From 73dc19893a80160fc1f1d9d2368149a33be2dd7c Mon Sep 17 00:00:00 2001 From: wangshaoyi Date: Fri, 7 Jul 2023 19:11:13 +0800 Subject: [PATCH 01/37] define rsync related header file and proto --- CMakeLists.txt | 4 +- include/rsync_client.h | 85 ++++++++++++++++++++++++++++ include/rsync_server.h | 106 +++++++++++++++++++++++++++++++++++ include/throttle.h | 21 +++++++ src/pstd/include/pstd_hash.h | 51 +++++++++++++++++ src/pstd/src/pstd_hash.cc | 51 ----------------- src/rsync_service.proto | 38 +++++++++++++ 7 files changed, 303 insertions(+), 53 deletions(-) create mode 100644 include/rsync_client.h create mode 100644 include/rsync_server.h create mode 100644 include/throttle.h create mode 100644 src/rsync_service.proto diff --git a/CMakeLists.txt b/CMakeLists.txt index f53d8065f..470f7040b 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -728,7 +728,7 @@ set(PIKA_BUILD_VERSION_CC ${CMAKE_BINARY_DIR}/pika_build_version.cc) message("PIKA_BUILD_VERSION_CC : " ${PIKA_BUILD_VERSION_CC}) configure_file(${CMAKE_CURRENT_SOURCE_DIR}/src/build_version.cc.in ${PIKA_BUILD_VERSION_CC} @ONLY) -set(PROTO_FILES ${CMAKE_CURRENT_SOURCE_DIR}/src/pika_inner_message.proto) +set(PROTO_FILES ${CMAKE_CURRENT_SOURCE_DIR}/src/pika_inner_message.proto ${CMAKE_CURRENT_SOURCE_DIR}/src/rsync_service.proto) custom_protobuf_generate_cpp(PROTO_SRCS PROTO_HDRS ${PROTO_FILES}) message("pika PROTO_SRCS = ${PROTO_SRCS}") message("pika PROTO_HDRS = ${PROTO_HDRS}") @@ -800,4 +800,4 @@ add_custom_target(clang-tidy-fix -header-filter='${PROJECT_SOURCE_DIR}(/include|/tools|/src)/.*' -clang-apply-replacements-binary ${CLANG_APPLY_REPLACEMENTS_BIN} -fix -) \ No newline at end of file +) diff --git a/include/rsync_client.h b/include/rsync_client.h new file mode 100644 index 000000000..8a18bcbe8 --- /dev/null +++ b/include/rsync_client.h @@ -0,0 +1,85 @@ +#ifndef RSYNC_CLIENT_H_ +#define RSYNC_CLIENT_H_ +#include + +#include "throttle.h" +#include "pstd/include/env.h" +#include "net/include/net_cli.h" +#include "pstd/include/pstd_hash.h" +#include "net/include/bg_thread.h" +#include "rsync_service.pb.h" + +using namespace pstd; +using namespace net; +using namespace RsyncService; + +namespace rsync { + +class RsyncWriter; +class Session; +class RsyncClient { +public: + RsyncClient(const std::string& dir, int64_t reader_id, int ip_port); + Status Start(); + void Recover(); + Status Copy(); +private: + Status LoadMetaTable(); + Status FlushMetaTable(); + +private: + //一个RsyncClient包含多个后台线程,并行地从master拉取文件 + std::vector> bg_workers_; + + //待拉取的文件集合 + std::set file_set_; + //已经下载完成的文件名与checksum值,用于宕机重启时恢复,减少重复文件下载,周期性flush到磁盘上 + std::map meta_table_; + + int64_t remote_reader_id_; + std::string ip_port_; + std::string dir_; + + std::atomic is_running_; +}; + +class RsyncWriter { +public: + RsyncWriter(const std::string& filepath); + ~RsyncWriter(); + Status Write(uint64_t offset, size_t n, Slice* result); + +private: + std::string filepath_; + std::unique_ptr file_; +}; + +class Session { +public: + Session(const std::string& filepath, std::shared_ptr conn, + std::shared_ptr throttle); + ~Session(); + Status Result(Slice* content) const; + Status CopyFile(); + +private: + Status HandleResponse(); + Status SendRequest(); + +private: + bool stop_; + Status s_; + + std::unique_ptr writer_; + std::string filepath_; + int64_t reader_id_; + size_t offset_; + MD5 md5_; + + std::shared_ptr throttle_; + std::shared_ptr conn_; +}; + +} // end namespace rsync + +#endif \ No newline at end of file diff --git a/include/rsync_server.h b/include/rsync_server.h new file mode 100644 index 000000000..317753f76 --- /dev/null +++ b/include/rsync_server.h @@ -0,0 +1,106 @@ +#ifndef RSYNC_SERVER_H_ +#define RSYNC_SERVER_H_ +#include +#include +#include +#include + +#include "rsync_service.pb.h" +#include "net/include/net_conn.h" +#include "net/include/net_thread.h" +#include "net/include/thread_pool.h" +#include "net/include/pb_conn.h" +#include "net/include/server_thread.h" +#include "net/src/net_multiplexer.h" +#include "pstd/include/env.h" +#include "net/src/holy_thread.h" + +using namespace net; +using namespace RsyncService; +using namespace pstd; + +namespace Rsync { + +struct RsyncServerTaskArg { + std::shared_ptr req; + std::shared_ptr conn; + RsyncServerTaskArg(std::shared_ptr _req, std::shared_ptr _conn) + : req(std::move(_req)), conn(std::move(_conn)) {} +}; +class RsyncReader; +class RsyncServerThread; + +class RsyncServer { +public: + RsyncServer(const std::string& ip_port, void* worker_specific_data, + const std::string& dir); + ~RsyncServer(); +private: + std::string dir_; + std::map > file_map_; + std::unique_ptr work_thread_ = nullptr; + std::unique_ptr rsync_server_thread_ = nullptr; +}; + +class RsyncServerConn : public PbConn { +public: + RsyncServerConn(int connfd, const std::string& ip_port, + Thread* thread, void* worker_specific_data, + NetMultiplexer* mpx); + virtual ~RsyncServerConn() override; + int DealMessage() override; + //处理slave发来的meta请求,arg参数类型为RsyncServerTaskArg, + //请求处理完成之后将序列化好的response通过conn->WriteResp进行发送 + void HandleMetaRsyncRequest(void* arg); + //处理slave发来的file请求,arg参数类型为RsyncServerTaskArg + //请求处理完成之后将序列化好的response通过conn->WriteResp进行发送 + void HandleFileRsyncRequest(void* arg); +}; + +class RsyncServerThread : public HolyThread { +public: + RsyncServerThread(const std::set& ips, int port, int cron_internal); + ~RsyncServerThread(); + +private: + class RsyncServerConnFactory : public ConnFactory { + public: + explicit RsyncServerConnFactory(RsyncServerThread* thread); + + std::shared_ptr NewNetConn(int connfd, const std::string& ip_port, + Thread* thread, void* worker_specific_data, + NetMultiplexer* net) const override; + }; + class RsyncServerHandle : public ServerHandle { + public: + void FdClosedHandle(int fd, const std::string& ip_port) const override; + void FdTimeoutHandle(int fd, const std::string& ip_port); + bool AccessHandle(int fd, std::string& ip); + bool AccessHandle(std::string& ip); + void CronHandle(); + }; +private: + RsyncServerConnFactory conn_factory_; + RsyncServerHandle handle_; +}; + +class RsyncServerConnFactory : public ConnFactory { + public: + virtual std::shared_ptr NewNetConn(int connfd, const std::string& ip_port, Thread* thread, + void* worker_specific_data, + NetMultiplexer* net_epoll) const override; +}; + +class RSyncReader { +public: + RSyncReader(const std::string& filepath); + ~RSyncReader(); + Status Read(uint64_t offset, size_t n, Slice* result); + +private: + std::string filepath_; + std::unique_ptr file_; +}; +} //end namespace rsync + +#endif \ No newline at end of file diff --git a/include/throttle.h b/include/throttle.h new file mode 100644 index 000000000..32b9dddf1 --- /dev/null +++ b/include/throttle.h @@ -0,0 +1,21 @@ +#ifndef THROTTLE_H_ +#define THROTTLE_H_ +#include + +class Throttle { +public: + Throttle(size_t throttle_throughput_bytes, size_t check_cycle); + ~Throttle(); + size_t ThrottledByThroughput(size_t bytes); + void ReturnUnusedThroughput( + size_t acquired, size_t consumed, size_t elaspe_time_us); +private: + std::atomic throttle_throughput_bytes_; + // the num of tasks doing install_snapshot + std::atomic last_throughput_check_time_us_; + std::atomic cur_throughput_bytes_; + // user defined check cycles of throughput per second + size_t check_cycle_; +}; + +#endif \ No newline at end of file diff --git a/src/pstd/include/pstd_hash.h b/src/pstd/include/pstd_hash.h index 7a23a4de1..54c12bc33 100644 --- a/src/pstd/include/pstd_hash.h +++ b/src/pstd/include/pstd_hash.h @@ -83,6 +83,57 @@ namespace pstd { std::string md5(const std::string& str, bool raw = false); std::string sha256(const std::string& input, bool raw = false); +// a small class for calculating MD5 hashes of strings or byte arrays +// it is not meant to be fast or secure +// +// usage: 1) feed it blocks of uchars with update() +// 2) finalize() +// 3) get hexdigest() string +// or +// MD5(std::string).hexdigest() +// +// assumes that char is 8 bit and int is 32 bit +class MD5 { + public: + using size_type = unsigned int; // must be 32bit + + MD5(); + MD5(const std::string& text); + void update(const unsigned char* input, size_type length); + void update(const char* input, size_type length); + MD5& finalize(); + std::string hexdigest() const; + std::string rawdigest() const; + friend std::ostream& operator<<(std::ostream& /*out*/, MD5 md5); + + private: + void init(); + using uint1 = unsigned char; // 8bit + using uint4 = unsigned int; // 32bit + enum { blocksize = 64 }; // VC6 won't eat a const static int here + + void transform(const uint1 block[blocksize]); + static void decode(uint4 output[], const uint1 input[], size_type len); + static void encode(uint1 output[], const uint4 input[], size_type len); + + bool finalized; + uint1 buffer[blocksize]; // bytes that didn't fit in last 64 byte chunk + uint4 count[2]; // 64bit counter for number of bits (lo, hi) + uint4 state[4]; // digest so far + uint1 digest[16]; // the result + + // low level logic operations + static inline uint4 F(uint4 x, uint4 y, uint4 z); + static inline uint4 G(uint4 x, uint4 y, uint4 z); + static inline uint4 H(uint4 x, uint4 y, uint4 z); + static inline uint4 I(uint4 x, uint4 y, uint4 z); + static inline uint4 rotate_left(uint4 x, int n); + static inline void FF(uint4& a, uint4 b, uint4 c, uint4 d, uint4 x, uint4 s, uint4 ac); + static inline void GG(uint4& a, uint4 b, uint4 c, uint4 d, uint4 x, uint4 s, uint4 ac); + static inline void HH(uint4& a, uint4 b, uint4 c, uint4 d, uint4 x, uint4 s, uint4 ac); + static inline void II(uint4& a, uint4 b, uint4 c, uint4 d, uint4 x, uint4 s, uint4 ac); +}; + } // namespace pstd #endif // __PSTD_HASH_H__ diff --git a/src/pstd/src/pstd_hash.cc b/src/pstd/src/pstd_hash.cc index 120da79c0..da5a5b682 100644 --- a/src/pstd/src/pstd_hash.cc +++ b/src/pstd/src/pstd_hash.cc @@ -124,57 +124,6 @@ class SHA256 { ((uint32) * ((str) + 0) << 24); \ } -// a small class for calculating MD5 hashes of strings or byte arrays -// it is not meant to be fast or secure -// -// usage: 1) feed it blocks of uchars with update() -// 2) finalize() -// 3) get hexdigest() string -// or -// MD5(std::string).hexdigest() -// -// assumes that char is 8 bit and int is 32 bit -class MD5 { - public: - using size_type = unsigned int; // must be 32bit - - MD5(); - MD5(const std::string& text); - void update(const unsigned char* input, size_type length); - void update(const char* input, size_type length); - MD5& finalize(); - std::string hexdigest() const; - std::string rawdigest() const; - friend std::ostream& operator<<(std::ostream& /*out*/, MD5 md5); - - private: - void init(); - using uint1 = unsigned char; // 8bit - using uint4 = unsigned int; // 32bit - enum { blocksize = 64 }; // VC6 won't eat a const static int here - - void transform(const uint1 block[blocksize]); - static void decode(uint4 output[], const uint1 input[], size_type len); - static void encode(uint1 output[], const uint4 input[], size_type len); - - bool finalized; - uint1 buffer[blocksize]; // bytes that didn't fit in last 64 byte chunk - uint4 count[2]; // 64bit counter for number of bits (lo, hi) - uint4 state[4]; // digest so far - uint1 digest[16]; // the result - - // low level logic operations - static inline uint4 F(uint4 x, uint4 y, uint4 z); - static inline uint4 G(uint4 x, uint4 y, uint4 z); - static inline uint4 H(uint4 x, uint4 y, uint4 z); - static inline uint4 I(uint4 x, uint4 y, uint4 z); - static inline uint4 rotate_left(uint4 x, int n); - static inline void FF(uint4& a, uint4 b, uint4 c, uint4 d, uint4 x, uint4 s, uint4 ac); - static inline void GG(uint4& a, uint4 b, uint4 c, uint4 d, uint4 x, uint4 s, uint4 ac); - static inline void HH(uint4& a, uint4 b, uint4 c, uint4 d, uint4 x, uint4 s, uint4 ac); - static inline void II(uint4& a, uint4 b, uint4 c, uint4 d, uint4 x, uint4 s, uint4 ac); -}; - const unsigned int SHA256::sha256_k[64] = { // UL = uint32 0x428a2f98, 0x71374491, 0xb5c0fbcf, 0xe9b5dba5, 0x3956c25b, 0x59f111f1, 0x923f82a4, 0xab1c5ed5, 0xd807aa98, 0x12835b01, 0x243185be, 0x550c7dc3, 0x72be5d74, 0x80deb1fe, 0x9bdc06a7, 0xc19bf174, diff --git a/src/rsync_service.proto b/src/rsync_service.proto new file mode 100644 index 000000000..0f7ec683d --- /dev/null +++ b/src/rsync_service.proto @@ -0,0 +1,38 @@ +syntax = "proto2"; +package RsyncService; + +enum Type { + kRsyncMeta = 1; + kRsyncFile = 2; +} + +message MetaResponse { + repeated string filenames = 1; +} + +message FileRequest { + required string filename = 1; + required uint64 count = 2; + required uint64 offset = 3; +} + +message FileResponse { + required int32 eof = 1; + required uint64 count = 2; + required uint64 offset = 3; + required bytes data = 4; + required string checksum = 5; + required string filename = 6; +} + +message RsyncRequest { + required Type type = 1; + required string uuid = 2; + optional FileRequest file_req = 3; +} + +message RsyncResponse { + required Type type = 1; + optional MetaResponse meta_resp = 2; + optional FileResponse file_resp = 3; +} \ No newline at end of file From c93f5d604406ea5b852ca4004f98c457b24a9297 Mon Sep 17 00:00:00 2001 From: chejinge <945997690@qq.com> Date: Wed, 12 Jul 2023 14:05:05 +0800 Subject: [PATCH 02/37] =?UTF-8?q?feat=EF=BC=9Aadd=20throttle=20(#167)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * add_throttle --- include/throttle.h | 32 ++++++++++++++----------- src/throttle.cc | 58 ++++++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 77 insertions(+), 13 deletions(-) create mode 100644 src/throttle.cc diff --git a/include/throttle.h b/include/throttle.h index 32b9dddf1..fb447aff4 100644 --- a/include/throttle.h +++ b/include/throttle.h @@ -1,21 +1,27 @@ #ifndef THROTTLE_H_ #define THROTTLE_H_ #include +#include "pstd/include/pstd_mutex.h" class Throttle { -public: - Throttle(size_t throttle_throughput_bytes, size_t check_cycle); - ~Throttle(); - size_t ThrottledByThroughput(size_t bytes); - void ReturnUnusedThroughput( - size_t acquired, size_t consumed, size_t elaspe_time_us); -private: - std::atomic throttle_throughput_bytes_; - // the num of tasks doing install_snapshot - std::atomic last_throughput_check_time_us_; - std::atomic cur_throughput_bytes_; - // user defined check cycles of throughput per second - size_t check_cycle_; + public: + Throttle(size_t throttle_throughput_bytes, size_t check_cycle); + ~Throttle(); + size_t ThrottledByThroughput(size_t bytes); + void ReturnUnusedThroughput(size_t acquired, size_t consumed, size_t elaspe_time_us); + + private: + std::atomic throttle_throughput_bytes_; + // the num of tasks doing install_snapshot + std::atomic last_throughput_check_time_us_; + std::atomic cur_throughput_bytes_; + // user defined check cycles of throughput per second + size_t check_cycle_; + pstd::Mutex keys_mutex_; + size_t caculate_check_time_us_(int64_t current_time_us, int64_t check_cycle) { + size_t base_aligning_time_us = 1000 * 1000 / check_cycle; + return current_time_us / base_aligning_time_us * base_aligning_time_us; + } }; #endif \ No newline at end of file diff --git a/src/throttle.cc b/src/throttle.cc new file mode 100644 index 000000000..fd6e56914 --- /dev/null +++ b/src/throttle.cc @@ -0,0 +1,58 @@ +// Copyright (c) 2023-present, Qihoo, Inc. All rights reserved. +// This source code is licensed under the BSD-style license found in the +// LICENSE file in the root directory of this source tree. An additional grant +// of patent rights can be found in the PATENTS file in the same directory. + +#include "include/throttle.h" +#include +#include +#include "pstd/include/env.h" + +DEFINE_uint64(raft_minimal_throttle_threshold_mb, 0, "minimal throttle throughput threshold per second"); + +Throttle::Throttle(size_t throttle_throughput_bytes, size_t check_cycle) + : throttle_throughput_bytes_(throttle_throughput_bytes), + last_throughput_check_time_us_(caculate_check_time_us_(pstd::NowMicros(), check_cycle)), + cur_throughput_bytes_(0) {} + +Throttle::~Throttle() {} + +size_t Throttle::ThrottledByThroughput(size_t bytes) { + size_t available_size = bytes; + size_t now = pstd::NowMicros(); + size_t limit_throughput_bytes_s = std::max(static_cast(throttle_throughput_bytes_), + FLAGS_raft_minimal_throttle_threshold_mb * 1024 * 1024); + size_t limit_per_cycle = limit_throughput_bytes_s / check_cycle_; + std::unique_lock lock(keys_mutex_); + if (cur_throughput_bytes_ + bytes > limit_per_cycle) { + // reading another |bytes| excceds the limit + if (now - last_throughput_check_time_us_ <= 1 * 1000 * 1000 / check_cycle_) { + // if a time interval is less than or equal to a cycle, read more data + // to make full use of the throughput of the current cycle. + available_size = limit_per_cycle > cur_throughput_bytes_ ? limit_per_cycle - cur_throughput_bytes_ : 0; + cur_throughput_bytes_ = limit_per_cycle; + } else { + // otherwise, read the data in the next cycle. + available_size = bytes > limit_per_cycle ? limit_per_cycle : bytes; + cur_throughput_bytes_ = available_size; + last_throughput_check_time_us_ = caculate_check_time_us_(now, check_cycle_); + } + } else { + // reading another |bytes| doesn't excced limit (less than or equal to), + // put it in the current cycle + available_size = bytes; + cur_throughput_bytes_ += available_size; + } + keys_mutex_.unlock(); + return available_size; +} + +void Throttle::ReturnUnusedThroughput(size_t acquired, size_t consumed, size_t elaspe_time_us) { + size_t now = pstd::NowMicros(); + std::unique_lock lock(keys_mutex_); + if (now - elaspe_time_us < last_throughput_check_time_us_) { + // Tokens are aqured in last cycle, ignore + return; + } + cur_throughput_bytes_ = std::max(cur_throughput_bytes_ - (acquired - consumed), size_t(0)); +} \ No newline at end of file From 8ba038f63839572335c60ae2729a86bab7f21e33 Mon Sep 17 00:00:00 2001 From: wangshao1 <30471730+wangshao1@users.noreply.github.com> Date: Wed, 12 Jul 2023 14:53:39 +0800 Subject: [PATCH 03/37] feat: implement rsync network tansform (#169) implement rsync network transfer Co-authored-by: wangshaoyi --- include/rsync_client.h | 86 ++++++++-------- include/rsync_client_thread.h | 50 ++++++++++ include/rsync_server.h | 43 +++++--- include/throttle.h | 4 +- src/rsync_client.cc | 178 ++++++++++++++++++++++++++++++++++ src/rsync_client_thread.cc | 38 ++++++++ src/rsync_server.cc | 155 +++++++++++++++++++++++++++++ src/rsync_service.proto | 2 +- 8 files changed, 498 insertions(+), 58 deletions(-) create mode 100644 include/rsync_client_thread.h create mode 100644 src/rsync_client.cc create mode 100644 src/rsync_client_thread.cc create mode 100644 src/rsync_server.cc diff --git a/include/rsync_client.h b/include/rsync_client.h index 8a18bcbe8..e2c540576 100644 --- a/include/rsync_client.h +++ b/include/rsync_client.h @@ -1,12 +1,18 @@ #ifndef RSYNC_CLIENT_H_ #define RSYNC_CLIENT_H_ #include +#include +#include +#include +#include #include "throttle.h" #include "pstd/include/env.h" #include "net/include/net_cli.h" #include "pstd/include/pstd_hash.h" #include "net/include/bg_thread.h" +#include "pstd/include/pstd_status.h" +#include "include/rsync_client_thread.h" #include "rsync_service.pb.h" using namespace pstd; @@ -17,69 +23,67 @@ namespace rsync { class RsyncWriter; class Session; -class RsyncClient { + +class RsyncClient : public net::Thread { public: - RsyncClient(const std::string& dir, int64_t reader_id, int ip_port); + enum State { + IDLE, + RUNNING, + STOP, + }; + enum Type { + kMeta, + kFile, + }; + RsyncClient(const std::string& dir, const std::string& ip, const int port); + + void* ThreadMain() override; + bool Init(); Status Start(); - void Recover(); - Status Copy(); + Status Stop(); + void OnReceive(RsyncResponse* resp); private: + void Recover(); + Status SendRequest(const std::string& filename, size_t offset, Type type); + Status HandleResponse(const std::string& filename, size_t& offset, MD5& md5, RsyncWriter& writer); + Status LoadFile(const std::string& filename); Status LoadMetaTable(); Status FlushMetaTable(); private: - //一个RsyncClient包含多个后台线程,并行地从master拉取文件 - std::vector> bg_workers_; - + //已经下载完成的文件名与checksum值,用于宕机重启时恢复, + //减少重复文件下载,周期性flush到磁盘上 + std::map meta_table_; //待拉取的文件集合 std::set file_set_; - //已经下载完成的文件名与checksum值,用于宕机重启时恢复,减少重复文件下载,周期性flush到磁盘上 - std::map meta_table_; + std::string snapshot_uuid_; - int64_t remote_reader_id_; - std::string ip_port_; std::string dir_; + std::string ip_; + int port_; + + std::unique_ptr client_thread_; + std::atomic state_; - std::atomic is_running_; + std::list resp_list_; + std::condition_variable cond_; + std::mutex mu_; }; +//TODO: jinge class RsyncWriter { public: - RsyncWriter(const std::string& filepath); - ~RsyncWriter(); - Status Write(uint64_t offset, size_t n, Slice* result); + RsyncWriter(const std::string& filepath) {} + ~RsyncWriter() {} + Status Write(uint64_t offset, size_t n, Slice* result) { + return Status::OK(); + } private: std::string filepath_; std::unique_ptr file_; }; -class Session { -public: - Session(const std::string& filepath, std::shared_ptr conn, - std::shared_ptr throttle); - ~Session(); - Status Result(Slice* content) const; - Status CopyFile(); - -private: - Status HandleResponse(); - Status SendRequest(); - -private: - bool stop_; - Status s_; - - std::unique_ptr writer_; - std::string filepath_; - int64_t reader_id_; - size_t offset_; - MD5 md5_; - - std::shared_ptr throttle_; - std::shared_ptr conn_; -}; - } // end namespace rsync #endif \ No newline at end of file diff --git a/include/rsync_client_thread.h b/include/rsync_client_thread.h new file mode 100644 index 000000000..39a36cceb --- /dev/null +++ b/include/rsync_client_thread.h @@ -0,0 +1,50 @@ +#ifndef RSYNC_CLIENT_THREAD_H_ +#define RSYNC_CLIENT_THREAD_H_ + +#include "net/include/client_thread.h" +#include "net/include/net_conn.h" +#include "net/include/pb_conn.h" +#include "rsync_service.pb.h" + +using namespace pstd; +using namespace net; + +namespace rsync { + +class RsyncClientConn : public PbConn { +public: + RsyncClientConn(int fd, const std::string& ip_port, + net::Thread* thread, void* cb_handler, + NetMultiplexer* mpx); + ~RsyncClientConn() override; + int DealMessage() override; + +private: + void* cb_handler_; +}; + +class RsyncClientConnFactory : public ConnFactory { +public: + RsyncClientConnFactory(void* scheduler) : cb_handler_(scheduler) {} + std::shared_ptr NewNetConn(int connfd, const std::string& ip_port, + net::Thread* thread, void* cb_handler, + net::NetMultiplexer* net) const override { + return std::static_pointer_cast( + std::make_shared(connfd, ip_port, thread, cb_handler_, net)); + } +private: + void* cb_handler_; +}; + +class RsyncClientThread : public ClientThread { +public: + RsyncClientThread(int cron_interval, int keepalive_timeout, void* scheduler); + ~RsyncClientThread() override; +private: + RsyncClientConnFactory conn_factory_; + ClientHandle handle_; +}; + +} //end namespace rsync + +#endif \ No newline at end of file diff --git a/include/rsync_server.h b/include/rsync_server.h index 317753f76..0cdbf82c8 100644 --- a/include/rsync_server.h +++ b/include/rsync_server.h @@ -19,7 +19,7 @@ using namespace net; using namespace RsyncService; using namespace pstd; -namespace Rsync { +namespace rsync { struct RsyncServerTaskArg { std::shared_ptr req; @@ -32,10 +32,15 @@ class RsyncServerThread; class RsyncServer { public: - RsyncServer(const std::string& ip_port, void* worker_specific_data, + RsyncServer(const std::string& ip_port, const int port, void* worker_specific_data, const std::string& dir); ~RsyncServer(); + void Schedule(net::TaskFunc func, void* arg); + int Start(); + int Stop(); private: + int port_; + std::string ip_; std::string dir_; std::map > file_map_; std::unique_ptr work_thread_ = nullptr; @@ -51,35 +56,43 @@ class RsyncServerConn : public PbConn { int DealMessage() override; //处理slave发来的meta请求,arg参数类型为RsyncServerTaskArg, //请求处理完成之后将序列化好的response通过conn->WriteResp进行发送 - void HandleMetaRsyncRequest(void* arg); + static void HandleMetaRsyncRequest(void* arg); //处理slave发来的file请求,arg参数类型为RsyncServerTaskArg //请求处理完成之后将序列化好的response通过conn->WriteResp进行发送 - void HandleFileRsyncRequest(void* arg); + static void HandleFileRsyncRequest(void* arg); +private: + void* data_; }; class RsyncServerThread : public HolyThread { public: - RsyncServerThread(const std::set& ips, int port, int cron_internal); + RsyncServerThread(const std::set& ips, int port, int cron_internal, RsyncServer* arg); ~RsyncServerThread(); private: - class RsyncServerConnFactory : public ConnFactory { - public: - explicit RsyncServerConnFactory(RsyncServerThread* thread); + class RsyncServerConnFactory : public ConnFactory { + public: + explicit RsyncServerConnFactory(RsyncServer* sched) : scheduler_(sched) {} + + std::shared_ptr NewNetConn(int connfd, const std::string& ip_port, + Thread* thread, void* worker_specific_data, + NetMultiplexer* net) const override { + return std::static_pointer_cast( + std::make_shared(connfd, ip_port, thread, scheduler_, net)); + } + private: + RsyncServer* scheduler_; - std::shared_ptr NewNetConn(int connfd, const std::string& ip_port, - Thread* thread, void* worker_specific_data, - NetMultiplexer* net) const override; }; class RsyncServerHandle : public ServerHandle { public: void FdClosedHandle(int fd, const std::string& ip_port) const override; - void FdTimeoutHandle(int fd, const std::string& ip_port); - bool AccessHandle(int fd, std::string& ip); - bool AccessHandle(std::string& ip); - void CronHandle(); + void FdTimeoutHandle(int fd, const std::string& ip_port) const override; + bool AccessHandle(int fd, std::string& ip) const override; + void CronHandle() const override; }; private: + void* arg_; RsyncServerConnFactory conn_factory_; RsyncServerHandle handle_; }; diff --git a/include/throttle.h b/include/throttle.h index fb447aff4..c9bc21f8f 100644 --- a/include/throttle.h +++ b/include/throttle.h @@ -3,6 +3,7 @@ #include #include "pstd/include/pstd_mutex.h" +namespace rsync { class Throttle { public: Throttle(size_t throttle_throughput_bytes, size_t check_cycle); @@ -22,6 +23,7 @@ class Throttle { size_t base_aligning_time_us = 1000 * 1000 / check_cycle; return current_time_us / base_aligning_time_us * base_aligning_time_us; } -}; +} +} // end namespace rsync #endif \ No newline at end of file diff --git a/src/rsync_client.cc b/src/rsync_client.cc new file mode 100644 index 000000000..27d3106d9 --- /dev/null +++ b/src/rsync_client.cc @@ -0,0 +1,178 @@ +#include "include/rsync_client.h" +using namespace net; +using namespace pstd; +using namespace RsyncService; +namespace rsync { +RsyncClient::RsyncClient(const std::string& dir, const std::string& ip, const int port) + : dir_ (dir), ip_ (ip), port_(port), state_ (IDLE) { + client_thread_ = std::make_unique(10 * 1000, 60 * 1000, this); +} + +bool RsyncClient::Init() { + if (state_.load(std::memory_order_relaxed) != IDLE) { + Stop(); + Start(); + } + Recover(); + state_ = RUNNING; + return true; +} + +void* RsyncClient::ThreadMain() { + int period = 1000; + Status s = Status::OK(); + while (state_.load(std::memory_order_relaxed) == RUNNING) { + for (const auto& file : file_set_) { + s = LoadFile(file); + if (!s.ok()) { + //TODO: 同步状态下,是保持最大重试次数还是死循环地搞? + LOG(WARNING) << "rsync LoadFile failed"; + return nullptr; + } + if (state_.load(std::memory_order_relaxed) != RUNNING) { + break; + } + if (--period == 0) { + period = 10; + FlushMetaTable(); + } + } + if (meta_table_.size() == file_set_.size()) { + LOG(INFO) << "rsync done..."; + state_.save(std::memory_order_relaxed, STOP); + break; + } + } + return nullptr; +} + +void RsyncClient::OnReceive(RsyncResponse* resp) { + std::unique_lock lock(mu_); + resp_list_.push_back(resp); + cond_.notify_all(); +} + +Status RsyncClient::SendRequest(const std::string& filename, size_t offset, Type type) { + Status s = Status::OK(); + RsyncRequest request; + switch (type) { + case kMeta: { + request.set_type(kRsyncMeta); + break; + } + case kFile: { + request.set_type(kRsyncFile); + FileRequest* file_req = request.mutable_file_req(); + file_req->set_filename(filename); + file_req->set_offset(offset); + file_req->set_count(4096); + break; + } + default: + break; + } + std::string to_send; + request.SerializeToString(&to_send); + s = client_thread_->Write(ip_, port_, to_send); + if (!s.ok()) { + LOG(WARNING) << "send rsync request failed"; + } + + return s; +} + +Status RsyncClient::HandleResponse(const std::string& filename, size_t& offset, MD5& md5, RsyncWriter& writer) { + std::unique_lock lock(mu_); + cond_.wait_for(lock, std::chrono::seconds(3), [this]{return !resp_list_.empty();}); + + Status s = Status::Timeout("rsync timeout", "timeout"); + auto iter = resp_list_.begin(); + while (iter != resp_list_.end()) { + RsyncResponse* response = *iter; + switch (response->type()) { + case kRsyncMeta: { + LOG(INFO) << "receive rsync meta infos, uuid: " << response->meta_resp().uuid() + << "files count: " << response->meta_resp().filenames_size(); + for (int i = 0; i < response->meta_resp().filenames_size(); i++) { + LOG(WARNING) << "filename: " << response->meta_resp().filenames(i); + } + break; + } + case kRsyncFile: { + s = Status::OK(); + //比对filename和offset + LOG(WARNING) << "receive rsync file infos: " + << "filename: " << response->file_resp().filename() + << "offset: " << response->file_resp().offset() + << "count: " << response->file_resp().count(); + md5.update(response->file_resp().data().c_str(), response->file_resp().count()); + break; + } + default: + break; + } + //TODO: 比对正常,退出resp_list + delete response; + break; + } + return s; +} + +Status RsyncClient::LoadFile(const std::string& filename) { + Status s; + int retries = 0; + int max_retries_ = 10; + size_t offset = 0; + Type type = kFile; + MD5 md5; + RsyncWriter writer(dir_ + "/" + filename); + while (retries < max_retries_) { + s = SendRequest(filename, offset, kFile); + if (!s.ok()) { + retries++; + continue; + } + s = HandleResponse(filename, offset, md5, writer); + if (!s.ok()) { + retries++; + break; + } + retries = 0; + } + return s; +} + +Status RsyncClient::Start() { + StartThread(); + client_thread_->StartThread(); + return Status::OK(); +} + +Status RsyncClient::Stop() { + state_ = STOP; + StopThread(); + client_thread_->StopThread(); + JoinThread(); + client_thread_->JoinThread(); + state_ = IDLE; + return Status::OK(); +} + +//TODO: yuecai +void RsyncClient::Recover() { + file_set_.insert("filename"); +} + +//TODO: shaoyi +Status RsyncClient::LoadMetaTable() { + LOG(WARNING) << "LoadMetaTable called"; + return Status::OK(); +} + +//TODO: shaoyi +Status RsyncClient::FlushMetaTable() { + LOG(WARNING) << "FlushMetaTable called"; + return Status::OK(); +} + +} // end namespace rsync \ No newline at end of file diff --git a/src/rsync_client_thread.cc b/src/rsync_client_thread.cc new file mode 100644 index 000000000..f29c904b1 --- /dev/null +++ b/src/rsync_client_thread.cc @@ -0,0 +1,38 @@ +#include "include/rsync_client_thread.h" +#include "include/rsync_client.h" +#include "include/pika_define.h" + +using namespace pstd; +using namespace net; +using namespace RsyncService; + +namespace rsync { +class RsyncClient; +RsyncClientConn::RsyncClientConn(int fd, const std::string& ip_port, + net::Thread* thread, void* worker_specific_data, NetMultiplexer* mpx) + : PbConn(fd, ip_port, thread, mpx), cb_handler_(worker_specific_data) {} + +RsyncClientConn::~RsyncClientConn() {} + +int RsyncClientConn::DealMessage() { + RsyncResponse* response = new RsyncResponse(); + ::google::protobuf::io::ArrayInputStream input(rbuf_ + cur_pos_ - header_len_, header_len_); + ::google::protobuf::io::CodedInputStream decoder(&input); + decoder.SetTotalBytesLimit(PIKA_MAX_CONN_RBUF); + bool success = response->ParseFromCodedStream(&decoder) && decoder.ConsumedEntireMessage(); + if (!success) { + LOG(WARNING) << "ParseFromArray FAILED! " + << " msg_len: " << header_len_; + return -1; + } + RsyncClient* handler = (RsyncClient*)cb_handler_; + handler->OnReceive(response); + return 0; +} + +RsyncClientThread::RsyncClientThread(int cron_interval, int keepalive_timeout, void* scheduler) + : ClientThread(&conn_factory_, cron_interval, keepalive_timeout, &handle_, nullptr), + conn_factory_(scheduler) {} + +RsyncClientThread::~RsyncClientThread() {} +} \ No newline at end of file diff --git a/src/rsync_server.cc b/src/rsync_server.cc new file mode 100644 index 000000000..b117c8417 --- /dev/null +++ b/src/rsync_server.cc @@ -0,0 +1,155 @@ +#include "include/rsync_server.h" +#include + +namespace rsync { + +RsyncServer::RsyncServer(const std::string& ip, const int port, void* worker_specific_data, + const std::string& dir) : dir_(dir), ip_(ip), port_(port) { + work_thread_ = std::make_unique(2, 100000); + //TODO parse parameter + std::set ips = {ip_}; + rsync_server_thread_ = std::make_unique(ips, port, 60 * 1000, this); +} + +RsyncServer::~RsyncServer() { + LOG(INFO) << "Rsync server destroyed"; +} + +void RsyncServer::Schedule(net::TaskFunc func, void* arg) { + work_thread_->Schedule(func, arg); +} + +int RsyncServer::Start() { + int res = rsync_server_thread_->StartThread(); + LOG(WARNING) << "after RsyncServer::Start"; + if (res != net::kSuccess) { + LOG(FATAL) << "Start rsync Server Thread Error: " << res; + } + res = work_thread_->start_thread_pool(); + if (res != net::kSuccess) { + LOG(FATAL) << "Start ThreadPool Error: " << res + << (res == net::kCreateThreadError ? ": create thread error " : ": other error"); + } + LOG(WARNING) << "after start_thread_pool"; + return res; +} + +int RsyncServer::Stop() { + work_thread_->stop_thread_pool(); + rsync_server_thread_->StopThread(); + return 0; +} + +/*-------------------------RsyncServerConn--------------------*/ +RsyncServerConn::RsyncServerConn(int connfd, const std::string& ip_port, + Thread* thread, void* worker_specific_data, + NetMultiplexer* mpx) : PbConn(connfd, ip_port, thread, mpx), data_(worker_specific_data) {} + +RsyncServerConn::~RsyncServerConn() { + LOG(INFO) << "RsyncServerConn destroyed"; +} + +int RsyncServerConn::DealMessage() { + std::shared_ptr req = std::make_shared(); + bool parse_res = req->ParseFromArray(rbuf_ + cur_pos_ - header_len_, header_len_); + LOG(WARNING) << "RsyncServer DealMessage..."; + if (!parse_res) { + LOG(WARNING) << "Pika rsync server connection pb parse error."; + return -1; + } + switch (req->type()) { + case RsyncService::kRsyncMeta: { + auto task_arg = + new RsyncServerTaskArg(req, std::dynamic_pointer_cast(shared_from_this())); + ((RsyncServer*)(data_))->Schedule(&RsyncServerConn::HandleMetaRsyncRequest, task_arg); + break; + } + case RsyncService::kRsyncFile: { + auto task_arg = + new RsyncServerTaskArg(req, std::dynamic_pointer_cast(shared_from_this())); + ((RsyncServer*)(data_))->Schedule(&RsyncServerConn::HandleFileRsyncRequest, task_arg); + break; + } + default: { + LOG(WARNING) << "Invalid RsyncRequest type"; + } + } + return 0; +} + +void RsyncServerConn::HandleMetaRsyncRequest(void* arg) { + std::unique_ptr task_arg(static_cast(arg)); + const std::shared_ptr req = task_arg->req; + std::shared_ptr conn = task_arg->conn; + + RsyncService::RsyncResponse response; + response.set_type(RsyncService::kRsyncMeta); + LOG(INFO) << "Receive RsyncMeta request"; + + //TODO: temporarily mock response + RsyncService::MetaResponse* meta_resp = response.mutable_meta_resp(); + meta_resp->set_uuid("uuid"); + + std::string reply_str; + if (!response.SerializeToString(&reply_str) || (conn->WriteResp(reply_str) != 0)) { + LOG(WARNING) << "Process MetaRsync request serialization failed"; + conn->NotifyClose(); + return; + } + conn->NotifyWrite(); +} + +void RsyncServerConn::HandleFileRsyncRequest(void* arg) { + std::unique_ptr task_arg(static_cast(arg)); + const std::shared_ptr req = task_arg->req; + std::shared_ptr conn = task_arg->conn; + + RsyncService::RsyncResponse response; + response.set_type(RsyncService::kRsyncFile); + LOG(INFO) << "Receive RsyncFile request " << "filename: " << req->file_req().filename() + << " offset: " << req->file_req().offset() + << " count: " << req->file_req().count(); + + //TODO: temporarily mock response + RsyncService::FileResponse* file_resp = response.mutable_file_resp(); + file_resp->set_eof(1); + file_resp->set_count(1024); + file_resp->set_offset(1024); + file_resp->set_data("mocking data"); + file_resp->set_checksum("checksum"); + file_resp->set_filename("filename"); + + std::string reply_str; + if (!response.SerializeToString(&reply_str) || (conn->WriteResp(reply_str) != 0)) { + LOG(WARNING) << "Process FileRsync request serialization failed"; + conn->NotifyClose(); + return; + } + conn->NotifyWrite(); +} + +RsyncServerThread::RsyncServerThread(const std::set& ips, int port, int cron_interval, RsyncServer* arg) + : HolyThread(ips, port, &conn_factory_, cron_interval, &handle_, true), conn_factory_(arg) {} + +RsyncServerThread::~RsyncServerThread() { + LOG(WARNING) << "RsyncServerThread destroyed"; +} + +void RsyncServerThread::RsyncServerHandle::FdClosedHandle(int fd, const std::string& ip_port) const { + LOG(WARNING) << "ip_port: " << ip_port << " connection closed"; +} + +void RsyncServerThread::RsyncServerHandle::FdTimeoutHandle(int fd, const std::string& ip_port) const { + LOG(WARNING) << "ip_port: " << ip_port << " connection timeout"; +} + +bool RsyncServerThread::RsyncServerHandle::AccessHandle(int fd, std::string& ip_port) const { + LOG(WARNING) << "fd: "<< fd << " ip_port: " << ip_port << " connection accepted"; + return true; +} + +void RsyncServerThread::RsyncServerHandle::CronHandle() const { + LOG(WARNING) << "CronHandle called"; +} + +} // end namespace rsync \ No newline at end of file diff --git a/src/rsync_service.proto b/src/rsync_service.proto index 0f7ec683d..baeb52594 100644 --- a/src/rsync_service.proto +++ b/src/rsync_service.proto @@ -8,6 +8,7 @@ enum Type { message MetaResponse { repeated string filenames = 1; + required string uuid = 2; } message FileRequest { @@ -27,7 +28,6 @@ message FileResponse { message RsyncRequest { required Type type = 1; - required string uuid = 2; optional FileRequest file_req = 3; } From dd51cde5803ca87b049d1ec5e22bca6b968c7b10 Mon Sep 17 00:00:00 2001 From: Yuecai Liu <38887641+luky116@users.noreply.github.com> Date: Wed, 12 Jul 2023 14:56:01 +0800 Subject: [PATCH 04/37] fix action (#171) * fix action --- .github/workflows/codeql.yml | 4 ++-- .github/workflows/codis.yml | 4 ++-- .github/workflows/operator.yml | 4 ++-- .github/workflows/pika.yml | 4 ++-- .github/workflows/tools_go.yml | 4 ++-- 5 files changed, 10 insertions(+), 10 deletions(-) diff --git a/.github/workflows/codeql.yml b/.github/workflows/codeql.yml index 23f547db9..dd1b733e6 100644 --- a/.github/workflows/codeql.yml +++ b/.github/workflows/codeql.yml @@ -13,10 +13,10 @@ name: "CodeQL" on: push: - branches: [ "unstable" ] + branches: [ "unstable", "*" ] pull_request: # The branches below must be a subset of the branches above - branches: [ "unstable" ] + branches: [ "unstable", "*" ] schedule: - cron: '25 19 * * 6' diff --git a/.github/workflows/codis.yml b/.github/workflows/codis.yml index b097078bf..917c45d64 100644 --- a/.github/workflows/codis.yml +++ b/.github/workflows/codis.yml @@ -5,9 +5,9 @@ name: Codis on: push: - branches: [ "unstable" ] + branches: [ "unstable", "*"] pull_request: - branches: [ "unstable" ] + branches: [ "unstable", "*"] jobs: diff --git a/.github/workflows/operator.yml b/.github/workflows/operator.yml index 0a2e8eb3d..51c3dbfa9 100644 --- a/.github/workflows/operator.yml +++ b/.github/workflows/operator.yml @@ -5,9 +5,9 @@ name: Operator on: push: - branches: [ "unstable" ] + branches: [ "unstable", "*" ] pull_request: - branches: [ "unstable" ] + branches: [ "unstable", "*" ] jobs: diff --git a/.github/workflows/pika.yml b/.github/workflows/pika.yml index d32c218ad..aef931044 100644 --- a/.github/workflows/pika.yml +++ b/.github/workflows/pika.yml @@ -2,9 +2,9 @@ name: Pika on: push: - branches: [ "unstable" ] + branches: [ "unstable", "*" ] pull_request: - branches: [ "unstable" ] + branches: [ "unstable", "*" ] env: # Customize the CMake build type here (Release, Debug, RelWithDebInfo, etc.) diff --git a/.github/workflows/tools_go.yml b/.github/workflows/tools_go.yml index 82d88e2ec..087ee6c34 100644 --- a/.github/workflows/tools_go.yml +++ b/.github/workflows/tools_go.yml @@ -2,11 +2,11 @@ name: Tools_go_build on: push: - branches: [ "unstable" ] + branches: [ "unstable", "*" ] paths: - 'tools/**' pull_request: - branches: [ "unstable" ] + branches: [ "unstable", "*" ] paths: - 'tools/**' From d23e398a85b1f9774f3d785ddfdbbe0606423f3e Mon Sep 17 00:00:00 2001 From: luky116 Date: Wed, 12 Jul 2023 15:51:31 +0800 Subject: [PATCH 05/37] fix --- include/throttle.h | 6 +++--- src/rsync_client.cc | 2 +- src/throttle.cc | 8 ++++---- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/include/throttle.h b/include/throttle.h index c9bc21f8f..0f61084db 100644 --- a/include/throttle.h +++ b/include/throttle.h @@ -17,13 +17,13 @@ class Throttle { std::atomic last_throughput_check_time_us_; std::atomic cur_throughput_bytes_; // user defined check cycles of throughput per second - size_t check_cycle_; + size_t check_cycle_{}; pstd::Mutex keys_mutex_; size_t caculate_check_time_us_(int64_t current_time_us, int64_t check_cycle) { size_t base_aligning_time_us = 1000 * 1000 / check_cycle; return current_time_us / base_aligning_time_us * base_aligning_time_us; } -} -} // end namespace rsync +}; +} // end namespace rsync #endif \ No newline at end of file diff --git a/src/rsync_client.cc b/src/rsync_client.cc index 27d3106d9..84a2b10c7 100644 --- a/src/rsync_client.cc +++ b/src/rsync_client.cc @@ -39,7 +39,7 @@ void* RsyncClient::ThreadMain() { } if (meta_table_.size() == file_set_.size()) { LOG(INFO) << "rsync done..."; - state_.save(std::memory_order_relaxed, STOP); + state_.store(STOP); break; } } diff --git a/src/throttle.cc b/src/throttle.cc index fd6e56914..9b995de1b 100644 --- a/src/throttle.cc +++ b/src/throttle.cc @@ -10,14 +10,14 @@ DEFINE_uint64(raft_minimal_throttle_threshold_mb, 0, "minimal throttle throughput threshold per second"); -Throttle::Throttle(size_t throttle_throughput_bytes, size_t check_cycle) +rsync::Throttle::Throttle(size_t throttle_throughput_bytes, size_t check_cycle) : throttle_throughput_bytes_(throttle_throughput_bytes), last_throughput_check_time_us_(caculate_check_time_us_(pstd::NowMicros(), check_cycle)), cur_throughput_bytes_(0) {} -Throttle::~Throttle() {} +rsync::Throttle::~Throttle() {} -size_t Throttle::ThrottledByThroughput(size_t bytes) { +size_t rsync::Throttle::ThrottledByThroughput(size_t bytes) { size_t available_size = bytes; size_t now = pstd::NowMicros(); size_t limit_throughput_bytes_s = std::max(static_cast(throttle_throughput_bytes_), @@ -47,7 +47,7 @@ size_t Throttle::ThrottledByThroughput(size_t bytes) { return available_size; } -void Throttle::ReturnUnusedThroughput(size_t acquired, size_t consumed, size_t elaspe_time_us) { +void rsync::Throttle::ReturnUnusedThroughput(size_t acquired, size_t consumed, size_t elaspe_time_us) { size_t now = pstd::NowMicros(); std::unique_lock lock(keys_mutex_); if (now - elaspe_time_us < last_throughput_check_time_us_) { From 7785b429404900a584b9e21540d5fe990508af8b Mon Sep 17 00:00:00 2001 From: Yuecai Liu <38887641+luky116@users.noreply.github.com> Date: Thu, 13 Jul 2023 18:28:48 +0800 Subject: [PATCH 06/37] feat: add load local meta file (#175) * add load meta file --- include/rsync_client.h | 12 ++++++--- include/rsync_server.h | 2 ++ src/rsync_client.cc | 59 ++++++++++++++++++++++++++++++++++++++++-- 3 files changed, 67 insertions(+), 6 deletions(-) diff --git a/include/rsync_client.h b/include/rsync_client.h index e2c540576..da5c80776 100644 --- a/include/rsync_client.h +++ b/include/rsync_client.h @@ -6,19 +6,22 @@ #include #include -#include "throttle.h" -#include "pstd/include/env.h" +#include "include/rsync_client_thread.h" +#include "net/include/bg_thread.h" #include "net/include/net_cli.h" +#include "pstd/include/env.h" #include "pstd/include/pstd_hash.h" -#include "net/include/bg_thread.h" #include "pstd/include/pstd_status.h" -#include "include/rsync_client_thread.h" #include "rsync_service.pb.h" +#include "throttle.h" using namespace pstd; using namespace net; using namespace RsyncService; +const std::string kDumpMetaFileName = "DUMP_META_DATA"; +const std::string kUuidPrefix = "snapshot-uuid:"; + namespace rsync { class RsyncWriter; @@ -49,6 +52,7 @@ class RsyncClient : public net::Thread { Status LoadFile(const std::string& filename); Status LoadMetaTable(); Status FlushMetaTable(); + void HandleRsyncMetaResponse(RsyncResponse* response); private: //已经下载完成的文件名与checksum值,用于宕机重启时恢复, diff --git a/include/rsync_server.h b/include/rsync_server.h index 0cdbf82c8..07d55fc55 100644 --- a/include/rsync_server.h +++ b/include/rsync_server.h @@ -113,6 +113,8 @@ class RSyncReader { private: std::string filepath_; std::unique_ptr file_; +public: + std::string GetFilePath() { return filepath_; } }; } //end namespace rsync diff --git a/src/rsync_client.cc b/src/rsync_client.cc index 84a2b10c7..396c0008c 100644 --- a/src/rsync_client.cc +++ b/src/rsync_client.cc @@ -1,7 +1,12 @@ #include "include/rsync_client.h" +#include "pstd/src/env.cc" +#include +#include + using namespace net; using namespace pstd; using namespace RsyncService; + namespace rsync { RsyncClient::RsyncClient(const std::string& dir, const std::string& ip, const int port) : dir_ (dir), ip_ (ip), port_(port), state_ (IDLE) { @@ -161,11 +166,61 @@ Status RsyncClient::Stop() { //TODO: yuecai void RsyncClient::Recover() { file_set_.insert("filename"); + // 从远程读取 meta } -//TODO: shaoyi Status RsyncClient::LoadMetaTable() { - LOG(WARNING) << "LoadMetaTable called"; + if (!FileExists(dir_)) { + return Status::OK(); + } + + FILE* fp; + char* line = nullptr; + size_t len = 0; + size_t read = 0; + int32_t line_num = 0; + + std::atomic_int8_t retry_times = 5; + + while (retry_times -- > 0) { + fp = fopen(dir_.c_str(), "r"); + if (fp == nullptr) { + LOG(WARNING) << "open meta file failed, meta_path: " << dir_; + } else { + break; + } + } + // if the file cannot be read from disk, use the remote file directly + if (fp == nullptr) { + LOG(WARNING) << "open meta file failed, meta_path: " << dir_ << ", retry times: " << retry_times; + return Status::IOError("open meta file failed, dir: ", dir_); + } + + while ((read = getline(&line, &len, fp)) != -1) { + std::string str(line); + std::string::size_type pos; + while ((pos = str.find("\r")) != std::string::npos) { + str.erase(pos, 1); + } + while ((pos = str.find("\n")) != std::string::npos) { + str.erase(pos, 1); + } + + if (str.empty()) { + continue; + } + + if (line_num == 0) { + snapshot_uuid_ = str.erase(0, kUuidPrefix.size()); + } else { + if ((pos = str.find(":")) != std::string::npos) { + str.erase(pos, str.size() - pos); + } + file_set_.insert(str); + } + + line_num++; + } return Status::OK(); } From e4aa9488c3ba5c561a7003e645f5e1af0686823d Mon Sep 17 00:00:00 2001 From: wangshao1 <30471730+wangshao1@users.noreply.github.com> Date: Thu, 13 Jul 2023 21:05:13 +0800 Subject: [PATCH 07/37] [feat] add rsync client/server code (#177) * implement rsync network transfer --- include/pika_server.h | 5 + include/rsync_client.h | 68 +++++++++-- src/pika_server.cc | 20 +++ src/rsync_client.cc | 262 +++++++++++++++++++++++++++------------- src/rsync_server.cc | 77 ++++++++++-- src/rsync_service.proto | 12 +- src/throttle.cc | 10 +- 7 files changed, 341 insertions(+), 113 deletions(-) diff --git a/include/pika_server.h b/include/pika_server.h index e01f7b023..62121470e 100644 --- a/include/pika_server.h +++ b/include/pika_server.h @@ -259,6 +259,11 @@ class PikaServer : public pstd::noncopyable { /* * DBSync used */ + //TODO: yuecai + size_t ReadDumpFile(const std::string& db_name, uint32_t slot_id, const std::string& filename, + const size_t offset, const size_t count, char* data); + //TODO: yuecai + void GetDumpMeta(const std::string& db_name, const uint32_t slot_id, std::vector* files, std::string* snapshot_uuid); void DBSync(const std::string& ip, int port, const std::string& db_name, uint32_t slot_id); void TryDBSync(const std::string& ip, int port, const std::string& db_name, uint32_t slot_id, int32_t top); void DbSyncSendFile(const std::string& ip, int port, const std::string& db_name, uint32_t slot_id); diff --git a/include/rsync_client.h b/include/rsync_client.h index da5c80776..1eaa11d01 100644 --- a/include/rsync_client.h +++ b/include/rsync_client.h @@ -1,6 +1,9 @@ #ifndef RSYNC_CLIENT_H_ #define RSYNC_CLIENT_H_ #include +#include +#include +#include #include #include #include @@ -8,6 +11,8 @@ #include "include/rsync_client_thread.h" #include "net/include/bg_thread.h" +#include "pstd/include/pstd_status.h" +#include "include/rsync_client_thread.h" #include "net/include/net_cli.h" #include "pstd/include/env.h" #include "pstd/include/pstd_hash.h" @@ -26,6 +31,7 @@ namespace rsync { class RsyncWriter; class Session; +class WaitObject; class RsyncClient : public net::Thread { public: @@ -34,10 +40,6 @@ class RsyncClient : public net::Thread { RUNNING, STOP, }; - enum Type { - kMeta, - kFile, - }; RsyncClient(const std::string& dir, const std::string& ip, const int port); void* ThreadMain() override; @@ -46,10 +48,10 @@ class RsyncClient : public net::Thread { Status Stop(); void OnReceive(RsyncResponse* resp); private: - void Recover(); - Status SendRequest(const std::string& filename, size_t offset, Type type); - Status HandleResponse(const std::string& filename, size_t& offset, MD5& md5, RsyncWriter& writer); - Status LoadFile(const std::string& filename); + bool Recover(); + Status Wait(WaitObject* wo); + Status CopyRemoteFile(const std::string& filename); + Status CopyRemoteMeta(std::string* snapshot_uuid, std::set* file_set); Status LoadMetaTable(); Status FlushMetaTable(); void HandleRsyncMetaResponse(RsyncResponse* response); @@ -58,6 +60,7 @@ class RsyncClient : public net::Thread { //已经下载完成的文件名与checksum值,用于宕机重启时恢复, //减少重复文件下载,周期性flush到磁盘上 std::map meta_table_; + int flush_period_; //待拉取的文件集合 std::set file_set_; std::string snapshot_uuid_; @@ -66,8 +69,12 @@ class RsyncClient : public net::Thread { std::string ip_; int port_; + std::string db_name_; + uint32_t slot_id_; + std::unique_ptr client_thread_; std::atomic state_; + int max_retries_; std::list resp_list_; std::condition_variable cond_; @@ -77,15 +84,52 @@ class RsyncClient : public net::Thread { //TODO: jinge class RsyncWriter { public: - RsyncWriter(const std::string& filepath) {} + RsyncWriter(const std::string& filepath) { + filepath_ = filepath; + fd_ = open(filepath.c_str(), O_RDWR | O_APPEND | O_CREAT, 0644); + LOG(WARNING) << "rsyncwriter fd: " << fd_; + } ~RsyncWriter() {} - Status Write(uint64_t offset, size_t n, Slice* result) { - return Status::OK(); + Status Write(uint64_t offset, size_t n, const char* data) { + const char* ptr = data; + size_t left = n; + Status s; + while (left != 0) { + ssize_t done = write(fd_, ptr, left); + if (done < 0) { + if (errno == EINTR) continue; + LOG(WARNING) << "pwrite failed, filename: " << filepath_ << "errno: " << strerror(errno) << "n: " << n; + return Status::IOError(filepath_, "pwrite failed"); + } + left -= done; + ptr += done; + offset += done; + } + return Status::OK(); + } + Status Close() { + close(fd_); + return Status::OK(); + } + Status Fsync() { + fsync(fd_); + return Status::OK(); } private: std::string filepath_; - std::unique_ptr file_; + int fd_; +}; + +class WaitObject { +public: + WaitObject(const std::string& filename, RsyncService::Type t, size_t offset) + : filename_(filename), type_(t), offset_(offset), resp_(nullptr) {} + WaitObject(RsyncService::Type t) : filename_(""), type_(t), offset_(-1), resp_(nullptr) {} + std::string filename_; + RsyncService::Type type_; + size_t offset_; + RsyncResponse* resp_; }; } // end namespace rsync diff --git a/src/pika_server.cc b/src/pika_server.cc index 0b9abb9f5..04526baf9 100644 --- a/src/pika_server.cc +++ b/src/pika_server.cc @@ -905,6 +905,26 @@ void PikaServer::DBSync(const std::string& ip, int port, const std::string& db_n bgsave_thread_.Schedule(&DoDBSync, reinterpret_cast(arg)); } +void PikaServer::GetDumpMeta(const std::string& db_name, const uint32_t slot_id, std::vector* files, std::string* snapshot_uuid) { + std::shared_ptr slot = GetDBSlotById(db_name, slot_id); + pstd::GetChildren(slot->bgsave_info().path, *files); + *snapshot_uuid = "demo_snapshot_uuid"; +} + +size_t PikaServer::ReadDumpFile(const std::string& db_name, uint32_t slot_id, const std::string& filename, + const size_t offset, const size_t count, char* data) { + std::shared_ptr slot = GetDBSlotById(db_name, slot_id); + if (!slot) { + LOG(WARNING) << "cannot find slot for db_name " << db_name << "slot_id: " << slot_id; + return 0; + } + const std::string filepath = slot->bgsave_info().path + "/" + filename; + int fd = open(filepath.c_str(), O_RDONLY, 0644); + ssize_t n = pread(fd, data, count, offset); + close(fd); + return n; +} + void PikaServer::TryDBSync(const std::string& ip, int port, const std::string& db_name, uint32_t slot_id, int32_t top) { std::shared_ptr slot = GetDBSlotById(db_name, slot_id); diff --git a/src/rsync_client.cc b/src/rsync_client.cc index 396c0008c..bd5f78f7e 100644 --- a/src/rsync_client.cc +++ b/src/rsync_client.cc @@ -1,4 +1,5 @@ #include "include/rsync_client.h" +#include "pstd/include/pstd_defer.h" #include "pstd/src/env.cc" #include #include @@ -7,44 +8,54 @@ using namespace net; using namespace pstd; using namespace RsyncService; +using namespace pstd; namespace rsync { RsyncClient::RsyncClient(const std::string& dir, const std::string& ip, const int port) - : dir_ (dir), ip_ (ip), port_(port), state_ (IDLE) { + : dir_ (dir), flush_period_(100), ip_(ip), port_(port), state_(IDLE), + db_name_(""), slot_id_(0), max_retries_(10) { client_thread_ = std::make_unique(10 * 1000, 60 * 1000, this); } bool RsyncClient::Init() { if (state_.load(std::memory_order_relaxed) != IDLE) { Stop(); - Start(); } - Recover(); - state_ = RUNNING; + client_thread_->StartThread(); + bool ret = Recover(); + if (!ret) { + client_thread_->StopThread(); + return false; + } return true; } void* RsyncClient::ThreadMain() { - int period = 1000; + int cnt = 0; + int period = 0; Status s = Status::OK(); while (state_.load(std::memory_order_relaxed) == RUNNING) { for (const auto& file : file_set_) { - s = LoadFile(file); - if (!s.ok()) { - //TODO: 同步状态下,是保持最大重试次数还是死循环地搞? - LOG(WARNING) << "rsync LoadFile failed"; - return nullptr; + LOG(INFO) << "CopyRemoteFile: " << file; + while (state_.load() == RUNNING) { + s = CopyRemoteFile(file); + if (!s.ok()) { + LOG(WARNING) << "rsync CopyRemoteFile failed, filename: " << file; + continue; + } + LOG(WARNING) << "CopyRemoteFile "<< file << "success..."; + break; } if (state_.load(std::memory_order_relaxed) != RUNNING) { break; } - if (--period == 0) { - period = 10; + if (++period == flush_period_) { + period = 0; FlushMetaTable(); } } if (meta_table_.size() == file_set_.size()) { - LOG(INFO) << "rsync done..."; - state_.store(STOP); + LOG(INFO) << "rsync success..."; + state_.store(STOP, std::memory_order_relaxed); break; } } @@ -57,99 +68,136 @@ void RsyncClient::OnReceive(RsyncResponse* resp) { cond_.notify_all(); } -Status RsyncClient::SendRequest(const std::string& filename, size_t offset, Type type) { - Status s = Status::OK(); - RsyncRequest request; - switch (type) { - case kMeta: { - request.set_type(kRsyncMeta); - break; +Status RsyncClient::Wait(WaitObject* wo) { + Status s = Status::Timeout("rsync timeout", "timeout"); + std::list resp_list; + { + std::unique_lock lock(mu_); + cond_.wait_for(lock, std::chrono::seconds(3), [this]{return !resp_list_.empty();}); + resp_list.swap(resp_list_); + } + auto iter = resp_list.begin(); + while (iter != resp_list.end()) { + RsyncResponse* resp = *iter; + if (resp->type() != wo->type_) { + LOG(WARNING) << "mismatch request/response type, skip"; + iter++; + continue; } - case kFile: { - request.set_type(kRsyncFile); - FileRequest* file_req = request.mutable_file_req(); - file_req->set_filename(filename); - file_req->set_offset(offset); - file_req->set_count(4096); - break; + if (resp->type() == kRsyncFile && + (resp->file_resp().filename() != wo->filename_ || + resp->file_resp().offset() != wo->offset_)) { + LOG(WARNING) << "mismatch rsync response, skip"; + continue; } - default: - break; - } - std::string to_send; - request.SerializeToString(&to_send); - s = client_thread_->Write(ip_, port_, to_send); - if (!s.ok()) { - LOG(WARNING) << "send rsync request failed"; + s = Status::OK(); + wo->resp_ = resp; + resp_list.erase(iter); + break; } - return s; -} - -Status RsyncClient::HandleResponse(const std::string& filename, size_t& offset, MD5& md5, RsyncWriter& writer) { - std::unique_lock lock(mu_); - cond_.wait_for(lock, std::chrono::seconds(3), [this]{return !resp_list_.empty();}); - - Status s = Status::Timeout("rsync timeout", "timeout"); - auto iter = resp_list_.begin(); - while (iter != resp_list_.end()) { - RsyncResponse* response = *iter; - switch (response->type()) { - case kRsyncMeta: { - LOG(INFO) << "receive rsync meta infos, uuid: " << response->meta_resp().uuid() - << "files count: " << response->meta_resp().filenames_size(); - for (int i = 0; i < response->meta_resp().filenames_size(); i++) { - LOG(WARNING) << "filename: " << response->meta_resp().filenames(i); - } - break; - } - case kRsyncFile: { - s = Status::OK(); - //比对filename和offset - LOG(WARNING) << "receive rsync file infos: " - << "filename: " << response->file_resp().filename() - << "offset: " << response->file_resp().offset() - << "count: " << response->file_resp().count(); - md5.update(response->file_resp().data().c_str(), response->file_resp().count()); - break; - } - default: - break; - } - //TODO: 比对正常,退出resp_list - delete response; - break; + iter = resp_list.begin(); + while (iter != resp_list.end()) { + delete (*iter); + iter++; } return s; } -Status RsyncClient::LoadFile(const std::string& filename) { +Status RsyncClient::CopyRemoteFile(const std::string& filename) { Status s; int retries = 0; - int max_retries_ = 10; size_t offset = 0; - Type type = kFile; + size_t count = 1 << 10; MD5 md5; - RsyncWriter writer(dir_ + "/" + filename); + std::unique_ptr writer(new RsyncWriter(dir_ + "/" + filename)); + DEFER { + if (writer) { + writer->Close(); + writer.reset(); + } + if (!s.ok()) { + DeleteFile(filename); + } + }; while (retries < max_retries_) { - s = SendRequest(filename, offset, kFile); + RsyncRequest request; + request.set_type(kRsyncFile); + request.set_db_name(db_name_); + request.set_slot_id(slot_id_); + FileRequest* file_req = request.mutable_file_req(); + file_req->set_filename(filename); + file_req->set_offset(offset); + file_req->set_count(1 << 20); + std::string to_send; + request.SerializeToString(&to_send); + + s = client_thread_->Write(ip_, port_, to_send); if (!s.ok()) { + LOG(WARNING) << "send rsync request failed"; + continue; + } + + WaitObject wo(filename, kRsyncFile, offset); + LOG(INFO) << "wait CopyRemoteFile response....."; + s = Wait(&wo); + if (s.IsTimeout() || wo.resp_ == nullptr) { + LOG(WARNING) << "rsync request timeout"; retries++; continue; } - s = HandleResponse(filename, offset, md5, writer); + RsyncResponse* resp = wo.resp_; + + LOG(INFO) << "receive fileresponse, snapshot_uuid: " << resp->snapshot_uuid() + << "filename: " << resp->file_resp().filename() + << "offset: " << resp->file_resp().offset() + << "count: " << resp->file_resp().count(); + + if (resp->snapshot_uuid() != snapshot_uuid_) { + LOG(WARNING) << "receive newer dump, reset state to STOP"; + state_.store(STOP); + delete resp; + return s; + } + + size_t offset = resp->file_resp().offset(); + size_t count = resp->file_resp().count(); + resp->file_resp().data(); + s = writer->Write((uint64_t)offset, count, resp->file_resp().data().c_str()); if (!s.ok()) { - retries++; + LOG(WARNING) << "rsync client write file error"; + break; + } + + md5.update(resp->file_resp().data().c_str(), resp->file_resp().count()); + if (resp->file_resp().eof()) { + if (md5.finalize().hexdigest() != resp->file_resp().checksum()) { + LOG(WARNING) << "mismatch file checksum for file: " << filename; + //TODO: 处理返回status + s = Status::IOError("mismatch checksum", "mismatch checksum"); + } + s = writer->Fsync(); + if (!s.ok()) { + return s; + } + s = writer->Close(); + if (!s.ok()) { + return s; + } + writer.reset(); + meta_table_.insert(std::make_pair(filename, resp->file_resp().checksum())); break; + } else { + offset += resp->file_resp().count(); } retries = 0; } + return s; } Status RsyncClient::Start() { StartThread(); - client_thread_->StartThread(); return Status::OK(); } @@ -163,10 +211,56 @@ Status RsyncClient::Stop() { return Status::OK(); } -//TODO: yuecai -void RsyncClient::Recover() { - file_set_.insert("filename"); - // 从远程读取 meta +//TODO: shaoyi +Status RsyncClient::CopyRemoteMeta(std::string* snapshot_uuid, std::set* file_set) { + Status s; + int retries = 0; + RsyncRequest request; + request.set_db_name(db_name_); + request.set_slot_id(slot_id_); + request.set_type(kRsyncMeta); + std::string to_send; + request.SerializeToString(&to_send); + while (retries < max_retries_) { + s = client_thread_->Write(ip_, port_, to_send); + if (!s.ok()) { + retries++; + } + WaitObject wo(kRsyncMeta); + s = Wait(&wo); + if (s.IsTimeout() || wo.resp_ == nullptr) { + LOG(WARNING) << "rsync CopyRemoteMeta request timeout, retry times: " << retries; + retries++; + continue; + } + RsyncResponse* resp = wo.resp_; + LOG(INFO) << "receive rsync meta infos, snapshot_uuid: " << resp->snapshot_uuid() + << "files count: " << resp->meta_resp().filenames_size(); + for (int i = 0; i < resp->meta_resp().filenames_size(); i++) { + LOG(INFO) << "file: " << resp->meta_resp().filenames(i); + } + *snapshot_uuid = resp->snapshot_uuid(); + for (int i = 0; i < resp->meta_resp().filenames_size(); i++) { + file_set->insert(resp->meta_resp().filenames(i)); + } + break; + } + return s; +} +bool RsyncClient::Recover() { + std::string snapshot_uuid; + std::set file_set; + Status s = CopyRemoteMeta(&snapshot_uuid, &file_set); + if (!s.ok()) { + LOG(WARNING) << "copy remote meta failed"; + return false; + } + //TODO: yuecai 加载本地元信息文件,与master回包内容diff + snapshot_uuid_ = snapshot_uuid; + file_set_.insert(file_set.begin(), file_set.end()); + state_ = RUNNING; + LOG(WARNING) << "copy remote meta done"; + return true; } Status RsyncClient::LoadMetaTable() { diff --git a/src/rsync_server.cc b/src/rsync_server.cc index b117c8417..ca7b80424 100644 --- a/src/rsync_server.cc +++ b/src/rsync_server.cc @@ -1,17 +1,48 @@ +#include #include "include/rsync_server.h" #include +#include "include/pika_server.h" +//extern PikaServer* g_pika_server; namespace rsync { +//TODO: mock code, need removed +void GetDumpMeta(const std::string& db_name, const uint32_t slot_id, std::vector* files, std::string* snapshot_uuid) { + pstd::GetChildren(".", *files); + auto iter = files->begin(); + while (iter != files->end()) { + if (std::filesystem::is_directory(*iter)) { + iter = files->erase(iter); + continue; + } + iter++; + } + *snapshot_uuid = "demo_snapshot_uuid"; +} + +//TODO: mock code, need removed +ssize_t ReadDumpFile(const std::string& db_name, uint32_t slot_id, const std::string& filename, + const size_t offset, const size_t count, char* data) { + const std::string filepath = std::string("./") + filename; + int fd = open(filepath.c_str(), O_RDONLY, 0644); + ssize_t n = pread(fd, data, count, offset); + LOG(WARNING) << "read n: " << n; + if (n < 0) { + LOG(WARNING) << "pread error, errno:" << strerror(errno); + } + close(fd); + return n; +} + RsyncServer::RsyncServer(const std::string& ip, const int port, void* worker_specific_data, const std::string& dir) : dir_(dir), ip_(ip), port_(port) { work_thread_ = std::make_unique(2, 100000); - //TODO parse parameter std::set ips = {ip_}; rsync_server_thread_ = std::make_unique(ips, port, 60 * 1000, this); } RsyncServer::~RsyncServer() { + //TODO: handle destory LOG(INFO) << "Rsync server destroyed"; } @@ -30,7 +61,6 @@ int RsyncServer::Start() { LOG(FATAL) << "Start ThreadPool Error: " << res << (res == net::kCreateThreadError ? ": create thread error " : ": other error"); } - LOG(WARNING) << "after start_thread_pool"; return res; } @@ -40,7 +70,6 @@ int RsyncServer::Stop() { return 0; } -/*-------------------------RsyncServerConn--------------------*/ RsyncServerConn::RsyncServerConn(int connfd, const std::string& ip_port, Thread* thread, void* worker_specific_data, NetMultiplexer* mpx) : PbConn(connfd, ip_port, thread, mpx), data_(worker_specific_data) {} @@ -83,12 +112,26 @@ void RsyncServerConn::HandleMetaRsyncRequest(void* arg) { std::shared_ptr conn = task_arg->conn; RsyncService::RsyncResponse response; + response.set_db_name("db_name"); + response.set_slot_id(0); response.set_type(RsyncService::kRsyncMeta); LOG(INFO) << "Receive RsyncMeta request"; + std::string db_name = req->db_name(); + uint32_t slot_id = req->slot_id(); + std::vector filenames; + std::string snapshot_uuid; + GetDumpMeta(db_name, slot_id, &filenames, &snapshot_uuid); + LOG(WARNING) << "snapshot_uuid: " << snapshot_uuid; + std::for_each(filenames.begin(), filenames.end(), [](auto& file) { + LOG(WARNING) << "file:" << file; + }); //TODO: temporarily mock response RsyncService::MetaResponse* meta_resp = response.mutable_meta_resp(); - meta_resp->set_uuid("uuid"); + response.set_snapshot_uuid(snapshot_uuid); + for (const auto& filename : filenames) { + meta_resp->add_filenames(filename); + } std::string reply_str; if (!response.SerializeToString(&reply_str) || (conn->WriteResp(reply_str) != 0)) { @@ -106,25 +149,41 @@ void RsyncServerConn::HandleFileRsyncRequest(void* arg) { RsyncService::RsyncResponse response; response.set_type(RsyncService::kRsyncFile); + response.set_snapshot_uuid("demo_snapshot_uuid"); + response.set_db_name("db_name"); + response.set_slot_id(0); LOG(INFO) << "Receive RsyncFile request " << "filename: " << req->file_req().filename() << " offset: " << req->file_req().offset() << " count: " << req->file_req().count(); + std::string db_name = req->db_name(); + std::string filename = req->file_req().filename(); + uint32_t slot_id = req->slot_id(); + size_t offset = req->file_req().offset(); + size_t count = req->file_req().count(); + char* buffer = new char[req->file_req().count() + 1]; + LOG(INFO) << "....... ReadDumpFile: " << filename; + auto r = ReadDumpFile(db_name, slot_id, filename, offset, count, buffer); + LOG(INFO) << "ReadDumpFile: " << filename << " read size: " << r; + //TODO: temporarily mock response RsyncService::FileResponse* file_resp = response.mutable_file_resp(); - file_resp->set_eof(1); - file_resp->set_count(1024); - file_resp->set_offset(1024); - file_resp->set_data("mocking data"); + file_resp->set_eof(r != count); + file_resp->set_count(r); + file_resp->set_offset(offset); + file_resp->set_data(buffer, r); file_resp->set_checksum("checksum"); - file_resp->set_filename("filename"); + file_resp->set_filename(filename); + LOG(INFO) << "....... before serializetostring: " << filename; std::string reply_str; if (!response.SerializeToString(&reply_str) || (conn->WriteResp(reply_str) != 0)) { LOG(WARNING) << "Process FileRsync request serialization failed"; conn->NotifyClose(); + delete []buffer; return; } + delete []buffer; conn->NotifyWrite(); } diff --git a/src/rsync_service.proto b/src/rsync_service.proto index baeb52594..a35a6835b 100644 --- a/src/rsync_service.proto +++ b/src/rsync_service.proto @@ -8,7 +8,6 @@ enum Type { message MetaResponse { repeated string filenames = 1; - required string uuid = 2; } message FileRequest { @@ -28,11 +27,16 @@ message FileResponse { message RsyncRequest { required Type type = 1; - optional FileRequest file_req = 3; + required string db_name = 2; + required uint32 slot_id = 3; + optional FileRequest file_req = 4; } message RsyncResponse { required Type type = 1; - optional MetaResponse meta_resp = 2; - optional FileResponse file_resp = 3; + required string snapshot_uuid = 2; + required string db_name = 3; + required uint32 slot_id = 4; + optional MetaResponse meta_resp = 5; + optional FileResponse file_resp = 6; } \ No newline at end of file diff --git a/src/throttle.cc b/src/throttle.cc index 9b995de1b..0346a6227 100644 --- a/src/throttle.cc +++ b/src/throttle.cc @@ -9,15 +9,16 @@ #include "pstd/include/env.h" DEFINE_uint64(raft_minimal_throttle_threshold_mb, 0, "minimal throttle throughput threshold per second"); +namespace rsync{ -rsync::Throttle::Throttle(size_t throttle_throughput_bytes, size_t check_cycle) +Throttle::Throttle(size_t throttle_throughput_bytes, size_t check_cycle) : throttle_throughput_bytes_(throttle_throughput_bytes), last_throughput_check_time_us_(caculate_check_time_us_(pstd::NowMicros(), check_cycle)), cur_throughput_bytes_(0) {} -rsync::Throttle::~Throttle() {} +Throttle::~Throttle() {} -size_t rsync::Throttle::ThrottledByThroughput(size_t bytes) { +size_t Throttle::ThrottledByThroughput(size_t bytes) { size_t available_size = bytes; size_t now = pstd::NowMicros(); size_t limit_throughput_bytes_s = std::max(static_cast(throttle_throughput_bytes_), @@ -47,7 +48,7 @@ size_t rsync::Throttle::ThrottledByThroughput(size_t bytes) { return available_size; } -void rsync::Throttle::ReturnUnusedThroughput(size_t acquired, size_t consumed, size_t elaspe_time_us) { +void Throttle::ReturnUnusedThroughput(size_t acquired, size_t consumed, size_t elaspe_time_us) { size_t now = pstd::NowMicros(); std::unique_lock lock(keys_mutex_); if (now - elaspe_time_us < last_throughput_check_time_us_) { @@ -55,4 +56,5 @@ void rsync::Throttle::ReturnUnusedThroughput(size_t acquired, size_t consumed, s return; } cur_throughput_bytes_ = std::max(cur_throughput_bytes_ - (acquired - consumed), size_t(0)); +} } \ No newline at end of file From 26d4db8f58a0fb1b7c9e27f4b5c6d34f3502823d Mon Sep 17 00:00:00 2001 From: wangshao1 <30471730+wangshao1@users.noreply.github.com> Date: Fri, 14 Jul 2023 15:04:01 +0800 Subject: [PATCH 08/37] add rsyncclient to syncslaveslot (#182) add rsyncclient to syncslaveslot --- include/pika_rm.h | 3 +++ include/rsync_client.h | 5 +++-- src/pika_rm.cc | 13 ++++++++++++- src/rsync_client.cc | 23 +++++++++++++---------- 4 files changed, 31 insertions(+), 13 deletions(-) diff --git a/include/pika_rm.h b/include/pika_rm.h index 7c50bbe05..7211f77d7 100644 --- a/include/pika_rm.h +++ b/include/pika_rm.h @@ -157,7 +157,10 @@ class SyncSlaveSlot : public SyncSlot { std::string LocalIp(); + void ActivateRsync(); + private: + std::unique_ptr sync_cli_; pstd::Mutex slot_mu_; RmNode m_info_; ReplState repl_state_{kNoConnect}; diff --git a/include/rsync_client.h b/include/rsync_client.h index 1eaa11d01..b82990545 100644 --- a/include/rsync_client.h +++ b/include/rsync_client.h @@ -40,12 +40,13 @@ class RsyncClient : public net::Thread { RUNNING, STOP, }; - RsyncClient(const std::string& dir, const std::string& ip, const int port); + RsyncClient(const std::string& dir, const std::string& db_name, const uint32_t slot_id); void* ThreadMain() override; - bool Init(); + bool Init(const std::string& local_ip); Status Start(); Status Stop(); + State State() { return state_;} void OnReceive(RsyncResponse* resp); private: bool Recover(); diff --git a/src/pika_rm.cc b/src/pika_rm.cc index 7af72c97a..90f602456 100644 --- a/src/pika_rm.cc +++ b/src/pika_rm.cc @@ -537,7 +537,8 @@ Status SyncMasterSlot::ConsensusReset(const LogOffset& applied_offset) { return /* SyncSlaveSlot */ SyncSlaveSlot::SyncSlaveSlot(const std::string& db_name, uint32_t slot_id) - : SyncSlot(db_name, slot_id) { + : SyncSlot(db_name, slot_id) { + rsync_cli_.reset(new rsync::RsyncClient(dbsync_path_, db_name, slot_id)); m_info_.SetLastRecvTime(pstd::NowMicros()); } @@ -637,6 +638,15 @@ std::string SyncSlaveSlot::LocalIp() { return local_ip_; } +void SyncSlaveSlot::ActivateRsync() { + if (rsync_cli_->State() == RsyncCient::RUNNING) { + return; + } + if (rsync_cli_->Init(local_ip_)) { + rsnyc_cli_->Start(); + } +} + /* PikaReplicaManger */ PikaReplicaManager::PikaReplicaManager() { @@ -1142,6 +1152,7 @@ Status PikaReplicaManager::RunSyncSlaveSlotStateMachine() { std::shared_ptr slot = g_pika_server->GetDBSlotById(p_info.db_name_, p_info.slot_id_); if (slot) { + slot->ActivateRsync(); slot->TryUpdateMasterOffset(); } else { LOG(WARNING) << "Slot not found, DB Name: " << p_info.db_name_ diff --git a/src/rsync_client.cc b/src/rsync_client.cc index bd5f78f7e..d7833f230 100644 --- a/src/rsync_client.cc +++ b/src/rsync_client.cc @@ -10,15 +10,16 @@ using namespace RsyncService; using namespace pstd; namespace rsync { -RsyncClient::RsyncClient(const std::string& dir, const std::string& ip, const int port) - : dir_ (dir), flush_period_(100), ip_(ip), port_(port), state_(IDLE), +RsyncClient::RsyncClient(const std::string& dir, const std::string& db_name, const size_t slot_id) + : dir_ (dir), flush_period_(100), db_name_(ip), slot_id_(port), state_(IDLE), db_name_(""), slot_id_(0), max_retries_(10) { client_thread_ = std::make_unique(10 * 1000, 60 * 1000, this); } -bool RsyncClient::Init() { - if (state_.load(std::memory_order_relaxed) != IDLE) { - Stop(); +bool RsyncClient::Init(const std::string ip_port) { + if (!ParseIpPortString(ip_port, ip_, port_)) { + LOG(WARNING) << "Parse ip_port error " << ip_port; + return false; } client_thread_->StartThread(); bool ret = Recover(); @@ -73,9 +74,12 @@ Status RsyncClient::Wait(WaitObject* wo) { std::list resp_list; { std::unique_lock lock(mu_); - cond_.wait_for(lock, std::chrono::seconds(3), [this]{return !resp_list_.empty();}); + cond_.wait_for(lock, std::chrono::seconds(3), [this]{ + return !resp_list_.empty();} + ); resp_list.swap(resp_list_); } + auto iter = resp_list.begin(); while (iter != resp_list.end()) { RsyncResponse* resp = *iter; @@ -160,16 +164,15 @@ Status RsyncClient::CopyRemoteFile(const std::string& filename) { return s; } - size_t offset = resp->file_resp().offset(); - size_t count = resp->file_resp().count(); + size_t ret_count = resp->file_resp().count(); resp->file_resp().data(); - s = writer->Write((uint64_t)offset, count, resp->file_resp().data().c_str()); + s = writer->Write((uint64_t)offset, ret_count, resp->file_resp().data().c_str()); if (!s.ok()) { LOG(WARNING) << "rsync client write file error"; break; } - md5.update(resp->file_resp().data().c_str(), resp->file_resp().count()); + md5.update(resp->file_resp().data().c_str(), ret_count); if (resp->file_resp().eof()) { if (md5.finalize().hexdigest() != resp->file_resp().checksum()) { LOG(WARNING) << "mismatch file checksum for file: " << filename; From 547530c2fd520b4fcaec35da12f5ae680d994443 Mon Sep 17 00:00:00 2001 From: Yuecai Liu <38887641+luky116@users.noreply.github.com> Date: Fri, 14 Jul 2023 15:27:12 +0800 Subject: [PATCH 09/37] feat: add read meta file and data (#179) * add read meta file and data --- include/pika_server.h | 4 +-- include/pika_slot.h | 1 + include/rsync_server.h | 8 +++--- src/pika_server.cc | 56 ++++++++++++++++++++++++++++++++++-------- src/pika_slot.cc | 23 +++++++++++++++++ 5 files changed, 77 insertions(+), 15 deletions(-) diff --git a/include/pika_server.h b/include/pika_server.h index 62121470e..9e856948e 100644 --- a/include/pika_server.h +++ b/include/pika_server.h @@ -260,10 +260,10 @@ class PikaServer : public pstd::noncopyable { * DBSync used */ //TODO: yuecai - size_t ReadDumpFile(const std::string& db_name, uint32_t slot_id, const std::string& filename, + pstd::Status ReadDumpFile(const std::string& db_name, uint32_t slot_id, const std::string& filename, const size_t offset, const size_t count, char* data); //TODO: yuecai - void GetDumpMeta(const std::string& db_name, const uint32_t slot_id, std::vector* files, std::string* snapshot_uuid); + pstd::Status GetDumpMeta(const std::string& db_name, const uint32_t slot_id, std::vector* files, std::string* snapshot_uuid); void DBSync(const std::string& ip, int port, const std::string& db_name, uint32_t slot_id); void TryDBSync(const std::string& ip, int port, const std::string& db_name, uint32_t slot_id, int32_t top); void DbSyncSendFile(const std::string& ip, int port, const std::string& db_name, uint32_t slot_id); diff --git a/include/pika_slot.h b/include/pika_slot.h index 0aa2a49cf..c41b906ca 100644 --- a/include/pika_slot.h +++ b/include/pika_slot.h @@ -76,6 +76,7 @@ class Slot : public std::enable_shared_from_this,public pstd::noncopyable bool IsBgSaving(); void BgSaveSlot(); BgSaveInfo bgsave_info(); + void GetBgSaveMetaData(std::vector* fileNames, std::string* snapshot_uuid); // FlushDB & FlushSubDB use bool FlushDB(); diff --git a/include/rsync_server.h b/include/rsync_server.h index 07d55fc55..0f3550e47 100644 --- a/include/rsync_server.h +++ b/include/rsync_server.h @@ -5,15 +5,16 @@ #include #include -#include "rsync_service.pb.h" #include "net/include/net_conn.h" #include "net/include/net_thread.h" -#include "net/include/thread_pool.h" #include "net/include/pb_conn.h" #include "net/include/server_thread.h" +#include "net/include/thread_pool.h" +#include "net/src/holy_thread.h" #include "net/src/net_multiplexer.h" #include "pstd/include/env.h" -#include "net/src/holy_thread.h" +#include "pstd_hash.h" +#include "rsync_service.pb.h" using namespace net; using namespace RsyncService; @@ -42,6 +43,7 @@ class RsyncServer { int port_; std::string ip_; std::string dir_; + std::string info_dir_; std::map > file_map_; std::unique_ptr work_thread_ = nullptr; std::unique_ptr rsync_server_thread_ = nullptr; diff --git a/src/pika_server.cc b/src/pika_server.cc index 04526baf9..d67a6d23e 100644 --- a/src/pika_server.cc +++ b/src/pika_server.cc @@ -21,10 +21,12 @@ #include "net/include/redis_cli.h" #include "pstd/include/env.h" #include "pstd/include/rsync.h" +# include "pstd/include/pstd_defer.h" #include "include/pika_cmd_table_manager.h" #include "include/pika_dispatch_thread.h" #include "include/pika_rm.h" +#include "pstd_hash.h" using pstd::Status; extern PikaServer* g_pika_server; @@ -905,24 +907,58 @@ void PikaServer::DBSync(const std::string& ip, int port, const std::string& db_n bgsave_thread_.Schedule(&DoDBSync, reinterpret_cast(arg)); } -void PikaServer::GetDumpMeta(const std::string& db_name, const uint32_t slot_id, std::vector* files, std::string* snapshot_uuid) { - std::shared_ptr slot = GetDBSlotById(db_name, slot_id); - pstd::GetChildren(slot->bgsave_info().path, *files); - *snapshot_uuid = "demo_snapshot_uuid"; +pstd::Status PikaServer::GetDumpMeta(const std::string& db_name, const uint32_t slot_id, std::vector* fileNames, std::string* snapshot_uuid) { + std::shared_ptr slot = GetDBSlotById(db_name, slot_id); + if (!slot) { + LOG(WARNING) << "cannot find slot for db_name " << db_name << "slot_id: " << slot_id; + return pstd::Status::NotFound("slot no found"); + } + slot->GetBgSaveMetaData(fileNames, snapshot_uuid); } -size_t PikaServer::ReadDumpFile(const std::string& db_name, uint32_t slot_id, const std::string& filename, +pstd::Status PikaServer::ReadDumpFile(const std::string& db_name, uint32_t slot_id, const std::string& filename, const size_t offset, const size_t count, char* data) { std::shared_ptr slot = GetDBSlotById(db_name, slot_id); if (!slot) { LOG(WARNING) << "cannot find slot for db_name " << db_name << "slot_id: " << slot_id; - return 0; + return pstd::Status::NotFound("slot no found"); } const std::string filepath = slot->bgsave_info().path + "/" + filename; - int fd = open(filepath.c_str(), O_RDONLY, 0644); - ssize_t n = pread(fd, data, count, offset); - close(fd); - return n; + int fd = open(filepath.c_str(), O_RDONLY); + if (fd < 0) { + return Status::IOError("fd open failed"); + } + DEFER { close(fd); }; + + const int kMaxCopyBlockSize = 8 << 10; + size_t read_offset = offset; + size_t read_count = count; + if (read_count > kMaxCopyBlockSize) { + read_count = kMaxCopyBlockSize; + } + + size_t bytesin = 0; + size_t left_read_count = count; + + while ((bytesin = pread(fd, data, read_count, read_offset)) > 0) { + left_read_count -= bytesin; + if (left_read_count <= 0) { + break ; + } + if (read_count > left_read_count) { + read_count = left_read_count; + } + + data += bytesin; + read_offset += bytesin; + } + + if (bytesin == -1) { + LOG(ERROR) << "unable to read from " << filename; + return pstd::Status::IOError("unable to read from " + filename); + } + + return pstd::Status::OK(); } void PikaServer::TryDBSync(const std::string& ip, int port, const std::string& db_name, uint32_t slot_id, diff --git a/src/pika_slot.cc b/src/pika_slot.cc index 1970b1c54..58abc0a9e 100644 --- a/src/pika_slot.cc +++ b/src/pika_slot.cc @@ -13,6 +13,7 @@ #include "include/pika_server.h" #include "pstd/include/mutex_impl.h" +#include "pstd_hash.h" using pstd::Status; @@ -296,6 +297,28 @@ BgSaveInfo Slot::bgsave_info() { return bgsave_info_; } +void Slot::GetBgSaveMetaData(std::vector* fileNames, std::string* snapshot_uuid) { + const std::string dbFilePath = bgsave_info().path; + // todo 待确认 info 文件的路径 + const std::string infoFilePath = bgsave_info().path + "/../info"; + + int ret = pstd::GetChildren(dbFilePath, *fileNames); + if (ret) { + LOG(WARNING) << dbFilePath << " read dump meta files failed! error:" << ret; + return; + } + + std::string info_data; + rocksdb::Status s = rocksdb::ReadFileToString(rocksdb::Env::Default(), infoFilePath, &info_data); + if (!s.ok()) { + LOG(WARNING) << "read dump meta info failed! error:" << s.ToString(); + return; + } + + pstd::MD5 md5 = pstd::MD5(info_data); + *snapshot_uuid = md5.hexdigest(); +} + void Slot::DoBgSave(void* arg) { std::unique_ptr bg_task_arg(static_cast(arg)); From 7bac3362098698ab842f99a559f192bfedc02d08 Mon Sep 17 00:00:00 2001 From: wangshao1 <30471730+wangshao1@users.noreply.github.com> Date: Fri, 14 Jul 2023 16:23:20 +0800 Subject: [PATCH 10/37] fix compile error (#183) * fix bug * add rsyncclient to syncslaveslot * fix compile error * fix compile error --------- Co-authored-by: wangshaoyi --- include/pika_rm.h | 1 + include/rsync_client.h | 3 ++- include/rsync_server.h | 5 +---- src/pika_rm.cc | 2 +- src/rsync_server.cc | 5 ++--- 5 files changed, 7 insertions(+), 9 deletions(-) diff --git a/include/pika_rm.h b/include/pika_rm.h index 7211f77d7..61821eab5 100644 --- a/include/pika_rm.h +++ b/include/pika_rm.h @@ -21,6 +21,7 @@ #include "include/pika_repl_server.h" #include "include/pika_slave_node.h" #include "include/pika_stable_log.h" +#include "include/rsync_client.h" #define kBinlogSendPacketNum 40 #define kBinlogSendBatchNum 100 diff --git a/include/rsync_client.h b/include/rsync_client.h index b82990545..3e636a60b 100644 --- a/include/rsync_client.h +++ b/include/rsync_client.h @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -46,7 +47,7 @@ class RsyncClient : public net::Thread { bool Init(const std::string& local_ip); Status Start(); Status Stop(); - State State() { return state_;} + bool IsRunning() { return state_.load() == RUNNING;} void OnReceive(RsyncResponse* resp); private: bool Recover(); diff --git a/include/rsync_server.h b/include/rsync_server.h index 0f3550e47..f87a1df8d 100644 --- a/include/rsync_server.h +++ b/include/rsync_server.h @@ -33,8 +33,7 @@ class RsyncServerThread; class RsyncServer { public: - RsyncServer(const std::string& ip_port, const int port, void* worker_specific_data, - const std::string& dir); + RsyncServer(const std::string& ip_port, const int port); ~RsyncServer(); void Schedule(net::TaskFunc func, void* arg); int Start(); @@ -42,8 +41,6 @@ class RsyncServer { private: int port_; std::string ip_; - std::string dir_; - std::string info_dir_; std::map > file_map_; std::unique_ptr work_thread_ = nullptr; std::unique_ptr rsync_server_thread_ = nullptr; diff --git a/src/pika_rm.cc b/src/pika_rm.cc index 90f602456..16cb2d1ab 100644 --- a/src/pika_rm.cc +++ b/src/pika_rm.cc @@ -639,7 +639,7 @@ std::string SyncSlaveSlot::LocalIp() { } void SyncSlaveSlot::ActivateRsync() { - if (rsync_cli_->State() == RsyncCient::RUNNING) { + if (rsync_cli_->IsRunning()) { return; } if (rsync_cli_->Init(local_ip_)) { diff --git a/src/rsync_server.cc b/src/rsync_server.cc index ca7b80424..c07e1ae6c 100644 --- a/src/rsync_server.cc +++ b/src/rsync_server.cc @@ -3,7 +3,7 @@ #include #include "include/pika_server.h" -//extern PikaServer* g_pika_server; +// extern PikaServer* g_pika_server; namespace rsync { //TODO: mock code, need removed @@ -34,8 +34,7 @@ ssize_t ReadDumpFile(const std::string& db_name, uint32_t slot_id, const std::st return n; } -RsyncServer::RsyncServer(const std::string& ip, const int port, void* worker_specific_data, - const std::string& dir) : dir_(dir), ip_(ip), port_(port) { +RsyncServer::RsyncServer(const std::string& ip, const int port) : ip_(ip), port_(port) { work_thread_ = std::make_unique(2, 100000); std::set ips = {ip_}; rsync_server_thread_ = std::make_unique(ips, port, 60 * 1000, this); From b2f4091897b28fd323b0d8d6d963ed00103c2635 Mon Sep 17 00:00:00 2001 From: wangshao1 <30471730+wangshao1@users.noreply.github.com> Date: Fri, 14 Jul 2023 16:40:26 +0800 Subject: [PATCH 11/37] fix compile error (#184) * fix bug --- include/pika_rm.h | 2 +- include/rsync_client.h | 1 + src/pika_rm.cc | 8 +++++--- src/rsync_client.cc | 8 ++++---- 4 files changed, 11 insertions(+), 8 deletions(-) diff --git a/include/pika_rm.h b/include/pika_rm.h index 61821eab5..0d1c14755 100644 --- a/include/pika_rm.h +++ b/include/pika_rm.h @@ -161,7 +161,7 @@ class SyncSlaveSlot : public SyncSlot { void ActivateRsync(); private: - std::unique_ptr sync_cli_; + std::unique_ptr rsync_cli_; pstd::Mutex slot_mu_; RmNode m_info_; ReplState repl_state_{kNoConnect}; diff --git a/include/rsync_client.h b/include/rsync_client.h index 3e636a60b..57999717d 100644 --- a/include/rsync_client.h +++ b/include/rsync_client.h @@ -17,6 +17,7 @@ #include "net/include/net_cli.h" #include "pstd/include/env.h" #include "pstd/include/pstd_hash.h" +#include "pstd/include/pstd_string.h" #include "pstd/include/pstd_status.h" #include "rsync_service.pb.h" #include "throttle.h" diff --git a/src/pika_rm.cc b/src/pika_rm.cc index 16cb2d1ab..d2663ee21 100644 --- a/src/pika_rm.cc +++ b/src/pika_rm.cc @@ -538,7 +538,9 @@ Status SyncMasterSlot::ConsensusReset(const LogOffset& applied_offset) { return /* SyncSlaveSlot */ SyncSlaveSlot::SyncSlaveSlot(const std::string& db_name, uint32_t slot_id) : SyncSlot(db_name, slot_id) { - rsync_cli_.reset(new rsync::RsyncClient(dbsync_path_, db_name, slot_id)); + //TODO: get dbsync_path from slot class + std::string dbsync_path = g_pika_conf->db_sync_path() + "/" + db_name; + rsync_cli_.reset(new rsync::RsyncClient(dbsync_path, db_name, slot_id)); m_info_.SetLastRecvTime(pstd::NowMicros()); } @@ -643,7 +645,7 @@ void SyncSlaveSlot::ActivateRsync() { return; } if (rsync_cli_->Init(local_ip_)) { - rsnyc_cli_->Start(); + rsync_cli_->Start(); } } @@ -1152,7 +1154,7 @@ Status PikaReplicaManager::RunSyncSlaveSlotStateMachine() { std::shared_ptr slot = g_pika_server->GetDBSlotById(p_info.db_name_, p_info.slot_id_); if (slot) { - slot->ActivateRsync(); + s_slot->ActivateRsync(); slot->TryUpdateMasterOffset(); } else { LOG(WARNING) << "Slot not found, DB Name: " << p_info.db_name_ diff --git a/src/rsync_client.cc b/src/rsync_client.cc index d7833f230..c487372e5 100644 --- a/src/rsync_client.cc +++ b/src/rsync_client.cc @@ -10,13 +10,13 @@ using namespace RsyncService; using namespace pstd; namespace rsync { -RsyncClient::RsyncClient(const std::string& dir, const std::string& db_name, const size_t slot_id) - : dir_ (dir), flush_period_(100), db_name_(ip), slot_id_(port), state_(IDLE), - db_name_(""), slot_id_(0), max_retries_(10) { +RsyncClient::RsyncClient(const std::string& dir, const std::string& db_name, const uint32_t slot_id) + : dir_ (dir), flush_period_(100), db_name_(db_name), slot_id_(slot_id), state_(IDLE), + max_retries_(10) { client_thread_ = std::make_unique(10 * 1000, 60 * 1000, this); } -bool RsyncClient::Init(const std::string ip_port) { +bool RsyncClient::Init(const std::string& ip_port) { if (!ParseIpPortString(ip_port, ip_, port_)) { LOG(WARNING) << "Parse ip_port error " << ip_port; return false; From 1b4b9796db577fa6b528fde64bbbaf51ef8968a3 Mon Sep 17 00:00:00 2001 From: chejinge <945997690@qq.com> Date: Fri, 14 Jul 2023 19:45:13 +0800 Subject: [PATCH 12/37] optimize: add_throttle (#189) optimize throttle --- include/rsync_client.h | 3 ++- include/throttle.h | 25 +++++++++++++------------ src/rsync_client.cc | 10 +++++++--- 3 files changed, 22 insertions(+), 16 deletions(-) diff --git a/include/rsync_client.h b/include/rsync_client.h index 57999717d..3baf53f6e 100644 --- a/include/rsync_client.h +++ b/include/rsync_client.h @@ -20,7 +20,7 @@ #include "pstd/include/pstd_string.h" #include "pstd/include/pstd_status.h" #include "rsync_service.pb.h" -#include "throttle.h" +#include "include/throttle.h" using namespace pstd; using namespace net; @@ -82,6 +82,7 @@ class RsyncClient : public net::Thread { std::list resp_list_; std::condition_variable cond_; std::mutex mu_; + std::unique_ptr throttle_; }; //TODO: jinge diff --git a/include/throttle.h b/include/throttle.h index 0f61084db..b63795bc7 100644 --- a/include/throttle.h +++ b/include/throttle.h @@ -6,20 +6,21 @@ namespace rsync { class Throttle { public: - Throttle(size_t throttle_throughput_bytes, size_t check_cycle); - ~Throttle(); - size_t ThrottledByThroughput(size_t bytes); - void ReturnUnusedThroughput(size_t acquired, size_t consumed, size_t elaspe_time_us); + Throttle(); + Throttle(size_t throttle_throughput_bytes, size_t check_cycle); + ~Throttle(); + size_t ThrottledByThroughput(size_t bytes); + void ReturnUnusedThroughput(size_t acquired, size_t consumed, size_t elaspe_time_us); private: - std::atomic throttle_throughput_bytes_; - // the num of tasks doing install_snapshot - std::atomic last_throughput_check_time_us_; - std::atomic cur_throughput_bytes_; - // user defined check cycles of throughput per second - size_t check_cycle_{}; - pstd::Mutex keys_mutex_; - size_t caculate_check_time_us_(int64_t current_time_us, int64_t check_cycle) { + std::atomic throttle_throughput_bytes_ = 100 * 1024 * 1024; + // the num of tasks doing install_snapshot + std::atomic last_throughput_check_time_us_; + std::atomic cur_throughput_bytes_; + // user defined check cycles of throughput per second + size_t check_cycle_ = 10; + pstd::Mutex keys_mutex_; + size_t caculate_check_time_us_(int64_t current_time_us, int64_t check_cycle) { size_t base_aligning_time_us = 1000 * 1000 / check_cycle; return current_time_us / base_aligning_time_us * base_aligning_time_us; } diff --git a/src/rsync_client.cc b/src/rsync_client.cc index c487372e5..3196d9ce4 100644 --- a/src/rsync_client.cc +++ b/src/rsync_client.cc @@ -112,7 +112,8 @@ Status RsyncClient::CopyRemoteFile(const std::string& filename) { Status s; int retries = 0; size_t offset = 0; - size_t count = 1 << 10; + size_t copy_file_begin_time = pstd::NowMicros(); + size_t count = throttle_->ThrottledByThroughput(1024 * 1024); MD5 md5; std::unique_ptr writer(new RsyncWriter(dir_ + "/" + filename)); DEFER { @@ -132,7 +133,7 @@ Status RsyncClient::CopyRemoteFile(const std::string& filename) { FileRequest* file_req = request.mutable_file_req(); file_req->set_filename(filename); file_req->set_offset(offset); - file_req->set_count(1 << 20); + file_req->set_count(count); std::string to_send; request.SerializeToString(&to_send); @@ -193,6 +194,9 @@ Status RsyncClient::CopyRemoteFile(const std::string& filename) { } else { offset += resp->file_resp().count(); } + size_t copy_file_end_time = pstd::NowMicros(); + size_t elaspe_time_us = copy_file_end_time - copy_file_begin_time; + throttle_->ReturnUnusedThroughput(count, ret_count, elaspe_time_us); retries = 0; } @@ -327,4 +331,4 @@ Status RsyncClient::FlushMetaTable() { return Status::OK(); } -} // end namespace rsync \ No newline at end of file +} // end namespace rsync From fa96b9dd5fd0446d9409f5c311dc114b3d8cc460 Mon Sep 17 00:00:00 2001 From: wangshao1 <30471730+wangshao1@users.noreply.github.com> Date: Fri, 14 Jul 2023 20:28:51 +0800 Subject: [PATCH 13/37] rsyncclient periodically flush meta table (#192) rsyncclient periodically flush meta table --- include/pika_server.h | 2 ++ include/rsync_client.h | 3 +-- src/pika_server.cc | 4 ++++ src/rsync_client.cc | 21 ++++++++++++--------- src/rsync_server.cc | 2 +- 5 files changed, 20 insertions(+), 12 deletions(-) diff --git a/include/pika_server.h b/include/pika_server.h index 9e856948e..8e810655b 100644 --- a/include/pika_server.h +++ b/include/pika_server.h @@ -35,6 +35,7 @@ #include "include/pika_repl_client.h" #include "include/pika_repl_server.h" #include "include/pika_rsync_service.h" +#include "include/rsync_server.h" #include "include/pika_statistic.h" #include "include/pika_slot_command.h" #include "include/pika_migrate_thread.h" @@ -554,6 +555,7 @@ class PikaServer : public pstd::noncopyable { * Rsync used */ std::unique_ptr pika_rsync_service_; + std::unique_ptr rsync_server_; /* * Pubsub used diff --git a/include/rsync_client.h b/include/rsync_client.h index 3baf53f6e..7c4ec1e75 100644 --- a/include/rsync_client.h +++ b/include/rsync_client.h @@ -50,6 +50,7 @@ class RsyncClient : public net::Thread { Status Stop(); bool IsRunning() { return state_.load() == RUNNING;} void OnReceive(RsyncResponse* resp); + private: bool Recover(); Status Wait(WaitObject* wo); @@ -60,8 +61,6 @@ class RsyncClient : public net::Thread { void HandleRsyncMetaResponse(RsyncResponse* response); private: - //已经下载完成的文件名与checksum值,用于宕机重启时恢复, - //减少重复文件下载,周期性flush到磁盘上 std::map meta_table_; int flush_period_; //待拉取的文件集合 diff --git a/src/pika_server.cc b/src/pika_server.cc index d67a6d23e..85ade40a4 100644 --- a/src/pika_server.cc +++ b/src/pika_server.cc @@ -75,6 +75,8 @@ PikaServer::PikaServer() pika_dispatch_thread_ = std::make_unique(ips, port_, worker_num_, 3000, worker_queue_limit, g_pika_conf->max_conn_rbuf_size()); pika_rsync_service_ = std::make_unique(g_pika_conf->db_sync_path(), g_pika_conf->port() + kPortShiftRSync); + //TODO 删除pika_rsync_service_服务,使用pika_rsync_service_端口 + rsync_server_ = std::make_unique("127.0.0.1", g_pika_conf->port() + kPortShiftRSync + 1); pika_pubsub_thread_ = std::make_unique(); pika_auxiliary_thread_ = std::make_unique(); pika_migrate_ = std::make_unique(); @@ -85,6 +87,7 @@ PikaServer::PikaServer() } PikaServer::~PikaServer() { + rsync_server_->Stop(); // DispatchThread will use queue of worker thread, // so we need to delete dispatch before worker. pika_client_processor_->Stop(); @@ -182,6 +185,7 @@ void PikaServer::Start() { } LOG(INFO) << "Pika Server going to start"; + rsync_server_->Start(); while (!exit_) { DoTimingTask(); // wake up every 5 seconds diff --git a/src/rsync_client.cc b/src/rsync_client.cc index 3196d9ce4..3dde06fb3 100644 --- a/src/rsync_client.cc +++ b/src/rsync_client.cc @@ -34,6 +34,11 @@ void* RsyncClient::ThreadMain() { int cnt = 0; int period = 0; Status s = Status::OK(); + std::string meta_file_path = dir_ + "/" + kDumpMetaFileName; + int meta_fd = open(meta_file_path.c_str(), O_CREAT | O_RDWR, 0644); + std::string meta_rep(kUuidPrefix); + meta_rep.append(snapshot_uuid_); + meta_rep.append("\n"); while (state_.load(std::memory_order_relaxed) == RUNNING) { for (const auto& file : file_set_) { LOG(INFO) << "CopyRemoteFile: " << file; @@ -51,7 +56,11 @@ void* RsyncClient::ThreadMain() { } if (++period == flush_period_) { period = 0; - FlushMetaTable(); + meta_rep.append(file + ":" + meta_table_[file]); + meta_rep.append("\n"); + write(meta_fd, meta_rep.data(), meta_rep.size()); + fsync(meta_fd); + meta_rep.clear(); } } if (meta_table_.size() == file_set_.size()) { @@ -60,6 +69,7 @@ void* RsyncClient::ThreadMain() { break; } } + close(meta_fd); return nullptr; } @@ -189,7 +199,7 @@ Status RsyncClient::CopyRemoteFile(const std::string& filename) { return s; } writer.reset(); - meta_table_.insert(std::make_pair(filename, resp->file_resp().checksum())); + meta_table_[filename] = resp->file_resp().checksum(); break; } else { offset += resp->file_resp().count(); @@ -324,11 +334,4 @@ Status RsyncClient::LoadMetaTable() { } return Status::OK(); } - -//TODO: shaoyi -Status RsyncClient::FlushMetaTable() { - LOG(WARNING) << "FlushMetaTable called"; - return Status::OK(); -} - } // end namespace rsync diff --git a/src/rsync_server.cc b/src/rsync_server.cc index c07e1ae6c..903d0ce00 100644 --- a/src/rsync_server.cc +++ b/src/rsync_server.cc @@ -3,7 +3,7 @@ #include #include "include/pika_server.h" -// extern PikaServer* g_pika_server; +extern PikaServer* g_pika_server; namespace rsync { //TODO: mock code, need removed From 84e69dad14b60a0477c3614ee5b486689d473024 Mon Sep 17 00:00:00 2001 From: Yuecai Liu <38887641+luky116@users.noreply.github.com> Date: Mon, 17 Jul 2023 18:14:27 +0800 Subject: [PATCH 14/37] change rsync response (#190) * change rsync response --- include/pika_define.h | 2 + include/pika_server.h | 2 +- include/rsync_client.h | 10 +- src/pika.cc | 2 +- src/pika_rm.cc | 2 +- src/pika_server.cc | 7 +- src/pika_slot.cc | 29 ++- src/rsync_client.cc | 479 +++++++++++++++++++++++++---------------- src/rsync_server.cc | 47 +--- 9 files changed, 339 insertions(+), 241 deletions(-) diff --git a/include/pika_define.h b/include/pika_define.h index 2a09a995e..123192136 100644 --- a/include/pika_define.h +++ b/include/pika_define.h @@ -34,6 +34,8 @@ class PikaServer; /* Port shift */ const int kPortShiftRSync = 1000; const int kPortShiftReplServer = 2000; +// todo 待移除,使用 kPortShiftRSync +const int kPortShiftRsync2 = 10001; const std::string kPikaPidFile = "pika.pid"; const std::string kPikaSecretFile = "rsync.secret"; diff --git a/include/pika_server.h b/include/pika_server.h index 8e810655b..fcd17c3a2 100644 --- a/include/pika_server.h +++ b/include/pika_server.h @@ -262,7 +262,7 @@ class PikaServer : public pstd::noncopyable { */ //TODO: yuecai pstd::Status ReadDumpFile(const std::string& db_name, uint32_t slot_id, const std::string& filename, - const size_t offset, const size_t count, char* data); + const size_t offset, const size_t count, char* data, size_t* bytes_read); //TODO: yuecai pstd::Status GetDumpMeta(const std::string& db_name, const uint32_t slot_id, std::vector* files, std::string* snapshot_uuid); void DBSync(const std::string& ip, int port, const std::string& db_name, uint32_t slot_id); diff --git a/include/rsync_client.h b/include/rsync_client.h index 7c4ec1e75..9a51d294a 100644 --- a/include/rsync_client.h +++ b/include/rsync_client.h @@ -45,7 +45,7 @@ class RsyncClient : public net::Thread { RsyncClient(const std::string& dir, const std::string& db_name, const uint32_t slot_id); void* ThreadMain() override; - bool Init(const std::string& local_ip); + bool Init(); Status Start(); Status Stop(); bool IsRunning() { return state_.load() == RUNNING;} @@ -56,8 +56,11 @@ class RsyncClient : public net::Thread { Status Wait(WaitObject* wo); Status CopyRemoteFile(const std::string& filename); Status CopyRemoteMeta(std::string* snapshot_uuid, std::set* file_set); - Status LoadMetaTable(); + Status LoadLocalMeta(std::string* snapshot_uuid, std::map* file_map); + std::string GetLocalMetaFilePath(); Status FlushMetaTable(); + Status CleanUpExpiredFiles(bool need_reset_path, std::set files); + Status UpdateLocalMeta(std::string& snapshot_uuid, std::set& expired_files, std::map& localFileMap); void HandleRsyncMetaResponse(RsyncResponse* response); private: @@ -68,9 +71,6 @@ class RsyncClient : public net::Thread { std::string snapshot_uuid_; std::string dir_; - std::string ip_; - int port_; - std::string db_name_; uint32_t slot_id_; diff --git a/src/pika.cc b/src/pika.cc index 17bc22d67..6f220e922 100644 --- a/src/pika.cc +++ b/src/pika.cc @@ -14,8 +14,8 @@ #include "include/pika_define.h" #include "include/pika_rm.h" #include "include/pika_server.h" -#include "include/pika_version.h" #include "include/pika_slot_command.h" +#include "include/pika_version.h" #include "pstd/include/env.h" #include "pstd/include/pstd_defer.h" diff --git a/src/pika_rm.cc b/src/pika_rm.cc index d2663ee21..4c96a76be 100644 --- a/src/pika_rm.cc +++ b/src/pika_rm.cc @@ -644,7 +644,7 @@ void SyncSlaveSlot::ActivateRsync() { if (rsync_cli_->IsRunning()) { return; } - if (rsync_cli_->Init(local_ip_)) { + if (rsync_cli_->Init()) { rsync_cli_->Start(); } } diff --git a/src/pika_server.cc b/src/pika_server.cc index 85ade40a4..0cdbbe4a0 100644 --- a/src/pika_server.cc +++ b/src/pika_server.cc @@ -76,7 +76,7 @@ PikaServer::PikaServer() std::make_unique(ips, port_, worker_num_, 3000, worker_queue_limit, g_pika_conf->max_conn_rbuf_size()); pika_rsync_service_ = std::make_unique(g_pika_conf->db_sync_path(), g_pika_conf->port() + kPortShiftRSync); //TODO 删除pika_rsync_service_服务,使用pika_rsync_service_端口 - rsync_server_ = std::make_unique("127.0.0.1", g_pika_conf->port() + kPortShiftRSync + 1); + rsync_server_ = std::make_unique("127.0.0.1", g_pika_server->master_port() + kPortShiftRsync2); pika_pubsub_thread_ = std::make_unique(); pika_auxiliary_thread_ = std::make_unique(); pika_migrate_ = std::make_unique(); @@ -918,10 +918,12 @@ pstd::Status PikaServer::GetDumpMeta(const std::string& db_name, const uint32_t return pstd::Status::NotFound("slot no found"); } slot->GetBgSaveMetaData(fileNames, snapshot_uuid); + return pstd::Status::OK(); } +// todo 参数太长了,待优化 pstd::Status PikaServer::ReadDumpFile(const std::string& db_name, uint32_t slot_id, const std::string& filename, - const size_t offset, const size_t count, char* data) { + const size_t offset, const size_t count, char* data, size_t* bytes_read) { std::shared_ptr slot = GetDBSlotById(db_name, slot_id); if (!slot) { LOG(WARNING) << "cannot find slot for db_name " << db_name << "slot_id: " << slot_id; @@ -954,6 +956,7 @@ pstd::Status PikaServer::ReadDumpFile(const std::string& db_name, uint32_t slot_ } data += bytesin; + *bytes_read += bytesin; read_offset += bytesin; } diff --git a/src/pika_slot.cc b/src/pika_slot.cc index 58abc0a9e..184316c8d 100644 --- a/src/pika_slot.cc +++ b/src/pika_slot.cc @@ -143,6 +143,7 @@ void Slot::PrepareRsync() { // 3, Update master offset, and the PikaAuxiliaryThread cron will connect and do slaveof task with master bool Slot::TryUpdateMasterOffset() { std::string info_path = dbsync_path_ + kBgsaveInfoFile; + // todo 这里要改动,定期向 master 发送 meta_rsync 的请求 if (!pstd::FileExists(info_path)) { LOG(WARNING) << "info path: " << info_path << " not exist"; return false; @@ -298,18 +299,32 @@ BgSaveInfo Slot::bgsave_info() { } void Slot::GetBgSaveMetaData(std::vector* fileNames, std::string* snapshot_uuid) { - const std::string dbFilePath = bgsave_info().path; + const std::string slotPath = bgsave_info().path; // todo 待确认 info 文件的路径 - const std::string infoFilePath = bgsave_info().path + "/../info"; + const std::string infoPath = bgsave_info().path + "/info"; - int ret = pstd::GetChildren(dbFilePath, *fileNames); - if (ret) { - LOG(WARNING) << dbFilePath << " read dump meta files failed! error:" << ret; - return; + std::string types[] = {storage::STRINGS_DB, storage::HASHES_DB, storage::LISTS_DB, storage::ZSETS_DB, storage::SETS_DB}; + for (const auto& type : types) { + std::string typePath = slotPath + ((slotPath.back() != '/') ? "/" : "") + type; + if (!pstd::FileExists(typePath)) { + continue ; + } + + std::vector tmpFileNames; + int ret = pstd::GetChildren(typePath, tmpFileNames); + if (ret) { + LOG(WARNING) << slotPath << " read dump meta files failed, path " << typePath; + return; + } + + for (const std::string fileName : tmpFileNames) { + fileNames -> push_back(type + "/" + fileName); + } } std::string info_data; - rocksdb::Status s = rocksdb::ReadFileToString(rocksdb::Env::Default(), infoFilePath, &info_data); + // todo 这里待替换 + rocksdb::Status s = rocksdb::ReadFileToString(rocksdb::Env::Default(), infoPath, &info_data); if (!s.ok()) { LOG(WARNING) << "read dump meta info failed! error:" << s.ToString(); return; diff --git a/src/rsync_client.cc b/src/rsync_client.cc index 3dde06fb3..0975080ba 100644 --- a/src/rsync_client.cc +++ b/src/rsync_client.cc @@ -1,33 +1,34 @@ #include "include/rsync_client.h" +#include +#include "include/pika_server.h" #include "pstd/include/pstd_defer.h" #include "pstd/src/env.cc" -#include -#include +#include "rocksdb/env.h" using namespace net; using namespace pstd; using namespace RsyncService; - using namespace pstd; + +extern PikaServer* g_pika_server; + namespace rsync { RsyncClient::RsyncClient(const std::string& dir, const std::string& db_name, const uint32_t slot_id) - : dir_ (dir), flush_period_(100), db_name_(db_name), slot_id_(slot_id), state_(IDLE), - max_retries_(10) { - client_thread_ = std::make_unique(10 * 1000, 60 * 1000, this); + : dir_(dir), flush_period_(100), db_name_(db_name), slot_id_(slot_id), state_(IDLE), max_retries_(10) { + client_thread_ = std::make_unique(10 * 1000, 60 * 1000, this); } -bool RsyncClient::Init(const std::string& ip_port) { - if (!ParseIpPortString(ip_port, ip_, port_)) { - LOG(WARNING) << "Parse ip_port error " << ip_port; - return false; - } - client_thread_->StartThread(); - bool ret = Recover(); - if (!ret) { - client_thread_->StopThread(); - return false; - } - return true; +bool RsyncClient::Init() { + // todo client 的 StartThread 只能被调用一次,如果一个 slot 进行多次主从同步,这里会出问题吗? + client_thread_->StartThread(); + bool ret = Recover(); + if (!ret) { + LOG(WARNING) << "RsyncClient recover failed..."; + client_thread_->StopThread(); + return false; + } + LOG(INFO) << "RsyncClient recover success..."; + return true; } void* RsyncClient::ThreadMain() { @@ -74,48 +75,45 @@ void* RsyncClient::ThreadMain() { } void RsyncClient::OnReceive(RsyncResponse* resp) { - std::unique_lock lock(mu_); - resp_list_.push_back(resp); - cond_.notify_all(); + std::unique_lock lock(mu_); + resp_list_.push_back(resp); + cond_.notify_all(); } Status RsyncClient::Wait(WaitObject* wo) { - Status s = Status::Timeout("rsync timeout", "timeout"); - std::list resp_list; - { - std::unique_lock lock(mu_); - cond_.wait_for(lock, std::chrono::seconds(3), [this]{ - return !resp_list_.empty();} - ); - resp_list.swap(resp_list_); - } + Status s = Status::Timeout("rsync timeout", "timeout"); + std::list resp_list; + { + std::unique_lock lock(mu_); + cond_.wait_for(lock, std::chrono::seconds(3), [this] { return !resp_list_.empty(); }); + resp_list.swap(resp_list_); + } - auto iter = resp_list.begin(); - while (iter != resp_list.end()) { - RsyncResponse* resp = *iter; - if (resp->type() != wo->type_) { - LOG(WARNING) << "mismatch request/response type, skip"; - iter++; - continue; - } - if (resp->type() == kRsyncFile && - (resp->file_resp().filename() != wo->filename_ || - resp->file_resp().offset() != wo->offset_)) { - LOG(WARNING) << "mismatch rsync response, skip"; - continue; - } - s = Status::OK(); - wo->resp_ = resp; - resp_list.erase(iter); - break; + auto iter = resp_list.begin(); + while (iter != resp_list.end()) { + RsyncResponse* resp = *iter; + if (resp->type() != wo->type_) { + LOG(WARNING) << "mismatch request/response type, skip"; + iter++; + continue; } - - iter = resp_list.begin(); - while (iter != resp_list.end()) { - delete (*iter); - iter++; + if (resp->type() == kRsyncFile && + (resp->file_resp().filename() != wo->filename_ || resp->file_resp().offset() != wo->offset_)) { + LOG(WARNING) << "mismatch rsync response, skip"; + continue; } - return s; + s = Status::OK(); + wo->resp_ = resp; + resp_list.erase(iter); + break; + } + + iter = resp_list.begin(); + while (iter != resp_list.end()) { + delete (*iter); + iter++; + } + return s; } Status RsyncClient::CopyRemoteFile(const std::string& filename) { @@ -147,41 +145,40 @@ Status RsyncClient::CopyRemoteFile(const std::string& filename) { std::string to_send; request.SerializeToString(&to_send); - s = client_thread_->Write(ip_, port_, to_send); - if (!s.ok()) { - LOG(WARNING) << "send rsync request failed"; - continue; - } + s = client_thread_->Write(g_pika_server->master_ip(), g_pika_server->master_port() + kPortShiftRsync2, to_send); + if (!s.ok()) { + LOG(WARNING) << "send rsync request failed"; + continue; + } - WaitObject wo(filename, kRsyncFile, offset); - LOG(INFO) << "wait CopyRemoteFile response....."; - s = Wait(&wo); - if (s.IsTimeout() || wo.resp_ == nullptr) { - LOG(WARNING) << "rsync request timeout"; - retries++; - continue; - } - RsyncResponse* resp = wo.resp_; - - LOG(INFO) << "receive fileresponse, snapshot_uuid: " << resp->snapshot_uuid() - << "filename: " << resp->file_resp().filename() - << "offset: " << resp->file_resp().offset() - << "count: " << resp->file_resp().count(); - - if (resp->snapshot_uuid() != snapshot_uuid_) { - LOG(WARNING) << "receive newer dump, reset state to STOP"; - state_.store(STOP); - delete resp; - return s; - } + WaitObject wo(filename, kRsyncFile, offset); + LOG(INFO) << "wait CopyRemoteFile response....."; + s = Wait(&wo); + if (s.IsTimeout() || wo.resp_ == nullptr) { + LOG(WARNING) << "rsync request timeout"; + retries++; + continue; + } + RsyncResponse* resp = wo.resp_; - size_t ret_count = resp->file_resp().count(); - resp->file_resp().data(); - s = writer->Write((uint64_t)offset, ret_count, resp->file_resp().data().c_str()); - if (!s.ok()) { - LOG(WARNING) << "rsync client write file error"; - break; - } + LOG(INFO) << "receive fileresponse, snapshot_uuid: " << resp->snapshot_uuid() + << "filename: " << resp->file_resp().filename() << "offset: " << resp->file_resp().offset() + << "count: " << resp->file_resp().count(); + + if (resp->snapshot_uuid() != snapshot_uuid_) { + LOG(WARNING) << "receive newer dump, reset state to STOP"; + state_.store(STOP); + delete resp; + return s; + } + + size_t ret_count = resp->file_resp().count(); + resp->file_resp().data(); + s = writer->Write((uint64_t)offset, ret_count, resp->file_resp().data().c_str()); + if (!s.ok()) { + LOG(WARNING) << "rsync client write file error"; + break; + } md5.update(resp->file_resp().data().c_str(), ret_count); if (resp->file_resp().eof()) { @@ -210,128 +207,234 @@ Status RsyncClient::CopyRemoteFile(const std::string& filename) { retries = 0; } - return s; + return s; } Status RsyncClient::Start() { - StartThread(); - return Status::OK(); + StartThread(); + return Status::OK(); } Status RsyncClient::Stop() { - state_ = STOP; - StopThread(); - client_thread_->StopThread(); - JoinThread(); - client_thread_->JoinThread(); - state_ = IDLE; - return Status::OK(); + state_ = STOP; + StopThread(); + client_thread_->StopThread(); + JoinThread(); + client_thread_->JoinThread(); + state_ = IDLE; + return Status::OK(); } -//TODO: shaoyi -Status RsyncClient::CopyRemoteMeta(std::string* snapshot_uuid, std::set* file_set) { - Status s; - int retries = 0; - RsyncRequest request; - request.set_db_name(db_name_); - request.set_slot_id(slot_id_); - request.set_type(kRsyncMeta); - std::string to_send; - request.SerializeToString(&to_send); - while (retries < max_retries_) { - s = client_thread_->Write(ip_, port_, to_send); - if (!s.ok()) { - retries++; - } - WaitObject wo(kRsyncMeta); - s = Wait(&wo); - if (s.IsTimeout() || wo.resp_ == nullptr) { - LOG(WARNING) << "rsync CopyRemoteMeta request timeout, retry times: " << retries; - retries++; - continue; - } - RsyncResponse* resp = wo.resp_; - LOG(INFO) << "receive rsync meta infos, snapshot_uuid: " << resp->snapshot_uuid() - << "files count: " << resp->meta_resp().filenames_size(); - for (int i = 0; i < resp->meta_resp().filenames_size(); i++) { - LOG(INFO) << "file: " << resp->meta_resp().filenames(i); - } - *snapshot_uuid = resp->snapshot_uuid(); - for (int i = 0; i < resp->meta_resp().filenames_size(); i++) { - file_set->insert(resp->meta_resp().filenames(i)); - } - break; - } - return s; -} bool RsyncClient::Recover() { - std::string snapshot_uuid; - std::set file_set; - Status s = CopyRemoteMeta(&snapshot_uuid, &file_set); + std::string remote_snapshot_uuid; + std::set remote_file_set; + std::string local_snapshot_uuid; + std::map local_file_map; + std::set local_file_set; + + Status s = CopyRemoteMeta(&remote_snapshot_uuid, &remote_file_set); + if (!s.ok()) { + LOG(WARNING) << "copy remote meta failed"; + return false; + } + + s = LoadLocalMeta(&local_snapshot_uuid, &local_file_map); + if (!s.ok()) { + LOG(WARNING) << "load local meta failed"; + return false; + } + for (auto const& file : local_file_map) { + local_file_set.insert(file.first); + } + + std::set expired_files; + if (remote_snapshot_uuid != local_snapshot_uuid) { + snapshot_uuid_ = remote_snapshot_uuid; + file_set_ = remote_file_set; + expired_files = local_file_set; + } else { + std::set newly_files; + set_difference(remote_file_set.begin(), remote_file_set.end(), local_file_set.begin(), local_file_set.end(), + inserter(newly_files, newly_files.begin())); + set_difference(local_file_set.begin(), local_file_set.end(), remote_file_set.begin(), remote_file_set.end(), + inserter(expired_files, expired_files.begin())); + file_set_.insert(newly_files.begin(), newly_files.end()); + } + + s = CleanUpExpiredFiles(local_snapshot_uuid != remote_snapshot_uuid, expired_files); + if (!s.ok()) { + LOG(WARNING) << "clean up expired files failed"; + return false; + } + s = UpdateLocalMeta(snapshot_uuid_, expired_files, local_file_map); + if (!s.ok()) { + LOG(WARNING) << "update local meta failed"; + return false; + } + + state_ = RUNNING; + LOG(INFO) << "copy meta data done, slot_id: " << slot_id_ << "snapshot_uuid: " << snapshot_uuid_ + << "file count: " << file_set_.size() << "expired file count: " << expired_files.size() + << ", local file count: " << local_file_set.size() << "remote file count: " << remote_file_set.size() + << "remote snapshot_uuid: " << remote_snapshot_uuid << "local snapshot_uuid: " << local_snapshot_uuid; + return true; +} + +Status RsyncClient::CopyRemoteMeta(std::string* snapshot_uuid, std::set* file_set) { + Status s; + int retries = 0; + RsyncRequest request; + request.set_db_name(db_name_); + request.set_slot_id(slot_id_); + request.set_type(kRsyncMeta); + std::string to_send; + request.SerializeToString(&to_send); + while (retries < max_retries_) { + s = client_thread_->Write(g_pika_server->master_ip(), g_pika_server->master_port() + kPortShiftRsync2, to_send); if (!s.ok()) { - LOG(WARNING) << "copy remote meta failed"; - return false; + retries++; } - //TODO: yuecai 加载本地元信息文件,与master回包内容diff - snapshot_uuid_ = snapshot_uuid; - file_set_.insert(file_set.begin(), file_set.end()); - state_ = RUNNING; - LOG(WARNING) << "copy remote meta done"; - return true; -} + WaitObject wo(kRsyncMeta); + s = Wait(&wo); + if (s.IsTimeout() || wo.resp_ == nullptr) { + LOG(WARNING) << "rsync CopyRemoteMeta request timeout, retry times: " << retries; + retries++; + continue; + } + RsyncResponse* resp = wo.resp_; + LOG(INFO) << "receive rsync meta infos, snapshot_uuid: " << resp->snapshot_uuid() + << "files count: " << resp->meta_resp().filenames_size(); -Status RsyncClient::LoadMetaTable() { - if (!FileExists(dir_)) { - return Status::OK(); + for (std::string item : resp->meta_resp().filenames()) { + file_set->insert(item); + } + *snapshot_uuid = resp->snapshot_uuid(); + for (int i = 0; i < resp->meta_resp().filenames_size(); i++) { + file_set->insert(resp->meta_resp().filenames(i)); } + break; + } + return s; +} - FILE* fp; - char* line = nullptr; - size_t len = 0; - size_t read = 0; - int32_t line_num = 0; +Status RsyncClient::LoadLocalMeta(std::string* snapshot_uuid, std::map* file_map) { + std::string meta_file_path = GetLocalMetaFilePath(); + if (!FileExists(meta_file_path)) { + return Status::OK(); + } - std::atomic_int8_t retry_times = 5; + FILE* fp; + char* line = nullptr; + size_t len = 0; + size_t read = 0; + int32_t line_num = 0; - while (retry_times -- > 0) { - fp = fopen(dir_.c_str(), "r"); - if (fp == nullptr) { - LOG(WARNING) << "open meta file failed, meta_path: " << dir_; - } else { - break; - } - } - // if the file cannot be read from disk, use the remote file directly + std::atomic_int8_t retry_times = 5; + + while (retry_times-- > 0) { + fp = fopen(meta_file_path.c_str(), "r"); if (fp == nullptr) { - LOG(WARNING) << "open meta file failed, meta_path: " << dir_ << ", retry times: " << retry_times; - return Status::IOError("open meta file failed, dir: ", dir_); + LOG(WARNING) << "open meta file failed, meta_path: " << dir_; + } else { + break; } + } + // if the file cannot be read from disk, use the remote file directly + if (fp == nullptr) { + LOG(WARNING) << "open meta file failed, meta_path: " << meta_file_path << ", retry times: " << retry_times; + return Status::IOError("open meta file failed, dir: ", meta_file_path); + } - while ((read = getline(&line, &len, fp)) != -1) { - std::string str(line); - std::string::size_type pos; - while ((pos = str.find("\r")) != std::string::npos) { - str.erase(pos, 1); - } - while ((pos = str.find("\n")) != std::string::npos) { - str.erase(pos, 1); - } - - if (str.empty()) { - continue; - } + while ((read = getline(&line, &len, fp)) != -1) { + std::string str(line); + std::string::size_type pos; + while ((pos = str.find("\r")) != std::string::npos) { + str.erase(pos, 1); + } + while ((pos = str.find("\n")) != std::string::npos) { + str.erase(pos, 1); + } - if (line_num == 0) { - snapshot_uuid_ = str.erase(0, kUuidPrefix.size()); - } else { - if ((pos = str.find(":")) != std::string::npos) { - str.erase(pos, str.size() - pos); - } - file_set_.insert(str); - } + if (str.empty()) { + continue; + } - line_num++; + if (line_num == 0) { + *snapshot_uuid = str.erase(0, kUuidPrefix.size()); + } else { + if ((pos = str.find(":")) != std::string::npos) { + std::string filename = str.substr(0, pos); + std::string shecksum = str.substr(pos + 1, str.size()); + (*file_map)[filename] = shecksum; + } } + + line_num++; + } + return Status::OK(); +} + +Status RsyncClient::CleanUpExpiredFiles(bool need_reset_path, std::set files) { + if (need_reset_path) { + std::string db_path = dir_ + (dir_.back() == '/' ? "" : "/"); + pstd::DeleteDirIfExist(db_path); + pstd::CreatePath(db_path + "strings"); + pstd::CreatePath(db_path + "hashes"); + pstd::CreatePath(db_path + "lists"); + pstd::CreatePath(db_path + "sets"); + pstd::CreatePath(db_path + "zsets"); return Status::OK(); + } + + std::string db_path = dir_ + (dir_.back() == '/' ? "" : "/"); + for (const auto& file : files) { + bool b = pstd::DeleteDirIfExist(db_path + file); + if (!b) { + LOG(WARNING) << "delete file failed, file: " << file; + return Status::IOError("delete file failed"); + } + } + return Status::OK(); } -} // end namespace rsync + +Status RsyncClient::UpdateLocalMeta(std::string& snapshot_uuid, std::set& expired_files, + std::map& localFileMap) { + localFileMap[kUuidPrefix] = snapshot_uuid; + for (const auto& item : expired_files) { + localFileMap.erase(item); + } + + std::string meta_file_path = GetLocalMetaFilePath(); + pstd::DeleteFile(meta_file_path); + + std::unique_ptr file; + pstd::Status s = pstd::NewWritableFile(meta_file_path, file); + if (!s.ok()) { + LOG(WARNING) << "create meta file failed, meta_file_path: " << meta_file_path; + return s; + } + + for (const auto& item : localFileMap) { + std::string line = item.first + item.second + "\n"; + file->Append(line); + } + s = file->Flush(); + if (!s.ok()) { + LOG(WARNING) << "flush meta file failed, meta_file_path: " << meta_file_path; + return s; + } + return Status::OK(); +} + +std::string RsyncClient::GetLocalMetaFilePath() { + std::string db_path = dir_ + (dir_.back() == '/' ? "" : "/"); + return db_path + kDumpMetaFileName; +} + +// TODO: shaoyi +Status RsyncClient::FlushMetaTable() { + LOG(WARNING) << "FlushMetaTable called"; + return Status::OK(); +} + +} // end namespace rsync \ No newline at end of file diff --git a/src/rsync_server.cc b/src/rsync_server.cc index 903d0ce00..c83cfd729 100644 --- a/src/rsync_server.cc +++ b/src/rsync_server.cc @@ -2,38 +2,11 @@ #include "include/rsync_server.h" #include #include "include/pika_server.h" +#include extern PikaServer* g_pika_server; namespace rsync { -//TODO: mock code, need removed -void GetDumpMeta(const std::string& db_name, const uint32_t slot_id, std::vector* files, std::string* snapshot_uuid) { - pstd::GetChildren(".", *files); - auto iter = files->begin(); - while (iter != files->end()) { - if (std::filesystem::is_directory(*iter)) { - iter = files->erase(iter); - continue; - } - iter++; - } - *snapshot_uuid = "demo_snapshot_uuid"; -} - -//TODO: mock code, need removed -ssize_t ReadDumpFile(const std::string& db_name, uint32_t slot_id, const std::string& filename, - const size_t offset, const size_t count, char* data) { - const std::string filepath = std::string("./") + filename; - int fd = open(filepath.c_str(), O_RDONLY, 0644); - ssize_t n = pread(fd, data, count, offset); - LOG(WARNING) << "read n: " << n; - if (n < 0) { - LOG(WARNING) << "pread error, errno:" << strerror(errno); - } - close(fd); - return n; -} - RsyncServer::RsyncServer(const std::string& ip, const int port) : ip_(ip), port_(port) { work_thread_ = std::make_unique(2, 100000); std::set ips = {ip_}; @@ -118,18 +91,19 @@ void RsyncServerConn::HandleMetaRsyncRequest(void* arg) { std::string db_name = req->db_name(); uint32_t slot_id = req->slot_id(); + std::vector filenames; std::string snapshot_uuid; - GetDumpMeta(db_name, slot_id, &filenames, &snapshot_uuid); + g_pika_server->GetDumpMeta(db_name, slot_id, &filenames, &snapshot_uuid); LOG(WARNING) << "snapshot_uuid: " << snapshot_uuid; std::for_each(filenames.begin(), filenames.end(), [](auto& file) { - LOG(WARNING) << "file:" << file; + LOG(WARNING) << "file:" << file; }); //TODO: temporarily mock response RsyncService::MetaResponse* meta_resp = response.mutable_meta_resp(); response.set_snapshot_uuid(snapshot_uuid); for (const auto& filename : filenames) { - meta_resp->add_filenames(filename); + meta_resp->add_filenames(filename); } std::string reply_str; @@ -161,16 +135,17 @@ void RsyncServerConn::HandleFileRsyncRequest(void* arg) { size_t offset = req->file_req().offset(); size_t count = req->file_req().count(); char* buffer = new char[req->file_req().count() + 1]; + size_t bytes_read{0}; LOG(INFO) << "....... ReadDumpFile: " << filename; - auto r = ReadDumpFile(db_name, slot_id, filename, offset, count, buffer); - LOG(INFO) << "ReadDumpFile: " << filename << " read size: " << r; + auto status = g_pika_server -> ReadDumpFile(db_name, slot_id, filename, offset, count, buffer, &bytes_read); + LOG(INFO) << "ReadDumpFile: " << filename << " read size: " << status.ToString(); //TODO: temporarily mock response RsyncService::FileResponse* file_resp = response.mutable_file_resp(); - file_resp->set_eof(r != count); - file_resp->set_count(r); + file_resp->set_eof(bytes_read != count); + file_resp->set_count(bytes_read); file_resp->set_offset(offset); - file_resp->set_data(buffer, r); + file_resp->set_data(buffer, bytes_read); file_resp->set_checksum("checksum"); file_resp->set_filename(filename); From 3fe94e43fe46493cffc96f87a171d7704d5371d5 Mon Sep 17 00:00:00 2001 From: wangshaoyi Date: Tue, 18 Jul 2023 10:57:41 +0800 Subject: [PATCH 15/37] add debug log for test --- include/pika_server.h | 1 + include/pika_slot.h | 2 ++ src/pika_server.cc | 10 +++++++++ src/pika_slot.cc | 30 ++++++++++++++++--------- src/rsync_server.cc | 51 ++++++++++++++++++++++++------------------- 5 files changed, 62 insertions(+), 32 deletions(-) diff --git a/include/pika_server.h b/include/pika_server.h index fcd17c3a2..414b468d1 100644 --- a/include/pika_server.h +++ b/include/pika_server.h @@ -263,6 +263,7 @@ class PikaServer : public pstd::noncopyable { //TODO: yuecai pstd::Status ReadDumpFile(const std::string& db_name, uint32_t slot_id, const std::string& filename, const size_t offset, const size_t count, char* data, size_t* bytes_read); + pstd::Status GetDumpUUID(const std::string& db_name, const uint32_t slot_id, std::string* snapshot_uuid); //TODO: yuecai pstd::Status GetDumpMeta(const std::string& db_name, const uint32_t slot_id, std::vector* files, std::string* snapshot_uuid); void DBSync(const std::string& ip, int port, const std::string& db_name, uint32_t slot_id); diff --git a/include/pika_slot.h b/include/pika_slot.h index c41b906ca..617805c46 100644 --- a/include/pika_slot.h +++ b/include/pika_slot.h @@ -77,6 +77,7 @@ class Slot : public std::enable_shared_from_this,public pstd::noncopyable void BgSaveSlot(); BgSaveInfo bgsave_info(); void GetBgSaveMetaData(std::vector* fileNames, std::string* snapshot_uuid); + pstd::Status GetBgSaveUUID(std::string* snapshot_uuid); // FlushDB & FlushSubDB use bool FlushDB(); @@ -94,6 +95,7 @@ class Slot : public std::enable_shared_from_this,public pstd::noncopyable private: std::string db_name_; uint32_t slot_id_ = 0; + std::string snapshot_uuid_; std::string db_path_; std::string bgsave_sub_path_; diff --git a/src/pika_server.cc b/src/pika_server.cc index 0cdbbe4a0..f27cfb5c8 100644 --- a/src/pika_server.cc +++ b/src/pika_server.cc @@ -911,6 +911,16 @@ void PikaServer::DBSync(const std::string& ip, int port, const std::string& db_n bgsave_thread_.Schedule(&DoDBSync, reinterpret_cast(arg)); } +pstd::Status PikaServer::GetDumpUUID(const std::string& db_name, const uint32_t slot_id, std::string* snapshot_uuid) { + std::shared_ptr slot = GetDBSlotById(db_name, slot_id); + if (!slot) { + LOG(WARNING) << "cannot find slot for db_name " << db_name << "slot_id: " << slot_id; + return pstd::Status::NotFound("slot no found"); + } + slot->GetBgSaveUUID(snapshot_uuid); + return pstd::Status::OK(); +} + pstd::Status PikaServer::GetDumpMeta(const std::string& db_name, const uint32_t slot_id, std::vector* fileNames, std::string* snapshot_uuid) { std::shared_ptr slot = GetDBSlotById(db_name, slot_id); if (!slot) { diff --git a/src/pika_slot.cc b/src/pika_slot.cc index 184316c8d..94e469f31 100644 --- a/src/pika_slot.cc +++ b/src/pika_slot.cc @@ -300,8 +300,6 @@ BgSaveInfo Slot::bgsave_info() { void Slot::GetBgSaveMetaData(std::vector* fileNames, std::string* snapshot_uuid) { const std::string slotPath = bgsave_info().path; - // todo 待确认 info 文件的路径 - const std::string infoPath = bgsave_info().path + "/info"; std::string types[] = {storage::STRINGS_DB, storage::HASHES_DB, storage::LISTS_DB, storage::ZSETS_DB, storage::SETS_DB}; for (const auto& type : types) { @@ -321,17 +319,29 @@ void Slot::GetBgSaveMetaData(std::vector* fileNames, std::string* s fileNames -> push_back(type + "/" + fileName); } } - - std::string info_data; - // todo 这里待替换 - rocksdb::Status s = rocksdb::ReadFileToString(rocksdb::Env::Default(), infoPath, &info_data); + pstd::Status s = GetBgSaveUUID(snapshot_uuid); if (!s.ok()) { - LOG(WARNING) << "read dump meta info failed! error:" << s.ToString(); - return; + LOG(WARNING) << "read dump meta info failed! error:" << s.ToString(); + return; } +} - pstd::MD5 md5 = pstd::MD5(info_data); - *snapshot_uuid = md5.hexdigest(); +Status Slot::GetBgSaveUUID(std::string* snapshot_uuid) { + if (snapshot_uuid_.empty()) { + std::string info_data; + // todo 待确认 info 文件的路径 + const std::string infoPath = bgsave_info().path + "/info"; + // todo 这里待替换 + rocksdb::Status s = rocksdb::ReadFileToString(rocksdb::Env::Default(), infoPath, &info_data); + if (!s.ok()) { + LOG(WARNING) << "read dump meta info failed! error:" << s.ToString(); + return Status::IOError("read dump meta info failed", infoPath); + } + pstd::MD5 md5 = pstd::MD5(info_data); + snapshot_uuid_ = md5.hexdigest(); + } + *snapshot_uuid = snapshot_uuid_; + return Status::OK(); } void Slot::DoBgSave(void* arg) { diff --git a/src/rsync_server.cc b/src/rsync_server.cc index c83cfd729..d22bd9f1d 100644 --- a/src/rsync_server.cc +++ b/src/rsync_server.cc @@ -23,8 +23,8 @@ void RsyncServer::Schedule(net::TaskFunc func, void* arg) { } int RsyncServer::Start() { + LOG(INFO) << "start RsyncServer ..."; int res = rsync_server_thread_->StartThread(); - LOG(WARNING) << "after RsyncServer::Start"; if (res != net::kSuccess) { LOG(FATAL) << "Start rsync Server Thread Error: " << res; } @@ -33,12 +33,15 @@ int RsyncServer::Start() { LOG(FATAL) << "Start ThreadPool Error: " << res << (res == net::kCreateThreadError ? ": create thread error " : ": other error"); } + LOG(INFO) << "start RsyncServer done ..."; return res; } int RsyncServer::Stop() { + LOG(INFO) << "stop RsyncServer ..."; work_thread_->stop_thread_pool(); rsync_server_thread_->StopThread(); + LOG(INFO) << "stop RsyncServer done..."; return 0; } @@ -53,7 +56,7 @@ RsyncServerConn::~RsyncServerConn() { int RsyncServerConn::DealMessage() { std::shared_ptr req = std::make_shared(); bool parse_res = req->ParseFromArray(rbuf_ + cur_pos_ - header_len_, header_len_); - LOG(WARNING) << "RsyncServer DealMessage..."; + LOG(INFO) << "RsyncServer receives new request..."; if (!parse_res) { LOG(WARNING) << "Pika rsync server connection pb parse error."; return -1; @@ -82,22 +85,22 @@ void RsyncServerConn::HandleMetaRsyncRequest(void* arg) { std::unique_ptr task_arg(static_cast(arg)); const std::shared_ptr req = task_arg->req; std::shared_ptr conn = task_arg->conn; + std::string db_name = req->db_name(); + uint32_t slot_id = req->slot_id(); RsyncService::RsyncResponse response; - response.set_db_name("db_name"); - response.set_slot_id(0); + response.set_db_name(db_name); + response.set_slot_id(slot_id); response.set_type(RsyncService::kRsyncMeta); - LOG(INFO) << "Receive RsyncMeta request"; + LOG(INFO) << "RsyncServer receives RsyncMeta request..."; - std::string db_name = req->db_name(); - uint32_t slot_id = req->slot_id(); std::vector filenames; std::string snapshot_uuid; g_pika_server->GetDumpMeta(db_name, slot_id, &filenames, &snapshot_uuid); LOG(WARNING) << "snapshot_uuid: " << snapshot_uuid; std::for_each(filenames.begin(), filenames.end(), [](auto& file) { - LOG(WARNING) << "file:" << file; + LOG(WARNING) << "meta file name: " << file; }); //TODO: temporarily mock response RsyncService::MetaResponse* meta_resp = response.mutable_meta_resp(); @@ -113,43 +116,46 @@ void RsyncServerConn::HandleMetaRsyncRequest(void* arg) { return; } conn->NotifyWrite(); + LOG(INFO) << "RsyncServer RsyncMeta request done..."; } void RsyncServerConn::HandleFileRsyncRequest(void* arg) { std::unique_ptr task_arg(static_cast(arg)); const std::shared_ptr req = task_arg->req; std::shared_ptr conn = task_arg->conn; + LOG(INFO) << "RsyncServer RsyncFile request ..."; - RsyncService::RsyncResponse response; - response.set_type(RsyncService::kRsyncFile); - response.set_snapshot_uuid("demo_snapshot_uuid"); - response.set_db_name("db_name"); - response.set_slot_id(0); - LOG(INFO) << "Receive RsyncFile request " << "filename: " << req->file_req().filename() - << " offset: " << req->file_req().offset() - << " count: " << req->file_req().count(); - + uint32_t slot_id = req->slot_id(); std::string db_name = req->db_name(); std::string filename = req->file_req().filename(); - uint32_t slot_id = req->slot_id(); size_t offset = req->file_req().offset(); size_t count = req->file_req().count(); + RsyncService::RsyncResponse response; + std::string snapshot_uuid; + Status s = g_pika_server->GetDumpUUID(db_name, slot_id, &snapshot_uuid); + LOG(INFO) << "Receive RsyncFile request " << "filename: " << filename + << " offset: " << offset + << " count: " << count; + char* buffer = new char[req->file_req().count() + 1]; size_t bytes_read{0}; - LOG(INFO) << "....... ReadDumpFile: " << filename; auto status = g_pika_server -> ReadDumpFile(db_name, slot_id, filename, offset, count, buffer, &bytes_read); - LOG(INFO) << "ReadDumpFile: " << filename << " read size: " << status.ToString(); + LOG(INFO) << "RsyncServer ReadDumpFile: " << filename << " read size: " << bytes_read << "status: " << status.ToString(); + + response.set_type(RsyncService::kRsyncFile); + response.set_snapshot_uuid(snapshot_uuid); + response.set_db_name(db_name); + response.set_slot_id(slot_id); - //TODO: temporarily mock response RsyncService::FileResponse* file_resp = response.mutable_file_resp(); file_resp->set_eof(bytes_read != count); file_resp->set_count(bytes_read); file_resp->set_offset(offset); file_resp->set_data(buffer, bytes_read); + //TODO: checksum file_resp->set_checksum("checksum"); file_resp->set_filename(filename); - LOG(INFO) << "....... before serializetostring: " << filename; std::string reply_str; if (!response.SerializeToString(&reply_str) || (conn->WriteResp(reply_str) != 0)) { LOG(WARNING) << "Process FileRsync request serialization failed"; @@ -159,6 +165,7 @@ void RsyncServerConn::HandleFileRsyncRequest(void* arg) { } delete []buffer; conn->NotifyWrite(); + LOG(INFO) << "RsyncServer RsyncFile request ..."; } RsyncServerThread::RsyncServerThread(const std::set& ips, int port, int cron_interval, RsyncServer* arg) From 0d35c3b71fd9ec6436ebaf5ea1e923a3a6673a08 Mon Sep 17 00:00:00 2001 From: wangshao1 <459439411@qq.com> Date: Tue, 18 Jul 2023 16:38:20 +0800 Subject: [PATCH 16/37] fix rsync client/server bugs --- include/rsync_server.h | 6 ++---- include/throttle.h | 4 ++-- src/pika_server.cc | 11 ++++++----- src/rsync_client.cc | 23 +++++++++++++++++++---- src/rsync_server.cc | 5 ++--- 5 files changed, 31 insertions(+), 18 deletions(-) diff --git a/include/rsync_server.h b/include/rsync_server.h index f87a1df8d..1c8fb4f05 100644 --- a/include/rsync_server.h +++ b/include/rsync_server.h @@ -33,14 +33,12 @@ class RsyncServerThread; class RsyncServer { public: - RsyncServer(const std::string& ip_port, const int port); + RsyncServer(const std::set& ips, const int port); ~RsyncServer(); void Schedule(net::TaskFunc func, void* arg); int Start(); int Stop(); private: - int port_; - std::string ip_; std::map > file_map_; std::unique_ptr work_thread_ = nullptr; std::unique_ptr rsync_server_thread_ = nullptr; @@ -117,4 +115,4 @@ class RSyncReader { }; } //end namespace rsync -#endif \ No newline at end of file +#endif diff --git a/include/throttle.h b/include/throttle.h index b63795bc7..2ffa52a9b 100644 --- a/include/throttle.h +++ b/include/throttle.h @@ -6,7 +6,7 @@ namespace rsync { class Throttle { public: - Throttle(); + Throttle() {} Throttle(size_t throttle_throughput_bytes, size_t check_cycle); ~Throttle(); size_t ThrottledByThroughput(size_t bytes); @@ -27,4 +27,4 @@ class Throttle { }; } // end namespace rsync -#endif \ No newline at end of file +#endif diff --git a/src/pika_server.cc b/src/pika_server.cc index f27cfb5c8..539fdf88c 100644 --- a/src/pika_server.cc +++ b/src/pika_server.cc @@ -76,7 +76,7 @@ PikaServer::PikaServer() std::make_unique(ips, port_, worker_num_, 3000, worker_queue_limit, g_pika_conf->max_conn_rbuf_size()); pika_rsync_service_ = std::make_unique(g_pika_conf->db_sync_path(), g_pika_conf->port() + kPortShiftRSync); //TODO 删除pika_rsync_service_服务,使用pika_rsync_service_端口 - rsync_server_ = std::make_unique("127.0.0.1", g_pika_server->master_port() + kPortShiftRsync2); + rsync_server_ = std::make_unique(ips, port_ + kPortShiftRsync2); pika_pubsub_thread_ = std::make_unique(); pika_auxiliary_thread_ = std::make_unique(); pika_migrate_ = std::make_unique(); @@ -134,7 +134,7 @@ bool PikaServer::ServerInit() { void PikaServer::Start() { int ret = 0; // start rsync first, rocksdb opened fd will not appear in this fork - ret = pika_rsync_service_->StartRsync(); + //ret = pika_rsync_service_->StartRsync(); if (0 != ret) { dbs_.clear(); LOG(FATAL) << "Start Rsync Error: bind port " + std::to_string(pika_rsync_service_->ListenPort()) + " failed" @@ -953,12 +953,12 @@ pstd::Status PikaServer::ReadDumpFile(const std::string& db_name, uint32_t slot_ read_count = kMaxCopyBlockSize; } - size_t bytesin = 0; + ssize_t bytesin = 0; size_t left_read_count = count; while ((bytesin = pread(fd, data, read_count, read_offset)) > 0) { left_read_count -= bytesin; - if (left_read_count <= 0) { + if (left_read_count < 0) { break ; } if (read_count > left_read_count) { @@ -1001,6 +1001,7 @@ void PikaServer::TryDBSync(const std::string& ip, int port, const std::string& d // Need Bgsave first slot->BgSaveSlot(); } + return; DBSync(ip, port, db_name, slot_id); } @@ -1332,7 +1333,7 @@ void PikaServer::DoTimingTask() { // Delete expired dump AutoDeleteExpiredDump(); // Cheek Rsync Status - AutoKeepAliveRSync(); + //AutoKeepAliveRSync(); // Reset server qps ResetLastSecQuerynum(); } diff --git a/src/rsync_client.cc b/src/rsync_client.cc index 0975080ba..b50fc1f9d 100644 --- a/src/rsync_client.cc +++ b/src/rsync_client.cc @@ -16,6 +16,7 @@ namespace rsync { RsyncClient::RsyncClient(const std::string& dir, const std::string& db_name, const uint32_t slot_id) : dir_(dir), flush_period_(100), db_name_(db_name), slot_id_(slot_id), state_(IDLE), max_retries_(10) { client_thread_ = std::make_unique(10 * 1000, 60 * 1000, this); + throttle_.reset(new Throttle()); } bool RsyncClient::Init() { @@ -42,7 +43,7 @@ void* RsyncClient::ThreadMain() { meta_rep.append("\n"); while (state_.load(std::memory_order_relaxed) == RUNNING) { for (const auto& file : file_set_) { - LOG(INFO) << "CopyRemoteFile: " << file; + LOG(INFO) << "CopyRemoteFile: " << file << "state_: " << state_.load(); while (state_.load() == RUNNING) { s = CopyRemoteFile(file); if (!s.ok()) { @@ -65,6 +66,16 @@ void* RsyncClient::ThreadMain() { } } if (meta_table_.size() == file_set_.size()) { + LOG(INFO) << "CopyRemoteFile: " << kBgsaveInfoFile << "state_: " << state_.load(); + while (state_.load() == RUNNING) { + s = CopyRemoteFile(kBgsaveInfoFile); + if (!s.ok()) { + LOG(WARNING) << "rsync CopyRemoteFile failed, filename: " << kBgsaveInfoFile; + continue; + } + LOG(WARNING) << "CopyRemoteFile "<< kBgsaveInfoFile << "success..."; + break; + } LOG(INFO) << "rsync success..."; state_.store(STOP, std::memory_order_relaxed); break; @@ -145,6 +156,7 @@ Status RsyncClient::CopyRemoteFile(const std::string& filename) { std::string to_send; request.SerializeToString(&to_send); + LOG(WARNING) << "master ip: " << g_pika_server->master_ip() << " master_port: " << g_pika_server->master_port() << " portshift: " << kPortShiftRsync2; s = client_thread_->Write(g_pika_server->master_ip(), g_pika_server->master_port() + kPortShiftRsync2, to_send); if (!s.ok()) { LOG(WARNING) << "send rsync request failed"; @@ -163,7 +175,7 @@ Status RsyncClient::CopyRemoteFile(const std::string& filename) { LOG(INFO) << "receive fileresponse, snapshot_uuid: " << resp->snapshot_uuid() << "filename: " << resp->file_resp().filename() << "offset: " << resp->file_resp().offset() - << "count: " << resp->file_resp().count(); + << "count: " << resp->file_resp().count() << "eof: " << resp->file_resp().eof(); if (resp->snapshot_uuid() != snapshot_uuid_) { LOG(WARNING) << "receive newer dump, reset state to STOP"; @@ -182,11 +194,13 @@ Status RsyncClient::CopyRemoteFile(const std::string& filename) { md5.update(resp->file_resp().data().c_str(), ret_count); if (resp->file_resp().eof()) { + /* if (md5.finalize().hexdigest() != resp->file_resp().checksum()) { LOG(WARNING) << "mismatch file checksum for file: " << filename; - //TODO: 处理返回status + //TODO: wangshaoyi处理返回status s = Status::IOError("mismatch checksum", "mismatch checksum"); } + */ s = writer->Fsync(); if (!s.ok()) { return s; @@ -290,6 +304,7 @@ Status RsyncClient::CopyRemoteMeta(std::string* snapshot_uuid, std::setmaster_ip() << " master_port: " << g_pika_server->master_port() << " portshift: " << kPortShiftRsync2; s = client_thread_->Write(g_pika_server->master_ip(), g_pika_server->master_port() + kPortShiftRsync2, to_send); if (!s.ok()) { retries++; @@ -437,4 +452,4 @@ Status RsyncClient::FlushMetaTable() { return Status::OK(); } -} // end namespace rsync \ No newline at end of file +} // end namespace rsync diff --git a/src/rsync_server.cc b/src/rsync_server.cc index d22bd9f1d..d8a1b958b 100644 --- a/src/rsync_server.cc +++ b/src/rsync_server.cc @@ -7,9 +7,8 @@ extern PikaServer* g_pika_server; namespace rsync { -RsyncServer::RsyncServer(const std::string& ip, const int port) : ip_(ip), port_(port) { +RsyncServer::RsyncServer(const std::set& ips, const int port) { work_thread_ = std::make_unique(2, 100000); - std::set ips = {ip_}; rsync_server_thread_ = std::make_unique(ips, port, 60 * 1000, this); } @@ -192,4 +191,4 @@ void RsyncServerThread::RsyncServerHandle::CronHandle() const { LOG(WARNING) << "CronHandle called"; } -} // end namespace rsync \ No newline at end of file +} // end namespace rsync From 11301306150f74a64ac79072d75e9946bdba44e9 Mon Sep 17 00:00:00 2001 From: wangshao1 <459439411@qq.com> Date: Tue, 18 Jul 2023 21:19:23 +0800 Subject: [PATCH 17/37] fix bugs --- include/rsync_client.h | 4 ++-- src/pika_rm.cc | 1 + src/pika_server.cc | 1 + src/rsync_client.cc | 13 +++++++++---- src/rsync_server.cc | 1 + 5 files changed, 14 insertions(+), 6 deletions(-) diff --git a/include/rsync_client.h b/include/rsync_client.h index 9a51d294a..ed4f0d356 100644 --- a/include/rsync_client.h +++ b/include/rsync_client.h @@ -54,7 +54,7 @@ class RsyncClient : public net::Thread { private: bool Recover(); Status Wait(WaitObject* wo); - Status CopyRemoteFile(const std::string& filename); + Status CopyRemoteFile(const std::string& filename, const std::string& rename = ""); Status CopyRemoteMeta(std::string* snapshot_uuid, std::set* file_set); Status LoadLocalMeta(std::string* snapshot_uuid, std::map* file_map); std::string GetLocalMetaFilePath(); @@ -137,4 +137,4 @@ class WaitObject { } // end namespace rsync -#endif \ No newline at end of file +#endif diff --git a/src/pika_rm.cc b/src/pika_rm.cc index 4c96a76be..e762facfa 100644 --- a/src/pika_rm.cc +++ b/src/pika_rm.cc @@ -602,6 +602,7 @@ void SyncSlaveSlot::Deactivate() { std::lock_guard l(slot_mu_); m_info_ = RmNode(); repl_state_ = ReplState::kNoConnect; + rsync_cli_->Stop(); } std::string SyncSlaveSlot::ToStringStatus() { diff --git a/src/pika_server.cc b/src/pika_server.cc index 539fdf88c..53d0a7931 100644 --- a/src/pika_server.cc +++ b/src/pika_server.cc @@ -72,6 +72,7 @@ PikaServer::PikaServer() // We estimate the queue size int worker_queue_limit = g_pika_conf->maxclients() / worker_num_ + 100; LOG(INFO) << "Worker queue limit is " << worker_queue_limit; + for_each(ips.begin(), ips.end(), [](auto& ip) {LOG(WARNING) << ip;}); pika_dispatch_thread_ = std::make_unique(ips, port_, worker_num_, 3000, worker_queue_limit, g_pika_conf->max_conn_rbuf_size()); pika_rsync_service_ = std::make_unique(g_pika_conf->db_sync_path(), g_pika_conf->port() + kPortShiftRSync); diff --git a/src/rsync_client.cc b/src/rsync_client.cc index b50fc1f9d..bdfe5930c 100644 --- a/src/rsync_client.cc +++ b/src/rsync_client.cc @@ -68,11 +68,12 @@ void* RsyncClient::ThreadMain() { if (meta_table_.size() == file_set_.size()) { LOG(INFO) << "CopyRemoteFile: " << kBgsaveInfoFile << "state_: " << state_.load(); while (state_.load() == RUNNING) { - s = CopyRemoteFile(kBgsaveInfoFile); + s = CopyRemoteFile(kBgsaveInfoFile, kBgsaveInfoFile + ".bak"); if (!s.ok()) { LOG(WARNING) << "rsync CopyRemoteFile failed, filename: " << kBgsaveInfoFile; continue; } + RenameFile(dir_ + "/" + kBgsaveInfoFile + ".bak", dir_+ "/" + kBgsaveInfoFile); LOG(WARNING) << "CopyRemoteFile "<< kBgsaveInfoFile << "success..."; break; } @@ -110,7 +111,8 @@ Status RsyncClient::Wait(WaitObject* wo) { } if (resp->type() == kRsyncFile && (resp->file_resp().filename() != wo->filename_ || resp->file_resp().offset() != wo->offset_)) { - LOG(WARNING) << "mismatch rsync response, skip"; + LOG(WARNING) << "mismatch rsync response, skip expect filename: " << wo->filename_ << " offset: " << wo->offset_ << " resp filename: " << resp->file_resp().filename() << " resp offset: " << resp->file_resp().offset(); + iter++; continue; } s = Status::OK(); @@ -127,14 +129,14 @@ Status RsyncClient::Wait(WaitObject* wo) { return s; } -Status RsyncClient::CopyRemoteFile(const std::string& filename) { +Status RsyncClient::CopyRemoteFile(const std::string& filename, const std::string& rename) { Status s; int retries = 0; size_t offset = 0; size_t copy_file_begin_time = pstd::NowMicros(); size_t count = throttle_->ThrottledByThroughput(1024 * 1024); MD5 md5; - std::unique_ptr writer(new RsyncWriter(dir_ + "/" + filename)); + std::unique_ptr writer(new RsyncWriter(dir_ + "/" + (rename.empty() ? filename : rename))); DEFER { if (writer) { writer->Close(); @@ -230,6 +232,9 @@ Status RsyncClient::Start() { } Status RsyncClient::Stop() { + if (state_ == IDLE) { + return Status::OK(); + } state_ = STOP; StopThread(); client_thread_->StopThread(); diff --git a/src/rsync_server.cc b/src/rsync_server.cc index d8a1b958b..db5c8a812 100644 --- a/src/rsync_server.cc +++ b/src/rsync_server.cc @@ -9,6 +9,7 @@ namespace rsync { RsyncServer::RsyncServer(const std::set& ips, const int port) { work_thread_ = std::make_unique(2, 100000); + for_each(ips.begin(), ips.end(), [&port](auto& ip) {LOG(WARNING) << ip << "port: " << port;}); rsync_server_thread_ = std::make_unique(ips, port, 60 * 1000, this); } From 612f5e41baa815edc0b9520bdf34f8e9c6006741 Mon Sep 17 00:00:00 2001 From: wangshaoyi Date: Tue, 18 Jul 2023 21:52:15 +0800 Subject: [PATCH 18/37] add debug log for test --- include/rsync_client.h | 33 +++++++++++++++++--- src/rsync_client.cc | 70 +++++++++++++++++++----------------------- 2 files changed, 59 insertions(+), 44 deletions(-) diff --git a/include/rsync_client.h b/include/rsync_client.h index ed4f0d356..93e766805 100644 --- a/include/rsync_client.h +++ b/include/rsync_client.h @@ -53,7 +53,7 @@ class RsyncClient : public net::Thread { private: bool Recover(); - Status Wait(WaitObject* wo); + Status Wait(RsyncResponse* resp); Status CopyRemoteFile(const std::string& filename, const std::string& rename = ""); Status CopyRemoteMeta(std::string* snapshot_uuid, std::set* file_set); Status LoadLocalMeta(std::string* snapshot_uuid, std::map* file_map); @@ -78,7 +78,7 @@ class RsyncClient : public net::Thread { std::atomic state_; int max_retries_; - std::list resp_list_; + std::unique_ptr wo_; std::condition_variable cond_; std::mutex mu_; std::unique_ptr throttle_; @@ -126,9 +126,32 @@ class RsyncWriter { class WaitObject { public: - WaitObject(const std::string& filename, RsyncService::Type t, size_t offset) - : filename_(filename), type_(t), offset_(offset), resp_(nullptr) {} - WaitObject(RsyncService::Type t) : filename_(""), type_(t), offset_(-1), resp_(nullptr) {} + WaitObject() : filename_(""), type_(RsyncService::kRsyncMeta), offset_(0), resp_(nullptr) {} + ~WaitObject() { + if (resp_) { + delete resp_; + resp_ = nullptr; + } + } + void Reset(const std::string& filename, RsyncService::Type t, size_t offset) { + if (resp_) { + delete resp_; + resp_ = nullptr; + } + filename_ = filename; + type_ = t; + offset_ = offset; + } + + void Reset(RsyncService::Type t) { + if (resp_) { + delete resp_; + resp_ = nullptr; + } + filename_ = ""; + type_ = t; + offset_ = 0xFFFFFFFF; + } std::string filename_; RsyncService::Type type_; size_t offset_; diff --git a/src/rsync_client.cc b/src/rsync_client.cc index bdfe5930c..c83df81b6 100644 --- a/src/rsync_client.cc +++ b/src/rsync_client.cc @@ -88,43 +88,25 @@ void* RsyncClient::ThreadMain() { void RsyncClient::OnReceive(RsyncResponse* resp) { std::unique_lock lock(mu_); - resp_list_.push_back(resp); + if (resp->type() != wo_->type_) { + delete resp; + return; + } + if (resp->type() == kRsyncFile && + (resp->file_resp().filename() != wo_->filename_ || resp->file_resp().offset() != wo_->offset_)) { + delete resp; + return; + } + wo_->resp_ = resp; cond_.notify_all(); } -Status RsyncClient::Wait(WaitObject* wo) { +Status RsyncClient::Wait(RsyncResponse* resp) { Status s = Status::Timeout("rsync timeout", "timeout"); - std::list resp_list; { std::unique_lock lock(mu_); - cond_.wait_for(lock, std::chrono::seconds(3), [this] { return !resp_list_.empty(); }); - resp_list.swap(resp_list_); - } - - auto iter = resp_list.begin(); - while (iter != resp_list.end()) { - RsyncResponse* resp = *iter; - if (resp->type() != wo->type_) { - LOG(WARNING) << "mismatch request/response type, skip"; - iter++; - continue; - } - if (resp->type() == kRsyncFile && - (resp->file_resp().filename() != wo->filename_ || resp->file_resp().offset() != wo->offset_)) { - LOG(WARNING) << "mismatch rsync response, skip expect filename: " << wo->filename_ << " offset: " << wo->offset_ << " resp filename: " << resp->file_resp().filename() << " resp offset: " << resp->file_resp().offset(); - iter++; - continue; - } - s = Status::OK(); - wo->resp_ = resp; - resp_list.erase(iter); - break; - } - - iter = resp_list.begin(); - while (iter != resp_list.end()) { - delete (*iter); - iter++; + cond_.wait_for(lock, std::chrono::seconds(3), [this] { return this->wo_->resp_ != nullptr; }); + *resp = *wo_->resp_; } return s; } @@ -164,16 +146,19 @@ Status RsyncClient::CopyRemoteFile(const std::string& filename, const std::strin LOG(WARNING) << "send rsync request failed"; continue; } + { + std::lock_guard lock(mu_); + wo_->Reset(filename, kRsyncFile, offset); + } - WaitObject wo(filename, kRsyncFile, offset); LOG(INFO) << "wait CopyRemoteFile response....."; - s = Wait(&wo); - if (s.IsTimeout() || wo.resp_ == nullptr) { + RsyncResponse* resp = nullptr; + s = Wait(resp); + if (s.IsTimeout() || resp == nullptr) { LOG(WARNING) << "rsync request timeout"; retries++; continue; } - RsyncResponse* resp = wo.resp_; LOG(INFO) << "receive fileresponse, snapshot_uuid: " << resp->snapshot_uuid() << "filename: " << resp->file_resp().filename() << "offset: " << resp->file_resp().offset() @@ -314,14 +299,19 @@ Status RsyncClient::CopyRemoteMeta(std::string* snapshot_uuid, std::set lock(mu_); + wo_->Reset(kRsyncMeta); + } + RsyncResponse* resp = nullptr; + s = Wait(resp); + if (s.IsTimeout() || resp == nullptr) { LOG(WARNING) << "rsync CopyRemoteMeta request timeout, retry times: " << retries; retries++; + delete resp; + resp = nullptr; continue; } - RsyncResponse* resp = wo.resp_; LOG(INFO) << "receive rsync meta infos, snapshot_uuid: " << resp->snapshot_uuid() << "files count: " << resp->meta_resp().filenames_size(); @@ -332,6 +322,8 @@ Status RsyncClient::CopyRemoteMeta(std::string* snapshot_uuid, std::setmeta_resp().filenames_size(); i++) { file_set->insert(resp->meta_resp().filenames(i)); } + delete resp; + resp = nullptr; break; } return s; From 63c4a7e5699f3c12d253a608aaddbfc00692fc51 Mon Sep 17 00:00:00 2001 From: wangshao1 <459439411@qq.com> Date: Wed, 19 Jul 2023 11:37:52 +0800 Subject: [PATCH 19/37] fix bugs --- include/rsync_client.h | 2 +- src/rsync_client.cc | 20 +++++++++++--------- 2 files changed, 12 insertions(+), 10 deletions(-) diff --git a/include/rsync_client.h b/include/rsync_client.h index 93e766805..15c147d54 100644 --- a/include/rsync_client.h +++ b/include/rsync_client.h @@ -53,7 +53,7 @@ class RsyncClient : public net::Thread { private: bool Recover(); - Status Wait(RsyncResponse* resp); + Status Wait(RsyncResponse*& resp); Status CopyRemoteFile(const std::string& filename, const std::string& rename = ""); Status CopyRemoteMeta(std::string* snapshot_uuid, std::set* file_set); Status LoadLocalMeta(std::string* snapshot_uuid, std::map* file_map); diff --git a/src/rsync_client.cc b/src/rsync_client.cc index c83df81b6..6c366b912 100644 --- a/src/rsync_client.cc +++ b/src/rsync_client.cc @@ -16,6 +16,7 @@ namespace rsync { RsyncClient::RsyncClient(const std::string& dir, const std::string& db_name, const uint32_t slot_id) : dir_(dir), flush_period_(100), db_name_(db_name), slot_id_(slot_id), state_(IDLE), max_retries_(10) { client_thread_ = std::make_unique(10 * 1000, 60 * 1000, this); + wo_.reset(new WaitObject()); throttle_.reset(new Throttle()); } @@ -43,14 +44,14 @@ void* RsyncClient::ThreadMain() { meta_rep.append("\n"); while (state_.load(std::memory_order_relaxed) == RUNNING) { for (const auto& file : file_set_) { - LOG(INFO) << "CopyRemoteFile: " << file << "state_: " << state_.load(); + LOG(INFO) << "CopyRemoteFile: " << file << " state_: " << state_.load(); while (state_.load() == RUNNING) { s = CopyRemoteFile(file); if (!s.ok()) { LOG(WARNING) << "rsync CopyRemoteFile failed, filename: " << file; continue; } - LOG(WARNING) << "CopyRemoteFile "<< file << "success..."; + LOG(WARNING) << "CopyRemoteFile "<< file << " success..."; break; } if (state_.load(std::memory_order_relaxed) != RUNNING) { @@ -65,8 +66,8 @@ void* RsyncClient::ThreadMain() { meta_rep.clear(); } } - if (meta_table_.size() == file_set_.size()) { - LOG(INFO) << "CopyRemoteFile: " << kBgsaveInfoFile << "state_: " << state_.load(); + if (state_.load(std::memory_order_relaxed) == RUNNING) { + LOG(INFO) << "CopyRemoteFile: " << kBgsaveInfoFile << " state_: " << state_.load(); while (state_.load() == RUNNING) { s = CopyRemoteFile(kBgsaveInfoFile, kBgsaveInfoFile + ".bak"); if (!s.ok()) { @@ -90,23 +91,26 @@ void RsyncClient::OnReceive(RsyncResponse* resp) { std::unique_lock lock(mu_); if (resp->type() != wo_->type_) { delete resp; + resp = nullptr; return; } if (resp->type() == kRsyncFile && (resp->file_resp().filename() != wo_->filename_ || resp->file_resp().offset() != wo_->offset_)) { delete resp; + resp = nullptr; return; } wo_->resp_ = resp; cond_.notify_all(); } -Status RsyncClient::Wait(RsyncResponse* resp) { +Status RsyncClient::Wait(RsyncResponse*& resp) { Status s = Status::Timeout("rsync timeout", "timeout"); { std::unique_lock lock(mu_); cond_.wait_for(lock, std::chrono::seconds(3), [this] { return this->wo_->resp_ != nullptr; }); - *resp = *wo_->resp_; + resp = wo_->resp_; + s = Status::OK(); } return s; } @@ -116,7 +120,7 @@ Status RsyncClient::CopyRemoteFile(const std::string& filename, const std::strin int retries = 0; size_t offset = 0; size_t copy_file_begin_time = pstd::NowMicros(); - size_t count = throttle_->ThrottledByThroughput(1024 * 1024); + size_t count = throttle_->ThrottledByThroughput(4 * 1024 * 1024); MD5 md5; std::unique_ptr writer(new RsyncWriter(dir_ + "/" + (rename.empty() ? filename : rename))); DEFER { @@ -308,8 +312,6 @@ Status RsyncClient::CopyRemoteMeta(std::string* snapshot_uuid, std::setsnapshot_uuid() From dc867798308498a7c9f1909200169c8efe36b60d Mon Sep 17 00:00:00 2001 From: wangshaoyi Date: Wed, 19 Jul 2023 13:43:09 +0800 Subject: [PATCH 20/37] fix bugs --- include/pika_rm.h | 2 + include/rsync_client.h | 3 +- src/pika_rm.cc | 8 +- src/pika_slot.cc | 1 + src/rsync_client.cc | 237 ++++++++++++++++++++--------------------- 5 files changed, 124 insertions(+), 127 deletions(-) diff --git a/include/pika_rm.h b/include/pika_rm.h index 0d1c14755..e58a861be 100644 --- a/include/pika_rm.h +++ b/include/pika_rm.h @@ -160,6 +160,8 @@ class SyncSlaveSlot : public SyncSlot { void ActivateRsync(); + bool IsRsyncIdle() {return rsync_cli_->IsIdle();} + private: std::unique_ptr rsync_cli_; pstd::Mutex slot_mu_; diff --git a/include/rsync_client.h b/include/rsync_client.h index 15c147d54..6d3260237 100644 --- a/include/rsync_client.h +++ b/include/rsync_client.h @@ -49,12 +49,13 @@ class RsyncClient : public net::Thread { Status Start(); Status Stop(); bool IsRunning() { return state_.load() == RUNNING;} + bool IsIdle() { return state_.load() == IDLE;} void OnReceive(RsyncResponse* resp); private: bool Recover(); Status Wait(RsyncResponse*& resp); - Status CopyRemoteFile(const std::string& filename, const std::string& rename = ""); + Status CopyRemoteFile(const std::string& filename); Status CopyRemoteMeta(std::string* snapshot_uuid, std::set* file_set); Status LoadLocalMeta(std::string* snapshot_uuid, std::map* file_map); std::string GetLocalMetaFilePath(); diff --git a/src/pika_rm.cc b/src/pika_rm.cc index e762facfa..b89191b23 100644 --- a/src/pika_rm.cc +++ b/src/pika_rm.cc @@ -642,7 +642,7 @@ std::string SyncSlaveSlot::LocalIp() { } void SyncSlaveSlot::ActivateRsync() { - if (rsync_cli_->IsRunning()) { + if (!rsync_cli_->IsIdle()) { return; } if (rsync_cli_->Init()) { @@ -1152,11 +1152,13 @@ Status PikaReplicaManager::RunSyncSlaveSlotStateMachine() { } else if (s_slot->State() == ReplState::kWaitReply) { continue; } else if (s_slot->State() == ReplState::kWaitDBSync) { + s_slot->ActivateRsync(); std::shared_ptr slot = g_pika_server->GetDBSlotById(p_info.db_name_, p_info.slot_id_); if (slot) { - s_slot->ActivateRsync(); - slot->TryUpdateMasterOffset(); + if (s_slot->IsRsyncIdle()) { + slot->TryUpdateMasterOffset(); + } } else { LOG(WARNING) << "Slot not found, DB Name: " << p_info.db_name_ << " Slot Id: " << p_info.slot_id_; diff --git a/src/pika_slot.cc b/src/pika_slot.cc index 94e469f31..b871767ce 100644 --- a/src/pika_slot.cc +++ b/src/pika_slot.cc @@ -319,6 +319,7 @@ void Slot::GetBgSaveMetaData(std::vector* fileNames, std::string* s fileNames -> push_back(type + "/" + fileName); } } + fileNames->push_back(kBgsaveInfoFile); pstd::Status s = GetBgSaveUUID(snapshot_uuid); if (!s.ok()) { LOG(WARNING) << "read dump meta info failed! error:" << s.ToString(); diff --git a/src/rsync_client.cc b/src/rsync_client.cc index 6c366b912..cbcdb9291 100644 --- a/src/rsync_client.cc +++ b/src/rsync_client.cc @@ -34,57 +34,45 @@ bool RsyncClient::Init() { } void* RsyncClient::ThreadMain() { - int cnt = 0; - int period = 0; - Status s = Status::OK(); - std::string meta_file_path = dir_ + "/" + kDumpMetaFileName; - int meta_fd = open(meta_file_path.c_str(), O_CREAT | O_RDWR, 0644); - std::string meta_rep(kUuidPrefix); - meta_rep.append(snapshot_uuid_); - meta_rep.append("\n"); - while (state_.load(std::memory_order_relaxed) == RUNNING) { - for (const auto& file : file_set_) { - LOG(INFO) << "CopyRemoteFile: " << file << " state_: " << state_.load(); - while (state_.load() == RUNNING) { - s = CopyRemoteFile(file); - if (!s.ok()) { - LOG(WARNING) << "rsync CopyRemoteFile failed, filename: " << file; - continue; - } - LOG(WARNING) << "CopyRemoteFile "<< file << " success..."; - break; - } - if (state_.load(std::memory_order_relaxed) != RUNNING) { - break; - } - if (++period == flush_period_) { - period = 0; - meta_rep.append(file + ":" + meta_table_[file]); - meta_rep.append("\n"); - write(meta_fd, meta_rep.data(), meta_rep.size()); - fsync(meta_fd); - meta_rep.clear(); - } - } - if (state_.load(std::memory_order_relaxed) == RUNNING) { - LOG(INFO) << "CopyRemoteFile: " << kBgsaveInfoFile << " state_: " << state_.load(); - while (state_.load() == RUNNING) { - s = CopyRemoteFile(kBgsaveInfoFile, kBgsaveInfoFile + ".bak"); - if (!s.ok()) { - LOG(WARNING) << "rsync CopyRemoteFile failed, filename: " << kBgsaveInfoFile; - continue; - } - RenameFile(dir_ + "/" + kBgsaveInfoFile + ".bak", dir_+ "/" + kBgsaveInfoFile); - LOG(WARNING) << "CopyRemoteFile "<< kBgsaveInfoFile << "success..."; - break; - } - LOG(INFO) << "rsync success..."; - state_.store(STOP, std::memory_order_relaxed); - break; - } + int cnt = 0; + int period = 0; + Status s = Status::OK(); + + std::string meta_file_path = GetLocalMetaFilePath(); + int meta_fd = open(meta_file_path.c_str(), O_CREAT | O_RDWR, 0644); + std::string meta_rep(kUuidPrefix); + meta_rep.append(snapshot_uuid_); + meta_rep.append("\n"); + + for (const auto& file : file_set_) { + LOG(INFO) << "CopyRemoteFile: " << file << " state_: " << state_.load(); + while (state_.load() == RUNNING) { + s = CopyRemoteFile(file); + if (!s.ok()) { + LOG(WARNING) << "rsync CopyRemoteFile failed, filename: " << file; + continue; + } + LOG(WARNING) << "CopyRemoteFile "<< file << " success..."; + break; } - close(meta_fd); - return nullptr; + if (state_.load(std::memory_order_relaxed) != RUNNING) { + break; + } + if (++period == flush_period_) { + period = 0; + meta_rep.append(file + ":" + meta_table_[file]); + meta_rep.append("\n"); + write(meta_fd, meta_rep.data(), meta_rep.size()); + fsync(meta_fd); + meta_rep.clear(); + } + } + if (meta_rep.empty()) { + write(meta_fd, meta_rep.data(), meta_rep.size()); + fsync(meta_fd); + } + close(meta_fd); + return nullptr; } void RsyncClient::OnReceive(RsyncResponse* resp) { @@ -115,14 +103,14 @@ Status RsyncClient::Wait(RsyncResponse*& resp) { return s; } -Status RsyncClient::CopyRemoteFile(const std::string& filename, const std::string& rename) { +Status RsyncClient::CopyRemoteFile(const std::string& filename) { Status s; int retries = 0; size_t offset = 0; size_t copy_file_begin_time = pstd::NowMicros(); size_t count = throttle_->ThrottledByThroughput(4 * 1024 * 1024); MD5 md5; - std::unique_ptr writer(new RsyncWriter(dir_ + "/" + (rename.empty() ? filename : rename))); + std::unique_ptr writer(new RsyncWriter(dir_ + "/" + filename)); DEFER { if (writer) { writer->Close(); @@ -132,84 +120,87 @@ Status RsyncClient::CopyRemoteFile(const std::string& filename, const std::strin DeleteFile(filename); } }; + while (retries < max_retries_) { - RsyncRequest request; - request.set_type(kRsyncFile); - request.set_db_name(db_name_); - request.set_slot_id(slot_id_); - FileRequest* file_req = request.mutable_file_req(); - file_req->set_filename(filename); - file_req->set_offset(offset); - file_req->set_count(count); - std::string to_send; - request.SerializeToString(&to_send); + RsyncRequest request; + request.set_type(kRsyncFile); + request.set_db_name(db_name_); + request.set_slot_id(slot_id_); + FileRequest* file_req = request.mutable_file_req(); + file_req->set_filename(filename); + file_req->set_offset(offset); + file_req->set_count(count); + std::string to_send; + request.SerializeToString(&to_send); + + LOG(WARNING) << "master ip: " << g_pika_server->master_ip() << " master_port: " << g_pika_server->master_port() << " portshift: " << kPortShiftRsync2; + s = client_thread_->Write(g_pika_server->master_ip(), g_pika_server->master_port() + kPortShiftRsync2, to_send); + if (!s.ok()) { + LOG(WARNING) << "send rsync request failed"; + continue; + } - LOG(WARNING) << "master ip: " << g_pika_server->master_ip() << " master_port: " << g_pika_server->master_port() << " portshift: " << kPortShiftRsync2; - s = client_thread_->Write(g_pika_server->master_ip(), g_pika_server->master_port() + kPortShiftRsync2, to_send); - if (!s.ok()) { - LOG(WARNING) << "send rsync request failed"; - continue; - } - { - std::lock_guard lock(mu_); - wo_->Reset(filename, kRsyncFile, offset); - } + { + std::lock_guard lock(mu_); + wo_->Reset(filename, kRsyncFile, offset); + } - LOG(INFO) << "wait CopyRemoteFile response....."; - RsyncResponse* resp = nullptr; - s = Wait(resp); - if (s.IsTimeout() || resp == nullptr) { - LOG(WARNING) << "rsync request timeout"; - retries++; - continue; - } + LOG(INFO) << "wait CopyRemoteFile response....."; + RsyncResponse* resp = nullptr; + s = Wait(resp); + if (s.IsTimeout() || resp == nullptr) { + LOG(WARNING) << "rsync request timeout"; + retries++; + continue; + } - LOG(INFO) << "receive fileresponse, snapshot_uuid: " << resp->snapshot_uuid() - << "filename: " << resp->file_resp().filename() << "offset: " << resp->file_resp().offset() - << "count: " << resp->file_resp().count() << "eof: " << resp->file_resp().eof(); + size_t copy_file_end_time = pstd::NowMicros(); + size_t elaspe_time_us = copy_file_end_time - copy_file_begin_time; + throttle_->ReturnUnusedThroughput(count, resp->file_resp().count(), elaspe_time_us); - if (resp->snapshot_uuid() != snapshot_uuid_) { - LOG(WARNING) << "receive newer dump, reset state to STOP"; - state_.store(STOP); - delete resp; - return s; - } + LOG(INFO) << "receive fileresponse, snapshot_uuid: " << resp->snapshot_uuid() + << "filename: " << resp->file_resp().filename() << "offset: " << resp->file_resp().offset() + << "count: " << resp->file_resp().count() << "eof: " << resp->file_resp().eof(); - size_t ret_count = resp->file_resp().count(); - resp->file_resp().data(); - s = writer->Write((uint64_t)offset, ret_count, resp->file_resp().data().c_str()); - if (!s.ok()) { - LOG(WARNING) << "rsync client write file error"; - break; - } + if (resp->snapshot_uuid() != snapshot_uuid_) { + LOG(WARNING) << "receive newer dump, reset state to STOP"; + state_.store(STOP); + delete resp; + return s; + } - md5.update(resp->file_resp().data().c_str(), ret_count); - if (resp->file_resp().eof()) { - /* - if (md5.finalize().hexdigest() != resp->file_resp().checksum()) { - LOG(WARNING) << "mismatch file checksum for file: " << filename; - //TODO: wangshaoyi处理返回status - s = Status::IOError("mismatch checksum", "mismatch checksum"); - } - */ - s = writer->Fsync(); - if (!s.ok()) { - return s; - } - s = writer->Close(); - if (!s.ok()) { - return s; - } - writer.reset(); - meta_table_[filename] = resp->file_resp().checksum(); - break; - } else { - offset += resp->file_resp().count(); + size_t ret_count = resp->file_resp().count(); + resp->file_resp().data(); + s = writer->Write((uint64_t)offset, ret_count, resp->file_resp().data().c_str()); + if (!s.ok()) { + LOG(WARNING) << "rsync client write file error"; + break; + } + + md5.update(resp->file_resp().data().c_str(), ret_count); + offset += resp->file_resp().count(); + if (resp->file_resp().eof()) { + /* + if (md5.finalize().hexdigest() != resp->file_resp().checksum()) { + LOG(WARNING) << "mismatch file checksum for file: " << filename; + //TODO: wangshaoyi处理返回status + s = Status::IOError("mismatch checksum", "mismatch checksum"); + return s; } - size_t copy_file_end_time = pstd::NowMicros(); - size_t elaspe_time_us = copy_file_end_time - copy_file_begin_time; - throttle_->ReturnUnusedThroughput(count, ret_count, elaspe_time_us); - retries = 0; + */ + s = writer->Fsync(); + if (!s.ok()) { + return s; + } + s = writer->Close(); + if (!s.ok()) { + return s; + } + writer.reset(); + meta_table_[filename] = resp->file_resp().checksum(); + break; + } + retries = 0; } return s; @@ -413,7 +404,6 @@ Status RsyncClient::CleanUpExpiredFiles(bool need_reset_path, std::set& expired_files, std::map& localFileMap) { - localFileMap[kUuidPrefix] = snapshot_uuid; for (const auto& item : expired_files) { localFileMap.erase(item); } @@ -427,6 +417,7 @@ Status RsyncClient::UpdateLocalMeta(std::string& snapshot_uuid, std::setAppend(kUuidPrefix + snapshot_uuid + "\n"); for (const auto& item : localFileMap) { std::string line = item.first + item.second + "\n"; From 002d34d998c75fd5c21c8d2d270892447dba0d4c Mon Sep 17 00:00:00 2001 From: wangshao1 <459439411@qq.com> Date: Wed, 19 Jul 2023 22:09:25 +0800 Subject: [PATCH 21/37] fix bugs --- CMakeLists.txt | 2 +- include/pika_rm.h | 2 +- include/rsync_client.h | 25 +++++------- src/pika_rm.cc | 6 ++- src/pika_server.cc | 2 +- src/rsync_client.cc | 88 +++++++++++++++++++++++++++++------------- src/rsync_server.cc | 1 - 7 files changed, 79 insertions(+), 47 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 465a5354a..0e08c861e 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -47,7 +47,7 @@ elseif(${BUILD_TYPE} STREQUAL RELWITHDEBINFO) set(LIB_BUILD_TYPE RELWITHDEBINFO) else() set(LIB_BUILD_TYPE RElEASE) - set(CMAKE_CXX_FLAGS_RELEASE "-O2 -DNDEBUG") + set(CMAKE_CXX_FLAGS_RELEASE "-g -DNDEBUG") endif() if(CMAKE_SYSTEM_NAME MATCHES "Darwin") diff --git a/include/pika_rm.h b/include/pika_rm.h index e58a861be..50d5f4af9 100644 --- a/include/pika_rm.h +++ b/include/pika_rm.h @@ -160,7 +160,7 @@ class SyncSlaveSlot : public SyncSlot { void ActivateRsync(); - bool IsRsyncIdle() {return rsync_cli_->IsIdle();} + bool IsRsyncRunning() {return rsync_cli_->IsRunning();} private: std::unique_ptr rsync_cli_; diff --git a/include/rsync_client.h b/include/rsync_client.h index 6d3260237..d38fdd483 100644 --- a/include/rsync_client.h +++ b/include/rsync_client.h @@ -48,7 +48,10 @@ class RsyncClient : public net::Thread { bool Init(); Status Start(); Status Stop(); - bool IsRunning() { return state_.load() == RUNNING;} + bool IsRunning() { + LOG(WARNING) << "current state_: " << state_.load(); + return state_.load() == RUNNING; + } bool IsIdle() { return state_.load() == IDLE;} void OnReceive(RsyncResponse* resp); @@ -83,6 +86,9 @@ class RsyncClient : public net::Thread { std::condition_variable cond_; std::mutex mu_; std::unique_ptr throttle_; + + std::string master_ip_; + int master_port_; }; //TODO: jinge @@ -128,27 +134,16 @@ class RsyncWriter { class WaitObject { public: WaitObject() : filename_(""), type_(RsyncService::kRsyncMeta), offset_(0), resp_(nullptr) {} - ~WaitObject() { - if (resp_) { - delete resp_; - resp_ = nullptr; - } - } + ~WaitObject() {} void Reset(const std::string& filename, RsyncService::Type t, size_t offset) { - if (resp_) { - delete resp_; - resp_ = nullptr; - } + resp_ = nullptr; filename_ = filename; type_ = t; offset_ = offset; } void Reset(RsyncService::Type t) { - if (resp_) { - delete resp_; - resp_ = nullptr; - } + resp_ = nullptr; filename_ = ""; type_ = t; offset_ = 0xFFFFFFFF; diff --git a/src/pika_rm.cc b/src/pika_rm.cc index b89191b23..c5858c81a 100644 --- a/src/pika_rm.cc +++ b/src/pika_rm.cc @@ -645,8 +645,12 @@ void SyncSlaveSlot::ActivateRsync() { if (!rsync_cli_->IsIdle()) { return; } + LOG(WARNING) << "ActivateRsync ..."; if (rsync_cli_->Init()) { rsync_cli_->Start(); + LOG(WARNING) << "ActivateRsync done..."; + } else { + LOG(WARNING) << "ActivateRsync init failed..."; } } @@ -1156,7 +1160,7 @@ Status PikaReplicaManager::RunSyncSlaveSlotStateMachine() { std::shared_ptr slot = g_pika_server->GetDBSlotById(p_info.db_name_, p_info.slot_id_); if (slot) { - if (s_slot->IsRsyncIdle()) { + if (!s_slot->IsRsyncRunning()) { slot->TryUpdateMasterOffset(); } } else { diff --git a/src/pika_server.cc b/src/pika_server.cc index 53d0a7931..3384ac906 100644 --- a/src/pika_server.cc +++ b/src/pika_server.cc @@ -947,7 +947,7 @@ pstd::Status PikaServer::ReadDumpFile(const std::string& db_name, uint32_t slot_ } DEFER { close(fd); }; - const int kMaxCopyBlockSize = 8 << 10; + const int kMaxCopyBlockSize = 1 << 20; size_t read_offset = offset; size_t read_count = count; if (read_count > kMaxCopyBlockSize) { diff --git a/src/rsync_client.cc b/src/rsync_client.cc index cbcdb9291..1a19f5b02 100644 --- a/src/rsync_client.cc +++ b/src/rsync_client.cc @@ -14,13 +14,20 @@ extern PikaServer* g_pika_server; namespace rsync { RsyncClient::RsyncClient(const std::string& dir, const std::string& db_name, const uint32_t slot_id) - : dir_(dir), flush_period_(100), db_name_(db_name), slot_id_(slot_id), state_(IDLE), max_retries_(10) { + : dir_(dir), flush_period_(10), db_name_(db_name), slot_id_(slot_id), state_(IDLE), max_retries_(10) { client_thread_ = std::make_unique(10 * 1000, 60 * 1000, this); wo_.reset(new WaitObject()); throttle_.reset(new Throttle()); } bool RsyncClient::Init() { + if (state_ != IDLE) { + LOG(WARNING) << "State should be IDLE when Init"; + return false; + } + master_ip_ = g_pika_server->master_ip(); + master_port_ = g_pika_server->master_port() + kPortShiftRsync2; + file_set_.clear(); // todo client 的 StartThread 只能被调用一次,如果一个 slot 进行多次主从同步,这里会出问题吗? client_thread_->StartThread(); bool ret = Recover(); @@ -37,12 +44,18 @@ void* RsyncClient::ThreadMain() { int cnt = 0; int period = 0; Status s = Status::OK(); + LOG(INFO) << "RsyncClient ThreadMain..."; + if (file_set_.empty()) { + LOG(INFO) << "no newly files need to download..."; + state_.store(STOP); + LOG(INFO) << "RsyncClient ThreadMain done..."; + return nullptr; + } std::string meta_file_path = GetLocalMetaFilePath(); - int meta_fd = open(meta_file_path.c_str(), O_CREAT | O_RDWR, 0644); - std::string meta_rep(kUuidPrefix); - meta_rep.append(snapshot_uuid_); - meta_rep.append("\n"); + std::ofstream outfile; + outfile.open(meta_file_path, std::ios_base::app); // append instead of overwrite + std::string meta_rep; for (const auto& file : file_set_) { LOG(INFO) << "CopyRemoteFile: " << file << " state_: " << state_.load(); @@ -58,20 +71,22 @@ void* RsyncClient::ThreadMain() { if (state_.load(std::memory_order_relaxed) != RUNNING) { break; } + meta_rep.append(file + ":" + meta_table_[file]); + meta_rep.append("\n"); if (++period == flush_period_) { period = 0; - meta_rep.append(file + ":" + meta_table_[file]); - meta_rep.append("\n"); - write(meta_fd, meta_rep.data(), meta_rep.size()); - fsync(meta_fd); + LOG(INFO) << "periodically flush meta table..., meta_rep: " << meta_rep; + outfile << meta_rep; + outfile.flush(); meta_rep.clear(); } } - if (meta_rep.empty()) { - write(meta_fd, meta_rep.data(), meta_rep.size()); - fsync(meta_fd); + if (!meta_rep.empty()) { + outfile << meta_rep; + outfile.flush(); } - close(meta_fd); + state_.store(STOP); + LOG(INFO) << "RsyncClient ThreadMain done..."; return nullptr; } @@ -96,7 +111,11 @@ Status RsyncClient::Wait(RsyncResponse*& resp) { Status s = Status::Timeout("rsync timeout", "timeout"); { std::unique_lock lock(mu_); - cond_.wait_for(lock, std::chrono::seconds(3), [this] { return this->wo_->resp_ != nullptr; }); + auto cv_s = cond_.wait_for(lock, std::chrono::seconds(3), [this] { return this->wo_->resp_ != nullptr; }); + if (!cv_s) { + LOG(WARNING) << "wait timeout"; + return s; + } resp = wo_->resp_; s = Status::OK(); } @@ -133,16 +152,18 @@ Status RsyncClient::CopyRemoteFile(const std::string& filename) { std::string to_send; request.SerializeToString(&to_send); - LOG(WARNING) << "master ip: " << g_pika_server->master_ip() << " master_port: " << g_pika_server->master_port() << " portshift: " << kPortShiftRsync2; - s = client_thread_->Write(g_pika_server->master_ip(), g_pika_server->master_port() + kPortShiftRsync2, to_send); + LOG(WARNING) << "master ip: " << master_ip_ << " master_port: " << master_port_; + s = client_thread_->Write(master_ip_, master_port_, to_send); if (!s.ok()) { LOG(WARNING) << "send rsync request failed"; continue; } { + LOG(WARNING) << "reset waitobject"; std::lock_guard lock(mu_); wo_->Reset(filename, kRsyncFile, offset); + LOG(WARNING) << "reset waitobject done"; } LOG(INFO) << "wait CopyRemoteFile response....."; @@ -158,12 +179,14 @@ Status RsyncClient::CopyRemoteFile(const std::string& filename) { size_t elaspe_time_us = copy_file_end_time - copy_file_begin_time; throttle_->ReturnUnusedThroughput(count, resp->file_resp().count(), elaspe_time_us); + /* LOG(INFO) << "receive fileresponse, snapshot_uuid: " << resp->snapshot_uuid() << "filename: " << resp->file_resp().filename() << "offset: " << resp->file_resp().offset() << "count: " << resp->file_resp().count() << "eof: " << resp->file_resp().eof(); + */ if (resp->snapshot_uuid() != snapshot_uuid_) { - LOG(WARNING) << "receive newer dump, reset state to STOP"; + LOG(WARNING) << "receive newer dump, reset state to STOP, local_snapshot_uuid:" << snapshot_uuid_ << "remote snapshot uuid: " << resp->snapshot_uuid(); state_.store(STOP); delete resp; return s; @@ -207,7 +230,11 @@ Status RsyncClient::CopyRemoteFile(const std::string& filename) { } Status RsyncClient::Start() { + LOG(WARNING) << "RsyncClient Start ..."; + LOG(WARNING) << "RsyncClient StartThread ..."; + LOG(WARNING) << "RsyncClient shouldstop: " << should_stop() << " is running: " << is_running(); StartThread(); + LOG(WARNING) << "RsyncClient StartThread done..."; return Status::OK(); } @@ -217,9 +244,13 @@ Status RsyncClient::Stop() { } state_ = STOP; StopThread(); + LOG(WARNING) << "RsyncClient StopThread done..."; client_thread_->StopThread(); + LOG(WARNING) << "RsyncClient Stop clientThread done..."; JoinThread(); + LOG(WARNING) << "RsyncClient JoinThread done..."; client_thread_->JoinThread(); + LOG(WARNING) << "RsyncClient join clientThread done..."; state_ = IDLE; return Status::OK(); } @@ -247,11 +278,16 @@ bool RsyncClient::Recover() { } std::set expired_files; + LOG(WARNING) << "file_set origin size: " << file_set_.size() << " local_snapshot_uuid: " << local_snapshot_uuid << " remote_snapshot_uuid: " << remote_snapshot_uuid; if (remote_snapshot_uuid != local_snapshot_uuid) { + LOG(WARNING) << "snapshot uuid mismatch"; snapshot_uuid_ = remote_snapshot_uuid; file_set_ = remote_file_set; expired_files = local_file_set; } else { + LOG(WARNING) << "snapshot uuid equal, file_set_ size: " << file_set_.size() << "remote_file_set: " << remote_file_set.size() << " local_file_set: " << local_file_set.size(); + for_each(remote_file_set.begin(), remote_file_set.end(), [](auto& file) {LOG(WARNING) << "remote_file_set: " << file;}); + for_each(local_file_set.begin(), local_file_set.end(), [](auto& file) {LOG(WARNING) << "local_file_set: " << file;}); std::set newly_files; set_difference(remote_file_set.begin(), remote_file_set.end(), local_file_set.begin(), local_file_set.end(), inserter(newly_files, newly_files.begin())); @@ -259,6 +295,7 @@ bool RsyncClient::Recover() { inserter(expired_files, expired_files.begin())); file_set_.insert(newly_files.begin(), newly_files.end()); } + LOG(WARNING) << "file_set merged size: " << file_set_.size(); s = CleanUpExpiredFiles(local_snapshot_uuid != remote_snapshot_uuid, expired_files); if (!s.ok()) { @@ -275,7 +312,9 @@ bool RsyncClient::Recover() { LOG(INFO) << "copy meta data done, slot_id: " << slot_id_ << "snapshot_uuid: " << snapshot_uuid_ << "file count: " << file_set_.size() << "expired file count: " << expired_files.size() << ", local file count: " << local_file_set.size() << "remote file count: " << remote_file_set.size() - << "remote snapshot_uuid: " << remote_snapshot_uuid << "local snapshot_uuid: " << local_snapshot_uuid; + << "remote snapshot_uuid: " << remote_snapshot_uuid << "local snapshot_uuid: " << local_snapshot_uuid + << "file_set_: " << file_set_.size(); + for_each(file_set_.begin(), file_set_.end(), [](auto& file) {LOG(WARNING) << "file_set: " << file;}); return true; } @@ -289,8 +328,8 @@ Status RsyncClient::CopyRemoteMeta(std::string* snapshot_uuid, std::setmaster_ip() << " master_port: " << g_pika_server->master_port() << " portshift: " << kPortShiftRsync2; - s = client_thread_->Write(g_pika_server->master_ip(), g_pika_server->master_port() + kPortShiftRsync2, to_send); + LOG(WARNING) << "master ip: " << master_ip_ << " master_port: " << master_port_; + s = client_thread_->Write(master_ip_, master_port_, to_send); if (!s.ok()) { retries++; } @@ -376,6 +415,7 @@ Status RsyncClient::LoadLocalMeta(std::string* snapshot_uuid, std::mapAppend(kUuidPrefix + snapshot_uuid + "\n"); for (const auto& item : localFileMap) { - std::string line = item.first + item.second + "\n"; + std::string line = item.first + ":" + item.second + "\n"; file->Append(line); } s = file->Flush(); @@ -436,10 +476,4 @@ std::string RsyncClient::GetLocalMetaFilePath() { return db_path + kDumpMetaFileName; } -// TODO: shaoyi -Status RsyncClient::FlushMetaTable() { - LOG(WARNING) << "FlushMetaTable called"; - return Status::OK(); -} - } // end namespace rsync diff --git a/src/rsync_server.cc b/src/rsync_server.cc index db5c8a812..403e2ade6 100644 --- a/src/rsync_server.cc +++ b/src/rsync_server.cc @@ -189,7 +189,6 @@ bool RsyncServerThread::RsyncServerHandle::AccessHandle(int fd, std::string& ip_ } void RsyncServerThread::RsyncServerHandle::CronHandle() const { - LOG(WARNING) << "CronHandle called"; } } // end namespace rsync From 1918b900fa51c571fe72109ff48035c8e5d59020 Mon Sep 17 00:00:00 2001 From: wangshao1 <30471730+wangshao1@users.noreply.github.com> Date: Thu, 20 Jul 2023 10:12:50 +0800 Subject: [PATCH 22/37] rix rsync bugs (#194) * fix pika rsync bug --- CMakeLists.txt | 2 +- include/pika_rm.h | 2 + include/pika_server.h | 1 + include/pika_slot.h | 2 + include/rsync_client.h | 33 +++- include/rsync_server.h | 6 +- include/throttle.h | 4 +- src/pika_rm.cc | 13 +- src/pika_server.cc | 24 ++- src/pika_slot.cc | 31 ++-- src/rsync_client.cc | 335 +++++++++++++++++++++++------------------ src/rsync_server.cc | 58 +++---- 12 files changed, 304 insertions(+), 207 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 465a5354a..0e08c861e 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -47,7 +47,7 @@ elseif(${BUILD_TYPE} STREQUAL RELWITHDEBINFO) set(LIB_BUILD_TYPE RELWITHDEBINFO) else() set(LIB_BUILD_TYPE RElEASE) - set(CMAKE_CXX_FLAGS_RELEASE "-O2 -DNDEBUG") + set(CMAKE_CXX_FLAGS_RELEASE "-g -DNDEBUG") endif() if(CMAKE_SYSTEM_NAME MATCHES "Darwin") diff --git a/include/pika_rm.h b/include/pika_rm.h index 0d1c14755..50d5f4af9 100644 --- a/include/pika_rm.h +++ b/include/pika_rm.h @@ -160,6 +160,8 @@ class SyncSlaveSlot : public SyncSlot { void ActivateRsync(); + bool IsRsyncRunning() {return rsync_cli_->IsRunning();} + private: std::unique_ptr rsync_cli_; pstd::Mutex slot_mu_; diff --git a/include/pika_server.h b/include/pika_server.h index fcd17c3a2..414b468d1 100644 --- a/include/pika_server.h +++ b/include/pika_server.h @@ -263,6 +263,7 @@ class PikaServer : public pstd::noncopyable { //TODO: yuecai pstd::Status ReadDumpFile(const std::string& db_name, uint32_t slot_id, const std::string& filename, const size_t offset, const size_t count, char* data, size_t* bytes_read); + pstd::Status GetDumpUUID(const std::string& db_name, const uint32_t slot_id, std::string* snapshot_uuid); //TODO: yuecai pstd::Status GetDumpMeta(const std::string& db_name, const uint32_t slot_id, std::vector* files, std::string* snapshot_uuid); void DBSync(const std::string& ip, int port, const std::string& db_name, uint32_t slot_id); diff --git a/include/pika_slot.h b/include/pika_slot.h index c41b906ca..617805c46 100644 --- a/include/pika_slot.h +++ b/include/pika_slot.h @@ -77,6 +77,7 @@ class Slot : public std::enable_shared_from_this,public pstd::noncopyable void BgSaveSlot(); BgSaveInfo bgsave_info(); void GetBgSaveMetaData(std::vector* fileNames, std::string* snapshot_uuid); + pstd::Status GetBgSaveUUID(std::string* snapshot_uuid); // FlushDB & FlushSubDB use bool FlushDB(); @@ -94,6 +95,7 @@ class Slot : public std::enable_shared_from_this,public pstd::noncopyable private: std::string db_name_; uint32_t slot_id_ = 0; + std::string snapshot_uuid_; std::string db_path_; std::string bgsave_sub_path_; diff --git a/include/rsync_client.h b/include/rsync_client.h index 9a51d294a..d38fdd483 100644 --- a/include/rsync_client.h +++ b/include/rsync_client.h @@ -48,12 +48,16 @@ class RsyncClient : public net::Thread { bool Init(); Status Start(); Status Stop(); - bool IsRunning() { return state_.load() == RUNNING;} + bool IsRunning() { + LOG(WARNING) << "current state_: " << state_.load(); + return state_.load() == RUNNING; + } + bool IsIdle() { return state_.load() == IDLE;} void OnReceive(RsyncResponse* resp); private: bool Recover(); - Status Wait(WaitObject* wo); + Status Wait(RsyncResponse*& resp); Status CopyRemoteFile(const std::string& filename); Status CopyRemoteMeta(std::string* snapshot_uuid, std::set* file_set); Status LoadLocalMeta(std::string* snapshot_uuid, std::map* file_map); @@ -78,10 +82,13 @@ class RsyncClient : public net::Thread { std::atomic state_; int max_retries_; - std::list resp_list_; + std::unique_ptr wo_; std::condition_variable cond_; std::mutex mu_; std::unique_ptr throttle_; + + std::string master_ip_; + int master_port_; }; //TODO: jinge @@ -126,9 +133,21 @@ class RsyncWriter { class WaitObject { public: - WaitObject(const std::string& filename, RsyncService::Type t, size_t offset) - : filename_(filename), type_(t), offset_(offset), resp_(nullptr) {} - WaitObject(RsyncService::Type t) : filename_(""), type_(t), offset_(-1), resp_(nullptr) {} + WaitObject() : filename_(""), type_(RsyncService::kRsyncMeta), offset_(0), resp_(nullptr) {} + ~WaitObject() {} + void Reset(const std::string& filename, RsyncService::Type t, size_t offset) { + resp_ = nullptr; + filename_ = filename; + type_ = t; + offset_ = offset; + } + + void Reset(RsyncService::Type t) { + resp_ = nullptr; + filename_ = ""; + type_ = t; + offset_ = 0xFFFFFFFF; + } std::string filename_; RsyncService::Type type_; size_t offset_; @@ -137,4 +156,4 @@ class WaitObject { } // end namespace rsync -#endif \ No newline at end of file +#endif diff --git a/include/rsync_server.h b/include/rsync_server.h index f87a1df8d..1c8fb4f05 100644 --- a/include/rsync_server.h +++ b/include/rsync_server.h @@ -33,14 +33,12 @@ class RsyncServerThread; class RsyncServer { public: - RsyncServer(const std::string& ip_port, const int port); + RsyncServer(const std::set& ips, const int port); ~RsyncServer(); void Schedule(net::TaskFunc func, void* arg); int Start(); int Stop(); private: - int port_; - std::string ip_; std::map > file_map_; std::unique_ptr work_thread_ = nullptr; std::unique_ptr rsync_server_thread_ = nullptr; @@ -117,4 +115,4 @@ class RSyncReader { }; } //end namespace rsync -#endif \ No newline at end of file +#endif diff --git a/include/throttle.h b/include/throttle.h index b63795bc7..2ffa52a9b 100644 --- a/include/throttle.h +++ b/include/throttle.h @@ -6,7 +6,7 @@ namespace rsync { class Throttle { public: - Throttle(); + Throttle() {} Throttle(size_t throttle_throughput_bytes, size_t check_cycle); ~Throttle(); size_t ThrottledByThroughput(size_t bytes); @@ -27,4 +27,4 @@ class Throttle { }; } // end namespace rsync -#endif \ No newline at end of file +#endif diff --git a/src/pika_rm.cc b/src/pika_rm.cc index 4c96a76be..c5858c81a 100644 --- a/src/pika_rm.cc +++ b/src/pika_rm.cc @@ -602,6 +602,7 @@ void SyncSlaveSlot::Deactivate() { std::lock_guard l(slot_mu_); m_info_ = RmNode(); repl_state_ = ReplState::kNoConnect; + rsync_cli_->Stop(); } std::string SyncSlaveSlot::ToStringStatus() { @@ -641,11 +642,15 @@ std::string SyncSlaveSlot::LocalIp() { } void SyncSlaveSlot::ActivateRsync() { - if (rsync_cli_->IsRunning()) { + if (!rsync_cli_->IsIdle()) { return; } + LOG(WARNING) << "ActivateRsync ..."; if (rsync_cli_->Init()) { rsync_cli_->Start(); + LOG(WARNING) << "ActivateRsync done..."; + } else { + LOG(WARNING) << "ActivateRsync init failed..."; } } @@ -1151,11 +1156,13 @@ Status PikaReplicaManager::RunSyncSlaveSlotStateMachine() { } else if (s_slot->State() == ReplState::kWaitReply) { continue; } else if (s_slot->State() == ReplState::kWaitDBSync) { + s_slot->ActivateRsync(); std::shared_ptr slot = g_pika_server->GetDBSlotById(p_info.db_name_, p_info.slot_id_); if (slot) { - s_slot->ActivateRsync(); - slot->TryUpdateMasterOffset(); + if (!s_slot->IsRsyncRunning()) { + slot->TryUpdateMasterOffset(); + } } else { LOG(WARNING) << "Slot not found, DB Name: " << p_info.db_name_ << " Slot Id: " << p_info.slot_id_; diff --git a/src/pika_server.cc b/src/pika_server.cc index 0cdbbe4a0..3384ac906 100644 --- a/src/pika_server.cc +++ b/src/pika_server.cc @@ -72,11 +72,12 @@ PikaServer::PikaServer() // We estimate the queue size int worker_queue_limit = g_pika_conf->maxclients() / worker_num_ + 100; LOG(INFO) << "Worker queue limit is " << worker_queue_limit; + for_each(ips.begin(), ips.end(), [](auto& ip) {LOG(WARNING) << ip;}); pika_dispatch_thread_ = std::make_unique(ips, port_, worker_num_, 3000, worker_queue_limit, g_pika_conf->max_conn_rbuf_size()); pika_rsync_service_ = std::make_unique(g_pika_conf->db_sync_path(), g_pika_conf->port() + kPortShiftRSync); //TODO 删除pika_rsync_service_服务,使用pika_rsync_service_端口 - rsync_server_ = std::make_unique("127.0.0.1", g_pika_server->master_port() + kPortShiftRsync2); + rsync_server_ = std::make_unique(ips, port_ + kPortShiftRsync2); pika_pubsub_thread_ = std::make_unique(); pika_auxiliary_thread_ = std::make_unique(); pika_migrate_ = std::make_unique(); @@ -134,7 +135,7 @@ bool PikaServer::ServerInit() { void PikaServer::Start() { int ret = 0; // start rsync first, rocksdb opened fd will not appear in this fork - ret = pika_rsync_service_->StartRsync(); + //ret = pika_rsync_service_->StartRsync(); if (0 != ret) { dbs_.clear(); LOG(FATAL) << "Start Rsync Error: bind port " + std::to_string(pika_rsync_service_->ListenPort()) + " failed" @@ -911,6 +912,16 @@ void PikaServer::DBSync(const std::string& ip, int port, const std::string& db_n bgsave_thread_.Schedule(&DoDBSync, reinterpret_cast(arg)); } +pstd::Status PikaServer::GetDumpUUID(const std::string& db_name, const uint32_t slot_id, std::string* snapshot_uuid) { + std::shared_ptr slot = GetDBSlotById(db_name, slot_id); + if (!slot) { + LOG(WARNING) << "cannot find slot for db_name " << db_name << "slot_id: " << slot_id; + return pstd::Status::NotFound("slot no found"); + } + slot->GetBgSaveUUID(snapshot_uuid); + return pstd::Status::OK(); +} + pstd::Status PikaServer::GetDumpMeta(const std::string& db_name, const uint32_t slot_id, std::vector* fileNames, std::string* snapshot_uuid) { std::shared_ptr slot = GetDBSlotById(db_name, slot_id); if (!slot) { @@ -936,19 +947,19 @@ pstd::Status PikaServer::ReadDumpFile(const std::string& db_name, uint32_t slot_ } DEFER { close(fd); }; - const int kMaxCopyBlockSize = 8 << 10; + const int kMaxCopyBlockSize = 1 << 20; size_t read_offset = offset; size_t read_count = count; if (read_count > kMaxCopyBlockSize) { read_count = kMaxCopyBlockSize; } - size_t bytesin = 0; + ssize_t bytesin = 0; size_t left_read_count = count; while ((bytesin = pread(fd, data, read_count, read_offset)) > 0) { left_read_count -= bytesin; - if (left_read_count <= 0) { + if (left_read_count < 0) { break ; } if (read_count > left_read_count) { @@ -991,6 +1002,7 @@ void PikaServer::TryDBSync(const std::string& ip, int port, const std::string& d // Need Bgsave first slot->BgSaveSlot(); } + return; DBSync(ip, port, db_name, slot_id); } @@ -1322,7 +1334,7 @@ void PikaServer::DoTimingTask() { // Delete expired dump AutoDeleteExpiredDump(); // Cheek Rsync Status - AutoKeepAliveRSync(); + //AutoKeepAliveRSync(); // Reset server qps ResetLastSecQuerynum(); } diff --git a/src/pika_slot.cc b/src/pika_slot.cc index 184316c8d..b871767ce 100644 --- a/src/pika_slot.cc +++ b/src/pika_slot.cc @@ -300,8 +300,6 @@ BgSaveInfo Slot::bgsave_info() { void Slot::GetBgSaveMetaData(std::vector* fileNames, std::string* snapshot_uuid) { const std::string slotPath = bgsave_info().path; - // todo 待确认 info 文件的路径 - const std::string infoPath = bgsave_info().path + "/info"; std::string types[] = {storage::STRINGS_DB, storage::HASHES_DB, storage::LISTS_DB, storage::ZSETS_DB, storage::SETS_DB}; for (const auto& type : types) { @@ -321,17 +319,30 @@ void Slot::GetBgSaveMetaData(std::vector* fileNames, std::string* s fileNames -> push_back(type + "/" + fileName); } } - - std::string info_data; - // todo 这里待替换 - rocksdb::Status s = rocksdb::ReadFileToString(rocksdb::Env::Default(), infoPath, &info_data); + fileNames->push_back(kBgsaveInfoFile); + pstd::Status s = GetBgSaveUUID(snapshot_uuid); if (!s.ok()) { - LOG(WARNING) << "read dump meta info failed! error:" << s.ToString(); - return; + LOG(WARNING) << "read dump meta info failed! error:" << s.ToString(); + return; } +} - pstd::MD5 md5 = pstd::MD5(info_data); - *snapshot_uuid = md5.hexdigest(); +Status Slot::GetBgSaveUUID(std::string* snapshot_uuid) { + if (snapshot_uuid_.empty()) { + std::string info_data; + // todo 待确认 info 文件的路径 + const std::string infoPath = bgsave_info().path + "/info"; + // todo 这里待替换 + rocksdb::Status s = rocksdb::ReadFileToString(rocksdb::Env::Default(), infoPath, &info_data); + if (!s.ok()) { + LOG(WARNING) << "read dump meta info failed! error:" << s.ToString(); + return Status::IOError("read dump meta info failed", infoPath); + } + pstd::MD5 md5 = pstd::MD5(info_data); + snapshot_uuid_ = md5.hexdigest(); + } + *snapshot_uuid = snapshot_uuid_; + return Status::OK(); } void Slot::DoBgSave(void* arg) { diff --git a/src/rsync_client.cc b/src/rsync_client.cc index 0975080ba..1a19f5b02 100644 --- a/src/rsync_client.cc +++ b/src/rsync_client.cc @@ -14,11 +14,20 @@ extern PikaServer* g_pika_server; namespace rsync { RsyncClient::RsyncClient(const std::string& dir, const std::string& db_name, const uint32_t slot_id) - : dir_(dir), flush_period_(100), db_name_(db_name), slot_id_(slot_id), state_(IDLE), max_retries_(10) { + : dir_(dir), flush_period_(10), db_name_(db_name), slot_id_(slot_id), state_(IDLE), max_retries_(10) { client_thread_ = std::make_unique(10 * 1000, 60 * 1000, this); + wo_.reset(new WaitObject()); + throttle_.reset(new Throttle()); } bool RsyncClient::Init() { + if (state_ != IDLE) { + LOG(WARNING) << "State should be IDLE when Init"; + return false; + } + master_ip_ = g_pika_server->master_ip(); + master_port_ = g_pika_server->master_port() + kPortShiftRsync2; + file_set_.clear(); // todo client 的 StartThread 只能被调用一次,如果一个 slot 进行多次主从同步,这里会出问题吗? client_thread_->StartThread(); bool ret = Recover(); @@ -32,86 +41,83 @@ bool RsyncClient::Init() { } void* RsyncClient::ThreadMain() { - int cnt = 0; - int period = 0; - Status s = Status::OK(); - std::string meta_file_path = dir_ + "/" + kDumpMetaFileName; - int meta_fd = open(meta_file_path.c_str(), O_CREAT | O_RDWR, 0644); - std::string meta_rep(kUuidPrefix); - meta_rep.append(snapshot_uuid_); + int cnt = 0; + int period = 0; + Status s = Status::OK(); + LOG(INFO) << "RsyncClient ThreadMain..."; + if (file_set_.empty()) { + LOG(INFO) << "no newly files need to download..."; + state_.store(STOP); + LOG(INFO) << "RsyncClient ThreadMain done..."; + return nullptr; + } + + std::string meta_file_path = GetLocalMetaFilePath(); + std::ofstream outfile; + outfile.open(meta_file_path, std::ios_base::app); // append instead of overwrite + std::string meta_rep; + + for (const auto& file : file_set_) { + LOG(INFO) << "CopyRemoteFile: " << file << " state_: " << state_.load(); + while (state_.load() == RUNNING) { + s = CopyRemoteFile(file); + if (!s.ok()) { + LOG(WARNING) << "rsync CopyRemoteFile failed, filename: " << file; + continue; + } + LOG(WARNING) << "CopyRemoteFile "<< file << " success..."; + break; + } + if (state_.load(std::memory_order_relaxed) != RUNNING) { + break; + } + meta_rep.append(file + ":" + meta_table_[file]); meta_rep.append("\n"); - while (state_.load(std::memory_order_relaxed) == RUNNING) { - for (const auto& file : file_set_) { - LOG(INFO) << "CopyRemoteFile: " << file; - while (state_.load() == RUNNING) { - s = CopyRemoteFile(file); - if (!s.ok()) { - LOG(WARNING) << "rsync CopyRemoteFile failed, filename: " << file; - continue; - } - LOG(WARNING) << "CopyRemoteFile "<< file << "success..."; - break; - } - if (state_.load(std::memory_order_relaxed) != RUNNING) { - break; - } - if (++period == flush_period_) { - period = 0; - meta_rep.append(file + ":" + meta_table_[file]); - meta_rep.append("\n"); - write(meta_fd, meta_rep.data(), meta_rep.size()); - fsync(meta_fd); - meta_rep.clear(); - } - } - if (meta_table_.size() == file_set_.size()) { - LOG(INFO) << "rsync success..."; - state_.store(STOP, std::memory_order_relaxed); - break; - } + if (++period == flush_period_) { + period = 0; + LOG(INFO) << "periodically flush meta table..., meta_rep: " << meta_rep; + outfile << meta_rep; + outfile.flush(); + meta_rep.clear(); } - close(meta_fd); - return nullptr; + } + if (!meta_rep.empty()) { + outfile << meta_rep; + outfile.flush(); + } + state_.store(STOP); + LOG(INFO) << "RsyncClient ThreadMain done..."; + return nullptr; } void RsyncClient::OnReceive(RsyncResponse* resp) { std::unique_lock lock(mu_); - resp_list_.push_back(resp); + if (resp->type() != wo_->type_) { + delete resp; + resp = nullptr; + return; + } + if (resp->type() == kRsyncFile && + (resp->file_resp().filename() != wo_->filename_ || resp->file_resp().offset() != wo_->offset_)) { + delete resp; + resp = nullptr; + return; + } + wo_->resp_ = resp; cond_.notify_all(); } -Status RsyncClient::Wait(WaitObject* wo) { +Status RsyncClient::Wait(RsyncResponse*& resp) { Status s = Status::Timeout("rsync timeout", "timeout"); - std::list resp_list; { std::unique_lock lock(mu_); - cond_.wait_for(lock, std::chrono::seconds(3), [this] { return !resp_list_.empty(); }); - resp_list.swap(resp_list_); - } - - auto iter = resp_list.begin(); - while (iter != resp_list.end()) { - RsyncResponse* resp = *iter; - if (resp->type() != wo->type_) { - LOG(WARNING) << "mismatch request/response type, skip"; - iter++; - continue; - } - if (resp->type() == kRsyncFile && - (resp->file_resp().filename() != wo->filename_ || resp->file_resp().offset() != wo->offset_)) { - LOG(WARNING) << "mismatch rsync response, skip"; - continue; + auto cv_s = cond_.wait_for(lock, std::chrono::seconds(3), [this] { return this->wo_->resp_ != nullptr; }); + if (!cv_s) { + LOG(WARNING) << "wait timeout"; + return s; } + resp = wo_->resp_; s = Status::OK(); - wo->resp_ = resp; - resp_list.erase(iter); - break; - } - - iter = resp_list.begin(); - while (iter != resp_list.end()) { - delete (*iter); - iter++; } return s; } @@ -121,7 +127,7 @@ Status RsyncClient::CopyRemoteFile(const std::string& filename) { int retries = 0; size_t offset = 0; size_t copy_file_begin_time = pstd::NowMicros(); - size_t count = throttle_->ThrottledByThroughput(1024 * 1024); + size_t count = throttle_->ThrottledByThroughput(4 * 1024 * 1024); MD5 md5; std::unique_ptr writer(new RsyncWriter(dir_ + "/" + filename)); DEFER { @@ -133,94 +139,118 @@ Status RsyncClient::CopyRemoteFile(const std::string& filename) { DeleteFile(filename); } }; + while (retries < max_retries_) { - RsyncRequest request; - request.set_type(kRsyncFile); - request.set_db_name(db_name_); - request.set_slot_id(slot_id_); - FileRequest* file_req = request.mutable_file_req(); - file_req->set_filename(filename); - file_req->set_offset(offset); - file_req->set_count(count); - std::string to_send; - request.SerializeToString(&to_send); - - s = client_thread_->Write(g_pika_server->master_ip(), g_pika_server->master_port() + kPortShiftRsync2, to_send); - if (!s.ok()) { - LOG(WARNING) << "send rsync request failed"; - continue; - } + RsyncRequest request; + request.set_type(kRsyncFile); + request.set_db_name(db_name_); + request.set_slot_id(slot_id_); + FileRequest* file_req = request.mutable_file_req(); + file_req->set_filename(filename); + file_req->set_offset(offset); + file_req->set_count(count); + std::string to_send; + request.SerializeToString(&to_send); + + LOG(WARNING) << "master ip: " << master_ip_ << " master_port: " << master_port_; + s = client_thread_->Write(master_ip_, master_port_, to_send); + if (!s.ok()) { + LOG(WARNING) << "send rsync request failed"; + continue; + } - WaitObject wo(filename, kRsyncFile, offset); - LOG(INFO) << "wait CopyRemoteFile response....."; - s = Wait(&wo); - if (s.IsTimeout() || wo.resp_ == nullptr) { - LOG(WARNING) << "rsync request timeout"; - retries++; - continue; - } - RsyncResponse* resp = wo.resp_; + { + LOG(WARNING) << "reset waitobject"; + std::lock_guard lock(mu_); + wo_->Reset(filename, kRsyncFile, offset); + LOG(WARNING) << "reset waitobject done"; + } - LOG(INFO) << "receive fileresponse, snapshot_uuid: " << resp->snapshot_uuid() - << "filename: " << resp->file_resp().filename() << "offset: " << resp->file_resp().offset() - << "count: " << resp->file_resp().count(); + LOG(INFO) << "wait CopyRemoteFile response....."; + RsyncResponse* resp = nullptr; + s = Wait(resp); + if (s.IsTimeout() || resp == nullptr) { + LOG(WARNING) << "rsync request timeout"; + retries++; + continue; + } - if (resp->snapshot_uuid() != snapshot_uuid_) { - LOG(WARNING) << "receive newer dump, reset state to STOP"; - state_.store(STOP); - delete resp; - return s; - } + size_t copy_file_end_time = pstd::NowMicros(); + size_t elaspe_time_us = copy_file_end_time - copy_file_begin_time; + throttle_->ReturnUnusedThroughput(count, resp->file_resp().count(), elaspe_time_us); + + /* + LOG(INFO) << "receive fileresponse, snapshot_uuid: " << resp->snapshot_uuid() + << "filename: " << resp->file_resp().filename() << "offset: " << resp->file_resp().offset() + << "count: " << resp->file_resp().count() << "eof: " << resp->file_resp().eof(); + */ + + if (resp->snapshot_uuid() != snapshot_uuid_) { + LOG(WARNING) << "receive newer dump, reset state to STOP, local_snapshot_uuid:" << snapshot_uuid_ << "remote snapshot uuid: " << resp->snapshot_uuid(); + state_.store(STOP); + delete resp; + return s; + } - size_t ret_count = resp->file_resp().count(); - resp->file_resp().data(); - s = writer->Write((uint64_t)offset, ret_count, resp->file_resp().data().c_str()); - if (!s.ok()) { - LOG(WARNING) << "rsync client write file error"; - break; - } + size_t ret_count = resp->file_resp().count(); + resp->file_resp().data(); + s = writer->Write((uint64_t)offset, ret_count, resp->file_resp().data().c_str()); + if (!s.ok()) { + LOG(WARNING) << "rsync client write file error"; + break; + } - md5.update(resp->file_resp().data().c_str(), ret_count); - if (resp->file_resp().eof()) { - if (md5.finalize().hexdigest() != resp->file_resp().checksum()) { - LOG(WARNING) << "mismatch file checksum for file: " << filename; - //TODO: 处理返回status - s = Status::IOError("mismatch checksum", "mismatch checksum"); - } - s = writer->Fsync(); - if (!s.ok()) { - return s; - } - s = writer->Close(); - if (!s.ok()) { - return s; - } - writer.reset(); - meta_table_[filename] = resp->file_resp().checksum(); - break; - } else { - offset += resp->file_resp().count(); + md5.update(resp->file_resp().data().c_str(), ret_count); + offset += resp->file_resp().count(); + if (resp->file_resp().eof()) { + /* + if (md5.finalize().hexdigest() != resp->file_resp().checksum()) { + LOG(WARNING) << "mismatch file checksum for file: " << filename; + //TODO: wangshaoyi处理返回status + s = Status::IOError("mismatch checksum", "mismatch checksum"); + return s; + } + */ + s = writer->Fsync(); + if (!s.ok()) { + return s; + } + s = writer->Close(); + if (!s.ok()) { + return s; } - size_t copy_file_end_time = pstd::NowMicros(); - size_t elaspe_time_us = copy_file_end_time - copy_file_begin_time; - throttle_->ReturnUnusedThroughput(count, ret_count, elaspe_time_us); - retries = 0; + writer.reset(); + meta_table_[filename] = resp->file_resp().checksum(); + break; + } + retries = 0; } return s; } Status RsyncClient::Start() { + LOG(WARNING) << "RsyncClient Start ..."; + LOG(WARNING) << "RsyncClient StartThread ..."; + LOG(WARNING) << "RsyncClient shouldstop: " << should_stop() << " is running: " << is_running(); StartThread(); + LOG(WARNING) << "RsyncClient StartThread done..."; return Status::OK(); } Status RsyncClient::Stop() { + if (state_ == IDLE) { + return Status::OK(); + } state_ = STOP; StopThread(); + LOG(WARNING) << "RsyncClient StopThread done..."; client_thread_->StopThread(); + LOG(WARNING) << "RsyncClient Stop clientThread done..."; JoinThread(); + LOG(WARNING) << "RsyncClient JoinThread done..."; client_thread_->JoinThread(); + LOG(WARNING) << "RsyncClient join clientThread done..."; state_ = IDLE; return Status::OK(); } @@ -248,11 +278,16 @@ bool RsyncClient::Recover() { } std::set expired_files; + LOG(WARNING) << "file_set origin size: " << file_set_.size() << " local_snapshot_uuid: " << local_snapshot_uuid << " remote_snapshot_uuid: " << remote_snapshot_uuid; if (remote_snapshot_uuid != local_snapshot_uuid) { + LOG(WARNING) << "snapshot uuid mismatch"; snapshot_uuid_ = remote_snapshot_uuid; file_set_ = remote_file_set; expired_files = local_file_set; } else { + LOG(WARNING) << "snapshot uuid equal, file_set_ size: " << file_set_.size() << "remote_file_set: " << remote_file_set.size() << " local_file_set: " << local_file_set.size(); + for_each(remote_file_set.begin(), remote_file_set.end(), [](auto& file) {LOG(WARNING) << "remote_file_set: " << file;}); + for_each(local_file_set.begin(), local_file_set.end(), [](auto& file) {LOG(WARNING) << "local_file_set: " << file;}); std::set newly_files; set_difference(remote_file_set.begin(), remote_file_set.end(), local_file_set.begin(), local_file_set.end(), inserter(newly_files, newly_files.begin())); @@ -260,6 +295,7 @@ bool RsyncClient::Recover() { inserter(expired_files, expired_files.begin())); file_set_.insert(newly_files.begin(), newly_files.end()); } + LOG(WARNING) << "file_set merged size: " << file_set_.size(); s = CleanUpExpiredFiles(local_snapshot_uuid != remote_snapshot_uuid, expired_files); if (!s.ok()) { @@ -276,7 +312,9 @@ bool RsyncClient::Recover() { LOG(INFO) << "copy meta data done, slot_id: " << slot_id_ << "snapshot_uuid: " << snapshot_uuid_ << "file count: " << file_set_.size() << "expired file count: " << expired_files.size() << ", local file count: " << local_file_set.size() << "remote file count: " << remote_file_set.size() - << "remote snapshot_uuid: " << remote_snapshot_uuid << "local snapshot_uuid: " << local_snapshot_uuid; + << "remote snapshot_uuid: " << remote_snapshot_uuid << "local snapshot_uuid: " << local_snapshot_uuid + << "file_set_: " << file_set_.size(); + for_each(file_set_.begin(), file_set_.end(), [](auto& file) {LOG(WARNING) << "file_set: " << file;}); return true; } @@ -290,18 +328,22 @@ Status RsyncClient::CopyRemoteMeta(std::string* snapshot_uuid, std::setWrite(g_pika_server->master_ip(), g_pika_server->master_port() + kPortShiftRsync2, to_send); + LOG(WARNING) << "master ip: " << master_ip_ << " master_port: " << master_port_; + s = client_thread_->Write(master_ip_, master_port_, to_send); if (!s.ok()) { retries++; } - WaitObject wo(kRsyncMeta); - s = Wait(&wo); - if (s.IsTimeout() || wo.resp_ == nullptr) { + { + std::lock_guard lock(mu_); + wo_->Reset(kRsyncMeta); + } + RsyncResponse* resp = nullptr; + s = Wait(resp); + if (s.IsTimeout() || resp == nullptr) { LOG(WARNING) << "rsync CopyRemoteMeta request timeout, retry times: " << retries; retries++; continue; } - RsyncResponse* resp = wo.resp_; LOG(INFO) << "receive rsync meta infos, snapshot_uuid: " << resp->snapshot_uuid() << "files count: " << resp->meta_resp().filenames_size(); @@ -312,6 +354,8 @@ Status RsyncClient::CopyRemoteMeta(std::string* snapshot_uuid, std::setmeta_resp().filenames_size(); i++) { file_set->insert(resp->meta_resp().filenames(i)); } + delete resp; + resp = nullptr; break; } return s; @@ -371,6 +415,7 @@ Status RsyncClient::LoadLocalMeta(std::string* snapshot_uuid, std::map& expired_files, std::map& localFileMap) { - localFileMap[kUuidPrefix] = snapshot_uuid; for (const auto& item : expired_files) { localFileMap.erase(item); } @@ -413,9 +457,10 @@ Status RsyncClient::UpdateLocalMeta(std::string& snapshot_uuid, std::setAppend(kUuidPrefix + snapshot_uuid + "\n"); for (const auto& item : localFileMap) { - std::string line = item.first + item.second + "\n"; + std::string line = item.first + ":" + item.second + "\n"; file->Append(line); } s = file->Flush(); @@ -431,10 +476,4 @@ std::string RsyncClient::GetLocalMetaFilePath() { return db_path + kDumpMetaFileName; } -// TODO: shaoyi -Status RsyncClient::FlushMetaTable() { - LOG(WARNING) << "FlushMetaTable called"; - return Status::OK(); -} - -} // end namespace rsync \ No newline at end of file +} // end namespace rsync diff --git a/src/rsync_server.cc b/src/rsync_server.cc index c83cfd729..403e2ade6 100644 --- a/src/rsync_server.cc +++ b/src/rsync_server.cc @@ -7,9 +7,9 @@ extern PikaServer* g_pika_server; namespace rsync { -RsyncServer::RsyncServer(const std::string& ip, const int port) : ip_(ip), port_(port) { +RsyncServer::RsyncServer(const std::set& ips, const int port) { work_thread_ = std::make_unique(2, 100000); - std::set ips = {ip_}; + for_each(ips.begin(), ips.end(), [&port](auto& ip) {LOG(WARNING) << ip << "port: " << port;}); rsync_server_thread_ = std::make_unique(ips, port, 60 * 1000, this); } @@ -23,8 +23,8 @@ void RsyncServer::Schedule(net::TaskFunc func, void* arg) { } int RsyncServer::Start() { + LOG(INFO) << "start RsyncServer ..."; int res = rsync_server_thread_->StartThread(); - LOG(WARNING) << "after RsyncServer::Start"; if (res != net::kSuccess) { LOG(FATAL) << "Start rsync Server Thread Error: " << res; } @@ -33,12 +33,15 @@ int RsyncServer::Start() { LOG(FATAL) << "Start ThreadPool Error: " << res << (res == net::kCreateThreadError ? ": create thread error " : ": other error"); } + LOG(INFO) << "start RsyncServer done ..."; return res; } int RsyncServer::Stop() { + LOG(INFO) << "stop RsyncServer ..."; work_thread_->stop_thread_pool(); rsync_server_thread_->StopThread(); + LOG(INFO) << "stop RsyncServer done..."; return 0; } @@ -53,7 +56,7 @@ RsyncServerConn::~RsyncServerConn() { int RsyncServerConn::DealMessage() { std::shared_ptr req = std::make_shared(); bool parse_res = req->ParseFromArray(rbuf_ + cur_pos_ - header_len_, header_len_); - LOG(WARNING) << "RsyncServer DealMessage..."; + LOG(INFO) << "RsyncServer receives new request..."; if (!parse_res) { LOG(WARNING) << "Pika rsync server connection pb parse error."; return -1; @@ -82,22 +85,22 @@ void RsyncServerConn::HandleMetaRsyncRequest(void* arg) { std::unique_ptr task_arg(static_cast(arg)); const std::shared_ptr req = task_arg->req; std::shared_ptr conn = task_arg->conn; + std::string db_name = req->db_name(); + uint32_t slot_id = req->slot_id(); RsyncService::RsyncResponse response; - response.set_db_name("db_name"); - response.set_slot_id(0); + response.set_db_name(db_name); + response.set_slot_id(slot_id); response.set_type(RsyncService::kRsyncMeta); - LOG(INFO) << "Receive RsyncMeta request"; + LOG(INFO) << "RsyncServer receives RsyncMeta request..."; - std::string db_name = req->db_name(); - uint32_t slot_id = req->slot_id(); std::vector filenames; std::string snapshot_uuid; g_pika_server->GetDumpMeta(db_name, slot_id, &filenames, &snapshot_uuid); LOG(WARNING) << "snapshot_uuid: " << snapshot_uuid; std::for_each(filenames.begin(), filenames.end(), [](auto& file) { - LOG(WARNING) << "file:" << file; + LOG(WARNING) << "meta file name: " << file; }); //TODO: temporarily mock response RsyncService::MetaResponse* meta_resp = response.mutable_meta_resp(); @@ -113,43 +116,46 @@ void RsyncServerConn::HandleMetaRsyncRequest(void* arg) { return; } conn->NotifyWrite(); + LOG(INFO) << "RsyncServer RsyncMeta request done..."; } void RsyncServerConn::HandleFileRsyncRequest(void* arg) { std::unique_ptr task_arg(static_cast(arg)); const std::shared_ptr req = task_arg->req; std::shared_ptr conn = task_arg->conn; + LOG(INFO) << "RsyncServer RsyncFile request ..."; - RsyncService::RsyncResponse response; - response.set_type(RsyncService::kRsyncFile); - response.set_snapshot_uuid("demo_snapshot_uuid"); - response.set_db_name("db_name"); - response.set_slot_id(0); - LOG(INFO) << "Receive RsyncFile request " << "filename: " << req->file_req().filename() - << " offset: " << req->file_req().offset() - << " count: " << req->file_req().count(); - + uint32_t slot_id = req->slot_id(); std::string db_name = req->db_name(); std::string filename = req->file_req().filename(); - uint32_t slot_id = req->slot_id(); size_t offset = req->file_req().offset(); size_t count = req->file_req().count(); + RsyncService::RsyncResponse response; + std::string snapshot_uuid; + Status s = g_pika_server->GetDumpUUID(db_name, slot_id, &snapshot_uuid); + LOG(INFO) << "Receive RsyncFile request " << "filename: " << filename + << " offset: " << offset + << " count: " << count; + char* buffer = new char[req->file_req().count() + 1]; size_t bytes_read{0}; - LOG(INFO) << "....... ReadDumpFile: " << filename; auto status = g_pika_server -> ReadDumpFile(db_name, slot_id, filename, offset, count, buffer, &bytes_read); - LOG(INFO) << "ReadDumpFile: " << filename << " read size: " << status.ToString(); + LOG(INFO) << "RsyncServer ReadDumpFile: " << filename << " read size: " << bytes_read << "status: " << status.ToString(); + + response.set_type(RsyncService::kRsyncFile); + response.set_snapshot_uuid(snapshot_uuid); + response.set_db_name(db_name); + response.set_slot_id(slot_id); - //TODO: temporarily mock response RsyncService::FileResponse* file_resp = response.mutable_file_resp(); file_resp->set_eof(bytes_read != count); file_resp->set_count(bytes_read); file_resp->set_offset(offset); file_resp->set_data(buffer, bytes_read); + //TODO: checksum file_resp->set_checksum("checksum"); file_resp->set_filename(filename); - LOG(INFO) << "....... before serializetostring: " << filename; std::string reply_str; if (!response.SerializeToString(&reply_str) || (conn->WriteResp(reply_str) != 0)) { LOG(WARNING) << "Process FileRsync request serialization failed"; @@ -159,6 +165,7 @@ void RsyncServerConn::HandleFileRsyncRequest(void* arg) { } delete []buffer; conn->NotifyWrite(); + LOG(INFO) << "RsyncServer RsyncFile request ..."; } RsyncServerThread::RsyncServerThread(const std::set& ips, int port, int cron_interval, RsyncServer* arg) @@ -182,7 +189,6 @@ bool RsyncServerThread::RsyncServerHandle::AccessHandle(int fd, std::string& ip_ } void RsyncServerThread::RsyncServerHandle::CronHandle() const { - LOG(WARNING) << "CronHandle called"; } -} // end namespace rsync \ No newline at end of file +} // end namespace rsync From 9583007147d57f5a728c5d031759f71c06c56ff6 Mon Sep 17 00:00:00 2001 From: wangshao1 <459439411@qq.com> Date: Thu, 20 Jul 2023 10:13:15 +0800 Subject: [PATCH 23/37] fix bugs --- CMakeLists.txt | 2 +- include/rsync_client.h | 1 - src/rsync_server.cc | 1 - 3 files changed, 1 insertion(+), 3 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 0e08c861e..465a5354a 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -47,7 +47,7 @@ elseif(${BUILD_TYPE} STREQUAL RELWITHDEBINFO) set(LIB_BUILD_TYPE RELWITHDEBINFO) else() set(LIB_BUILD_TYPE RElEASE) - set(CMAKE_CXX_FLAGS_RELEASE "-g -DNDEBUG") + set(CMAKE_CXX_FLAGS_RELEASE "-O2 -DNDEBUG") endif() if(CMAKE_SYSTEM_NAME MATCHES "Darwin") diff --git a/include/rsync_client.h b/include/rsync_client.h index d38fdd483..3677b3233 100644 --- a/include/rsync_client.h +++ b/include/rsync_client.h @@ -49,7 +49,6 @@ class RsyncClient : public net::Thread { Status Start(); Status Stop(); bool IsRunning() { - LOG(WARNING) << "current state_: " << state_.load(); return state_.load() == RUNNING; } bool IsIdle() { return state_.load() == IDLE;} diff --git a/src/rsync_server.cc b/src/rsync_server.cc index 403e2ade6..24b97b582 100644 --- a/src/rsync_server.cc +++ b/src/rsync_server.cc @@ -9,7 +9,6 @@ namespace rsync { RsyncServer::RsyncServer(const std::set& ips, const int port) { work_thread_ = std::make_unique(2, 100000); - for_each(ips.begin(), ips.end(), [&port](auto& ip) {LOG(WARNING) << ip << "port: " << port;}); rsync_server_thread_ = std::make_unique(ips, port, 60 * 1000, this); } From 852b8d6e702e49c08c5b844b05123a7bc5573d44 Mon Sep 17 00:00:00 2001 From: wangshaoyi Date: Thu, 20 Jul 2023 15:45:57 +0800 Subject: [PATCH 24/37] fix bugs --- include/pika_server.h | 3 - include/rsync_server.h | 1 - src/pika_server.cc | 48 ----------- src/rsync_client.cc | 43 +++++----- src/rsync_server.cc | 172 ++++++++++++++++++++++++++++------------ src/rsync_service.proto | 10 ++- 6 files changed, 155 insertions(+), 122 deletions(-) diff --git a/include/pika_server.h b/include/pika_server.h index 414b468d1..0712908e1 100644 --- a/include/pika_server.h +++ b/include/pika_server.h @@ -260,9 +260,6 @@ class PikaServer : public pstd::noncopyable { /* * DBSync used */ - //TODO: yuecai - pstd::Status ReadDumpFile(const std::string& db_name, uint32_t slot_id, const std::string& filename, - const size_t offset, const size_t count, char* data, size_t* bytes_read); pstd::Status GetDumpUUID(const std::string& db_name, const uint32_t slot_id, std::string* snapshot_uuid); //TODO: yuecai pstd::Status GetDumpMeta(const std::string& db_name, const uint32_t slot_id, std::vector* files, std::string* snapshot_uuid); diff --git a/include/rsync_server.h b/include/rsync_server.h index 1c8fb4f05..7906d12db 100644 --- a/include/rsync_server.h +++ b/include/rsync_server.h @@ -39,7 +39,6 @@ class RsyncServer { int Start(); int Stop(); private: - std::map > file_map_; std::unique_ptr work_thread_ = nullptr; std::unique_ptr rsync_server_thread_ = nullptr; }; diff --git a/src/pika_server.cc b/src/pika_server.cc index 3384ac906..ed67f5e19 100644 --- a/src/pika_server.cc +++ b/src/pika_server.cc @@ -21,7 +21,6 @@ #include "net/include/redis_cli.h" #include "pstd/include/env.h" #include "pstd/include/rsync.h" -# include "pstd/include/pstd_defer.h" #include "include/pika_cmd_table_manager.h" #include "include/pika_dispatch_thread.h" @@ -932,53 +931,6 @@ pstd::Status PikaServer::GetDumpMeta(const std::string& db_name, const uint32_t return pstd::Status::OK(); } -// todo 参数太长了,待优化 -pstd::Status PikaServer::ReadDumpFile(const std::string& db_name, uint32_t slot_id, const std::string& filename, - const size_t offset, const size_t count, char* data, size_t* bytes_read) { - std::shared_ptr slot = GetDBSlotById(db_name, slot_id); - if (!slot) { - LOG(WARNING) << "cannot find slot for db_name " << db_name << "slot_id: " << slot_id; - return pstd::Status::NotFound("slot no found"); - } - const std::string filepath = slot->bgsave_info().path + "/" + filename; - int fd = open(filepath.c_str(), O_RDONLY); - if (fd < 0) { - return Status::IOError("fd open failed"); - } - DEFER { close(fd); }; - - const int kMaxCopyBlockSize = 1 << 20; - size_t read_offset = offset; - size_t read_count = count; - if (read_count > kMaxCopyBlockSize) { - read_count = kMaxCopyBlockSize; - } - - ssize_t bytesin = 0; - size_t left_read_count = count; - - while ((bytesin = pread(fd, data, read_count, read_offset)) > 0) { - left_read_count -= bytesin; - if (left_read_count < 0) { - break ; - } - if (read_count > left_read_count) { - read_count = left_read_count; - } - - data += bytesin; - *bytes_read += bytesin; - read_offset += bytesin; - } - - if (bytesin == -1) { - LOG(ERROR) << "unable to read from " << filename; - return pstd::Status::IOError("unable to read from " + filename); - } - - return pstd::Status::OK(); -} - void PikaServer::TryDBSync(const std::string& ip, int port, const std::string& db_name, uint32_t slot_id, int32_t top) { std::shared_ptr slot = GetDBSlotById(db_name, slot_id); diff --git a/src/rsync_client.cc b/src/rsync_client.cc index 1a19f5b02..1c4dd6f72 100644 --- a/src/rsync_client.cc +++ b/src/rsync_client.cc @@ -22,7 +22,7 @@ RsyncClient::RsyncClient(const std::string& dir, const std::string& db_name, con bool RsyncClient::Init() { if (state_ != IDLE) { - LOG(WARNING) << "State should be IDLE when Init"; + LOG(WARNING) << "State should be IDLE when Init"; return false; } master_ip_ = g_pika_server->master_ip(); @@ -111,9 +111,10 @@ Status RsyncClient::Wait(RsyncResponse*& resp) { Status s = Status::Timeout("rsync timeout", "timeout"); { std::unique_lock lock(mu_); - auto cv_s = cond_.wait_for(lock, std::chrono::seconds(3), [this] { return this->wo_->resp_ != nullptr; }); + auto cv_s = cond_.wait_for(lock, std::chrono::seconds(3), [this] { + return this->wo_->resp_ != nullptr; + }); if (!cv_s) { - LOG(WARNING) << "wait timeout"; return s; } resp = wo_->resp_; @@ -179,6 +180,12 @@ Status RsyncClient::CopyRemoteFile(const std::string& filename) { size_t elaspe_time_us = copy_file_end_time - copy_file_begin_time; throttle_->ReturnUnusedThroughput(count, resp->file_resp().count(), elaspe_time_us); + if (resp->code() != RsyncService::kOk) { + //TODO: handle different error + delete resp; + continue; + } + /* LOG(INFO) << "receive fileresponse, snapshot_uuid: " << resp->snapshot_uuid() << "filename: " << resp->file_resp().filename() << "offset: " << resp->file_resp().offset() @@ -186,7 +193,8 @@ Status RsyncClient::CopyRemoteFile(const std::string& filename) { */ if (resp->snapshot_uuid() != snapshot_uuid_) { - LOG(WARNING) << "receive newer dump, reset state to STOP, local_snapshot_uuid:" << snapshot_uuid_ << "remote snapshot uuid: " << resp->snapshot_uuid(); + LOG(WARNING) << "receive newer dump, reset state to STOP, local_snapshot_uuid:" + << snapshot_uuid_ << "remote snapshot uuid: " << resp->snapshot_uuid(); state_.store(STOP); delete resp; return s; @@ -203,14 +211,11 @@ Status RsyncClient::CopyRemoteFile(const std::string& filename) { md5.update(resp->file_resp().data().c_str(), ret_count); offset += resp->file_resp().count(); if (resp->file_resp().eof()) { - /* if (md5.finalize().hexdigest() != resp->file_resp().checksum()) { LOG(WARNING) << "mismatch file checksum for file: " << filename; - //TODO: wangshaoyi处理返回status s = Status::IOError("mismatch checksum", "mismatch checksum"); return s; } - */ s = writer->Fsync(); if (!s.ok()) { return s; @@ -230,11 +235,8 @@ Status RsyncClient::CopyRemoteFile(const std::string& filename) { } Status RsyncClient::Start() { - LOG(WARNING) << "RsyncClient Start ..."; - LOG(WARNING) << "RsyncClient StartThread ..."; - LOG(WARNING) << "RsyncClient shouldstop: " << should_stop() << " is running: " << is_running(); + LOG(WARNING) << "RsyncClient start ..."; StartThread(); - LOG(WARNING) << "RsyncClient StartThread done..."; return Status::OK(); } @@ -242,15 +244,12 @@ Status RsyncClient::Stop() { if (state_ == IDLE) { return Status::OK(); } + LOG(WARNING) << "RsyncClient stop ..."; state_ = STOP; StopThread(); - LOG(WARNING) << "RsyncClient StopThread done..."; client_thread_->StopThread(); - LOG(WARNING) << "RsyncClient Stop clientThread done..."; JoinThread(); - LOG(WARNING) << "RsyncClient JoinThread done..."; client_thread_->JoinThread(); - LOG(WARNING) << "RsyncClient join clientThread done..."; state_ = IDLE; return Status::OK(); } @@ -278,16 +277,16 @@ bool RsyncClient::Recover() { } std::set expired_files; - LOG(WARNING) << "file_set origin size: " << file_set_.size() << " local_snapshot_uuid: " << local_snapshot_uuid << " remote_snapshot_uuid: " << remote_snapshot_uuid; + //LOG(WARNING) << "file_set origin size: " << file_set_.size() << " local_snapshot_uuid: " << local_snapshot_uuid << " remote_snapshot_uuid: " << remote_snapshot_uuid; if (remote_snapshot_uuid != local_snapshot_uuid) { LOG(WARNING) << "snapshot uuid mismatch"; snapshot_uuid_ = remote_snapshot_uuid; file_set_ = remote_file_set; expired_files = local_file_set; } else { - LOG(WARNING) << "snapshot uuid equal, file_set_ size: " << file_set_.size() << "remote_file_set: " << remote_file_set.size() << " local_file_set: " << local_file_set.size(); for_each(remote_file_set.begin(), remote_file_set.end(), [](auto& file) {LOG(WARNING) << "remote_file_set: " << file;}); for_each(local_file_set.begin(), local_file_set.end(), [](auto& file) {LOG(WARNING) << "local_file_set: " << file;}); + std::set newly_files; set_difference(remote_file_set.begin(), remote_file_set.end(), local_file_set.begin(), local_file_set.end(), inserter(newly_files, newly_files.begin())); @@ -328,7 +327,6 @@ Status RsyncClient::CopyRemoteMeta(std::string* snapshot_uuid, std::setWrite(master_ip_, master_port_, to_send); if (!s.ok()) { retries++; @@ -340,10 +338,17 @@ Status RsyncClient::CopyRemoteMeta(std::string* snapshot_uuid, std::setcode() != RsyncService::kOk) { + //TODO: handle different error + delete resp; + continue; + } LOG(INFO) << "receive rsync meta infos, snapshot_uuid: " << resp->snapshot_uuid() << "files count: " << resp->meta_resp().filenames_size(); diff --git a/src/rsync_server.cc b/src/rsync_server.cc index 24b97b582..de2107be9 100644 --- a/src/rsync_server.cc +++ b/src/rsync_server.cc @@ -1,12 +1,81 @@ #include -#include "include/rsync_server.h" + #include -#include "include/pika_server.h" #include +#include "pstd_hash.h" +#include "include/pika_server.h" +#include "include/rsync_server.h" +#include "pstd/include/pstd_defer.h" + extern PikaServer* g_pika_server; namespace rsync { +//TODO: optimzie file read +Status ReadDumpFile(const std::string filepath, const size_t offset, const size_t count, + char* data, size_t* bytes_read, std::string* checksum) { + int fd = open(filepath.c_str(), O_RDONLY); + if (fd < 0) { + return Status::IOError("fd open failed"); + } + DEFER { close(fd); }; + + const int kMaxCopyBlockSize = 1 << 20; + size_t read_offset = offset; + size_t read_count = count; + if (read_count > kMaxCopyBlockSize) { + read_count = kMaxCopyBlockSize; + } + ssize_t bytesin = 0; + size_t left_read_count = count; + + while ((bytesin = pread(fd, data, read_count, read_offset)) > 0) { + left_read_count -= bytesin; + if (left_read_count < 0) { + break ; + } + if (read_count > left_read_count) { + read_count = left_read_count; + } + + data += bytesin; + *bytes_read += bytesin; + read_offset += bytesin; + } + + if (bytesin == -1) { + LOG(ERROR) << "unable to read from " << filepath; + return pstd::Status::IOError("unable to read from " + filepath); + } + + if (bytesin == 0) { + char* buffer = new char[kMaxCopyBlockSize]; + pstd::MD5 md5; + + while ((bytesin = read(fd, buffer, kMaxCopyBlockSize)) > 0) { + md5.update(buffer, bytesin); + } + if (bytesin == -1) { + LOG(ERROR) << "unable to read from " << filepath; + delete []buffer; + return pstd::Status::IOError("unable to read from " + filepath); + } + delete []buffer; + *checksum = md5.finalize().hexdigest(); + } + return pstd::Status::OK(); +} + +void RsyncWriteResp(RsyncService::RsyncResponse& response, std::shared_ptr conn) { + std::string reply_str; + if (!response.SerializeToString(&reply_str) || (conn->WriteResp(reply_str) != 0)) { + LOG(WARNING) << "Process FileRsync request serialization failed"; + conn->NotifyClose(); + return; + } + conn->NotifyWrite(); +} + RsyncServer::RsyncServer(const std::set& ips, const int port) { work_thread_ = std::make_unique(2, 100000); rsync_server_thread_ = std::make_unique(ips, port, 60 * 1000, this); @@ -44,13 +113,11 @@ int RsyncServer::Stop() { return 0; } -RsyncServerConn::RsyncServerConn(int connfd, const std::string& ip_port, - Thread* thread, void* worker_specific_data, - NetMultiplexer* mpx) : PbConn(connfd, ip_port, thread, mpx), data_(worker_specific_data) {} +RsyncServerConn::RsyncServerConn(int connfd, const std::string& ip_port, Thread* thread, + void* worker_specific_data, NetMultiplexer* mpx) + : PbConn(connfd, ip_port, thread, mpx), data_(worker_specific_data) {} -RsyncServerConn::~RsyncServerConn() { - LOG(INFO) << "RsyncServerConn destroyed"; -} +RsyncServerConn::~RsyncServerConn() {} int RsyncServerConn::DealMessage() { std::shared_ptr req = std::make_shared(); @@ -88,82 +155,89 @@ void RsyncServerConn::HandleMetaRsyncRequest(void* arg) { uint32_t slot_id = req->slot_id(); RsyncService::RsyncResponse response; + response.set_code(RsyncService::kOk); + response.set_type(RsyncService::kRsyncMeta); response.set_db_name(db_name); response.set_slot_id(slot_id); - response.set_type(RsyncService::kRsyncMeta); - LOG(INFO) << "RsyncServer receives RsyncMeta request..."; - std::vector filenames; std::string snapshot_uuid; g_pika_server->GetDumpMeta(db_name, slot_id, &filenames, &snapshot_uuid); - LOG(WARNING) << "snapshot_uuid: " << snapshot_uuid; + response.set_snapshot_uuid(snapshot_uuid); + + LOG(INFO) << "Rsync Meta request, snapshot_uuid: " << snapshot_uuid + << "files count: " << filenames.size() << "file list: "; std::for_each(filenames.begin(), filenames.end(), [](auto& file) { - LOG(WARNING) << "meta file name: " << file; + LOG(INFO) << "rsync snapshot file: " << file; }); - //TODO: temporarily mock response + RsyncService::MetaResponse* meta_resp = response.mutable_meta_resp(); - response.set_snapshot_uuid(snapshot_uuid); for (const auto& filename : filenames) { meta_resp->add_filenames(filename); } - - std::string reply_str; - if (!response.SerializeToString(&reply_str) || (conn->WriteResp(reply_str) != 0)) { - LOG(WARNING) << "Process MetaRsync request serialization failed"; - conn->NotifyClose(); - return; - } - conn->NotifyWrite(); - LOG(INFO) << "RsyncServer RsyncMeta request done..."; + RsyncWriteResp(response, conn); } void RsyncServerConn::HandleFileRsyncRequest(void* arg) { - std::unique_ptr task_arg(static_cast(arg)); - const std::shared_ptr req = task_arg->req; + std::unique_ptr task_arg(static_cast(arg)); + const std::shared_ptr req = task_arg->req; std::shared_ptr conn = task_arg->conn; - LOG(INFO) << "RsyncServer RsyncFile request ..."; uint32_t slot_id = req->slot_id(); std::string db_name = req->db_name(); std::string filename = req->file_req().filename(); size_t offset = req->file_req().offset(); size_t count = req->file_req().count(); + RsyncService::RsyncResponse response; + response.set_code(RsyncService::kOk); + response.set_type(RsyncService::kRsyncFile); + response.set_db_name(db_name); + response.set_slot_id(slot_id); + std::string snapshot_uuid; Status s = g_pika_server->GetDumpUUID(db_name, slot_id, &snapshot_uuid); - LOG(INFO) << "Receive RsyncFile request " << "filename: " << filename - << " offset: " << offset - << " count: " << count; + response.set_snapshot_uuid(snapshot_uuid); + if (!s.ok()) { + LOG(WARNING) << "rsyncserver get snapshotUUID failed"; + response.set_code(RsyncService::kErr); + RsyncWriteResp(response, conn); + return; + } + + std::shared_ptr slot = g_pika_server->GetDBSlotById(db_name, slot_id); + if (!slot) { + LOG(WARNING) << "cannot find slot for db_name " << db_name + << "slot_id: " << slot_id; + response.set_code(RsyncService::kErr); + RsyncWriteResp(response, conn); + } + const std::string filepath = slot->bgsave_info().path + "/" + filename; char* buffer = new char[req->file_req().count() + 1]; + std::string checksum = ""; size_t bytes_read{0}; - auto status = g_pika_server -> ReadDumpFile(db_name, slot_id, filename, offset, count, buffer, &bytes_read); - LOG(INFO) << "RsyncServer ReadDumpFile: " << filename << " read size: " << bytes_read << "status: " << status.ToString(); - - response.set_type(RsyncService::kRsyncFile); - response.set_snapshot_uuid(snapshot_uuid); - response.set_db_name(db_name); - response.set_slot_id(slot_id); + s = ReadDumpFile(filepath, offset, count, buffer, &bytes_read, &checksum); + if (!s.ok()) { + response.set_code(RsyncService::kErr); + RsyncWriteResp(response, conn); + delete []buffer; + return; + } + LOG(INFO) << "RsyncServer receives FileRequest " << "filename: " + << filename << " offset: " << offset << " count: " + << count << " read_count: " << bytes_read; RsyncService::FileResponse* file_resp = response.mutable_file_resp(); + file_resp->set_data(buffer, bytes_read); file_resp->set_eof(bytes_read != count); + file_resp->set_checksum(checksum); + file_resp->set_filename(filename); file_resp->set_count(bytes_read); file_resp->set_offset(offset); - file_resp->set_data(buffer, bytes_read); - //TODO: checksum - file_resp->set_checksum("checksum"); - file_resp->set_filename(filename); - std::string reply_str; - if (!response.SerializeToString(&reply_str) || (conn->WriteResp(reply_str) != 0)) { - LOG(WARNING) << "Process FileRsync request serialization failed"; - conn->NotifyClose(); - delete []buffer; - return; - } + RsyncWriteResp(response, conn); delete []buffer; - conn->NotifyWrite(); LOG(INFO) << "RsyncServer RsyncFile request ..."; } diff --git a/src/rsync_service.proto b/src/rsync_service.proto index a35a6835b..a8b96bcbe 100644 --- a/src/rsync_service.proto +++ b/src/rsync_service.proto @@ -6,6 +6,11 @@ enum Type { kRsyncFile = 2; } +enum StatusCode { + kOk = 1; + kErr = 2; +} + message MetaResponse { repeated string filenames = 1; } @@ -37,6 +42,7 @@ message RsyncResponse { required string snapshot_uuid = 2; required string db_name = 3; required uint32 slot_id = 4; - optional MetaResponse meta_resp = 5; - optional FileResponse file_resp = 6; + required StatusCode code = 5; + optional MetaResponse meta_resp = 6; + optional FileResponse file_resp = 7; } \ No newline at end of file From f39a0243b32dbd349bfbdf76b08bf911bbfe265b Mon Sep 17 00:00:00 2001 From: wangshao1 <459439411@qq.com> Date: Fri, 21 Jul 2023 19:27:41 +0800 Subject: [PATCH 25/37] fix bugs 1 --- src/pika_conf.cc | 2 +- src/pika_repl_server_conn.cc | 1 + src/pika_server.cc | 2 ++ src/pika_slot.cc | 16 +++++++++------- src/rsync_client.cc | 9 ++++----- src/rsync_server.cc | 13 ++++++++++--- 6 files changed, 27 insertions(+), 16 deletions(-) diff --git a/src/pika_conf.cc b/src/pika_conf.cc index dfc5292f2..4d71f8797 100644 --- a/src/pika_conf.cc +++ b/src/pika_conf.cc @@ -535,7 +535,7 @@ int PikaConf::Load() { if (slaveof_ != "") { std::string master_run_id; GetConfStr("master-run-id", &master_run_id); - if (master_run_id_.length() == configRunIDSize) { + if (master_run_id.length() == configRunIDSize) { master_run_id_ = master_run_id; } } diff --git a/src/pika_repl_server_conn.cc b/src/pika_repl_server_conn.cc index 330ec6051..31e834fe3 100644 --- a/src/pika_repl_server_conn.cc +++ b/src/pika_repl_server_conn.cc @@ -343,6 +343,7 @@ void PikaReplServerConn::HandleDBSyncRequest(void* arg) { } g_pika_server->TryDBSync(node.ip(), node.port() + kPortShiftRSync, db_name, slot_id, slave_boffset.filenum()); + master_slot->ActivateSlaveDbSync(node.ip(), node.port()); std::string reply_str; if (!response.SerializeToString(&reply_str) || (conn->WriteResp(reply_str) != 0)) { diff --git a/src/pika_server.cc b/src/pika_server.cc index ed67f5e19..c96a96ab2 100644 --- a/src/pika_server.cc +++ b/src/pika_server.cc @@ -172,6 +172,8 @@ void PikaServer::Start() { time(&start_time_s_); + std::string master_run_id = g_pika_conf->master_run_id(); + set_master_run_id(master_run_id); std::string slaveof = g_pika_conf->slaveof(); if (!slaveof.empty()) { int32_t sep = slaveof.find(':'); diff --git a/src/pika_slot.cc b/src/pika_slot.cc index b871767ce..4c0769aa3 100644 --- a/src/pika_slot.cc +++ b/src/pika_slot.cc @@ -353,17 +353,20 @@ void Slot::DoBgSave(void* arg) { // Some output BgSaveInfo info = bg_task_arg->slot->bgsave_info(); + std::stringstream info_content; std::ofstream out; out.open(info.path + "/" + kBgsaveInfoFile, std::ios::in | std::ios::trunc); if (out.is_open()) { - out << (time(nullptr) - info.start_time) << "s\n" - << g_pika_server->host() << "\n" - << g_pika_server->port() << "\n" - << info.offset.b_offset.filenum << "\n" - << info.offset.b_offset.offset << "\n"; + info_content << (time(nullptr) - info.start_time) << "s\n" + << g_pika_server->host() << "\n" + << g_pika_server->port() << "\n" + << info.offset.b_offset.filenum << "\n" + << info.offset.b_offset.offset << "\n"; if (g_pika_conf->consensus_level() != 0) { - out << info.offset.l_offset.term << "\n" << info.offset.l_offset.index << "\n"; + info_content << info.offset.l_offset.term << "\n" << info.offset.l_offset.index << "\n"; } + bg_task_arg->slot->snapshot_uuid_ = md5(info_content.str()); + out << info_content.rdbuf(); out.close(); } if (!success) { @@ -371,7 +374,6 @@ void Slot::DoBgSave(void* arg) { pstd::RenameFile(info.path, fail_path); } bg_task_arg->slot->FinishBgsave(); - } bool Slot::RunBgsaveEngine() { diff --git a/src/rsync_client.cc b/src/rsync_client.cc index 1c4dd6f72..4a70ec128 100644 --- a/src/rsync_client.cc +++ b/src/rsync_client.cc @@ -65,7 +65,7 @@ void* RsyncClient::ThreadMain() { LOG(WARNING) << "rsync CopyRemoteFile failed, filename: " << file; continue; } - LOG(WARNING) << "CopyRemoteFile "<< file << " success..."; + //LOG(WARNING) << "CopyRemoteFile "<< file << " success..."; break; } if (state_.load(std::memory_order_relaxed) != RUNNING) { @@ -153,7 +153,7 @@ Status RsyncClient::CopyRemoteFile(const std::string& filename) { std::string to_send; request.SerializeToString(&to_send); - LOG(WARNING) << "master ip: " << master_ip_ << " master_port: " << master_port_; + //LOG(WARNING) << "master ip: " << master_ip_ << " master_port: " << master_port_; s = client_thread_->Write(master_ip_, master_port_, to_send); if (!s.ok()) { LOG(WARNING) << "send rsync request failed"; @@ -161,13 +161,11 @@ Status RsyncClient::CopyRemoteFile(const std::string& filename) { } { - LOG(WARNING) << "reset waitobject"; std::lock_guard lock(mu_); wo_->Reset(filename, kRsyncFile, offset); - LOG(WARNING) << "reset waitobject done"; } - LOG(INFO) << "wait CopyRemoteFile response....."; + //LOG(INFO) << "wait CopyRemoteFile response....."; RsyncResponse* resp = nullptr; s = Wait(resp); if (s.IsTimeout() || resp == nullptr) { @@ -211,6 +209,7 @@ Status RsyncClient::CopyRemoteFile(const std::string& filename) { md5.update(resp->file_resp().data().c_str(), ret_count); offset += resp->file_resp().count(); if (resp->file_resp().eof()) { + LOG(WARNING) << "filename: " << filename << " md5: " << md5.finalize().hexdigest(); if (md5.finalize().hexdigest() != resp->file_resp().checksum()) { LOG(WARNING) << "mismatch file checksum for file: " << filename; s = Status::IOError("mismatch checksum", "mismatch checksum"); diff --git a/src/rsync_server.cc b/src/rsync_server.cc index de2107be9..b28bf33b9 100644 --- a/src/rsync_server.cc +++ b/src/rsync_server.cc @@ -122,7 +122,7 @@ RsyncServerConn::~RsyncServerConn() {} int RsyncServerConn::DealMessage() { std::shared_ptr req = std::make_shared(); bool parse_res = req->ParseFromArray(rbuf_ + cur_pos_ - header_len_, header_len_); - LOG(INFO) << "RsyncServer receives new request..."; + //LOG(INFO) << "RsyncServer receives new request..."; if (!parse_res) { LOG(WARNING) << "Pika rsync server connection pb parse error."; return -1; @@ -153,6 +153,11 @@ void RsyncServerConn::HandleMetaRsyncRequest(void* arg) { std::shared_ptr conn = task_arg->conn; std::string db_name = req->db_name(); uint32_t slot_id = req->slot_id(); + std::shared_ptr slot = g_pika_server->GetDBSlotById(db_name, slot_id); + if (!slot || slot->IsBgSaving()) { + LOG(WARNING) << "waiting bgsave done..."; + return; + } RsyncService::RsyncResponse response; response.set_code(RsyncService::kOk); @@ -224,9 +229,11 @@ void RsyncServerConn::HandleFileRsyncRequest(void* arg) { delete []buffer; return; } +/* LOG(INFO) << "RsyncServer receives FileRequest " << "filename: " << filename << " offset: " << offset << " count: " - << count << " read_count: " << bytes_read; + << count << " read_count: " << bytes_read << "checksum:" << checksum; +*/ RsyncService::FileResponse* file_resp = response.mutable_file_resp(); file_resp->set_data(buffer, bytes_read); @@ -238,7 +245,7 @@ void RsyncServerConn::HandleFileRsyncRequest(void* arg) { RsyncWriteResp(response, conn); delete []buffer; - LOG(INFO) << "RsyncServer RsyncFile request ..."; + //LOG(INFO) << "RsyncServer RsyncFile request ..."; } RsyncServerThread::RsyncServerThread(const std::set& ips, int port, int cron_interval, RsyncServer* arg) From bbf65852649497976b17a783627189d99e902309 Mon Sep 17 00:00:00 2001 From: wangshaoyi Date: Sat, 22 Jul 2023 11:59:27 +0800 Subject: [PATCH 26/37] fix bugs --- src/rsync_server.cc | 7 ------- 1 file changed, 7 deletions(-) diff --git a/src/rsync_server.cc b/src/rsync_server.cc index efa04a0e4..e393142a3 100644 --- a/src/rsync_server.cc +++ b/src/rsync_server.cc @@ -165,13 +165,6 @@ void RsyncServerConn::HandleMetaRsyncRequest(void* arg) { response.set_db_name(db_name); response.set_slot_id(slot_id); - RsyncService::RsyncResponse response; - response.set_db_name(db_name); - response.set_slot_id(slot_id); - response.set_type(RsyncService::kRsyncMeta); - LOG(INFO) << "RsyncServer receives RsyncMeta request..."; - - std::vector filenames; std::string snapshot_uuid; g_pika_server->GetDumpMeta(db_name, slot_id, &filenames, &snapshot_uuid); From 17217ac9bbbdc12ce76b1f9e996d5fb2084cb1f7 Mon Sep 17 00:00:00 2001 From: wangshao1 <30471730+wangshao1@users.noreply.github.com> Date: Sat, 22 Jul 2023 15:51:12 +0800 Subject: [PATCH 27/37] fix rsync bugs (#195) * add debug log for test * fix rsync client/server bugs * fix bugs * add debug log for test * fix bugs * fix bugs * fix bugs * fix bugs * fix bugs * fix bugs 1 * fix bugs --------- Co-authored-by: wangshaoyi --- include/pika_server.h | 3 - include/rsync_client.h | 3 +- include/rsync_server.h | 1 - src/pika_conf.cc | 2 +- src/pika_repl_server_conn.cc | 1 + src/pika_server.cc | 50 +--------- src/pika_slot.cc | 16 +-- src/rsync_client.cc | 47 +++++---- src/rsync_server.cc | 183 +++++++++++++++++++++++++---------- src/rsync_service.proto | 10 +- 10 files changed, 182 insertions(+), 134 deletions(-) diff --git a/include/pika_server.h b/include/pika_server.h index 414b468d1..0712908e1 100644 --- a/include/pika_server.h +++ b/include/pika_server.h @@ -260,9 +260,6 @@ class PikaServer : public pstd::noncopyable { /* * DBSync used */ - //TODO: yuecai - pstd::Status ReadDumpFile(const std::string& db_name, uint32_t slot_id, const std::string& filename, - const size_t offset, const size_t count, char* data, size_t* bytes_read); pstd::Status GetDumpUUID(const std::string& db_name, const uint32_t slot_id, std::string* snapshot_uuid); //TODO: yuecai pstd::Status GetDumpMeta(const std::string& db_name, const uint32_t slot_id, std::vector* files, std::string* snapshot_uuid); diff --git a/include/rsync_client.h b/include/rsync_client.h index d38fdd483..4b3cf53d7 100644 --- a/include/rsync_client.h +++ b/include/rsync_client.h @@ -48,8 +48,7 @@ class RsyncClient : public net::Thread { bool Init(); Status Start(); Status Stop(); - bool IsRunning() { - LOG(WARNING) << "current state_: " << state_.load(); + bool IsRunning() { return state_.load() == RUNNING; } bool IsIdle() { return state_.load() == IDLE;} diff --git a/include/rsync_server.h b/include/rsync_server.h index 1c8fb4f05..7906d12db 100644 --- a/include/rsync_server.h +++ b/include/rsync_server.h @@ -39,7 +39,6 @@ class RsyncServer { int Start(); int Stop(); private: - std::map > file_map_; std::unique_ptr work_thread_ = nullptr; std::unique_ptr rsync_server_thread_ = nullptr; }; diff --git a/src/pika_conf.cc b/src/pika_conf.cc index dfc5292f2..4d71f8797 100644 --- a/src/pika_conf.cc +++ b/src/pika_conf.cc @@ -535,7 +535,7 @@ int PikaConf::Load() { if (slaveof_ != "") { std::string master_run_id; GetConfStr("master-run-id", &master_run_id); - if (master_run_id_.length() == configRunIDSize) { + if (master_run_id.length() == configRunIDSize) { master_run_id_ = master_run_id; } } diff --git a/src/pika_repl_server_conn.cc b/src/pika_repl_server_conn.cc index 330ec6051..31e834fe3 100644 --- a/src/pika_repl_server_conn.cc +++ b/src/pika_repl_server_conn.cc @@ -343,6 +343,7 @@ void PikaReplServerConn::HandleDBSyncRequest(void* arg) { } g_pika_server->TryDBSync(node.ip(), node.port() + kPortShiftRSync, db_name, slot_id, slave_boffset.filenum()); + master_slot->ActivateSlaveDbSync(node.ip(), node.port()); std::string reply_str; if (!response.SerializeToString(&reply_str) || (conn->WriteResp(reply_str) != 0)) { diff --git a/src/pika_server.cc b/src/pika_server.cc index 3384ac906..c96a96ab2 100644 --- a/src/pika_server.cc +++ b/src/pika_server.cc @@ -21,7 +21,6 @@ #include "net/include/redis_cli.h" #include "pstd/include/env.h" #include "pstd/include/rsync.h" -# include "pstd/include/pstd_defer.h" #include "include/pika_cmd_table_manager.h" #include "include/pika_dispatch_thread.h" @@ -173,6 +172,8 @@ void PikaServer::Start() { time(&start_time_s_); + std::string master_run_id = g_pika_conf->master_run_id(); + set_master_run_id(master_run_id); std::string slaveof = g_pika_conf->slaveof(); if (!slaveof.empty()) { int32_t sep = slaveof.find(':'); @@ -932,53 +933,6 @@ pstd::Status PikaServer::GetDumpMeta(const std::string& db_name, const uint32_t return pstd::Status::OK(); } -// todo 参数太长了,待优化 -pstd::Status PikaServer::ReadDumpFile(const std::string& db_name, uint32_t slot_id, const std::string& filename, - const size_t offset, const size_t count, char* data, size_t* bytes_read) { - std::shared_ptr slot = GetDBSlotById(db_name, slot_id); - if (!slot) { - LOG(WARNING) << "cannot find slot for db_name " << db_name << "slot_id: " << slot_id; - return pstd::Status::NotFound("slot no found"); - } - const std::string filepath = slot->bgsave_info().path + "/" + filename; - int fd = open(filepath.c_str(), O_RDONLY); - if (fd < 0) { - return Status::IOError("fd open failed"); - } - DEFER { close(fd); }; - - const int kMaxCopyBlockSize = 1 << 20; - size_t read_offset = offset; - size_t read_count = count; - if (read_count > kMaxCopyBlockSize) { - read_count = kMaxCopyBlockSize; - } - - ssize_t bytesin = 0; - size_t left_read_count = count; - - while ((bytesin = pread(fd, data, read_count, read_offset)) > 0) { - left_read_count -= bytesin; - if (left_read_count < 0) { - break ; - } - if (read_count > left_read_count) { - read_count = left_read_count; - } - - data += bytesin; - *bytes_read += bytesin; - read_offset += bytesin; - } - - if (bytesin == -1) { - LOG(ERROR) << "unable to read from " << filename; - return pstd::Status::IOError("unable to read from " + filename); - } - - return pstd::Status::OK(); -} - void PikaServer::TryDBSync(const std::string& ip, int port, const std::string& db_name, uint32_t slot_id, int32_t top) { std::shared_ptr slot = GetDBSlotById(db_name, slot_id); diff --git a/src/pika_slot.cc b/src/pika_slot.cc index b871767ce..4c0769aa3 100644 --- a/src/pika_slot.cc +++ b/src/pika_slot.cc @@ -353,17 +353,20 @@ void Slot::DoBgSave(void* arg) { // Some output BgSaveInfo info = bg_task_arg->slot->bgsave_info(); + std::stringstream info_content; std::ofstream out; out.open(info.path + "/" + kBgsaveInfoFile, std::ios::in | std::ios::trunc); if (out.is_open()) { - out << (time(nullptr) - info.start_time) << "s\n" - << g_pika_server->host() << "\n" - << g_pika_server->port() << "\n" - << info.offset.b_offset.filenum << "\n" - << info.offset.b_offset.offset << "\n"; + info_content << (time(nullptr) - info.start_time) << "s\n" + << g_pika_server->host() << "\n" + << g_pika_server->port() << "\n" + << info.offset.b_offset.filenum << "\n" + << info.offset.b_offset.offset << "\n"; if (g_pika_conf->consensus_level() != 0) { - out << info.offset.l_offset.term << "\n" << info.offset.l_offset.index << "\n"; + info_content << info.offset.l_offset.term << "\n" << info.offset.l_offset.index << "\n"; } + bg_task_arg->slot->snapshot_uuid_ = md5(info_content.str()); + out << info_content.rdbuf(); out.close(); } if (!success) { @@ -371,7 +374,6 @@ void Slot::DoBgSave(void* arg) { pstd::RenameFile(info.path, fail_path); } bg_task_arg->slot->FinishBgsave(); - } bool Slot::RunBgsaveEngine() { diff --git a/src/rsync_client.cc b/src/rsync_client.cc index 1a19f5b02..f9b52bcb2 100644 --- a/src/rsync_client.cc +++ b/src/rsync_client.cc @@ -22,7 +22,7 @@ RsyncClient::RsyncClient(const std::string& dir, const std::string& db_name, con bool RsyncClient::Init() { if (state_ != IDLE) { - LOG(WARNING) << "State should be IDLE when Init"; + LOG(WARNING) << "State should be IDLE when Init"; return false; } master_ip_ = g_pika_server->master_ip(); @@ -65,7 +65,7 @@ void* RsyncClient::ThreadMain() { LOG(WARNING) << "rsync CopyRemoteFile failed, filename: " << file; continue; } - LOG(WARNING) << "CopyRemoteFile "<< file << " success..."; + //LOG(WARNING) << "CopyRemoteFile "<< file << " success..."; break; } if (state_.load(std::memory_order_relaxed) != RUNNING) { @@ -111,9 +111,10 @@ Status RsyncClient::Wait(RsyncResponse*& resp) { Status s = Status::Timeout("rsync timeout", "timeout"); { std::unique_lock lock(mu_); - auto cv_s = cond_.wait_for(lock, std::chrono::seconds(3), [this] { return this->wo_->resp_ != nullptr; }); + auto cv_s = cond_.wait_for(lock, std::chrono::seconds(3), [this] { + return this->wo_->resp_ != nullptr; + }); if (!cv_s) { - LOG(WARNING) << "wait timeout"; return s; } resp = wo_->resp_; @@ -152,7 +153,7 @@ Status RsyncClient::CopyRemoteFile(const std::string& filename) { std::string to_send; request.SerializeToString(&to_send); - LOG(WARNING) << "master ip: " << master_ip_ << " master_port: " << master_port_; + //LOG(WARNING) << "master ip: " << master_ip_ << " master_port: " << master_port_; s = client_thread_->Write(master_ip_, master_port_, to_send); if (!s.ok()) { LOG(WARNING) << "send rsync request failed"; @@ -160,13 +161,11 @@ Status RsyncClient::CopyRemoteFile(const std::string& filename) { } { - LOG(WARNING) << "reset waitobject"; std::lock_guard lock(mu_); wo_->Reset(filename, kRsyncFile, offset); - LOG(WARNING) << "reset waitobject done"; } - LOG(INFO) << "wait CopyRemoteFile response....."; + //LOG(INFO) << "wait CopyRemoteFile response....."; RsyncResponse* resp = nullptr; s = Wait(resp); if (s.IsTimeout() || resp == nullptr) { @@ -179,6 +178,12 @@ Status RsyncClient::CopyRemoteFile(const std::string& filename) { size_t elaspe_time_us = copy_file_end_time - copy_file_begin_time; throttle_->ReturnUnusedThroughput(count, resp->file_resp().count(), elaspe_time_us); + if (resp->code() != RsyncService::kOk) { + //TODO: handle different error + delete resp; + continue; + } + /* LOG(INFO) << "receive fileresponse, snapshot_uuid: " << resp->snapshot_uuid() << "filename: " << resp->file_resp().filename() << "offset: " << resp->file_resp().offset() @@ -186,7 +191,8 @@ Status RsyncClient::CopyRemoteFile(const std::string& filename) { */ if (resp->snapshot_uuid() != snapshot_uuid_) { - LOG(WARNING) << "receive newer dump, reset state to STOP, local_snapshot_uuid:" << snapshot_uuid_ << "remote snapshot uuid: " << resp->snapshot_uuid(); + LOG(WARNING) << "receive newer dump, reset state to STOP, local_snapshot_uuid:" + << snapshot_uuid_ << "remote snapshot uuid: " << resp->snapshot_uuid(); state_.store(STOP); delete resp; return s; @@ -203,14 +209,12 @@ Status RsyncClient::CopyRemoteFile(const std::string& filename) { md5.update(resp->file_resp().data().c_str(), ret_count); offset += resp->file_resp().count(); if (resp->file_resp().eof()) { - /* + LOG(WARNING) << "filename: " << filename << " md5: " << md5.finalize().hexdigest(); if (md5.finalize().hexdigest() != resp->file_resp().checksum()) { LOG(WARNING) << "mismatch file checksum for file: " << filename; - //TODO: wangshaoyi处理返回status s = Status::IOError("mismatch checksum", "mismatch checksum"); return s; } - */ s = writer->Fsync(); if (!s.ok()) { return s; @@ -230,9 +234,7 @@ Status RsyncClient::CopyRemoteFile(const std::string& filename) { } Status RsyncClient::Start() { - LOG(WARNING) << "RsyncClient Start ..."; - LOG(WARNING) << "RsyncClient StartThread ..."; - LOG(WARNING) << "RsyncClient shouldstop: " << should_stop() << " is running: " << is_running(); + LOG(WARNING) << "RsyncClient start ..."; StartThread(); LOG(WARNING) << "RsyncClient StartThread done..."; return Status::OK(); @@ -242,6 +244,7 @@ Status RsyncClient::Stop() { if (state_ == IDLE) { return Status::OK(); } + LOG(WARNING) << "RsyncClient stop ..."; state_ = STOP; StopThread(); LOG(WARNING) << "RsyncClient StopThread done..."; @@ -278,16 +281,16 @@ bool RsyncClient::Recover() { } std::set expired_files; - LOG(WARNING) << "file_set origin size: " << file_set_.size() << " local_snapshot_uuid: " << local_snapshot_uuid << " remote_snapshot_uuid: " << remote_snapshot_uuid; + //LOG(WARNING) << "file_set origin size: " << file_set_.size() << " local_snapshot_uuid: " << local_snapshot_uuid << " remote_snapshot_uuid: " << remote_snapshot_uuid; if (remote_snapshot_uuid != local_snapshot_uuid) { LOG(WARNING) << "snapshot uuid mismatch"; snapshot_uuid_ = remote_snapshot_uuid; file_set_ = remote_file_set; expired_files = local_file_set; } else { - LOG(WARNING) << "snapshot uuid equal, file_set_ size: " << file_set_.size() << "remote_file_set: " << remote_file_set.size() << " local_file_set: " << local_file_set.size(); for_each(remote_file_set.begin(), remote_file_set.end(), [](auto& file) {LOG(WARNING) << "remote_file_set: " << file;}); for_each(local_file_set.begin(), local_file_set.end(), [](auto& file) {LOG(WARNING) << "local_file_set: " << file;}); + std::set newly_files; set_difference(remote_file_set.begin(), remote_file_set.end(), local_file_set.begin(), local_file_set.end(), inserter(newly_files, newly_files.begin())); @@ -328,7 +331,6 @@ Status RsyncClient::CopyRemoteMeta(std::string* snapshot_uuid, std::setWrite(master_ip_, master_port_, to_send); if (!s.ok()) { retries++; @@ -340,10 +342,17 @@ Status RsyncClient::CopyRemoteMeta(std::string* snapshot_uuid, std::setcode() != RsyncService::kOk) { + //TODO: handle different error + delete resp; + continue; + } LOG(INFO) << "receive rsync meta infos, snapshot_uuid: " << resp->snapshot_uuid() << "files count: " << resp->meta_resp().filenames_size(); diff --git a/src/rsync_server.cc b/src/rsync_server.cc index 403e2ade6..e393142a3 100644 --- a/src/rsync_server.cc +++ b/src/rsync_server.cc @@ -1,15 +1,83 @@ #include -#include "include/rsync_server.h" + #include -#include "include/pika_server.h" #include +#include "pstd_hash.h" +#include "include/pika_server.h" +#include "include/rsync_server.h" +#include "pstd/include/pstd_defer.h" + extern PikaServer* g_pika_server; namespace rsync { +//TODO: optimzie file read +Status ReadDumpFile(const std::string filepath, const size_t offset, const size_t count, + char* data, size_t* bytes_read, std::string* checksum) { + int fd = open(filepath.c_str(), O_RDONLY); + if (fd < 0) { + return Status::IOError("fd open failed"); + } + DEFER { close(fd); }; + + const int kMaxCopyBlockSize = 1 << 20; + size_t read_offset = offset; + size_t read_count = count; + if (read_count > kMaxCopyBlockSize) { + read_count = kMaxCopyBlockSize; + } + ssize_t bytesin = 0; + size_t left_read_count = count; + + while ((bytesin = pread(fd, data, read_count, read_offset)) > 0) { + left_read_count -= bytesin; + if (left_read_count < 0) { + break ; + } + if (read_count > left_read_count) { + read_count = left_read_count; + } + + data += bytesin; + *bytes_read += bytesin; + read_offset += bytesin; + } + + if (bytesin == -1) { + LOG(ERROR) << "unable to read from " << filepath; + return pstd::Status::IOError("unable to read from " + filepath); + } + + if (bytesin == 0) { + char* buffer = new char[kMaxCopyBlockSize]; + pstd::MD5 md5; + + while ((bytesin = read(fd, buffer, kMaxCopyBlockSize)) > 0) { + md5.update(buffer, bytesin); + } + if (bytesin == -1) { + LOG(ERROR) << "unable to read from " << filepath; + delete []buffer; + return pstd::Status::IOError("unable to read from " + filepath); + } + delete []buffer; + *checksum = md5.finalize().hexdigest(); + } + return pstd::Status::OK(); +} + +void RsyncWriteResp(RsyncService::RsyncResponse& response, std::shared_ptr conn) { + std::string reply_str; + if (!response.SerializeToString(&reply_str) || (conn->WriteResp(reply_str) != 0)) { + LOG(WARNING) << "Process FileRsync request serialization failed"; + conn->NotifyClose(); + return; + } + conn->NotifyWrite(); +} + RsyncServer::RsyncServer(const std::set& ips, const int port) { work_thread_ = std::make_unique(2, 100000); - for_each(ips.begin(), ips.end(), [&port](auto& ip) {LOG(WARNING) << ip << "port: " << port;}); rsync_server_thread_ = std::make_unique(ips, port, 60 * 1000, this); } @@ -45,18 +113,16 @@ int RsyncServer::Stop() { return 0; } -RsyncServerConn::RsyncServerConn(int connfd, const std::string& ip_port, - Thread* thread, void* worker_specific_data, - NetMultiplexer* mpx) : PbConn(connfd, ip_port, thread, mpx), data_(worker_specific_data) {} +RsyncServerConn::RsyncServerConn(int connfd, const std::string& ip_port, Thread* thread, + void* worker_specific_data, NetMultiplexer* mpx) + : PbConn(connfd, ip_port, thread, mpx), data_(worker_specific_data) {} -RsyncServerConn::~RsyncServerConn() { - LOG(INFO) << "RsyncServerConn destroyed"; -} +RsyncServerConn::~RsyncServerConn() {} int RsyncServerConn::DealMessage() { std::shared_ptr req = std::make_shared(); bool parse_res = req->ParseFromArray(rbuf_ + cur_pos_ - header_len_, header_len_); - LOG(INFO) << "RsyncServer receives new request..."; + //LOG(INFO) << "RsyncServer receives new request..."; if (!parse_res) { LOG(WARNING) << "Pika rsync server connection pb parse error."; return -1; @@ -87,41 +153,39 @@ void RsyncServerConn::HandleMetaRsyncRequest(void* arg) { std::shared_ptr conn = task_arg->conn; std::string db_name = req->db_name(); uint32_t slot_id = req->slot_id(); + std::shared_ptr slot = g_pika_server->GetDBSlotById(db_name, slot_id); + if (!slot || slot->IsBgSaving()) { + LOG(WARNING) << "waiting bgsave done..."; + return; + } RsyncService::RsyncResponse response; + response.set_code(RsyncService::kOk); + response.set_type(RsyncService::kRsyncMeta); response.set_db_name(db_name); response.set_slot_id(slot_id); - response.set_type(RsyncService::kRsyncMeta); - LOG(INFO) << "RsyncServer receives RsyncMeta request..."; - std::vector filenames; std::string snapshot_uuid; g_pika_server->GetDumpMeta(db_name, slot_id, &filenames, &snapshot_uuid); - LOG(WARNING) << "snapshot_uuid: " << snapshot_uuid; + response.set_snapshot_uuid(snapshot_uuid); + + LOG(INFO) << "Rsync Meta request, snapshot_uuid: " << snapshot_uuid + << "files count: " << filenames.size() << "file list: "; std::for_each(filenames.begin(), filenames.end(), [](auto& file) { - LOG(WARNING) << "meta file name: " << file; + LOG(INFO) << "rsync snapshot file: " << file; }); - //TODO: temporarily mock response + RsyncService::MetaResponse* meta_resp = response.mutable_meta_resp(); - response.set_snapshot_uuid(snapshot_uuid); for (const auto& filename : filenames) { meta_resp->add_filenames(filename); } - - std::string reply_str; - if (!response.SerializeToString(&reply_str) || (conn->WriteResp(reply_str) != 0)) { - LOG(WARNING) << "Process MetaRsync request serialization failed"; - conn->NotifyClose(); - return; - } - conn->NotifyWrite(); - LOG(INFO) << "RsyncServer RsyncMeta request done..."; + RsyncWriteResp(response, conn); } void RsyncServerConn::HandleFileRsyncRequest(void* arg) { - std::unique_ptr task_arg(static_cast(arg)); - const std::shared_ptr req = task_arg->req; + std::unique_ptr task_arg(static_cast(arg)); + const std::shared_ptr req = task_arg->req; std::shared_ptr conn = task_arg->conn; LOG(INFO) << "RsyncServer RsyncFile request ..."; @@ -130,42 +194,59 @@ void RsyncServerConn::HandleFileRsyncRequest(void* arg) { std::string filename = req->file_req().filename(); size_t offset = req->file_req().offset(); size_t count = req->file_req().count(); + RsyncService::RsyncResponse response; + response.set_code(RsyncService::kOk); + response.set_type(RsyncService::kRsyncFile); + response.set_db_name(db_name); + response.set_slot_id(slot_id); + std::string snapshot_uuid; Status s = g_pika_server->GetDumpUUID(db_name, slot_id, &snapshot_uuid); - LOG(INFO) << "Receive RsyncFile request " << "filename: " << filename - << " offset: " << offset - << " count: " << count; + response.set_snapshot_uuid(snapshot_uuid); + if (!s.ok()) { + LOG(WARNING) << "rsyncserver get snapshotUUID failed"; + response.set_code(RsyncService::kErr); + RsyncWriteResp(response, conn); + return; + } + std::shared_ptr slot = g_pika_server->GetDBSlotById(db_name, slot_id); + if (!slot) { + LOG(WARNING) << "cannot find slot for db_name " << db_name + << "slot_id: " << slot_id; + response.set_code(RsyncService::kErr); + RsyncWriteResp(response, conn); + } + + const std::string filepath = slot->bgsave_info().path + "/" + filename; char* buffer = new char[req->file_req().count() + 1]; + std::string checksum = ""; size_t bytes_read{0}; - auto status = g_pika_server -> ReadDumpFile(db_name, slot_id, filename, offset, count, buffer, &bytes_read); - LOG(INFO) << "RsyncServer ReadDumpFile: " << filename << " read size: " << bytes_read << "status: " << status.ToString(); - - response.set_type(RsyncService::kRsyncFile); - response.set_snapshot_uuid(snapshot_uuid); - response.set_db_name(db_name); - response.set_slot_id(slot_id); + s = ReadDumpFile(filepath, offset, count, buffer, &bytes_read, &checksum); + if (!s.ok()) { + response.set_code(RsyncService::kErr); + RsyncWriteResp(response, conn); + delete []buffer; + return; + } +/* + LOG(INFO) << "RsyncServer receives FileRequest " << "filename: " + << filename << " offset: " << offset << " count: " + << count << " read_count: " << bytes_read << "checksum:" << checksum; +*/ RsyncService::FileResponse* file_resp = response.mutable_file_resp(); + file_resp->set_data(buffer, bytes_read); file_resp->set_eof(bytes_read != count); + file_resp->set_checksum(checksum); + file_resp->set_filename(filename); file_resp->set_count(bytes_read); file_resp->set_offset(offset); - file_resp->set_data(buffer, bytes_read); - //TODO: checksum - file_resp->set_checksum("checksum"); - file_resp->set_filename(filename); - std::string reply_str; - if (!response.SerializeToString(&reply_str) || (conn->WriteResp(reply_str) != 0)) { - LOG(WARNING) << "Process FileRsync request serialization failed"; - conn->NotifyClose(); - delete []buffer; - return; - } + RsyncWriteResp(response, conn); delete []buffer; - conn->NotifyWrite(); - LOG(INFO) << "RsyncServer RsyncFile request ..."; + //LOG(INFO) << "RsyncServer RsyncFile request ..."; } RsyncServerThread::RsyncServerThread(const std::set& ips, int port, int cron_interval, RsyncServer* arg) diff --git a/src/rsync_service.proto b/src/rsync_service.proto index a35a6835b..a8b96bcbe 100644 --- a/src/rsync_service.proto +++ b/src/rsync_service.proto @@ -6,6 +6,11 @@ enum Type { kRsyncFile = 2; } +enum StatusCode { + kOk = 1; + kErr = 2; +} + message MetaResponse { repeated string filenames = 1; } @@ -37,6 +42,7 @@ message RsyncResponse { required string snapshot_uuid = 2; required string db_name = 3; required uint32 slot_id = 4; - optional MetaResponse meta_resp = 5; - optional FileResponse file_resp = 6; + required StatusCode code = 5; + optional MetaResponse meta_resp = 6; + optional FileResponse file_resp = 7; } \ No newline at end of file From 44c57ed4d029344d39561af88f2d24e61cc3634c Mon Sep 17 00:00:00 2001 From: wangshaoyi Date: Sun, 23 Jul 2023 21:02:57 +0800 Subject: [PATCH 28/37] remove unused code --- CMakeLists.txt | 2 +- include/pika_server.h | 1 - include/rsync_client.h | 189 ++++++++++++++++------------------ include/rsync_client_thread.h | 36 +++---- include/rsync_server.h | 71 +++++-------- src/pika_rm.cc | 4 - src/pika_server.cc | 10 +- src/pika_slot.cc | 1 - src/rsync_client.cc | 30 +----- src/rsync_client_thread.cc | 28 ++--- src/rsync_server.cc | 94 ++++++++--------- 11 files changed, 204 insertions(+), 262 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 0e08c861e..465a5354a 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -47,7 +47,7 @@ elseif(${BUILD_TYPE} STREQUAL RELWITHDEBINFO) set(LIB_BUILD_TYPE RELWITHDEBINFO) else() set(LIB_BUILD_TYPE RElEASE) - set(CMAKE_CXX_FLAGS_RELEASE "-g -DNDEBUG") + set(CMAKE_CXX_FLAGS_RELEASE "-O2 -DNDEBUG") endif() if(CMAKE_SYSTEM_NAME MATCHES "Darwin") diff --git a/include/pika_server.h b/include/pika_server.h index 0712908e1..938ec237f 100644 --- a/include/pika_server.h +++ b/include/pika_server.h @@ -261,7 +261,6 @@ class PikaServer : public pstd::noncopyable { * DBSync used */ pstd::Status GetDumpUUID(const std::string& db_name, const uint32_t slot_id, std::string* snapshot_uuid); - //TODO: yuecai pstd::Status GetDumpMeta(const std::string& db_name, const uint32_t slot_id, std::vector* files, std::string* snapshot_uuid); void DBSync(const std::string& ip, int port, const std::string& db_name, uint32_t slot_id); void TryDBSync(const std::string& ip, int port, const std::string& db_name, uint32_t slot_id, int32_t top); diff --git a/include/rsync_client.h b/include/rsync_client.h index 4b3cf53d7..25217d407 100644 --- a/include/rsync_client.h +++ b/include/rsync_client.h @@ -37,120 +37,111 @@ class WaitObject; class RsyncClient : public net::Thread { public: - enum State { - IDLE, - RUNNING, - STOP, - }; - RsyncClient(const std::string& dir, const std::string& db_name, const uint32_t slot_id); - - void* ThreadMain() override; - bool Init(); - Status Start(); - Status Stop(); - bool IsRunning() { - return state_.load() == RUNNING; - } - bool IsIdle() { return state_.load() == IDLE;} - void OnReceive(RsyncResponse* resp); + enum State { + IDLE, + RUNNING, + STOP, + }; + RsyncClient(const std::string& dir, const std::string& db_name, const uint32_t slot_id); + void* ThreadMain() override; + bool Init(); + Status Start(); + Status Stop(); + bool IsRunning() { + return state_.load() == RUNNING; + } + bool IsIdle() { return state_.load() == IDLE;} + void OnReceive(RsyncResponse* resp); private: - bool Recover(); - Status Wait(RsyncResponse*& resp); - Status CopyRemoteFile(const std::string& filename); - Status CopyRemoteMeta(std::string* snapshot_uuid, std::set* file_set); - Status LoadLocalMeta(std::string* snapshot_uuid, std::map* file_map); - std::string GetLocalMetaFilePath(); - Status FlushMetaTable(); - Status CleanUpExpiredFiles(bool need_reset_path, std::set files); - Status UpdateLocalMeta(std::string& snapshot_uuid, std::set& expired_files, std::map& localFileMap); - void HandleRsyncMetaResponse(RsyncResponse* response); + bool Recover(); + Status Wait(RsyncResponse*& resp); + Status CopyRemoteFile(const std::string& filename); + Status CopyRemoteMeta(std::string* snapshot_uuid, std::set* file_set); + Status LoadLocalMeta(std::string* snapshot_uuid, std::map* file_map); + std::string GetLocalMetaFilePath(); + Status FlushMetaTable(); + Status CleanUpExpiredFiles(bool need_reset_path, std::set files); + Status UpdateLocalMeta(std::string& snapshot_uuid, std::set& expired_files, std::map& localFileMap); + void HandleRsyncMetaResponse(RsyncResponse* response); private: - std::map meta_table_; - int flush_period_; - //待拉取的文件集合 - std::set file_set_; - std::string snapshot_uuid_; - - std::string dir_; - std::string db_name_; - uint32_t slot_id_; - - std::unique_ptr client_thread_; - std::atomic state_; - int max_retries_; - - std::unique_ptr wo_; - std::condition_variable cond_; - std::mutex mu_; - std::unique_ptr throttle_; - - std::string master_ip_; - int master_port_; + std::map meta_table_; + int flush_period_; + std::set file_set_; + std::string snapshot_uuid_; + std::string dir_; + std::string db_name_; + uint32_t slot_id_; + std::unique_ptr client_thread_; + std::atomic state_; + int max_retries_; + std::unique_ptr wo_; + std::condition_variable cond_; + std::mutex mu_; + std::unique_ptr throttle_; + std::string master_ip_; + int master_port_; }; -//TODO: jinge class RsyncWriter { public: - RsyncWriter(const std::string& filepath) { - filepath_ = filepath; - fd_ = open(filepath.c_str(), O_RDWR | O_APPEND | O_CREAT, 0644); - LOG(WARNING) << "rsyncwriter fd: " << fd_; - } - ~RsyncWriter() {} - Status Write(uint64_t offset, size_t n, const char* data) { - const char* ptr = data; - size_t left = n; - Status s; - while (left != 0) { - ssize_t done = write(fd_, ptr, left); - if (done < 0) { - if (errno == EINTR) continue; - LOG(WARNING) << "pwrite failed, filename: " << filepath_ << "errno: " << strerror(errno) << "n: " << n; - return Status::IOError(filepath_, "pwrite failed"); - } - left -= done; - ptr += done; - offset += done; - } - return Status::OK(); - } - Status Close() { - close(fd_); - return Status::OK(); - } - Status Fsync() { - fsync(fd_); - return Status::OK(); + RsyncWriter(const std::string& filepath) { + filepath_ = filepath; + fd_ = open(filepath.c_str(), O_RDWR | O_APPEND | O_CREAT, 0644); + } + ~RsyncWriter() {} + Status Write(uint64_t offset, size_t n, const char* data) { + const char* ptr = data; + size_t left = n; + Status s; + while (left != 0) { + ssize_t done = write(fd_, ptr, left); + if (done < 0) { + if (errno == EINTR) continue; + LOG(WARNING) << "pwrite failed, filename: " << filepath_ << "errno: " << strerror(errno) << "n: " << n; + return Status::IOError(filepath_, "pwrite failed"); + } + left -= done; + ptr += done; + offset += done; } + return Status::OK(); + } + Status Close() { + close(fd_); + return Status::OK(); + } + Status Fsync() { + fsync(fd_); + return Status::OK(); + } private: - std::string filepath_; - int fd_; + std::string filepath_; + int fd_; }; class WaitObject { public: - WaitObject() : filename_(""), type_(RsyncService::kRsyncMeta), offset_(0), resp_(nullptr) {} - ~WaitObject() {} - void Reset(const std::string& filename, RsyncService::Type t, size_t offset) { - resp_ = nullptr; - filename_ = filename; - type_ = t; - offset_ = offset; - } - - void Reset(RsyncService::Type t) { - resp_ = nullptr; - filename_ = ""; - type_ = t; - offset_ = 0xFFFFFFFF; - } - std::string filename_; - RsyncService::Type type_; - size_t offset_; - RsyncResponse* resp_; + WaitObject() : filename_(""), type_(RsyncService::kRsyncMeta), offset_(0), resp_(nullptr) {} + ~WaitObject() {} + void Reset(const std::string& filename, RsyncService::Type t, size_t offset) { + resp_ = nullptr; + filename_ = filename; + type_ = t; + offset_ = offset; + } + void Reset(RsyncService::Type t) { + resp_ = nullptr; + filename_ = ""; + type_ = t; + offset_ = 0xFFFFFFFF; + } + std::string filename_; + RsyncService::Type type_; + size_t offset_; + RsyncResponse* resp_; }; } // end namespace rsync diff --git a/include/rsync_client_thread.h b/include/rsync_client_thread.h index 39a36cceb..6ebdc7de2 100644 --- a/include/rsync_client_thread.h +++ b/include/rsync_client_thread.h @@ -13,36 +13,36 @@ namespace rsync { class RsyncClientConn : public PbConn { public: - RsyncClientConn(int fd, const std::string& ip_port, - net::Thread* thread, void* cb_handler, - NetMultiplexer* mpx); - ~RsyncClientConn() override; - int DealMessage() override; + RsyncClientConn(int fd, const std::string& ip_port, + net::Thread* thread, void* cb_handler, + NetMultiplexer* mpx); + ~RsyncClientConn() override; + int DealMessage() override; private: - void* cb_handler_; + void* cb_handler_; }; class RsyncClientConnFactory : public ConnFactory { public: - RsyncClientConnFactory(void* scheduler) : cb_handler_(scheduler) {} - std::shared_ptr NewNetConn(int connfd, const std::string& ip_port, - net::Thread* thread, void* cb_handler, - net::NetMultiplexer* net) const override { - return std::static_pointer_cast( - std::make_shared(connfd, ip_port, thread, cb_handler_, net)); - } + RsyncClientConnFactory(void* scheduler) : cb_handler_(scheduler) {} + std::shared_ptr NewNetConn(int connfd, const std::string& ip_port, + net::Thread* thread, void* cb_handler, + net::NetMultiplexer* net) const override { + return std::static_pointer_cast( + std::make_shared(connfd, ip_port, thread, cb_handler_, net)); + } private: - void* cb_handler_; + void* cb_handler_; }; class RsyncClientThread : public ClientThread { public: - RsyncClientThread(int cron_interval, int keepalive_timeout, void* scheduler); - ~RsyncClientThread() override; + RsyncClientThread(int cron_interval, int keepalive_timeout, void* scheduler); + ~RsyncClientThread() override; private: - RsyncClientConnFactory conn_factory_; - ClientHandle handle_; + RsyncClientConnFactory conn_factory_; + ClientHandle handle_; }; } //end namespace rsync diff --git a/include/rsync_server.h b/include/rsync_server.h index 7906d12db..c77dfba1a 100644 --- a/include/rsync_server.h +++ b/include/rsync_server.h @@ -33,31 +33,27 @@ class RsyncServerThread; class RsyncServer { public: - RsyncServer(const std::set& ips, const int port); - ~RsyncServer(); - void Schedule(net::TaskFunc func, void* arg); - int Start(); - int Stop(); + RsyncServer(const std::set& ips, const int port); + ~RsyncServer(); + void Schedule(net::TaskFunc func, void* arg); + int Start(); + int Stop(); private: - std::unique_ptr work_thread_ = nullptr; - std::unique_ptr rsync_server_thread_ = nullptr; + std::unique_ptr work_thread_ = nullptr; + std::unique_ptr rsync_server_thread_ = nullptr; }; class RsyncServerConn : public PbConn { public: - RsyncServerConn(int connfd, const std::string& ip_port, - Thread* thread, void* worker_specific_data, - NetMultiplexer* mpx); - virtual ~RsyncServerConn() override; - int DealMessage() override; - //处理slave发来的meta请求,arg参数类型为RsyncServerTaskArg, - //请求处理完成之后将序列化好的response通过conn->WriteResp进行发送 - static void HandleMetaRsyncRequest(void* arg); - //处理slave发来的file请求,arg参数类型为RsyncServerTaskArg - //请求处理完成之后将序列化好的response通过conn->WriteResp进行发送 - static void HandleFileRsyncRequest(void* arg); + RsyncServerConn(int connfd, const std::string& ip_port, + Thread* thread, void* worker_specific_data, + NetMultiplexer* mpx); + virtual ~RsyncServerConn() override; + int DealMessage() override; + static void HandleMetaRsyncRequest(void* arg); + static void HandleFileRsyncRequest(void* arg); private: - void* data_; + void* data_; }; class RsyncServerThread : public HolyThread { @@ -66,19 +62,18 @@ class RsyncServerThread : public HolyThread { ~RsyncServerThread(); private: - class RsyncServerConnFactory : public ConnFactory { - public: - explicit RsyncServerConnFactory(RsyncServer* sched) : scheduler_(sched) {} - - std::shared_ptr NewNetConn(int connfd, const std::string& ip_port, - Thread* thread, void* worker_specific_data, - NetMultiplexer* net) const override { - return std::static_pointer_cast( - std::make_shared(connfd, ip_port, thread, scheduler_, net)); - } - private: - RsyncServer* scheduler_; + class RsyncServerConnFactory : public ConnFactory { + public: + explicit RsyncServerConnFactory(RsyncServer* sched) : scheduler_(sched) {} + std::shared_ptr NewNetConn(int connfd, const std::string& ip_port, + Thread* thread, void* worker_specific_data, + NetMultiplexer* net) const override { + return std::static_pointer_cast( + std::make_shared(connfd, ip_port, thread, scheduler_, net)); + } + private: + RsyncServer* scheduler_; }; class RsyncServerHandle : public ServerHandle { public: @@ -94,24 +89,12 @@ class RsyncServerThread : public HolyThread { }; class RsyncServerConnFactory : public ConnFactory { - public: +public: virtual std::shared_ptr NewNetConn(int connfd, const std::string& ip_port, Thread* thread, void* worker_specific_data, NetMultiplexer* net_epoll) const override; }; -class RSyncReader { -public: - RSyncReader(const std::string& filepath); - ~RSyncReader(); - Status Read(uint64_t offset, size_t n, Slice* result); - -private: - std::string filepath_; - std::unique_ptr file_; -public: - std::string GetFilePath() { return filepath_; } -}; } //end namespace rsync #endif diff --git a/src/pika_rm.cc b/src/pika_rm.cc index c5858c81a..4f1871612 100644 --- a/src/pika_rm.cc +++ b/src/pika_rm.cc @@ -538,7 +538,6 @@ Status SyncMasterSlot::ConsensusReset(const LogOffset& applied_offset) { return /* SyncSlaveSlot */ SyncSlaveSlot::SyncSlaveSlot(const std::string& db_name, uint32_t slot_id) : SyncSlot(db_name, slot_id) { - //TODO: get dbsync_path from slot class std::string dbsync_path = g_pika_conf->db_sync_path() + "/" + db_name; rsync_cli_.reset(new rsync::RsyncClient(dbsync_path, db_name, slot_id)); m_info_.SetLastRecvTime(pstd::NowMicros()); @@ -648,9 +647,6 @@ void SyncSlaveSlot::ActivateRsync() { LOG(WARNING) << "ActivateRsync ..."; if (rsync_cli_->Init()) { rsync_cli_->Start(); - LOG(WARNING) << "ActivateRsync done..."; - } else { - LOG(WARNING) << "ActivateRsync init failed..."; } } diff --git a/src/pika_server.cc b/src/pika_server.cc index c96a96ab2..002102ca4 100644 --- a/src/pika_server.cc +++ b/src/pika_server.cc @@ -134,12 +134,15 @@ bool PikaServer::ServerInit() { void PikaServer::Start() { int ret = 0; // start rsync first, rocksdb opened fd will not appear in this fork - //ret = pika_rsync_service_->StartRsync(); + // TODO: temporarily disable rsync server + /* + ret = pika_rsync_service_->StartRsync(); if (0 != ret) { dbs_.clear(); LOG(FATAL) << "Start Rsync Error: bind port " + std::to_string(pika_rsync_service_->ListenPort()) + " failed" << ", Listen on this port to receive Master FullSync Data"; } + */ // We Init DB Struct Before Start The following thread InitDBStruct(); @@ -956,8 +959,8 @@ void PikaServer::TryDBSync(const std::string& ip, int port, const std::string& d // Need Bgsave first slot->BgSaveSlot(); } - return; - DBSync(ip, port, db_name, slot_id); + //TODO: temporarily disable rsync server + //DBSync(ip, port, db_name, slot_id); } void PikaServer::DbSyncSendFile(const std::string& ip, int port, const std::string& db_name, uint32_t slot_id) { @@ -1288,6 +1291,7 @@ void PikaServer::DoTimingTask() { // Delete expired dump AutoDeleteExpiredDump(); // Cheek Rsync Status + //TODO: temporarily disable rsync //AutoKeepAliveRSync(); // Reset server qps ResetLastSecQuerynum(); diff --git a/src/pika_slot.cc b/src/pika_slot.cc index 4c0769aa3..8dafdd9e5 100644 --- a/src/pika_slot.cc +++ b/src/pika_slot.cc @@ -330,7 +330,6 @@ void Slot::GetBgSaveMetaData(std::vector* fileNames, std::string* s Status Slot::GetBgSaveUUID(std::string* snapshot_uuid) { if (snapshot_uuid_.empty()) { std::string info_data; - // todo 待确认 info 文件的路径 const std::string infoPath = bgsave_info().path + "/info"; // todo 这里待替换 rocksdb::Status s = rocksdb::ReadFileToString(rocksdb::Env::Default(), infoPath, &info_data); diff --git a/src/rsync_client.cc b/src/rsync_client.cc index f9b52bcb2..3350cd150 100644 --- a/src/rsync_client.cc +++ b/src/rsync_client.cc @@ -46,9 +46,7 @@ void* RsyncClient::ThreadMain() { Status s = Status::OK(); LOG(INFO) << "RsyncClient ThreadMain..."; if (file_set_.empty()) { - LOG(INFO) << "no newly files need to download..."; state_.store(STOP); - LOG(INFO) << "RsyncClient ThreadMain done..."; return nullptr; } @@ -58,14 +56,13 @@ void* RsyncClient::ThreadMain() { std::string meta_rep; for (const auto& file : file_set_) { - LOG(INFO) << "CopyRemoteFile: " << file << " state_: " << state_.load(); + LOG(INFO) << "CopyRemoteFile: " << file; while (state_.load() == RUNNING) { s = CopyRemoteFile(file); if (!s.ok()) { LOG(WARNING) << "rsync CopyRemoteFile failed, filename: " << file; continue; } - //LOG(WARNING) << "CopyRemoteFile "<< file << " success..."; break; } if (state_.load(std::memory_order_relaxed) != RUNNING) { @@ -75,7 +72,6 @@ void* RsyncClient::ThreadMain() { meta_rep.append("\n"); if (++period == flush_period_) { period = 0; - LOG(INFO) << "periodically flush meta table..., meta_rep: " << meta_rep; outfile << meta_rep; outfile.flush(); meta_rep.clear(); @@ -86,7 +82,7 @@ void* RsyncClient::ThreadMain() { outfile.flush(); } state_.store(STOP); - LOG(INFO) << "RsyncClient ThreadMain done..."; + LOG(INFO) << "RsyncClient fetch copy remote files done..."; return nullptr; } @@ -153,7 +149,6 @@ Status RsyncClient::CopyRemoteFile(const std::string& filename) { std::string to_send; request.SerializeToString(&to_send); - //LOG(WARNING) << "master ip: " << master_ip_ << " master_port: " << master_port_; s = client_thread_->Write(master_ip_, master_port_, to_send); if (!s.ok()) { LOG(WARNING) << "send rsync request failed"; @@ -165,7 +160,6 @@ Status RsyncClient::CopyRemoteFile(const std::string& filename) { wo_->Reset(filename, kRsyncFile, offset); } - //LOG(INFO) << "wait CopyRemoteFile response....."; RsyncResponse* resp = nullptr; s = Wait(resp); if (s.IsTimeout() || resp == nullptr) { @@ -184,12 +178,6 @@ Status RsyncClient::CopyRemoteFile(const std::string& filename) { continue; } - /* - LOG(INFO) << "receive fileresponse, snapshot_uuid: " << resp->snapshot_uuid() - << "filename: " << resp->file_resp().filename() << "offset: " << resp->file_resp().offset() - << "count: " << resp->file_resp().count() << "eof: " << resp->file_resp().eof(); - */ - if (resp->snapshot_uuid() != snapshot_uuid_) { LOG(WARNING) << "receive newer dump, reset state to STOP, local_snapshot_uuid:" << snapshot_uuid_ << "remote snapshot uuid: " << resp->snapshot_uuid(); @@ -209,7 +197,6 @@ Status RsyncClient::CopyRemoteFile(const std::string& filename) { md5.update(resp->file_resp().data().c_str(), ret_count); offset += resp->file_resp().count(); if (resp->file_resp().eof()) { - LOG(WARNING) << "filename: " << filename << " md5: " << md5.finalize().hexdigest(); if (md5.finalize().hexdigest() != resp->file_resp().checksum()) { LOG(WARNING) << "mismatch file checksum for file: " << filename; s = Status::IOError("mismatch checksum", "mismatch checksum"); @@ -234,9 +221,7 @@ Status RsyncClient::CopyRemoteFile(const std::string& filename) { } Status RsyncClient::Start() { - LOG(WARNING) << "RsyncClient start ..."; StartThread(); - LOG(WARNING) << "RsyncClient StartThread done..."; return Status::OK(); } @@ -247,13 +232,9 @@ Status RsyncClient::Stop() { LOG(WARNING) << "RsyncClient stop ..."; state_ = STOP; StopThread(); - LOG(WARNING) << "RsyncClient StopThread done..."; client_thread_->StopThread(); - LOG(WARNING) << "RsyncClient Stop clientThread done..."; JoinThread(); - LOG(WARNING) << "RsyncClient JoinThread done..."; client_thread_->JoinThread(); - LOG(WARNING) << "RsyncClient join clientThread done..."; state_ = IDLE; return Status::OK(); } @@ -281,9 +262,7 @@ bool RsyncClient::Recover() { } std::set expired_files; - //LOG(WARNING) << "file_set origin size: " << file_set_.size() << " local_snapshot_uuid: " << local_snapshot_uuid << " remote_snapshot_uuid: " << remote_snapshot_uuid; if (remote_snapshot_uuid != local_snapshot_uuid) { - LOG(WARNING) << "snapshot uuid mismatch"; snapshot_uuid_ = remote_snapshot_uuid; file_set_ = remote_file_set; expired_files = local_file_set; @@ -298,7 +277,6 @@ bool RsyncClient::Recover() { inserter(expired_files, expired_files.begin())); file_set_.insert(newly_files.begin(), newly_files.end()); } - LOG(WARNING) << "file_set merged size: " << file_set_.size(); s = CleanUpExpiredFiles(local_snapshot_uuid != remote_snapshot_uuid, expired_files); if (!s.ok()) { @@ -353,12 +331,13 @@ Status RsyncClient::CopyRemoteMeta(std::string* snapshot_uuid, std::setsnapshot_uuid() << "files count: " << resp->meta_resp().filenames_size(); - for (std::string item : resp->meta_resp().filenames()) { file_set->insert(item); } + *snapshot_uuid = resp->snapshot_uuid(); for (int i = 0; i < resp->meta_resp().filenames_size(); i++) { file_set->insert(resp->meta_resp().filenames(i)); @@ -392,6 +371,7 @@ Status RsyncClient::LoadLocalMeta(std::string* snapshot_uuid, std::mapParseFromCodedStream(&decoder) && decoder.ConsumedEntireMessage(); - if (!success) { - LOG(WARNING) << "ParseFromArray FAILED! " - << " msg_len: " << header_len_; - return -1; - } - RsyncClient* handler = (RsyncClient*)cb_handler_; - handler->OnReceive(response); - return 0; + RsyncResponse* response = new RsyncResponse(); + ::google::protobuf::io::ArrayInputStream input(rbuf_ + cur_pos_ - header_len_, header_len_); + ::google::protobuf::io::CodedInputStream decoder(&input); + decoder.SetTotalBytesLimit(PIKA_MAX_CONN_RBUF); + bool success = response->ParseFromCodedStream(&decoder) && decoder.ConsumedEntireMessage(); + if (!success) { + LOG(WARNING) << "ParseFromArray FAILED! " + << " msg_len: " << header_len_; + return -1; + } + RsyncClient* handler = (RsyncClient*)cb_handler_; + handler->OnReceive(response); + return 0; } RsyncClientThread::RsyncClientThread(int cron_interval, int keepalive_timeout, void* scheduler) @@ -35,4 +35,4 @@ RsyncClientThread::RsyncClientThread(int cron_interval, int keepalive_timeout, v conn_factory_(scheduler) {} RsyncClientThread::~RsyncClientThread() {} -} \ No newline at end of file +} //end namespace rsync \ No newline at end of file diff --git a/src/rsync_server.cc b/src/rsync_server.cc index e393142a3..be517a424 100644 --- a/src/rsync_server.cc +++ b/src/rsync_server.cc @@ -11,7 +11,7 @@ extern PikaServer* g_pika_server; namespace rsync { -//TODO: optimzie file read +//TODO: optimzie file read and calculate checksum, maybe use RsyncReader prefeching file content Status ReadDumpFile(const std::string filepath, const size_t offset, const size_t count, char* data, size_t* bytes_read, std::string* checksum) { int fd = open(filepath.c_str(), O_RDONLY); @@ -77,40 +77,38 @@ void RsyncWriteResp(RsyncService::RsyncResponse& response, std::shared_ptr& ips, const int port) { - work_thread_ = std::make_unique(2, 100000); - rsync_server_thread_ = std::make_unique(ips, port, 60 * 1000, this); + work_thread_ = std::make_unique(2, 100000); + rsync_server_thread_ = std::make_unique(ips, port, 60 * 1000, this); } RsyncServer::~RsyncServer() { - //TODO: handle destory - LOG(INFO) << "Rsync server destroyed"; + //TODO: handle destory + LOG(INFO) << "Rsync server destroyed"; } void RsyncServer::Schedule(net::TaskFunc func, void* arg) { - work_thread_->Schedule(func, arg); + work_thread_->Schedule(func, arg); } int RsyncServer::Start() { - LOG(INFO) << "start RsyncServer ..."; - int res = rsync_server_thread_->StartThread(); - if (res != net::kSuccess) { - LOG(FATAL) << "Start rsync Server Thread Error: " << res; - } - res = work_thread_->start_thread_pool(); - if (res != net::kSuccess) { - LOG(FATAL) << "Start ThreadPool Error: " << res - << (res == net::kCreateThreadError ? ": create thread error " : ": other error"); - } - LOG(INFO) << "start RsyncServer done ..."; + LOG(INFO) << "start RsyncServer ..."; + int res = rsync_server_thread_->StartThread(); + if (res != net::kSuccess) { + LOG(FATAL) << "Start rsync Server Thread Error: " << res; + } + res = work_thread_->start_thread_pool(); + if (res != net::kSuccess) { + LOG(FATAL) << "Start ThreadPool Error: " << res + << (res == net::kCreateThreadError ? ": create thread error " : ": other error"); + } return res; } int RsyncServer::Stop() { - LOG(INFO) << "stop RsyncServer ..."; - work_thread_->stop_thread_pool(); - rsync_server_thread_->StopThread(); - LOG(INFO) << "stop RsyncServer done..."; - return 0; + LOG(INFO) << "stop RsyncServer ..."; + work_thread_->stop_thread_pool(); + rsync_server_thread_->StopThread(); + return 0; } RsyncServerConn::RsyncServerConn(int connfd, const std::string& ip_port, Thread* thread, @@ -120,36 +118,35 @@ RsyncServerConn::RsyncServerConn(int connfd, const std::string& ip_port, Thread* RsyncServerConn::~RsyncServerConn() {} int RsyncServerConn::DealMessage() { - std::shared_ptr req = std::make_shared(); - bool parse_res = req->ParseFromArray(rbuf_ + cur_pos_ - header_len_, header_len_); - //LOG(INFO) << "RsyncServer receives new request..."; - if (!parse_res) { - LOG(WARNING) << "Pika rsync server connection pb parse error."; - return -1; - } - switch (req->type()) { - case RsyncService::kRsyncMeta: { - auto task_arg = - new RsyncServerTaskArg(req, std::dynamic_pointer_cast(shared_from_this())); - ((RsyncServer*)(data_))->Schedule(&RsyncServerConn::HandleMetaRsyncRequest, task_arg); - break; - } - case RsyncService::kRsyncFile: { - auto task_arg = - new RsyncServerTaskArg(req, std::dynamic_pointer_cast(shared_from_this())); + std::shared_ptr req = std::make_shared(); + bool parse_res = req->ParseFromArray(rbuf_ + cur_pos_ - header_len_, header_len_); + if (!parse_res) { + LOG(WARNING) << "Pika rsync server connection pb parse error."; + return -1; + } + switch (req->type()) { + case RsyncService::kRsyncMeta: { + auto task_arg = + new RsyncServerTaskArg(req, std::dynamic_pointer_cast(shared_from_this())); + ((RsyncServer*)(data_))->Schedule(&RsyncServerConn::HandleMetaRsyncRequest, task_arg); + break; + } + case RsyncService::kRsyncFile: { + auto task_arg = + new RsyncServerTaskArg(req, std::dynamic_pointer_cast(shared_from_this())); ((RsyncServer*)(data_))->Schedule(&RsyncServerConn::HandleFileRsyncRequest, task_arg); break; - } - default: { - LOG(WARNING) << "Invalid RsyncRequest type"; - } + } + default: { + LOG(WARNING) << "Invalid RsyncRequest type"; + } } return 0; } void RsyncServerConn::HandleMetaRsyncRequest(void* arg) { - std::unique_ptr task_arg(static_cast(arg)); - const std::shared_ptr req = task_arg->req; + std::unique_ptr task_arg(static_cast(arg)); + const std::shared_ptr req = task_arg->req; std::shared_ptr conn = task_arg->conn; std::string db_name = req->db_name(); uint32_t slot_id = req->slot_id(); @@ -187,7 +184,6 @@ void RsyncServerConn::HandleFileRsyncRequest(void* arg) { std::unique_ptr task_arg(static_cast(arg)); const std::shared_ptr req = task_arg->req; std::shared_ptr conn = task_arg->conn; - LOG(INFO) << "RsyncServer RsyncFile request ..."; uint32_t slot_id = req->slot_id(); std::string db_name = req->db_name(); @@ -230,11 +226,6 @@ void RsyncServerConn::HandleFileRsyncRequest(void* arg) { delete []buffer; return; } -/* - LOG(INFO) << "RsyncServer receives FileRequest " << "filename: " - << filename << " offset: " << offset << " count: " - << count << " read_count: " << bytes_read << "checksum:" << checksum; -*/ RsyncService::FileResponse* file_resp = response.mutable_file_resp(); file_resp->set_data(buffer, bytes_read); @@ -246,7 +237,6 @@ void RsyncServerConn::HandleFileRsyncRequest(void* arg) { RsyncWriteResp(response, conn); delete []buffer; - //LOG(INFO) << "RsyncServer RsyncFile request ..."; } RsyncServerThread::RsyncServerThread(const std::set& ips, int port, int cron_interval, RsyncServer* arg) From fe2508083adbd18630d8fd0c7a652db44f141124 Mon Sep 17 00:00:00 2001 From: wangshaoyi Date: Sun, 23 Jul 2023 21:11:52 +0800 Subject: [PATCH 29/37] remove unused code --- src/rsync_client.cc | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/src/rsync_client.cc b/src/rsync_client.cc index 2fb6492c6..0e6797725 100644 --- a/src/rsync_client.cc +++ b/src/rsync_client.cc @@ -160,6 +160,10 @@ Status RsyncClient::CopyRemoteFile(const std::string& filename) { wo_->Reset(filename, kRsyncFile, offset); } + RsyncResponse* resp = nullptr; + s = Wait(resp); + if (s.IsTimeout() || resp == nullptr) { + LOG(WARNING) << "rsync request timeout"; retries++; continue; } @@ -198,6 +202,10 @@ Status RsyncClient::CopyRemoteFile(const std::string& filename) { s = Status::IOError("mismatch checksum", "mismatch checksum"); return s; } + s = writer->Fsync(); + if (!s.ok()) { + return s; + } s = writer->Close(); if (!s.ok()) { return s; @@ -323,7 +331,6 @@ Status RsyncClient::CopyRemoteMeta(std::string* snapshot_uuid, std::setsnapshot_uuid() << "files count: " << resp->meta_resp().filenames_size(); for (std::string item : resp->meta_resp().filenames()) { From f48726b2580f1d81aa0d7916c96b6d1d6ae30715 Mon Sep 17 00:00:00 2001 From: wangshaoyi Date: Mon, 24 Jul 2023 20:15:00 +0800 Subject: [PATCH 30/37] remove unused code --- .github/workflows/codeql.yml | 4 ++-- .github/workflows/codis.yml | 4 ++-- .github/workflows/pika.yml | 4 ++-- .github/workflows/tools_go.yml | 4 ++-- 4 files changed, 8 insertions(+), 8 deletions(-) diff --git a/.github/workflows/codeql.yml b/.github/workflows/codeql.yml index dd1b733e6..23f547db9 100644 --- a/.github/workflows/codeql.yml +++ b/.github/workflows/codeql.yml @@ -13,10 +13,10 @@ name: "CodeQL" on: push: - branches: [ "unstable", "*" ] + branches: [ "unstable" ] pull_request: # The branches below must be a subset of the branches above - branches: [ "unstable", "*" ] + branches: [ "unstable" ] schedule: - cron: '25 19 * * 6' diff --git a/.github/workflows/codis.yml b/.github/workflows/codis.yml index 917c45d64..b097078bf 100644 --- a/.github/workflows/codis.yml +++ b/.github/workflows/codis.yml @@ -5,9 +5,9 @@ name: Codis on: push: - branches: [ "unstable", "*"] + branches: [ "unstable" ] pull_request: - branches: [ "unstable", "*"] + branches: [ "unstable" ] jobs: diff --git a/.github/workflows/pika.yml b/.github/workflows/pika.yml index 30f8d643e..b4ed354a1 100644 --- a/.github/workflows/pika.yml +++ b/.github/workflows/pika.yml @@ -2,9 +2,9 @@ name: Pika on: push: - branches: [ "unstable", "*" ] + branches: [ "unstable" ] pull_request: - branches: [ "unstable", "*" ] + branches: [ "unstable" ] env: # Customize the CMake build type here (Release, Debug, RelWithDebInfo, etc.) diff --git a/.github/workflows/tools_go.yml b/.github/workflows/tools_go.yml index 087ee6c34..82d88e2ec 100644 --- a/.github/workflows/tools_go.yml +++ b/.github/workflows/tools_go.yml @@ -2,11 +2,11 @@ name: Tools_go_build on: push: - branches: [ "unstable", "*" ] + branches: [ "unstable" ] paths: - 'tools/**' pull_request: - branches: [ "unstable", "*" ] + branches: [ "unstable" ] paths: - 'tools/**' From c5936b268e66d49498048266f0131368c42aa911 Mon Sep 17 00:00:00 2001 From: wangshaoyi Date: Mon, 24 Jul 2023 20:18:30 +0800 Subject: [PATCH 31/37] remove unused code --- .github/workflows/operator.yml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/workflows/operator.yml b/.github/workflows/operator.yml index 51c3dbfa9..fc8045872 100644 --- a/.github/workflows/operator.yml +++ b/.github/workflows/operator.yml @@ -5,9 +5,9 @@ name: Operator on: push: - branches: [ "unstable", "*" ] + branches: [ "unstable" ] pull_request: - branches: [ "unstable", "*" ] + branches: [ "unstable" ] jobs: @@ -29,7 +29,7 @@ jobs: - name: Build run: | - cd tools/pika_operator && make + cd tools/pika_operator && make - name: Unit Test run: | From afd133a8131cc7ec163d6bb1336f4d93c9c123a4 Mon Sep 17 00:00:00 2001 From: wangshaoyi Date: Mon, 24 Jul 2023 20:21:57 +0800 Subject: [PATCH 32/37] add copyright --- include/rsync_client.h | 5 +++++ include/rsync_client_thread.h | 5 +++++ include/rsync_server.h | 5 +++++ include/throttle.h | 5 +++++ src/rsync_client.cc | 5 +++++ src/rsync_client_thread.cc | 5 +++++ src/rsync_server.cc | 5 +++++ 7 files changed, 35 insertions(+) diff --git a/include/rsync_client.h b/include/rsync_client.h index 25217d407..145b67567 100644 --- a/include/rsync_client.h +++ b/include/rsync_client.h @@ -1,3 +1,8 @@ +// Copyright (c) 2023-present, Qihoo, Inc. All rights reserved. +// This source code is licensed under the BSD-style license found in the +// LICENSE file in the root directory of this source tree. An additional grant +// of patent rights can be found in the PATENTS file in the same directory. + #ifndef RSYNC_CLIENT_H_ #define RSYNC_CLIENT_H_ #include diff --git a/include/rsync_client_thread.h b/include/rsync_client_thread.h index 6ebdc7de2..e9784323e 100644 --- a/include/rsync_client_thread.h +++ b/include/rsync_client_thread.h @@ -1,3 +1,8 @@ +// Copyright (c) 2023-present, Qihoo, Inc. All rights reserved. +// This source code is licensed under the BSD-style license found in the +// LICENSE file in the root directory of this source tree. An additional grant +// of patent rights can be found in the PATENTS file in the same directory. + #ifndef RSYNC_CLIENT_THREAD_H_ #define RSYNC_CLIENT_THREAD_H_ diff --git a/include/rsync_server.h b/include/rsync_server.h index c77dfba1a..8316ebe61 100644 --- a/include/rsync_server.h +++ b/include/rsync_server.h @@ -1,3 +1,8 @@ +// Copyright (c) 2023-present, Qihoo, Inc. All rights reserved. +// This source code is licensed under the BSD-style license found in the +// LICENSE file in the root directory of this source tree. An additional grant +// of patent rights can be found in the PATENTS file in the same directory. + #ifndef RSYNC_SERVER_H_ #define RSYNC_SERVER_H_ #include diff --git a/include/throttle.h b/include/throttle.h index 2ffa52a9b..f1242d5c4 100644 --- a/include/throttle.h +++ b/include/throttle.h @@ -1,3 +1,8 @@ +// Copyright (c) 2023-present, Qihoo, Inc. All rights reserved. +// This source code is licensed under the BSD-style license found in the +// LICENSE file in the root directory of this source tree. An additional grant +// of patent rights can be found in the PATENTS file in the same directory. + #ifndef THROTTLE_H_ #define THROTTLE_H_ #include diff --git a/src/rsync_client.cc b/src/rsync_client.cc index 0e6797725..002f06237 100644 --- a/src/rsync_client.cc +++ b/src/rsync_client.cc @@ -1,3 +1,8 @@ +// Copyright (c) 2023-present, Qihoo, Inc. All rights reserved. +// This source code is licensed under the BSD-style license found in the +// LICENSE file in the root directory of this source tree. An additional grant +// of patent rights can be found in the PATENTS file in the same directory. + #include "include/rsync_client.h" #include #include "include/pika_server.h" diff --git a/src/rsync_client_thread.cc b/src/rsync_client_thread.cc index 6ffcc1f2d..926b3aa05 100644 --- a/src/rsync_client_thread.cc +++ b/src/rsync_client_thread.cc @@ -1,3 +1,8 @@ +// Copyright (c) 2023-present, Qihoo, Inc. All rights reserved. +// This source code is licensed under the BSD-style license found in the +// LICENSE file in the root directory of this source tree. An additional grant +// of patent rights can be found in the PATENTS file in the same directory. + #include "include/rsync_client_thread.h" #include "include/rsync_client.h" #include "include/pika_define.h" diff --git a/src/rsync_server.cc b/src/rsync_server.cc index be517a424..ef0d39c0e 100644 --- a/src/rsync_server.cc +++ b/src/rsync_server.cc @@ -1,3 +1,8 @@ +// Copyright (c) 2023-present, Qihoo, Inc. All rights reserved. +// This source code is licensed under the BSD-style license found in the +// LICENSE file in the root directory of this source tree. An additional grant +// of patent rights can be found in the PATENTS file in the same directory. + #include #include From bbd489c613fc862f48316d6adbabf601899bf9bd Mon Sep 17 00:00:00 2001 From: wangshao1 <30471730+wangshao1@users.noreply.github.com> Date: Tue, 25 Jul 2023 14:06:11 +0800 Subject: [PATCH 33/37] fix by review comments (#213) Co-authored-by: wangshaoyi --- include/rsync_client.h | 8 +++++--- include/rsync_server.h | 1 - src/pika_server.cc | 6 +++--- src/rsync_client.cc | 16 ++++++++-------- src/rsync_client_thread.cc | 2 +- src/rsync_service.proto | 2 +- src/throttle.cc | 2 +- 7 files changed, 19 insertions(+), 18 deletions(-) diff --git a/include/rsync_client.h b/include/rsync_client.h index 145b67567..7e65fa29c 100644 --- a/include/rsync_client.h +++ b/include/rsync_client.h @@ -66,8 +66,8 @@ class RsyncClient : public net::Thread { Status LoadLocalMeta(std::string* snapshot_uuid, std::map* file_map); std::string GetLocalMetaFilePath(); Status FlushMetaTable(); - Status CleanUpExpiredFiles(bool need_reset_path, std::set files); - Status UpdateLocalMeta(std::string& snapshot_uuid, std::set& expired_files, std::map& localFileMap); + Status CleanUpExpiredFiles(bool need_reset_path, const std::set& files); + Status UpdateLocalMeta(const std::string& snapshot_uuid, const std::set& expired_files, std::map* localFileMap); void HandleRsyncMetaResponse(RsyncResponse* response); private: @@ -103,7 +103,9 @@ class RsyncWriter { while (left != 0) { ssize_t done = write(fd_, ptr, left); if (done < 0) { - if (errno == EINTR) continue; + if (errno == EINTR) { + continue; + } LOG(WARNING) << "pwrite failed, filename: " << filepath_ << "errno: " << strerror(errno) << "n: " << n; return Status::IOError(filepath_, "pwrite failed"); } diff --git a/include/rsync_server.h b/include/rsync_server.h index 8316ebe61..a8477f414 100644 --- a/include/rsync_server.h +++ b/include/rsync_server.h @@ -88,7 +88,6 @@ class RsyncServerThread : public HolyThread { void CronHandle() const override; }; private: - void* arg_; RsyncServerConnFactory conn_factory_; RsyncServerHandle handle_; }; diff --git a/src/pika_server.cc b/src/pika_server.cc index 10889ea49..ac1c9ffcf 100644 --- a/src/pika_server.cc +++ b/src/pika_server.cc @@ -79,7 +79,7 @@ PikaServer::PikaServer() pika_dispatch_thread_ = std::make_unique(ips, port_, worker_num_, 3000, worker_queue_limit, g_pika_conf->max_conn_rbuf_size()); pika_rsync_service_ = std::make_unique(g_pika_conf->db_sync_path(), g_pika_conf->port() + kPortShiftRSync); - //TODO 删除pika_rsync_service_服务,使用pika_rsync_service_端口 + //TODO: remove pika_rsync_service_,reuse pika_rsync_service_ port rsync_server_ = std::make_unique(ips, port_ + kPortShiftRsync2); pika_pubsub_thread_ = std::make_unique(); pika_auxiliary_thread_ = std::make_unique(); @@ -928,7 +928,7 @@ void PikaServer::DBSync(const std::string& ip, int port, const std::string& db_n pstd::Status PikaServer::GetDumpUUID(const std::string& db_name, const uint32_t slot_id, std::string* snapshot_uuid) { std::shared_ptr slot = GetDBSlotById(db_name, slot_id); if (!slot) { - LOG(WARNING) << "cannot find slot for db_name " << db_name << "slot_id: " << slot_id; + LOG(WARNING) << "cannot find slot for db_name " << db_name << " slot_id: " << slot_id; return pstd::Status::NotFound("slot no found"); } slot->GetBgSaveUUID(snapshot_uuid); @@ -938,7 +938,7 @@ pstd::Status PikaServer::GetDumpUUID(const std::string& db_name, const uint32_t pstd::Status PikaServer::GetDumpMeta(const std::string& db_name, const uint32_t slot_id, std::vector* fileNames, std::string* snapshot_uuid) { std::shared_ptr slot = GetDBSlotById(db_name, slot_id); if (!slot) { - LOG(WARNING) << "cannot find slot for db_name " << db_name << "slot_id: " << slot_id; + LOG(WARNING) << "cannot find slot for db_name " << db_name << " slot_id: " << slot_id; return pstd::Status::NotFound("slot no found"); } slot->GetBgSaveMetaData(fileNames, snapshot_uuid); diff --git a/src/rsync_client.cc b/src/rsync_client.cc index 002f06237..69a9d8659 100644 --- a/src/rsync_client.cc +++ b/src/rsync_client.cc @@ -19,7 +19,8 @@ extern PikaServer* g_pika_server; namespace rsync { RsyncClient::RsyncClient(const std::string& dir, const std::string& db_name, const uint32_t slot_id) - : dir_(dir), flush_period_(10), db_name_(db_name), slot_id_(slot_id), state_(IDLE), max_retries_(10) { + : flush_period_(10), snapshot_uuid_(""), dir_(dir), db_name_(db_name), slot_id_(slot_id), + state_(IDLE), max_retries_(10), master_ip_(""), master_port_(0) { client_thread_ = std::make_unique(10 * 1000, 60 * 1000, this); wo_.reset(new WaitObject()); throttle_.reset(new Throttle()); @@ -33,7 +34,6 @@ bool RsyncClient::Init() { master_ip_ = g_pika_server->master_ip(); master_port_ = g_pika_server->master_port() + kPortShiftRsync2; file_set_.clear(); - // todo client 的 StartThread 只能被调用一次,如果一个 slot 进行多次主从同步,这里会出问题吗? client_thread_->StartThread(); bool ret = Recover(); if (!ret) { @@ -288,7 +288,7 @@ bool RsyncClient::Recover() { LOG(WARNING) << "clean up expired files failed"; return false; } - s = UpdateLocalMeta(snapshot_uuid_, expired_files, local_file_map); + s = UpdateLocalMeta(snapshot_uuid_, expired_files, &local_file_map); if (!s.ok()) { LOG(WARNING) << "update local meta failed"; return false; @@ -412,7 +412,7 @@ Status RsyncClient::LoadLocalMeta(std::string* snapshot_uuid, std::map files) { +Status RsyncClient::CleanUpExpiredFiles(bool need_reset_path, const std::set& files) { if (need_reset_path) { std::string db_path = dir_ + (dir_.back() == '/' ? "" : "/"); pstd::DeleteDirIfExist(db_path); @@ -435,10 +435,10 @@ Status RsyncClient::CleanUpExpiredFiles(bool need_reset_path, std::set& expired_files, - std::map& localFileMap) { +Status RsyncClient::UpdateLocalMeta(const std::string& snapshot_uuid, const std::set& expired_files, + std::map* localFileMap) { for (const auto& item : expired_files) { - localFileMap.erase(item); + localFileMap->erase(item); } std::string meta_file_path = GetLocalMetaFilePath(); @@ -452,7 +452,7 @@ Status RsyncClient::UpdateLocalMeta(std::string& snapshot_uuid, std::setAppend(kUuidPrefix + snapshot_uuid + "\n"); - for (const auto& item : localFileMap) { + for (const auto& item : *localFileMap) { std::string line = item.first + ":" + item.second + "\n"; file->Append(line); } diff --git a/src/rsync_client_thread.cc b/src/rsync_client_thread.cc index 926b3aa05..75e38ce23 100644 --- a/src/rsync_client_thread.cc +++ b/src/rsync_client_thread.cc @@ -40,4 +40,4 @@ RsyncClientThread::RsyncClientThread(int cron_interval, int keepalive_timeout, v conn_factory_(scheduler) {} RsyncClientThread::~RsyncClientThread() {} -} //end namespace rsync \ No newline at end of file +} //end namespace rsync diff --git a/src/rsync_service.proto b/src/rsync_service.proto index a8b96bcbe..68cbc8684 100644 --- a/src/rsync_service.proto +++ b/src/rsync_service.proto @@ -45,4 +45,4 @@ message RsyncResponse { required StatusCode code = 5; optional MetaResponse meta_resp = 6; optional FileResponse file_resp = 7; -} \ No newline at end of file +} diff --git a/src/throttle.cc b/src/throttle.cc index 0346a6227..a97460fa6 100644 --- a/src/throttle.cc +++ b/src/throttle.cc @@ -57,4 +57,4 @@ void Throttle::ReturnUnusedThroughput(size_t acquired, size_t consumed, size_t e } cur_throughput_bytes_ = std::max(cur_throughput_bytes_ - (acquired - consumed), size_t(0)); } -} \ No newline at end of file +} From 17095de2378c4e6c6f6b4809d01c65f2542a128a Mon Sep 17 00:00:00 2001 From: wangshao1 <30471730+wangshao1@users.noreply.github.com> Date: Wed, 26 Jul 2023 16:53:34 +0800 Subject: [PATCH 34/37] fix by review comments (#214) * fix by review comments Co-authored-by: wangshaoyi --- include/rsync_client.h | 15 ++++++------ include/rsync_client_thread.h | 6 ++--- include/rsync_server.h | 43 ++++++++++++----------------------- include/throttle.h | 3 ++- src/rsync_client.cc | 10 ++++---- src/rsync_server.cc | 11 ++++++--- src/throttle.cc | 1 + 7 files changed, 43 insertions(+), 46 deletions(-) diff --git a/include/rsync_client.h b/include/rsync_client.h index 7e65fa29c..bdb7a5f67 100644 --- a/include/rsync_client.h +++ b/include/rsync_client.h @@ -5,6 +5,7 @@ #ifndef RSYNC_CLIENT_H_ #define RSYNC_CLIENT_H_ + #include #include #include @@ -72,15 +73,15 @@ class RsyncClient : public net::Thread { private: std::map meta_table_; - int flush_period_; + int flush_period_ = 10; std::set file_set_; std::string snapshot_uuid_; std::string dir_; std::string db_name_; - uint32_t slot_id_; + uint32_t slot_id_ = 0; std::unique_ptr client_thread_; std::atomic state_; - int max_retries_; + int max_retries_ = 10; std::unique_ptr wo_; std::condition_variable cond_; std::mutex mu_; @@ -126,7 +127,7 @@ class RsyncWriter { private: std::string filepath_; - int fd_; + int fd_ = -1; }; class WaitObject { @@ -147,10 +148,10 @@ class WaitObject { } std::string filename_; RsyncService::Type type_; - size_t offset_; - RsyncResponse* resp_; + size_t offset_ = 0xFFFFFFFF; + RsyncResponse* resp_ = nullptr; }; } // end namespace rsync - #endif + diff --git a/include/rsync_client_thread.h b/include/rsync_client_thread.h index e9784323e..91d5cf383 100644 --- a/include/rsync_client_thread.h +++ b/include/rsync_client_thread.h @@ -25,7 +25,7 @@ class RsyncClientConn : public PbConn { int DealMessage() override; private: - void* cb_handler_; + void* cb_handler_ = nullptr; }; class RsyncClientConnFactory : public ConnFactory { @@ -38,7 +38,7 @@ class RsyncClientConnFactory : public ConnFactory { std::make_shared(connfd, ip_port, thread, cb_handler_, net)); } private: - void* cb_handler_; + void* cb_handler_ = nullptr; }; class RsyncClientThread : public ClientThread { @@ -51,5 +51,5 @@ class RsyncClientThread : public ClientThread { }; } //end namespace rsync +#endif -#endif \ No newline at end of file diff --git a/include/rsync_server.h b/include/rsync_server.h index a8477f414..eef55fc66 100644 --- a/include/rsync_server.h +++ b/include/rsync_server.h @@ -5,10 +5,9 @@ #ifndef RSYNC_SERVER_H_ #define RSYNC_SERVER_H_ + #include #include -#include -#include #include "net/include/net_conn.h" #include "net/include/net_thread.h" @@ -21,12 +20,7 @@ #include "pstd_hash.h" #include "rsync_service.pb.h" -using namespace net; -using namespace RsyncService; -using namespace pstd; - namespace rsync { - struct RsyncServerTaskArg { std::shared_ptr req; std::shared_ptr conn; @@ -44,43 +38,43 @@ class RsyncServer { int Start(); int Stop(); private: - std::unique_ptr work_thread_ = nullptr; - std::unique_ptr rsync_server_thread_ = nullptr; + std::unique_ptr work_thread_; + std::unique_ptr rsync_server_thread_; }; -class RsyncServerConn : public PbConn { +class RsyncServerConn : public net::PbConn { public: RsyncServerConn(int connfd, const std::string& ip_port, - Thread* thread, void* worker_specific_data, - NetMultiplexer* mpx); + net::Thread* thread, void* worker_specific_data, + net::NetMultiplexer* mpx); virtual ~RsyncServerConn() override; int DealMessage() override; static void HandleMetaRsyncRequest(void* arg); static void HandleFileRsyncRequest(void* arg); private: - void* data_; + void* data_ = nullptr; }; -class RsyncServerThread : public HolyThread { +class RsyncServerThread : public net::HolyThread { public: RsyncServerThread(const std::set& ips, int port, int cron_internal, RsyncServer* arg); ~RsyncServerThread(); private: - class RsyncServerConnFactory : public ConnFactory { + class RsyncServerConnFactory : public net::ConnFactory { public: explicit RsyncServerConnFactory(RsyncServer* sched) : scheduler_(sched) {} - std::shared_ptr NewNetConn(int connfd, const std::string& ip_port, - Thread* thread, void* worker_specific_data, - NetMultiplexer* net) const override { + std::shared_ptr NewNetConn(int connfd, const std::string& ip_port, + net::Thread* thread, void* worker_specific_data, + net::NetMultiplexer* net) const override { return std::static_pointer_cast( std::make_shared(connfd, ip_port, thread, scheduler_, net)); } private: - RsyncServer* scheduler_; + RsyncServer* scheduler_ = nullptr; }; - class RsyncServerHandle : public ServerHandle { + class RsyncServerHandle : public net::ServerHandle { public: void FdClosedHandle(int fd, const std::string& ip_port) const override; void FdTimeoutHandle(int fd, const std::string& ip_port) const override; @@ -92,13 +86,6 @@ class RsyncServerThread : public HolyThread { RsyncServerHandle handle_; }; -class RsyncServerConnFactory : public ConnFactory { -public: - virtual std::shared_ptr NewNetConn(int connfd, const std::string& ip_port, Thread* thread, - void* worker_specific_data, - NetMultiplexer* net_epoll) const override; -}; - } //end namespace rsync - #endif + diff --git a/include/throttle.h b/include/throttle.h index f1242d5c4..d80f43aac 100644 --- a/include/throttle.h +++ b/include/throttle.h @@ -5,6 +5,7 @@ #ifndef THROTTLE_H_ #define THROTTLE_H_ + #include #include "pstd/include/pstd_mutex.h" @@ -31,5 +32,5 @@ class Throttle { } }; } // end namespace rsync - #endif + diff --git a/src/rsync_client.cc b/src/rsync_client.cc index 69a9d8659..dc74f5204 100644 --- a/src/rsync_client.cc +++ b/src/rsync_client.cc @@ -3,12 +3,13 @@ // LICENSE file in the root directory of this source tree. An additional grant // of patent rights can be found in the PATENTS file in the same directory. -#include "include/rsync_client.h" #include -#include "include/pika_server.h" -#include "pstd/include/pstd_defer.h" -#include "pstd/src/env.cc" +#include + #include "rocksdb/env.h" +#include "pstd/include/pstd_defer.h" +#include "include/pika_server.h" +#include "include/rsync_client.h" using namespace net; using namespace pstd; @@ -470,3 +471,4 @@ std::string RsyncClient::GetLocalMetaFilePath() { } } // end namespace rsync + diff --git a/src/rsync_server.cc b/src/rsync_server.cc index ef0d39c0e..55af5f255 100644 --- a/src/rsync_server.cc +++ b/src/rsync_server.cc @@ -16,6 +16,10 @@ extern PikaServer* g_pika_server; namespace rsync { +using namespace net; +using namespace RsyncService; +using namespace pstd; + //TODO: optimzie file read and calculate checksum, maybe use RsyncReader prefeching file content Status ReadDumpFile(const std::string filepath, const size_t offset, const size_t count, char* data, size_t* bytes_read, std::string* checksum) { @@ -173,7 +177,7 @@ void RsyncServerConn::HandleMetaRsyncRequest(void* arg) { response.set_snapshot_uuid(snapshot_uuid); LOG(INFO) << "Rsync Meta request, snapshot_uuid: " << snapshot_uuid - << "files count: " << filenames.size() << "file list: "; + << " files count: " << filenames.size() << " file list: "; std::for_each(filenames.begin(), filenames.end(), [](auto& file) { LOG(INFO) << "rsync snapshot file: " << file; }); @@ -214,8 +218,8 @@ void RsyncServerConn::HandleFileRsyncRequest(void* arg) { std::shared_ptr slot = g_pika_server->GetDBSlotById(db_name, slot_id); if (!slot) { - LOG(WARNING) << "cannot find slot for db_name " << db_name - << "slot_id: " << slot_id; + LOG(WARNING) << "cannot find slot for db_name: " << db_name + << " slot_id: " << slot_id; response.set_code(RsyncService::kErr); RsyncWriteResp(response, conn); } @@ -268,3 +272,4 @@ void RsyncServerThread::RsyncServerHandle::CronHandle() const { } } // end namespace rsync + diff --git a/src/throttle.cc b/src/throttle.cc index a97460fa6..39f93d025 100644 --- a/src/throttle.cc +++ b/src/throttle.cc @@ -58,3 +58,4 @@ void Throttle::ReturnUnusedThroughput(size_t acquired, size_t consumed, size_t e cur_throughput_bytes_ = std::max(cur_throughput_bytes_ - (acquired - consumed), size_t(0)); } } + From 2090367fc721057fef1e09386ea30db3b25f7ab8 Mon Sep 17 00:00:00 2001 From: wangshao1 <30471730+wangshao1@users.noreply.github.com> Date: Wed, 26 Jul 2023 17:53:59 +0800 Subject: [PATCH 35/37] fix by review comments (#216) * fix by review comments --------- Co-authored-by: wangshaoyi --- src/rsync_client_thread.cc | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/rsync_client_thread.cc b/src/rsync_client_thread.cc index 75e38ce23..e9ddb76a5 100644 --- a/src/rsync_client_thread.cc +++ b/src/rsync_client_thread.cc @@ -26,9 +26,10 @@ int RsyncClientConn::DealMessage() { decoder.SetTotalBytesLimit(PIKA_MAX_CONN_RBUF); bool success = response->ParseFromCodedStream(&decoder) && decoder.ConsumedEntireMessage(); if (!success) { - LOG(WARNING) << "ParseFromArray FAILED! " - << " msg_len: " << header_len_; - return -1; + delete response; + LOG(WARNING) << "ParseFromArray FAILED! " + << " msg_len: " << header_len_; + return -1; } RsyncClient* handler = (RsyncClient*)cb_handler_; handler->OnReceive(response); From 2be71f41e8f4db2bd1965c06c6a055502a7e0daa Mon Sep 17 00:00:00 2001 From: wangshao1 <30471730+wangshao1@users.noreply.github.com> Date: Wed, 26 Jul 2023 18:32:32 +0800 Subject: [PATCH 36/37] Optimize rsync wangsy (#217) * fix by review comments Co-authored-by: wangshaoyi --- include/pika_define.h | 3 +-- include/rsync_client.h | 19 +++++++------------ src/pika_server.cc | 6 ++---- src/pika_slot.cc | 7 ++----- src/rsync_client.cc | 1 - src/rsync_client_thread.cc | 1 + src/rsync_service.proto | 1 + 7 files changed, 14 insertions(+), 24 deletions(-) diff --git a/include/pika_define.h b/include/pika_define.h index 149675c49..ac0eb46c4 100644 --- a/include/pika_define.h +++ b/include/pika_define.h @@ -34,9 +34,8 @@ class PikaServer; /* Port shift */ const int kPortShiftRSync = 1000; const int kPortShiftReplServer = 2000; -// todo 待移除,使用 kPortShiftRSync +//TODO: Temporarily used for rsync server port shift const int kPortShiftRsync2 = 10001; - const std::string kPikaPidFile = "pika.pid"; const std::string kPikaSecretFile = "rsync.secret"; const std::string kDefaultRsyncAuth = "default"; diff --git a/include/rsync_client.h b/include/rsync_client.h index bdb7a5f67..45d9894e3 100644 --- a/include/rsync_client.h +++ b/include/rsync_client.h @@ -16,21 +16,16 @@ #include #include -#include "include/rsync_client_thread.h" #include "net/include/bg_thread.h" -#include "pstd/include/pstd_status.h" -#include "include/rsync_client_thread.h" #include "net/include/net_cli.h" #include "pstd/include/env.h" +#include "pstd/include/pstd_status.h" #include "pstd/include/pstd_hash.h" #include "pstd/include/pstd_string.h" #include "pstd/include/pstd_status.h" -#include "rsync_service.pb.h" +#include "include/rsync_client_thread.h" #include "include/throttle.h" - -using namespace pstd; -using namespace net; -using namespace RsyncService; +#include "rsync_service.pb.h" const std::string kDumpMetaFileName = "DUMP_META_DATA"; const std::string kUuidPrefix = "snapshot-uuid:"; @@ -57,11 +52,11 @@ class RsyncClient : public net::Thread { return state_.load() == RUNNING; } bool IsIdle() { return state_.load() == IDLE;} - void OnReceive(RsyncResponse* resp); + void OnReceive(RsyncService::RsyncResponse* resp); private: bool Recover(); - Status Wait(RsyncResponse*& resp); + Status Wait(RsyncService::RsyncResponse*& resp); Status CopyRemoteFile(const std::string& filename); Status CopyRemoteMeta(std::string* snapshot_uuid, std::set* file_set); Status LoadLocalMeta(std::string* snapshot_uuid, std::map* file_map); @@ -69,7 +64,7 @@ class RsyncClient : public net::Thread { Status FlushMetaTable(); Status CleanUpExpiredFiles(bool need_reset_path, const std::set& files); Status UpdateLocalMeta(const std::string& snapshot_uuid, const std::set& expired_files, std::map* localFileMap); - void HandleRsyncMetaResponse(RsyncResponse* response); + void HandleRsyncMetaResponse(RsyncService::RsyncResponse* response); private: std::map meta_table_; @@ -149,7 +144,7 @@ class WaitObject { std::string filename_; RsyncService::Type type_; size_t offset_ = 0xFFFFFFFF; - RsyncResponse* resp_ = nullptr; + RsyncService::RsyncResponse* resp_ = nullptr; }; } // end namespace rsync diff --git a/src/pika_server.cc b/src/pika_server.cc index ac1c9ffcf..0226313a0 100644 --- a/src/pika_server.cc +++ b/src/pika_server.cc @@ -3,8 +3,6 @@ // LICENSE file in the root directory of this source tree. An additional grant // of patent rights can be found in the PATENTS file in the same directory. -#include "include/pika_server.h" - #include #include #include @@ -25,10 +23,10 @@ #include "include/pika_cmd_table_manager.h" #include "include/pika_dispatch_thread.h" -#include "include/pika_rm.h" -#include "pstd_hash.h" #include "include/pika_monotonic_time.h" #include "include/pika_instant.h" +#include "include/pika_server.h" +#include "include/pika_rm.h" using pstd::Status; extern PikaServer* g_pika_server; diff --git a/src/pika_slot.cc b/src/pika_slot.cc index b6101c00d..0e6c0e0c1 100644 --- a/src/pika_slot.cc +++ b/src/pika_slot.cc @@ -3,17 +3,16 @@ // LICENSE file in the root directory of this source tree. An additional grant // of patent rights can be found in the PATENTS file in the same directory. -#include "include/pika_slot.h" - #include #include #include "include/pika_conf.h" #include "include/pika_rm.h" #include "include/pika_server.h" +#include "include/pika_slot.h" #include "pstd/include/mutex_impl.h" -#include "pstd_hash.h" +#include "pstd/include/pstd_hash.h" using pstd::Status; @@ -143,7 +142,6 @@ void Slot::PrepareRsync() { // 3, Update master offset, and the PikaAuxiliaryThread cron will connect and do slaveof task with master bool Slot::TryUpdateMasterOffset() { std::string info_path = dbsync_path_ + kBgsaveInfoFile; - // todo 这里要改动,定期向 master 发送 meta_rsync 的请求 if (!pstd::FileExists(info_path)) { LOG(WARNING) << "info path: " << info_path << " not exist"; return false; @@ -331,7 +329,6 @@ Status Slot::GetBgSaveUUID(std::string* snapshot_uuid) { if (snapshot_uuid_.empty()) { std::string info_data; const std::string infoPath = bgsave_info().path + "/info"; - // todo 这里待替换 rocksdb::Status s = rocksdb::ReadFileToString(rocksdb::Env::Default(), infoPath, &info_data); if (!s.ok()) { LOG(WARNING) << "read dump meta info failed! error:" << s.ToString(); diff --git a/src/rsync_client.cc b/src/rsync_client.cc index dc74f5204..0dd886320 100644 --- a/src/rsync_client.cc +++ b/src/rsync_client.cc @@ -14,7 +14,6 @@ using namespace net; using namespace pstd; using namespace RsyncService; -using namespace pstd; extern PikaServer* g_pika_server; diff --git a/src/rsync_client_thread.cc b/src/rsync_client_thread.cc index e9ddb76a5..1672f906f 100644 --- a/src/rsync_client_thread.cc +++ b/src/rsync_client_thread.cc @@ -42,3 +42,4 @@ RsyncClientThread::RsyncClientThread(int cron_interval, int keepalive_timeout, v RsyncClientThread::~RsyncClientThread() {} } //end namespace rsync + diff --git a/src/rsync_service.proto b/src/rsync_service.proto index 68cbc8684..73f6005bd 100644 --- a/src/rsync_service.proto +++ b/src/rsync_service.proto @@ -46,3 +46,4 @@ message RsyncResponse { optional MetaResponse meta_resp = 6; optional FileResponse file_resp = 7; } + From 01b6a22a58bdf169135f0b67ed18f2765c2327dc Mon Sep 17 00:00:00 2001 From: wangshao1 <30471730+wangshao1@users.noreply.github.com> Date: Wed, 26 Jul 2023 20:04:17 +0800 Subject: [PATCH 37/37] fix by review comments (#218) * fix by review comments --------- Co-authored-by: wangshaoyi --- include/pika_define.h | 2 +- src/pika_slot.cc | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/include/pika_define.h b/include/pika_define.h index ac0eb46c4..fb3a1fed7 100644 --- a/include/pika_define.h +++ b/include/pika_define.h @@ -34,7 +34,7 @@ class PikaServer; /* Port shift */ const int kPortShiftRSync = 1000; const int kPortShiftReplServer = 2000; -//TODO: Temporarily used for rsync server port shift +//TODO: Temporarily used for rsync server port shift. will be deleted. const int kPortShiftRsync2 = 10001; const std::string kPikaPidFile = "pika.pid"; const std::string kPikaSecretFile = "rsync.secret"; diff --git a/src/pika_slot.cc b/src/pika_slot.cc index 0e6c0e0c1..72206f121 100644 --- a/src/pika_slot.cc +++ b/src/pika_slot.cc @@ -329,6 +329,7 @@ Status Slot::GetBgSaveUUID(std::string* snapshot_uuid) { if (snapshot_uuid_.empty()) { std::string info_data; const std::string infoPath = bgsave_info().path + "/info"; + //TODO: using file read function to replace rocksdb::ReadFileToString rocksdb::Status s = rocksdb::ReadFileToString(rocksdb::Env::Default(), infoPath, &info_data); if (!s.ok()) { LOG(WARNING) << "read dump meta info failed! error:" << s.ToString();