From 1f1a2bac2a020038e405de78d791db2f14549897 Mon Sep 17 00:00:00 2001 From: wuhanqing Date: Sat, 5 Sep 2020 18:18:20 +0800 Subject: [PATCH] optimize curve-client --- conf/client.conf | 4 +- .../roles/generate_config/defaults/main.yml | 5 +- .../templates/nebd-client.conf.j2 | 2 + curvesnapshot_python/libcurveSnapshot.cpp | 9 +- nebd/etc/nebd/nebd-client.conf | 2 + nebd/src/part1/BUILD | 1 + nebd/src/part1/nebd_client.cpp | 143 +++++--- nebd/src/part1/nebd_client.h | 22 ++ nebd/src/part1/nebd_common.h | 2 + src/chunkserver/clone_manager.cpp | 2 +- src/chunkserver/clone_manager.h | 2 +- src/chunkserver/copyset_node_manager.cpp | 2 +- src/chunkserver/copyset_node_manager.h | 2 +- src/client/chunk_closure.cpp | 155 ++++----- src/client/chunk_closure.h | 164 ++------- src/client/client_common.h | 41 +-- src/client/client_config.cpp | 98 +++--- src/client/client_config.h | 21 +- src/client/client_metric.h | 33 +- src/client/config_info.h | 206 +++++------ src/client/copyset_client.cpp | 4 +- src/client/copyset_client.h | 29 +- src/client/file_instance.cpp | 60 +++- src/client/file_instance.h | 31 +- src/client/inflight_controller.h | 39 ++- src/client/io_condition_varaiable.h | 2 + src/client/io_tracker.cpp | 39 +-- src/client/io_tracker.h | 1 - src/client/iomanager.h | 15 +- src/client/iomanager4chunk.cpp | 2 +- src/client/iomanager4chunk.h | 4 +- src/client/iomanager4file.cpp | 3 +- src/client/iomanager4file.h | 18 +- src/client/lease_executor.cpp | 2 +- src/client/lease_executor.h | 4 +- src/client/libcurve_file.cpp | 84 ++--- src/client/libcurve_file.h | 17 +- src/client/libcurve_snapshot.cpp | 11 +- src/client/libcurve_snapshot.h | 6 +- src/client/mds_client.cpp | 165 +++++---- src/client/mds_client.h | 327 +++++++++--------- src/client/mds_client_base.cpp | 222 ++++++------ src/client/mds_client_base.h | 175 +++++----- src/client/metacache.cpp | 112 +++--- src/client/metacache.h | 114 +++--- src/client/metacache_struct.h | 131 +++---- src/client/request_closure.cpp | 62 +--- src/client/request_closure.h | 135 ++++---- src/client/request_context.h | 15 +- src/client/request_scheduler.cpp | 117 +++---- src/client/request_scheduler.h | 40 ++- src/client/request_sender.cpp | 33 +- src/client/request_sender.h | 4 +- src/client/request_sender_manager.cpp | 45 +-- src/client/request_sender_manager.h | 16 +- src/client/service_helper.cpp | 3 +- src/client/service_helper.h | 13 +- src/client/splitor.cpp | 289 +++++++++------- src/client/splitor.h | 23 +- src/client/unstable_helper.cpp | 69 ++++ src/client/unstable_helper.h | 106 ++++++ .../concurrent/bounded_blocking_queue.h | 5 +- src/common/concurrent/concurrent.h | 3 - src/common/concurrent/task_thread_pool.cpp | 117 ------- src/common/concurrent/task_thread_pool.h | 121 +++++-- src/common/math_util.h | 43 +++ src/mds/nameserver2/clean_task_manager.cpp | 2 +- src/mds/nameserver2/clean_task_manager.h | 2 +- src/snapshotcloneserver/common/thread_pool.h | 2 +- test/client/backoff_test.cpp | 59 ++-- .../client_mdsclient_metacache_unittest.cpp | 49 +-- test/client/client_metric_test.cpp | 10 +- test/client/client_unstable_helper_test.cpp | 18 +- test/client/client_userifo_unittest.cpp | 42 +-- test/client/copyset_client_test.cpp | 35 +- .../client/fake/client_workflow_test4snap.cpp | 6 +- test/client/fake/fakeMDS.h | 3 +- test/client/iotracker_splitor_unittest.cpp | 6 +- test/client/libcurve_interface_unittest.cpp | 58 ++-- test/client/mds_failover_test.cpp | 14 +- test/client/request_scheduler_test.cpp | 2 +- test/client/request_sender_manager_test.cpp | 4 +- test/client/snapshot_service_unittest.cpp | 51 ++- test/common/math_util_test.cpp | 51 +++ test/common/task_thread_pool_test.cpp | 18 +- test/integration/cluster_common/cluster.cpp | 2 +- test/integration/cluster_common/cluster.h | 2 +- .../cluster_common/cluster_basic_test.cpp | 4 +- 88 files changed, 2145 insertions(+), 2087 deletions(-) create mode 100644 src/client/unstable_helper.cpp create mode 100644 src/client/unstable_helper.h delete mode 100644 src/common/concurrent/task_thread_pool.cpp create mode 100644 src/common/math_util.h create mode 100644 test/common/math_util_test.cpp diff --git a/conf/client.conf b/conf/client.conf index 36f95c16d7..9b5fe31a8e 100644 --- a/conf/client.conf +++ b/conf/client.conf @@ -52,7 +52,7 @@ schedule.queueCapacity=1000000 # 队列取出到发送完rpc请求大概在(20us-100us),20us是正常情况下不需要获取leader的时候 # 如果在发送的时候需要获取leader,时间会在100us左右,一个线程的吞吐在10w-50w # 性能已经满足需求 -schedule.threadpoolSize=1 +schedule.threadpoolSize=2 # 为隔离qemu侧线程引入的任务队列,因为qemu一侧只有一个IO线程 # 当qemu一侧调用aio接口的时候直接将调用push到任务队列就返回, @@ -112,7 +112,7 @@ chunkserver.maxRetryTimesBeforeConsiderSuspend=20 ################# 文件级别配置项 ############# # # libcurve底层rpc调度允许最大的未返回rpc数量,每个文件的inflight RPC独立 -global.fileMaxInFlightRPCNum=64 +global.fileMaxInFlightRPCNum=128 # 文件IO下发到底层chunkserver最大的分片KB global.fileIOSplitMaxSizeKB=64 diff --git a/curve-ansible/roles/generate_config/defaults/main.yml b/curve-ansible/roles/generate_config/defaults/main.yml index 5825145750..e2c7f318e7 100644 --- a/curve-ansible/roles/generate_config/defaults/main.yml +++ b/curve-ansible/roles/generate_config/defaults/main.yml @@ -169,7 +169,7 @@ client_metacache_get_leader_timeout_ms: 500 client_metacache_get_leader_retry: 5 client_metacache_rpc_retry_interval_us: 100000 client_schedule_queue_capacity: 1000000 -client_schedule_threadpool_size: 1 +client_schedule_threadpool_size: 2 client_isolation_task_queue_capacity: 1000000 client_isolation_task_thread_pool_size: 1 client_chunkserver_op_retry_interval_us: 100000 @@ -183,7 +183,7 @@ client_chunkserver_check_health_timeout_ms: 100 client_chunkserver_server_stable_threshold: 3 client_chunkserver_min_retry_times_force_timeout_backoff: 5 client_chunkserver_max_retry_times_before_consider_suspend: 20 -client_file_max_inflight_rpc_num: 64 +client_file_max_inflight_rpc_num: 128 client_file_io_split_max_size_kb: 64 client_log_level: 0 client_log_path: /data/log/curve/ @@ -198,6 +198,7 @@ nebd_client_rpc_retry_max_inverval_us: 64000000 nebd_client_rpc_hostdown_retry_inverval_us: 10000 nebd_client_health_check_internal_s: 1 nebd_client_delay_health_check_internal_ms: 100 +nebd_client_rpc_send_exec_queue_num: 2 nebd_client_heartbeat_inverval_s: 5 nebd_client_heartbeat_rpc_timeout_ms: 500 nebd_server_heartbeat_timeout_s: 30 diff --git a/curve-ansible/roles/generate_config/templates/nebd-client.conf.j2 b/curve-ansible/roles/generate_config/templates/nebd-client.conf.j2 index 36b8b9b94a..d7121c6dad 100644 --- a/curve-ansible/roles/generate_config/templates/nebd-client.conf.j2 +++ b/curve-ansible/roles/generate_config/templates/nebd-client.conf.j2 @@ -16,6 +16,8 @@ request.rpcHostDownRetryIntervalUs={{ nebd_client_rpc_hostdown_retry_inverval_us request.rpcHealthCheckIntervalS={{ nebd_client_health_check_internal_s }} # brpc从rpc失败到进行健康检查的最大时间间隔,单位ms request.rpcMaxDelayHealthCheckIntervalMs={{ nebd_client_delay_health_check_internal_ms }} +# rpc发送执行队列个数 +request.rpcSendExecQueueNum={{ nebd_client_rpc_send_exec_queue_num }} # heartbeat间隔 heartbeat.intervalS={{ nebd_client_heartbeat_inverval_s }} diff --git a/curvesnapshot_python/libcurveSnapshot.cpp b/curvesnapshot_python/libcurveSnapshot.cpp index e62ade6c09..5cdce45219 100644 --- a/curvesnapshot_python/libcurveSnapshot.cpp +++ b/curvesnapshot_python/libcurveSnapshot.cpp @@ -30,11 +30,16 @@ #include "src/client/client_config.h" #include "include/client/libcurve.h" #include "src/client/client_common.h" +#include "src/common/concurrent/concurrent.h" using curve::client::UserInfo; using curve::client::ClientConfig; using curve::client::SnapshotClient; using curve::client::SnapCloneClosure; +using curve::client::FileServiceOption; +using curve::client::ClientConfigOption; +using curve::common::Mutex; +using curve::common::ConditionVariable; class TaskTracker { public: @@ -128,8 +133,8 @@ int Init(const char* path) { return -LIBCURVE_ERROR::FAILED; } - FileServiceOption_t fileopt = cc.GetFileServiceOption(); - ClientConfigOption_t copt; + FileServiceOption fileopt = cc.GetFileServiceOption(); + ClientConfigOption copt; copt.loginfo = fileopt.loginfo; copt.ioOpt = fileopt.ioOpt; copt.metaServerOpt = fileopt.metaServerOpt; diff --git a/nebd/etc/nebd/nebd-client.conf b/nebd/etc/nebd/nebd-client.conf index c0c02c37b8..dcd2e3fca4 100644 --- a/nebd/etc/nebd/nebd-client.conf +++ b/nebd/etc/nebd/nebd-client.conf @@ -16,6 +16,8 @@ request.rpcHostDownRetryIntervalUs=10000 request.rpcHealthCheckIntervalS=1 # brpc从rpc失败到进行健康检查的最大时间间隔,单位ms request.rpcMaxDelayHealthCheckIntervalMs=100 +# rpc发送执行队列个数 +request.rpcSendExecQueueNum=2 # heartbeat间隔 heartbeat.intervalS=5 diff --git a/nebd/src/part1/BUILD b/nebd/src/part1/BUILD index c8c713cc68..89d741d322 100644 --- a/nebd/src/part1/BUILD +++ b/nebd/src/part1/BUILD @@ -51,6 +51,7 @@ cc_library( "//external:brpc", "//nebd/src/common:nebd_common", "//nebd/proto:client_cc_proto", + "//include:include-common" ], copts = COPTS, linkopts = ["-Wl,-rpath=/usr/lib/nebd"], diff --git a/nebd/src/part1/nebd_client.cpp b/nebd/src/part1/nebd_client.cpp index a0b3305316..04c6bffbc3 100644 --- a/nebd/src/part1/nebd_client.cpp +++ b/nebd/src/part1/nebd_client.cpp @@ -102,6 +102,17 @@ int NebdClient::Init(const char* confpath) { heartbeatMgr_->Run(); + // init rpc send exec-queue + rpcTaskQueues_.resize(option_.requestOption.rpcSendExecQueueNum); + for (auto& q : rpcTaskQueues_) { + int rc = bthread::execution_queue_start( + &q, nullptr, &NebdClient::ExecAsyncRpcTask, this); + if (rc != 0) { + LOG(ERROR) << "Init AsyncRpcQueues failed"; + return -1; + } + } + return 0; } @@ -109,6 +120,13 @@ void NebdClient::Uninit() { if (heartbeatMgr_ != nullptr) { heartbeatMgr_->Stop(); } + + // stop exec queue + for (auto& q : rpcTaskQueues_) { + bthread::execution_queue_stop(q); + bthread::execution_queue_join(q); + } + LOG(INFO) << "NebdClient uninit success."; google::ShutdownGoogleLogging(); } @@ -289,67 +307,85 @@ int64_t NebdClient::GetFileSize(int fd) { } int NebdClient::Discard(int fd, NebdClientAioContext* aioctx) { - nebd::client::NebdFileService_Stub stub(&channel_); - nebd::client::DiscardRequest request; - request.set_fd(fd); - request.set_offset(aioctx->offset); - request.set_size(aioctx->length); - - AioDiscardClosure* done = new(std::nothrow) AioDiscardClosure( - fd, aioctx, option_.requestOption); - done->cntl.set_timeout_ms(-1); - done->cntl.set_log_id(logId_.fetch_add(1, std::memory_order_relaxed)); - stub.Discard(&done->cntl, &request, &done->response, done); + auto task = [this, fd, aioctx]() { + nebd::client::NebdFileService_Stub stub(&channel_); + nebd::client::DiscardRequest request; + request.set_fd(fd); + request.set_offset(aioctx->offset); + request.set_size(aioctx->length); + + AioDiscardClosure* done = new(std::nothrow) AioDiscardClosure( + fd, aioctx, option_.requestOption); + done->cntl.set_timeout_ms(-1); + done->cntl.set_log_id(logId_.fetch_add(1, std::memory_order_relaxed)); + stub.Discard(&done->cntl, &request, &done->response, done); + }; + + PushAsyncTask(task); return 0; } int NebdClient::AioRead(int fd, NebdClientAioContext* aioctx) { - nebd::client::NebdFileService_Stub stub(&channel_); - nebd::client::ReadRequest request; - request.set_fd(fd); - request.set_offset(aioctx->offset); - request.set_size(aioctx->length); - - AioReadClosure* done = new(std::nothrow) AioReadClosure( - fd, aioctx, option_.requestOption); - done->cntl.set_timeout_ms(-1); - done->cntl.set_log_id(logId_.fetch_add(1, std::memory_order_relaxed)); - stub.Read(&done->cntl, &request, &done->response, done); + auto task = [this, fd, aioctx]() { + nebd::client::NebdFileService_Stub stub(&channel_); + nebd::client::ReadRequest request; + request.set_fd(fd); + request.set_offset(aioctx->offset); + request.set_size(aioctx->length); + + AioReadClosure* done = new(std::nothrow) AioReadClosure( + fd, aioctx, option_.requestOption); + done->cntl.set_timeout_ms(-1); + done->cntl.set_log_id(logId_.fetch_add(1, std::memory_order_relaxed)); + stub.Read(&done->cntl, &request, &done->response, done); + }; + + PushAsyncTask(task); + return 0; } static void EmptyDeleter(void* m) {} int NebdClient::AioWrite(int fd, NebdClientAioContext* aioctx) { - nebd::client::NebdFileService_Stub stub(&channel_); - nebd::client::WriteRequest request; - request.set_fd(fd); - request.set_offset(aioctx->offset); - request.set_size(aioctx->length); + auto task = [this, fd, aioctx]() { + nebd::client::NebdFileService_Stub stub(&channel_); + nebd::client::WriteRequest request; + request.set_fd(fd); + request.set_offset(aioctx->offset); + request.set_size(aioctx->length); + + AioWriteClosure* done = new(std::nothrow) AioWriteClosure( + fd, aioctx, option_.requestOption); - AioWriteClosure* done = new(std::nothrow) AioWriteClosure( - fd, aioctx, option_.requestOption); + done->cntl.set_timeout_ms(-1); + done->cntl.set_log_id(logId_.fetch_add(1, std::memory_order_relaxed)); + done->cntl.request_attachment().append_user_data( + aioctx->buf, aioctx->length, EmptyDeleter); + stub.Write(&done->cntl, &request, &done->response, done); + }; - done->cntl.set_timeout_ms(-1); - done->cntl.set_log_id(logId_.fetch_add(1, std::memory_order_relaxed)); - done->cntl.request_attachment().append_user_data( - aioctx->buf, aioctx->length, EmptyDeleter); - stub.Write(&done->cntl, &request, &done->response, done); + PushAsyncTask(task); return 0; } int NebdClient::Flush(int fd, NebdClientAioContext* aioctx) { - nebd::client::NebdFileService_Stub stub(&channel_); - nebd::client::FlushRequest request; - request.set_fd(fd); - - AioFlushClosure* done = new(std::nothrow) AioFlushClosure( - fd, aioctx, option_.requestOption); - done->cntl.set_timeout_ms(-1); - done->cntl.set_log_id(logId_.fetch_add(1, std::memory_order_relaxed)); - stub.Flush(&done->cntl, &request, &done->response, done); + auto task = [this, fd, aioctx]() { + nebd::client::NebdFileService_Stub stub(&channel_); + nebd::client::FlushRequest request; + request.set_fd(fd); + + AioFlushClosure* done = new(std::nothrow) AioFlushClosure( + fd, aioctx, option_.requestOption); + done->cntl.set_timeout_ms(-1); + done->cntl.set_log_id(logId_.fetch_add(1, std::memory_order_relaxed)); + stub.Flush(&done->cntl, &request, &done->response, done); + }; + + PushAsyncTask(task); + return 0; } @@ -473,6 +509,13 @@ int NebdClient::InitNebdClientOption(Configuration* conf) { LOG_IF(ERROR, ret != true) << "Load request.rpcMaxDelayHealthCheckIntervalMs failed"; // NOLINT RETURN_IF_FALSE(ret); + ret = conf->GetUInt32Value("request.rpcSendExecQueueNum", + &requestOption.rpcSendExecQueueNum); + LOG_IF(ERROR, ret != true) + << "Load request.rpcSendExecQueueNum from config file failed, current " + "value is " + << requestOption.rpcSendExecQueueNum; + option_.requestOption = requestOption; ret = conf->GetStringValue("log.path", &option_.logOption.logPath); @@ -564,5 +607,19 @@ void NebdClient::InitLogger(const LogOption& logOption) { google::InitGoogleLogging(kProcessName); } +int NebdClient::ExecAsyncRpcTask(void* meta, + bthread::TaskIterator& iter) { // NOLINT + if (iter.is_queue_stopped()) { + return 0; + } + + for (; iter; ++iter) { + auto& task = *iter; + task(); + } + + return 0; +} + } // namespace client } // namespace nebd diff --git a/nebd/src/part1/nebd_client.h b/nebd/src/part1/nebd_client.h index c6a8ee7e77..cb02361335 100644 --- a/nebd/src/part1/nebd_client.h +++ b/nebd/src/part1/nebd_client.h @@ -24,10 +24,12 @@ #define NEBD_SRC_PART1_NEBD_CLIENT_H_ #include +#include #include #include #include +#include #include "nebd/src/part1/nebd_common.h" #include "nebd/src/common/configuration.h" @@ -36,6 +38,8 @@ #include "nebd/src/part1/heartbeat_manager.h" #include "nebd/src/part1/nebd_metacache.h" +#include "include/curve_compiler_specific.h" + namespace nebd { namespace client { @@ -171,6 +175,24 @@ class NebdClient { brpc::Channel channel_; std::atomic logId_{1}; + + private: + using AsyncRpcTask = std::function; + + std::vector> rpcTaskQueues_; + + static int ExecAsyncRpcTask(void* meta, bthread::TaskIterator& iter); // NOLINT + + void PushAsyncTask(const AsyncRpcTask& task) { + static thread_local unsigned int seed = time(nullptr); + + int idx = rand_r(&seed) % rpcTaskQueues_.size(); + int rc = bthread::execution_queue_execute(rpcTaskQueues_[idx], task); + + if (CURVE_UNLIKELY(rc != 0)) { + task(); + } + } }; extern NebdClient &nebdClient; diff --git a/nebd/src/part1/nebd_common.h b/nebd/src/part1/nebd_common.h index 94ef121a9a..30b5cb292f 100644 --- a/nebd/src/part1/nebd_common.h +++ b/nebd/src/part1/nebd_common.h @@ -39,6 +39,8 @@ struct RequestOption { int64_t rpcHealthCheckIntervalS; // brpc从rpc失败到进行健康检查的最大时间间隔 int64_t rpcMaxDelayHealthCheckIntervalMs; + // rpc发送执行队列个数 + uint32_t rpcSendExecQueueNum = 2; }; // 日志配置项 diff --git a/src/chunkserver/clone_manager.cpp b/src/chunkserver/clone_manager.cpp index 1f1bf4696e..6fc428bdba 100644 --- a/src/chunkserver/clone_manager.cpp +++ b/src/chunkserver/clone_manager.cpp @@ -42,7 +42,7 @@ int CloneManager::Run() { return 0; // 启动线程池 LOG(INFO) << "Begin to run clone manager."; - tp_ = std::make_shared(); + tp_ = std::make_shared>(); int ret = tp_->Start(options_.threadNum, options_.queueCapacity); if (ret < 0) { LOG(ERROR) << "clone manager start error." diff --git a/src/chunkserver/clone_manager.h b/src/chunkserver/clone_manager.h index 529ea54229..01f7088218 100644 --- a/src/chunkserver/clone_manager.h +++ b/src/chunkserver/clone_manager.h @@ -105,7 +105,7 @@ class CloneManager { // 克隆任务管理相关的选项,调Init的时候初始化 CloneOptions options_; // 处理克隆任务的异步线程池 - std::shared_ptr tp_; + std::shared_ptr> tp_; // 当前线程池是否处于工作状态 std::atomic isRunning_; }; diff --git a/src/chunkserver/copyset_node_manager.cpp b/src/chunkserver/copyset_node_manager.cpp index e432ed9e7f..dd03d63418 100755 --- a/src/chunkserver/copyset_node_manager.cpp +++ b/src/chunkserver/copyset_node_manager.cpp @@ -51,7 +51,7 @@ std::once_flag addServiceFlag; int CopysetNodeManager::Init(const CopysetNodeOptions ©setNodeOptions) { copysetNodeOptions_ = copysetNodeOptions; if (copysetNodeOptions_.loadConcurrency > 0) { - copysetLoader_ = std::make_shared(); + copysetLoader_ = std::make_shared>(); } else { copysetLoader_ = nullptr; } diff --git a/src/chunkserver/copyset_node_manager.h b/src/chunkserver/copyset_node_manager.h index d8ac867953..512a4accae 100755 --- a/src/chunkserver/copyset_node_manager.h +++ b/src/chunkserver/copyset_node_manager.h @@ -205,7 +205,7 @@ class CopysetNodeManager : public curve::common::Uncopyable { // 复制组配置选项 CopysetNodeOptions copysetNodeOptions_; // 控制copyset并发启动的数量 - std::shared_ptr copysetLoader_; + std::shared_ptr> copysetLoader_; // 表示copyset node manager当前是否正在运行 Atomic running_; // 表示copyset node manager当前是否已经完成加载 diff --git a/src/client/chunk_closure.cpp b/src/client/chunk_closure.cpp index a52073263c..328772b687 100644 --- a/src/client/chunk_closure.cpp +++ b/src/client/chunk_closure.cpp @@ -22,75 +22,27 @@ #include "src/client/chunk_closure.h" -#include #include #include -#include #include #include "src/client/client_common.h" -#include "src/client/request_sender.h" -#include "src/client/request_sender_manager.h" #include "src/client/copyset_client.h" #include "src/client/metacache.h" #include "src/client/request_closure.h" #include "src/client/request_context.h" +#include "src/client/service_helper.h" #include "src/client/io_tracker.h" // TODO(tongguangxun) :优化重试逻辑,将重试逻辑与RPC返回逻辑拆开 namespace curve { namespace client { -ClientClosure::BackoffParam ClientClosure::backoffParam_; -FailureRequestOption_t ClientClosure::failReqOpt_; - -UnstableState UnstableHelper::GetCurrentUnstableState( - ChunkServerID csId, - const butil::EndPoint& csEndPoint) { - - std::string ip = butil::ip2str(csEndPoint.ip).c_str(); - - lock_.Lock(); - // 如果当前ip已经超过阈值,则直接返回chunkserver unstable - int unstabled = serverUnstabledChunkservers_[ip].size(); - if (unstabled >= option_.serverUnstableThreshold) { - serverUnstabledChunkservers_[ip].emplace(csId); - lock_.UnLock(); - return UnstableState::ChunkServerUnstable; - } - - bool exceed = - timeoutTimes_[csId] > option_.maxStableChunkServerTimeoutTimes; - lock_.UnLock(); - - if (exceed == false) { - return UnstableState::NoUnstable; - } - - bool health = CheckChunkServerHealth(csEndPoint); - if (health) { - ClearTimeout(csId, csEndPoint); - return UnstableState::NoUnstable; - } - lock_.Lock(); - auto ret = serverUnstabledChunkservers_[ip].emplace(csId); - unstabled = serverUnstabledChunkservers_[ip].size(); - lock_.UnLock(); - - if (ret.second && unstabled == option_.serverUnstableThreshold) { - return UnstableState::ServerUnstable; - } else { - return UnstableState::ChunkServerUnstable; - } -} +ClientClosure::BackoffParam ClientClosure::backoffParam_; +FailureRequestOption ClientClosure::failReqOpt_; void ClientClosure::PreProcessBeforeRetry(int rpcstatus, int cntlstatus) { - RequestClosure *reqDone = dynamic_cast(done_); - - RequestContext *reqCtx = reqDone->GetReqCtx(); - LogicPoolID logicPoolId = reqCtx->idinfo_.lpid_; - CopysetID copysetId = reqCtx->idinfo_.cpid_; - ChunkID chunkid = reqCtx->idinfo_.cid_; + RequestClosure* reqDone = static_cast(done_); // 如果对应的cooysetId leader可能发生变更 // 那么设置这次重试请求超时时间为默认值 @@ -103,7 +55,7 @@ void ClientClosure::PreProcessBeforeRetry(int rpcstatus, int cntlstatus) { uint64_t nextTimeout = 0; uint64_t retriedTimes = reqDone->GetRetriedTimes(); bool leaderMayChange = metaCache_->IsLeaderMayChange( - logicPoolId, copysetId); + chunkIdInfo_.lpid_, chunkIdInfo_.cpid_); // 当某一个IO重试超过一定次数后,超时时间一定进行指数退避 // 当底层chunkserver压力大时,可能也会触发unstable @@ -120,27 +72,29 @@ void ClientClosure::PreProcessBeforeRetry(int rpcstatus, int cntlstatus) { reqDone->SetNextTimeOutMS(nextTimeout); LOG(WARNING) << "rpc timeout, next timeout = " << nextTimeout - << ", " << *reqCtx + << ", " << *reqCtx_ << ", retried times = " << reqDone->GetRetriedTimes() << ", IO id = " << reqDone->GetIOTracker()->GetID() - << ", request id = " << reqCtx->id_ - << ", remote side = " << remoteAddress_; + << ", request id = " << reqCtx_->id_ + << ", remote side = " + << butil::endpoint2str(cntl_->remote_side()).c_str(); return; } if (rpcstatus == CHUNK_OP_STATUS::CHUNK_OP_STATUS_OVERLOAD) { uint64_t nextsleeptime = OverLoadBackOff(reqDone->GetRetriedTimes()); LOG(WARNING) << "chunkserver overload, sleep(us) = " << nextsleeptime - << ", " << *reqCtx + << ", " << *reqCtx_ << ", retried times = " << reqDone->GetRetriedTimes() << ", IO id = " << reqDone->GetIOTracker()->GetID() - << ", request id = " << reqCtx->id_ - << ", remote side = " << remoteAddress_; + << ", request id = " << reqCtx_->id_ + << ", remote side = " + << butil::endpoint2str(cntl_->remote_side()).c_str(); bthread_usleep(nextsleeptime); return; } - auto nextSleepUS = 0; + uint64_t nextSleepUS = 0; if (!retryDirectly_) { nextSleepUS = failReqOpt_.chunkserverOPRetryIntervalUS; @@ -153,14 +107,15 @@ void ClientClosure::PreProcessBeforeRetry(int rpcstatus, int cntlstatus) { << "Rpc failed " << (retryDirectly_ ? "retry directly, " : "sleep " + std::to_string(nextSleepUS) + " us, ") - << *reqCtx << ", cntl status = " << cntlstatus + << *reqCtx_ << ", cntl status = " << cntlstatus << ", response status = " << curve::chunkserver::CHUNK_OP_STATUS_Name( static_cast(rpcstatus)) << ", retried times = " << reqDone->GetRetriedTimes() << ", IO id = " << reqDone->GetIOTracker()->GetID() - << ", request id = " << reqCtx->id_ - << ", remote side = " << remoteAddress_; + << ", request id = " << reqCtx_->id_ + << ", remote side = " + << butil::endpoint2str(cntl_->remote_side()).c_str(); if (nextSleepUS != 0) { bthread_usleep(nextSleepUS); @@ -168,29 +123,31 @@ void ClientClosure::PreProcessBeforeRetry(int rpcstatus, int cntlstatus) { } uint64_t ClientClosure::OverLoadBackOff(uint64_t currentRetryTimes) { - uint64_t curpowTime = std::min(currentRetryTimes, - backoffParam_.maxOverloadPow); + uint64_t curpowTime = + std::min(currentRetryTimes, backoffParam_.maxOverloadPow); - uint64_t nextsleeptime = failReqOpt_.chunkserverOPRetryIntervalUS - * std::pow(2, curpowTime); + uint64_t nextsleeptime = + failReqOpt_.chunkserverOPRetryIntervalUS * (1 << curpowTime); - int random_time = std::rand() % (nextsleeptime/5 + 1); - random_time -= nextsleeptime/10; + // 20 percent jitter + uint64_t random_time = std::rand() % (nextsleeptime / 5 + 1); + random_time -= nextsleeptime / 10; nextsleeptime += random_time; - nextsleeptime = std::min(nextsleeptime, failReqOpt_.chunkserverMaxRetrySleepIntervalUS); // NOLINT - nextsleeptime = std::max(nextsleeptime, failReqOpt_.chunkserverOPRetryIntervalUS); // NOLINT + nextsleeptime = std::min(nextsleeptime, failReqOpt_.chunkserverMaxRetrySleepIntervalUS); // NOLINT + nextsleeptime = std::max(nextsleeptime, failReqOpt_.chunkserverOPRetryIntervalUS); // NOLINT return nextsleeptime; } uint64_t ClientClosure::TimeoutBackOff(uint64_t currentRetryTimes) { - uint64_t curpowTime = std::min(currentRetryTimes, - backoffParam_.maxTimeoutPow); + uint64_t curpowTime = + std::min(currentRetryTimes, backoffParam_.maxTimeoutPow); - uint64_t nextTimeout = failReqOpt_.chunkserverRPCTimeoutMS - * std::pow(2, curpowTime); - nextTimeout = std::min(nextTimeout, failReqOpt_.chunkserverMaxRPCTimeoutMS); // NOLINT + uint64_t nextTimeout = + failReqOpt_.chunkserverRPCTimeoutMS * (1 << curpowTime); + + nextTimeout = std::min(nextTimeout, failReqOpt_.chunkserverMaxRPCTimeoutMS); nextTimeout = std::max(nextTimeout, failReqOpt_.chunkserverRPCTimeoutMS); return nextTimeout; @@ -206,13 +163,12 @@ void ClientClosure::Run() { brpc::ClosureGuard doneGuard(done_); metaCache_ = client_->GetMetaCache(); - reqDone_ = dynamic_cast(done_); + reqDone_ = static_cast(done_); fileMetric_ = reqDone_->GetMetric(); reqCtx_ = reqDone_->GetReqCtx(); chunkIdInfo_ = reqCtx_->idinfo_; status_ = -1; cntlstatus_ = cntl_->ErrorCode(); - remoteAddress_ = butil::endpoint2str(cntl_->remote_side()).c_str(); bool needRetry = false; @@ -221,7 +177,7 @@ void ClientClosure::Run() { OnRpcFailed(); } else { // 只要rpc正常返回,就清空超时计数器 - UnstableHelper::GetInstance().ClearTimeout( + metaCache_->GetUnstableHelper().ClearTimeout( chunkserverID_, chunkserverEndPoint_); status_ = GetResponseStatus(); @@ -268,7 +224,8 @@ void ClientClosure::Run() { << ", retried times = " << reqDone_->GetRetriedTimes() << ", IO id = " << reqDone_->GetIOTracker()->GetID() << ", request id = " << reqCtx_->id_ - << ", remote side = " << remoteAddress_; + << ", remote side = " + << butil::endpoint2str(cntl_->remote_side()).c_str(); } break; @@ -287,7 +244,8 @@ void ClientClosure::Run() { << ", retried times = " << reqDone_->GetRetriedTimes() << ", IO id = " << reqDone_->GetIOTracker()->GetID() << ", request id = " << reqCtx_->id_ - << ", remote side = " << remoteAddress_; + << ", remote side = " + << butil::endpoint2str(cntl_->remote_side()).c_str(); } } @@ -305,7 +263,7 @@ void ClientClosure::OnRpcFailed() { // 如果连接失败,再等一定时间再重试 if (cntlstatus_ == brpc::ERPCTIMEDOUT) { // 如果RPC超时, 对应的chunkserver超时请求次数+1 - UnstableHelper::GetInstance().IncreTimeout(chunkserverID_); + metaCache_->GetUnstableHelper().IncreTimeout(chunkserverID_); MetricHelper::IncremTimeOutRPCCount(fileMetric_, reqCtx_->optype_); } @@ -317,7 +275,8 @@ void ClientClosure::OnRpcFailed() { << ", retried times = " << reqDone_->GetRetriedTimes() << ", IO id = " << reqDone_->GetIOTracker()->GetID() << ", request id = " << reqCtx_->id_ - << ", remote side = " << remoteAddress_; + << ", remote side = " + << butil::endpoint2str(cntl_->remote_side()).c_str(); // it will be invoked in brpc's bthread if (reqCtx_->optype_ == OpType::WRITE) { @@ -329,8 +288,9 @@ void ClientClosure::OnRpcFailed() { } void ClientClosure::ProcessUnstableState() { - UnstableState state = UnstableHelper::GetInstance().GetCurrentUnstableState( - chunkserverID_, chunkserverEndPoint_); + UnstableState state = + metaCache_->GetUnstableHelper().GetCurrentUnstableState( + chunkserverID_, chunkserverEndPoint_); switch (state) { case UnstableState::ServerUnstable: { @@ -359,7 +319,7 @@ void ClientClosure::ProcessUnstableState() { void ClientClosure::OnSuccess() { reqDone_->SetFailed(0); - auto duration = TimeUtility::GetTimeofDayUs() - reqDone_->GetStartTime(); + auto duration = cntl_->latency_us(); MetricHelper::LatencyRecord(fileMetric_, duration, reqCtx_->optype_); MetricHelper::IncremRPCQPSCount( fileMetric_, reqCtx_->rawlength_, reqCtx_->optype_); @@ -374,9 +334,10 @@ void ClientClosure::OnChunkNotExist() { << ", retried times = " << reqDone_->GetRetriedTimes() << ", IO id = " << reqDone_->GetIOTracker()->GetID() << ", request id = " << reqCtx_->id_ - << ", remote side = " << remoteAddress_; + << ", remote side = " + << butil::endpoint2str(cntl_->remote_side()).c_str(); - auto duration = TimeUtility::GetTimeofDayUs() - reqDone_->GetStartTime(); + auto duration = cntl_->latency_us(); MetricHelper::LatencyRecord(fileMetric_, duration, reqCtx_->optype_); MetricHelper::IncremRPCQPSCount( fileMetric_, reqCtx_->rawlength_, reqCtx_->optype_); @@ -391,7 +352,8 @@ void ClientClosure::OnChunkExist() { << ", retried times = " << reqDone_->GetRetriedTimes() << ", IO id = " << reqDone_->GetIOTracker()->GetID() << ", request id = " << reqCtx_->id_ - << ", remote side = " << remoteAddress_; + << ", remote side = " + << butil::endpoint2str(cntl_->remote_side()).c_str(); } void ClientClosure::OnRedirected() { @@ -403,7 +365,8 @@ void ClientClosure::OnRedirected() { << ", request id = " << reqCtx_->id_ << ", redirect leader is " << (response_->has_redirect() ? response_->redirect() : "empty") - << ", remote side = " << remoteAddress_; + << ", remote side = " + << butil::endpoint2str(cntl_->remote_side()).c_str(); if (response_->has_redirect()) { int ret = UpdateLeaderWithRedirectInfo(response_->redirect()); @@ -422,7 +385,8 @@ void ClientClosure::OnCopysetNotExist() { << ", retried times = " << reqDone_->GetRetriedTimes() << ", IO id = " << reqDone_->GetIOTracker()->GetID() << ", request id = " << reqCtx_->id_ - << ", remote side = " << remoteAddress_; + << ", remote side = " + << butil::endpoint2str(cntl_->remote_side()).c_str(); RefreshLeader(); } @@ -484,7 +448,8 @@ void ClientClosure::OnBackward() { << ", retried times = " << reqDone_->GetRetriedTimes() << ", IO id = " << reqDone_->GetIOTracker()->GetID() << ", request id = " << reqCtx_->id_ - << ", remote side = " << remoteAddress_; + << ", remote side = " + << butil::endpoint2str(cntl_->remote_side()).c_str(); reqCtx_->seq_ = latestSn; } @@ -497,7 +462,8 @@ void ClientClosure::OnInvalidRequest() { << ", retried times = " << reqDone_->GetRetriedTimes() << ", IO id = " << reqDone_->GetIOTracker()->GetID() << ", request id = " << reqCtx_->id_ - << ", remote side = " << remoteAddress_; + << ", remote side = " + << butil::endpoint2str(cntl_->remote_side()).c_str(); MetricHelper::IncremFailRPCCount(fileMetric_, reqCtx_->optype_); } @@ -591,7 +557,8 @@ void GetChunkInfoClosure::OnRedirected() { << ", redirect leader is " << (chunkinforesponse_->has_redirect() ? chunkinforesponse_->redirect() : "empty") - << ", remote side = " << remoteAddress_; + << ", remote side = " + << butil::endpoint2str(cntl_->remote_side()).c_str(); if (chunkinforesponse_->has_redirect()) { int ret = UpdateLeaderWithRedirectInfo(chunkinforesponse_->redirect()); diff --git a/src/client/chunk_closure.h b/src/client/chunk_closure.h index 383c5ad738..035f9b6e4a 100644 --- a/src/client/chunk_closure.h +++ b/src/client/chunk_closure.h @@ -25,10 +25,7 @@ #include #include -#include #include -#include // NOLINT -#include // NOLINT #include #include @@ -37,8 +34,7 @@ #include "src/client/client_common.h" #include "src/client/client_metric.h" #include "src/client/request_closure.h" -#include "src/common/concurrent/concurrent.h" -#include "src/client/service_helper.h" +#include "src/common/math_util.h" namespace curve { namespace client { @@ -48,97 +44,18 @@ using curve::chunkserver::ChunkResponse; using curve::chunkserver::GetChunkInfoResponse; using ::google::protobuf::Message; using ::google::protobuf::Closure; -using curve::common::SpinLock; -class RequestSenderManager; class MetaCache; class CopysetClient; -enum class UnstableState { - NoUnstable, - ChunkServerUnstable, - ServerUnstable -}; - -// 如果chunkserver宕机或者网络不可达, 发往对应chunkserver的rpc会超时 -// 返回之后, 回去refresh leader然后再去发送请求 -// 这种情况下不同copyset上的请求,总会先rpc timedout然后重新refresh leader -// 为了避免一次多余的rpc timedout -// 记录一下发往同一个chunkserver上超时请求的次数 -// 如果超过一定的阈值,会发送http请求检查chunkserver是否健康 -// 如果不健康,则通知所有leader在这台chunkserver上的copyset -// 主动去refresh leader,而不是根据缓存的leader信息直接发送rpc -class UnstableHelper { - public: - static UnstableHelper& GetInstance() { - static UnstableHelper helper; - return helper; - } - - void IncreTimeout(ChunkServerID csId) { - lock_.Lock(); - ++timeoutTimes_[csId]; - lock_.UnLock(); - } - - UnstableState GetCurrentUnstableState( - ChunkServerID csId, - const butil::EndPoint& csEndPoint); - - void ClearTimeout(ChunkServerID csId, - const butil::EndPoint& csEndPoint) { - std::string ip = butil::ip2str(csEndPoint.ip).c_str(); - - lock_.Lock(); - timeoutTimes_[csId] = 0; - serverUnstabledChunkservers_[ip].erase(csId); - lock_.UnLock(); - } - - void SetUnstableChunkServerOption( - const ChunkServerUnstableOption& opt) { - option_ = opt; - } - - // 测试使用,重置计数器 - void ResetState() { - timeoutTimes_.clear(); - serverUnstabledChunkservers_.clear(); - } - - private: - UnstableHelper() = default; - - /** - * @brief 检查chunkserver状态 - * - * @param: endPoint chunkserver的ip:port地址 - * @return: true 健康 / false 不健康 - */ - bool CheckChunkServerHealth(const butil::EndPoint& endPoint) { - return ServiceHelper::CheckChunkServerHealth( - endPoint, option_.checkHealthTimeoutMS) == 0; - } - - ChunkServerUnstableOption option_; - - SpinLock lock_; - - // 同一chunkserver连续超时请求次数 - std::unordered_map timeoutTimes_; - - // 同一server上unstable chunkserver的id - std::unordered_map> serverUnstabledChunkservers_; // NOLINT -}; - /** * ClientClosure,负责保存Rpc上下文, * 包含cntl和response已经重试次数 */ class ClientClosure : public Closure { public: - ClientClosure(CopysetClient *client, Closure *done) - : client_(client), done_(done) {} + ClientClosure(CopysetClient* client, Closure* done) + : client_(client), done_(done) {} virtual ~ClientClosure() = default; @@ -147,7 +64,7 @@ class ClientClosure : public Closure { } virtual void SetResponse(Message* response) { - response_.reset(dynamic_cast(response)); + response_.reset(static_cast(response)); } void SetChunkServerID(ChunkServerID csid) { @@ -205,12 +122,9 @@ class ClientClosure : public Closure { } static void SetFailureRequestOption( - const FailureRequestOption_t& failRequestOpt) { + const FailureRequestOption& failRequestOpt) { failReqOpt_ = failRequestOpt; - UnstableHelper::GetInstance().SetUnstableChunkServerOption( - failReqOpt_.chunkserverUnstableOption); - std::srand(std::time(nullptr)); SetBackoffParam(); @@ -230,7 +144,7 @@ class ClientClosure : public Closure { done_ = done; } - static FailureRequestOption_t GetFailOpt() { + static FailureRequestOption GetFailOpt() { return failReqOpt_; } @@ -242,18 +156,20 @@ class ClientClosure : public Closure { * @param: cntlstatus为本次rpc controller返回值 */ void PreProcessBeforeRetry(int rpcstatue, int cntlstatus); + /** * 底层chunkserver overload之后需要根据重试次数进行退避 * @param: currentRetryTimes为当前已重试的次数 * @return: 返回当前的需要睡眠的时间 */ - uint64_t OverLoadBackOff(uint64_t currentRetryTimes); + static uint64_t OverLoadBackOff(uint64_t currentRetryTimes); + /** * rpc timeout之后需要根据重试次数进行退避 * @param: currentRetryTimes为当前已重试的次数 * @return: 返回下一次RPC 超时时间 */ - uint64_t TimeoutBackOff(uint64_t currentRetryTimes); + static uint64_t TimeoutBackOff(uint64_t currentRetryTimes); struct BackoffParam { uint64_t maxTimeoutPow; @@ -265,23 +181,18 @@ class ClientClosure : public Closure { }; static void SetBackoffParam() { - uint64_t overloadTimes = failReqOpt_.chunkserverMaxRetrySleepIntervalUS - / failReqOpt_.chunkserverOPRetryIntervalUS; - backoffParam_.maxOverloadPow = GetPowTime(overloadTimes); + using curve::common::MaxPowerTimesLessEqualValue; + uint64_t overloadTimes = + failReqOpt_.chunkserverMaxRetrySleepIntervalUS / + failReqOpt_.chunkserverOPRetryIntervalUS; - uint64_t timeoutTimes = failReqOpt_.chunkserverMaxRPCTimeoutMS - / failReqOpt_.chunkserverRPCTimeoutMS; - backoffParam_.maxTimeoutPow = GetPowTime(timeoutTimes); - } + backoffParam_.maxOverloadPow = + MaxPowerTimesLessEqualValue(overloadTimes); - static uint64_t GetPowTime(uint64_t value) { - int pow = 0; - while (value > 1) { - value>>=1; - pow++; - } - return pow; + uint64_t timeoutTimes = failReqOpt_.chunkserverMaxRPCTimeoutMS / + failReqOpt_.chunkserverRPCTimeoutMS; + backoffParam_.maxTimeoutPow = MaxPowerTimesLessEqualValue(timeoutTimes); } static BackoffParam backoffParam_; @@ -293,7 +204,7 @@ class ClientClosure : public Closure { void RefreshLeader(); - static FailureRequestOption_t failReqOpt_; + static FailureRequestOption failReqOpt_; brpc::Controller* cntl_; std::unique_ptr response_; @@ -312,22 +223,19 @@ class ClientClosure : public Closure { ChunkIDInfo chunkIdInfo_; // 发送重试请求前是否睡眠 - bool retryDirectly_{false}; + bool retryDirectly_ = false; // response 状态码 int status_; // rpc 状态码 int cntlstatus_; - - // rpc remote side address - std::string remoteAddress_; }; class WriteChunkClosure : public ClientClosure { public: - WriteChunkClosure(CopysetClient *client, Closure *done) - : ClientClosure(client, done) {} + WriteChunkClosure(CopysetClient* client, Closure* done) + : ClientClosure(client, done) {} void OnSuccess() override; void SendRetryRequest() override; @@ -335,8 +243,8 @@ class WriteChunkClosure : public ClientClosure { class ReadChunkClosure : public ClientClosure { public: - ReadChunkClosure(CopysetClient *client, Closure *done) - : ClientClosure(client, done) {} + ReadChunkClosure(CopysetClient* client, Closure* done) + : ClientClosure(client, done) {} void OnSuccess() override; void OnChunkNotExist() override; @@ -345,8 +253,8 @@ class ReadChunkClosure : public ClientClosure { class ReadChunkSnapClosure : public ClientClosure { public: - ReadChunkSnapClosure(CopysetClient *client, Closure *done) - : ClientClosure(client, done) {} + ReadChunkSnapClosure(CopysetClient* client, Closure* done) + : ClientClosure(client, done) {} void OnSuccess() override; void SendRetryRequest() override; @@ -354,19 +262,19 @@ class ReadChunkSnapClosure : public ClientClosure { class DeleteChunkSnapClosure : public ClientClosure { public: - DeleteChunkSnapClosure(CopysetClient *client, Closure *done) - : ClientClosure(client, done) {} + DeleteChunkSnapClosure(CopysetClient* client, Closure* done) + : ClientClosure(client, done) {} void SendRetryRequest() override; }; class GetChunkInfoClosure : public ClientClosure { public: - GetChunkInfoClosure(CopysetClient *client, Closure *done) - : ClientClosure(client, done) {} + GetChunkInfoClosure(CopysetClient* client, Closure* done) + : ClientClosure(client, done) {} void SetResponse(Message* message) override { - chunkinforesponse_.reset(dynamic_cast(message)); + chunkinforesponse_.reset(static_cast(message)); } CHUNK_OP_STATUS GetResponseStatus() const override { @@ -383,16 +291,16 @@ class GetChunkInfoClosure : public ClientClosure { class CreateCloneChunkClosure : public ClientClosure { public: - CreateCloneChunkClosure(CopysetClient *client, Closure *done) - : ClientClosure(client, done) {} + CreateCloneChunkClosure(CopysetClient* client, Closure* done) + : ClientClosure(client, done) {} void SendRetryRequest() override; }; class RecoverChunkClosure : public ClientClosure { public: - RecoverChunkClosure(CopysetClient *client, Closure *done) - : ClientClosure(client, done) {} + RecoverChunkClosure(CopysetClient* client, Closure* done) + : ClientClosure(client, done) {} void SendRetryRequest() override; }; diff --git a/src/client/client_common.h b/src/client/client_common.h index d5a0802a34..4c312355db 100644 --- a/src/client/client_common.h +++ b/src/client/client_common.h @@ -25,11 +25,9 @@ #include #include -#include +#include -#include #include -#include #include #include "include/client/libcurve.h" @@ -37,6 +35,7 @@ namespace curve { namespace client { + using ChunkID = uint64_t; using CopysetID = uint32_t; using LogicPoolID = uint32_t; @@ -73,35 +72,19 @@ enum class FileStatus { }; typedef struct ChunkIDInfo { - ChunkID cid_; - CopysetID cpid_; - LogicPoolID lpid_; - ChunkIDInfo() { - cid_ = 0; - lpid_ = 0; - cpid_ = 0; - } + ChunkID cid_ = 0; + CopysetID cpid_ = 0; + LogicPoolID lpid_ = 0; - ChunkIDInfo(ChunkID cid, LogicPoolID lpid, CopysetID cpid) { - cid_ = cid; - lpid_ = lpid; - cpid_ = cpid; - } + ChunkIDInfo() = default; - ChunkIDInfo(const ChunkIDInfo& chunkinfo) { - cid_ = chunkinfo.cid_; - lpid_ = chunkinfo.lpid_; - cpid_ = chunkinfo.cpid_; - } + ChunkIDInfo(ChunkID cid, LogicPoolID lpid, CopysetID cpid) + : cid_(cid), cpid_(cpid), lpid_(lpid) {} - ChunkIDInfo& operator=(const ChunkIDInfo& chunkinfo) { - cid_ = chunkinfo.cid_; - lpid_ = chunkinfo.lpid_; - cpid_ = chunkinfo.cpid_; - return *this; - } + ChunkIDInfo(const ChunkIDInfo& chunkinfo) = default; + ChunkIDInfo& operator=(const ChunkIDInfo& chunkinfo) = default; - bool Valid() { + bool Valid() const { return lpid_ > 0 && cpid_ > 0; } } ChunkIDInfo_t; @@ -212,7 +195,7 @@ struct ChunkServerAddr { return std::string(str); } - bool operator==(const ChunkServerAddr& other) { + bool operator==(const ChunkServerAddr& other) const { return addr_ == other.addr_; } }; diff --git a/src/client/client_config.cpp b/src/client/client_config.cpp index bdf5ea56a4..479cf16f93 100644 --- a/src/client/client_config.cpp +++ b/src/client/client_config.cpp @@ -20,13 +20,20 @@ * Author: tongguangxun */ +#include "src/client/client_config.h" + +#include + +#include #include -#include "src/client/client_config.h" -#include "src/common/string_util.h" #include "src/common/net_common.h" +#include "src/common/string_util.h" -#define RETURN_IF_FALSE(x) if (x == false) return -1; +#define RETURN_IF_FALSE(x) \ + if (x == false) { \ + return -1; \ + } namespace curve { namespace client { @@ -43,36 +50,36 @@ int ClientConfig::Init(const char* configpath) { ret = conf_.GetIntValue("global.logLevel", &fileServiceOption_.loginfo.logLevel); // NOLINT LOG_IF(ERROR, ret == false) << "config no global.logLevel info"; - RETURN_IF_FALSE(ret) + RETURN_IF_FALSE(ret); ret = conf_.GetStringValue("global.logPath", &fileServiceOption_.loginfo.logPath); // NOLINT LOG_IF(ERROR, ret == false) << "config no global.logPath info"; - RETURN_IF_FALSE(ret) + RETURN_IF_FALSE(ret); ret = conf_.GetUInt64Value("global.fileIOSplitMaxSizeKB", &fileServiceOption_.ioOpt.ioSplitOpt.fileIOSplitMaxSizeKB); LOG_IF(ERROR, ret == false) << "config no global.fileIOSplitMaxSizeKB info"; // NOLINT - RETURN_IF_FALSE(ret) + RETURN_IF_FALSE(ret); ret = conf_.GetBoolValue("chunkserver.enableAppliedIndexRead", &fileServiceOption_.ioOpt.ioSenderOpt.chunkserverEnableAppliedIndexRead); // NOLINT LOG_IF(ERROR, ret == false) << "config no chunkserver.enableAppliedIndexRead info"; // NOLINT - RETURN_IF_FALSE(ret) + RETURN_IF_FALSE(ret); ret = conf_.GetUInt32Value("chunkserver.opMaxRetry", &fileServiceOption_.ioOpt.ioSenderOpt.failRequestOpt.chunkserverOPMaxRetry); // NOLINT LOG_IF(ERROR, ret == false) << "config no chunkserver.opMaxRetry info"; - RETURN_IF_FALSE(ret) + RETURN_IF_FALSE(ret); ret = conf_.GetUInt64Value("chunkserver.opRetryIntervalUS", &fileServiceOption_.ioOpt.ioSenderOpt.failRequestOpt.chunkserverOPRetryIntervalUS); // NOLINT LOG_IF(ERROR, ret == false) << "config no chunkserver.opRetryIntervalUS info"; // NOLINT - RETURN_IF_FALSE(ret) + RETURN_IF_FALSE(ret); ret = conf_.GetUInt64Value("chunkserver.rpcTimeoutMS", &fileServiceOption_.ioOpt.ioSenderOpt.failRequestOpt.chunkserverRPCTimeoutMS); // NOLINT LOG_IF(ERROR, ret == false) << "config no chunkserver.rpcTimeoutMS info"; - RETURN_IF_FALSE(ret) + RETURN_IF_FALSE(ret); ret = conf_.GetUInt64Value("chunkserver.maxRetrySleepIntervalUS", &fileServiceOption_.ioOpt.ioSenderOpt.failRequestOpt.chunkserverMaxRetrySleepIntervalUS); // NOLINT @@ -82,20 +89,26 @@ int ClientConfig::Init(const char* configpath) { &fileServiceOption_.ioOpt.ioSenderOpt.failRequestOpt.chunkserverMaxRPCTimeoutMS); // NOLINT LOG_IF(ERROR, ret == false) << "config no chunkserver.maxRPCTimeoutMS info"; - ret = conf_.GetUInt32Value("chunkserver.maxStableTimeoutTimes", - &fileServiceOption_.ioOpt.ioSenderOpt.failRequestOpt.chunkserverUnstableOption.maxStableChunkServerTimeoutTimes); // NOLINT - LOG_IF(ERROR, ret == false) << "config no chunkserver.maxStableTimeoutTimes info"; // NOLINT - RETURN_IF_FALSE(ret) + ret = conf_.GetUInt32Value( + "chunkserver.maxStableTimeoutTimes", + &fileServiceOption_.ioOpt.metaCacheOpt.chunkserverUnstableOption.maxStableChunkServerTimeoutTimes); // NOLINT + LOG_IF(ERROR, ret == false) + << "config no chunkserver.maxStableTimeoutTimes info"; + RETURN_IF_FALSE(ret); - ret = conf_.GetUInt32Value("chunkserver.checkHealthTimeoutMs", - &fileServiceOption_.ioOpt.ioSenderOpt.failRequestOpt.chunkserverUnstableOption.checkHealthTimeoutMS); // NOLINT - LOG_IF(ERROR, ret == false) << "config no chunkserver.checkHealthTimeoutMs info"; // NOLINT - RETURN_IF_FALSE(ret) + ret = conf_.GetUInt32Value( + "chunkserver.checkHealthTimeoutMs", + &fileServiceOption_.ioOpt.metaCacheOpt.chunkserverUnstableOption.checkHealthTimeoutMS); // NOLINT + LOG_IF(ERROR, ret == false) + << "config no chunkserver.checkHealthTimeoutMs info"; // NOLINT + RETURN_IF_FALSE(ret); - ret = conf_.GetUInt32Value("chunkserver.serverStableThreshold", - &fileServiceOption_.ioOpt.ioSenderOpt.failRequestOpt.chunkserverUnstableOption.serverUnstableThreshold); // NOLINT - LOG_IF(ERROR, ret == false) << "config no chunkserver.serverStableThreshold info"; // NOLINT - RETURN_IF_FALSE(ret) + ret = conf_.GetUInt32Value( + "chunkserver.serverStableThreshold", + &fileServiceOption_.ioOpt.metaCacheOpt.chunkserverUnstableOption.serverUnstableThreshold); // NOLINT + LOG_IF(ERROR, ret == false) + << "config no chunkserver.serverStableThreshold info"; // NOLINT + RETURN_IF_FALSE(ret); ret = conf_.GetUInt64Value("chunkserver.minRetryTimesForceTimeoutBackoff", &fileServiceOption_.ioOpt.ioSenderOpt.failRequestOpt.chunkserverMinRetryTimesForceTimeoutBackoff); // NOLINT @@ -110,62 +123,61 @@ int ClientConfig::Init(const char* configpath) { ret = conf_.GetUInt64Value("global.fileMaxInFlightRPCNum", &fileServiceOption_.ioOpt.ioSenderOpt.inflightOpt.fileMaxInFlightRPCNum); // NOLINT LOG_IF(ERROR, ret == false) << "config no global.fileMaxInFlightRPCNum info"; // NOLINT - RETURN_IF_FALSE(ret) + RETURN_IF_FALSE(ret); ret = conf_.GetUInt32Value("metacache.getLeaderRetry", &fileServiceOption_.ioOpt.metaCacheOpt.metacacheGetLeaderRetry); LOG_IF(ERROR, ret == false) << "config no metacache.getLeaderRetry info"; - RETURN_IF_FALSE(ret) + RETURN_IF_FALSE(ret); ret = conf_.GetUInt32Value("metacache.rpcRetryIntervalUS", &fileServiceOption_.ioOpt.metaCacheOpt.metacacheRPCRetryIntervalUS); LOG_IF(ERROR, ret == false) << "config no metacache.rpcRetryIntervalUS info"; // NOLINT - RETURN_IF_FALSE(ret) + RETURN_IF_FALSE(ret); ret = conf_.GetUInt32Value("metacache.getLeaderTimeOutMS", &fileServiceOption_.ioOpt.metaCacheOpt.metacacheGetLeaderRPCTimeOutMS); LOG_IF(ERROR, ret == false) << "config no metacache.getLeaderTimeOutMS info"; // NOLINT - RETURN_IF_FALSE(ret) + RETURN_IF_FALSE(ret); ret = conf_.GetUInt32Value("schedule.queueCapacity", &fileServiceOption_.ioOpt.reqSchdulerOpt.scheduleQueueCapacity); LOG_IF(ERROR, ret == false) << "config no schedule.queueCapacity info"; - RETURN_IF_FALSE(ret) + RETURN_IF_FALSE(ret); ret = conf_.GetUInt32Value("schedule.threadpoolSize", &fileServiceOption_.ioOpt.reqSchdulerOpt.scheduleThreadpoolSize); LOG_IF(ERROR, ret == false) << "config no schedule.threadpoolSize info"; - RETURN_IF_FALSE(ret) + RETURN_IF_FALSE(ret); ret = conf_.GetUInt32Value("mds.refreshTimesPerLease", &fileServiceOption_.leaseOpt.mdsRefreshTimesPerLease); LOG_IF(ERROR, ret == false) << "config no mds.refreshTimesPerLease info"; - RETURN_IF_FALSE(ret) - - fileServiceOption_.ioOpt.reqSchdulerOpt.ioSenderOpt - = fileServiceOption_.ioOpt.ioSenderOpt; + RETURN_IF_FALSE(ret); + fileServiceOption_.ioOpt.reqSchdulerOpt.ioSenderOpt = + fileServiceOption_.ioOpt.ioSenderOpt; ret = conf_.GetUInt64Value("isolation.taskQueueCapacity", &fileServiceOption_.ioOpt.taskThreadOpt.isolationTaskQueueCapacity); LOG_IF(ERROR, ret == false) << "config no isolation.taskQueueCapacity info"; - RETURN_IF_FALSE(ret) + RETURN_IF_FALSE(ret); ret = conf_.GetUInt32Value("isolation.taskThreadPoolSize", &fileServiceOption_.ioOpt.taskThreadOpt.isolationTaskThreadPoolSize); LOG_IF(ERROR, ret == false) << "config no isolation.taskThreadPoolSize info"; // NOLINT - RETURN_IF_FALSE(ret) + RETURN_IF_FALSE(ret); std::string metaAddr; ret = conf_.GetStringValue("mds.listen.addr", &metaAddr); LOG_IF(ERROR, ret == false) << "config no mds.listen.addr info"; - RETURN_IF_FALSE(ret) + RETURN_IF_FALSE(ret); std::vector mdsAddr; common::SplitString(metaAddr, ",", &mdsAddr); - fileServiceOption_.metaServerOpt.metaaddrvec.assign(mdsAddr.begin(), + fileServiceOption_.metaServerOpt.mdsAddrs.assign(mdsAddr.begin(), mdsAddr.end()); - for (auto& addr : fileServiceOption_.metaServerOpt.metaaddrvec) { + for (auto& addr : fileServiceOption_.metaServerOpt.mdsAddrs) { if (!curve::common::NetCommon::CheckAddressValid(addr)) { LOG(ERROR) << "address valid!"; return -1; @@ -175,17 +187,17 @@ int ClientConfig::Init(const char* configpath) { ret = conf_.GetUInt64Value("mds.rpcTimeoutMS", &fileServiceOption_.metaServerOpt.mdsRPCTimeoutMs); LOG_IF(ERROR, ret == false) << "config no mds.rpcTimeoutMS info"; - RETURN_IF_FALSE(ret) + RETURN_IF_FALSE(ret); ret = conf_.GetUInt32Value("mds.rpcRetryIntervalUS", &fileServiceOption_.metaServerOpt.mdsRPCRetryIntervalUS); LOG_IF(ERROR, ret == false) << "config no mds.rpcRetryIntervalUS info"; - RETURN_IF_FALSE(ret) + RETURN_IF_FALSE(ret); ret = conf_.GetUInt64Value("mds.maxRPCTimeoutMS", &fileServiceOption_.metaServerOpt.mdsMaxRPCTimeoutMS); LOG_IF(ERROR, ret == false) << "config no mds.maxRPCTimeoutMS info"; - RETURN_IF_FALSE(ret) + RETURN_IF_FALSE(ret); ret = conf_.GetUInt64Value("mds.maxRetryMS", &fileServiceOption_.metaServerOpt.mdsMaxRetryMS); @@ -198,7 +210,7 @@ int ClientConfig::Init(const char* configpath) { ret = conf_.GetBoolValue("mds.registerToMDS", &fileServiceOption_.commonOpt.mdsRegisterToMDS); LOG_IF(ERROR, ret == false) << "config no mds.registerToMDS info"; - RETURN_IF_FALSE(ret) + RETURN_IF_FALSE(ret); ret = conf_.GetBoolValue("global.turnOffHealthCheck", &fileServiceOption_.commonOpt.turnOffHealthCheck); @@ -209,10 +221,6 @@ int ClientConfig::Init(const char* configpath) { return 0; } -FileServiceOption_t ClientConfig::GetFileServiceOption() { - return fileServiceOption_; -} - uint16_t ClientConfig::GetDummyserverStartPort() { return conf_.GetIntValue("global.metricDummyServerStartPort", 9000); } diff --git a/src/client/client_config.h b/src/client/client_config.h index feb539fcd1..2721a80f27 100644 --- a/src/client/client_config.h +++ b/src/client/client_config.h @@ -23,25 +23,28 @@ #ifndef SRC_CLIENT_CLIENT_CONFIG_H_ #define SRC_CLIENT_CLIENT_CONFIG_H_ -#include -#include -#include "src/common/configuration.h" #include "src/client/config_info.h" +#include "src/common/configuration.h" namespace curve { namespace client { + class ClientConfig { public: int Init(const char* configpath); - FileServiceOption_t GetFileServiceOption(); - uint16_t GetDummyserverStartPort(); + FileServiceOption GetFileServiceOption() const { + return fileServiceOption_; + } + + uint16_t GetDummyserverStartPort(); private: - FileServiceOption_t fileServiceOption_; - common::Configuration conf_; + FileServiceOption fileServiceOption_; + common::Configuration conf_; }; -} // namespace client -} // namespace curve + +} // namespace client +} // namespace curve #endif // SRC_CLIENT_CLIENT_CONFIG_H_ diff --git a/src/client/client_metric.h b/src/client/client_metric.h index a369200ce3..ead3da7258 100644 --- a/src/client/client_metric.h +++ b/src/client/client_metric.h @@ -25,9 +25,7 @@ #include -#include #include -#include #include #include "src/common/timeutility.h" @@ -50,6 +48,7 @@ static uint64_t GetUnInt64Value(void* arg) { struct IOSuspendMetric { // 当前persecond计数总数 bvar::Adder count; + IOSuspendMetric(const std::string& prefix, const std::string& name) : count(prefix, name + "_total_count") {} }; @@ -60,6 +59,7 @@ struct PerSecondMetric { bvar::Adder count; // persecond真实数据,这个数据依赖于count bvar::PerSecond> value; + PerSecondMetric(const std::string& prefix, const std::string& name) : count(prefix, name + "_total_count"), value(prefix, name, &count, 1) {} @@ -94,16 +94,17 @@ struct InterfaceMetric { // 文件级别metric信息统计 struct FileMetric { + const std::string prefix = "curve_client"; + // 当前metric归属于哪个文件 std::string filename; - const std::string prefix = "curve client"; // 当前文件inflight io数量 bvar::Adder inflightRPCNum; // 当前文件请求的最大请求字节数,这种统计方式可以很方便的看到最大值,分位值 - bvar::LatencyRecorder writeSizeRecorder; bvar::LatencyRecorder readSizeRecorder; + bvar::LatencyRecorder writeSizeRecorder; // libcurve最底层read rpc接口统计信息metric统计 InterfaceMetric readRPC; @@ -121,20 +122,20 @@ struct FileMetric { explicit FileMetric(const std::string& name) : filename(name), - userRead(prefix, filename + "_read"), - userWrite(prefix, filename + "_write"), + inflightRPCNum(prefix, filename + "_inflight_rpc_num"), + readSizeRecorder(prefix, filename + "_read_request_size_recoder"), + writeSizeRecorder(prefix, filename + "_write_request_size_recoder"), readRPC(prefix, filename + "_read_rpc"), writeRPC(prefix, filename + "_write_rpc"), - inflightRPCNum(prefix, filename + "_inflight_rpc_num"), + userRead(prefix, filename + "_read"), + userWrite(prefix, filename + "_write"), getLeaderRetryQPS(prefix, filename + "_get_leader_retry_rpc"), - writeSizeRecorder(prefix, filename + "_write_request_size_recoder"), - readSizeRecorder(prefix, filename + "_read_request_size_recoder"), suspendRPCMetric(prefix, filename + "_suspend_io_num") {} }; // 用于全局mds接口统计信息调用信息统计 struct MDSClientMetric { - const std::string prefix = "curve mds client"; + const std::string prefix = "curve_mds_client"; // mds的地址信息 std::string metaserverAddr; @@ -177,7 +178,6 @@ struct MDSClientMetric { MDSClientMetric() : metaserverAddress(prefix, "current_metaserver_addr", GetStringValue, &metaserverAddr), - mdsServerChangeTimes(prefix, "mds_server_change_times"), openFile(prefix, "openFile"), createFile(prefix, "createFile"), closeFile(prefix, "closeFile"), @@ -192,20 +192,21 @@ struct MDSClientMetric { listDir(prefix, "listDir"), registerClient(prefix, "registerClient"), getChunkServerId(prefix, "GetChunkServerId"), - listChunkserverInServer(prefix, "ListChunkServerInServer") {} + listChunkserverInServer(prefix, "ListChunkServerInServer"), + mdsServerChangeTimes(prefix, "mds_server_change_times") {} }; struct LatencyGuard { bvar::LatencyRecorder* latencyRec; - uint64_t timeelapse; + uint64_t startTimeUs; + explicit LatencyGuard(bvar::LatencyRecorder* latency) { latencyRec = latency; - timeelapse = TimeUtility::GetTimeofDayUs(); + startTimeUs = TimeUtility::GetTimeofDayUs(); } ~LatencyGuard() { - timeelapse = TimeUtility::GetTimeofDayUs() - timeelapse; - *latencyRec << timeelapse; + *latencyRec << (TimeUtility::GetTimeofDayUs() - startTimeUs); } }; diff --git a/src/client/config_info.h b/src/client/config_info.h index fb45c11cef..302078292b 100644 --- a/src/client/config_info.h +++ b/src/client/config_info.h @@ -27,29 +27,26 @@ #include #include +namespace curve { +namespace client { + /** * log的基本配置信息 * @logLevel: 是log打印等级 * @logPath: log打印位置 */ -typedef struct LogInfo { - int logLevel; +struct LogInfo { + int logLevel = 2; std::string logPath; - LogInfo() { - logLevel = 2; - } -} LogInfo_t; +}; /** * in flight IO控制信息 * @fileMaxInFlightRPCNum: 为一个文件中最大允许的inflight IO数量 */ -typedef struct InFlightIOCntlInfo { - uint64_t fileMaxInFlightRPCNum; - InFlightIOCntlInfo() { - fileMaxInFlightRPCNum = 2048; - } -} InFlightIOCntlInfo_t; +struct InFlightIOCntlInfo { + uint64_t fileMaxInFlightRPCNum = 2048; +}; /** * mds client的基本配置 @@ -62,23 +59,16 @@ typedef struct InFlightIOCntlInfo { * mds重试,不会立即重试请求,而是选择睡眠一小段时间之后再去发RPC。 * @mdsMaxFailedTimesBeforeChangeMDS: 如果重试的rpc在一个mds节点上连续失败超过该值 * 就需要主动触发切换mds再重试。 - * @metaaddrvec: mds server地址,存放mds集群的多个地址信息 + * @mdsAddrs: mds server地址,存放mds集群的多个地址信息 */ -typedef struct MetaServerOption { - uint64_t mdsMaxRetryMS; - uint64_t mdsRPCTimeoutMs; - uint64_t mdsMaxRPCTimeoutMS; - uint32_t mdsRPCRetryIntervalUS; - uint32_t mdsMaxFailedTimesBeforeChangeMDS; - std::vector metaaddrvec; - MetaServerOption() { - mdsMaxRetryMS = 8000; - mdsMaxRPCTimeoutMS = 2000; - mdsRPCTimeoutMs = 500; - mdsRPCRetryIntervalUS = 50000; - mdsMaxFailedTimesBeforeChangeMDS = 5; - } -} MetaServerOption_t; +struct MetaServerOption { + uint64_t mdsMaxRetryMS = 8000; + uint64_t mdsMaxRPCTimeoutMS = 2000; + uint64_t mdsRPCTimeoutMs = 500; + uint32_t mdsRPCRetryIntervalUS = 50000; + uint32_t mdsMaxFailedTimesBeforeChangeMDS = 5; + std::vector mdsAddrs; +}; /** * 租约基本配置 @@ -88,12 +78,9 @@ typedef struct MetaServerOption { * 那么client认为当前mds存在异常,会阻塞后续的IO,直到 * 续约成功。 */ -typedef struct LeaseOption { - uint32_t mdsRefreshTimesPerLease; - LeaseOption() { - mdsRefreshTimesPerLease = 5; - } -} LeaseOption_t; +struct LeaseOption { + uint32_t mdsRefreshTimesPerLease = 5; +}; /** * rpc超时,判断是否unstable的参数 @@ -107,9 +94,9 @@ typedef struct LeaseOption { * 整个server上的所有chunkserver都标记为unstable */ struct ChunkServerUnstableOption { - uint32_t maxStableChunkServerTimeoutTimes{64}; - uint32_t checkHealthTimeoutMS{100}; - uint32_t serverUnstableThreshold{3}; + uint32_t maxStableChunkServerTimeoutTimes = 64; + uint32_t checkHealthTimeoutMS = 100; + uint32_t serverUnstableThreshold = 3; }; /** @@ -144,26 +131,15 @@ struct ChunkServerUnstableOption { * 失败超过该阈值的时候,可以认为当前IO处于悬挂状态,通过metric * 向上报警。 */ -typedef struct FailureRequestOption { - uint32_t chunkserverOPMaxRetry; - uint64_t chunkserverOPRetryIntervalUS; - uint64_t chunkserverRPCTimeoutMS; - uint64_t chunkserverMaxRPCTimeoutMS; - uint64_t chunkserverMaxRetrySleepIntervalUS; - uint64_t chunkserverMinRetryTimesForceTimeoutBackoff; - uint64_t chunkserverMaxRetryTimesBeforeConsiderSuspend; - ChunkServerUnstableOption chunkserverUnstableOption; - - FailureRequestOption() { - chunkserverOPMaxRetry = 3; - chunkserverOPRetryIntervalUS = 200; - chunkserverRPCTimeoutMS = 1000; - chunkserverMaxRPCTimeoutMS = 64000; - chunkserverMaxRetrySleepIntervalUS = 64 * 1000 * 1000; - chunkserverMinRetryTimesForceTimeoutBackoff = 5; - chunkserverMaxRetryTimesBeforeConsiderSuspend = 20; - } -} FailureRequestOption_t; +struct FailureRequestOption { + uint32_t chunkserverOPMaxRetry = 3; + uint64_t chunkserverOPRetryIntervalUS = 200; + uint64_t chunkserverRPCTimeoutMS = 1000; + uint64_t chunkserverMaxRPCTimeoutMS = 64000; + uint64_t chunkserverMaxRetrySleepIntervalUS = 64ull * 1000 * 1000; + uint64_t chunkserverMinRetryTimesForceTimeoutBackoff = 5; + uint64_t chunkserverMaxRetryTimesBeforeConsiderSuspend = 20; +}; /** * 发送rpc给chunkserver的配置 @@ -171,11 +147,11 @@ typedef struct FailureRequestOption { * @inflightOpt: 一个文件向chunkserver发送请求时的inflight 请求控制配置 * @failRequestOpt: rpc发送失败之后,需要进行rpc重试的相关配置 */ -typedef struct IOSenderOption { +struct IOSenderOption { bool chunkserverEnableAppliedIndexRead; - InFlightIOCntlInfo_t inflightOpt; - FailureRequestOption_t failRequestOpt; -} IOSenderOption_t; + InFlightIOCntlInfo inflightOpt; + FailureRequestOption failRequestOpt; +}; /** * scheduler模块基本配置信息,schedule模块是用于分发用户请求,每个文件有自己的schedule @@ -183,15 +159,11 @@ typedef struct IOSenderOption { * @scheduleQueueCapacity: schedule模块配置的队列深度 * @scheduleThreadpoolSize: schedule模块线程池大小 */ -typedef struct RequestScheduleOption { - uint32_t scheduleQueueCapacity; - uint32_t scheduleThreadpoolSize; - IOSenderOption_t ioSenderOpt; - RequestScheduleOption() { - scheduleQueueCapacity = 1024; - scheduleThreadpoolSize = 2; - } -} RequestScheduleOption_t; +struct RequestScheduleOption { + uint32_t scheduleQueueCapacity = 1024; + uint32_t scheduleThreadpoolSize = 2; + IOSenderOption ioSenderOpt; +}; /** * metaccache模块配置信息 @@ -211,32 +183,23 @@ typedef struct RequestScheduleOption { * @metacacheGetLeaderBackupRequestLbName: 为getleader backup rpc * 选择底层服务节点的策略 */ -typedef struct MetaCacheOption { - uint32_t metacacheGetLeaderRetry; - uint32_t metacacheRPCRetryIntervalUS; - uint32_t metacacheGetLeaderRPCTimeOutMS; - uint32_t metacacheGetLeaderBackupRequestMS; - std::string metacacheGetLeaderBackupRequestLbName; - MetaCacheOption() { - metacacheGetLeaderRetry = 3; - metacacheRPCRetryIntervalUS = 500; - metacacheGetLeaderRPCTimeOutMS = 1000; - metacacheGetLeaderBackupRequestMS = 100; - metacacheGetLeaderBackupRequestLbName = "rr"; - } -} MetaCacheOption_t; +struct MetaCacheOption { + uint32_t metacacheGetLeaderRetry = 3; + uint32_t metacacheRPCRetryIntervalUS = 500; + uint32_t metacacheGetLeaderRPCTimeOutMS = 1000; + uint32_t metacacheGetLeaderBackupRequestMS = 100; + std::string metacacheGetLeaderBackupRequestLbName = "rr"; + ChunkServerUnstableOption chunkserverUnstableOption; +}; /** * IO 拆分模块配置信息 * @fileIOSplitMaxSizeKB: 用户下发IO大小client没有限制,但是client会将用户的IO进行拆分, * 发向同一个chunkserver的请求锁携带的数据大小不能超过该值。 */ -typedef struct IOSplitOPtion { - uint64_t fileIOSplitMaxSizeKB; - IOSplitOPtion() { - fileIOSplitMaxSizeKB = 64; - } -} IOSplitOPtion_t; +struct IOSplitOption { + uint64_t fileIOSplitMaxSizeKB = 64; +}; /** * 线程隔离任务队列配置信息 @@ -245,25 +208,21 @@ typedef struct IOSplitOPtion { * @isolationTaskQueueCapacity: 隔离线程池的队列深度 * @isolationTaskThreadPoolSize: 隔离线程池容量 */ -typedef struct TaskThreadOption { - uint64_t isolationTaskQueueCapacity; - uint32_t isolationTaskThreadPoolSize; - TaskThreadOption() { - isolationTaskQueueCapacity = 500000; - isolationTaskThreadPoolSize = 1; - } -} TaskThreadOption_t; +struct TaskThreadOption { + uint64_t isolationTaskQueueCapacity = 500000; + uint32_t isolationTaskThreadPoolSize = 1; +}; /** * IOOption存储了当前io 操作所需要的所有配置信息 */ -typedef struct IOOption { - IOSplitOPtion_t ioSplitOpt; - IOSenderOption_t ioSenderOpt; - MetaCacheOption_t metaCacheOpt; - TaskThreadOption_t taskThreadOpt; - RequestScheduleOption_t reqSchdulerOpt; -} IOOption_t; +struct IOOption { + IOSplitOption ioSplitOpt; + IOSenderOption ioSenderOpt; + MetaCacheOption metaCacheOpt; + TaskThreadOption taskThreadOpt; + RequestScheduleOption reqSchdulerOpt; +}; /** * client一侧常规的共同的配置信息 @@ -272,30 +231,33 @@ typedef struct IOOption { * ip和端口信息发送给mds。 * @turnOffHealthCheck: 是否关闭健康检查 */ -typedef struct CommonConfigOpt { - bool mdsRegisterToMDS{false}; - bool turnOffHealthCheck{false}; -} CommonConfigOpt_t; +struct CommonConfigOpt { + bool mdsRegisterToMDS = false; + bool turnOffHealthCheck = false; +}; /** * ClientConfigOption是外围快照系统需要设置的配置信息 */ -typedef struct ClientConfigOption { - LogInfo_t loginfo; - IOOption_t ioOpt; - CommonConfigOpt_t commonOpt; - MetaServerOption_t metaServerOpt; -} ClientConfigOption_t; +struct ClientConfigOption { + LogInfo loginfo; + IOOption ioOpt; + CommonConfigOpt commonOpt; + MetaServerOption metaServerOpt; +}; /** * FileServiceOption是QEMU侧总体配置信息 */ -typedef struct FileServiceOption { - LogInfo_t loginfo; - IOOption_t ioOpt; - LeaseOption_t leaseOpt; - CommonConfigOpt_t commonOpt; - MetaServerOption_t metaServerOpt; -} FileServiceOption_t; +struct FileServiceOption { + LogInfo loginfo; + IOOption ioOpt; + LeaseOption leaseOpt; + CommonConfigOpt commonOpt; + MetaServerOption metaServerOpt; +}; + +} // namespace client +} // namespace curve #endif // SRC_CLIENT_CONFIG_INFO_H_ diff --git a/src/client/copyset_client.cpp b/src/client/copyset_client.cpp index 86d662479d..2331025221 100644 --- a/src/client/copyset_client.cpp +++ b/src/client/copyset_client.cpp @@ -33,11 +33,11 @@ #include "src/client/request_scheduler.h" #include "src/client/request_closure.h" -using google::protobuf::Closure; namespace curve { namespace client { + int CopysetClient::Init(MetaCache *metaCache, - const IOSenderOption_t& ioSenderOpt, RequestScheduler* scheduler, + const IOSenderOption& ioSenderOpt, RequestScheduler* scheduler, FileMetric* fileMetric) { if (nullptr == metaCache || scheduler == nullptr) { LOG(ERROR) << "metacache or scheduler is null!"; diff --git a/src/client/copyset_client.h b/src/client/copyset_client.h index 83a6adee08..ac72b36277 100644 --- a/src/client/copyset_client.h +++ b/src/client/copyset_client.h @@ -24,29 +24,22 @@ #define SRC_CLIENT_COPYSET_CLIENT_H_ #include -#include -#include -#include #include #include #include -#include "src/common/concurrent/concurrent.h" -#include "src/client/client_metric.h" -#include "src/client/request_context.h" +#include "include/curve_compiler_specific.h" #include "src/client/client_common.h" +#include "src/client/client_metric.h" #include "src/client/config_info.h" -#include "src/common/uncopyable.h" +#include "src/client/request_context.h" #include "src/client/request_sender_manager.h" -#include "include/curve_compiler_specific.h" -#include "src/client/inflight_controller.h" +#include "src/common/concurrent/concurrent.h" namespace curve { namespace client { -using curve::common::Mutex; -using curve::common::ConditionVariable; using curve::common::Uncopyable; using ::google::protobuf::Closure; @@ -57,7 +50,7 @@ class RequestScheduler; * 负责管理 ChunkServer 的链接,向上层提供访问 * 指定 copyset 的 chunk 的 read/write 等接口 */ -class CopysetClient : public Uncopyable { +class CopysetClient { public: CopysetClient() : sessionNotValid_(false), @@ -66,19 +59,24 @@ class CopysetClient : public Uncopyable { scheduler_(nullptr), exitFlag_(false) {} + CopysetClient(const CopysetClient&) = delete; + CopysetClient& operator=(const CopysetClient&) = delete; + virtual ~CopysetClient() { delete senderManager_; senderManager_ = nullptr; } int Init(MetaCache *metaCache, - const IOSenderOption_t& ioSenderOpt, + const IOSenderOption& ioSenderOpt, RequestScheduler* scheduler = nullptr, FileMetric* fileMetic = nullptr); /** * 返回依赖的Meta Cache */ - MetaCache *GetMetaCache() { return metaCache_; } + MetaCache* GetMetaCache() { + return metaCache_; + } /** * 读Chunk @@ -194,7 +192,6 @@ class CopysetClient : public Uncopyable { sessionNotValid_ = true; } - /** * session恢复通知不再回收重试的RPC */ @@ -241,7 +238,7 @@ class CopysetClient : public Uncopyable { // 所有ChunkServer的链接管理者 RequestSenderManager *senderManager_; // 配置 - IOSenderOption_t iosenderopt_; + IOSenderOption iosenderopt_; // session是否有效,如果session无效那么需要将重试的RPC停住 // RPC停住通过将这个rpc重新push到request scheduler队列,这样不会 diff --git a/src/client/file_instance.cpp b/src/client/file_instance.cpp index 973dab7d14..288a52b2f2 100644 --- a/src/client/file_instance.cpp +++ b/src/client/file_instance.cpp @@ -20,35 +20,39 @@ * Author: tongguangxun */ -#include -#include +#include "src/client/file_instance.h" + #include +#include -#include "proto/topology.pb.h" #include "proto/nameserver2.pb.h" -#include "src/common/timeutility.h" -#include "src/client/file_instance.h" -#include "src/client/metacache.h" -#include "src/client/mds_client.h" +#include "proto/topology.pb.h" #include "src/client/iomanager4file.h" +#include "src/client/mds_client.h" +#include "src/client/metacache.h" #include "src/client/request_scheduler.h" #include "src/client/request_sender_manager.h" +#include "src/common/timeutility.h" + +namespace curve { +namespace client { using curve::client::ClientConfig; using curve::common::TimeUtility; using curve::mds::SessionStatus; -namespace curve { -namespace client { -FileInstance::FileInstance() { - finfo_.chunksize = 4 * 1024 * 1024; - finfo_.segmentsize = 1 * 1024 * 1024 * 1024ul; -} +FileInstance::FileInstance() + : finfo_(), + fileopt_(), + mdsclient_(nullptr), + leaseExecutor_(), + iomanager4file_(), + readonly_(false) {} bool FileInstance::Initialize(const std::string& filename, MDSClient* mdsclient, const UserInfo_t& userinfo, - FileServiceOption_t fileservicopt, + const FileServiceOption& fileservicopt, bool readonly) { readonly_ = readonly; fileopt_ = fileservicopt; @@ -143,7 +147,7 @@ int FileInstance::Open(const std::string& filename, ret = mdsclient_->OpenFile(filename, finfo_.userinfo, &finfo_, &lease); if (ret == LIBCURVE_ERROR::OK) { ret = leaseExecutor_->Start(finfo_, lease) ? LIBCURVE_ERROR::OK - : LIBCURVE_ERROR::FAILED; + : LIBCURVE_ERROR::FAILED; if (nullptr != sessionId) { sessionId->assign(lease.sessionID); } @@ -175,5 +179,31 @@ int FileInstance::Close() { return -ret; } +FileInstance* FileInstance::NewInitedFileInstance( + const FileServiceOption& fileServiceOption, + MDSClient* mdsClient, + const std::string& filename, + const UserInfo& userInfo, + bool readonly) { + FileInstance* instance = new (std::nothrow) FileInstance(); + if (instance == nullptr) { + LOG(ERROR) << "Create FileInstance failed, filename: " << filename; + return nullptr; + } + + bool ret = instance->Initialize(filename, mdsClient, userInfo, + fileServiceOption, readonly); + if (!ret) { + LOG(ERROR) << "FileInstance initialize failed" + << ", filename = " << filename + << ", ower = " << userInfo.owner + << ", readonly = " << readonly; + delete instance; + return nullptr; + } + + return instance; +} + } // namespace client } // namespace curve diff --git a/src/client/file_instance.h b/src/client/file_instance.h index 9c0ac42b45..e729ee6a41 100644 --- a/src/client/file_instance.h +++ b/src/client/file_instance.h @@ -22,15 +22,8 @@ #ifndef SRC_CLIENT_FILE_INSTANCE_H_ #define SRC_CLIENT_FILE_INSTANCE_H_ -#include -#include - -#include -#include // NOLINT #include -#include //NOLINT #include -#include #include "src/client/mds_client.h" #include "include/client/libcurve.h" @@ -42,6 +35,7 @@ namespace curve { namespace client { + class CURVE_CACHELINE_ALIGNMENT FileInstance { public: FileInstance(); @@ -60,7 +54,7 @@ class CURVE_CACHELINE_ALIGNMENT FileInstance { bool Initialize(const std::string& filename, MDSClient* mdsclient, const UserInfo_t& userinfo, - FileServiceOption_t fileservicopt, + const FileServiceOption& fileservicopt, bool readonly = false); /** * 打开文件 @@ -119,13 +113,15 @@ class CURVE_CACHELINE_ALIGNMENT FileInstance { void UnInitialize(); - IOManager4File* GetIOManager4File() {return &iomanager4file_;} + IOManager4File* GetIOManager4File() { + return &iomanager4file_; + } /** * 获取lease, 测试代码使用 */ - LeaseExecutor* GetLeaseExecutor() { - return leaseExecutor_.get(); + LeaseExecutor* GetLeaseExecutor() const { + return leaseExecutor_.get(); } int GetFileInfo(const std::string& filename, FInfo_t* fi); @@ -136,15 +132,22 @@ class CURVE_CACHELINE_ALIGNMENT FileInstance { * @return 当前instance对应文件的信息 */ FInfo GetCurrentFileInfo() const { - return finfo_; + return finfo_; } + static FileInstance* NewInitedFileInstance( + const FileServiceOption& fileServiceOption, + MDSClient* mdsClient, + const std::string& filename, + const UserInfo& userInfo, + bool readonly); + private: // 保存当前file的文件信息 FInfo_t finfo_; // 当前FileInstance的初始化配置信息 - FileServiceOption_t fileopt_; + FileServiceOption fileopt_; // MDSClient是FileInstance与mds通信的唯一出口 MDSClient* mdsclient_; @@ -158,6 +161,8 @@ class CURVE_CACHELINE_ALIGNMENT FileInstance { // 是否为只读方式 bool readonly_; }; + } // namespace client } // namespace curve + #endif // SRC_CLIENT_FILE_INSTANCE_H_ diff --git a/src/client/inflight_controller.h b/src/client/inflight_controller.h index 2eb46c00fb..5c59f4edce 100644 --- a/src/client/inflight_controller.h +++ b/src/client/inflight_controller.h @@ -25,23 +25,23 @@ #include "src/common/concurrent/concurrent.h" +namespace curve { +namespace client { + using curve::common::Mutex; using curve::common::ConditionVariable; -namespace curve { -namespace client { class InflightControl { public: - InflightControl() { - curInflightIONum_.store(0); - } + InflightControl() = default; void SetMaxInflightNum(uint64_t maxInflightNum) { maxInflightNum_ = maxInflightNum; } /** - * 调用该接口等待inflight全部回来,这段期间是hang的 + * @brief 调用该接口等待inflight全部回来,这段期间是hang的, + * 在close文件时调用 */ void WaitInflightAllComeBack() { LOG(INFO) << "wait inflight to complete, count = " << curInflightIONum_; @@ -53,33 +53,36 @@ class InflightControl { } /** - * 调用该接口等待inflight回来,这段期间是hang的 + * @brief 调用该接口等待inflight回来,这段期间是hang的 */ void WaitInflightComeBack() { - if (curInflightIONum_.load() >= maxInflightNum_) { + if (curInflightIONum_.load(std::memory_order_acquire) >= + maxInflightNum_) { std::unique_lock lk(inflightComeBackmtx_); inflightComeBackcv_.wait(lk, [this]() { - return curInflightIONum_.load() < maxInflightNum_; + return curInflightIONum_.load(std::memory_order_acquire) < + maxInflightNum_; }); } } /** - * 递增inflight num + * @brief 递增inflight num */ void IncremInflightNum() { curInflightIONum_.fetch_add(1, std::memory_order_release); } /** - * 递减inflight num + * @brief 递减inflight num */ void DecremInflightNum() { std::lock_guard lk(inflightComeBackmtx_); { std::lock_guard lk(inflightAllComeBackmtx_); - curInflightIONum_.fetch_sub(1, std::memory_order_release); - if (curInflightIONum_.load() == 0) { + const auto cnt = + curInflightIONum_.fetch_sub(1, std::memory_order_acq_rel); + if (cnt == 1) { inflightAllComeBackcv_.notify_all(); } } @@ -106,13 +109,17 @@ class InflightControl { DecremInflightNum(); } + /** + * @brief Get current inflight io num, only use in test code + */ uint64_t GetCurrentInflightNum() const { - return curInflightIONum_; + return curInflightIONum_.load(std::memory_order_acquire); } private: - uint64_t maxInflightNum_; - std::atomic curInflightIONum_; + uint64_t maxInflightNum_ = 0; + std::atomic curInflightIONum_{0}; + Mutex inflightComeBackmtx_; ConditionVariable inflightComeBackcv_; Mutex inflightAllComeBackmtx_; diff --git a/src/client/io_condition_varaiable.h b/src/client/io_condition_varaiable.h index ce1b4eaed1..a220168db3 100644 --- a/src/client/io_condition_varaiable.h +++ b/src/client/io_condition_varaiable.h @@ -70,6 +70,8 @@ class IOConditionVariable { // 条件变量用于等待 std::condition_variable cv_; }; + } // namespace client } // namespace curve + #endif // SRC_CLIENT_IO_CONDITION_VARAIABLE_H_ diff --git a/src/client/io_tracker.cpp b/src/client/io_tracker.cpp index 329523b028..0214017540 100644 --- a/src/client/io_tracker.cpp +++ b/src/client/io_tracker.cpp @@ -31,22 +31,22 @@ #include "src/client/request_closure.h" #include "src/common/timeutility.h" -using curve::chunkserver::CHUNK_OP_STATUS; - namespace curve { namespace client { +using curve::chunkserver::CHUNK_OP_STATUS; + std::atomic IOTracker::tracekerID_(1); IOTracker::IOTracker(IOManager* iomanager, - MetaCache* mc, - RequestScheduler* scheduler, - FileMetric* clientMetric): - mc_(mc), - iomanager_(iomanager), - scheduler_(scheduler), - fileMetric_(clientMetric) { - id_ = tracekerID_.fetch_add(1); + MetaCache* mc, + RequestScheduler* scheduler, + FileMetric* clientMetric) + : mc_(mc), + iomanager_(iomanager), + scheduler_(scheduler), + fileMetric_(clientMetric) { + id_ = tracekerID_.fetch_add(1, std::memory_order_relaxed); scc_ = nullptr; aioctx_ = nullptr; data_ = nullptr; @@ -214,7 +214,7 @@ void IOTracker::DeleteSnapChunkOrCorrectSn(const ChunkIDInfo &cinfo, int ret = -1; do { - RequestContext* newreqNode = GetInitedRequestContext(); + RequestContext* newreqNode = RequestContext::NewInitedRequestContext(); if (newreqNode == nullptr) { break; } @@ -241,7 +241,7 @@ void IOTracker::GetChunkInfo(const ChunkIDInfo &cinfo, int ret = -1; do { - RequestContext* newreqNode = GetInitedRequestContext(); + RequestContext* newreqNode = RequestContext::NewInitedRequestContext(); if (newreqNode == nullptr) { break; } @@ -271,7 +271,7 @@ void IOTracker::CreateCloneChunk(const std::string& location, int ret = -1; do { - RequestContext* newreqNode = GetInitedRequestContext(); + RequestContext* newreqNode = RequestContext::NewInitedRequestContext(); if (newreqNode == nullptr) { break; } @@ -302,7 +302,7 @@ void IOTracker::RecoverChunk(const ChunkIDInfo& cinfo, uint64_t offset, int ret = -1; do { - RequestContext* newreqNode = GetInitedRequestContext(); + RequestContext* newreqNode = RequestContext::NewInitedRequestContext(); if (newreqNode == nullptr) { break; } @@ -469,16 +469,5 @@ void IOTracker::ChunkServerErr2LibcurveErr(CHUNK_OP_STATUS errcode, } } -RequestContext* IOTracker::GetInitedRequestContext() const { - RequestContext* reqNode = new (std::nothrow) RequestContext(); - if (reqNode != nullptr && reqNode->Init()) { - return reqNode; - } else { - LOG(ERROR) << "allocate req node failed!"; - delete reqNode; - return nullptr; - } -} - } // namespace client } // namespace curve diff --git a/src/client/io_tracker.h b/src/client/io_tracker.h index c48101b841..e88b468062 100644 --- a/src/client/io_tracker.h +++ b/src/client/io_tracker.h @@ -24,7 +24,6 @@ #include -#include #include #include #include diff --git a/src/client/iomanager.h b/src/client/iomanager.h index 6eed0115d0..e985b1527f 100644 --- a/src/client/iomanager.h +++ b/src/client/iomanager.h @@ -27,21 +27,22 @@ #include "src/client/client_common.h" #include "src/common/concurrent/concurrent.h" -using curve::common::Atomic; - namespace curve { namespace client { + +using curve::common::Atomic; + class IOManager { public: IOManager() { - id_ = idRecorder_.fetch_add(1); + id_ = idRecorder_.fetch_add(1, std::memory_order_relaxed); } virtual ~IOManager() = default; /** - * 获取当前iomanager的ID信息 + * @brief 获取当前iomanager的ID信息 */ - virtual IOManagerID ID() { + virtual IOManagerID ID() const { return id_; } @@ -60,7 +61,7 @@ class IOManager { } /** - * 处理异步返回的response + * @brief 处理异步返回的response * @param: iotracker是当前reponse的归属 */ virtual void HandleAsyncIOResponse(IOTracker* iotracker) = 0; @@ -69,9 +70,11 @@ class IOManager { // iomanager id目的是为了让底层RPC知道自己归属于哪个iomanager IOManagerID id_; + private: // global id recorder static Atomic idRecorder_; }; + } // namespace client } // namespace curve diff --git a/src/client/iomanager4chunk.cpp b/src/client/iomanager4chunk.cpp index 744c009381..4a2c990a5c 100644 --- a/src/client/iomanager4chunk.cpp +++ b/src/client/iomanager4chunk.cpp @@ -30,7 +30,7 @@ namespace client { IOManager4Chunk::IOManager4Chunk() { } -bool IOManager4Chunk::Initialize(IOOption_t ioOpt, MDSClient* mdsclient) { +bool IOManager4Chunk::Initialize(IOOption ioOpt, MDSClient* mdsclient) { ioopt_ = ioOpt; mc_.Init(ioopt_.metaCacheOpt, mdsclient); Splitor::Init(ioopt_.ioSplitOpt); diff --git a/src/client/iomanager4chunk.h b/src/client/iomanager4chunk.h index d457cd36bf..f9cedeca02 100644 --- a/src/client/iomanager4chunk.h +++ b/src/client/iomanager4chunk.h @@ -41,7 +41,7 @@ class IOManager4Chunk : public IOManager { public: IOManager4Chunk(); ~IOManager4Chunk() = default; - bool Initialize(IOOption_t ioOpt, MDSClient* mdsclient); + bool Initialize(IOOption ioOpt, MDSClient* mdsclient); /** * 读取seq版本号的快照数据 @@ -133,7 +133,7 @@ class IOManager4Chunk : public IOManager { private: // 每个IOManager都有其IO配置,保存在iooption里 - IOOption_t ioopt_; + IOOption ioopt_; // metacache存储当前snapshot client元数据信息 MetaCache mc_; diff --git a/src/client/iomanager4file.cpp b/src/client/iomanager4file.cpp index 41785d2d2a..63e9c89720 100644 --- a/src/client/iomanager4file.cpp +++ b/src/client/iomanager4file.cpp @@ -20,7 +20,6 @@ * Author: tongguangxun */ -#include #include #include // NOLINT @@ -38,7 +37,7 @@ IOManager4File::IOManager4File(): scheduler_(nullptr), exit_(false) { } bool IOManager4File::Initialize(const std::string& filename, - const IOOption_t& ioOpt, + const IOOption& ioOpt, MDSClient* mdsclient) { ioopt_ = ioOpt; diff --git a/src/client/iomanager4file.h b/src/client/iomanager4file.h index adb6d1b4a3..a6d544f333 100644 --- a/src/client/iomanager4file.h +++ b/src/client/iomanager4file.h @@ -23,6 +23,9 @@ #ifndef SRC_CLIENT_IOMANAGER4FILE_H_ #define SRC_CLIENT_IOMANAGER4FILE_H_ +#include +#include + #include #include // NOLINT #include // NOLINT @@ -40,7 +43,11 @@ namespace curve { namespace client { + +using curve::common::Atomic; + class FlightIOGuard; + class IOManager4File : public IOManager { public: IOManager4File(); @@ -53,7 +60,8 @@ class IOManager4File : public IOManager { * @param: mdsclient向下透传给metacache * @return: 成功true,失败false */ - bool Initialize(const std::string& filename, const IOOption_t& ioOpt, + bool Initialize(const std::string& filename, + const IOOption& ioOpt, MDSClient* mdsclient); /** @@ -132,7 +140,7 @@ class IOManager4File : public IOManager { /** * 重新设置io配置信息,测试使用 */ - void SetIOOpt(const IOOption_t& opt) { + void SetIOOpt(const IOOption& opt) { ioopt_ = opt; } @@ -211,7 +219,7 @@ class IOManager4File : public IOManager { private: // 每个IOManager都有其IO配置,保存在iooption里 - IOOption_t ioopt_; + IOOption ioopt_; // metacache存储当前文件的所有元数据信息 MetaCache mc_; @@ -223,7 +231,8 @@ class IOManager4File : public IOManager { FileMetric* fileMetric_; // task thread pool为了将qemu线程与curve线程隔离 - curve::common::TaskThreadPool taskPool_; + curve::common::TaskThreadPool + taskPool_; // inflight IO控制 InflightControl inflightCntl_; @@ -248,5 +257,4 @@ class IOManager4File : public IOManager { } // namespace client } // namespace curve - #endif // SRC_CLIENT_IOMANAGER4FILE_H_ diff --git a/src/client/lease_executor.cpp b/src/client/lease_executor.cpp index 810931c3f5..57faa66ef9 100644 --- a/src/client/lease_executor.cpp +++ b/src/client/lease_executor.cpp @@ -29,7 +29,7 @@ using curve::common::TimeUtility; namespace curve { namespace client { -LeaseExecutor::LeaseExecutor(const LeaseOption_t& leaseOpt, +LeaseExecutor::LeaseExecutor(const LeaseOption& leaseOpt, UserInfo_t userinfo, MDSClient* mdsclient, IOManager4File* iomanager): diff --git a/src/client/lease_executor.h b/src/client/lease_executor.h index 9309b8303e..89a5156086 100644 --- a/src/client/lease_executor.h +++ b/src/client/lease_executor.h @@ -68,7 +68,7 @@ class LeaseExecutor { * @param: mdsclient是与mds续约的client * @param: iomanager会在续约失败或者版本变更的时候进行io调度 */ - LeaseExecutor(const LeaseOption_t& leaseOpt, + LeaseExecutor(const LeaseOption& leaseOpt, UserInfo_t userinfo, MDSClient* mdscllent, IOManager4File* iomanager); @@ -146,7 +146,7 @@ class LeaseExecutor { IOManager4File* iomanager_; // 当前lease执行的配置信息 - LeaseOption_t leaseoption_; + LeaseOption leaseoption_; // mds端传过来的lease信息,包含当前文件的lease时长,及sessionid LeaseSession_t leasesession_; diff --git a/src/client/libcurve_file.cpp b/src/client/libcurve_file.cpp index 71743f8045..e28465437a 100644 --- a/src/client/libcurve_file.cpp +++ b/src/client/libcurve_file.cpp @@ -20,38 +20,34 @@ * Author: tongguangxun */ +#include "src/client/libcurve_file.h" + #include -#include #include -#include // NOLINT -#include // NOLINT -#include #include -#include "src/client/libcurve_file.h" +#include +#include // NOLINT +#include // NOLINT + +#include "include/client/libcurve.h" +#include "include/curve_compiler_specific.h" +#include "proto/nameserver2.pb.h" #include "src/client/client_common.h" #include "src/client/client_config.h" -#include "include/client/libcurve.h" #include "src/client/file_instance.h" -#include "include/curve_compiler_specific.h" #include "src/client/iomanager4file.h" #include "src/client/service_helper.h" -#include "proto/nameserver2.pb.h" +#include "src/common/curve_version.h" #include "src/common/net_common.h" #include "src/common/uuid.h" -#include "src/common/curve_version.h" - -using curve::client::UserInfo; -using curve::common::ReadLockGuard; -using curve::common::WriteLockGuard; #define PORT_LIMIT 65535 bool globalclientinited_ = false; curve::client::FileClient* globalclient = nullptr; -static const int PROCESS_NAME_MAX = 32; -static char g_processname[PROCESS_NAME_MAX]; +using curve::client::UserInfo; namespace brpc { DECLARE_int32(health_check_interval); @@ -60,6 +56,12 @@ namespace brpc { namespace curve { namespace client { +using curve::common::ReadLockGuard; +using curve::common::WriteLockGuard; + +static const int PROCESS_NAME_MAX = 32; +static char g_processname[PROCESS_NAME_MAX]; + void LoggerGuard::InitInternal(const std::string& confPath) { curve::common::Configuration conf; conf.SetConfigPath(confPath); @@ -166,9 +168,11 @@ void FileClient::UnInit() { int FileClient::Open(const std::string& filename, const UserInfo_t& userinfo, std::string* sessionId) { - FileInstance* fileserv = GetInitedFileInstance(filename, userinfo, false); + FileInstance* fileserv = FileInstance::NewInitedFileInstance( + clientconfig_.GetFileServiceOption(), mdsClient_, filename, userinfo, + false); if (fileserv == nullptr) { - LOG(ERROR) << "GetInitedFileInstance fail"; + LOG(ERROR) << "NewInitedFileInstance fail"; return -1; } @@ -181,7 +185,7 @@ int FileClient::Open(const std::string& filename, return ret; } - int fd = fdcount_.fetch_add(1, std::memory_order_acq_rel); + int fd = fdcount_.fetch_add(1, std::memory_order_relaxed); { WriteLockGuard lk(rwlock_); @@ -197,10 +201,11 @@ int FileClient::ReOpen(const std::string& filename, const std::string& sessionId, const UserInfo& userInfo, std::string* newSessionId) { - FileInstance* fileInstance = - GetInitedFileInstance(filename, userInfo, false); + FileInstance* fileInstance = FileInstance::NewInitedFileInstance( + clientconfig_.GetFileServiceOption(), mdsClient_, filename, userInfo, + false); if (nullptr == fileInstance) { - LOG(ERROR) << "GetInitedFileInstance fail"; + LOG(ERROR) << "NewInitedFileInstance fail"; return -1; } @@ -212,7 +217,7 @@ int FileClient::ReOpen(const std::string& filename, return ret; } - int fd = fdcount_.fetch_add(1, std::memory_order_acq_rel); + int fd = fdcount_.fetch_add(1, std::memory_order_relaxed); { WriteLockGuard wlk(rwlock_); @@ -226,9 +231,11 @@ int FileClient::ReOpen(const std::string& filename, int FileClient::Open4ReadOnly(const std::string& filename, const UserInfo_t& userinfo) { - FileInstance* fileserv = GetInitedFileInstance(filename, userinfo, true); + FileInstance* fileserv = FileInstance::NewInitedFileInstance( + clientconfig_.GetFileServiceOption(), mdsClient_, filename, userinfo, + true); if (fileserv == nullptr) { - LOG(ERROR) << "GetInitedFileInstance fail"; + LOG(ERROR) << "NewInitedFileInstance fail"; return -1; } @@ -245,7 +252,7 @@ int FileClient::Open4ReadOnly(const std::string& filename, finfo.fullPathName = filename; fileserv->GetIOManager4File()->UpdateFileInfo(finfo); - int fd = fdcount_.fetch_add(1, std::memory_order_acq_rel); + int fd = fdcount_.fetch_add(1, std::memory_order_relaxed); { WriteLockGuard lk(rwlock_); @@ -521,33 +528,6 @@ int FileClient::Close(int fd) { return -LIBCURVE_ERROR::FAILED; } -bool FileClient::CheckAligned(off_t offset, size_t length) { - return (offset % IO_ALIGNED_BLOCK_SIZE == 0) && - (length % IO_ALIGNED_BLOCK_SIZE == 0); -} - -FileInstance* FileClient::GetInitedFileInstance(const std::string& filename, - const UserInfo& userinfo, bool readonly) { - FileInstance* fileserv = new (std::nothrow) FileInstance(); - if (fileserv == nullptr) { - LOG(ERROR) << "create FileInstance failed!"; - return nullptr; - } - - bool ret = fileserv->Initialize(filename, mdsClient_, userinfo, - clientconfig_.GetFileServiceOption(), readonly); - if (!ret) { - LOG(ERROR) << "FileInstance initialize failed" - << ", filename = " << filename - << ", ower = " << userinfo.owner - << ", readonly = " << readonly; - delete fileserv; - return nullptr; - } - - return fileserv; -} - int FileClient::GetClusterId(char* buf, int len) { std::string result = GetClusterId(); diff --git a/src/client/libcurve_file.h b/src/client/libcurve_file.h index 74d47b4fac..6637d9ff9d 100644 --- a/src/client/libcurve_file.h +++ b/src/client/libcurve_file.h @@ -36,13 +36,11 @@ #include "src/common/concurrent/rw_lock.h" #include "src/common/uuid.h" -using curve::common::BthreadRWLock; - // TODO(tongguangxun) :添加关键函数trace功能 namespace curve { namespace client { -void InitLogging(const std::string& confpath); +using curve::common::BthreadRWLock; class LoggerGuard { public: @@ -274,13 +272,10 @@ class FileClient { private: bool StartDummyServer(); - inline bool CheckAligned(off_t offset, size_t length); - - // 获取一个初始化的FileInstance对象 - // return: 成功返回指向对象的指针,否则返回nullptr - FileInstance* GetInitedFileInstance(const std::string& filename, - const UserInfo& userinfo, - bool readonly); + bool CheckAligned(off_t offset, size_t length) { + return (offset % IO_ALIGNED_BLOCK_SIZE == 0) && + (length % IO_ALIGNED_BLOCK_SIZE == 0); + } private: BthreadRWLock rwlock_; @@ -303,6 +298,8 @@ class FileClient { // 挂载文件数量 bvar::Adder openedFileNum_; }; + } // namespace client } // namespace curve + #endif // SRC_CLIENT_LIBCURVE_FILE_H_ diff --git a/src/client/libcurve_snapshot.cpp b/src/client/libcurve_snapshot.cpp index fd7b3cd7ca..53e2324e53 100644 --- a/src/client/libcurve_snapshot.cpp +++ b/src/client/libcurve_snapshot.cpp @@ -20,20 +20,19 @@ * Author: tongguangxun */ -#include +#include "src/client/libcurve_snapshot.h" + #include -#include // NOLINT +#include "include/curve_compiler_specific.h" #include "src/client/client_common.h" #include "src/client/client_config.h" -#include "src/client/libcurve_snapshot.h" -#include "include/curve_compiler_specific.h" namespace curve { namespace client { SnapshotClient::SnapshotClient() {} -int SnapshotClient::Init(ClientConfigOption_t clientopt) { +int SnapshotClient::Init(ClientConfigOption clientopt) { google::SetCommandLineOption("minloglevel", std::to_string(clientopt.loginfo.logLevel).c_str()); int ret = -LIBCURVE_ERROR::FAILED; @@ -152,7 +151,7 @@ int SnapshotClient::GetSnapshotSegmentInfo(const std::string& filename, int SnapshotClient::GetServerList(const LogicPoolID& lpid, const std::vector& csid) { - std::vector cpinfoVec; + std::vector cpinfoVec; int ret = mdsclient_.GetServerList(lpid, csid, &cpinfoVec); for (auto iter : cpinfoVec) { diff --git a/src/client/libcurve_snapshot.h b/src/client/libcurve_snapshot.h index 9b0335b5c6..326756822b 100644 --- a/src/client/libcurve_snapshot.h +++ b/src/client/libcurve_snapshot.h @@ -23,13 +23,9 @@ #ifndef SRC_CLIENT_LIBCURVE_SNAPSHOT_H_ #define SRC_CLIENT_LIBCURVE_SNAPSHOT_H_ -#include - #include #include -#include #include -#include #include "src/client/mds_client.h" #include "src/client/config_info.h" @@ -48,7 +44,7 @@ class SnapshotClient { * @param: opt为外围配置选项 * @return:0为成功,-1为失败 */ - int Init(ClientConfigOption_t opt); + int Init(ClientConfigOption opt); /** * file对象初始化函数 diff --git a/src/client/mds_client.cpp b/src/client/mds_client.cpp index 5e1e03bc0b..373f2fa523 100644 --- a/src/client/mds_client.cpp +++ b/src/client/mds_client.cpp @@ -19,22 +19,21 @@ * File Created: Monday, 18th February 2019 6:25:25 pm * Author: tongguangxun */ +#include "src/client/mds_client.h" + #include -#include -#include #include -#include // NOLINT -#include // NOLINT -#include +#include -#include "src/common/uuid.h" -#include "src/common/net_common.h" +#include "src/client/lease_executor.h" #include "src/client/metacache.h" -#include "src/client/mds_client.h" -#include "src/common/timeutility.h" +#include "src/common/net_common.h" #include "src/common/string_util.h" -#include "src/client/lease_executor.h" +#include "src/common/timeutility.h" + +namespace curve { +namespace client { using curve::common::NetCommon; using curve::common::TimeUtility; @@ -42,7 +41,6 @@ using curve::common::ReadLockGuard; using curve::common::WriteLockGuard; using curve::mds::StatusCode; using curve::mds::FileInfo; -using curve::mds::FileStatus; using curve::mds::PageFileChunkInfo; using curve::mds::PageFileSegment; using curve::mds::CreateFileResponse; @@ -68,13 +66,8 @@ using curve::mds::topology::CopySetServerInfo; using curve::mds::topology::ChunkServerLocation; using curve::mds::topology::GetChunkServerListInCopySetsResponse; -namespace curve { -namespace client { -MDSClient::MDSClient() { - inited_ = false; -} -LIBCURVE_ERROR MDSClient::Initialize(const MetaServerOption_t& metaServerOpt) { +LIBCURVE_ERROR MDSClient::Initialize(const MetaServerOption& metaServerOpt) { if (inited_) { LOG(INFO) << "MDSClient already started!"; return LIBCURVE_ERROR::OK; @@ -99,8 +92,8 @@ void MDSClient::UnInitialize() { } // rpc发送和mds地址切换状态机 -LIBCURVE_ERROR MDSClient::MDSRPCExcutor::DoRPCTask( - RPCFunc rpctask, uint64_t maxRetryTimeMS) { +LIBCURVE_ERROR MDSClient::MDSRPCExcutor::DoRPCTask(RPCFunc rpctask, + uint64_t maxRetryTimeMS) { // 记录上一次正在服务的mds index int lastWorkingMDSIndex = currentWorkingMDSAddrIndex_; @@ -141,16 +134,17 @@ LIBCURVE_ERROR MDSClient::MDSRPCExcutor::DoRPCTask( : LIBCURVE_ERROR::FAILED; } -bool MDSClient::MDSRPCExcutor::GoOnRetry( - uint64_t startTimeMS, uint64_t maxRetryTimeMS) { +bool MDSClient::MDSRPCExcutor::GoOnRetry(uint64_t startTimeMS, + uint64_t maxRetryTimeMS) { uint64_t currentTime = TimeUtility::GetTimeofDayMs(); return currentTime - startTimeMS < maxRetryTimeMS; } -int MDSClient::MDSRPCExcutor::PreProcessBeforeRetry( - int status, uint64_t* curMDSRetryCount, int curRetryMDSIndex, - int* lastWorkingMDSIndex, uint64_t* timeOutMS) { - +int MDSClient::MDSRPCExcutor::PreProcessBeforeRetry(int status, + uint64_t* curMDSRetryCount, + int curRetryMDSIndex, + int* lastWorkingMDSIndex, + uint64_t* timeOutMS) { int nextMDSIndex = 0; bool rpcTimeout = false; bool needChangeMDS = false; @@ -210,12 +204,13 @@ int MDSClient::MDSRPCExcutor::PreProcessBeforeRetry( * 说明有其他接口更新了currentWorkingMDSAddrIndex_,那么本次切换 * 直接切换到currentWorkingMDSAddrIndex_ */ -int MDSClient::MDSRPCExcutor::GetNextMDSIndex( - bool needChangeMDS, int currentRetryIndex, int* lastWorkingindex) { +int MDSClient::MDSRPCExcutor::GetNextMDSIndex(bool needChangeMDS, + int currentRetryIndex, + int* lastWorkingindex) { int nextMDSIndex = 0; if (std::atomic_compare_exchange_strong(¤tWorkingMDSAddrIndex_, lastWorkingindex, currentWorkingMDSAddrIndex_.load())) { - int size = metaServerOpt_.metaaddrvec.size(); + int size = metaServerOpt_.mdsAddrs.size(); nextMDSIndex = needChangeMDS ? (currentRetryIndex + 1) % size : currentRetryIndex; } else { @@ -225,9 +220,10 @@ int MDSClient::MDSRPCExcutor::GetNextMDSIndex( return nextMDSIndex; } -int MDSClient::MDSRPCExcutor::ExcuteTask( - int mdsindex, uint64_t rpcTimeOutMS, RPCFunc task) { - auto mdsaddr = metaServerOpt_.metaaddrvec[mdsindex]; +int MDSClient::MDSRPCExcutor::ExcuteTask(int mdsindex, + uint64_t rpcTimeOutMS, + RPCFunc task) { + const std::string& mdsaddr = metaServerOpt_.mdsAddrs[mdsindex]; brpc::Channel channel; int ret = channel.Init(mdsaddr.c_str(), nullptr); @@ -244,10 +240,14 @@ int MDSClient::MDSRPCExcutor::ExcuteTask( return task(mdsindex, rpcTimeOutMS, &channel, &cntl); } -#define RPCTaskDefine [&](int mdsindex, uint64_t rpctimeoutMS, brpc::Channel* channel, brpc::Controller* cntl)->int // NOLINT +#define RPCTaskDefine \ + [&](int mdsindex, uint64_t rpctimeoutMS, brpc::Channel* channel, \ + brpc::Controller* cntl) -> int LIBCURVE_ERROR MDSClient::OpenFile(const std::string& filename, - const UserInfo_t& userinfo, FInfo_t* fi, LeaseSession* lease) { + const UserInfo_t& userinfo, + FInfo_t* fi, + LeaseSession* lease) { auto task = RPCTaskDefine { OpenFileResponse response; mdsClientMetric_.openFile.qps.count << 1; @@ -292,7 +292,9 @@ LIBCURVE_ERROR MDSClient::OpenFile(const std::string& filename, } LIBCURVE_ERROR MDSClient::CreateFile(const std::string& filename, - const UserInfo_t& userinfo, size_t size, bool normalFile) { + const UserInfo_t& userinfo, + size_t size, + bool normalFile) { auto task = RPCTaskDefine { CreateFileResponse response; mdsClientMetric_.createFile.qps.count << 1; @@ -324,7 +326,8 @@ LIBCURVE_ERROR MDSClient::CreateFile(const std::string& filename, } LIBCURVE_ERROR MDSClient::CloseFile(const std::string& filename, - const UserInfo_t& userinfo, const std::string& sessionid) { + const UserInfo_t& userinfo, + const std::string& sessionid) { auto task = RPCTaskDefine { CloseFileResponse response; mdsClientMetric_.closeFile.qps.count << 1; @@ -356,7 +359,8 @@ LIBCURVE_ERROR MDSClient::CloseFile(const std::string& filename, } LIBCURVE_ERROR MDSClient::GetFileInfo(const std::string& filename, - const UserInfo_t& uinfo, FInfo_t* fi) { + const UserInfo_t& uinfo, + FInfo_t* fi) { auto task = RPCTaskDefine { GetFileInfoResponse response; mdsClientMetric_.getFile.qps.count << 1; @@ -390,7 +394,8 @@ LIBCURVE_ERROR MDSClient::GetFileInfo(const std::string& filename, } LIBCURVE_ERROR MDSClient::CreateSnapShot(const std::string& filename, - const UserInfo_t& userinfo, uint64_t* seq) { + const UserInfo_t& userinfo, + uint64_t* seq) { auto task = RPCTaskDefine { CreateSnapShotResponse response; mdsClientBase_.CreateSnapShot(filename, userinfo, @@ -444,7 +449,8 @@ LIBCURVE_ERROR MDSClient::CreateSnapShot(const std::string& filename, } LIBCURVE_ERROR MDSClient::DeleteSnapShot(const std::string& filename, - const UserInfo_t& userinfo, uint64_t seq) { + const UserInfo_t& userinfo, + uint64_t seq) { auto task = RPCTaskDefine { DeleteSnapShotResponse response; mdsClientBase_.DeleteSnapShot(filename, userinfo, seq, @@ -472,8 +478,9 @@ LIBCURVE_ERROR MDSClient::DeleteSnapShot(const std::string& filename, } LIBCURVE_ERROR MDSClient::ListSnapShot(const std::string& filename, - const UserInfo_t& userinfo, const std::vector* seq, - std::map* snapif) { + const UserInfo_t& userinfo, + const std::vector* seq, + std::map* snapif) { auto task = RPCTaskDefine { ListSnapShotFileInfoResponse response; mdsClientBase_.ListSnapShot(filename, userinfo, seq, @@ -517,8 +524,10 @@ LIBCURVE_ERROR MDSClient::ListSnapShot(const std::string& filename, } LIBCURVE_ERROR MDSClient::GetSnapshotSegmentInfo(const std::string& filename, - const UserInfo_t& userinfo, uint64_t seq, - uint64_t offset, SegmentInfo *segInfo) { + const UserInfo_t& userinfo, + uint64_t seq, + uint64_t offset, + SegmentInfo* segInfo) { auto task = RPCTaskDefine { GetOrAllocateSegmentResponse response; mdsClientBase_.GetSnapshotSegmentInfo(filename, userinfo, seq, offset, @@ -579,8 +588,10 @@ LIBCURVE_ERROR MDSClient::GetSnapshotSegmentInfo(const std::string& filename, } LIBCURVE_ERROR MDSClient::RefreshSession(const std::string& filename, - const UserInfo_t& userinfo, const std::string& sessionid, - LeaseRefreshResult* resp, LeaseSession* lease) { + const UserInfo_t& userinfo, + const std::string& sessionid, + LeaseRefreshResult* resp, + LeaseSession* lease) { auto task = RPCTaskDefine { ReFreshSessionResponse response; mdsClientMetric_.refreshSession.qps.count << 1; @@ -651,7 +662,9 @@ LIBCURVE_ERROR MDSClient::RefreshSession(const std::string& filename, } LIBCURVE_ERROR MDSClient::CheckSnapShotStatus(const std::string& filename, - const UserInfo_t& userinfo, uint64_t seq, FileStatus* filestatus) { + const UserInfo_t& userinfo, + uint64_t seq, + FileStatus* filestatus) { auto task = RPCTaskDefine { CheckSnapShotStatusResponse response; mdsClientBase_.CheckSnapShotStatus(filename, userinfo, seq, @@ -683,9 +696,11 @@ LIBCURVE_ERROR MDSClient::CheckSnapShotStatus(const std::string& filename, } #define IOPathMaxRetryMS UINT64_MAX -LIBCURVE_ERROR MDSClient::GetServerList(const LogicPoolID& logicalpooid, + +LIBCURVE_ERROR MDSClient::GetServerList( + const LogicPoolID& logicalpooid, const std::vector& copysetidvec, - std::vector* cpinfoVec) { + std::vector* cpinfoVec) { auto task = RPCTaskDefine { GetChunkServerListInCopySetsResponse response; mdsClientMetric_.getServerList.qps.count << 1; @@ -704,13 +719,13 @@ LIBCURVE_ERROR MDSClient::GetServerList(const LogicPoolID& logicalpooid, int csinfonum = response.csinfo_size(); for (int i = 0; i < csinfonum; i++) { std::string copyset_peer; - CopysetInfo_t copysetseverl; + CopysetInfo copysetseverl; CopySetServerInfo info = response.csinfo(i); copysetseverl.cpid_ = info.copysetid(); int cslocsNum = info.cslocs_size(); for (int j = 0; j < cslocsNum; j++) { - CopysetPeerInfo_t csinfo; + CopysetPeerInfo csinfo; ChunkServerLocation csl = info.cslocs(j); uint16_t port = csl.port(); std::string internalIp = csl.hostip(); @@ -808,19 +823,21 @@ LIBCURVE_ERROR MDSClient::CreateCloneFile(const std::string& source, return rpcExcutor.DoRPCTask(task, metaServerOpt_.mdsMaxRetryMS); } -LIBCURVE_ERROR MDSClient::CompleteCloneMeta(const std::string &destination, - const UserInfo_t& userinfo) { - return SetCloneFileStatus(destination, - FileStatus::CloneMetaInstalled, userinfo); +LIBCURVE_ERROR MDSClient::CompleteCloneMeta(const std::string& destination, + const UserInfo_t& userinfo) { + return SetCloneFileStatus(destination, FileStatus::CloneMetaInstalled, + userinfo); } -LIBCURVE_ERROR MDSClient::CompleteCloneFile(const std::string &destination, - const UserInfo_t& userinfo) { +LIBCURVE_ERROR MDSClient::CompleteCloneFile(const std::string& destination, + const UserInfo_t& userinfo) { return SetCloneFileStatus(destination, FileStatus::Cloned, userinfo); } -LIBCURVE_ERROR MDSClient::SetCloneFileStatus(const std::string &filename, - const FileStatus& filestatus, const UserInfo_t& userinfo, uint64_t fileID) { +LIBCURVE_ERROR MDSClient::SetCloneFileStatus(const std::string& filename, + const FileStatus& filestatus, + const UserInfo_t& userinfo, + uint64_t fileID) { auto task = RPCTaskDefine { SetCloneFileStatusResponse response; mdsClientBase_.SetCloneFileStatus(filename, filestatus, userinfo, @@ -848,8 +865,10 @@ LIBCURVE_ERROR MDSClient::SetCloneFileStatus(const std::string &filename, return rpcExcutor.DoRPCTask(task, metaServerOpt_.mdsMaxRetryMS); } -LIBCURVE_ERROR MDSClient::GetOrAllocateSegment(bool allocate, uint64_t offset, - const FInfo_t* fi, SegmentInfo *segInfo) { +LIBCURVE_ERROR MDSClient::GetOrAllocateSegment(bool allocate, + uint64_t offset, + const FInfo_t* fi, + SegmentInfo* segInfo) { auto task = RPCTaskDefine { GetOrAllocateSegmentResponse response; mdsClientMetric_.getOrAllocateSegment.qps.count << 1; @@ -904,8 +923,10 @@ LIBCURVE_ERROR MDSClient::GetOrAllocateSegment(bool allocate, uint64_t offset, } LIBCURVE_ERROR MDSClient::RenameFile(const UserInfo_t& userinfo, - const std::string &origin, const std::string &destination, - uint64_t originId, uint64_t destinationId) { + const std::string& origin, + const std::string& destination, + uint64_t originId, + uint64_t destinationId) { auto task = RPCTaskDefine { RenameFileResponse response; mdsClientMetric_.renameFile.qps.count << 1; @@ -939,7 +960,8 @@ LIBCURVE_ERROR MDSClient::RenameFile(const UserInfo_t& userinfo, } LIBCURVE_ERROR MDSClient::Extend(const std::string& filename, - const UserInfo_t& userinfo, uint64_t newsize) { + const UserInfo_t& userinfo, + uint64_t newsize) { auto task = RPCTaskDefine { ExtendFileResponse response; mdsClientMetric_.extendFile.qps.count << 1; @@ -972,7 +994,9 @@ LIBCURVE_ERROR MDSClient::Extend(const std::string& filename, } LIBCURVE_ERROR MDSClient::DeleteFile(const std::string& filename, - const UserInfo_t& userinfo, bool deleteforce, uint64_t fileid) { + const UserInfo_t& userinfo, + bool deleteforce, + uint64_t fileid) { auto task = RPCTaskDefine { DeleteFileResponse response; mdsClientMetric_.deleteFile.qps.count << 1; @@ -1002,7 +1026,8 @@ LIBCURVE_ERROR MDSClient::DeleteFile(const std::string& filename, } LIBCURVE_ERROR MDSClient::ChangeOwner(const std::string& filename, - const std::string& newOwner, const UserInfo_t& userinfo) { + const std::string& newOwner, + const UserInfo_t& userinfo) { auto task = RPCTaskDefine { ChangeOwnerResponse response; mdsClientMetric_.changeOwner.qps.count << 1; @@ -1033,7 +1058,8 @@ LIBCURVE_ERROR MDSClient::ChangeOwner(const std::string& filename, } LIBCURVE_ERROR MDSClient::Listdir(const std::string& dirpath, - const UserInfo_t& userinfo, std::vector* filestatVec) { + const UserInfo_t& userinfo, + std::vector* filestatVec) { auto task = RPCTaskDefine { ListDirResponse response; mdsClientMetric_.listDir.qps.count << 1; @@ -1082,7 +1108,7 @@ LIBCURVE_ERROR MDSClient::Listdir(const std::string& dirpath, } LIBCURVE_ERROR MDSClient::GetChunkServerInfo(const ChunkServerAddr& csAddr, - CopysetPeerInfo_t* chunkserverInfo) { + CopysetPeerInfo* chunkserverInfo) { if (!chunkserverInfo) { LOG(ERROR) << "chunkserverInfo pointer is null!"; return LIBCURVE_ERROR::FAILED; @@ -1144,8 +1170,7 @@ LIBCURVE_ERROR MDSClient::GetChunkServerInfo(const ChunkServerAddr& csAddr, } LIBCURVE_ERROR MDSClient::ListChunkServerInServer( - const std::string& serverIp, - std::vector* csIds) { + const std::string& serverIp, std::vector* csIds) { auto task = RPCTaskDefine { curve::mds::topology::ListChunkServerResponse response; @@ -1185,8 +1210,8 @@ LIBCURVE_ERROR MDSClient::ListChunkServerInServer( return rpcExcutor.DoRPCTask(task, metaServerOpt_.mdsMaxRetryMS); } -void MDSClient::MDSStatusCode2LibcurveError(const StatusCode& status - , LIBCURVE_ERROR* errcode) { +void MDSClient::MDSStatusCode2LibcurveError(const StatusCode& status, + LIBCURVE_ERROR* errcode) { switch (status) { case StatusCode::kOK: *errcode = LIBCURVE_ERROR::OK; diff --git a/src/client/mds_client.h b/src/client/mds_client.h index c4a48ee321..4916c5aa0f 100644 --- a/src/client/mds_client.h +++ b/src/client/mds_client.h @@ -26,47 +26,47 @@ #include #include #include -#include #include -#include -#include #include -#include +#include -#include "proto/topology.pb.h" +#include "include/client/libcurve.h" #include "proto/nameserver2.pb.h" +#include "proto/topology.pb.h" #include "src/client/client_common.h" #include "src/client/client_config.h" -#include "include/client/libcurve.h" +#include "src/client/client_metric.h" +#include "src/client/mds_client_base.h" #include "src/client/metacache_struct.h" -#include "src/common/concurrent/rw_lock.h" #include "src/common/concurrent/concurrent.h" -#include "src/client/mds_client_base.h" -#include "src/client/client_metric.h" +#include "src/common/concurrent/rw_lock.h" + +namespace curve { +namespace client { using curve::common::RWLock; using curve::common::ReadLockGuard; using curve::common::Authenticator; -namespace curve { -namespace client { class MetaCache; struct LeaseRefreshResult; // MDSClient是client与MDS通信的唯一窗口 class MDSClient { public: - MDSClient(); + MDSClient() = default; ~MDSClient() = default; - using RPCFunc = std::function; + + using RPCFunc = + std::function; + /** * 初始化函数 * @param: metaopt为mdsclient的配置信息 * @return: 成功返回LIBCURVE_ERROR::OK,否则返回LIBCURVE_ERROR::FAILED */ - LIBCURVE_ERROR Initialize(const MetaServerOption_t& metaopt); + LIBCURVE_ERROR Initialize(const MetaServerOption& metaopt); /** * 创建文件 * @param: filename创建文件的文件名 @@ -79,9 +79,9 @@ class MDSClient { * 如果认证失败返回LIBCURVE_ERROR::AUTHFAIL, */ LIBCURVE_ERROR CreateFile(const std::string& filename, - const UserInfo_t& userinfo, - size_t size = 0, - bool normalFile = true); + const UserInfo_t& userinfo, + size_t size = 0, + bool normalFile = true); /** * 打开文件 * @param: filename是文件名 @@ -103,15 +103,15 @@ class MDSClient { * @param: cpinfoVec保存获取到的server信息 * @return: 成功返回LIBCURVE_ERROR::OK,否则返回LIBCURVE_ERROR::FAILED */ - LIBCURVE_ERROR GetServerList(const LogicPoolID &logicPoolId, - const std::vector& csid, - std::vector* cpinfoVec); - - /** - * 获取当前mds所属的集群信息 - * @param[out]: clsctx 为要获取的集群信息 - * @return: 成功返回LIBCURVE_ERROR::OK,否则返回LIBCURVE_ERROR::FAILED - */ + LIBCURVE_ERROR GetServerList(const LogicPoolID& logicPoolId, + const std::vector& csid, + std::vector* cpinfoVec); + + /** + * 获取当前mds所属的集群信息 + * @param[out]: clsctx 为要获取的集群信息 + * @return: 成功返回LIBCURVE_ERROR::OK,否则返回LIBCURVE_ERROR::FAILED + */ LIBCURVE_ERROR GetClusterInfo(ClusterContext* clsctx); /** @@ -124,9 +124,9 @@ class MDSClient { * 否则返回LIBCURVE_ERROR::FAILED */ LIBCURVE_ERROR GetOrAllocateSegment(bool allocate, - uint64_t offset, - const FInfo_t* fi, - SegmentInfo *segInfo); + uint64_t offset, + const FInfo_t* fi, + SegmentInfo* segInfo); /** * 获取文件信息,fi是出参 * @param: filename是文件名 @@ -136,8 +136,8 @@ class MDSClient { * 否则返回LIBCURVE_ERROR::FAILED */ LIBCURVE_ERROR GetFileInfo(const std::string& filename, - const UserInfo_t& userinfo, - FInfo_t* fi); + const UserInfo_t& userinfo, + FInfo_t* fi); /** * 扩展文件 * @param: userinfo是用户信息 @@ -167,8 +167,8 @@ class MDSClient { * 否则返回LIBCURVE_ERROR::FAILED */ LIBCURVE_ERROR CreateSnapShot(const std::string& filename, - const UserInfo_t& userinfo, - uint64_t* seq); + const UserInfo_t& userinfo, + uint64_t* seq); /** * 删除版本号为seq的快照 * @param: userinfo是用户信息 @@ -178,8 +178,8 @@ class MDSClient { * 否则返回LIBCURVE_ERROR::FAILED */ LIBCURVE_ERROR DeleteSnapShot(const std::string& filename, - const UserInfo_t& userinfo, - uint64_t seq); + const UserInfo_t& userinfo, + uint64_t seq); /** * 以列表的形式获取版本号为seq的snapshot文件信息,snapif是出参 @@ -191,9 +191,9 @@ class MDSClient { * 否则返回LIBCURVE_ERROR::FAILED */ LIBCURVE_ERROR ListSnapShot(const std::string& filename, - const UserInfo_t& userinfo, - const std::vector* seq, - std::map* snapif); + const UserInfo_t& userinfo, + const std::vector* seq, + std::map* snapif); /** * 获取快照的chunk信息并更新到metacache,segInfo是出参 * @param: filename是要快照的文件名 @@ -205,10 +205,10 @@ class MDSClient { * 否则返回LIBCURVE_ERROR::FAILED */ LIBCURVE_ERROR GetSnapshotSegmentInfo(const std::string& filename, - const UserInfo_t& userinfo, - uint64_t seq, - uint64_t offset, - SegmentInfo *segInfo); + const UserInfo_t& userinfo, + uint64_t seq, + uint64_t offset, + SegmentInfo* segInfo); /** * 获取快照状态 * @param: filenam文件名 @@ -217,9 +217,9 @@ class MDSClient { * @param[out]: filestatus为快照状态 */ LIBCURVE_ERROR CheckSnapShotStatus(const std::string& filename, - const UserInfo_t& userinfo, - uint64_t seq, - FileStatus* filestatus); + const UserInfo_t& userinfo, + uint64_t seq, + FileStatus* filestatus); /** * 文件接口在打开文件的时候需要与mds保持心跳,refresh用来续约 @@ -232,10 +232,10 @@ class MDSClient { * 否则返回LIBCURVE_ERROR::FAILED */ LIBCURVE_ERROR RefreshSession(const std::string& filename, - const UserInfo_t& userinfo, - const std::string& sessionid, - LeaseRefreshResult* resp, - LeaseSession* lease = nullptr); + const UserInfo_t& userinfo, + const std::string& sessionid, + LeaseRefreshResult* resp, + LeaseSession* lease = nullptr); /** * 关闭文件,需要携带sessionid,这样mds端会在数据库删除该session信息 * @param: filename是要续约的文件名 @@ -244,8 +244,8 @@ class MDSClient { * 否则返回LIBCURVE_ERROR::FAILED */ LIBCURVE_ERROR CloseFile(const std::string& filename, - const UserInfo_t& userinfo, - const std::string& sessionid); + const UserInfo_t& userinfo, + const std::string& sessionid); /** * @brief 创建clone文件 @@ -265,8 +265,10 @@ class MDSClient { */ LIBCURVE_ERROR CreateCloneFile(const std::string& source, const std::string& destination, - const UserInfo_t& userinfo, uint64_t size, - uint64_t sn, uint32_t chunksize, + const UserInfo_t& userinfo, + uint64_t size, + uint64_t sn, + uint32_t chunksize, FInfo* fileinfo); /** @@ -277,8 +279,8 @@ class MDSClient { * * @return 错误码 */ - LIBCURVE_ERROR CompleteCloneMeta(const std::string &destination, - const UserInfo_t& userinfo); + LIBCURVE_ERROR CompleteCloneMeta(const std::string& destination, + const UserInfo_t& userinfo); /** * @brief 通知mds完成Clone Chunk @@ -288,8 +290,8 @@ class MDSClient { * * @return 错误码 */ - LIBCURVE_ERROR CompleteCloneFile(const std::string &destination, - const UserInfo_t& userinfo); + LIBCURVE_ERROR CompleteCloneFile(const std::string& destination, + const UserInfo_t& userinfo); /** * @brief 通知mds完成Clone Meta @@ -301,10 +303,10 @@ class MDSClient { * * @return 错误码 */ - LIBCURVE_ERROR SetCloneFileStatus(const std::string &filename, - const FileStatus& filestatus, - const UserInfo_t& userinfo, - uint64_t fileID = 0); + LIBCURVE_ERROR SetCloneFileStatus(const std::string& filename, + const FileStatus& filestatus, + const UserInfo_t& userinfo, + uint64_t fileID = 0); /** * @brief 重名文件 @@ -332,26 +334,26 @@ class MDSClient { * 否则返回LIBCURVE_ERROR::FAILED,LIBCURVE_ERROR::AUTHFAILED等 */ LIBCURVE_ERROR ChangeOwner(const std::string& filename, - const std::string& newOwner, - const UserInfo_t& userinfo); + const std::string& newOwner, + const UserInfo_t& userinfo); - /** - * 枚举目录内容 - * @param: userinfo是用户信息 - * @param: dirpath是目录路径 - * @param[out]: filestatVec当前文件夹内的文件信息 - */ + /** + * 枚举目录内容 + * @param: userinfo是用户信息 + * @param: dirpath是目录路径 + * @param[out]: filestatVec当前文件夹内的文件信息 + */ LIBCURVE_ERROR Listdir(const std::string& dirpath, - const UserInfo_t& userinfo, - std::vector* filestatVec); + const UserInfo_t& userinfo, + std::vector* filestatVec); - /** - * 向mds注册client metric监听的地址和端口 - * @param: ip客户端ip - * @param: dummyServerPort为监听端口 - * @return: 成功返回0, - * 否则返回LIBCURVE_ERROR::FAILED,LIBCURVE_ERROR::AUTHFAILED等 - */ + /** + * 向mds注册client metric监听的地址和端口 + * @param: ip客户端ip + * @param: dummyServerPort为监听端口 + * @return: 成功返回0, + * 否则返回LIBCURVE_ERROR::FAILED,LIBCURVE_ERROR::AUTHFAILED等 + */ LIBCURVE_ERROR Register(const std::string& ip, uint16_t port); /** @@ -361,20 +363,20 @@ class MDSClient { * @return:成功返回ok */ LIBCURVE_ERROR GetChunkServerInfo(const ChunkServerAddr& addr, - CopysetPeerInfo_t* chunkserverInfo); + CopysetPeerInfo* chunkserverInfo); - /** - * 获取server上所有chunkserver的id - * @param[in]: ip为server的ip地址 - * @param[out]: csIds用于保存chunkserver的id - * @return: 成功返回LIBCURVE_ERROR::OK,失败返回LIBCURVE_ERROR::FAILED - */ + /** + * 获取server上所有chunkserver的id + * @param[in]: ip为server的ip地址 + * @param[out]: csIds用于保存chunkserver的id + * @return: 成功返回LIBCURVE_ERROR::OK,失败返回LIBCURVE_ERROR::FAILED + */ LIBCURVE_ERROR ListChunkServerInServer(const std::string& ip, std::vector* csIds); - /** - * 析构,回收资源 - */ + /** + * 析构,回收资源 + */ void UnInitialize(); // 测试使用 @@ -385,71 +387,74 @@ class MDSClient { protected: class MDSRPCExcutor { public: - MDSRPCExcutor() { + MDSRPCExcutor() { cntlID_.store(1); currentWorkingMDSAddrIndex_.store(0); - } + } - void SetOption(const MetaServerOption_t& option) { + void SetOption(const MetaServerOption& option) { metaServerOpt_ = option; - } - - /** - * 将client与mds的重试相关逻辑抽离 - * @param: task为当前要进行的具体rpc任务 - * @param: maxRetryTimeMS是当前执行最大的重试时间 - * @return: 返回当前RPC的结果 - */ - LIBCURVE_ERROR DoRPCTask(RPCFunc task, uint64_t maxRetryTimeMS); - - /** - * 测试使用: 设置当前正在服务的mdsindex - */ - void SetCurrentWorkIndex(int index) { + } + + /** + * 将client与mds的重试相关逻辑抽离 + * @param: task为当前要进行的具体rpc任务 + * @param: maxRetryTimeMS是当前执行最大的重试时间 + * @return: 返回当前RPC的结果 + */ + LIBCURVE_ERROR DoRPCTask(RPCFunc task, uint64_t maxRetryTimeMS); + + /** + * 测试使用: 设置当前正在服务的mdsindex + */ + void SetCurrentWorkIndex(int index) { currentWorkingMDSAddrIndex_.store(index); - } + } - /** - * 测试使用:获取当前正在服务的mdsindex - */ - int GetCurrentWorkIndex() { + /** + * 测试使用:获取当前正在服务的mdsindex + */ + int GetCurrentWorkIndex() const { return currentWorkingMDSAddrIndex_.load(); - } + } private: - /** - * rpc失败需要重试,根据cntl返回的不同的状态,确定应该做什么样的预处理。 - * 主要做了以下几件事: - * 1. 如果上一次的RPC是超时返回,那么执行rpc 超时指数退避逻辑 - * 2. 如果上一次rpc返回not connect等返回值,会主动触发切换mds地址重试 - * 3. 更新重试信息,比如在当前mds上连续重试的次数 - * @param[in]: status为当前rpc的失败返回的状态 - * @param[in][out]: curMDSRetryCount当前mds节点上的重试次数,如果切换mds - * 该值会被重置为1. - * @param[in]: curRetryMDSIndex代表当前正在重试的mds索引 - * @param[out]: lastWorkingMDSIndex上一次正在提供服务的mds索引 - * @param[out]: timeOutMS根据status对rpctimeout进行调整 - * - * @return: 返回下一次重试的mds索引 - */ - int PreProcessBeforeRetry(int status, uint64_t* curMDSRetryCount, - int curRetryMDSIndex, int* lastWorkingMDSIndex, - uint64_t* timeOutMS); - /** + /** + * rpc失败需要重试,根据cntl返回的不同的状态,确定应该做什么样的预处理。 + * 主要做了以下几件事: + * 1. 如果上一次的RPC是超时返回,那么执行rpc 超时指数退避逻辑 + * 2. 如果上一次rpc返回not connect等返回值,会主动触发切换mds地址重试 + * 3. 更新重试信息,比如在当前mds上连续重试的次数 + * @param[in]: status为当前rpc的失败返回的状态 + * @param[in][out]: curMDSRetryCount当前mds节点上的重试次数,如果切换mds + * 该值会被重置为1. + * @param[in]: curRetryMDSIndex代表当前正在重试的mds索引 + * @param[out]: lastWorkingMDSIndex上一次正在提供服务的mds索引 + * @param[out]: timeOutMS根据status对rpctimeout进行调整 + * + * @return: 返回下一次重试的mds索引 + */ + int PreProcessBeforeRetry(int status, + uint64_t* curMDSRetryCount, + int curRetryMDSIndex, + int* lastWorkingMDSIndex, + uint64_t* timeOutMS); + /** * 执行rpc发送任务 * @param[in]: mdsindex为mds对应的地址索引 * @param[in]: rpcTimeOutMS是rpc超时时间 * @param[in]: task为待执行的任务 * @return: channel获取成功则返回0,否则-1 */ - int ExcuteTask(int mdsindex, uint64_t rpcTimeOutMS, RPCFunc task); - /** + int ExcuteTask(int mdsindex, uint64_t rpcTimeOutMS, RPCFunc task); + /** * 根据输入状态获取下一次需要重试的mds索引,mds切换逻辑: * 记录三个状态:curRetryMDSIndex、lastWorkingMDSIndex、 * currentWorkingMDSIndex * 1. 开始的时候curRetryMDSIndex = currentWorkingMDSIndex * lastWorkingMDSIndex = currentWorkingMDSIndex - * 2. 如果rpc失败,会触发切换curRetryMDSIndex,如果这时候lastWorkingMDSIndex + * 2. + * 如果rpc失败,会触发切换curRetryMDSIndex,如果这时候lastWorkingMDSIndex * 与currentWorkingMDSIndex相等,这时候会顺序切换到下一个mds索引, * 如果lastWorkingMDSIndex与currentWorkingMDSIndex不相等,那么 * 说明有其他接口更新了currentWorkingMDSAddrIndex_,那么本次切换 @@ -457,37 +462,41 @@ class MDSClient { * @param[in]: needChangeMDS表示当前外围需不需要切换mds,这个值由 * PreProcessBeforeRetry函数确定 * @param[in]: currentRetryIndex为当前正在重试的mds索引 - * @param[in][out]: lastWorkingindex为上一次正在服务的mds索引,正在重试的mds + * @param[in][out]: + * lastWorkingindex为上一次正在服务的mds索引,正在重试的mds * 与正在服务的mds索引可能是不同的mds。 * @return: 返回下一次要重试的mds索引 */ - int GetNextMDSIndex(bool needChangeMDS, int currentRetryIndex, + int GetNextMDSIndex(bool needChangeMDS, + int currentRetryIndex, int* lastWorkingindex); - /** - * 根据输入参数,决定是否继续重试,重试退出条件是重试时间超出最大允许时间 - * IO路径上和非IO路径上的重试时间不一样,非IO路径的重试时间由配置文件的 - * mdsMaxRetryMS参数指定,IO路径为无限循环重试。 - * @param[in]: startTimeMS - * @param[in]: maxRetryTimeMS为最大重试时间 - * @return:需要继续重试返回true, 否则返回false - */ - bool GoOnRetry(uint64_t startTimeMS, uint64_t maxRetryTimeMS); - - /** - * 递增controller id并返回id - */ - inline uint64_t GetLogId() { return cntlID_.fetch_add(1); } + /** + * 根据输入参数,决定是否继续重试,重试退出条件是重试时间超出最大允许时间 + * IO路径上和非IO路径上的重试时间不一样,非IO路径的重试时间由配置文件的 + * mdsMaxRetryMS参数指定,IO路径为无限循环重试。 + * @param[in]: startTimeMS + * @param[in]: maxRetryTimeMS为最大重试时间 + * @return:需要继续重试返回true, 否则返回false + */ + bool GoOnRetry(uint64_t startTimeMS, uint64_t maxRetryTimeMS); + + /** + * 递增controller id并返回id + */ + uint64_t GetLogId() { + return cntlID_.fetch_add(1, std::memory_order_relaxed); + } private: - // 执行rpc时必要的配置信息 - MetaServerOption_t metaServerOpt_; + // 执行rpc时必要的配置信息 + MetaServerOption metaServerOpt_; - // 记录上一次重试过的leader信息 - std::atomic currentWorkingMDSAddrIndex_; + // 记录上一次重试过的leader信息 + std::atomic currentWorkingMDSAddrIndex_; - // controller id,用于trace整个rpc IO链路 - // 这里直接用uint64即可,在可预测的范围内,不会溢出 - std::atomic cntlID_; + // controller id,用于trace整个rpc IO链路 + // 这里直接用uint64即可,在可预测的范围内,不会溢出 + std::atomic cntlID_; }; /** @@ -500,10 +509,10 @@ class MDSClient { private: // 初始化标志,放置重复初始化 - bool inited_; + bool inited_ = false; // 当前模块的初始化option配置 - MetaServerOption_t metaServerOpt_; + MetaServerOption metaServerOpt_; // client与mds通信的metric统计 MDSClientMetric mdsClientMetric_; diff --git a/src/client/mds_client_base.cpp b/src/client/mds_client_base.cpp index 1371d7cfae..35ff229c95 100644 --- a/src/client/mds_client_base.cpp +++ b/src/client/mds_client_base.cpp @@ -23,24 +23,25 @@ #include "src/client/mds_client_base.h" #include "src/common/curve_version.h" -const char* kRootUserName = "root"; - namespace curve { namespace client { -int MDSClientBase::Init(const MetaServerOption_t& metaServerOpt) { + +const char* kRootUserName = "root"; + +int MDSClientBase::Init(const MetaServerOption& metaServerOpt) { metaServerOpt_ = metaServerOpt; return 0; } void MDSClientBase::OpenFile(const std::string& filename, - const UserInfo_t& userinfo, - OpenFileResponse* response, - brpc::Controller* cntl, - brpc::Channel* channel) { + const UserInfo_t& userinfo, + OpenFileResponse* response, + brpc::Controller* cntl, + brpc::Channel* channel) { OpenFileRequest request; request.set_filename(filename); request.set_clientversion(curve::common::CurveVersion()); - FillUserInfo(&request, userinfo); + FillUserInfo(&request, userinfo); LOG(INFO) << "OpenFile: filename = " << filename << ", owner = " << userinfo.owner @@ -51,12 +52,12 @@ void MDSClientBase::OpenFile(const std::string& filename, } void MDSClientBase::CreateFile(const std::string& filename, - const UserInfo_t& userinfo, - size_t size, - bool normalFile, - CreateFileResponse* response, - brpc::Controller* cntl, - brpc::Channel* channel) { + const UserInfo_t& userinfo, + size_t size, + bool normalFile, + CreateFileResponse* response, + brpc::Controller* cntl, + brpc::Channel* channel) { CreateFileRequest request; request.set_filename(filename); if (normalFile) { @@ -66,7 +67,7 @@ void MDSClientBase::CreateFile(const std::string& filename, request.set_filetype(curve::mds::FileType::INODE_DIRECTORY); } - FillUserInfo(&request, userinfo); + FillUserInfo(&request, userinfo); LOG(INFO) << "CreateFile: filename = " << filename << ", owner = " << userinfo.owner @@ -78,15 +79,15 @@ void MDSClientBase::CreateFile(const std::string& filename, } void MDSClientBase::CloseFile(const std::string& filename, - const UserInfo_t& userinfo, - const std::string& sessionid, - CloseFileResponse* response, - brpc::Controller* cntl, - brpc::Channel* channel) { + const UserInfo_t& userinfo, + const std::string& sessionid, + CloseFileResponse* response, + brpc::Controller* cntl, + brpc::Channel* channel) { CloseFileRequest request; request.set_filename(filename); request.set_sessionid(sessionid); - FillUserInfo(&request, userinfo); + FillUserInfo(&request, userinfo); LOG(INFO) << "CloseFile: filename = " << filename << ", owner = " << userinfo.owner @@ -104,7 +105,7 @@ void MDSClientBase::GetFileInfo(const std::string& filename, brpc::Channel* channel) { GetFileInfoRequest request; request.set_filename(filename); - FillUserInfo(&request, userinfo); + FillUserInfo(&request, userinfo); LOG_EVERY_SECOND(INFO) << "GetFileInfo: filename = " << filename << ", owner = " << userinfo.owner @@ -115,13 +116,13 @@ void MDSClientBase::GetFileInfo(const std::string& filename, } void MDSClientBase::CreateSnapShot(const std::string& filename, - const UserInfo_t& userinfo, - CreateSnapShotResponse* response, - brpc::Controller* cntl, - brpc::Channel* channel) { + const UserInfo_t& userinfo, + CreateSnapShotResponse* response, + brpc::Controller* cntl, + brpc::Channel* channel) { CreateSnapShotRequest request; request.set_filename(filename); - FillUserInfo<::curve::mds::CreateSnapShotRequest>(&request, userinfo); + FillUserInfo(&request, userinfo); LOG(INFO) << "CreateSnapShot: filename = " << filename << ", owner = " << userinfo.owner @@ -132,15 +133,15 @@ void MDSClientBase::CreateSnapShot(const std::string& filename, } void MDSClientBase::DeleteSnapShot(const std::string& filename, - const UserInfo_t& userinfo, - uint64_t seq, - DeleteSnapShotResponse* response, - brpc::Controller* cntl, - brpc::Channel* channel) { + const UserInfo_t& userinfo, + uint64_t seq, + DeleteSnapShotResponse* response, + brpc::Controller* cntl, + brpc::Channel* channel) { DeleteSnapShotRequest request;; request.set_seq(seq); request.set_filename(filename); - FillUserInfo<::curve::mds::DeleteSnapShotRequest>(&request, userinfo); + FillUserInfo(&request, userinfo); LOG(INFO) << "DeleteSnapShot: filename = " << filename << ", owner = " << userinfo.owner @@ -152,17 +153,17 @@ void MDSClientBase::DeleteSnapShot(const std::string& filename, } void MDSClientBase::ListSnapShot(const std::string& filename, - const UserInfo_t& userinfo, - const std::vector* seq, - ListSnapShotFileInfoResponse* response, - brpc::Controller* cntl, - brpc::Channel* channel) { + const UserInfo_t& userinfo, + const std::vector* seq, + ListSnapShotFileInfoResponse* response, + brpc::Controller* cntl, + brpc::Channel* channel) { ListSnapShotFileInfoRequest request; for (unsigned int i = 0; i < (*seq).size(); i++) { request.add_seq((*seq)[i]); } request.set_filename(filename); - FillUserInfo(&request, userinfo); + FillUserInfo(&request, userinfo); LOG(INFO) << "ListSnapShot: filename = " << filename << ", owner = " << userinfo.owner @@ -181,19 +182,20 @@ void MDSClientBase::ListSnapShot(const std::string& filename, stub.ListSnapShot(cntl, &request, response, nullptr); } -void MDSClientBase::GetSnapshotSegmentInfo(const std::string& filename, - const UserInfo_t& userinfo, - uint64_t seq, - uint64_t offset, - GetOrAllocateSegmentResponse* response, - brpc::Controller* cntl, - brpc::Channel* channel) { +void MDSClientBase::GetSnapshotSegmentInfo( + const std::string& filename, + const UserInfo_t& userinfo, + uint64_t seq, + uint64_t offset, + GetOrAllocateSegmentResponse* response, + brpc::Controller* cntl, + brpc::Channel* channel) { GetOrAllocateSegmentRequest request; request.set_filename(filename); request.set_offset(offset); request.set_allocateifnotexist(false); request.set_seqnum(seq); - FillUserInfo(&request, userinfo); + FillUserInfo(&request, userinfo); LOG(INFO) << "GetSnapshotSegmentInfo: filename = " << filename << ", owner = " << userinfo.owner @@ -224,7 +226,7 @@ void MDSClientBase::RefreshSession(const std::string& filename, request.set_clientport(clientInfo.GetPort()); } - FillUserInfo(&request, userinfo); + FillUserInfo(&request, userinfo); LOG_EVERY_N(INFO, 10) << "RefreshSession: filename = " << filename << ", owner = " << userinfo.owner @@ -236,15 +238,15 @@ void MDSClientBase::RefreshSession(const std::string& filename, } void MDSClientBase::CheckSnapShotStatus(const std::string& filename, - const UserInfo_t& userinfo, - uint64_t seq, - CheckSnapShotStatusResponse* response, - brpc::Controller* cntl, - brpc::Channel* channel) { + const UserInfo_t& userinfo, + uint64_t seq, + CheckSnapShotStatusResponse* response, + brpc::Controller* cntl, + brpc::Channel* channel) { CheckSnapShotStatusRequest request; request.set_seq(seq); request.set_filename(filename); - FillUserInfo(&request, userinfo); + FillUserInfo(&request, userinfo); LOG(INFO) << "CheckSnapShotStatus: filename = " << filename << ", owner = " << userinfo.owner @@ -255,11 +257,12 @@ void MDSClientBase::CheckSnapShotStatus(const std::string& filename, stub.CheckSnapShotStatus(cntl, &request, response, nullptr); } -void MDSClientBase::GetServerList(const LogicPoolID& logicalpooid, - const std::vector& copysetidvec, - GetChunkServerListInCopySetsResponse* response, - brpc::Controller* cntl, - brpc::Channel* channel) { +void MDSClientBase::GetServerList( + const LogicPoolID& logicalpooid, + const std::vector& copysetidvec, + GetChunkServerListInCopySetsResponse* response, + brpc::Controller* cntl, + brpc::Channel* channel) { GetChunkServerListInCopySetsRequest request; request.set_logicalpoolid(logicalpooid); std::string requestCopysets; @@ -283,8 +286,10 @@ void MDSClientBase::GetClusterInfo(GetClusterInfoResponse* response, void MDSClientBase::CreateCloneFile(const std::string& source, const std::string& destination, - const UserInfo_t& userinfo, uint64_t size, - uint64_t sn, uint32_t chunksize, + const UserInfo_t& userinfo, + uint64_t size, + uint64_t sn, + uint32_t chunksize, CreateCloneFileResponse* response, brpc::Controller* cntl, brpc::Channel* channel) { @@ -295,7 +300,7 @@ void MDSClientBase::CreateCloneFile(const std::string& source, request.set_chunksize(chunksize); request.set_filetype(curve::mds::FileType::INODE_PAGEFILE); request.set_clonesource(source); - FillUserInfo(&request, userinfo); + FillUserInfo(&request, userinfo); LOG(INFO) << "CreateCloneFile: source = " << source << ", destination = " << destination @@ -307,20 +312,20 @@ void MDSClientBase::CreateCloneFile(const std::string& source, stub.CreateCloneFile(cntl, &request, response, NULL); } -void MDSClientBase::SetCloneFileStatus(const std::string &filename, - const FileStatus& filestatus, - const UserInfo_t& userinfo, - uint64_t fileID, - SetCloneFileStatusResponse* response, - brpc::Controller* cntl, - brpc::Channel* channel) { +void MDSClientBase::SetCloneFileStatus(const std::string& filename, + const FileStatus& filestatus, + const UserInfo_t& userinfo, + uint64_t fileID, + SetCloneFileStatusResponse* response, + brpc::Controller* cntl, + brpc::Channel* channel) { SetCloneFileStatusRequest request; request.set_filename(filename); request.set_filestatus(static_cast(filestatus)); if (fileID > 0) { request.set_fileid(fileID); } - FillUserInfo(&request, userinfo); + FillUserInfo(&request, userinfo); LOG(INFO) << "SetCloneFileStatus: filename = " << filename << ", owner = " << userinfo.owner @@ -333,11 +338,11 @@ void MDSClientBase::SetCloneFileStatus(const std::string &filename, } void MDSClientBase::GetOrAllocateSegment(bool allocate, - uint64_t offset, - const FInfo_t* fi, - GetOrAllocateSegmentResponse* response, - brpc::Controller* cntl, - brpc::Channel* channel) { + uint64_t offset, + const FInfo_t* fi, + GetOrAllocateSegmentResponse* response, + brpc::Controller* cntl, + brpc::Channel* channel) { GetOrAllocateSegmentRequest request; // convert the user offset to seg offset @@ -347,7 +352,7 @@ void MDSClientBase::GetOrAllocateSegment(bool allocate, request.set_filename(fi->fullPathName); request.set_offset(seg_offset); request.set_allocateifnotexist(allocate); - FillUserInfo(&request, fi->userinfo); + FillUserInfo(&request, fi->userinfo); LOG(INFO) << "GetOrAllocateSegment: allocate = " << allocate << ", owner = " << fi->owner @@ -360,13 +365,13 @@ void MDSClientBase::GetOrAllocateSegment(bool allocate, } void MDSClientBase::RenameFile(const UserInfo_t& userinfo, - const std::string &origin, - const std::string &destination, - uint64_t originId, - uint64_t destinationId, - RenameFileResponse* response, - brpc::Controller* cntl, - brpc::Channel* channel) { + const std::string& origin, + const std::string& destination, + uint64_t originId, + uint64_t destinationId, + RenameFileResponse* response, + brpc::Controller* cntl, + brpc::Channel* channel) { RenameFileRequest request; request.set_oldfilename(origin); request.set_newfilename(destination); @@ -374,7 +379,7 @@ void MDSClientBase::RenameFile(const UserInfo_t& userinfo, request.set_oldfileid(originId); request.set_newfileid(destinationId); } - FillUserInfo(&request, userinfo); + FillUserInfo(&request, userinfo); LOG(INFO) << "RenameFile: origin = " << origin << ", destination = " << destination @@ -388,15 +393,15 @@ void MDSClientBase::RenameFile(const UserInfo_t& userinfo, } void MDSClientBase::Extend(const std::string& filename, - const UserInfo_t& userinfo, - uint64_t newsize, - ExtendFileResponse* response, - brpc::Controller* cntl, - brpc::Channel* channel) { + const UserInfo_t& userinfo, + uint64_t newsize, + ExtendFileResponse* response, + brpc::Controller* cntl, + brpc::Channel* channel) { ExtendFileRequest request; request.set_filename(filename); request.set_newsize(newsize); - FillUserInfo(&request, userinfo); + FillUserInfo(&request, userinfo); LOG(INFO) << "Extend: filename = " << filename << ", owner = " << userinfo.owner @@ -408,19 +413,19 @@ void MDSClientBase::Extend(const std::string& filename, } void MDSClientBase::DeleteFile(const std::string& filename, - const UserInfo_t& userinfo, - bool deleteforce, - uint64_t fileid, - DeleteFileResponse* response, - brpc::Controller* cntl, - brpc::Channel* channel) { + const UserInfo_t& userinfo, + bool deleteforce, + uint64_t fileid, + DeleteFileResponse* response, + brpc::Controller* cntl, + brpc::Channel* channel) { DeleteFileRequest request; request.set_filename(filename); request.set_forcedelete(deleteforce); if (fileid > 0) { request.set_fileid(fileid); } - FillUserInfo(&request, userinfo); + FillUserInfo(&request, userinfo); LOG(INFO) << "DeleteFile: filename = " << filename << ", owner = " << userinfo.owner @@ -454,14 +459,14 @@ void MDSClientBase::ChangeOwner(const std::string& filename, } void MDSClientBase::Listdir(const std::string& dirpath, - const UserInfo_t& userinfo, - ListDirResponse* response, - brpc::Controller* cntl, - brpc::Channel* channel) { + const UserInfo_t& userinfo, + ListDirResponse* response, + brpc::Controller* cntl, + brpc::Channel* channel) { curve::mds::ListDirRequest request; request.set_filename(dirpath); - FillUserInfo<::curve::mds::ListDirRequest>(&request, userinfo); + FillUserInfo(&request, userinfo); LOG(INFO) << "Listdir: filename = " << dirpath << ", owner = " << userinfo.owner @@ -472,8 +477,10 @@ void MDSClientBase::Listdir(const std::string& dirpath, } void MDSClientBase::GetChunkServerInfo(const std::string& ip, - uint16_t port, GetChunkServerInfoResponse* response, - brpc::Controller* cntl, brpc::Channel* channel) { + uint16_t port, + GetChunkServerInfoResponse* response, + brpc::Controller* cntl, + brpc::Channel* channel) { curve::mds::topology::GetChunkServerInfoRequest request; request.set_hostip(ip); request.set_port(port); @@ -486,9 +493,10 @@ void MDSClientBase::GetChunkServerInfo(const std::string& ip, stub.GetChunkServer(cntl, &request, response, NULL); } -void MDSClientBase::ListChunkServerInServer( - const std::string& ip, ListChunkServerResponse* response, - brpc::Controller* cntl, brpc::Channel* channel) { +void MDSClientBase::ListChunkServerInServer(const std::string& ip, + ListChunkServerResponse* response, + brpc::Controller* cntl, + brpc::Channel* channel) { curve::mds::topology::ListChunkServerRequest request; request.set_ip(ip); LOG(INFO) << "ListChunkServerInServer from mds: " diff --git a/src/client/mds_client_base.h b/src/client/mds_client_base.h index f0bbbe00cc..4703098662 100644 --- a/src/client/mds_client_base.h +++ b/src/client/mds_client_base.h @@ -29,13 +29,16 @@ #include #include -#include "proto/topology.pb.h" -#include "proto/nameserver2.pb.h" #include "include/client/libcurve.h" +#include "proto/nameserver2.pb.h" +#include "proto/topology.pb.h" #include "src/client/client_common.h" #include "src/client/config_info.h" -#include "src/common/timeutility.h" #include "src/common/authenticator.h" +#include "src/common/timeutility.h" + +namespace curve { +namespace client { using curve::common::Authenticator; @@ -88,8 +91,6 @@ using curve::mds::topology::ListChunkServerResponse; extern const char* kRootUserName; -namespace curve { -namespace client { // MDSClientBase将所有与mds的RPC接口抽离,与业务逻辑解耦 // 这里只负责rpc的发送,具体的业务处理逻辑通过reponse和controller向上 // 返回给调用者,有调用者处理 @@ -99,7 +100,7 @@ class MDSClientBase { * @param: metaServerOpt为mdsclient的配置信息 * @return: 成功0, 否则-1 */ - int Init(const MetaServerOption_t& metaServerOpt); + int Init(const MetaServerOption& metaServerOpt); /** * 打开文件 @@ -142,11 +143,11 @@ class MDSClientBase { * @param[in]:channel是当前与mds建立的通道 */ void CloseFile(const std::string& filename, - const UserInfo_t& userinfo, - const std::string& sessionid, - CloseFileResponse* response, - brpc::Controller* cntl, - brpc::Channel* channel); + const UserInfo_t& userinfo, + const std::string& sessionid, + CloseFileResponse* response, + brpc::Controller* cntl, + brpc::Channel* channel); /** * 获取文件信息,fi是出参 * @param: filename是文件名 @@ -156,10 +157,10 @@ class MDSClientBase { * @param[in]:channel是当前与mds建立的通道 */ void GetFileInfo(const std::string& filename, - const UserInfo_t& userinfo, - GetFileInfoResponse* response, - brpc::Controller* cntl, - brpc::Channel* channel); + const UserInfo_t& userinfo, + GetFileInfoResponse* response, + brpc::Controller* cntl, + brpc::Channel* channel); /** * 创建版本号为seq的快照 * @param: userinfo是用户信息 @@ -169,10 +170,10 @@ class MDSClientBase { * @param[in]:channel是当前与mds建立的通道 */ void CreateSnapShot(const std::string& filename, - const UserInfo_t& userinfo, - CreateSnapShotResponse* response, - brpc::Controller* cntl, - brpc::Channel* channel); + const UserInfo_t& userinfo, + CreateSnapShotResponse* response, + brpc::Controller* cntl, + brpc::Channel* channel); /** * 删除版本号为seq的快照 * @param: userinfo是用户信息 @@ -183,11 +184,11 @@ class MDSClientBase { * @param[in]:channel是当前与mds建立的通道 */ void DeleteSnapShot(const std::string& filename, - const UserInfo_t& userinfo, - uint64_t seq, - DeleteSnapShotResponse* response, - brpc::Controller* cntl, - brpc::Channel* channel); + const UserInfo_t& userinfo, + uint64_t seq, + DeleteSnapShotResponse* response, + brpc::Controller* cntl, + brpc::Channel* channel); /** * 以列表的形式获取版本号为seq的snapshot文件信息,snapif是出参 * @param: filename是要快照的文件名 @@ -198,11 +199,11 @@ class MDSClientBase { * @param[in]:channel是当前与mds建立的通道 */ void ListSnapShot(const std::string& filename, - const UserInfo_t& userinfo, - const std::vector* seq, - ListSnapShotFileInfoResponse* response, - brpc::Controller* cntl, - brpc::Channel* channel); + const UserInfo_t& userinfo, + const std::vector* seq, + ListSnapShotFileInfoResponse* response, + brpc::Controller* cntl, + brpc::Channel* channel); /** * 获取快照的chunk信息并更新到metacache,segInfo是出参 * @param: filename是要快照的文件名 @@ -214,12 +215,12 @@ class MDSClientBase { * @param[in]:channel是当前与mds建立的通道 */ void GetSnapshotSegmentInfo(const std::string& filename, - const UserInfo_t& userinfo, - uint64_t seq, - uint64_t offset, - GetOrAllocateSegmentResponse* response, - brpc::Controller* cntl, - brpc::Channel* channel); + const UserInfo_t& userinfo, + uint64_t seq, + uint64_t offset, + GetOrAllocateSegmentResponse* response, + brpc::Controller* cntl, + brpc::Channel* channel); /** * 文件接口在打开文件的时候需要与mds保持心跳,refresh用来续约 @@ -246,11 +247,11 @@ class MDSClientBase { * @param[in]:channel是当前与mds建立的通道 */ void CheckSnapShotStatus(const std::string& filename, - const UserInfo_t& userinfo, - uint64_t seq, - CheckSnapShotStatusResponse* response, - brpc::Controller* cntl, - brpc::Channel* channel); + const UserInfo_t& userinfo, + uint64_t seq, + CheckSnapShotStatusResponse* response, + brpc::Controller* cntl, + brpc::Channel* channel); /** * 获取copysetid对应的serverlist信息并更新到metacache * @param: logicPoolId逻辑池信息 @@ -260,10 +261,10 @@ class MDSClientBase { * @param[in]:channel是当前与mds建立的通道 */ void GetServerList(const LogicPoolID& logicalpooid, - const std::vector& copysetidvec, - GetChunkServerListInCopySetsResponse* response, - brpc::Controller* cntl, - brpc::Channel* channel); + const std::vector& copysetidvec, + GetChunkServerListInCopySetsResponse* response, + brpc::Controller* cntl, + brpc::Channel* channel); /** * 获取mds对应的cluster id @@ -289,9 +290,13 @@ class MDSClientBase { */ void CreateCloneFile(const std::string& source, const std::string& destination, - const UserInfo_t& userinfo, uint64_t size, uint64_t sn, - uint32_t chunksize, CreateCloneFileResponse* response, - brpc::Controller* cntl, brpc::Channel* channel); + const UserInfo_t& userinfo, + uint64_t size, + uint64_t sn, + uint32_t chunksize, + CreateCloneFileResponse* response, + brpc::Controller* cntl, + brpc::Channel* channel); /** * @brief 通知mds完成Clone Meta @@ -303,13 +308,13 @@ class MDSClientBase { * @param[in|out]: cntl既是入参,也是出参,返回RPC状态 * @param[in]:channel是当前与mds建立的通道 */ - void SetCloneFileStatus(const std::string &filename, - const FileStatus& filestatus, - const UserInfo_t& userinfo, - uint64_t fileID, - SetCloneFileStatusResponse* response, - brpc::Controller* cntl, - brpc::Channel* channel); + void SetCloneFileStatus(const std::string& filename, + const FileStatus& filestatus, + const UserInfo_t& userinfo, + uint64_t fileID, + SetCloneFileStatusResponse* response, + brpc::Controller* cntl, + brpc::Channel* channel); /** * 获取segment的chunk信息,并更新到Metacache * @param: allocate为true的时候mds端发现不存在就分配,为false的时候不分配 @@ -320,11 +325,11 @@ class MDSClientBase { * @param[in]:channel是当前与mds建立的通道 */ void GetOrAllocateSegment(bool allocate, - uint64_t offset, - const FInfo_t* fi, - GetOrAllocateSegmentResponse* response, - brpc::Controller* cntl, - brpc::Channel* channel); + uint64_t offset, + const FInfo_t* fi, + GetOrAllocateSegmentResponse* response, + brpc::Controller* cntl, + brpc::Channel* channel); /** * @brief 重名文件 * @param:userinfo 用户信息 @@ -354,11 +359,11 @@ class MDSClientBase { * @param[in]:channel是当前与mds建立的通道 */ void Extend(const std::string& filename, - const UserInfo_t& userinfo, - uint64_t newsize, - ExtendFileResponse* response, - brpc::Controller* cntl, - brpc::Channel* channel); + const UserInfo_t& userinfo, + uint64_t newsize, + ExtendFileResponse* response, + brpc::Controller* cntl, + brpc::Channel* channel); /** * 删除文件 * @param: userinfo是用户信息 @@ -386,11 +391,11 @@ class MDSClientBase { * @param[in]:channel是当前与mds建立的通道 */ void ChangeOwner(const std::string& filename, - const std::string& newOwner, - const UserInfo_t& userinfo, - ChangeOwnerResponse* response, - brpc::Controller* cntl, - brpc::Channel* channel); + const std::string& newOwner, + const UserInfo_t& userinfo, + ChangeOwnerResponse* response, + brpc::Controller* cntl, + brpc::Channel* channel); /** * 枚举目录内容 * @param: userinfo是用户信息 @@ -400,10 +405,10 @@ class MDSClientBase { * @param[in]:channel是当前与mds建立的通道 */ void Listdir(const std::string& dirpath, - const UserInfo_t& userinfo, - ListDirResponse* response, - brpc::Controller* cntl, - brpc::Channel* channel); + const UserInfo_t& userinfo, + ListDirResponse* response, + brpc::Controller* cntl, + brpc::Channel* channel); /** * 获取chunkserverID信息 * @param[in]: ip为当前client的监听地址 @@ -413,10 +418,10 @@ class MDSClientBase { * @param[in]:channel是当前与mds建立的通道 */ void GetChunkServerInfo(const std::string& ip, - uint16_t port, - GetChunkServerInfoResponse* reponse, - brpc::Controller* cntl, - brpc::Channel* channel); + uint16_t port, + GetChunkServerInfoResponse* reponse, + brpc::Controller* cntl, + brpc::Channel* channel); /** * 获取server上的所有chunkserver的id @@ -425,11 +430,10 @@ class MDSClientBase { * @param[in|out]: cntl既是入参也是出参 * @param[in]: channel是当前与mds建立的通道 */ - void ListChunkServerInServer( - const std::string& ip, - ListChunkServerResponse* response, - brpc::Controller* cntl, - brpc::Channel* channel); + void ListChunkServerInServer(const std::string& ip, + ListChunkServerResponse* response, + brpc::Controller* cntl, + brpc::Channel* channel); private: /** @@ -445,15 +449,16 @@ class MDSClientBase { } private: - inline bool IsRootUserAndHasPassword(const UserInfo& userinfo) const { - return userinfo.owner == kRootUserName && !userinfo.password.empty(); + bool IsRootUserAndHasPassword(const UserInfo& userinfo) const { + return userinfo.owner == kRootUserName && !userinfo.password.empty(); } std::string CalcSignature(const UserInfo& userinfo, uint64_t date) const; // 当前模块的初始化option配置 - MetaServerOption_t metaServerOpt_; + MetaServerOption metaServerOpt_; }; + } // namespace client } // namespace curve diff --git a/src/client/metacache.cpp b/src/client/metacache.cpp index ada4acf621..4cfcd627e0 100644 --- a/src/client/metacache.cpp +++ b/src/client/metacache.cpp @@ -19,12 +19,12 @@ * File Created: Tuesday, 25th September 2018 2:06:35 pm * Author: tongguangxun */ -#include #include #include #include +#include #include #include "proto/cli.pb.h" @@ -34,14 +34,15 @@ #include "src/client/client_common.h" #include "src/common/concurrent/concurrent.h" +namespace curve { +namespace client { + using curve::common::WriteLockGuard; using curve::common::ReadLockGuard; using curve::client::ClientConfig; -namespace curve { -namespace client { - -void MetaCache::Init(MetaCacheOption_t metaCacheOpt, MDSClient* mdsclient) { +void MetaCache::Init(const MetaCacheOption& metaCacheOpt, + MDSClient* mdsclient) { mdsclient_ = mdsclient; metacacheopt_ = metaCacheOpt; LOG(INFO) << "metacache init success!" @@ -51,9 +52,12 @@ void MetaCache::Init(MetaCacheOption_t metaCacheOpt, MDSClient* mdsclient) { << metacacheopt_.metacacheRPCRetryIntervalUS << ", get leader rpc time out ms = " << metacacheopt_.metacacheGetLeaderRPCTimeOutMS; + + unstableHelper_.Init(metacacheopt_.chunkserverUnstableOption); } -MetaCacheErrorType MetaCache::GetChunkInfoByIndex(ChunkIndex chunkidx, ChunkIDInfo_t* chunxinfo ) { // NOLINT +MetaCacheErrorType MetaCache::GetChunkInfoByIndex(ChunkIndex chunkidx, + ChunkIDInfo* chunxinfo) { ReadLockGuard rdlk(rwlock4ChunkInfo_); auto iter = chunkindex2idMap_.find(chunkidx); if (iter != chunkindex2idMap_.end()) { @@ -67,7 +71,7 @@ bool MetaCache::IsLeaderMayChange(LogicPoolID logicPoolId, CopysetID copysetId) { rwlock4ChunkInfo_.RDLock(); auto iter = lpcsid2CopsetInfoMap_.find( - LogicPoolCopysetID2Str(logicPoolId, copysetId)); + CalcLogicPoolCopysetID(logicPoolId, copysetId)); if (iter == lpcsid2CopsetInfoMap_.end()) { rwlock4ChunkInfo_.Unlock(); return false; @@ -79,16 +83,16 @@ bool MetaCache::IsLeaderMayChange(LogicPoolID logicPoolId, } int MetaCache::GetLeader(LogicPoolID logicPoolId, - CopysetID copysetId, - ChunkServerID* serverId, - EndPoint* serverAddr, - bool refresh, - FileMetric* fm) { - std::string mapkey = LogicPoolCopysetID2Str(logicPoolId, copysetId); - - CopysetInfo_t targetInfo; + CopysetID copysetId, + ChunkServerID* serverId, + EndPoint* serverAddr, + bool refresh, + FileMetric* fm) { + const auto key = CalcLogicPoolCopysetID(logicPoolId, copysetId); + + CopysetInfo targetInfo; rwlock4CopysetInfo_.RDLock(); - auto iter = lpcsid2CopsetInfoMap_.find(mapkey); + auto iter = lpcsid2CopsetInfoMap_.find(key); if (iter == lpcsid2CopsetInfoMap_.end()) { rwlock4CopysetInfo_.Unlock(); LOG(ERROR) << "server list not exist, LogicPoolID = " << logicPoolId @@ -159,7 +163,7 @@ int MetaCache::UpdateLeaderInternal(LogicPoolID logicPoolId, // 如果更新失败,说明leader地址不在当前配置组中,从mds获取chunkserver的信息 if (ret == -1 && !leaderaddr.IsEmpty()) { - CopysetPeerInfo_t csInfo; + CopysetPeerInfo csInfo; ret = mdsclient_->GetChunkServerInfo(leaderaddr, &csInfo); if (ret != LIBCURVE_ERROR::OK) { @@ -178,7 +182,7 @@ int MetaCache::UpdateLeaderInternal(LogicPoolID logicPoolId, } int MetaCache::UpdateCopysetInfoFromMDS(LogicPoolID logicPoolId, - CopysetID copysetId) { + CopysetID copysetId) { std::vector copysetInfos; int ret = @@ -201,7 +205,8 @@ int MetaCache::UpdateCopysetInfoFromMDS(LogicPoolID logicPoolId, } void MetaCache::UpdateCopysetInfoIfMatchCurrentLeader( - LogicPoolID logicPoolId, CopysetID copysetId, + LogicPoolID logicPoolId, + CopysetID copysetId, const ChunkServerAddr& leaderAddr) { std::vector copysetInfos; int ret = @@ -222,13 +227,13 @@ void MetaCache::UpdateCopysetInfoIfMatchCurrentLeader( } } -CopysetInfo_t MetaCache::GetServerList(LogicPoolID logicPoolId, - CopysetID copysetId) { - std::string mapkey = LogicPoolCopysetID2Str(logicPoolId, copysetId); - CopysetInfo_t ret; +CopysetInfo MetaCache::GetServerList(LogicPoolID logicPoolId, + CopysetID copysetId) { + const auto key = CalcLogicPoolCopysetID(logicPoolId, copysetId); + CopysetInfo ret; ReadLockGuard rdlk(rwlock4CopysetInfo_); - auto iter = lpcsid2CopsetInfoMap_.find(mapkey); + auto iter = lpcsid2CopsetInfoMap_.find(key); if (iter == lpcsid2CopsetInfoMap_.end()) { // it's impossible to get here return ret; @@ -242,11 +247,12 @@ CopysetInfo_t MetaCache::GetServerList(LogicPoolID logicPoolId, * return the ChunkServerID to invoker */ int MetaCache::UpdateLeader(LogicPoolID logicPoolId, - CopysetID copysetId, const EndPoint &leaderAddr) { - std::string mapkey = LogicPoolCopysetID2Str(logicPoolId, copysetId); + CopysetID copysetId, + const EndPoint& leaderAddr) { + const auto key = CalcLogicPoolCopysetID(logicPoolId, copysetId); ReadLockGuard rdlk(rwlock4CopysetInfo_); - auto iter = lpcsid2CopsetInfoMap_.find(mapkey); + auto iter = lpcsid2CopsetInfoMap_.find(key); if (iter == lpcsid2CopsetInfoMap_.end()) { // it's impossible to get here return -1; @@ -256,36 +262,39 @@ int MetaCache::UpdateLeader(LogicPoolID logicPoolId, return iter->second.UpdateLeaderInfo(csAddr); } -void MetaCache::UpdateChunkInfoByIndex(ChunkIndex cindex, ChunkIDInfo_t cinfo) { +void MetaCache::UpdateChunkInfoByIndex(ChunkIndex cindex, + const ChunkIDInfo& cinfo) { WriteLockGuard wrlk(rwlock4ChunkInfo_); chunkindex2idMap_[cindex] = cinfo; } void MetaCache::UpdateCopysetInfo(LogicPoolID logicPoolid, CopysetID copysetid, const CopysetInfo& csinfo) { - auto key = LogicPoolCopysetID2Str(logicPoolid, copysetid); + const auto key = CalcLogicPoolCopysetID(logicPoolid, copysetid); WriteLockGuard wrlk(rwlock4CopysetInfo_); lpcsid2CopsetInfoMap_[key] = csinfo; } void MetaCache::UpdateAppliedIndex(LogicPoolID logicPoolId, - CopysetID copysetId, uint64_t appliedindex) { - std::string mapkey = LogicPoolCopysetID2Str(logicPoolId, copysetId); + CopysetID copysetId, + uint64_t appliedindex) { + const auto key = CalcLogicPoolCopysetID(logicPoolId, copysetId); - WriteLockGuard wrlk(rwlock4CopysetInfo_); - auto iter = lpcsid2CopsetInfoMap_.find(mapkey); + ReadLockGuard wrlk(rwlock4CopysetInfo_); + auto iter = lpcsid2CopsetInfoMap_.find(key); if (iter == lpcsid2CopsetInfoMap_.end()) { return; } + iter->second.UpdateAppliedIndex(appliedindex); } uint64_t MetaCache::GetAppliedIndex(LogicPoolID logicPoolId, CopysetID copysetId) { - std::string mapkey = LogicPoolCopysetID2Str(logicPoolId, copysetId); + const auto key = CalcLogicPoolCopysetID(logicPoolId, copysetId); ReadLockGuard rdlk(rwlock4CopysetInfo_); - auto iter = lpcsid2CopsetInfoMap_.find(mapkey); + auto iter = lpcsid2CopsetInfoMap_.find(key); if (iter == lpcsid2CopsetInfoMap_.end()) { return 0; } @@ -293,7 +302,7 @@ uint64_t MetaCache::GetAppliedIndex(LogicPoolID logicPoolId, return iter->second.GetAppliedIndex(); } -void MetaCache::UpdateChunkInfoByID(ChunkID cid, ChunkIDInfo cidinfo) { +void MetaCache::UpdateChunkInfoByID(ChunkID cid, const ChunkIDInfo& cidinfo) { WriteLockGuard wrlk(rwlock4chunkInfoMap_); chunkid2chunkInfoMap_[cid] = cidinfo; } @@ -329,8 +338,8 @@ void MetaCache::SetChunkserverUnstable(ChunkServerID csid) { ReadLockGuard rdlk(rwlock4CopysetInfo_); for (auto it : copysetIDSet) { - std::string mapkey = LogicPoolCopysetID2Str(it.lpid, it.cpid); - auto cpinfo = lpcsid2CopsetInfoMap_.find(mapkey); + const auto key = CalcLogicPoolCopysetID(it.lpid, it.cpid); + auto cpinfo = lpcsid2CopsetInfoMap_.find(key); if (cpinfo != lpcsid2CopsetInfoMap_.end()) { ChunkServerID leaderid; if (cpinfo->second.GetCurrentLeaderServerID(&leaderid)) { @@ -347,17 +356,17 @@ void MetaCache::SetChunkserverUnstable(ChunkServerID csid) { } void MetaCache::AddCopysetIDInfo(ChunkServerID csid, - const CopysetIDInfo& cpidinfo) { + const CopysetIDInfo& cpidinfo) { WriteLockGuard wrlk(rwlock4CSCopysetIDMap_); chunkserverCopysetIDMap_[csid].emplace(cpidinfo); } void MetaCache::UpdateChunkserverCopysetInfo(LogicPoolID lpid, - const CopysetInfo_t& cpinfo) { + const CopysetInfo& cpinfo) { ReadLockGuard rdlk(rwlock4CopysetInfo_); - std::string mapkey = LogicPoolCopysetID2Str(lpid, cpinfo.cpid_); + const auto key = CalcLogicPoolCopysetID(lpid, cpinfo.cpid_); // 先获取原来的chunkserver到copyset映射 - auto previouscpinfo = lpcsid2CopsetInfoMap_.find(mapkey); + auto previouscpinfo = lpcsid2CopsetInfoMap_.find(key); if (previouscpinfo != lpcsid2CopsetInfoMap_.end()) { std::vector newID; std::vector changedID; @@ -391,31 +400,20 @@ void MetaCache::UpdateChunkserverCopysetInfo(LogicPoolID lpid, // 更新新的copyset信息到chunkserver for (auto chunkserverid : newID) { WriteLockGuard wrlk(rwlock4CSCopysetIDMap_); - chunkserverCopysetIDMap_[chunkserverid].emplace(lpid, cpinfo.cpid_); // NOLINT + chunkserverCopysetIDMap_[chunkserverid].emplace(lpid, cpinfo.cpid_); } } } -CopysetInfo_t MetaCache::GetCopysetinfo(LogicPoolID lpid, CopysetID csid) { +CopysetInfo MetaCache::GetCopysetinfo(LogicPoolID lpid, CopysetID csid) { ReadLockGuard rdlk(rwlock4CopysetInfo_); - std::string mapkey = LogicPoolCopysetID2Str(lpid, csid); - auto cpinfo = lpcsid2CopsetInfoMap_.find(mapkey); + const auto key = CalcLogicPoolCopysetID(lpid, csid); + auto cpinfo = lpcsid2CopsetInfoMap_.find(key); if (cpinfo != lpcsid2CopsetInfoMap_.end()) { return cpinfo->second; } return CopysetInfo(); } -std::string MetaCache::LogicPoolCopysetChunkID2Str(LogicPoolID lpid, - CopysetID csid, ChunkID chunkid) { - return std::to_string(lpid).append("_") - .append(std::to_string(csid)) - .append("_") - .append(std::to_string(chunkid)); -} -std::string MetaCache::LogicPoolCopysetID2Str(LogicPoolID lpid, - CopysetID csid) { - return std::to_string(lpid).append("_").append(std::to_string(csid)); -} } // namespace client } // namespace curve diff --git a/src/client/metacache.h b/src/client/metacache.h index ed7895f600..a24f3cee11 100644 --- a/src/client/metacache.h +++ b/src/client/metacache.h @@ -22,25 +22,24 @@ #ifndef SRC_CLIENT_METACACHE_H_ #define SRC_CLIENT_METACACHE_H_ -#include -#include -#include #include -#include +#include #include -#include "src/client/client_config.h" -#include "src/common/concurrent/rw_lock.h" #include "src/client/client_common.h" +#include "src/client/client_config.h" +#include "src/client/client_metric.h" +#include "src/client/mds_client.h" #include "src/client/metacache_struct.h" #include "src/client/service_helper.h" -#include "src/client/mds_client.h" -#include "src/client/client_metric.h" - -using curve::common::RWLock; +#include "src/client/unstable_helper.h" +#include "src/common/concurrent/rw_lock.h" namespace curve { namespace client { + +using curve::common::RWLock; + enum class MetaCacheErrorType { OK = 0, CHUNKINFO_NOT_FOUND = 1, @@ -51,10 +50,10 @@ enum class MetaCacheErrorType { class MetaCache { public: - using CopysetLogicPoolIDStr = std::string; - using ChunkInfoMap = std::unordered_map; // NOLINT - using CopysetInfoMap = std::unordered_map; // NOLINT - using ChunkIndexInfoMap = std::map; + using LogicPoolCopysetID = uint64_t; + using ChunkInfoMap = std::unordered_map; + using CopysetInfoMap = std::unordered_map; + using ChunkIndexInfoMap = std::unordered_map; MetaCache() = default; virtual ~MetaCache() = default; @@ -71,7 +70,7 @@ class MetaCache { * 然后再重试,如果leader获取不成功,需要向mds一侧查询当前copyset的最新信息, * 这里将查询mds封装在内部了,这样copyset client和chunk closure就不感知mds了 */ - void Init(MetaCacheOption_t metaCacheOpt, MDSClient* mdsclient); + void Init(const MetaCacheOption& metaCacheOpt, MDSClient* mdsclient); /** * 通过chunk index获取chunkid信息 @@ -80,15 +79,7 @@ class MetaCache { * @param: 成功返回OK, 否则返回UNKNOWN_ERROR */ virtual MetaCacheErrorType GetChunkInfoByIndex(ChunkIndex chunkidx, - ChunkIDInfo_t* chunkinfo); - /** - * 通过chunkid获取chunkinfo id信息 - * @param: chunkid是待查询的chunk id信息 - * @param: chunkinfo是出参,存储chunk的版本信息 - * @param: 成功返回OK, 否则返回UNKNOWN_ERROR - */ - // virtual MetaCacheErrorType GetChunkInfoByID(ChunkID chunkid, - // ChunkIDInfo_t* chunkinfo); + ChunkIDInfo_t* chunkinfo); /** * sender发送数据的时候需要知道对应的leader然后发送给对应的chunkserver @@ -106,11 +97,11 @@ class MetaCache { * @param: 成功返回0, 否则返回-1 */ virtual int GetLeader(LogicPoolID logicPoolId, - CopysetID copysetId, - ChunkServerID* serverId, - butil::EndPoint* serverAddr, - bool refresh = false, - FileMetric* fm = nullptr); + CopysetID copysetId, + ChunkServerID* serverId, + butil::EndPoint* serverAddr, + bool refresh = false, + FileMetric* fm = nullptr); /** * 更新某个copyset的leader信息 * @param logicPoolId 逻辑池id @@ -119,15 +110,16 @@ class MetaCache { * @return: 成功返回0, 否则返回-1 */ virtual int UpdateLeader(LogicPoolID logicPoolId, - CopysetID copysetId, - const butil::EndPoint &leaderAddr); + CopysetID copysetId, + const butil::EndPoint& leaderAddr); /** * 更新copyset数据信息,包含serverlist * @param: lpid逻辑池id * @param: cpid是copysetid * @param: csinfo是要更新的copyset info */ - virtual void UpdateCopysetInfo(LogicPoolID logicPoolId, CopysetID copysetId, + virtual void UpdateCopysetInfo(LogicPoolID logicPoolId, + CopysetID copysetId, const CopysetInfo& csinfo); /** * 通过chunk index更新chunkid信息 @@ -135,13 +127,13 @@ class MetaCache { * @param: chunkinfo为需要更新的info信息 */ virtual void UpdateChunkInfoByIndex(ChunkIndex cindex, - ChunkIDInfo_t chunkinfo); + const ChunkIDInfo& chunkinfo); /** * 通过chunk id更新chunkid信息 * @param: cid为chunkid * @param: cidinfo为当前chunk对应的id信息 */ - virtual void UpdateChunkInfoByID(ChunkID cid, ChunkIDInfo cidinfo); + virtual void UpdateChunkInfoByID(ChunkID cid, const ChunkIDInfo& cidinfo); /** * 当读写请求返回后,更新当前copyset的applyindex信息 @@ -150,16 +142,15 @@ class MetaCache { * @param: appliedindex是需要更新的applyindex */ virtual void UpdateAppliedIndex(LogicPoolID logicPoolId, - CopysetID copysetId, - uint64_t appliedindex); + CopysetID copysetId, + uint64_t appliedindex); /** * 当读数据时,需要获取当前copyset的applyindex信息 * @param: lpid逻辑池id * @param: cpid是copysetid * @return: 当前copyset的applyin信息 */ - uint64_t GetAppliedIndex(LogicPoolID logicPoolId, - CopysetID copysetId); + uint64_t GetAppliedIndex(LogicPoolID logicPoolId, CopysetID copysetId); /** * 获取当前copyset的server list信息 @@ -167,8 +158,8 @@ class MetaCache { * @param: cpid是copysetid * @return: 当前copyset的copysetinfo信息 */ - virtual CopysetInfo_t GetServerList(LogicPoolID logicPoolId, - CopysetID copysetId); + virtual CopysetInfo GetServerList(LogicPoolID logicPoolId, + CopysetID copysetId); /** * 将ID转化为cache的key @@ -176,18 +167,11 @@ class MetaCache { * @param: cpid是copysetid * @return: 为当前的key */ - inline std::string LogicPoolCopysetID2Str(LogicPoolID lpid, - CopysetID csid); - /** - * 将ID转化为cache的key - * @param: lpid逻辑池id - * @param: cpid是copysetid - * @param: chunkid是chunk的id - * @return: 为当前的key - */ - inline std::string LogicPoolCopysetChunkID2Str(LogicPoolID lpid, - CopysetID csid, - ChunkID chunkid); + static LogicPoolCopysetID CalcLogicPoolCopysetID(LogicPoolID logicPoolId, + CopysetID copysetId) { + return (static_cast(logicPoolId) << 32) | + static_cast(copysetId); + } /** * @brief: 标记整个server上的所有chunkserver为unstable状态 @@ -219,7 +203,7 @@ class MetaCache { const CopysetIDInfo& cpid); virtual void UpdateChunkserverCopysetInfo(LogicPoolID lpid, - const CopysetInfo_t& cpinfo); + const CopysetInfo& cpinfo); void UpdateFileInfo(const FInfo& fileInfo) { fileInfo_ = fileInfo; @@ -247,15 +231,11 @@ class MetaCache { * 测试使用 * 获取copysetinfo信息 */ - virtual CopysetInfo_t GetCopysetinfo(LogicPoolID lpid, CopysetID csid); + virtual CopysetInfo GetCopysetinfo(LogicPoolID lpid, CopysetID csid); - /** - * 测试使用 - * 获取CopysetIDInfo_t - */ - // virtual bool CopysetIDInfoIn(ChunkServerID csid, - // LogicPoolID lpid, - // CopysetID cpid); + UnstableHelper& GetUnstableHelper() { + return unstableHelper_; + } private: /** @@ -285,13 +265,13 @@ class MetaCache { * @param: leaderAddr 当前的leader address */ void UpdateCopysetInfoIfMatchCurrentLeader( - LogicPoolID logicPoolId, - CopysetID copysetId, - const ChunkServerAddr& leaderAddr); + LogicPoolID logicPoolId, + CopysetID copysetId, + const ChunkServerAddr& leaderAddr); private: MDSClient* mdsclient_; - MetaCacheOption_t metacacheopt_; + MetaCacheOption metacacheopt_; // chunkindex到chunkidinfo的映射表 CURVE_CACHELINE_ALIGNMENT ChunkIndexInfoMap chunkindex2idMap_; @@ -313,12 +293,14 @@ class MetaCache { // 是否需要刷新leader // chunkserverid到copyset的映射 - std::unordered_map> chunkserverCopysetIDMap_; // NOLINT + std::unordered_map> chunkserverCopysetIDMap_; // NOLINT // 读写锁保护unStableCSMap CURVE_CACHELINE_ALIGNMENT RWLock rwlock4CSCopysetIDMap_; // 当前文件信息 FInfo fileInfo_; + + UnstableHelper unstableHelper_; }; } // namespace client diff --git a/src/client/metacache_struct.h b/src/client/metacache_struct.h index 12d9c4b18a..af841cf688 100644 --- a/src/client/metacache_struct.h +++ b/src/client/metacache_struct.h @@ -25,93 +25,68 @@ #include #include -#include -#include -#include #include +#include #include "include/curve_compiler_specific.h" #include "src/client/client_common.h" #include "src/common/concurrent/spinlock.h" -using curve::common::SpinLock; - namespace curve { namespace client { -static inline bool -operator==(const ChunkServerAddr& addr1, const ChunkServerAddr& addr2) { - return (addr1.addr_ == addr2.addr_); -} +using curve::common::SpinLock; // copyset内的chunkserver节点的基本信息 // 包含当前chunkserver的id信息,以及chunkserver的地址信息 -typedef struct CURVE_CACHELINE_ALIGNMENT CopysetPeerInfo { +struct CURVE_CACHELINE_ALIGNMENT CopysetPeerInfo { // 当前chunkserver节点的ID - ChunkServerID chunkserverID; + ChunkServerID chunkserverID = 0; // 当前chunkserver节点的内部地址 ChunkServerAddr internalAddr; // 当前chunkserver节点的外部地址 ChunkServerAddr externalAddr; - CopysetPeerInfo() : chunkserverID(0) { - } + CopysetPeerInfo() = default; + CopysetPeerInfo& operator=(const CopysetPeerInfo& other) = default; - CopysetPeerInfo(const ChunkServerID& cid, const ChunkServerAddr& internal, + CopysetPeerInfo(const ChunkServerID& cid, + const ChunkServerAddr& internal, const ChunkServerAddr& external) - : chunkserverID(cid), - internalAddr(internal), - externalAddr(external) {} - - CopysetPeerInfo& operator=(const CopysetPeerInfo& other) { - this->chunkserverID = other.chunkserverID; - this->internalAddr = other.internalAddr; - this->externalAddr = other.externalAddr; - return *this; - } + : chunkserverID(cid), internalAddr(internal), externalAddr(external) {} - bool operator==(const CopysetPeerInfo& other) { - return this->internalAddr == other.internalAddr - && this->externalAddr == other.externalAddr; + bool operator==(const CopysetPeerInfo& other) const { + return this->internalAddr == other.internalAddr && + this->externalAddr == other.externalAddr; } - bool IsEmpty() { - return this->chunkserverID == 0 - && this->internalAddr.IsEmpty() - && this->externalAddr.IsEmpty(); + bool IsEmpty() const { + return this->chunkserverID == 0 && this->internalAddr.IsEmpty() && + this->externalAddr.IsEmpty(); } -} CopysetPeerInfo_t; +}; // copyset的基本信息,包含peer信息、leader信息、appliedindex信息 -typedef struct CURVE_CACHELINE_ALIGNMENT CopysetInfo { +struct CURVE_CACHELINE_ALIGNMENT CopysetInfo { // leader存在变更可能标志位 - bool leaderMayChange_; + bool leaderMayChange_ = false; // 当前copyset的节点信息 - std::vector csinfos_; + std::vector csinfos_; // 当前节点的apply信息,在read的时候需要,用来避免读IO进入raft std::atomic lastappliedindex_{0}; // leader在本copyset信息中的索引,用于后面避免重复尝试同一个leader - int16_t leaderindex_; + int16_t leaderindex_ = -1; // 当前copyset的id信息 - CopysetID cpid_; + CopysetID cpid_ = 0; // 用于保护对copyset信息的修改 - SpinLock spinlock_; + SpinLock spinlock_; - CopysetInfo() { - csinfos_.clear(); - leaderindex_ = -1; - lastappliedindex_ = 0; - leaderMayChange_ = false; - } - - ~CopysetInfo() { - csinfos_.clear(); - leaderindex_ = -1; - } + CopysetInfo() = default; + ~CopysetInfo() = default; CopysetInfo& operator=(const CopysetInfo& other) { this->cpid_ = other.cpid_; - this->csinfos_.assign(other.csinfos_.begin(), other.csinfos_.end()); + this->csinfos_ = other.csinfos_; this->leaderindex_ = other.leaderindex_; this->lastappliedindex_.store(other.lastappliedindex_); this->leaderMayChange_ = other.leaderMayChange_; @@ -137,7 +112,7 @@ typedef struct CURVE_CACHELINE_ALIGNMENT CopysetInfo { leaderMayChange_ = false; } - bool LeaderMayChange() { + bool LeaderMayChange() const { return leaderMayChange_; } @@ -164,11 +139,11 @@ typedef struct CURVE_CACHELINE_ALIGNMENT CopysetInfo { /** * 获取当前leader的索引 */ - int16_t GetCurrentLeaderIndex() { + int16_t GetCurrentLeaderIndex() const { return leaderindex_; } - bool GetCurrentLeaderServerID(ChunkServerID* id) { + bool GetCurrentLeaderServerID(ChunkServerID* id) const { if (leaderindex_ >= 0) { if (csinfos_.size() < leaderindex_) { return false; @@ -186,7 +161,7 @@ typedef struct CURVE_CACHELINE_ALIGNMENT CopysetInfo { * @param: addr为新的leader的地址信息 */ int UpdateLeaderInfo(const ChunkServerAddr& addr, - CopysetPeerInfo_t csInfo = CopysetPeerInfo()) { + CopysetPeerInfo csInfo = CopysetPeerInfo()) { spinlock_.Lock(); bool exists = false; uint16_t tempindex = 0; @@ -241,7 +216,7 @@ typedef struct CURVE_CACHELINE_ALIGNMENT CopysetInfo { /** * 当前CopysetInfo是否合法 */ - bool IsValid() { + bool IsValid() const { return !csinfos_.empty(); } @@ -266,42 +241,30 @@ typedef struct CURVE_CACHELINE_ALIGNMENT CopysetInfo { return false; } -} CopysetInfo_t; +}; -typedef struct CopysetIDInfo { - LogicPoolID lpid; - CopysetID cpid; +struct CopysetIDInfo { + LogicPoolID lpid = 0; + CopysetID cpid = 0; - CopysetIDInfo(LogicPoolID logicpoolid, CopysetID copysetid) { - lpid = logicpoolid; - cpid = copysetid; - } + CopysetIDInfo(LogicPoolID logicpoolid, CopysetID copysetid) + : lpid(logicpoolid), cpid(copysetid) {} - CopysetIDInfo(const CopysetIDInfo& other) { - lpid = other.lpid; - cpid = other.cpid; - } - - CopysetIDInfo& operator=(const CopysetIDInfo& other) { - lpid = other.lpid; - cpid = other.cpid; - return *this; - } -} CopysetIDInfo_t; + CopysetIDInfo(const CopysetIDInfo& other) = default; + CopysetIDInfo& operator=(const CopysetIDInfo& other) = default; +}; -static inline bool -operator<(const CopysetIDInfo& cpidinfo1, const CopysetIDInfo& cpidinfo2) { - return cpidinfo1.lpid <= cpidinfo2.lpid && - cpidinfo1.cpid < cpidinfo2.cpid; +inline bool operator<(const CopysetIDInfo& cpidinfo1, + const CopysetIDInfo& cpidinfo2) { + return cpidinfo1.lpid <= cpidinfo2.lpid && cpidinfo1.cpid < cpidinfo2.cpid; } -static inline bool -operator==(const CopysetIDInfo& cpidinfo1, const CopysetIDInfo& cpidinfo2) { - return cpidinfo1.cpid == cpidinfo2.cpid && - cpidinfo1.lpid == cpidinfo2.lpid; +inline bool operator==(const CopysetIDInfo& cpidinfo1, + const CopysetIDInfo& cpidinfo2) { + return cpidinfo1.cpid == cpidinfo2.cpid && cpidinfo1.lpid == cpidinfo2.lpid; } -} // namespace client -} // namespace curve +} // namespace client +} // namespace curve #endif // SRC_CLIENT_METACACHE_STRUCT_H_ diff --git a/src/client/request_closure.cpp b/src/client/request_closure.cpp index fbb21ac690..602e3a1139 100644 --- a/src/client/request_closure.cpp +++ b/src/client/request_closure.cpp @@ -21,39 +21,14 @@ */ #include "src/client/request_closure.h" + #include "src/client/io_tracker.h" -#include "src/client/request_context.h" -#include "src/client/chunk_closure.h" #include "src/client/iomanager.h" - -using curve::common::ReadLockGuard; -using curve::common::WriteLockGuard; +#include "src/client/request_context.h" namespace curve { namespace client { -RequestClosure::RequestClosure(RequestContext* reqctx) : ioManager_(nullptr) { - suspendRPC_ = false; - managerID_ = 0; - retryTimes_ = 0; - errcode_ = -1; - reqCtx_ = reqctx; - metric_ = nullptr; - nextTimeoutMS_ = 0; -} - -IOTracker* RequestClosure::GetIOTracker() { - return tracker_; -} - -void RequestClosure::SetIOTracker(IOTracker* track) { - tracker_ = track; -} - -void RequestClosure::SetFailed(int errorcode) { - errcode_ = errorcode; -} - void RequestClosure::Run() { ReleaseInflightRPCToken(); if (suspendRPC_) { @@ -62,34 +37,6 @@ void RequestClosure::Run() { tracker_->HandleResponse(reqCtx_); } -int RequestClosure::GetErrorCode() { - return errcode_; -} - -RequestContext* RequestClosure::GetReqCtx() { - return reqCtx_; -} - -void RequestClosure::SetFileMetric(FileMetric* fm) { - metric_ = fm; -} - -void RequestClosure::SetIOManager(IOManager* ioManager) { - ioManager_ = ioManager; -} - -FileMetric* RequestClosure::GetMetric() { - return metric_; -} - -void RequestClosure::SetStartTime(uint64_t start) { - starttime_ = start; -} - -uint64_t RequestClosure::GetStartTime() { - return starttime_; -} - void RequestClosure::GetInflightRPCToken() { if (ioManager_ != nullptr) { ioManager_->GetInflightRpcToken(); @@ -103,5 +50,6 @@ void RequestClosure::ReleaseInflightRPCToken() { MetricHelper::DecremInflightRPC(metric_); } } -} // namespace client -} // namespace curve + +} // namespace client +} // namespace curve diff --git a/src/client/request_closure.h b/src/client/request_closure.h index 855ac4b266..47c77543c6 100644 --- a/src/client/request_closure.h +++ b/src/client/request_closure.h @@ -25,162 +25,157 @@ // for Closure #include -#include -#include #include "include/curve_compiler_specific.h" -#include "src/client/inflight_controller.h" -#include "src/client/client_metric.h" -#include "src/client/config_info.h" #include "src/client/client_common.h" +#include "src/client/client_metric.h" +#include "src/client/inflight_controller.h" #include "src/common/concurrent/concurrent.h" -using curve::common::RWLock; - namespace curve { namespace client { + class IOTracker; class RequestContext; class IOManager; -class RequestClosure : public ::google::protobuf::Closure { +class CURVE_CACHELINE_ALIGNMENT RequestClosure + : public ::google::protobuf::Closure { public: - explicit RequestClosure(RequestContext* reqctx); + explicit RequestClosure(RequestContext* reqctx) : reqCtx_(reqctx) {} virtual ~RequestClosure() = default; + void Run() override; + /** - * clouser的callback执行函数 + * @brief Get the inflight token before sending rpc */ - virtual void Run(); + void GetInflightRPCToken(); /** - * 获取request的错误码 + * @brief Release the inflight token when rpc returned */ - virtual int GetErrorCode(); + void ReleaseInflightRPCToken(); /** - * 获取当前closure属于哪个request + * @brief Get error code */ - virtual RequestContext* GetReqCtx(); + virtual int GetErrorCode() { + return errcode_; + } /** - * 获取当前request属于哪个iotracker + * @brief Set error code, 0 means success */ - virtual IOTracker* GetIOTracker(); + virtual void SetFailed(int errorCode) { + errcode_ = errorCode; + } /** - * 设置返回错误 + * @brief 获取当前closure属于哪个request */ - virtual void SetFailed(int errorcode); + virtual RequestContext* GetReqCtx() { + return reqCtx_; + } /** - * 设置当前属于哪一个iotracker + * @brief 获取当前request属于哪个iotracker */ - void SetIOTracker(IOTracker* ioctx); + virtual IOTracker* GetIOTracker() { + return tracker_; + } + + /** + * @brief 设置当前属于哪一个iotracker + */ + void SetIOTracker(IOTracker* ioTracker) { + tracker_ = ioTracker; + } /** * @brief 设置所属的iomanager */ - void SetIOManager(IOManager* ioManager); + void SetIOManager(IOManager* ioManager) { + ioManager_ = ioManager; + } /** - * 设置当前closure重试次数 + * @brief 设置当前closure重试次数 */ void IncremRetriedTimes() { - retryTimes_++; + retryTimes_++; } - uint64_t GetRetriedTimes() { - return retryTimes_; + uint64_t GetRetriedTimes() const { + return retryTimes_; } /** * 设置metric */ - void SetFileMetric(FileMetric* fm); + void SetFileMetric(FileMetric* fm) { + metric_ = fm; + } /** * 获取metric指针 */ - FileMetric* GetMetric(); - - /** - * 设置rpc发送起始时间,用于RPC延时统计 - */ - void SetStartTime(uint64_t start); - - /** - * 获取rpc起始时间 - */ - uint64_t GetStartTime(); - - /** - * 发送rpc之前都需要先获取inflight token - */ - void GetInflightRPCToken(); - - /** - * 返回给用户或者重新进队的时候都要释放inflight token - */ - void ReleaseInflightRPCToken(); + FileMetric* GetMetric() const { + return metric_; + } /** * 获取下一次rpc超时时间, rpc超时时间实现了指数退避的策略 */ - uint64_t GetNextTimeoutMS() { - return nextTimeoutMS_; + uint64_t GetNextTimeoutMS() const { + return nextTimeoutMS_; } /** * 设置下次重试超时时间 */ void SetNextTimeOutMS(uint64_t timeout) { - nextTimeoutMS_ = timeout; + nextTimeoutMS_ = timeout; } /** * 设置当前的IO为悬挂IO */ void SetSuspendRPCFlag() { - suspendRPC_ = true; + suspendRPC_ = true; } - bool IsSuspendRPC() { - return suspendRPC_; + bool IsSuspendRPC() const { + return suspendRPC_; } private: // suspend io标志 - bool suspendRPC_; + bool suspendRPC_ = false; // 当前request的错误码 - int errcode_; + int errcode_ = -1; // 当前request的tracker信息 - IOTracker* tracker_; + IOTracker* tracker_ = nullptr; // closure的request信息 - RequestContext* reqCtx_; + RequestContext* reqCtx_ = nullptr; // metric信息 - FileMetric* metric_; - - // 起始时间 - uint64_t starttime_; + FileMetric* metric_ = nullptr; // 重试次数 - uint64_t retryTimes_; - - // 当前closure归属于哪个iomanager - IOManagerID managerID_; + uint64_t retryTimes_ = 0; // 当前closure属于的iomanager - IOManager* ioManager_; + IOManager* ioManager_ = nullptr; // 下一次rpc超时时间 - uint64_t nextTimeoutMS_; + uint64_t nextTimeoutMS_ = 0; }; -} // namespace client -} // namespace curve + +} // namespace client +} // namespace curve #endif // SRC_CLIENT_REQUEST_CLOSURE_H_ diff --git a/src/client/request_context.h b/src/client/request_context.h index f6ba3c3aae..63aff8f0e5 100644 --- a/src/client/request_context.h +++ b/src/client/request_context.h @@ -30,6 +30,7 @@ #include "src/client/client_common.h" #include "src/client/request_closure.h" +#include "include/curve_compiler_specific.h" namespace curve { namespace client { @@ -54,7 +55,7 @@ inline std::ostream& operator<<(std::ostream& os, return os; } -struct RequestContext { +struct CURVE_CACHELINE_ALIGNMENT RequestContext { RequestContext() : id_(GetNextRequestContextId()) {} ~RequestContext() = default; @@ -109,6 +110,18 @@ struct RequestContext { // 当前request context id uint64_t id_ = 0; + static RequestContext* NewInitedRequestContext() { + RequestContext* ctx = new (std::nothrow) RequestContext(); + if (ctx && ctx->Init()) { + return ctx; + } else { + LOG(ERROR) << "Allocate or Init RequestContext Failed"; + delete ctx; + return nullptr; + } + } + + private: static std::atomic requestId; static uint64_t GetNextRequestContextId() { diff --git a/src/client/request_scheduler.cpp b/src/client/request_scheduler.cpp index 9099917841..b5ec1dc161 100644 --- a/src/client/request_scheduler.cpp +++ b/src/client/request_scheduler.cpp @@ -32,11 +32,10 @@ namespace curve { namespace client { -RequestScheduler::~RequestScheduler() { -} +RequestScheduler::~RequestScheduler() {} -int RequestScheduler::Init(const RequestScheduleOption_t& reqSchdulerOpt, - MetaCache *metaCache, +int RequestScheduler::Init(const RequestScheduleOption& reqSchdulerOpt, + MetaCache* metaCache, FileMetric* fm) { blockIO_.store(false); reqschopt_ = reqSchdulerOpt; @@ -134,69 +133,14 @@ void RequestScheduler::WakeupBlockQueueAtExit() { } void RequestScheduler::Process() { - while ((running_.load(std::memory_order_acquire) - || !queue_.Empty()) // flush all request in the queue - && !stop_.load(std::memory_order_acquire)) { + while ((running_.load(std::memory_order_acquire) || + !queue_.Empty()) // flush all request in the queue + && !stop_.load(std::memory_order_acquire)) { WaitValidSession(); - BBQItem item = queue_.TakeFront(); + BBQItem item = queue_.TakeFront(); if (!item.IsStop()) { - RequestContext *req = item.Item(); - brpc::ClosureGuard guard(req->done_); - switch (req->optype_) { - case OpType::READ: - { - req->done_->GetInflightRPCToken(); - client_.ReadChunk(req->idinfo_, req->seq_, req->offset_, - req->rawlength_, req->appliedindex_, - req->sourceInfo_, guard.release()); - } - break; - case OpType::WRITE: - DVLOG(9) << "Processing write request, buf header: " - << " buf: " - << *(unsigned int*)(req->writeData_.fetch1()); - { - req->done_->GetInflightRPCToken(); - client_.WriteChunk(req->idinfo_, req->seq_, - req->writeData_, req->offset_, - req->rawlength_, req->sourceInfo_, - guard.release()); - } - break; - case OpType::READ_SNAP: - client_.ReadChunkSnapshot(req->idinfo_, - req->seq_, - req->offset_, - req->rawlength_, - guard.release()); - break; - case OpType::DELETE_SNAP: - client_.DeleteChunkSnapshotOrCorrectSn(req->idinfo_, - req->correctedSeq_, - guard.release()); - break; - case OpType::GET_CHUNK_INFO: - client_.GetChunkInfo(req->idinfo_, - guard.release()); - break; - case OpType::CREATE_CLONE: - client_.CreateCloneChunk(req->idinfo_, - req->location_, - req->seq_, - req->correctedSeq_, - req->chunksize_, - guard.release()); - break; - case OpType::RECOVER_CHUNK: - client_.RecoverChunk(req->idinfo_, - req->offset_, req->rawlength_, - guard.release()); - break; - default: - /* TODO(wudemiao) 后期整个链路错误发统一了在处理 */ - req->done_->SetFailed(-1); - LOG(ERROR) << "unknown op type: OpType::UNKNOWN"; - } + RequestContext* req = item.Item(); + ProcessOne(req); } else { /** * 一旦遇到stop item,所有线程都可以退出,因为此时 @@ -207,5 +151,48 @@ void RequestScheduler::Process() { } } +void RequestScheduler::ProcessOne(RequestContext* ctx) { + brpc::ClosureGuard guard(ctx->done_); + + switch (ctx->optype_) { + case OpType::READ: + ctx->done_->GetInflightRPCToken(); + client_.ReadChunk(ctx->idinfo_, ctx->seq_, ctx->offset_, + ctx->rawlength_, ctx->appliedindex_, + ctx->sourceInfo_, guard.release()); + break; + case OpType::WRITE: + ctx->done_->GetInflightRPCToken(); + client_.WriteChunk(ctx->idinfo_, ctx->seq_, ctx->writeData_, + ctx->offset_, ctx->rawlength_, ctx->sourceInfo_, + guard.release()); + break; + case OpType::READ_SNAP: + client_.ReadChunkSnapshot(ctx->idinfo_, ctx->seq_, ctx->offset_, + ctx->rawlength_, guard.release()); + break; + case OpType::DELETE_SNAP: + client_.DeleteChunkSnapshotOrCorrectSn( + ctx->idinfo_, ctx->correctedSeq_, guard.release()); + break; + case OpType::GET_CHUNK_INFO: + client_.GetChunkInfo(ctx->idinfo_, guard.release()); + break; + case OpType::CREATE_CLONE: + client_.CreateCloneChunk(ctx->idinfo_, ctx->location_, ctx->seq_, + ctx->correctedSeq_, ctx->chunksize_, + guard.release()); + break; + case OpType::RECOVER_CHUNK: + client_.RecoverChunk(ctx->idinfo_, ctx->offset_, ctx->rawlength_, + guard.release()); + break; + default: + /* TODO(wudemiao) 后期整个链路错误发统一了在处理 */ + ctx->done_->SetFailed(-1); + LOG(ERROR) << "unknown op type: OpType::UNKNOWN"; + } +} + } // namespace client } // namespace curve diff --git a/src/client/request_scheduler.h b/src/client/request_scheduler.h index 96153b0e4d..b68055a6ee 100644 --- a/src/client/request_scheduler.h +++ b/src/client/request_scheduler.h @@ -61,7 +61,7 @@ class RequestScheduler : public Uncopyable { * @param: metacache为meta信息 * @param: filematric为文件的metric信息 */ - virtual int Init(const RequestScheduleOption_t& reqSchdulerOpt, + virtual int Init(const RequestScheduleOption& reqSchdulerOpt, MetaCache *metaCache, FileMetric* fileMetric = nullptr); /** @@ -109,9 +109,9 @@ class RequestScheduler : public Uncopyable { * 后续的IO调度会被阻塞 */ void LeaseTimeoutBlockIO() { - std::unique_lock lk(leaseRefreshmtx_); - blockIO_.store(true); - client_.StartRecycleRetryRPC(); + std::unique_lock lk(leaseRefreshmtx_); + blockIO_.store(true); + client_.StartRecycleRetryRPC(); } /** @@ -119,17 +119,17 @@ class RequestScheduler : public Uncopyable { * IO调度被恢复 */ void RefeshSuccAndResumeIO() { - std::unique_lock lk(leaseRefreshmtx_); - blockIO_.store(false); - leaseRefreshcv_.notify_all(); - client_.ResumeRPCRetry(); + std::unique_lock lk(leaseRefreshmtx_); + blockIO_.store(false); + leaseRefreshcv_.notify_all(); + client_.ResumeRPCRetry(); } /** * 测试使用,获取队列 */ - BoundedBlockingDeque>* GetQueue() { - return &queue_; + BoundedBlockingDeque>* GetQueue() { + return &queue_; } private: @@ -138,19 +138,21 @@ class RequestScheduler : public Uncopyable { */ void Process(); - inline void WaitValidSession() { - // lease续约失败的时候需要阻塞IO直到续约成功 - if (blockIO_.load(std::memory_order_acquire) && blockingQueue_) { - std::unique_lock lk(leaseRefreshmtx_); - leaseRefreshcv_.wait(lk, [&]()->bool{ - return !blockIO_.load() || !blockingQueue_; - }); - } + void ProcessOne(RequestContext* ctx); + + void WaitValidSession() { + // lease续约失败的时候需要阻塞IO直到续约成功 + if (blockIO_.load(std::memory_order_acquire) && blockingQueue_) { + std::unique_lock lk(leaseRefreshmtx_); + leaseRefreshcv_.wait(lk, [&]() -> bool { + return !blockIO_.load() || !blockingQueue_; + }); + } } private: // 线程池和queue容量的配置参数 - RequestScheduleOption_t reqschopt_; + RequestScheduleOption reqschopt_; // 存放 request 的队列 BoundedBlockingDeque> queue_; // 处理 request 的线程池 diff --git a/src/client/request_sender.cpp b/src/client/request_sender.cpp index 5e0d05344d..22bf683295 100644 --- a/src/client/request_sender.cpp +++ b/src/client/request_sender.cpp @@ -30,12 +30,12 @@ #include "src/client/request_closure.h" #include "src/common/location_operator.h" -using curve::common::TimeUtility; - namespace curve { namespace client { -int RequestSender::Init(const IOSenderOption_t& ioSenderOpt) { +using curve::common::TimeUtility; + +int RequestSender::Init(const IOSenderOption& ioSenderOpt) { if (0 != channel_.Init(serverEndPoint_, NULL)) { LOG(ERROR) << "failed to init channel to server, id: " << chunkServerId_ << ", "<< serverEndPoint_.ip << ":" << serverEndPoint_.port; @@ -58,12 +58,11 @@ int RequestSender::ReadChunk(ChunkIDInfo idinfo, RequestClosure* rc = static_cast(done->GetClosure()); MetricHelper::IncremRPCRPSCount(rc->GetMetric(), OpType::READ); - rc->SetStartTime(TimeUtility::GetTimeofDayUs()); brpc::Controller *cntl = new brpc::Controller(); cntl->set_timeout_ms( - std::max(rc->GetNextTimeoutMS(), - iosenderopt_.failRequestOpt.chunkserverRPCTimeoutMS)); + std::max(rc->GetNextTimeoutMS(), + iosenderopt_.failRequestOpt.chunkserverRPCTimeoutMS)); done->SetCntl(cntl); ChunkResponse *response = new ChunkResponse(); done->SetResponse(response); @@ -86,6 +85,7 @@ int RequestSender::ReadChunk(ChunkIDInfo idinfo, if (iosenderopt_.chunkserverEnableAppliedIndexRead && appliedindex > 0) { request.set_appliedindex(appliedindex); } + ChunkService_Stub stub(&channel_); stub.ReadChunk(cntl, &request, response, doneGuard.release()); @@ -103,7 +103,6 @@ int RequestSender::WriteChunk(ChunkIDInfo idinfo, RequestClosure* rc = static_cast(done->GetClosure()); MetricHelper::IncremRPCRPSCount(rc->GetMetric(), OpType::WRITE); - rc->SetStartTime(TimeUtility::GetTimeofDayUs()); DVLOG(9) << "Sending request, buf header: " << " buf: " << *(unsigned int *)(data.fetch1()); @@ -148,8 +147,8 @@ int RequestSender::ReadChunkSnapshot(ChunkIDInfo idinfo, RequestClosure* rc = static_cast(done->GetClosure()); brpc::Controller *cntl = new brpc::Controller(); cntl->set_timeout_ms( - std::max(rc->GetNextTimeoutMS(), - iosenderopt_.failRequestOpt.chunkserverRPCTimeoutMS)); + std::max(rc->GetNextTimeoutMS(), + iosenderopt_.failRequestOpt.chunkserverRPCTimeoutMS)); done->SetCntl(cntl); ChunkResponse *response = new ChunkResponse(); done->SetResponse(response); @@ -176,8 +175,8 @@ int RequestSender::DeleteChunkSnapshotOrCorrectSn(ChunkIDInfo idinfo, RequestClosure* rc = static_cast(done->GetClosure()); brpc::Controller *cntl = new brpc::Controller(); cntl->set_timeout_ms( - std::max(rc->GetNextTimeoutMS(), - iosenderopt_.failRequestOpt.chunkserverRPCTimeoutMS)); + std::max(rc->GetNextTimeoutMS(), + iosenderopt_.failRequestOpt.chunkserverRPCTimeoutMS)); done->SetCntl(cntl); ChunkResponse *response = new ChunkResponse(); done->SetResponse(response); @@ -203,8 +202,8 @@ int RequestSender::GetChunkInfo(ChunkIDInfo idinfo, RequestClosure* rc = static_cast(done->GetClosure()); brpc::Controller *cntl = new brpc::Controller(); cntl->set_timeout_ms( - std::max(rc->GetNextTimeoutMS(), - iosenderopt_.failRequestOpt.chunkserverRPCTimeoutMS)); + std::max(rc->GetNextTimeoutMS(), + iosenderopt_.failRequestOpt.chunkserverRPCTimeoutMS)); done->SetCntl(cntl); GetChunkInfoResponse *response = new GetChunkInfoResponse(); done->SetResponse(response); @@ -229,8 +228,8 @@ int RequestSender::CreateCloneChunk(ChunkIDInfo idinfo, RequestClosure* rc = static_cast(done->GetClosure()); brpc::Controller *cntl = new brpc::Controller(); cntl->set_timeout_ms( - std::max(rc->GetNextTimeoutMS(), - iosenderopt_.failRequestOpt.chunkserverRPCTimeoutMS)); + std::max(rc->GetNextTimeoutMS(), + iosenderopt_.failRequestOpt.chunkserverRPCTimeoutMS)); done->SetCntl(cntl); ChunkResponse *response = new ChunkResponse(); done->SetResponse(response); @@ -259,8 +258,8 @@ int RequestSender::RecoverChunk(const ChunkIDInfo& idinfo, RequestClosure* rc = static_cast(done->GetClosure()); brpc::Controller *cntl = new brpc::Controller(); cntl->set_timeout_ms( - std::max(rc->GetNextTimeoutMS(), - iosenderopt_.failRequestOpt.chunkserverRPCTimeoutMS)); + std::max(rc->GetNextTimeoutMS(), + iosenderopt_.failRequestOpt.chunkserverRPCTimeoutMS)); done->SetCntl(cntl); ChunkResponse *response = new ChunkResponse(); done->SetResponse(response); diff --git a/src/client/request_sender.h b/src/client/request_sender.h index edcd742ac5..3087d898eb 100644 --- a/src/client/request_sender.h +++ b/src/client/request_sender.h @@ -58,7 +58,7 @@ class RequestSender { channel_() {} virtual ~RequestSender() {} - int Init(const IOSenderOption_t& ioSenderOpt); + int Init(const IOSenderOption& ioSenderOpt); /** * 读Chunk @@ -181,7 +181,7 @@ class RequestSender { private: // Rpc stub配置 - IOSenderOption_t iosenderopt_; + IOSenderOption iosenderopt_; // ChunkServer 的唯一标识 id ChunkServerID chunkServerId_; // ChunkServer 的地址 diff --git a/src/client/request_sender_manager.cpp b/src/client/request_sender_manager.cpp index f7656c6b7f..a5c77a793f 100644 --- a/src/client/request_sender_manager.cpp +++ b/src/client/request_sender_manager.cpp @@ -30,33 +30,36 @@ namespace curve { namespace client { RequestSenderManager::SenderPtr RequestSenderManager::GetOrCreateSender( - const ChunkServerID &leaderId, - const butil::EndPoint &leaderAddr, - IOSenderOption_t senderopt) { - std::shared_ptr senderPtr = nullptr; - - std::lock_guard guard(lock_); - auto leaderIter = senderPool_.find(leaderId); - if (senderPool_.end() != leaderIter) { - return leaderIter->second; - } else { - // 不存在则创建 - senderPtr = std::make_shared(leaderId, leaderAddr); - CHECK(nullptr != senderPtr) << "new RequestSender failed"; - - int rc = senderPtr->Init(senderopt); - if (0 != rc) { - return nullptr; + const ChunkServerID& leaderId, + const butil::EndPoint& leaderAddr, + const IOSenderOption& senderopt) { + { + curve::common::ReadLockGuard guard(rwlock_); + auto iter = senderPool_.find(leaderId); + if (senderPool_.end() != iter) { + return iter->second; } + } - senderPool_.insert(std::pair(leaderId, - senderPtr)); + curve::common::WriteLockGuard guard(rwlock_); + auto iter = senderPool_.find(leaderId); + if (senderPool_.end() != iter) { + return iter->second; } - return senderPtr; + + SenderPtr sender = std::make_shared(leaderId, leaderAddr); + int rc = sender->Init(senderopt); + if (rc != 0) { + return nullptr; + } + + senderPool_.emplace(leaderId, sender); + + return sender; } void RequestSenderManager::ResetSenderIfNotHealth(const ChunkServerID& csId) { - std::lock_guard guard(lock_); + curve::common::WriteLockGuard guard(rwlock_); auto iter = senderPool_.find(csId); if (iter == senderPool_.end()) { diff --git a/src/client/request_sender_manager.h b/src/client/request_sender_manager.h index a20d8e96f2..530d8c1c82 100644 --- a/src/client/request_sender_manager.h +++ b/src/client/request_sender_manager.h @@ -23,12 +23,12 @@ #ifndef SRC_CLIENT_REQUEST_SENDER_MANAGER_H_ #define SRC_CLIENT_REQUEST_SENDER_MANAGER_H_ -#include //NOLINT -#include #include +#include #include "src/client/client_common.h" #include "src/client/config_info.h" +#include "src/common/concurrent/rw_lock.h" #include "src/common/uncopyable.h" namespace curve { @@ -44,7 +44,7 @@ class RequestSender; class RequestSenderManager : public Uncopyable { public: using SenderPtr = std::shared_ptr; - RequestSenderManager() : lock_(), senderPool_() {} + RequestSenderManager() : rwlock_(), senderPool_() {} /** * 获取指定leader id的sender,如果没有则根据leader @@ -53,9 +53,9 @@ class RequestSenderManager : public Uncopyable { * @param leaderAddr:leader的地址 * @return nullptr:get或者create失败,否则成功 */ - SenderPtr GetOrCreateSender(const ChunkServerID &leaderId, - const butil::EndPoint &leaderAddr, - IOSenderOption_t senderopt); + SenderPtr GetOrCreateSender(const ChunkServerID& leaderId, + const butil::EndPoint& leaderAddr, + const IOSenderOption& senderopt); /** * @brief 如果csId对应的RequestSender不健康,就进行重置 @@ -64,8 +64,8 @@ class RequestSenderManager : public Uncopyable { void ResetSenderIfNotHealth(const ChunkServerID& csId); private: - // 互斥锁,保护senderPool_ - mutable std::mutex lock_; + // 读写锁,保护senderPool_ + curve::common::BthreadRWLock rwlock_; // 请求发送链接的map,以ChunkServer ID为key std::unordered_map senderPool_; }; diff --git a/src/client/service_helper.cpp b/src/client/service_helper.cpp index a8e407c3a3..937b3d6ff8 100644 --- a/src/client/service_helper.cpp +++ b/src/client/service_helper.cpp @@ -24,7 +24,6 @@ #include #include -#include #include #include @@ -85,7 +84,7 @@ class GetLeaderProxy : public std::enable_shared_from_this { friend struct GetLeaderClosure; public: GetLeaderProxy() - : proxyId_(getLeaderProxyId.fetch_add(1)), + : proxyId_(getLeaderProxyId.fetch_add(1, std::memory_order_relaxed)), finish_(false), success_(false) {} diff --git a/src/client/service_helper.h b/src/client/service_helper.h index 983044fc03..83239191d3 100644 --- a/src/client/service_helper.h +++ b/src/client/service_helper.h @@ -44,9 +44,8 @@ namespace client { struct GetLeaderRpcOption { uint32_t rpcTimeoutMs; - explicit GetLeaderRpcOption( - uint32_t rpcTimeoutMs = 500) - : rpcTimeoutMs(rpcTimeoutMs) {} + explicit GetLeaderRpcOption(uint32_t rpcTimeoutMs = 500) + : rpcTimeoutMs(rpcTimeoutMs) {} }; // GetLeader请求对应的copyset信息及rpc相关参数信息 @@ -120,8 +119,8 @@ class ServiceHelper { * @return: 获取到user信息为true,否则false */ static bool GetUserInfoFromFilename(const std::string& fname, - std::string* realfilename, - std::string* user); + std::string* realfilename, + std::string* user); /** * @brief: 发送http请求,判断chunkserver是否健康 @@ -131,8 +130,8 @@ class ServiceHelper { * * @return: 0 表示健康,-1表示不健康 */ - static int CheckChunkServerHealth( - const butil::EndPoint& endPoint, int32_t requestTimeoutMs); + static int CheckChunkServerHealth(const butil::EndPoint& endPoint, + int32_t requestTimeoutMs); }; } // namespace client diff --git a/src/client/splitor.cpp b/src/client/splitor.cpp index ae4a26bedd..2e7d0dc16f 100644 --- a/src/client/splitor.cpp +++ b/src/client/splitor.cpp @@ -20,33 +20,36 @@ * Author: tongguangxun */ -#include +#include "src/client/splitor.h" + #include + #include -#include #include -#include "src/client/splitor.h" -#include "src/client/mds_client.h" +#include + #include "src/client/file_instance.h" -#include "src/client/request_closure.h" +#include "src/client/mds_client.h" #include "src/client/metacache_struct.h" +#include "src/client/request_closure.h" #include "src/common/location_operator.h" namespace curve { namespace client { -IOSplitOPtion_t Splitor::iosplitopt_; -void Splitor::Init(IOSplitOPtion_t ioSplitOpt) { +IOSplitOption Splitor::iosplitopt_; + +void Splitor::Init(const IOSplitOption& ioSplitOpt) { iosplitopt_ = ioSplitOpt; LOG(INFO) << "io splitor init success!"; } -int Splitor::IO2ChunkRequests(IOTracker* iotracker, MetaCache* mc, +int Splitor::IO2ChunkRequests(IOTracker* iotracker, MetaCache* metaCache, std::vector* targetlist, butil::IOBuf* data, off_t offset, size_t length, - MDSClient* mdsclient, const FInfo_t* fi) { - if (targetlist == nullptr || mdsclient == nullptr || - mc == nullptr || iotracker == nullptr || fi == nullptr) { + MDSClient* mdsclient, const FInfo_t* fileInfo) { + if (targetlist == nullptr || mdsclient == nullptr || metaCache == nullptr || + iotracker == nullptr || fileInfo == nullptr) { return -1; } @@ -56,60 +59,62 @@ int Splitor::IO2ChunkRequests(IOTracker* iotracker, MetaCache* mc, targetlist->reserve(length / (iosplitopt_.fileIOSplitMaxSizeKB * 1024) + 1); - uint64_t chunksize = fi->chunksize; - - uint64_t startchunkindex = offset / chunksize; - uint64_t endchunkindex = (offset + length - 1) / chunksize; - uint64_t startoffset = offset; - uint64_t endoff = offset + length; - uint64_t currentoff = offset % chunksize; - uint64_t dataoff = 0; - - while (startchunkindex <= endchunkindex) { - uint64_t off = currentoff; - uint64_t chunkendpos = chunksize * (startchunkindex + 1); - uint64_t pos = chunkendpos > endoff ? endoff : chunkendpos; - uint64_t len = pos - startoffset; + const uint64_t chunksize = fileInfo->chunksize; + uint64_t currentChunkIndex = offset / chunksize; + const uint64_t endChunkIndex = (offset + length - 1) / chunksize; + uint64_t currentRequestOffset = offset; + const uint64_t endRequestOffest = offset + length; + uint64_t currentChunkOffset = offset % chunksize; + uint64_t dataOffset = 0; + + while (currentChunkIndex <= endChunkIndex) { + const uint64_t currentChunkEndOffset = + chunksize * (currentChunkIndex + 1); + uint64_t requestLength = + std::min(currentChunkEndOffset, endRequestOffest) - + currentRequestOffset; DVLOG(9) << "request split" - << ", off = " << off - << ", len = " << len - << ", seqnum = " << fi->seqnum - << ", endoff = " << endoff - << ", chunkendpos = " << chunkendpos + << ", off = " << currentChunkOffset + << ", len = " << requestLength + << ", seqnum = " << fileInfo->seqnum + << ", endoff = " << endRequestOffest + << ", chunkendpos = " << currentChunkEndOffset << ", chunksize = " << chunksize - << ", chunkindex = " << startchunkindex - << ", endchunkindex = " << endchunkindex; + << ", chunkindex = " << currentChunkIndex + << ", endchunkindex = " << endChunkIndex; - if (!AssignInternal(iotracker, mc, targetlist, data, - off, len, mdsclient, fi, startchunkindex)) { + if (!AssignInternal(iotracker, metaCache, targetlist, data, + currentChunkOffset, requestLength, mdsclient, + fileInfo, currentChunkIndex)) { LOG(ERROR) << "request split failed" - << ", off = " << off - << ", len = " << len - << ", seqnum = " << fi->seqnum - << ", endoff = " << endoff - << ", chunkendpos = " << chunkendpos + << ", off = " << currentChunkOffset + << ", len = " << requestLength + << ", seqnum = " << fileInfo->seqnum + << ", endoff = " << endRequestOffest + << ", chunkendpos = " << currentChunkEndOffset << ", chunksize = " << chunksize - << ", chunkindex = " << startchunkindex - << ", endchunkindex = " << endchunkindex; + << ", chunkindex = " << currentChunkIndex + << ", endchunkindex = " << endChunkIndex; return -1; } - currentoff = 0; - startchunkindex++; + currentChunkOffset = 0; + currentChunkIndex++; - dataoff += len; - startoffset += len; + dataOffset += requestLength; + currentRequestOffset += requestLength; } + return 0; } // this offset is begin by chunk int Splitor::SingleChunkIO2ChunkRequests( - IOTracker* iotracker, MetaCache* mc, - std::vector* targetlist, const ChunkIDInfo_t idinfo, + IOTracker* iotracker, MetaCache* metaCache, + std::vector* targetlist, const ChunkIDInfo& idinfo, butil::IOBuf* data, off_t offset, uint64_t length, uint64_t seq) { - if (targetlist == nullptr || mc == nullptr || iotracker == nullptr) { + if (targetlist == nullptr || metaCache == nullptr || iotracker == nullptr) { return -1; } @@ -117,146 +122,164 @@ int Splitor::SingleChunkIO2ChunkRequests( return -1; } - auto max_split_size_bytes = 1024 * iosplitopt_.fileIOSplitMaxSizeKB; + const auto maxSplitSizeBytes = 1024 * iosplitopt_.fileIOSplitMaxSizeKB; - uint64_t len = 0; - uint64_t off = 0; - uint64_t tempoff = offset; - uint64_t leftlength = length; - while (leftlength > 0) { - tempoff += len; - len = leftlength > max_split_size_bytes ? max_split_size_bytes : leftlength; // NOLINT + uint64_t dataOffset = 0; + uint64_t currentOffset = offset; + uint64_t leftLength = length; + while (leftLength > 0) { + uint64_t requestLength = std::min(leftLength, maxSplitSizeBytes); - RequestContext* newreqNode = GetInitedRequestContext(); + RequestContext* newreqNode = RequestContext::NewInitedRequestContext(); if (newreqNode == nullptr) { return -1; } - newreqNode->seq_ = seq; - if (iotracker->Optype() == OpType::WRITE) { - auto nc = data->cutn(&(newreqNode->writeData_), len); - if (nc != len) { - LOG(ERROR) << "IOBuf::cutn failed, expected: " << len + auto nc = data->cutn(&(newreqNode->writeData_), requestLength); + if (nc != requestLength) { + LOG(ERROR) << "IOBuf::cutn failed, expected: " << requestLength << ", return: " << nc; return -1; } } - newreqNode->offset_ = tempoff; - newreqNode->rawlength_ = len; + newreqNode->seq_ = seq; + newreqNode->offset_ = currentOffset; + newreqNode->rawlength_ = requestLength; newreqNode->optype_ = iotracker->Optype(); newreqNode->idinfo_ = idinfo; newreqNode->done_->SetIOTracker(iotracker); targetlist->push_back(newreqNode); DVLOG(9) << "request split" - << ", off = " << tempoff - << ", len = " << len + << ", off = " << currentOffset + << ", len = " << requestLength << ", seqnum = " << seq << ", chunkid = " << idinfo.cid_ << ", copysetid = " << idinfo.cpid_ << ", logicpoolid = " << idinfo.lpid_; - leftlength -= len; - off += len; + leftLength -= requestLength; + dataOffset += requestLength; + currentOffset += requestLength; } + return 0; } -bool Splitor::AssignInternal(IOTracker* iotracker, MetaCache* mc, +bool Splitor::AssignInternal(IOTracker* iotracker, MetaCache* metaCache, std::vector* targetlist, butil::IOBuf* data, off_t off, size_t len, MDSClient* mdsclient, const FInfo_t* fileinfo, ChunkIndex chunkidx) { - ChunkIDInfo_t chinfo; - SegmentInfo segInfo; - LogicalPoolCopysetIDInfo_t lpcsIDInfo; - MetaCacheErrorType chunkidxexist = - mc->GetChunkInfoByIndex(chunkidx, &chinfo); - - if (chunkidxexist == MetaCacheErrorType::CHUNKINFO_NOT_FOUND) { - LIBCURVE_ERROR re = mdsclient->GetOrAllocateSegment(true, - (off_t)chunkidx * fileinfo->chunksize, - fileinfo, - &segInfo); - if (re != LIBCURVE_ERROR::OK) { - LOG(ERROR) << "GetOrAllocateSegment failed! " - << "offset = " << chunkidx * fileinfo->chunksize; - return false; - } else { - int count = 0; - for (auto chunkidinfo : segInfo.chunkvec) { - uint64_t index = (segInfo.startoffset + - count * fileinfo->chunksize) / fileinfo->chunksize; - mc->UpdateChunkInfoByIndex(index, chunkidinfo); - ++count; - } + const auto maxSplitSizeBytes = 1024 * iosplitopt_.fileIOSplitMaxSizeKB; - std::vector cpinfoVec; - re = mdsclient->GetServerList(segInfo.lpcpIDInfo.lpid, - segInfo.lpcpIDInfo.cpidVec, &cpinfoVec); - for (auto cpinfo : cpinfoVec) { - for (auto peerinfo : cpinfo.csinfos_) { - mc->AddCopysetIDInfo(peerinfo.chunkserverID, - CopysetIDInfo(segInfo.lpcpIDInfo.lpid, cpinfo.cpid_)); - } - } + ChunkIDInfo chunkIdInfo; + MetaCacheErrorType errCode = + metaCache->GetChunkInfoByIndex(chunkidx, &chunkIdInfo); - if (re == LIBCURVE_ERROR::FAILED) { - std::string cpidstr; - for (auto id : segInfo.lpcpIDInfo.cpidVec) { - cpidstr.append(std::to_string(id)) - .append(","); - } - - LOG(ERROR) << "GetServerList failed! " - << "logicpool id = " << segInfo.lpcpIDInfo.lpid - << ", copyset list = " << cpidstr.c_str(); - return false; - } else { - for (auto cpinfo : cpinfoVec) { - mc->UpdateCopysetInfo(segInfo.lpcpIDInfo.lpid, - cpinfo.cpid_, cpinfo); - } - } + if (errCode == MetaCacheErrorType::CHUNKINFO_NOT_FOUND) { + if (false == GetOrAllocateSegment( + true, + static_cast(chunkidx) * fileinfo->chunksize, + mdsclient, metaCache, fileinfo)) { + return false; } - chunkidxexist = mc->GetChunkInfoByIndex(chunkidx, &chinfo); + errCode = metaCache->GetChunkInfoByIndex(chunkidx, &chunkIdInfo); } - if (chunkidxexist == MetaCacheErrorType::OK) { + if (errCode == MetaCacheErrorType::OK) { int ret = 0; - auto appliedindex_ = mc->GetAppliedIndex(chinfo.lpid_, chinfo.cpid_); + uint64_t appliedindex_ = 0; + + // only read needs applied-index + if (iotracker->Optype() == OpType::READ) { + appliedindex_ = metaCache->GetAppliedIndex(chunkIdInfo.lpid_, + chunkIdInfo.cpid_); + } + std::vector templist; - ret = SingleChunkIO2ChunkRequests(iotracker, mc, &templist, chinfo, - data, off, len, fileinfo->seqnum); + ret = SingleChunkIO2ChunkRequests(iotracker, metaCache, &templist, + chunkIdInfo, data, off, len, + fileinfo->seqnum); for (auto& ctx : templist) { ctx->appliedindex_ = appliedindex_; - ctx->sourceInfo_ = CalcRequestSourceInfo(iotracker, mc, chunkidx); + ctx->sourceInfo_ = + CalcRequestSourceInfo(iotracker, metaCache, chunkidx); } targetlist->insert(targetlist->end(), templist.begin(), - templist.end()); + templist.end()); return ret == 0; } LOG(ERROR) << "can not find the chunk index info!" << ", chunk index = " << chunkidx; + return false; } -RequestContext* Splitor::GetInitedRequestContext() { - RequestContext* ctx = new (std::nothrow) RequestContext(); - if (ctx && ctx->Init()) { - return ctx; - } else { - LOG(ERROR) << "Allocate RequestContext Failed!"; - delete ctx; - return nullptr; +bool Splitor::GetOrAllocateSegment(bool allocateIfNotExist, + uint64_t offset, + MDSClient* mdsClient, + MetaCache* metaCache, + const FInfo* fileInfo) { + SegmentInfo segmentInfo; + LIBCURVE_ERROR errCode = mdsClient->GetOrAllocateSegment( + allocateIfNotExist, offset, fileInfo, &segmentInfo); + + if (errCode == LIBCURVE_ERROR::FAILED || + errCode == LIBCURVE_ERROR::AUTHFAIL) { + LOG(ERROR) << "GetOrAllocateSegmen failed, filename: " + << fileInfo->filename << ", offset: " << offset; + return false; + } + + const auto chunksize = fileInfo->chunksize; + uint32_t count = 0; + for (const auto& chunkIdInfo : segmentInfo.chunkvec) { + uint64_t chunkIdx = + (segmentInfo.startoffset + count * chunksize) / chunksize; + metaCache->UpdateChunkInfoByIndex(chunkIdx, chunkIdInfo); + ++count; } + + std::vector copysetInfos; + errCode = mdsClient->GetServerList(segmentInfo.lpcpIDInfo.lpid, + segmentInfo.lpcpIDInfo.cpidVec, + ©setInfos); + + if (errCode == LIBCURVE_ERROR::FAILED) { + std::string failedCopysets; + for (const auto& id : segmentInfo.lpcpIDInfo.cpidVec) { + failedCopysets.append(std::to_string(id)).append(","); + } + + LOG(ERROR) << "GetServerList failed, logicpool id: " + << segmentInfo.lpcpIDInfo.lpid + << ", copysets: " << failedCopysets; + + return false; + } + + for (const auto& copysetInfo : copysetInfos) { + for (const auto& peerInfo : copysetInfo.csinfos_) { + metaCache->AddCopysetIDInfo( + peerInfo.chunkserverID, + CopysetIDInfo(segmentInfo.lpcpIDInfo.lpid, copysetInfo.cpid_)); + } + } + + for (const auto& copysetInfo : copysetInfos) { + metaCache->UpdateCopysetInfo(segmentInfo.lpcpIDInfo.lpid, + copysetInfo.cpid_, copysetInfo); + } + + return true; } RequestSourceInfo Splitor::CalcRequestSourceInfo(IOTracker* ioTracker, diff --git a/src/client/splitor.h b/src/client/splitor.h index fcaf94fb01..6975250b23 100644 --- a/src/client/splitor.h +++ b/src/client/splitor.h @@ -39,11 +39,12 @@ namespace client { class Splitor { public: - static void Init(IOSplitOPtion_t ioSplitOpt); + static void Init(const IOSplitOption& ioSplitOpt); + /** * 用户IO拆分成Chunk级别的IO * @param: iotracker大IO上下文信息 - * @param: mc是io拆分过程中需要使用的缓存信息 + * @param: metaCache是io拆分过程中需要使用的缓存信息 * @param: targetlist大IO被拆分之后的小IO存储列表 * @param: data 是待写的数据 * @param: offset用户下发IO的其实偏移 @@ -52,7 +53,7 @@ class Splitor { * @param: fi存储当前IO的一些基本信息,比如chunksize等 */ static int IO2ChunkRequests(IOTracker* iotracker, - MetaCache* mc, + MetaCache* metaCache, std::vector* targetlist, butil::IOBuf* data, off_t offset, @@ -62,7 +63,7 @@ class Splitor { /** * 对单ChunkIO进行细粒度拆分 * @param: iotracker大IO上下文信息 - * @param: mc是io拆分过程中需要使用的缓存信息 + * @param: metaCache是io拆分过程中需要使用的缓存信息 * @param: targetlist大IO被拆分之后的小IO存储列表 * @param: cid是当前chunk的ID信息 * @param: data是待写的数据 @@ -71,9 +72,9 @@ class Splitor { * @param: seq是当前chunk的版本号 */ static int SingleChunkIO2ChunkRequests(IOTracker* iotracker, - MetaCache* mc, + MetaCache* metaCache, std::vector* targetlist, - const ChunkIDInfo_t cid, + const ChunkIDInfo& cid, butil::IOBuf* data, off_t offset, size_t length, @@ -104,7 +105,7 @@ class Splitor { * @param: chunkidx是当前chunk在vdisk中的索引值 */ static bool AssignInternal(IOTracker* iotracker, - MetaCache* mc, + MetaCache* metaCache, std::vector* targetlist, butil::IOBuf* data, off_t offset, @@ -113,11 +114,15 @@ class Splitor { const FInfo_t* fi, ChunkIndex chunkidx); - static RequestContext* GetInitedRequestContext(); + static bool GetOrAllocateSegment(bool allocateIfNotExist, + uint64_t offset, + MDSClient* mdsClient, + MetaCache* metaCache, + const FInfo* fileInfo); private: // IO拆分模块所使用的配置信息 - static IOSplitOPtion_t iosplitopt_; + static IOSplitOption iosplitopt_; }; } // namespace client } // namespace curve diff --git a/src/client/unstable_helper.cpp b/src/client/unstable_helper.cpp new file mode 100644 index 0000000000..26fa1bbea8 --- /dev/null +++ b/src/client/unstable_helper.cpp @@ -0,0 +1,69 @@ +/* + * Copyright (c) 2020 NetEase Inc. + * + * Licensed 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. + */ + +/* + * Project: curve + * Date: Sat Sep 5 11:13:12 CST 2020 + */ + +#include "src/client/unstable_helper.h" + +namespace curve { +namespace client { + +UnstableState UnstableHelper::GetCurrentUnstableState( + ChunkServerID csId, + const butil::EndPoint& csEndPoint) { + + std::string ip = butil::ip2str(csEndPoint.ip).c_str(); + + mtx_.lock(); + // 如果当前ip已经超过阈值,则直接返回chunkserver unstable + int unstabled = serverUnstabledChunkservers_[ip].size(); + if (unstabled >= option_.serverUnstableThreshold) { + serverUnstabledChunkservers_[ip].emplace(csId); + mtx_.unlock(); + return UnstableState::ChunkServerUnstable; + } + + bool exceed = + timeoutTimes_[csId] > option_.maxStableChunkServerTimeoutTimes; + mtx_.unlock(); + + if (exceed == false) { + return UnstableState::NoUnstable; + } + + bool health = CheckChunkServerHealth(csEndPoint); + if (health) { + ClearTimeout(csId, csEndPoint); + return UnstableState::NoUnstable; + } + + mtx_.lock(); + auto ret = serverUnstabledChunkservers_[ip].emplace(csId); + unstabled = serverUnstabledChunkservers_[ip].size(); + mtx_.unlock(); + + if (ret.second && unstabled == option_.serverUnstableThreshold) { + return UnstableState::ServerUnstable; + } else { + return UnstableState::ChunkServerUnstable; + } +} + +} // namespace client +} // namespace curve diff --git a/src/client/unstable_helper.h b/src/client/unstable_helper.h new file mode 100644 index 0000000000..89cc22be8e --- /dev/null +++ b/src/client/unstable_helper.h @@ -0,0 +1,106 @@ +/* + * Copyright (c) 2020 NetEase Inc. + * + * Licensed 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. + */ + +/* + * Project: curve + * Date: Sat Sep 5 11:13:12 CST 2020 + */ + +#ifndef SRC_CLIENT_UNSTABLE_HELPER_H_ +#define SRC_CLIENT_UNSTABLE_HELPER_H_ + +#include + +#include +#include +#include + +#include "src/client/client_common.h" +#include "src/client/client_config.h" +#include "src/client/service_helper.h" + +namespace curve { +namespace client { + +enum class UnstableState { + NoUnstable, + ChunkServerUnstable, + ServerUnstable +}; + +// 如果chunkserver宕机或者网络不可达, 发往对应chunkserver的rpc会超时 +// 返回之后, 回去refresh leader然后再去发送请求 +// 这种情况下不同copyset上的请求,总会先rpc timedout然后重新refresh leader +// 为了避免一次多余的rpc timedout +// 记录一下发往同一个chunkserver上超时请求的次数 +// 如果超过一定的阈值,会发送http请求检查chunkserver是否健康 +// 如果不健康,则通知所有leader在这台chunkserver上的copyset +// 主动去refresh leader,而不是根据缓存的leader信息直接发送rpc +class UnstableHelper { + public: + UnstableHelper() = default; + + UnstableHelper(const UnstableHelper&) = delete; + UnstableHelper& operator=(const UnstableHelper&) = delete; + + void Init(const ChunkServerUnstableOption& opt) { + option_ = opt; + } + + void IncreTimeout(ChunkServerID csId) { + std::unique_lock guard(mtx_); + ++timeoutTimes_[csId]; + } + + UnstableState GetCurrentUnstableState(ChunkServerID csId, + const butil::EndPoint& csEndPoint); + + void ClearTimeout(ChunkServerID csId, const butil::EndPoint& csEndPoint) { + std::string ip = butil::ip2str(csEndPoint.ip).c_str(); + + std::unique_lock guard(mtx_); + timeoutTimes_[csId] = 0; + serverUnstabledChunkservers_[ip].clear(); + } + + private: + /** + * @brief 检查chunkserver状态 + * + * @param: endPoint chunkserver的ip:port地址 + * @return: true 健康 / false 不健康 + */ + bool CheckChunkServerHealth(const butil::EndPoint& endPoint) const { + return ServiceHelper::CheckChunkServerHealth( + endPoint, option_.checkHealthTimeoutMS) == 0; + } + + ChunkServerUnstableOption option_; + + bthread::Mutex mtx_; + + // 同一chunkserver连续超时请求次数 + std::unordered_map timeoutTimes_; + + // 同一server上unstable chunkserver的id + std::unordered_map> + serverUnstabledChunkservers_; +}; + +} // namespace client +} // namespace curve + +#endif // SRC_CLIENT_UNSTABLE_HELPER_H_ diff --git a/src/common/concurrent/bounded_blocking_queue.h b/src/common/concurrent/bounded_blocking_queue.h index 33118c2285..56c59fcfc1 100644 --- a/src/common/concurrent/bounded_blocking_queue.h +++ b/src/common/concurrent/bounded_blocking_queue.h @@ -29,6 +29,7 @@ #include #include //NOLINT #include +#include #include "src/common/uncopyable.h" @@ -113,7 +114,7 @@ class BoundedBlockingDeque : public Uncopyable { while (deque_.empty()) { notEmpty_.wait(guard); } - T front(deque_.front()); + T front(std::move(deque_.front())); deque_.pop_front(); notFull_.notify_one(); return front; @@ -124,7 +125,7 @@ class BoundedBlockingDeque : public Uncopyable { while (deque_.empty()) { notEmpty_.wait(guard); } - T back(deque_.back()); + T back(std::move(deque_.back())); deque_.pop_back(); notFull_.notify_one(); return back; diff --git a/src/common/concurrent/concurrent.h b/src/common/concurrent/concurrent.h index 0ae089f745..df79ea8ec8 100644 --- a/src/common/concurrent/concurrent.h +++ b/src/common/concurrent/concurrent.h @@ -56,10 +56,7 @@ using WriteLockGuard = WriteLockGuard; // curve内部定义的线程组件 using TaskQueue = TaskQueue; using ThreadPool = ThreadPool; -using TaskThreadPool = TaskThreadPool; } // namespace common } // namespace curve #endif // SRC_COMMON_CONCURRENT_CONCURRENT_H_ - - diff --git a/src/common/concurrent/task_thread_pool.cpp b/src/common/concurrent/task_thread_pool.cpp deleted file mode 100644 index c53e15f9e0..0000000000 --- a/src/common/concurrent/task_thread_pool.cpp +++ /dev/null @@ -1,117 +0,0 @@ -/* - * Copyright (c) 2020 NetEase Inc. - * - * Licensed 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. - */ - -/* - * Project: curve - * Created Date: 18-12-14 - * Author: wudemiao - */ - -#include "src/common/concurrent/task_thread_pool.h" - -namespace curve { -namespace common { - -TaskThreadPool::TaskThreadPool() - : mutex_(), - notEmpty_(), - notFull_(), - capacity_(-1), - running_(false) { -} - -TaskThreadPool::~TaskThreadPool() { - if (running_.load(std::memory_order_acquire)) { - Stop(); - } -} - -int TaskThreadPool::QueueCapacity() const { - return capacity_; -} - -int TaskThreadPool::ThreadOfNums() const { - return threads_.size(); -} - -int TaskThreadPool::QueueSize() const { - std::unique_lock guard(mutex_); - return queue_.size(); -} - -int TaskThreadPool::Start(int numThreads, int queueCapacity) { - if (0 >= queueCapacity) { - return -1; - } - capacity_ = queueCapacity; - - if (0 >= numThreads) { - return -1; - } - - if (!running_.exchange(true, std::memory_order_acq_rel)) { - threads_.reserve(numThreads); - for (int i = 0; i < numThreads; ++i) { - threads_.emplace_back(new std::thread(std::bind(&TaskThreadPool::ThreadFunc, // NOLINT - this))); - } - } - - return 0; -} - -void TaskThreadPool::Stop() { - if (running_.exchange(false, std::memory_order_acq_rel)) { - { - std::lock_guard guard(mutex_); - notEmpty_.notify_all(); - } - for (auto &thr : threads_) { - thr->join(); - } - } -} - -TaskThreadPool::Task TaskThreadPool::Take() { - std::unique_lock guard(mutex_); - while (queue_.empty() && running_.load(std::memory_order_acquire)) { - notEmpty_.wait(guard); - } - Task task; - if (!queue_.empty()) { - task = queue_.front(); - queue_.pop_front(); - notFull_.notify_one(); - } - return task; -} - -void TaskThreadPool::ThreadFunc() { - while (running_.load(std::memory_order_acquire)) { - Task task(Take()); - /* ThreadPool 退出的时候,queue 为空,那么会返回无效的 task */ - if (task) { - task(); - } - } -} - -bool TaskThreadPool::IsFullUnlock() const { - return queue_.size() >= capacity_; -} - -} // namespace common -} // namespace curve diff --git a/src/common/concurrent/task_thread_pool.h b/src/common/concurrent/task_thread_pool.h index d1b22fe15a..207af2ad5e 100644 --- a/src/common/concurrent/task_thread_pool.h +++ b/src/common/concurrent/task_thread_pool.h @@ -41,12 +41,20 @@ namespace curve { namespace common { // 异步运行回调的线程池 +template class TaskThreadPool : public Uncopyable { public: using Task = std::function; - TaskThreadPool(); - virtual ~TaskThreadPool(); + TaskThreadPool() + : mutex_(), notEmpty_(), notFull_(), capacity_(-1), running_(false) {} + + virtual ~TaskThreadPool() { + if (running_.load(std::memory_order_acquire)) { + Stop(); + } + } /** * 启动一个线程池 @@ -54,11 +62,42 @@ class TaskThreadPool : public Uncopyable { * @param queueCapacity queue 的容量,必须大于 0 * @return */ - int Start(int numThreads, int queueCapacity = INT_MAX); + int Start(int numThreads, int queueCapacity = INT_MAX) { + if (0 >= queueCapacity) { + return -1; + } + capacity_ = queueCapacity; + + if (0 >= numThreads) { + return -1; + } + + if (!running_.exchange(true, std::memory_order_acq_rel)) { + threads_.reserve(numThreads); + for (int i = 0; i < numThreads; ++i) { + threads_.emplace_back(new std::thread( + std::bind(&TaskThreadPool::ThreadFunc, this))); + } + } + + return 0; + } + /** * 关闭线程池 */ - void Stop(); + void Stop() { + if (running_.exchange(false, std::memory_order_acq_rel)) { + { + std::lock_guard guard(mutex_); + notEmpty_.notify_all(); + } + for (auto& thr : threads_) { + thr->join(); + } + } + } + /** * push 一个 task 给线程池处理,如果队列满,线程阻塞,直到 task push 进去 * 需要注意的是用户自己需要保证 task 的有效的。除此之外,此 TaskThreadPool @@ -69,45 +108,75 @@ class TaskThreadPool : public Uncopyable { * @param f * @param args */ - template - void Enqueue(F &&f, Args &&... args); + template + void Enqueue(F&& f, Args&&... args) { + std::unique_lock guard(mutex_); + while (IsFullUnlock()) { + notFull_.wait(guard); + } + auto task = std::bind(std::forward(f), std::forward(args)...); + queue_.push_back(std::move(task)); + notEmpty_.notify_one(); + } /* 返回线程池 queue 的容量 */ - int QueueCapacity() const; + int QueueCapacity() const { + return capacity_; + } + /* 返回线程池当前 queue 中的 task 数量,线程安全 */ - int QueueSize() const; + int QueueSize() const { + std::lock_guard guard(mutex_); + return queue_.size(); + } + /* 返回线程池的线程数 */ - int ThreadOfNums() const; + int ThreadOfNums() const { + return threads_.size(); + } protected: /*线程工作时执行的函数*/ - virtual void ThreadFunc(); + virtual void ThreadFunc() { + while (running_.load(std::memory_order_acquire)) { + Task task(Take()); + /* ThreadPool 退出的时候,queue 为空,那么会返回无效的 task */ + if (task) { + task(); + } + } + } + /* 判断线程池 queue 是否已经满了, 非线程安全,私有内部使用 */ - bool IsFullUnlock() const; + bool IsFullUnlock() const { + return queue_.size() >= capacity_; + } + /* 从线程池的 queue 中取一个 task 线程安全 */ - Task Take(); + Task Take() { + std::unique_lock guard(mutex_); + while (queue_.empty() && running_.load(std::memory_order_acquire)) { + notEmpty_.wait(guard); + } + Task task; + if (!queue_.empty()) { + task = std::move(queue_.front()); + queue_.pop_front(); + notFull_.notify_one(); + } + return task; + } protected: - mutable std::mutex mutex_; - std::condition_variable notEmpty_; - std::condition_variable notFull_; + mutable MutexT mutex_; + CondVarT notEmpty_; + CondVarT notFull_; std::vector> threads_; std::deque queue_; int capacity_; std::atomic running_; }; -template -void TaskThreadPool::Enqueue(F &&f, Args &&... args) { - std::unique_lock guard(mutex_); - while (IsFullUnlock()) { - notFull_.wait(guard); - } - auto task = std::bind(std::forward(f), std::forward(args)...); - queue_.push_back(std::move(task)); - notEmpty_.notify_one(); -} - } // namespace common } // namespace curve diff --git a/src/common/math_util.h b/src/common/math_util.h new file mode 100644 index 0000000000..97651a26cc --- /dev/null +++ b/src/common/math_util.h @@ -0,0 +1,43 @@ +/* + * Copyright (c) 2020 NetEase Inc. + * + * Licensed 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. + */ + +/* + * Project: curve + * Created Date: Sun Sep 6 17:13:58 CST 2020 + */ + +#ifndef SRC_COMMON_MATH_UTIL_H_ +#define SRC_COMMON_MATH_UTIL_H_ + +#include +#include + +namespace curve { +namespace common { + +inline uint64_t MaxPowerTimesLessEqualValue(uint64_t value) { + uint64_t pow = 0; + while (value > 1) { + value >>= 1; + pow++; + } + return pow; +} + +} // namespace common +} // namespace curve + +#endif // SRC_COMMON_MATH_UTIL_H_ diff --git a/src/mds/nameserver2/clean_task_manager.cpp b/src/mds/nameserver2/clean_task_manager.cpp index d56c061a28..400911bc15 100644 --- a/src/mds/nameserver2/clean_task_manager.cpp +++ b/src/mds/nameserver2/clean_task_manager.cpp @@ -69,7 +69,7 @@ bool CleanTaskManager::Start(void) { stopFlag_ = false; // start worker thread - cleanWorkers_ = new ::curve::common::TaskThreadPool(); + cleanWorkers_ = new ::curve::common::TaskThreadPool<>(); if (cleanWorkers_->Start(threadNum_) != 0) { LOG(ERROR) << "thread pool start error"; diff --git a/src/mds/nameserver2/clean_task_manager.h b/src/mds/nameserver2/clean_task_manager.h index 8c491e2ab5..9673a0b1c4 100644 --- a/src/mds/nameserver2/clean_task_manager.h +++ b/src/mds/nameserver2/clean_task_manager.h @@ -85,7 +85,7 @@ class CleanTaskManager { private: int threadNum_; - ::curve::common::TaskThreadPool *cleanWorkers_; + ::curve::common::TaskThreadPool<> *cleanWorkers_; // for period check snapshot delete status std::unordered_map> cleanTasks_; common::Mutex mutex_; diff --git a/src/snapshotcloneserver/common/thread_pool.h b/src/snapshotcloneserver/common/thread_pool.h index 15935393be..1f7b4ea697 100644 --- a/src/snapshotcloneserver/common/thread_pool.h +++ b/src/snapshotcloneserver/common/thread_pool.h @@ -74,7 +74,7 @@ class ThreadPool { /** * @brief 通用线程池 */ - curve::common::TaskThreadPool threadPool_; + curve::common::TaskThreadPool<> threadPool_; /** * @brief 线程数 */ diff --git a/test/client/backoff_test.cpp b/test/client/backoff_test.cpp index cbb416f8aa..80a28b66f3 100644 --- a/test/client/backoff_test.cpp +++ b/test/client/backoff_test.cpp @@ -20,36 +20,17 @@ * Author: tongguangxun */ -#include #include -#include "src/client/config_info.h" +#include + #include "src/client/chunk_closure.h" +#include "src/client/config_info.h" namespace curve { namespace client { -TEST(ClientClosure, GetPowTimeTest) { - ASSERT_EQ(0, ClientClosure::GetPowTime(0)); - ASSERT_EQ(0, ClientClosure::GetPowTime(1)); - ASSERT_EQ(2, ClientClosure::GetPowTime(4)); - ASSERT_EQ(1, ClientClosure::GetPowTime(2)); - ASSERT_EQ(1, ClientClosure::GetPowTime(3)); - ASSERT_EQ(2, ClientClosure::GetPowTime(7)); - ASSERT_EQ(3, ClientClosure::GetPowTime(10)); - ASSERT_EQ(3, ClientClosure::GetPowTime(15)); - ASSERT_EQ(5, ClientClosure::GetPowTime(32)); - ASSERT_EQ(5, ClientClosure::GetPowTime(63)); - ASSERT_EQ(6, ClientClosure::GetPowTime(64)); - ASSERT_EQ(7, ClientClosure::GetPowTime(255)); - ASSERT_EQ(8, ClientClosure::GetPowTime(256)); - ASSERT_EQ(8, ClientClosure::GetPowTime(257)); - ASSERT_EQ(10, ClientClosure::GetPowTime(1024)); - ASSERT_EQ(10, ClientClosure::GetPowTime(2047)); - ASSERT_EQ(11, ClientClosure::GetPowTime(2048)); - ASSERT_EQ(11, ClientClosure::GetPowTime(2049)); -} TEST(ClientClosure, OverLoadBackOffTest) { - FailureRequestOption_t failopt; + FailureRequestOption failopt; failopt.chunkserverMaxRetrySleepIntervalUS = 8000000; failopt.chunkserverOPRetryIntervalUS = 500000; @@ -59,16 +40,17 @@ TEST(ClientClosure, OverLoadBackOffTest) { for (int i = 1; i < 1000; i++) { if (i < ClientClosure::backoffParam_.maxOverloadPow) { - uint64_t curTime = failopt.chunkserverOPRetryIntervalUS*std::pow(2, i); // NOLINT + uint64_t curTime = + failopt.chunkserverOPRetryIntervalUS * std::pow(2, i); ASSERT_LT(cc.OverLoadBackOff(i), curTime + 0.1 * curTime); ASSERT_GT(cc.OverLoadBackOff(i), curTime - 0.1 * curTime); } else { ASSERT_LT(cc.OverLoadBackOff(i), - failopt.chunkserverMaxRetrySleepIntervalUS + - 0.1 * failopt.chunkserverMaxRetrySleepIntervalUS); + failopt.chunkserverMaxRetrySleepIntervalUS + + 0.1 * failopt.chunkserverMaxRetrySleepIntervalUS); ASSERT_GT(cc.OverLoadBackOff(i), - failopt.chunkserverMaxRetrySleepIntervalUS - - 0.1 * failopt.chunkserverMaxRetrySleepIntervalUS); + failopt.chunkserverMaxRetrySleepIntervalUS - + 0.1 * failopt.chunkserverMaxRetrySleepIntervalUS); } } @@ -79,22 +61,23 @@ TEST(ClientClosure, OverLoadBackOffTest) { for (int i = 1; i < 1000; i++) { if (i < ClientClosure::backoffParam_.maxOverloadPow) { - uint64_t curTime = failopt.chunkserverOPRetryIntervalUS*std::pow(2, i); // NOLINT + uint64_t curTime = + failopt.chunkserverOPRetryIntervalUS * std::pow(2, i); ASSERT_LT(cc.OverLoadBackOff(i), curTime + 0.1 * curTime); ASSERT_GT(cc.OverLoadBackOff(i), curTime - 0.1 * curTime); } else { ASSERT_LT(cc.OverLoadBackOff(i), - failopt.chunkserverMaxRetrySleepIntervalUS + - 0.1 * failopt.chunkserverMaxRetrySleepIntervalUS); + failopt.chunkserverMaxRetrySleepIntervalUS + + 0.1 * failopt.chunkserverMaxRetrySleepIntervalUS); ASSERT_GT(cc.OverLoadBackOff(i), - failopt.chunkserverMaxRetrySleepIntervalUS - - 0.1 * failopt.chunkserverMaxRetrySleepIntervalUS); + failopt.chunkserverMaxRetrySleepIntervalUS - + 0.1 * failopt.chunkserverMaxRetrySleepIntervalUS); } } } TEST(ClientClosure, TimeoutBackOffTest) { - FailureRequestOption_t failopt; + FailureRequestOption failopt; failopt.chunkserverMaxRPCTimeoutMS = 3000; failopt.chunkserverRPCTimeoutMS = 500; @@ -104,7 +87,7 @@ TEST(ClientClosure, TimeoutBackOffTest) { for (int i = 1; i < 1000; i++) { if (i < ClientClosure::backoffParam_.maxTimeoutPow) { - uint64_t curTime = failopt.chunkserverRPCTimeoutMS*std::pow(2, i); + uint64_t curTime = failopt.chunkserverRPCTimeoutMS * std::pow(2, i); ASSERT_EQ(cc.TimeoutBackOff(i), curTime); } else { ASSERT_EQ(cc.TimeoutBackOff(i), 2000); @@ -118,7 +101,7 @@ TEST(ClientClosure, TimeoutBackOffTest) { for (int i = 1; i < 1000; i++) { if (i < ClientClosure::backoffParam_.maxTimeoutPow) { - uint64_t curTime = failopt.chunkserverRPCTimeoutMS*std::pow(2, i); + uint64_t curTime = failopt.chunkserverRPCTimeoutMS * std::pow(2, i); ASSERT_EQ(cc.TimeoutBackOff(i), curTime); } else { ASSERT_EQ(cc.TimeoutBackOff(i), 4000); @@ -126,5 +109,5 @@ TEST(ClientClosure, TimeoutBackOffTest) { } } -} // namespace client -} // namespace curve +} // namespace client +} // namespace curve diff --git a/test/client/client_mdsclient_metacache_unittest.cpp b/test/client/client_mdsclient_metacache_unittest.cpp index 38a226ec62..a8a82387fd 100644 --- a/test/client/client_mdsclient_metacache_unittest.cpp +++ b/test/client/client_mdsclient_metacache_unittest.cpp @@ -58,28 +58,12 @@ extern uint32_t chunk_size; extern std::string configpath; extern curve::client::FileClient* globalclient; -using curve::client::MDSClient; -using curve::client::UserInfo_t; -using curve::client::CopysetPeerInfo; -using curve::client::CopysetInfo_t; -using curve::client::SegmentInfo; -using curve::client::FInfo; -using curve::client::LeaseSession; -using curve::client::LogicalPoolCopysetIDInfo_t; -using curve::client::MetaCacheErrorType; -using curve::client::MDSClient; -using curve::client::ServiceHelper; -using curve::client::FileClient; -using curve::client::LogicPoolID; -using curve::client::CopysetID; -using curve::client::ChunkServerID; -using curve::client::ChunkServerAddr; -using curve::client::FileInstance; +namespace curve { +namespace client { + using curve::mds::CurveFSService; using curve::mds::topology::TopologyService; -using ::curve::mds::topology::GetChunkServerListInCopySetsResponse; -using curve::client::GetLeaderInfo; -using curve::client::GetLeaderRpcOption; +using curve::mds::topology::GetChunkServerListInCopySetsResponse; using curve::mds::topology::ChunkServerStatus; using curve::mds::topology::DiskState; using curve::mds::topology::OnlineState; @@ -87,8 +71,8 @@ using curve::mds::topology::OnlineState; class MDSClientTest : public ::testing::Test { public: void SetUp() { - metaopt.metaaddrvec.push_back("127.0.0.1:9104"); - metaopt.metaaddrvec.push_back("127.0.0.1:9104"); + metaopt.mdsAddrs.push_back("127.0.0.1:9104"); + metaopt.mdsAddrs.push_back("127.0.0.1:9104"); metaopt.mdsMaxRetryMS = 1000; metaopt.mdsRPCTimeoutMs = 500; @@ -148,7 +132,7 @@ class MDSClientTest : public ::testing::Test { FileClient fileClient_; UserInfo_t userinfo; MDSClient mdsclient_; - MetaServerOption_t metaopt; + MetaServerOption metaopt; FakeTopologyService topologyservice; FakeMDSCurveFSService curvefsservice; static int i; @@ -993,7 +977,7 @@ TEST_F(MDSClientTest, GetOrAllocateSegment) { ++count; } - std::vector cpinfoVec; + std::vector cpinfoVec; mdsclient_.GetServerList(segInfo.lpcpIDInfo.lpid, segInfo.lpcpIDInfo.cpidVec, &cpinfoVec); for (auto iter : cpinfoVec) { @@ -1060,7 +1044,7 @@ TEST_F(MDSClientTest, GetOrAllocateSegment) { ASSERT_EQ(0, mc.GetAppliedIndex(1111, 0)); // test applied index update - curve::client::CopysetInfo_t csinfo; + curve::client::CopysetInfo csinfo; mc.UpdateCopysetInfo(111, 123, csinfo); ASSERT_EQ(0, mc.GetAppliedIndex(111, 123)); mc.UpdateAppliedIndex(111, 123, 4); @@ -1130,7 +1114,7 @@ TEST_F(MDSClientTest, GetServerList) { cpidvec.push_back(i); } - std::vector cpinfoVec; + std::vector cpinfoVec; curve::client::MetaCache mc; ASSERT_NE(LIBCURVE_ERROR::FAILED, mdsclient_.GetServerList(1234, cpidvec, &cpinfoVec)); @@ -1209,7 +1193,7 @@ TEST_F(MDSClientTest, GetLeaderTest) { curve::client::MetaCache mc; MetaCacheOption mcOpt; mc.Init(mcOpt, &mdsclient_); - curve::client::CopysetInfo_t cslist; + curve::client::CopysetInfo cslist; curve::client::ChunkServerAddr pd1; pd1.Parse("10.182.26.2:9120:0"); @@ -1343,9 +1327,9 @@ TEST_F(MDSClientTest, GetLeaderTest) { mc.GetLeader(1234, 1234, &ckid, &leaderep, true); - CopysetInfo_t cpinfo = mc.GetServerList(1234, 1234); + CopysetInfo cpinfo = mc.GetServerList(1234, 1234); ASSERT_EQ(cpinfo.csinfos_.size(), 5); - curve::client::CopysetPeerInfo_t cpeer; + curve::client::CopysetPeerInfo cpeer; cpeer.internalAddr.Parse("10.182.26.2:9124:0"); cpeer.externalAddr.Parse("127.0.0.1:9124:0"); auto it = std::find(cpinfo.csinfos_.begin(), cpinfo.csinfos_.end(), cpeer); @@ -2221,9 +2205,6 @@ TEST_F(MDSClientTest, StatFileStatusTest) { } } -namespace curve { -namespace client { - using ::testing::_; using ::testing::DoAll; using ::testing::ElementsAre; @@ -2274,7 +2255,7 @@ TEST_F(MDSClientRefreshSessionTest, StartDummyServerTest) { MDSClient mdsClient; MetaServerOption opt; - opt.metaaddrvec.push_back(kServerAddress); + opt.mdsAddrs.push_back(kServerAddress); ASSERT_EQ(0, mdsClient.Initialize(opt)); curve::mds::ReFreshSessionRequest request; @@ -2302,7 +2283,7 @@ TEST_F(MDSClientRefreshSessionTest, NoStartDummyServerTest) { MDSClient mdsClient; MetaServerOption opt; - opt.metaaddrvec.push_back(kServerAddress); + opt.mdsAddrs.push_back(kServerAddress); ASSERT_EQ(0, mdsClient.Initialize(opt)); curve::mds::ReFreshSessionRequest request; diff --git a/test/client/client_metric_test.cpp b/test/client/client_metric_test.cpp index 3ecda775a8..09247a2688 100644 --- a/test/client/client_metric_test.cpp +++ b/test/client/client_metric_test.cpp @@ -63,8 +63,8 @@ const std::vector clientConf { }; TEST(MetricTest, ChunkServer_MetricTest) { - MetaServerOption_t metaopt; - metaopt.metaaddrvec.push_back(mdsMetaServerAddr); + MetaServerOption metaopt; + metaopt.mdsAddrs.push_back(mdsMetaServerAddr); metaopt.mdsRPCTimeoutMs = 500; metaopt.mdsRPCRetryIntervalUS = 200; @@ -181,8 +181,8 @@ void cb(CurveAioContext* ctx) { } TEST(MetricTest, SuspendRPC_MetricTest) { - MetaServerOption_t metaopt; - metaopt.metaaddrvec.push_back(mdsMetaServerAddr); + MetaServerOption metaopt; + metaopt.mdsAddrs.push_back(mdsMetaServerAddr); metaopt.mdsRPCTimeoutMs = 500; metaopt.mdsRPCRetryIntervalUS = 200; @@ -208,7 +208,7 @@ TEST(MetricTest, SuspendRPC_MetricTest) { UserInfo_t userinfo; userinfo.owner = "test"; - FileServiceOption_t opt; + FileServiceOption opt; opt.ioOpt.reqSchdulerOpt. ioSenderOpt.failRequestOpt.chunkserverOPMaxRetry = 50; opt.ioOpt.reqSchdulerOpt. diff --git a/test/client/client_unstable_helper_test.cpp b/test/client/client_unstable_helper_test.cpp index dc358667f5..cbb62891a5 100644 --- a/test/client/client_unstable_helper_test.cpp +++ b/test/client/client_unstable_helper_test.cpp @@ -26,19 +26,19 @@ #include #include -#include "src/client/chunk_closure.h" +#include "src/client/unstable_helper.h" namespace curve { namespace client { TEST(UnstableHelperTest, normal_test) { - UnstableHelper& helper = UnstableHelper::GetInstance(); + UnstableHelper helper; ChunkServerUnstableOption opt; opt.maxStableChunkServerTimeoutTimes = 10; opt.serverUnstableThreshold = 3; - helper.SetUnstableChunkServerOption(opt); + helper.Init(opt); std::vector> chunkservers; for (int i = 1; i <= opt.serverUnstableThreshold; ++i) { @@ -61,32 +61,32 @@ TEST(UnstableHelperTest, normal_test) { // 再对每个chunkserver增加一次超时 // 前两个是chunkserver unstable状态,第三个是server unstable - UnstableHelper::GetInstance().IncreTimeout(chunkservers[0].first); + helper.IncreTimeout(chunkservers[0].first); ASSERT_EQ(UnstableState::ChunkServerUnstable, helper.GetCurrentUnstableState( chunkservers[0].first, chunkservers[0].second)); - UnstableHelper::GetInstance().IncreTimeout(chunkservers[1].first); + helper.IncreTimeout(chunkservers[1].first); ASSERT_EQ(UnstableState::ChunkServerUnstable, helper.GetCurrentUnstableState( chunkservers[1].first, chunkservers[1].second)); - UnstableHelper::GetInstance().IncreTimeout(chunkservers[2].first); + helper.IncreTimeout(chunkservers[2].first); ASSERT_EQ(UnstableState::ServerUnstable, helper.GetCurrentUnstableState( chunkservers[2].first, chunkservers[2].second)); // 继续增加超时次数 // 这种情况下,每次都是chunkserver unstable - UnstableHelper::GetInstance().IncreTimeout(chunkservers[0].first); + helper.IncreTimeout(chunkservers[0].first); ASSERT_EQ(UnstableState::ChunkServerUnstable, helper.GetCurrentUnstableState( chunkservers[0].first, chunkservers[0].second)); - UnstableHelper::GetInstance().IncreTimeout(chunkservers[1].first); + helper.IncreTimeout(chunkservers[1].first); ASSERT_EQ(UnstableState::ChunkServerUnstable, helper.GetCurrentUnstableState( chunkservers[1].first, chunkservers[1].second)); - UnstableHelper::GetInstance().IncreTimeout(chunkservers[2].first); + helper.IncreTimeout(chunkservers[2].first); ASSERT_EQ(UnstableState::ChunkServerUnstable, helper.GetCurrentUnstableState( chunkservers[2].first, chunkservers[2].second)); diff --git a/test/client/client_userifo_unittest.cpp b/test/client/client_userifo_unittest.cpp index c29715200a..8bc0306e4a 100644 --- a/test/client/client_userifo_unittest.cpp +++ b/test/client/client_userifo_unittest.cpp @@ -41,36 +41,15 @@ extern std::string mdsMetaServerAddr; extern std::string configpath; -using curve::client::MDSClient; -using curve::client::UserInfo_t; -using curve::client::ClientConfig; -using curve::client::FileClient; -using curve::client::FileInstance; -using curve::client::SegmentInfo; -using curve::client::ChunkInfoDetail; -using curve::client::SnapshotClient; -using curve::client::ChunkID; -using curve::client::LogicPoolID; -using curve::client::CopysetID; -using curve::client::ChunkIDInfo_t; -using curve::client::CopysetInfo_t; -using curve::client::MetaCache; -using curve::client::IOManager4Chunk; -using curve::client::LogicalPoolCopysetIDInfo; -using curve::client::FileMetric; - -/* -void sessioncallback(CurveAioContext* aioctx) { - ASSERT_EQ(-LIBCURVE_ERROR::DISABLEIO, aioctx->ret); -} -*/ +namespace curve { +namespace client { class CurveClientUserAuthFail : public ::testing::Test { public: void SetUp() { - metaopt.metaaddrvec.push_back("127.0.0.1:9104"); + metaopt.mdsAddrs.push_back("127.0.0.1:9104"); - metaopt.metaaddrvec.push_back("127.0.0.1:9104"); + metaopt.mdsAddrs.push_back("127.0.0.1:9104"); metaopt.mdsRPCTimeoutMs = 500; metaopt.mdsRPCRetryIntervalUS = 200; @@ -96,7 +75,7 @@ class CurveClientUserAuthFail : public ::testing::Test { } brpc::Server server; - MetaServerOption_t metaopt; + MetaServerOption metaopt; FakeMDSCurveFSService curvefsservice; FakeMDSTopologyService topologyservice; }; @@ -235,9 +214,9 @@ TEST_F(CurveClientUserAuthFail, CurveClientUserAuthFailTest) { } TEST_F(CurveClientUserAuthFail, CurveSnapClientUserAuthFailTest) { - ClientConfigOption_t opt; + ClientConfigOption opt; opt.metaServerOpt.mdsRPCTimeoutMs = 500; - opt.metaServerOpt.metaaddrvec.push_back("127.0.0.1:9104"); + opt.metaServerOpt.mdsAddrs.push_back("127.0.0.1:9104"); opt.ioOpt.reqSchdulerOpt.scheduleQueueCapacity = 4096; opt.ioOpt.reqSchdulerOpt.scheduleThreadpoolSize = 2; opt.ioOpt.ioSenderOpt.failRequestOpt.chunkserverOPMaxRetry = 3; @@ -369,9 +348,9 @@ TEST_F(CurveClientUserAuthFail, CurveSnapClientUserAuthFailTest) { // root user测试 TEST_F(CurveClientUserAuthFail, CurveSnapClientRootUserAuthTest) { - ClientConfigOption_t opt; + ClientConfigOption opt; opt.metaServerOpt.mdsRPCTimeoutMs = 500; - opt.metaServerOpt.metaaddrvec.push_back("127.0.0.1:9104"); + opt.metaServerOpt.mdsAddrs.push_back("127.0.0.1:9104"); opt.ioOpt.reqSchdulerOpt.scheduleQueueCapacity = 4096; opt.ioOpt.reqSchdulerOpt.scheduleThreadpoolSize = 2; opt.ioOpt.ioSenderOpt.failRequestOpt.chunkserverOPMaxRetry = 3; @@ -503,3 +482,6 @@ TEST_F(CurveClientUserAuthFail, CurveSnapClientRootUserAuthTest) { delete listfakeret; delete delfakeret; } + +} // namespace client +} // namespace curve diff --git a/test/client/copyset_client_test.cpp b/test/client/copyset_client_test.cpp index d602f8fd2d..9b64cfa1ab 100644 --- a/test/client/copyset_client_test.cpp +++ b/test/client/copyset_client_test.cpp @@ -174,7 +174,7 @@ TEST_F(CopysetClientTest, normal_test) { brpc::SERVER_DOESNT_OWN_SERVICE), 0); ASSERT_EQ(server_->Start(listenAddr_.c_str(), nullptr), 0); - IOSenderOption_t ioSenderOpt; + IOSenderOption ioSenderOpt; ioSenderOpt.failRequestOpt.chunkserverRPCTimeoutMS = 5000; ioSenderOpt.failRequestOpt.chunkserverOPMaxRetry = 3; ioSenderOpt.failRequestOpt.chunkserverOPRetryIntervalUS = 500; @@ -420,7 +420,7 @@ TEST_F(CopysetClientTest, write_error_test) { brpc::SERVER_DOESNT_OWN_SERVICE), 0); ASSERT_EQ(server_->Start(listenAddr_.c_str(), nullptr), 0); - IOSenderOption_t ioSenderOpt; + IOSenderOption ioSenderOpt; ioSenderOpt.failRequestOpt.chunkserverRPCTimeoutMS = 1000; ioSenderOpt.failRequestOpt.chunkserverOPMaxRetry = 3; ioSenderOpt.failRequestOpt.chunkserverOPRetryIntervalUS = 5000; @@ -428,7 +428,7 @@ TEST_F(CopysetClientTest, write_error_test) { ioSenderOpt.failRequestOpt.chunkserverMaxRetrySleepIntervalUS = 3500000; ioSenderOpt.chunkserverEnableAppliedIndexRead = 1; - RequestScheduleOption_t reqopt; + RequestScheduleOption reqopt; reqopt.ioSenderOpt = ioSenderOpt; CopysetClient copysetClient; @@ -801,7 +801,8 @@ TEST_F(CopysetClientTest, write_error_test) { auto elpased = curve::common::TimeUtility::GetTimeofDayUs() - startTimeUs; // chunkserverOPRetryIntervalUS = 5000 - // 每次redirect睡眠500us,共重试3次,所以总共耗费时间大于1500us + // 每次redirect睡眠500us,共重试2次(chunkserverOPMaxRetry=3,判断时大于等于就返回,所以共只重试了两次) + // 所以总共耗费时间大于1000us ASSERT_GE(elpased, 1000); ASSERT_EQ(CHUNK_OP_STATUS::CHUNK_OP_STATUS_REDIRECTED, reqDone->GetErrorCode()); @@ -888,7 +889,7 @@ TEST_F(CopysetClientTest, write_failed_test) { brpc::SERVER_DOESNT_OWN_SERVICE), 0); ASSERT_EQ(server_->Start(listenAddr_.c_str(), nullptr), 0); - IOSenderOption_t ioSenderOpt; + IOSenderOption ioSenderOpt; ioSenderOpt.failRequestOpt.chunkserverRPCTimeoutMS = 500; ioSenderOpt.failRequestOpt.chunkserverOPMaxRetry = 50; ioSenderOpt.failRequestOpt.chunkserverOPRetryIntervalUS = 5000; @@ -896,7 +897,7 @@ TEST_F(CopysetClientTest, write_failed_test) { ioSenderOpt.failRequestOpt.chunkserverMaxRetrySleepIntervalUS = 100000; ioSenderOpt.chunkserverEnableAppliedIndexRead = 1; - RequestScheduleOption_t reqopt; + RequestScheduleOption reqopt; reqopt.ioSenderOpt = ioSenderOpt; CopysetClient copysetClient; @@ -1026,7 +1027,7 @@ TEST_F(CopysetClientTest, read_failed_test) { brpc::SERVER_DOESNT_OWN_SERVICE), 0); ASSERT_EQ(server_->Start(listenAddr_.c_str(), nullptr), 0); - IOSenderOption_t ioSenderOpt; + IOSenderOption ioSenderOpt; ioSenderOpt.failRequestOpt.chunkserverRPCTimeoutMS = 500; ioSenderOpt.failRequestOpt.chunkserverOPMaxRetry = 50; ioSenderOpt.failRequestOpt.chunkserverOPRetryIntervalUS = 5000; @@ -1034,7 +1035,7 @@ TEST_F(CopysetClientTest, read_failed_test) { ioSenderOpt.failRequestOpt.chunkserverMaxRetrySleepIntervalUS = 100000; ioSenderOpt.chunkserverEnableAppliedIndexRead = 1; - RequestScheduleOption_t reqopt; + RequestScheduleOption reqopt; reqopt.ioSenderOpt = ioSenderOpt; CopysetClient copysetClient; @@ -1165,7 +1166,7 @@ TEST_F(CopysetClientTest, read_error_test) { brpc::SERVER_DOESNT_OWN_SERVICE), 0); ASSERT_EQ(server_->Start(listenAddr_.c_str(), nullptr), 0); - IOSenderOption_t ioSenderOpt; + IOSenderOption ioSenderOpt; ioSenderOpt.failRequestOpt.chunkserverRPCTimeoutMS = 1000; ioSenderOpt.failRequestOpt.chunkserverOPMaxRetry = 3; ioSenderOpt.failRequestOpt.chunkserverOPRetryIntervalUS = 500; @@ -1173,7 +1174,7 @@ TEST_F(CopysetClientTest, read_error_test) { ioSenderOpt.failRequestOpt.chunkserverMaxRetrySleepIntervalUS = 3500000; ioSenderOpt.chunkserverEnableAppliedIndexRead = 1; - RequestScheduleOption_t reqopt; + RequestScheduleOption reqopt; reqopt.ioSenderOpt = ioSenderOpt; CopysetClient copysetClient; @@ -1666,7 +1667,7 @@ TEST_F(CopysetClientTest, read_snapshot_error_test) { brpc::SERVER_DOESNT_OWN_SERVICE), 0); ASSERT_EQ(server_->Start(listenAddr_.c_str(), nullptr), 0); - IOSenderOption_t ioSenderOpt; + IOSenderOption ioSenderOpt; ioSenderOpt.failRequestOpt.chunkserverRPCTimeoutMS = 5000; ioSenderOpt.failRequestOpt.chunkserverOPMaxRetry = 3; ioSenderOpt.failRequestOpt.chunkserverOPRetryIntervalUS = 500; @@ -2091,7 +2092,7 @@ TEST_F(CopysetClientTest, delete_snapshot_error_test) { brpc::SERVER_DOESNT_OWN_SERVICE), 0); ASSERT_EQ(server_->Start(listenAddr_.c_str(), nullptr), 0); - IOSenderOption_t ioSenderOpt; + IOSenderOption ioSenderOpt; ioSenderOpt.failRequestOpt.chunkserverRPCTimeoutMS = 5000; ioSenderOpt.failRequestOpt.chunkserverOPMaxRetry = 3; ioSenderOpt.failRequestOpt.chunkserverOPRetryIntervalUS = 500; @@ -2475,7 +2476,7 @@ TEST_F(CopysetClientTest, create_s3_clone_error_test) { brpc::SERVER_DOESNT_OWN_SERVICE), 0); ASSERT_EQ(server_->Start(listenAddr_.c_str(), nullptr), 0); - IOSenderOption_t ioSenderOpt; + IOSenderOption ioSenderOpt; ioSenderOpt.failRequestOpt.chunkserverRPCTimeoutMS = 5000; ioSenderOpt.failRequestOpt.chunkserverOPMaxRetry = 3; ioSenderOpt.failRequestOpt.chunkserverOPRetryIntervalUS = 500; @@ -2844,7 +2845,7 @@ TEST_F(CopysetClientTest, recover_chunk_error_test) { brpc::SERVER_DOESNT_OWN_SERVICE), 0); ASSERT_EQ(server_->Start(listenAddr_.c_str(), nullptr), 0); - IOSenderOption_t ioSenderOpt; + IOSenderOption ioSenderOpt; ioSenderOpt.failRequestOpt.chunkserverRPCTimeoutMS = 5000; ioSenderOpt.failRequestOpt.chunkserverOPMaxRetry = 3; ioSenderOpt.failRequestOpt.chunkserverOPRetryIntervalUS = 500; @@ -3201,7 +3202,7 @@ TEST_F(CopysetClientTest, get_chunk_info_test) { brpc::SERVER_DOESNT_OWN_SERVICE), 0); ASSERT_EQ(server_->Start(listenAddr_.c_str(), nullptr), 0); - IOSenderOption_t ioSenderOpt; + IOSenderOption ioSenderOpt; ioSenderOpt.failRequestOpt.chunkserverRPCTimeoutMS = 5000; ioSenderOpt.failRequestOpt.chunkserverOPMaxRetry = 3; ioSenderOpt.failRequestOpt.chunkserverOPRetryIntervalUS = 500; @@ -3647,7 +3648,7 @@ TEST_F(CopysetClientTest, retry_rpc_sleep_test) { const uint64_t sleepUsBeforeRetry = 5 * 1000 * 1000; - IOSenderOption_t ioSenderOpt; + IOSenderOption ioSenderOpt; ioSenderOpt.failRequestOpt.chunkserverRPCTimeoutMS = 1000; ioSenderOpt.failRequestOpt.chunkserverOPMaxRetry = 3; ioSenderOpt.failRequestOpt.chunkserverOPRetryIntervalUS = @@ -3656,7 +3657,7 @@ TEST_F(CopysetClientTest, retry_rpc_sleep_test) { ioSenderOpt.failRequestOpt.chunkserverMaxRetrySleepIntervalUS = 3500000; ioSenderOpt.chunkserverEnableAppliedIndexRead = 1; - RequestScheduleOption_t reqopt; + RequestScheduleOption reqopt; reqopt.ioSenderOpt = ioSenderOpt; CopysetClient copysetClient; diff --git a/test/client/fake/client_workflow_test4snap.cpp b/test/client/fake/client_workflow_test4snap.cpp index 85b10150e2..515b9ddc4d 100644 --- a/test/client/fake/client_workflow_test4snap.cpp +++ b/test/client/fake/client_workflow_test4snap.cpp @@ -65,7 +65,7 @@ using curve::client::ChunkID; using curve::client::LogicPoolID; using curve::client::CopysetID; using curve::client::ChunkIDInfo; -using curve::client::CopysetInfo_t; +using curve::client::CopysetPeerInfo; using curve::client::MetaCache; using curve::client::LogicalPoolCopysetIDInfo; @@ -88,9 +88,9 @@ int main(int argc, char ** argv) { } } - ClientConfigOption_t opt; + ClientConfigOption opt; opt.metaServerOpt.chunkserverRPCTimeoutMS = 500; - opt.metaServerOpt.metaaddrvec.push_back("127.0.0.1:6666"); + opt.metaServerOpt.mdsAddrs.push_back("127.0.0.1:6666"); opt.ioOpt.reqSchdulerOpt.scheduleQueueCapacity = 4096; opt.ioOpt.reqSchdulerOpt.scheduleThreadpoolSize = 2; opt.ioOpt.ioSenderOpt.failRequestOpt.chunkserverOPMaxRetry = 3; diff --git a/test/client/fake/fakeMDS.h b/test/client/fake/fakeMDS.h index b96f377a63..f957138e1f 100644 --- a/test/client/fake/fakeMDS.h +++ b/test/client/fake/fakeMDS.h @@ -686,7 +686,7 @@ class FakeMDSCurveFSService : public curve::mds::CurveFSService { const std::string& filename, const std::string& owner, uint64_t date) { - if (owner == kRootUserName) { + if (owner == curve::client::kRootUserName) { std::string str2sig = Authenticator::GetString2Signature(date, owner); // NOLINT std::string sigtest = Authenticator::CalcString2Signature(str2sig, "123"); // NOLINT ASSERT_STREQ(sigtest.c_str(), signature.c_str()); @@ -1125,4 +1125,3 @@ class FakeMDS { }; #endif // TEST_CLIENT_FAKE_FAKEMDS_H_ - diff --git a/test/client/iotracker_splitor_unittest.cpp b/test/client/iotracker_splitor_unittest.cpp index 3250e9d288..a976396020 100644 --- a/test/client/iotracker_splitor_unittest.cpp +++ b/test/client/iotracker_splitor_unittest.cpp @@ -81,7 +81,7 @@ class IOTrackerSplitorTest : public ::testing::Test { public: void SetUp() { fiu_init(0); - fopt.metaServerOpt.metaaddrvec.push_back("127.0.0.1:9104"); + fopt.metaServerOpt.mdsAddrs.push_back("127.0.0.1:9104"); fopt.metaServerOpt.mdsRPCTimeoutMs = 500; fopt.metaServerOpt.mdsRPCRetryIntervalUS = 50000; fopt.loginfo.logLevel = 0; @@ -248,7 +248,7 @@ class IOTrackerSplitorTest : public ::testing::Test { ++count; } - std::vector cpinfoVec; + std::vector cpinfoVec; mdsclient_.GetServerList(lpcsIDInfo.lpid, lpcsIDInfo.cpidVec, &cpinfoVec); @@ -259,7 +259,7 @@ class IOTrackerSplitorTest : public ::testing::Test { UserInfo_t userinfo; MDSClient mdsclient_; - FileServiceOption_t fopt; + FileServiceOption fopt; curve::client::ClientConfig cc; FileInstance* fileinstance_; brpc::Server server; diff --git a/test/client/libcurve_interface_unittest.cpp b/test/client/libcurve_interface_unittest.cpp index a830eb1113..7c1c9d442a 100644 --- a/test/client/libcurve_interface_unittest.cpp +++ b/test/client/libcurve_interface_unittest.cpp @@ -339,7 +339,7 @@ TEST(TestLibcurveInterface, ChunkserverUnstableTest) { UserInfo_t userinfo; MDSClient mdsclient_; - FileServiceOption_t fopt; + FileServiceOption fopt; FileInstance fileinstance_; FLAGS_chunkserver_list = @@ -347,7 +347,7 @@ TEST(TestLibcurveInterface, ChunkserverUnstableTest) { userinfo.owner = "userinfo"; userinfo.password = "12345"; - fopt.metaServerOpt.metaaddrvec.push_back("127.0.0.1:9104"); + fopt.metaServerOpt.mdsAddrs.push_back("127.0.0.1:9104"); fopt.metaServerOpt.chunkserverRPCTimeoutMS = 500; fopt.loginfo.logLevel = 0; fopt.ioOpt.ioSplitOpt.fileIOSplitMaxSizeKB = 64; @@ -410,7 +410,7 @@ TEST(TestLibcurveInterface, ChunkserverUnstableTest) { MetaCacheErrorType rc = mc->GetChunkInfoByIndex(0, &chunkinfo1); ASSERT_EQ(rc, MetaCacheErrorType::OK); for (int i = 0; i < FLAGS_copyset_num; i++) { - CopysetInfo_t ci = mc->GetCopysetinfo(FLAGS_logic_pool_id, i); + CopysetPeerInfo ci = mc->GetCopysetinfo(FLAGS_logic_pool_id, i); if (i == chunkinfo1.cpid_) { ASSERT_NE(-1, ci.GetCurrentLeaderIndex()); ASSERT_FALSE(ci.LeaderMayChange()); @@ -435,7 +435,7 @@ TEST(TestLibcurveInterface, ChunkserverUnstableTest) { ASSERT_EQ(rc, MetaCacheErrorType::OK); ASSERT_NE(chunkinfo2.cpid_, chunkinfo1.cpid_); for (int i = 0; i < FLAGS_copyset_num; i++) { - CopysetInfo_t ci = mc->GetCopysetinfo(FLAGS_logic_pool_id, i); + CopysetPeerInfo ci = mc->GetCopysetinfo(FLAGS_logic_pool_id, i); if (i == chunkinfo1.cpid_ || i == chunkinfo2.cpid_) { ASSERT_NE(-1, ci.GetCurrentLeaderIndex()); // 这两个leader为该chunkserver的copyset的LeaderMayChange置位 @@ -453,7 +453,7 @@ TEST(TestLibcurveInterface, ChunkserverUnstableTest) { ASSERT_EQ(8192, fileinstance_.Write(buffer, 1 * chunk_size, length)); ASSERT_EQ(8192, fileinstance_.Read(buffer, 1 * chunk_size, length)); for (int i = 0; i < FLAGS_copyset_num; i++) { - CopysetInfo_t ci = mc->GetCopysetinfo(FLAGS_logic_pool_id, i); + CopysetPeerInfo ci = mc->GetCopysetinfo(FLAGS_logic_pool_id, i); if (i == chunkinfo2.cpid_) { ASSERT_NE(-1, ci.GetCurrentLeaderIndex()); // copyset2的LeaderMayChange置位 @@ -484,7 +484,7 @@ TEST(TestLibcurveInterface, ChunkserverUnstableTest) { ASSERT_EQ(1, cliservice->GetInvokeTimes()); // 这个时候 for (int i = 0; i < FLAGS_copyset_num; i++) { - CopysetInfo_t ci = mc->GetCopysetinfo(FLAGS_logic_pool_id, i); + CopysetPeerInfo ci = mc->GetCopysetinfo(FLAGS_logic_pool_id, i); if (i == chunkinfo2.cpid_) { ASSERT_NE(-1, ci.GetCurrentLeaderIndex()); // copyset2的LeaderMayChange置位 @@ -503,23 +503,23 @@ TEST(TestLibcurveInterface, ChunkserverUnstableTest) { // 验证copyset id信息更新 // copyset id = 888, chunkserver id = 100 101 102 // copyset id = 999, chunkserver id = 102 103 104 - CopysetInfo_t csinfo1; + CopysetPeerInfo csinfo1; ChunkServerAddr addr; csinfo1.cpid_ = 888; - curve::client::CopysetPeerInfo_t peer1(100, addr); + curve::client::CopysetPeerInfo peer1(100, addr); csinfo1.csinfos_.push_back(peer1); - curve::client::CopysetPeerInfo_t peer2(101, addr); + curve::client::CopysetPeerInfo peer2(101, addr); csinfo1.csinfos_.push_back(peer2); - curve::client::CopysetPeerInfo_t peer3(102, addr); + curve::client::CopysetPeerInfo peer3(102, addr); csinfo1.csinfos_.push_back(peer3); - CopysetInfo_t csinfo2; + CopysetPeerInfo csinfo2; csinfo2.cpid_ = 999; - curve::client::CopysetPeerInfo_t peer4(102, addr); + curve::client::CopysetPeerInfo peer4(102, addr); csinfo2.csinfos_.push_back(peer4); - curve::client::CopysetPeerInfo_t peer5(103, addr); + curve::client::CopysetPeerInfo peer5(103, addr); csinfo2.csinfos_.push_back(peer5); - curve::client::CopysetPeerInfo_t peer6(104, addr); + curve::client::CopysetPeerInfo peer6(104, addr); csinfo2.csinfos_.push_back(peer6); mc->UpdateCopysetInfo(FLAGS_logic_pool_id, 888, csinfo1); @@ -548,13 +548,13 @@ TEST(TestLibcurveInterface, ChunkserverUnstableTest) { ASSERT_FALSE(mc->CopysetIDInfoIn(101, FLAGS_logic_pool_id, 999)); - CopysetInfo_t csinfo3; + CopysetPeerInfo csinfo3; csinfo3.cpid_ = 999; - curve::client::CopysetPeerInfo_t peer7(100, addr); + curve::client::CopysetPeerInfo peer7(100, addr); csinfo3.csinfos_.push_back(peer7); - curve::client::CopysetPeerInfo_t peer8(101, addr); + curve::client::CopysetPeerInfo peer8(101, addr); csinfo3.csinfos_.push_back(peer8); - curve::client::CopysetPeerInfo_t peer9(103, addr); + curve::client::CopysetPeerInfo peer9(103, addr); csinfo3.csinfos_.push_back(peer9); // 更新copyset信息,chunkserver 104的信息被清除 @@ -653,7 +653,7 @@ TEST(TestLibcurveInterface, UnstableChunkserverTest) { UserInfo_t userinfo; MDSClient mdsclient_; - FileServiceOption_t fopt; + FileServiceOption fopt; FileInstance fileinstance_; FLAGS_chunkserver_list = @@ -661,7 +661,7 @@ TEST(TestLibcurveInterface, UnstableChunkserverTest) { userinfo.owner = "userinfo"; userinfo.password = "UnstableChunkserverTest"; - fopt.metaServerOpt.metaaddrvec.push_back("127.0.0.1:9104"); + fopt.metaServerOpt.mdsAddrs.push_back("127.0.0.1:9104"); fopt.metaServerOpt.mdsRPCTimeoutMs = 500; fopt.loginfo.logLevel = 0; fopt.ioOpt.ioSplitOpt.fileIOSplitMaxSizeKB = 64; @@ -675,7 +675,7 @@ TEST(TestLibcurveInterface, UnstableChunkserverTest) { fopt.ioOpt.reqSchdulerOpt.scheduleThreadpoolSize = 2; fopt.ioOpt.reqSchdulerOpt.ioSenderOpt = fopt.ioOpt.ioSenderOpt; fopt.leaseOpt.mdsRefreshTimesPerLease = 4; - fopt.ioOpt.reqSchdulerOpt.ioSenderOpt.failRequestOpt.chunkserverUnstableOption.maxStableChunkServerTimeoutTimes = 10; // NOLINT + fopt.ioOpt.metaCacheOpt.chunkserverUnstableOption.maxStableChunkServerTimeoutTimes = 10; // NOLINT LOG(INFO) << "fopt size " << sizeof(fopt); // curve::client::ClientClosure::SetFailureRequestOption( @@ -729,7 +729,7 @@ TEST(TestLibcurveInterface, UnstableChunkserverTest) { MetaCacheErrorType rc = mc->GetChunkInfoByIndex(0, &chunkinfo1); ASSERT_EQ(rc, MetaCacheErrorType::OK); for (int i = 0; i < FLAGS_copyset_num; i++) { - CopysetInfo_t ci = mc->GetCopysetinfo(FLAGS_logic_pool_id, i); + CopysetInfo ci = mc->GetCopysetinfo(FLAGS_logic_pool_id, i); if (i == chunkinfo1.cpid_) { ASSERT_NE(-1, ci.GetCurrentLeaderIndex()); ASSERT_FALSE(ci.LeaderMayChange()); @@ -759,7 +759,7 @@ TEST(TestLibcurveInterface, UnstableChunkserverTest) { ASSERT_EQ(rc, MetaCacheErrorType::OK); ASSERT_NE(chunkinfo2.cpid_, chunkinfo1.cpid_); for (int i = 0; i < FLAGS_copyset_num; ++i) { - CopysetInfo_t ci = mc->GetCopysetinfo(FLAGS_logic_pool_id, i); + CopysetInfo ci = mc->GetCopysetinfo(FLAGS_logic_pool_id, i); if (i == chunkinfo1.cpid_ || i == chunkinfo2.cpid_) { ASSERT_NE(-1, ci.GetCurrentLeaderIndex()); ASSERT_TRUE(ci.LeaderMayChange()); @@ -789,7 +789,7 @@ TEST(TestLibcurveInterface, UnstableChunkserverTest) { ASSERT_EQ(8192, fileinstance_.Write(buffer, 1 * chunk_size, length)); ASSERT_EQ(8192, fileinstance_.Read(buffer, 1 * chunk_size, length)); for (int i = 0; i < FLAGS_copyset_num; ++i) { - CopysetInfo_t ci = mc->GetCopysetinfo(FLAGS_logic_pool_id, i); + CopysetInfo ci = mc->GetCopysetinfo(FLAGS_logic_pool_id, i); if (i == chunkinfo2.cpid_) { ASSERT_NE(-1, ci.GetCurrentLeaderIndex()); ASSERT_FALSE(ci.LeaderMayChange()); @@ -817,7 +817,7 @@ TEST(TestLibcurveInterface, UnstableChunkserverTest) { ASSERT_EQ(8192, fileinstance_.Read(buffer, 1 * chunk_size, length)); for (int i = 0; i < FLAGS_copyset_num; ++i) { - CopysetInfo_t ci = mc->GetCopysetinfo(FLAGS_logic_pool_id, i); + CopysetInfo ci = mc->GetCopysetinfo(FLAGS_logic_pool_id, i); if (i == chunkinfo2.cpid_) { ASSERT_NE(-1, ci.GetCurrentLeaderIndex()); ASSERT_FALSE(ci.LeaderMayChange()); @@ -841,7 +841,7 @@ TEST(TestLibcurveInterface, ResumeTimeoutBackoff) { UserInfo_t userinfo; MDSClient mdsclient_; - FileServiceOption_t fopt; + FileServiceOption fopt; FileInstance fileinstance_; FLAGS_chunkserver_list = @@ -849,7 +849,7 @@ TEST(TestLibcurveInterface, ResumeTimeoutBackoff) { userinfo.owner = "userinfo"; userinfo.password = "ResumeTimeoutBackoff"; - fopt.metaServerOpt.metaaddrvec.push_back("127.0.0.1:9104"); + fopt.metaServerOpt.mdsAddrs.push_back("127.0.0.1:9104"); fopt.metaServerOpt.mdsRPCTimeoutMs = 500; fopt.loginfo.logLevel = 0; fopt.ioOpt.ioSplitOpt.fileIOSplitMaxSizeKB = 64; @@ -864,7 +864,7 @@ TEST(TestLibcurveInterface, ResumeTimeoutBackoff) { fopt.ioOpt.reqSchdulerOpt.scheduleThreadpoolSize = 2; fopt.ioOpt.reqSchdulerOpt.ioSenderOpt = fopt.ioOpt.ioSenderOpt; fopt.leaseOpt.mdsRefreshTimesPerLease = 4; - fopt.ioOpt.reqSchdulerOpt.ioSenderOpt.failRequestOpt.chunkserverUnstableOption.maxStableChunkServerTimeoutTimes = 10; // NOLINT + fopt.ioOpt.metaCacheOpt.chunkserverUnstableOption.maxStableChunkServerTimeoutTimes = 10; // NOLINT mdsclient_.Initialize(fopt.metaServerOpt); fileinstance_.Initialize( @@ -913,7 +913,7 @@ TEST(TestLibcurveInterface, ResumeTimeoutBackoff) { MetaCacheErrorType rc = mc->GetChunkInfoByIndex(0, &chunkinfo1); ASSERT_EQ(rc, MetaCacheErrorType::OK); for (int i = 0; i < FLAGS_copyset_num; i++) { - CopysetInfo_t ci = mc->GetCopysetinfo(FLAGS_logic_pool_id, i); + CopysetInfo ci = mc->GetCopysetinfo(FLAGS_logic_pool_id, i); if (i == chunkinfo1.cpid_) { ASSERT_NE(-1, ci.GetCurrentLeaderIndex()); ASSERT_FALSE(ci.LeaderMayChange()); diff --git a/test/client/mds_failover_test.cpp b/test/client/mds_failover_test.cpp index 62c61cacbd..00b9886066 100644 --- a/test/client/mds_failover_test.cpp +++ b/test/client/mds_failover_test.cpp @@ -47,7 +47,8 @@ #include "test/integration/cluster_common/cluster.h" #include "test/util/config_generator.h" -using curve::client::MDSClient; +namespace curve { +namespace client { // 测试mds failover切换状态机 TEST(MDSChangeTest, MDSFailoverTest) { @@ -56,10 +57,10 @@ TEST(MDSChangeTest, MDSFailoverTest) { MDSClient::MDSRPCExcutor rpcexcutor; }; - MetaServerOption_t metaopt; - metaopt.metaaddrvec.push_back("127.0.0.1:9903"); - metaopt.metaaddrvec.push_back("127.0.0.1:9904"); - metaopt.metaaddrvec.push_back("127.0.0.1:9905"); + MetaServerOption metaopt; + metaopt.mdsAddrs.push_back("127.0.0.1:9903"); + metaopt.mdsAddrs.push_back("127.0.0.1:9904"); + metaopt.mdsAddrs.push_back("127.0.0.1:9905"); metaopt.mdsRPCTimeoutMs = 1000; metaopt.mdsRPCRetryIntervalUS = 10000; // 10ms @@ -265,6 +266,9 @@ TEST(MDSChangeTest, MDSFailoverTest) { ASSERT_LE(calledTimes, 510); } +} // namespace client +} // namespace curve + const std::vector registConfOff { std::string("mds.listen.addr=127.0.0.1:9903,127.0.0.1:9904,127.0.0.1:9905"), std::string("rpcRetryTimes=3"), diff --git a/test/client/request_scheduler_test.cpp b/test/client/request_scheduler_test.cpp index 84799ddbc9..65b8f6c0e2 100644 --- a/test/client/request_scheduler_test.cpp +++ b/test/client/request_scheduler_test.cpp @@ -39,7 +39,7 @@ namespace client { using ::testing::AnyNumber; TEST(RequestSchedulerTest, fake_server_test) { - RequestScheduleOption_t opt; + RequestScheduleOption opt; opt.scheduleQueueCapacity = 4096; opt.scheduleThreadpoolSize = 2; opt.ioSenderOpt.failRequestOpt.chunkserverRPCTimeoutMS = 200; diff --git a/test/client/request_sender_manager_test.cpp b/test/client/request_sender_manager_test.cpp index 07d94ed24f..454b845d44 100644 --- a/test/client/request_sender_manager_test.cpp +++ b/test/client/request_sender_manager_test.cpp @@ -29,7 +29,7 @@ namespace curve { namespace client { TEST(RequestSenderManagerTest, basic_test) { - IOSenderOption_t ioSenderOpt; + IOSenderOption ioSenderOpt; ioSenderOpt.failRequestOpt.chunkserverOPMaxRetry = 3; ioSenderOpt.failRequestOpt.chunkserverOPRetryIntervalUS = 500; ioSenderOpt.chunkserverEnableAppliedIndexRead = 1; @@ -50,7 +50,7 @@ TEST(RequestSenderManagerTest, basic_test) { } TEST(RequestSenderManagerTest, fail_test) { - IOSenderOption_t ioSenderOpt; + IOSenderOption ioSenderOpt; ioSenderOpt.failRequestOpt.chunkserverOPMaxRetry = 3; ioSenderOpt.failRequestOpt.chunkserverOPRetryIntervalUS = 500; ioSenderOpt.chunkserverEnableAppliedIndexRead = 1; diff --git a/test/client/snapshot_service_unittest.cpp b/test/client/snapshot_service_unittest.cpp index d97c5d3dd6..cfc3ad4da2 100644 --- a/test/client/snapshot_service_unittest.cpp +++ b/test/client/snapshot_service_unittest.cpp @@ -43,19 +43,8 @@ extern std::string mdsMetaServerAddr; -using curve::client::UserInfo_t; -using curve::client::ChunkIDInfo; -using curve::client::SegmentInfo; -using curve::client::ChunkInfoDetail; -using curve::client::SnapshotClient; -using curve::client::ChunkID; -using curve::client::LogicPoolID; -using curve::client::CopysetID; -using curve::client::ChunkIDInfo_t; -using curve::client::CopysetInfo_t; -using curve::client::MetaCache; -using curve::client::IOManager4Chunk; -using curve::client::LogicalPoolCopysetIDInfo; +namespace curve { +namespace client { class SnapCloneClosureTest : public curve::client::SnapCloneClosure { public: @@ -63,10 +52,10 @@ class SnapCloneClosureTest : public curve::client::SnapCloneClosure { }; TEST(SnapInstance, SnapShotTest) { - ClientConfigOption_t opt; + ClientConfigOption opt; opt.metaServerOpt.mdsMaxRetryMS = 1000; opt.metaServerOpt.mdsRPCTimeoutMs = 500; - opt.metaServerOpt.metaaddrvec.push_back("127.0.0.1:9103"); + opt.metaServerOpt.mdsAddrs.push_back("127.0.0.1:9103"); opt.ioOpt.reqSchdulerOpt.scheduleQueueCapacity = 4096; opt.ioOpt.reqSchdulerOpt.scheduleThreadpoolSize = 2; opt.ioOpt.ioSenderOpt.failRequestOpt.chunkserverOPMaxRetry = 3; @@ -504,9 +493,9 @@ TEST(SnapInstance, SnapShotTest) { } TEST(SnapInstance, ReadChunkSnapshotTest) { - ClientConfigOption_t opt; + ClientConfigOption opt; opt.metaServerOpt.mdsMaxRetryMS = 1000; - opt.metaServerOpt.metaaddrvec.push_back("127.0.0.1:9103"); + opt.metaServerOpt.mdsAddrs.push_back("127.0.0.1:9103"); opt.ioOpt.reqSchdulerOpt.scheduleQueueCapacity = 4096; opt.ioOpt.reqSchdulerOpt.scheduleThreadpoolSize = 2; opt.ioOpt.ioSenderOpt.failRequestOpt.chunkserverOPMaxRetry = 3; @@ -528,7 +517,7 @@ TEST(SnapInstance, ReadChunkSnapshotTest) { // fake metacache MetaCache* mc = cl.GetIOManager4Chunk()->GetMetaCache(); ChunkID cid = 1; - CopysetInfo_t cpinfo; + CopysetInfo cpinfo; mc->UpdateChunkInfoByID(cid, ChunkIDInfo(cid, 2, 3)); mc->UpdateCopysetInfo(2, 3, cpinfo); @@ -561,9 +550,9 @@ TEST(SnapInstance, ReadChunkSnapshotTest) { } TEST(SnapInstance, DeleteChunkSnapshotTest) { - ClientConfigOption_t opt; + ClientConfigOption opt; opt.metaServerOpt.mdsMaxRetryMS = 1000; - opt.metaServerOpt.metaaddrvec.push_back("127.0.0.1:9103"); + opt.metaServerOpt.mdsAddrs.push_back("127.0.0.1:9103"); opt.ioOpt.reqSchdulerOpt.scheduleQueueCapacity = 4096; opt.ioOpt.reqSchdulerOpt.scheduleThreadpoolSize = 2; opt.ioOpt.ioSenderOpt.failRequestOpt.chunkserverOPMaxRetry = 3; @@ -584,7 +573,7 @@ TEST(SnapInstance, DeleteChunkSnapshotTest) { // fake metacache MetaCache* mc = cl.GetIOManager4Chunk()->GetMetaCache(); ChunkID cid = 1; - CopysetInfo_t cpinfo; + CopysetInfo cpinfo; mc->UpdateChunkInfoByID(cid, ChunkIDInfo(cid, 2, 3)); mc->UpdateCopysetInfo(2, 3, cpinfo); @@ -601,9 +590,9 @@ TEST(SnapInstance, DeleteChunkSnapshotTest) { } TEST(SnapInstance, GetChunkInfoTest) { - ClientConfigOption_t opt; + ClientConfigOption opt; opt.metaServerOpt.mdsMaxRetryMS = 1000; - opt.metaServerOpt.metaaddrvec.push_back("127.0.0.1:9103"); + opt.metaServerOpt.mdsAddrs.push_back("127.0.0.1:9103"); opt.ioOpt.reqSchdulerOpt.scheduleQueueCapacity = 4096; opt.ioOpt.reqSchdulerOpt.scheduleThreadpoolSize = 2; opt.ioOpt.ioSenderOpt.failRequestOpt.chunkserverOPMaxRetry = 3; @@ -622,7 +611,7 @@ TEST(SnapInstance, GetChunkInfoTest) { // fake metacache MetaCache* mc = cl.GetIOManager4Chunk()->GetMetaCache(); ChunkID cid = 1; - CopysetInfo_t cpinfo; + CopysetInfo cpinfo; mc->UpdateChunkInfoByID(cid, ChunkIDInfo(cid, 2, 3)); mc->UpdateCopysetInfo(2, 3, cpinfo); @@ -641,9 +630,9 @@ TEST(SnapInstance, GetChunkInfoTest) { } TEST(SnapInstance, RecoverChunkTest) { - ClientConfigOption_t opt; + ClientConfigOption opt; opt.metaServerOpt.mdsMaxRetryMS = 1000; - opt.metaServerOpt.metaaddrvec.push_back("127.0.0.1:9103"); + opt.metaServerOpt.mdsAddrs.push_back("127.0.0.1:9103"); opt.ioOpt.reqSchdulerOpt.scheduleQueueCapacity = 4096; opt.ioOpt.reqSchdulerOpt.scheduleThreadpoolSize = 2; opt.ioOpt.ioSenderOpt.failRequestOpt.chunkserverOPMaxRetry = 3; @@ -664,7 +653,7 @@ TEST(SnapInstance, RecoverChunkTest) { // fake metacache MetaCache* mc = cl.GetIOManager4Chunk()->GetMetaCache(); ChunkID cid = 1; - CopysetInfo_t cpinfo; + CopysetInfo cpinfo; mc->UpdateChunkInfoByID(cid, ChunkIDInfo(cid, 2, 3)); mc->UpdateCopysetInfo(2, 3, cpinfo); @@ -683,9 +672,9 @@ TEST(SnapInstance, RecoverChunkTest) { } TEST(SnapInstance, CreateCloneChunkTest) { - ClientConfigOption_t opt; + ClientConfigOption opt; opt.metaServerOpt.mdsMaxRetryMS = 1000; - opt.metaServerOpt.metaaddrvec.push_back("127.0.0.1:9103"); + opt.metaServerOpt.mdsAddrs.push_back("127.0.0.1:9103"); opt.ioOpt.reqSchdulerOpt.scheduleQueueCapacity = 4096; opt.ioOpt.reqSchdulerOpt.scheduleThreadpoolSize = 2; opt.ioOpt.ioSenderOpt.failRequestOpt.chunkserverOPMaxRetry = 3; @@ -706,7 +695,7 @@ TEST(SnapInstance, CreateCloneChunkTest) { // fake metacache MetaCache* mc = cl.GetIOManager4Chunk()->GetMetaCache(); ChunkID cid = 1; - CopysetInfo_t cpinfo; + CopysetInfo cpinfo; mc->UpdateChunkInfoByID(cid, ChunkIDInfo(cid, 2, 3)); mc->UpdateCopysetInfo(2, 3, cpinfo); @@ -726,6 +715,8 @@ TEST(SnapInstance, CreateCloneChunkTest) { cl.UnInit(); } +} // namespace client +} // namespace curve std::string mdsMetaServerAddr = "127.0.0.1:9103"; // NOLINT uint32_t segment_size = 1 * 1024 * 1024 * 1024ul; // NOLINT diff --git a/test/common/math_util_test.cpp b/test/common/math_util_test.cpp new file mode 100644 index 0000000000..bb436312f1 --- /dev/null +++ b/test/common/math_util_test.cpp @@ -0,0 +1,51 @@ +/* + * Copyright (c) 2020 NetEase Inc. + * + * Licensed 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. + */ + +/* + * Project: curve + * Created Date: Sun Sep 6 17:13:58 CST 2020 + */ + +#include "src/common/math_util.h" + +#include + +namespace curve { +namespace common { + +TEST(MathUtilTest, CommonTest) { + ASSERT_EQ(0, MaxPowerTimesLessEqualValue(0)); + ASSERT_EQ(0, MaxPowerTimesLessEqualValue(1)); + ASSERT_EQ(2, MaxPowerTimesLessEqualValue(4)); + ASSERT_EQ(1, MaxPowerTimesLessEqualValue(2)); + ASSERT_EQ(1, MaxPowerTimesLessEqualValue(3)); + ASSERT_EQ(2, MaxPowerTimesLessEqualValue(7)); + ASSERT_EQ(3, MaxPowerTimesLessEqualValue(10)); + ASSERT_EQ(3, MaxPowerTimesLessEqualValue(15)); + ASSERT_EQ(5, MaxPowerTimesLessEqualValue(32)); + ASSERT_EQ(5, MaxPowerTimesLessEqualValue(63)); + ASSERT_EQ(6, MaxPowerTimesLessEqualValue(64)); + ASSERT_EQ(7, MaxPowerTimesLessEqualValue(255)); + ASSERT_EQ(8, MaxPowerTimesLessEqualValue(256)); + ASSERT_EQ(8, MaxPowerTimesLessEqualValue(257)); + ASSERT_EQ(10, MaxPowerTimesLessEqualValue(1024)); + ASSERT_EQ(10, MaxPowerTimesLessEqualValue(2047)); + ASSERT_EQ(11, MaxPowerTimesLessEqualValue(2048)); + ASSERT_EQ(11, MaxPowerTimesLessEqualValue(2049)); +} + +} // namespace common +} // namespace curve diff --git a/test/common/task_thread_pool_test.cpp b/test/common/task_thread_pool_test.cpp index bfc86bfd76..0ac05897b6 100644 --- a/test/common/task_thread_pool_test.cpp +++ b/test/common/task_thread_pool_test.cpp @@ -47,33 +47,33 @@ int TestAdd2(int a, double b, CountDownEvent *cond) { TEST(TaskThreadPool, basic) { /* 测试线程池 start 入参 */ { - TaskThreadPool taskThreadPool; + TaskThreadPool<> taskThreadPool; ASSERT_EQ(-1, taskThreadPool.Start(2, 0)); } { - TaskThreadPool taskThreadPool; + TaskThreadPool<> taskThreadPool; ASSERT_EQ(-1, taskThreadPool.Start(2, -4)); } { - TaskThreadPool taskThreadPool; + TaskThreadPool<> taskThreadPool; ASSERT_EQ(-1, taskThreadPool.Start(0, 1)); } { - TaskThreadPool taskThreadPool; + TaskThreadPool<> taskThreadPool; ASSERT_EQ(-1, taskThreadPool.Start(-2, 1)); } { - TaskThreadPool taskThreadPool; + TaskThreadPool<> taskThreadPool; ASSERT_EQ(-1, taskThreadPool.Start(-2, -1)); } { /* 测试不设置,此时为 INT_MAX */ - TaskThreadPool taskThreadPool; + TaskThreadPool<> taskThreadPool; ASSERT_EQ(0, taskThreadPool.Start(4)); ASSERT_EQ(INT_MAX, taskThreadPool.QueueCapacity()); ASSERT_EQ(4, taskThreadPool.ThreadOfNums()); @@ -82,7 +82,7 @@ TEST(TaskThreadPool, basic) { } { - TaskThreadPool taskThreadPool; + TaskThreadPool<> taskThreadPool; ASSERT_EQ(0, taskThreadPool.Start(4, 15)); ASSERT_EQ(15, taskThreadPool.QueueCapacity()); ASSERT_EQ(4, taskThreadPool.ThreadOfNums()); @@ -112,7 +112,7 @@ TEST(TaskThreadPool, basic) { cond.Signal(); }; - TaskThreadPool taskThreadPool; + TaskThreadPool<> taskThreadPool; ASSERT_EQ(0, taskThreadPool.Start(kThreadNums, kQueueCapacity)); ASSERT_EQ(kQueueCapacity, taskThreadPool.QueueCapacity()); ASSERT_EQ(kThreadNums, taskThreadPool.ThreadOfNums()); @@ -162,7 +162,7 @@ TEST(TaskThreadPool, basic) { runTaskCount.fetch_add(1, std::memory_order_acq_rel); }; - TaskThreadPool taskThreadPool; + TaskThreadPool<> taskThreadPool; ASSERT_EQ(0, taskThreadPool.Start(kThreadNums, kQueueCapacity)); ASSERT_EQ(kQueueCapacity, taskThreadPool.QueueCapacity()); ASSERT_EQ(kThreadNums, taskThreadPool.ThreadOfNums()); diff --git a/test/integration/cluster_common/cluster.cpp b/test/integration/cluster_common/cluster.cpp index 954189cef5..80afc1c3e2 100644 --- a/test/integration/cluster_common/cluster.cpp +++ b/test/integration/cluster_common/cluster.cpp @@ -43,7 +43,7 @@ using ::curve::kvstorage::EtcdClientImp; using ::curve::snapshotcloneserver::SnapshotCloneCodec; namespace curve { -int CurveCluster::InitMdsClient(const MetaServerOption_t &op) { +int CurveCluster::InitMdsClient(const curve::client::MetaServerOption &op) { mdsClient_ = std::make_shared(); return mdsClient_->Initialize(op); } diff --git a/test/integration/cluster_common/cluster.h b/test/integration/cluster_common/cluster.h index 151aa36af6..da3d05592a 100644 --- a/test/integration/cluster_common/cluster.h +++ b/test/integration/cluster_common/cluster.h @@ -75,7 +75,7 @@ class CurveCluster { * @param op 参数设置 * @return 0.成功; 非0.失败 */ - int InitMdsClient(const MetaServerOption_t &op); + int InitMdsClient(const curve::client::MetaServerOption &op); /** diff --git a/test/integration/cluster_common/cluster_basic_test.cpp b/test/integration/cluster_common/cluster_basic_test.cpp index 20be4b8787..21d7894898 100644 --- a/test/integration/cluster_common/cluster_basic_test.cpp +++ b/test/integration/cluster_common/cluster_basic_test.cpp @@ -172,9 +172,9 @@ TEST_F(ClusterBasicTest, test_start_stop_module2) { LOG(INFO) << "mds 1 started on 127.0.0.1:3333, pid = " << pid; ASSERT_GT(pid, 0); // 初始化mdsclient - MetaServerOption_t op; + curve::client::MetaServerOption op; op.mdsRPCTimeoutMs = 500; - op.metaaddrvec = std::vector{ "127.0.0.1:3333" }; + op.mdsAddrs = std::vector{ "127.0.0.1:3333" }; ASSERT_EQ(0, curveCluster_->InitMdsClient(op)); // 创建物理池