From c025543202e50b561d2fb9ceac414197ba95a91e Mon Sep 17 00:00:00 2001 From: PFZheng Date: Tue, 23 Apr 2019 01:36:45 -0700 Subject: [PATCH 01/35] Optimize raft LOG to print group_id wherever possible, to make it more convenient when debuging. --- src/braft/fsm_caller.cpp | 5 +- src/braft/log.cpp | 38 +++--- src/braft/log_manager.cpp | 6 +- src/braft/node.cpp | 211 ++++++++++++++++---------------- src/braft/replicator.cpp | 99 +++++++++------ src/braft/snapshot.cpp | 35 +++--- src/braft/snapshot_executor.cpp | 25 ++-- src/braft/storage.cpp | 9 +- test/test_node.cpp | 4 +- 9 files changed, 243 insertions(+), 189 deletions(-) diff --git a/src/braft/fsm_caller.cpp b/src/braft/fsm_caller.cpp index 27a207bc..4aacf620 100644 --- a/src/braft/fsm_caller.cpp +++ b/src/braft/fsm_caller.cpp @@ -271,8 +271,9 @@ void FSMCaller::do_committed(int64_t committed_index) { } Iterator iter(&iter_impl); _fsm->on_apply(iter); - LOG_IF(ERROR, iter.valid()) - << "Iterator is still valid, did you return before iterator " + LOG_IF(ERROR, iter.valid()) + << "Node " << _node->node_id() + << " Iterator is still valid, did you return before iterator " " reached the end?"; // Try move to next in case that we pass the same log twice. iter.next(); diff --git a/src/braft/log.cpp b/src/braft/log.cpp index c2ed8749..b1006b3a 100644 --- a/src/braft/log.cpp +++ b/src/braft/log.cpp @@ -186,7 +186,7 @@ int Segment::_load_entry(off_t offset, EntryHeader* head, butil::IOBuf* data, if (!verify_checksum(tmp.checksum_type, p, ENTRY_HEADER_SIZE - 4, header_checksum)) { LOG(ERROR) << "Found corrupted header at offset=" << offset - << ", header=" << tmp; + << ", header=" << tmp << ", path: " << _path; return -1; } if (head != NULL) { @@ -207,7 +207,8 @@ int Segment::_load_entry(off_t offset, EntryHeader* head, butil::IOBuf* data, if (!verify_checksum(tmp.checksum_type, buf, tmp.data_checksum)) { LOG(ERROR) << "Found corrupted data at offset=" << offset + ENTRY_HEADER_SIZE - << " header=" << tmp; + << " header=" << tmp + << " path: " << _path; // TODO: abort()? return -1; } @@ -360,7 +361,8 @@ int Segment::append(const LogEntry* entry) { } break; default: - LOG(FATAL) << "unknow entry type: " << entry->type; + LOG(FATAL) << "unknow entry type: " << entry->type + << ", path: " << _path; return -1; } CHECK_LE(data.length(), 1ul << 56ul); @@ -383,7 +385,8 @@ int Segment::append(const LogEntry* entry) { const ssize_t n = butil::IOBuf::cut_multiple_into_file_descriptor( _fd, pieces + start, ARRAY_SIZE(pieces) - start); if (n < 0) { - LOG(ERROR) << "Fail to write to fd=" << _fd << ", " << berror(); + LOG(ERROR) << "Fail to write to fd=" << _fd + << ", path: " << _path << berror(); return -1; } written += n; @@ -450,7 +453,7 @@ LogEntry* Segment::get(const int64_t index) const { } break; default: - CHECK(false) << "Unknown entry type"; + CHECK(false) << "Unknown entry type, path: " << _path; break; } @@ -489,8 +492,10 @@ int Segment::close(bool will_sync) { // TODO: optimize index memory usage by reconstruct vector LOG(INFO) << "close a full segment. Current first_index: " << _first_index - << " last_index: " << _last_index << " raft_sync_segments: " << FLAGS_raft_sync_segments - << " will_sync: " << will_sync; + << " last_index: " << _last_index + << " raft_sync_segments: " << FLAGS_raft_sync_segments + << " will_sync: " << will_sync + << " path: " << new_path; int ret = 0; if (_last_index > _first_index) { if (FLAGS_raft_sync_segments && will_sync) { @@ -587,7 +592,8 @@ int Segment::truncate(const int64_t last_index_kept) { // seek fd off_t ret_off = ::lseek(_fd, truncate_size, SEEK_SET); if (ret_off < 0) { - PLOG(ERROR) << "Fail to lseek fd=" << _fd << " to size=" << truncate_size; + PLOG(ERROR) << "Fail to lseek fd=" << _fd << " to size=" << truncate_size + << " path: " << _path; return -1; } @@ -672,7 +678,8 @@ int SegmentLogStorage::append_entries(const std::vector& entries) { } if (_last_log_index.load(butil::memory_order_relaxed) + 1 != entries.front()->id.index) { - LOG(FATAL) << "There's gap between appending entries and _last_log_index"; + LOG(FATAL) << "There's gap between appending entries and _last_log_index" + << " path: " << _path; return -1; } scoped_refptr last_segment = NULL; @@ -772,7 +779,7 @@ int SegmentLogStorage::truncate_prefix(const int64_t first_index_kept) { // the deleting fails or the process crashes (which is unlikely to happen). // The new process would see the latest `first_log_index' if (save_meta(first_index_kept) != 0) { // NOTE - PLOG(ERROR) << "Fail to save meta"; + PLOG(ERROR) << "Fail to save meta, path: " << _path; return -1; } std::vector > popped; @@ -855,7 +862,8 @@ int SegmentLogStorage::truncate_suffix(const int64_t last_index_kept) { int SegmentLogStorage::reset(const int64_t next_log_index) { if (next_log_index <= 0) { - LOG(ERROR) << "Invalid next_log_index=" << next_log_index; + LOG(ERROR) << "Invalid next_log_index=" << next_log_index + << " path: " << _path; return EINVAL; } std::vector > popped; @@ -875,7 +883,7 @@ int SegmentLogStorage::reset(const int64_t next_log_index) { lck.unlock(); // NOTE: see the comments in truncate_prefix if (save_meta(next_log_index) != 0) { - PLOG(ERROR) << "Fail to save meta"; + PLOG(ERROR) << "Fail to save meta, path: " << _path; return -1; } for (size_t i = 0; i < popped.size(); ++i) { @@ -888,7 +896,8 @@ int SegmentLogStorage::reset(const int64_t next_log_index) { int SegmentLogStorage::list_segments(bool is_empty) { butil::DirReaderPosix dir_reader(_path.c_str()); if (!dir_reader.IsValid()) { - LOG(WARNING) << "directory reader failed, maybe NOEXIST or PERMISSION. path: " << _path; + LOG(WARNING) << "directory reader failed, maybe NOEXIST or PERMISSION." + << " path: " << _path; return -1; } @@ -1103,7 +1112,8 @@ scoped_refptr SegmentLogStorage::open_segment() { break; } } - PLOG(ERROR) << "Fail to close old open_segment or create new open_segment"; + PLOG(ERROR) << "Fail to close old open_segment or create new open_segment" + << " path: " << _path; // Failed, revert former changes BAIDU_SCOPED_LOCK(_mutex); _segments.erase(prev_open_segment->first_index()); diff --git a/src/braft/log_manager.cpp b/src/braft/log_manager.cpp index b1658974..083584c8 100644 --- a/src/braft/log_manager.cpp +++ b/src/braft/log_manager.cpp @@ -444,8 +444,8 @@ void LogManager::append_to_storage(std::vector* to_append, if (nappent != (int)to_append->size()) { // FIXME LOG(ERROR) << "Fail to append_entries, " - << "nappent=" << nappent - << ", to_append=" << to_append->size(); + << "nappent=" << nappent + << ", to_append=" << to_append->size(); report_error(EIO, "Fail to append entries"); } if (nappent > 0) { @@ -563,7 +563,7 @@ int LogManager::disk_thread(void* meta, dynamic_cast(done); if (tsc) { LOG(WARNING) << "Truncating storage to last_index_kept=" - << tsc->last_index_kept(); + << tsc->last_index_kept(); ret = log_manager->_log_storage->truncate_suffix( tsc->last_index_kept()); if (ret == 0) { diff --git a/src/braft/node.cpp b/src/braft/node.cpp index add928aa..1532597a 100644 --- a/src/braft/node.cpp +++ b/src/braft/node.cpp @@ -220,9 +220,8 @@ int NodeImpl::init_log_storage() { CHECK(_fsm_caller); _log_storage = LogStorage::create(_options.log_uri); if (!_log_storage) { - LOG(ERROR) << "Fail to find log storage of `" - << _options.log_uri - << '\''; + LOG(ERROR) << "node " << _group_id << ":" << _server_id + << " find log storage failed, uri " << _options.log_uri; return -1; } _log_manager = new LogManager(); @@ -369,14 +368,16 @@ int NodeImpl::bootstrap(const BootstrapOptions& options) { if (options.last_log_index > 0) { if (init_snapshot_storage() != 0) { - LOG(ERROR) << "Fail to init snapshot_storage"; + LOG(ERROR) << "Fail to init snapshot_storage from " + << _options.snapshot_uri; return -1; } SynchronizedClosure done; _snapshot_executor->do_snapshot(&done); done.wait(); if (!done.status().ok()) { - LOG(ERROR) << "Fail to save snapshot " << done.status(); + LOG(ERROR) << "Fail to save snapshot " << done.status() + << " from " << _options.snapshot_uri; return -1; } } @@ -408,12 +409,14 @@ int NodeImpl::init(const NodeOptions& options) { // check _server_id if (butil::IP_ANY == _server_id.addr.ip) { - LOG(ERROR) << "Node can't started from IP_ANY"; + LOG(ERROR) << "Group " << _group_id + << " Node can't started from IP_ANY"; return -1; } if (!NodeManager::GetInstance()->server_exists(_server_id.addr)) { - LOG(ERROR) << "No RPC Server attached to " << _server_id.addr + LOG(ERROR) << "Group " << _group_id + << " No RPC Server attached to " << _server_id.addr << ", did you forget to call braft::add_service()?"; return -1; } @@ -427,13 +430,15 @@ int NodeImpl::init(const NodeOptions& options) { if (bthread::execution_queue_start(&_apply_queue_id, NULL, execute_applying_tasks, this) != 0) { - LOG(ERROR) << "Fail to start execution_queue"; + LOG(ERROR) << "node " << _group_id << ":" << _server_id + << " fail to start execution_queue"; return -1; } _apply_queue = execution_queue_address(_apply_queue_id); if (!_apply_queue) { - LOG(ERROR) << "Fail to address execution_queue"; + LOG(ERROR) << "node " << _group_id << ":" << _server_id + << " fail to address execution_queue"; return -1; } @@ -443,20 +448,20 @@ int NodeImpl::init(const NodeOptions& options) { // log storage and log manager init if (init_log_storage() != 0) { LOG(ERROR) << "node " << _group_id << ":" << _server_id - << " init_log_storage failed"; + << " init_log_storage failed"; return -1; } // meta init if (init_meta_storage() != 0) { LOG(ERROR) << "node " << _group_id << ":" << _server_id - << " init_meta_storage failed"; + << " init_meta_storage failed"; return -1; } if (init_fsm_caller(LogId(0, 0)) != 0) { LOG(ERROR) << "node " << _group_id << ":" << _server_id - << " init_fsm_caller failed"; + << " init_fsm_caller failed"; return -1; } @@ -467,7 +472,7 @@ int NodeImpl::init(const NodeOptions& options) { ballot_box_options.closure_queue = _closure_queue; if (_ballot_box->init(ballot_box_options) != 0) { LOG(ERROR) << "node " << _group_id << ":" << _server_id - << " init _ballot_box failed"; + << " init _ballot_box failed"; return -1; } @@ -476,15 +481,15 @@ int NodeImpl::init(const NodeOptions& options) { // init log storage before snapshot storage, snapshot storage will update configration if (init_snapshot_storage() != 0) { LOG(ERROR) << "node " << _group_id << ":" << _server_id - << " init_snapshot_storage failed"; + << " init_snapshot_storage failed"; return -1; } butil::Status st = _log_manager->check_consistency(); if (!st.ok()) { LOG(ERROR) << "node " << _group_id << ":" << _server_id - << " is initialized with inconsitency log: " - << st; + << " is initialized with inconsitency log: " + << st; return -1; } @@ -523,7 +528,7 @@ int NodeImpl::init(const NodeOptions& options) { // start snapshot timer if (_snapshot_executor && _options.snapshot_interval_s > 0) { BRAFT_VLOG << "node " << _group_id << ":" << _server_id - << " term " << _current_term << " start snapshot_timer"; + << " term " << _current_term << " start snapshot_timer"; _snapshot_timer.start(); } @@ -801,7 +806,7 @@ butil::Status NodeImpl::reset_peers(const Configuration& new_peers) { // check state if (!is_active_state(_state)) { LOG(WARNING) << "node " << _group_id << ":" << _server_id - << " is in state " << state2str(_state) << ", can't set_peer"; + << " is in state " << state2str(_state) << ", can't reset_peer"; return butil::Status(EPERM, "Bad state %s", state2str(_state)); } // check bootstrap @@ -818,7 +823,7 @@ butil::Status NodeImpl::reset_peers(const Configuration& new_peers) { // check concurrent conf change if (_state == STATE_LEADER && _conf_ctx.is_busy()) { LOG(WARNING) << "node " << _group_id << ":" << _server_id - << " set_peer need wait current conf change"; + << " reset_peer need wait current conf change"; return butil::Status(EBUSY, "Changing to another configuration"); } @@ -1079,14 +1084,15 @@ int NodeImpl::transfer_leadership_to(const PeerId& peer) { // last_log_id will be selected. if (peer_id == ANY_PEER) { LOG(INFO) << "node " << _group_id << ":" << _server_id - << " starts to transfer leadership to any peer."; + << " starts to transfer leadership to any peer."; // find the next candidate which is the most possible to become new leader if (_replicator_group.find_the_next_candidate(&peer_id, _conf) != 0) { return -1; } } if (peer_id == _server_id) { - LOG(INFO) << "Transferring leadership to self"; + LOG(INFO) << "node " << _group_id << ":" << _server_id + << " transfering leadership to self"; return 0; } if (!_conf.contains(peer_id)) { @@ -1098,7 +1104,8 @@ int NodeImpl::transfer_leadership_to(const PeerId& peer) { const int64_t last_log_index = _log_manager->last_log_index(); const int rc = _replicator_group.transfer_leadership_to(peer_id, last_log_index); if (rc != 0) { - LOG(WARNING) << "No such peer=" << peer_id; + LOG(WARNING) << "node " << _group_id << ":" << _server_id + << " fail to transfer leadership, no such peer=" << peer_id; return EINVAL; } _state = STATE_TRANSFERRING; @@ -1186,22 +1193,22 @@ void NodeImpl::handle_request_vote_response(const PeerId& peer_id, const int64_t // check state if (_state != STATE_CANDIDATE) { LOG(WARNING) << "node " << _group_id << ":" << _server_id - << " received invalid RequestVoteResponse from " << peer_id - << " state not in CANDIDATE but " << state2str(_state); + << " received invalid RequestVoteResponse from " << peer_id + << " state not in CANDIDATE but " << state2str(_state); return; } // check stale response if (term != _current_term) { LOG(WARNING) << "node " << _group_id << ":" << _server_id - << " received stale RequestVoteResponse from " << peer_id - << " term " << term << " current_term " << _current_term; + << " received stale RequestVoteResponse from " << peer_id + << " term " << term << " current_term " << _current_term; return; } // check response term if (response.term() > _current_term) { LOG(WARNING) << "node " << _group_id << ":" << _server_id - << " received invalid RequestVoteResponse from " << peer_id - << " term " << response.term() << " expect " << _current_term; + << " received invalid RequestVoteResponse from " << peer_id + << " term " << response.term() << " expect " << _current_term; butil::Status status; status.set_error(EHIGHERTERMRESPONSE, "Raft node receives higher term " "request_vote_response."); @@ -1210,9 +1217,9 @@ void NodeImpl::handle_request_vote_response(const PeerId& peer_id, const int64_t } LOG(INFO) << "node " << _group_id << ":" << _server_id - << " received RequestVoteResponse from " << peer_id - << " term " << response.term() << " granted " << response.granted(); - // check granted quorum? + << " received RequestVoteResponse from " << peer_id + << " term " << response.term() << " granted " << response.granted(); + // check if the quorum granted if (response.granted()) { _vote_ctx.grant(peer_id); if (_vote_ctx.granted()) { @@ -1258,22 +1265,22 @@ void NodeImpl::handle_pre_vote_response(const PeerId& peer_id, const int64_t ter // check state if (_state != STATE_FOLLOWER) { LOG(WARNING) << "node " << _group_id << ":" << _server_id - << " received invalid PreVoteResponse from " << peer_id - << " state not in STATE_FOLLOWER but " << state2str(_state); + << " received invalid PreVoteResponse from " << peer_id + << " state not in STATE_FOLLOWER but " << state2str(_state); return; } // check stale response if (term != _current_term) { LOG(WARNING) << "node " << _group_id << ":" << _server_id - << " received stale PreVoteResponse from " << peer_id - << " term " << term << " current_term " << _current_term; + << " received stale PreVoteResponse from " << peer_id + << " term " << term << " current_term " << _current_term; return; } // check response term if (response.term() > _current_term) { LOG(WARNING) << "node " << _group_id << ":" << _server_id - << " received invalid PreVoteResponse from " << peer_id - << " term " << response.term() << " expect " << _current_term; + << " received invalid PreVoteResponse from " << peer_id + << " term " << response.term() << " expect " << _current_term; butil::Status status; status.set_error(EHIGHERTERMRESPONSE, "Raft node receives higher term " "pre_vote_response."); @@ -1282,9 +1289,9 @@ void NodeImpl::handle_pre_vote_response(const PeerId& peer_id, const int64_t ter } LOG(INFO) << "node " << _group_id << ":" << _server_id - << " received PreVoteResponse from " << peer_id - << " term " << response.term() << " granted " << response.granted(); - // check granted quorum? + << " received PreVoteResponse from " << peer_id + << " term " << response.term() << " granted " << response.granted(); + // check if the quorum granted if (response.granted()) { _pre_vote_ctx.grant(peer_id); if (_pre_vote_ctx.granted()) { @@ -1306,7 +1313,8 @@ struct OnPreVoteRPCDone : public google::protobuf::Closure { do { if (cntl.ErrorCode() != 0) { LOG(WARNING) << "node " << node->node_id() - << " request PreVote from " << peer << " error: " << cntl.ErrorText(); + << " request PreVote from " << peer + << " error: " << cntl.ErrorText(); break; } node->handle_pre_vote_response(peer, term, response); @@ -1324,13 +1332,12 @@ struct OnPreVoteRPCDone : public google::protobuf::Closure { void NodeImpl::pre_vote(std::unique_lock* lck) { LOG(INFO) << "node " << _group_id << ":" << _server_id - << " term " << _current_term - << " start pre_vote"; + << " term " << _current_term << " start pre_vote"; if (_snapshot_executor && _snapshot_executor->is_installing_snapshot()) { LOG(WARNING) << "node " << _group_id << ":" << _server_id - << " term " << _current_term - << " doesn't do pre_vote when installing snapshot as the " - " configuration is possibly out of date"; + << " term " << _current_term + << " doesn't do pre_vote when installing snapshot as the " + " configuration is possibly out of date"; return; } if (!_conf.contains(_server_id)) { @@ -1347,7 +1354,7 @@ void NodeImpl::pre_vote(std::unique_lock* lck) { // pre_vote need defense ABA after unlock&lock if (old_term != _current_term) { LOG(WARNING) << "node " << _group_id << ":" << _server_id - << " raise term " << _current_term << " when get last_log_id"; + << " raise term " << _current_term << " when get last_log_id"; return; } @@ -1365,7 +1372,7 @@ void NodeImpl::pre_vote(std::unique_lock* lck) { brpc::Channel channel; if (0 != channel.Init(iter->addr, &options)) { LOG(WARNING) << "node " << _group_id << ":" << _server_id - << " channel init failed, addr " << iter->addr; + << " channel init failed, addr " << iter->addr; continue; } @@ -1391,8 +1398,7 @@ void NodeImpl::pre_vote(std::unique_lock* lck) { // in lock void NodeImpl::elect_self(std::unique_lock* lck) { LOG(INFO) << "node " << _group_id << ":" << _server_id - << " term " << _current_term - << " start vote and grant vote self"; + << " term " << _current_term << " start vote and grant vote self"; if (!_conf.contains(_server_id)) { LOG(WARNING) << "node " << _group_id << ':' << _server_id << " can't do elect_self as it is not in " << _conf.conf; @@ -1401,7 +1407,7 @@ void NodeImpl::elect_self(std::unique_lock* lck) { // cancel follower election timer if (_state == STATE_FOLLOWER) { BRAFT_VLOG << "node " << _group_id << ":" << _server_id - << " term " << _current_term << " stop election_timer"; + << " term " << _current_term << " stop election_timer"; _election_timer.stop(); } // reset leader_id before vote @@ -1417,7 +1423,7 @@ void NodeImpl::elect_self(std::unique_lock* lck) { _voted_id = _server_id; BRAFT_VLOG << "node " << _group_id << ":" << _server_id - << " term " << _current_term << " start vote_timer"; + << " term " << _current_term << " start vote_timer"; _vote_timer.start(); _vote_ctx.init(_conf.conf, _conf.stable() ? NULL : &_conf.old_conf); @@ -1431,7 +1437,7 @@ void NodeImpl::elect_self(std::unique_lock* lck) { if (old_term != _current_term) { // term changed cause by step_down LOG(WARNING) << "node " << _group_id << ":" << _server_id - << " raise term " << _current_term << " when get last_log_id"; + << " raise term " << _current_term << " when get last_log_id"; return; } std::set peers; @@ -1448,7 +1454,7 @@ void NodeImpl::elect_self(std::unique_lock* lck) { brpc::Channel channel; if (0 != channel.Init(iter->addr, &options)) { LOG(WARNING) << "node " << _group_id << ":" << _server_id - << " channel init failed, addr " << iter->addr; + << " channel init failed, addr " << iter->addr; continue; } @@ -1593,10 +1599,8 @@ void NodeImpl::become_leader() { CHECK(_state == STATE_CANDIDATE); LOG(INFO) << "node " << _group_id << ":" << _server_id << " term " << _current_term - << " become leader of group " - << _conf.conf - << " " - << _conf.old_conf; + << " become leader of group " << _conf.conf + << " " << _conf.old_conf; // cancel candidate vote timer _vote_timer.stop(); @@ -1754,16 +1758,17 @@ int NodeImpl::handle_pre_vote_request(const RequestVoteRequest* request, const State saved_state = _state; lck.unlock(); LOG(WARNING) << "node " << _group_id << ":" << _server_id - << " is not in active state " << "current_term " << saved_current_term - << " state " << state2str(saved_state); + << " is not in active state " << "current_term " + << saved_current_term + << " state " << state2str(saved_state); return EINVAL; } PeerId candidate_id; if (0 != candidate_id.parse(request->server_id())) { LOG(WARNING) << "node " << _group_id << ":" << _server_id - << " received PreVote from " << request->server_id() - << " server_id bad format"; + << " received PreVote from " << request->server_id() + << " server_id bad format"; return EINVAL; } @@ -1772,9 +1777,9 @@ int NodeImpl::handle_pre_vote_request(const RequestVoteRequest* request, if (request->term() < _current_term) { // ignore older term LOG(INFO) << "node " << _group_id << ":" << _server_id - << " ignore PreVote from " << request->server_id() - << " in term " << request->term() - << " current_term " << _current_term; + << " ignore PreVote from " << request->server_id() + << " in term " << request->term() + << " current_term " << _current_term; break; } @@ -1788,10 +1793,10 @@ int NodeImpl::handle_pre_vote_request(const RequestVoteRequest* request, >= last_log_id); LOG(INFO) << "node " << _group_id << ":" << _server_id - << " received PreVote from " << request->server_id() - << " in term " << request->term() - << " current_term " << _current_term - << " granted " << granted; + << " received PreVote from " << request->server_id() + << " in term " << request->term() + << " current_term " << _current_term + << " granted " << granted; } while (0); @@ -1809,16 +1814,17 @@ int NodeImpl::handle_request_vote_request(const RequestVoteRequest* request, const State saved_state = _state; lck.unlock(); LOG(WARNING) << "node " << _group_id << ":" << _server_id - << " is not in active state " << "current_term " << saved_current_term - << " state " << state2str(saved_state); + << " is not in active state " << "current_term " + << saved_current_term + << " state " << state2str(saved_state); return EINVAL; } PeerId candidate_id; if (0 != candidate_id.parse(request->server_id())) { LOG(WARNING) << "node " << _group_id << ":" << _server_id - << " received RequestVote from " << request->server_id() - << " server_id bad format"; + << " received RequestVote from " << request->server_id() + << " server_id bad format"; return EINVAL; } @@ -1826,9 +1832,9 @@ int NodeImpl::handle_request_vote_request(const RequestVoteRequest* request, // check term if (request->term() >= _current_term) { LOG(INFO) << "node " << _group_id << ":" << _server_id - << " received RequestVote from " << request->server_id() - << " in term " << request->term() - << " current_term " << _current_term; + << " received RequestVote from " << request->server_id() + << " in term " << request->term() + << " current_term " << _current_term; // incress current term, change state to follower if (request->term() > _current_term) { butil::Status status; @@ -1839,9 +1845,9 @@ int NodeImpl::handle_request_vote_request(const RequestVoteRequest* request, } else { // ignore older term LOG(INFO) << "node " << _group_id << ":" << _server_id - << " ignore RequestVote from " << request->server_id() - << " in term " << request->term() - << " current_term " << _current_term; + << " ignore RequestVote from " << request->server_id() + << " in term " << request->term() + << " current_term " << _current_term; break; } @@ -1852,7 +1858,7 @@ int NodeImpl::handle_request_vote_request(const RequestVoteRequest* request, // vote need ABA check after unlock&lock if (request->term() != _current_term) { LOG(WARNING) << "node " << _group_id << ":" << _server_id - << " raise term " << _current_term << " when get last_log_id"; + << " raise term " << _current_term << " when get last_log_id"; break; } @@ -1977,8 +1983,8 @@ void NodeImpl::handle_append_entries_request(brpc::Controller* cntl, const State saved_state = _state; lck.unlock(); LOG(WARNING) << "node " << _group_id << ":" << _server_id - << " is not in active state " << "current_term " << saved_current_term - << " state " << state2str(saved_state); + << " is not in active state " << "current_term " + << saved_current_term << " state " << state2str(saved_state); cntl->SetFailed(EINVAL, "node %s:%s is not in active state, state %s", _group_id.c_str(), _server_id.to_string().c_str(), state2str(saved_state)); return; @@ -1988,8 +1994,8 @@ void NodeImpl::handle_append_entries_request(brpc::Controller* cntl, if (0 != server_id.parse(request->server_id())) { lck.unlock(); LOG(WARNING) << "node " << _group_id << ":" << _server_id - << " received AppendEntries from " << request->server_id() - << " server_id bad format"; + << " received AppendEntries from " << request->server_id() + << " server_id bad format"; cntl->SetFailed(brpc::EREQUEST, "Fail to parse server_id `%s'", request->server_id().c_str()); @@ -2001,9 +2007,9 @@ void NodeImpl::handle_append_entries_request(brpc::Controller* cntl, const int64_t saved_current_term = _current_term; lck.unlock(); LOG(WARNING) << "node " << _group_id << ":" << _server_id - << " ignore stale AppendEntries from " << request->server_id() - << " in term " << request->term() - << " current_term " << saved_current_term; + << " ignore stale AppendEntries from " << request->server_id() + << " in term " << request->term() + << " current_term " << saved_current_term; response->set_success(false); response->set_term(saved_current_term); return; @@ -2036,7 +2042,7 @@ void NodeImpl::handle_append_entries_request(brpc::Controller* cntl, (_snapshot_executor && _snapshot_executor->is_installing_snapshot())) { LOG(WARNING) << "node " << _group_id << ":" << _server_id - << " received append entries while installing snapshot"; + << " received append entries while installing snapshot"; cntl->SetFailed(EBUSY, "Is installing snapshot"); return; } @@ -2072,12 +2078,12 @@ void NodeImpl::handle_append_entries_request(brpc::Controller* cntl, response->set_last_log_index(last_index); lck.unlock(); LOG(WARNING) << "node " << _group_id << ":" << _server_id - << " reject term_unmatched AppendEntries from " << request->server_id() - << " in term " << request->term() - << " prev_log_index " << request->prev_log_index() - << " prev_log_term " << request->prev_log_term() - << " local_prev_log_term " << local_prev_log_term - << " last_log_index " << last_index + << " reject term_unmatched AppendEntries from " << request->server_id() + << " in term " << request->term() + << " prev_log_index " << request->prev_log_index() + << " prev_log_term " << request->prev_log_term() + << " local_prev_log_term " << local_prev_log_term + << " last_log_index " << last_index << " entries_size " << request->entries_size() << " from_append_entries_cache: " << from_append_entries_cache; return; @@ -2197,8 +2203,8 @@ void NodeImpl::handle_install_snapshot_request(brpc::Controller* controller, const State saved_state = _state; lck.unlock(); LOG(WARNING) << "node " << _group_id << ":" << _server_id - << " is not in active state " << "current_term " << saved_current_term - << " state " << state2str(saved_state); + << " is not in active state " << "current_term " + << saved_current_term << " state " << state2str(saved_state); cntl->SetFailed(EINVAL, "node %s:%s is not in active state, state %s", _group_id.c_str(), _server_id.to_string().c_str(), state2str(saved_state)); return; @@ -2207,9 +2213,9 @@ void NodeImpl::handle_install_snapshot_request(brpc::Controller* controller, // check stale term if (request->term() < _current_term) { LOG(WARNING) << "node " << _group_id << ":" << _server_id - << " ignore stale AppendEntries from " << request->server_id() - << " in term " << request->term() - << " current_term " << _current_term; + << " ignore stale AppendEntries from " << request->server_id() + << " in term " << request->term() + << " current_term " << _current_term; response->set_term(_current_term); response->set_success(false); return; @@ -2218,10 +2224,9 @@ void NodeImpl::handle_install_snapshot_request(brpc::Controller* controller, check_step_down(request->term(), server_id); if (server_id != _leader_id) { - LOG(ERROR) << "Another peer=" << server_id - << " declares that it is the leader at term=" - << _current_term << " which was occupied by leader=" - << _leader_id; + LOG(ERROR) << "Another peer " << _group_id << ":" << server_id + << " declares that it is the leader at term=" << _current_term + << " which was occupied by leader=" << _leader_id; // Increase the term by 1 and make both leaders step down to minimize the // loss of split brain butil::Status status; diff --git a/src/braft/replicator.cpp b/src/braft/replicator.cpp index e32aaf10..8682632a 100644 --- a/src/braft/replicator.cpp +++ b/src/braft/replicator.cpp @@ -100,7 +100,7 @@ Replicator::~Replicator() { int Replicator::start(const ReplicatorOptions& options, ReplicatorId *id) { if (options.log_manager == NULL || options.ballot_box == NULL || options.node == NULL) { - LOG(ERROR) << "Invalid arguments"; + LOG(ERROR) << "Invalid arguments, group " << options.group_id; return -1; } Replicator* r = new Replicator(); @@ -108,7 +108,8 @@ int Replicator::start(const ReplicatorOptions& options, ReplicatorId *id) { //channel_opt.connect_timeout_ms = *options.heartbeat_timeout_ms; channel_opt.timeout_ms = -1; // We don't need RPC timeout if (r->_sending_channel.Init(options.peer_id.addr, &channel_opt) != 0) { - LOG(ERROR) << "Fail to init sending channel"; + LOG(ERROR) << "Fail to init sending channel" + << ", group " << options.group_id; delete r; return -1; } @@ -120,7 +121,8 @@ int Replicator::start(const ReplicatorOptions& options, ReplicatorId *id) { r->_options = options; r->_next_index = r->_options.log_manager->last_log_index() + 1; if (bthread_id_create(&r->_id, r, _on_error) != 0) { - LOG(ERROR) << "Fail to create bthread_id"; + LOG(ERROR) << "Fail to create bthread_id" + << ", group " << options.group_id; delete r; return -1; } @@ -128,7 +130,8 @@ int Replicator::start(const ReplicatorOptions& options, ReplicatorId *id) { if (id) { *id = r->_id.value; } - LOG(INFO) << "Replicator=" << r->_id << "@" << r->_options.peer_id << " is started"; + LOG(INFO) << "Replicator=" << r->_id << "@" << r->_options.peer_id << " is started" + << ", group " << r->_options.group_id; r->_catchup_closure = NULL; r->_last_rpc_send_timestamp = butil::monotonic_time_ms(); r->_start_heartbeat_timer(butil::gettimeofday_us()); @@ -173,7 +176,8 @@ void Replicator::wait_for_caught_up(ReplicatorId id, if (r->_catchup_closure != NULL) { CHECK_EQ(0, bthread_id_unlock(dummy_id)) << "Fail to unlock " << dummy_id; - LOG(ERROR) << "Previous wait_for_caught_up is not over"; + LOG(ERROR) << "Previous wait_for_caught_up is not over" + << ", group " << r->_options.group_id; done->status().set_error(EINVAL, "Duplicated call"); run_closure_in_bthread(done); return; @@ -244,9 +248,9 @@ void Replicator::_block(long start_time_us, int error_code) { _on_block_timedout, (void*)_id.value); if (rc == 0) { std::stringstream ss; - ss << "Blocking " << _options.peer_id << " for " + ss << "Blocking " << _options.peer_id << " for " << blocking_time << "ms" << ", group " << _options.group_id; - BRAFT_VLOG << ss.str(); + BRAFT_VLOG << ss.str(); _st.st = BLOCKING; CHECK_EQ(0, bthread_id_unlock(_id)) << "Fail to unlock " << _id; return; @@ -282,8 +286,9 @@ void Replicator::_on_heartbeat_returned( BRAFT_VLOG << ss.str(); // TODO: Should it be VLOG? - LOG_IF(WARNING, (r->_consecutive_error_times++) % 10 == 0) - << "Fail to issue RPC to " << r->_options.peer_id + LOG_IF(WARNING, (r->_consecutive_error_times++) % 10 == 0) + << "Group " << r->_options.group_id + << " fail to issue RPC to " << r->_options.peer_id << " _consecutive_error_times=" << r->_consecutive_error_times << ", " << cntl->ErrorText(); r->_start_heartbeat_timer(start_time_us); @@ -301,7 +306,8 @@ void Replicator::_on_heartbeat_returned( // after _notify_on_caught_up. node_impl->AddRef(); r->_notify_on_caught_up(EPERM, true); - LOG(INFO) << "Replicator=" << dummy_id << " is going to quit"; + LOG(INFO) << "Replicator=" << dummy_id << " is going to quit" + << ", group " << r->_options.group_id; butil::Status status; status.set_error(EHIGHERTERMRESPONSE, "Leader receives higher term " "hearbeat_response from peer:%s", r->_options.peer_id.to_string().c_str()); @@ -335,9 +341,9 @@ void Replicator::_on_rpc_returned(ReplicatorId id, brpc::Controller* cntl, std::stringstream ss; ss << "node " << r->_options.group_id << ":" << r->_options.server_id - << " received AppendEntriesResponse from " - << r->_options.peer_id << " prev_log_index " << request->prev_log_index() - << " prev_log_term " << request->prev_log_term() << " count " << request->entries_size(); + << " received AppendEntriesResponse from " + << r->_options.peer_id << " prev_log_index " << request->prev_log_index() + << " prev_log_term " << request->prev_log_term() << " count " << request->entries_size(); bool valid_rpc = false; int64_t rpc_first_index = request->prev_log_index() + 1; @@ -364,8 +370,9 @@ void Replicator::_on_rpc_returned(ReplicatorId id, brpc::Controller* cntl, BRAFT_VLOG << ss.str(); // TODO: Should it be VLOG? - LOG_IF(WARNING, (r->_consecutive_error_times++) % 10 == 0) - << "Fail to issue RPC to " << r->_options.peer_id + LOG_IF(WARNING, (r->_consecutive_error_times++) % 10 == 0) + << "Group " << r->_options.group_id + << " fail to issue RPC to " << r->_options.peer_id << " _consecutive_error_times=" << r->_consecutive_error_times << ", " << cntl->ErrorText(); // If the follower crashes, any RPC to the follower fails immediately, @@ -405,18 +412,20 @@ void Replicator::_on_rpc_returned(ReplicatorId id, brpc::Controller* cntl, // prev_log_index and prev_log_term doesn't match r->_reset_next_index(); if (response->last_log_index() + 1 < r->_next_index) { - BRAFT_VLOG << "last_log_index at peer=" << r->_options.peer_id - << " is " << response->last_log_index(); + BRAFT_VLOG << "Group " << r->_options.group_id + << "last_log_index at peer=" << r->_options.peer_id + << " is " << response->last_log_index(); // The peer contains less logs than leader r->_next_index = response->last_log_index() + 1; } else { // The peer contains logs from old term which should be truncated, // decrease _last_log_at_peer by one to test the right index to keep if (BAIDU_LIKELY(r->_next_index > 1)) { - BRAFT_VLOG << "log_index=" << r->_next_index << " dismatch"; - --r->_next_index; + BRAFT_VLOG << "Group " << r->_options.group_id + << " log_index=" << r->_next_index << " dismatch"; } else { - LOG(ERROR) << "Peer=" << r->_options.peer_id + LOG(ERROR) << "Group " << r->_options.group_id + << " peer=" << r->_options.peer_id << " declares that log at index=0 doesn't match," " which is not supposed to happen"; } @@ -481,7 +490,8 @@ int Replicator::_fill_common_fields(AppendEntriesRequest* request, if (prev_log_term == 0 && prev_log_index != 0) { if (!is_heartbeat) { CHECK_LT(prev_log_index, _options.log_manager->first_log_index()); - BRAFT_VLOG << "log_index=" << prev_log_index << " was compacted"; + BRAFT_VLOG << "Group " << _options.group_id + << " log_index=" << prev_log_index << " was compacted"; return -1; } else { // The log at prev_log_index has been compacted, which indicates @@ -661,7 +671,8 @@ int Replicator::_continue_sending(void* arg, int error_code) { << " Replicator=" << id << " canceled waiter"; bthread_id_unlock(id); } else { - LOG(WARNING) << "Replicator=" << id << " stops sending entries"; + LOG(WARNING) << "Group " << r->_options.group_id + << " Replicator=" << id << " stops sending entries"; bthread_id_unlock(id); } return 0; @@ -707,14 +718,18 @@ void Replicator::_install_snapshot() { std::string uri = _reader->generate_uri_for_copy(); SnapshotMeta meta; // report error on failure - if (_reader->load_meta(&meta) != 0){ + if (uri.empty() || _reader->load_meta(&meta) != 0){ std::string snapshot_path = _reader->get_path(); NodeImpl *node_impl = _options.node; node_impl->AddRef(); CHECK_EQ(0, bthread_id_unlock(_id)) << "Fail to unlock " << _id; braft::Error e; e.set_type(ERROR_TYPE_SNAPSHOT); - e.status().set_error(EIO, "Fail to load meta from " + snapshot_path); + if (uri.empty()) { + e.status().set_error(EIO, "Fail to generate uri from " + snapshot_path); + } else { + e.status().set_error(EIO, "Fail to load meta from " + snapshot_path); + } node_impl->on_error(e); node_impl->Release(); return; @@ -732,9 +747,9 @@ void Replicator::_install_snapshot() { request->set_uri(uri); LOG(INFO) << "node " << _options.group_id << ":" << _options.server_id - << " send InstallSnapshotRequest to " << _options.peer_id - << " term " << _options.term << " last_included_term " << meta.last_included_term() - << " last_included_index " << meta.last_included_index() << " uri " << uri; + << " send InstallSnapshotRequest to " << _options.peer_id + << " term " << _options.term << " last_included_term " << meta.last_included_term() + << " last_included_index " << meta.last_included_index() << " uri " << uri; _install_snapshot_in_fly = cntl->call_id(); _install_snapshot_counter++; @@ -773,16 +788,17 @@ void Replicator::_on_install_snapshot_returned( } std::stringstream ss; ss << "received InstallSnapshotResponse from " - << r->_options.group_id << ":" << r->_options.peer_id - << " last_included_index " << request->meta().last_included_index() - << " last_included_term " << request->meta().last_included_term(); + << r->_options.group_id << ":" << r->_options.peer_id + << " last_included_index " << request->meta().last_included_index() + << " last_included_term " << request->meta().last_included_term(); do { if (cntl->Failed()) { ss << " error: " << cntl->ErrorText(); - LOG(INFO) << ss.str(); + LOG(INFO) << ss.str(); LOG_IF(WARNING, (r->_consecutive_error_times++) % 10 == 0) - << "Fail to install snapshot at peer=" + << "Group " << r->_options.group_id + << " Fail to install snapshot at peer=" << r->_options.peer_id <<", " << cntl->ErrorText(); succ = false; @@ -791,14 +807,14 @@ void Replicator::_on_install_snapshot_returned( if (!response->success()) { succ = false; ss << " fail."; - LOG(INFO) << ss.str(); + LOG(INFO) << ss.str(); // Let hearbeat do step down break; } // Success r->_next_index = request->meta().last_included_index() + 1; ss << " success."; - LOG(INFO) << ss.str(); + LOG(INFO) << ss.str(); } while (0); // We don't retry installing the snapshot explicitly. @@ -885,7 +901,8 @@ int Replicator::_on_error(bthread_id_t id, void* arg, int error_code) { r->_options.log_manager->remove_waiter(r->_wait_id); r->_notify_on_caught_up(error_code, true); r->_wait_id = 0; - LOG(INFO) << "Replicator=" << id << " is going to quit"; + LOG(INFO) << "Group " << r->_options.group_id + << " Replicator=" << id << " is going to quit"; r->_destroy(); return 0; } else if (error_code == ETIMEDOUT) { @@ -901,7 +918,8 @@ int Replicator::_on_error(bthread_id_t id, void* arg, int error_code) { } return 0; } else { - CHECK(false) << "Unknown error_code=" << error_code; + CHECK(false) << "Group " << r->_options.group_id + << " Unknown error_code=" << error_code; CHECK_EQ(0, bthread_id_unlock(id)) << "Fail to unlock " << id; return -1; } @@ -1182,7 +1200,8 @@ int ReplicatorGroup::add_replicator(const PeerId& peer) { options.peer_id = peer; ReplicatorId rid; if (Replicator::start(options, &rid) != 0) { - LOG(ERROR) << "Fail to start replicator to peer=" << peer; + LOG(ERROR) << "Group " << options.group_id + << " Fail to start replicator to peer=" << peer; return -1; } _rmap[peer] = rid; @@ -1284,10 +1303,12 @@ int ReplicatorGroup::stop_all_and_find_the_next_candidate( PeerId candidate_id; const int rc = find_the_next_candidate(&candidate_id, conf); if (rc == 0) { - LOG(INFO) << "Found " << candidate_id << " as the next candidate"; + LOG(INFO) << "Group " << _common_options.group_id + << " Found " << candidate_id << " as the next candidate"; *candidate = _rmap[candidate_id]; } else { - LOG(INFO) << "Fail to find the next candidate"; + LOG(INFO) << "Group " << _common_options.group_id + << " Fail to find the next candidate"; } for (std::map::const_iterator iter = _rmap.begin(); iter != _rmap.end(); ++iter) { diff --git a/src/braft/snapshot.cpp b/src/braft/snapshot.cpp index a170757d..a6e61ac6 100644 --- a/src/braft/snapshot.cpp +++ b/src/braft/snapshot.cpp @@ -241,7 +241,7 @@ int LocalSnapshotWriter::save_meta(const SnapshotMeta& meta) { int LocalSnapshotWriter::sync() { const int rc = _meta_table.save_to_file(_fs, _path + "/" BRAFT_SNAPSHOT_META_FILE); if (rc != 0 && ok()) { - LOG(ERROR) << "Fail to sync"; + LOG(ERROR) << "Fail to sync, path: " << _path; set_error(rc, "Fail to sync : %s", berror(rc)); } return rc; @@ -375,7 +375,7 @@ class SnapshotFileReader : public LocalDirReader { std::string LocalSnapshotReader::generate_uri_for_copy() { if (_addr == butil::EndPoint()) { - LOG(ERROR) << "Address is not specified"; + LOG(ERROR) << "Address is not specified, path:" << _path; return std::string(); } if (_reader_id == 0) { @@ -383,14 +383,12 @@ std::string LocalSnapshotReader::generate_uri_for_copy() { scoped_refptr reader( new SnapshotFileReader(_fs.get(), _path, _snapshot_throttle.get())); reader->set_meta_table(_meta_table); - - if (!reader->open()) { - LOG(ERROR) << "Open snapshot=" << _path << " failed"; - return std::string(); - } - + if (!reader->open()) { + LOG(ERROR) << "Open snapshot=" << _path << " failed"; + return std::string(); + } if (file_service_add(reader.get(), &_reader_id) != 0) { - LOG(ERROR) << "Fail to add reader to file_service"; + LOG(ERROR) << "Fail to add reader to file_service, path: " << _path; return std::string(); } } @@ -531,7 +529,7 @@ SnapshotWriter* LocalSnapshotStorage::create(bool from_empty) { writer = new LocalSnapshotWriter(snapshot_path, _fs.get()); if (writer->init() != 0) { - LOG(ERROR) << "Fail to init writer"; + LOG(ERROR) << "Fail to init writer, path: " << snapshot_path; delete writer; writer = NULL; break; @@ -548,7 +546,8 @@ SnapshotCopier* LocalSnapshotStorage::start_to_copy_from(const std::string& uri) copier->_fs = _fs.get(); copier->_throttle = _snapshot_throttle.get(); if (copier->init(uri) != 0) { - LOG(ERROR) << "Fail to init copier to " << uri; + LOG(ERROR) << "Fail to init copier from " << uri + << " path: " << _path; delete copier; return NULL; } @@ -673,7 +672,7 @@ int LocalSnapshotStorage::set_filter_before_copy_remote() { int LocalSnapshotStorage::set_file_system_adaptor(FileSystemAdaptor* fs) { if (fs == NULL) { - LOG(ERROR) << "file system is NULL"; + LOG(ERROR) << "file system is NULL, path: " << _path; return -1; } _fs = fs; @@ -802,8 +801,7 @@ int LocalSnapshotCopier::filter_before_copy(LocalSnapshotWriter* writer, if (local_meta.has_checksum() && local_meta.checksum() == remote_meta.checksum()) { LOG(INFO) << "Keep file=" << filename - << " checksum=" - << remote_meta.checksum() + << " checksum=" << remote_meta.checksum() << " in " << writer->get_path(); continue; } @@ -870,7 +868,8 @@ void LocalSnapshotCopier::filter() { SnapshotReader* reader = _storage->open(); if (filter_before_copy(_writer, reader) != 0) { LOG(WARNING) << "Fail to filter writer before copying" - << " destroy and create a new writer"; + ", path: " << _writer->get_path() + << ", destroy and create a new writer"; _writer->set_error(-1, "Fail to filter"); _storage->close(_writer, false); _writer = (LocalSnapshotWriter*)_storage->create(true); @@ -892,7 +891,8 @@ void LocalSnapshotCopier::filter() { void LocalSnapshotCopier::copy_file(const std::string& filename) { if (_writer->get_file_meta(filename, NULL) == 0) { - LOG(INFO) << "Skipped downloading " << filename; + LOG(INFO) << "Skipped downloading " << filename + << " path: " << _writer->get_path(); return; } std::string file_path = _writer->get_path() + '/' + filename; @@ -925,7 +925,8 @@ void LocalSnapshotCopier::copy_file(const std::string& filename) { scoped_refptr session = _copier.start_to_copy_to_file(filename, file_path, NULL); if (session == NULL) { - LOG(WARNING) << "Fail to copy " << filename; + LOG(WARNING) << "Fail to copy " << filename + << " path: " << _writer->get_path(); set_error(-1, "Fail to copy %s", filename.c_str()); return; } diff --git a/src/braft/snapshot_executor.cpp b/src/braft/snapshot_executor.cpp index a2a1f93d..177d7229 100644 --- a/src/braft/snapshot_executor.cpp +++ b/src/braft/snapshot_executor.cpp @@ -189,7 +189,7 @@ int SnapshotExecutor::on_snapshot_save_done( if (ret == 0) { if (writer->save_meta(meta)) { - LOG(WARNING) << "Fail to save snapshot"; + LOG(WARNING) << "node " << _node->node_id() << " fail to save snapshot"; ret = EIO; } } else { @@ -200,9 +200,12 @@ int SnapshotExecutor::on_snapshot_save_done( if (_snapshot_storage->close(writer) != 0) { ret = EIO; - LOG(WARNING) << "Fail to close writer"; + LOG(WARNING) << "node " << _node->node_id() << " fail to close writer"; } + if (_node) { + LOG(INFO) << "node " << _node->node_id() << ' ' << noflush; + } lck.lock(); if (ret == 0) { _last_snapshot_index = meta.last_included_index(); @@ -317,7 +320,7 @@ void SaveSnapshotDone::Run() { int SnapshotExecutor::init(const SnapshotExecutorOptions& options) { if (options.uri.empty()) { - LOG(ERROR) << "uri is empty()"; + LOG(ERROR) << "node " << _node->node_id() << " uri is empty()"; return -1; } _log_manager = options.log_manager; @@ -328,7 +331,8 @@ int SnapshotExecutor::init(const SnapshotExecutorOptions& options) { _snapshot_storage = SnapshotStorage::create(options.uri); if (!_snapshot_storage) { - LOG(ERROR) << "Fail to find snapshot storage, uri " << options.uri; + LOG(ERROR) << "node " << _node->node_id() + << " fail to find snapshot storage, uri " << options.uri; return -1; } if (options.filter_before_copy_remote) { @@ -342,7 +346,8 @@ int SnapshotExecutor::init(const SnapshotExecutorOptions& options) { _snapshot_storage->set_snapshot_throttle(options.snapshot_throttle); } if (_snapshot_storage->init() != 0) { - LOG(ERROR) << "Fail to init snapshot storage"; + LOG(ERROR) << "node " << _node->node_id() + << " fail to init snapshot storage, uri " << options.uri; return -1; } LocalSnapshotStorage* tmp = dynamic_cast(_snapshot_storage); @@ -396,6 +401,9 @@ void SnapshotExecutor::install_snapshot(brpc::Controller* cntl, // ^^^ DON'T access request, response, done and cntl after this point // as the retry snapshot will replace this one. if (ret != 0) { + if (_node) { + LOG(WARNING) << "node " << _node->node_id() << ' ' << noflush; + } LOG(WARNING) << "Fail to register_downloading_snapshot"; if (ret > 0) { // This RPC will be responded by the previous session @@ -468,7 +476,7 @@ void SnapshotExecutor::load_downloading_snapshot(DownloadingSnapshot* ds, new InstallSnapshotDone(this, reader); int ret = _fsm_caller->on_snapshot_load(install_snapshot_done); if (ret != 0) { - LOG(WARNING) << "Fail to call on_snapshot_load"; + LOG(WARNING) << "node " << _node->node_id() << " fail to call on_snapshot_load"; install_snapshot_done->status().set_error(EHOSTDOWN, "This raft node is down"); return install_snapshot_done->Run(); } @@ -578,7 +586,10 @@ void SnapshotExecutor::interrupt_downloading_snapshot(int64_t new_term) { } CHECK(_cur_copier); _cur_copier->cancel(); - LOG(INFO) << "Trying to cancel downloading snapshot : " + if (_node) { + LOG(INFO) << "node " << _node->node_id() << ' ' << noflush; + } + LOG(INFO) << " Trying to cancel downloading snapshot : " << _downloading_snapshot.load(butil::memory_order_relaxed) ->request->ShortDebugString(); } diff --git a/src/braft/storage.cpp b/src/braft/storage.cpp index 7396b431..7f9f3dea 100644 --- a/src/braft/storage.cpp +++ b/src/braft/storage.cpp @@ -73,7 +73,8 @@ LogStorage* LogStorage::create(const std::string& uri) { const LogStorage* type = log_storage_extension()->Find( protocol.as_string().c_str()); if (type == NULL) { - LOG(ERROR) << "Fail to find log storage type " << protocol; + LOG(ERROR) << "Fail to find log storage type " << protocol + << ", uri=" << uri; return NULL; } return type->new_instance(parameter); @@ -90,7 +91,8 @@ SnapshotStorage* SnapshotStorage::create(const std::string& uri) { const SnapshotStorage* type = snapshot_storage_extension()->Find( protocol.as_string().c_str()); if (type == NULL) { - LOG(ERROR) << "Fail to find snapshot storage type " << protocol; + LOG(ERROR) << "Fail to find snapshot storage type " << protocol + << ", uri=" << uri; return NULL; } return type->new_instance(parameter); @@ -107,7 +109,8 @@ RaftMetaStorage* RaftMetaStorage::create(const std::string& uri) { const RaftMetaStorage* type = meta_storage_extension()->Find( protocol.as_string().c_str()); if (type == NULL) { - LOG(ERROR) << "Fail to find meta storage type " << protocol; + LOG(ERROR) << "Fail to find meta storage type " << protocol + << ", uri=" << uri; return NULL; } return type->new_instance(parameter); diff --git a/test/test_node.cpp b/test/test_node.cpp index 07c3f9fa..6664ad89 100644 --- a/test/test_node.cpp +++ b/test/test_node.cpp @@ -2361,7 +2361,9 @@ TEST_P(NodeTest, on_start_following_and_on_stop_following) { cluster.wait_leader(); braft::Node* leader_first = cluster.leader(); ASSERT_TRUE(leader_first != NULL); - LOG(WARNING) << "leader_first is " << leader_first->node_id(); + LOG(WARNING) << "leader_first is " << leader_first->node_id() + << ", election_timeout is " + << leader_first->_impl->_options.election_timeout_ms; // apply something bthread::CountdownEvent cond(10); From 42cfd9ade8593b1d2b0dc775b6dc46fe79ff8f8f Mon Sep 17 00:00:00 2001 From: PFZheng Date: Tue, 23 Apr 2019 05:34:38 -0700 Subject: [PATCH 02/35] Fix bug, _node ref is not released when leader step_down during install_snapshot; add some log to test throttle of install_snapshot. --- src/braft/file_service.cpp | 6 +- src/braft/node.cpp | 38 ++++++++-- src/braft/remote_file_copier.cpp | 1 + src/braft/replicator.cpp | 15 +++- src/braft/snapshot.cpp | 1 + test/test_node.cpp | 120 +++++++++++++++++++++++++++++-- 6 files changed, 165 insertions(+), 16 deletions(-) diff --git a/src/braft/file_service.cpp b/src/braft/file_service.cpp index 23f7f510..516ff2df 100644 --- a/src/braft/file_service.cpp +++ b/src/braft/file_service.cpp @@ -46,9 +46,9 @@ void FileServiceImpl::get_file(::google::protobuf::RpcController* controller, // Don't touch iter ever after reader = iter->second; lck.unlock(); - BRAFT_VLOG << "get_file from " << cntl->remote_side() << " path=" << reader->path() - << " filename=" << request->filename() - << " offset=" << request->offset() << " count=" << request->count(); + BRAFT_VLOG << "get_file for " << cntl->remote_side() << " path=" << reader->path() + << " filename=" << request->filename() + << " offset=" << request->offset() << " count=" << request->count(); if (request->count() <= 0 || request->offset() < 0) { cntl->SetFailed(brpc::EREQUEST, "Invalid request=%s", diff --git a/src/braft/node.cpp b/src/braft/node.cpp index 1532597a..ef02861b 100644 --- a/src/braft/node.cpp +++ b/src/braft/node.cpp @@ -643,10 +643,14 @@ class OnCaughtUp : public CatchupClosure { void NodeImpl::on_caughtup(const PeerId& peer, int64_t term, int64_t version, const butil::Status& st) { BAIDU_SCOPED_LOCK(_mutex); - // CHECK _current_term and _state to avoid ABA problem - if (term != _current_term && _state != STATE_LEADER) { - // term has changed and nothing should be done, otherwise there will be - // an ABA problem. + // CHECK _state and _current_term to avoid ABA problem + if (_state != STATE_LEADER || term != _current_term) { + // if leader stepped down, reset() has already been called in step_down(), + // so nothing needs to be done here + LOG(WARNING) << "node " << node_id() << " stepped down when waiting peer " + << peer << " to catch up, current state is " << state2str(_state) + << ", current term is " << _current_term + << ", expect term is " << term; return; } @@ -661,8 +665,8 @@ void NodeImpl::on_caughtup(const PeerId& peer, int64_t term, - _replicator_group.last_rpc_send_timestamp(peer)) <= _options.election_timeout_ms) { - BRAFT_VLOG << "node " << _group_id << ":" << _server_id - << " waits peer " << peer << " to catch up"; + LOG(INFO) << "node " << _group_id << ":" << _server_id + << " waits peer " << peer << " to catch up"; OnCaughtUp* caught_up = new OnCaughtUp(this, _current_term, peer, version); timespec due_time = butil::milliseconds_from_now( @@ -1511,6 +1515,7 @@ void NodeImpl::step_down(const int64_t term, bool wakeup_a_candidate, // soft state in memory _state = STATE_FOLLOWER; + // _conf_ctx.reset() will stop replicators of catching up nodes _conf_ctx.reset(); _last_leader_timestamp = butil::monotonic_time_ms(); @@ -2681,9 +2686,15 @@ void NodeImpl::ConfigurationCtx::start(const Configuration& old_conf, Configuration removing; new_conf.diffs(old_conf, &adding, &removing); _nchanges = adding.size() + removing.size(); + + LOG(INFO) << "node " << _node->_group_id << ":" << _node->_server_id + << " change_peers from " << old_conf << " to " << new_conf << noflush; + if (adding.empty()) { + LOG(INFO) << ", begin removing."; return next_stage(); } + LOG(INFO) << ", begin caughtup."; adding.list_peers(&_adding_peers); for (std::set::const_iterator iter = _adding_peers.begin(); iter != _adding_peers.end(); ++iter) { @@ -2725,6 +2736,9 @@ void NodeImpl::ConfigurationCtx::flush(const Configuration& conf, void NodeImpl::ConfigurationCtx::on_caughtup( int64_t version, const PeerId& peer_id, bool succ) { if (version != _version) { + LOG(WARNING) << "Node " << _node->node_id() + << " on_caughtup with unmatched version=" << version + << ", expect version=" << _version; return; } CHECK_EQ(STAGE_CATCHING_UP, _stage); @@ -2782,9 +2796,21 @@ void NodeImpl::ConfigurationCtx::next_stage() { } void NodeImpl::ConfigurationCtx::reset(butil::Status* st) { + // reset() should be called only once + if (_stage == STAGE_NONE) { + BRAFT_VLOG << "node " << _node->node_id() + << " reset ConfigurationCtx when stage is STAGE_NONE already"; + return; + } + + LOG(INFO) << "node " << _node->node_id() + << " reset ConfigurationCtx, new_peers: " << Configuration(_new_peers) + << ", old_peers: " << Configuration(_old_peers); if (st && st->ok()) { _node->stop_replicator(_new_peers, _old_peers); } else { + // leader step_down may stop replicators of catching up nodes, leading to + // run catchup_closure _node->stop_replicator(_old_peers, _new_peers); } _new_peers.clear(); diff --git a/src/braft/remote_file_copier.cpp b/src/braft/remote_file_copier.cpp index 43825113..3e3e4c39 100644 --- a/src/braft/remote_file_copier.cpp +++ b/src/braft/remote_file_copier.cpp @@ -216,6 +216,7 @@ void RemoteFileCopier::Session::send_next_rpc() { new_max_count = _throttle->throttled_by_throughput(max_count); if (new_max_count == 0) { // Reset count to make next rpc retry the previous one + BRAFT_VLOG << "Copy file throttled, path: " << _dest_path; _request.set_count(0); AddRef(); int64_t retry_interval_ms_when_throttled = diff --git a/src/braft/replicator.cpp b/src/braft/replicator.cpp index 8682632a..21803280 100644 --- a/src/braft/replicator.cpp +++ b/src/braft/replicator.cpp @@ -142,6 +142,14 @@ int Replicator::start(const ReplicatorOptions& options, ReplicatorId *id) { int Replicator::stop(ReplicatorId id) { bthread_id_t dummy_id = { id }; + Replicator* r = NULL; + // already stopped + if (bthread_id_lock(dummy_id, (void**)&r) != 0) { + return 0; + } + // to run _catchup_closure if it is not NULL + r->_notify_on_caught_up(EPERM, true); + CHECK_EQ(0, bthread_id_unlock(dummy_id)) << "Fail to unlock " << dummy_id; return bthread_id_error(dummy_id, ESTOP); } @@ -836,9 +844,9 @@ void Replicator::_notify_on_caught_up(int error_code, bool before_destroy) { return; } if (error_code != ETIMEDOUT && error_code != EPERM) { - if (!_is_catchup(_catchup_closure->_max_margin)) { + if (!_is_catchup(_catchup_closure->_max_margin)) { return; - } + } if (_catchup_closure->_error_was_set) { return; } @@ -853,7 +861,7 @@ void Replicator::_notify_on_caught_up(int error_code, bool before_destroy) { return; } } - } else { // Timed out + } else { // Timed out or leader step_down if (!_catchup_closure->_error_was_set) { _catchup_closure->status().set_error(error_code, "%s", berror(error_code)); } @@ -929,6 +937,7 @@ void Replicator::_on_catch_up_timedout(void* arg) { bthread_id_t id = { (uint64_t)arg }; Replicator* r = NULL; if (bthread_id_lock(id, (void**)&r) != 0) { + LOG(WARNING) << "Replicator is destroyed when catch_up_timedout."; return; } r->_notify_on_caught_up(ETIMEDOUT, false); diff --git a/src/braft/snapshot.cpp b/src/braft/snapshot.cpp index a6e61ac6..c5a9824b 100644 --- a/src/braft/snapshot.cpp +++ b/src/braft/snapshot.cpp @@ -350,6 +350,7 @@ class SnapshotFileReader : public LocalDirReader { // if it's not allowed to read partly or it's allowed but // throughput is throttled to 0, try again. if (!read_partly || new_max_count == 0) { + BRAFT_VLOG << "Read file throttled, path: " << path(); ret = EAGAIN; } } diff --git a/test/test_node.cpp b/test/test_node.cpp index 6664ad89..810e9cf4 100644 --- a/test/test_node.cpp +++ b/test/test_node.cpp @@ -20,6 +20,7 @@ #include "braft/node.h" #include "braft/enum.pb.h" #include "braft/errno.pb.h" +#include namespace braft { extern bvar::Adder g_num_nodes; @@ -178,7 +179,6 @@ class ExpectClosure : public braft::Closure { if (_expect_err_code >= 0) { EXPECT_EQ(status().error_code(), _expect_err_code) << _pos << " : " << status(); - } if (_cond) { _cond->signal(); @@ -256,13 +256,22 @@ class Cluster { butil::endpoint2str(listen_addr).c_str()); butil::string_printf(&options.snapshot_uri, "local://./data/%s/snapshot", butil::endpoint2str(listen_addr).c_str()); - scoped_refptr tst(_throttle); + + int64_t throttle_throughput_bytes = 10 * 1024 * 1024; + int64_t check_cycle = 10; + braft::SnapshotThrottle* throttle = + new braft::ThroughputSnapshotThrottle(throttle_throughput_bytes, check_cycle); + scoped_refptr tst(throttle); options.snapshot_throttle = &tst; + options.catchup_margin = 2; + braft::Node* node = new braft::Node(_name, braft::PeerId(listen_addr, 0)); int ret = node->init(options); if (ret != 0) { LOG(WARNING) << "init_node failed, server: " << listen_addr; + delete throttle; + delete node; return ret; } else { LOG(NOTICE) << "init node " << listen_addr; @@ -388,6 +397,8 @@ class Cluster { fsm->lock(); if (first->logs.size() != fsm->logs.size()) { + LOG(INFO) << "logs size not match: " + << first->logs.size() << " vs " << fsm->logs.size(); fsm->unlock(); goto WAIT; } @@ -936,6 +947,104 @@ TEST_P(NodeTest, JoinNode) { cluster.stop_all(); } +TEST_F(NodeTest, Leader_step_down_during_install_snapshot) { + std::vector peers; + braft::PeerId peer0; + peer0.addr.ip = butil::my_ip(); + peer0.addr.port = 5006; + peer0.idx = 0; + + // start cluster + peers.push_back(peer0); + Cluster cluster("unittest", peers, 1000); + ASSERT_EQ(0, cluster.start(peer0.addr)); + LOG(NOTICE) << "start single cluster " << peer0; + + cluster.wait_leader(); + + braft::Node* leader = cluster.leader(); + ASSERT_TRUE(leader != NULL); + ASSERT_EQ(leader->node_id().peer_id, peer0); + LOG(WARNING) << "leader is " << leader->node_id(); + + bthread::CountdownEvent cond(10); + // apply something + for (int i = 0; i < 10; i++) { + butil::IOBuf data; + std::string data_buf; + data_buf.resize(256 * 1024, 'a'); + data.append(data_buf); + + braft::Task task; + task.data = &data; + task.done = NEW_APPLYCLOSURE(&cond, 0); + leader->apply(task); + } + cond.wait(); + + // trigger leader snapshot + LOG(WARNING) << "trigger leader snapshot "; + cond.reset(1); + leader->snapshot(NEW_SNAPSHOTCLOSURE(&cond, 0)); + cond.wait(); + + cond.reset(10); + // apply something + for (int i = 0; i < 10; i++) { + butil::IOBuf data; + std::string data_buf; + data_buf.resize(256 * 1024, 'b'); + data.append(data_buf); + + braft::Task task; + task.data = &data; + task.done = NEW_APPLYCLOSURE(&cond, 0); + leader->apply(task); + } + cond.wait(); + + // trigger leader snapshot again to compact logs + LOG(WARNING) << "trigger leader snapshot again"; + cond.reset(1); + leader->snapshot(NEW_SNAPSHOTCLOSURE(&cond, 0)); + cond.wait(); + + // start peer1 + braft::PeerId peer1; + peer1.addr.ip = butil::my_ip(); + peer1.addr.port = 5006 + 1; + peer1.idx = 0; + ASSERT_EQ(0, cluster.start(peer1.addr, true)); + LOG(NOTICE) << "start peer " << peer1; + // wait until started successfully + usleep(1000* 1000); + + // add peer1, leader step down while caught_up + cond.reset(1); + LOG(NOTICE) << "add peer: " << peer1; + leader->add_peer(peer1, NEW_ADDPEERCLOSURE(&cond, EPERM)); + usleep(500 * 1000); + LOG(NOTICE) << "leader " << leader->node_id() + << " step_down because of some error"; + butil::Status status; + status.set_error(braft::ERAFTTIMEDOUT, "Majority of the group dies"); + leader->_impl->step_down(leader->_impl->_current_term, false, status); + cond.wait(); + + // add peer1 again, success + LOG(NOTICE) << "add peer again: " << peer1; + cond.reset(1); + cluster.wait_leader(); + leader = cluster.leader(); + leader->add_peer(peer1, NEW_ADDPEERCLOSURE(&cond, 0)); + cond.wait(); + + cluster.ensure_same(); + + LOG(TRACE) << "stop cluster"; + cluster.stop_all(); +} + TEST_P(NodeTest, RemoveFollower) { std::vector peers; for (int i = 0; i < 3; i++) { @@ -1570,8 +1679,11 @@ TEST_P(NodeTest, InstallSnapshot) { } cond.wait(); - // wait leader to compact logs - usleep(5000 * 1000); + // trigger leader snapshot again to compact logs + LOG(WARNING) << "trigger leader snapshot again"; + cond.reset(1); + leader->snapshot(NEW_SNAPSHOTCLOSURE(&cond, 0)); + cond.wait(); LOG(WARNING) << "restart follower"; ASSERT_EQ(0, cluster.start(follower_addr)); From de34f5c896b69c5a0f32df4a7b3b054d3985b22a Mon Sep 17 00:00:00 2001 From: PFZheng Date: Tue, 23 Apr 2019 19:15:23 -0700 Subject: [PATCH 03/35] Bugfix fsm may has same queue_id with the node's apply queue in this case: node start apply_queue with queue_id values X -> fsm constructor with random queue_id value X -> node init_log_storage fail so fsm->join will block on apply_queue because node not have stop and reset apply_queue --- src/braft/fsm_caller.cpp | 19 +++++++++++++++---- src/braft/fsm_caller.h | 1 + 2 files changed, 16 insertions(+), 4 deletions(-) diff --git a/src/braft/fsm_caller.cpp b/src/braft/fsm_caller.cpp index 4aacf620..552cbab7 100644 --- a/src/braft/fsm_caller.cpp +++ b/src/braft/fsm_caller.cpp @@ -41,6 +41,7 @@ FSMCaller::FSMCaller() , _node(NULL) , _cur_task(IDLE) , _applying_index(0) + , _queue_started(false) { } @@ -155,15 +156,22 @@ int FSMCaller::init(const FSMCallerOptions &options) { execq_opt.bthread_attr = options.usercode_in_pthread ? BTHREAD_ATTR_PTHREAD : BTHREAD_ATTR_NORMAL; - bthread::execution_queue_start(&_queue_id, + if (bthread::execution_queue_start(&_queue_id, &execq_opt, FSMCaller::run, - this); + this) != 0) { + LOG(ERROR) << "fsm fail to start execution_queue"; + return -1; + } + _queue_started = true; return 0; } int FSMCaller::shutdown() { - return bthread::execution_queue_stop(_queue_id); + if (_queue_started) { + return bthread::execution_queue_stop(_queue_id); + } + return 0; } void FSMCaller::do_shutdown() { @@ -500,7 +508,10 @@ void FSMCaller::describe(std::ostream &os, bool use_html) { } void FSMCaller::join() { - bthread::execution_queue_join(_queue_id); + if (_queue_started) { + bthread::execution_queue_join(_queue_id); + _queue_started = false; + } } IteratorImpl::IteratorImpl(StateMachine* sm, LogManager* lm, diff --git a/src/braft/fsm_caller.h b/src/braft/fsm_caller.h index 7c508658..a7c55681 100644 --- a/src/braft/fsm_caller.h +++ b/src/braft/fsm_caller.h @@ -183,6 +183,7 @@ friend class IteratorImpl; TaskType _cur_task; butil::atomic _applying_index; Error _error; + bool _queue_started; }; }; From d8e4e2100f203e164ca8b8b704c31e01a85ae1ff Mon Sep 17 00:00:00 2001 From: PFZheng Date: Tue, 23 Apr 2019 19:28:41 -0700 Subject: [PATCH 04/35] Fix bug, failing to close writer when install_snapshot should set_error to LocalSnapshotCopier to make this task fail, or data consistency will be broken. --- src/braft/snapshot.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/braft/snapshot.cpp b/src/braft/snapshot.cpp index c5a9824b..b19c7e6e 100644 --- a/src/braft/snapshot.cpp +++ b/src/braft/snapshot.cpp @@ -733,7 +733,9 @@ void LocalSnapshotCopier::copy() { _writer->set_error(error_code(), error_data()); } if (_writer) { - _storage->close(_writer, _filter_before_copy_remote); + if (_storage->close(_writer, _filter_before_copy_remote) != 0) { + set_error(EIO, "Fail to close writer"); + } _writer = NULL; } if (ok()) { From 0a02365b4af3b09f8dffb23e6946645ff4667224 Mon Sep 17 00:00:00 2001 From: PFZheng Date: Tue, 23 Apr 2019 19:42:52 -0700 Subject: [PATCH 05/35] Save the last but one snapshot id in LogManager. --- src/braft/log_manager.cpp | 12 +++++++++--- src/braft/log_manager.h | 3 +++ 2 files changed, 12 insertions(+), 3 deletions(-) diff --git a/src/braft/log_manager.cpp b/src/braft/log_manager.cpp index 083584c8..2b2c9355 100644 --- a/src/braft/log_manager.cpp +++ b/src/braft/log_manager.cpp @@ -619,7 +619,7 @@ void LogManager::set_snapshot(const SnapshotMeta* meta) { _config_manager->set_snapshot(entry); int64_t term = unsafe_get_term(meta->last_included_index()); - const int64_t saved_last_snapshot_index = _last_snapshot_id.index; + _last_but_one_snapshot_id = _last_snapshot_id; _last_snapshot_id.index = meta->last_included_index(); _last_snapshot_id.term = meta->last_included_term(); if (_last_snapshot_id > _applied_id) { @@ -635,9 +635,9 @@ void LogManager::set_snapshot(const SnapshotMeta* meta) { // We don't truncate log before the lastest snapshot immediately since // some log around last_snapshot_index is probably needed by some // followers - if (saved_last_snapshot_index > 0) { + if (_last_but_one_snapshot_id.index > 0) { // We have last snapshot index - truncate_prefix(saved_last_snapshot_index + 1, lck); + truncate_prefix(_last_but_one_snapshot_id.index + 1, lck); } return; } else { @@ -678,6 +678,9 @@ int64_t LogManager::unsafe_get_term(const int64_t index) { } // check index equal snapshot_index, return snapshot_term + if (index == _last_but_one_snapshot_id.index) { + return _last_but_one_snapshot_id.term; + } if (index == _last_snapshot_id.index) { return _last_snapshot_id.term; } @@ -702,6 +705,9 @@ int64_t LogManager::get_term(const int64_t index) { } // check index equal snapshot_index, return snapshot_term + if (index == _last_but_one_snapshot_id.index) { + return _last_but_one_snapshot_id.term; + } if (index == _last_snapshot_id.index) { return _last_snapshot_id.term; } diff --git a/src/braft/log_manager.h b/src/braft/log_manager.h index d51054a1..9a4c0bfa 100644 --- a/src/braft/log_manager.h +++ b/src/braft/log_manager.h @@ -200,7 +200,10 @@ friend class AppendBatcher; std::deque _logs_in_memory; int64_t _first_log_index; int64_t _last_log_index; + // the last snapshot's log_id LogId _last_snapshot_id; + // the last but one snapshot's log_id + LogId _last_but_one_snapshot_id; bthread::ExecutionQueueId _disk_queue; }; From 3c24159d13a5c1bc0640592a0aa7669c8ef16deb Mon Sep 17 00:00:00 2001 From: PFZheng Date: Tue, 23 Apr 2019 21:13:21 -0700 Subject: [PATCH 06/35] Add 'close' interface in FileAdaptor. When a file read or write operation is completed, the 'close' interface should be called to close the corresponding resource --- src/braft/file_system_adaptor.cpp | 10 +++++++++- src/braft/file_system_adaptor.h | 9 +++++---- src/braft/remote_file_copier.cpp | 8 ++++++-- test/memory_file_system_adaptor.h | 5 +++++ 4 files changed, 25 insertions(+), 7 deletions(-) diff --git a/src/braft/file_system_adaptor.cpp b/src/braft/file_system_adaptor.cpp index 0a376f96..942883ce 100644 --- a/src/braft/file_system_adaptor.cpp +++ b/src/braft/file_system_adaptor.cpp @@ -37,7 +37,6 @@ const char* PosixDirReader::name() const { } PosixFileAdaptor::~PosixFileAdaptor() { - ::close(_fd); } ssize_t PosixFileAdaptor::write(const butil::IOBuf& data, off_t offset) { @@ -57,6 +56,15 @@ bool PosixFileAdaptor::sync() { return raft_fsync(_fd) == 0; } +bool PosixFileAdaptor::close() { + if (_fd > 0) { + bool res = ::close(_fd) == 0; + _fd = -1; + return res; + } + return true; +} + static pthread_once_t s_check_cloexec_once = PTHREAD_ONCE_INIT; static bool s_support_cloexec_on_open = false; diff --git a/src/braft/file_system_adaptor.h b/src/braft/file_system_adaptor.h index 31fc7af7..18e83b47 100644 --- a/src/braft/file_system_adaptor.h +++ b/src/braft/file_system_adaptor.h @@ -56,12 +56,13 @@ class DirReader { template struct DestroyObj { - void operator()(T* const obj) { obj->destroy(); } + void operator()(T* const obj) { obj->close(); delete obj; } }; class FileAdaptor { public: + virtual ~FileAdaptor() {} // Write to the file. Different from posix ::pwrite(), write will retry automatically // when occur EINTR. // Return |data.size()| if successful, -1 otherwise. @@ -79,13 +80,12 @@ class FileAdaptor { // Sync data of the file to disk device virtual bool sync() = 0; - // Destroy this adaptor - virtual void destroy() { delete this; } + // Close the descriptor of this file adaptor + virtual bool close() = 0; protected: FileAdaptor() {} - virtual ~FileAdaptor() {} private: DISALLOW_COPY_AND_ASSIGN(FileAdaptor); @@ -177,6 +177,7 @@ friend class PosixFileSystemAdaptor; virtual ssize_t read(butil::IOPortal* portal, off_t offset, size_t size); virtual ssize_t size(); virtual bool sync(); + virtual bool close(); protected: PosixFileAdaptor(int fd) : _fd(fd) {} diff --git a/src/braft/remote_file_copier.cpp b/src/braft/remote_file_copier.cpp index 3e3e4c39..d6338726 100644 --- a/src/braft/remote_file_copier.cpp +++ b/src/braft/remote_file_copier.cpp @@ -189,7 +189,8 @@ RemoteFileCopier::Session::Session() RemoteFileCopier::Session::~Session() { if (_file) { - _file->destroy(); + _file->close(); + delete _file; _file = NULL; } } @@ -347,7 +348,10 @@ void RemoteFileCopier::Session::on_timer(void* arg) { void RemoteFileCopier::Session::on_finished() { if (!_finished) { if (_file) { - _file->destroy(); + if (!_file->close()) { + _st.set_error(EIO, "%s", berror(EIO)); + } + delete _file; _file = NULL; } _finished = true; diff --git a/test/memory_file_system_adaptor.h b/test/memory_file_system_adaptor.h index 1bce5bfa..8706d8d7 100644 --- a/test/memory_file_system_adaptor.h +++ b/test/memory_file_system_adaptor.h @@ -97,6 +97,11 @@ class MemoryFileAdaptor : public braft::FileAdaptor { return true; } + bool close() { + return true; + } + + private: scoped_refptr _node_impl; }; From 857f4fba5c2116f22bd278a440cf1b56e837dd71 Mon Sep 17 00:00:00 2001 From: PFZheng Date: Tue, 23 Apr 2019 21:15:28 -0700 Subject: [PATCH 07/35] Fix bug caused by adding _last_but_one_snapshot_id, it may cause unexpect cases when this is hole between _last_but_one_snapshot_id and last_snapshot_id. --- src/braft/log_manager.cpp | 19 ++++++++++++------- src/braft/log_manager.h | 8 ++++++-- 2 files changed, 18 insertions(+), 9 deletions(-) diff --git a/src/braft/log_manager.cpp b/src/braft/log_manager.cpp index 2b2c9355..5275b64a 100644 --- a/src/braft/log_manager.cpp +++ b/src/braft/log_manager.cpp @@ -619,7 +619,7 @@ void LogManager::set_snapshot(const SnapshotMeta* meta) { _config_manager->set_snapshot(entry); int64_t term = unsafe_get_term(meta->last_included_index()); - _last_but_one_snapshot_id = _last_snapshot_id; + const LogId _last_but_one_snapshot_id = _last_snapshot_id; _last_snapshot_id.index = meta->last_included_index(); _last_snapshot_id.term = meta->last_included_term(); if (_last_snapshot_id > _applied_id) { @@ -628,6 +628,7 @@ void LogManager::set_snapshot(const SnapshotMeta* meta) { if (term == 0) { // last_included_index is larger than last_index // FIXME: what if last_included_index is less than first_index? + _virtual_first_log_id = _last_snapshot_id; truncate_prefix(meta->last_included_index() + 1, lck); return; } else if (term == meta->last_included_term()) { @@ -637,11 +638,13 @@ void LogManager::set_snapshot(const SnapshotMeta* meta) { // followers if (_last_but_one_snapshot_id.index > 0) { // We have last snapshot index + _virtual_first_log_id = _last_but_one_snapshot_id; truncate_prefix(_last_but_one_snapshot_id.index + 1, lck); } return; } else { // TODO: check the result of reset. + _virtual_first_log_id = _last_snapshot_id; reset(meta->last_included_index() + 1, lck); return; } @@ -677,10 +680,11 @@ int64_t LogManager::unsafe_get_term(const int64_t index) { return 0; } - // check index equal snapshot_index, return snapshot_term - if (index == _last_but_one_snapshot_id.index) { - return _last_but_one_snapshot_id.term; + // check virtual first log + if (index == _virtual_first_log_id.index) { + return _virtual_first_log_id.term; } + if (index == _last_snapshot_id.index) { return _last_snapshot_id.term; } @@ -704,10 +708,11 @@ int64_t LogManager::get_term(const int64_t index) { return 0; } - // check index equal snapshot_index, return snapshot_term - if (index == _last_but_one_snapshot_id.index) { - return _last_but_one_snapshot_id.term; + // check virtual first log + if (index == _virtual_first_log_id.index) { + return _virtual_first_log_id.term; } + if (index == _last_snapshot_id.index) { return _last_snapshot_id.term; } diff --git a/src/braft/log_manager.h b/src/braft/log_manager.h index 9a4c0bfa..a6a8cfac 100644 --- a/src/braft/log_manager.h +++ b/src/braft/log_manager.h @@ -202,8 +202,12 @@ friend class AppendBatcher; int64_t _last_log_index; // the last snapshot's log_id LogId _last_snapshot_id; - // the last but one snapshot's log_id - LogId _last_but_one_snapshot_id; + // the virtual first log, for finding next_index of replicator, which + // can avoid install_snapshot too often in extreme case where a follower's + // install_snapshot is slower than leader's save_snapshot + // [NOTICE] there should not be hole between this log_id and _last_snapshot_id, + // or may cause some unexpect cases + LogId _virtual_first_log_id; bthread::ExecutionQueueId _disk_queue; }; From 725a3830fd06fa457ac67d7d131e31adafa955ee Mon Sep 17 00:00:00 2001 From: PFZheng Date: Tue, 23 Apr 2019 21:17:03 -0700 Subject: [PATCH 08/35] fix variable name, only member variable starts with '_' --- src/braft/log_manager.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/braft/log_manager.cpp b/src/braft/log_manager.cpp index 5275b64a..61d30c0f 100644 --- a/src/braft/log_manager.cpp +++ b/src/braft/log_manager.cpp @@ -619,7 +619,7 @@ void LogManager::set_snapshot(const SnapshotMeta* meta) { _config_manager->set_snapshot(entry); int64_t term = unsafe_get_term(meta->last_included_index()); - const LogId _last_but_one_snapshot_id = _last_snapshot_id; + const LogId last_but_one_snapshot_id = _last_snapshot_id; _last_snapshot_id.index = meta->last_included_index(); _last_snapshot_id.term = meta->last_included_term(); if (_last_snapshot_id > _applied_id) { @@ -636,10 +636,10 @@ void LogManager::set_snapshot(const SnapshotMeta* meta) { // We don't truncate log before the lastest snapshot immediately since // some log around last_snapshot_index is probably needed by some // followers - if (_last_but_one_snapshot_id.index > 0) { + if (last_but_one_snapshot_id.index > 0) { // We have last snapshot index - _virtual_first_log_id = _last_but_one_snapshot_id; - truncate_prefix(_last_but_one_snapshot_id.index + 1, lck); + _virtual_first_log_id = last_but_one_snapshot_id; + truncate_prefix(last_but_one_snapshot_id.index + 1, lck); } return; } else { From e23c96bfcde162aa0f51bbb1071f382003c71f41 Mon Sep 17 00:00:00 2001 From: PFZheng Date: Tue, 23 Apr 2019 21:18:24 -0700 Subject: [PATCH 09/35] Fix compile error with gcc4.4 --- src/braft/log.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/braft/log.h b/src/braft/log.h index 549fa4fa..6c9fa81d 100644 --- a/src/braft/log.h +++ b/src/braft/log.h @@ -96,7 +96,7 @@ class BAIDU_CACHELINE_ALIGNMENT Segment std::string file_name(); private: -friend butil::RefCountedThreadSafe; +friend class butil::RefCountedThreadSafe; ~Segment() { if (_fd >= 0) { ::close(_fd); From 858b26d8b3e228e091f840e4578ed034f5a7c114 Mon Sep 17 00:00:00 2001 From: PFZheng Date: Tue, 23 Apr 2019 21:25:51 -0700 Subject: [PATCH 10/35] Fix NodeImpl::handle_append_entries_request use variables after tranfer ownership to out-of-order cache, which may cause thread concurrency issues. --- src/braft/node.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/braft/node.cpp b/src/braft/node.cpp index ef02861b..e5eef05f 100644 --- a/src/braft/node.cpp +++ b/src/braft/node.cpp @@ -2059,6 +2059,7 @@ void NodeImpl::handle_append_entries_request(brpc::Controller* cntl, int64_t last_index = _log_manager->last_log_index(); int64_t saved_term = request->term(); int saved_entries_size = request->entries_size(); + std::string rpc_server_id = request->server_id(); if (!from_append_entries_cache && handle_out_of_order_append_entries( cntl, request, response, done, last_index)) { @@ -2068,7 +2069,7 @@ void NodeImpl::handle_append_entries_request(brpc::Controller* cntl, done_guard.release(); LOG(WARNING) << "node " << _group_id << ":" << _server_id << " cache out-of-order AppendEntries from " - << request->server_id() + << rpc_server_id << " in term " << saved_term << " prev_log_index " << prev_log_index << " prev_log_term " << prev_log_term From e00ab529c54797d3cdbe0cac6de3056cdd65b626 Mon Sep 17 00:00:00 2001 From: PFZheng Date: Tue, 23 Apr 2019 21:27:33 -0700 Subject: [PATCH 11/35] test_node set default election timeout from 300ms to 3s, to avoid unit test fails on SATA disks. --- test/test_node.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/test_node.cpp b/test/test_node.cpp index 810e9cf4..d69cf4f4 100644 --- a/test/test_node.cpp +++ b/test/test_node.cpp @@ -217,7 +217,7 @@ typedef ExpectClosure SnapshotClosure; class Cluster { public: Cluster(const std::string& name, const std::vector& peers, - int32_t election_timeout_ms = 300) + int32_t election_timeout_ms = 3000) : _name(name), _peers(peers) , _election_timeout_ms(election_timeout_ms) { int64_t throttle_throughput_bytes = 10 * 1024 * 1024; From 2c0cf7564fbac34d303da567bfe3d0db2cdd31c0 Mon Sep 17 00:00:00 2001 From: PFZheng Date: Tue, 23 Apr 2019 21:29:49 -0700 Subject: [PATCH 12/35] Temporary fix ut problem of on_start_following_and_on_stop_following. --- test/test_node.cpp | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/test/test_node.cpp b/test/test_node.cpp index d69cf4f4..7c7308e0 100644 --- a/test/test_node.cpp +++ b/test/test_node.cpp @@ -2535,15 +2535,15 @@ TEST_P(NodeTest, on_start_following_and_on_stop_following) { // When it was still in follower state, it would do handle_election_timeout and // trigger on_stop_following when not receiving heartbeat for a long // time(election_timeout_ms). - ASSERT_EQ(static_cast(leader_second->_impl->_options.fsm)->_on_start_following_times, 1); - ASSERT_EQ(static_cast(leader_second->_impl->_options.fsm)->_on_stop_following_times, 1); + ASSERT_GE(static_cast(leader_second->_impl->_options.fsm)->_on_start_following_times, 1); + ASSERT_GE(static_cast(leader_second->_impl->_options.fsm)->_on_stop_following_times, 1); for (int i = 0; i < 3; i++) { // Firstly these followers have a leader, but it stops and a candidate // sends request_vote_request to them, which triggers on_stop_following. // When the candidate becomes new leader, on_start_following is triggled // again so _on_start_following_times increase by 1. - ASSERT_EQ(static_cast(followers_second[i]->_impl->_options.fsm)->_on_start_following_times, 2); - ASSERT_EQ(static_cast(followers_second[i]->_impl->_options.fsm)->_on_stop_following_times, 1); + ASSERT_GE(static_cast(followers_second[i]->_impl->_options.fsm)->_on_start_following_times, 2); + ASSERT_GE(static_cast(followers_second[i]->_impl->_options.fsm)->_on_stop_following_times, 1); } // transfer leadership to a follower @@ -2575,20 +2575,20 @@ TEST_P(NodeTest, on_start_following_and_on_stop_following) { // leader_third's _on_start_following_times and _on_stop_following_times should both be 2. // When it was still in follower state, it would do handle_timeout_now_request and // trigger on_stop_following when leader_second transferred leadership to it. - ASSERT_EQ(static_cast(leader_third->_impl->_options.fsm)->_on_start_following_times, 2); - ASSERT_EQ(static_cast(leader_third->_impl->_options.fsm)->_on_stop_following_times, 2); + ASSERT_GE(static_cast(leader_third->_impl->_options.fsm)->_on_start_following_times, 2); + ASSERT_GE(static_cast(leader_third->_impl->_options.fsm)->_on_stop_following_times, 2); for (int i = 0; i < 3; i++) { // leader_second became follower when it transferred leadership to target, // and when it receives leader_third's append_entries_request on_start_following is triggled. if (followers_third[i]->node_id().peer_id == leader_second->node_id().peer_id) { - ASSERT_EQ(static_cast(followers_third[i]->_impl->_options.fsm)->_on_start_following_times, 2); - ASSERT_EQ(static_cast(followers_third[i]->_impl->_options.fsm)->_on_stop_following_times, 1); + ASSERT_GE(static_cast(followers_third[i]->_impl->_options.fsm)->_on_start_following_times, 2); + ASSERT_GE(static_cast(followers_third[i]->_impl->_options.fsm)->_on_stop_following_times, 1); continue; } // other followers just lose the leader_second and get leader_third, so _on_stop_following_times and // _on_start_following_times both increase by 1. - ASSERT_EQ(static_cast(followers_third[i]->_impl->_options.fsm)->_on_start_following_times, 3); - ASSERT_EQ(static_cast(followers_third[i]->_impl->_options.fsm)->_on_stop_following_times, 2); + ASSERT_GE(static_cast(followers_third[i]->_impl->_options.fsm)->_on_start_following_times, 3); + ASSERT_GE(static_cast(followers_third[i]->_impl->_options.fsm)->_on_stop_following_times, 2); } cluster.ensure_same(); From a48f2f42fb3082eb495b965d7adc5ee9e088ba74 Mon Sep 17 00:00:00 2001 From: PFZheng Date: Tue, 23 Apr 2019 21:34:08 -0700 Subject: [PATCH 13/35] test_node use ASSERT_EQ rather than EXPECT_EQ to break from unittests, when check expect error code failed --- test/test_node.cpp | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/test/test_node.cpp b/test/test_node.cpp index 7c7308e0..e6b40a47 100644 --- a/test/test_node.cpp +++ b/test/test_node.cpp @@ -83,7 +83,9 @@ class MockFSM : public braft::StateMachine { virtual void on_apply(braft::Iterator& iter) { for (; iter.valid(); iter.next()) { - LOG_IF(TRACE, !g_dont_print_apply_log) << "addr " << address << " apply " << iter.index(); + LOG_IF(TRACE, !g_dont_print_apply_log) << "addr " << address + << " apply " << iter.index() + << " data " << iter.data(); ::brpc::ClosureGuard guard(iter.done()); lock(); logs.push_back(iter.data()); @@ -177,7 +179,7 @@ class ExpectClosure : public braft::Closure { public: void Run() { if (_expect_err_code >= 0) { - EXPECT_EQ(status().error_code(), _expect_err_code) + ASSERT_EQ(status().error_code(), _expect_err_code) << _pos << " : " << status(); } if (_cond) { @@ -397,8 +399,10 @@ class Cluster { fsm->lock(); if (first->logs.size() != fsm->logs.size()) { - LOG(INFO) << "logs size not match: " - << first->logs.size() << " vs " << fsm->logs.size(); + LOG(INFO) << "logs size not match, " + << " addr: " << first->address << " vs " + << fsm->address << ", log num " + << first->logs.size() << " vs " << fsm->logs.size(); fsm->unlock(); goto WAIT; } @@ -409,9 +413,9 @@ class Cluster { if (first_data.to_string() != fsm_data.to_string()) { LOG(INFO) << "log data of index=" << j << " not match, " << " addr: " << first->address << " vs " - << fsm->address << ", data " - << first_data.to_string() << " vs " - << fsm_data.to_string(); + << fsm->address << ", data (" + << first_data.to_string() << ") vs " + << fsm_data.to_string() << ")"; fsm->unlock(); goto WAIT; } From 463428a5dc81341a73c630a4b2f1bec4c865feba Mon Sep 17 00:00:00 2001 From: PFZheng Date: Tue, 23 Apr 2019 21:39:08 -0700 Subject: [PATCH 14/35] Add configuration change info in NodeImpl::describe() --- src/braft/node.cpp | 45 +++++++++++++++++++++++++++++++++++++++++++++ src/braft/node.h | 33 +++++++++++++++++++++++++++++---- test/test_node.cpp | 23 ++++++++++++++++++++++- 3 files changed, 96 insertions(+), 5 deletions(-) diff --git a/src/braft/node.cpp b/src/braft/node.cpp index e5eef05f..4fa7493e 100644 --- a/src/braft/node.cpp +++ b/src/braft/node.cpp @@ -2309,6 +2309,16 @@ void NodeImpl::describe(std::ostream& os, bool use_html) { //const int ref_count = ref_count_; std::vector peers; _conf.conf.list_peers(&peers); + + const std::string is_changing_conf = _conf_ctx.is_busy() ? "YES" : "NO"; + const char* conf_statge = _conf_ctx.stage_str(); + // new_peers and old_peers during all conf-change stages, namely + // STAGE_CATCHING_UP->STAGE_JOINT->STAGE_STABLE + std::vector new_peers; + _conf_ctx.list_new_peers(&new_peers); + std::vector old_peers; + _conf_ctx.list_old_peers(&old_peers); + // No replicator attached to nodes that are not leader; _replicator_group.list_replicators(&replicators); const int64_t leader_timestamp = _last_leader_timestamp; @@ -2331,6 +2341,41 @@ void NodeImpl::describe(std::ostream& os, bool use_html) { } } os << newline; // newline for peers + // info of configuration change + if (st == STATE_LEADER) { + os << "changing_conf: " << is_changing_conf + << " stage: " << conf_statge << newline; + } + if (!new_peers.empty()) { + os << "new_peers:"; + for (size_t j = 0; j < new_peers.size(); ++j) { + os << ' '; + if (use_html && new_peers[j] != _server_id) { + os << ""; + } + os << new_peers[j]; + if (use_html && new_peers[j] != _server_id) { + os << ""; + } + } + os << newline; // newline for new_peers + } + if (!old_peers.empty()) { + os << "old_peers:"; + for (size_t j = 0; j < old_peers.size(); ++j) { + os << ' '; + if (use_html && old_peers[j] != _server_id) { + os << ""; + } + os << old_peers[j]; + if (use_html && old_peers[j] != _server_id) { + os << ""; + } + } + os << newline; // newline for old_peers + } if (st == STATE_FOLLOWER) { os << "leader: "; diff --git a/src/braft/node.h b/src/braft/node.h index b2e43b20..3f8590e4 100644 --- a/src/braft/node.h +++ b/src/braft/node.h @@ -284,13 +284,38 @@ friend class butil::RefCountedThreadSafe; DISALLOW_COPY_AND_ASSIGN(ConfigurationCtx); public: enum Stage { - STAGE_NONE, - STAGE_CATCHING_UP, - STAGE_JOINT, - STAGE_STABLE, + // Don't change the order if you are not sure about the usage + STAGE_NONE = 0, + STAGE_CATCHING_UP = 1, + STAGE_JOINT = 2, + STAGE_STABLE = 3, }; ConfigurationCtx(NodeImpl* node) : _node(node), _stage(STAGE_NONE), _version(0), _done(NULL) {} + void list_new_peers(std::vector* new_peers) const { + new_peers->clear(); + std::set::iterator it; + for (it = _new_peers.begin(); it != _new_peers.end(); ++it) { + new_peers->push_back(*it); + } + } + void list_old_peers(std::vector* old_peers) const { + old_peers->clear(); + std::set::iterator it; + for (it = _old_peers.begin(); it != _old_peers.end(); ++it) { + old_peers->push_back(*it); + } + } + const char* stage_str() { + const char* str[] = {"STAGE_NONE", "STAGE_CATCHING_UP", + "STAGE_JOINT", "STAGE_STABLE", }; + if (_stage <= STAGE_STABLE) { + return str[(int)_stage]; + } else { + return "UNKNOWN"; + } + } + int32_t stage() const { return _stage; } void reset(butil::Status* st = NULL); bool is_busy() const { return _stage != STAGE_NONE; } // Start change configuration. diff --git a/test/test_node.cpp b/test/test_node.cpp index e6b40a47..3e627a28 100644 --- a/test/test_node.cpp +++ b/test/test_node.cpp @@ -1027,7 +1027,28 @@ TEST_F(NodeTest, Leader_step_down_during_install_snapshot) { cond.reset(1); LOG(NOTICE) << "add peer: " << peer1; leader->add_peer(peer1, NEW_ADDPEERCLOSURE(&cond, EPERM)); - usleep(500 * 1000); + usleep(50 * 1000); + + { + brpc::Channel channel; + brpc::ChannelOptions options; + options.protocol = brpc::PROTOCOL_HTTP; + + if (channel.Init(leader->node_id().peer_id.addr, &options) != 0) { + LOG(ERROR) << "Fail to initialize channel"; + } + + { + brpc::Controller cntl; + cntl.http_request().uri() = "/raft_stat/unittest"; + cntl.http_request().set_method(brpc::HTTP_METHOD_GET); + + channel.CallMethod(NULL, &cntl, NULL, NULL, NULL/*done*/); + + LOG(NOTICE) << "http return: \n" << cntl.response_attachment(); + } + } + LOG(NOTICE) << "leader " << leader->node_id() << " step_down because of some error"; butil::Status status; From aeed67315fcbfbf397161e52c5e234fe52bffa27 Mon Sep 17 00:00:00 2001 From: PFZheng Date: Tue, 23 Apr 2019 21:41:31 -0700 Subject: [PATCH 15/35] on_configurtion_committed callback with log index --- src/braft/fsm_caller.cpp | 5 +++-- src/braft/raft.cpp | 5 +++++ src/braft/raft.h | 1 + test/test_node.cpp | 4 ++-- 4 files changed, 11 insertions(+), 4 deletions(-) diff --git a/src/braft/fsm_caller.cpp b/src/braft/fsm_caller.cpp index 552cbab7..f70ef908 100644 --- a/src/braft/fsm_caller.cpp +++ b/src/braft/fsm_caller.cpp @@ -265,7 +265,8 @@ void FSMCaller::do_committed(int64_t committed_index) { if (iter_impl.entry()->old_peers == NULL) { // Joint stage is not supposed to be noticeable by end users. _fsm->on_configuration_committed( - Configuration(*iter_impl.entry()->peers)); + Configuration(*iter_impl.entry()->peers), + iter_impl.entry()->id.index); } } // For other entries, we have nothing to do besides flush the @@ -399,7 +400,7 @@ void FSMCaller::do_snapshot_load(LoadSnapshotClosure* done) { for (int i = 0; i < meta.peers_size(); ++i) { conf.add_peer(meta.peers(i)); } - _fsm->on_configuration_committed(conf); + _fsm->on_configuration_committed(conf, meta.last_included_index()); } _last_applied_index.store(meta.last_included_index(), diff --git a/src/braft/raft.cpp b/src/braft/raft.cpp index 1e66fe09..73695f4d 100644 --- a/src/braft/raft.cpp +++ b/src/braft/raft.cpp @@ -228,6 +228,11 @@ void StateMachine::on_configuration_committed(const Configuration& conf) { return; } +void StateMachine::on_configuration_committed(const Configuration& conf, int64_t index) { + (void)index; + return on_configuration_committed(conf); +} + void StateMachine::on_stop_following(const LeaderChangeContext&) {} void StateMachine::on_start_following(const LeaderChangeContext&) {} diff --git a/src/braft/raft.h b/src/braft/raft.h index 2b7c79f0..d19eae25 100644 --- a/src/braft/raft.h +++ b/src/braft/raft.h @@ -241,6 +241,7 @@ class StateMachine { // Invoked when a configuration has been committed to the group virtual void on_configuration_committed(const ::braft::Configuration& conf); + virtual void on_configuration_committed(const ::braft::Configuration& conf, int64_t index); // this method is called when a follower stops following a leader and its leader_id becomes NULL, // situations including: diff --git a/test/test_node.cpp b/test/test_node.cpp index 3e627a28..9c995de6 100644 --- a/test/test_node.cpp +++ b/test/test_node.cpp @@ -169,8 +169,8 @@ class MockFSM : public braft::StateMachine { ++_on_stop_following_times; } - virtual void on_configuration_committed(const ::braft::Configuration& conf) { - LOG(TRACE) << "address " << address << " commit conf: " << conf; + virtual void on_configuration_committed(const ::braft::Configuration& conf, int64_t index) { + LOG(TRACE) << "address " << address << " commit conf: " << conf << " at index " << index; } }; From ceb4d1f93f8742cbe788a892df2a3740db5c47b0 Mon Sep 17 00:00:00 2001 From: PFZheng Date: Tue, 23 Apr 2019 22:15:37 -0700 Subject: [PATCH 16/35] add Node::get_status interface to get internal status of raft --- src/braft/ballot_box.cpp | 12 ++++++ src/braft/ballot_box.h | 11 ++++++ src/braft/fsm_caller.cpp | 9 +++++ src/braft/fsm_caller.h | 1 + src/braft/log_manager.cpp | 11 ++++++ src/braft/log_manager.h | 13 +++++++ src/braft/node.cpp | 61 +++++++++++++++++++++++++++++ src/braft/node.h | 4 ++ src/braft/raft.cpp | 4 ++ src/braft/raft.h | 80 +++++++++++++++++++++++++++++++++++++++ src/braft/replicator.cpp | 32 ++++++++++++++++ src/braft/replicator.h | 7 ++++ test/test_node.cpp | 32 ++++++++++++---- 13 files changed, 270 insertions(+), 7 deletions(-) diff --git a/src/braft/ballot_box.cpp b/src/braft/ballot_box.cpp index 705c72e3..45272718 100644 --- a/src/braft/ballot_box.cpp +++ b/src/braft/ballot_box.cpp @@ -173,4 +173,16 @@ void BallotBox::describe(std::ostream& os, bool use_html) { } } +void BallotBox::get_status(BallotBoxStatus* status) { + if (!status) { + return; + } + std::unique_lock lck(_mutex); + status->committed_index = _last_committed_index; + if (_pending_index != 0) { + status->pending_index = _pending_index; + status->pending_queue_size = _pending_meta_queue.size(); + } +} + } // namespace braft diff --git a/src/braft/ballot_box.h b/src/braft/ballot_box.h index 6148831c..f824f3ea 100644 --- a/src/braft/ballot_box.h +++ b/src/braft/ballot_box.h @@ -39,6 +39,15 @@ struct BallotBoxOptions { ClosureQueue* closure_queue; }; +struct BallotBoxStatus { + BallotBoxStatus() + : committed_index(0), pending_index(0), pending_queue_size(0) + {} + int64_t committed_index; + int64_t pending_index; + int64_t pending_queue_size; +}; + class BallotBox { public: BallotBox(); @@ -79,6 +88,8 @@ class BallotBox { void describe(std::ostream& os, bool use_html); + void get_status(BallotBoxStatus* ballot_box_status); + private: FSMCaller* _waiter; diff --git a/src/braft/fsm_caller.cpp b/src/braft/fsm_caller.cpp index f70ef908..8fcb0922 100644 --- a/src/braft/fsm_caller.cpp +++ b/src/braft/fsm_caller.cpp @@ -508,6 +508,15 @@ void FSMCaller::describe(std::ostream &os, bool use_html) { os << newline; } +int64_t FSMCaller::applying_index() const { + TaskType cur_task = _cur_task; + if (cur_task != COMMITTED) { + return 0; + } else { + return _applying_index.load(butil::memory_order_relaxed); + } +} + void FSMCaller::join() { if (_queue_started) { bthread::execution_queue_join(_queue_id); diff --git a/src/braft/fsm_caller.h b/src/braft/fsm_caller.h index a7c55681..de8a67b3 100644 --- a/src/braft/fsm_caller.h +++ b/src/braft/fsm_caller.h @@ -119,6 +119,7 @@ class BAIDU_CACHELINE_ALIGNMENT FSMCaller { int64_t last_applied_index() const { return _last_applied_index.load(butil::memory_order_relaxed); } + int64_t applying_index() const; void describe(std::ostream& os, bool use_html); void join(); private: diff --git a/src/braft/log_manager.cpp b/src/braft/log_manager.cpp index 61d30c0f..91a98c91 100644 --- a/src/braft/log_manager.cpp +++ b/src/braft/log_manager.cpp @@ -892,6 +892,17 @@ void LogManager::describe(std::ostream& os, bool use_html) { os << "last_log_id: " << last_log_id() << newline; } +void LogManager::get_status(LogManagerStatus* status) { + if (!status) { + return; + } + std::unique_lock lck(_mutex); + status->first_index = _log_storage->first_log_index(); + status->last_index = _log_storage->last_log_index(); + status->disk_index = _disk_id.index; + status->known_applied_index = _applied_id.index; +} + void LogManager::report_error(int error_code, const char* fmt, ...) { _has_error.store(true, butil::memory_order_relaxed); va_list ap; diff --git a/src/braft/log_manager.h b/src/braft/log_manager.h index a6a8cfac..8dd4cef6 100644 --- a/src/braft/log_manager.h +++ b/src/braft/log_manager.h @@ -40,6 +40,16 @@ struct LogManagerOptions { FSMCaller* fsm_caller; // To report log error }; +struct LogManagerStatus { + LogManagerStatus() + : first_index(1), last_index(0), disk_index(0), known_applied_index(0) + {} + int64_t first_index; + int64_t last_index; + int64_t disk_index; + int64_t known_applied_index; +}; + class SnapshotMeta; class BAIDU_CACHELINE_ALIGNMENT LogManager { @@ -133,6 +143,9 @@ class BAIDU_CACHELINE_ALIGNMENT LogManager { void describe(std::ostream& os, bool use_html); + // Get the internal status of LogManager. + void get_status(LogManagerStatus* status); + private: friend class AppendBatcher; struct WaitMeta { diff --git a/src/braft/node.cpp b/src/braft/node.cpp index 4fa7493e..1bd43d7d 100644 --- a/src/braft/node.cpp +++ b/src/braft/node.cpp @@ -2341,6 +2341,7 @@ void NodeImpl::describe(std::ostream& os, bool use_html) { } } os << newline; // newline for peers + // info of configuration change if (st == STATE_LEADER) { os << "changing_conf: " << is_changing_conf @@ -2415,6 +2416,66 @@ void NodeImpl::describe(std::ostream& os, bool use_html) { Replicator::describe(replicators[i], os, use_html); } } + +void NodeImpl::get_status(NodeStatus* status) { + if (status == NULL) { + return; + } + + std::vector peers; + std::vector > replicators; + std::unique_lock lck(_mutex); + status->state = _state; + status->term = _current_term; + status->peer_id = _server_id; + _conf.conf.list_peers(&peers); + _replicator_group.list_replicators(&replicators); + lck.unlock(); + + if (status->state == STATE_LEADER || + status->state == STATE_TRANSFERRING) { + status->leader_id = _server_id; + } else if (status->state == STATE_FOLLOWER) { + status->leader_id = _leader_id; + } + + LogManagerStatus log_manager_status; + _log_manager->get_status(&log_manager_status); + status->known_applied_index = log_manager_status.known_applied_index; + status->first_index = log_manager_status.first_index; + status->last_index = log_manager_status.last_index; + status->disk_index = log_manager_status.disk_index; + + BallotBoxStatus ballot_box_status; + _ballot_box->get_status(&ballot_box_status); + status->committed_index = ballot_box_status.committed_index; + status->pending_index = ballot_box_status.pending_index; + status->pending_queue_size = ballot_box_status.pending_queue_size; + + status->applying_index = _fsm_caller->applying_index(); + + if (replicators.size() == 0) { + return; + } + + for (size_t i = 0; i < peers.size(); ++i) { + if (peers[i] == _server_id) { + continue; + } + status->stable_followers.insert(std::make_pair(peers[i], PeerStatus())); + } + + for (size_t i = 0; i < replicators.size(); ++i) { + NodeStatus::PeerStatusMap::iterator it = + status->stable_followers.find(replicators[i].first); + if (it == status->stable_followers.end()) { + it = status->unstable_followers.insert( + std::make_pair(replicators[i].first, PeerStatus())).first; + } + Replicator::get_status(replicators[i].second, &(it->second)); + } +} + void NodeImpl::stop_replicator(const std::set& keep, const std::set& drop) { for (std::set::const_iterator diff --git a/src/braft/node.h b/src/braft/node.h index 3f8590e4..a392410f 100644 --- a/src/braft/node.h +++ b/src/braft/node.h @@ -195,6 +195,10 @@ friend class ConfigurationChangeDone; void update_configuration_after_installing_snapshot(); void describe(std::ostream& os, bool use_html); + + // Get the internal status of this node, the information is mostly the same as we + // see from the website, which is generated by |describe| actually. + void get_status(NodeStatus* status); // Call on_error when some error happens, after this is called. // After this point: diff --git a/src/braft/raft.cpp b/src/braft/raft.cpp index 73695f4d..dbb09398 100644 --- a/src/braft/raft.cpp +++ b/src/braft/raft.cpp @@ -164,6 +164,10 @@ butil::Status Node::read_committed_user_log(const int64_t index, UserLog* user_l return _impl->read_committed_user_log(index, user_log); } +void Node::get_status(NodeStatus* status) { + return _impl->get_status(status); +} + // ------------- Iterator void Iterator::next() { if (valid()) { diff --git a/src/braft/raft.h b/src/braft/raft.h index d19eae25..ea037ed2 100644 --- a/src/braft/raft.h +++ b/src/braft/raft.h @@ -353,6 +353,82 @@ inline std::ostream& operator<<(std::ostream& os, const UserLog& user_log) { return os; } +// Status of a peer +struct PeerStatus { + PeerStatus() + : valid(false), installing_snapshot(false), next_index(0) + , last_rpc_send_timestamp(0), flying_append_entries_size(0) + , consecutive_error_times(0) + {} + + bool valid; + bool installing_snapshot; + int64_t next_index; + int64_t last_rpc_send_timestamp; + int64_t flying_append_entries_size; + int consecutive_error_times; +}; + +// Status of Node +class NodeStatus { +friend class NodeImpl; +public: + typedef std::map PeerStatusMap; + + NodeStatus() + : state(STATE_END), term(0), committed_index(0), known_applied_index(0) + , pending_index(0), pending_queue_size(0), applying_index(0), first_index(0) + , last_index(-1), disk_index(0) + {} + + State state; + PeerId peer_id; + PeerId leader_id; + int64_t term; + int64_t committed_index; + int64_t known_applied_index; + + // The start index of the logs waiting to be committed. + // If the value is 0, means no pending logs. + // + // WARNING: if this value is not 0, and keep the same in a long time, + // means something happend to prevent the node to commit logs in a + // large probability, and users should check carefully to find out + // the reasons. + int64_t pending_index; + + // How many pending logs waiting to be committed. + // + // WARNING: too many pending logs, means the processing rate can't catup with + // the writing rate. Users can consider to slow down the writing rate to avoid + // exhaustion of resources. + int64_t pending_queue_size; + + // The current applying index. If the value is 0, means no applying log. + // + // WARNING: if this value is not 0, and keep the same in a long time, means + // the apply thread hung, users should check if a deadlock happend, or some + // time-consuming operations is handling in place. + int64_t applying_index; + + // The first log of the node, including the logs in memory and disk. + int64_t first_index; + + // The last log of the node, including the logs in memory and disk. + int64_t last_index; + + // The max log in disk. + int64_t disk_index; + + // Stable followers are peers in current configuration. + // If the node is not leader, this map is empty. + PeerStatusMap stable_followers; + + // Unstable followers are peers not in current configurations. For example, + // if a new peer is added and not catchup now, it's in this map. + PeerStatusMap unstable_followers; +}; + struct NodeOptions { // A follower would become a candidate if it doesn't receive any message // from the leader in |election_timeout_ms| milliseconds @@ -530,6 +606,10 @@ class Node { // in code implementation. butil::Status read_committed_user_log(const int64_t index, UserLog* user_log); + // Get the internal status of this node, the information is mostly the same as we + // see from the website. + void get_status(NodeStatus* status); + private: NodeImpl* _impl; }; diff --git a/src/braft/replicator.cpp b/src/braft/replicator.cpp index 21803280..6a502e2c 100644 --- a/src/braft/replicator.cpp +++ b/src/braft/replicator.cpp @@ -1153,6 +1153,16 @@ void Replicator::_describe(std::ostream& os, bool use_html) { os << " hc=" << heartbeat_counter << " ac=" << append_entries_counter << " ic=" << install_snapshot_counter << new_line; } +void Replicator::_get_status(PeerStatus* status) { + status->valid = true; + status->installing_snapshot = (_st.st == INSTALLING_SNAPSHOT); + status->next_index = _next_index; + status->flying_append_entries_size = _flying_append_entries_size; + status->last_rpc_send_timestamp = _last_rpc_send_timestamp; + status->consecutive_error_times = _consecutive_error_times; + CHECK_EQ(0, bthread_id_unlock(_id)); +} + void Replicator::describe(ReplicatorId id, std::ostream& os, bool use_html) { bthread_id_t dummy_id = { id }; Replicator* r = NULL; @@ -1163,6 +1173,18 @@ void Replicator::describe(ReplicatorId id, std::ostream& os, bool use_html) { return r->_describe(os, use_html); } +void Replicator::get_status(ReplicatorId id, PeerStatus* status) { + if (!status) { + return; + } + bthread_id_t dummy_id = { id }; + Replicator* r = NULL; + if (bthread_id_lock(dummy_id, (void**)&r) != 0) { + return; + } + return r->_get_status(status); +} + // ==================== ReplicatorGroup ========================== ReplicatorGroupOptions::ReplicatorGroupOptions() @@ -1360,4 +1382,14 @@ void ReplicatorGroup::list_replicators(std::vector* out) const { } } +void ReplicatorGroup::list_replicators( + std::vector >* out) const { + out->clear(); + out->reserve(_rmap.size()); + for (std::map::const_iterator + iter = _rmap.begin(); iter != _rmap.end(); ++iter) { + out->push_back(*iter); + } +} + } // namespace braft diff --git a/src/braft/replicator.h b/src/braft/replicator.h index 0a8ea5ed..c3cc6d10 100644 --- a/src/braft/replicator.h +++ b/src/braft/replicator.h @@ -106,6 +106,9 @@ class BAIDU_CACHELINE_ALIGNMENT Replicator { static int64_t get_next_index(ReplicatorId id); static void describe(ReplicatorId id, std::ostream& os, bool use_html); + + // Get replicator internal status. + static void get_status(ReplicatorId id, PeerStatus* status); private: enum St { @@ -181,6 +184,7 @@ class BAIDU_CACHELINE_ALIGNMENT Replicator { InstallSnapshotResponse* response); void _destroy(); void _describe(std::ostream& os, bool use_html); + void _get_status(PeerStatus* status); bool _is_catchup(int64_t max_margin) { // We should wait until install snapshot finish. If the process is throttled, // it maybe very slow. @@ -317,6 +321,9 @@ class ReplicatorGroup { // List all the existing replicators void list_replicators(std::vector* out) const; + // List all the existing replicators with PeerId + void list_replicators(std::vector >* out) const; + private: int _add_replicator(const PeerId& peer, ReplicatorId *rid); diff --git a/test/test_node.cpp b/test/test_node.cpp index 9c995de6..ae1532df 100644 --- a/test/test_node.cpp +++ b/test/test_node.cpp @@ -367,6 +367,26 @@ class Cluster { } } + void check_node_status() { + std::vector nodes; + { + std::lock_guard guard(_mutex); + for (size_t i = 0; i < _nodes.size(); i++) { + nodes.push_back(_nodes[i]); + } + } + for (size_t i = 0; i < _nodes.size(); ++i) { + braft::NodeStatus status; + nodes[i]->get_status(&status); + if (nodes[i]->is_leader()) { + ASSERT_EQ(status.state, braft::STATE_LEADER); + } else { + ASSERT_NE(status.state, braft::STATE_LEADER); + ASSERT_EQ(status.stable_followers.size(), 0); + } + } + } + void ensure_leader(const butil::EndPoint& expect_addr) { CHECK: std::lock_guard guard(_mutex); @@ -384,7 +404,7 @@ class Cluster { } bool ensure_same(int wait_time_s = -1) { - std::lock_guard guard(_mutex); + std::unique_lock guard(_mutex); if (_fsms.size() <= 1) { return true; } @@ -424,6 +444,8 @@ class Cluster { fsm->unlock(); } first->unlock(); + guard.unlock(); + check_node_status(); return true; WAIT: @@ -1027,24 +1049,20 @@ TEST_F(NodeTest, Leader_step_down_during_install_snapshot) { cond.reset(1); LOG(NOTICE) << "add peer: " << peer1; leader->add_peer(peer1, NEW_ADDPEERCLOSURE(&cond, EPERM)); - usleep(50 * 1000); + usleep(500 * 1000); { brpc::Channel channel; brpc::ChannelOptions options; options.protocol = brpc::PROTOCOL_HTTP; - if (channel.Init(leader->node_id().peer_id.addr, &options) != 0) { LOG(ERROR) << "Fail to initialize channel"; } - { brpc::Controller cntl; cntl.http_request().uri() = "/raft_stat/unittest"; cntl.http_request().set_method(brpc::HTTP_METHOD_GET); - - channel.CallMethod(NULL, &cntl, NULL, NULL, NULL/*done*/); - + channel.CallMethod(NULL, &cntl, NULL, NULL, NULL/* done*/); LOG(NOTICE) << "http return: \n" << cntl.response_attachment(); } } From 4cc0ea0bc191080eeea2b090acaa58753094c3a1 Mon Sep 17 00:00:00 2001 From: PFZheng Date: Tue, 23 Apr 2019 22:18:03 -0700 Subject: [PATCH 17/35] Make the usage of |node_owns_log_storage| and |log_storage| initial options more clearly. |node_owns_log_storage| only indicate that the life cycle of log storage is managed by NodeImpl, no matter the log storage is created inside NodeImpl, or assigned by user defined |log_storage| option. --- src/braft/node.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/braft/node.cpp b/src/braft/node.cpp index 1bd43d7d..b84de035 100644 --- a/src/braft/node.cpp +++ b/src/braft/node.cpp @@ -218,7 +218,11 @@ int NodeImpl::init_snapshot_storage() { int NodeImpl::init_log_storage() { CHECK(_fsm_caller); - _log_storage = LogStorage::create(_options.log_uri); + if (_options.log_storage) { + _log_storage = _options.log_storage; + } else { + _log_storage = LogStorage::create(_options.log_uri); + } if (!_log_storage) { LOG(ERROR) << "node " << _group_id << ":" << _server_id << " find log storage failed, uri " << _options.log_uri; From 93d248b70cf813f157e68965079ef95884d8ef74 Mon Sep 17 00:00:00 2001 From: PFZheng Date: Tue, 23 Apr 2019 22:30:58 -0700 Subject: [PATCH 18/35] Raise an error if a full segment is found to be incomplete at the loading stage. The data lost may happend when: 1) Disk has errors; 2) |raft_sync_segments| is set to false, and the kernel fail to flush page cache after disk full or machine crash. --- src/braft/log.cpp | 16 +++++-- test/test_log.cpp | 105 ++++++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 118 insertions(+), 3 deletions(-) diff --git a/src/braft/log.cpp b/src/braft/log.cpp index b1006b3a..cf413072 100644 --- a/src/braft/log.cpp +++ b/src/braft/log.cpp @@ -272,6 +272,7 @@ int Segment::load(ConfigurationManager* configuration_manager) { // load entry index int64_t file_size = st_buf.st_size; int64_t entry_off = 0; + int64_t actual_last_index = _first_index - 1; for (int64_t i = _first_index; entry_off < file_size; i++) { EntryHeader header; const int rc = _load_entry(entry_off, &header, NULL, ENTRY_HEADER_SIZE); @@ -310,12 +311,21 @@ int Segment::load(ConfigurationManager* configuration_manager) { } } _offset_and_term.push_back(std::make_pair(entry_off, header.term)); - if (_is_open) { - ++_last_index; - } + ++actual_last_index; entry_off += skip_len; } + if (ret == 0 && !_is_open && actual_last_index < _last_index) { + LOG(ERROR) << "data lost in a full segment, path: " << _path + << " first_index: " << _first_index << " expect_last_index: " + << _last_index << " actual_last_index: " << actual_last_index; + ret = -1; + } + + if (_is_open) { + _last_index = actual_last_index; + } + // truncate last uncompleted entry if (ret == 0 && entry_off != file_size) { LOG(INFO) << "truncate last uncompleted write entry, path: " << _path diff --git a/test/test_log.cpp b/test/test_log.cpp index 893aa7fb..3cc4aa1d 100644 --- a/test/test_log.cpp +++ b/test/test_log.cpp @@ -12,6 +12,7 @@ #include #include #include +#include #include #include #include "braft/log.h" @@ -482,6 +483,110 @@ TEST_F(LogStorageTest, append_close_load_append) { delete configuration_manager; } +ssize_t file_size(const char* filename) { + struct stat st; + stat(filename, &st); + return st.st_size; +} + +int truncate_uninterrupted(const char* filename, off_t length) { + int rc = 0; + do { + rc = truncate(filename, length); + } while (rc == -1 && errno == EINTR); + return rc; +} + +TEST_F(LogStorageTest, data_lost) { + ::system("rm -rf data"); + braft::LogStorage* storage = new braft::SegmentLogStorage("./data"); + braft::ConfigurationManager* configuration_manager = new braft::ConfigurationManager; + ASSERT_EQ(0, storage->init(configuration_manager)); + + // append entry + for (int i = 0; i < 100000; i++) { + std::vector entries; + for (int j = 0; j < 5; j++) { + int64_t index = 5*i + j + 1; + braft::LogEntry* entry = new braft::LogEntry(); + entry->type = braft::ENTRY_TYPE_DATA; + entry->id.term = 1; + entry->id.index = index; + + char data_buf[128]; + snprintf(data_buf, sizeof(data_buf), "hello, world: %ld", index); + entry->data.append(data_buf); + entries.push_back(entry); + } + + ASSERT_EQ(5, storage->append_entries(entries)); + + for (size_t j = 0; j < entries.size(); j++) { + delete entries[j]; + } + } + + delete storage; + delete configuration_manager; + + // reinit + storage = new braft::SegmentLogStorage("./data"); + configuration_manager = new braft::ConfigurationManager; + ASSERT_EQ(0, storage->init(configuration_manager)); + + ASSERT_EQ(storage->first_log_index(), 1); + ASSERT_EQ(storage->last_log_index(), 100000*5); + + delete storage; + delete configuration_manager; + + // last segment lost data + butil::DirReaderPosix dir_reader1("./data"); + ASSERT_TRUE(dir_reader1.IsValid()); + while (dir_reader1.Next()) { + int64_t first_index = 0; + int match = sscanf(dir_reader1.name(), "log_inprogress_%020ld", + &first_index); + std::string path; + butil::string_appendf(&path, "./data/%s", dir_reader1.name()); + if (match == 1) { + ASSERT_EQ(truncate_uninterrupted(path.c_str(), file_size(path.c_str()) - 1), 0); + } + } + + storage = new braft::SegmentLogStorage("./data"); + configuration_manager = new braft::ConfigurationManager; + ASSERT_EQ(0, storage->init(configuration_manager)); + + ASSERT_EQ(storage->first_log_index(), 1); + ASSERT_EQ(storage->last_log_index(), 100000*5 - 1); + + delete storage; + delete configuration_manager; + + // middle segment lost data + butil::DirReaderPosix dir_reader2("./data"); + ASSERT_TRUE(dir_reader2.IsValid()); + while (dir_reader2.Next()) { + int64_t first_index = 0; + int64_t last_index = 0; + int match = sscanf(dir_reader2.name(), "log_%020ld_%020ld", + &first_index, &last_index); + std::string path; + butil::string_appendf(&path, "./data/%s", dir_reader2.name()); + if (match == 2) { + ASSERT_EQ(truncate_uninterrupted(path.c_str(), file_size(path.c_str()) - 1), 0); + } + } + + storage = new braft::SegmentLogStorage("./data"); + configuration_manager = new braft::ConfigurationManager; + ASSERT_NE(0, storage->init(configuration_manager)); + + delete storage; + delete configuration_manager; +} + TEST_F(LogStorageTest, append_read_badcase) { ::system("rm -rf data"); braft::LogStorage* storage = new braft::SegmentLogStorage("./data"); From 0bffe8b3cfdf0dea021feb0c655f02d4b81f617b Mon Sep 17 00:00:00 2001 From: PFZheng Date: Tue, 23 Apr 2019 22:40:39 -0700 Subject: [PATCH 19/35] Website and get_status() interface only show pending index when pending queue is not empty. --- src/braft/ballot_box.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/braft/ballot_box.cpp b/src/braft/ballot_box.cpp index 45272718..cda64577 100644 --- a/src/braft/ballot_box.cpp +++ b/src/braft/ballot_box.cpp @@ -167,7 +167,7 @@ void BallotBox::describe(std::ostream& os, bool use_html) { lck.unlock(); const char *newline = use_html ? "
" : "\r\n"; os << "last_committed_index: " << committed_index << newline; - if (pending_index != 0) { + if (pending_queue_size != 0) { os << "pending_index: " << pending_index << newline; os << "pending_queue_size: " << pending_queue_size << newline; } @@ -179,7 +179,7 @@ void BallotBox::get_status(BallotBoxStatus* status) { } std::unique_lock lck(_mutex); status->committed_index = _last_committed_index; - if (_pending_index != 0) { + if (_pending_queue_size != 0) { status->pending_index = _pending_index; status->pending_queue_size = _pending_meta_queue.size(); } From 5dd342fa4877f80f2a56f146bc0e724da26143f1 Mon Sep 17 00:00:00 2001 From: PFZheng Date: Tue, 23 Apr 2019 22:43:50 -0700 Subject: [PATCH 20/35] Fixbug, log_manager get_term() should check if index < first_log_index considering a node without log changes leader when installing snapshot from an old leader, the new leader may use a stale next_index, and after the old snapshot is loaded, new leader will replicate logs whose index is smaller than last_included_index which will cause core. after this fix, new leader will fail to append_entries at the first time and get a latest next_index in the case above. --- src/braft/log_manager.cpp | 27 ++++++++++++++------------- 1 file changed, 14 insertions(+), 13 deletions(-) diff --git a/src/braft/log_manager.cpp b/src/braft/log_manager.cpp index 91a98c91..085108a3 100644 --- a/src/braft/log_manager.cpp +++ b/src/braft/log_manager.cpp @@ -675,19 +675,20 @@ int64_t LogManager::unsafe_get_term(const int64_t index) { if (index == 0) { return 0; } - - if (index > _last_log_index) { - return 0; - } - // check virtual first log if (index == _virtual_first_log_id.index) { return _virtual_first_log_id.term; } - + // check last_snapshot_id if (index == _last_snapshot_id.index) { return _last_snapshot_id.term; } + // out of range, direct return NULL + // check this after check last_snapshot_id, because it is likely that + // last_snapshot_id < first_log_index + if (index > _last_log_index || index < _first_log_index) { + return 0; + } LogEntry* entry = get_entry_from_memory(index); if (entry) { @@ -701,21 +702,21 @@ int64_t LogManager::get_term(const int64_t index) { if (index == 0) { return 0; } - std::unique_lock lck(_mutex); - // out of range, direct return NULL - if (index > _last_log_index) { - return 0; - } - // check virtual first log if (index == _virtual_first_log_id.index) { return _virtual_first_log_id.term; } - + // check last_snapshot_id if (index == _last_snapshot_id.index) { return _last_snapshot_id.term; } + // out of range, direct return NULL + // check this after check last_snapshot_id, because it is likely that + // last_snapshot_id < first_log_index + if (index > _last_log_index || index < _first_log_index) { + return 0; + } LogEntry* entry = get_entry_from_memory(index); if (entry) { From c50fa097b5e8d22c1f1aac2c959a27b81c9f1d74 Mon Sep 17 00:00:00 2001 From: PFZheng Date: Tue, 23 Apr 2019 22:46:09 -0700 Subject: [PATCH 21/35] Fixbug, _virtual_first_log_id should be reset in clear_bufferred_logs. --- src/braft/log_manager.cpp | 1 + src/braft/node.cpp | 2 ++ src/braft/snapshot_executor.cpp | 7 +++++++ 3 files changed, 10 insertions(+) diff --git a/src/braft/log_manager.cpp b/src/braft/log_manager.cpp index 085108a3..64618f82 100644 --- a/src/braft/log_manager.cpp +++ b/src/braft/log_manager.cpp @@ -654,6 +654,7 @@ void LogManager::set_snapshot(const SnapshotMeta* meta) { void LogManager::clear_bufferred_logs() { std::unique_lock lck(_mutex); if (_last_snapshot_id.index != 0) { + _virtual_first_log_id = _last_snapshot_id; truncate_prefix(_last_snapshot_id.index + 1, lck); } } diff --git a/src/braft/node.cpp b/src/braft/node.cpp index b84de035..3385e404 100644 --- a/src/braft/node.cpp +++ b/src/braft/node.cpp @@ -858,6 +858,8 @@ void NodeImpl::snapshot(Closure* done) { } void NodeImpl::do_snapshot(Closure* done) { + LOG(INFO) << "node " << _group_id << ":" << _server_id + << " starts to do snapshot"; if (_snapshot_executor) { _snapshot_executor->do_snapshot(done); } else { diff --git a/src/braft/snapshot_executor.cpp b/src/braft/snapshot_executor.cpp index 177d7229..b03e3590 100644 --- a/src/braft/snapshot_executor.cpp +++ b/src/braft/snapshot_executor.cpp @@ -108,6 +108,8 @@ SnapshotExecutor::~SnapshotExecutor() { void SnapshotExecutor::do_snapshot(Closure* done) { std::unique_lock lck(_mutex); + int64_t saved_last_snapshot_index = _last_snapshot_index; + int64_t saved_last_snapshot_term = _last_snapshot_term; if (_stopped) { lck.unlock(); if (done) { @@ -140,6 +142,11 @@ void SnapshotExecutor::do_snapshot(Closure* done) { // updated. But it's fine since we will do next snapshot saving in a // predictable time. lck.unlock(); + LOG_IF(INFO, _node != NULL) << "node " << _node->node_id() + << " has no applied logs since last snapshot, " + << " last_snapshot_index " << saved_last_snapshot_index + << " last_snapshot_term " << saved_last_snapshot_term + << ", will clear bufferred log and return success"; _log_manager->clear_bufferred_logs(); if (done) { run_closure_in_bthread(done, _usercode_in_pthread); From 72326108beaaec4839cae495f4fb562636db5ef2 Mon Sep 17 00:00:00 2001 From: PFZheng Date: Tue, 23 Apr 2019 23:32:28 -0700 Subject: [PATCH 22/35] Self-protective readonly mode. If a follower become readonly, the leader stop replicate new logs to it. This may cause the data for behind the leader, in the case that the leader is still writable. After the follower exit readonly mode, the leader will resume to replicate missing logs. A leader is readonly, if the node itself is readonly, or writable nodes (nodes that are not marked as readonly) in the group is less than majority. Once a leader become readonly, no new users logs will be acceptted. --- src/braft/ballot_box.cpp | 2 +- src/braft/errno.proto | 2 + src/braft/node.cpp | 88 ++++++++++++++++-- src/braft/node.h | 12 +++ src/braft/raft.cpp | 12 +++ src/braft/raft.h | 48 +++++++++- src/braft/raft.proto | 1 + src/braft/replicator.cpp | 121 +++++++++++++++++++++++-- src/braft/replicator.h | 15 +++ test/test_node.cpp | 191 +++++++++++++++++++++++++++++++++++++++ 10 files changed, 475 insertions(+), 17 deletions(-) diff --git a/src/braft/ballot_box.cpp b/src/braft/ballot_box.cpp index cda64577..2b8471af 100644 --- a/src/braft/ballot_box.cpp +++ b/src/braft/ballot_box.cpp @@ -179,7 +179,7 @@ void BallotBox::get_status(BallotBoxStatus* status) { } std::unique_lock lck(_mutex); status->committed_index = _last_committed_index; - if (_pending_queue_size != 0) { + if (_pending_meta_queue.size() != 0) { status->pending_index = _pending_index; status->pending_queue_size = _pending_meta_queue.size(); } diff --git a/src/braft/errno.proto b/src/braft/errno.proto index ae7f64a9..3fc909ec 100644 --- a/src/braft/errno.proto +++ b/src/braft/errno.proto @@ -30,5 +30,7 @@ enum RaftError { ELOGDELETED = 10014; // No available user log to read ENOMOREUSERLOG = 10015; + // Raft node in readonly mode + EREADONLY = 10016; }; diff --git a/src/braft/node.cpp b/src/braft/node.cpp index 3385e404..be6fa55f 100644 --- a/src/braft/node.cpp +++ b/src/braft/node.cpp @@ -117,7 +117,9 @@ NodeImpl::NodeImpl(const GroupId& group_id, const PeerId& peer_id) , _stop_transfer_arg(NULL) , _waking_candidate(0) , _append_entries_cache(NULL) - , _append_entries_cache_version(0) { + , _append_entries_cache_version(0) + , _node_readonly(false) + , _majority_nodes_readonly(false) { _server_id = peer_id; AddRef(); g_num_nodes << 1; @@ -168,9 +170,11 @@ NodeImpl::~NodeImpl() { _ballot_box = NULL; } - if (_log_storage) { - delete _log_storage; - _log_storage = NULL; + if (_options.node_owns_log_storage) { + if (_log_storage) { + delete _log_storage; + _log_storage = NULL; + } } if (_closure_queue) { delete _closure_queue; @@ -1524,6 +1528,7 @@ void NodeImpl::step_down(const int64_t term, bool wakeup_a_candidate, // _conf_ctx.reset() will stop replicators of catching up nodes _conf_ctx.reset(); _last_leader_timestamp = butil::monotonic_time_ms(); + _majority_nodes_readonly = false; clear_append_entries_cache(); @@ -1684,9 +1689,12 @@ void NodeImpl::apply(LogEntryAndClosure tasks[], size_t size) { std::vector entries; entries.reserve(size); std::unique_lock lck(_mutex); - if (_state != STATE_LEADER) { + bool reject_new_user_logs = (_node_readonly || _majority_nodes_readonly); + if (_state != STATE_LEADER || reject_new_user_logs) { butil::Status st; - if (_state != STATE_TRANSFERRING) { + if (_state == STATE_LEADER && reject_new_user_logs) { + st.set_error(EREADONLY, "readonly mode reject new user logs"); + } else if (_state != STATE_TRANSFERRING) { st.set_error(EPERM, "is not leader"); } else { st.set_error(EBUSY, "is transferring leadership"); @@ -2105,6 +2113,7 @@ void NodeImpl::handle_append_entries_request(brpc::Controller* cntl, response->set_success(true); response->set_term(_current_term); response->set_last_log_index(_log_manager->last_log_index()); + response->set_readonly(_node_readonly); lck.unlock(); // see the comments at FollowerStableClosure::run() _ballot_box->set_last_committed_index( @@ -2328,10 +2337,12 @@ void NodeImpl::describe(std::ostream& os, bool use_html) { // No replicator attached to nodes that are not leader; _replicator_group.list_replicators(&replicators); const int64_t leader_timestamp = _last_leader_timestamp; + const bool readonly = (_node_readonly || _majority_nodes_readonly); lck.unlock(); const char *newline = use_html ? "
" : "\r\n"; os << "peer_id: " << _server_id << newline; os << "state: " << state2str(st) << newline; + os << "readonly: " << readonly << newline; os << "term: " << term << newline; os << "conf_index: " << conf_index << newline; os << "peers:"; @@ -2434,6 +2445,7 @@ void NodeImpl::get_status(NodeStatus* status) { status->state = _state; status->term = _current_term; status->peer_id = _server_id; + status->readonly = (_node_readonly || _majority_nodes_readonly); _conf.conf.list_peers(&peers); _replicator_group.list_replicators(&replicators); lck.unlock(); @@ -2932,6 +2944,7 @@ void NodeImpl::ConfigurationCtx::reset(butil::Status* st) { ++_version; _stage = STAGE_NONE; _nchanges = 0; + _node->check_majority_nodes_readonly(); if (_done) { if (!st) { _done->status().set_error(EPERM, "leader stepped down"); @@ -2943,6 +2956,69 @@ void NodeImpl::ConfigurationCtx::reset(butil::Status* st) { } } +void NodeImpl::enter_readonly_mode() { + BAIDU_SCOPED_LOCK(_mutex); + if (!_node_readonly) { + LOG(INFO) << "node " << _group_id << ":" << _server_id + << " enter readonly mode"; + _node_readonly = true; + } +} + +void NodeImpl::leave_readonly_mode() { + BAIDU_SCOPED_LOCK(_mutex); + if (_node_readonly) { + LOG(INFO) << "node " << _group_id << ":" << _server_id + << " leave readonly mode"; + _node_readonly = false; + } +} + +bool NodeImpl::readonly() { + BAIDU_SCOPED_LOCK(_mutex); + return _node_readonly || _majority_nodes_readonly; +} + +int NodeImpl::change_readonly_config(int64_t term, const PeerId& peer_id, bool readonly) { + BAIDU_SCOPED_LOCK(_mutex); + if (term != _current_term && _state != STATE_LEADER) { + return EINVAL; + } + _replicator_group.change_readonly_config(peer_id, readonly); + check_majority_nodes_readonly(); + return 0; +} + +void NodeImpl::check_majority_nodes_readonly() { + check_majority_nodes_readonly(_conf.conf); + if (!_conf.old_conf.empty()) { + check_majority_nodes_readonly(_conf.old_conf); + } +} + +void NodeImpl::check_majority_nodes_readonly(const Configuration& conf) { + std::vector peers; + conf.list_peers(&peers); + size_t readonly_nodes = 0; + for (size_t i = 0; i < peers.size(); i++) { + if (peers[i] == _server_id) { + readonly_nodes += ((_node_readonly) ? 1: 0); + continue; + } + if (_replicator_group.readonly(peers[i])) { + ++readonly_nodes; + } + } + size_t writable_nodes = peers.size() - readonly_nodes; + bool prev_readonly = _majority_nodes_readonly; + _majority_nodes_readonly = !(writable_nodes >= (peers.size() / 2 + 1)); + if (prev_readonly != _majority_nodes_readonly) { + LOG(INFO) << "node " << _group_id << ":" << _server_id + << " majority readonly change from " << (prev_readonly ? "enable" : "disable") + << " to " << (_majority_nodes_readonly ? " enable" : "disable"); + } +} + // Timers int NodeTimer::init(NodeImpl* node, int timeout_ms) { BRAFT_RETURN_IF(RepeatedTimerTask::init(timeout_ms) != 0, -1); diff --git a/src/braft/node.h b/src/braft/node.h index a392410f..e2cd35e6 100644 --- a/src/braft/node.h +++ b/src/braft/node.h @@ -200,6 +200,14 @@ friend class ConfigurationChangeDone; // see from the website, which is generated by |describe| actually. void get_status(NodeStatus* status); + // Readonly mode func + void enter_readonly_mode(); + void leave_readonly_mode(); + bool readonly(); + int change_readonly_config(int64_t term, const PeerId& peer_id, bool readonly); + void check_majority_nodes_readonly(); + void check_majority_nodes_readonly(const Configuration& conf); + // Call on_error when some error happens, after this is called. // After this point: // - This node is to step down immediately if it was the leader. @@ -428,6 +436,10 @@ friend class butil::RefCountedThreadSafe; bthread::ExecutionQueue::scoped_ptr_t _apply_queue; AppendEntriesCache* _append_entries_cache; int64_t _append_entries_cache_version; + + // for readonly mode + bool _node_readonly; + bool _majority_nodes_readonly; }; } diff --git a/src/braft/raft.cpp b/src/braft/raft.cpp index dbb09398..c45e7573 100644 --- a/src/braft/raft.cpp +++ b/src/braft/raft.cpp @@ -168,6 +168,18 @@ void Node::get_status(NodeStatus* status) { return _impl->get_status(status); } +void Node::enter_readonly_mode() { + return _impl->enter_readonly_mode(); +} + +void Node::leave_readonly_mode() { + return _impl->leave_readonly_mode(); +} + +bool Node::readonly() { + return _impl->readonly(); +} + // ------------- Iterator void Iterator::next() { if (valid()) { diff --git a/src/braft/raft.h b/src/braft/raft.h index ea037ed2..857090f5 100644 --- a/src/braft/raft.h +++ b/src/braft/raft.h @@ -43,6 +43,7 @@ class SnapshotHook; class LeaderChangeContext; class FileSystemAdaptor; class SnapshotThrottle; +class LogStorage; const PeerId ANY_PEER(butil::EndPoint(butil::IP_ANY, 0), 0); @@ -358,7 +359,7 @@ struct PeerStatus { PeerStatus() : valid(false), installing_snapshot(false), next_index(0) , last_rpc_send_timestamp(0), flying_append_entries_size(0) - , consecutive_error_times(0) + , readonly_index(0), consecutive_error_times(0) {} bool valid; @@ -366,6 +367,7 @@ struct PeerStatus { int64_t next_index; int64_t last_rpc_send_timestamp; int64_t flying_append_entries_size; + int64_t readonly_index; int consecutive_error_times; }; @@ -376,7 +378,7 @@ friend class NodeImpl; typedef std::map PeerStatusMap; NodeStatus() - : state(STATE_END), term(0), committed_index(0), known_applied_index(0) + : state(STATE_END), readonly(false), term(0), committed_index(0), known_applied_index(0) , pending_index(0), pending_queue_size(0), applying_index(0), first_index(0) , last_index(-1), disk_index(0) {} @@ -384,6 +386,7 @@ friend class NodeImpl; State state; PeerId peer_id; PeerId leader_id; + bool readonly; int64_t term; int64_t committed_index; int64_t known_applied_index; @@ -467,6 +470,18 @@ struct NodeOptions { // Default: false bool node_owns_fsm; + // If |node_owns_log_storage| is true. |log_storage| would be destroyed when the backing + // Node is no longer referenced. + // + // Default: true + bool node_owns_log_storage; + + // The specific LogStorage implemented at the bussiness layer, which should be a valid + // instance, otherwise use SegmentLogStorage by default. + // + // Default: null + LogStorage* log_storage; + // Run the user callbacks and user closures in pthread rather than bthread // // Default: false @@ -509,6 +524,8 @@ inline NodeOptions::NodeOptions() , catchup_margin(1000) , fsm(NULL) , node_owns_fsm(false) + , node_owns_log_storage(true) + , log_storage(NULL) , usercode_in_pthread(false) , filter_before_copy_remote(false) , snapshot_file_system_adaptor(NULL) @@ -610,6 +627,33 @@ class Node { // see from the website. void get_status(NodeStatus* status); + // Make this node enter readonly mode. + // Readonly mode should only be used to protect the system in some extreme cases. + // For exampe, in a storage system, too many write requests flood into the system + // unexpectly, and the system is in the danger of exhaust capacity. There's not enough + // time to add new machines, and wait for capacity balance. Once many disks become + // full, quorum dead happen to raft groups. One choice in this example is readonly + // mode, to let leader reject new write requests, but still handle reads request, + // and configuration changes. + // If a follower become readonly, the leader stop replicate new logs to it. This + // may cause the data far behind the leader, in the case that the leader is still + // writable. After the follower exit readonly mode, the leader will resume to + // replicate missing logs. + // A leader is readonly, if the node itself is readonly, or writable nodes (nodes that + // are not marked as readonly) in the group is less than majority. Once a leader become + // readonly, no new users logs will be acceptted. + void enter_readonly_mode(); + + // Node leave readonly node. + void leave_readonly_mode(); + + // Check if this node is readonly. + // There are two situations that if a node is readonly: + // - This node is marked as readonly, by calling enter_readonly_mode(); + // - This node is a leader, and the count of writable nodes in the group + // is less than the majority. + bool readonly(); + private: NodeImpl* _impl; }; diff --git a/src/braft/raft.proto b/src/braft/raft.proto index 19294b9c..bc057082 100644 --- a/src/braft/raft.proto +++ b/src/braft/raft.proto @@ -45,6 +45,7 @@ message AppendEntriesResponse { required int64 term = 1; required bool success = 2; optional int64 last_log_index = 3; + optional bool readonly = 4; }; message SnapshotMeta { diff --git a/src/braft/replicator.cpp b/src/braft/replicator.cpp index 6a502e2c..d534abfc 100644 --- a/src/braft/replicator.cpp +++ b/src/braft/replicator.cpp @@ -70,6 +70,7 @@ Replicator::Replicator() , _heartbeat_counter(0) , _append_entries_counter(0) , _install_snapshot_counter(0) + , _readonly_index(0) , _wait_id(0) , _is_waiter_canceled(false) , _reader(NULL) @@ -291,7 +292,7 @@ void Replicator::_on_heartbeat_returned( << " prev_log_term " << request->prev_log_term(); if (cntl->Failed()) { ss << " fail, sleep."; - BRAFT_VLOG << ss.str(); + BRAFT_VLOG << ss.str(); // TODO: Should it be VLOG? LOG_IF(WARNING, (r->_consecutive_error_times++) % 10 == 0) @@ -306,8 +307,8 @@ void Replicator::_on_heartbeat_returned( r->_consecutive_error_times = 0; if (response->term() > r->_options.term) { ss << " fail, greater term " << response->term() - << " expect term " << r->_options.term; - BRAFT_VLOG << ss.str(); + << " expect term " << r->_options.term; + BRAFT_VLOG << ss.str(); NodeImpl *node_impl = r->_options.node; // Acquire a reference of Node here in case that Node is detroyed @@ -324,12 +325,28 @@ void Replicator::_on_heartbeat_returned( node_impl->Release(); return; } - BRAFT_VLOG << ss.str(); + bool readonly = response->has_readonly() && response->readonly(); + BRAFT_VLOG << ss.str() << " readonly " << readonly; if (rpc_send_time > r->_last_rpc_send_timestamp){ r->_last_rpc_send_timestamp = rpc_send_time; } r->_start_heartbeat_timer(start_time_us); + NodeImpl* node_impl = NULL; + // Check if readonly config changed + if ((readonly && r->_readonly_index == 0) || + (!readonly && r->_readonly_index != 0)) { + node_impl = r->_options.node; + node_impl->AddRef(); + } + if (!node_impl) { + CHECK_EQ(0, bthread_id_unlock(dummy_id)) << "Fail to unlock " << dummy_id; + return; + } + const PeerId peer_id = r->_options.peer_id; + int64_t term = r->_options.term; CHECK_EQ(0, bthread_id_unlock(dummy_id)) << "Fail to unlock " << dummy_id; + node_impl->change_readonly_config(term, peer_id, readonly); + node_impl->Release(); return; } @@ -564,12 +581,22 @@ void Replicator::_send_empty_entries(bool is_heartbeat) { int Replicator::_prepare_entry(int offset, EntryMeta* em, butil::IOBuf *data) { if (data->length() >= (size_t)FLAGS_raft_max_body_size) { - return -1; + return ERANGE; } - const size_t log_index = _next_index + offset; + const int64_t log_index = _next_index + offset; LogEntry *entry = _options.log_manager->get_entry(log_index); if (entry == NULL) { - return -1; + return ENOENT; + } + // When leader become readonly, no new user logs can submit. On the other side, + // if any user log are accepted after this replicator become readonly, the leader + // still have enough followers to commit logs, we can safely stop waiting new logs + // until the replicator leave readonly mode. + if (_readonly_index != 0 && log_index >= _readonly_index) { + if (entry->type != ENTRY_TYPE_CONFIGURATION) { + return EREADONLY; + } + _readonly_index = log_index + 1; } em->set_term(entry->id.term); em->set_type(entry->type); @@ -613,9 +640,11 @@ void Replicator::_send_entries() { } EntryMeta em; const int max_entries_size = FLAGS_raft_max_entries_size - _flying_append_entries_size; + int prepare_entry_rc = 0; CHECK_GT(max_entries_size, 0); for (int i = 0; i < max_entries_size; ++i) { - if (_prepare_entry(i, &em, &cntl->request_attachment()) != 0) { + prepare_entry_rc = _prepare_entry(i, &em, &cntl->request_attachment()); + if (prepare_entry_rc != 0) { break; } request->add_entries()->Swap(&em); @@ -626,6 +655,13 @@ void Replicator::_send_entries() { _reset_next_index(); return _install_snapshot(); } + if (prepare_entry_rc == EREADONLY) { + if (_flying_append_entries_size == 0) { + _st.st = IDLE; + } + CHECK_EQ(0, bthread_id_unlock(_id)) << "Fail to unlock " << _id; + return; + } return _wait_more_entries(); } @@ -1111,6 +1147,52 @@ int64_t Replicator::get_next_index(ReplicatorId id) { return next_index; } +int Replicator::change_readonly_config(ReplicatorId id, bool readonly) { + Replicator *r = NULL; + bthread_id_t dummy_id = { id }; + if (bthread_id_lock(dummy_id, (void**)&r) != 0) { + return 0; + } + return r->_change_readonly_config(readonly); +} + +int Replicator::_change_readonly_config(bool readonly) { + if ((readonly && _readonly_index != 0) || + (!readonly && _readonly_index == 0)) { + // Check if readonly already set + BRAFT_VLOG << "node " << _options.group_id << ":" << _options.server_id + << " ignore change readonly config of " << _options.peer_id + << " to " << readonly << ", readonly_index " << _readonly_index; + CHECK_EQ(0, bthread_id_unlock(_id)) << "Fail to unlock " << _id; + return 0; + } + if (readonly) { + // Keep a readonly index here to make sure the pending logs can be committed. + _readonly_index = _options.log_manager->last_log_index() + 1; + LOG(INFO) << "node " << _options.group_id << ":" << _options.server_id + << " enable readonly for " << _options.peer_id + << ", readonly_index " << _readonly_index; + CHECK_EQ(0, bthread_id_unlock(_id)) << "Fail to unlock " << _id; + } else { + _readonly_index = 0; + LOG(INFO) << "node " << _options.group_id << ":" << _options.server_id + << " disable readonly for " << _options.peer_id; + _wait_more_entries(); + } + return 0; +} + +bool Replicator::readonly(ReplicatorId id) { + Replicator *r = NULL; + bthread_id_t dummy_id = { id }; + if (bthread_id_lock(dummy_id, (void**)&r) != 0) { + return 0; + } + bool readonly = (r->_readonly_index != 0); + CHECK_EQ(0, bthread_id_unlock(dummy_id)) << "Fail to unlock " << dummy_id; + return readonly; +} + void Replicator::_destroy() { bthread_id_t saved_id = _id; CHECK_EQ(0, bthread_id_unlock_and_destroy(saved_id)); @@ -1129,12 +1211,16 @@ void Replicator::_describe(std::ostream& os, bool use_html) { const int64_t heartbeat_counter = _heartbeat_counter; const int64_t append_entries_counter = _append_entries_counter; const int64_t install_snapshot_counter = _install_snapshot_counter; + const int64_t readonly_index = _readonly_index; CHECK_EQ(0, bthread_id_unlock(_id)); // Don't touch *this ever after const char* new_line = use_html ? "
" : "\r\n"; os << "replicator_" << id << '@' << peer_id << ':'; os << " next_index=" << next_index << ' '; os << " flying_append_entries_size=" << flying_append_entries_size << ' '; + if (readonly_index != 0) { + os << " readonly_index=" << readonly_index << ' '; + } switch (st.st) { case IDLE: os << "idle"; @@ -1160,6 +1246,7 @@ void Replicator::_get_status(PeerStatus* status) { status->flying_append_entries_size = _flying_append_entries_size; status->last_rpc_send_timestamp = _last_rpc_send_timestamp; status->consecutive_error_times = _consecutive_error_times; + status->readonly_index = _readonly_index; CHECK_EQ(0, bthread_id_unlock(_id)); } @@ -1391,5 +1478,23 @@ void ReplicatorGroup::list_replicators( out->push_back(*iter); } } + +int ReplicatorGroup::change_readonly_config(const PeerId& peer, bool readonly) { + std::map::const_iterator iter = _rmap.find(peer); + if (iter == _rmap.end()) { + return -1; + } + ReplicatorId rid = iter->second; + return Replicator::change_readonly_config(rid, readonly); +} + +bool ReplicatorGroup::readonly(const PeerId& peer) const { + std::map::const_iterator iter = _rmap.find(peer); + if (iter == _rmap.end()) { + return false; + } + ReplicatorId rid = iter->second; + return Replicator::readonly(rid); +} } // namespace braft diff --git a/src/braft/replicator.h b/src/braft/replicator.h index c3cc6d10..15315698 100644 --- a/src/braft/replicator.h +++ b/src/braft/replicator.h @@ -109,6 +109,13 @@ class BAIDU_CACHELINE_ALIGNMENT Replicator { // Get replicator internal status. static void get_status(ReplicatorId id, PeerStatus* status); + + // Change the readonly config. + // Return 0 if success, the error code otherwise. + static int change_readonly_config(ReplicatorId id, bool readonly); + + // Check if a replicator is readonly + static bool readonly(ReplicatorId id); private: enum St { @@ -150,6 +157,7 @@ class BAIDU_CACHELINE_ALIGNMENT Replicator { int64_t _min_flying_index() { return _next_index - _flying_append_entries_size; } + int _change_readonly_config(bool readonly); static void _on_rpc_returned( ReplicatorId id, brpc::Controller* cntl, @@ -218,6 +226,7 @@ class BAIDU_CACHELINE_ALIGNMENT Replicator { int64_t _heartbeat_counter; int64_t _append_entries_counter; int64_t _install_snapshot_counter; + int64_t _readonly_index; Stat _st; std::deque _append_entries_in_fly; brpc::CallId _install_snapshot_in_fly; @@ -324,6 +333,12 @@ class ReplicatorGroup { // List all the existing replicators with PeerId void list_replicators(std::vector >* out) const; + // Change the readonly config for a peer + int change_readonly_config(const PeerId& peer, bool readonly); + + // Check if a replicator is in readonly + bool readonly(const PeerId& peer) const; + private: int _add_replicator(const PeerId& peer, ReplicatorId *rid); diff --git a/test/test_node.cpp b/test/test_node.cpp index ae1532df..cbcc010c 100644 --- a/test/test_node.cpp +++ b/test/test_node.cpp @@ -3460,6 +3460,197 @@ TEST_P(NodeTest, follower_handle_out_of_order_append_entries) { cluster.stop_all(); } +TEST_P(NodeTest, readonly) { + std::vector peers; + for (int i = 0; i < 3; i++) { + braft::PeerId peer; + peer.addr.ip = butil::my_ip(); + peer.addr.port = 5006 + i; + peer.idx = 0; + + peers.push_back(peer); + } + + // start cluster + Cluster cluster("unittest", peers); + for (size_t i = 0; i < peers.size(); i++) { + ASSERT_EQ(0, cluster.start(peers[i].addr)); + } + + // elect leader + cluster.wait_leader(); + braft::Node* leader = cluster.leader(); + ASSERT_TRUE(leader != NULL); + LOG(WARNING) << "leader is " << leader->node_id(); + + // apply something + bthread::CountdownEvent cond(10); + int start_index = 0; + for (int i = start_index; i < start_index + 10; i++) { + butil::IOBuf data; + char data_buf[128]; + snprintf(data_buf, sizeof(data_buf), "hello: %d", i); + data.append(data_buf); + + braft::Task task; + task.data = &data; + task.done = NEW_APPLYCLOSURE(&cond, 0); + leader->apply(task); + } + cond.wait(); + + // let leader enter readonly mode, reject user logs + leader->enter_readonly_mode(); + ASSERT_TRUE(leader->readonly()); + cond.reset(10); + start_index += 10; + for (int i = start_index; i < start_index + 10; i++) { + butil::IOBuf data; + char data_buf[128]; + snprintf(data_buf, sizeof(data_buf), "hello: %d", i); + data.append(data_buf); + + braft::Task task; + task.data = &data; + task.done = NEW_APPLYCLOSURE(&cond, braft::EREADONLY); + leader->apply(task); + } + cond.wait(); + + // let leader leave readonly mode, accept user logs + leader->leave_readonly_mode(); + ASSERT_FALSE(leader->readonly()); + cond.reset(10); + start_index += 10; + for (int i = start_index; i < start_index + 10; i++) { + butil::IOBuf data; + char data_buf[128]; + snprintf(data_buf, sizeof(data_buf), "hello: %d", i); + data.append(data_buf); + + braft::Task task; + task.data = &data; + task.done = NEW_APPLYCLOSURE(&cond, 0); + leader->apply(task); + } + cond.wait(); + + std::vector followers; + cluster.followers(&followers); + ASSERT_EQ(2, followers.size()); + + // Let follower 0 enter readonly mode, still can accept user logs + followers[0]->enter_readonly_mode(); + bthread_usleep(2000 * 1000); // wait a while for heartbeat + cond.reset(10); + start_index += 10; + for (int i = start_index; i < start_index + 10; i++) { + butil::IOBuf data; + char data_buf[128]; + snprintf(data_buf, sizeof(data_buf), "hello: %d", i); + data.append(data_buf); + + braft::Task task; + task.data = &data; + task.done = NEW_APPLYCLOSURE(&cond, 0); + leader->apply(task); + } + cond.wait(); + + // Let follower 1 enter readonly mode, majority readonly, reject user logs + followers[1]->enter_readonly_mode(); + int retry = 5; + while (!leader->readonly() && --retry >= 0) { + bthread_usleep(1000 * 1000); + } + ASSERT_TRUE(leader->readonly()); + cond.reset(10); + start_index += 10; + for (int i = start_index; i < start_index + 10; i++) { + butil::IOBuf data; + char data_buf[128]; + snprintf(data_buf, sizeof(data_buf), "hello: %d", i); + data.append(data_buf); + + braft::Task task; + task.data = &data; + task.done = NEW_APPLYCLOSURE(&cond, braft::EREADONLY); + leader->apply(task); + } + cond.wait(); + + // Add a new follower + braft::PeerId peer3; + peer3.addr.ip = butil::my_ip(); + peer3.addr.port = 5006 + 3; + peer3.idx = 0; + ASSERT_EQ(0, cluster.start(peer3.addr, true)); + bthread_usleep(1000* 1000); + cond.reset(1); + leader->add_peer(peer3, NEW_ADDPEERCLOSURE(&cond, 0)); + cond.wait(); + + // Trigger follower 0 do snapshot + cond.reset(1); + followers[0]->snapshot(NEW_SNAPSHOTCLOSURE(&cond, 0)); + cond.wait(); + + // 2/4 readonly, leader still in readonly + retry = 5; + while (!leader->readonly() && --retry >= 0) { + bthread_usleep(1000 * 1000); + } + ASSERT_TRUE(leader->readonly()); + start_index += 10; + cond.reset(10); + for (int i = start_index; i < start_index + 10; i++) { + butil::IOBuf data; + char data_buf[128]; + snprintf(data_buf, sizeof(data_buf), "hello: %d", i); + data.append(data_buf); + + braft::Task task; + task.data = &data; + task.done = NEW_APPLYCLOSURE(&cond, braft::EREADONLY); + leader->apply(task); + } + cond.wait(); + + // Remove follower 0 + cond.reset(1); + leader->remove_peer(followers[0]->node_id().peer_id, NEW_REMOVEPEERCLOSURE(&cond, 0)); + cond.wait(); + cluster.stop(followers[0]->node_id().peer_id.addr); + + // 1/3 readonly, leader leave Readonly + retry = 5; + while (leader->readonly() && --retry >= 0) { + bthread_usleep(1000 * 1000); + } + ASSERT_TRUE(!leader->readonly()); + cond.reset(10); + start_index += 10; + for (int i = start_index; i < start_index + 10; i++) { + butil::IOBuf data; + char data_buf[128]; + snprintf(data_buf, sizeof(data_buf), "hello: %d", i); + data.append(data_buf); + + braft::Task task; + task.data = &data; + task.done = NEW_APPLYCLOSURE(&cond, 0); + leader->apply(task); + } + cond.wait(); + + // Follower 1 leave readonly, catch up logs + followers[1]->leave_readonly_mode(); + cluster.ensure_same(); + + LOG(WARNING) << "cluster stop"; + cluster.stop_all(); +} + INSTANTIATE_TEST_CASE_P(NodeTestWithoutPipelineReplication, NodeTest, ::testing::Values("NoReplcation")); From 3d2e65c48da9e41f844ac5a49761f7209de67595 Mon Sep 17 00:00:00 2001 From: PFZheng Date: Tue, 23 Apr 2019 23:41:21 -0700 Subject: [PATCH 23/35] Fix bug, should delete incomplete files when init snapshot writer. A incomplete file means it exist in temp/ but not exist in meta_table, this could happen when a raft node crashed during do_snapshot. --- src/braft/log.cpp | 8 ++++---- src/braft/snapshot.cpp | 29 +++++++++++++++++++++++++++++ test/test_snapshot.cpp | 24 +++++++++++++++++------- 3 files changed, 50 insertions(+), 11 deletions(-) diff --git a/src/braft/log.cpp b/src/braft/log.cpp index cf413072..09921e5f 100644 --- a/src/braft/log.cpp +++ b/src/braft/log.cpp @@ -1069,8 +1069,8 @@ int SegmentLogStorage::save_meta(const int64_t log_index) { timer.stop(); PLOG_IF(ERROR, ret != 0) << "Fail to save meta to " << meta_path; - BRAFT_VLOG << "log save_meta " << meta_path << " log_index: " << log_index - << " time: " << timer.u_elapsed(); + LOG(INFO) << "log save_meta " << meta_path << " first_log_index: " << log_index + << " time: " << timer.u_elapsed(); return ret; } @@ -1091,8 +1091,8 @@ int SegmentLogStorage::load_meta() { _first_log_index.store(meta.first_log_index()); timer.stop(); - BRAFT_VLOG << "log load_meta " << meta_path << " log_index: " << meta.first_log_index() - << " time: " << timer.u_elapsed(); + LOG(INFO) << "log load_meta " << meta_path << " first_log_index: " << meta.first_log_index() + << " time: " << timer.u_elapsed(); return 0; } diff --git a/src/braft/snapshot.cpp b/src/braft/snapshot.cpp index b19c7e6e..a4e6c7a6 100644 --- a/src/braft/snapshot.cpp +++ b/src/braft/snapshot.cpp @@ -194,6 +194,34 @@ int LocalSnapshotWriter::init() { set_error(EIO, "Fail to load metatable from %s", meta_path.c_str()); return EIO; } + + // remove file if it's not in _meta_table to avoid dirty data + if (_fs->path_exists(meta_path)) { + std::vector to_remove; + DirReader* dir_reader = _fs->directory_reader(_path); + if (!dir_reader->is_valid()) { + LOG(WARNING) << "directory reader failed, maybe NOEXIST or PERMISSION," + " path: " << _path; + delete dir_reader; + return EIO; + } + while (dir_reader->next()) { + std::string filename = dir_reader->name(); + if (filename != BRAFT_SNAPSHOT_META_FILE) { + if (get_file_meta(filename, NULL) != 0) { + to_remove.push_back(filename); + } + } + } + delete dir_reader; + for (size_t i = 0; i < to_remove.size(); ++i) { + std::string file_path = _path + "/" + to_remove[i]; + _fs->delete_file(file_path, false); + LOG(WARNING) << "Snapshot file exist but meta not found so delete it," + " path: " << file_path; + } + } + return 0; } @@ -535,6 +563,7 @@ SnapshotWriter* LocalSnapshotStorage::create(bool from_empty) { writer = NULL; break; } + BRAFT_VLOG << "Create writer success, path: " << snapshot_path; } while (0); return writer; diff --git a/test/test_snapshot.cpp b/test/test_snapshot.cpp index 98d050a7..5224640e 100644 --- a/test/test_snapshot.cpp +++ b/test/test_snapshot.cpp @@ -405,10 +405,17 @@ void add_file_meta(braft::FileSystemAdaptor* fs, braft::SnapshotWriter* writer, if (checksum) { file_meta.set_checksum(*checksum); } - write_file(fs, writer->get_path() + "/" + path.str(), path.str() + data); + write_file(fs, writer->get_path() + "/" + path.str(), path.str() + ": " + data); ASSERT_EQ(0, writer->add_file(path.str(), &file_meta)); } +void add_file_without_meta(braft::FileSystemAdaptor* fs, braft::SnapshotWriter* writer, int index, + const std::string& data) { + std::stringstream path; + path << "file" << index; + write_file(fs, writer->get_path() + "/" + path.str(), path.str() + ": " + data); +} + bool check_file_exist(braft::FileSystemAdaptor* fs, const std::string& path, int index) { if (fs == NULL) { fs = braft::default_file_system(); @@ -522,6 +529,8 @@ TEST_F(SnapshotTest, filter_before_copy) { add_file_meta(fs, writer2, 4, &checksum2, data2); // file not exist in remote, will delete add_file_meta(fs, writer2, 100, &checksum2, data2); + // file exit but meta not exit, will delete + add_file_without_meta(fs, writer2, 102, data2); ASSERT_EQ(0, writer2->save_meta(meta)); ASSERT_EQ(0, storage2->close(writer2)); @@ -537,15 +546,15 @@ TEST_F(SnapshotTest, filter_before_copy) { meta.set_last_included_index(901); const std::string data3("ccc"); const std::string checksum3("3"); - // same checksum, will not copy + // same checksum, will copy from last_snapshot with index=901 add_file_meta(fs, writer2, 6, &checksum1, data3); - // remote checksum not set, local set, will copy + // remote checksum not set, local last_snapshot set, will copy add_file_meta(fs, writer2, 7, &checksum1, data3); - // remote checksum set, local not set, will copy + // remote checksum set, local last_snapshot not set, will copy add_file_meta(fs, writer2, 8, NULL, data3); - // different checksum, will copy + // remote and local last_snapshot different checksum, will copy add_file_meta(fs, writer2, 9, &checksum3, data3); - // file not exist in remote, will delete + // file not exist in remote, will not copy add_file_meta(fs, writer2, 101, &checksum3, data3); ASSERT_EQ(0, writer2->save_meta(meta)); ASSERT_EQ(0, storage2->close(writer2)); @@ -566,7 +575,7 @@ TEST_F(SnapshotTest, filter_before_copy) { for (int i = 1; i <= 9; ++i) { ASSERT_TRUE(check_file_exist(fs, snapshot_path, i)); std::stringstream content; - content << "file" << i; + content << "file" << i << ": "; if (i == 1) { content << data2; } else if (i == 6) { @@ -578,6 +587,7 @@ TEST_F(SnapshotTest, filter_before_copy) { } ASSERT_TRUE(!check_file_exist(fs, snapshot_path, 100)); ASSERT_TRUE(!check_file_exist(fs, snapshot_path, 101)); + ASSERT_TRUE(!check_file_exist(fs, snapshot_path, 102)); delete storage2; delete storage1; From 8afd3e60c646d2694cdd3849d3f0df6d758711f2 Mon Sep 17 00:00:00 2001 From: PFZheng Date: Wed, 24 Apr 2019 00:33:33 -0700 Subject: [PATCH 24/35] Fix compatibility problem, fail to get uri when install_snapshot should not make raft Node ERROE immediately, because FileSystemAdaptor layer of reader is user defined and may have some control logic. --- src/braft/remote_file_copier.cpp | 1 + src/braft/replicator.cpp | 20 +++++++++++++------- 2 files changed, 14 insertions(+), 7 deletions(-) diff --git a/src/braft/remote_file_copier.cpp b/src/braft/remote_file_copier.cpp index d6338726..3ab2e5d0 100644 --- a/src/braft/remote_file_copier.cpp +++ b/src/braft/remote_file_copier.cpp @@ -45,6 +45,7 @@ BRPC_VALIDATE_GFLAG(raft_enable_throttle_when_install_snapshot, RemoteFileCopier::RemoteFileCopier() : _reader_id(0) , _throttle(NULL) + , _throttle_token_acquire_time_us(0) {} int RemoteFileCopier::init(const std::string& uri, FileSystemAdaptor* fs, diff --git a/src/braft/replicator.cpp b/src/braft/replicator.cpp index d534abfc..32c01ae4 100644 --- a/src/braft/replicator.cpp +++ b/src/braft/replicator.cpp @@ -740,7 +740,8 @@ void Replicator::_wait_more_entries() { void Replicator::_install_snapshot() { CHECK(!_reader); - if (_options.snapshot_throttle && !_options.snapshot_throttle->add_one_more_task(true)) { + if (_options.snapshot_throttle && !_options.snapshot_throttle-> + add_one_more_task(true)) { return _block(butil::gettimeofday_us(), EBUSY); } @@ -760,20 +761,25 @@ void Replicator::_install_snapshot() { return; } std::string uri = _reader->generate_uri_for_copy(); + // NOTICE: If uri is something wrong, retry later instead of reporting error + // immediately(making raft Node error), as FileSystemAdaptor layer of _reader is + // user defined and may need some control logic when opened + if (uri.empty()) { + LOG(WARNING) << "node " << _options.group_id << ":" << _options.server_id + << " refuse to send InstallSnapshotRequest to " << _options.peer_id + << " because snapshot uri is empty"; + return _block(butil::gettimeofday_us(), EBUSY); + } SnapshotMeta meta; // report error on failure - if (uri.empty() || _reader->load_meta(&meta) != 0){ + if (_reader->load_meta(&meta) != 0){ std::string snapshot_path = _reader->get_path(); NodeImpl *node_impl = _options.node; node_impl->AddRef(); CHECK_EQ(0, bthread_id_unlock(_id)) << "Fail to unlock " << _id; braft::Error e; e.set_type(ERROR_TYPE_SNAPSHOT); - if (uri.empty()) { - e.status().set_error(EIO, "Fail to generate uri from " + snapshot_path); - } else { - e.status().set_error(EIO, "Fail to load meta from " + snapshot_path); - } + e.status().set_error(EIO, "Fail to load meta from " + snapshot_path); node_impl->on_error(e); node_impl->Release(); return; From 643f0d7d0e38dd0aecdfa7fb52d157b80ce78801 Mon Sep 17 00:00:00 2001 From: PFZheng Date: Wed, 24 Apr 2019 00:41:56 -0700 Subject: [PATCH 25/35] Fix bug, in replicator block timedout should check wait_id, start_send_empty_entries only when wait_id = 0, which means there is no another replication pipeline bad case happends when 1. pipeline is enabled and 2. disable readonly mode triggers a new replication --- src/braft/remote_file_copier.cpp | 1 - src/braft/replicator.cpp | 10 ++- src/braft/snapshot.cpp | 13 ++-- src/braft/snapshot_executor.cpp | 2 +- test/test_node.cpp | 106 +++++++++++++++++++++++++++++++ 5 files changed, 124 insertions(+), 8 deletions(-) diff --git a/src/braft/remote_file_copier.cpp b/src/braft/remote_file_copier.cpp index 3ab2e5d0..d6338726 100644 --- a/src/braft/remote_file_copier.cpp +++ b/src/braft/remote_file_copier.cpp @@ -45,7 +45,6 @@ BRPC_VALIDATE_GFLAG(raft_enable_throttle_when_install_snapshot, RemoteFileCopier::RemoteFileCopier() : _reader_id(0) , _throttle(NULL) - , _throttle_token_acquire_time_us(0) {} int RemoteFileCopier::init(const std::string& uri, FileSystemAdaptor* fs, diff --git a/src/braft/replicator.cpp b/src/braft/replicator.cpp index 32c01ae4..fe74eaa5 100644 --- a/src/braft/replicator.cpp +++ b/src/braft/replicator.cpp @@ -696,11 +696,17 @@ int Replicator::_continue_sending(void* arg, int error_code) { return -1; } if (error_code == ETIMEDOUT) { + // Replication is in progress when block timedout, no need to start again + // this case can happen when + // 1. pipeline is enabled and + // 2. disable readonly mode triggers another replication + if (r->_wait_id != 0) { + return 0; + } // Send empty entries after block timeout to check the correct // _next_index otherwise the replictor is likely waits in // _wait_more_entries and no further logs would be replicated even if the // last_index of this followers is less than |next_index - 1| - CHECK_EQ(r->_wait_id, 0); r->_send_empty_entries(false); } else if (error_code != ESTOP && !r->_is_waiter_canceled) { // id is unlock in _send_entries @@ -729,7 +735,7 @@ void Replicator::_wait_more_entries() { _next_index - 1, _continue_sending, (void*)_id.value); _is_waiter_canceled = false; BRAFT_VLOG << "node " << _options.group_id << ":" << _options.peer_id - << " wait more entries"; + << " wait more entries, wait_id " << _wait_id; } if (_flying_append_entries_size == 0) { _st.st = IDLE; diff --git a/src/braft/snapshot.cpp b/src/braft/snapshot.cpp index a4e6c7a6..fff10381 100644 --- a/src/braft/snapshot.cpp +++ b/src/braft/snapshot.cpp @@ -759,10 +759,15 @@ void LocalSnapshotCopier::copy() { } } while (0); if (!ok() && _writer && _writer->ok()) { - _writer->set_error(error_code(), error_data()); + LOG(WARNING) << "Fail to copy, error_code " << error_code() + << " error_msg " << error_cstr() + << " writer path " << _writer->get_path(); + _writer->set_error(error_code(), error_cstr()); } if (_writer) { - if (_storage->close(_writer, _filter_before_copy_remote) != 0) { + // set_error for copier only when failed to close writer and copier was + // ok before this moment + if (_storage->close(_writer, _filter_before_copy_remote) != 0 && ok()) { set_error(EIO, "Fail to close writer"); } _writer = NULL; @@ -790,7 +795,7 @@ void LocalSnapshotCopier::load_meta_table() { lck.unlock(); if (!session->status().ok()) { LOG(WARNING) << "Fail to copy meta file : " << session->status(); - set_error(session->status().error_code(), session->status().error_data()); + set_error(session->status().error_code(), session->status().error_cstr()); return; } if (_remote_snapshot._meta_table.load_from_iobuf_as_remote(meta_buf) != 0) { @@ -969,7 +974,7 @@ void LocalSnapshotCopier::copy_file(const std::string& filename) { _cur_session = NULL; lck.unlock(); if (!session->status().ok()) { - set_error(session->status().error_code(), session->status().error_data()); + set_error(session->status().error_code(), session->status().error_cstr()); return; } if (_writer->add_file(filename, &meta) != 0) { diff --git a/src/braft/snapshot_executor.cpp b/src/braft/snapshot_executor.cpp index b03e3590..59553daf 100644 --- a/src/braft/snapshot_executor.cpp +++ b/src/braft/snapshot_executor.cpp @@ -606,7 +606,7 @@ void SnapshotExecutor::report_error(int error_code, const char* fmt, ...) { va_start(ap, fmt); Error e; e.set_type(ERROR_TYPE_SNAPSHOT); - e.status().set_error(error_code, fmt, ap); + e.status().set_errorv(error_code, fmt, ap); va_end(ap); _fsm_caller->on_error(e); } diff --git a/test/test_node.cpp b/test/test_node.cpp index cbcc010c..5901c023 100644 --- a/test/test_node.cpp +++ b/test/test_node.cpp @@ -21,6 +21,7 @@ #include "braft/enum.pb.h" #include "braft/errno.pb.h" #include +#include namespace braft { extern bvar::Adder g_num_nodes; @@ -1088,6 +1089,111 @@ TEST_F(NodeTest, Leader_step_down_during_install_snapshot) { cluster.stop_all(); } + +TEST_P(NodeTest, Report_error_during_install_snapshot) { + std::vector peers; + for (int i = 0; i < 3; i++) { + braft::PeerId peer; + peer.addr.ip = butil::my_ip(); + peer.addr.port = 5006 + i; + peer.idx = 0; + + peers.push_back(peer); + } + + // start cluster + Cluster cluster("unittest", peers); + for (size_t i = 0; i < peers.size(); i++) { + ASSERT_EQ(0, cluster.start(peers[i].addr)); + } + + // elect leader + cluster.wait_leader(); + braft::Node* leader = cluster.leader(); + ASSERT_TRUE(leader != NULL); + LOG(WARNING) << "leader is " << leader->node_id(); + + // apply something + bthread::CountdownEvent cond(10); + for (int i = 0; i < 10; i++) { + butil::IOBuf data; + std::string data_buf; + data_buf.resize(256 * 1024, 'a'); + data.append(data_buf); + + braft::Task task; + task.data = &data; + task.done = NEW_APPLYCLOSURE(&cond, 0); + leader->apply(task); + } + cond.wait(); + + cluster.ensure_same(); + + std::vector nodes; + cluster.followers(&nodes); + ASSERT_EQ(2, nodes.size()); + + // stop follower + LOG(WARNING) << "stop follower"; + butil::EndPoint follower_addr = nodes[0]->node_id().peer_id.addr; + cluster.stop(follower_addr); + + // apply something + cond.reset(10); + for (int i = 10; i < 20; i++) { + butil::IOBuf data; + std::string data_buf; + data_buf.resize(256 * 1024, 'b'); + data.append(data_buf); + + braft::Task task; + task.data = &data; + task.done = NEW_APPLYCLOSURE(&cond, 0); + leader->apply(task); + } + cond.wait(); + + // trigger leader snapshot + LOG(WARNING) << "trigger leader snapshot "; + cond.reset(1); + leader->snapshot(NEW_SNAPSHOTCLOSURE(&cond, 0)); + cond.wait(); + + // apply something + cond.reset(10); + for (int i = 20; i < 30; i++) { + butil::IOBuf data; + std::string data_buf; + data_buf.resize(256 * 1024, 'c'); + data.append(data_buf); + + braft::Task task; + task.data = &data; + task.done = NEW_APPLYCLOSURE(&cond, 0); + leader->apply(task); + } + cond.wait(); + + // trigger leader snapshot again to compact logs + LOG(WARNING) << "trigger leader snapshot again"; + cond.reset(1); + leader->snapshot(NEW_SNAPSHOTCLOSURE(&cond, 0)); + cond.wait(); + + LOG(WARNING) << "restart follower"; + ASSERT_EQ(0, cluster.start(follower_addr)); + usleep(1*1000*1000); + + // trigger newly-started follower report_error when install_snapshot + cluster._nodes.back()->_impl->_snapshot_executor->report_error(EIO, "%s", + "Fail to close writer"); + + sleep(2); + LOG(WARNING) << "cluster stop"; + cluster.stop_all(); +} + TEST_P(NodeTest, RemoveFollower) { std::vector peers; for (int i = 0; i < 3; i++) { From 902cc435befe758f0ffa0c81da8c976cdeb231e7 Mon Sep 17 00:00:00 2001 From: PFZheng Date: Wed, 24 Apr 2019 04:47:08 -0700 Subject: [PATCH 26/35] Fix bug, pre-set replicator state to INSTALLING_SNAPSHOT in Replicator::_install_snapshot so it could be blocked again if something is wrong. A known case: send_empty_entries trigger install_snapshot, and install_snapshot is throttled for a period of time. --- src/braft/replicator.cpp | 17 ++++++++++++----- 1 file changed, 12 insertions(+), 5 deletions(-) diff --git a/src/braft/replicator.cpp b/src/braft/replicator.cpp index fe74eaa5..9d3b8cab 100644 --- a/src/braft/replicator.cpp +++ b/src/braft/replicator.cpp @@ -235,15 +235,18 @@ void Replicator::_on_block_timedout(void *arg) { } void Replicator::_block(long start_time_us, int error_code) { + // mainly for pipeline case, to avoid too many block timer when this + // replicator is something wrong + if (_st.st == BLOCKING) { + CHECK_EQ(0, bthread_id_unlock(_id)) << "Fail to unlock " << _id; + return; + } + // TODO: Currently we don't care about error_code which indicates why the // very RPC fails. To make it better there should be different timeout for // each individual error (e.g. we don't need check every // heartbeat_timeout_ms whether a dead follower has come back), but it's just // fine now. - if (_st.st == BLOCKING) { - CHECK_EQ(0, bthread_id_unlock(_id)) << "Fail to unlock " << _id; - return; - } int blocking_time = 0; if (error_code == EBUSY || error_code == EINTR) { blocking_time = FLAGS_raft_retry_replicate_interval_ms; @@ -703,6 +706,7 @@ int Replicator::_continue_sending(void* arg, int error_code) { if (r->_wait_id != 0) { return 0; } + // Send empty entries after block timeout to check the correct // _next_index otherwise the replictor is likely waits in // _wait_more_entries and no further logs would be replicated even if the @@ -750,6 +754,10 @@ void Replicator::_install_snapshot() { add_one_more_task(true)) { return _block(butil::gettimeofday_us(), EBUSY); } + + // pre-set replictor state to INSTALLING_SNAPSHOT, so replicator could be + // blocked if something is wrong, such as throttled for a period of time + _st.st = INSTALLING_SNAPSHOT; _reader = _options.snapshot_storage->open(); if (!_reader){ @@ -809,7 +817,6 @@ void Replicator::_install_snapshot() { _install_snapshot_in_fly = cntl->call_id(); _install_snapshot_counter++; - _st.st = INSTALLING_SNAPSHOT; _st.last_log_included = meta.last_included_index(); _st.last_term_included = meta.last_included_term(); google::protobuf::Closure* done = brpc::NewCallback< From b24858cc3c9805bd36f931f9ece4cb550519b01d Mon Sep 17 00:00:00 2001 From: PFZheng Date: Wed, 24 Apr 2019 05:39:10 -0700 Subject: [PATCH 27/35] Fix bug, follower's readonly mode changing from enabled to disabled may cause two install_snapshot at the same time, which is not allowed --- src/braft/replicator.cpp | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/src/braft/replicator.cpp b/src/braft/replicator.cpp index 9d3b8cab..75c77ff3 100644 --- a/src/braft/replicator.cpp +++ b/src/braft/replicator.cpp @@ -658,6 +658,9 @@ void Replicator::_send_entries() { _reset_next_index(); return _install_snapshot(); } + // NOTICE: a follower's readonly mode does not prevent install_snapshot + // as we need followers to commit conf log(like add_node) when + // leader reaches readonly as well if (prepare_entry_rc == EREADONLY) { if (_flying_append_entries_size == 0) { _st.st = IDLE; @@ -748,7 +751,16 @@ void Replicator::_wait_more_entries() { } void Replicator::_install_snapshot() { - CHECK(!_reader); + if (_reader) { + // follower's readonly mode change may cause two install_snapshot + // one possible case is: + // enable -> install_snapshot -> disable -> wait_more_entries -> + // install_snapshot again + LOG(WARNING) << "node " << _options.group_id << ":" << _options.server_id + << " refuse to send InstallSnapshotRequest to " << _options.peer_id + << " because there is an running one"; + return; + } if (_options.snapshot_throttle && !_options.snapshot_throttle-> add_one_more_task(true)) { From 53e929940988fafc2fa9d06a4216b002b6704a98 Mon Sep 17 00:00:00 2001 From: PFZheng Date: Wed, 24 Apr 2019 05:47:15 -0700 Subject: [PATCH 28/35] Fix bug, init writer should remove tmp files when meta_path not exist at all. --- src/braft/snapshot.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/braft/snapshot.cpp b/src/braft/snapshot.cpp index fff10381..e8a15035 100644 --- a/src/braft/snapshot.cpp +++ b/src/braft/snapshot.cpp @@ -195,8 +195,9 @@ int LocalSnapshotWriter::init() { return EIO; } - // remove file if it's not in _meta_table to avoid dirty data - if (_fs->path_exists(meta_path)) { + // remove file if meta_path not exist or it's not in _meta_table + // to avoid dirty data + { std::vector to_remove; DirReader* dir_reader = _fs->directory_reader(_path); if (!dir_reader->is_valid()) { From ed605b3224a6e8904215acfe2d3f580b865bae30 Mon Sep 17 00:00:00 2001 From: PFZheng Date: Wed, 24 Apr 2019 05:54:45 -0700 Subject: [PATCH 29/35] Fix compatibility problem when install snapshot. Do not reset request.count with reponse.size if FLAGS_raft_allow_read_partly_when_install_snapshot is false. --- src/braft/remote_file_copier.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/braft/remote_file_copier.cpp b/src/braft/remote_file_copier.cpp index d6338726..8ebeb26a 100644 --- a/src/braft/remote_file_copier.cpp +++ b/src/braft/remote_file_copier.cpp @@ -198,7 +198,7 @@ RemoteFileCopier::Session::~Session() { void RemoteFileCopier::Session::send_next_rpc() { _cntl.Reset(); _response.Clear(); - // Not clear request as we need some fields of the previouse RPC + // Not clear request as we need some fields of the previous RPC off_t offset = _request.offset() + _request.count(); const size_t max_count = (!_buf) ? FLAGS_raft_max_byte_count_per_rpc : UINT_MAX; @@ -295,7 +295,8 @@ void RemoteFileCopier::Session::on_rpc_returned() { } _retry_times = 0; // Reset count to |real_read_size| to make next rpc get the right offset - if (_response.has_read_size() && (_response.read_size() != 0)) { + if (_response.has_read_size() && (_response.read_size() != 0) + && FLAGS_raft_allow_read_partly_when_install_snapshot) { _request.set_count(_response.read_size()); } if (_file) { From 43f9dcd853bcf6d02b8b2dcedcc64edceab45127 Mon Sep 17 00:00:00 2001 From: PFZheng Date: Wed, 24 Apr 2019 05:59:49 -0700 Subject: [PATCH 30/35] Fix a deadlock problem: Replicator::_install_snapshot forget to unlock bthread id, after detecting duplicate snapshot readers. --- src/braft/replicator.cpp | 20 +++++++++++++------- src/braft/replicator.h | 1 + 2 files changed, 14 insertions(+), 7 deletions(-) diff --git a/src/braft/replicator.cpp b/src/braft/replicator.cpp index 75c77ff3..d28da48b 100644 --- a/src/braft/replicator.cpp +++ b/src/braft/replicator.cpp @@ -85,13 +85,7 @@ Replicator::Replicator() Replicator::~Replicator() { // bind lifecycle with node, Release // Replicator stop is async - if (_reader) { - _options.snapshot_storage->close(_reader); - _reader = NULL; - if (_options.snapshot_throttle) { - _options.snapshot_throttle->finish_one_task(true); - } - } + _close_reader(); if (_options.node) { _options.node->Release(); _options.node = NULL; @@ -759,6 +753,7 @@ void Replicator::_install_snapshot() { LOG(WARNING) << "node " << _options.group_id << ":" << _options.server_id << " refuse to send InstallSnapshotRequest to " << _options.peer_id << " because there is an running one"; + CHECK_EQ(0, bthread_id_unlock(_id)) << "Fail to unlock " << _id; return; } @@ -794,6 +789,7 @@ void Replicator::_install_snapshot() { LOG(WARNING) << "node " << _options.group_id << ":" << _options.server_id << " refuse to send InstallSnapshotRequest to " << _options.peer_id << " because snapshot uri is empty"; + _close_reader(); return _block(butil::gettimeofday_us(), EBUSY); } SnapshotMeta meta; @@ -1303,6 +1299,16 @@ void Replicator::get_status(ReplicatorId id, PeerStatus* status) { return r->_get_status(status); } +void Replicator::_close_reader() { + if (_reader) { + _options.snapshot_storage->close(_reader); + _reader = NULL; + if (_options.snapshot_throttle) { + _options.snapshot_throttle->finish_one_task(true); + } + } +} + // ==================== ReplicatorGroup ========================== ReplicatorGroupOptions::ReplicatorGroupOptions() diff --git a/src/braft/replicator.h b/src/braft/replicator.h index 15315698..9bdf380e 100644 --- a/src/braft/replicator.h +++ b/src/braft/replicator.h @@ -205,6 +205,7 @@ class BAIDU_CACHELINE_ALIGNMENT Replicator { } return true; } + void _close_reader(); private: struct FlyingAppendEntriesRpc { From e01b56db57ebe14ff8e20c2e278b34fbf085976a Mon Sep 17 00:00:00 2001 From: PFZheng Date: Wed, 24 Apr 2019 06:03:39 -0700 Subject: [PATCH 31/35] Randomlize the first snapshot trigger timeout, to disperse the snapshot pressure. A multi-raft node usually load all raft nodes immediately after start. Since the nodes have the same snapshot interval, they will start to do snapshot almost at the same time. Those systems with heavy snapshot implementation, suffer huge pressure. --- src/braft/node.cpp | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/src/braft/node.cpp b/src/braft/node.cpp index be6fa55f..724506e6 100644 --- a/src/braft/node.cpp +++ b/src/braft/node.cpp @@ -60,6 +60,17 @@ static bvar::Adder g_num_nodes("raft_node_count"); bvar::Adder g_num_nodes("raft_node_count"); #endif +int SnapshotTimer::adjust_timeout_ms(int timeout_ms) { + if (!_first_schedule) { + return timeout_ms; + } + if (timeout_ms > 0) { + timeout_ms = butil::fast_rand_less_than(timeout_ms) + 1; + } + _first_schedule = false; + return timeout_ms; +} + class ConfigurationChangeDone : public Closure { public: void Run() { @@ -345,7 +356,6 @@ int NodeImpl::bootstrap(const BootstrapOptions& options) { _options.log_uri = options.log_uri; _options.raft_meta_uri = options.raft_meta_uri; _options.snapshot_uri = options.snapshot_uri; - _options.snapshot_file_system_adaptor = _options.snapshot_file_system_adaptor; _config_manager = new ConfigurationManager(); // Create _fsm_caller first as log_manager needs it to report error From 5db5212fe99bb7fd379b7097dee728cf9c2d5174 Mon Sep 17 00:00:00 2001 From: PFZheng Date: Wed, 24 Apr 2019 06:23:31 -0700 Subject: [PATCH 32/35] Add some bvar to record batch counter --- src/braft/fsm_caller.cpp | 9 ++ src/braft/log.cpp | 2 +- src/braft/log_entry.cpp | 1 - src/braft/log_entry.h | 1 - src/braft/log_manager.cpp | 10 +- src/braft/node.cpp | 5 + src/braft/node.h | 5 + src/braft/replicator.cpp | 13 ++- src/braft/util.cpp | 226 +++++++++++++++++++++++++++++++++++++- src/braft/util.h | 126 +++++++++++++++++++++ 10 files changed, 388 insertions(+), 10 deletions(-) diff --git a/src/braft/fsm_caller.cpp b/src/braft/fsm_caller.cpp index 8fcb0922..2a1eca26 100644 --- a/src/braft/fsm_caller.cpp +++ b/src/braft/fsm_caller.cpp @@ -31,6 +31,9 @@ namespace braft { +static bvar::CounterRecorder g_commit_tasks_batch_counter( + "raft_commit_tasks_batch_counter"); + FSMCaller::FSMCaller() : _log_manager(NULL) , _fsm(NULL) @@ -56,16 +59,20 @@ int FSMCaller::run(void* meta, bthread::TaskIterator& iter) { return 0; } int64_t max_committed_index = -1; + int64_t counter = 0; for (; iter; ++iter) { if (iter->type == COMMITTED) { if (iter->committed_index > max_committed_index) { max_committed_index = iter->committed_index; + counter++; } } else { if (max_committed_index >= 0) { caller->_cur_task = COMMITTED; caller->do_committed(max_committed_index); max_committed_index = -1; + g_commit_tasks_batch_counter << counter; + counter = 0; } switch (iter->type) { case COMMITTED: @@ -116,6 +123,8 @@ int FSMCaller::run(void* meta, bthread::TaskIterator& iter) { if (max_committed_index >= 0) { caller->_cur_task = COMMITTED; caller->do_committed(max_committed_index); + g_commit_tasks_batch_counter << counter; + counter = 0; } caller->_cur_task = IDLE; return 0; diff --git a/src/braft/log.cpp b/src/braft/log.cpp index 09921e5f..507ab629 100644 --- a/src/braft/log.cpp +++ b/src/braft/log.cpp @@ -993,7 +993,7 @@ int SegmentLogStorage::list_segments(bool is_empty) { } last_log_index = segment->last_index(); - it++; + ++it; } if (_open_segment) { if (last_log_index == -1 && diff --git a/src/braft/log_entry.cpp b/src/braft/log_entry.cpp index 83b536b9..540ac17a 100644 --- a/src/braft/log_entry.cpp +++ b/src/braft/log_entry.cpp @@ -14,7 +14,6 @@ // Authors: Zhangyi Chen(chenzhangyi01@baidu.com) -#include #include "braft/log_entry.h" #include "braft/local_storage.pb.h" diff --git a/src/braft/log_entry.h b/src/braft/log_entry.h index 4de5d50f..2d1a106c 100644 --- a/src/braft/log_entry.h +++ b/src/braft/log_entry.h @@ -19,7 +19,6 @@ #include // butil::IOBuf #include // butil::RefCountedThreadSafe -#include #include // fmix64 #include "braft/configuration.h" #include "braft/raft.pb.h" diff --git a/src/braft/log_manager.cpp b/src/braft/log_manager.cpp index 64618f82..c5c4a193 100644 --- a/src/braft/log_manager.cpp +++ b/src/braft/log_manager.cpp @@ -40,8 +40,13 @@ static bvar::Adder g_read_term_from_storage static bvar::PerSecond > g_read_term_from_storage_second ("raft_read_term_from_storage_second", &g_read_term_from_storage); -static bvar::LatencyRecorder g_storage_append_entries_latency("raft_storage_append_entries"); -static bvar::LatencyRecorder g_nomralized_append_entries_latency("raft_storage_append_entries_normalized"); +static bvar::LatencyRecorder g_storage_append_entries_latency( + "raft_storage_append_entries"); +static bvar::LatencyRecorder g_nomralized_append_entries_latency( + "raft_storage_append_entries_normalized"); + +static bvar::CounterRecorder g_storage_flush_batch_counter( + "raft_storage_flush_batch_counter"); LogManagerOptions::LogManagerOptions() : log_storage(NULL) @@ -483,6 +488,7 @@ class AppendBatcher { void flush() { if (_size > 0) { _lm->append_to_storage(&_to_append, _last_id); + g_storage_flush_batch_counter << _size; for (size_t i = 0; i < _size; ++i) { _storage[i]->_entries.clear(); if (_lm->_has_error.load(butil::memory_order_relaxed)) { diff --git a/src/braft/node.cpp b/src/braft/node.cpp index 724506e6..3ed456ef 100644 --- a/src/braft/node.cpp +++ b/src/braft/node.cpp @@ -60,6 +60,9 @@ static bvar::Adder g_num_nodes("raft_node_count"); bvar::Adder g_num_nodes("raft_node_count"); #endif +static bvar::CounterRecorder g_apply_tasks_batch_counter( + "raft_apply_tasks_batch_counter"); + int SnapshotTimer::adjust_timeout_ms(int timeout_ms) { if (!_first_schedule) { return timeout_ms; @@ -1696,6 +1699,8 @@ void LeaderStableClosure::Run() { } void NodeImpl::apply(LogEntryAndClosure tasks[], size_t size) { + g_apply_tasks_batch_counter << size; + std::vector entries; entries.reserve(size); std::unique_lock lck(_mutex); diff --git a/src/braft/node.h b/src/braft/node.h index e2cd35e6..c37e402f 100644 --- a/src/braft/node.h +++ b/src/braft/node.h @@ -76,8 +76,13 @@ class StepdownTimer : public NodeTimer { }; class SnapshotTimer : public NodeTimer { +public: + SnapshotTimer() : _first_schedule(true) {} protected: void run(); + int adjust_timeout_ms(int timeout_ms); +private: + bool _first_schedule; }; class BAIDU_CACHELINE_ALIGNMENT NodeImpl diff --git a/src/braft/replicator.cpp b/src/braft/replicator.cpp index d28da48b..6df635e1 100644 --- a/src/braft/replicator.cpp +++ b/src/braft/replicator.cpp @@ -37,7 +37,8 @@ BRPC_VALIDATE_GFLAG(raft_max_entries_size, ::brpc::PositiveInteger); DEFINE_int32(raft_max_parallel_append_entries_rpc_num, 1, "The max number of parallel AppendEntries requests"); -BRPC_VALIDATE_GFLAG(raft_max_parallel_append_entries_rpc_num, ::brpc::PositiveInteger); +BRPC_VALIDATE_GFLAG(raft_max_parallel_append_entries_rpc_num, + ::brpc::PositiveInteger); DEFINE_int32(raft_max_body_size, 512 * 1024, "The max byte size of AppendEntriesRequest"); @@ -45,10 +46,14 @@ BRPC_VALIDATE_GFLAG(raft_max_body_size, ::brpc::PositiveInteger); DEFINE_int32(raft_retry_replicate_interval_ms, 1000, "Interval of retry to append entries or install snapshot"); -BRPC_VALIDATE_GFLAG(raft_retry_replicate_interval_ms, brpc::PositiveInteger); +BRPC_VALIDATE_GFLAG(raft_retry_replicate_interval_ms, + brpc::PositiveInteger); static bvar::LatencyRecorder g_send_entries_latency("raft_send_entries"); -static bvar::LatencyRecorder g_normalized_send_entries_latency("raft_send_entries_normalized"); +static bvar::LatencyRecorder g_normalized_send_entries_latency( + "raft_send_entries_normalized"); +static bvar::CounterRecorder g_send_entries_batch_counter( + "raft_send_entries_batch_counter"); ReplicatorOptions::ReplicatorOptions() : dynamic_heartbeat_timeout_ms(NULL) @@ -670,6 +675,8 @@ void Replicator::_send_entries() { _append_entries_counter++; _next_index += request->entries_size(); _flying_append_entries_size += request->entries_size(); + + g_send_entries_batch_counter << request->entries_size(); BRAFT_VLOG << "node " << _options.group_id << ":" << _options.server_id << " send AppendEntriesRequest to " << _options.peer_id << " term " << _options.term diff --git a/src/braft/util.cpp b/src/braft/util.cpp index 7fed4873..053d7384 100644 --- a/src/braft/util.cpp +++ b/src/braft/util.cpp @@ -16,14 +16,236 @@ // Wang,Yao(wangyao02@baidu.com) #include "braft/util.h" - +#include #include #include #include // butil::RawPacker #include - #include "braft/raft.h" +namespace bvar { + +// Reloading following gflags does not change names of the corresponding bvars. +// Avoid reloading in practice. +DEFINE_int32(bvar_counter_p1, 80, "First counter percentile"); +DEFINE_int32(bvar_counter_p2, 90, "Second counter percentile"); +DEFINE_int32(bvar_counter_p3, 99, "Third counter percentile"); + +static bool valid_percentile(const char*, int32_t v) { + return v > 0 && v < 100; +} + +const bool ALLOW_UNUSED dummy_bvar_counter_p1 = ::google::RegisterFlagValidator( + &FLAGS_bvar_counter_p1, valid_percentile); +const bool ALLOW_UNUSED dummy_bvar_counter_p2 = ::google::RegisterFlagValidator( + &FLAGS_bvar_counter_p2, valid_percentile); +const bool ALLOW_UNUSED dummy_bvar_counter_p3 = ::google::RegisterFlagValidator( + &FLAGS_bvar_counter_p3, valid_percentile); + +namespace detail { + +typedef PercentileSamples<1022> CombinedPercentileSamples; + +static int64_t get_window_recorder_qps(void* arg) { + detail::Sample s; + static_cast(arg)->get_span(1, &s); + // Use floating point to avoid overflow. + if (s.time_us <= 0) { + return 0; + } + return static_cast(round(s.data.num * 1000000.0 / s.time_us)); +} + +static int64_t get_recorder_count(void* arg) { + return static_cast(arg)->get_value().num; +} + +// Caller is responsible for deleting the return value. +static CombinedPercentileSamples* combine(PercentileWindow* w) { + CombinedPercentileSamples* cb = new CombinedPercentileSamples; + std::vector buckets; + w->get_samples(&buckets); + cb->combine_of(buckets.begin(), buckets.end()); + return cb; +} + +template +static int64_t get_counter_percetile(void* arg) { + return ((CounterRecorder*)arg)->counter_percentile( + (double)numerator / double(denominator)); +} + +static int64_t get_p1_counter(void* arg) { + CounterRecorder* cr = static_cast(arg); + return cr->counter_percentile(FLAGS_bvar_counter_p1 / 100.0); +} +static int64_t get_p2_counter(void* arg) { + CounterRecorder* cr = static_cast(arg); + return cr->counter_percentile(FLAGS_bvar_counter_p2 / 100.0); +} +static int64_t get_p3_counter(void* arg) { + CounterRecorder* cr = static_cast(arg); + return cr->counter_percentile(FLAGS_bvar_counter_p3 / 100.0); +} + +static Vector get_counters(void *arg) { + std::unique_ptr cb( + combine((PercentileWindow*)arg)); + // NOTE: We don't show 99.99% since it's often significantly larger than + // other values and make other curves on the plotted graph small and + // hard to read.ggggnnn + Vector result; + result[0] = cb->get_number(FLAGS_bvar_counter_p1 / 100.0); + result[1] = cb->get_number(FLAGS_bvar_counter_p2 / 100.0); + result[2] = cb->get_number(FLAGS_bvar_counter_p3 / 100.0); + result[3] = cb->get_number(0.999); + return result; +} + +CounterRecorderBase::CounterRecorderBase(time_t window_size) + : _max_counter() + , _avg_counter_window(&_avg_counter, window_size) + , _max_counter_window(&_max_counter, window_size) + , _counter_percentile_window(&_counter_percentile, window_size) + , _total_times(get_recorder_count, &_avg_counter) + , _qps(get_window_recorder_qps, &_avg_counter_window) + , _counter_p1(get_p1_counter, this) + , _counter_p2(get_p2_counter, this) + , _counter_p3(get_p3_counter, this) + , _counter_999(get_counter_percetile<999, 1000>, this) + , _counter_9999(get_counter_percetile<9999, 10000>, this) + , _counter_cdf(&_counter_percentile_window) + , _counter_percentiles(get_counters, &_counter_percentile_window) +{} + +} // namespace detail + +// CounterRecorder +Vector CounterRecorder::counter_percentiles() const { + // const_cast here is just to adapt parameter type and safe. + return detail::get_counters( + const_cast(&_counter_percentile_window)); +} + +int64_t CounterRecorder::qps(time_t window_size) const { + detail::Sample s; + _avg_counter_window.get_span(window_size, &s); + // Use floating point to avoid overflow. + if (s.time_us <= 0) { + return 0; + } + return static_cast(round(s.data.num * 1000000.0 / s.time_us)); +} + +int CounterRecorder::expose(const butil::StringPiece& prefix1, + const butil::StringPiece& prefix2) { + if (prefix2.empty()) { + LOG(ERROR) << "Parameter[prefix2] is empty"; + return -1; + } + butil::StringPiece prefix = prefix2; + // User may add "_counter" as the suffix, remove it. + if (prefix.ends_with("counter") || prefix.ends_with("Counter")) { + prefix.remove_suffix(7); + if (prefix.empty()) { + LOG(ERROR) << "Invalid prefix2=" << prefix2; + return -1; + } + } + std::string tmp; + if (!prefix1.empty()) { + tmp.reserve(prefix1.size() + prefix.size() + 1); + tmp.append(prefix1.data(), prefix1.size()); + tmp.push_back('_'); // prefix1 ending with _ is good. + tmp.append(prefix.data(), prefix.size()); + prefix = tmp; + } + + // set debug names for printing helpful error log. + _avg_counter.set_debug_name(prefix); + _counter_percentile.set_debug_name(prefix); + + if (_avg_counter_window.expose_as(prefix, "avg_counter") != 0) { + return -1; + } + if (_max_counter_window.expose_as(prefix, "max_counter") != 0) { + return -1; + } + if (_total_times.expose_as(prefix, "total_times") != 0) { + return -1; + } + if (_qps.expose_as(prefix, "qps") != 0) { + return -1; + } + char namebuf[32]; + snprintf(namebuf, sizeof(namebuf), "counter_%d", (int)FLAGS_bvar_counter_p1); + if (_counter_p1.expose_as(prefix, namebuf, DISPLAY_ON_PLAIN_TEXT) != 0) { + return -1; + } + snprintf(namebuf, sizeof(namebuf), "counter_%d", (int)FLAGS_bvar_counter_p2); + if (_counter_p2.expose_as(prefix, namebuf, DISPLAY_ON_PLAIN_TEXT) != 0) { + return -1; + } + snprintf(namebuf, sizeof(namebuf), "counter_%u", (int)FLAGS_bvar_counter_p3); + if (_counter_p3.expose_as(prefix, namebuf, DISPLAY_ON_PLAIN_TEXT) != 0) { + return -1; + } + if (_counter_999.expose_as(prefix, "counter_999", DISPLAY_ON_PLAIN_TEXT) != 0) { + return -1; + } + if (_counter_9999.expose_as(prefix, "counter_9999") != 0) { + return -1; + } + if (_counter_cdf.expose_as(prefix, "counter_cdf", DISPLAY_ON_HTML) != 0) { + return -1; + } + if (_counter_percentiles.expose_as(prefix, "counter_percentiles", DISPLAY_ON_HTML) != 0) { + return -1; + } + snprintf(namebuf, sizeof(namebuf), "%d%%,%d%%,%d%%,99.9%%", + (int)FLAGS_bvar_counter_p1, (int)FLAGS_bvar_counter_p2, + (int)FLAGS_bvar_counter_p3); + CHECK_EQ(0, _counter_percentiles.set_vector_names(namebuf)); + return 0; +} + +int64_t CounterRecorder::counter_percentile(double ratio) const { + std::unique_ptr cb( + combine((detail::PercentileWindow*)&_counter_percentile_window)); + return cb->get_number(ratio); +} + +void CounterRecorder::hide() { + _avg_counter_window.hide(); + _max_counter_window.hide(); + _total_times.hide(); + _qps.hide(); + _counter_p1.hide(); + _counter_p2.hide(); + _counter_p3.hide(); + _counter_999.hide(); + _counter_9999.hide(); + _counter_cdf.hide(); + _counter_percentiles.hide(); +} + +CounterRecorder& CounterRecorder::operator<<(int64_t count_num) { + _avg_counter << count_num; + _max_counter << count_num; + _counter_percentile << count_num; + return *this; +} + +std::ostream& operator<<(std::ostream& os, const CounterRecorder& rec) { + return os << "{avg=" << rec.avg_counter() + << " max" << rec.window_size() << '=' << rec.max_counter() + << " qps=" << rec.qps() + << " count=" << rec.total_times() << '}'; +} + +} // namespace bvar + + namespace braft { static void* run_closure(void* arg) { diff --git a/src/braft/util.h b/src/braft/util.h index 50359845..50bec34b 100644 --- a/src/braft/util.h +++ b/src/braft/util.h @@ -40,9 +40,135 @@ #include #include #include +#include #include "braft/macros.h" #include "braft/raft.h" +namespace bvar { +namespace detail { + +class Percentile; + +typedef Window RecorderWindow; +typedef Window, SERIES_IN_SECOND> MaxUint64Window; +typedef Window PercentileWindow; + +// For mimic constructor inheritance. +class CounterRecorderBase { +public: + explicit CounterRecorderBase(time_t window_size); + time_t window_size() const { return _avg_counter_window.window_size(); } +protected: + IntRecorder _avg_counter; + Maxer _max_counter; + Percentile _counter_percentile; + RecorderWindow _avg_counter_window; + MaxUint64Window _max_counter_window; + PercentileWindow _counter_percentile_window; + + PassiveStatus _total_times; + PassiveStatus _qps; + PassiveStatus _counter_p1; + PassiveStatus _counter_p2; + PassiveStatus _counter_p3; + PassiveStatus _counter_999; // 99.9% + PassiveStatus _counter_9999; // 99.99% + CDF _counter_cdf; + PassiveStatus > _counter_percentiles; +}; +} // namespace detail + +// Specialized structure to record counter. +// It's not a Variable, but it contains multiple bvar inside. +class CounterRecorder : public detail::CounterRecorderBase { + typedef detail::CounterRecorderBase Base; +public: + CounterRecorder() : Base(-1) {} + explicit CounterRecorder(time_t window_size) : Base(window_size) {} + explicit CounterRecorder(const butil::StringPiece& prefix) : Base(-1) { + expose(prefix); + } + CounterRecorder(const butil::StringPiece& prefix, + time_t window_size) : Base(window_size) { + expose(prefix); + } + CounterRecorder(const butil::StringPiece& prefix1, + const butil::StringPiece& prefix2) : Base(-1) { + expose(prefix1, prefix2); + } + CounterRecorder(const butil::StringPiece& prefix1, + const butil::StringPiece& prefix2, + time_t window_size) : Base(window_size) { + expose(prefix1, prefix2); + } + + ~CounterRecorder() { hide(); } + + // Record the counter num. + CounterRecorder& operator<<(int64_t count_num); + + // Expose all internal variables using `prefix' as prefix. + // Returns 0 on success, -1 otherwise. + // Example: + // CounterRecorder rec; + // rec.expose("foo_bar_add"); // foo_bar_add_avg_counter + // // foo_bar_add_max_counter + // // foo_bar_add_total_times + // // foo_bar_add_qps + // rec.expose("foo_bar", "apply"); // foo_bar_apply_avg_counter + // // foo_bar_apply_max_counter + // // foo_bar_apply_total_times + // // foo_bar_apply_qps + int expose(const butil::StringPiece& prefix) { + return expose(butil::StringPiece(), prefix); + } + int expose(const butil::StringPiece& prefix1, + const butil::StringPiece& prefix2); + + // Hide all internal variables, called in dtor as well. + void hide(); + + // Get the average counter num in recent |window_size| seconds + // If |window_size| is absent, use the window_size to ctor. + int64_t avg_counter(time_t window_size) const + { return _avg_counter_window.get_value(window_size).get_average_int(); } + int64_t avg_counter() const + { return _avg_counter_window.get_value().get_average_int(); } + + // Get p1/p2/p3/99.9-ile counter num in recent window_size-to-ctor seconds. + Vector counter_percentiles() const; + + // Get the max counter numer in recent window_size-to-ctor seconds. + int64_t max_counter() const { return _max_counter_window.get_value(); } + + // Get the total number of recorded counter nums + int64_t total_times() const { return _avg_counter.get_value().num; } + + // Get qps in recent |window_size| seconds. The `q' means counter nums. + // recorded by operator<<(). + // If |window_size| is absent, use the window_size to ctor. + int64_t qps(time_t window_size) const; + int64_t qps() const { return _qps.get_value(); } + + // Get |ratio|-ile counter num in recent |window_size| seconds + // E.g. 0.99 means 99%-ile + int64_t counter_percentile(double ratio) const; + + // Get name of a sub-bvar. + const std::string& avg_counter_name() const { return _avg_counter_window.name(); } + const std::string& counter_percentiles_name() const + { return _counter_percentiles.name(); } + const std::string& counter_cdf_name() const { return _counter_cdf.name(); } + const std::string& max_counter_name() const + { return _max_counter_window.name(); } + const std::string& total_times_name() const { return _total_times.name(); } + const std::string& qps_name() const { return _qps.name(); } +}; + +std::ostream& operator<<(std::ostream& os, const CounterRecorder&); + +} // namespace bvar + namespace braft { class Closure; From 3b7d9c25412c06490610aca43d44d53b401a4aa3 Mon Sep 17 00:00:00 2001 From: PFZheng Date: Thu, 25 Apr 2019 01:20:05 -0700 Subject: [PATCH 33/35] Add vote() to trigger vote manually, the election timeout should be set back after a period. --- src/braft/node.cpp | 23 ++++++++++++++++++++++- src/braft/node.h | 4 ++++ src/braft/raft.cpp | 4 ++++ src/braft/raft.h | 5 +++++ test/test_node.cpp | 9 ++------- 5 files changed, 37 insertions(+), 8 deletions(-) diff --git a/src/braft/node.cpp b/src/braft/node.cpp index 3ed456ef..0468d0fc 100644 --- a/src/braft/node.cpp +++ b/src/braft/node.cpp @@ -129,6 +129,7 @@ NodeImpl::NodeImpl(const GroupId& group_id, const PeerId& peer_id) , _ballot_box(NULL) , _snapshot_executor(NULL) , _stop_transfer_arg(NULL) + , _vote_triggered(false) , _waking_candidate(0) , _append_entries_cache(NULL) , _append_entries_cache_version(0) @@ -154,6 +155,7 @@ NodeImpl::NodeImpl() , _ballot_box(NULL) , _snapshot_executor(NULL) , _stop_transfer_arg(NULL) + , _vote_triggered(false) , _waking_candidate(0) { AddRef(); g_num_nodes << 1; @@ -973,10 +975,13 @@ void NodeImpl::handle_election_timeout() { return; } - if ((butil::monotonic_time_ms() - _last_leader_timestamp) + // check timestamp, skip one cycle check when trigger vote + if (!_vote_triggered && + (butil::monotonic_time_ms() - _last_leader_timestamp) < _options.election_timeout_ms) { return; } + _vote_triggered = false; // Reset leader as the leader is uncerntain on election timeout. PeerId empty_id; @@ -1154,6 +1159,22 @@ int NodeImpl::transfer_leadership_to(const PeerId& peer) { return 0; } +void NodeImpl::vote(int election_timeout) { + std::unique_lock lck(_mutex); + _options.election_timeout_ms = election_timeout; + _replicator_group.reset_heartbeat_interval( + heartbeat_timeout(_options.election_timeout_ms)); + _replicator_group.reset_election_timeout_interval(_options.election_timeout_ms); + if (_state != STATE_FOLLOWER) { + return; + } + _vote_triggered = true; + LOG(INFO) << "node " << _group_id << ":" << _server_id << " trigger-vote," + " current_term " << _current_term << " state " << state2str(_state) << + " election_timeout " << election_timeout; + _election_timer.reset(election_timeout); +} + void NodeImpl::reset_election_timeout_ms(int election_timeout_ms) { std::unique_lock lck(_mutex); _options.election_timeout_ms = election_timeout_ms; diff --git a/src/braft/node.h b/src/braft/node.h index c37e402f..8bf60cc7 100644 --- a/src/braft/node.h +++ b/src/braft/node.h @@ -144,6 +144,9 @@ friend class ConfigurationChangeDone; // trigger snapshot void snapshot(Closure* done); + // trigger vote + void vote(int election_timeout); + // reset the election_timeout for the very node void reset_election_timeout_ms(int election_timeout_ms); @@ -436,6 +439,7 @@ friend class butil::RefCountedThreadSafe; SnapshotTimer _snapshot_timer; bthread_timer_t _transfer_timer; StopTransferArg* _stop_transfer_arg; + bool _vote_triggered; ReplicatorId _waking_candidate; bthread::ExecutionQueueId _apply_queue_id; bthread::ExecutionQueue::scoped_ptr_t _apply_queue; diff --git a/src/braft/raft.cpp b/src/braft/raft.cpp index c45e7573..93b3f1c4 100644 --- a/src/braft/raft.cpp +++ b/src/braft/raft.cpp @@ -152,6 +152,10 @@ void Node::snapshot(Closure* done) { _impl->snapshot(done); } +void Node::vote(int election_timeout) { + _impl->vote(election_timeout); +} + void Node::reset_election_timeout_ms(int election_timeout_ms) { _impl->reset_election_timeout_ms(election_timeout_ms); } diff --git a/src/braft/raft.h b/src/braft/raft.h index 857090f5..1604b14b 100644 --- a/src/braft/raft.h +++ b/src/braft/raft.h @@ -603,6 +603,11 @@ class Node { // when the snapshot finishes, describing the detailed result. void snapshot(Closure* done); + // user trigger vote + // reset election_timeout, suggest some peer to become the leader in a + // higher probability + void vote(int election_timeout); + // reset the election_timeout for the very node void reset_election_timeout_ms(int election_timeout_ms); diff --git a/test/test_node.cpp b/test/test_node.cpp index 5901c023..862c58de 100644 --- a/test/test_node.cpp +++ b/test/test_node.cpp @@ -260,11 +260,7 @@ class Cluster { butil::string_printf(&options.snapshot_uri, "local://./data/%s/snapshot", butil::endpoint2str(listen_addr).c_str()); - int64_t throttle_throughput_bytes = 10 * 1024 * 1024; - int64_t check_cycle = 10; - braft::SnapshotThrottle* throttle = - new braft::ThroughputSnapshotThrottle(throttle_throughput_bytes, check_cycle); - scoped_refptr tst(throttle); + scoped_refptr tst(_throttle); options.snapshot_throttle = &tst; options.catchup_margin = 2; @@ -273,7 +269,6 @@ class Cluster { int ret = node->init(options); if (ret != 0) { LOG(WARNING) << "init_node failed, server: " << listen_addr; - delete throttle; delete node; return ret; } else { @@ -974,7 +969,7 @@ TEST_P(NodeTest, JoinNode) { cluster.stop_all(); } -TEST_F(NodeTest, Leader_step_down_during_install_snapshot) { +TEST_P(NodeTest, Leader_step_down_during_install_snapshot) { std::vector peers; braft::PeerId peer0; peer0.addr.ip = butil::my_ip(); From 821372daf7184a9f7b06d1bd5df24ea5e07207d0 Mon Sep 17 00:00:00 2001 From: PFZheng Date: Thu, 25 Apr 2019 02:27:04 -0700 Subject: [PATCH 34/35] Fix some typo. --- src/braft/node.cpp | 6 +++--- src/braft/replicator.cpp | 31 +++++++++++++++++++------------ 2 files changed, 22 insertions(+), 15 deletions(-) diff --git a/src/braft/node.cpp b/src/braft/node.cpp index 53630ce3..91d70fd8 100644 --- a/src/braft/node.cpp +++ b/src/braft/node.cpp @@ -1291,7 +1291,7 @@ struct OnRequestVoteRPCDone : public google::protobuf::Closure { if (cntl.ErrorCode() != 0) { LOG(WARNING) << "node " << node->node_id() << " received RequestVoteResponse from " << peer - << " error: " << cntl.ErrorText(); + << " error: " << cntl.ErrorText(); break; } node->handle_request_vote_response(peer, term, response); @@ -1534,8 +1534,8 @@ void NodeImpl::step_down(const int64_t term, bool wakeup_a_candidate, BRAFT_VLOG << "node " << _group_id << ":" << _server_id << " term " << _current_term << " stepdown from " << state2str(_state) - << " new_term " << term << - " wakeup_a_candidate=" << wakeup_a_candidate; + << " new_term " << term + << " wakeup_a_candidate=" << wakeup_a_candidate; if (!is_active_state(_state)) { return; diff --git a/src/braft/replicator.cpp b/src/braft/replicator.cpp index ec675876..a0272d73 100644 --- a/src/braft/replicator.cpp +++ b/src/braft/replicator.cpp @@ -386,13 +386,15 @@ void Replicator::_on_rpc_returned(ReplicatorId id, brpc::Controller* cntl, } } if (!valid_rpc) { - BRAFT_VLOG << " ignore invalid rpc"; + ss << " ignore invalid rpc"; + BRAFT_VLOG << ss.str(); CHECK_EQ(0, bthread_id_unlock(r->_id)) << "Fail to unlock " << r->_id; return; } if (cntl->Failed()) { - BRAFT_VLOG << " fail, sleep."; + ss << " fail, sleep."; + BRAFT_VLOG << ss.str(); // TODO: Should it be VLOG? LOG_IF(WARNING, (r->_consecutive_error_times++) % 10 == 0) @@ -427,9 +429,10 @@ void Replicator::_on_rpc_returned(ReplicatorId id, brpc::Controller* cntl, node_impl->Release(); return; } - BRAFT_VLOG << " fail, find next_index remote last_log_index " << response->last_log_index() - << " local next_index " << r->_next_index - << " rpc prev_log_index " << request->prev_log_index(); + ss << " fail, find next_index remote last_log_index " << response->last_log_index() + << " local next_index " << r->_next_index + << " rpc prev_log_index " << request->prev_log_index(); + BRAFT_VLOG << ss.str(); if (rpc_send_time > r->_last_rpc_send_timestamp) { r->_last_rpc_send_timestamp = rpc_send_time; } @@ -460,7 +463,8 @@ void Replicator::_on_rpc_returned(ReplicatorId id, brpc::Controller* cntl, return; } - BRAFT_VLOG << " success"; + ss << " success"; + BRAFT_VLOG << ss.str(); if (response->term() != r->_options.term) { LOG(ERROR) << "Group " << r->_options.group_id @@ -476,7 +480,7 @@ void Replicator::_on_rpc_returned(ReplicatorId id, brpc::Controller* cntl, const int entries_size = request->entries_size(); const int64_t rpc_last_log_index = request->prev_log_index() + entries_size; BRAFT_VLOG_IF(entries_size > 0) << "Group " << r->_options.group_id - << " Replicated logs in [" + << " replicated logs in [" << min_flying_index << ", " << rpc_last_log_index << "] to peer " << r->_options.peer_id; @@ -1111,12 +1115,14 @@ void Replicator::_on_timeout_now_returned( return; } - BRAFT_VLOG << "node " << r->_options.group_id << ":" << r->_options.server_id - << " received TimeoutNowResponse from " - << r->_options.peer_id; + std::stringstream ss; + ss << "node " << r->_options.group_id << ":" << r->_options.server_id + << " received TimeoutNowResponse from " + << r->_options.peer_id; if (cntl->Failed()) { - BRAFT_VLOG << " fail : " << cntl->ErrorText(); + ss << " fail : " << cntl->ErrorText(); + BRAFT_VLOG << ss.str(); if (stop_after_finish) { r->_notify_on_caught_up(ESTOP, true); @@ -1126,7 +1132,8 @@ void Replicator::_on_timeout_now_returned( } return; } - BRAFT_VLOG << (response->success() ? " success " : "fail:"); + ss << (response->success() ? " success " : "fail:"); + BRAFT_VLOG << ss.str(); if (response->term() > r->_options.term) { NodeImpl *node_impl = r->_options.node; From 20ae73bdca73d22a286c2cacac2d1cd55ef6bce5 Mon Sep 17 00:00:00 2001 From: PFZheng Date: Thu, 25 Apr 2019 05:38:01 -0700 Subject: [PATCH 35/35] Reduce log size of test_node to make Travis CI happy. --- test/test_node.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/test/test_node.cpp b/test/test_node.cpp index 862c58de..a1744ec6 100644 --- a/test/test_node.cpp +++ b/test/test_node.cpp @@ -85,8 +85,7 @@ class MockFSM : public braft::StateMachine { virtual void on_apply(braft::Iterator& iter) { for (; iter.valid(); iter.next()) { LOG_IF(TRACE, !g_dont_print_apply_log) << "addr " << address - << " apply " << iter.index() - << " data " << iter.data(); + << " apply " << iter.index(); ::brpc::ClosureGuard guard(iter.done()); lock(); logs.push_back(iter.data());