From 103e86e78b57163184b2258973cd5b02e02d3ff8 Mon Sep 17 00:00:00 2001 From: YuCai18 <1512875381@qq.com> Date: Tue, 26 Aug 2025 15:45:53 +0800 Subject: [PATCH 1/5] Added pika_command_collector mechanism --- conf/pika.conf | 19 + include/pika_binlog.h | 4 + include/pika_command_collector.h | 191 ++++++++ include/pika_conf.h | 62 ++- include/pika_consensus.h | 26 +- include/pika_define.h | 20 +- include/pika_rm.h | 34 +- include/pika_server.h | 2 +- include/pika_stable_log.h | 6 + src/net/src/pb_conn.cc | 2 +- src/pika_binlog.cc | 7 + src/pika_client_conn.cc | 2 + src/pika_command_collector.cc | 505 ++++++++++++++++++++ src/pika_conf.cc | 28 +- src/pika_consensus.cc | 350 ++++++++++++-- src/pika_repl_bgworker.cc | 62 ++- src/pika_repl_server.cc | 25 +- src/pika_rm.cc | 384 ++++++++++++--- src/pika_server.cc | 32 +- src/pika_stable_log.cc | 15 + src/storage/src/redis.cc | 1 + tests/integration/clean_start.sh | 54 +++ tests/integration/start_master_and_slave.sh | 264 +++++----- 23 files changed, 1825 insertions(+), 270 deletions(-) create mode 100644 include/pika_command_collector.h create mode 100644 src/pika_command_collector.cc create mode 100644 tests/integration/clean_start.sh diff --git a/conf/pika.conf b/conf/pika.conf index 97d171d419..6486574996 100644 --- a/conf/pika.conf +++ b/conf/pika.conf @@ -153,6 +153,25 @@ replication-num : 0 # The default value of consensus-level is 0, which means this feature is not enabled. consensus-level : 0 +# Batch processing configuration (used by both command collection and consensus mechanism) +# The maximum number of items in a batch (both command collection and consensus) +# Default: 100 +batch-size : 100 + +# Batch processing configuration (used by both command collection and consensus mechanism) +# The maximum waiting batch for (both command collection and consensus) +# Default: 5 +batch-max-wait-time : 5 + +# The timeout in milliseconds for waiting for a batch ACK from a slave. +# Default: 500 +replication-ack-timeout : 500 + +# Enable command batch processing for better performance +# When enabled, write commands will be collected and processed in batches +# Default: no +command-batch-enabled : yes + # The Prefix of dump file's name. # All the files that generated by command "bgsave" will be name with this prefix. dump-prefix : diff --git a/include/pika_binlog.h b/include/pika_binlog.h index 43615ae0b4..778eb08b4f 100644 --- a/include/pika_binlog.h +++ b/include/pika_binlog.h @@ -61,6 +61,10 @@ class Binlog : public pstd::noncopyable { * Set Producer pro_num and pro_offset with lock */ pstd::Status SetProducerStatus(uint32_t pro_num, uint64_t pro_offset, uint32_t term = 0, uint64_t index = 0); + + // Force sync data to disk + pstd::Status Sync(); + // Need to hold Lock(); pstd::Status Truncate(uint32_t pro_num, uint64_t pro_offset, uint64_t index); diff --git a/include/pika_command_collector.h b/include/pika_command_collector.h new file mode 100644 index 0000000000..4e3fabcaca --- /dev/null +++ b/include/pika_command_collector.h @@ -0,0 +1,191 @@ +// Copyright (c) 2015-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 PIKA_COMMAND_COLLECTOR_H_ +#define PIKA_COMMAND_COLLECTOR_H_ + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include "include/pika_command.h" +#include "include/pika_define.h" +#include "pstd/include/pstd_status.h" + +#include "include/pika_consensus.h" + +/** + * @brief PikaCommandCollector is used to collect write commands and process them in batches + * + * Main functions: + * 1. Collect write commands and process them in optimized batches after reaching the threshold + * 2. Handle the conflict of the same key (the later command will overwrite the earlier command) + * 3. Send commands in batches to the consensus coordinator with batch-level synchronization + * 4. Support asynchronous callback notification of command processing results + * 5. Track performance metrics for batch processing + * 6. Provide intelligent retry mechanisms for failed batches +*/ +class PikaCommandCollector { + public: + // Callback function type after command processing is completed + using CommandCallback = std::function; + + /** + * @brief constructor + * @param coordinator consensus coordinator reference + * @param batch_size batch size (number of commands) + * @param batch_max_wait_time forced flush interval (milliseconds) + */ + // Constructor with raw pointer (original) + PikaCommandCollector(ConsensusCoordinator* coordinator, size_t batch_size = 100, int batch_max_wait_time = 5); + + // Constructor with shared_ptr (for compatibility with make_shared calls) + PikaCommandCollector(std::shared_ptr coordinator, size_t batch_size = 100, int batch_max_wait_time = 5); + + ~PikaCommandCollector(); + + /** + * @brief Add command to collector + * @param cmd_ptr command pointer + * @param callback callback function after processing is completed + * @return whether the addition was successful + */ + bool AddCommand(const std::shared_ptr& cmd_ptr, CommandCallback callback); + + /** + * @brief Called periodically by external systems to process batches + * @param force Force processing even if batch is not full or timeout not reached + * @return Number of commands processed + */ + + /** + * @brief Immediately process all currently collected commands + * @return The number of commands processed + */ + size_t FlushCommands(bool force = false); + + + /** + * @brief Get the current number of pending commands + * @return number of commands + */ + size_t PendingCommands() const; + + /** + * @brief Set the batch size + * @param batch_size batch size + */ + void SetBatchSize(size_t batch_size); + + /** + * @brief Set the batch max wait time + * @param batch_max_wait_time maximum wait time in milliseconds + */ + void SetBatchMaxWaitTime(int batch_max_wait_time); + + /** + * @brief Get batch processing statistics + * @return Pair of (total_processed_commands, total_batches) + */ + std::pair GetBatchStats() const; + + /** + * @brief Get average batch processing time in milliseconds + * @return Average processing time or nullopt if no batches processed + */ + std::optional GetAverageBatchTime() const; + + private: + + /** + * @brief batch processing command + * @param batch command batch + * @return Whether the processing is successful + */ + pstd::Status ProcessBatch(const std::vector>& commands, + const std::vector& callbacks); + + /** + * @brief Check for conflicts based on command type and key name + * @param cmd_ptr command pointer + * @return true if there is a conflict (should be replaced), false if there is no conflict + */ + bool CheckConflict(const std::shared_ptr& cmd_ptr) const; + + /** + * @brief Handle key conflicts and remove conflicting commands + * @param cmd_ptr new command + */ + void HandleConflict(const std::shared_ptr& cmd_ptr); + + /** + * @brief Retry batch processing commands + * @param commands List of commands to retry + * @param callbacks Corresponding callback function list + * @param priority Priority level for the retry (higher means more urgent) + * @return Whether the commands were successfully requeued + */ + bool RetryBatch(const std::vector>& commands, + const std::vector& callbacks, + int priority = 100); + + private: + //Consensus coordinator reference + ConsensusCoordinator* coordinator_; + + // Batch processing configuration + std::atomic batch_size_; + std::atomic batch_max_wait_time_; + + // Retry configuration + std::atomic max_retry_attempts_{3}; + std::atomic retry_backoff_ms_{50}; + + // Command collection and processing + mutable std::mutex mutex_; + + // Pending command queue and callbacks + std::list, CommandCallback>> pending_commands_; + + // Priority queue for retries + std::deque>, std::vector>> retry_queue_; + + // Command key mapping, used to handle same-key conflicts + std::unordered_map, CommandCallback>>::iterator> key_map_; + + // Batch statistics + std::atomic total_processed_{0}; + std::atomic total_batches_{0}; + std::atomic total_retries_{0}; + std::atomic total_conflicts_{0}; + std::atomic total_batch_time_ms_{0}; + std::chrono::time_point batch_start_time_; + + // Performance tracking + struct BatchMetrics { + uint64_t batch_size; + uint64_t processing_time_ms; + uint64_t wait_time_ms; + bool successful; + }; + + // Circular buffer for recent batch metrics + static constexpr size_t kMetricsBufferSize = 100; + std::vector recent_metrics_; + std::mutex metrics_mutex_; +}; + +#endif // PIKA_COMMAND_COLLECTOR_H_ \ No newline at end of file diff --git a/include/pika_conf.h b/include/pika_conf.h index 80d5abe8f0..d9cd9a91bd 100644 --- a/include/pika_conf.h +++ b/include/pika_conf.h @@ -69,6 +69,21 @@ class PikaConf : public pstd::BaseConf { std::shared_lock l(rwlock_); return sync_thread_num_; } + + bool command_batch_enabled() { + std::shared_lock l(rwlock_); + return command_batch_enabled_; + } + + int batch_size() { + std::shared_lock l(rwlock_); + return batch_size_; + } + + int batch_max_wait_time() { + std::shared_lock l(rwlock_); + return batch_max_wait_time_; + } int sync_binlog_thread_num() { std::shared_lock l(rwlock_); return sync_binlog_thread_num_; @@ -350,6 +365,16 @@ class PikaConf : public pstd::BaseConf { int max_conn_rbuf_size() { return max_conn_rbuf_size_.load(); } int consensus_level() { return consensus_level_.load(); } int replication_num() { return replication_num_.load(); } + int replication_ack_timeout() { + std::shared_lock l(rwlock_); + return replication_ack_timeout_; + } + + // Function to set replication acknowledgment timeout (used by batch system) + void SetReplicationAckTimeout(int timeout) { + std::lock_guard l(rwlock_); + replication_ack_timeout_ = timeout; + } int rate_limiter_mode() { std::shared_lock l(rwlock_); return rate_limiter_mode_; @@ -436,7 +461,6 @@ class PikaConf : public pstd::BaseConf { bool is_admin_cmd(const std::string& cmd) { return admin_cmd_set_.find(cmd) != admin_cmd_set_.end(); } - // Immutable config items, we don't use lock. bool daemonize() { return daemonize_; } bool rtc_cache_read_enabled() { return rtc_cache_read_enabled_; } @@ -462,6 +486,23 @@ class PikaConf : public pstd::BaseConf { std::lock_guard l(rwlock_); thread_num_ = value; } + + void SetCommandBatchEnabled(const bool value) { + std::lock_guard l(rwlock_); + TryPushDiffCommands("command-batch-enabled", value ? "yes" : "no"); + command_batch_enabled_ = value; + } + + void SetCommandBatchSize(const int value) { + std::lock_guard l(rwlock_); + TryPushDiffCommands("batch-size", std::to_string(value)); + batch_size_ = value; + } + void SetCommandBatchMaxWaitTime(const int value) { + std::lock_guard l(rwlock_); + TryPushDiffCommands("batch-max-wait-time", std::to_string(value)); + batch_max_wait_time_ = value; + } void SetTimeout(const int value) { std::lock_guard l(rwlock_); TryPushDiffCommands("timeout", std::to_string(value)); @@ -665,6 +706,17 @@ class PikaConf : public pstd::BaseConf { TryPushDiffCommands("max-conn-rbuf-size", std::to_string(value)); max_conn_rbuf_size_.store(value); } + void SetConsensusBatchSize(const int value) { + std::lock_guard l(rwlock_); + TryPushDiffCommands("batch-size", std::to_string(value)); + batch_size_ = value; + } + // This method is used by config update system + void UpdateReplicationAckTimeout(const int value) { + std::lock_guard l(rwlock_); + TryPushDiffCommands("replication-ack-timeout", std::to_string(value)); + replication_ack_timeout_ = value; + } void SetMaxCacheFiles(const int& value) { std::lock_guard l(rwlock_); TryPushDiffCommands("max-cache-files", std::to_string(value)); @@ -929,6 +981,12 @@ class PikaConf : public pstd::BaseConf { std::string server_id_; std::string run_id_; std::string replication_id_; + + // 命令批处理相关配置 + bool command_batch_enabled_ = true; + int batch_size_ = 100; + int batch_max_wait_time_ = 5; + int replication_ack_timeout_ = 5000; std::string requirepass_; std::string masterauth_; std::string userpass_; @@ -1047,7 +1105,7 @@ class PikaConf : public pstd::BaseConf { int throttle_bytes_per_second_ = 200 << 20; // 200MB/s int max_rsync_parallel_num_ = kMaxRsyncParallelNum; std::atomic_int64_t rsync_timeout_ms_ = 1000; - + /* kUninitialized = 0, // unknown setting kDisable = 1, // disable perf stats diff --git a/include/pika_consensus.h b/include/pika_consensus.h index 78e20eb3ab..41a36d3e4e 100644 --- a/include/pika_consensus.h +++ b/include/pika_consensus.h @@ -155,6 +155,8 @@ class ConsensusCoordinator { pstd::Status Reset(const LogOffset& offset); pstd::Status ProposeLog(const std::shared_ptr& cmd_ptr); + // Batch processing of commands + pstd::Status BatchProposeLog(const std::vector>& cmd_ptrs, std::vector* offsets); pstd::Status UpdateSlave(const std::string& ip, int port, const LogOffset& start, const LogOffset& end); pstd::Status AddSlaveNode(const std::string& ip, int port, int session_id); pstd::Status RemoveSlaveNode(const std::string& ip, int port); @@ -244,6 +246,10 @@ class ConsensusCoordinator { SyncProgress sync_pros_; std::shared_ptr stable_logger_; std::shared_ptr mem_logger_; + + // Make db_name accessible to external classes + public: + const std::string& db_name() const { return db_name_; } // pacificA public: @@ -258,8 +264,12 @@ class ConsensusCoordinator { pstd::Status CommitAppLog(const LogOffset& master_committed_id); pstd::Status UpdateCommittedID(); pstd::Status ApplyBinlog(const std::shared_ptr& cmd_ptr); + pstd::Status BatchApplyBinlogs(const std::vector& logs_to_apply); + void BatchInternalApplyFollower(const std::vector>& cmd_ptrs); pstd::Status ProcessCoordination(); + // Batch operations for slave entries + pstd::Status BatchPersistAppendBinlog(const std::vector>& cmd_ptrs,const std::vector& attributes,std::vector* offsets); LogOffset GetCommittedId() { std::lock_guard l(committed_id_rwlock_); return committed_id_; @@ -273,9 +283,16 @@ class ConsensusCoordinator { prepared_id_ = offset; } void SetCommittedId(const LogOffset& offset) { - std::lock_guard l(committed_id_rwlock_); - committed_id_ = offset; - context_->UpdateAppliedIndex(committed_id_); + { + std::lock_guard l(committed_id_rwlock_); + if (committed_id_ >= offset) { + return; + } + committed_id_ = offset; + context_->UpdateAppliedIndex(committed_id_); + } + notification_counter_.fetch_add(1); + LOG(INFO) << "SetCommittedId: Updated to " << offset.ToString(); } private: @@ -283,9 +300,10 @@ class ConsensusCoordinator { private: std::shared_mutex is_consistency_rwlock_; - bool is_consistency_ = false; + bool is_consistency_ = true; std::shared_mutex committed_id_rwlock_; LogOffset committed_id_ = LogOffset(); + std::atomic notification_counter_{0}; std::shared_mutex prepared_id__rwlock_; LogOffset prepared_id_ = LogOffset(); std::shared_ptr logs_; diff --git a/include/pika_define.h b/include/pika_define.h index c09d0d7c38..8605f2a06b 100644 --- a/include/pika_define.h +++ b/include/pika_define.h @@ -140,7 +140,9 @@ struct LogOffset { bool operator<=(const LogOffset& other) const { return b_offset <= other.b_offset; } bool operator>=(const LogOffset& other) const { return b_offset >= other.b_offset; } bool operator>(const LogOffset& other) const { return b_offset > other.b_offset; } + bool operator!=(const LogOffset& other) const { return b_offset != other.b_offset; } std::string ToString() const { return b_offset.ToString() + " " + l_offset.ToString(); } + bool IsValid() const { return b_offset.filenum > 0 || b_offset.offset > 0; } BinlogOffset b_offset; LogicOffset l_offset; }; @@ -178,10 +180,18 @@ const std::string BinlogSyncStateMsg[] = {"NotSync", "ReadFromCache", "ReadFromF struct BinlogChip { LogOffset offset_; std::string binlog_; - BinlogChip(const LogOffset& offset, std::string binlog) : offset_(offset), binlog_(std::move(binlog)) {} + bool is_batch_ = false; + + BinlogChip(const LogOffset& offset, std::string binlog) + : offset_(offset), binlog_(std::move(binlog)), is_batch_(false) {} + + BinlogChip(const LogOffset& offset, std::string binlog, bool is_batch) + : offset_(offset), binlog_(std::move(binlog)), is_batch_(is_batch) {} + BinlogChip(const BinlogChip& binlog_chip) { offset_ = binlog_chip.offset_; binlog_ = binlog_chip.binlog_; + is_batch_ = binlog_chip.is_batch_; } }; @@ -354,6 +364,14 @@ const int64_t kPoolSize = 1073741824; const std::string kBinlogPrefix = "write2file"; const size_t kBinlogPrefixLen = 10; +/* + * PIKA_BATCH_MAGIC: Core identifier for binlog batch processing. + * - Master: Prefixes batched binlogs with this magic in SendBinlog + * - Slave: Detects this magic in HandleBGWorkerWriteBinlog + * to switch between batch and single-binlog parsing modes. + */ +const uint32_t PIKA_BATCH_MAGIC = 0x42544348; // "BTCH" in ASCII + const std::string kPikaMeta = "meta"; const std::string kManifest = "manifest"; const std::string kContext = "context"; diff --git a/include/pika_rm.h b/include/pika_rm.h index 709d5722cc..e72bad4992 100644 --- a/include/pika_rm.h +++ b/include/pika_rm.h @@ -22,6 +22,7 @@ #include "include/pika_slave_node.h" #include "include/pika_stable_log.h" #include "include/rsync_client.h" +#include "include/pika_command_collector.h" #define kBinlogSendPacketNum 40 #define kBinlogSendBatchNum 100 @@ -69,6 +70,7 @@ class SyncMasterDB : public SyncDB { // consensus use pstd::Status ConsensusUpdateSlave(const std::string& ip, int port, const LogOffset& start, const LogOffset& end); pstd::Status ConsensusProposeLog(const std::shared_ptr& cmd_ptr); + pstd::Status ConsensusBatchProposeLog(const std::vector>& cmd_ptrs, std::vector* offsets); pstd::Status ConsensusProcessLeaderLog(const std::shared_ptr& cmd_ptr, const BinlogItem& attribute); LogOffset ConsensusCommittedIndex(); @@ -83,16 +85,22 @@ class SyncMasterDB : public SyncDB { return coordinator_.StableLogger()->Logger(); } + std::shared_ptr GetSlaveNode(const std::string& ip, int port); + // Make coordinator_ accessible to StableLog class + ConsensusCoordinator& GetCoordinator() { return coordinator_; } + std::shared_ptr GetCommandCollector(); + private: // invoker need to hold slave_mu_ pstd::Status ReadBinlogFileToWq(const std::shared_ptr& slave_ptr); - std::shared_ptr GetSlaveNode(const std::string& ip, int port); + //std::shared_ptr GetSlaveNode(const std::string& ip, int port); std::unordered_map> GetAllSlaveNodes(); pstd::Mutex session_mu_; int32_t session_id_ = 0; ConsensusCoordinator coordinator_; + std::shared_ptr command_collector_; //pacificA public: public: @@ -108,12 +116,12 @@ class SyncMasterDB : public SyncDB { LogOffset GetPreparedId(); LogOffset GetCommittedId(); pstd::Status AppendSlaveEntries(const std::shared_ptr& cmd_ptr, const BinlogItem& attribute); + pstd::Status BatchAppendSlaveEntries(const std::vector>& cmd_ptrs, const std::vector& attributes); pstd::Status AppendCandidateBinlog(const std::string& ip, int port, const LogOffset& offset); pstd::Status UpdateCommittedID(); pstd::Status CommitAppLog(const LogOffset& master_committed_id); pstd::Status Truncate(const LogOffset& offset); - - + pstd::Status WaitForSlaveAcks(const LogOffset& target_offset, int timeout_ms); }; class SyncSlaveDB : public SyncDB { @@ -163,6 +171,7 @@ class PikaReplicaManager { pstd::Status SendMetaSyncRequest(); pstd::Status SendRemoveSlaveNodeRequest(const std::string& table); pstd::Status SendTrySyncRequest(const std::string& db_name); + std::shared_ptr GetConsensusCoordinator(const std::string& db_name); pstd::Status SendDBSyncRequest(const std::string& db_name); pstd::Status SendBinlogSyncAckRequest(const std::string& table, const LogOffset& ack_start, const LogOffset& ack_end, bool is_first_send = false); @@ -243,10 +252,27 @@ class PikaReplicaManager { pstd::Mutex write_queue_mu_; + // db_name -> a queue of write task + using DBWriteTaskQueue = std::map>; + // ip:port -> a map of DBWriteTaskQueue + using SlaveWriteTaskQueue = std::map; + // every host owns a queue, the key is "ip + port" - std::unordered_map>> write_queues_; + SlaveWriteTaskQueue write_queues_; + + // client for replica std::unique_ptr pika_repl_client_; std::unique_ptr pika_repl_server_; + + // one-shot switch to force immediate send on next SendBinlog + std::atomic immediate_send_once_{false}; + + // Condition variable for signaling when the write queue has new items + pstd::CondVar write_queue_cv_; + + std::shared_mutex is_consistency_rwlock_; + bool is_consistency_ = true; + std::shared_mutex committed_id_rwlock_; }; #endif // PIKA_RM_H diff --git a/include/pika_server.h b/include/pika_server.h index 41a8c9b346..f67e569645 100644 --- a/include/pika_server.h +++ b/include/pika_server.h @@ -581,7 +581,7 @@ class PikaServer : public pstd::noncopyable { std::string master_ip_; int master_port_ = 0; int repl_state_ = PIKA_REPL_NO_CONNECT; - bool is_consistency_ = false; + bool is_consistency_ = true; int role_ = PIKA_ROLE_SINGLE; int last_role_ = PIKA_ROLE_SINGLE; int last_meta_sync_timestamp_ = 0; diff --git a/include/pika_stable_log.h b/include/pika_stable_log.h index 300e0d0fc5..8cb6dc6267 100644 --- a/include/pika_stable_log.h +++ b/include/pika_stable_log.h @@ -11,6 +11,9 @@ #include "include/pika_binlog.h" +// Forward declaration to avoid circular dependency +class ConsensusCoordinator; + class StableLog : public std::enable_shared_from_this { public: StableLog(std::string table_name, std::string log_path); @@ -25,6 +28,9 @@ class StableLog : public std::enable_shared_from_this { std::shared_lock l(offset_rwlock_); return first_offset_; } + // Return a direct reference to the ConsensusCoordinator without copying + std::shared_ptr coordinator(); + void set_coordinator(std::shared_ptr coordinator); // Need to hold binlog lock pstd::Status TruncateTo(const LogOffset& offset); diff --git a/src/net/src/pb_conn.cc b/src/net/src/pb_conn.cc index 5185e8f51d..15e4024da0 100644 --- a/src/net/src/pb_conn.cc +++ b/src/net/src/pb_conn.cc @@ -153,7 +153,7 @@ WriteStatus PbConn::SendReply() { if (item_len - write_buf_.item_pos_ != 0) { return kWriteHalf; } - LOG(ERROR) << "write item success"; + //LOG(ERROR) << "write item success"; } return kWriteAll; } diff --git a/src/pika_binlog.cc b/src/pika_binlog.cc index 187d63d8ad..e051108317 100644 --- a/src/pika_binlog.cc +++ b/src/pika_binlog.cc @@ -469,3 +469,10 @@ Status Binlog::Truncate(uint32_t pro_num, uint64_t pro_offset, uint64_t index) { return Status::OK(); } + +Status Binlog::Sync() { + if (queue_) { + return queue_->Sync(); + } + return Status::Corruption("Logger not initialized"); +} \ No newline at end of file diff --git a/src/pika_client_conn.cc b/src/pika_client_conn.cc index a6cd5ec62f..e768cefa6f 100644 --- a/src/pika_client_conn.cc +++ b/src/pika_client_conn.cc @@ -16,9 +16,11 @@ #include "include/pika_define.h" #include "include/pika_rm.h" #include "include/pika_server.h" +#include "include/pika_command_collector.h" #include "net/src/dispatch_thread.h" #include "net/src/worker_thread.h" #include "src/pstd/include/scope_record_lock.h" +#include #include "rocksdb/perf_context.h" #include "rocksdb/iostats_context.h" diff --git a/src/pika_command_collector.cc b/src/pika_command_collector.cc new file mode 100644 index 0000000000..89eec53d22 --- /dev/null +++ b/src/pika_command_collector.cc @@ -0,0 +1,505 @@ +// Copyright (c) 2015-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/pika_command_collector.h" +#include "include/pika_conf.h" +#include "include/pika_server.h" +#include "include/pika_rm.h" +#include "include/pika_define.h" +#include +#include + +extern std::unique_ptr g_pika_conf; +extern std::unique_ptr g_pika_rm; + +PikaCommandCollector::PikaCommandCollector(ConsensusCoordinator* coordinator, size_t batch_size, int batch_max_wait_time) + : coordinator_(coordinator), + batch_size_(batch_size), + batch_max_wait_time_(batch_max_wait_time) { + + // Check if coordinator is null + if (!coordinator_) { + LOG(FATAL) << "PikaCommandCollector: ConsensusCoordinator cannot be null! " + << "This usually means SyncMasterDB is not initialized yet."; + return; + } + + LOG(INFO) << "PikaCommandCollector created with batch_size=" << batch_size << ", batch_max_wait_time=" << batch_max_wait_time << "ms"; + + // Initialize metrics buffer + recent_metrics_.reserve(kMetricsBufferSize); +} + +// Constructor with shared_ptr (for compatibility with make_shared calls) +PikaCommandCollector::PikaCommandCollector(std::shared_ptr coordinator, size_t batch_size, int batch_max_wait_time) + : coordinator_(coordinator.get()), + batch_size_(batch_size), + batch_max_wait_time_(batch_max_wait_time) { + + // Check if coordinator is null + if (!coordinator_) { + LOG(FATAL) << "PikaCommandCollector: ConsensusCoordinator cannot be null! " + << "This usually means SyncMasterDB is not initialized yet."; + return; + } + + LOG(INFO) << "PikaCommandCollector created from shared_ptr with batch_size=" << batch_size << ", batch_max_wait_time=" << batch_max_wait_time << "ms"; + + // Initialize metrics buffer + recent_metrics_.reserve(kMetricsBufferSize); +} + +PikaCommandCollector::~PikaCommandCollector() { + // Process any remaining commands + FlushCommands(true); + LOG(INFO) << "PikaCommandCollector stopped, processed " << total_processed_.load() + << ", conflicts: " << total_conflicts_.load(); +} + +bool PikaCommandCollector::AddCommand(const std::shared_ptr& cmd_ptr, CommandCallback callback) { + if (!cmd_ptr || !cmd_ptr->is_write()) { + LOG(WARNING) << "Attempt to add non-write command to CommandCollector"; + return false; + } + + std::lock_guard lock(mutex_); + + if (pending_commands_.empty()) { + batch_start_time_ = std::chrono::steady_clock::now(); + } + + // Check if we should immediately flush the batch + bool should_flush = pending_commands_.size() >= static_cast(batch_size_.load()); + if (should_flush) { + FlushCommands(false); + } + + // Handle same Key conflict - counts updated inside HandleConflict + HandleConflict(cmd_ptr); + + // Add command to queue + pending_commands_.emplace_back(cmd_ptr, std::move(callback)); + + // Update Key Mapping + std::vector keys = cmd_ptr->current_key(); + for (const auto& key : keys) { + key_map_[key] = std::prev(pending_commands_.end()); + } + + + return true; +} + +size_t PikaCommandCollector::FlushCommands(bool force) { + std::vector> commands; + std::vector callbacks; + + // Record batch metrics variables + auto batch_start = std::chrono::steady_clock::now(); + uint64_t wait_time_ms = 0; + bool batch_successful = false; + + { + std::lock_guard lock(mutex_); + if (pending_commands_.empty()) { + // Check if there are any retries to process + if (!retry_queue_.empty() && force) { + auto& [pri, cmds, cbs] = retry_queue_.front(); + commands = cmds; + callbacks = cbs; + retry_queue_.pop_front(); + LOG(INFO) << "FlushCommands: Processing retry batch with priority " << pri << ", size: " << commands.size(); + } else { + return 0; + } + } else { + auto now = std::chrono::steady_clock::now(); + auto elapsed_ms = std::chrono::duration_cast(now - batch_start_time_).count(); + wait_time_ms = elapsed_ms; // Record wait time for metrics + + bool should_flush = force || + pending_commands_.size() >= static_cast(batch_size_.load()) || + elapsed_ms > batch_max_wait_time_.load(); + + if (!should_flush) { + return 0; + } + + size_t batch_count = pending_commands_.size(); + if (!force) { + batch_count = std::min(batch_count, static_cast(batch_size_.load())); + } + + commands.reserve(batch_count); + callbacks.reserve(batch_count); + + auto it = pending_commands_.begin(); + for (size_t i = 0; i < batch_count; ++i, ++it) { + commands.push_back(it->first); + callbacks.push_back(std::move(it->second)); + } + + // Clear queue and map + for (const auto& cmd : commands) { + std::vector keys = cmd->current_key(); + for (const auto& key : keys) { + key_map_.erase(key); + } + } + pending_commands_.erase(pending_commands_.begin(), std::next(pending_commands_.begin(), batch_count)); + + if (!pending_commands_.empty()) { + // Reset timer for the next batch + batch_start_time_ = std::chrono::steady_clock::now(); + } + } + } + + size_t batch_size = commands.size(); + if (batch_size > 0) { + LOG(INFO) << "Processing batch of " << batch_size << " commands"; + + auto process_start = std::chrono::steady_clock::now(); + pstd::Status status = ProcessBatch(commands, callbacks); + auto process_end = std::chrono::steady_clock::now(); + + uint64_t processing_time_ms = std::chrono::duration_cast(process_end - process_start).count(); + batch_successful = status.ok(); + + if (!batch_successful) { + LOG(ERROR) << "Error processing command batch: " << status.ToString(); + } else { + LOG(INFO) << "Successfully processed batch in " << processing_time_ms << "ms"; + } + + // Update statistics + total_processed_.fetch_add(batch_size); + total_batches_.fetch_add(1); + total_batch_time_ms_.fetch_add(processing_time_ms); + + // Record batch metrics + { + std::lock_guard metrics_lock(metrics_mutex_); + if (recent_metrics_.size() >= kMetricsBufferSize) { + recent_metrics_.erase(recent_metrics_.begin()); + } + recent_metrics_.push_back({batch_size, processing_time_ms, wait_time_ms, batch_successful}); + } + + // Process any retries if there were failures but we have pending retries + if (!batch_successful) { + std::lock_guard lock(mutex_); + if (!retry_queue_.empty()) { + LOG(INFO) << "FlushCommands: Processing retries due to batch failure"; + // Schedule immediate follow-up flush to process retries + return batch_size + FlushCommands(true); + } + } + } + + return batch_size; +} + +pstd::Status PikaCommandCollector::ProcessBatch( + const std::vector>& commands, + const std::vector& callbacks) { + + if (commands.empty()) { + return pstd::Status::OK(); + } + + // Implement batch processing logic here + // 1. Generate binlogs for each command + // 2. Write binlogs to production queue in batches + // 3. Main node will update memory data structures in batches + // 4. Trigger asynchronous persistence + + // Store the log offset for each command + std::vector offsets; + + // Check if coordinator is valid + if (!coordinator_) { + LOG(ERROR) << "ProcessBatch: ConsensusCoordinator is null"; + return pstd::Status::InvalidArgument("ConsensusCoordinator is null"); + } + + // Get SyncMasterDB and submit commands in batch + DBInfo db_info(coordinator_->db_name()); + auto master_db = g_pika_rm->GetSyncMasterDBByName(db_info); + if (!master_db) { + LOG(ERROR) << "Failed to get SyncMasterDB for " << coordinator_->db_name(); + return pstd::Status::NotFound("SyncMasterDB not found"); + } + + // Submit to consensus coordinator in batch + LOG(INFO) << "ProcessBatch: Processing " << commands.size() << " commands in batch"; + pstd::Status batch_status = master_db->ConsensusBatchProposeLog(commands, &offsets); + + // Log the batch status + if (!batch_status.ok()) { + LOG(WARNING) << "ProcessBatch: Batch operation failed with status: " << batch_status.ToString(); + if (batch_status.IsTimeout()) { + LOG(WARNING) << "ProcessBatch: Timeout occurred, triggering batch retry mechanism"; + + // Get the last command's offset + LogOffset last_offset; + if (!offsets.empty()) { + last_offset = offsets.back(); + } + + // Roll back committed_id on master and all slave nodes + if (last_offset.IsValid()) { + LOG(WARNING) << "ProcessBatch: Rolling back committed_id to before " << last_offset.ToString(); + + // Get current committed_id + LogOffset current_committed_id = master_db->GetCommittedId(); + + // Calculate rollback target committed_id (assuming rollback to the previous batch's committed_id) + // In actual implementation, you may need to adjust the rollback target based on specific situations + LogOffset rollback_target = current_committed_id; + rollback_target.l_offset.index -= commands.size(); // Simple rollback, may need more complex logic in practice + if (rollback_target.l_offset.index < 0) { + rollback_target.l_offset.index = 0; + } + + // Execute rollback operation + LOG(WARNING) << "ProcessBatch: Rolling back from " << current_committed_id.ToString() + << " to " << rollback_target.ToString(); + + // Call rollback API + pstd::Status truncate_status = master_db->Truncate(rollback_target); + if (!truncate_status.ok()) { + LOG(ERROR) << "ProcessBatch: Failed to rollback committed_id: " << truncate_status.ToString(); + } else { + LOG(INFO) << "ProcessBatch: Successfully rolled back committed_id"; + + // Call batch retry mechanism + LOG(INFO) << "ProcessBatch: Triggering batch retry mechanism for " << commands.size() << " commands"; + bool retry_result = RetryBatch(commands, callbacks, 100); // Use high priority for timeouts + if (retry_result) { + LOG(INFO) << "ProcessBatch: Successfully requeued commands for retry"; + // Already requeued, no need to execute callbacks + return batch_status; + } else { + LOG(ERROR) << "ProcessBatch: Failed to requeue commands for retry"; + } + } + } + } + } else { + LOG(INFO) << "ProcessBatch: Batch operation completed successfully"; + } + + // Execute callback for each command + for (size_t i = 0; i < commands.size() && i < callbacks.size(); ++i) { + if (callbacks[i]) { + pstd::Status cmd_status; + // If batch processing status failed, all commands should fail + if (!batch_status.ok()) { + // Pass the upper layer error status to the client + cmd_status = batch_status; + } else { + // If the offset is empty, it means the command failed to be added + cmd_status = offsets[i].IsValid() ? pstd::Status::OK() : pstd::Status::IOError("Failed to append command"); + } + + // Log information before executing each callback function + LOG(INFO) << "ProcessBatch: Executing callback for command " << i + << ", cmd=" << (commands[i] ? commands[i]->name() : "null") + << ", status=" << cmd_status.ToString() + << ", offset=" << (offsets[i].IsValid() ? offsets[i].ToString() : "invalid"); + // Execute callback + callbacks[i](offsets[i], cmd_status); + + // Log information after callback execution + LOG(INFO) << "ProcessBatch: Callback executed for command " << i; + } + } + return batch_status; +} + +bool PikaCommandCollector::CheckConflict(const std::shared_ptr& cmd_ptr) const { + if (!cmd_ptr) { + return false; + } + + std::vector keys = cmd_ptr->current_key(); + for (const auto& key : keys) { + if (key_map_.find(key) != key_map_.end()) { + return true; + } + } + + return false; +} + +void PikaCommandCollector::HandleConflict(const std::shared_ptr& cmd_ptr) { + if (!cmd_ptr) { + return; + } + + std::vector keys = cmd_ptr->current_key(); + std::vector, CommandCallback>>::iterator> to_remove; + + // Find all conflicting commands + for (const auto& key : keys) { + auto it = key_map_.find(key); + if (it != key_map_.end()) { + // Check if this iterator is already added + bool already_added = false; + for (const auto& iter : to_remove) { + if (iter == it->second) { + already_added = true; + break; + } + } + if (!already_added) { + to_remove.push_back(it->second); + } + key_map_.erase(it); + } + } + + // Track conflict metrics + if (!to_remove.empty()) { + total_conflicts_.fetch_add(to_remove.size()); + } + + // Check command importance to prevent important commands from being overwritten + for (auto it : to_remove) { + auto old_cmd = it->first; + auto new_cmd = cmd_ptr; + + // Determine command importance + // If the old command type exists in the important command list, consider it important + bool is_important_cmd = false; + // Check commands with keywords such as "EXEC", "MULTI", "WATCH", etc. + std::string cmd_name = old_cmd->name(); + if (cmd_name == "MULTI" || cmd_name == "EXEC" || cmd_name == "WATCH") { + is_important_cmd = true; + } + + if (is_important_cmd) { + // 对于重要命令,我们保留原来的命令,并拒绝新命令 + // 恢复已移除的key映射 + std::vector old_keys = old_cmd->current_key(); + for (const auto& key : old_keys) { + key_map_[key] = it; + } + + // 从待删除列表中移除该命令 + for (auto iter = to_remove.begin(); iter != to_remove.end(); ++iter) { + if (*iter == it) { + to_remove.erase(iter); + break; + } + } + + } + } + + // 删除非重要的冲突命令 + for (auto it : to_remove) { + // 执行回调通知命令被取消 + if (it->second) { + it->second(LogOffset(), pstd::Status::Busy("Command replaced by newer command with same key")); + } + + // 从队列中移除 + pending_commands_.erase(it); + } +} + + + + + +void PikaCommandCollector::SetBatchSize(size_t batch_size) { + batch_size_.store(batch_size); + LOG(INFO) << "BatchSize set to " << batch_size; +} + +void PikaCommandCollector::SetBatchMaxWaitTime(int batch_max_wait_time) { + batch_max_wait_time_.store(batch_max_wait_time); + LOG(INFO) << "BatchMaxWaitTime set to " << batch_max_wait_time << "ms"; +} + +std::pair PikaCommandCollector::GetBatchStats() const { + return {total_processed_.load(), total_batches_.load()}; +} + +std::optional PikaCommandCollector::GetAverageBatchTime() const { + uint64_t total_batches = total_batches_.load(); + if (total_batches == 0) { + return std::nullopt; + } + return static_cast(total_batch_time_ms_.load()) / total_batches; +} + +size_t PikaCommandCollector::PendingCommands() const { + std::lock_guard lock(mutex_); + return pending_commands_.size(); +} + + + +bool PikaCommandCollector::RetryBatch( + const std::vector>& commands, + const std::vector& callbacks, + int priority) { + + if (commands.empty() || callbacks.empty() || commands.size() != callbacks.size()) { + LOG(WARNING) << "RetryBatch: Invalid input parameters, commands size: " << commands.size() + << ", callbacks size: " << callbacks.size(); + return false; + } + + std::lock_guard lock(mutex_); + + LOG(INFO) << "RetryBatch: Retrying " << commands.size() << " commands with priority " << priority; + + // Add to retry queue with priority + retry_queue_.emplace_front(std::make_tuple(priority, commands, callbacks)); + + // Update statistics + total_retries_.fetch_add(1); + + // Sort retry queue by priority (higher values first) + std::sort(retry_queue_.begin(), retry_queue_.end(), + [](const auto& a, const auto& b) { return std::get<0>(a) > std::get<0>(b); }); + + // Process highest priority retry immediately if possible + if (priority > 50 && pending_commands_.empty()) { + // Process retry queue directly + auto& [pri, cmds, cbs] = retry_queue_.front(); + + // Add each command to the queue + for (size_t i = 0; i < cmds.size(); ++i) { + if (cmds[i]) { + // Handle key conflicts + HandleConflict(cmds[i]); + + // Add to queue front for priority processing + pending_commands_.push_front(std::make_pair(cmds[i], cbs[i])); + + // Update key mapping + std::vector keys = cmds[i]->current_key(); + for (const auto& key : keys) { + key_map_[key] = pending_commands_.begin(); + } + + LOG(INFO) << "RetryBatch: Immediately requeued command " << i << ": " << cmds[i]->name(); + } + } + + // Remove from retry queue + retry_queue_.pop_front(); + + // Trigger immediate flush + FlushCommands(true); + } + + return true; +} \ No newline at end of file diff --git a/src/pika_conf.cc b/src/pika_conf.cc index 94071eac7f..a4a7178cb7 100644 --- a/src/pika_conf.cc +++ b/src/pika_conf.cc @@ -412,6 +412,20 @@ int PikaConf::Load() { max_cache_statistic_keys_ = 0; } + // 命令批处理相关配置 + std::string command_batch_enabled; + GetConfStr("command-batch-enabled", &command_batch_enabled); + command_batch_enabled_ = (command_batch_enabled == "yes"); + + GetConfInt("batch-size", &batch_size_); + if (batch_size_ <= 0) { + batch_size_ = 100; + } + + GetConfInt("batch_max_wait_time", &batch_max_wait_time_); + if (batch_max_wait_time_ <= 0) { + batch_max_wait_time_ = 5; + } // disable_auto_compactions GetConfBool("disable_auto_compactions", &disable_auto_compactions_); @@ -707,9 +721,13 @@ int PikaConf::Load() { rsync_timeout_ms_.store(tmp_rsync_timeout_ms); } - return ret; -} - + GetConfInt("replication-ack-timeout", &replication_ack_timeout_); + if (replication_ack_timeout_ <= 0) { + replication_ack_timeout_ = 5000; + } + return ret; + } + void PikaConf::TryPushDiffCommands(const std::string& command, const std::string& value) { if (!CheckConfExist(command)) { diff_commands_[command] = value; @@ -770,6 +788,9 @@ int PikaConf::ConfigRewrite() { SetConfStr("run-id", run_id_); SetConfStr("replication-id", replication_id_); SetConfInt("max-cache-statistic-keys", max_cache_statistic_keys_); + SetConfStr("command-batch-enabled", command_batch_enabled_ ? "yes" : "no"); + SetConfInt("batch-size", batch_size_); + SetConfInt("batch-max-wait-time", batch_max_wait_time_); SetConfInt("small-compaction-threshold", small_compaction_threshold_); SetConfInt("small-compaction-duration-threshold", small_compaction_duration_threshold_); SetConfInt("max-client-response-size", static_cast(max_client_response_size_)); @@ -790,6 +811,7 @@ int PikaConf::ConfigRewrite() { SetConfInt("replication-num", replication_num_.load()); SetConfStr("slow-cmd-list", pstd::Set2String(slow_cmd_set_, ',')); SetConfInt("max-conn-rbuf-size", max_conn_rbuf_size_.load()); + SetConfInt("replication-ack-timeout", replication_ack_timeout_); // options for storage engine SetConfInt("max-cache-files", max_cache_files_); SetConfInt("max-background-compactions", max_background_compactions_); diff --git a/src/pika_consensus.cc b/src/pika_consensus.cc index ef1960d589..00490c3162 100644 --- a/src/pika_consensus.cc +++ b/src/pika_consensus.cc @@ -12,6 +12,7 @@ #include "include/pika_conf.h" #include "include/pika_rm.h" #include "include/pika_server.h" +#include "include/pika_repl_bgworker.h" using pstd::Status; @@ -228,6 +229,15 @@ ConsensusCoordinator::ConsensusCoordinator(const std::string& db_name) stable_logger_ = std::make_shared(db_name, log_path); mem_logger_ = std::make_shared(); logs_ = std::make_shared(); + // Initialize prepared_id + LogOffset last_offset; + BinlogOffset b_offset; + Status s = stable_logger_->Logger()->GetProducerStatus(&b_offset.filenum, &b_offset.offset); + if (s.ok()) { + last_offset.b_offset = b_offset; + SetPreparedId(last_offset); + LOG(INFO) << "Init: Initialized prepared_id to " << last_offset.ToString(); + } } ConsensusCoordinator::~ConsensusCoordinator() = default; @@ -335,6 +345,66 @@ Status ConsensusCoordinator::ProposeLog(const std::shared_ptr& cmd_ptr) { return Status::OK(); } +Status ConsensusCoordinator::BatchProposeLog(const std::vector>& cmd_ptrs, std::vector* offsets) { + if (cmd_ptrs.empty()) { + return Status::OK(); + } + // Reserve space for all command offsets + offsets->resize(cmd_ptrs.size()); + // Process each command in the batch + for (size_t i = 0; i < cmd_ptrs.size(); i++) { + const auto& cmd_ptr = cmd_ptrs[i]; + std::vector keys = cmd_ptr->current_key(); + // slotkey shouldn't add binlog + if (cmd_ptr->name() == kCmdNameSAdd && !keys.empty() &&(keys[0].compare(0, SlotKeyPrefix.length(), SlotKeyPrefix) == 0 || keys[0].compare(0, SlotTagPrefix.length(), SlotTagPrefix) == 0)) { + continue; + } + + // Generate binlog content + std::string content = cmd_ptr->ToRedisProtocol(); + std::string binlog; + + // Get offset for this command + LogOffset offset; + Status s = stable_logger_->Logger()->Put(content, &offset, binlog); + + if (!s.ok()) { + std::string db_name = cmd_ptr->db_name().empty() ? g_pika_conf->default_db() : cmd_ptr->db_name(); + std::shared_ptr db = g_pika_server->GetDB(db_name); + if (db) { + db->SetBinlogIoError(); + } + return s; + } + + // Store the offset for this command + (*offsets)[i] = offset; + + // Append to logs for replication, regardless of consistency mode. + logs_->AppendLog(Log::LogItem(offset, cmd_ptr, binlog)); + } + + // Signal auxiliary thread to handle these binlogs + g_pika_server->SignalAuxiliary(); + + // Proactively start sending binlogs to slaves in parallel with flushing to disk + std::shared_ptr db = g_pika_rm->GetSyncMasterDBByName(DBInfo(db_name_)); + if (db) { + db->WakeUpSlaveBinlogSync(); + } + + // Master node batch Binlog flush disk + stable_logger_->Logger()->Sync(); + + // update prepared_id to the offset of the last log + if (!offsets->empty()) { + SetPreparedId(offsets->back()); + LOG(INFO) << "BatchProposeLog: Updated prepared_id to " << offsets->back().ToString(); + } + + return Status::OK(); +} + Status ConsensusCoordinator::InternalAppendLog(const std::shared_ptr& cmd_ptr) { return InternalAppendBinlog(cmd_ptr); } @@ -444,7 +514,8 @@ uint32_t ConsensusCoordinator::term() { } void ConsensusCoordinator::InternalApplyFollower(const std::shared_ptr& cmd_ptr) { - g_pika_rm->ScheduleWriteDBTask(cmd_ptr, db_name_); + // g_pika_rm->ScheduleWriteDBTask(cmd_ptr, db_name_); + PikaReplBgWorker::WriteDBInSyncWay(cmd_ptr); } int ConsensusCoordinator::InitCmd(net::RedisParser* parser, const net::RedisCmdArgsType& argv) { @@ -823,6 +894,7 @@ bool ConsensusCoordinator::checkFinished(const LogOffset& offset) { //// pacificA private: Status ConsensusCoordinator::PersistAppendBinlog(const std::shared_ptr& cmd_ptr, LogOffset& cur_offset) { + std::lock_guard l(order_mu_); std::string content = cmd_ptr->ToRedisProtocol(); std::string binlog = std::string(); LogOffset offset = LogOffset(); @@ -848,6 +920,56 @@ Status ConsensusCoordinator::PersistAppendBinlog(const std::shared_ptr& cmd return stable_logger_->Logger()->IsOpened(); } +Status ConsensusCoordinator::BatchPersistAppendBinlog(const std::vector>& cmd_ptrs, + const std::vector& attributes, + std::vector* offsets) { + if (cmd_ptrs.empty() || cmd_ptrs.size() != attributes.size()) { + return Status::InvalidArgument("Invalid batch parameters"); + } + + std::lock_guard l(order_mu_); + offsets->resize(cmd_ptrs.size()); + for (size_t i = 0; i < cmd_ptrs.size(); ++i) { + const auto& cmd_ptr = cmd_ptrs[i]; + const auto& attribute = attributes[i]; + + LogOffset last_index = logs_->LastOffset(); + if (attribute.logic_id() < last_index.l_offset.index) { + LOG(WARNING) << DBInfo(db_name_).ToString() << "Drop log from leader logic_id " << attribute.logic_id() + << " cur last index " << last_index.l_offset.index; + (*offsets)[i] = LogOffset(); + continue; + } + + std::string content = cmd_ptr->ToRedisProtocol(); + std::string binlog = std::string(); + LogOffset offset = LogOffset(); + + Status s = stable_logger_->Logger()->Put(content, &offset, binlog); + if (!s.ok()) { + std::string db_name = cmd_ptr->db_name().empty() ? g_pika_conf->default_db() : cmd_ptr->db_name(); + std::shared_ptr db = g_pika_server->GetDB(db_name); + if (db) { + db->SetBinlogIoError(); + } + return s; + } + + logs_->AppendLog(Log::LogItem(offset, cmd_ptr, binlog)); + (*offsets)[i] = offset; + } + + // 更新prepared_id为最后一个有效的offset + for (auto it = offsets->rbegin(); it != offsets->rend(); ++it) { + if (it->IsValid()) { + SetPreparedId(*it); + break; + } + } + + return stable_logger_->Logger()->IsOpened(); +} + Status ConsensusCoordinator::AppendEntries(const std::shared_ptr& cmd_ptr, LogOffset& cur_logoffset) { std::vector keys = cmd_ptr->current_key(); // slotkey shouldn't add binlog @@ -887,12 +1009,17 @@ Status ConsensusCoordinator::AppendSlaveEntries(const std::shared_ptr& cmd_ Status ConsensusCoordinator::CommitAppLog(const LogOffset& master_committed_id) { int index = logs_->FindOffset(logs_->FirstOffset()); int log_size = logs_->Size(); // Cache log size + std::vector logs_to_apply; for (int i = index; i < log_size; ++i) { Log::LogItem log = logs_->At(i); if (master_committed_id >= log.offset) { - LOG(INFO) << "PacificA master_committed_id: " << master_committed_id.ToString() - << ", ApplyLog: " << log.offset.ToString(); - ApplyBinlog(log.cmd_ptr); + logs_to_apply.push_back(log); + } + } + if (!logs_to_apply.empty()) { + Status s = BatchApplyBinlogs(logs_to_apply); + if (!s.ok()) { + return s; } } @@ -902,29 +1029,113 @@ Status ConsensusCoordinator::CommitAppLog(const LogOffset& master_committed_id) } /** - * @brief Update the committed ID based on the Prepared ID of the slave + * @brief Update the committed ID based on the acknowledged offsets of all slaves. */ Status ConsensusCoordinator::UpdateCommittedID() { + // 获取当前的prepared_id + LogOffset current_prepared_id; + { + std::lock_guard l(prepared_id__rwlock_); + current_prepared_id = prepared_id_; + } + + // 获取当前的committed_id + LogOffset current_committed_id; + { + std::lock_guard l(committed_id_rwlock_); + current_committed_id = committed_id_; + } + std::unordered_map> slaves = sync_pros_.GetAllSlaveNodes(); - LogOffset slave_prepared_id = LogOffset(); - - for (const auto& slave : slaves) { - if (slave.second->slave_state == kSlaveBinlogSync) { - if (slave_prepared_id == LogOffset()) { - slave_prepared_id = slave.second->acked_offset; - } else if (slave.second->acked_offset < slave_prepared_id) { - slave_prepared_id = slave.second->acked_offset; + LogOffset min_acked_offset; + int active_slave_count = 0; + int total_slave_count = slaves.size(); + int binlog_sync_slave_count = 0; + // 初始化min_acked_offset为主节点的prepared_id + min_acked_offset = current_prepared_id; + + LOG(INFO) << "UpdateCommittedID: Master prepared_id: " << current_prepared_id.ToString() + << ", current_committed_id: " << current_committed_id.ToString() + << ", Total slaves: " << total_slave_count; + + // 如果没有从节点或没有活跃从节点,且prepared_id > committed_id,直接更新committed_id + if (total_slave_count == 0 && current_prepared_id > current_committed_id) { + LOG(INFO) << "UpdateCommittedID: No slaves, updating committed_id to prepared_id: " + << current_prepared_id.ToString(); + SetCommittedId(current_prepared_id); + return Status::OK(); + } + + // 查找所有活跃从节点中的最小acked_offset + for (const auto& slave_pair : slaves) { + const auto& slave = slave_pair.second; + // 记录从节点的状态和acked_offset,无论状态如何 + LOG(INFO) << "UpdateCommittedID: Slave " << slave->Ip() << ":" << slave->Port() + << " state: " << SlaveStateMsg[slave->slave_state] + << " acked_offset: " << slave->acked_offset.ToString(); + + // 考虑所有状态的从节点,只要它们有有效的acked_offset + if (slave->acked_offset.IsValid()) { + if (active_slave_count == 0) { + min_acked_offset = slave->acked_offset; + } else { + if (slave->acked_offset < min_acked_offset) { + min_acked_offset = slave->acked_offset; + } + } + active_slave_count++; + + // 如果是BinlogSync或Candidate状态,计数 + if (slave->slave_state == kSlaveBinlogSync || slave->slave_state == KCandidate) { + binlog_sync_slave_count++; } } } - if (slave_prepared_id < GetCommittedId()) { - LOG(WARNING) << "Error: slave_prepared_id (" << slave_prepared_id.ToString() << ") < master_committedId (" - << GetCommittedId().ToString() << ")"; - return Status::Error("slave_prepared_id < master_committedId"); + // 确定新的committed_id + LogOffset new_committed_id; + if (active_slave_count == 0) { + // 如果没有活跃的从节点,不应该推进committed_id + // 保持当前的committed_id不变,确保数据一致性 + new_committed_id = current_committed_id; + LOG(INFO) << "UpdateCommittedID: No active slaves, keeping current committed_id: " << current_committed_id.ToString(); + } else if (active_slave_count < total_slave_count) { + // 如果有部分从节点活跃,使用min_acked_offset作为new_committed_id + new_committed_id = min_acked_offset; + LOG(INFO) << "UpdateCommittedID: Partial slaves acknowledged (" << active_slave_count << "/" << total_slave_count + << "), using minimum acked offset: " << min_acked_offset.ToString(); + } else { + // 如果所有从节点都活跃,使用min_acked_offset作为new_committed_id + new_committed_id = min_acked_offset; + LOG(INFO) << "UpdateCommittedID: All slaves acknowledged (" << active_slave_count << "/" << total_slave_count + << "), using minimum acked offset: " << min_acked_offset.ToString(); + } + + // 确保new_committed_id不低于当前的committed_id + if (new_committed_id < current_committed_id) { + LOG(WARNING) << "UpdateCommittedID: New committed_id " << new_committed_id.ToString() + << " is less than current committed_id " << current_committed_id.ToString() + << ", keeping current value"; + new_committed_id = current_committed_id; + } + + // 确保new_committed_id不超过prepared_id + if (new_committed_id > current_prepared_id) { + LOG(WARNING) << "UpdateCommittedID: new_committed_id " << new_committed_id.ToString() + << " exceeds prepared_id " << current_prepared_id.ToString() + << ", adjusting to prepared_id"; + new_committed_id = current_prepared_id; + } + + // 如果committed_id有变化,更新它 + if (new_committed_id > current_committed_id) { + LOG(INFO) << "UpdateCommittedID: Updating committed_id from " << current_committed_id.ToString() + << " to " << new_committed_id.ToString(); + SetCommittedId(new_committed_id); + return Status::OK(); + } else { + LOG(INFO) << "UpdateCommittedID: No update needed, current committed ID: " << current_committed_id.ToString(); + return Status::OK(); } - SetCommittedId(slave_prepared_id); - LOG(INFO) << "PacificA update CommittedID: " << GetCommittedId().ToString(); - return Status::OK(); } Status ConsensusCoordinator::ProcessCoordination() { LogOffset offset = LogOffset(); @@ -964,30 +1175,107 @@ Status ConsensusCoordinator::ApplyBinlog(const std::shared_ptr& cmd_ptr) { return Status::OK(); } +// Batch apply commands to slave database +Status ConsensusCoordinator::BatchApplyBinlogs(const std::vector& logs_to_apply) { + if (logs_to_apply.empty()) { + return Status::OK(); + } + for (const auto& log_item : logs_to_apply) { + PikaReplBgWorker::WriteDBInSyncWay(log_item.cmd_ptr); + } + + return Status::OK(); +} + +void ConsensusCoordinator::BatchInternalApplyFollower(const std::vector>& cmd_ptrs) { + for (const auto& cmd_ptr : cmd_ptrs) { + // g_pika_rm->ScheduleWriteDBTask(cmd_ptr, db_name_); + PikaReplBgWorker::WriteDBInSyncWay(cmd_ptr); + } +} + Status ConsensusCoordinator::SendBinlog(std::shared_ptr slave_ptr, std::string db_name) { std::vector tasks; + const int MAX_BATCH_SIZE = 100; // Maximum number of logs to send in a single batch + + // Get current committed_id to ensure it's sent to the slave + LogOffset current_committed_id = GetCommittedId(); + LOG(INFO) << "SendBinlog: [Thread " << std::this_thread::get_id() << "] Current committed_id: " << current_committed_id.ToString() + << ", sending to slave " << slave_ptr->Ip() << ":" << slave_ptr->Port() + << ", logs_ addr: " << logs_.get() << ", db_name: " << db_name_; // Check if there are new log entries that need to be sent to the slave - if (logs_->LastOffset() >= slave_ptr->acked_offset) { + LOG(INFO) << "SendBinlog: logs_->LastOffset()=" << logs_->LastOffset().ToString() + << ", slave_ptr->acked_offset=" << slave_ptr->acked_offset.ToString() + << ", logs_->Size()=" << logs_->Size(); + + if (logs_->Size() > 0 && logs_->LastOffset() >= slave_ptr->acked_offset) { // Find the index of the log entry corresponding to the slave's acknowledged offset int index = logs_->FindOffset(slave_ptr->acked_offset); + int entries_to_send = logs_->Size() - index; + LOG(INFO) << "SendBinlog: Found " << entries_to_send << " new log entries to send, " + << "starting from index " << index << " of " << logs_->Size(); + if (index < logs_->Size()) { - for (int i = index; i < logs_->Size(); ++i) { - const Log::LogItem& item = logs_->At(i); - - slave_ptr->SetLastSendTime(pstd::NowMicros()); - - RmNode rm_node(slave_ptr->Ip(), slave_ptr->Port(), slave_ptr->DBName(), slave_ptr->SessionId()); - WriteTask task(rm_node, BinlogChip(item.offset, item.binlog_), slave_ptr->sent_offset, GetCommittedId()); - tasks.emplace_back(std::move(task)); - - slave_ptr->sent_offset = item.offset; + // Send log entries in optimized batches + RmNode rm_node(slave_ptr->Ip(), slave_ptr->Port(), db_name, slave_ptr->SessionId()); + + // For large batches, use specialized batch handling + if (entries_to_send > MAX_BATCH_SIZE) { + LOG(INFO) << "SendBinlog: Using optimized batch sending for " << entries_to_send << " entries"; + + // Process in chunks of MAX_BATCH_SIZE + for (int batch_start = index; batch_start < logs_->Size(); batch_start += MAX_BATCH_SIZE) { + int batch_end = std::min(batch_start + MAX_BATCH_SIZE, logs_->Size()); + std::vector batch_tasks; + + for (int i = batch_start; i < batch_end; ++i) { + Log::LogItem item = logs_->At(i); + WriteTask task(rm_node, BinlogChip(item.offset, item.binlog_), item.offset, current_committed_id); + batch_tasks.push_back(task); + } + + g_pika_rm->ProduceWriteQueue(slave_ptr->Ip(), slave_ptr->Port(), db_name, batch_tasks); + LOG(INFO) << "SendBinlog: Sent batch " << (batch_start - index) / MAX_BATCH_SIZE + 1 + << " with " << (batch_end - batch_start) << " entries"; + } + } else { + // Send all entries in a single batch + for (int i = index; i < logs_->Size(); ++i) { + Log::LogItem item = logs_->At(i); + WriteTask task(rm_node, BinlogChip(item.offset, item.binlog_), item.offset, current_committed_id); + tasks.push_back(task); + } } + } else { + LOG(INFO) << "SendBinlog: No new log entries to send, index " << index << " is out of range (logs size: " << logs_->Size() << ")"; + } + } else { + if (logs_->Size() == 0) { + LOG(INFO) << "SendBinlog: No logs available yet (logs_->Size()=0), will send empty binlog to maintain connection"; + } else { + LOG(INFO) << "SendBinlog: Slave is already up to date, last offset: " << logs_->LastOffset().ToString() + << ", slave acked offset: " << slave_ptr->acked_offset.ToString(); } } + // Only send empty binlog if there are no actual log entries to send + // This prevents the deadlock where master waits for slave ACK and slave waits for master data + if (tasks.empty() && logs_->Size() == 0) { + LOG(INFO) << "SendBinlog: Sending empty binlog with current committed_id: " << current_committed_id.ToString(); + RmNode rm_node(slave_ptr->Ip(), slave_ptr->Port(), db_name, slave_ptr->SessionId()); + // Create an empty WriteTask that includes the current committed_id + WriteTask empty_task(rm_node, BinlogChip(LogOffset(), ""), LogOffset(), current_committed_id); + tasks.push_back(empty_task); + } + + // Send the tasks to the slave if (!tasks.empty()) { + LOG(INFO) << "SendBinlog: Sending " << tasks.size() << " tasks to slave " << slave_ptr->Ip() << ":" << slave_ptr->Port(); + extern std::unique_ptr g_pika_rm; g_pika_rm->ProduceWriteQueue(slave_ptr->Ip(), slave_ptr->Port(), db_name, tasks); + } else { + LOG(INFO) << "SendBinlog: No tasks to send to slave " << slave_ptr->Ip() << ":" << slave_ptr->Port(); } return Status::OK(); } diff --git a/src/pika_repl_bgworker.cc b/src/pika_repl_bgworker.cc index 5340533160..048d06ba16 100644 --- a/src/pika_repl_bgworker.cc +++ b/src/pika_repl_bgworker.cc @@ -145,16 +145,61 @@ void PikaReplBgWorker::HandleBGWorkerWriteBinlog(void* arg) { if (binlog_res.binlog().empty()) { continue; } - if (!PikaBinlogTransverter::BinlogItemWithoutContentDecode(TypeFirst, binlog_res.binlog(), &worker->binlog_item_)) { - LOG(WARNING) << "Binlog item decode failed"; + // Handle binlog data with batch magic number detection + const std::string& received_binlog = binlog_res.binlog(); + std::string binlog_str = received_binlog; + // Check if this is the first binlog entry in a batch (contains PIKA_BATCH_MAGIC) + bool has_batch_magic = false; + if (i == (*index)[0] && received_binlog.size() >= sizeof(uint32_t)) { + uint32_t magic_num = 0; + memcpy(&magic_num, received_binlog.data(), sizeof(uint32_t)); + if (magic_num == PIKA_BATCH_MAGIC) { + has_batch_magic = true; + // Remove the magic number from the binlog data + binlog_str = received_binlog.substr(sizeof(uint32_t)); + LOG(INFO) << "HandleBGWorkerWriteBinlog: Detected PIKA_BATCH_MAGIC in binlog entry " << i + << ", processing as batch, original size: " << received_binlog.size() + << ", new size after removing magic: " << binlog_str.size(); + } + } + + // Validate binlog data + if (binlog_str.empty()) { + only_keepalive = true; + continue; + } + + if (binlog_str.size() < BINLOG_ENCODE_LEN) { + LOG(WARNING) << "HandleBGWorkerWriteBinlog: Binlog data too small (" << binlog_str.size() + << " bytes), minimum required: " << BINLOG_ENCODE_LEN; + continue; + } + + // Decode the binlog item + if (!PikaBinlogTransverter::BinlogItemWithoutContentDecode(TypeFirst, binlog_str, &worker->binlog_item_)) { slave_db->SetReplState(ReplState::kTryConnect); return; } - const char* redis_parser_start = binlog_res.binlog().data() + BINLOG_ENCODE_LEN; - int redis_parser_len = static_cast(binlog_res.binlog().size()) - BINLOG_ENCODE_LEN; + + // Extract Redis command data + const char* redis_parser_start = binlog_str.data() + BINLOG_ENCODE_LEN; + int redis_parser_len = static_cast(binlog_str.size()) - BINLOG_ENCODE_LEN; + + if (redis_parser_len <= 0) { + LOG(WARNING) << "HandleBGWorkerWriteBinlog: No Redis command data after binlog header for entry " << i; + continue; + } + + // Create a new parser instance for each binlog entry to ensure clean state + net::RedisParser temp_parser; + net::RedisParserSettings settings; + settings.DealMessage = &(PikaReplBgWorker::HandleWriteBinlog); + temp_parser.RedisParserInit(REDIS_PARSER_REQUEST, settings); + temp_parser.data = worker; + int processed_len = 0; net::RedisParserStatus ret = - worker->redis_parser_.ProcessInputBuffer(redis_parser_start, redis_parser_len, &processed_len); + temp_parser.ProcessInputBuffer(redis_parser_start, redis_parser_len, &processed_len); if (ret != net::kRedisParserDone) { LOG(WARNING) << "Redis parser failed"; slave_db->SetReplState(ReplState::kTryConnect); @@ -177,6 +222,9 @@ void PikaReplBgWorker::HandleBGWorkerWriteBinlog(void* arg) { &productor_status.l_offset.term, &productor_status.l_offset.index); ack_end = productor_status; ack_end.l_offset.term = pb_end.l_offset.term; + + // slave nodes batch Binlog flush disk + logger->Sync(); } g_pika_rm->SendBinlogSyncAckRequest(db_name, ack_start, ack_end); @@ -226,8 +274,8 @@ int PikaReplBgWorker::HandleWriteBinlog(net::RedisParser* parser, const net::Red } void PikaReplBgWorker::HandleBGWorkerWriteDB(void* arg) { - std::unique_ptr task_arg(static_cast(arg)); - const std::shared_ptr c_ptr = task_arg->cmd_ptr; + std::unique_ptr> cmd_ptr_ptr(static_cast*>(arg)); + const std::shared_ptr c_ptr = *cmd_ptr_ptr; WriteDBInSyncWay(c_ptr); } diff --git a/src/pika_repl_server.cc b/src/pika_repl_server.cc index c8f1c9f9dc..463c17a42e 100644 --- a/src/pika_repl_server.cc +++ b/src/pika_repl_server.cc @@ -88,9 +88,16 @@ void PikaReplServer::BuildBinlogOffset(const LogOffset& offset, InnerMessage::Bi } void PikaReplServer::BuildBinlogSyncResp(const std::vector& tasks, InnerMessage::InnerResponse* response) { + if (tasks.empty()) { + return; + } response->set_code(InnerMessage::kOk); response->set_type(InnerMessage::Type::kBinlogSync); - for (const auto& task : tasks) { + + // Since CommandCollector is enabled, all data should be treated as batch processing + bool is_batch = g_pika_conf->command_batch_enabled(); + for (size_t task_idx = 0; task_idx < tasks.size(); task_idx++) { + const auto& task = tasks[task_idx]; InnerMessage::InnerResponse::BinlogSync* binlog_sync = response->add_binlog_sync(); binlog_sync->set_session_id(task.rm_node_.SessionId()); InnerMessage::Slot* db = binlog_sync->mutable_slot(); @@ -103,11 +110,23 @@ void PikaReplServer::BuildBinlogSyncResp(const std::vector& tasks, In db->set_slot_id(0); InnerMessage::BinlogOffset* boffset = binlog_sync->mutable_binlog_offset(); BuildBinlogOffset(task.binlog_chip_.offset_, boffset); - if(g_pika_server->IsConsistency()){ + + // For batch binlog transmission (when CommandCollector is enabled), + // add PIKA_BATCH_MAGIC at the beginning of the first binlog entry + if (is_batch && task_idx == 0) { + std::string magic_binlog; + magic_binlog.resize(sizeof(uint32_t)); + memcpy(&magic_binlog[0], &PIKA_BATCH_MAGIC, sizeof(uint32_t)); + magic_binlog.append(task.binlog_chip_.binlog_); + binlog_sync->set_binlog(magic_binlog); + } else { + binlog_sync->set_binlog(task.binlog_chip_.binlog_); + } + + if (g_pika_server->IsConsistency()) { InnerMessage::BinlogOffset* committed_id = binlog_sync->mutable_committed_id(); BuildBinlogOffset(task.committed_id_, committed_id); } - binlog_sync->set_binlog(task.binlog_chip_.binlog_); } } diff --git a/src/pika_rm.cc b/src/pika_rm.cc index 9c777339ab..71781a4a13 100644 --- a/src/pika_rm.cc +++ b/src/pika_rm.cc @@ -12,6 +12,7 @@ #include #include +#include #include "net/include/net_cli.h" @@ -25,6 +26,7 @@ using pstd::Status; extern std::unique_ptr g_pika_rm; extern PikaServer* g_pika_server; +extern std::unique_ptr g_pika_conf; /* SyncDB */ @@ -53,11 +55,14 @@ Status SyncMasterDB::GetSlaveNodeSession(const std::string& ip, int port, int32_ return Status::NotFound("slave " + ip + ":" + std::to_string(port) + " not found"); } - slave_ptr->Lock(); - *session = slave_ptr->SessionId(); - slave_ptr->Unlock(); - - return Status::OK(); + if (slave_ptr->slave_mu.try_lock()) { + *session = slave_ptr->SessionId(); + slave_ptr->slave_mu.unlock(); + return Status::OK(); + } else { + LOG(WARNING) << "GetSlaveNodeSession: Failed to acquire lock for " << ip << ":" << port << ", slave may be busy"; + return Status::Busy("Slave node is busy, try again later"); + } } Status SyncMasterDB::AddSlaveNode(const std::string& ip, int port, int session_id) { @@ -229,34 +234,32 @@ Status SyncMasterDB::GetSlaveState(const std::string& ip, int port, SlaveState* } Status SyncMasterDB::WakeUpSlaveBinlogSync() { - std::unordered_map> slaves = GetAllSlaveNodes(); - std::vector> to_del; - for (auto& slave_iter : slaves) { - std::shared_ptr slave_ptr = slave_iter.second; - std::lock_guard l(slave_ptr->slave_mu); - if (slave_ptr->sent_offset == slave_ptr->acked_offset) { - Status s; - if (coordinator_.GetISConsistency()) { - if(slave_ptr->slave_state == SlaveState::kSlaveBinlogSync||slave_ptr->slave_state == SlaveState::KCandidate){ - s = coordinator_.SendBinlog(slave_ptr, db_info_.db_name_); - } - } else { - s = ReadBinlogFileToWq(slave_ptr); - } - if (!s.ok()) { - to_del.push_back(slave_ptr); - LOG(WARNING) << "WakeUpSlaveBinlogSync failed, marking for deletion: " - << slave_ptr->ToStringStatus() << " - " << s.ToString(); - } - } - } + std::unordered_map> slaves = GetAllSlaveNodes(); + std::vector> to_del; + for (auto& slave_iter : slaves) { + std::shared_ptr slave_ptr = slave_iter.second; + + slave_ptr->Lock(); + SlaveState current_state = slave_ptr->slave_state; + slave_ptr->Unlock(); - for (const auto& to_del_slave : to_del) { - RemoveSlaveNode(to_del_slave->Ip(), to_del_slave->Port()); - LOG(INFO) << "Removed slave: " << to_del_slave->ToStringStatus(); + // Only send to slaves that are actively waiting for binlogs + if (current_state == SlaveState::kSlaveBinlogSync || current_state == SlaveState::KCandidate) { + Status s = coordinator_.SendBinlog(slave_ptr, db_info_.db_name_); + if (!s.ok()) { + to_del.push_back(slave_ptr); + LOG(WARNING) << "WakeUpSlaveBinlogSync: SendBinlog failed for slave " << slave_ptr->ToString() << ": " + << s.ToString(); + } } + } - return Status::OK(); + for (const auto& to_del_slave : to_del) { + RemoveSlaveNode(to_del_slave->Ip(), to_del_slave->Port()); + LOG(INFO) << "Removed slave due to SendBinlog failure: " << to_del_slave->ToStringStatus(); + } + + return Status::OK(); } @@ -424,11 +427,33 @@ LogOffset SyncMasterDB::GetCommittedId(){ Status SyncMasterDB::AppendSlaveEntries(const std::shared_ptr& cmd_ptr, const BinlogItem& attribute) { return coordinator_.AppendSlaveEntries(cmd_ptr, attribute); } + +Status SyncMasterDB::BatchAppendSlaveEntries(const std::vector>& cmd_ptrs, + const std::vector& attributes) { + if (cmd_ptrs.empty() || cmd_ptrs.size() != attributes.size()) { + return Status::InvalidArgument("Invalid batch parameters"); + } + + std::vector offsets; + Status s = coordinator_.BatchPersistAppendBinlog(cmd_ptrs, attributes, &offsets); + if (!s.ok()) { + return s; + } + + return Status::OK(); +} Status SyncMasterDB::ProcessCoordination(){ return coordinator_.ProcessCoordination(); } Status SyncMasterDB::UpdateCommittedID(){ - return coordinator_.UpdateCommittedID(); + Status s = coordinator_.UpdateCommittedID(); + if (s.ok()) { + // UpdateCommittedID success - removed verbose logging + int slave_count = GetNumberOfSlaveNode(); + } else { + LOG(WARNING) << "UpdateCommittedID failed: " << s.ToString(); + } + return s; } Status SyncMasterDB::Truncate(const LogOffset& offset){ return coordinator_.Truncate(offset); @@ -471,6 +496,71 @@ Status SyncMasterDB::AppendCandidateBinlog(const std::string& ip, int port, cons return Status::OK(); } +pstd::Status SyncMasterDB::WaitForSlaveAcks(const LogOffset& target_offset, int timeout_ms) { + // bug: 重发有问题 + // Get slave count + int slave_count = GetNumberOfSlaveNode(); + LOG(INFO) << "WaitForSlaveAcks: Waiting for ACKs from master and " << slave_count << " slave(s) for target " << target_offset.ToString(); + + // If no slaves, return success immediately + if (slave_count == 0) { + LOG(INFO) << "WaitForSlaveAcks: No slaves connected, returning success immediately"; + // Update committed_id + coordinator_.SetCommittedId(target_offset); + return Status::OK(); + } + + g_pika_rm->WakeUpBinlogSync(); + + // Use efficient polling mechanism to avoid creating extra threads + auto start_time = std::chrono::steady_clock::now(); + auto timeout_duration = std::chrono::milliseconds(timeout_ms); + const int POLL_INTERVAL_MS = 10; // 10ms polling interval + + while (true) { + // Check if timeout + auto elapsed = std::chrono::steady_clock::now() - start_time; + if (elapsed >= timeout_duration) { + LOG(WARNING) << "WaitForSlaveAcks: after " << timeout_ms << "ms waiting for target " << target_offset.ToString(); + return Status::Timeout("Strong consistency replication timed out"); + } + + // Check acknowledgment status of each slave node + std::unordered_map> slaves = GetAllSlaveNodes(); + int ack_count = 1; // Master node is already acknowledged + + for (const auto& slave_pair : slaves) { + std::shared_ptr slave = slave_pair.second; + if (slave) { + slave->Lock(); + LogOffset acked_offset = slave->acked_offset; + slave->Unlock(); + + if (acked_offset >= target_offset) { + ack_count++; + } + } + } + + // Check if all nodes have acknowledged (1 master + N slaves) + int expected_acks = 1 + slave_count; + if (ack_count >= expected_acks) { + LOG(INFO) << "WaitForSlaveAcks: All " << expected_acks << " nodes have acknowledged target " << target_offset.ToString(); + // Update committed_id + coordinator_.SetCommittedId(target_offset); + return Status::OK(); + } + + // Sleep briefly then retry + std::this_thread::sleep_for(std::chrono::milliseconds(POLL_INTERVAL_MS)); + + // Periodically trigger binlog sync to ensure slave nodes receive data + if (elapsed.count() % 100 == 0) { // Trigger every 100ms + g_pika_rm->WakeUpBinlogSync(); + } + } +} + Status SyncMasterDB::ConsensusProposeLog(const std::shared_ptr& cmd_ptr) { // If consistency is not required, directly propose the log without waiting for consensus if (!coordinator_.GetISConsistency()) { @@ -498,6 +588,114 @@ Status SyncMasterDB::ConsensusProposeLog(const std::shared_ptr& cmd_ptr) { return Status::Timeout("No consistency achieved within 10 seconds"); } +Status SyncMasterDB::ConsensusBatchProposeLog(const std::vector>& cmd_ptrs, std::vector* offsets) { + // If the batch is empty, nothing to do + if (cmd_ptrs.empty()) { + return Status::OK(); + } + + // For large batches, log batch size for monitoring + if (cmd_ptrs.size() > 10) { + LOG(INFO) << "ConsensusBatchProposeLog: Processing large batch of " << cmd_ptrs.size() << " commands"; + } + + // First, propose the log batch to the coordinator with optimized performance + auto batch_start = std::chrono::steady_clock::now(); + Status s = coordinator_.BatchProposeLog(cmd_ptrs, offsets); + auto batch_end = std::chrono::steady_clock::now(); + auto batch_time_ms = std::chrono::duration_cast(batch_end - batch_start).count(); + + if (!s.ok()) { + LOG(WARNING) << "ConsensusBatchProposeLog: Failed to propose log batch: " << s.ToString(); + return s; + } + + // Performance logging for large batches + if (cmd_ptrs.size() > 10) { + LOG(INFO) << "ConsensusBatchProposeLog: Successfully proposed batch of " << cmd_ptrs.size() + << " commands in " << batch_time_ms << "ms (" + << (cmd_ptrs.size() / (batch_time_ms ? batch_time_ms : 1)) << " commands/ms)"; + } + + // If consistency is not required, return immediately without waiting for replication + if (!coordinator_.GetISConsistency()) { + LOG(INFO) << "ConsensusBatchProposeLog: Consistency not required, returning immediately"; + return s; + } + + // Record the current committed_id and slave node count + LogOffset current_committed_id = GetCommittedId(); + int slave_count = GetNumberOfSlaveNode(); + + LOG(INFO) << "ConsensusBatchProposeLog: Before BatchProposeLog - Current committed_id: " << current_committed_id.ToString() + << ", expecting ACKs from 1 master and " << slave_count << " slave(s)"; + + // 在BatchProposeLog完成后,获取实际的prepared_id作为等待目标 + LogOffset actual_prepared_id = GetPreparedId(); + LogOffset last_cmd_offset; + if (!offsets->empty()) { + last_cmd_offset = offsets->back(); + LOG(INFO) << "ConsensusBatchProposeLog: Last command offset in batch: " << last_cmd_offset.ToString(); + LOG(INFO) << "ConsensusBatchProposeLog: After BatchProposeLog - actual prepared_id: " << actual_prepared_id.ToString(); + } + + // For strong consistency mode, set a batch-level timeout that applies to the entire batch + int batch_timeout_ms = g_pika_conf->replication_ack_timeout(); + // Adjust timeout based on batch size for large batches + if (cmd_ptrs.size() > 100) { + // Scale timeout logarithmically with batch size + batch_timeout_ms = static_cast(batch_timeout_ms * (1 + log10(cmd_ptrs.size() / 100.0))); + LOG(INFO) << "ConsensusBatchProposeLog: Adjusted batch timeout to " << batch_timeout_ms << "ms for large batch"; + } + + // For strong consistency mode, wait for the batch to be committed + LOG(INFO) << "ConsensusBatchProposeLog: Waiting for batch to be committed (target: " << actual_prepared_id.ToString() + << ") with timeout of " << batch_timeout_ms << "ms"; + + s = WaitForSlaveAcks(actual_prepared_id, batch_timeout_ms); + + // Process synchronization results + if (!s.ok()) { + if (s.IsTimeout()) { + LOG(WARNING) << "ConsensusBatchProposeLog: Batch timed out waiting for ACKs: " << s.ToString(); + } else if (s.IsIncomplete()) { + LOG(WARNING) << "ConsensusBatchProposeLog: Not all nodes acknowledged the batch: " << s.ToString(); + } else { + LOG(WARNING) << "ConsensusBatchProposeLog: Batch operation failed with status: " << s.ToString(); + } + LOG(WARNING) << "ConsensusBatchProposeLog: Batch operation could not be confirmed with strong consistency, " + << "batch size: " << cmd_ptrs.size(); + } else { + LogOffset new_committed_id = GetCommittedId(); + LOG(INFO) << "ConsensusBatchProposeLog: Successfully received ACKs for entire batch, " + << "new committed_id: " << new_committed_id.ToString(); + + // Verify that synchronization successfully included all offsets in this batch of commands + if (!offsets->empty() && new_committed_id < last_cmd_offset) { + LOG(WARNING) << "ConsensusBatchProposeLog: New committed_id " << new_committed_id.ToString() + << " is less than last command offset " << last_cmd_offset.ToString() + << ", some commands in batch may not be fully replicated"; + + // For strong consistency, we should ensure all commands are replicated + // But if there are no slaves, we don't need to worry about replication + if (slave_count > 0) { + LOG(WARNING) << "ConsensusBatchProposeLog: Some commands may not be fully replicated to all slaves, but proceeding"; + } else { + LOG(INFO) << "ConsensusBatchProposeLog: No slaves connected, no replication needed"; + } + } + } + return s; +} + +// Per-DB global batching window across threads +struct WindowState { + pstd::Mutex mu; + std::atomic start_us{0}; + std::atomic accepted{0}; +}; +static std::unordered_map g_db_windows; +static pstd::Mutex g_db_windows_mu; Status SyncMasterDB::ConsensusProcessLeaderLog(const std::shared_ptr& cmd_ptr, const BinlogItem& attribute) { return coordinator_.ProcessLeaderLog(cmd_ptr, attribute); @@ -515,6 +713,8 @@ std::unordered_map> SyncMasterDB::GetAll return coordinator_.SyncPros().GetAllSlaveNodes(); } +std::shared_ptr SyncMasterDB::GetCommandCollector() { return command_collector_; } + /* SyncSlaveDB */ SyncSlaveDB::SyncSlaveDB(const std::string& db_name) : SyncDB(db_name) { @@ -704,63 +904,85 @@ void PikaReplicaManager::ProduceWriteQueue(const std::string& ip, int port, std: } int PikaReplicaManager::ConsumeWriteQueue() { - std::unordered_map>> to_send_map; - int counter = 0; + // Quick check if there are any tasks { std::lock_guard l(write_queue_mu_); - for (auto& iter : write_queues_) { - const std::string& ip_port = iter.first; - std::unordered_map>& p_map = iter.second; - for (auto& db_queue : p_map) { - std::queue& queue = db_queue.second; - for (int i = 0; i < kBinlogSendPacketNum; ++i) { - if (queue.empty()) { - break; - } - size_t batch_index = queue.size() > kBinlogSendBatchNum ? kBinlogSendBatchNum : queue.size(); - std::vector to_send; - size_t batch_size = 0; - for (size_t i = 0; i < batch_index; ++i) { - WriteTask& task = queue.front(); - batch_size += task.binlog_chip_.binlog_.size(); - // make sure SerializeToString will not over 2G - if (batch_size > PIKA_MAX_CONN_RBUF_HB) { - break; - } - to_send.push_back(task); - queue.pop(); - counter++; - } - if (!to_send.empty()) { - to_send_map[ip_port].push_back(std::move(to_send)); - } + if (write_queues_.empty()) { + static int empty_counter = 0; + return 0; + } + } + if (g_pika_conf->command_batch_enabled()) { + for (auto& db_item : sync_master_dbs_) { + if (db_item.second) { + auto command_collector = db_item.second->GetCommandCollector(); + if (command_collector) { + command_collector->FlushCommands(); } } } } + // A list of sending jobs to be executed outside the lock. + // Each job is a tuple of (ip, port, tasks_to_send). + std::vector>> all_sends; + int counter = 0; - std::vector to_delete; - for (auto& iter : to_send_map) { - std::string ip; - int port = 0; - if (!pstd::ParseIpPortString(iter.first, ip, port)) { - LOG(WARNING) << "Parse ip_port error " << iter.first; - continue; - } - for (auto& to_send : iter.second) { - Status s = pika_repl_server_->SendSlaveBinlogChips(ip, port, to_send); - if (!s.ok()) { - LOG(WARNING) << "send binlog to " << ip << ":" << port << " failed, " << s.ToString(); - to_delete.push_back(iter.first); + // === Start of Critical Section === + { + std::lock_guard l(write_queue_mu_); + LOG(INFO) << "ConsumeWriteQueue: write_queues_ size: " << write_queues_.size(); + auto slave_iter = write_queues_.begin(); + while (slave_iter != write_queues_.end()) { + std::string ip; + int port = 0; + if (!pstd::ParseIpPortString(slave_iter->first, ip, port)) { + LOG(WARNING) << "Parse ip_port error " << slave_iter->first; + slave_iter = write_queues_.erase(slave_iter); continue; } + + // Collect all tasks for this slave from all its dbs + std::vector tasks_for_this_slave; + auto& p_map = slave_iter->second; + auto db_iter = p_map.begin(); + while (db_iter != p_map.end()) { + auto& queue = db_iter->second; + while (!queue.empty()) { + tasks_for_this_slave.push_back(std::move(queue.front())); + queue.pop(); + } + // Since the queue is now empty, erase this db entry + db_iter = p_map.erase(db_iter); + } + + if (!tasks_for_this_slave.empty()) { + LOG(INFO) << "ConsumeWriteQueue: Found " << tasks_for_this_slave.size() << " tasks for slave " << ip << ":" << port; + all_sends.emplace_back(ip, port, std::move(tasks_for_this_slave)); + } else { + LOG(INFO) << "ConsumeWriteQueue: No tasks found for slave " << ip << ":" << port; + } + + // Since all db entries for this slave are processed and erased, + // erase the slave entry itself. + slave_iter = write_queues_.erase(slave_iter); } } + // === End of Critical Section === - if (!to_delete.empty()) { - std::lock_guard l(write_queue_mu_); - for (auto& del_queue : to_delete) { - write_queues_.erase(del_queue); + // Now, execute all the prepared network IO jobs outside the lock. + for (auto& send_job : all_sends) { + std::string& ip = std::get<0>(send_job); + int port = std::get<1>(send_job); + std::vector& to_send = std::get<2>(send_job); + + counter += to_send.size(); + LOG(INFO) << "ConsumeWriteQueue: Sending " << to_send.size() << " tasks to " << ip << ":" << port; + Status s = pika_repl_server_->SendSlaveBinlogChips(ip, port, to_send); + if (!s.ok()) { + LOG(WARNING) << "send binlog to " << ip << ":" << port << " failed, " << s.ToString(); + DropItemInWriteQueue(ip, port); + } else { + LOG(INFO) << "ConsumeWriteQueue: Successfully sent " << to_send.size() << " tasks to " << ip << ":" << port; } } return counter; @@ -1158,6 +1380,16 @@ void PikaReplicaManager::FindCommonMaster(std::string* master) { } } +std::shared_ptr PikaReplicaManager::GetConsensusCoordinator(const std::string& db_name) { + std::shared_lock l(dbs_rw_); + DBInfo p_info(db_name); + if (sync_master_dbs_.find(p_info) == sync_master_dbs_.end()) { + return nullptr; + } + // Return a pointer to the existing coordinator instead of creating a copy + return sync_master_dbs_[p_info]->StableLogger()->coordinator(); +} + void PikaReplicaManager::RmStatus(std::string* info) { std::shared_lock l(dbs_rw_); std::stringstream tmp_stream; diff --git a/src/pika_server.cc b/src/pika_server.cc index a7d50b1e71..e688202c75 100644 --- a/src/pika_server.cc +++ b/src/pika_server.cc @@ -24,6 +24,7 @@ #include "include/pika_monotonic_time.h" #include "include/pika_rm.h" #include "include/pika_server.h" +#include "include/pika_command_collector.h" using pstd::Status; extern PikaServer* g_pika_server; @@ -169,6 +170,14 @@ void PikaServer::Start() { } */ + if (g_pika_conf->command_batch_enabled()) { + auto master_db = g_pika_rm->GetSyncMasterDBByName(DBInfo(g_pika_conf->default_db())); + if (master_db) { + auto command_collector = master_db->GetCommandCollector(); + LOG(INFO) << "Command batch enabled, command collector accessed successfully"; + } + } + ret = pika_client_processor_->Start(); if (ret != net::kSuccess) { dbs_.clear(); @@ -588,10 +597,17 @@ void PikaServer::DeleteSlave(int fd) { } if (slave_num == 0) { - std::lock_guard l(state_protector_); - last_role_ = role_; - role_ &= ~PIKA_ROLE_MASTER; - leader_protected_mode_ = false; // explicitly cancel protected mode + // Check if slaveof is configured, if so, do not remove MASTER role + // Because a Pika configured as a slave is still the master node for its clients + std::string slaveof = g_pika_conf->slaveof(); + if (slaveof.empty()) { + // Only remove MASTER role when slaveof is not configured + std::lock_guard l(state_protector_); + last_role_ = role_; + role_ &= ~PIKA_ROLE_MASTER; + leader_protected_mode_ = false; // explicitly cancel protected mode + LOG(INFO) << "DeleteSlave: Removed MASTER role for standalone node"; + } } } @@ -1098,7 +1114,13 @@ int PikaServer::SendToPeer() { return g_pika_rm->ConsumeWriteQueue(); } void PikaServer::SignalAuxiliary() { pika_auxiliary_thread_->cv_.notify_one(); } -Status PikaServer::TriggerSendBinlogSync() { return g_pika_rm->WakeUpBinlogSync(); } +Status PikaServer::TriggerSendBinlogSync() { + // Only execute on master nodes + if (!(role_ & PIKA_ROLE_MASTER)) { + return Status::OK(); + } + return g_pika_rm->WakeUpBinlogSync(); +} int PikaServer::PubSubNumPat() { return pika_pubsub_thread_->PubSubNumPat(); } diff --git a/src/pika_stable_log.cc b/src/pika_stable_log.cc index b1e9fc278a..dcc781354d 100644 --- a/src/pika_stable_log.cc +++ b/src/pika_stable_log.cc @@ -35,6 +35,21 @@ StableLog::StableLog(std::string db_name, std::string log_path) StableLog::~StableLog() = default; +std::shared_ptr StableLog::coordinator() { + // Get and return the coordinator pointer directly from SyncMasterDB + auto master_db = g_pika_rm->GetSyncMasterDBByName(DBInfo(db_name_)); + if (master_db) { + // Return a nullptr if GetCoordinator is not ready + try { + // Use aliasing constructor instead of dangerous custom deleter + return std::shared_ptr(master_db, &master_db->GetCoordinator()); + } catch (const std::exception& e) { + LOG(ERROR) << "Failed to get coordinator for " << db_name_ << ": " << e.what(); + return nullptr; + } + } + return nullptr; +} void StableLog::Leave() { Close(); RemoveStableLogDir(); diff --git a/src/storage/src/redis.cc b/src/storage/src/redis.cc index 3066a62759..0c4aa52cae 100644 --- a/src/storage/src/redis.cc +++ b/src/storage/src/redis.cc @@ -14,6 +14,7 @@ Redis::Redis(Storage* const s, const DataType& type) lock_mgr_(std::make_shared(1000, 0, std::make_shared())), small_compaction_threshold_(5000), small_compaction_duration_threshold_(10000) { + default_write_options_.disableWAL = true; statistics_store_ = std::make_unique>(); scan_cursors_store_ = std::make_unique>(); scan_cursors_store_->SetCapacity(5000); diff --git a/tests/integration/clean_start.sh b/tests/integration/clean_start.sh new file mode 100644 index 0000000000..589cfb7059 --- /dev/null +++ b/tests/integration/clean_start.sh @@ -0,0 +1,54 @@ +#!/bin/bash + +cd /home/pika/caiyu/pikiwidb || exit + +clean_ports() { + echo "Checking and cleaning ports..." + + sudo killall -9 pika + + sleep 1 +} + +echo "Building project..." +sudo ./build.sh +if [ $? -ne 0 ]; then + echo "Build failed!" + exit 1 +fi +echo "Build successful." + +echo "Cleaning up test directory..." +sudo rm -rf ./output/pacifica_test/ +echo "Cleanup completed." + +clean_ports + +cd output || exit + +echo "Starting master and slave servers..." +sudo ../tests/integration/start_master_and_slave.sh +sleep 10 + +echo "Setting up strong consistency replication..." + +redis-cli -p 9302 slaveof 127.0.0.1 9301 strong +sleep 1 +echo "Replication setup successful." + +echo "Running benchmark..." + +redis-benchmark -p 9301 -t set -r 100000 -n 100000 -c 500 --threads 4 +echo "Benchmark finished." + +echo -e "\n==== 主节点 INFO 日志 ====" +tail -n 150 ./pacifica_test/master/log/pika.INFO + +echo -e "\n==== 主节点 WARNING 日志 ====" +tail -n 150 ./pacifica_test/master/log/pika.WARNING + +echo -e "\n==== 从节点 INFO 日志 ====" +tail -n 150 ./pacifica_test/slave1/log/pika.INFO + +echo -e "\n==== 从节点 WARNING 日志 ====" +tail -n 150 ./pacifica_test/slave1/log/pika.WARNING \ No newline at end of file diff --git a/tests/integration/start_master_and_slave.sh b/tests/integration/start_master_and_slave.sh index d3d0f1257d..5517ade141 100755 --- a/tests/integration/start_master_and_slave.sh +++ b/tests/integration/start_master_and_slave.sh @@ -1,135 +1,135 @@ -#!/bin/bash -# This script is used by .github/workflows/pika.yml, Do not modify this file unless you know what you are doing. -# it's used to start pika master and slave, running path: build -cp ../conf/pika.conf ./pika_single.conf -cp ../conf/pika.conf ./pika_master.conf -cp ../conf/pika.conf ./pika_slave.conf -cp ../conf/pika.conf ./pika_rename.conf -cp ../conf/pika.conf ./pika_master_rename.conf -cp ../conf/pika.conf ./pika_slave_rename.conf -cp ../conf/pika.conf ./pika_acl_both_password.conf -cp ../conf/pika.conf ./pika_acl_only_admin_password.conf -cp ../conf/pika.conf ./pika_has_other_acl_user.conf -# Create folders for storing data on the primary and secondary nodes -mkdir master_data -mkdir slave_data -# Example Change the location for storing data on primary and secondary nodes in the configuration file -sed -i.bak \ - -e 's|databases : 1|databases : 2|' \ - -e 's|#daemonize : yes|daemonize : yes|' \ - -e 's|timeout : 60|timeout : 500|' ./pika_single.conf - -sed -i.bak \ - -e 's|databases : 1|databases : 2|' \ - -e 's|port : 9221|port : 9241|' \ - -e 's|log-path : ./log/|log-path : ./master_data/log/|' \ - -e 's|db-path : ./db/|db-path : ./master_data/db/|' \ - -e 's|dump-path : ./dump/|dump-path : ./master_data/dump/|' \ - -e 's|pidfile : ./pika.pid|pidfile : ./master_data/pika.pid|' \ - -e 's|db-sync-path : ./dbsync/|db-sync-path : ./master_data/dbsync/|' \ - -e 's|#daemonize : yes|daemonize : yes|' \ - -e 's|timeout : 60|timeout : 500|' ./pika_master.conf - -sed -i.bak \ - -e 's|databases : 1|databases : 2|' \ - -e 's|port : 9221|port : 9231|' \ - -e 's|log-path : ./log/|log-path : ./slave_data/log/|' \ - -e 's|db-path : ./db/|db-path : ./slave_data/db/|' \ - -e 's|dump-path : ./dump/|dump-path : ./slave_data/dump/|' \ - -e 's|pidfile : ./pika.pid|pidfile : ./slave_data/pika.pid|' \ - -e 's|db-sync-path : ./dbsync/|db-sync-path : ./slave_data/dbsync/|' \ - -e 's|#daemonize : yes|daemonize : yes|' \ - -e 's|timeout : 60|timeout : 500|' ./pika_slave.conf - -sed -i.bak \ - -e 's|# rename-command : FLUSHALL 360flushall|rename-command : FLUSHALL 360flushall|' \ - -e 's|# rename-command : FLUSHDB 360flushdb|rename-command : FLUSHDB 360flushdb|' \ - -e 's|databases : 1|databases : 2|' \ - -e 's|port : 9221|port : 9251|' \ - -e 's|log-path : ./log/|log-path : ./rename_data/log/|' \ - -e 's|db-path : ./db/|db-path : ./rename_data/db/|' \ - -e 's|dump-path : ./dump/|dump-path : ./rename_data/dump/|' \ - -e 's|pidfile : ./pika.pid|pidfile : ./rename_data/pika.pid|' \ - -e 's|db-sync-path : ./dbsync/|db-sync-path : ./rename_data/dbsync/|' \ - -e 's|#daemonize : yes|daemonize : yes|' \ - -e 's|timeout : 60|timeout : 500|' ./pika_rename.conf - -sed -i.bak \ - -e 's|requirepass :|requirepass : requirepass|' \ - -e 's|masterauth :|masterauth : requirepass|' \ - -e 's|# userpass :|userpass : userpass|' \ - -e 's|# userblacklist :|userblacklist : flushall,flushdb|' \ - -e 's|port : 9221|port : 9261|' \ - -e 's|log-path : ./log/|log-path : ./acl1_data/log/|' \ - -e 's|db-path : ./db/|db-path : ./acl1_data/db/|' \ - -e 's|dump-path : ./dump/|dump-path : ./acl1_data/dump/|' \ - -e 's|pidfile : ./pika.pid|pidfile : ./acl1_data/pika.pid|' \ - -e 's|db-sync-path : ./dbsync/|db-sync-path : ./acl1_data/dbsync/|' \ - -e 's|#daemonize : yes|daemonize : yes|' \ - -e 's|timeout : 60|timeout : 500|' ./pika_acl_both_password.conf - -sed -i.bak \ - -e 's|requirepass :|requirepass : requirepass|' \ - -e 's|masterauth :|masterauth : requirepass|' \ - -e 's|# userblacklist :|userblacklist : flushall,flushdb|' \ - -e 's|port : 9221|port : 9271|' \ - -e 's|log-path : ./log/|log-path : ./acl2_data/log/|' \ - -e 's|db-path : ./db/|db-path : ./acl2_data/db/|' \ - -e 's|dump-path : ./dump/|dump-path : ./acl2_data/dump/|' \ - -e 's|pidfile : ./pika.pid|pidfile : ./acl2_data/pika.pid|' \ - -e 's|db-sync-path : ./dbsync/|db-sync-path : ./acl2_data/dbsync/|' \ - -e 's|#daemonize : yes|daemonize : yes|' \ - -e 's|timeout : 60|timeout : 500|' ./pika_acl_only_admin_password.conf - -sed -i.bak \ - -e 's|requirepass :|requirepass : requirepass|' \ - -e 's|masterauth :|masterauth : requirepass|' \ - -e 's|# userpass :|userpass : userpass|' \ - -e 's|# userblacklist :|userblacklist : flushall,flushdb|' \ - -e 's|port : 9221|port : 9281|' \ - -e 's|log-path : ./log/|log-path : ./acl3_data/log/|' \ - -e 's|db-path : ./db/|db-path : ./acl3_data/db/|' \ - -e 's|dump-path : ./dump/|dump-path : ./acl3_data/dump/|' \ - -e 's|pidfile : ./pika.pid|pidfile : ./acl3_data/pika.pid|' \ - -e 's|db-sync-path : ./dbsync/|db-sync-path : ./acl3_data/dbsync/|' \ - -e 's|#daemonize : yes|daemonize : yes|' \ - -e 's|timeout : 60|timeout : 500|' ./pika_has_other_acl_user.conf -echo -e '\nuser : limit on >limitpass ~* +@all &*' >> ./pika_has_other_acl_user.conf - -sed -i '' \ - -e 's|# rename-command : FLUSHDB 360flushdb|rename-command : FLUSHDB 360flushdb|' \ - -e 's|port : 9221|port : 9291|' \ - -e 's|log-path : ./log/|log-path : ./master_rename_data/log/|' \ - -e 's|db-path : ./db/|db-path : ./master_rename_data/db/|' \ - -e 's|dump-path : ./dump/|dump-path : ./master_rename_data/dump/|' \ - -e 's|pidfile : ./pika.pid|pidfile : ./master_rename_data/pika.pid|' \ - -e 's|db-sync-path : ./dbsync/|db-sync-path : ./master_rename_data/dbsync/|' \ - -e 's|#daemonize : yes|daemonize : yes|' \ - -e 's|timeout : 60|timeout : 500|' ./pika_master_rename.conf - -sed -i '' \ - -e 's|# rename-command : FLUSHDB 360flushdb|rename-command : FLUSHDB 360flushdb|' \ - -e 's|port : 9221|port : 9301|' \ - -e 's|log-path : ./log/|log-path : ./slave_rename_data/log/|' \ - -e 's|db-path : ./db/|db-path : ./slave_rename_data/db/|' \ - -e 's|dump-path : ./dump/|dump-path : ./slave_rename_data/dump/|' \ - -e 's|pidfile : ./pika.pid|pidfile : ./slave_rename_data/pika.pid|' \ - -e 's|db-sync-path : ./dbsync/|db-sync-path : ./slave_rename_data/dbsync/|' \ - -e 's|#daemonize : yes|daemonize : yes|' \ - -e 's|timeout : 60|timeout : 500|' ./pika_slave_rename.conf - -# Start three nodes -./pika -c ./pika_single.conf -./pika -c ./pika_master.conf -./pika -c ./pika_slave.conf -./pika -c ./pika_rename.conf -./pika -c ./pika_acl_both_password.conf -./pika -c ./pika_acl_only_admin_password.conf -./pika -c ./pika_has_other_acl_user.conf -./pika -c ./pika_master_rename.conf -./pika -c ./pika_slave_rename.conf -#ensure both master and slave are ready -sleep 10 +# #!/bin/bash +# # This script is used by .github/workflows/pika.yml, Do not modify this file unless you know what you are doing. +# # it's used to start pika master and slave, running path: build +# cp ../conf/pika.conf ./pika_single.conf +# cp ../conf/pika.conf ./pika_master.conf +# cp ../conf/pika.conf ./pika_slave.conf +# cp ../conf/pika.conf ./pika_rename.conf +# cp ../conf/pika.conf ./pika_master_rename.conf +# cp ../conf/pika.conf ./pika_slave_rename.conf +# cp ../conf/pika.conf ./pika_acl_both_password.conf +# cp ../conf/pika.conf ./pika_acl_only_admin_password.conf +# cp ../conf/pika.conf ./pika_has_other_acl_user.conf +# # Create folders for storing data on the primary and secondary nodes +# mkdir master_data +# mkdir slave_data +# # Example Change the location for storing data on primary and secondary nodes in the configuration file +# sed -i.bak \ +# -e 's|databases : 1|databases : 2|' \ +# -e 's|#daemonize : yes|daemonize : yes|' \ +# -e 's|timeout : 60|timeout : 500|' ./pika_single.conf + +# sed -i.bak \ +# -e 's|databases : 1|databases : 2|' \ +# -e 's|port : 9221|port : 9241|' \ +# -e 's|log-path : ./log/|log-path : ./master_data/log/|' \ +# -e 's|db-path : ./db/|db-path : ./master_data/db/|' \ +# -e 's|dump-path : ./dump/|dump-path : ./master_data/dump/|' \ +# -e 's|pidfile : ./pika.pid|pidfile : ./master_data/pika.pid|' \ +# -e 's|db-sync-path : ./dbsync/|db-sync-path : ./master_data/dbsync/|' \ +# -e 's|#daemonize : yes|daemonize : yes|' \ +# -e 's|timeout : 60|timeout : 500|' ./pika_master.conf + +# sed -i.bak \ +# -e 's|databases : 1|databases : 2|' \ +# -e 's|port : 9221|port : 9231|' \ +# -e 's|log-path : ./log/|log-path : ./slave_data/log/|' \ +# -e 's|db-path : ./db/|db-path : ./slave_data/db/|' \ +# -e 's|dump-path : ./dump/|dump-path : ./slave_data/dump/|' \ +# -e 's|pidfile : ./pika.pid|pidfile : ./slave_data/pika.pid|' \ +# -e 's|db-sync-path : ./dbsync/|db-sync-path : ./slave_data/dbsync/|' \ +# -e 's|#daemonize : yes|daemonize : yes|' \ +# -e 's|timeout : 60|timeout : 500|' ./pika_slave.conf + +# sed -i.bak \ +# -e 's|# rename-command : FLUSHALL 360flushall|rename-command : FLUSHALL 360flushall|' \ +# -e 's|# rename-command : FLUSHDB 360flushdb|rename-command : FLUSHDB 360flushdb|' \ +# -e 's|databases : 1|databases : 2|' \ +# -e 's|port : 9221|port : 9251|' \ +# -e 's|log-path : ./log/|log-path : ./rename_data/log/|' \ +# -e 's|db-path : ./db/|db-path : ./rename_data/db/|' \ +# -e 's|dump-path : ./dump/|dump-path : ./rename_data/dump/|' \ +# -e 's|pidfile : ./pika.pid|pidfile : ./rename_data/pika.pid|' \ +# -e 's|db-sync-path : ./dbsync/|db-sync-path : ./rename_data/dbsync/|' \ +# -e 's|#daemonize : yes|daemonize : yes|' \ +# -e 's|timeout : 60|timeout : 500|' ./pika_rename.conf + +# sed -i.bak \ +# -e 's|requirepass :|requirepass : requirepass|' \ +# -e 's|masterauth :|masterauth : requirepass|' \ +# -e 's|# userpass :|userpass : userpass|' \ +# -e 's|# userblacklist :|userblacklist : flushall,flushdb|' \ +# -e 's|port : 9221|port : 9261|' \ +# -e 's|log-path : ./log/|log-path : ./acl1_data/log/|' \ +# -e 's|db-path : ./db/|db-path : ./acl1_data/db/|' \ +# -e 's|dump-path : ./dump/|dump-path : ./acl1_data/dump/|' \ +# -e 's|pidfile : ./pika.pid|pidfile : ./acl1_data/pika.pid|' \ +# -e 's|db-sync-path : ./dbsync/|db-sync-path : ./acl1_data/dbsync/|' \ +# -e 's|#daemonize : yes|daemonize : yes|' \ +# -e 's|timeout : 60|timeout : 500|' ./pika_acl_both_password.conf + +# sed -i.bak \ +# -e 's|requirepass :|requirepass : requirepass|' \ +# -e 's|masterauth :|masterauth : requirepass|' \ +# -e 's|# userblacklist :|userblacklist : flushall,flushdb|' \ +# -e 's|port : 9221|port : 9271|' \ +# -e 's|log-path : ./log/|log-path : ./acl2_data/log/|' \ +# -e 's|db-path : ./db/|db-path : ./acl2_data/db/|' \ +# -e 's|dump-path : ./dump/|dump-path : ./acl2_data/dump/|' \ +# -e 's|pidfile : ./pika.pid|pidfile : ./acl2_data/pika.pid|' \ +# -e 's|db-sync-path : ./dbsync/|db-sync-path : ./acl2_data/dbsync/|' \ +# -e 's|#daemonize : yes|daemonize : yes|' \ +# -e 's|timeout : 60|timeout : 500|' ./pika_acl_only_admin_password.conf + +# sed -i.bak \ +# -e 's|requirepass :|requirepass : requirepass|' \ +# -e 's|masterauth :|masterauth : requirepass|' \ +# -e 's|# userpass :|userpass : userpass|' \ +# -e 's|# userblacklist :|userblacklist : flushall,flushdb|' \ +# -e 's|port : 9221|port : 9281|' \ +# -e 's|log-path : ./log/|log-path : ./acl3_data/log/|' \ +# -e 's|db-path : ./db/|db-path : ./acl3_data/db/|' \ +# -e 's|dump-path : ./dump/|dump-path : ./acl3_data/dump/|' \ +# -e 's|pidfile : ./pika.pid|pidfile : ./acl3_data/pika.pid|' \ +# -e 's|db-sync-path : ./dbsync/|db-sync-path : ./acl3_data/dbsync/|' \ +# -e 's|#daemonize : yes|daemonize : yes|' \ +# -e 's|timeout : 60|timeout : 500|' ./pika_has_other_acl_user.conf +# echo -e '\nuser : limit on >limitpass ~* +@all &*' >> ./pika_has_other_acl_user.conf + +# sed -i '' \ +# -e 's|# rename-command : FLUSHDB 360flushdb|rename-command : FLUSHDB 360flushdb|' \ +# -e 's|port : 9221|port : 9291|' \ +# -e 's|log-path : ./log/|log-path : ./master_rename_data/log/|' \ +# -e 's|db-path : ./db/|db-path : ./master_rename_data/db/|' \ +# -e 's|dump-path : ./dump/|dump-path : ./master_rename_data/dump/|' \ +# -e 's|pidfile : ./pika.pid|pidfile : ./master_rename_data/pika.pid|' \ +# -e 's|db-sync-path : ./dbsync/|db-sync-path : ./master_rename_data/dbsync/|' \ +# -e 's|#daemonize : yes|daemonize : yes|' \ +# -e 's|timeout : 60|timeout : 500|' ./pika_master_rename.conf + +# sed -i '' \ +# -e 's|# rename-command : FLUSHDB 360flushdb|rename-command : FLUSHDB 360flushdb|' \ +# -e 's|port : 9221|port : 9301|' \ +# -e 's|log-path : ./log/|log-path : ./slave_rename_data/log/|' \ +# -e 's|db-path : ./db/|db-path : ./slave_rename_data/db/|' \ +# -e 's|dump-path : ./dump/|dump-path : ./slave_rename_data/dump/|' \ +# -e 's|pidfile : ./pika.pid|pidfile : ./slave_rename_data/pika.pid|' \ +# -e 's|db-sync-path : ./dbsync/|db-sync-path : ./slave_rename_data/dbsync/|' \ +# -e 's|#daemonize : yes|daemonize : yes|' \ +# -e 's|timeout : 60|timeout : 500|' ./pika_slave_rename.conf + +# # Start three nodes +# ./pika -c ./pika_single.conf +# ./pika -c ./pika_master.conf +# ./pika -c ./pika_slave.conf +# ./pika -c ./pika_rename.conf +# ./pika -c ./pika_acl_both_password.conf +# ./pika -c ./pika_acl_only_admin_password.conf +# ./pika -c ./pika_has_other_acl_user.conf +# ./pika -c ./pika_master_rename.conf +# ./pika -c ./pika_slave_rename.conf +# #ensure both master and slave are ready +# sleep 10 # 创建PacificA一致性测试的数据目录 mkdir -p pacifica_test/master From 46a398dbe59bacbedf81000904fdfa27e6786a8c Mon Sep 17 00:00:00 2001 From: YuCai18 <1512875381@qq.com> Date: Tue, 26 Aug 2025 17:02:45 +0800 Subject: [PATCH 2/5] Solve the bug that the slave node also generates Binlog --- src/pika_repl_bgworker.cc | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/src/pika_repl_bgworker.cc b/src/pika_repl_bgworker.cc index 048d06ba16..d465431405 100644 --- a/src/pika_repl_bgworker.cc +++ b/src/pika_repl_bgworker.cc @@ -215,15 +215,11 @@ void PikaReplBgWorker::HandleBGWorkerWriteBinlog(void* arg) { if (only_keepalive) { ack_end = LogOffset(); } else { - LogOffset productor_status; - // Reply Ack to master immediately - std::shared_ptr logger = db->Logger(); - logger->GetProducerStatus(&productor_status.b_offset.filenum, &productor_status.b_offset.offset, - &productor_status.l_offset.term, &productor_status.l_offset.index); - ack_end = productor_status; - ack_end.l_offset.term = pb_end.l_offset.term; + // The slave node uses the binlog offset sent by the master node as ACK + ack_end = pb_end; // slave nodes batch Binlog flush disk + std::shared_ptr logger = db->Logger(); logger->Sync(); } @@ -266,7 +262,7 @@ int PikaReplBgWorker::HandleWriteBinlog(net::RedisParser* parser, const net::Red return -1; } if(db->GetISConsistency()){ - db->AppendSlaveEntries(c_ptr, worker->binlog_item_); + PikaReplBgWorker::WriteDBInSyncWay(c_ptr); }else{ db->ConsensusProcessLeaderLog(c_ptr, worker->binlog_item_); } From aea793b12e3a3991377f7633906fd3124cf2a9fe Mon Sep 17 00:00:00 2001 From: YuCai18 <1512875381@qq.com> Date: Fri, 29 Aug 2025 16:03:31 +0800 Subject: [PATCH 3/5] New Architecture --- include/pika_command_collector.h | 108 +---- include/pika_command_queue.h | 99 ++++ include/pika_consensus.h | 6 +- include/pika_define.h | 8 - include/pika_rm.h | 56 ++- include/pika_stable_log.h | 6 - src/pika_client_conn.cc | 40 +- src/pika_command_collector.cc | 489 +------------------- src/pika_command_queue.cc | 107 +++++ src/pika_consensus.cc | 254 +---------- src/pika_repl_bgworker.cc | 72 +-- src/pika_repl_server.cc | 25 +- src/pika_rm.cc | 750 +++++++++++++++++++++---------- src/pika_server.cc | 23 +- src/pika_stable_log.cc | 15 - tests/integration/clean_start.sh | 2 +- 16 files changed, 876 insertions(+), 1184 deletions(-) create mode 100644 include/pika_command_queue.h create mode 100644 src/pika_command_queue.cc diff --git a/include/pika_command_collector.h b/include/pika_command_collector.h index 4e3fabcaca..de4e19dafb 100644 --- a/include/pika_command_collector.h +++ b/include/pika_command_collector.h @@ -36,7 +36,7 @@ * 3. Send commands in batches to the consensus coordinator with batch-level synchronization * 4. Support asynchronous callback notification of command processing results * 5. Track performance metrics for batch processing - * 6. Provide intelligent retry mechanisms for failed batches + */ class PikaCommandCollector { public: @@ -46,14 +46,13 @@ class PikaCommandCollector { /** * @brief constructor * @param coordinator consensus coordinator reference - * @param batch_size batch size (number of commands) - * @param batch_max_wait_time forced flush interval (milliseconds) + * @param batch_max_wait_time maximum wait time in milliseconds */ // Constructor with raw pointer (original) - PikaCommandCollector(ConsensusCoordinator* coordinator, size_t batch_size = 100, int batch_max_wait_time = 5); + PikaCommandCollector(ConsensusCoordinator* coordinator, int batch_max_wait_time = 5); // Constructor with shared_ptr (for compatibility with make_shared calls) - PikaCommandCollector(std::shared_ptr coordinator, size_t batch_size = 100, int batch_max_wait_time = 5); + PikaCommandCollector(std::shared_ptr coordinator, int batch_max_wait_time = 5); ~PikaCommandCollector(); @@ -65,31 +64,6 @@ class PikaCommandCollector { */ bool AddCommand(const std::shared_ptr& cmd_ptr, CommandCallback callback); - /** - * @brief Called periodically by external systems to process batches - * @param force Force processing even if batch is not full or timeout not reached - * @return Number of commands processed - */ - - /** - * @brief Immediately process all currently collected commands - * @return The number of commands processed - */ - size_t FlushCommands(bool force = false); - - - /** - * @brief Get the current number of pending commands - * @return number of commands - */ - size_t PendingCommands() const; - - /** - * @brief Set the batch size - * @param batch_size batch size - */ - void SetBatchSize(size_t batch_size); - /** * @brief Set the batch max wait time * @param batch_max_wait_time maximum wait time in milliseconds @@ -101,91 +75,17 @@ class PikaCommandCollector { * @return Pair of (total_processed_commands, total_batches) */ std::pair GetBatchStats() const; - - /** - * @brief Get average batch processing time in milliseconds - * @return Average processing time or nullopt if no batches processed - */ - std::optional GetAverageBatchTime() const; - - private: - - /** - * @brief batch processing command - * @param batch command batch - * @return Whether the processing is successful - */ - pstd::Status ProcessBatch(const std::vector>& commands, - const std::vector& callbacks); - - /** - * @brief Check for conflicts based on command type and key name - * @param cmd_ptr command pointer - * @return true if there is a conflict (should be replaced), false if there is no conflict - */ - bool CheckConflict(const std::shared_ptr& cmd_ptr) const; - - /** - * @brief Handle key conflicts and remove conflicting commands - * @param cmd_ptr new command - */ - void HandleConflict(const std::shared_ptr& cmd_ptr); - - /** - * @brief Retry batch processing commands - * @param commands List of commands to retry - * @param callbacks Corresponding callback function list - * @param priority Priority level for the retry (higher means more urgent) - * @return Whether the commands were successfully requeued - */ - bool RetryBatch(const std::vector>& commands, - const std::vector& callbacks, - int priority = 100); private: //Consensus coordinator reference ConsensusCoordinator* coordinator_; // Batch processing configuration - std::atomic batch_size_; std::atomic batch_max_wait_time_; - // Retry configuration - std::atomic max_retry_attempts_{3}; - std::atomic retry_backoff_ms_{50}; - - // Command collection and processing - mutable std::mutex mutex_; - - // Pending command queue and callbacks - std::list, CommandCallback>> pending_commands_; - - // Priority queue for retries - std::deque>, std::vector>> retry_queue_; - - // Command key mapping, used to handle same-key conflicts - std::unordered_map, CommandCallback>>::iterator> key_map_; - // Batch statistics std::atomic total_processed_{0}; std::atomic total_batches_{0}; - std::atomic total_retries_{0}; - std::atomic total_conflicts_{0}; - std::atomic total_batch_time_ms_{0}; - std::chrono::time_point batch_start_time_; - - // Performance tracking - struct BatchMetrics { - uint64_t batch_size; - uint64_t processing_time_ms; - uint64_t wait_time_ms; - bool successful; - }; - - // Circular buffer for recent batch metrics - static constexpr size_t kMetricsBufferSize = 100; - std::vector recent_metrics_; - std::mutex metrics_mutex_; }; #endif // PIKA_COMMAND_COLLECTOR_H_ \ No newline at end of file diff --git a/include/pika_command_queue.h b/include/pika_command_queue.h new file mode 100644 index 0000000000..62574fcd2d --- /dev/null +++ b/include/pika_command_queue.h @@ -0,0 +1,99 @@ +// Copyright (c) 2015-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 PIKA_COMMAND_QUEUE_H_ +#define PIKA_COMMAND_QUEUE_H_ + +#include +#include +#include +#include +#include +#include +#include +#include + +#include "pstd/include/pstd_mutex.h" +#include "pstd/include/env.h" +#include "include/pika_command.h" +#include "include/pika_define.h" +#include "pstd/include/env.h" +#include "include/pika_define.h" + +// Callback function type for command completion notification +using CommandCallback = std::function; + +// Structure representing a batch of commands +struct CommandBatch { + std::vector> commands; + std::vector callbacks; + uint64_t batch_id; + uint64_t create_time; + std::string db_name; + std::vector binlog_offsets; // Binlog offsets for each command + + CommandBatch(const std::vector>& cmds, + const std::vector& cbs, + const std::string& db) + : commands(cmds), callbacks(cbs), db_name(db) { + static std::atomic next_id{1}; + batch_id = next_id.fetch_add(1); + create_time = pstd::NowMicros(); + } + + bool Empty() const { + return commands.empty(); + } + + size_t Size() const { + return commands.size(); + } +}; + +// New structure to group multiple CommandBatches for RocksDB processing +struct BatchGroup { + std::vector> batches; + LogOffset end_offset; // Only store the final offset of the last batch + BatchGroup() = default; + BatchGroup(const std::vector>& batches, + const LogOffset& final_offset) + : batches(batches), end_offset(final_offset) {} + bool Empty() const { return batches.empty(); } + size_t BatchCount() const { return batches.size(); } +}; + +// Thread-safe command queue for batched command processing +class CommandQueue { +public: + explicit CommandQueue(size_t max_size); + ~CommandQueue(); + + // Enqueue a command batch (blocking if queue is full) + bool EnqueueBatch(std::shared_ptr batch); + + // Dequeue a command batch (blocking if queue is empty) + std::shared_ptr DequeueBatch(); + + // Dequeue all available batches (non-blocking) + std::vector> DequeueAllBatches(); + + // Get current queue size + size_t Size() const; + + // Check if queue is empty + bool Empty() const; + + // Shutdown the queue + void Shutdown(); + +private: + std::queue> cmd_queue_; + mutable std::mutex queue_mutex_; + std::condition_variable queue_cv_; + size_t max_size_; + std::atomic shutdown_{false}; +}; + +#endif // PIKA_COMMAND_QUEUE_H_ \ No newline at end of file diff --git a/include/pika_consensus.h b/include/pika_consensus.h index 41a36d3e4e..f13a147054 100644 --- a/include/pika_consensus.h +++ b/include/pika_consensus.h @@ -155,8 +155,6 @@ class ConsensusCoordinator { pstd::Status Reset(const LogOffset& offset); pstd::Status ProposeLog(const std::shared_ptr& cmd_ptr); - // Batch processing of commands - pstd::Status BatchProposeLog(const std::vector>& cmd_ptrs, std::vector* offsets); pstd::Status UpdateSlave(const std::string& ip, int port, const LogOffset& start, const LogOffset& end); pstd::Status AddSlaveNode(const std::string& ip, int port, int session_id); pstd::Status RemoveSlaveNode(const std::string& ip, int port); @@ -268,8 +266,6 @@ class ConsensusCoordinator { void BatchInternalApplyFollower(const std::vector>& cmd_ptrs); pstd::Status ProcessCoordination(); - // Batch operations for slave entries - pstd::Status BatchPersistAppendBinlog(const std::vector>& cmd_ptrs,const std::vector& attributes,std::vector* offsets); LogOffset GetCommittedId() { std::lock_guard l(committed_id_rwlock_); return committed_id_; @@ -300,7 +296,7 @@ class ConsensusCoordinator { private: std::shared_mutex is_consistency_rwlock_; - bool is_consistency_ = true; + bool is_consistency_ = false; std::shared_mutex committed_id_rwlock_; LogOffset committed_id_ = LogOffset(); std::atomic notification_counter_{0}; diff --git a/include/pika_define.h b/include/pika_define.h index 8605f2a06b..6568685a33 100644 --- a/include/pika_define.h +++ b/include/pika_define.h @@ -364,14 +364,6 @@ const int64_t kPoolSize = 1073741824; const std::string kBinlogPrefix = "write2file"; const size_t kBinlogPrefixLen = 10; -/* - * PIKA_BATCH_MAGIC: Core identifier for binlog batch processing. - * - Master: Prefixes batched binlogs with this magic in SendBinlog - * - Slave: Detects this magic in HandleBGWorkerWriteBinlog - * to switch between batch and single-binlog parsing modes. - */ -const uint32_t PIKA_BATCH_MAGIC = 0x42544348; // "BTCH" in ASCII - const std::string kPikaMeta = "meta"; const std::string kManifest = "manifest"; const std::string kContext = "context"; diff --git a/include/pika_rm.h b/include/pika_rm.h index e72bad4992..7db1bbf322 100644 --- a/include/pika_rm.h +++ b/include/pika_rm.h @@ -12,6 +12,10 @@ #include #include #include +#include +#include +#include +#include #include "pstd/include/pstd_status.h" @@ -23,6 +27,7 @@ #include "include/pika_stable_log.h" #include "include/rsync_client.h" #include "include/pika_command_collector.h" +#include "include/pika_command_queue.h" #define kBinlogSendPacketNum 40 #define kBinlogSendBatchNum 100 @@ -70,7 +75,6 @@ class SyncMasterDB : public SyncDB { // consensus use pstd::Status ConsensusUpdateSlave(const std::string& ip, int port, const LogOffset& start, const LogOffset& end); pstd::Status ConsensusProposeLog(const std::shared_ptr& cmd_ptr); - pstd::Status ConsensusBatchProposeLog(const std::vector>& cmd_ptrs, std::vector* offsets); pstd::Status ConsensusProcessLeaderLog(const std::shared_ptr& cmd_ptr, const BinlogItem& attribute); LogOffset ConsensusCommittedIndex(); @@ -116,12 +120,11 @@ class SyncMasterDB : public SyncDB { LogOffset GetPreparedId(); LogOffset GetCommittedId(); pstd::Status AppendSlaveEntries(const std::shared_ptr& cmd_ptr, const BinlogItem& attribute); - pstd::Status BatchAppendSlaveEntries(const std::vector>& cmd_ptrs, const std::vector& attributes); pstd::Status AppendCandidateBinlog(const std::string& ip, int port, const LogOffset& offset); pstd::Status UpdateCommittedID(); pstd::Status CommitAppLog(const LogOffset& master_committed_id); pstd::Status Truncate(const LogOffset& offset); - pstd::Status WaitForSlaveAcks(const LogOffset& target_offset, int timeout_ms); + // pstd::Status WaitForSlaveAcks(const LogOffset& target_offset, int timeout_ms); }; class SyncSlaveDB : public SyncDB { @@ -171,7 +174,6 @@ class PikaReplicaManager { pstd::Status SendMetaSyncRequest(); pstd::Status SendRemoveSlaveNodeRequest(const std::string& table); pstd::Status SendTrySyncRequest(const std::string& db_name); - std::shared_ptr GetConsensusCoordinator(const std::string& db_name); pstd::Status SendDBSyncRequest(const std::string& db_name); pstd::Status SendBinlogSyncAckRequest(const std::string& table, const LogOffset& ack_start, const LogOffset& ack_end, bool is_first_send = false); @@ -242,6 +244,14 @@ class PikaReplicaManager { return pika_repl_client_->GetUnfinishedAsyncWriteDBTaskCount(db_name); } + // Command Queue related methods + void EnqueueCommandBatch(std::shared_ptr batch); + std::shared_ptr DequeueCommandBatch(); + size_t GetCommandQueueSize() const; + bool IsCommandQueueEmpty() const; + // CommittedID notification for RocksDB thread + void NotifyCommittedID(const LogOffset& committed_id); + private: void InitDB(); pstd::Status SelectLocalIp(const std::string& remote_ip, int remote_port, std::string* local_ip); @@ -263,9 +273,6 @@ class PikaReplicaManager { // client for replica std::unique_ptr pika_repl_client_; std::unique_ptr pika_repl_server_; - - // one-shot switch to force immediate send on next SendBinlog - std::atomic immediate_send_once_{false}; // Condition variable for signaling when the write queue has new items pstd::CondVar write_queue_cv_; @@ -273,6 +280,41 @@ class PikaReplicaManager { std::shared_mutex is_consistency_rwlock_; bool is_consistency_ = true; std::shared_mutex committed_id_rwlock_; + + // Command queue for collected batches + std::unique_ptr command_queue_; + + // Background thread for processing command queue + std::unique_ptr command_queue_thread_; + std::atomic command_queue_running_{false}; + std::mutex command_queue_mutex_; + std::condition_variable command_queue_cv_; + + // RocksDB background thread for Put operations and client responses + std::unique_ptr rocksdb_back_thread_; + std::atomic rocksdb_thread_running_{false}; + std::mutex rocksdb_thread_mutex_; + std::condition_variable rocksdb_thread_cv_; + + // Pending batch groups waiting for CommittedID + std::queue> pending_batch_groups_; + std::mutex pending_batch_groups_mutex_; + + // Last committed ID for RocksDB thread processing + LogOffset last_committed_id_; + std::mutex last_committed_id_mutex_; + + // Background thread processing methods + void StartCommandQueueThread(); + void StopCommandQueueThread(); + void CommandQueueLoop(); + void ProcessCommandBatches(const std::vector>& batches); + + // RocksDB background thread methods + void StartRocksDBThread(); + void StopRocksDBThread(); + void RocksDBThreadLoop(); + size_t ProcessCommittedBatchGroups(const LogOffset& committed_id); }; #endif // PIKA_RM_H diff --git a/include/pika_stable_log.h b/include/pika_stable_log.h index 8cb6dc6267..300e0d0fc5 100644 --- a/include/pika_stable_log.h +++ b/include/pika_stable_log.h @@ -11,9 +11,6 @@ #include "include/pika_binlog.h" -// Forward declaration to avoid circular dependency -class ConsensusCoordinator; - class StableLog : public std::enable_shared_from_this { public: StableLog(std::string table_name, std::string log_path); @@ -28,9 +25,6 @@ class StableLog : public std::enable_shared_from_this { std::shared_lock l(offset_rwlock_); return first_offset_; } - // Return a direct reference to the ConsensusCoordinator without copying - std::shared_ptr coordinator(); - void set_coordinator(std::shared_ptr coordinator); // Need to hold binlog lock pstd::Status TruncateTo(const LogOffset& offset); diff --git a/src/pika_client_conn.cc b/src/pika_client_conn.cc index e768cefa6f..0ad970311c 100644 --- a/src/pika_client_conn.cc +++ b/src/pika_client_conn.cc @@ -219,8 +219,44 @@ std::shared_ptr PikaClientConn::DoCmd(const PikaCmdArgsType& argv, const st // Perform some operations rocksdb::get_perf_context()->Reset(); - // Process Command - c_ptr->Execute(); + + // Process Command - route write commands through CommandCollector for batching + if (c_ptr->is_write() && g_pika_conf->command_batch_enabled()) { + // Get the appropriate SyncMasterDB for command batching + auto sync_db = g_pika_rm->GetSyncMasterDBByName(DBInfo(c_ptr->db_name())); + if (sync_db) { + auto command_collector = sync_db->GetCommandCollector(); + if (command_collector) { + // Create callback to handle command completion + auto callback = [this, c_ptr](const LogOffset& offset, pstd::Status status) { + if (status.ok()) { + // Command was successfully processed through the pipeline + LOG(INFO) << "Command " << c_ptr->name() << " completed via CommandCollector"; + } else { + // Set error response + c_ptr->res().SetRes(CmdRes::kErrOther, "Command processing failed: " + status.ToString()); + LOG(ERROR) << "Command " << c_ptr->name() << " failed in CommandCollector: " << status.ToString(); + } + }; + + // Add command to collector for batch processing + bool added = command_collector->AddCommand(c_ptr, callback); + if (!added) { + LOG(WARNING) << "Failed to add command " << c_ptr->name() << " to CommandCollector, executing directly"; + c_ptr->Execute(); + } + } else { + LOG(WARNING) << "CommandCollector not available, executing command directly"; + c_ptr->Execute(); + } + } else { + LOG(WARNING) << "SyncMasterDB not found for " << c_ptr->db_name() << ", executing command directly"; + c_ptr->Execute(); + } + } else { + // Non-write commands or batching disabled - execute directly + c_ptr->Execute(); + } time_stat_->process_done_ts_ = pstd::NowMicros(); auto cmdstat_map = g_pika_cmd_table_manager->GetCommandStatMap(); diff --git a/src/pika_command_collector.cc b/src/pika_command_collector.cc index 89eec53d22..fb206eb9b1 100644 --- a/src/pika_command_collector.cc +++ b/src/pika_command_collector.cc @@ -3,59 +3,31 @@ // 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 +#include #include "include/pika_command_collector.h" -#include "include/pika_conf.h" -#include "include/pika_server.h" #include "include/pika_rm.h" -#include "include/pika_define.h" -#include -#include +#include "include/pika_conf.h" extern std::unique_ptr g_pika_conf; extern std::unique_ptr g_pika_rm; -PikaCommandCollector::PikaCommandCollector(ConsensusCoordinator* coordinator, size_t batch_size, int batch_max_wait_time) - : coordinator_(coordinator), - batch_size_(batch_size), - batch_max_wait_time_(batch_max_wait_time) { - - // Check if coordinator is null - if (!coordinator_) { - LOG(FATAL) << "PikaCommandCollector: ConsensusCoordinator cannot be null! " - << "This usually means SyncMasterDB is not initialized yet."; - return; - } - - LOG(INFO) << "PikaCommandCollector created with batch_size=" << batch_size << ", batch_max_wait_time=" << batch_max_wait_time << "ms"; - - // Initialize metrics buffer - recent_metrics_.reserve(kMetricsBufferSize); +PikaCommandCollector::PikaCommandCollector(ConsensusCoordinator* coordinator, int batch_max_wait_time) + : coordinator_(coordinator), batch_max_wait_time_(batch_max_wait_time) { + LOG(INFO) << "PikaCommandCollector started for DB: " << coordinator_->db_name() + << " with batch_max_wait_time: " << batch_max_wait_time << "ms"; } -// Constructor with shared_ptr (for compatibility with make_shared calls) -PikaCommandCollector::PikaCommandCollector(std::shared_ptr coordinator, size_t batch_size, int batch_max_wait_time) - : coordinator_(coordinator.get()), - batch_size_(batch_size), - batch_max_wait_time_(batch_max_wait_time) { - - // Check if coordinator is null - if (!coordinator_) { - LOG(FATAL) << "PikaCommandCollector: ConsensusCoordinator cannot be null! " - << "This usually means SyncMasterDB is not initialized yet."; - return; - } - - LOG(INFO) << "PikaCommandCollector created from shared_ptr with batch_size=" << batch_size << ", batch_max_wait_time=" << batch_max_wait_time << "ms"; - - // Initialize metrics buffer - recent_metrics_.reserve(kMetricsBufferSize); +PikaCommandCollector::PikaCommandCollector(std::shared_ptr coordinator, int batch_max_wait_time) + : coordinator_(coordinator.get()), batch_max_wait_time_(batch_max_wait_time) { + LOG(INFO) << "PikaCommandCollector started for DB: " << coordinator_->db_name() + << " with batch_max_wait_time: " << batch_max_wait_time << "ms"; } PikaCommandCollector::~PikaCommandCollector() { - // Process any remaining commands - FlushCommands(true); LOG(INFO) << "PikaCommandCollector stopped, processed " << total_processed_.load() - << ", conflicts: " << total_conflicts_.load(); + << " commands, " << total_batches_.load() << " batches"; } bool PikaCommandCollector::AddCommand(const std::shared_ptr& cmd_ptr, CommandCallback callback) { @@ -64,363 +36,24 @@ bool PikaCommandCollector::AddCommand(const std::shared_ptr& cmd_ptr, Comma return false; } - std::lock_guard lock(mutex_); + // Create a single-command batch directly + std::vector> commands = {cmd_ptr}; + std::vector callbacks = {std::move(callback)}; - if (pending_commands_.empty()) { - batch_start_time_ = std::chrono::steady_clock::now(); - } - - // Check if we should immediately flush the batch - bool should_flush = pending_commands_.size() >= static_cast(batch_size_.load()); - if (should_flush) { - FlushCommands(false); - } + std::string db_name = cmd_ptr->db_name().empty() ? g_pika_conf->default_db() : cmd_ptr->db_name(); + auto command_batch = std::make_shared(commands, callbacks, db_name); - // Handle same Key conflict - counts updated inside HandleConflict - HandleConflict(cmd_ptr); + // Enqueue the batch directly to PikaReplicaManager + g_pika_rm->EnqueueCommandBatch(command_batch); - // Add command to queue - pending_commands_.emplace_back(cmd_ptr, std::move(callback)); + // Update statistics + total_processed_.fetch_add(1); + total_batches_.fetch_add(1); - // Update Key Mapping - std::vector keys = cmd_ptr->current_key(); - for (const auto& key : keys) { - key_map_[key] = std::prev(pending_commands_.end()); - } - - + LOG(INFO) << "Added single command " << cmd_ptr->name() << " to CommandQueue"; return true; } -size_t PikaCommandCollector::FlushCommands(bool force) { - std::vector> commands; - std::vector callbacks; - - // Record batch metrics variables - auto batch_start = std::chrono::steady_clock::now(); - uint64_t wait_time_ms = 0; - bool batch_successful = false; - - { - std::lock_guard lock(mutex_); - if (pending_commands_.empty()) { - // Check if there are any retries to process - if (!retry_queue_.empty() && force) { - auto& [pri, cmds, cbs] = retry_queue_.front(); - commands = cmds; - callbacks = cbs; - retry_queue_.pop_front(); - LOG(INFO) << "FlushCommands: Processing retry batch with priority " << pri << ", size: " << commands.size(); - } else { - return 0; - } - } else { - auto now = std::chrono::steady_clock::now(); - auto elapsed_ms = std::chrono::duration_cast(now - batch_start_time_).count(); - wait_time_ms = elapsed_ms; // Record wait time for metrics - - bool should_flush = force || - pending_commands_.size() >= static_cast(batch_size_.load()) || - elapsed_ms > batch_max_wait_time_.load(); - - if (!should_flush) { - return 0; - } - - size_t batch_count = pending_commands_.size(); - if (!force) { - batch_count = std::min(batch_count, static_cast(batch_size_.load())); - } - - commands.reserve(batch_count); - callbacks.reserve(batch_count); - - auto it = pending_commands_.begin(); - for (size_t i = 0; i < batch_count; ++i, ++it) { - commands.push_back(it->first); - callbacks.push_back(std::move(it->second)); - } - - // Clear queue and map - for (const auto& cmd : commands) { - std::vector keys = cmd->current_key(); - for (const auto& key : keys) { - key_map_.erase(key); - } - } - pending_commands_.erase(pending_commands_.begin(), std::next(pending_commands_.begin(), batch_count)); - - if (!pending_commands_.empty()) { - // Reset timer for the next batch - batch_start_time_ = std::chrono::steady_clock::now(); - } - } - } - - size_t batch_size = commands.size(); - if (batch_size > 0) { - LOG(INFO) << "Processing batch of " << batch_size << " commands"; - - auto process_start = std::chrono::steady_clock::now(); - pstd::Status status = ProcessBatch(commands, callbacks); - auto process_end = std::chrono::steady_clock::now(); - - uint64_t processing_time_ms = std::chrono::duration_cast(process_end - process_start).count(); - batch_successful = status.ok(); - - if (!batch_successful) { - LOG(ERROR) << "Error processing command batch: " << status.ToString(); - } else { - LOG(INFO) << "Successfully processed batch in " << processing_time_ms << "ms"; - } - - // Update statistics - total_processed_.fetch_add(batch_size); - total_batches_.fetch_add(1); - total_batch_time_ms_.fetch_add(processing_time_ms); - - // Record batch metrics - { - std::lock_guard metrics_lock(metrics_mutex_); - if (recent_metrics_.size() >= kMetricsBufferSize) { - recent_metrics_.erase(recent_metrics_.begin()); - } - recent_metrics_.push_back({batch_size, processing_time_ms, wait_time_ms, batch_successful}); - } - - // Process any retries if there were failures but we have pending retries - if (!batch_successful) { - std::lock_guard lock(mutex_); - if (!retry_queue_.empty()) { - LOG(INFO) << "FlushCommands: Processing retries due to batch failure"; - // Schedule immediate follow-up flush to process retries - return batch_size + FlushCommands(true); - } - } - } - - return batch_size; -} - -pstd::Status PikaCommandCollector::ProcessBatch( - const std::vector>& commands, - const std::vector& callbacks) { - - if (commands.empty()) { - return pstd::Status::OK(); - } - - // Implement batch processing logic here - // 1. Generate binlogs for each command - // 2. Write binlogs to production queue in batches - // 3. Main node will update memory data structures in batches - // 4. Trigger asynchronous persistence - - // Store the log offset for each command - std::vector offsets; - - // Check if coordinator is valid - if (!coordinator_) { - LOG(ERROR) << "ProcessBatch: ConsensusCoordinator is null"; - return pstd::Status::InvalidArgument("ConsensusCoordinator is null"); - } - - // Get SyncMasterDB and submit commands in batch - DBInfo db_info(coordinator_->db_name()); - auto master_db = g_pika_rm->GetSyncMasterDBByName(db_info); - if (!master_db) { - LOG(ERROR) << "Failed to get SyncMasterDB for " << coordinator_->db_name(); - return pstd::Status::NotFound("SyncMasterDB not found"); - } - - // Submit to consensus coordinator in batch - LOG(INFO) << "ProcessBatch: Processing " << commands.size() << " commands in batch"; - pstd::Status batch_status = master_db->ConsensusBatchProposeLog(commands, &offsets); - - // Log the batch status - if (!batch_status.ok()) { - LOG(WARNING) << "ProcessBatch: Batch operation failed with status: " << batch_status.ToString(); - if (batch_status.IsTimeout()) { - LOG(WARNING) << "ProcessBatch: Timeout occurred, triggering batch retry mechanism"; - - // Get the last command's offset - LogOffset last_offset; - if (!offsets.empty()) { - last_offset = offsets.back(); - } - - // Roll back committed_id on master and all slave nodes - if (last_offset.IsValid()) { - LOG(WARNING) << "ProcessBatch: Rolling back committed_id to before " << last_offset.ToString(); - - // Get current committed_id - LogOffset current_committed_id = master_db->GetCommittedId(); - - // Calculate rollback target committed_id (assuming rollback to the previous batch's committed_id) - // In actual implementation, you may need to adjust the rollback target based on specific situations - LogOffset rollback_target = current_committed_id; - rollback_target.l_offset.index -= commands.size(); // Simple rollback, may need more complex logic in practice - if (rollback_target.l_offset.index < 0) { - rollback_target.l_offset.index = 0; - } - - // Execute rollback operation - LOG(WARNING) << "ProcessBatch: Rolling back from " << current_committed_id.ToString() - << " to " << rollback_target.ToString(); - - // Call rollback API - pstd::Status truncate_status = master_db->Truncate(rollback_target); - if (!truncate_status.ok()) { - LOG(ERROR) << "ProcessBatch: Failed to rollback committed_id: " << truncate_status.ToString(); - } else { - LOG(INFO) << "ProcessBatch: Successfully rolled back committed_id"; - - // Call batch retry mechanism - LOG(INFO) << "ProcessBatch: Triggering batch retry mechanism for " << commands.size() << " commands"; - bool retry_result = RetryBatch(commands, callbacks, 100); // Use high priority for timeouts - if (retry_result) { - LOG(INFO) << "ProcessBatch: Successfully requeued commands for retry"; - // Already requeued, no need to execute callbacks - return batch_status; - } else { - LOG(ERROR) << "ProcessBatch: Failed to requeue commands for retry"; - } - } - } - } - } else { - LOG(INFO) << "ProcessBatch: Batch operation completed successfully"; - } - - // Execute callback for each command - for (size_t i = 0; i < commands.size() && i < callbacks.size(); ++i) { - if (callbacks[i]) { - pstd::Status cmd_status; - // If batch processing status failed, all commands should fail - if (!batch_status.ok()) { - // Pass the upper layer error status to the client - cmd_status = batch_status; - } else { - // If the offset is empty, it means the command failed to be added - cmd_status = offsets[i].IsValid() ? pstd::Status::OK() : pstd::Status::IOError("Failed to append command"); - } - - // Log information before executing each callback function - LOG(INFO) << "ProcessBatch: Executing callback for command " << i - << ", cmd=" << (commands[i] ? commands[i]->name() : "null") - << ", status=" << cmd_status.ToString() - << ", offset=" << (offsets[i].IsValid() ? offsets[i].ToString() : "invalid"); - // Execute callback - callbacks[i](offsets[i], cmd_status); - - // Log information after callback execution - LOG(INFO) << "ProcessBatch: Callback executed for command " << i; - } - } - return batch_status; -} - -bool PikaCommandCollector::CheckConflict(const std::shared_ptr& cmd_ptr) const { - if (!cmd_ptr) { - return false; - } - - std::vector keys = cmd_ptr->current_key(); - for (const auto& key : keys) { - if (key_map_.find(key) != key_map_.end()) { - return true; - } - } - - return false; -} - -void PikaCommandCollector::HandleConflict(const std::shared_ptr& cmd_ptr) { - if (!cmd_ptr) { - return; - } - - std::vector keys = cmd_ptr->current_key(); - std::vector, CommandCallback>>::iterator> to_remove; - - // Find all conflicting commands - for (const auto& key : keys) { - auto it = key_map_.find(key); - if (it != key_map_.end()) { - // Check if this iterator is already added - bool already_added = false; - for (const auto& iter : to_remove) { - if (iter == it->second) { - already_added = true; - break; - } - } - if (!already_added) { - to_remove.push_back(it->second); - } - key_map_.erase(it); - } - } - - // Track conflict metrics - if (!to_remove.empty()) { - total_conflicts_.fetch_add(to_remove.size()); - } - - // Check command importance to prevent important commands from being overwritten - for (auto it : to_remove) { - auto old_cmd = it->first; - auto new_cmd = cmd_ptr; - - // Determine command importance - // If the old command type exists in the important command list, consider it important - bool is_important_cmd = false; - // Check commands with keywords such as "EXEC", "MULTI", "WATCH", etc. - std::string cmd_name = old_cmd->name(); - if (cmd_name == "MULTI" || cmd_name == "EXEC" || cmd_name == "WATCH") { - is_important_cmd = true; - } - - if (is_important_cmd) { - // 对于重要命令,我们保留原来的命令,并拒绝新命令 - // 恢复已移除的key映射 - std::vector old_keys = old_cmd->current_key(); - for (const auto& key : old_keys) { - key_map_[key] = it; - } - - // 从待删除列表中移除该命令 - for (auto iter = to_remove.begin(); iter != to_remove.end(); ++iter) { - if (*iter == it) { - to_remove.erase(iter); - break; - } - } - - } - } - - // 删除非重要的冲突命令 - for (auto it : to_remove) { - // 执行回调通知命令被取消 - if (it->second) { - it->second(LogOffset(), pstd::Status::Busy("Command replaced by newer command with same key")); - } - - // 从队列中移除 - pending_commands_.erase(it); - } -} - - - - - -void PikaCommandCollector::SetBatchSize(size_t batch_size) { - batch_size_.store(batch_size); - LOG(INFO) << "BatchSize set to " << batch_size; -} - void PikaCommandCollector::SetBatchMaxWaitTime(int batch_max_wait_time) { batch_max_wait_time_.store(batch_max_wait_time); LOG(INFO) << "BatchMaxWaitTime set to " << batch_max_wait_time << "ms"; @@ -428,78 +61,4 @@ void PikaCommandCollector::SetBatchMaxWaitTime(int batch_max_wait_time) { std::pair PikaCommandCollector::GetBatchStats() const { return {total_processed_.load(), total_batches_.load()}; -} - -std::optional PikaCommandCollector::GetAverageBatchTime() const { - uint64_t total_batches = total_batches_.load(); - if (total_batches == 0) { - return std::nullopt; - } - return static_cast(total_batch_time_ms_.load()) / total_batches; -} - -size_t PikaCommandCollector::PendingCommands() const { - std::lock_guard lock(mutex_); - return pending_commands_.size(); -} - - - -bool PikaCommandCollector::RetryBatch( - const std::vector>& commands, - const std::vector& callbacks, - int priority) { - - if (commands.empty() || callbacks.empty() || commands.size() != callbacks.size()) { - LOG(WARNING) << "RetryBatch: Invalid input parameters, commands size: " << commands.size() - << ", callbacks size: " << callbacks.size(); - return false; - } - - std::lock_guard lock(mutex_); - - LOG(INFO) << "RetryBatch: Retrying " << commands.size() << " commands with priority " << priority; - - // Add to retry queue with priority - retry_queue_.emplace_front(std::make_tuple(priority, commands, callbacks)); - - // Update statistics - total_retries_.fetch_add(1); - - // Sort retry queue by priority (higher values first) - std::sort(retry_queue_.begin(), retry_queue_.end(), - [](const auto& a, const auto& b) { return std::get<0>(a) > std::get<0>(b); }); - - // Process highest priority retry immediately if possible - if (priority > 50 && pending_commands_.empty()) { - // Process retry queue directly - auto& [pri, cmds, cbs] = retry_queue_.front(); - - // Add each command to the queue - for (size_t i = 0; i < cmds.size(); ++i) { - if (cmds[i]) { - // Handle key conflicts - HandleConflict(cmds[i]); - - // Add to queue front for priority processing - pending_commands_.push_front(std::make_pair(cmds[i], cbs[i])); - - // Update key mapping - std::vector keys = cmds[i]->current_key(); - for (const auto& key : keys) { - key_map_[key] = pending_commands_.begin(); - } - - LOG(INFO) << "RetryBatch: Immediately requeued command " << i << ": " << cmds[i]->name(); - } - } - - // Remove from retry queue - retry_queue_.pop_front(); - - // Trigger immediate flush - FlushCommands(true); - } - - return true; } \ No newline at end of file diff --git a/src/pika_command_queue.cc b/src/pika_command_queue.cc new file mode 100644 index 0000000000..f9013f65c9 --- /dev/null +++ b/src/pika_command_queue.cc @@ -0,0 +1,107 @@ +// Copyright (c) 2015-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/pika_command_queue.h" +#include + +CommandQueue::CommandQueue(size_t max_size) : max_size_(max_size) { + LOG(INFO) << "CommandQueue created with max_size: " << max_size_; +} + +CommandQueue::~CommandQueue() { + Shutdown(); + LOG(INFO) << "CommandQueue destroyed"; +} + +bool CommandQueue::EnqueueBatch(std::shared_ptr batch) { + if (!batch || batch->Empty()) { + LOG(WARNING) << "Attempt to enqueue empty or null batch"; + return false; + } + + std::lock_guard lock(queue_mutex_); + + if (shutdown_.load()) { + LOG(WARNING) << "Cannot enqueue batch: queue is shutdown"; + return false; + } + + if (cmd_queue_.size() >= max_size_) { + LOG(WARNING) << "Command queue is full (size: " << cmd_queue_.size() + << ", max: " << max_size_ << "), dropping batch"; + return false; + } + + cmd_queue_.push(batch); + + LOG(INFO) << "Enqueued command batch with " << batch->Size() + << " commands, queue size: " << cmd_queue_.size(); + + queue_cv_.notify_one(); + return true; +} + +std::shared_ptr CommandQueue::DequeueBatch() { + std::unique_lock lock(queue_mutex_); + + while (cmd_queue_.empty() && !shutdown_.load()) { + queue_cv_.wait(lock); + } + + if (shutdown_.load() && cmd_queue_.empty()) { + return nullptr; + } + + auto batch = cmd_queue_.front(); + cmd_queue_.pop(); + + LOG(INFO) << "Dequeued command batch with " << batch->Size() + << " commands, remaining queue size: " << cmd_queue_.size(); + + return batch; +} + +std::vector> CommandQueue::DequeueAllBatches() { + std::vector> batches; + std::lock_guard lock(queue_mutex_); + + if (shutdown_.load()) { + return batches; + } + + // Take all available batches + while (!cmd_queue_.empty()) { + batches.push_back(cmd_queue_.front()); + cmd_queue_.pop(); + } + + if (!batches.empty()) { + size_t total_commands = 0; + for (const auto& batch : batches) { + total_commands += batch->Size(); + } + LOG(INFO) << "Dequeued all batches: " << batches.size() + << " batches with " << total_commands << " total commands"; + } + + return batches; +} + +size_t CommandQueue::Size() const { + std::lock_guard lock(queue_mutex_); + return cmd_queue_.size(); +} + +bool CommandQueue::Empty() const { + std::lock_guard lock(queue_mutex_); + return cmd_queue_.empty(); +} + +void CommandQueue::Shutdown() { + std::lock_guard lock(queue_mutex_); + shutdown_.store(true); + queue_cv_.notify_all(); + LOG(INFO) << "CommandQueue shutdown, remaining batches: " << cmd_queue_.size(); +} diff --git a/src/pika_consensus.cc b/src/pika_consensus.cc index 00490c3162..ed20e4811b 100644 --- a/src/pika_consensus.cc +++ b/src/pika_consensus.cc @@ -229,15 +229,6 @@ ConsensusCoordinator::ConsensusCoordinator(const std::string& db_name) stable_logger_ = std::make_shared(db_name, log_path); mem_logger_ = std::make_shared(); logs_ = std::make_shared(); - // Initialize prepared_id - LogOffset last_offset; - BinlogOffset b_offset; - Status s = stable_logger_->Logger()->GetProducerStatus(&b_offset.filenum, &b_offset.offset); - if (s.ok()) { - last_offset.b_offset = b_offset; - SetPreparedId(last_offset); - LOG(INFO) << "Init: Initialized prepared_id to " << last_offset.ToString(); - } } ConsensusCoordinator::~ConsensusCoordinator() = default; @@ -345,66 +336,6 @@ Status ConsensusCoordinator::ProposeLog(const std::shared_ptr& cmd_ptr) { return Status::OK(); } -Status ConsensusCoordinator::BatchProposeLog(const std::vector>& cmd_ptrs, std::vector* offsets) { - if (cmd_ptrs.empty()) { - return Status::OK(); - } - // Reserve space for all command offsets - offsets->resize(cmd_ptrs.size()); - // Process each command in the batch - for (size_t i = 0; i < cmd_ptrs.size(); i++) { - const auto& cmd_ptr = cmd_ptrs[i]; - std::vector keys = cmd_ptr->current_key(); - // slotkey shouldn't add binlog - if (cmd_ptr->name() == kCmdNameSAdd && !keys.empty() &&(keys[0].compare(0, SlotKeyPrefix.length(), SlotKeyPrefix) == 0 || keys[0].compare(0, SlotTagPrefix.length(), SlotTagPrefix) == 0)) { - continue; - } - - // Generate binlog content - std::string content = cmd_ptr->ToRedisProtocol(); - std::string binlog; - - // Get offset for this command - LogOffset offset; - Status s = stable_logger_->Logger()->Put(content, &offset, binlog); - - if (!s.ok()) { - std::string db_name = cmd_ptr->db_name().empty() ? g_pika_conf->default_db() : cmd_ptr->db_name(); - std::shared_ptr db = g_pika_server->GetDB(db_name); - if (db) { - db->SetBinlogIoError(); - } - return s; - } - - // Store the offset for this command - (*offsets)[i] = offset; - - // Append to logs for replication, regardless of consistency mode. - logs_->AppendLog(Log::LogItem(offset, cmd_ptr, binlog)); - } - - // Signal auxiliary thread to handle these binlogs - g_pika_server->SignalAuxiliary(); - - // Proactively start sending binlogs to slaves in parallel with flushing to disk - std::shared_ptr db = g_pika_rm->GetSyncMasterDBByName(DBInfo(db_name_)); - if (db) { - db->WakeUpSlaveBinlogSync(); - } - - // Master node batch Binlog flush disk - stable_logger_->Logger()->Sync(); - - // update prepared_id to the offset of the last log - if (!offsets->empty()) { - SetPreparedId(offsets->back()); - LOG(INFO) << "BatchProposeLog: Updated prepared_id to " << offsets->back().ToString(); - } - - return Status::OK(); -} - Status ConsensusCoordinator::InternalAppendLog(const std::shared_ptr& cmd_ptr) { return InternalAppendBinlog(cmd_ptr); } @@ -514,8 +445,7 @@ uint32_t ConsensusCoordinator::term() { } void ConsensusCoordinator::InternalApplyFollower(const std::shared_ptr& cmd_ptr) { - // g_pika_rm->ScheduleWriteDBTask(cmd_ptr, db_name_); - PikaReplBgWorker::WriteDBInSyncWay(cmd_ptr); + g_pika_rm->ScheduleWriteDBTask(cmd_ptr, db_name_); } int ConsensusCoordinator::InitCmd(net::RedisParser* parser, const net::RedisCmdArgsType& argv) { @@ -913,63 +843,16 @@ Status ConsensusCoordinator::PersistAppendBinlog(const std::shared_ptr& cmd // If successful, append the log entry to the logs // TODO: 这里logs_的appendlog操作和上边的stable_logger_->Logger()->Put不是原子的,可能导致offset大的先被追加到logs_中, // 多线程写入的时候窗口会对不上,最终主从断开连接。需要加逻辑保证原子性 + LOG(INFO) << "PersistAppendBinlog: About to append to logs_, current size=" << logs_->Size() + << ", offset=" << cur_offset.ToString() << ", cmd=" << cmd_ptr->name(); logs_->AppendLog(Log::LogItem(cur_offset, cmd_ptr, binlog)); + LOG(INFO) << "PersistAppendBinlog: Successfully appended to logs_, new size=" << logs_->Size(); SetPreparedId(cur_offset); return stable_logger_->Logger()->IsOpened(); } -Status ConsensusCoordinator::BatchPersistAppendBinlog(const std::vector>& cmd_ptrs, - const std::vector& attributes, - std::vector* offsets) { - if (cmd_ptrs.empty() || cmd_ptrs.size() != attributes.size()) { - return Status::InvalidArgument("Invalid batch parameters"); - } - - std::lock_guard l(order_mu_); - offsets->resize(cmd_ptrs.size()); - for (size_t i = 0; i < cmd_ptrs.size(); ++i) { - const auto& cmd_ptr = cmd_ptrs[i]; - const auto& attribute = attributes[i]; - - LogOffset last_index = logs_->LastOffset(); - if (attribute.logic_id() < last_index.l_offset.index) { - LOG(WARNING) << DBInfo(db_name_).ToString() << "Drop log from leader logic_id " << attribute.logic_id() - << " cur last index " << last_index.l_offset.index; - (*offsets)[i] = LogOffset(); - continue; - } - - std::string content = cmd_ptr->ToRedisProtocol(); - std::string binlog = std::string(); - LogOffset offset = LogOffset(); - - Status s = stable_logger_->Logger()->Put(content, &offset, binlog); - if (!s.ok()) { - std::string db_name = cmd_ptr->db_name().empty() ? g_pika_conf->default_db() : cmd_ptr->db_name(); - std::shared_ptr db = g_pika_server->GetDB(db_name); - if (db) { - db->SetBinlogIoError(); - } - return s; - } - - logs_->AppendLog(Log::LogItem(offset, cmd_ptr, binlog)); - (*offsets)[i] = offset; - } - - // 更新prepared_id为最后一个有效的offset - for (auto it = offsets->rbegin(); it != offsets->rend(); ++it) { - if (it->IsValid()) { - SetPreparedId(*it); - break; - } - } - - return stable_logger_->Logger()->IsOpened(); -} - Status ConsensusCoordinator::AppendEntries(const std::shared_ptr& cmd_ptr, LogOffset& cur_logoffset) { std::vector keys = cmd_ptr->current_key(); // slotkey shouldn't add binlog @@ -1009,17 +892,12 @@ Status ConsensusCoordinator::AppendSlaveEntries(const std::shared_ptr& cmd_ Status ConsensusCoordinator::CommitAppLog(const LogOffset& master_committed_id) { int index = logs_->FindOffset(logs_->FirstOffset()); int log_size = logs_->Size(); // Cache log size - std::vector logs_to_apply; for (int i = index; i < log_size; ++i) { Log::LogItem log = logs_->At(i); if (master_committed_id >= log.offset) { - logs_to_apply.push_back(log); - } - } - if (!logs_to_apply.empty()) { - Status s = BatchApplyBinlogs(logs_to_apply); - if (!s.ok()) { - return s; + LOG(INFO) << "PacificA master_committed_id: " << master_committed_id.ToString() + << ", ApplyLog: " << log.offset.ToString(); + ApplyBinlog(log.cmd_ptr); } } @@ -1029,113 +907,29 @@ Status ConsensusCoordinator::CommitAppLog(const LogOffset& master_committed_id) } /** - * @brief Update the committed ID based on the acknowledged offsets of all slaves. + * @brief Update the committed ID based on the Prepared ID of the slave */ Status ConsensusCoordinator::UpdateCommittedID() { - // 获取当前的prepared_id - LogOffset current_prepared_id; - { - std::lock_guard l(prepared_id__rwlock_); - current_prepared_id = prepared_id_; - } - - // 获取当前的committed_id - LogOffset current_committed_id; - { - std::lock_guard l(committed_id_rwlock_); - current_committed_id = committed_id_; - } - std::unordered_map> slaves = sync_pros_.GetAllSlaveNodes(); - LogOffset min_acked_offset; - int active_slave_count = 0; - int total_slave_count = slaves.size(); - int binlog_sync_slave_count = 0; - // 初始化min_acked_offset为主节点的prepared_id - min_acked_offset = current_prepared_id; - - LOG(INFO) << "UpdateCommittedID: Master prepared_id: " << current_prepared_id.ToString() - << ", current_committed_id: " << current_committed_id.ToString() - << ", Total slaves: " << total_slave_count; - - // 如果没有从节点或没有活跃从节点,且prepared_id > committed_id,直接更新committed_id - if (total_slave_count == 0 && current_prepared_id > current_committed_id) { - LOG(INFO) << "UpdateCommittedID: No slaves, updating committed_id to prepared_id: " - << current_prepared_id.ToString(); - SetCommittedId(current_prepared_id); - return Status::OK(); - } - - // 查找所有活跃从节点中的最小acked_offset - for (const auto& slave_pair : slaves) { - const auto& slave = slave_pair.second; - // 记录从节点的状态和acked_offset,无论状态如何 - LOG(INFO) << "UpdateCommittedID: Slave " << slave->Ip() << ":" << slave->Port() - << " state: " << SlaveStateMsg[slave->slave_state] - << " acked_offset: " << slave->acked_offset.ToString(); - - // 考虑所有状态的从节点,只要它们有有效的acked_offset - if (slave->acked_offset.IsValid()) { - if (active_slave_count == 0) { - min_acked_offset = slave->acked_offset; - } else { - if (slave->acked_offset < min_acked_offset) { - min_acked_offset = slave->acked_offset; - } - } - active_slave_count++; - - // 如果是BinlogSync或Candidate状态,计数 - if (slave->slave_state == kSlaveBinlogSync || slave->slave_state == KCandidate) { - binlog_sync_slave_count++; + LogOffset slave_prepared_id = LogOffset(); + + for (const auto& slave : slaves) { + if (slave.second->slave_state == kSlaveBinlogSync) { + if (slave_prepared_id == LogOffset()) { + slave_prepared_id = slave.second->acked_offset; + } else if (slave.second->acked_offset < slave_prepared_id) { + slave_prepared_id = slave.second->acked_offset; } } } - // 确定新的committed_id - LogOffset new_committed_id; - if (active_slave_count == 0) { - // 如果没有活跃的从节点,不应该推进committed_id - // 保持当前的committed_id不变,确保数据一致性 - new_committed_id = current_committed_id; - LOG(INFO) << "UpdateCommittedID: No active slaves, keeping current committed_id: " << current_committed_id.ToString(); - } else if (active_slave_count < total_slave_count) { - // 如果有部分从节点活跃,使用min_acked_offset作为new_committed_id - new_committed_id = min_acked_offset; - LOG(INFO) << "UpdateCommittedID: Partial slaves acknowledged (" << active_slave_count << "/" << total_slave_count - << "), using minimum acked offset: " << min_acked_offset.ToString(); - } else { - // 如果所有从节点都活跃,使用min_acked_offset作为new_committed_id - new_committed_id = min_acked_offset; - LOG(INFO) << "UpdateCommittedID: All slaves acknowledged (" << active_slave_count << "/" << total_slave_count - << "), using minimum acked offset: " << min_acked_offset.ToString(); - } - - // 确保new_committed_id不低于当前的committed_id - if (new_committed_id < current_committed_id) { - LOG(WARNING) << "UpdateCommittedID: New committed_id " << new_committed_id.ToString() - << " is less than current committed_id " << current_committed_id.ToString() - << ", keeping current value"; - new_committed_id = current_committed_id; - } - - // 确保new_committed_id不超过prepared_id - if (new_committed_id > current_prepared_id) { - LOG(WARNING) << "UpdateCommittedID: new_committed_id " << new_committed_id.ToString() - << " exceeds prepared_id " << current_prepared_id.ToString() - << ", adjusting to prepared_id"; - new_committed_id = current_prepared_id; - } - - // 如果committed_id有变化,更新它 - if (new_committed_id > current_committed_id) { - LOG(INFO) << "UpdateCommittedID: Updating committed_id from " << current_committed_id.ToString() - << " to " << new_committed_id.ToString(); - SetCommittedId(new_committed_id); - return Status::OK(); - } else { - LOG(INFO) << "UpdateCommittedID: No update needed, current committed ID: " << current_committed_id.ToString(); - return Status::OK(); + if (slave_prepared_id < GetCommittedId()) { + LOG(WARNING) << "Error: slave_prepared_id (" << slave_prepared_id.ToString() << ") < master_committedId (" + << GetCommittedId().ToString() << ")"; + return Status::Error("slave_prepared_id < master_committedId"); } + SetCommittedId(slave_prepared_id); + LOG(INFO) << "PacificA update CommittedID: " << GetCommittedId().ToString(); + return Status::OK(); } Status ConsensusCoordinator::ProcessCoordination() { LogOffset offset = LogOffset(); @@ -1262,7 +1056,7 @@ Status ConsensusCoordinator::SendBinlog(std::shared_ptr slave_ptr, st // Only send empty binlog if there are no actual log entries to send // This prevents the deadlock where master waits for slave ACK and slave waits for master data if (tasks.empty() && logs_->Size() == 0) { - LOG(INFO) << "SendBinlog: Sending empty binlog with current committed_id: " << current_committed_id.ToString(); + // LOG(INFO) << "SendBinlog: Sending empty binlog with current committed_id: " << current_committed_id.ToString(); RmNode rm_node(slave_ptr->Ip(), slave_ptr->Port(), db_name, slave_ptr->SessionId()); // Create an empty WriteTask that includes the current committed_id WriteTask empty_task(rm_node, BinlogChip(LogOffset(), ""), LogOffset(), current_committed_id); diff --git a/src/pika_repl_bgworker.cc b/src/pika_repl_bgworker.cc index d465431405..5340533160 100644 --- a/src/pika_repl_bgworker.cc +++ b/src/pika_repl_bgworker.cc @@ -145,61 +145,16 @@ void PikaReplBgWorker::HandleBGWorkerWriteBinlog(void* arg) { if (binlog_res.binlog().empty()) { continue; } - // Handle binlog data with batch magic number detection - const std::string& received_binlog = binlog_res.binlog(); - std::string binlog_str = received_binlog; - // Check if this is the first binlog entry in a batch (contains PIKA_BATCH_MAGIC) - bool has_batch_magic = false; - if (i == (*index)[0] && received_binlog.size() >= sizeof(uint32_t)) { - uint32_t magic_num = 0; - memcpy(&magic_num, received_binlog.data(), sizeof(uint32_t)); - if (magic_num == PIKA_BATCH_MAGIC) { - has_batch_magic = true; - // Remove the magic number from the binlog data - binlog_str = received_binlog.substr(sizeof(uint32_t)); - LOG(INFO) << "HandleBGWorkerWriteBinlog: Detected PIKA_BATCH_MAGIC in binlog entry " << i - << ", processing as batch, original size: " << received_binlog.size() - << ", new size after removing magic: " << binlog_str.size(); - } - } - - // Validate binlog data - if (binlog_str.empty()) { - only_keepalive = true; - continue; - } - - if (binlog_str.size() < BINLOG_ENCODE_LEN) { - LOG(WARNING) << "HandleBGWorkerWriteBinlog: Binlog data too small (" << binlog_str.size() - << " bytes), minimum required: " << BINLOG_ENCODE_LEN; - continue; - } - - // Decode the binlog item - if (!PikaBinlogTransverter::BinlogItemWithoutContentDecode(TypeFirst, binlog_str, &worker->binlog_item_)) { + if (!PikaBinlogTransverter::BinlogItemWithoutContentDecode(TypeFirst, binlog_res.binlog(), &worker->binlog_item_)) { + LOG(WARNING) << "Binlog item decode failed"; slave_db->SetReplState(ReplState::kTryConnect); return; } - - // Extract Redis command data - const char* redis_parser_start = binlog_str.data() + BINLOG_ENCODE_LEN; - int redis_parser_len = static_cast(binlog_str.size()) - BINLOG_ENCODE_LEN; - - if (redis_parser_len <= 0) { - LOG(WARNING) << "HandleBGWorkerWriteBinlog: No Redis command data after binlog header for entry " << i; - continue; - } - - // Create a new parser instance for each binlog entry to ensure clean state - net::RedisParser temp_parser; - net::RedisParserSettings settings; - settings.DealMessage = &(PikaReplBgWorker::HandleWriteBinlog); - temp_parser.RedisParserInit(REDIS_PARSER_REQUEST, settings); - temp_parser.data = worker; - + const char* redis_parser_start = binlog_res.binlog().data() + BINLOG_ENCODE_LEN; + int redis_parser_len = static_cast(binlog_res.binlog().size()) - BINLOG_ENCODE_LEN; int processed_len = 0; net::RedisParserStatus ret = - temp_parser.ProcessInputBuffer(redis_parser_start, redis_parser_len, &processed_len); + worker->redis_parser_.ProcessInputBuffer(redis_parser_start, redis_parser_len, &processed_len); if (ret != net::kRedisParserDone) { LOG(WARNING) << "Redis parser failed"; slave_db->SetReplState(ReplState::kTryConnect); @@ -215,12 +170,13 @@ void PikaReplBgWorker::HandleBGWorkerWriteBinlog(void* arg) { if (only_keepalive) { ack_end = LogOffset(); } else { - // The slave node uses the binlog offset sent by the master node as ACK - ack_end = pb_end; - - // slave nodes batch Binlog flush disk + LogOffset productor_status; + // Reply Ack to master immediately std::shared_ptr logger = db->Logger(); - logger->Sync(); + logger->GetProducerStatus(&productor_status.b_offset.filenum, &productor_status.b_offset.offset, + &productor_status.l_offset.term, &productor_status.l_offset.index); + ack_end = productor_status; + ack_end.l_offset.term = pb_end.l_offset.term; } g_pika_rm->SendBinlogSyncAckRequest(db_name, ack_start, ack_end); @@ -262,7 +218,7 @@ int PikaReplBgWorker::HandleWriteBinlog(net::RedisParser* parser, const net::Red return -1; } if(db->GetISConsistency()){ - PikaReplBgWorker::WriteDBInSyncWay(c_ptr); + db->AppendSlaveEntries(c_ptr, worker->binlog_item_); }else{ db->ConsensusProcessLeaderLog(c_ptr, worker->binlog_item_); } @@ -270,8 +226,8 @@ int PikaReplBgWorker::HandleWriteBinlog(net::RedisParser* parser, const net::Red } void PikaReplBgWorker::HandleBGWorkerWriteDB(void* arg) { - std::unique_ptr> cmd_ptr_ptr(static_cast*>(arg)); - const std::shared_ptr c_ptr = *cmd_ptr_ptr; + std::unique_ptr task_arg(static_cast(arg)); + const std::shared_ptr c_ptr = task_arg->cmd_ptr; WriteDBInSyncWay(c_ptr); } diff --git a/src/pika_repl_server.cc b/src/pika_repl_server.cc index 463c17a42e..c8f1c9f9dc 100644 --- a/src/pika_repl_server.cc +++ b/src/pika_repl_server.cc @@ -88,16 +88,9 @@ void PikaReplServer::BuildBinlogOffset(const LogOffset& offset, InnerMessage::Bi } void PikaReplServer::BuildBinlogSyncResp(const std::vector& tasks, InnerMessage::InnerResponse* response) { - if (tasks.empty()) { - return; - } response->set_code(InnerMessage::kOk); response->set_type(InnerMessage::Type::kBinlogSync); - - // Since CommandCollector is enabled, all data should be treated as batch processing - bool is_batch = g_pika_conf->command_batch_enabled(); - for (size_t task_idx = 0; task_idx < tasks.size(); task_idx++) { - const auto& task = tasks[task_idx]; + for (const auto& task : tasks) { InnerMessage::InnerResponse::BinlogSync* binlog_sync = response->add_binlog_sync(); binlog_sync->set_session_id(task.rm_node_.SessionId()); InnerMessage::Slot* db = binlog_sync->mutable_slot(); @@ -110,23 +103,11 @@ void PikaReplServer::BuildBinlogSyncResp(const std::vector& tasks, In db->set_slot_id(0); InnerMessage::BinlogOffset* boffset = binlog_sync->mutable_binlog_offset(); BuildBinlogOffset(task.binlog_chip_.offset_, boffset); - - // For batch binlog transmission (when CommandCollector is enabled), - // add PIKA_BATCH_MAGIC at the beginning of the first binlog entry - if (is_batch && task_idx == 0) { - std::string magic_binlog; - magic_binlog.resize(sizeof(uint32_t)); - memcpy(&magic_binlog[0], &PIKA_BATCH_MAGIC, sizeof(uint32_t)); - magic_binlog.append(task.binlog_chip_.binlog_); - binlog_sync->set_binlog(magic_binlog); - } else { - binlog_sync->set_binlog(task.binlog_chip_.binlog_); - } - - if (g_pika_server->IsConsistency()) { + if(g_pika_server->IsConsistency()){ InnerMessage::BinlogOffset* committed_id = binlog_sync->mutable_committed_id(); BuildBinlogOffset(task.committed_id_, committed_id); } + binlog_sync->set_binlog(task.binlog_chip_.binlog_); } } diff --git a/src/pika_rm.cc b/src/pika_rm.cc index 71781a4a13..de337fd157 100644 --- a/src/pika_rm.cc +++ b/src/pika_rm.cc @@ -40,7 +40,9 @@ std::string SyncDB::DBName() { /* SyncMasterDB*/ SyncMasterDB::SyncMasterDB(const std::string& db_name) - : SyncDB(db_name), coordinator_(db_name) {} + : SyncDB(db_name), coordinator_(db_name) { + command_collector_ = std::make_shared(&coordinator_, 5); +} int SyncMasterDB::GetNumberOfSlaveNode() { return coordinator_.SyncPros().SlaveSize(); } @@ -55,14 +57,11 @@ Status SyncMasterDB::GetSlaveNodeSession(const std::string& ip, int port, int32_ return Status::NotFound("slave " + ip + ":" + std::to_string(port) + " not found"); } - if (slave_ptr->slave_mu.try_lock()) { - *session = slave_ptr->SessionId(); - slave_ptr->slave_mu.unlock(); - return Status::OK(); - } else { - LOG(WARNING) << "GetSlaveNodeSession: Failed to acquire lock for " << ip << ":" << port << ", slave may be busy"; - return Status::Busy("Slave node is busy, try again later"); - } + slave_ptr->Lock(); + *session = slave_ptr->SessionId(); + slave_ptr->Unlock(); + + return Status::OK(); } Status SyncMasterDB::AddSlaveNode(const std::string& ip, int port, int session_id) { @@ -234,32 +233,34 @@ Status SyncMasterDB::GetSlaveState(const std::string& ip, int port, SlaveState* } Status SyncMasterDB::WakeUpSlaveBinlogSync() { - std::unordered_map> slaves = GetAllSlaveNodes(); - std::vector> to_del; - for (auto& slave_iter : slaves) { - std::shared_ptr slave_ptr = slave_iter.second; - - slave_ptr->Lock(); - SlaveState current_state = slave_ptr->slave_state; - slave_ptr->Unlock(); - - // Only send to slaves that are actively waiting for binlogs - if (current_state == SlaveState::kSlaveBinlogSync || current_state == SlaveState::KCandidate) { - Status s = coordinator_.SendBinlog(slave_ptr, db_info_.db_name_); - if (!s.ok()) { - to_del.push_back(slave_ptr); - LOG(WARNING) << "WakeUpSlaveBinlogSync: SendBinlog failed for slave " << slave_ptr->ToString() << ": " - << s.ToString(); + std::unordered_map> slaves = GetAllSlaveNodes(); + std::vector> to_del; + for (auto& slave_iter : slaves) { + std::shared_ptr slave_ptr = slave_iter.second; + std::lock_guard l(slave_ptr->slave_mu); + if (slave_ptr->sent_offset == slave_ptr->acked_offset) { + Status s; + if (coordinator_.GetISConsistency()) { + if(slave_ptr->slave_state == SlaveState::kSlaveBinlogSync||slave_ptr->slave_state == SlaveState::KCandidate){ + s = coordinator_.SendBinlog(slave_ptr, db_info_.db_name_); + } + } else { + s = ReadBinlogFileToWq(slave_ptr); + } + if (!s.ok()) { + to_del.push_back(slave_ptr); + LOG(WARNING) << "WakeUpSlaveBinlogSync failed, marking for deletion: " + << slave_ptr->ToStringStatus() << " - " << s.ToString(); + } } - } } - for (const auto& to_del_slave : to_del) { - RemoveSlaveNode(to_del_slave->Ip(), to_del_slave->Port()); - LOG(INFO) << "Removed slave due to SendBinlog failure: " << to_del_slave->ToStringStatus(); - } + for (const auto& to_del_slave : to_del) { + RemoveSlaveNode(to_del_slave->Ip(), to_del_slave->Port()); + LOG(INFO) << "Removed slave: " << to_del_slave->ToStringStatus(); + } - return Status::OK(); + return Status::OK(); } @@ -428,28 +429,20 @@ Status SyncMasterDB::AppendSlaveEntries(const std::shared_ptr& cmd_ptr, con return coordinator_.AppendSlaveEntries(cmd_ptr, attribute); } -Status SyncMasterDB::BatchAppendSlaveEntries(const std::vector>& cmd_ptrs, - const std::vector& attributes) { - if (cmd_ptrs.empty() || cmd_ptrs.size() != attributes.size()) { - return Status::InvalidArgument("Invalid batch parameters"); - } - - std::vector offsets; - Status s = coordinator_.BatchPersistAppendBinlog(cmd_ptrs, attributes, &offsets); - if (!s.ok()) { - return s; - } - - return Status::OK(); -} Status SyncMasterDB::ProcessCoordination(){ return coordinator_.ProcessCoordination(); } Status SyncMasterDB::UpdateCommittedID(){ Status s = coordinator_.UpdateCommittedID(); if (s.ok()) { - // UpdateCommittedID success - removed verbose logging - int slave_count = GetNumberOfSlaveNode(); + // Notify RocksDB thread of new CommittedID + LogOffset committed_id = coordinator_.GetCommittedId(); + if (committed_id.IsValid()) { + extern std::unique_ptr g_pika_rm; + if (g_pika_rm) { + g_pika_rm->NotifyCommittedID(committed_id); + } + } } else { LOG(WARNING) << "UpdateCommittedID failed: " << s.ToString(); } @@ -496,6 +489,7 @@ Status SyncMasterDB::AppendCandidateBinlog(const std::string& ip, int port, cons return Status::OK(); } +/* pstd::Status SyncMasterDB::WaitForSlaveAcks(const LogOffset& target_offset, int timeout_ms) { // bug: 重发有问题 // Get slave count @@ -560,6 +554,7 @@ pstd::Status SyncMasterDB::WaitForSlaveAcks(const LogOffset& target_offset, int } } } +*/ Status SyncMasterDB::ConsensusProposeLog(const std::shared_ptr& cmd_ptr) { // If consistency is not required, directly propose the log without waiting for consensus @@ -567,7 +562,6 @@ Status SyncMasterDB::ConsensusProposeLog(const std::shared_ptr& cmd_ptr) { return coordinator_.ProposeLog(cmd_ptr); } - auto start = std::chrono::steady_clock::now(); LogOffset offset; Status s = coordinator_.AppendEntries(cmd_ptr, offset); // Append the log entry to the coordinator @@ -575,128 +569,23 @@ Status SyncMasterDB::ConsensusProposeLog(const std::shared_ptr& cmd_ptr) { return s; } - // Wait for consensus to be achieved within 10 seconds - while (std::chrono::duration_cast(std::chrono::steady_clock::now() - start).count() < 10) { - // Check if consensus has been achieved for the given log offset - if (checkFinished(offset)) { - return Status::OK(); - } - // TODO: 这里暂时注掉了sleep等待,50ms耗时过长,影响写入链路,后期需要改成条件变量唤醒方式 - //std::this_thread::sleep_for(std::chrono::milliseconds(50)); - } + // // Wait for consensus to be achieved within 10 seconds + // while (std::chrono::duration_cast(std::chrono::steady_clock::now() - start).count() < 10) { + // // Check if consensus has been achieved for the given log offset + // if (checkFinished(offset)) { + // return Status::OK(); + // } + // // TODO: 这里暂时注掉了sleep等待,50ms耗时过长,影响写入链路,后期需要改成条件变量唤醒方式 + // //std::this_thread::sleep_for(std::chrono::milliseconds(50)); + // } - return Status::Timeout("No consistency achieved within 10 seconds"); -} + // return Status::Timeout("No consistency achieved within 10 seconds"); + LOG(INFO) << "ConsensusProposeLog: Successfully appended cmd " << cmd_ptr->name() + << " with offset " << offset.ToString() << ", delegating to RocksDBThreadLoop"; -Status SyncMasterDB::ConsensusBatchProposeLog(const std::vector>& cmd_ptrs, std::vector* offsets) { - // If the batch is empty, nothing to do - if (cmd_ptrs.empty()) { - return Status::OK(); - } - - // For large batches, log batch size for monitoring - if (cmd_ptrs.size() > 10) { - LOG(INFO) << "ConsensusBatchProposeLog: Processing large batch of " << cmd_ptrs.size() << " commands"; - } - - // First, propose the log batch to the coordinator with optimized performance - auto batch_start = std::chrono::steady_clock::now(); - Status s = coordinator_.BatchProposeLog(cmd_ptrs, offsets); - auto batch_end = std::chrono::steady_clock::now(); - auto batch_time_ms = std::chrono::duration_cast(batch_end - batch_start).count(); - - if (!s.ok()) { - LOG(WARNING) << "ConsensusBatchProposeLog: Failed to propose log batch: " << s.ToString(); - return s; - } - - // Performance logging for large batches - if (cmd_ptrs.size() > 10) { - LOG(INFO) << "ConsensusBatchProposeLog: Successfully proposed batch of " << cmd_ptrs.size() - << " commands in " << batch_time_ms << "ms (" - << (cmd_ptrs.size() / (batch_time_ms ? batch_time_ms : 1)) << " commands/ms)"; - } - - // If consistency is not required, return immediately without waiting for replication - if (!coordinator_.GetISConsistency()) { - LOG(INFO) << "ConsensusBatchProposeLog: Consistency not required, returning immediately"; - return s; - } - - // Record the current committed_id and slave node count - LogOffset current_committed_id = GetCommittedId(); - int slave_count = GetNumberOfSlaveNode(); - - LOG(INFO) << "ConsensusBatchProposeLog: Before BatchProposeLog - Current committed_id: " << current_committed_id.ToString() - << ", expecting ACKs from 1 master and " << slave_count << " slave(s)"; - - // 在BatchProposeLog完成后,获取实际的prepared_id作为等待目标 - LogOffset actual_prepared_id = GetPreparedId(); - LogOffset last_cmd_offset; - if (!offsets->empty()) { - last_cmd_offset = offsets->back(); - LOG(INFO) << "ConsensusBatchProposeLog: Last command offset in batch: " << last_cmd_offset.ToString(); - LOG(INFO) << "ConsensusBatchProposeLog: After BatchProposeLog - actual prepared_id: " << actual_prepared_id.ToString(); - } - - // For strong consistency mode, set a batch-level timeout that applies to the entire batch - int batch_timeout_ms = g_pika_conf->replication_ack_timeout(); - // Adjust timeout based on batch size for large batches - if (cmd_ptrs.size() > 100) { - // Scale timeout logarithmically with batch size - batch_timeout_ms = static_cast(batch_timeout_ms * (1 + log10(cmd_ptrs.size() / 100.0))); - LOG(INFO) << "ConsensusBatchProposeLog: Adjusted batch timeout to " << batch_timeout_ms << "ms for large batch"; - } - - // For strong consistency mode, wait for the batch to be committed - LOG(INFO) << "ConsensusBatchProposeLog: Waiting for batch to be committed (target: " << actual_prepared_id.ToString() - << ") with timeout of " << batch_timeout_ms << "ms"; - - s = WaitForSlaveAcks(actual_prepared_id, batch_timeout_ms); - - // Process synchronization results - if (!s.ok()) { - if (s.IsTimeout()) { - LOG(WARNING) << "ConsensusBatchProposeLog: Batch timed out waiting for ACKs: " << s.ToString(); - } else if (s.IsIncomplete()) { - LOG(WARNING) << "ConsensusBatchProposeLog: Not all nodes acknowledged the batch: " << s.ToString(); - } else { - LOG(WARNING) << "ConsensusBatchProposeLog: Batch operation failed with status: " << s.ToString(); - } - LOG(WARNING) << "ConsensusBatchProposeLog: Batch operation could not be confirmed with strong consistency, " - << "batch size: " << cmd_ptrs.size(); - } else { - LogOffset new_committed_id = GetCommittedId(); - LOG(INFO) << "ConsensusBatchProposeLog: Successfully received ACKs for entire batch, " - << "new committed_id: " << new_committed_id.ToString(); - - // Verify that synchronization successfully included all offsets in this batch of commands - if (!offsets->empty() && new_committed_id < last_cmd_offset) { - LOG(WARNING) << "ConsensusBatchProposeLog: New committed_id " << new_committed_id.ToString() - << " is less than last command offset " << last_cmd_offset.ToString() - << ", some commands in batch may not be fully replicated"; - - // For strong consistency, we should ensure all commands are replicated - // But if there are no slaves, we don't need to worry about replication - if (slave_count > 0) { - LOG(WARNING) << "ConsensusBatchProposeLog: Some commands may not be fully replicated to all slaves, but proceeding"; - } else { - LOG(INFO) << "ConsensusBatchProposeLog: No slaves connected, no replication needed"; - } - } - } - return s; + return Status::OK(); } -// Per-DB global batching window across threads -struct WindowState { - pstd::Mutex mu; - std::atomic start_us{0}; - std::atomic accepted{0}; -}; -static std::unordered_map g_db_windows; -static pstd::Mutex g_db_windows_mu; - Status SyncMasterDB::ConsensusProcessLeaderLog(const std::shared_ptr& cmd_ptr, const BinlogItem& attribute) { return coordinator_.ProcessLeaderLog(cmd_ptr, attribute); } @@ -846,6 +735,9 @@ PikaReplicaManager::PikaReplicaManager() { int port = g_pika_conf->port() + kPortShiftReplServer; pika_repl_client_ = std::make_unique(3000, 60); pika_repl_server_ = std::make_unique(ips, port, 3000); + command_queue_ = std::make_unique(500); + // Initialize CommittedID tracking + last_committed_id_ = LogOffset(); InitDB(); } @@ -862,9 +754,17 @@ void PikaReplicaManager::Start() { LOG(FATAL) << "Start Repl Server Error: " << ret << (ret == net::kCreateThreadError ? ": create thread error " : ": other error"); } + // Start command queue background thread + StartCommandQueueThread(); + // Start RocksDB background thread + StartRocksDBThread(); } void PikaReplicaManager::Stop() { + // Stop RocksDB background thread first + StopRocksDBThread(); + // Stop command queue background thread + StopCommandQueueThread(); pika_repl_client_->Stop(); pika_repl_server_->Stop(); } @@ -904,85 +804,63 @@ void PikaReplicaManager::ProduceWriteQueue(const std::string& ip, int port, std: } int PikaReplicaManager::ConsumeWriteQueue() { - // Quick check if there are any tasks + std::unordered_map>> to_send_map; + int counter = 0; { std::lock_guard l(write_queue_mu_); - if (write_queues_.empty()) { - static int empty_counter = 0; - return 0; - } - } - if (g_pika_conf->command_batch_enabled()) { - for (auto& db_item : sync_master_dbs_) { - if (db_item.second) { - auto command_collector = db_item.second->GetCommandCollector(); - if (command_collector) { - command_collector->FlushCommands(); + for (auto& iter : write_queues_) { + const std::string& ip_port = iter.first; + std::map>& p_map = iter.second; + for (auto& db_queue : p_map) { + std::queue& queue = db_queue.second; + for (int i = 0; i < kBinlogSendPacketNum; ++i) { + if (queue.empty()) { + break; + } + size_t batch_index = queue.size() > kBinlogSendBatchNum ? kBinlogSendBatchNum : queue.size(); + std::vector to_send; + size_t batch_size = 0; + for (size_t i = 0; i < batch_index; ++i) { + WriteTask& task = queue.front(); + batch_size += task.binlog_chip_.binlog_.size(); + // make sure SerializeToString will not over 2G + if (batch_size > PIKA_MAX_CONN_RBUF_HB) { + break; + } + to_send.push_back(task); + queue.pop(); + counter++; + } + if (!to_send.empty()) { + to_send_map[ip_port].push_back(std::move(to_send)); + } } } } } - // A list of sending jobs to be executed outside the lock. - // Each job is a tuple of (ip, port, tasks_to_send). - std::vector>> all_sends; - int counter = 0; - // === Start of Critical Section === - { - std::lock_guard l(write_queue_mu_); - LOG(INFO) << "ConsumeWriteQueue: write_queues_ size: " << write_queues_.size(); - auto slave_iter = write_queues_.begin(); - while (slave_iter != write_queues_.end()) { - std::string ip; - int port = 0; - if (!pstd::ParseIpPortString(slave_iter->first, ip, port)) { - LOG(WARNING) << "Parse ip_port error " << slave_iter->first; - slave_iter = write_queues_.erase(slave_iter); + std::vector to_delete; + for (auto& iter : to_send_map) { + std::string ip; + int port = 0; + if (!pstd::ParseIpPortString(iter.first, ip, port)) { + LOG(WARNING) << "Parse ip_port error " << iter.first; + continue; + } + for (auto& to_send : iter.second) { + Status s = pika_repl_server_->SendSlaveBinlogChips(ip, port, to_send); + if (!s.ok()) { + LOG(WARNING) << "send binlog to " << ip << ":" << port << " failed, " << s.ToString(); + to_delete.push_back(iter.first); continue; } - - // Collect all tasks for this slave from all its dbs - std::vector tasks_for_this_slave; - auto& p_map = slave_iter->second; - auto db_iter = p_map.begin(); - while (db_iter != p_map.end()) { - auto& queue = db_iter->second; - while (!queue.empty()) { - tasks_for_this_slave.push_back(std::move(queue.front())); - queue.pop(); - } - // Since the queue is now empty, erase this db entry - db_iter = p_map.erase(db_iter); - } - - if (!tasks_for_this_slave.empty()) { - LOG(INFO) << "ConsumeWriteQueue: Found " << tasks_for_this_slave.size() << " tasks for slave " << ip << ":" << port; - all_sends.emplace_back(ip, port, std::move(tasks_for_this_slave)); - } else { - LOG(INFO) << "ConsumeWriteQueue: No tasks found for slave " << ip << ":" << port; - } - - // Since all db entries for this slave are processed and erased, - // erase the slave entry itself. - slave_iter = write_queues_.erase(slave_iter); } } - // === End of Critical Section === - // Now, execute all the prepared network IO jobs outside the lock. - for (auto& send_job : all_sends) { - std::string& ip = std::get<0>(send_job); - int port = std::get<1>(send_job); - std::vector& to_send = std::get<2>(send_job); - - counter += to_send.size(); - LOG(INFO) << "ConsumeWriteQueue: Sending " << to_send.size() << " tasks to " << ip << ":" << port; - Status s = pika_repl_server_->SendSlaveBinlogChips(ip, port, to_send); - if (!s.ok()) { - LOG(WARNING) << "send binlog to " << ip << ":" << port << " failed, " << s.ToString(); - DropItemInWriteQueue(ip, port); - } else { - LOG(INFO) << "ConsumeWriteQueue: Successfully sent " << to_send.size() << " tasks to " << ip << ":" << port; + if (!to_delete.empty()) { + std::lock_guard l(write_queue_mu_); + for (auto& del_queue : to_delete) { + write_queues_.erase(del_queue); } } return counter; @@ -1380,16 +1258,6 @@ void PikaReplicaManager::FindCommonMaster(std::string* master) { } } -std::shared_ptr PikaReplicaManager::GetConsensusCoordinator(const std::string& db_name) { - std::shared_lock l(dbs_rw_); - DBInfo p_info(db_name); - if (sync_master_dbs_.find(p_info) == sync_master_dbs_.end()) { - return nullptr; - } - // Return a pointer to the existing coordinator instead of creating a copy - return sync_master_dbs_[p_info]->StableLogger()->coordinator(); -} - void PikaReplicaManager::RmStatus(std::string* info) { std::shared_lock l(dbs_rw_); std::stringstream tmp_stream; @@ -1413,3 +1281,399 @@ void PikaReplicaManager::BuildBinlogOffset(const LogOffset& offset, InnerMessage boffset->set_term(offset.l_offset.term); boffset->set_index(offset.l_offset.index); } + +// Command Queue related implementations +void PikaReplicaManager::EnqueueCommandBatch(std::shared_ptr batch) { + if (command_queue_) { + bool success = command_queue_->EnqueueBatch(batch); + if (success) { + // Notify the command queue thread that new data is available + { + std::lock_guard lock(command_queue_mutex_); + command_queue_cv_.notify_one(); + } + } else { + LOG(ERROR) << "Failed to enqueue command batch with " << batch->Size() << " commands"; + } + } else { + LOG(ERROR) << "Command queue not initialized"; + } +} + +std::shared_ptr PikaReplicaManager::DequeueCommandBatch() { + if (command_queue_) { + return command_queue_->DequeueBatch(); + } else { + LOG(ERROR) << "Command queue not initialized"; + return nullptr; + } +} + +size_t PikaReplicaManager::GetCommandQueueSize() const { + if (command_queue_) { + return command_queue_->Size(); + } + return 0; +} + +bool PikaReplicaManager::IsCommandQueueEmpty() const { + if (command_queue_) { + return command_queue_->Empty(); + } + return true; +} + +void PikaReplicaManager::StartCommandQueueThread() { + if (command_queue_running_.load()) { + LOG(WARNING) << "Command queue thread is already running"; + return; + } + command_queue_running_.store(true); + command_queue_thread_ = std::make_unique(&PikaReplicaManager::CommandQueueLoop, this); + LOG(INFO) << "Command queue background thread started"; +} + +void PikaReplicaManager::StopCommandQueueThread() { + if (!command_queue_running_.load()) { + return; + } + command_queue_running_.store(false); + // Notify the condition variable to wake up the thread + { + std::lock_guard lock(command_queue_mutex_); + command_queue_cv_.notify_one(); + } + + if (command_queue_thread_ && command_queue_thread_->joinable()) { + command_queue_thread_->join(); + } + LOG(INFO) << "Command queue background thread stopped"; +} + +void PikaReplicaManager::CommandQueueLoop() { + LOG(INFO) << "Command queue loop started"; + while (command_queue_running_.load()) { + try { + std::unique_lock lock(command_queue_mutex_); + // Wait for notification or timeout + command_queue_cv_.wait(lock, [this] { + return !command_queue_running_.load() || !command_queue_->Empty(); + }); + + // Check if we should exit + if (!command_queue_running_.load()) { + break; + } + + // Release lock before processing + lock.unlock(); + + // Continuously process batches until queue is empty + bool processed_any_batch = false; + do { + // Non-blocking dequeue all available batches + auto batches = command_queue_->DequeueAllBatches(); + + if (!batches.empty()) { + // Process all batches + ProcessCommandBatches(batches); + processed_any_batch = true; + LOG(INFO) << "Processed " << batches.size() << " batches, checking for more..."; + } else { + processed_any_batch = false; + } + // Continue processing if we processed any batches (there might be more) + } while (processed_any_batch && command_queue_running_.load()); + } catch (const std::exception& e) { + LOG(ERROR) << "Error in command queue loop: " << e.what(); + std::this_thread::sleep_for(std::chrono::milliseconds(100)); + } + } + LOG(INFO) << "Command queue loop ended"; +} + +void PikaReplicaManager::ProcessCommandBatches(const std::vector>& batches) { + if (batches.empty()) { + return; + } + size_t total_commands = 0; + for (const auto& batch : batches) { + total_commands += batch->Size(); + } + LOG(INFO) << "Processing " << batches.size() << " batches with " << total_commands << " total commands"; + + // Merge all commands from all batches for unified batch processing + std::vector> all_commands; + std::map, size_t>> command_to_batch_map; + size_t global_cmd_idx = 0; + for (const auto& batch : batches) { + if (!batch || batch->Empty()) { + continue; + } + for (size_t i = 0; i < batch->commands.size(); ++i) { + all_commands.push_back(batch->commands[i]); + command_to_batch_map[global_cmd_idx] = std::make_pair(batch, i); + global_cmd_idx++; + } + } + + if (all_commands.empty()) { + LOG(WARNING) << "No valid commands found in batches"; + return; + } + + // Use the first batch's database for consensus + std::string db_name = batches[0]->db_name; + std::shared_ptr sync_db = GetSyncMasterDBByName(DBInfo(db_name)); + if (!sync_db) { + LOG(WARNING) << "SyncMasterDB not found for database: " << db_name; + // Call callbacks with error for all batches + for (const auto& batch : batches) { + for (size_t i = 0; i < batch->callbacks.size(); ++i) { + if (batch->callbacks[i]) { + batch->callbacks[i](LogOffset(), pstd::Status::NotFound("Database not found")); + } + } + } + return; + } + + try { + // Process each command individually using AppendEntries + std::vector all_offsets; + all_offsets.resize(all_commands.size()); + + // Process each command using the original single-command logic + for (size_t cmd_idx = 0; cmd_idx < all_commands.size(); ++cmd_idx) { + const auto& cmd_ptr = all_commands[cmd_idx]; + LogOffset cmd_offset; + + if (sync_db->GetISConsistency()) { + // For consistency mode, use ConsensusProposeLog directly + pstd::Status s = sync_db->ConsensusProposeLog(cmd_ptr); + if (!s.ok()) { + LOG(ERROR) << "Failed to append command " << cmd_ptr->name() << ": " << s.ToString(); + // Call callbacks with error for remaining commands + for (size_t error_idx = cmd_idx; error_idx < all_commands.size(); ++error_idx) { + auto& batch_info = command_to_batch_map[error_idx]; + auto batch = batch_info.first; + size_t batch_cmd_idx = batch_info.second; + if (batch_cmd_idx < batch->callbacks.size() && batch->callbacks[batch_cmd_idx]) { + batch->callbacks[batch_cmd_idx](LogOffset(), s); + } + } + return; + } + // Get the prepared ID as the offset + cmd_offset = sync_db->GetPreparedId(); + } else { + // For non-consistency mode, use ConsensusProposeLog as well + pstd::Status s = sync_db->ConsensusProposeLog(cmd_ptr); + if (!s.ok()) { + LOG(ERROR) << "Failed to propose command " << cmd_ptr->name() << ": " << s.ToString(); + // Call callbacks with error for remaining commands + for (size_t error_idx = cmd_idx; error_idx < all_commands.size(); ++error_idx) { + auto& batch_info = command_to_batch_map[error_idx]; + auto batch = batch_info.first; + size_t batch_cmd_idx = batch_info.second; + if (batch_cmd_idx < batch->callbacks.size() && batch->callbacks[batch_cmd_idx]) { + batch->callbacks[batch_cmd_idx](LogOffset(), s); + } + } + return; + } + // Get the prepared ID as the offset + cmd_offset = sync_db->GetPreparedId(); + } + + all_offsets[cmd_idx] = cmd_offset; + + // Update individual batch offsets + auto& batch_info = command_to_batch_map[cmd_idx]; + auto batch = batch_info.first; + size_t batch_cmd_idx = batch_info.second; + + // Ensure the batch has enough space for offsets + if (batch->binlog_offsets.size() <= batch_cmd_idx) { + batch->binlog_offsets.resize(batch->commands.size()); + } + batch->binlog_offsets[batch_cmd_idx] = cmd_offset; + } + + LOG(INFO) << "Successfully processed " << all_commands.size() << " commands individually"; + + // Create BatchGroup with the last offset as end_offset + LogOffset end_offset = all_offsets.back(); + auto batch_group = std::make_shared(batches, end_offset); + + // Enqueue the BatchGroup + { + std::lock_guard lock(pending_batch_groups_mutex_); + pending_batch_groups_.push(batch_group); + } + LOG(INFO) << "Created BatchGroup with " << batches.size() << " batches and end_offset: " << end_offset.ToString(); + } catch (const std::exception& e) { + LOG(ERROR) << "Exception in ProcessCommandBatches: " << e.what(); + // Call callbacks with error for all batches + for (const auto& batch : batches) { + for (size_t i = 0; i < batch->callbacks.size(); ++i) { + if (batch->callbacks[i]) { + batch->callbacks[i](LogOffset(), pstd::Status::Corruption("Exception in processing")); + } + } + } + } + + // Signal auxiliary thread once after processing all batches + g_pika_server->SignalAuxiliary(); +} + +// RocksDB Background Thread Implementation +void PikaReplicaManager::StartRocksDBThread() { + if (rocksdb_thread_running_.load()) { + LOG(WARNING) << "RocksDB background thread is already running"; + return; + } + rocksdb_thread_running_.store(true); + rocksdb_back_thread_ = std::make_unique(&PikaReplicaManager::RocksDBThreadLoop, this); + LOG(INFO) << "RocksDB background thread started"; +} + +void PikaReplicaManager::StopRocksDBThread() { + if (!rocksdb_thread_running_.load()) { + return; + } + rocksdb_thread_running_.store(false); + // Notify the condition variable to wake up the thread + { + std::lock_guard lock(rocksdb_thread_mutex_); + rocksdb_thread_cv_.notify_one(); + } + if (rocksdb_back_thread_ && rocksdb_back_thread_->joinable()) { + rocksdb_back_thread_->join(); + } + LOG(INFO) << "RocksDB background thread stopped"; +} + +void PikaReplicaManager::RocksDBThreadLoop() { + LOG(INFO) << "RocksDB_back_thread started"; + while (rocksdb_thread_running_.load()) { + try { + std::unique_lock lock(rocksdb_thread_mutex_); + // Wait for CommittedID notification or shutdown signal + rocksdb_thread_cv_.wait(lock, [this] { + bool has_pending; + { + std::lock_guard pending_lock(pending_batch_groups_mutex_); + has_pending = !pending_batch_groups_.empty(); + } + return !rocksdb_thread_running_.load() || has_pending; + }); + // Check if we should exit + if (!rocksdb_thread_running_.load()) { + break; + } + lock.unlock(); + // Get current committed ID + LogOffset committed_id; + { + std::lock_guard id_lock(last_committed_id_mutex_); + committed_id = last_committed_id_; + } + + // Process committed BatchGroups + if (committed_id.IsValid()) { + size_t groups_processed = ProcessCommittedBatchGroups(committed_id); + if (groups_processed > 0) { + LOG(INFO) << "Processed " << groups_processed << " committed BatchGroups"; + } + } + } catch (const std::exception& e) { + LOG(ERROR) << "Error in RocksDB thread loop: " << e.what(); + std::this_thread::sleep_for(std::chrono::milliseconds(100)); + } + } + LOG(INFO) << "RocksDB_back_thread ended"; +} + +size_t PikaReplicaManager::ProcessCommittedBatchGroups(const LogOffset& committed_id) { + std::queue> groups_to_process; + // Get pending BatchGroups and separate committed from uncommitted + { + std::lock_guard lock(pending_batch_groups_mutex_); + while (!pending_batch_groups_.empty()) { + auto batch_group = pending_batch_groups_.front(); + // Check if this BatchGroup is committed by comparing its end_offset with committed_id + bool is_committed = (batch_group->end_offset <= committed_id); + if (is_committed) { + groups_to_process.push(batch_group); + pending_batch_groups_.pop(); + LOG(INFO) << "BatchGroup with end_offset " << batch_group->end_offset.ToString() + << " is committed (committed_id: " << committed_id.ToString() << ")"; + } else { + // First uncommitted BatchGroup found, stop processing + static LogOffset last_uncommitted_offset; + if (last_uncommitted_offset != batch_group->end_offset) { + LOG(INFO) << "BatchGroup with end_offset " << batch_group->end_offset.ToString() + << " is not yet committed (committed_id: " << committed_id.ToString() << ")"; + last_uncommitted_offset = batch_group->end_offset; + } + break; + } + } + } + // Store the number of groups to process for return value + size_t groups_count = groups_to_process.size(); + // Process committed BatchGroups + while (!groups_to_process.empty()) { + auto batch_group = groups_to_process.front(); + groups_to_process.pop(); + LOG(INFO) << "Processing committed BatchGroup with " << batch_group->BatchCount() + << " batches for RocksDB Put operations and client callbacks"; + + // Process all batches in this group + for (const auto& batch : batch_group->batches) { + // Execute RocksDB Put operations for each command in the batch + for (size_t i = 0; i < batch->commands.size(); ++i) { + const auto& cmd_ptr = batch->commands[i]; + try { + // Execute the command (this will perform the RocksDB Put operation) + cmd_ptr->Execute(); + if (cmd_ptr->res().ok()) { + LOG(INFO) << "Successfully executed RocksDB Put for command: " << cmd_ptr->name(); + // Execute callback with BatchGroup's end_offset (all commands in the group get the same offset) + if (i < batch->callbacks.size() && batch->callbacks[i]) { + batch->callbacks[i](batch_group->end_offset, pstd::Status::OK()); + } + } else { + LOG(WARNING) << "Command " << cmd_ptr->name() << " execution failed: " << cmd_ptr->res().message(); + // Execute callback with command error, still use BatchGroup's end_offset + if (i < batch->callbacks.size() && batch->callbacks[i]) { + batch->callbacks[i](batch_group->end_offset, pstd::Status::Corruption("Command execution failed: " + cmd_ptr->res().message())); + } + } + } catch (const std::exception& e) { + LOG(ERROR) << "Exception during command execution: " << e.what(); + // Execute callback with exception error, still use BatchGroup's end_offset + if (i < batch->callbacks.size() && batch->callbacks[i]) { + batch->callbacks[i](batch_group->end_offset, pstd::Status::Corruption("Exception during execution: " + std::string(e.what()))); + } + } + } + } + } + return groups_count; +} + +void PikaReplicaManager::NotifyCommittedID(const LogOffset& committed_id) { + { + std::lock_guard lock(last_committed_id_mutex_); + last_committed_id_ = committed_id; + } + // Notify RocksDB thread + { + std::lock_guard lock(rocksdb_thread_mutex_); + rocksdb_thread_cv_.notify_one(); + } + LOG(INFO) << "Notified RocksDB thread of new CommittedID: " << committed_id.ToString(); +} diff --git a/src/pika_server.cc b/src/pika_server.cc index e688202c75..1edbeaa789 100644 --- a/src/pika_server.cc +++ b/src/pika_server.cc @@ -597,17 +597,10 @@ void PikaServer::DeleteSlave(int fd) { } if (slave_num == 0) { - // Check if slaveof is configured, if so, do not remove MASTER role - // Because a Pika configured as a slave is still the master node for its clients - std::string slaveof = g_pika_conf->slaveof(); - if (slaveof.empty()) { - // Only remove MASTER role when slaveof is not configured - std::lock_guard l(state_protector_); - last_role_ = role_; - role_ &= ~PIKA_ROLE_MASTER; - leader_protected_mode_ = false; // explicitly cancel protected mode - LOG(INFO) << "DeleteSlave: Removed MASTER role for standalone node"; - } + std::lock_guard l(state_protector_); + last_role_ = role_; + role_ &= ~PIKA_ROLE_MASTER; + leader_protected_mode_ = false; // explicitly cancel protected mode } } @@ -1114,13 +1107,7 @@ int PikaServer::SendToPeer() { return g_pika_rm->ConsumeWriteQueue(); } void PikaServer::SignalAuxiliary() { pika_auxiliary_thread_->cv_.notify_one(); } -Status PikaServer::TriggerSendBinlogSync() { - // Only execute on master nodes - if (!(role_ & PIKA_ROLE_MASTER)) { - return Status::OK(); - } - return g_pika_rm->WakeUpBinlogSync(); -} +Status PikaServer::TriggerSendBinlogSync() { return g_pika_rm->WakeUpBinlogSync(); } int PikaServer::PubSubNumPat() { return pika_pubsub_thread_->PubSubNumPat(); } diff --git a/src/pika_stable_log.cc b/src/pika_stable_log.cc index dcc781354d..b1e9fc278a 100644 --- a/src/pika_stable_log.cc +++ b/src/pika_stable_log.cc @@ -35,21 +35,6 @@ StableLog::StableLog(std::string db_name, std::string log_path) StableLog::~StableLog() = default; -std::shared_ptr StableLog::coordinator() { - // Get and return the coordinator pointer directly from SyncMasterDB - auto master_db = g_pika_rm->GetSyncMasterDBByName(DBInfo(db_name_)); - if (master_db) { - // Return a nullptr if GetCoordinator is not ready - try { - // Use aliasing constructor instead of dangerous custom deleter - return std::shared_ptr(master_db, &master_db->GetCoordinator()); - } catch (const std::exception& e) { - LOG(ERROR) << "Failed to get coordinator for " << db_name_ << ": " << e.what(); - return nullptr; - } - } - return nullptr; -} void StableLog::Leave() { Close(); RemoveStableLogDir(); diff --git a/tests/integration/clean_start.sh b/tests/integration/clean_start.sh index 589cfb7059..4a41b47561 100644 --- a/tests/integration/clean_start.sh +++ b/tests/integration/clean_start.sh @@ -38,7 +38,7 @@ echo "Replication setup successful." echo "Running benchmark..." -redis-benchmark -p 9301 -t set -r 100000 -n 100000 -c 500 --threads 4 +redis-benchmark -p 9301 -t set -r 100000 -n 100000 -c 10000 --threads 4 echo "Benchmark finished." echo -e "\n==== 主节点 INFO 日志 ====" From d6c0db3a7bc8c900f1d0e5866e90f03a32c526c0 Mon Sep 17 00:00:00 2001 From: YuCai18 <1512875381@qq.com> Date: Fri, 5 Sep 2025 11:18:25 +0800 Subject: [PATCH 4/5] fix some bugs --- include/pika_client_conn.h | 4 +- include/pika_command_collector.h | 2 +- include/pika_kv.h | 1 + src/pika_client_conn.cc | 27 ++++--- src/pika_command.cc | 59 ++++++++-------- src/pika_command_collector.cc | 2 +- src/pika_consensus.cc | 118 +++++++++++-------------------- src/pika_kv.cc | 14 ++++ src/pika_repl_server.cc | 2 +- src/pika_rm.cc | 81 +++++++++++---------- src/pika_server.cc | 12 +++- tests/integration/clean_start.sh | 3 +- 12 files changed, 160 insertions(+), 165 deletions(-) diff --git a/include/pika_client_conn.h b/include/pika_client_conn.h index bc4c28db6a..de01ce5034 100644 --- a/include/pika_client_conn.h +++ b/include/pika_client_conn.h @@ -113,7 +113,7 @@ class PikaClientConn : public net::RedisConn { std::vector> resp_array; std::shared_ptr time_stat_; - + void TryWriteResp(); private: net::ServerThread* const server_thread_; std::string current_db_; @@ -134,7 +134,7 @@ class PikaClientConn : public net::RedisConn { void ProcessMonitor(const PikaCmdArgsType& argv); void ExecRedisCmd(const PikaCmdArgsType& argv, std::shared_ptr& resp_ptr, bool cache_miss_in_rtc); - void TryWriteResp(); + // void TryWriteResp(); }; struct ClientInfo { diff --git a/include/pika_command_collector.h b/include/pika_command_collector.h index de4e19dafb..fe74bcf6ab 100644 --- a/include/pika_command_collector.h +++ b/include/pika_command_collector.h @@ -62,7 +62,7 @@ class PikaCommandCollector { * @param callback callback function after processing is completed * @return whether the addition was successful */ - bool AddCommand(const std::shared_ptr& cmd_ptr, CommandCallback callback); + bool AddCommand(std::shared_ptr cmd_ptr, CommandCallback callback); /** * @brief Set the batch max wait time diff --git a/include/pika_kv.h b/include/pika_kv.h index 82939d29d9..8e5e5a6f32 100644 --- a/include/pika_kv.h +++ b/include/pika_kv.h @@ -24,6 +24,7 @@ class SetCmd : public Cmd { res.push_back(key_); return res; } + ~SetCmd() {} void Do() override; void DoUpdateCache() override; void DoThroughDB() override; diff --git a/src/pika_client_conn.cc b/src/pika_client_conn.cc index 0ad970311c..59f196a189 100644 --- a/src/pika_client_conn.cc +++ b/src/pika_client_conn.cc @@ -229,14 +229,16 @@ std::shared_ptr PikaClientConn::DoCmd(const PikaCmdArgsType& argv, const st if (command_collector) { // Create callback to handle command completion auto callback = [this, c_ptr](const LogOffset& offset, pstd::Status status) { - if (status.ok()) { - // Command was successfully processed through the pipeline - LOG(INFO) << "Command " << c_ptr->name() << " completed via CommandCollector"; - } else { - // Set error response - c_ptr->res().SetRes(CmdRes::kErrOther, "Command processing failed: " + status.ToString()); - LOG(ERROR) << "Command " << c_ptr->name() << " failed in CommandCollector: " << status.ToString(); - } + LOG(INFO) << "Command completed"; + auto pc = dynamic_cast(c_ptr->GetConn().get()); + if (pc) { + auto resp_ptr = c_ptr->GetResp(); + if (resp_ptr) { + *resp_ptr = std::move(c_ptr->res().message()); + } + pc->resp_num--; + pc->TryWriteResp(); + } }; // Add command to collector for batch processing @@ -591,8 +593,13 @@ void PikaClientConn::ExecRedisCmd(const PikaCmdArgsType& argv, std::shared_ptr cmd_ptr = DoCmd(argv, opt, resp_ptr, cache_miss_in_rtc); - *resp_ptr = std::move(cmd_ptr->res().message()); - resp_num--; + // *resp_ptr = std::move(cmd_ptr->res().message()); + // resp_num--; + if (opt == kCmdNameSet) { + } else { + *resp_ptr = std::move(cmd_ptr->res().message()); + resp_num--; + } } std::queue> PikaClientConn::GetTxnCmdQue() { return txn_cmd_que_; } diff --git a/src/pika_command.cc b/src/pika_command.cc index fa27505844..d72f5c355b 100644 --- a/src/pika_command.cc +++ b/src/pika_command.cc @@ -952,35 +952,36 @@ bool Cmd::DoReadCommandInCache() { void Cmd::DoBinlog() { - if (res().ok() && is_write() && g_pika_conf->write_binlog()) { - std::shared_ptr conn_ptr = GetConn(); - std::shared_ptr resp_ptr = GetResp(); - // Consider that dummy cmd appended by system, both conn and resp are null. - if ((!conn_ptr || !resp_ptr) && (name_ != kCmdDummy)) { - if (!conn_ptr) { - LOG(WARNING) << sync_db_->SyncDBInfo().ToString() << " conn empty."; - } - if (!resp_ptr) { - LOG(WARNING) << sync_db_->SyncDBInfo().ToString() << " resp empty."; - } - res().SetRes(CmdRes::kErrOther); - return; - } - - Status s = sync_db_->ConsensusProposeLog(shared_from_this()); - if (!s.ok()) { - if(g_pika_server->IsConsistency()&&s.IsTimeout()){ - res().SetRes(CmdRes::kConsistencyTimeout, "Timeout waiting for consistency"); - LOG(WARNING) << sync_db_->SyncDBInfo().ToString() << " Slave node consistency timeout" - << s.ToString(); - }else{ - LOG(WARNING) << sync_db_->SyncDBInfo().ToString() << " Writing binlog failed, maybe no space left on device " - << s.ToString(); - res().SetRes(CmdRes::kErrOther, s.ToString()); - } - return; - } - } +// if (res().ok() && is_write() && g_pika_conf->write_binlog()) { +// std::shared_ptr conn_ptr = GetConn(); +// std::shared_ptr resp_ptr = GetResp(); +// // Consider that dummy cmd appended by system, both conn and resp are null. +// if ((!conn_ptr || !resp_ptr) && (name_ != kCmdDummy)) { +// if (!conn_ptr) { +// LOG(WARNING) << sync_db_->SyncDBInfo().ToString() << " conn empty."; +// } +// if (!resp_ptr) { +// LOG(WARNING) << sync_db_->SyncDBInfo().ToString() << " resp empty."; +// } +// res().SetRes(CmdRes::kErrOther); +// return; +// } + +// Status s = sync_db_->ConsensusProposeLog(shared_from_this()); +// if (!s.ok()) { +// if(g_pika_server->IsConsistency()&&s.IsTimeout()){ +// res().SetRes(CmdRes::kConsistencyTimeout, "Timeout waiting for consistency"); +// LOG(WARNING) << sync_db_->SyncDBInfo().ToString() << " Slave node consistency timeout" +// << s.ToString(); +// }else{ +// LOG(WARNING) << sync_db_->SyncDBInfo().ToString() << " Writing binlog failed, maybe no space left on device " +// << s.ToString(); +// res().SetRes(CmdRes::kErrOther, s.ToString()); +// } +// return; +// } +// } + return; } #define PIKA_STAGE_DURATION_OUTPUT(duration) \ diff --git a/src/pika_command_collector.cc b/src/pika_command_collector.cc index fb206eb9b1..8e64f814cd 100644 --- a/src/pika_command_collector.cc +++ b/src/pika_command_collector.cc @@ -30,7 +30,7 @@ PikaCommandCollector::~PikaCommandCollector() { << " commands, " << total_batches_.load() << " batches"; } -bool PikaCommandCollector::AddCommand(const std::shared_ptr& cmd_ptr, CommandCallback callback) { +bool PikaCommandCollector::AddCommand(std::shared_ptr cmd_ptr, CommandCallback callback) { if (!cmd_ptr || !cmd_ptr->is_write()) { LOG(WARNING) << "Attempt to add non-write command to CommandCollector"; return false; diff --git a/src/pika_consensus.cc b/src/pika_consensus.cc index ed20e4811b..9900c5a41f 100644 --- a/src/pika_consensus.cc +++ b/src/pika_consensus.cc @@ -824,7 +824,6 @@ bool ConsensusCoordinator::checkFinished(const LogOffset& offset) { //// pacificA private: Status ConsensusCoordinator::PersistAppendBinlog(const std::shared_ptr& cmd_ptr, LogOffset& cur_offset) { - std::lock_guard l(order_mu_); std::string content = cmd_ptr->ToRedisProtocol(); std::string binlog = std::string(); LogOffset offset = LogOffset(); @@ -843,12 +842,16 @@ Status ConsensusCoordinator::PersistAppendBinlog(const std::shared_ptr& cmd // If successful, append the log entry to the logs // TODO: 这里logs_的appendlog操作和上边的stable_logger_->Logger()->Put不是原子的,可能导致offset大的先被追加到logs_中, // 多线程写入的时候窗口会对不上,最终主从断开连接。需要加逻辑保证原子性 - LOG(INFO) << "PersistAppendBinlog: About to append to logs_, current size=" << logs_->Size() - << ", offset=" << cur_offset.ToString() << ", cmd=" << cmd_ptr->name(); - logs_->AppendLog(Log::LogItem(cur_offset, cmd_ptr, binlog)); - LOG(INFO) << "PersistAppendBinlog: Successfully appended to logs_, new size=" << logs_->Size(); - - SetPreparedId(cur_offset); + //LOG(INFO) << "PersistAppendBinlog: About to append to logs_, current size=" << logs_->Size() + // << ", offset=" << cur_offset.ToString() << ", cmd=" << cmd_ptr->name(); + // logs_->AppendLog(Log::LogItem(cur_offset, cmd_ptr, binlog)); + //LOG(INFO) << "PersistAppendBinlog: Successfully appended to logs_, new size=" << logs_->Size(); + { + std::lock_guard l(order_mu_); + // Append to logs_ under order lock to maintain ordering + logs_->AppendLog(Log::LogItem(cur_offset, cmd_ptr, binlog)); + SetPreparedId(cur_offset); + } return stable_logger_->Logger()->IsOpened(); } @@ -868,7 +871,7 @@ Status ConsensusCoordinator::AppendEntries(const std::shared_ptr& cmd_ptr, return s; } - g_pika_server->SignalAuxiliary(); + // g_pika_server->SignalAuxiliary(); return Status::OK(); } Status ConsensusCoordinator::AppendSlaveEntries(const std::shared_ptr& cmd_ptr, const BinlogItem& attribute) { @@ -922,6 +925,23 @@ Status ConsensusCoordinator::UpdateCommittedID() { } } } + // if (!has_active_slaves) { + // LogOffset master_prepared_id = GetPreparedId(); + // if (master_prepared_id.IsValid() && master_prepared_id >= GetCommittedId()) { + // SetCommittedId(master_prepared_id); + // LOG(INFO) << "PacificA update CommittedID (no active slaves): " << GetCommittedId().ToString() + // << ", Total slaves: " << total_slaves + // << ", kSlaveBinlogSync: " << binlog_sync_slaves + // << ", Other states: " << other_state_slaves; + // } else { + // LOG(INFO) << "PacificA update CommittedID: No active slaves, keeping current CommittedID: " << GetCommittedId().ToString() + // << ", Total slaves: " << total_slaves + // << ", kSlaveBinlogSync: " << binlog_sync_slaves + // << ", Other states: " << other_state_slaves; + // } + // // g_pika_rm->NotifyCommittedID(GetCommittedId()); + // return Status::OK(); + // } if (slave_prepared_id < GetCommittedId()) { LOG(WARNING) << "Error: slave_prepared_id (" << slave_prepared_id.ToString() << ") < master_committedId (" << GetCommittedId().ToString() << ")"; @@ -953,7 +973,9 @@ Status ConsensusCoordinator::ProcessCoordination() { // Execute the operation of writing to DB Status ConsensusCoordinator::ApplyBinlog(const std::shared_ptr& cmd_ptr) { auto opt = cmd_ptr->argv()[0]; + LOG(INFO) << "[ApplyBinlog] Received command: " << opt << " for db: " << db_name_; if (pstd::StringToLower(opt) != kCmdNameFlushdb) { + LOG(INFO) << "[ApplyBinlog] Scheduling async task for " << opt; InternalApplyFollower(cmd_ptr); } else { int32_t wait_ms = 250; @@ -990,86 +1012,28 @@ void ConsensusCoordinator::BatchInternalApplyFollower(const std::vector slave_ptr, std::string db_name) { std::vector tasks; - const int MAX_BATCH_SIZE = 100; // Maximum number of logs to send in a single batch - - // Get current committed_id to ensure it's sent to the slave - LogOffset current_committed_id = GetCommittedId(); - LOG(INFO) << "SendBinlog: [Thread " << std::this_thread::get_id() << "] Current committed_id: " << current_committed_id.ToString() - << ", sending to slave " << slave_ptr->Ip() << ":" << slave_ptr->Port() - << ", logs_ addr: " << logs_.get() << ", db_name: " << db_name_; // Check if there are new log entries that need to be sent to the slave - LOG(INFO) << "SendBinlog: logs_->LastOffset()=" << logs_->LastOffset().ToString() - << ", slave_ptr->acked_offset=" << slave_ptr->acked_offset.ToString() - << ", logs_->Size()=" << logs_->Size(); - - if (logs_->Size() > 0 && logs_->LastOffset() >= slave_ptr->acked_offset) { + if (logs_->LastOffset() >= slave_ptr->acked_offset) { // Find the index of the log entry corresponding to the slave's acknowledged offset int index = logs_->FindOffset(slave_ptr->acked_offset); - int entries_to_send = logs_->Size() - index; - LOG(INFO) << "SendBinlog: Found " << entries_to_send << " new log entries to send, " - << "starting from index " << index << " of " << logs_->Size(); - if (index < logs_->Size()) { - // Send log entries in optimized batches - RmNode rm_node(slave_ptr->Ip(), slave_ptr->Port(), db_name, slave_ptr->SessionId()); - - // For large batches, use specialized batch handling - if (entries_to_send > MAX_BATCH_SIZE) { - LOG(INFO) << "SendBinlog: Using optimized batch sending for " << entries_to_send << " entries"; - - // Process in chunks of MAX_BATCH_SIZE - for (int batch_start = index; batch_start < logs_->Size(); batch_start += MAX_BATCH_SIZE) { - int batch_end = std::min(batch_start + MAX_BATCH_SIZE, logs_->Size()); - std::vector batch_tasks; - - for (int i = batch_start; i < batch_end; ++i) { - Log::LogItem item = logs_->At(i); - WriteTask task(rm_node, BinlogChip(item.offset, item.binlog_), item.offset, current_committed_id); - batch_tasks.push_back(task); - } - - g_pika_rm->ProduceWriteQueue(slave_ptr->Ip(), slave_ptr->Port(), db_name, batch_tasks); - LOG(INFO) << "SendBinlog: Sent batch " << (batch_start - index) / MAX_BATCH_SIZE + 1 - << " with " << (batch_end - batch_start) << " entries"; - } - } else { - // Send all entries in a single batch - for (int i = index; i < logs_->Size(); ++i) { - Log::LogItem item = logs_->At(i); - WriteTask task(rm_node, BinlogChip(item.offset, item.binlog_), item.offset, current_committed_id); - tasks.push_back(task); - } + for (int i = index; i < logs_->Size(); ++i) { + const Log::LogItem& item = logs_->At(i); + + slave_ptr->SetLastSendTime(pstd::NowMicros()); + + RmNode rm_node(slave_ptr->Ip(), slave_ptr->Port(), slave_ptr->DBName(), slave_ptr->SessionId()); + WriteTask task(rm_node, BinlogChip(item.offset, item.binlog_), slave_ptr->sent_offset, GetCommittedId()); + tasks.emplace_back(std::move(task)); + + slave_ptr->sent_offset = item.offset; } - } else { - LOG(INFO) << "SendBinlog: No new log entries to send, index " << index << " is out of range (logs size: " << logs_->Size() << ")"; - } - } else { - if (logs_->Size() == 0) { - LOG(INFO) << "SendBinlog: No logs available yet (logs_->Size()=0), will send empty binlog to maintain connection"; - } else { - LOG(INFO) << "SendBinlog: Slave is already up to date, last offset: " << logs_->LastOffset().ToString() - << ", slave acked offset: " << slave_ptr->acked_offset.ToString(); } } - // Only send empty binlog if there are no actual log entries to send - // This prevents the deadlock where master waits for slave ACK and slave waits for master data - if (tasks.empty() && logs_->Size() == 0) { - // LOG(INFO) << "SendBinlog: Sending empty binlog with current committed_id: " << current_committed_id.ToString(); - RmNode rm_node(slave_ptr->Ip(), slave_ptr->Port(), db_name, slave_ptr->SessionId()); - // Create an empty WriteTask that includes the current committed_id - WriteTask empty_task(rm_node, BinlogChip(LogOffset(), ""), LogOffset(), current_committed_id); - tasks.push_back(empty_task); - } - - // Send the tasks to the slave if (!tasks.empty()) { - LOG(INFO) << "SendBinlog: Sending " << tasks.size() << " tasks to slave " << slave_ptr->Ip() << ":" << slave_ptr->Port(); - extern std::unique_ptr g_pika_rm; g_pika_rm->ProduceWriteQueue(slave_ptr->Ip(), slave_ptr->Port(), db_name, tasks); - } else { - LOG(INFO) << "SendBinlog: No tasks to send to slave " << slave_ptr->Ip() << ":" << slave_ptr->Port(); } return Status::OK(); } diff --git a/src/pika_kv.cc b/src/pika_kv.cc index 1c1abdd4cf..79dc6cd291 100644 --- a/src/pika_kv.cc +++ b/src/pika_kv.cc @@ -15,6 +15,20 @@ extern std::unique_ptr g_pika_conf; /* SET key value [NX] [XX] [EX ] [PX ] */ +// SetCmd::~SetCmd() { +// auto tmp_conn = GetConn(); +// if (!tmp_conn) { +// return; +// } + +// auto pc = dynamic_cast(tmp_conn.get()); +// std::shared_ptr resp_ptr = std::make_shared(); +// *resp_ptr = std::move(res().message()); +// pc->resp_num--; +// pc->resp_array.push_back(resp_ptr); +// pc->TryWriteResp(); +// LOG(INFO) << "SetCmd::~SetCmd() is completed"; +// } void SetCmd::DoInitial() { if (!CheckArg(argv_.size())) { res_.SetRes(CmdRes::kWrongNum, kCmdNameSet); diff --git a/src/pika_repl_server.cc b/src/pika_repl_server.cc index c8f1c9f9dc..2617a59aab 100644 --- a/src/pika_repl_server.cc +++ b/src/pika_repl_server.cc @@ -17,7 +17,7 @@ extern PikaServer* g_pika_server; extern std::unique_ptr g_pika_rm; PikaReplServer::PikaReplServer(const std::set& ips, int port, int cron_interval) { - server_tp_ = std::make_unique(PIKA_REPL_SERVER_TP_SIZE, 100000, "PikaReplServer"); + server_tp_ = std::make_unique(1, 100000, "PikaReplServer"); pika_repl_server_thread_ = std::make_unique(ips, port, cron_interval); pika_repl_server_thread_->set_thread_name("PikaReplServer"); } diff --git a/src/pika_rm.cc b/src/pika_rm.cc index de337fd157..a411e906bc 100644 --- a/src/pika_rm.cc +++ b/src/pika_rm.cc @@ -249,8 +249,8 @@ Status SyncMasterDB::WakeUpSlaveBinlogSync() { } if (!s.ok()) { to_del.push_back(slave_ptr); - LOG(WARNING) << "WakeUpSlaveBinlogSync failed, marking for deletion: " - << slave_ptr->ToStringStatus() << " - " << s.ToString(); + // LOG(WARNING) << "WakeUpSlaveBinlogSync failed, marking for deletion: " + // << slave_ptr->ToStringStatus() << " - " << s.ToString(); } } } @@ -580,8 +580,8 @@ Status SyncMasterDB::ConsensusProposeLog(const std::shared_ptr& cmd_ptr) { // } // return Status::Timeout("No consistency achieved within 10 seconds"); - LOG(INFO) << "ConsensusProposeLog: Successfully appended cmd " << cmd_ptr->name() - << " with offset " << offset.ToString() << ", delegating to RocksDBThreadLoop"; + //LOG(INFO) << "ConsensusProposeLog: Successfully appended cmd " << cmd_ptr->name() + // << " with offset " << offset.ToString() << ", delegating to RocksDBThreadLoop"; return Status::OK(); } @@ -798,9 +798,12 @@ void PikaReplicaManager::ProduceWriteQueue(const std::string& ip, int port, std: const std::vector& tasks) { std::lock_guard l(write_queue_mu_); std::string index = ip + ":" + std::to_string(port); + LOG(INFO) << "[Queue] Entering ProduceWriteQueue for " << ip << ":" << port << ", db: " << db_name << ", current queue size: " << write_queues_.size() << ", tasks to add: " << tasks.size(); for (auto& task : tasks) { write_queues_[index][db_name].push(task); } + LOG(INFO) << "[Queue] Added " << tasks.size() << " tasks to queue for " << ip << ":" << port << ", db: " << db_name << ", new queue size: " << write_queues_[index][db_name].size(); + LOG(INFO) << "[Queue] Exiting ProduceWriteQueue"; } int PikaReplicaManager::ConsumeWriteQueue() { @@ -850,9 +853,11 @@ int PikaReplicaManager::ConsumeWriteQueue() { for (auto& to_send : iter.second) { Status s = pika_repl_server_->SendSlaveBinlogChips(ip, port, to_send); if (!s.ok()) { - LOG(WARNING) << "send binlog to " << ip << ":" << port << " failed, " << s.ToString(); + LOG(WARNING) << "SendSlaveBinlogChips to " << iter.first << " failed, " << s.ToString(); to_delete.push_back(iter.first); continue; + } else { + LOG(INFO) << "[Queue] SendSlaveBinlogChips to " << iter.first << " success."; } } } @@ -863,6 +868,9 @@ int PikaReplicaManager::ConsumeWriteQueue() { write_queues_.erase(del_queue); } } + // LOG(INFO) << "[Consume] Entering ConsumeWriteQueue, total queues: " << write_queues_.size(); + // LOG(INFO) << "[Consume] Consumed " << counter << " tasks"; + // LOG(INFO) << "[Consume] Exiting ConsumeWriteQueue"; return counter; } @@ -1448,43 +1456,23 @@ void PikaReplicaManager::ProcessCommandBatches(const std::vectorGetISConsistency()) { - // For consistency mode, use ConsensusProposeLog directly - pstd::Status s = sync_db->ConsensusProposeLog(cmd_ptr); - if (!s.ok()) { - LOG(ERROR) << "Failed to append command " << cmd_ptr->name() << ": " << s.ToString(); - // Call callbacks with error for remaining commands - for (size_t error_idx = cmd_idx; error_idx < all_commands.size(); ++error_idx) { - auto& batch_info = command_to_batch_map[error_idx]; - auto batch = batch_info.first; - size_t batch_cmd_idx = batch_info.second; - if (batch_cmd_idx < batch->callbacks.size() && batch->callbacks[batch_cmd_idx]) { - batch->callbacks[batch_cmd_idx](LogOffset(), s); - } - } - return; - } - // Get the prepared ID as the offset - cmd_offset = sync_db->GetPreparedId(); - } else { - // For non-consistency mode, use ConsensusProposeLog as well - pstd::Status s = sync_db->ConsensusProposeLog(cmd_ptr); - if (!s.ok()) { - LOG(ERROR) << "Failed to propose command " << cmd_ptr->name() << ": " << s.ToString(); - // Call callbacks with error for remaining commands - for (size_t error_idx = cmd_idx; error_idx < all_commands.size(); ++error_idx) { - auto& batch_info = command_to_batch_map[error_idx]; - auto batch = batch_info.first; - size_t batch_cmd_idx = batch_info.second; - if (batch_cmd_idx < batch->callbacks.size() && batch->callbacks[batch_cmd_idx]) { - batch->callbacks[batch_cmd_idx](LogOffset(), s); - } + pstd::Status s = sync_db->ConsensusProposeLog(cmd_ptr); + if (!s.ok()) { + LOG(ERROR) << "Failed to " << (sync_db->GetISConsistency() ? "append" : "propose") + << " command " << cmd_ptr->name() << ": " << s.ToString(); + // Call callbacks with error for remaining commands + for (size_t error_idx = cmd_idx; error_idx < all_commands.size(); ++error_idx) { + auto& batch_info = command_to_batch_map[error_idx]; + auto batch = batch_info.first; + size_t batch_cmd_idx = batch_info.second; + if (batch_cmd_idx < batch->callbacks.size() && batch->callbacks[batch_cmd_idx]) { + batch->callbacks[batch_cmd_idx](LogOffset(), s); } - return; } - // Get the prepared ID as the offset - cmd_offset = sync_db->GetPreparedId(); + return; } + // Get the prepared ID as the offset + cmd_offset = sync_db->GetPreparedId(); all_offsets[cmd_idx] = cmd_offset; @@ -1628,21 +1616,31 @@ size_t PikaReplicaManager::ProcessCommittedBatchGroups(const LogOffset& committe while (!groups_to_process.empty()) { auto batch_group = groups_to_process.front(); groups_to_process.pop(); - LOG(INFO) << "Processing committed BatchGroup with " << batch_group->BatchCount() - << " batches for RocksDB Put operations and client callbacks"; + //LOG(INFO) << "Processing committed BatchGroup with " << batch_group->BatchCount() + // << " batches for RocksDB Put operations and client callbacks"; // Process all batches in this group for (const auto& batch : batch_group->batches) { // Execute RocksDB Put operations for each command in the batch for (size_t i = 0; i < batch->commands.size(); ++i) { const auto& cmd_ptr = batch->commands[i]; + LOG(INFO) << "[RocksDB] RocksDBThreadLoop" << "Processing BatchGroup with end_offset: " << batch_group->end_offset.ToString(); try { // Execute the command (this will perform the RocksDB Put operation) cmd_ptr->Execute(); + // auto tmp_conn = cmd_ptr->GetConn(); + // auto pc = dynamic_cast(tmp_conn.get()); + // std::shared_ptr resp_ptr = std::make_shared(); + // *resp_ptr = std::move(cmd_ptr->res().message()); + // pc->resp_num--; + // pc->resp_array.push_back(resp_ptr); + // pc->TryWriteResp(); if (cmd_ptr->res().ok()) { LOG(INFO) << "Successfully executed RocksDB Put for command: " << cmd_ptr->name(); // Execute callback with BatchGroup's end_offset (all commands in the group get the same offset) if (i < batch->callbacks.size() && batch->callbacks[i]) { + LOG(INFO) << "[Callback] RocksDBThreadLoop" + << ", Executing callback for end_offset: " << batch_group->end_offset.ToString(); batch->callbacks[i](batch_group->end_offset, pstd::Status::OK()); } } else { @@ -1662,6 +1660,7 @@ size_t PikaReplicaManager::ProcessCommittedBatchGroups(const LogOffset& committe } } } + return groups_count; } diff --git a/src/pika_server.cc b/src/pika_server.cc index 1edbeaa789..74f439060a 100644 --- a/src/pika_server.cc +++ b/src/pika_server.cc @@ -1105,9 +1105,17 @@ std::unordered_map PikaServer::ServerAllDBStat() { re int PikaServer::SendToPeer() { return g_pika_rm->ConsumeWriteQueue(); } -void PikaServer::SignalAuxiliary() { pika_auxiliary_thread_->cv_.notify_one(); } +void PikaServer::SignalAuxiliary() { + // LOG(INFO) << "[Signal] SignalAuxiliary called, notifying auxiliary thread"; + pika_auxiliary_thread_->cv_.notify_one(); +} -Status PikaServer::TriggerSendBinlogSync() { return g_pika_rm->WakeUpBinlogSync(); } +Status PikaServer::TriggerSendBinlogSync() { + // LOG(INFO) << "[Trigger] TriggerSendBinlogSync called, calling WakeUpBinlogSync"; + Status s = g_pika_rm->WakeUpBinlogSync(); + // LOG(INFO) << "[Trigger] WakeUpBinlogSync result: " << s.ToString(); + return s; +} int PikaServer::PubSubNumPat() { return pika_pubsub_thread_->PubSubNumPat(); } diff --git a/tests/integration/clean_start.sh b/tests/integration/clean_start.sh index 4a41b47561..f527ccedfc 100644 --- a/tests/integration/clean_start.sh +++ b/tests/integration/clean_start.sh @@ -38,7 +38,8 @@ echo "Replication setup successful." echo "Running benchmark..." -redis-benchmark -p 9301 -t set -r 100000 -n 100000 -c 10000 --threads 4 +# redis-cli -p 9301 set key "12313" +redis-benchmark -p 9301 -t set -n 100000 -c 10 --threads 1 echo "Benchmark finished." echo -e "\n==== 主节点 INFO 日志 ====" From 98e482136f647c2c79bf1fe1847399260c82a232 Mon Sep 17 00:00:00 2001 From: wuxianrong Date: Tue, 16 Sep 2025 15:34:24 +0800 Subject: [PATCH 5/5] constst pika --- CMakeLists.txt | 4 +- conf/pika.conf | 27 ++++---- include/pika_binlog.h | 1 + include/pika_consensus.h | 4 +- include/pika_rm.h | 7 ++- src/net/src/bg_thread.cc | 3 +- src/net/src/pb_conn.cc | 12 +++- src/pika_auxiliary_thread.cc | 2 + src/pika_binlog.cc | 4 +- src/pika_client_conn.cc | 3 +- src/pika_command_collector.cc | 2 +- src/pika_command_queue.cc | 12 ++-- src/pika_consensus.cc | 53 +++++++++++----- src/pika_repl_bgworker.cc | 22 ++++--- src/pika_repl_client_conn.cc | 8 +-- src/pika_repl_server_conn.cc | 2 + src/pika_rm.cc | 114 ++++++++++++++++++++++------------ src/pstd/src/env.cc | 20 +++++- 18 files changed, 202 insertions(+), 98 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index f3bb015d5c..491f40f22b 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -457,9 +457,9 @@ set(LZ4_INCLUDE_DIR ${INSTALL_INCLUDEDIR}) ExternalProject_Add(zlib DEPENDS URL - https://github.com/madler/zlib/releases/download/v1.2.13/zlib-1.2.13.tar.gz + https://github.com/madler/zlib/releases/download/v1.3.1/zlib-1.3.1.tar.gz URL_HASH - MD5=9b8aa094c4e5765dabf4da391f00d15c + MD5=9855b6d802d7fe5b7bd5b196a2271655 DOWNLOAD_NO_PROGRESS 1 UPDATE_COMMAND diff --git a/conf/pika.conf b/conf/pika.conf index 6486574996..3475cd0034 100644 --- a/conf/pika.conf +++ b/conf/pika.conf @@ -42,7 +42,7 @@ slow-cmd-thread-pool-size : 1 admin-thread-pool-size : 2 # Slow cmd list e.g. hgetall, mset -slow-cmd-list : +slow-cmd-list : # List of commands considered as administrative. These commands will be handled by the admin thread pool. Modify this list as needed. # Default commands: info, ping, monitor @@ -95,7 +95,7 @@ proto-max-bulk-len : 512M # If <= 0, a proper value is automatically calculated. # (usually 1/8 of writer-buffer-size, rounded up to a multiple of 4KB) # Supported Units [K|M|G], arena-block-size default unit is in [bytes]. -arena-block-size : +arena-block-size : # Timeout of Pika's connection, counting down starts When there are no requests # on a connection (it enters sleep state), when the countdown reaches 0, the connection @@ -109,12 +109,12 @@ timeout : 60 # [NOTICE] If this admin password is the same as user password (including both being empty), # in this scenario, users are not subject to the restrictions imposed by the userblacklist. # PS: "user password" refers to value of the parameter below: userpass. -requirepass : +requirepass : # Password for replication verify, used for authentication when a slave # connects to a master to request replication. # [NOTICE] The value of this parameter must match the "requirepass" setting on the master. -masterauth : +masterauth : # The [password of user], which is empty by default. # [NOTICE] If this user password is the same as admin password (including both being empty), @@ -174,7 +174,7 @@ command-batch-enabled : yes # The Prefix of dump file's name. # All the files that generated by command "bgsave" will be name with this prefix. -dump-prefix : +dump-prefix : # daemonize [yes | no]. #daemonize : yes @@ -555,13 +555,13 @@ cache-num : 16 # cache-model 0:cache_none 1:cache_read cache-model : 1 # cache-type: string, set, zset, list, hash, bit -cache-type: string, set, zset, list, hash, bit +cache-type : string, set, zset, list, hash, bit # Set the maximum number of elements in the cache of the Set, list, Zset data types -cache-value-item-max-size: 1024 +cache-value-item-max-size : 1024 # Sets the maximum number of bytes for Key when the String data type is updated in the cache -max-key-size-in-cache: 1048576 +max-key-size-in-cache : 1048576 # Maximum number of keys in the zset redis cache # On the disk DB, a zset field may have many fields. In the memory cache, we limit the maximum @@ -593,10 +593,10 @@ cache-maxmemory : 10737418240 cache-maxmemory-policy : 1 # cache-maxmemory-samples -cache-maxmemory-samples: 5 +cache-maxmemory-samples : 5 # cache-lfu-decay-time -cache-lfu-decay-time: 1 +cache-lfu-decay-time : 1 # is possible to manage access to Pub/Sub channels with ACL rules as well. The @@ -650,12 +650,12 @@ cache-lfu-decay-time: 1 # 'internal-used-unfinished-full-sync' is used to generate a metric 'is_eligible_for_master_election' # which serves for the scenario of codis-pika cluster reelection # You'd better [DO NOT MODIFY IT UNLESS YOU KNOW WHAT YOU ARE DOING] -internal-used-unfinished-full-sync : +internal-used-unfinished-full-sync : # for wash data from 4.0.0 to 4.0.1 # https://github.com/OpenAtomFoundation/pika/issues/2886 # default value: true -wash-data: true +wash-data : true # Pika automatic compact compact strategy, a complement to rocksdb compact. # Trigger the compact background task periodically according to `compact-interval` @@ -690,3 +690,6 @@ dont-compact-sst-created-in-seconds : 20 # According to the number of sst files in rocksdb, # compact every `compact-every-num-of-files` file. best-delete-min-ratio : 10 +# Generated by ReplicationID CONFIG REWRITE +replication-id : d605afa1b464ddf3e571966482dd934ec7336a4bac49aa0a6b +run-id : 58ec490577bb7defd64f6fe642d7609af67896b1 diff --git a/include/pika_binlog.h b/include/pika_binlog.h index 778eb08b4f..26dd350c80 100644 --- a/include/pika_binlog.h +++ b/include/pika_binlog.h @@ -57,6 +57,7 @@ class Binlog : public pstd::noncopyable { pstd::Status Put(const std::string& item, LogOffset *cur_logoffset,std::string& binlog); pstd::Status IsOpened(); pstd::Status GetProducerStatus(uint32_t* filenum, uint64_t* pro_offset, uint32_t* term = nullptr, uint64_t* logic_id = nullptr); + pstd::WritableFile* GetQueue() { return queue_.get(); } /* * Set Producer pro_num and pro_offset with lock */ diff --git a/include/pika_consensus.h b/include/pika_consensus.h index f13a147054..94258ef638 100644 --- a/include/pika_consensus.h +++ b/include/pika_consensus.h @@ -288,7 +288,7 @@ class ConsensusCoordinator { context_->UpdateAppliedIndex(committed_id_); } notification_counter_.fetch_add(1); - LOG(INFO) << "SetCommittedId: Updated to " << offset.ToString(); + //LOG(INFO) << "Master SetCommittedId: Updated to " << offset.ToString(); } private: @@ -296,7 +296,7 @@ class ConsensusCoordinator { private: std::shared_mutex is_consistency_rwlock_; - bool is_consistency_ = false; + bool is_consistency_ = true; std::shared_mutex committed_id_rwlock_; LogOffset committed_id_ = LogOffset(); std::atomic notification_counter_{0}; diff --git a/include/pika_rm.h b/include/pika_rm.h index 7db1bbf322..a5e7a13519 100644 --- a/include/pika_rm.h +++ b/include/pika_rm.h @@ -33,8 +33,9 @@ #define kBinlogSendBatchNum 100 // unit seconds -#define kSendKeepAliveTimeout (2 * 1000000) -#define kRecvKeepAliveTimeout (20 * 1000000) +// WXR +#define kSendKeepAliveTimeout (100 * 1000000) +#define kRecvKeepAliveTimeout (200 * 1000000) class SyncDB { @@ -302,7 +303,7 @@ class PikaReplicaManager { // Last committed ID for RocksDB thread processing LogOffset last_committed_id_; - std::mutex last_committed_id_mutex_; + std::shared_mutex last_committed_id_mutex_; // Background thread processing methods void StartCommandQueueThread(); diff --git a/src/net/src/bg_thread.cc b/src/net/src/bg_thread.cc index b0835330f9..a15e600494 100644 --- a/src/net/src/bg_thread.cc +++ b/src/net/src/bg_thread.cc @@ -6,6 +6,7 @@ #include "net/include/bg_thread.h" #include #include +#include namespace net { @@ -79,7 +80,7 @@ void* BGThread::ThreadMain() { while (!should_stop()) { std::unique_lock lock(mu_); - rsignal_.wait(lock, [this]() { return !queue_.empty() || !timer_queue_.empty() || should_stop(); }); + rsignal_.wait(lock, [this]() { return !queue_.empty() || should_stop(); }); if (should_stop()) { break; diff --git a/src/net/src/pb_conn.cc b/src/net/src/pb_conn.cc index 15e4024da0..fae60f1ea7 100644 --- a/src/net/src/pb_conn.cc +++ b/src/net/src/pb_conn.cc @@ -42,7 +42,10 @@ ReadStatus PbConn::GetRequest() { case kHeader: { int quickack = 1; ssize_t nread = read(fd(), rbuf_ + cur_pos_, COMMAND_HEADER_LENGTH - cur_pos_); - setsockopt(fd(), IPPROTO_TCP, TCP_QUICKACK, &quickack, sizeof(quickack)); + #ifdef __linux__ + int quickack = 1; + setsockopt(fd(), IPPROTO_TCP, TCP_QUICKACK, &quickack, sizeof(quickack)); +#endif if (nread == -1) { if (errno == EAGAIN) { return kReadHalf; @@ -82,7 +85,10 @@ ReadStatus PbConn::GetRequest() { // read msg body ssize_t nread = read(fd(), rbuf_ + cur_pos_, remain_packet_len_); int quickack = 1; - setsockopt(fd(), IPPROTO_TCP, TCP_QUICKACK, &quickack, sizeof(quickack)); + #ifdef __linux__ + int quickack = 1; + setsockopt(fd(), IPPROTO_TCP, TCP_QUICKACK, &quickack, sizeof(quickack)); +#endif if (nread == -1) { if (errno == EAGAIN) { return kReadHalf; @@ -131,7 +137,7 @@ WriteStatus PbConn::SendReply() { while (item_len - write_buf_.item_pos_ > 0) { nwritten = write(fd(), item.data() + write_buf_.item_pos_, item_len - write_buf_.item_pos_); if (nwritten <= 0) { - LOG(ERROR) << "nwritten less than 0"; + //LOG(ERROR) << "nwritten less than 0"; break; } g_network_statistic->IncrReplOutputBytes(nwritten); diff --git a/src/pika_auxiliary_thread.cc b/src/pika_auxiliary_thread.cc index e94104b442..ebc02a42b0 100644 --- a/src/pika_auxiliary_thread.cc +++ b/src/pika_auxiliary_thread.cc @@ -35,11 +35,13 @@ void* PikaAuxiliaryThread::ThreadMain() { g_pika_server->CheckLeaderProtectedMode(); // TODO(whoiami) timeout + // 将增量数据写入 Write_queue_ s = g_pika_server->TriggerSendBinlogSync(); if (!s.ok()) { LOG(WARNING) << s.ToString(); } // send to peer + // 将 Write_queue 中的数据发送给从节点 int res = g_pika_server->SendToPeer(); if (res == 0) { // sleep 100 ms diff --git a/src/pika_binlog.cc b/src/pika_binlog.cc index e051108317..76cb70b64f 100644 --- a/src/pika_binlog.cc +++ b/src/pika_binlog.cc @@ -298,7 +298,9 @@ Status Binlog::EmitPhysicalRecord(RecordType t, const char* ptr, size_t n, int* if (s.ok()) { s = queue_->Append(pstd::Slice(ptr, n)); if (s.ok()) { - s = queue_->Flush(); + //LOG(INFO) << "EmitPhysicalRecord Flush"; + //s = queue_->Sync(); + //s = queue_->Flush(); } } block_offset_ += static_cast(kHeaderSize + n); diff --git a/src/pika_client_conn.cc b/src/pika_client_conn.cc index 59f196a189..153f3c1756 100644 --- a/src/pika_client_conn.cc +++ b/src/pika_client_conn.cc @@ -224,12 +224,13 @@ std::shared_ptr PikaClientConn::DoCmd(const PikaCmdArgsType& argv, const st if (c_ptr->is_write() && g_pika_conf->command_batch_enabled()) { // Get the appropriate SyncMasterDB for command batching auto sync_db = g_pika_rm->GetSyncMasterDBByName(DBInfo(c_ptr->db_name())); + // 查看 DB 是不是 Master DB if (sync_db) { auto command_collector = sync_db->GetCommandCollector(); if (command_collector) { // Create callback to handle command completion auto callback = [this, c_ptr](const LogOffset& offset, pstd::Status status) { - LOG(INFO) << "Command completed"; + //LOG(INFO) << "Command completed"; auto pc = dynamic_cast(c_ptr->GetConn().get()); if (pc) { auto resp_ptr = c_ptr->GetResp(); diff --git a/src/pika_command_collector.cc b/src/pika_command_collector.cc index 8e64f814cd..424b2b28b4 100644 --- a/src/pika_command_collector.cc +++ b/src/pika_command_collector.cc @@ -50,7 +50,7 @@ bool PikaCommandCollector::AddCommand(std::shared_ptr cmd_ptr, CommandCallb total_processed_.fetch_add(1); total_batches_.fetch_add(1); - LOG(INFO) << "Added single command " << cmd_ptr->name() << " to CommandQueue"; + //LOG(INFO) << "Added single command " << cmd_ptr->name() << " to CommandQueue"; return true; } diff --git a/src/pika_command_queue.cc b/src/pika_command_queue.cc index f9013f65c9..e0c6694e31 100644 --- a/src/pika_command_queue.cc +++ b/src/pika_command_queue.cc @@ -36,8 +36,8 @@ bool CommandQueue::EnqueueBatch(std::shared_ptr batch) { cmd_queue_.push(batch); - LOG(INFO) << "Enqueued command batch with " << batch->Size() - << " commands, queue size: " << cmd_queue_.size(); + //LOG(INFO) << "Enqueued command batch with " << batch->Size() + //<< " commands, queue size: " << cmd_queue_.size(); queue_cv_.notify_one(); return true; @@ -57,8 +57,8 @@ std::shared_ptr CommandQueue::DequeueBatch() { auto batch = cmd_queue_.front(); cmd_queue_.pop(); - LOG(INFO) << "Dequeued command batch with " << batch->Size() - << " commands, remaining queue size: " << cmd_queue_.size(); + //LOG(INFO) << "Dequeued command batch with " << batch->Size() + //<< " commands, remaining queue size: " << cmd_queue_.size(); return batch; } @@ -82,8 +82,8 @@ std::vector> CommandQueue::DequeueAllBatches() { for (const auto& batch : batches) { total_commands += batch->Size(); } - LOG(INFO) << "Dequeued all batches: " << batches.size() - << " batches with " << total_commands << " total commands"; + // LOG(INFO) << "Dequeued all batches: " << batches.size() + // << " batches with " << total_commands << " total commands"; } return batches; diff --git a/src/pika_consensus.cc b/src/pika_consensus.cc index 9900c5a41f..6e9d587f5c 100644 --- a/src/pika_consensus.cc +++ b/src/pika_consensus.cc @@ -143,14 +143,19 @@ Status SyncProgress::Update(const std::string& ip, int port, const LogOffset& st LogOffset acked_offset; { // update slave_ptr + LOG(INFO) << "UPdate"; std::lock_guard l(slave_ptr->slave_mu); Status s = slave_ptr->Update(start, end, &acked_offset); + slave_ptr->acked_offset = acked_offset; + if (!s.ok()) { return s; } // update match_index_ - // shared slave_ptr->slave_mu + // shared slave_ptr->slave_mu= match_index_[ip + std::to_string(port)] = acked_offset; + LOG(INFO) << "slave ip: " << ip << ", port :" << port << ",slave acked_offset " + << slave_ptr->acked_offset.ToString(); } return Status::OK(); @@ -375,6 +380,7 @@ Status ConsensusCoordinator::ProcessLeaderLog(const std::shared_ptr& cmd_pt Status ConsensusCoordinator::UpdateSlave(const std::string& ip, int port, const LogOffset& start, const LogOffset& end) { if (is_consistency_) { + //LOG(INFO) << "is_consistency"; std::shared_ptr slave_ptr = sync_pros_.GetSlaveNode(ip, port); if (!slave_ptr) { return Status::NotFound("ip " + ip + " port " + std::to_string(port)); @@ -383,8 +389,8 @@ Status ConsensusCoordinator::UpdateSlave(const std::string& ip, int port, const std::lock_guard l(slave_ptr->slave_mu); slave_ptr->acked_offset = end; sync_pros_.AddMatchIndex(ip, port, slave_ptr->acked_offset); - LOG(INFO) << "PacificA slave ip: " << ip << ", port :" << port << ",slave acked_offset " - << slave_ptr->acked_offset.ToString(); + //LOG(INFO) << "PacificA slave ip: " << ip << ", port :" << port << ",slave acked_offset " + //<< slave_ptr->acked_offset.ToString(); if (slave_ptr->slave_state != kSlaveBinlogSync && slave_ptr->acked_offset >= slave_ptr->target_offset) { slave_ptr->slave_state = kSlaveBinlogSync; LOG(INFO) << "PacificA change slave_state kSlaveBinlogSync acked_offset: " << slave_ptr->acked_offset.ToString() @@ -392,6 +398,7 @@ Status ConsensusCoordinator::UpdateSlave(const std::string& ip, int port, const } } } else { + LOG(INFO) << "no_consistency"; LogOffset committed_index; Status s = sync_pros_.Update(ip, port, start, end, &committed_index); if (!s.ok()) { @@ -823,12 +830,13 @@ bool ConsensusCoordinator::checkFinished(const LogOffset& offset) { //// pacificA private: +// 持久化 Binlog Status ConsensusCoordinator::PersistAppendBinlog(const std::shared_ptr& cmd_ptr, LogOffset& cur_offset) { std::string content = cmd_ptr->ToRedisProtocol(); std::string binlog = std::string(); LogOffset offset = LogOffset(); Status s = stable_logger_->Logger()->Put(content, &offset, binlog); - LOG(INFO) << "PacificA binlog_offset :" << offset.ToString(); + //LOG(INFO) << "PacificA binlog_offset :" << offset.ToString(); cur_offset = offset; if (!s.ok()) { std::string db_name = cmd_ptr->db_name().empty() ? g_pika_conf->default_db() : cmd_ptr->db_name(); @@ -856,7 +864,9 @@ Status ConsensusCoordinator::PersistAppendBinlog(const std::shared_ptr& cmd return stable_logger_->Logger()->IsOpened(); } +// 主节点持久化日志 Status ConsensusCoordinator::AppendEntries(const std::shared_ptr& cmd_ptr, LogOffset& cur_logoffset) { + //LOG(INFO) << "AppendEntries"; std::vector keys = cmd_ptr->current_key(); // slotkey shouldn't add binlog if (cmd_ptr->name() == kCmdNameSAdd && !keys.empty() && @@ -874,11 +884,14 @@ Status ConsensusCoordinator::AppendEntries(const std::shared_ptr& cmd_ptr, // g_pika_server->SignalAuxiliary(); return Status::OK(); } + +// 从节点持久化 LOG Status ConsensusCoordinator::AppendSlaveEntries(const std::shared_ptr& cmd_ptr, const BinlogItem& attribute) { + //LOG(INFO) << "AppendSlaveEntries"; LogOffset last_index = logs_->LastOffset(); if (attribute.logic_id() < last_index.l_offset.index) { - LOG(WARNING) << DBInfo(db_name_).ToString() << "Drop log from leader logic_id " << attribute.logic_id() - << " cur last index " << last_index.l_offset.index; + //LOG(WARNING) << DBInfo(db_name_).ToString() << "Drop log from leader logic_id " << attribute.logic_id() + //<< " cur last index " << last_index.l_offset.index; return Status::OK(); } LogOffset offset = LogOffset(); @@ -892,18 +905,21 @@ Status ConsensusCoordinator::AppendSlaveEntries(const std::shared_ptr& cmd_ /** * @brief Commit logs up to the given offset and update the committed ID. */ + +//从节点更新自己 CommittedID Status ConsensusCoordinator::CommitAppLog(const LogOffset& master_committed_id) { int index = logs_->FindOffset(logs_->FirstOffset()); int log_size = logs_->Size(); // Cache log size + //LOG(INFO) << "CommitAppLog, logs size: " << log_size; for (int i = index; i < log_size; ++i) { Log::LogItem log = logs_->At(i); if (master_committed_id >= log.offset) { - LOG(INFO) << "PacificA master_committed_id: " << master_committed_id.ToString() - << ", ApplyLog: " << log.offset.ToString(); + //LOG(INFO) << "PacificA master_committed_id: " << master_committed_id.ToString() + //<< ", ApplyLog: " << log.offset.ToString(); ApplyBinlog(log.cmd_ptr); } } - + // 日志截断 logs_->TruncateFrom(master_committed_id); // Truncate logs SetCommittedId(master_committed_id); // Update committed ID return Status::OK(); @@ -912,17 +928,20 @@ Status ConsensusCoordinator::CommitAppLog(const LogOffset& master_committed_id) /** * @brief Update the committed ID based on the Prepared ID of the slave */ +// 更新 CommittedID Status ConsensusCoordinator::UpdateCommittedID() { std::unordered_map> slaves = sync_pros_.GetAllSlaveNodes(); LogOffset slave_prepared_id = LogOffset(); for (const auto& slave : slaves) { if (slave.second->slave_state == kSlaveBinlogSync) { - if (slave_prepared_id == LogOffset()) { + slave_prepared_id = slave.second->acked_offset; + //LOG(INFO) << "slave_prepared_id: " << slave_prepared_id.ToString(); + /*if (slave_prepared_id == LogOffset()) { slave_prepared_id = slave.second->acked_offset; } else if (slave.second->acked_offset < slave_prepared_id) { slave_prepared_id = slave.second->acked_offset; - } + }*/ } } // if (!has_active_slaves) { @@ -947,10 +966,13 @@ Status ConsensusCoordinator::UpdateCommittedID() { << GetCommittedId().ToString() << ")"; return Status::Error("slave_prepared_id < master_committedId"); } + // Master 节点更新 Committed ID SetCommittedId(slave_prepared_id); - LOG(INFO) << "PacificA update CommittedID: " << GetCommittedId().ToString(); + //LOG(INFO) << "PacificA update CommittedID: " << GetCommittedId().ToString(); return Status::OK(); } + +// 从节点应用日志 Status ConsensusCoordinator::ProcessCoordination() { LogOffset offset = LogOffset(); Status s = stable_logger_->Logger()->GetProducerStatus(&(offset.b_offset.filenum), &(offset.b_offset.offset), @@ -963,6 +985,7 @@ Status ConsensusCoordinator::ProcessCoordination() { } SetPreparedId(offset); if (g_pika_server->role() & PIKA_ROLE_MASTER && g_pika_server->last_role() & PIKA_ROLE_SLAVE) { + LOG(INFO) << "CommitAppLOG"; Status s = CommitAppLog(GetPreparedId()); if (!s.ok()) { return s; @@ -973,15 +996,16 @@ Status ConsensusCoordinator::ProcessCoordination() { // Execute the operation of writing to DB Status ConsensusCoordinator::ApplyBinlog(const std::shared_ptr& cmd_ptr) { auto opt = cmd_ptr->argv()[0]; - LOG(INFO) << "[ApplyBinlog] Received command: " << opt << " for db: " << db_name_; + //LOG(INFO) << "[ApplyBinlog] Received command: " << opt << " for db: " << db_name_; if (pstd::StringToLower(opt) != kCmdNameFlushdb) { - LOG(INFO) << "[ApplyBinlog] Scheduling async task for " << opt; + //LOG(INFO) << "[ApplyBinlog] Scheduling async task for " << opt; InternalApplyFollower(cmd_ptr); } else { int32_t wait_ms = 250; while (g_pika_rm->GetUnfinishedAsyncWriteDBTaskCount(db_name_) > 0) { // TODO: 暂时去掉了sleep的逻辑,考虑使用条件变量唤醒 //std::this_thread::sleep_for(std::chrono::milliseconds(wait_ms)); + // WXR wait_ms *= 2; wait_ms = wait_ms < 3000 ? wait_ms : 3000; } @@ -1033,6 +1057,7 @@ Status ConsensusCoordinator::SendBinlog(std::shared_ptr slave_ptr, st } if (!tasks.empty()) { + //LOG(INFO) << "task.size: " << tasks.size(); g_pika_rm->ProduceWriteQueue(slave_ptr->Ip(), slave_ptr->Port(), db_name, tasks); } return Status::OK(); diff --git a/src/pika_repl_bgworker.cc b/src/pika_repl_bgworker.cc index 5340533160..0d3c7cbf7b 100644 --- a/src/pika_repl_bgworker.cc +++ b/src/pika_repl_bgworker.cc @@ -46,6 +46,7 @@ void PikaReplBgWorker::ParseBinlogOffset(const InnerMessage::BinlogOffset& pb_of } void PikaReplBgWorker::HandleBGWorkerWriteBinlog(void* arg) { + //LOG(INFO) << "Handel BGWorkerWriteBinlog 1"; auto task_arg = static_cast(arg); const std::shared_ptr res = task_arg->res; std::shared_ptr conn = task_arg->conn; @@ -133,14 +134,6 @@ void PikaReplBgWorker::HandleBGWorkerWriteBinlog(void* arg) { slave_db->SetReplState(ReplState::kTryConnect); return; } - if(db->GetISConsistency()){ - const InnerMessage::BinlogOffset& committed_id = binlog_res.committed_id(); - LogOffset master_committed_id(BinlogOffset(committed_id.filenum(),committed_id.offset()),LogicOffset(committed_id.term(),committed_id.index())); - Status s= db->CommitAppLog(master_committed_id); - if(!s.ok()){ - return; - } - } // empty binlog treated as keepalive packet if (binlog_res.binlog().empty()) { continue; @@ -165,6 +158,15 @@ void PikaReplBgWorker::HandleBGWorkerWriteBinlog(void* arg) { LOG(WARNING) << "DB " << worker->db_name_ << " Not Found"; return; } + if(db->GetISConsistency()){ + const InnerMessage::BinlogOffset& committed_id = binlog_res.committed_id(); + LogOffset master_committed_id(BinlogOffset(committed_id.filenum(),committed_id.offset()),LogicOffset(committed_id.term(),committed_id.index())); + Status s= db->CommitAppLog(master_committed_id); + if(!s.ok()){ + return; + } + } + } if (only_keepalive) { @@ -178,11 +180,12 @@ void PikaReplBgWorker::HandleBGWorkerWriteBinlog(void* arg) { ack_end = productor_status; ack_end.l_offset.term = pb_end.l_offset.term; } - + db->GetCoordinator()->StableLogger()->Logger()->GetQueue()->Flush(); g_pika_rm->SendBinlogSyncAckRequest(db_name, ack_start, ack_end); } int PikaReplBgWorker::HandleWriteBinlog(net::RedisParser* parser, const net::RedisCmdArgsType& argv) { + //LOG(INFO) << "Start Handel WriteBinlog 2"; std::string opt = argv[0]; auto worker = static_cast(parser->data); // Monitor related @@ -218,6 +221,7 @@ int PikaReplBgWorker::HandleWriteBinlog(net::RedisParser* parser, const net::Red return -1; } if(db->GetISConsistency()){ + // 如果是一致性就调用这个函数 db->AppendSlaveEntries(c_ptr, worker->binlog_item_); }else{ db->ConsensusProcessLeaderLog(c_ptr, worker->binlog_item_); diff --git a/src/pika_repl_client_conn.cc b/src/pika_repl_client_conn.cc index 498cd1d274..0891660d11 100644 --- a/src/pika_repl_client_conn.cc +++ b/src/pika_repl_client_conn.cc @@ -223,13 +223,13 @@ void PikaReplClientConn::HandleTrySyncResponse(void* arg) { db->Logger()->GetProducerStatus(&boffset.filenum, &boffset.offset); slave_db->SetMasterSessionId(session_id); LogOffset offset(boffset, logic_last_offset); - LOG(INFO)<<"PacificA slave first binlog stable offset : "<< offset.ToString(); + //LOG(INFO)<<"PacificA slave first binlog stable offset : "<< offset.ToString(); if(db->GetISConsistency()){ if (try_sync_response.has_prepared_id()){ const InnerMessage::BinlogOffset& prepared_id = try_sync_response.prepared_id(); LogOffset master_prepared_id(BinlogOffset(prepared_id.filenum(),prepared_id.offset()),LogicOffset(prepared_id.term(),prepared_id.index())); - LOG(INFO)<<"PacificA master TrySync Response master_prepared_id: "<GetPreparedId().ToString(); + //LOG(INFO)<<"PacificA master TrySync Response master_prepared_id: "<GetPreparedId().ToString(); if(master_prepared_id < db->GetPreparedId()){ if(master_prepared_id < db->GetCommittedId()){ @@ -250,7 +250,7 @@ void PikaReplClientConn::HandleTrySyncResponse(void* arg) { slave_db->SetReplState(ReplState::kConnected); // after connected, update receive time first to avoid connection timeout slave_db->SetLastRecvTime(pstd::NowMicros()); - LOG(INFO) << "DB: " << db_name << " TrySync Ok"; + //LOG(INFO) << "DB: " << db_name << " TrySync Ok"; } else if (try_sync_response.reply_code() == InnerMessage::InnerResponse::TrySync::kSyncPointBePurged) { slave_db->SetReplState(ReplState::kTryDBSync); LOG(INFO) << "DB: " << db_name << " Need To Try DBSync"; diff --git a/src/pika_repl_server_conn.cc b/src/pika_repl_server_conn.cc index 091c85a0de..911ab7dbc4 100644 --- a/src/pika_repl_server_conn.cc +++ b/src/pika_repl_server_conn.cc @@ -355,6 +355,7 @@ void PikaReplServerConn::HandleDBSyncRequest(void* arg) { conn->NotifyWrite(); } +// 主节点处理从节点的 Binlog 回应 void PikaReplServerConn::HandleBinlogSyncRequest(void* arg) { std::unique_ptr task_arg(static_cast(arg)); const std::shared_ptr req = task_arg->req; @@ -401,6 +402,7 @@ void PikaReplServerConn::HandleBinlogSyncRequest(void* arg) { } if (is_first_send) { + LOG(INFO) << "first send"; if (range_start.b_offset != range_end.b_offset) { LOG(WARNING) << "first binlogsync request pb argument invalid"; conn->NotifyClose(); diff --git a/src/pika_rm.cc b/src/pika_rm.cc index a411e906bc..dd189e9d24 100644 --- a/src/pika_rm.cc +++ b/src/pika_rm.cc @@ -195,6 +195,7 @@ Status SyncMasterDB::ReadBinlogFileToWq(const std::shared_ptr& slave_ return Status::OK(); } +// Master节点更查看 Slave 节点偏移量 Status SyncMasterDB::ConsensusUpdateSlave(const std::string& ip, int port, const LogOffset& start, const LogOffset& end) { Status s = coordinator_.UpdateSlave(ip, port, start, end); if (!s.ok()) { @@ -238,13 +239,16 @@ Status SyncMasterDB::WakeUpSlaveBinlogSync() { for (auto& slave_iter : slaves) { std::shared_ptr slave_ptr = slave_iter.second; std::lock_guard l(slave_ptr->slave_mu); + // slave 节点这次要发送的 sent_offset 一定要等于上次回包的 acked_offset,否则就不用发送了 if (slave_ptr->sent_offset == slave_ptr->acked_offset) { Status s; if (coordinator_.GetISConsistency()) { if(slave_ptr->slave_state == SlaveState::kSlaveBinlogSync||slave_ptr->slave_state == SlaveState::KCandidate){ + // 强一致性给 slave 发送 binlog s = coordinator_.SendBinlog(slave_ptr, db_info_.db_name_); } } else { + // 非强一致性 s = ReadBinlogFileToWq(slave_ptr); } if (!s.ok()) { @@ -253,6 +257,8 @@ Status SyncMasterDB::WakeUpSlaveBinlogSync() { // << slave_ptr->ToStringStatus() << " - " << s.ToString(); } } + + } for (const auto& to_del_slave : to_del) { @@ -329,6 +335,7 @@ bool SyncMasterDB::BinlogCloudPurge(uint32_t index) { return true; } +//WXR Status SyncMasterDB::CheckSyncTimeout(uint64_t now) { std::unordered_map> slaves = GetAllSlaveNodes(); @@ -432,7 +439,11 @@ Status SyncMasterDB::AppendSlaveEntries(const std::shared_ptr& cmd_ptr, con Status SyncMasterDB::ProcessCoordination(){ return coordinator_.ProcessCoordination(); } + +// Master 节点更新自己的 CommittedID Status SyncMasterDB::UpdateCommittedID(){ + // Master 更新自己的 CommittedID + //LOG(INFO) << "UpdateCommittedID"; Status s = coordinator_.UpdateCommittedID(); if (s.ok()) { // Notify RocksDB thread of new CommittedID @@ -440,6 +451,7 @@ Status SyncMasterDB::UpdateCommittedID(){ if (committed_id.IsValid()) { extern std::unique_ptr g_pika_rm; if (g_pika_rm) { + // 唤醒 RocksDB 线程执行 g_pika_rm->NotifyCommittedID(committed_id); } } @@ -448,12 +460,15 @@ Status SyncMasterDB::UpdateCommittedID(){ } return s; } + Status SyncMasterDB::Truncate(const LogOffset& offset){ return coordinator_.Truncate(offset); } Status SyncMasterDB::CommitAppLog(const LogOffset& master_committed_id){ return coordinator_.CommitAppLog(master_committed_id); } + + Status SyncMasterDB::AppendCandidateBinlog(const std::string& ip, int port, const LogOffset& offset) { std::shared_ptr slave_ptr = GetSlaveNode(ip, port); if (!slave_ptr) { @@ -477,14 +492,15 @@ Status SyncMasterDB::AppendCandidateBinlog(const std::string& ip, int port, cons if (!s.ok()) { return Status::Corruption("Init binlog file reader failed" + s.ToString()); // 如果初始化失败,返回错误状态 } + // 删除写队列中的数据 g_pika_rm->DropItemInOneWriteQueue(ip, port, slave_ptr->DBName()); slave_ptr->b_state = kReadFromFile; } - Status s = coordinator_.SendBinlog(slave_ptr, slave_ptr->DBName()); + /*Status s = coordinator_.SendBinlog(slave_ptr, slave_ptr->DBName()); if (!s.ok()) { return s; - } + }*/ return Status::OK(); } @@ -798,12 +814,12 @@ void PikaReplicaManager::ProduceWriteQueue(const std::string& ip, int port, std: const std::vector& tasks) { std::lock_guard l(write_queue_mu_); std::string index = ip + ":" + std::to_string(port); - LOG(INFO) << "[Queue] Entering ProduceWriteQueue for " << ip << ":" << port << ", db: " << db_name << ", current queue size: " << write_queues_.size() << ", tasks to add: " << tasks.size(); + //LOG(INFO) << "[Queue] Entering ProduceWriteQueue for " << ip << ":" << port << ", db: " << db_name << ", current queue size: " << write_queues_.size() << ", tasks to add: " << tasks.size(); for (auto& task : tasks) { write_queues_[index][db_name].push(task); } - LOG(INFO) << "[Queue] Added " << tasks.size() << " tasks to queue for " << ip << ":" << port << ", db: " << db_name << ", new queue size: " << write_queues_[index][db_name].size(); - LOG(INFO) << "[Queue] Exiting ProduceWriteQueue"; + //LOG(INFO) << "[Queue] Added " << tasks.size() << " tasks to queue for " << ip << ":" << port << ", db: " << db_name << ", new queue size: " << write_queues_[index][db_name].size(); + //LOG(INFO) << "[Queue] Exiting ProduceWriteQueue"; } int PikaReplicaManager::ConsumeWriteQueue() { @@ -850,6 +866,8 @@ int PikaReplicaManager::ConsumeWriteQueue() { LOG(WARNING) << "Parse ip_port error " << iter.first; continue; } + //LOG(INFO) << "message size: " << iter.second.size(); + //LOG(INFO) << "SendSlaveBinlogChips to " << iter.first << " start. for (auto& to_send : iter.second) { Status s = pika_repl_server_->SendSlaveBinlogChips(ip, port, to_send); if (!s.ok()) { @@ -857,7 +875,7 @@ int PikaReplicaManager::ConsumeWriteQueue() { to_delete.push_back(iter.first); continue; } else { - LOG(INFO) << "[Queue] SendSlaveBinlogChips to " << iter.first << " success."; + //LOG(INFO) << "[Queue] SendSlaveBinlogChips to " << iter.first << " success."; } } } @@ -916,6 +934,7 @@ void PikaReplicaManager::ReplServerUpdateClientConnMap(const std::string& ip_por pika_repl_server_->UpdateClientConnMap(ip_port, fd); } +//更新 Binlog 状态 Status PikaReplicaManager::UpdateSyncBinlogStatus(const RmNode& slave, const LogOffset& offset_start, const LogOffset& offset_end) { std::shared_lock l(dbs_rw_); @@ -928,15 +947,16 @@ Status PikaReplicaManager::UpdateSyncBinlogStatus(const RmNode& slave, const Log return s; } if(db->GetISConsistency()){ + // Master 节点更新 CommittedID s = db->UpdateCommittedID(); if (!s.ok()) { return s; } } - s = db->SyncBinlogToWq(slave.Ip(), slave.Port()); + /*s = db->SyncBinlogToWq(slave.Ip(), slave.Port()); if (!s.ok()) { return s; - } + }*/ return Status::OK(); } @@ -1386,7 +1406,7 @@ void PikaReplicaManager::CommandQueueLoop() { // Process all batches ProcessCommandBatches(batches); processed_any_batch = true; - LOG(INFO) << "Processed " << batches.size() << " batches, checking for more..."; + //LOG(INFO) << "Processed " << batches.size() << " batches, checking for more..."; } else { processed_any_batch = false; } @@ -1397,7 +1417,7 @@ void PikaReplicaManager::CommandQueueLoop() { std::this_thread::sleep_for(std::chrono::milliseconds(100)); } } - LOG(INFO) << "Command queue loop ended"; + //LOG(INFO) << "Command queue loop ended"; } void PikaReplicaManager::ProcessCommandBatches(const std::vector>& batches) { @@ -1408,7 +1428,7 @@ void PikaReplicaManager::ProcessCommandBatches(const std::vectorSize(); } - LOG(INFO) << "Processing " << batches.size() << " batches with " << total_commands << " total commands"; + //LOG(INFO) << "Processing " << batches.size() << " batches with " << total_commands << " total commands"; // Merge all commands from all batches for unified batch processing std::vector> all_commands; @@ -1452,10 +1472,11 @@ void PikaReplicaManager::ProcessCommandBatches(const std::vectorConsensusProposeLog(cmd_ptr); if (!s.ok()) { LOG(ERROR) << "Failed to " << (sync_db->GetISConsistency() ? "append" : "propose") @@ -1472,8 +1493,9 @@ void PikaReplicaManager::ProcessCommandBatches(const std::vectorGetPreparedId(); - + // 将 PrepardID 填充到 all_offsets 中 all_offsets[cmd_idx] = cmd_offset; // Update individual batch offsets @@ -1488,18 +1510,21 @@ void PikaReplicaManager::ProcessCommandBatches(const std::vectorbinlog_offsets[batch_cmd_idx] = cmd_offset; } - LOG(INFO) << "Successfully processed " << all_commands.size() << " commands individually"; + //LOG(INFO) << "Successfully processed " << all_commands.size() << " commands individually"; // Create BatchGroup with the last offset as end_offset LogOffset end_offset = all_offsets.back(); + //LOG(INFO) << "end_offset: " << end_offset.ToString(); auto batch_group = std::make_shared(batches, end_offset); // Enqueue the BatchGroup { std::lock_guard lock(pending_batch_groups_mutex_); pending_batch_groups_.push(batch_group); + //rocksdb_thread_cv_.notify_one(); } - LOG(INFO) << "Created BatchGroup with " << batches.size() << " batches and end_offset: " << end_offset.ToString(); + //LOG(INFO) << "Created BatchGroup with " << batches.size() << " batches and end_offset: " << end_offset.ToString(); + //./LOG(INFO) << "Pending BatchGroup with " << pending_batch_groups_.size() << " batches"; } catch (const std::exception& e) { LOG(ERROR) << "Exception in ProcessCommandBatches: " << e.what(); // Call callbacks with error for all batches @@ -1544,7 +1569,7 @@ void PikaReplicaManager::StopRocksDBThread() { } void PikaReplicaManager::RocksDBThreadLoop() { - LOG(INFO) << "RocksDB_back_thread started"; + //LOG(INFO) << "RocksDB_back_thread started"; while (rocksdb_thread_running_.load()) { try { std::unique_lock lock(rocksdb_thread_mutex_); @@ -1552,7 +1577,7 @@ void PikaReplicaManager::RocksDBThreadLoop() { rocksdb_thread_cv_.wait(lock, [this] { bool has_pending; { - std::lock_guard pending_lock(pending_batch_groups_mutex_); + //std::lock_guard pending_lock(pending_batch_groups_mutex_); has_pending = !pending_batch_groups_.empty(); } return !rocksdb_thread_running_.load() || has_pending; @@ -1565,51 +1590,61 @@ void PikaReplicaManager::RocksDBThreadLoop() { // Get current committed ID LogOffset committed_id; { - std::lock_guard id_lock(last_committed_id_mutex_); + std::unique_lock lock(last_committed_id_mutex_); committed_id = last_committed_id_; + //LOG(INFO) << "committed_id: " << committed_id.ToString(); } - + //LOG(INFO) << "RocksDBThreadLoop"; // Process committed BatchGroups - if (committed_id.IsValid()) { + //if (committed_id.IsValid()) { size_t groups_processed = ProcessCommittedBatchGroups(committed_id); - if (groups_processed > 0) { - LOG(INFO) << "Processed " << groups_processed << " committed BatchGroups"; - } - } + //if (groups_processed > 0) { + //LOG(INFO) << "Processed " << groups_processed << " committed BatchGroups"; + //} + //} } catch (const std::exception& e) { LOG(ERROR) << "Error in RocksDB thread loop: " << e.what(); std::this_thread::sleep_for(std::chrono::milliseconds(100)); } } - LOG(INFO) << "RocksDB_back_thread ended"; + //LOG(INFO) << "RocksDB_back_thread ended"; } size_t PikaReplicaManager::ProcessCommittedBatchGroups(const LogOffset& committed_id) { std::queue> groups_to_process; // Get pending BatchGroups and separate committed from uncommitted { + //LOG(INFO) << "Start Process"; std::lock_guard lock(pending_batch_groups_mutex_); + //LOG(INFO) << pending_batch_groups_.size() << " pending batch groups"; while (!pending_batch_groups_.empty()) { auto batch_group = pending_batch_groups_.front(); // Check if this BatchGroup is committed by comparing its end_offset with committed_id - bool is_committed = (batch_group->end_offset <= committed_id); + bool is_committed; + { + std::shared_lock lock(last_committed_id_mutex_); + is_committed = (batch_group->end_offset <= last_committed_id_); + //LOG(INFO) << "committed_id: " << last_committed_id_.ToString() + // << " And BatchGroup with end_offset " << batch_group->end_offset.ToString() + //<< " is committed"; + } if (is_committed) { groups_to_process.push(batch_group); pending_batch_groups_.pop(); - LOG(INFO) << "BatchGroup with end_offset " << batch_group->end_offset.ToString() - << " is committed (committed_id: " << committed_id.ToString() << ")"; } else { + //rocksdb_thread_cv_.notify_one(); // First uncommitted BatchGroup found, stop processing static LogOffset last_uncommitted_offset; if (last_uncommitted_offset != batch_group->end_offset) { - LOG(INFO) << "BatchGroup with end_offset " << batch_group->end_offset.ToString() - << " is not yet committed (committed_id: " << committed_id.ToString() << ")"; + //LOG(INFO) << "BatchGroup with end_offset " << batch_group->end_offset.ToString() + //<< " is not yet committed (committed_id: " << committed_id.ToString() << ")"; last_uncommitted_offset = batch_group->end_offset; } break; } } } + //LOG(INFO) << "RocksDB Start Process Committed BatchGroups"; // Store the number of groups to process for return value size_t groups_count = groups_to_process.size(); // Process committed BatchGroups @@ -1624,7 +1659,7 @@ size_t PikaReplicaManager::ProcessCommittedBatchGroups(const LogOffset& committe // Execute RocksDB Put operations for each command in the batch for (size_t i = 0; i < batch->commands.size(); ++i) { const auto& cmd_ptr = batch->commands[i]; - LOG(INFO) << "[RocksDB] RocksDBThreadLoop" << "Processing BatchGroup with end_offset: " << batch_group->end_offset.ToString(); + //LOG(INFO) << "[RocksDB] RocksDBThreadLoop" << "Processing BatchGroup with end_offset: " << batch_group->end_offset.ToString(); try { // Execute the command (this will perform the RocksDB Put operation) cmd_ptr->Execute(); @@ -1636,13 +1671,13 @@ size_t PikaReplicaManager::ProcessCommittedBatchGroups(const LogOffset& committe // pc->resp_array.push_back(resp_ptr); // pc->TryWriteResp(); if (cmd_ptr->res().ok()) { - LOG(INFO) << "Successfully executed RocksDB Put for command: " << cmd_ptr->name(); + //LOG(INFO) << "Successfully executed RocksDB Put for command: " << cmd_ptr->name(); // Execute callback with BatchGroup's end_offset (all commands in the group get the same offset) - if (i < batch->callbacks.size() && batch->callbacks[i]) { - LOG(INFO) << "[Callback] RocksDBThreadLoop" - << ", Executing callback for end_offset: " << batch_group->end_offset.ToString(); + //if (i < batch->callbacks.size() && batch->callbacks[i]) { + //LOG(INFO) << "[Callback] RocksDBThreadLoop" + // << ", Executing callback for end_offset: " << batch_group->end_offset.ToString(); batch->callbacks[i](batch_group->end_offset, pstd::Status::OK()); - } + //} } else { LOG(WARNING) << "Command " << cmd_ptr->name() << " execution failed: " << cmd_ptr->res().message(); // Execute callback with command error, still use BatchGroup's end_offset @@ -1664,15 +1699,18 @@ size_t PikaReplicaManager::ProcessCommittedBatchGroups(const LogOffset& committe return groups_count; } +// 唤醒 RocksDB 线程 void PikaReplicaManager::NotifyCommittedID(const LogOffset& committed_id) { { - std::lock_guard lock(last_committed_id_mutex_); + std::unique_lock lock(last_committed_id_mutex_); last_committed_id_ = committed_id; + //LOG(INFO) << "last_committed_id: " << last_committed_id_.ToString(); } // Notify RocksDB thread { + //LOG(INFO) << "NotifyCommittedID"; std::lock_guard lock(rocksdb_thread_mutex_); rocksdb_thread_cv_.notify_one(); } - LOG(INFO) << "Notified RocksDB thread of new CommittedID: " << committed_id.ToString(); + //LOG(INFO) << "Notified RocksDB thread of new CommittedID: " << committed_id.ToString(); } diff --git a/src/pstd/src/env.cc b/src/pstd/src/env.cc index 7dadf924ea..27dff0958a 100644 --- a/src/pstd/src/env.cc +++ b/src/pstd/src/env.cc @@ -425,7 +425,25 @@ class PosixMmapFile : public WritableFile { return s; } - Status Flush() override { return Status::OK(); } +Status Flush() override { + LOG(INFO) << "Flush"; + if (fd_ < 0) { + return IOError(filename_, EINVAL); + } + + // 刷写内核缓冲区到磁盘 + #if defined(__APPLE__) + LOG(INFO) << "fsync"; + if (fsync(fd_) != 0) { + #else + LOG(INFO) << "fdatasync"; + if (fdatasync(fd_) != 0) { + #endif + LOG(INFO) << "fdatasync error"; + return IOError(filename_, errno); + } + return Status::OK(); +} Status Sync() override { Status s;