diff --git a/src/braft/builtin_service_impl.cpp b/src/braft/builtin_service_impl.cpp index 6e7a8233..d7764a64 100644 --- a/src/braft/builtin_service_impl.cpp +++ b/src/braft/builtin_service_impl.cpp @@ -41,11 +41,10 @@ void RaftStatImpl::default_method(::google::protobuf::RpcController* controller, brpc::Controller* cntl = (brpc::Controller*)controller; std::string group_id = cntl->http_request().unresolved_path(); std::vector > nodes; - NodeManager* nm = NodeManager::GetInstance(); if (group_id.empty()) { - nm->get_all_nodes(&nodes); + global_node_manager->get_all_nodes(&nodes); } else { - nm->get_nodes_by_group_id(group_id, &nodes); + global_node_manager->get_nodes_by_group_id(group_id, &nodes); } const bool html = brpc::UseHTML(cntl->http_request()); if (html) { diff --git a/src/braft/cli_service.cpp b/src/braft/cli_service.cpp index 813157d1..5f84079d 100644 --- a/src/braft/cli_service.cpp +++ b/src/braft/cli_service.cpp @@ -198,7 +198,6 @@ void CliServiceImpl::get_leader(::google::protobuf::RpcController* controller, brpc::Controller* cntl = (brpc::Controller*)controller; brpc::ClosureGuard done_guard(done); std::vector > nodes; - NodeManager* const nm = NodeManager::GetInstance(); if (request->has_peer_id()) { PeerId peer; if (peer.parse(request->peer_id()) != 0) { @@ -206,12 +205,13 @@ void CliServiceImpl::get_leader(::google::protobuf::RpcController* controller, request->peer_id().c_str()); return; } - scoped_refptr node = nm->get(request->group_id(), peer); + scoped_refptr node = + global_node_manager->get(request->group_id(), peer); if (node) { nodes.push_back(node); } } else { - nm->get_nodes_by_group_id(request->group_id(), &nodes); + global_node_manager->get_nodes_by_group_id(request->group_id(), &nodes); } if (nodes.empty()) { cntl->SetFailed(ENOENT, "No nodes in group %s", @@ -232,9 +232,8 @@ void CliServiceImpl::get_leader(::google::protobuf::RpcController* controller, butil::Status CliServiceImpl::get_node(scoped_refptr* node, const GroupId& group_id, const std::string& peer_id) { - NodeManager* const nm = NodeManager::GetInstance(); if (!peer_id.empty()) { - *node = nm->get(group_id, peer_id); + *node = global_node_manager->get(group_id, peer_id); if (!(*node)) { return butil::Status(ENOENT, "Fail to find node %s in group %s", peer_id.c_str(), @@ -242,7 +241,7 @@ butil::Status CliServiceImpl::get_node(scoped_refptr* node, } } else { std::vector > nodes; - nm->get_nodes_by_group_id(group_id, &nodes); + global_node_manager->get_nodes_by_group_id(group_id, &nodes); if (nodes.empty()) { return butil::Status(ENOENT, "Fail to find node in group %s", group_id.c_str()); diff --git a/src/braft/configuration.h b/src/braft/configuration.h index c34ecebc..1e3acade 100644 --- a/src/braft/configuration.h +++ b/src/braft/configuration.h @@ -31,6 +31,8 @@ namespace braft { typedef std::string GroupId; +// GroupId with version, format: {group_id}_{index} +typedef std::string VersionedGroupId; // Represent a participant in a replicating group. struct PeerId { diff --git a/src/braft/fsm_caller.cpp b/src/braft/fsm_caller.cpp index 2a1eca26..89defa50 100644 --- a/src/braft/fsm_caller.cpp +++ b/src/braft/fsm_caller.cpp @@ -98,7 +98,8 @@ int FSMCaller::run(void* meta, bthread::TaskIterator& iter) { delete iter->status; break; case LEADER_START: - caller->do_leader_start(iter->term); + caller->do_leader_start(*(iter->leader_start_context)); + delete iter->leader_start_context; break; case START_FOLLOWING: caller->_cur_task = START_FOLLOWING; @@ -430,19 +431,26 @@ int FSMCaller::on_leader_stop(const butil::Status& status) { return 0; } -int FSMCaller::on_leader_start(int64_t term) { +int FSMCaller::on_leader_start(int64_t term, int64_t lease_epoch) { ApplyTask task; task.type = LEADER_START; - task.term = term; - return bthread::execution_queue_execute(_queue_id, task); + LeaderStartContext* on_leader_start_context = + new LeaderStartContext(term, lease_epoch); + task.leader_start_context = on_leader_start_context; + if (bthread::execution_queue_execute(_queue_id, task) != 0) { + delete on_leader_start_context; + return -1; + } + return 0; } void FSMCaller::do_leader_stop(const butil::Status& status) { _fsm->on_leader_stop(status); } -void FSMCaller::do_leader_start(int64_t term) { - _fsm->on_leader_start(term); +void FSMCaller::do_leader_start(const LeaderStartContext& leader_start_context) { + _node->leader_lease_start(leader_start_context.lease_epoch); + _fsm->on_leader_start(leader_start_context.term); } int FSMCaller::on_start_following(const LeaderChangeContext& start_following_context) { diff --git a/src/braft/fsm_caller.h b/src/braft/fsm_caller.h index 0b01ebb6..897a50f9 100644 --- a/src/braft/fsm_caller.h +++ b/src/braft/fsm_caller.h @@ -26,6 +26,7 @@ #include "braft/closure_queue.h" #include "braft/macros.h" #include "braft/log_entry.h" +#include "braft/lease.h" namespace braft { @@ -112,7 +113,7 @@ class BAIDU_CACHELINE_ALIGNMENT FSMCaller { BRAFT_MOCK int on_snapshot_load(LoadSnapshotClosure* done); BRAFT_MOCK int on_snapshot_save(SaveSnapshotClosure* done); int on_leader_stop(const butil::Status& status); - int on_leader_start(int64_t term); + int on_leader_start(int64_t term, int64_t lease_epoch); int on_start_following(const LeaderChangeContext& start_following_context); int on_stop_following(const LeaderChangeContext& stop_following_context); BRAFT_MOCK int on_error(const Error& e); @@ -138,6 +139,15 @@ friend class IteratorImpl; ERROR, }; + struct LeaderStartContext { + LeaderStartContext(int64_t term_, int64_t lease_epoch_) + : term(term_), lease_epoch(lease_epoch_) + {} + + int64_t term; + int64_t lease_epoch; + }; + struct ApplyTask { TaskType type; union { @@ -145,7 +155,7 @@ friend class IteratorImpl; int64_t committed_index; // For on_leader_start - int64_t term; + LeaderStartContext* leader_start_context; // For on_leader_stop butil::Status* status; @@ -167,7 +177,7 @@ friend class IteratorImpl; void do_snapshot_load(LoadSnapshotClosure* done); void do_on_error(OnErrorClousre* done); void do_leader_stop(const butil::Status& status); - void do_leader_start(int64_t term); + void do_leader_start(const LeaderStartContext& leader_start_context); void do_start_following(const LeaderChangeContext& start_following_context); void do_stop_following(const LeaderChangeContext& stop_following_context); void set_error(const Error& e); diff --git a/src/braft/lease.cpp b/src/braft/lease.cpp new file mode 100644 index 00000000..fa7c88d7 --- /dev/null +++ b/src/braft/lease.cpp @@ -0,0 +1,141 @@ +// Copyright (c) 2019 Baidu.com, Inc. All Rights Reserved +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Authors: Pengfei Zheng (zhengpengfei@baidu.com) + +#include +#include +#include "braft/lease.h" + +namespace braft { + +DEFINE_bool(raft_enable_leader_lease, false, + "Enable or disable leader lease. only when all peers in a raft group " + "set this configuration to true, leader lease check and vote are safe."); +BRPC_VALIDATE_GFLAG(raft_enable_leader_lease, ::brpc::PassValidate); + +void LeaderLease::init(int64_t election_timeout_ms) { + _election_timeout_ms = election_timeout_ms; +} + +void LeaderLease::on_leader_start(int64_t term) { + BAIDU_SCOPED_LOCK(_mutex); + ++_lease_epoch; + _term = term; + _last_active_timestamp = 0; +} + +void LeaderLease::on_leader_stop() { + BAIDU_SCOPED_LOCK(_mutex); + _last_active_timestamp = 0; + _term = 0; +} + +void LeaderLease::on_lease_start(int64_t expect_lease_epoch, int64_t last_active_timestamp) { + BAIDU_SCOPED_LOCK(_mutex); + if (_term == 0 || expect_lease_epoch != _lease_epoch) { + return; + } + _last_active_timestamp = last_active_timestamp; +} + +void LeaderLease::renew(int64_t last_active_timestamp) { + BAIDU_SCOPED_LOCK(_mutex); + _last_active_timestamp = last_active_timestamp; +} + +void LeaderLease::get_lease_info(LeaseInfo* lease_info) { + lease_info->term = 0; + lease_info->lease_epoch = 0; + if (!FLAGS_raft_enable_leader_lease) { + lease_info->state = LeaderLease::DISABLED; + return; + } + + BAIDU_SCOPED_LOCK(_mutex); + if (_term == 0) { + lease_info->state = LeaderLease::EXPIRED; + return; + } + if (_last_active_timestamp == 0) { + lease_info->state = LeaderLease::NOT_READY; + return; + } + if (butil::monotonic_time_ms() < _last_active_timestamp + _election_timeout_ms) { + lease_info->term = _term; + lease_info->lease_epoch = _lease_epoch; + lease_info->state = LeaderLease::VALID; + } else { + lease_info->state = LeaderLease::SUSPECT; + } +} + +int64_t LeaderLease::lease_epoch() { + BAIDU_SCOPED_LOCK(_mutex); + return _lease_epoch; +} + +void LeaderLease::reset_election_timeout_ms(int64_t election_timeout_ms) { + BAIDU_SCOPED_LOCK(_mutex); + _election_timeout_ms = election_timeout_ms; +} + +void FollowerLease::init(int64_t election_timeout_ms, int64_t max_clock_drift_ms) { + _election_timeout_ms = election_timeout_ms; + _max_clock_drift_ms = max_clock_drift_ms; + // When the node restart, we are not sure when the lease will be expired actually, + // so just be conservative. + _last_leader_timestamp = butil::monotonic_time_ms(); +} + +void FollowerLease::renew(const PeerId& leader_id) { + _last_leader = leader_id; + _last_leader_timestamp = butil::monotonic_time_ms(); +} + +int64_t FollowerLease::votable_time_from_now() { + if (!FLAGS_raft_enable_leader_lease) { + return 0; + } + + int64_t now = butil::monotonic_time_ms(); + int64_t votable_timestamp = _last_leader_timestamp + _election_timeout_ms + + _max_clock_drift_ms; + if (now >= votable_timestamp) { + return 0; + } + return votable_timestamp - now; +} + +const PeerId& FollowerLease::last_leader() { + return _last_leader; +} + +bool FollowerLease::expired() { + return butil::monotonic_time_ms() - _last_leader_timestamp + >= _election_timeout_ms + _max_clock_drift_ms; +} + +void FollowerLease::reset() { + _last_leader = PeerId(); + _last_leader_timestamp = 0; +} + +void FollowerLease::reset_election_timeout_ms(int64_t election_timeout_ms, + int64_t max_clock_drift_ms) { + _election_timeout_ms = election_timeout_ms; + _max_clock_drift_ms = max_clock_drift_ms; +} + +} // namespace braft diff --git a/src/braft/lease.h b/src/braft/lease.h new file mode 100644 index 00000000..be2844e7 --- /dev/null +++ b/src/braft/lease.h @@ -0,0 +1,88 @@ +// Copyright (c) 2019 Baidu.com, Inc. All Rights Reserved +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Authors: Pengfei Zheng (zhengpengfei@baidu.com) + +#ifndef PUBLIC_RAFT_LEASE_H +#define PUBLIC_RAFT_LEASE_H + +#include "braft/util.h" + +namespace braft { + +class LeaderLease { +public: + enum InternalState { + DISABLED, + EXPIRED, + NOT_READY, + VALID, + SUSPECT, + }; + + struct LeaseInfo { + InternalState state; + int64_t term; + int64_t lease_epoch; + }; + + LeaderLease() + : _election_timeout_ms(0) + , _last_active_timestamp(0) + , _term(0) + , _lease_epoch(0) + {} + + void init(int64_t election_timeout_ms); + void on_leader_start(int64_t term); + void on_leader_stop(); + void on_lease_start(int64_t expect_lease_epoch, int64_t last_active_timestamp); + void get_lease_info(LeaseInfo* lease_info); + void renew(int64_t last_active_timestamp); + int64_t lease_epoch(); + void reset_election_timeout_ms(int64_t election_timeout_ms); + +private: + raft_mutex_t _mutex; + int64_t _election_timeout_ms; + int64_t _last_active_timestamp; + int64_t _term; + int64_t _lease_epoch; +}; + +class FollowerLease { +public: + FollowerLease() + : _election_timeout_ms(0), _max_clock_drift_ms(0) + , _last_leader_timestamp(0) + {} + + void init(int64_t election_timeout_ms, int64_t max_clock_drift_ms); + void renew(const PeerId& leader_id); + int64_t votable_time_from_now(); + const PeerId& last_leader(); + bool expired(); + void reset(); + void reset_election_timeout_ms(int64_t election_timeout_ms, int64_t max_clock_drift_ms); + +private: + int64_t _election_timeout_ms; + int64_t _max_clock_drift_ms; + PeerId _last_leader; + int64_t _last_leader_timestamp; +}; + +} // namespace braft + +#endif // PUBLIC_RAFT_LEADER_LEASE_H diff --git a/src/braft/log.cpp b/src/braft/log.cpp index c4fc1dce..94eba838 100644 --- a/src/braft/log.cpp +++ b/src/braft/log.cpp @@ -44,6 +44,7 @@ namespace braft { using ::butil::RawPacker; using ::butil::RawUnpacker; +DECLARE_bool(raft_trace_append_entry_latency); DEFINE_int32(raft_max_segment_size, 8 * 1024 * 1024 /*8M*/, "Max size of one segment file"); BRPC_VALIDATE_GFLAG(raft_max_segment_size, brpc::PositiveInteger); @@ -51,6 +52,10 @@ BRPC_VALIDATE_GFLAG(raft_max_segment_size, brpc::PositiveInteger); DEFINE_bool(raft_sync_segments, false, "call fsync when a segment is closed"); BRPC_VALIDATE_GFLAG(raft_sync_segments, ::brpc::PassValidate); +static bvar::LatencyRecorder g_open_segment_latency("raft_open_segment"); +static bvar::LatencyRecorder g_segment_append_entry_latency("raft_segment_append_entry"); +static bvar::LatencyRecorder g_sync_segment_latency("raft_sync_segment"); + int ftruncate_uninterrupted(int fd, off_t length) { int rc = 0; do { @@ -701,7 +706,7 @@ int64_t SegmentLogStorage::last_log_index() { return _last_log_index.load(butil::memory_order_acquire); } -int SegmentLogStorage::append_entries(const std::vector& entries) { +int SegmentLogStorage::append_entries(const std::vector& entries, IOMetric* metric) { if (entries.empty()) { return 0; } @@ -712,10 +717,18 @@ int SegmentLogStorage::append_entries(const std::vector& entries) { return -1; } scoped_refptr last_segment = NULL; + int64_t now = 0; + int64_t delta_time_us = 0; for (size_t i = 0; i < entries.size(); i++) { + now = butil::cpuwide_time_us(); LogEntry* entry = entries[i]; - + scoped_refptr segment = open_segment(); + if (FLAGS_raft_trace_append_entry_latency && metric) { + delta_time_us = butil::cpuwide_time_us() - now; + metric->open_segment_time_us += delta_time_us; + g_open_segment_latency << delta_time_us; + } if (NULL == segment) { return i; } @@ -723,10 +736,21 @@ int SegmentLogStorage::append_entries(const std::vector& entries) { if (0 != ret) { return i; } + if (FLAGS_raft_trace_append_entry_latency && metric) { + delta_time_us = butil::cpuwide_time_us() - now; + metric->append_entry_time_us += delta_time_us; + g_segment_append_entry_latency << delta_time_us; + } _last_log_index.fetch_add(1, butil::memory_order_release); last_segment = segment; } + now = butil::cpuwide_time_us(); last_segment->sync(_enable_sync); + if (FLAGS_raft_trace_append_entry_latency && metric) { + delta_time_us = butil::cpuwide_time_us() - now; + metric->sync_segment_time_us += delta_time_us; + g_sync_segment_latency << delta_time_us; + } return entries.size(); } @@ -1229,4 +1253,16 @@ LogStorage* SegmentLogStorage::new_instance(const std::string& uri) const { return new SegmentLogStorage(uri); } +butil::Status SegmentLogStorage::gc_instance(const std::string& uri) const { + butil::Status status; + if (gc_dir(uri) != 0) { + LOG(WARNING) << "Failed to gc log storage from path " << _path; + status.set_error(EINVAL, "Failed to gc log storage from path %s", + uri.c_str()); + return status; + } + LOG(INFO) << "Succeed to gc log storage from path " << uri; + return status; +} + } diff --git a/src/braft/log.h b/src/braft/log.h index 6c9fa81d..1bb682fc 100644 --- a/src/braft/log.h +++ b/src/braft/log.h @@ -176,8 +176,8 @@ class SegmentLogStorage : public LogStorage { // append entry to log int append_entry(const LogEntry* entry); - // append entries to log, return success append number - virtual int append_entries(const std::vector& entries); + // append entries to log and update IOMetric, return success append number + virtual int append_entries(const std::vector& entries, IOMetric* metric); // delete logs from storage's head, [1, first_index_kept) will be discarded virtual int truncate_prefix(const int64_t first_index_kept); @@ -188,6 +188,8 @@ class SegmentLogStorage : public LogStorage { virtual int reset(const int64_t next_log_index); LogStorage* new_instance(const std::string& uri) const; + + butil::Status gc_instance(const std::string& uri) const; SegmentMap& segments() { return _segments; diff --git a/src/braft/log_manager.cpp b/src/braft/log_manager.cpp index b251557f..6c5aa275 100644 --- a/src/braft/log_manager.cpp +++ b/src/braft/log_manager.cpp @@ -48,6 +48,12 @@ static bvar::LatencyRecorder g_nomralized_append_entries_latency( static bvar::CounterRecorder g_storage_flush_batch_counter( "raft_storage_flush_batch_counter"); +void LogManager::StableClosure::update_metric(IOMetric* m) { + metric.open_segment_time_us = m->open_segment_time_us; + metric.append_entry_time_us = m->append_entry_time_us; + metric.sync_segment_time_us = m->sync_segment_time_us; +} + LogManagerOptions::LogManagerOptions() : log_storage(NULL) , configuration_manager(NULL) @@ -84,6 +90,8 @@ int LogManager::init(const LogManagerOptions &options) { _first_log_index = _log_storage->first_log_index(); _last_log_index = _log_storage->last_log_index(); _disk_id.index = _last_log_index; + // Term will be 0 if the node has no logs, and we will correct the value + // after snapshot load finish. _disk_id.term = _log_storage->get_term(_last_log_index); _fsm_caller = options.fsm_caller; return 0; @@ -436,7 +444,7 @@ void LogManager::append_entries( } void LogManager::append_to_storage(std::vector* to_append, - LogId* last_id) { + LogId* last_id, IOMetric* metric) { if (!_has_error.load(butil::memory_order_relaxed)) { size_t written_size = 0; for (size_t i = 0; i < to_append->size(); ++i) { @@ -444,7 +452,7 @@ void LogManager::append_to_storage(std::vector* to_append, } butil::Timer timer; timer.start(); - int nappent = _log_storage->append_entries(*to_append); + int nappent = _log_storage->append_entries(*to_append, metric); timer.stop(); if (nappent != (int)to_append->size()) { // FIXME @@ -487,7 +495,8 @@ class AppendBatcher { void flush() { if (_size > 0) { - _lm->append_to_storage(&_to_append, _last_id); + IOMetric metric; + _lm->append_to_storage(&_to_append, _last_id, &metric); g_storage_flush_batch_counter << _size; for (size_t i = 0; i < _size; ++i) { _storage[i]->_entries.clear(); @@ -495,6 +504,7 @@ class AppendBatcher { _storage[i]->status().set_error( EIO, "Corrupted LogStorage"); } + _storage[i]->update_metric(&metric); _storage[i]->Run(); } _to_append.clear(); @@ -541,6 +551,8 @@ int LogManager::disk_thread(void* meta, // ^^^ Must iterate to the end to release to corresponding // even if some error has occurred StableClosure* done = *iter; + done->metric.bthread_queue_time_us = butil::cpuwide_time_us() - + done->metric.start_time_us; if (!done->_entries.empty()) { ab.append(done); } else { @@ -631,6 +643,9 @@ void LogManager::set_snapshot(const SnapshotMeta* meta) { if (_last_snapshot_id > _applied_id) { _applied_id = _last_snapshot_id; } + if (_last_snapshot_id > _disk_id) { + _disk_id = _last_snapshot_id; + } if (term == 0) { // last_included_index is larger than last_index // FIXME: what if last_included_index is less than first_index? diff --git a/src/braft/log_manager.h b/src/braft/log_manager.h index 8dd4cef6..fcd52dc3 100644 --- a/src/braft/log_manager.h +++ b/src/braft/log_manager.h @@ -27,6 +27,7 @@ #include "braft/util.h" // raft_mutex_t #include "braft/log_entry.h" // LogEntry #include "braft/configuration_manager.h" // ConfigurationManager +#include "braft/storage.h" // Storage namespace braft { @@ -59,8 +60,10 @@ class BAIDU_CACHELINE_ALIGNMENT LogManager { class StableClosure : public Closure { public: StableClosure() : _first_log_index(0) {} + void update_metric(IOMetric* metric); protected: int64_t _first_log_index; + IOMetric metric; private: friend class LogManager; friend class AppendBatcher; @@ -133,7 +136,7 @@ class BAIDU_CACHELINE_ALIGNMENT LogManager { void set_applied_id(const LogId& applied_id); // Check the consistency between log and snapshot, which must satisfy ANY - // one of the follower condition + // one of the following condition // - Log starts from 1. OR // - Log starts from a positive position and there must be a snapshot // of which the last_included_id is in the range @@ -154,7 +157,7 @@ friend class AppendBatcher; int error_code; }; - void append_to_storage(std::vector* to_append, LogId* last_id); + void append_to_storage(std::vector* to_append, LogId* last_id, IOMetric* metric); static int disk_thread(void* meta, bthread::TaskIterator& iter); diff --git a/src/braft/memory_log.cpp b/src/braft/memory_log.cpp index 393550f4..40e20516 100644 --- a/src/braft/memory_log.cpp +++ b/src/braft/memory_log.cpp @@ -71,7 +71,8 @@ int MemoryLogStorage::append_entry(const LogEntry* input_entry) { return 0; } -int MemoryLogStorage::append_entries(const std::vector& entries) { +int MemoryLogStorage::append_entries(const std::vector& entries, + IOMetric* metric) { if (entries.empty()) { return 0; } @@ -158,4 +159,8 @@ LogStorage* MemoryLogStorage::new_instance(const std::string& uri) const { return new MemoryLogStorage(uri); } +butil::Status MemoryLogStorage::gc_instance(const std::string& uri) const { + return butil::Status::OK(); +} + } // namespace braft diff --git a/src/braft/memory_log.h b/src/braft/memory_log.h index 89e0d8a4..19901f74 100644 --- a/src/braft/memory_log.h +++ b/src/braft/memory_log.h @@ -63,8 +63,8 @@ class BAIDU_CACHELINE_ALIGNMENT MemoryLogStorage : public LogStorage { // append entries to log virtual int append_entry(const LogEntry* entry); - // append entries to log, return append success number - virtual int append_entries(const std::vector& entries); + // append entries to log and update IOMetric, return append success number + virtual int append_entries(const std::vector& entries, IOMetric* metric); // delete logs from storage's head, [first_log_index, first_index_kept) will be discarded virtual int truncate_prefix(const int64_t first_index_kept); @@ -81,6 +81,10 @@ class BAIDU_CACHELINE_ALIGNMENT MemoryLogStorage : public LogStorage { // Return the address referenced to the instance on success, NULL otherwise. virtual LogStorage* new_instance(const std::string& uri) const; + // GC an instance of this kind of LogStorage with the parameters encoded + // in |uri| + virtual butil::Status gc_instance(const std::string& uri) const; + private: std::string _path; butil::atomic _first_log_index; diff --git a/src/braft/node.cpp b/src/braft/node.cpp index 67dd6d19..8e2c4f6e 100644 --- a/src/braft/node.cpp +++ b/src/braft/node.cpp @@ -14,7 +14,7 @@ // Authors: Wang,Yao(wangyao02@baidu.com) // Zhangyi Chen(chenzhangyi01@baidu.com) -// Xiong,Kai(xionkai@baidu.com) +// Xiong,Kai(xiongkai@baidu.com) #include #include @@ -53,6 +53,14 @@ DEFINE_int32(raft_max_append_entries_cache_size, 8, "the max size of out-of-order append entries cache"); BRPC_VALIDATE_GFLAG(raft_max_append_entries_cache_size, ::brpc::PositiveInteger); +DEFINE_int64(raft_append_entry_high_lat_us, 1000 * 1000, + "append entry high latency us"); +BRPC_VALIDATE_GFLAG(raft_append_entry_high_lat_us, brpc::PositiveInteger); + +DEFINE_bool(raft_trace_append_entry_latency, false, + "trace append entry latency"); +BRPC_VALIDATE_GFLAG(raft_trace_append_entry_latency, brpc::PassValidate); + #ifndef UNIT_TEST static bvar::Adder g_num_nodes("raft_node_count"); #else @@ -80,6 +88,7 @@ class ConfigurationChangeDone : public Closure { if (status().ok()) { _node->on_configuration_change_done(_term); if (_leader_start) { + _node->leader_lease_start(_lease_epoch); _node->_options.fsm->on_leader_start(_term); } } @@ -87,10 +96,11 @@ class ConfigurationChangeDone : public Closure { } private: ConfigurationChangeDone( - NodeImpl* node, int64_t term, bool leader_start) + NodeImpl* node, int64_t term, bool leader_start, int64_t lease_epoch) : _node(node) , _term(term) , _leader_start(leader_start) + , _lease_epoch(lease_epoch) { _node->AddRef(); } @@ -102,6 +112,7 @@ friend class NodeImpl; NodeImpl* _node; int64_t _term; bool _leader_start; + int64_t _lease_epoch; }; inline int random_timeout(int timeout_ms) { @@ -119,6 +130,7 @@ NodeImpl::NodeImpl(const GroupId& group_id, const PeerId& peer_id) , _current_term(0) , _last_leader_timestamp(butil::monotonic_time_ms()) , _group_id(group_id) + , _server_id(peer_id) , _conf_ctx(this) , _log_storage(NULL) , _meta_storage(NULL) @@ -135,7 +147,7 @@ NodeImpl::NodeImpl(const GroupId& group_id, const PeerId& peer_id) , _append_entries_cache_version(0) , _node_readonly(false) , _majority_nodes_readonly(false) { - _server_id = peer_id; + butil::string_printf(&_v_group_id, "%s_%d", _group_id.c_str(), _server_id.idx); AddRef(); g_num_nodes << 1; } @@ -145,6 +157,7 @@ NodeImpl::NodeImpl() , _current_term(0) , _last_leader_timestamp(butil::monotonic_time_ms()) , _group_id() + , _server_id() , _conf_ctx(this) , _log_storage(NULL) , _meta_storage(NULL) @@ -156,8 +169,13 @@ NodeImpl::NodeImpl() , _snapshot_executor(NULL) , _stop_transfer_arg(NULL) , _vote_triggered(false) - , _waking_candidate(0) { - AddRef(); + , _waking_candidate(0) + , _append_entries_cache(NULL) + , _append_entries_cache_version(0) + , _node_readonly(false) + , _majority_nodes_readonly(false) { + butil::string_printf(&_v_group_id, "%s_%d", _group_id.c_str(), _server_id.idx); + AddRef(); g_num_nodes << 1; } @@ -185,17 +203,16 @@ NodeImpl::~NodeImpl() { delete _ballot_box; _ballot_box = NULL; } - + if (_closure_queue) { + delete _closure_queue; + _closure_queue = NULL; + } if (_options.node_owns_log_storage) { if (_log_storage) { delete _log_storage; _log_storage = NULL; } } - if (_closure_queue) { - delete _closure_queue; - _closure_queue = NULL; - } if (_meta_storage) { delete _meta_storage; _meta_storage = NULL; @@ -257,36 +274,37 @@ int NodeImpl::init_log_storage() { } int NodeImpl::init_meta_storage() { - int ret = 0; - - do { - _meta_storage = RaftMetaStorage::create(_options.raft_meta_uri); - if (!_meta_storage) { - LOG(WARNING) << "node " << _group_id << ":" << _server_id - << " find meta storage failed, uri " << _options.raft_meta_uri; - ret = ENOENT; - break; - } - - ret = _meta_storage->init(); - if (ret != 0) { - LOG(WARNING) << "node " << _group_id << ":" << _server_id - << " init meta storage failed, uri " << _options.raft_meta_uri - << " ret " << ret; - break; - } + // create stable storage + _meta_storage = RaftMetaStorage::create(_options.raft_meta_uri); + if (!_meta_storage) { + LOG(WARNING) << "node " << _group_id << ":" << _server_id + << " failed to create meta storage, uri " + << _options.raft_meta_uri; + return ENOENT; + } - _current_term = _meta_storage->get_term(); - ret = _meta_storage->get_votedfor(&_voted_id); - if (ret != 0) { - LOG(WARNING) << "node " << _group_id << ":" << _server_id - << " meta storage get_votedfor failed, uri " - << _options.raft_meta_uri << " ret " << ret; - break; - } - } while (0); + // check init + butil::Status status = _meta_storage->init(); + if (!status.ok()) { + LOG(WARNING) << "node " << _group_id << ":" << _server_id + << " failed to init meta storage, uri " + << _options.raft_meta_uri + << ", error " << status; + return status.error_code(); + } + + // get term and votedfor + status = _meta_storage-> + get_term_and_votedfor(&_current_term, &_voted_id, _v_group_id); + if (!status.ok()) { + LOG(WARNING) << "node " << _group_id << ":" << _server_id + << " failed to get term and voted_id when init meta storage," + << " uri " << _options.raft_meta_uri + << ", error " << status; + return status.error_code(); + } - return ret; + return 0; } void NodeImpl::handle_snapshot_timeout() { @@ -378,8 +396,12 @@ int NodeImpl::bootstrap(const BootstrapOptions& options) { } if (_current_term == 0) { _current_term = 1; - if (_meta_storage->set_term_and_votedfor(1, PeerId()) != 0) { - LOG(ERROR) << "Fail to set term"; + butil::Status status = _meta_storage-> + set_term_and_votedfor(1, PeerId(), _v_group_id); + if (!status.ok()) { + // TODO add group_id + LOG(ERROR) << "Fail to set term and votedfor when bootstrap," + " error: " << status; return -1; } return -1; @@ -438,14 +460,14 @@ int NodeImpl::init(const NodeOptions& options) { return -1; } - if (!NodeManager::GetInstance()->server_exists(_server_id.addr)) { + if (!global_node_manager->server_exists(_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; } - CHECK_EQ(0, _vote_timer.init(this, options.election_timeout_ms)); + CHECK_EQ(0, _vote_timer.init(this, options.election_timeout_ms + options.max_clock_drift_ms)); CHECK_EQ(0, _election_timer.init(this, options.election_timeout_ms)); CHECK_EQ(0, _stepdown_timer.init(this, options.election_timeout_ms)); CHECK_EQ(0, _snapshot_timer.init(this, options.snapshot_interval_s * 1000)); @@ -469,6 +491,9 @@ int NodeImpl::init(const NodeOptions& options) { // Create _fsm_caller first as log_manager needs it to report error _fsm_caller = new FSMCaller(); + _leader_lease.init(options.election_timeout_ms); + _follower_lease.init(options.election_timeout_ms, options.max_clock_drift_ms); + // log storage and log manager init if (init_log_storage() != 0) { LOG(ERROR) << "node " << _group_id << ":" << _server_id @@ -561,7 +586,7 @@ int NodeImpl::init(const NodeOptions& options) { } // add node to NodeManager - if (!NodeManager::GetInstance()->add(this)) { + if (!global_node_manager->add(this)) { LOG(ERROR) << "NodeManager add " << _group_id << ":" << _server_id << " failed"; return -1; @@ -686,7 +711,7 @@ void NodeImpl::on_caughtup(const PeerId& peer, int64_t term, // Retry if this peer is still alive if (st.error_code() == ETIMEDOUT && (butil::monotonic_time_ms() - - _replicator_group.last_rpc_send_timestamp(peer)) + - _replicator_group.last_rpc_send_timestamp(peer)) <= _options.election_timeout_ms) { LOG(INFO) << "node " << _group_id << ":" << _server_id @@ -902,7 +927,7 @@ void NodeImpl::shutdown(Closure* done) { if (_state < STATE_SHUTTING) { // Got the right to shut // Remove node from NodeManager and |this| would not be accessed by // the coming RPCs - NodeManager::GetInstance()->remove(this); + global_node_manager->remove(this); // if it is leader, set the wakeup_a_candidate with true, // if it is follower, call on_stop_following in step_down if (_state <= STATE_FOLLOWER) { @@ -976,10 +1001,9 @@ void NodeImpl::handle_election_timeout() { return; } - // check timestamp, skip one cycle check when trigger vote - if (!_vote_triggered && - (butil::monotonic_time_ms() - _last_leader_timestamp) - < _options.election_timeout_ms) { + // Trigger vote manually, or wait until follower lease expire. + if (!_vote_triggered && !_follower_lease.expired()) { + return; } _vote_triggered = false; @@ -1079,7 +1103,8 @@ void NodeImpl::handle_transfer_timeout(int64_t term, const PeerId& peer) { if (term == _current_term) { _replicator_group.stop_transfer_leadership(peer); if (_state == STATE_TRANSFERRING) { - _fsm_caller->on_leader_start(term); + _leader_lease.on_leader_start(term); + _fsm_caller->on_leader_start(term, _leader_lease.lease_epoch()); _state = STATE_LEADER; _stop_transfer_arg = NULL; } @@ -1145,6 +1170,7 @@ int NodeImpl::transfer_leadership_to(const PeerId& peer) { butil::Status status; status.set_error(ETRANSFERLEADERSHIP, "Raft leader is transferring " "leadership to %s", peer_id.to_string().c_str()); + _leader_lease.on_leader_stop(); _fsm_caller->on_leader_stop(status); LOG(INFO) << "node " << _group_id << ":" << _server_id << " starts to transfer leadership to " << peer_id; @@ -1160,33 +1186,73 @@ int NodeImpl::transfer_leadership_to(const PeerId& peer) { return 0; } -void NodeImpl::vote(int election_timeout) { +butil::Status NodeImpl::vote(int election_timeout_ms) { 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; + return butil::Status(EPERM, "is not follower"); } + int max_election_timeout_ms = _options.max_clock_drift_ms + _options.election_timeout_ms; + if (election_timeout_ms > max_election_timeout_ms) { + return butil::Status(EINVAL, "election_timeout_ms larger than safety threshold"); + } + election_timeout_ms = std::min(election_timeout_ms, max_election_timeout_ms); + int max_clock_drift_ms = max_election_timeout_ms - election_timeout_ms; + unsafe_reset_election_timeout_ms(election_timeout_ms, max_clock_drift_ms); _vote_triggered = true; + const int64_t saved_current_term = _current_term; + const State saved_state = _state; + lck.unlock(); + 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); + " current_term " << saved_current_term << " state " << state2str(saved_state) << + " election_timeout " << election_timeout_ms; + return butil::Status(); } -void NodeImpl::reset_election_timeout_ms(int election_timeout_ms) { +butil::Status NodeImpl::reset_election_timeout_ms(int election_timeout_ms) { std::unique_lock lck(_mutex); + int max_election_timeout_ms = _options.max_clock_drift_ms + _options.election_timeout_ms; + if (election_timeout_ms > max_election_timeout_ms) { + return butil::Status(EINVAL, "election_timeout_ms larger than safety threshold"); + } + election_timeout_ms = std::min(election_timeout_ms, max_election_timeout_ms); + int max_clock_drift_ms = max_election_timeout_ms - election_timeout_ms; + unsafe_reset_election_timeout_ms(election_timeout_ms, max_clock_drift_ms); + const int64_t saved_current_term = _current_term; + const State saved_state = _state; + lck.unlock(); + + LOG(INFO) << "node " << _group_id << ":" << _server_id << " reset_election_timeout," + " current_term " << saved_current_term << " state " << state2str(saved_state) << + " new election_timeout " << election_timeout_ms << " new clock_drift_ms " << + max_clock_drift_ms; + return butil::Status(); +} + +void NodeImpl::reset_election_timeout_ms(int election_timeout_ms, + int max_clock_drift_ms) { + std::unique_lock lck(_mutex); + unsafe_reset_election_timeout_ms(election_timeout_ms, max_clock_drift_ms); + const int64_t saved_current_term = _current_term; + const State saved_state = _state; + lck.unlock(); + + LOG(INFO) << "node " << _group_id << ":" << _server_id << " reset_election_timeout," + " current_term " << saved_current_term << " state " << state2str(saved_state) << + " new election_timeout " << election_timeout_ms << " new clock_drift_ms " << + max_clock_drift_ms; +} + +void NodeImpl::unsafe_reset_election_timeout_ms(int election_timeout_ms, + int max_clock_drift_ms) { _options.election_timeout_ms = election_timeout_ms; + _options.max_clock_drift_ms = max_clock_drift_ms; _replicator_group.reset_heartbeat_interval( heartbeat_timeout(_options.election_timeout_ms)); _replicator_group.reset_election_timeout_interval(_options.election_timeout_ms); - LOG(INFO) << "node " << _group_id << ":" << _server_id << " reset_election_timeout," - " current_term " << _current_term << " state " << state2str(_state) << - " new election_timeout " << election_timeout_ms; - _election_timer.reset(election_timeout_ms); + _leader_lease.reset_election_timeout_ms(election_timeout_ms); + _follower_lease.reset_election_timeout_ms(election_timeout_ms, _options.max_clock_drift_ms); } void NodeImpl::on_error(const Error& e) { @@ -1236,9 +1302,18 @@ void NodeImpl::handle_vote_timeout() { } void NodeImpl::handle_request_vote_response(const PeerId& peer_id, const int64_t term, + const int64_t ctx_version, const RequestVoteResponse& response) { BAIDU_SCOPED_LOCK(_mutex); + if (ctx_version != _vote_ctx.version()) { + LOG(WARNING) << "node " << _group_id << ":" << _server_id + << " received invalid RequestVoteResponse from " << peer_id + << " ctx_version " << ctx_version + << " current_ctx_version " << _vote_ctx.version(); + return; + } + // check state if (_state != STATE_CANDIDATE) { LOG(WARNING) << "node " << _group_id << ":" << _server_id @@ -1271,6 +1346,10 @@ void NodeImpl::handle_request_vote_response(const PeerId& peer_id, const int64_t // check if the quorum granted if (response.granted()) { _vote_ctx.grant(peer_id); + if (peer_id == _follower_lease.last_leader()) { + _vote_ctx.grant(_server_id); + _vote_ctx.stop_grant_self_timer(this); + } if (_vote_ctx.granted()) { become_leader(); } @@ -1278,9 +1357,10 @@ void NodeImpl::handle_request_vote_response(const PeerId& peer_id, const int64_t } struct OnRequestVoteRPCDone : public google::protobuf::Closure { - OnRequestVoteRPCDone(const PeerId& peer_id_, const int64_t term_, NodeImpl* node_) - : peer(peer_id_), term(term_), node(node_) { - node->AddRef(); + OnRequestVoteRPCDone(const PeerId& peer_id_, const int64_t term_, + const int64_t ctx_version_, NodeImpl* node_) + : peer(peer_id_), term(term_), ctx_version(ctx_version_), node(node_) { + node->AddRef(); } virtual ~OnRequestVoteRPCDone() { node->Release(); @@ -1294,13 +1374,14 @@ struct OnRequestVoteRPCDone : public google::protobuf::Closure { << " error: " << cntl.ErrorText(); break; } - node->handle_request_vote_response(peer, term, response); + node->handle_request_vote_response(peer, term, ctx_version, response); } while (0); delete this; } PeerId peer; int64_t term; + int64_t ctx_version; RequestVoteRequest request; RequestVoteResponse response; brpc::Controller cntl; @@ -1308,9 +1389,18 @@ struct OnRequestVoteRPCDone : public google::protobuf::Closure { }; void NodeImpl::handle_pre_vote_response(const PeerId& peer_id, const int64_t term, - const RequestVoteResponse& response) { + const int64_t ctx_version, + const RequestVoteResponse& response) { std::unique_lock lck(_mutex); + if (ctx_version != _pre_vote_ctx.version()) { + LOG(WARNING) << "node " << _group_id << ":" << _server_id + << " received invalid PreVoteResponse from " << peer_id + << " ctx_version " << ctx_version + << "current_ctx_version " << _pre_vote_ctx.version(); + return; + } + // check state if (_state != STATE_FOLLOWER) { LOG(WARNING) << "node " << _group_id << ":" << _server_id @@ -1343,6 +1433,10 @@ void NodeImpl::handle_pre_vote_response(const PeerId& peer_id, const int64_t ter // check if the quorum granted if (response.granted()) { _pre_vote_ctx.grant(peer_id); + if (peer_id == _follower_lease.last_leader()) { + _pre_vote_ctx.grant(_server_id); + _pre_vote_ctx.stop_grant_self_timer(this); + } if (_pre_vote_ctx.granted()) { elect_self(&lck); } @@ -1350,9 +1444,10 @@ void NodeImpl::handle_pre_vote_response(const PeerId& peer_id, const int64_t ter } struct OnPreVoteRPCDone : public google::protobuf::Closure { - OnPreVoteRPCDone(const PeerId& peer_id_, const int64_t term_, NodeImpl* node_) - : peer(peer_id_), term(term_), node(node_) { - node->AddRef(); + OnPreVoteRPCDone(const PeerId& peer_id_, const int64_t term_, + const int64_t ctx_version_, NodeImpl* node_) + : peer(peer_id_), term(term_), ctx_version(ctx_version_), node(node_) { + node->AddRef(); } virtual ~OnPreVoteRPCDone() { node->Release(); @@ -1366,13 +1461,14 @@ struct OnPreVoteRPCDone : public google::protobuf::Closure { << " error: " << cntl.ErrorText(); break; } - node->handle_pre_vote_response(peer, term, response); + node->handle_pre_vote_response(peer, term, ctx_version, response); } while (0); delete this; } PeerId peer; int64_t term; + int64_t ctx_version; RequestVoteRequest request; RequestVoteResponse response; brpc::Controller cntl; @@ -1407,9 +1503,10 @@ void NodeImpl::pre_vote(std::unique_lock* lck) { return; } - _pre_vote_ctx.init(_conf.conf, _conf.stable() ? NULL : &_conf.old_conf); + _pre_vote_ctx.init(this); std::set peers; _conf.list_peers(&peers); + for (std::set::const_iterator iter = peers.begin(); iter != peers.end(); ++iter) { if (*iter == _server_id) { @@ -1425,7 +1522,8 @@ void NodeImpl::pre_vote(std::unique_lock* lck) { continue; } - OnPreVoteRPCDone* done = new OnPreVoteRPCDone(*iter, _current_term, this); + OnPreVoteRPCDone* done = new OnPreVoteRPCDone( + *iter, _current_term, _pre_vote_ctx.version(), this); done->cntl.set_timeout_ms(_options.election_timeout_ms); done->request.set_group_id(_group_id); done->request.set_server_id(_server_id.to_string()); @@ -1437,11 +1535,7 @@ void NodeImpl::pre_vote(std::unique_lock* lck) { RaftService_Stub stub(&channel); stub.pre_vote(&done->cntl, &done->request, &done->response, done); } - _pre_vote_ctx.grant(_server_id); - - if (_pre_vote_ctx.granted()) { - elect_self(lck); - } + grant_self(&_pre_vote_ctx, lck); } // in lock @@ -1462,9 +1556,8 @@ void NodeImpl::elect_self(std::unique_lock* lck) { // reset leader_id before vote PeerId empty_id; butil::Status status; - status.set_error(ERAFTTIMEDOUT, - "A follower's leader_id is reset to NULL " - "as it begins to request_vote."); + status.set_error(ERAFTTIMEDOUT, "A follower's leader_id is reset to NULL " + "as it begins to request_vote."); reset_leader_id(empty_id, status); _state = STATE_CANDIDATE; @@ -1474,8 +1567,8 @@ void NodeImpl::elect_self(std::unique_lock* lck) { BRAFT_VLOG << "node " << _group_id << ":" << _server_id << " term " << _current_term << " start vote_timer"; _vote_timer.start(); - - _vote_ctx.init(_conf.conf, _conf.stable() ? NULL : &_conf.old_conf); + _pre_vote_ctx.reset(this); + _vote_ctx.init(this); int64_t old_term = _current_term; // get last_log_id outof node mutex @@ -1507,7 +1600,8 @@ void NodeImpl::elect_self(std::unique_lock* lck) { continue; } - OnRequestVoteRPCDone* done = new OnRequestVoteRPCDone(*iter, _current_term, this); + OnRequestVoteRPCDone* done = + new OnRequestVoteRPCDone(*iter, _current_term, _vote_ctx.version(), this); done->cntl.set_timeout_ms(_options.election_timeout_ms); done->request.set_group_id(_group_id); done->request.set_server_id(_server_id.to_string()); @@ -1521,11 +1615,17 @@ void NodeImpl::elect_self(std::unique_lock* lck) { } //TODO: outof lock - _meta_storage->set_term_and_votedfor(_current_term, _server_id); - _vote_ctx.grant(_server_id); - if (_vote_ctx.granted()) { - become_leader(); + status = _meta_storage-> + set_term_and_votedfor(_current_term, _server_id, _v_group_id); + if (!status.ok()) { + LOG(ERROR) << "node " << _group_id << ":" << _server_id + << " fail to set_term_and_votedfor itself when elect_self," + " error: " << status; + // reset _voted_id to avoid inconsistent cases + // return immediately without granting _vote_ctx + _voted_id.reset(); } + grant_self(&_vote_ctx, lck); } // in lock @@ -1543,13 +1643,16 @@ void NodeImpl::step_down(const int64_t term, bool wakeup_a_candidate, // delete timer and something else if (_state == STATE_CANDIDATE) { _vote_timer.stop(); + _vote_ctx.reset(this); + } else if (_state == STATE_FOLLOWER) { + _pre_vote_ctx.reset(this); } else if (_state <= STATE_TRANSFERRING) { _stepdown_timer.stop(); - _ballot_box->clear_pending_tasks(); // signal fsm leader stop immediately - if (_state == STATE_LEADER) { + if (_state == STATE_LEADER) { + _leader_lease.on_leader_stop(); _fsm_caller->on_leader_stop(status); } } @@ -1562,7 +1665,6 @@ void NodeImpl::step_down(const int64_t term, bool wakeup_a_candidate, _state = STATE_FOLLOWER; // _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(); @@ -1576,7 +1678,14 @@ void NodeImpl::step_down(const int64_t term, bool wakeup_a_candidate, _current_term = term; _voted_id.reset(); //TODO: outof lock - _meta_storage->set_term_and_votedfor(term, _voted_id); + butil::Status status = _meta_storage-> + set_term_and_votedfor(term, _voted_id, _v_group_id); + if (!status.ok()) { + LOG(ERROR) << "node " << _group_id << ":" << _server_id + << " fail to set_term_and_votedfor when step_down, error: " + << status; + // TODO report error + } } // stop stagging new node @@ -1615,6 +1724,7 @@ void NodeImpl::reset_leader_id(const PeerId& new_leader_id, _leader_id.reset(); } else { if (_leader_id.is_empty()) { + _pre_vote_ctx.reset(this); LeaderChangeContext start_following_context(new_leader_id, _current_term, status); _fsm_caller->on_start_following(start_following_context); @@ -1645,6 +1755,21 @@ void NodeImpl::check_step_down(const int64_t request_term, const PeerId& server_ } } +class LeaderStartClosure : public Closure { +public: + LeaderStartClosure(StateMachine* fsm, int64_t term) : _fsm(fsm), _term(term) {} + ~LeaderStartClosure() {} + void Run() { + if (status().ok()) { + _fsm->on_leader_start(_term); + } + delete this; + } +private: + StateMachine* _fsm; + int64_t _term; +}; + // in lock void NodeImpl::become_leader() { CHECK(_state == STATE_CANDIDATE); @@ -1654,11 +1779,14 @@ void NodeImpl::become_leader() { << " " << _conf.old_conf; // cancel candidate vote timer _vote_timer.stop(); + _vote_ctx.reset(this); _state = STATE_LEADER; _leader_id = _server_id; _replicator_group.reset_term(_current_term); + _follower_lease.reset(); + _leader_lease.on_leader_start(_current_term); std::set peers; _conf.list_peers(&peers); @@ -1713,6 +1841,18 @@ void LeaderStableClosure::Run() { _ballot_box->commit_at( _first_log_index, _first_log_index + _nentries - 1, _node_id.peer_id); } + int64_t now = butil::cpuwide_time_us(); + if (FLAGS_raft_trace_append_entry_latency && + now - metric.start_time_us > (int64_t)FLAGS_raft_append_entry_high_lat_us) { + LOG(WARNING) << "leader append entry latency us " << (now - metric.start_time_us) + << " greater than " + << FLAGS_raft_append_entry_high_lat_us + << metric + << " node " << _node_id + << " log_index [" << _first_log_index + << ", " << _first_log_index + _nentries - 1 + << "]"; + } } else { LOG(ERROR) << "node " << _node_id << " append [" << _first_log_index << ", " << _first_log_index + _nentries - 1 << "] failed"; @@ -1792,7 +1932,7 @@ void NodeImpl::unsafe_apply_configuration(const Configuration& new_conf, old_conf->list_peers(entry->old_peers); } ConfigurationChangeDone* configuration_change_done = - new ConfigurationChangeDone(this, _current_term, leader_start); + new ConfigurationChangeDone(this, _current_term, leader_start, _leader_lease.lease_epoch()); // Use the new_conf to deal the quorum of this very log _ballot_box->append_pending_task(new_conf, old_conf, configuration_change_done); @@ -1830,12 +1970,14 @@ int NodeImpl::handle_pre_vote_request(const RequestVoteRequest* request, bool granted = false; do { - if (request->term() < _current_term) { + int64_t votable_time = _follower_lease.votable_time_from_now(); + if (request->term() < _current_term || votable_time > 0) { // ignore older term LOG(INFO) << "node " << _group_id << ":" << _server_id << " ignore PreVote from " << request->server_id() << " in term " << request->term() - << " current_term " << _current_term; + << " current_term " << _current_term + << " votable_time_from_now " << votable_time; break; } @@ -1886,7 +2028,8 @@ int NodeImpl::handle_request_vote_request(const RequestVoteRequest* request, do { // check term - if (request->term() >= _current_term) { + int64_t votable_time = _follower_lease.votable_time_from_now(); + if (request->term() >= _current_term && votable_time == 0) { LOG(INFO) << "node " << _group_id << ":" << _server_id << " received RequestVote from " << request->server_id() << " in term " << request->term() @@ -1903,7 +2046,8 @@ int NodeImpl::handle_request_vote_request(const RequestVoteRequest* request, LOG(INFO) << "node " << _group_id << ":" << _server_id << " ignore RequestVote from " << request->server_id() << " in term " << request->term() - << " current_term " << _current_term; + << " current_term " << _current_term + << " votable_time_from_now " << votable_time; break; } @@ -1928,7 +2072,16 @@ int NodeImpl::handle_request_vote_request(const RequestVoteRequest* request, step_down(request->term(), false, status); _voted_id = candidate_id; //TODO: outof lock - _meta_storage->set_votedfor(candidate_id); + status = _meta_storage-> + set_term_and_votedfor(_current_term, candidate_id, _v_group_id); + if (!status.ok()) { + LOG(ERROR) << "node " << _group_id << ":" << _server_id + << " refuse to vote for " << request->server_id() + << " because failed to set_votedfor it, error: " + << status; + // reset _voted_id to response set_granted(false) + _voted_id.reset(); + } } } while (0); @@ -2009,8 +2162,20 @@ class FollowerStableClosure : public LogManager::StableClosure { // untrustable so we can't commit them even if their // indexes are less than request->committed_index() ); - //_ballot_box is thread safe and tolerats disorder. + //_ballot_box is thread safe and tolerates disorder. _node->_ballot_box->set_last_committed_index(committed_index); + int64_t now = butil::cpuwide_time_us(); + if (FLAGS_raft_trace_append_entry_latency && now - metric.start_time_us > + (int64_t)FLAGS_raft_append_entry_high_lat_us) { + LOG(WARNING) << "follower append entry latency us " << (now - metric.start_time_us) + << " greater than " + << FLAGS_raft_append_entry_high_lat_us + << metric + << " node " << _node->node_id() + << " log_index [" << _request->prev_log_index() + 1 + << ", " << _request->prev_log_index() + _request->entries_size() - 1 + << "]"; + } } brpc::Controller* _cntl; @@ -2090,7 +2255,7 @@ void NodeImpl::handle_append_entries_request(brpc::Controller* cntl, if (!from_append_entries_cache) { // Requests from cache already updated timestamp - _last_leader_timestamp = butil::monotonic_time_ms(); + _follower_lease.renew(_leader_id); } if (request->entries_size() > 0 && @@ -3058,6 +3223,198 @@ void NodeImpl::check_majority_nodes_readonly(const Configuration& conf) { } } +bool NodeImpl::is_leader_lease_valid() { + LeaderLeaseStatus lease_status; + get_leader_lease_status(&lease_status); + return lease_status.state == LEASE_VALID; +} + +void NodeImpl::get_leader_lease_status(LeaderLeaseStatus* lease_status) { + // Fast path for leader to lease check + LeaderLease::LeaseInfo internal_info; + _leader_lease.get_lease_info(&internal_info); + switch (internal_info.state) { + case LeaderLease::DISABLED: + lease_status->state = LEASE_DISABLED; + return; + case LeaderLease::EXPIRED: + lease_status->state = LEASE_EXPIRED; + return; + case LeaderLease::NOT_READY: + lease_status->state = LEASE_NOT_READY; + return; + case LeaderLease::VALID: + lease_status->term = internal_info.term; + lease_status->lease_epoch = internal_info.lease_epoch; + lease_status->state = LEASE_VALID; + return; + case LeaderLease::SUSPECT: + // Need do heavy check to judge if a lease still valid. + break; + } + + BAIDU_SCOPED_LOCK(_mutex); + if (_state != STATE_LEADER) { + lease_status->state = LEASE_EXPIRED; + return; + } + int64_t last_active_timestamp = last_leader_active_timestamp(); + _leader_lease.renew(last_active_timestamp); + _leader_lease.get_lease_info(&internal_info); + if (internal_info.state != LeaderLease::VALID && internal_info.state != LeaderLease::DISABLED) { + butil::Status status; + status.set_error(ERAFTTIMEDOUT, "Leader lease expired"); + step_down(_current_term, false, status); + lease_status->state = LEASE_EXPIRED; + } else if (internal_info.state == LeaderLease::VALID) { + lease_status->term = internal_info.term; + lease_status->lease_epoch = internal_info.lease_epoch; + lease_status->state = LEASE_VALID; + } else { + lease_status->state = LEASE_DISABLED; + } +} + +void NodeImpl::VoteBallotCtx::init(NodeImpl* node) { + ++_version; + _ballot.init(node->_conf.conf, node->_conf.stable() ? NULL : &(node->_conf.old_conf)); + stop_grant_self_timer(node); +} + +void NodeImpl::VoteBallotCtx::start_grant_self_timer(int64_t wait_ms, NodeImpl* node) { + timespec duetime = butil::milliseconds_from_now(wait_ms); + GrantSelfArg* timer_arg = new GrantSelfArg; + timer_arg->node = node; + timer_arg->vote_ctx_version = _version; + timer_arg->vote_ctx = this; + node->AddRef(); + _grant_self_arg = timer_arg; + if (bthread_timer_add( + &_timer, duetime, NodeImpl::on_grant_self_timedout, + timer_arg) != 0) { + LOG(ERROR) << "Fail to add timer"; + delete timer_arg; + _grant_self_arg = NULL; + node->Release(); + } +} + +void NodeImpl::VoteBallotCtx::stop_grant_self_timer(NodeImpl* node) { + if (_timer == bthread_timer_t()) { + return; + } + if (bthread_timer_del(_timer) == 0) { + node->Release(); + delete _grant_self_arg; + _grant_self_arg = NULL; + _timer = bthread_timer_t(); + } +} + +void NodeImpl::VoteBallotCtx::reset(NodeImpl* node) { + stop_grant_self_timer(node); + ++_version; +} + +void NodeImpl::grant_self(VoteBallotCtx* vote_ctx, std::unique_lock* lck) { + // If follower lease expired, we can safely grant self. Otherwise, we wait util: + // 1. last active leader vote the node, and we grant two votes together; + // 2. follower lease expire. + int64_t wait_ms = _follower_lease.votable_time_from_now(); + if (wait_ms == 0) { + vote_ctx->grant(_server_id); + if (!vote_ctx->granted()) { + return; + } + if (vote_ctx == &_pre_vote_ctx) { + elect_self(lck); + } else { + become_leader(); + } + return; + } + vote_ctx->start_grant_self_timer(wait_ms, this); +} + +void NodeImpl::on_grant_self_timedout(void* arg) { + bthread_t tid; + if (bthread_start_background( + &tid, NULL, NodeImpl::handle_grant_self_timedout, + arg) != 0) { + PLOG(ERROR) << "Fail to start bthread"; + NodeImpl::handle_grant_self_timedout(arg); + } +} + +void* NodeImpl::handle_grant_self_timedout(void* arg) { + GrantSelfArg* grant_arg = (GrantSelfArg*)arg; + NodeImpl* node = grant_arg->node; + VoteBallotCtx* vote_ctx = grant_arg->vote_ctx; + int64_t vote_ctx_version = grant_arg->vote_ctx_version; + + delete grant_arg; + + std::unique_lock lck(node->_mutex); + if (!is_active_state(node->_state) || + vote_ctx->version() != vote_ctx_version) { + lck.unlock(); + node->Release(); + return NULL; + } + node->grant_self(vote_ctx, &lck); + lck.unlock(); + node->Release(); + return NULL; +} + +void NodeImpl::leader_lease_start(int64_t lease_epoch) { + BAIDU_SCOPED_LOCK(_mutex); + if (_state == STATE_LEADER) { + _leader_lease.on_lease_start( + lease_epoch, last_leader_active_timestamp()); + } +} + +int64_t NodeImpl::last_leader_active_timestamp() { + int64_t timestamp = last_leader_active_timestamp(_conf.conf); + if (!_conf.old_conf.empty()) { + timestamp = std::min(timestamp, last_leader_active_timestamp(_conf.old_conf)); + } + return timestamp; +} + +struct LastActiveTimestampCompare { + bool operator()(const int64_t& a, const int64_t& b) { + return a > b; + } +}; + +int64_t NodeImpl::last_leader_active_timestamp(const Configuration& conf) { + std::vector peers; + conf.list_peers(&peers); + std::vector last_rpc_send_timestamps; + LastActiveTimestampCompare compare; + for (size_t i = 0; i < peers.size(); i++) { + if (peers[i] == _server_id) { + continue; + } + + int64_t timestamp = _replicator_group.last_rpc_send_timestamp(peers[i]); + last_rpc_send_timestamps.push_back(timestamp); + std::push_heap(last_rpc_send_timestamps.begin(), last_rpc_send_timestamps.end(), compare); + if (last_rpc_send_timestamps.size() > peers.size() / 2) { + std::pop_heap(last_rpc_send_timestamps.begin(), last_rpc_send_timestamps.end(), compare); + last_rpc_send_timestamps.pop_back(); + } + } + // Only one peer in the group. + if (last_rpc_send_timestamps.empty()) { + return butil::monotonic_time_ms(); + } + std::pop_heap(last_rpc_send_timestamps.begin(), last_rpc_send_timestamps.end(), compare); + return last_rpc_send_timestamps.back(); +} + // 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 8bf60cc7..e399aead 100644 --- a/src/braft/node.h +++ b/src/braft/node.h @@ -14,7 +14,7 @@ // Authors: Wang,Yao(wangyao02@baidu.com) // Zhangyi Chen(chenzhangyi01@baidu.com) -// Xiong,Kai(xionkai@baidu.com) +// Xiong,Kai(xiongkai@baidu.com) #ifndef BRAFT_RAFT_NODE_H #define BRAFT_RAFT_NODE_H @@ -91,6 +91,7 @@ friend class RaftServiceImpl; friend class RaftStatImpl; friend class FollowerStableClosure; friend class ConfigurationChangeDone; +friend class VoteBallotCtx; public: NodeImpl(const GroupId& group_id, const PeerId& peer_id); NodeImpl(); @@ -145,10 +146,11 @@ friend class ConfigurationChangeDone; void snapshot(Closure* done); // trigger vote - void vote(int election_timeout); + butil::Status vote(int election_timeout); // reset the election_timeout for the very node - void reset_election_timeout_ms(int election_timeout_ms); + butil::Status reset_election_timeout_ms(int election_timeout_ms); + void reset_election_timeout_ms(int election_timeout_ms, int max_clock_drift_ms); // rpc request proc func // @@ -186,8 +188,10 @@ friend class ConfigurationChangeDone; // Closure call func // void handle_pre_vote_response(const PeerId& peer_id, const int64_t term, - const RequestVoteResponse& response); + const int64_t ctx_version, + const RequestVoteResponse& response); void handle_request_vote_response(const PeerId& peer_id, const int64_t term, + const int64_t ctx_version, const RequestVoteResponse& response); void on_caughtup(const PeerId& peer, int64_t term, int64_t version, const butil::Status& st); @@ -196,6 +200,9 @@ friend class ConfigurationChangeDone; // called when leader change configuration done, ref with FSMCaller void on_configuration_change_done(int64_t term); + // Called when leader lease is safe to start. + void leader_lease_start(int64_t lease_epoch); + // called when leader recv greater term in AppendEntriesResponse, ref with Replicator int increase_term_to(int64_t new_term, const butil::Status& status); @@ -216,6 +223,10 @@ friend class ConfigurationChangeDone; void check_majority_nodes_readonly(); void check_majority_nodes_readonly(const Configuration& conf); + // Lease func + bool is_leader_lease_valid(); + void get_leader_lease_status(LeaderLeaseStatus* status); + // 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. @@ -269,6 +280,12 @@ friend class butil::RefCountedThreadSafe; // elect self to candidate void elect_self(std::unique_lock* lck); + // grant self a vote + class VoteBallotCtx; + void grant_self(VoteBallotCtx* vote_ctx, std::unique_lock* lck); + static void on_grant_self_timedout(void* arg); + static void* handle_grant_self_timedout(void* arg); + // leader async apply configuration void unsafe_apply_configuration(const Configuration& new_conf, const Configuration* old_conf, @@ -298,6 +315,11 @@ friend class butil::RefCountedThreadSafe; static void on_append_entries_cache_timedout(void* arg); static void* handle_append_entries_cache_timedout(void* arg); + int64_t last_leader_active_timestamp(); + int64_t last_leader_active_timestamp(const Configuration& conf); + void unsafe_reset_election_timeout_ms(int election_timeout_ms, + int max_clock_drift_ms); + private: class ConfigurationCtx { @@ -408,16 +430,51 @@ friend class butil::RefCountedThreadSafe; int64_t _timer_version; }; + // A versioned ballot for vote and prevote + struct GrantSelfArg; + class VoteBallotCtx { + public: + VoteBallotCtx() : _timer(bthread_timer_t()), _version(0), _grant_self_arg(NULL) { + } + void init(NodeImpl* node); + void grant(const PeerId& peer) { + _ballot.grant(peer); + } + bool granted() { + return _ballot.granted(); + } + int64_t version() { + return _version; + } + void start_grant_self_timer(int64_t wait_ms, NodeImpl* node); + void stop_grant_self_timer(NodeImpl* node); + void reset(NodeImpl* node); + private: + bthread_timer_t _timer; + Ballot _ballot; + // Each time the vote ctx restarted, increase the version to avoid + // ABA problem. + int64_t _version; + GrantSelfArg* _grant_self_arg; + }; + + struct GrantSelfArg { + NodeImpl* node; + int64_t vote_ctx_version; + VoteBallotCtx* vote_ctx; + }; + State _state; int64_t _current_term; int64_t _last_leader_timestamp; PeerId _leader_id; PeerId _voted_id; - Ballot _vote_ctx; - Ballot _pre_vote_ctx; + VoteBallotCtx _vote_ctx; // candidate vote ctx + VoteBallotCtx _pre_vote_ctx; // prevote ctx ConfigurationEntry _conf; GroupId _group_id; + VersionedGroupId _v_group_id; PeerId _server_id; NodeOptions _options; @@ -449,6 +506,9 @@ friend class butil::RefCountedThreadSafe; // for readonly mode bool _node_readonly; bool _majority_nodes_readonly; + + LeaderLease _leader_lease; + FollowerLease _follower_lease; }; } diff --git a/src/braft/node_manager.h b/src/braft/node_manager.h index e8dd7f95..5d443995 100644 --- a/src/braft/node_manager.h +++ b/src/braft/node_manager.h @@ -82,6 +82,8 @@ class NodeManager { std::set _addr_set; }; +#define global_node_manager NodeManager::GetInstance() + } // namespace braft #endif // BRAFT_NODE_MANAGER_H diff --git a/src/braft/raft.cpp b/src/braft/raft.cpp index 93b3f1c4..6069f706 100644 --- a/src/braft/raft.cpp +++ b/src/braft/raft.cpp @@ -48,7 +48,14 @@ static pthread_once_t global_init_once = PTHREAD_ONCE_INIT; struct GlobalExtension { SegmentLogStorage local_log; MemoryLogStorage memory_log; - LocalRaftMetaStorage local_meta; + + // manage only one raft instance + FileBasedSingleMetaStorage single_meta; + // manage a batch of raft instances + KVBasedMergedMetaStorage merged_meta; + // mix two types for double write when upgrade and downgrade + MixedMetaStorage mixed_meta; + LocalSnapshotStorage local_snapshot; }; @@ -57,7 +64,17 @@ static void global_init_or_die_impl() { log_storage_extension()->RegisterOrDie("local", &s_ext.local_log); log_storage_extension()->RegisterOrDie("memory", &s_ext.memory_log); - meta_storage_extension()->RegisterOrDie("local", &s_ext.local_meta); + + // uri = local://{single_path} + // |single_path| usually ends with `/meta' + // NOTICE: not change "local" to "local-single" because of compatibility + meta_storage_extension()->RegisterOrDie("local", &s_ext.single_meta); + // uri = local-merged://{merged_path} + // |merged_path| usually ends with `/merged_meta' + meta_storage_extension()->RegisterOrDie("local-merged", &s_ext.merged_meta); + // uri = local-mixed://merged_path={merged_path}&&single_path={single_path} + meta_storage_extension()->RegisterOrDie("local-mixed", &s_ext.mixed_meta); + snapshot_storage_extension()->RegisterOrDie("local", &s_ext.local_snapshot); } @@ -71,13 +88,14 @@ void global_init_once_or_die() { int add_service(brpc::Server* server, const butil::EndPoint& listen_addr) { global_init_once_or_die(); - return NodeManager::GetInstance()->add_service(server, listen_addr); + return global_node_manager->add_service(server, listen_addr); } int add_service(brpc::Server* server, int port) { butil::EndPoint addr(butil::IP_ANY, port); return add_service(server, addr); } + int add_service(brpc::Server* server, const char* listen_ip_and_port) { butil::EndPoint addr; if (butil::str2endpoint(listen_ip_and_port, &addr) != 0) { @@ -87,6 +105,34 @@ int add_service(brpc::Server* server, const char* listen_ip_and_port) { return add_service(server, addr); } +// GC +int gc_raft_data(const GCOptions& gc_options) { + const VersionedGroupId vgid = gc_options.vgid; + const std::string log_uri = gc_options.log_uri; + const std::string raft_meta_uri = gc_options.raft_meta_uri; + const std::string snapshot_uri = gc_options.snapshot_uri; + bool is_success = true; + + butil::Status status = LogStorage::destroy(log_uri); + if (!status.ok()) { + is_success = false; + LOG(WARNING) << "Group " << vgid << " failed to gc raft log, uri " << log_uri; + } + // TODO encode vgid into raft_meta_uri ? + status = RaftMetaStorage::destroy(raft_meta_uri, vgid); + if (!status.ok()) { + is_success = false; + LOG(WARNING) << "Group " << vgid << " failed to gc raft stable, uri " << raft_meta_uri; + } + status = SnapshotStorage::destroy(snapshot_uri); + if (!status.ok()) { + is_success = false; + LOG(WARNING) << "Group " << vgid << " failed to gc raft snapshot, uri " << snapshot_uri; + } + return is_success ? 0 : -1; +} + +// ------------- Node Node::Node(const GroupId& group_id, const PeerId& peer_id) { _impl = new NodeImpl(group_id, peer_id); } @@ -112,6 +158,14 @@ bool Node::is_leader() { return _impl->is_leader(); } +bool Node::is_leader_lease_valid() { + return _impl->is_leader_lease_valid(); +} + +void Node::get_leader_lease_status(LeaderLeaseStatus* status) { + return _impl->get_leader_lease_status(status); +} + int Node::init(const NodeOptions& options) { return _impl->init(options); } @@ -152,12 +206,16 @@ void Node::snapshot(Closure* done) { _impl->snapshot(done); } -void Node::vote(int election_timeout) { - _impl->vote(election_timeout); +butil::Status Node::vote(int election_timeout) { + return _impl->vote(election_timeout); +} + +butil::Status Node::reset_election_timeout_ms(int election_timeout_ms) { + return _impl->reset_election_timeout_ms(election_timeout_ms); } -void Node::reset_election_timeout_ms(int election_timeout_ms) { - _impl->reset_election_timeout_ms(election_timeout_ms); +void Node::reset_election_timeout_ms(int election_timeout_ms, int max_clock_drift_ms) { + _impl->reset_election_timeout_ms(election_timeout_ms, max_clock_drift_ms); } int Node::transfer_leadership_to(const PeerId& peer) { @@ -211,7 +269,7 @@ void Iterator::set_error_and_rollback(size_t ntail, const butil::Status* st) { return _impl->set_error_and_rollback(ntail, st); } -// ----------------- Default Implementation of StateMachine +// ------------- Default Implementation of StateMachine StateMachine::~StateMachine() {} void StateMachine::on_shutdown() {} diff --git a/src/braft/raft.h b/src/braft/raft.h index af56ab12..8d785525 100644 --- a/src/braft/raft.h +++ b/src/braft/raft.h @@ -373,9 +373,7 @@ struct PeerStatus { }; // Status of Node -class NodeStatus { -friend class NodeImpl; -public: +struct NodeStatus { typedef std::map PeerStatusMap; NodeStatus() @@ -433,12 +431,59 @@ friend class NodeImpl; PeerStatusMap unstable_followers; }; +// State of a lease. Following is a typical lease state change diagram: +// +// event: become leader become follower +// ^ on leader start ^ on leader stop +// | ^ | ^ +// time: ----------|-----------|-----------------|---|------- +// lease state: EXPIRED | NOT_READY | VALID | EXPIRED +// +enum LeaseState { + // Lease is disabled, this state will only be returned when + // |raft_enable_leader_lease == false|. + LEASE_DISABLED = 1, + + // Lease is expired, this node is not leader any more. + LEASE_EXPIRED = 2, + + // This node is leader, but we are not sure the data is up to date. This state + // continue until |on_leader_start| or the leader step down. + LEASE_NOT_READY = 3, + + // Lease is valid. + LEASE_VALID = 4, +}; + +// Status of a leader lease. +struct LeaderLeaseStatus { + LeaderLeaseStatus() + : state(LEASE_DISABLED), term(0), lease_epoch(0) + {} + + LeaseState state; + + // These followering fields are only meaningful when |state == LEASE_VALID|. + + // The term of this lease + int64_t term; + + // A specific term may have more than one lease, when transfer leader timeout + // happen. Lease epoch will be guranteed to be monotinically increase, in the + // life cycle of a node. + int64_t lease_epoch; +}; + struct NodeOptions { // A follower would become a candidate if it doesn't receive any message // from the leader in |election_timeout_ms| milliseconds // Default: 1000 (1s) int election_timeout_ms; //follower to candidate timeout + // Max clock drift time. It will be used to keep the safety of leader lease. + // Default: 1000 (1s) + int max_clock_drift_ms; + // A snapshot saving would be triggered every |snapshot_interval_s| seconds // if this was reset as a positive number // If |snapshot_interval_s| <= 0, the time based snapshot would be disabled. @@ -461,6 +506,11 @@ struct NodeOptions { // Default: A empty group Configuration initial_conf; + // Run the user callbacks and user closures in pthread rather than bthread + // + // Default: false + bool usercode_in_pthread; + // The specific StateMachine implemented your business logic, which must be // a valid instance. StateMachine* fsm; @@ -471,27 +521,45 @@ 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 - bool usercode_in_pthread; + // 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; // Describe a specific LogStorage in format ${type}://${parameters} + // It's valid iff |log_storage| is null std::string log_uri; // Describe a specific RaftMetaStorage in format ${type}://${parameters} + // Three types are provided up till now: + // 1. type=local + // FileBasedSingleMetaStorage(old name is LocalRaftMetaStorage) will be + // used, which is based on protobuf file and manages stable meta of + // only one Node + // typical format: local://${node_path} + // 2. type=local-merged + // KVBasedMergedMetaStorage will be used, whose under layer is based + // on KV storage and manages a batch of Nodes one the same disk. It's + // designed to solve performance problems caused by lots of small + // synchronous IO during leader electing, when there are huge number of + // Nodes in Multi-raft situation. + // typical format: local-merged://${disk_path} + // 3. type=local-mixed + // MixedMetaStorage will be used, which will double write the above + // two types of meta storages when upgrade an downgrade. + // typical format: + // local-mixed://merged_path=${disk_path}&&single_path=${node_path} + // + // Upgrade and Downgrade steps: + // upgrade from Single to Merged: local -> mixed -> merged + // downgrade from Merged to Single: merged -> mixed -> local std::string raft_meta_uri; // Describe a specific SnapshotStorage in format ${type}://${parameters} @@ -507,7 +575,7 @@ struct NodeOptions { // Default: NULL scoped_refptr* snapshot_file_system_adaptor; - // If non-null, we will pass this throughput_snapshot_throttle to SnapshotExecutor + // If non-null, we will pass this snapshot_throttle to SnapshotExecutor // Default: NULL scoped_refptr* snapshot_throttle; @@ -521,13 +589,14 @@ struct NodeOptions { inline NodeOptions::NodeOptions() : election_timeout_ms(1000) + , max_clock_drift_ms(1000) , snapshot_interval_s(3600) , catchup_margin(1000) + , usercode_in_pthread(false) , fsm(NULL) , node_owns_fsm(false) - , node_owns_log_storage(true) , log_storage(NULL) - , usercode_in_pthread(false) + , node_owns_log_storage(true) , filter_before_copy_remote(false) , snapshot_file_system_adaptor(NULL) , snapshot_throttle(NULL) @@ -549,6 +618,18 @@ class Node { // Return true if this is the leader of the belonging group bool is_leader(); + // Return true if this is the leader, and leader lease is valid. It's always + // false when |raft_enable_leader_lease == false|. + // In the follwing situations, the returned true is unbeleivable: + // - Not all nodes in the raft group set |raft_enable_leader_lease| to true, + // and tranfer leader/vote interfaces are used; + // - In the raft group, the value of |election_timeout_ms| in one node is larger + // than |election_timeout_ms + max_clock_drift_ms| in another peer. + bool is_leader_lease_valid(); + + // Get leader lease status for more complex checking + void get_leader_lease_status(LeaderLeaseStatus* status); + // init node int init(const NodeOptions& options); @@ -607,10 +688,31 @@ class Node { // 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); + butil::Status vote(int election_timeout); + + // Reset the |election_timeout_ms| for the very node, the |max_clock_drift_ms| + // is also adjusted to keep the sum of |election_timeout_ms| and |the max_clock_drift_ms| + // unchanged. + butil::Status reset_election_timeout_ms(int election_timeout_ms); + + // Forcely reset |election_timeout_ms| and |max_clock_drift_ms|. It may break + // leader lease safety, should be careful. + // Following are suggestions for you to change |election_timeout_ms| safely. + // 1. Three steps to safely upgrade |election_timeout_ms| to a larger one: + // - Enlarge |max_clock_drift_ms| in all peers to make sure + // |old election_timeout_ms + new max_clock_drift_ms| larger than + // |new election_timeout_ms + old max_clock_drift_ms|. + // - Wait at least |old election_timeout_ms + new max_clock_drift_ms| times to make + // sure all previous elections complete. + // - Upgrade |election_timeout_ms| to new one, meanwhiles |max_clock_drift_ms| + // can set back to the old value. + // 2. Three steps to safely upgrade |election_timeout_ms| to a smaller one: + // - Adjust |election_timeout_ms| and |max_clock_drift_ms| at the same time, + // to make the sum of |election_timeout_ms + max_clock_drift_ms| unchanged. + // - Wait at least |election_timeout_ms + max_clock_drift_ms| times to make + // sure all previous elections complete. + // - Upgrade |max_clock_drift_ms| back to the old value. + void reset_election_timeout_ms(int election_timeout_ms, int max_clock_drift_ms); // Try transferring leadership to |peer|. // If peer is ANY_PEER, a proper follower will be chosen as the leader for @@ -635,7 +737,7 @@ class Node { // 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 + // For example, 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 @@ -718,6 +820,26 @@ int add_service(brpc::Server* server, const butil::EndPoint& listen_addr); int add_service(brpc::Server* server, int port); int add_service(brpc::Server* server, const char* listen_ip_and_port); +// GC +struct GCOptions { + // Versioned-groupid of this raft instance. + // Version is necessary because instance with the same groupid may be created + // again very soon after destroyed. + VersionedGroupId vgid; + std::string log_uri; + std::string raft_meta_uri; + std::string snapshot_uri; +}; + +// TODO What if a disk is dropped and added again without released from +// global_mss_manager? It seems ok because all the instance on that disk would +// be destroyed before dropping the disk itself, so there would be no garbage. +// +// GC the data of a raft instance when destroying the instance by some reason. +// +// Returns 0 on success, -1 otherwise. +int gc_raft_data(const GCOptions& gc_options); + } // namespace braft #endif //BRAFT_RAFT_H diff --git a/src/braft/raft_meta.cpp b/src/braft/raft_meta.cpp index 8b0fbc3b..8f9e6fea 100644 --- a/src/braft/raft_meta.cpp +++ b/src/braft/raft_meta.cpp @@ -13,11 +13,14 @@ // limitations under the License. // Authors: Wang,Yao(wangyao02@baidu.com) +// Xiong,Kai(xiongkai@baidu.com) #include #include #include #include // butil::CreateDirectory +#include +#include #include "braft/util.h" #include "braft/protobuf_file.h" #include "braft/local_storage.pb.h" @@ -25,69 +28,427 @@ namespace braft { -const char* LocalRaftMetaStorage::_s_raft_meta = "raft_meta"; +static bvar::LatencyRecorder g_load_pb_raft_meta("raft_load_pb_raft_meta"); +static bvar::LatencyRecorder g_save_pb_raft_meta("raft_save_pb_raft_meta"); +static bvar::LatencyRecorder g_load_kv_raft_meta("raft_load_kv_raft_meta"); +static bvar::LatencyRecorder g_save_kv_raft_meta("raft_save_kv_raft_meta"); +static bvar::LatencyRecorder g_delete_kv_raft_meta("raft_delete_kv_raft_meta"); -int LocalRaftMetaStorage::init() { - if (_is_inited) { +const char* FileBasedSingleMetaStorage::_s_raft_meta = "stable_meta"; + +// MetaStorageManager +// +// To manage all KVBasedMergedMetaStorageImpl of all the raft instances. +// Typically nodes on the same disk will share a KVBasedMergedMetaStorageImpl, +// so we use disk_path as the KEY to manage all the instances. +class MetaStorageManager { +public: + static MetaStorageManager* GetInstance() { + return Singleton::get(); + } + + scoped_refptr + register_meta_storage(const std::string& path) { + scoped_refptr mss = get_meta_storage(path); + if (mss != NULL) { + return mss; + } + + mss = new KVBasedMergedMetaStorageImpl(path); + { + _ss_map.Modify(_add, path, mss); + } + return get_meta_storage(path); + } + + scoped_refptr + get_meta_storage(const std::string& path) { + DoublyBufferedMetaStorageMap::ScopedPtr ptr; + CHECK_EQ(0, _ss_map.Read(&ptr)); + MetaStorageMap::const_iterator it = ptr->find(path); + if (it != ptr->end()) { + return it->second; + } + return NULL; + } + + // GC an invalid item in KVBasedMergedMetaStorageImpl when destroying + // an raft instance on the disk for some reason, such as IO error. + int remove_instance_from_meta_storage(const std::string& path, + const VersionedGroupId& v_group_id) { + scoped_refptr mss = + get_meta_storage(path); + if (mss == NULL) { + return 0; + } + butil::Status status = mss->delete_meta(v_group_id); + if (!status.ok()) { + return -1; + } return 0; } - butil::FilePath dir_path(_path); - butil::File::Error e; - if (!butil::CreateDirectoryAndGetError( - dir_path, &e, FLAGS_raft_create_parent_directories)) { - LOG(ERROR) << "Fail to create " << dir_path.value() << " : " << e; - return -1; + +private: + MetaStorageManager() {}; + ~MetaStorageManager() {}; + DISALLOW_COPY_AND_ASSIGN(MetaStorageManager); + friend struct DefaultSingletonTraits; + + typedef std::map > + MetaStorageMap; + typedef butil::DoublyBufferedData DoublyBufferedMetaStorageMap; + + static size_t _add(MetaStorageMap& m, const std::string& path, + const scoped_refptr mss) { + std::pair iter = + m.insert(std::make_pair(path, mss)); + if (iter.second) { + return 1lu; + } + return 0lu; + } + + static size_t _remove(MetaStorageMap& m, const std::string& path) { + return m.erase(path); } - int ret = load(); - if (ret == 0) { - _is_inited = true; + DoublyBufferedMetaStorageMap _ss_map; +}; + +#define global_mss_manager MetaStorageManager::GetInstance() + +// MixedMetaStorage +// +// Uri of Multi-raft using mixed stable storage is: +// local-mixed://merged_path={merged_path}&&single_path={single_path} +int MixedMetaStorage::parse_mixed_path(const std::string& uri, + std::string& merged_path, + std::string& single_path) { + // here uri has removed protocol already, check just for safety + butil::StringPiece copied_uri(uri); + size_t pos = copied_uri.find("://"); + if (pos != butil::StringPiece::npos) { + copied_uri.remove_prefix(pos + 3/* length of '://' */); } - return ret; + + pos = copied_uri.find("merged_path="); + if (pos == butil::StringPiece::npos) { + return -1; + } + copied_uri.remove_prefix(pos + 12/* length of 'merged_path=' */); + + pos = copied_uri.find("&&single_path="); + if (pos == butil::StringPiece::npos) { + return -1; + } + merged_path = copied_uri.substr(0, pos).as_string(); + copied_uri.remove_prefix(pos + 14/* length of '&&single_path=' */); + single_path = copied_uri.as_string(); + + return 0; } -int LocalRaftMetaStorage::set_term(const int64_t term) { - if (_is_inited) { - _term = term; - return save(); +MixedMetaStorage::MixedMetaStorage(const std::string& uri) { + _is_inited = false; + _is_bad = false; + + std::string merged_path; + std::string single_path; + + int ret = parse_mixed_path(uri, merged_path, single_path); + if (ret != 0) { + LOG(ERROR) << "node parse mixed path failed, uri " << uri; + _is_bad = true; } else { - LOG(WARNING) << "LocalRaftMetaStorage not init(), path: " << _path; - return -1; + // Use single_path as the path of MixedMetaStorage as it usually + // contains group_id + _path = single_path; + + _single_impl = new FileBasedSingleMetaStorage(single_path); + _merged_impl = global_mss_manager->register_meta_storage(merged_path); + + if (!_single_impl || !_merged_impl) { + // Both _single_impl and _merged_impl are needed in MixedMetaStorage + LOG(ERROR) << "MixedMetaStorage failed to create both" + " sub stable storage, uri " << uri; + _is_bad = true; + } } } -int64_t LocalRaftMetaStorage::get_term() { - if (_is_inited) { - return _term; - } else { - LOG(WARNING) << "LocalRaftMetaStorage not init(), path: " << _path; - return -1; +MixedMetaStorage::~MixedMetaStorage() { + if (_single_impl) { + delete _single_impl; + _single_impl = NULL; + } + if (_merged_impl) { + _merged_impl = NULL; } } -int LocalRaftMetaStorage::set_votedfor(const PeerId& peer_id) { - if (_is_inited) { - _votedfor = peer_id; - return save(); - } else { - LOG(WARNING) << "LocalRaftMetaStorage not init(), path: " << _path; - return -1; +butil::Status MixedMetaStorage::init() { + butil::Status status; + if (_is_inited) { + return status; } + // check bad + if (_is_bad) { + status.set_error(EINVAL, "MixedMetaStorage is bad, path %s", + _path.c_str()); + return status; + } + + // both _single_impl and _merged_impl are valid since _is_bad is false + status = _single_impl->init(); + if (!status.ok()) { + LOG(ERROR) << "Init Mixed stable storage failed because init Single" + " stable storage failed, path " << _path; + return status; + } + + status = _merged_impl->init(); + if (!status.ok()) { + LOG(ERROR) << "Init Mixed stable storage failed because init merged" + " stable storage failed, path " << _path; + return status; + } + + _is_inited = true; + LOG(NOTICE) << "Succeed to init MixedMetaStorage, path: " << _path; + return status; } -int LocalRaftMetaStorage::set_term_and_votedfor(const int64_t term, const PeerId& peer_id) { - if (_is_inited) { - _term = term; - _votedfor = peer_id; - return save(); +butil::Status MixedMetaStorage::set_term_and_votedfor(const int64_t term, + const PeerId& peer_id, const VersionedGroupId& group) { + butil::Status status; + if (!_is_inited) { + LOG(WARNING) << "MixedMetaStorage not init, path: " << _path; + status.set_error(EINVAL, "MixedMetaStorage of group %s not init, path: %s", + group.c_str(), _path.c_str()); + return status; + } + + status = _single_impl->set_term_and_votedfor(term, peer_id, group); + if (!status.ok()) { + LOG(WARNING) << "node " << group + << " single stable storage failed to set_term_and_votedfor, path: " + << _path; + return status; + } + + status = _merged_impl->set_term_and_votedfor(term, peer_id, group); + if (!status.ok()) { + LOG(WARNING) << "node " << group + << " merged stable storage failed to set_term_and_votedfor, path: " + << _path; + return status; + } + return status; +} + +// [NOTICE] Conflict cases may occur in this mode, it's important to ensure consistency +// 1. Single is newer than Merged: +// case 1: upgrade storage from Single to Mixed, data in Merged is stale +// case 2: last set_term_and_votedfor succeeded in Single but failed in Merged +// 2. Merged is newer than Single: +// case: downgrade storage from Merged to Mixed, data in Single is stale +butil::Status MixedMetaStorage::get_term_and_votedfor(int64_t* term, PeerId* peer_id, + const VersionedGroupId& group) { + butil::Status status; + if (!_is_inited) { + LOG(WARNING) << "MixedMetaStorage not init, path: " << _path; + status.set_error(EINVAL, "MixedMetaStorage of group %s not init, path: %s", + group.c_str(), _path.c_str()); + return status; + } + + // If data from single stable storage is newer than that from merged stable storage, + // merged stable storage should catch up the newer data to ensure safety; Vice versa. + bool single_newer_than_merged = false; + + int64_t term_1; + PeerId peer_id_1; + status = _single_impl->get_term_and_votedfor(&term_1, &peer_id_1, group); + if (!status.ok()) { + LOG(WARNING) << "node " << group + << " single stable storage failed to get_term_and_votedfor, path: " + << _path << ", error: " << status.error_cstr(); + return status; + } + + int64_t term_2; + PeerId peer_id_2; + status = _merged_impl->get_term_and_votedfor(&term_2, + &peer_id_2, group); + if (!status.ok()) { + LOG(WARNING) << "node " << group + << " merged stable storage failed to get_term_and_votdfor," + << " path: " << _path << ", error: " << status.error_cstr(); + return status; + // check consistency of two stable storage + } else if (term_1 == term_2 && peer_id_1 == peer_id_2) { + // if two results are consistent, just return success + *term = term_1; + *peer_id = peer_id_1; + return status; + } + + // this case is theoretically impossible, pay much attention to it if happens + if (term_1 == term_2 && peer_id_1 != ANY_PEER + && peer_id_2 != ANY_PEER) { + CHECK(false) << "Unexpected conflict when mixed stable storage of " + << group << " get_term_and_votedfor, the same term " << term_1 + << ", but different non-empty votdfor(" << peer_id_1 + << " from single stable storage and " << peer_id_2 + << " from merged stable storage), path: " << _path; + status.set_error(EINVAL, "Unexpected conflict"); + return status; + } + + // if two results are not consistent, check out which is newer and catch up + // data for the stale one + single_newer_than_merged = term_1 > term_2 || + (term_1 == term_2 && peer_id_1 != ANY_PEER && peer_id_2 == ANY_PEER); + + if (single_newer_than_merged) { + *term = term_1; + *peer_id = peer_id_1; + status = _merged_impl->set_term_and_votedfor(*term, *peer_id, group); + if (!status.ok()) { + LOG(WARNING) << "node " << group + << " merged stable storage failed to set term " << *term + << " and vote for peer " << *peer_id + << " when catch up data, path " << _path + << ", error: " << status.error_cstr(); + return status; + } + LOG(NOTICE) << "node " << group + << " merged stable storage succeed to set term " << *term + << " and vote for peer " << *peer_id + << " when catch up data, path " << _path; } else { - LOG(WARNING) << "LocalRaftMetaStorage not init(), path: " << _path; - return -1; + LOG(WARNING) << "LocalMetaStorage not init(), path: " << _path; + *term = term_2; + *peer_id = peer_id_2; + status = _single_impl->set_term_and_votedfor(*term, *peer_id, group); + if (!status.ok()) { + LOG(WARNING) << "node " << group + << " single stable storage failed to set term " << *term + << " and vote for peer " << *peer_id + << " when catch up data, path " << _path + << ", error: " << status.error_cstr(); + return status; + } + LOG(NOTICE) << "node " << group + << " single stable storage succeed to set term " << *term + << " and vote for peer " << *peer_id + << " when catch up data, path " << _path; + } + + return status; +} + +RaftMetaStorage* MixedMetaStorage::new_instance(const std::string& uri) const { + return new MixedMetaStorage(uri); +} + +butil::Status MixedMetaStorage::gc_instance(const std::string& uri, + const VersionedGroupId& vgid) const { + butil::Status status; + std::string merged_path; + std::string single_path; + + int ret = parse_mixed_path(uri, merged_path, single_path); + if (ret != 0) { + LOG(WARNING) << "node parse mixed path failed, uri " << uri; + status.set_error(EINVAL, "Group %s failed to parse mixed path, uri %s", + vgid.c_str(), uri.c_str()); + return status; + } + if (0 != gc_dir(single_path)) { + LOG(WARNING) << "Group " << vgid << " failed to gc path " << single_path; + status.set_error(EIO, "Group %s failed to gc path %s", + vgid.c_str(), single_path.c_str()); + return status; } + if (0 != global_mss_manager-> + remove_instance_from_meta_storage(merged_path, vgid)) { + LOG(ERROR) << "Group " << vgid << " failed to gc kv from path: " + << merged_path; + status.set_error(EIO, "Group %s failed to gc kv from path %s", + vgid.c_str(), merged_path.c_str()); + return status; + } + LOG(INFO) << "Group " << vgid << " succeed to gc from single path: " + << single_path << " and merged path: " << merged_path; + return status; + } + +// FileBasedSingleMetaStorage +butil::Status FileBasedSingleMetaStorage::init() { + butil::Status status; + if (_is_inited) { + return status; + } + + butil::FilePath dir_path(_path); + butil::File::Error e; + if (!butil::CreateDirectoryAndGetError( + dir_path, &e, FLAGS_raft_create_parent_directories)) { + LOG(ERROR) << "Fail to create " << dir_path.value() << " : " << e; + status.set_error(e, "Fail to create dir when init SingleMetaStorage, " + "path: %s", _path.c_str()); + return status; + } + + int ret = load(); + if (ret != 0) { + LOG(ERROR) << "Fail to load pb meta when init single stable storage" + ", path: " << _path; + status.set_error(EIO, "Fail to load pb meta when init stabel storage" + ", path: %s", _path.c_str()); + return status; + } + + _is_inited = true; + return status; } -int LocalRaftMetaStorage::load() { +butil::Status FileBasedSingleMetaStorage::set_term_and_votedfor(const int64_t term, + const PeerId& peer_id, const VersionedGroupId&) { + butil::Status status; + if (!_is_inited) { + status.set_error(EINVAL, "SingleMetaStorage not init, path: %s", + _path.c_str()); + return status; + } + _term = term; + _votedfor = peer_id; + if (save() != 0) { + status.set_error(EIO, "SingleMetaStorage failed to save pb meta, path: %s", + _path.c_str()); + return status; + } + return status; +} + +butil::Status FileBasedSingleMetaStorage::get_term_and_votedfor(int64_t* term, + PeerId* peer_id, const VersionedGroupId& group) { + butil::Status status; + if (!_is_inited) { + status.set_error(EINVAL, "SingleMetaStorage not init, path: %s", + _path.c_str()); + return status; + } + *term = _term; + *peer_id = _votedfor; + return status; +} +int FileBasedSingleMetaStorage::load() { + butil::Timer timer; + timer.start(); + std::string path(_path); path.append("/"); path.append(_s_raft_meta); @@ -104,11 +465,19 @@ int LocalRaftMetaStorage::load() { } else { PLOG(ERROR) << "Fail to load meta from " << path; } - + + timer.stop(); + // Only reload process will load stable meta of raft instances, + // reading just go through memory + g_load_pb_raft_meta << timer.u_elapsed(); + LOG(INFO) << "Loaded single stable meta, path " << _path + << " term " << _term + << " votedfor " << _votedfor.to_string() + << " time: " << timer.u_elapsed(); return ret; } -int LocalRaftMetaStorage::save() { +int FileBasedSingleMetaStorage::save() { butil::Timer timer; timer.start(); @@ -125,23 +494,261 @@ int LocalRaftMetaStorage::save() { PLOG_IF(ERROR, ret != 0) << "Fail to save meta to " << path; timer.stop(); - LOG(INFO) << "save raft meta, path " << _path - << " term " << _term << " votedfor " << _votedfor.to_string() << " time: " << timer.u_elapsed(); + g_save_pb_raft_meta << timer.u_elapsed(); + LOG(INFO) << "Saved single stable meta, path " << _path + << " term " << _term + << " votedfor " << _votedfor.to_string() + << " time: " << timer.u_elapsed(); return ret; } -int LocalRaftMetaStorage::get_votedfor(PeerId* peer_id) { +RaftMetaStorage* FileBasedSingleMetaStorage::new_instance( + const std::string& uri) const { + return new FileBasedSingleMetaStorage(uri); +} + +butil::Status FileBasedSingleMetaStorage::gc_instance(const std::string& uri, + const VersionedGroupId& vgid) const { + butil::Status status; + if (0 != gc_dir(uri)) { + LOG(WARNING) << "Group " << vgid << " failed to gc single stable storage" + ", path: " << uri; + status.set_error(EIO, "Group %s failed to gc single stable storage" + ", path: %s", vgid.c_str(), uri.c_str()); + return status; + } + LOG(INFO) << "Group " << vgid << " succeed to gc single stable storage" + ", path: " << uri; + return status; +} + +// KVBasedMergedMetaStorage +KVBasedMergedMetaStorage::KVBasedMergedMetaStorage(const std::string& path) { + _merged_impl = global_mss_manager->register_meta_storage(path); +} + +KVBasedMergedMetaStorage::~KVBasedMergedMetaStorage() { + if (_merged_impl) { + _merged_impl = NULL; + } +} + +butil::Status KVBasedMergedMetaStorage::init() { + return _merged_impl->init(); +}; + +butil::Status KVBasedMergedMetaStorage::set_term_and_votedfor(const int64_t term, + const PeerId& peer_id, const VersionedGroupId& group) { + return _merged_impl->set_term_and_votedfor(term, peer_id, group); +}; + +butil::Status KVBasedMergedMetaStorage::get_term_and_votedfor(int64_t* term, + PeerId* peer_id, const VersionedGroupId& group) { + return _merged_impl->get_term_and_votedfor(term, peer_id, group); +}; + +RaftMetaStorage* KVBasedMergedMetaStorage::new_instance( + const std::string& uri) const { + return new KVBasedMergedMetaStorage(uri); +} + +butil::Status KVBasedMergedMetaStorage::gc_instance(const std::string& uri, + const VersionedGroupId& vgid) const { + butil::Status status; + if (0 != global_mss_manager-> + remove_instance_from_meta_storage(uri, vgid)) { + LOG(WARNING) << "Group " << vgid << " failed to gc kv, path: " << uri; + status.set_error(EIO, "Group %s failed to gc kv in path: %s", + vgid.c_str(), uri.c_str()); + return status; + } + LOG(INFO) << "Group " << vgid << " succeed to gc kv, path: " << uri; + return status; +}; + +butil::Status KVBasedMergedMetaStorage::delete_meta( + const VersionedGroupId& group) { + return _merged_impl->delete_meta(group); +}; + +// KVBasedMergedMetaStorageImpl +butil::Status KVBasedMergedMetaStorageImpl::init() { + std::unique_lock lck(_mutex); + + butil::Status status; if (_is_inited) { - *peer_id = _votedfor; - return 0; - } else { - LOG(WARNING) << "LocalRaftMetaStorage not init(), path: " << _path; - return -1; + return status; } + + butil::FilePath dir_path(_path); + butil::File::Error e; + if (!butil::CreateDirectoryAndGetError( + dir_path, &e, FLAGS_raft_create_parent_directories)) { + lck.unlock(); + LOG(ERROR) << "Fail to create " << dir_path.value() << " : " << e; + status.set_error(e, "Fail to create dir when init MergedMetaStorage, " + "path: %s", _path.c_str()); + return status; + } + + leveldb::Options options; + options.create_if_missing = true; + //options.error_if_exists = true; + + leveldb::Status st; + st = leveldb::DB::Open(options, _path.c_str(), &_db); + if (!st.ok()) { + lck.unlock(); + LOG(ERROR) << "Fail to open db: " << st.ToString() << " path: " << _path; + status.set_error(EIO, "Fail to open db, path: %s, error: %s", + _path.c_str(), st.ToString().c_str()); + return status; + } + + _is_inited = true; + return status; +} + +butil::Status KVBasedMergedMetaStorageImpl::set_term_and_votedfor( + const int64_t term, const PeerId& peer_id, const VersionedGroupId& group) { + butil::Status status; + if (!_is_inited) { + status.set_error(EINVAL, "MergedMetaStorage of group %s not init" + ", path: %s", group.c_str(), _path.c_str()); + return status; + } + + butil::Timer timer; + timer.start(); + leveldb::WriteOptions options; + options.sync = raft_sync_meta(); + + // TODO replace pb + StablePBMeta meta; + meta.set_term(term); + meta.set_votedfor(peer_id.to_string()); + std::string meta_string; + meta.SerializeToString(&meta_string); + leveldb::Slice value(meta_string.data(), meta_string.size()); + + leveldb::Slice key(group.data(), group.size()); + leveldb::Status st = _db->Put(options, key, value); + if (!st.ok()) { + LOG(ERROR) << "Fail to put vote info into db, group " << group + << " term " << term << " vote for " << peer_id + << ", path: " << _path << ", error: " << st.ToString(); + status.set_error(EIO, "MergedMetaStorage of group %s failed" + "to put value(term %ld, votedfor %s), path: %s, error: %s", + group.c_str(), term, peer_id.to_string().c_str(), + _path.c_str(), st.ToString().c_str()); + return status; + } + + timer.stop(); + g_save_kv_raft_meta << timer.u_elapsed(); + LOG(INFO) << "Saved merged stable meta, path " << _path + << " group " << group + << " term " << term + << " votedfor " << peer_id.to_string() + << " time: " << timer.u_elapsed(); + return status; } -RaftMetaStorage* LocalRaftMetaStorage::new_instance(const std::string& uri) const { - return new LocalRaftMetaStorage(uri); +butil::Status KVBasedMergedMetaStorageImpl::get_term_and_votedfor(int64_t* term, + PeerId* peer_id, const VersionedGroupId& group) { + butil::Status status; + if (!_is_inited) { + status.set_error(EINVAL, "MergedMetaStorage of group %s not init, path: %s", + group.c_str(), _path.c_str()); + return status; + } + + butil::Timer timer; + timer.start(); + + leveldb::Slice key(group.data(), group.size()); + std::string value; + leveldb::Status st = _db->Get(leveldb::ReadOptions(), key, &value); + if (st.IsNotFound()) { + // Not exist in db, set initial term 1 and votedfor 0.0.0.0:0:0 + *term = 1; + *peer_id = ANY_PEER; + status = set_term_and_votedfor(*term, *peer_id, group); + if (!status.ok()) { + LOG(ERROR) << "node " << group + << " failed to set initial term and votedfor when first time init" + << ", path " << _path + << ", error " << status.error_cstr(); + return status; + } + LOG(NOTICE) << "node " << group + << " succeed to set initial term and votedfor when first time init" + << ", path " << _path; + return status; + } else if (!st.ok()) { + LOG(ERROR) << "node " << group + << " failed to get value from db, path " << _path + << ", error " << st.ToString().c_str(); + status.set_error(EIO, "MergedMetaStorage of group %s failed to" + "get value from db, path: %s, error: %s", + group.c_str(), _path.c_str(), st.ToString().c_str()); + return status; + } + + // TODO replace pb + StablePBMeta meta; + meta.ParseFromString(value); + *term = meta.term(); + if (0 != peer_id->parse(meta.votedfor())) { + LOG(ERROR) << "node " << group + << " failed to parse votedfor when loading meta from db, path " + << _path; + status.set_error(EINVAL, "MergedMetaStorage of group %s failed to" + " parse votedfor when loading meta from db, path: %s", + group.c_str(), _path.c_str()); + return status; + } + + timer.stop(); + g_load_kv_raft_meta << timer.u_elapsed(); + LOG(INFO) << "Loaded merged stable meta, path " << _path + << " group " << group + << " term " << *term + << " votedfor " << *peer_id + << " time: " << timer.u_elapsed(); + return status; +} + +butil::Status KVBasedMergedMetaStorageImpl::delete_meta( + const VersionedGroupId& group) { + butil::Status status; + if (!_is_inited) { + status.set_error(EINVAL, "MergedMetaStorage of group %s not init, path: %s", + group.c_str(), _path.c_str()); + return status; + } + + butil::Timer timer; + timer.start(); + leveldb::WriteOptions options; + options.sync = raft_sync_meta(); + + leveldb::Slice key(group.data(), group.size()); + leveldb::Status st = _db->Delete(options, key); + if (!st.ok()) { + LOG(ERROR) << "Fail to delete meta info from db, group " << group; + status.set_error(EIO, "MergedMetaStorage failed to delete group %s" + ", path: %s, error: %s", + group.c_str(), _path.c_str(), st.ToString().c_str()); + return status; + } + + timer.stop(); + g_delete_kv_raft_meta << timer.u_elapsed(); + LOG(INFO) << "Deleted merged stable meta, path " << _path + << " group " << group + << " time: " << timer.u_elapsed(); + return status; } } diff --git a/src/braft/raft_meta.h b/src/braft/raft_meta.h index d80a02e9..6873a71a 100644 --- a/src/braft/raft_meta.h +++ b/src/braft/raft_meta.h @@ -13,40 +13,82 @@ // limitations under the License. // Authors: Wang,Yao(wangyao02@baidu.com) +// Xiong,Kai(xiongkai@baidu.com) #ifndef BRAFT_STABLE_H #define BRAFT_STABLE_H +#include +#include #include "braft/storage.h" namespace braft { -class LocalRaftMetaStorage : public RaftMetaStorage { +class FileBasedSingleMetaStorage; +class KVBasedMergedMetaStorageImpl; + +class MixedMetaStorage : public RaftMetaStorage { public: - explicit LocalRaftMetaStorage(const std::string& path) - : _is_inited(false), _path(path), _term(1) {} - LocalRaftMetaStorage() {} - virtual ~LocalRaftMetaStorage() {} + explicit MixedMetaStorage(const std::string& path); + MixedMetaStorage() {} + virtual ~MixedMetaStorage(); + + // init meta storage + virtual butil::Status init(); - // init stable storage, check consistency and integrity - virtual int init(); + // set term and votedfor information + virtual butil::Status set_term_and_votedfor(const int64_t term, + const PeerId& peer_id, const VersionedGroupId& group); - // set current term - virtual int set_term(const int64_t term); + // get term and votedfor information + virtual butil::Status get_term_and_votedfor(int64_t* term, PeerId* peer_id, + const VersionedGroupId& group); - // get current term - virtual int64_t get_term(); + RaftMetaStorage* new_instance(const std::string& uri) const; + + butil::Status gc_instance(const std::string& uri, + const VersionedGroupId& vgid) const; + + bool is_bad() { return _is_bad; } + +private: - // set votefor information - virtual int set_votedfor(const PeerId& peer_id); + static int parse_mixed_path(const std::string& uri, std::string& merged_path, + std::string& single_path); + + bool _is_inited; + bool _is_bad; + std::string _path; + // Origin stable storage for each raft node + FileBasedSingleMetaStorage* _single_impl; + // Merged stable storage for raft nodes on the same disk + scoped_refptr _merged_impl; +}; - // get votefor information - virtual int get_votedfor(PeerId* peer_id); +// Manage meta info of ONLY ONE raft instance +class FileBasedSingleMetaStorage : public RaftMetaStorage { +public: + explicit FileBasedSingleMetaStorage(const std::string& path) + : _is_inited(false), _path(path), _term(1) {} + FileBasedSingleMetaStorage() {} + virtual ~FileBasedSingleMetaStorage() {} - // set term and peer_id - virtual int set_term_and_votedfor(const int64_t term, const PeerId& peer_id); + // init stable storage + virtual butil::Status init(); + + // set term and votedfor information + virtual butil::Status set_term_and_votedfor(const int64_t term, + const PeerId& peer_id, const VersionedGroupId& group); + + // get term and votedfor information + virtual butil::Status get_term_and_votedfor(int64_t* term, PeerId* peer_id, + const VersionedGroupId& group); RaftMetaStorage* new_instance(const std::string& uri) const; + + butil::Status gc_instance(const std::string& uri, + const VersionedGroupId& vgid) const; + private: static const char* _s_raft_meta; int load(); @@ -58,6 +100,83 @@ class LocalRaftMetaStorage : public RaftMetaStorage { PeerId _votedfor; }; +// Manage meta info of A BATCH of raft instances who share the same disk_path prefix +class KVBasedMergedMetaStorage : public RaftMetaStorage { + +public: + explicit KVBasedMergedMetaStorage(const std::string& path); + KVBasedMergedMetaStorage() {} + + virtual ~KVBasedMergedMetaStorage(); + + // init stable storage + virtual butil::Status init(); + + // set term and votedfor information + virtual butil::Status set_term_and_votedfor(const int64_t term, + const PeerId& peer_id, + const VersionedGroupId& group); + + // get term and votedfor information + virtual butil::Status get_term_and_votedfor(int64_t* term, PeerId* peer_id, + const VersionedGroupId& group); + + RaftMetaStorage* new_instance(const std::string& uri) const; + + butil::Status gc_instance(const std::string& uri, + const VersionedGroupId& vgid) const; + + // GC meta info of a raft instance indicated by |group| + virtual butil::Status delete_meta(const VersionedGroupId& group); + +private: + + scoped_refptr _merged_impl; +}; + +// Inner class of KVBasedMergedMetaStorage +class KVBasedMergedMetaStorageImpl : + public butil::RefCountedThreadSafe { +friend class scoped_refptr; + +public: + explicit KVBasedMergedMetaStorageImpl(const std::string& path) + : _is_inited(false), _path(path) {} + KVBasedMergedMetaStorageImpl() {} + virtual ~KVBasedMergedMetaStorageImpl() { + if (_db) { + delete _db; + } + } + + // init stable storage + virtual butil::Status init(); + + // set term and votedfor information + virtual butil::Status set_term_and_votedfor(const int64_t term, + const PeerId& peer_id, + const VersionedGroupId& group); + + // get term and votedfor information + // [NOTICE] If some new instance init stable storage for the first time, + // no record would be found from db, in which case initial term and votedfor + // will be set. + // Initial term: 1 Initial votedfor: ANY_PEER + virtual butil::Status get_term_and_votedfor(int64_t* term, PeerId* peer_id, + const VersionedGroupId& group); + + // GC meta info of a raft instance indicated by |group| + virtual butil::Status delete_meta(const VersionedGroupId& group); + +private: + friend class butil::RefCountedThreadSafe; + + raft_mutex_t _mutex; + bool _is_inited; + std::string _path; + leveldb::DB* _db; +}; + } #endif //~BRAFT_STABLE_H diff --git a/src/braft/raft_service.cpp b/src/braft/raft_service.cpp index 0abde6ce..0bdd6bfa 100644 --- a/src/braft/raft_service.cpp +++ b/src/braft/raft_service.cpp @@ -25,7 +25,7 @@ namespace braft { RaftServiceImpl::~RaftServiceImpl() { - NodeManager::GetInstance()->remove_address(_addr); + global_node_manager->remove_address(_addr); } void RaftServiceImpl::pre_vote(google::protobuf::RpcController* cntl_base, @@ -42,8 +42,8 @@ void RaftServiceImpl::pre_vote(google::protobuf::RpcController* cntl_base, return; } - scoped_refptr node_ptr = NodeManager::GetInstance()->get(request->group_id(), - peer_id); + scoped_refptr node_ptr = + global_node_manager->get(request->group_id(), peer_id); NodeImpl* node = node_ptr.get(); if (!node) { cntl->SetFailed(ENOENT, "peer_id not exist"); @@ -72,8 +72,8 @@ void RaftServiceImpl::request_vote(google::protobuf::RpcController* cntl_base, return; } - scoped_refptr node_ptr = NodeManager::GetInstance()->get(request->group_id(), - peer_id); + scoped_refptr node_ptr = + global_node_manager->get(request->group_id(), peer_id); NodeImpl* node = node_ptr.get(); if (!node) { cntl->SetFailed(ENOENT, "peer_id not exist"); @@ -101,8 +101,8 @@ void RaftServiceImpl::append_entries(google::protobuf::RpcController* cntl_base, return; } - scoped_refptr node_ptr = NodeManager::GetInstance()->get(request->group_id(), - peer_id); + scoped_refptr node_ptr = + global_node_manager->get(request->group_id(), peer_id); NodeImpl* node = node_ptr.get(); if (!node) { cntl->SetFailed(ENOENT, "peer_id not exist"); @@ -127,8 +127,8 @@ void RaftServiceImpl::install_snapshot(google::protobuf::RpcController* cntl_bas return; } - scoped_refptr node_ptr = NodeManager::GetInstance()->get(request->group_id(), - peer_id); + scoped_refptr node_ptr = + global_node_manager->get(request->group_id(), peer_id); NodeImpl* node = node_ptr.get(); if (!node) { cntl->SetFailed(ENOENT, "peer_id not exist"); @@ -154,8 +154,8 @@ void RaftServiceImpl::timeout_now(::google::protobuf::RpcController* controller, return; } - scoped_refptr node_ptr = NodeManager::GetInstance()->get(request->group_id(), - peer_id); + scoped_refptr node_ptr = + global_node_manager->get(request->group_id(), peer_id); NodeImpl* node = node_ptr.get(); if (!node) { cntl->SetFailed(ENOENT, "peer_id not exist"); diff --git a/src/braft/replicator.cpp b/src/braft/replicator.cpp index 788277ca..2c41b693 100644 --- a/src/braft/replicator.cpp +++ b/src/braft/replicator.cpp @@ -49,6 +49,9 @@ DEFINE_int32(raft_retry_replicate_interval_ms, 1000, BRPC_VALIDATE_GFLAG(raft_retry_replicate_interval_ms, brpc::PositiveInteger); +DECLARE_int64(raft_append_entry_high_lat_us); +DECLARE_bool(raft_trace_append_entry_latency); + static bvar::LatencyRecorder g_send_entries_latency("raft_send_entries"); static bvar::LatencyRecorder g_normalized_send_entries_latency( "raft_send_entries_normalized"); @@ -489,6 +492,18 @@ void Replicator::_on_rpc_returned(ReplicatorId id, brpc::Controller* cntl, r->_options.ballot_box->commit_at( min_flying_index, rpc_last_log_index, r->_options.peer_id); + int64_t rpc_latency_us = cntl->latency_us(); + if (FLAGS_raft_trace_append_entry_latency && + rpc_latency_us > FLAGS_raft_append_entry_high_lat_us) { + LOG(WARNING) << "append entry rpc latency us " << rpc_latency_us + << " greater than " + << FLAGS_raft_append_entry_high_lat_us + << " Group " << r->_options.group_id + << " to peer " << r->_options.peer_id + << " request entry size " << entries_size + << " request data size " + << cntl->request_attachment().size(); + } g_send_entries_latency << cntl->latency_us(); if (cntl->request_attachment().size() > 0) { g_normalized_send_entries_latency << diff --git a/src/braft/snapshot.cpp b/src/braft/snapshot.cpp index e8a15035..fe9a227d 100644 --- a/src/braft/snapshot.cpp +++ b/src/braft/snapshot.cpp @@ -719,6 +719,18 @@ SnapshotStorage* LocalSnapshotStorage::new_instance(const std::string& uri) cons return new LocalSnapshotStorage(uri); } +butil::Status LocalSnapshotStorage::gc_instance(const std::string& uri) const { + butil::Status status; + if (gc_dir(uri) != 0) { + LOG(WARNING) << "Failed to gc snapshot storage from path " << _path; + status.set_error(EINVAL, "Failed to gc snapshot storage from path %s", + uri.c_str()); + return status; + } + LOG(INFO) << "Succeed to gc snapshot storage from path " << uri; + return status; +} + // LocalSnapshotCopier LocalSnapshotCopier::LocalSnapshotCopier() diff --git a/src/braft/snapshot.h b/src/braft/snapshot.h index 49a2061d..448ff4cd 100644 --- a/src/braft/snapshot.h +++ b/src/braft/snapshot.h @@ -200,6 +200,8 @@ friend class LocalSnapshotCopier; virtual int set_snapshot_throttle(SnapshotThrottle* snapshot_throttle); SnapshotStorage* new_instance(const std::string& uri) const; + butil::Status gc_instance(const std::string& uri) const; + void set_server_addr(butil::EndPoint server_addr) { _addr = server_addr; } bool has_server_addr() { return _addr != butil::EndPoint(); } private: diff --git a/src/braft/storage.cpp b/src/braft/storage.cpp index 7f9f3dea..ce0a09f4 100644 --- a/src/braft/storage.cpp +++ b/src/braft/storage.cpp @@ -18,7 +18,6 @@ #include #include #include -#include #include #include @@ -37,31 +36,6 @@ DEFINE_bool(raft_create_parent_directories, true, DEFINE_bool(raft_sync_meta, false, "sync log meta, snapshot meta and raft meta"); BRPC_VALIDATE_GFLAG(raft_sync_meta, ::brpc::PassValidate); -inline butil::StringPiece parse_uri(butil::StringPiece* uri, std::string* parameter) { - // ${protocol}://${parameters} - size_t pos = uri->find("://"); - if (pos == butil::StringPiece::npos) { - return butil::StringPiece(); - } - butil::StringPiece protocol = uri->substr(0, pos); - uri->remove_prefix(pos + 3/* length of '://' */); - protocol.trim_spaces(); - parameter->reserve(uri->size()); - parameter->clear(); - size_t removed_spaces = 0; - for (butil::StringPiece::const_iterator - iter = uri->begin(); iter != uri->end(); ++iter) { - if (!isspace(*iter)) { - parameter->push_back(*iter); - } else { - ++removed_spaces; - } - } - LOG_IF(WARNING, removed_spaces) << "Removed " << removed_spaces - << " spaces from `" << *uri << '\''; - return protocol; -} - LogStorage* LogStorage::create(const std::string& uri) { butil::StringPiece copied_uri(uri); std::string parameter; @@ -80,6 +54,28 @@ LogStorage* LogStorage::create(const std::string& uri) { return type->new_instance(parameter); } +butil::Status LogStorage::destroy(const std::string& uri) { + butil::Status status; + butil::StringPiece copied_uri(uri); + std::string parameter; + butil::StringPiece protocol = parse_uri(&copied_uri, ¶meter); + if (protocol.empty()) { + LOG(ERROR) << "Invalid log storage uri=`" << uri << '\''; + status.set_error(EINVAL, "Invalid log storage uri = %s", uri.c_str()); + return status; + } + const LogStorage* type = log_storage_extension()->Find( + protocol.as_string().c_str()); + if (type == NULL) { + LOG(ERROR) << "Fail to find log storage type " << protocol + << ", uri=" << uri; + status.set_error(EINVAL, "Fail to find log storage type %s uri %s", + protocol.as_string().c_str(), uri.c_str()); + return status; + } + return type->gc_instance(parameter); +} + SnapshotStorage* SnapshotStorage::create(const std::string& uri) { butil::StringPiece copied_uri(uri); std::string parameter; @@ -98,6 +94,28 @@ SnapshotStorage* SnapshotStorage::create(const std::string& uri) { return type->new_instance(parameter); } +butil::Status SnapshotStorage::destroy(const std::string& uri) { + butil::Status status; + butil::StringPiece copied_uri(uri); + std::string parameter; + butil::StringPiece protocol = parse_uri(&copied_uri, ¶meter); + if (protocol.empty()) { + LOG(ERROR) << "Invalid snapshot storage uri=`" << uri << '\''; + status.set_error(EINVAL, "Invalid log storage uri = %s", uri.c_str()); + return status; + } + const SnapshotStorage* type = snapshot_storage_extension()->Find( + protocol.as_string().c_str()); + if (type == NULL) { + LOG(ERROR) << "Fail to find snapshot storage type " << protocol + << ", uri=" << uri; + status.set_error(EINVAL, "Fail to find snapshot storage type %s uri %s", + protocol.as_string().c_str(), uri.c_str()); + return status; + } + return type->gc_instance(parameter); +} + RaftMetaStorage* RaftMetaStorage::create(const std::string& uri) { butil::StringPiece copied_uri(uri); std::string parameter; @@ -116,4 +134,27 @@ RaftMetaStorage* RaftMetaStorage::create(const std::string& uri) { return type->new_instance(parameter); } +butil::Status RaftMetaStorage::destroy(const std::string& uri, + const VersionedGroupId& vgid) { + butil::Status status; + butil::StringPiece copied_uri(uri); + std::string parameter; + butil::StringPiece protocol = parse_uri(&copied_uri, ¶meter); + if (protocol.empty()) { + LOG(ERROR) << "Invalid meta storage uri=`" << uri << '\''; + status.set_error(EINVAL, "Invalid meta storage uri = %s", uri.c_str()); + return status; + } + const RaftMetaStorage* type = meta_storage_extension()->Find( + protocol.as_string().c_str()); + if (type == NULL) { + LOG(ERROR) << "Fail to find meta storage type " << protocol + << ", uri=" << uri; + status.set_error(EINVAL, "Fail to find meta storage type %s uri %s", + protocol.as_string().c_str(), uri.c_str()); + return status; + } + return type->gc_instance(parameter, vgid); +} + } // namespace braft diff --git a/src/braft/storage.h b/src/braft/storage.h index 9334c422..bcdb07ec 100644 --- a/src/braft/storage.h +++ b/src/braft/storage.h @@ -24,7 +24,7 @@ #include #include #include - +#include #include "braft/configuration.h" #include "braft/configuration_manager.h" @@ -42,6 +42,83 @@ DECLARE_bool(raft_create_parent_directories); struct LogEntry; +struct IOMetric { +public: + IOMetric() + : start_time_us(butil::cpuwide_time_us()) + , bthread_queue_time_us(0) + , open_segment_time_us(0) + , append_entry_time_us(0) + , sync_segment_time_us(0) {} + + int64_t start_time_us; + int64_t bthread_queue_time_us; + int64_t open_segment_time_us; + int64_t append_entry_time_us; + int64_t sync_segment_time_us; +}; + +inline std::ostream& operator<<(std::ostream& os, const IOMetric& m) { + return os << " bthread_queue_time_us: " << m.bthread_queue_time_us + << " open_segment_time_us: " << m.open_segment_time_us + << " append_entry_time_us: " << m.append_entry_time_us + << " sync_segment_time_us: " << m.sync_segment_time_us; +} + +inline butil::StringPiece parse_uri(butil::StringPiece* uri, std::string* parameter) { + // ${protocol}://${parameters} + size_t pos = uri->find("://"); + if (pos == butil::StringPiece::npos) { + return butil::StringPiece(); + } + butil::StringPiece protocol = uri->substr(0, pos); + uri->remove_prefix(pos + 3/* length of '://' */); + protocol.trim_spaces(); + parameter->reserve(uri->size()); + parameter->clear(); + size_t removed_spaces = 0; + for (butil::StringPiece::const_iterator + iter = uri->begin(); iter != uri->end(); ++iter) { + if (!isspace(*iter)) { + parameter->push_back(*iter); + } else { + ++removed_spaces; + } + } + LOG_IF(WARNING, removed_spaces) << "Removed " << removed_spaces + << " spaces from `" << *uri << '\''; + return protocol; +} + +inline int gc_dir(const std::string& path) { + butil::File::Error e; + butil::FilePath target_path(path); + butil::FilePath tmp_path(path + ".tmp"); + // delete tmp path firstly in case there is garbage + if (!butil::DeleteFile(tmp_path, true)) { + LOG(ERROR) << "Fail to delete tmp file, path: " << tmp_path.value(); + return -1; + } + + if (butil::PathExists(target_path)) { + const bool rc = butil::ReplaceFile(butil::FilePath(target_path), + butil::FilePath(tmp_path), &e); + if (!rc) { + LOG(ERROR) << "Fail to rename `" << target_path.value() + << " to `" << tmp_path.value() << "' : " << e; + return -1; + } + if (!butil::DeleteFile(tmp_path, true)) { + LOG(ERROR) << "Fail to delete tmp file, path: " << tmp_path.value(); + return -1; + } + } else { + LOG(NOTICE) << "Target path not exist, so no need to gc, path: " + << target_path.value(); + } + return 0; +} + class LogStorage { public: virtual ~LogStorage() {} @@ -64,8 +141,8 @@ class LogStorage { // append entries to log virtual int append_entry(const LogEntry* entry) = 0; - // append entries to log, return append success number - virtual int append_entries(const std::vector& entries) = 0; + // append entries to log and update IOMetric, return append success number + virtual int append_entries(const std::vector& entries, IOMetric* metric) = 0; // delete logs from storage's head, [first_log_index, first_index_kept) will be discarded virtual int truncate_prefix(const int64_t first_index_kept) = 0; @@ -83,36 +160,59 @@ class LogStorage { virtual LogStorage* new_instance(const std::string& uri) const = 0; static LogStorage* create(const std::string& uri); + + // GC an instance of this kind of LogStorage with the parameters encoded + // in |uri| + virtual butil::Status gc_instance(const std::string& uri) const { + CHECK(false) << butil::class_name_str(*this) + << " didn't implement gc_instance interface while deleting" + " raft log in " << uri; + butil::Status status; + status.set_error(ENOSYS, "gc_instance interface is not implemented"); + return status; + } + + static butil::Status destroy(const std::string& uri); }; class RaftMetaStorage { public: virtual ~RaftMetaStorage() {} - // init stable storage, check consistency and integrity - virtual int init() = 0; - - // set current term - virtual int set_term(const int64_t term) = 0; - - // get current term - virtual int64_t get_term() = 0; - - // set votefor information - virtual int set_votedfor(const PeerId& peer_id) = 0; - - // get votefor information - virtual int get_votedfor(PeerId* peer_id) = 0; + // init stable storage + virtual butil::Status init() = 0; // set term and votedfor information - virtual int set_term_and_votedfor(const int64_t term, const PeerId& peer_id) = 0; + virtual butil::Status set_term_and_votedfor(const int64_t term, + const PeerId& peer_id, + const VersionedGroupId& group) = 0; - // Create an instance of this kind of LogStorage with the parameters encoded + // get term and votedfor information + virtual butil::Status get_term_and_votedfor(int64_t* term, PeerId* peer_id, + const VersionedGroupId& group) = 0; + + // Create an instance of this kind of RaftMetaStorage with the parameters encoded // in |uri| // Return the address referenced to the instance on success, NULL otherwise. virtual RaftMetaStorage* new_instance(const std::string& uri) const = 0; static RaftMetaStorage* create(const std::string& uri); + + // GC an instance of this kind of StableStorage with the parameters encoded + // in |uri| + virtual butil::Status gc_instance(const std::string& uri, + const VersionedGroupId& vgid) const { + CHECK(false) << butil::class_name_str(*this) + << " didn't implement gc_instance interface while deleting" + " raft stable meta in " << uri; + butil::Status status; + status.set_error(ENOSYS, "gc_instance interface is not implemented"); + return status; + } + + static butil::Status destroy(const std::string& uri, + const VersionedGroupId& vgid); + }; // Snapshot @@ -198,19 +298,22 @@ class SnapshotStorage { virtual ~SnapshotStorage() {} virtual int set_filter_before_copy_remote() { - CHECK(false) << butil::class_name_str(*this) << " doesn't support filter before copy remote"; + CHECK(false) << butil::class_name_str(*this) + << " doesn't support filter before copy remote"; return -1; } virtual int set_file_system_adaptor(FileSystemAdaptor* fs) { (void)fs; - CHECK(false) << butil::class_name_str(*this) << " doesn't support file system adaptor"; + CHECK(false) << butil::class_name_str(*this) + << " doesn't support file system adaptor"; return -1; } virtual int set_snapshot_throttle(SnapshotThrottle* st) { (void)st; - CHECK(false) << butil::class_name_str(*this) << " doesn't support snapshot throttle"; + CHECK(false) << butil::class_name_str(*this) + << " doesn't support snapshot throttle"; return -1; } @@ -234,8 +337,24 @@ class SnapshotStorage { virtual SnapshotCopier* start_to_copy_from(const std::string& uri) = 0; virtual int close(SnapshotCopier* copier) = 0; + // Create an instance of this kind of SnapshotStorage with the parameters encoded + // in |uri| + // Return the address referenced to the instance on success, NULL otherwise. virtual SnapshotStorage* new_instance(const std::string& uri) const WARN_UNUSED_RESULT = 0; static SnapshotStorage* create(const std::string& uri); + + // GC an instance of this kind of SnapshotStorage with the parameters encoded + // in |uri| + virtual butil::Status gc_instance(const std::string& uri) const { + CHECK(false) << butil::class_name_str(*this) + << " didn't implement gc_instance interface while deleting" + " raft snapshot in " << uri; + butil::Status status; + status.set_error(ENOSYS, "gc_instance interface is not implemented"); + return status; + } + + static butil::Status destroy(const std::string& uri); }; inline brpc::Extension* log_storage_extension() { diff --git a/src/braft/test/util.h b/src/braft/test/util.h new file mode 100644 index 00000000..210c241e --- /dev/null +++ b/src/braft/test/util.h @@ -0,0 +1,520 @@ +// Copyright (c) 2019 Baidu.com, Inc. All Rights Reserved +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Authors: Pengfei Zheng (zhengpengfei@baidu.com) + +#ifndef PUBLIC_RAFT_TEST_UTIL_H +#define PUBLIC_RAFT_TEST_UTIL_H + +#include "braft/node.h" +#include "braft/enum.pb.h" +#include "braft/errno.pb.h" +#include "braft/snapshot_throttle.h" +#include "braft/snapshot_executor.h" + +using namespace braft; +bool g_dont_print_apply_log = false; + +class MockFSM : public braft::StateMachine { +public: + MockFSM(const butil::EndPoint& address_) + : address(address_) + , applied_index(0) + , snapshot_index(0) + , _on_start_following_times(0) + , _on_stop_following_times(0) + , _leader_term(-1) + , _on_leader_start_closure(NULL) + { + pthread_mutex_init(&mutex, NULL); + } + virtual ~MockFSM() { + pthread_mutex_destroy(&mutex); + } + + butil::EndPoint address; + std::vector logs; + pthread_mutex_t mutex; + int64_t applied_index; + int64_t snapshot_index; + int64_t _on_start_following_times; + int64_t _on_stop_following_times; + volatile int64_t _leader_term; + braft::Closure* _on_leader_start_closure; + + void lock() { + pthread_mutex_lock(&mutex); + } + + void unlock() { + pthread_mutex_unlock(&mutex); + } + + void set_on_leader_start_closure(braft::Closure* closure) { + _on_leader_start_closure = closure; + } + + void on_leader_start(int64_t term) { + _leader_term = term; + if (_on_leader_start_closure) { + _on_leader_start_closure->Run(); + _on_leader_start_closure = NULL; + } + } + void on_leader_stop(const braft::LeaderChangeContext&) { + _leader_term = -1; + } + + bool is_leader() { return _leader_term > 0; } + + 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_size " << iter.data().size(); + BRAFT_VLOG << "data " << iter.data(); + ::brpc::ClosureGuard guard(iter.done()); + lock(); + logs.push_back(iter.data()); + unlock(); + applied_index = iter.index(); + } + } + + virtual void on_shutdown() { + LOG(TRACE) << "addr " << address << " shutdowned"; + } + + virtual void on_snapshot_save(braft::SnapshotWriter* writer, braft::Closure* done) { + std::string file_path = writer->get_path(); + file_path.append("/data"); + brpc::ClosureGuard done_guard(done); + + LOG(NOTICE) << "on_snapshot_save to " << file_path; + + int fd = ::creat(file_path.c_str(), 0644); + if (fd < 0) { + LOG(ERROR) << "create file failed, path: " << file_path << " err: " << berror(); + done->status().set_error(EIO, "Fail to create file"); + return; + } + lock(); + // write snapshot and log to file + for (size_t i = 0; i < logs.size(); i++) { + butil::IOBuf data = logs[i]; + int len = data.size(); + int ret = write(fd, &len, sizeof(int)); + CHECK_EQ(ret, 4); + data.cut_into_file_descriptor(fd, len); + } + ::close(fd); + snapshot_index = applied_index; + unlock(); + writer->add_file("data"); + } + + virtual int on_snapshot_load(braft::SnapshotReader* reader) { + std::string file_path = reader->get_path(); + file_path.append("/data"); + + LOG(INFO) << "on_snapshot_load from " << file_path; + + int fd = ::open(file_path.c_str(), O_RDONLY); + if (fd < 0) { + LOG(ERROR) << "creat file failed, path: " << file_path << " err: " << berror(); + return EIO; + } + + lock(); + logs.clear(); + while (true) { + int len = 0; + int ret = read(fd, &len, sizeof(int)); + if (ret <= 0) { + break; + } + + butil::IOPortal data; + data.append_from_file_descriptor(fd, len); + logs.push_back(data); + } + + ::close(fd); + unlock(); + return 0; + } + + virtual void on_start_following(const braft::LeaderChangeContext& start_following_context) { + LOG(TRACE) << "address " << address << " start following new leader: " + << start_following_context; + ++_on_start_following_times; + } + + virtual void on_stop_following(const braft::LeaderChangeContext& stop_following_context) { + LOG(TRACE) << "address " << address << " stop following old leader: " + << stop_following_context; + ++_on_stop_following_times; + } + + virtual void on_configuration_committed(const ::braft::Configuration& conf, int64_t index) { + LOG(TRACE) << "address " << address << " commit conf: " << conf << " at index " << index; + } +}; + +class ExpectClosure : public braft::Closure { +public: + void Run() { + if (_expect_err_code >= 0) { + ASSERT_EQ(status().error_code(), _expect_err_code) + << _pos << " : " << status(); + } + if (_cond) { + _cond->signal(); + } + delete this; + } +private: + ExpectClosure(bthread::CountdownEvent* cond, int expect_err_code, const char* pos) + : _cond(cond), _expect_err_code(expect_err_code), _pos(pos) {} + + ExpectClosure(bthread::CountdownEvent* cond, const char* pos) + : _cond(cond), _expect_err_code(-1), _pos(pos) {} + + bthread::CountdownEvent* _cond; + int _expect_err_code; + const char* _pos; +}; + +typedef ExpectClosure ShutdownClosure; +typedef ExpectClosure ApplyClosure; +typedef ExpectClosure AddPeerClosure; +typedef ExpectClosure RemovePeerClosure; +typedef ExpectClosure SnapshotClosure; + +#define NEW_SHUTDOWNCLOSURE(arg...) \ + (new ExpectClosure(arg, __FILE__ ":" BAIDU_SYMBOLSTR(__LINE__))) +#define NEW_APPLYCLOSURE(arg...) \ + (new ExpectClosure(arg, __FILE__ ":" BAIDU_SYMBOLSTR(__LINE__))) +#define NEW_ADDPEERCLOSURE(arg...) \ + (new ExpectClosure(arg, __FILE__ ":" BAIDU_SYMBOLSTR(__LINE__))) +#define NEW_REMOVEPEERCLOSURE(arg...) \ + (new ExpectClosure(arg, __FILE__ ":" BAIDU_SYMBOLSTR(__LINE__))) +#define NEW_SNAPSHOTCLOSURE(arg...) \ + (new ExpectClosure(arg, __FILE__ ":" BAIDU_SYMBOLSTR(__LINE__))) + +class Cluster { +public: + Cluster(const std::string& name, const std::vector& peers, + int32_t election_timeout_ms = 3000, int max_clock_drift_ms = 1000) + : _name(name), _peers(peers) + , _election_timeout_ms(election_timeout_ms) + , _max_clock_drift_ms(max_clock_drift_ms) { + + int64_t throttle_throughput_bytes = 10 * 1024 * 1024; + int64_t check_cycle = 10; + _throttle = new braft::ThroughputSnapshotThrottle(throttle_throughput_bytes, check_cycle); + } + ~Cluster() { + stop_all(); + } + + int start(const butil::EndPoint& listen_addr, bool empty_peers = false, + int snapshot_interval_s = 30, + braft::Closure* leader_start_closure = NULL) { + if (_server_map[listen_addr] == NULL) { + brpc::Server* server = new brpc::Server(); + if (braft::add_service(server, listen_addr) != 0 + || server->Start(listen_addr, NULL) != 0) { + LOG(ERROR) << "Fail to start raft service"; + delete server; + return -1; + } + _server_map[listen_addr] = server; + } + + braft::NodeOptions options; + options.election_timeout_ms = _election_timeout_ms; + options.max_clock_drift_ms = _max_clock_drift_ms; + options.snapshot_interval_s = snapshot_interval_s; + if (!empty_peers) { + options.initial_conf = braft::Configuration(_peers); + } + MockFSM* fsm = new MockFSM(listen_addr); + if (leader_start_closure) { + fsm->set_on_leader_start_closure(leader_start_closure); + } + options.fsm = fsm; + options.node_owns_fsm = true; + butil::string_printf(&options.log_uri, "local://./data/%s/log", + butil::endpoint2str(listen_addr).c_str()); + butil::string_printf(&options.raft_meta_uri, "local://./data/%s/raft_meta", + 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); + 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 node; + return ret; + } else { + LOG(NOTICE) << "init node " << listen_addr; + } + + { + std::lock_guard guard(_mutex); + _nodes.push_back(node); + _fsms.push_back(fsm); + } + return 0; + } + + int stop(const butil::EndPoint& listen_addr) { + + bthread::CountdownEvent cond; + braft::Node* node = remove_node(listen_addr); + if (node) { + node->shutdown(NEW_SHUTDOWNCLOSURE(&cond)); + cond.wait(); + node->join(); + } + + if (_server_map[listen_addr] != NULL) { + delete _server_map[listen_addr]; + _server_map.erase(listen_addr); + } + _server_map.erase(listen_addr); + delete node; + return node ? 0 : -1; + } + + void stop_all() { + std::vector addrs; + all_nodes(&addrs); + + for (size_t i = 0; i < addrs.size(); i++) { + stop(addrs[i]); + } + } + + void clean(const butil::EndPoint& listen_addr) { + std::string data_path; + butil::string_printf(&data_path, "./data/%s", + butil::endpoint2str(listen_addr).c_str()); + + if (!butil::DeleteFile(butil::FilePath(data_path), true)) { + LOG(ERROR) << "delete path failed, path: " << data_path; + } + } + + braft::Node* leader() { + std::lock_guard guard(_mutex); + braft::Node* node = NULL; + for (size_t i = 0; i < _nodes.size(); i++) { + if (_nodes[i]->is_leader() && + _fsms[i]->_leader_term == _nodes[i]->_impl->_current_term) { + node = _nodes[i]; + break; + } + } + return node; + } + + void followers(std::vector* nodes) { + nodes->clear(); + + std::lock_guard guard(_mutex); + for (size_t i = 0; i < _nodes.size(); i++) { + if (!_nodes[i]->is_leader()) { + nodes->push_back(_nodes[i]); + } + } + } + + void all_nodes(std::vector* nodes) { + nodes->clear(); + + std::lock_guard guard(_mutex); + for (size_t i = 0; i < _nodes.size(); i++) { + nodes->push_back(_nodes[i]); + } + } + + braft::Node* find_node(const braft::PeerId& peer_id) { + std::lock_guard guard(_mutex); + for (size_t i = 0; i < _nodes.size(); i++) { + if (peer_id == _nodes[i]->node_id().peer_id) { + return _nodes[i]; + } + } + return NULL; + } + + void wait_leader() { + while (true) { + braft::Node* node = leader(); + if (node) { + return; + } else { + usleep(100 * 1000); + } + } + } + + 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); + for (size_t i = 0; i < _nodes.size(); i++) { + braft::PeerId leader_id = _nodes[i]->leader_id(); + if (leader_id.addr != expect_addr) { + goto WAIT; + } + } + + return; +WAIT: + usleep(100 * 1000); + goto CHECK; + } + + bool ensure_same(int wait_time_s = -1) { + std::unique_lock guard(_mutex); + if (_fsms.size() <= 1) { + return true; + } + LOG(INFO) << "_fsms.size()=" << _fsms.size(); + + int nround = 0; + MockFSM* first = _fsms[0]; +CHECK: + first->lock(); + for (size_t i = 1; i < _fsms.size(); i++) { + MockFSM* fsm = _fsms[i]; + fsm->lock(); + + if (first->logs.size() != 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; + } + + for (size_t j = 0; j < first->logs.size(); j++) { + butil::IOBuf& first_data = first->logs[j]; + butil::IOBuf& fsm_data = fsm->logs[j]; + 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->unlock(); + goto WAIT; + } + } + + fsm->unlock(); + } + first->unlock(); + guard.unlock(); + check_node_status(); + + return true; +WAIT: + first->unlock(); + sleep(1); + ++nround; + if (wait_time_s > 0 && nround > wait_time_s) { + return false; + } + goto CHECK; + } + +private: + void all_nodes(std::vector* addrs) { + addrs->clear(); + + std::lock_guard guard(_mutex); + for (size_t i = 0; i < _nodes.size(); i++) { + addrs->push_back(_nodes[i]->node_id().peer_id.addr); + } + } + + braft::Node* remove_node(const butil::EndPoint& addr) { + std::lock_guard guard(_mutex); + + // remove node + braft::Node* node = NULL; + std::vector new_nodes; + for (size_t i = 0; i < _nodes.size(); i++) { + if (addr.port == _nodes[i]->node_id().peer_id.addr.port) { + node = _nodes[i]; + } else { + new_nodes.push_back(_nodes[i]); + } + } + _nodes.swap(new_nodes); + + // remove fsm + std::vector new_fsms; + for (size_t i = 0; i < _fsms.size(); i++) { + if (_fsms[i]->address != addr) { + new_fsms.push_back(_fsms[i]); + } + } + _fsms.swap(new_fsms); + + return node; + } + + std::string _name; + std::vector _peers; + std::vector _nodes; + std::vector _fsms; + std::map _server_map; + int32_t _election_timeout_ms; + int32_t _max_clock_drift_ms; + raft_mutex_t _mutex; + braft::SnapshotThrottle* _throttle; +}; + +#endif // ~PUBLIC_RAFT_TEST_UTIL_H diff --git a/test/test_cli.cpp b/test/test_cli.cpp index d9e85b4a..ed816abe 100644 --- a/test/test_cli.cpp +++ b/test/test_cli.cpp @@ -134,7 +134,7 @@ TEST_F(CliTest, set_peer) { st = braft::cli::reset_peer("test", node1.peer_id(), conf2, braft::cli::CliOptions()); ASSERT_TRUE(st.ok()); - usleep(2 * 1000 * 1000); + usleep(4 * 1000 * 1000); ASSERT_TRUE(node1._node->is_leader()); } @@ -156,7 +156,7 @@ TEST_F(CliTest, change_peers) { st = braft::cli::reset_peer("test", node1.peer_id(), conf2, braft::cli::CliOptions()); ASSERT_TRUE(st.ok()); - usleep(2 * 1000 * 1000); + usleep(4 * 1000 * 1000); ASSERT_TRUE(node1._node->is_leader()); } diff --git a/test/test_leader_lease.cpp b/test/test_leader_lease.cpp new file mode 100644 index 00000000..bc3d565f --- /dev/null +++ b/test/test_leader_lease.cpp @@ -0,0 +1,675 @@ +// Copyright (c) 2019 Baidu.com, Inc. All Rights Reserved +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Authors: Pengfei Zheng (zhengpengfei@baidu.com) + +#include +#include +#include "braft/util.h" +#include "braft/node.h" +#include "braft/lease.h" +#include "braft/test/util.h" + +namespace braft { +DECLARE_bool(raft_enable_leader_lease); +DECLARE_int32(raft_election_heartbeat_factor); +} + +class LeaseTest : public testing::Test { +protected: + void SetUp() { + ::system("rm -rf data"); + //logging::FLAGS_v = 90; + braft::FLAGS_raft_sync = false; + braft::FLAGS_raft_enable_leader_lease = true; + braft::FLAGS_raft_election_heartbeat_factor = 3; + g_dont_print_apply_log = true; + } + void TearDown() { + ::system("rm -rf data"); + } +}; + +void check_if_stale_leader_exist(Cluster* cluster, int line) { + // Only one peer can be LEASE_NOT_READY or LEASE_VALID + std::vector nodes; + braft::LeaderLeaseStatus lease_status; + cluster->all_nodes(&nodes); + for (int i = 0; i < 2; ++i) { + int lease_valid_num = 0; + braft::Node* leader_node = NULL; + int64_t leader_term = -1; + for (auto& n : nodes) { + n->get_leader_lease_status(&lease_status); + if (lease_status.state == braft::LEASE_VALID) { + ++lease_valid_num; + if (lease_valid_num == 1) { + leader_node = n; + leader_term = lease_status.term; + } else if (lease_valid_num > 2) { + LOG(ERROR) << "found more than two leaders, leader_num: " << lease_valid_num + << ", line: " << line; + ASSERT_TRUE(false); + return; + } else if (lease_status.term == leader_term || i == 2) { + LOG(ERROR) << "found more leaders, leader 1: " << leader_node->node_id() + << ", leader 2: " << n->node_id() << " line: " << line; + ASSERT_TRUE(false); + return; + } + } + } + if (lease_valid_num == 2) { + // Little chance that we found two leaders because of leader change, + // try again to check again + LOG(WARNING) << "found two leaders, check again"; + continue; + } + if (lease_valid_num == 0) { + LOG(NOTICE) << "no leader"; + return; + } else { + LOG(NOTICE) << "leader with lease: " << leader_node->node_id(); + return; + } + } +} + +#define CHECK_NO_STALE_LEADER(cluster) \ + check_if_stale_leader_exist(cluster, __LINE__); + +bool g_check_lease_in_thread_stop = true; + +void* check_lease_in_thread(void* arg) { + Cluster* cluster = static_cast(arg); + int round = 0; + while (!g_check_lease_in_thread_stop) { + BRAFT_VLOG << "check stale lease, round: " << round++; + usleep(10 * 1000); + CHECK_NO_STALE_LEADER(cluster); + } + return NULL; +} + +TEST_F(LeaseTest, triple_node) { + ::system("rm -rf data"); + 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, 500, 10); + 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(); + std::vector followers; + cluster.followers(&followers); + ASSERT_TRUE(leader != NULL); + LOG(WARNING) << "leader is elected " << leader->node_id(); + + braft::LeaderLeaseStatus lease_status; + leader->get_leader_lease_status(&lease_status); + int64_t leader_term = lease_status.term; + int64_t lease_epoch = lease_status.lease_epoch; + int64_t start_ms = butil::monotonic_time_ms(); + while (lease_status.state == braft::LEASE_NOT_READY || + butil::monotonic_time_ms() - start_ms < 1000) { + BRAFT_VLOG << "waiting lease become valid"; + bthread_usleep(100 * 1000); + leader->get_leader_lease_status(&lease_status); + } + ASSERT_EQ(lease_status.state, braft::LEASE_VALID); + ASSERT_EQ(lease_status.term, leader_term); + ASSERT_EQ(lease_status.lease_epoch, lease_epoch); + ASSERT_TRUE(leader->is_leader_lease_valid()); + + for (int i = 0; i < 3; ++i) { + bthread_usleep(100 * 1000); + + leader->get_leader_lease_status(&lease_status); + ASSERT_EQ(lease_status.state, braft::LEASE_VALID); + ASSERT_TRUE(leader->is_leader_lease_valid()); + ASSERT_EQ(lease_status.term, leader_term); + ASSERT_EQ(lease_status.lease_epoch, lease_epoch); + + // lease lazily extend + bthread_usleep(600 * 1000); + + leader->get_leader_lease_status(&lease_status); + ASSERT_EQ(lease_status.state, braft::LEASE_VALID); + ASSERT_TRUE(leader->is_leader_lease_valid()); + ASSERT_EQ(lease_status.term, leader_term); + ASSERT_EQ(lease_status.lease_epoch, lease_epoch); + } + + // check followrs + for (auto& f : followers) { + ASSERT_FALSE(f->is_leader_lease_valid()); + f->get_leader_lease_status(&lease_status); + ASSERT_EQ(lease_status.state, braft::LEASE_EXPIRED); + } + + // disable + braft::FLAGS_raft_enable_leader_lease = false; + leader->get_leader_lease_status(&lease_status); + ASSERT_EQ(lease_status.state, braft::LEASE_DISABLED); + for (auto& f : followers) { + ASSERT_FALSE(f->is_leader_lease_valid()); + f->get_leader_lease_status(&lease_status); + ASSERT_EQ(lease_status.state, braft::LEASE_DISABLED); + } + + braft::FLAGS_raft_enable_leader_lease = true; + + // stop a follower, lease still valid + LOG(NOTICE) << "stop a follower"; + cluster.stop(followers[0]->node_id().peer_id.addr); + + leader->get_leader_lease_status(&lease_status); + ASSERT_EQ(lease_status.state, braft::LEASE_VALID); + ASSERT_EQ(lease_status.term, leader_term); + ASSERT_EQ(lease_status.lease_epoch, lease_epoch); + ASSERT_TRUE(leader->is_leader_lease_valid()); + + bthread_usleep(600 * 1000); + + leader->get_leader_lease_status(&lease_status); + ASSERT_EQ(lease_status.state, braft::LEASE_VALID); + ASSERT_EQ(lease_status.term, leader_term); + ASSERT_EQ(lease_status.lease_epoch, lease_epoch); + ASSERT_TRUE(leader->is_leader_lease_valid()); + + // stop all follwers, lease expired + LOG(NOTICE) << "stop a another"; + cluster.stop(followers[1]->node_id().peer_id.addr); + + leader->get_leader_lease_status(&lease_status); + ASSERT_EQ(lease_status.state, braft::LEASE_VALID); + ASSERT_EQ(lease_status.term, leader_term); + ASSERT_EQ(lease_status.lease_epoch, lease_epoch); + ASSERT_TRUE(leader->is_leader_lease_valid()); + + bthread_usleep(600 * 1000); + + leader->get_leader_lease_status(&lease_status); + ASSERT_EQ(lease_status.state, braft::LEASE_EXPIRED); + ASSERT_FALSE(leader->is_leader_lease_valid()); + + cluster.stop_all(); +} + +TEST_F(LeaseTest, change_peers) { + ::system("rm -rf data"); + 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, 500, 10); + ASSERT_EQ(0, cluster.start(peer0.addr)); + LOG(NOTICE) << "start single cluster " << peer0; + + // start a thread to check leader lease + g_check_lease_in_thread_stop = false; + pthread_t tid; + ASSERT_EQ(0, pthread_create(&tid, NULL, check_lease_in_thread, &cluster)); + + cluster.wait_leader(); + + for (int i = 1; i < 10; ++i) { + LOG(NOTICE) << "start peer " << i; + braft::PeerId peer = peer0; + peer.addr.port += i; + ASSERT_EQ(0, cluster.start(peer.addr, true)); + } + + for (int i = 1; i < 10; ++i) { + LOG(NOTICE) << "add peer " << i; + cluster.wait_leader(); + braft::Node* leader = cluster.leader(); + braft::PeerId peer = peer0; + peer.addr.port += i; + braft::SynchronizedClosure done; + leader->add_peer(peer, &done); + usleep(50 * 1000); + done.wait(); + ASSERT_TRUE(done.status().ok()) << done.status(); + } + + for (int i = 1; i < 10; ++i) { + LOG(NOTICE) << "remove peer " << i; + cluster.wait_leader(); + braft::Node* leader = cluster.leader(); + braft::PeerId peer = peer0; + peer.addr.port += i; + braft::SynchronizedClosure done; + leader->remove_peer(peer, &done); + usleep(50 * 1000); + done.wait(); + ASSERT_TRUE(done.status().ok()) << done.status(); + } + + g_check_lease_in_thread_stop = true; + pthread_join(tid, NULL); + + cluster.stop_all(); +} + +TEST_F(LeaseTest, transfer_leadership_success) { + ::system("rm -rf data"); + 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, 500, 10); + 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(); + std::vector nodes; + cluster.followers(&nodes); + braft::PeerId target = nodes[0]->node_id().peer_id; + + braft::LeaderLeaseStatus old_leader_lease; + braft::LeaderLeaseStatus new_leader_lease; + leader->get_leader_lease_status(&old_leader_lease); + nodes[0]->get_leader_lease_status(&new_leader_lease); + ASSERT_EQ(old_leader_lease.state, braft::LEASE_VALID); + ASSERT_EQ(new_leader_lease.state, braft::LEASE_EXPIRED); + + braft::SynchronizedClosure done; + static_cast(nodes[0]->_impl->_options.fsm)->set_on_leader_start_closure(&done); + + ASSERT_EQ(0, leader->transfer_leadership_to(target)); + done.wait(); + + leader->get_leader_lease_status(&old_leader_lease); + nodes[0]->get_leader_lease_status(&new_leader_lease); + + ASSERT_EQ(old_leader_lease.state, braft::LEASE_EXPIRED); + ASSERT_EQ(new_leader_lease.state, braft::LEASE_VALID); + + leader = cluster.leader(); + ASSERT_EQ(target, leader->node_id().peer_id); + + cluster.stop_all(); +} + +TEST_F(LeaseTest, transfer_leadership_timeout) { + ::system("rm -rf data"); + 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, 500, 10); + 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(); + std::vector nodes; + cluster.followers(&nodes); + braft::PeerId target = nodes[0]->node_id().peer_id; + + braft::LeaderLeaseStatus old_leader_lease; + braft::LeaderLeaseStatus new_leader_lease; + leader->get_leader_lease_status(&old_leader_lease); + ASSERT_EQ(old_leader_lease.state, braft::LEASE_VALID); + + cluster.stop(nodes[0]->node_id().peer_id.addr); + + braft::SynchronizedClosure done; + static_cast(leader->_impl->_options.fsm)->set_on_leader_start_closure(&done); + ASSERT_EQ(0, leader->transfer_leadership_to(target)); + done.wait(); + + leader->get_leader_lease_status(&new_leader_lease); + + ASSERT_EQ(old_leader_lease.state, braft::LEASE_VALID); + ASSERT_EQ(old_leader_lease.term, new_leader_lease.term); + ASSERT_LT(old_leader_lease.lease_epoch, new_leader_lease.lease_epoch); + + cluster.stop_all(); +} + +TEST_F(LeaseTest, vote) { + ::system("rm -rf data"); + 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, 500, 100); + 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(); + std::vector nodes; + cluster.followers(&nodes); + braft::PeerId target = nodes[0]->node_id().peer_id; + + braft::LeaderLeaseStatus old_leader_lease; + braft::LeaderLeaseStatus new_leader_lease; + leader->get_leader_lease_status(&old_leader_lease); + nodes[0]->get_leader_lease_status(&new_leader_lease); + ASSERT_EQ(old_leader_lease.state, braft::LEASE_VALID); + ASSERT_EQ(new_leader_lease.state, braft::LEASE_EXPIRED); + + braft::SynchronizedClosure done; + static_cast(nodes[0]->_impl->_options.fsm)->set_on_leader_start_closure(&done); + + int64_t vote_begin_ms = butil::monotonic_time_ms(); + nodes[0]->vote(50); + done.wait(); + ASSERT_LT(butil::monotonic_time_ms() - vote_begin_ms, 500); + + leader->get_leader_lease_status(&old_leader_lease); + nodes[0]->get_leader_lease_status(&new_leader_lease); + + ASSERT_EQ(old_leader_lease.state, braft::LEASE_EXPIRED); + ASSERT_EQ(new_leader_lease.state, braft::LEASE_VALID); + + leader = cluster.leader(); + ASSERT_EQ(target, leader->node_id().peer_id); + + ASSERT_EQ(500 + 100 - 50, nodes[0]->_impl->_follower_lease._max_clock_drift_ms); + nodes[0]->reset_election_timeout_ms(500); + ASSERT_EQ(100, nodes[0]->_impl->_follower_lease._max_clock_drift_ms); + + cluster.stop_all(); +} + +TEST_F(LeaseTest, leader_step_down) { + ::system("rm -rf data"); + 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, 500, 1000); + 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(); + std::vector nodes; + cluster.followers(&nodes); + + braft::SynchronizedClosure done; + static_cast(nodes[0]->_impl->_options.fsm)->set_on_leader_start_closure(&done); + static_cast(nodes[1]->_impl->_options.fsm)->set_on_leader_start_closure(&done); + + int64_t begin_ms = butil::monotonic_time_ms(); + cluster.stop(leader->node_id().peer_id.addr); + done.wait(); + ASSERT_GT(butil::monotonic_time_ms() - begin_ms, 500 / 2 + 1000); + + leader = cluster.leader(); + braft::LeaderLeaseStatus lease_status; + leader->get_leader_lease_status(&lease_status); + + cluster.stop_all(); +} + +class OnLeaderStartHungClosure : public braft::SynchronizedClosure { +public: + OnLeaderStartHungClosure(int i) + : braft::SynchronizedClosure(), idx(i), hung(true), running(false) {} + + void Run() { + running = true; + LOG(WARNING) << "start run on leader start hung closure " << idx; + while (hung) { + bthread_usleep(10 * 1000); + } + LOG(WARNING) << "finish run on leader start hung closure" << idx; + braft::SynchronizedClosure::Run(); + running = false; + } + + int idx; + butil::atomic hung; + butil::atomic running; +}; + +TEST_F(LeaseTest, apply_thread_hung) { + ::system("rm -rf data"); + braft::LeaderLeaseStatus lease_status; + std::vector peers; + std::vector on_leader_start_closures; + 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); + on_leader_start_closures.push_back(new OnLeaderStartHungClosure(i)); + } + + // start cluster + Cluster cluster("unittest", peers, 500, 10); + for (size_t i = 0; i < peers.size(); i++) { + ASSERT_EQ(0, cluster.start(peers[i].addr, false, 30, on_leader_start_closures[i])); + } + + /* + cluster.all_nodes(&nodes); + for (size_t i = 0; i < nodes.size(); ++i) { + static_cast(nodes[i]->_impl->_options.fsm) + ->set_on_leader_start_closure(on_leader_start_closures[i]); + } + */ + + // elect leader + cluster.wait_leader(); + braft::Node* leader = cluster.leader(); + ASSERT_TRUE(leader != NULL); + LOG(WARNING) << "leader is " << leader->node_id(); + std::vector nodes; + cluster.followers(&nodes); + braft::PeerId target = nodes[0]->node_id().peer_id; + + cluster.stop(nodes[0]->node_id().peer_id.addr); + + ASSERT_EQ(0, leader->transfer_leadership_to(target)); + cluster.wait_leader(); + + // apply thread hung, lease status always be ready + leader->get_leader_lease_status(&lease_status); + ASSERT_EQ(lease_status.state, braft::LEASE_NOT_READY); + + // apply thread resume to work + for (auto c : on_leader_start_closures) { + c->hung = false; + if (c->running) { + LOG(WARNING) << "waiting leader resume"; + c->wait(); + LOG(WARNING) << "leader resumed"; + } + } + usleep(100 * 1000); + leader->get_leader_lease_status(&lease_status); + ASSERT_EQ(lease_status.state, braft::LEASE_VALID); + cluster.stop_all(); + + for (auto c : on_leader_start_closures) { + delete c; + } +} + +TEST_F(LeaseTest, chaos) { + ::system("rm -rf data"); + std::vector started_nodes; + for (int i = 0; i < 3; i++) { + braft::PeerId peer; + peer.addr.ip = butil::my_ip(); + peer.addr.port = 5006 + i; + peer.idx = 0; + started_nodes.push_back(peer); + } + + // start cluster + Cluster cluster("unittest", started_nodes, 500, 10); + for (size_t i = 0; i < started_nodes.size(); i++) { + ASSERT_EQ(0, cluster.start(started_nodes[i].addr)); + } + + g_check_lease_in_thread_stop = false; + pthread_t tid; + ASSERT_EQ(0, pthread_create(&tid, NULL, check_lease_in_thread, &cluster)); + + enum OpType { + NODE_START, + NODE_STOP, + TRANSFER_LEADER, + VOTE, + RESET_ELECTION_TIMEOUT, + OP_END, + }; + + std::vector stopped_nodes; + for (size_t i = 0; i < 500; ++i) { + OpType type = static_cast(butil::fast_rand() % OP_END); + if (type == NODE_START) { + if (stopped_nodes.empty()) { + type = NODE_STOP; + } + } else if (type == NODE_STOP || type == VOTE || type == TRANSFER_LEADER) { + if (stopped_nodes.size() == 2) { + type = NODE_START; + } + } + braft::Node* target_node = NULL; + switch (type) { + case NODE_START: { + BRAFT_VLOG << "chaos round " << i << ", node start"; + std::vector tmp_nodes; + size_t j = butil::fast_rand() % stopped_nodes.size(); + for (size_t t = 0; t < stopped_nodes.size(); ++t) { + if (t == j) { + cluster.start(stopped_nodes[t].addr); + started_nodes.push_back(stopped_nodes[t]); + } else { + tmp_nodes.push_back(stopped_nodes[t]); + } + } + tmp_nodes.swap(stopped_nodes); + break; + } + case NODE_STOP: { + BRAFT_VLOG << "chaos round " << i << ", node stop"; + std::vector tmp_nodes; + size_t j = butil::fast_rand() % started_nodes.size(); + for (size_t t = 0; t < started_nodes.size(); ++t) { + if (t == j) { + cluster.stop(started_nodes[t].addr); + stopped_nodes.push_back(started_nodes[t]); + } else { + tmp_nodes.push_back(started_nodes[t]); + } + } + tmp_nodes.swap(started_nodes); + break; + } + case TRANSFER_LEADER: { + BRAFT_VLOG << "chaos round " << i << ", transfer leader"; + cluster.wait_leader(); + braft::Node* leader = cluster.leader(); + braft::PeerId target; + if (stopped_nodes.empty() || butil::fast_rand() % 2 == 0) { + target = started_nodes[butil::fast_rand() % started_nodes.size()]; + } else { + target = stopped_nodes[butil::fast_rand() % stopped_nodes.size()]; + } + leader->transfer_leadership_to(target); + break; + } + case VOTE: { + BRAFT_VLOG << "chaos round " << i << ", vote"; + braft::PeerId target = started_nodes[butil::fast_rand() % started_nodes.size()]; + target_node = cluster.find_node(target); + target_node->vote(50); + break; + } + case RESET_ELECTION_TIMEOUT: { + BRAFT_VLOG << "chaos round " << i << ", vote"; + braft::PeerId target = started_nodes[butil::fast_rand() % started_nodes.size()]; + target_node = cluster.find_node(target); + target_node->reset_election_timeout_ms(butil::fast_rand_in(50, 500)); + break; + } + case OP_END: + CHECK(false); + } + + usleep(20 * 1000); + if (type == VOTE) { + target_node->reset_election_timeout_ms(500); + } + } + + g_check_lease_in_thread_stop = true; + pthread_join(tid, NULL); + + cluster.stop_all(); +} diff --git a/test/test_log.cpp b/test/test_log.cpp index 51be7c9e..d68b80f5 100644 --- a/test/test_log.cpp +++ b/test/test_log.cpp @@ -17,11 +17,17 @@ #include #include "braft/util.h" #include "braft/log.h" +#include "braft/storage.h" + +namespace braft { +DECLARE_bool(raft_trace_append_entry_latency); +} class LogStorageTest : public testing::Test { protected: void SetUp() { braft::FLAGS_raft_sync = false; + GFLAGS_NS::SetCommandLineOption("minloglevel", "3"); } void TearDown() {} }; @@ -268,7 +274,7 @@ TEST_F(LogStorageTest, multi_segment_and_segment_logstorage) { entries.push_back(entry); } - ASSERT_EQ(5, storage->append_entries(entries)); + ASSERT_EQ(5, storage->append_entries(entries, NULL)); for (size_t j = 0; j < entries.size(); j++) { entries[j]->Release(); @@ -343,7 +349,7 @@ TEST_F(LogStorageTest, multi_segment_and_segment_logstorage) { entries.push_back(entry); } - ASSERT_EQ(5, storage->append_entries(entries)); + ASSERT_EQ(5, storage->append_entries(entries, NULL)); for (size_t j = 0; j < entries.size(); j++) { delete entries[j]; @@ -420,7 +426,7 @@ TEST_F(LogStorageTest, append_close_load_append) { entries.push_back(entry); } - ASSERT_EQ(5, storage->append_entries(entries)); + ASSERT_EQ(5, storage->append_entries(entries, NULL)); for (size_t j = 0; j < entries.size(); j++) { delete entries[j]; @@ -451,7 +457,7 @@ TEST_F(LogStorageTest, append_close_load_append) { entries.push_back(entry); } - ASSERT_EQ(5, storage->append_entries(entries)); + ASSERT_EQ(5, storage->append_entries(entries, NULL)); for (size_t j = 0; j < entries.size(); j++) { delete entries[j]; @@ -519,7 +525,7 @@ TEST_F(LogStorageTest, data_lost) { entries.push_back(entry); } - ASSERT_EQ(5, storage->append_entries(entries)); + ASSERT_EQ(5, storage->append_entries(entries, NULL)); for (size_t j = 0; j < entries.size(); j++) { delete entries[j]; @@ -609,7 +615,7 @@ TEST_F(LogStorageTest, full_segment_has_garbage) { entries.push_back(entry); } - ASSERT_EQ(5, storage->append_entries(entries)); + ASSERT_EQ(5, storage->append_entries(entries, NULL)); for (size_t j = 0; j < entries.size(); j++) { delete entries[j]; @@ -690,7 +696,7 @@ TEST_F(LogStorageTest, append_read_badcase) { entries.push_back(entry); } - ASSERT_EQ(5, storage->append_entries(entries)); + ASSERT_EQ(5, storage->append_entries(entries, NULL)); for (size_t j = 0; j < entries.size(); j++) { delete entries[j]; @@ -817,7 +823,7 @@ TEST_F(LogStorageTest, configuration) { entry->data.append(data_buf); entries.push_back(entry); } - ASSERT_EQ(5, storage->append_entries(entries)); + ASSERT_EQ(5, storage->append_entries(entries, NULL)); for (size_t j = 0; j < entries.size(); j++) { delete entries[j]; @@ -1171,3 +1177,95 @@ TEST_F(LogStorageTest, joint_configuration) { } } +TEST_F(LogStorageTest, append_close_load_append_with_io_metric) { + ::system("rm -rf data"); + braft::IOMetric metric; + braft::FLAGS_raft_trace_append_entry_latency = true; + 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, &metric)); + + for (size_t j = 0; j < entries.size(); j++) { + delete entries[j]; + } + } + + ASSERT_NE(0, metric.open_segment_time_us); + ASSERT_NE(0, metric.append_entry_time_us); + ASSERT_NE(0, metric.sync_segment_time_us); + + LOG(NOTICE) << metric; + + delete storage; + delete configuration_manager; + + // reinit + storage = new braft::SegmentLogStorage("./data"); + configuration_manager = new braft::ConfigurationManager; + ASSERT_EQ(0, storage->init(configuration_manager)); + + // append entry + for (int i = 100000; i < 200000; 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 = 2; + 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, &metric)); + + for (size_t j = 0; j < entries.size(); j++) { + delete entries[j]; + } + } + + // check and read + ASSERT_EQ(storage->first_log_index(), 1); + ASSERT_EQ(storage->last_log_index(), 200000*5); + + for (int i = 0; i < 200000*5; i++) { + int64_t index = i + 1; + braft::LogEntry* entry = storage->get_entry(index); + if (i < 100000*5) { + ASSERT_EQ(entry->id.term, 1); + } else { + ASSERT_EQ(entry->id.term, 2); + } + ASSERT_EQ(entry->type, braft::ENTRY_TYPE_DATA); + ASSERT_EQ(entry->id.index, index); + + char data_buf[128]; + snprintf(data_buf, sizeof(data_buf), "hello, world: %ld", index); + ASSERT_EQ(data_buf, entry->data.to_string()); + entry->Release(); + } + + delete storage; + delete configuration_manager; +} diff --git a/test/test_memory_storage.cpp b/test/test_memory_storage.cpp index 000f58a1..88e16ce1 100644 --- a/test/test_memory_storage.cpp +++ b/test/test_memory_storage.cpp @@ -41,7 +41,7 @@ TEST_F(MemStorageTest, entry_operation) { entry->type = braft::ENTRY_TYPE_DATA; std::vector entries; entries.push_back(entry); - ASSERT_EQ(1u, log_storage->append_entries(entries)); + ASSERT_EQ(1u, log_storage->append_entries(entries, NULL)); ASSERT_EQ(1, log_storage->first_log_index()); ASSERT_EQ(1, log_storage->last_log_index()); @@ -81,7 +81,7 @@ TEST_F(MemStorageTest, trunk_operation) { entry2->id = braft::LogId(3, 1); entry2->type = braft::ENTRY_TYPE_DATA; entries.push_back(entry2); - ASSERT_EQ(3u, log_storage->append_entries(entries)); + ASSERT_EQ(3u, log_storage->append_entries(entries, NULL)); size_t ret = log_storage->truncate_suffix(2); ASSERT_EQ(0, ret); diff --git a/test/test_meta.cpp b/test/test_meta.cpp index c871ef16..47433b74 100644 --- a/test/test_meta.cpp +++ b/test/test_meta.cpp @@ -1,62 +1,531 @@ +#include #include +#include +#include "braft/raft.h" #include "braft/raft_meta.h" +namespace braft { +extern void global_init_once_or_die(); +}; + class TestUsageSuits : public testing::Test { protected: - void SetUp() {} + void SetUp() { + braft::global_init_once_or_die(); + } void TearDown() {} }; -TEST_F(TestUsageSuits, sanity) { - braft::LocalRaftMetaStorage* storage = new braft::LocalRaftMetaStorage("./data"); +TEST_F(TestUsageSuits, single_stable_storage) { + system("rm -rf stable"); + braft::FileBasedSingleMetaStorage* storage = + new braft::FileBasedSingleMetaStorage("./stable"); + int64_t term; + braft::PeerId any_peer; + butil::Status st; + // not init + { + term = 10; + braft::PeerId candidate; + ASSERT_EQ(0, candidate.parse("1.1.1.1:1000:0")); + ASSERT_NE(0, candidate.parse("1.1.1.1,1000,0")); + st = storage->set_term_and_votedfor(term, candidate, ""); + ASSERT_FALSE(st.ok()); + int64_t term_bak = 0; + braft::PeerId peer_bak; + st = storage->get_term_and_votedfor(&term_bak, &peer_bak, ""); + ASSERT_FALSE(st.ok()); + ASSERT_EQ(0, term_bak); + ASSERT_EQ(braft::ANY_PEER, peer_bak); + } + + ASSERT_TRUE(storage->init().ok()); + ASSERT_TRUE(storage->init().ok()); + { + term = 10; + st = storage->set_term_and_votedfor(term, any_peer, ""); + ASSERT_TRUE(st.ok()); + int64_t term_bak; + braft::PeerId peer_bak; + st = storage->get_term_and_votedfor(&term_bak, &peer_bak, ""); + ASSERT_TRUE(st.ok()); + ASSERT_EQ(10, term_bak); + ASSERT_EQ(braft::ANY_PEER, peer_bak); + + braft::PeerId candidate; + ASSERT_EQ(0, candidate.parse("1.1.1.1:1000:0")); + ASSERT_TRUE(storage-> + set_term_and_votedfor(term, candidate, "").ok()); + ASSERT_TRUE(storage-> + get_term_and_votedfor(&term_bak, &peer_bak, "").ok()); + ASSERT_EQ(peer_bak.addr, candidate.addr); + ASSERT_EQ(peer_bak.idx, candidate.idx); + + term = 11; + braft::PeerId candidate2; + ASSERT_EQ(0, candidate2.parse("2.2.2.2:2000:0")); + ASSERT_TRUE(storage-> + set_term_and_votedfor(term, candidate2, "").ok()); + } + delete storage; - // no init + storage = new braft::FileBasedSingleMetaStorage("./stable"); + ASSERT_TRUE(storage->init().ok()); { - ASSERT_EQ(-1, storage->set_term(10)); - int64_t term = storage->get_term(); - ASSERT_EQ(term, -1L); + int64_t term_bak; + braft::PeerId peer_bak; + st = storage->get_term_and_votedfor(&term_bak, &peer_bak, ""); + ASSERT_TRUE(st.ok()); + ASSERT_EQ(term, term_bak); + + butil::ip_t ip; + butil::str2ip("2.2.2.2", &ip); + ASSERT_EQ(peer_bak.addr.ip, ip); + ASSERT_EQ(peer_bak.addr.port, 2000); + ASSERT_EQ(peer_bak.idx, 0); + } + delete storage; +} + +TEST_F(TestUsageSuits, merged_stable_storage) { + system("rm -rf merged_stable"); + braft::KVBasedMergedMetaStorage* storage = + new braft::KVBasedMergedMetaStorage("./merged_stable"); + // group_id = "pool_ssd_0", index = 0 + std::string v_group_id = "pool_ssd_0_0"; + int64_t term; + braft::PeerId any_peer; + butil::Status st; + // not init + { + term = 10; braft::PeerId candidate; ASSERT_EQ(0, candidate.parse("1.1.1.1:1000:0")); ASSERT_NE(0, candidate.parse("1.1.1.1,1000,0")); - ASSERT_EQ(-1, storage->set_votedfor(candidate)); + st = storage->set_term_and_votedfor(term, candidate, v_group_id); + ASSERT_FALSE(st.ok()); + int64_t term_bak = 0; + braft::PeerId peer_bak; + st = storage->get_term_and_votedfor(&term_bak, &peer_bak, v_group_id); + ASSERT_FALSE(st.ok()); + ASSERT_EQ(0, term_bak); + ASSERT_EQ(braft::ANY_PEER, peer_bak); + } + + ASSERT_TRUE(storage->init().ok()); + ASSERT_TRUE(storage->init().ok()); + { + term = 10; + st = storage->set_term_and_votedfor(term, any_peer, v_group_id); + ASSERT_TRUE(st.ok()); + int64_t term_bak; + braft::PeerId peer_bak; + st = storage->get_term_and_votedfor(&term_bak, &peer_bak, v_group_id); + ASSERT_TRUE(st.ok()); + ASSERT_EQ(10, term_bak); + ASSERT_EQ(braft::ANY_PEER, peer_bak); + + braft::PeerId candidate; + ASSERT_EQ(0, candidate.parse("1.1.1.1:1000:0")); + ASSERT_TRUE(storage-> + set_term_and_votedfor(term, candidate, v_group_id).ok()); + ASSERT_TRUE(storage-> + get_term_and_votedfor(&term_bak, &peer_bak, v_group_id).ok()); + ASSERT_EQ(peer_bak.addr, candidate.addr); + ASSERT_EQ(peer_bak.idx, candidate.idx); + + term = 11; braft::PeerId candidate2; - ASSERT_EQ(-1, storage->get_votedfor(&candidate2)); - ASSERT_NE(0, storage->set_term_and_votedfor(10, candidate)); + ASSERT_EQ(0, candidate2.parse("2.2.2.2:2000:0")); + ASSERT_TRUE(storage-> + set_term_and_votedfor(term, candidate2, v_group_id).ok()); + } + delete storage; + + storage = new braft::KVBasedMergedMetaStorage("./merged_stable"); + ASSERT_TRUE(storage->init().ok()); + { + int64_t term_bak; + braft::PeerId peer_bak; + st = storage->get_term_and_votedfor(&term_bak, &peer_bak, v_group_id); + ASSERT_TRUE(st.ok()); + ASSERT_EQ(term, term_bak); + + butil::ip_t ip; + butil::str2ip("2.2.2.2", &ip); + ASSERT_EQ(peer_bak.addr.ip, ip); + ASSERT_EQ(peer_bak.addr.port, 2000); + ASSERT_EQ(peer_bak.idx, 0); } + delete storage; +} - ASSERT_EQ(0, storage->init()); - ASSERT_EQ(0, storage->init()); +TEST_F(TestUsageSuits, mixed_stable_storage_upgrade) { + const std::string uri = "local://./disk1/replica_pool_ssd_0_0/stable"; + const std::string uri_mixed = "local-mixed://merged_path=./disk1/merged_stable" + "&&single_path=./disk1/replica_pool_ssd_0_0/stable"; + const std::string uri_merged = "local-merged://./disk1/merged_stable"; + // group_id = "pool_ssd_0", index = 0 + std::string v_group_id = "pool_ssd_0_0"; + braft::RaftMetaStorage::destroy(uri_merged, v_group_id); + system("rm -rf ./disk1"); + + // check init with only single_stable_storage + braft::RaftMetaStorage* storage = braft::RaftMetaStorage::create(uri); + { + ASSERT_TRUE(storage->init().ok()); + braft::FileBasedSingleMetaStorage* tmp = + dynamic_cast(storage); + ASSERT_TRUE(tmp); + } + int64_t term; + braft::PeerId any_peer; + butil::Status st; + + // test default value { - ASSERT_EQ(0, storage->set_term(10)); - int64_t term = storage->get_term(); - ASSERT_EQ(term, 10); + int64_t term_bak; + braft::PeerId peer_bak; + st = storage->get_term_and_votedfor(&term_bak, &peer_bak, v_group_id); + ASSERT_TRUE(st.ok()); + ASSERT_EQ(1, term_bak); + ASSERT_EQ(braft::ANY_PEER, peer_bak); + } + + // test single stable storage alone + { + term = 10; + st = storage->set_term_and_votedfor(term, any_peer, v_group_id); + ASSERT_TRUE(st.ok()); + int64_t term_bak; + braft::PeerId peer_bak; + st = storage->get_term_and_votedfor(&term_bak, &peer_bak, v_group_id); + ASSERT_TRUE(st.ok()); + ASSERT_EQ(10, term_bak); + ASSERT_EQ(braft::ANY_PEER, peer_bak); + braft::PeerId candidate; ASSERT_EQ(0, candidate.parse("1.1.1.1:1000:0")); - ASSERT_EQ(0, storage->set_votedfor(candidate)); + ASSERT_TRUE(storage-> + set_term_and_votedfor(term, candidate, v_group_id).ok()); + ASSERT_TRUE(storage-> + get_term_and_votedfor(&term_bak, &peer_bak, v_group_id).ok()); + ASSERT_EQ(peer_bak.addr, candidate.addr); + ASSERT_EQ(peer_bak.idx, candidate.idx); + + term = 11; braft::PeerId candidate2; - ASSERT_EQ(0, storage->get_votedfor(&candidate2)); - ASSERT_EQ(candidate2.addr, candidate.addr); - ASSERT_EQ(candidate2.idx, candidate.idx); + ASSERT_EQ(0, candidate2.parse("2.2.2.2:2000:0")); + ASSERT_TRUE(storage-> + set_term_and_votedfor(term, candidate2, v_group_id).ok()); + } + delete storage; + + // test reload with only single stable storage + storage = braft::RaftMetaStorage::create(uri); + ASSERT_TRUE(storage->init().ok()); + { + int64_t term_bak; + braft::PeerId peer_bak; + st = storage->get_term_and_votedfor(&term_bak, &peer_bak, v_group_id); + ASSERT_TRUE(st.ok()); + ASSERT_EQ(term, term_bak); + + butil::ip_t ip; + butil::str2ip("2.2.2.2", &ip); + ASSERT_EQ(peer_bak.addr.ip, ip); + ASSERT_EQ(peer_bak.addr.port, 2000); + ASSERT_EQ(peer_bak.idx, 0); + } + delete storage; - braft::PeerId candidate3; - ASSERT_EQ(0, candidate3.parse("2.2.2.2:2000:0")); - ASSERT_EQ(0, storage->set_term_and_votedfor(11, candidate3)); + // test upgrade stable storage from Single to Merged + // stage1: use mixed stable storage + + // test init state with both + storage = braft::RaftMetaStorage::create(uri_mixed); + ASSERT_TRUE(storage->init().ok()); + braft::MixedMetaStorage* tmp = + dynamic_cast(storage); + ASSERT_TRUE(tmp); + ASSERT_FALSE(tmp->is_bad()); + ASSERT_TRUE(tmp->_single_impl); + ASSERT_TRUE(tmp->_merged_impl); + + // test _merged_impl catch up data + { + // initial data of _merged_impl + int64_t term_bak = 0; + braft::PeerId peer_bak; + st = tmp->_merged_impl->get_term_and_votedfor(&term_bak, &peer_bak, v_group_id); + ASSERT_TRUE(st.ok()); + ASSERT_EQ(1, term_bak); + ASSERT_EQ(braft::ANY_PEER, peer_bak); + } + { + // _merged_impl catch up data when Mixed first load + int64_t term_bak; + braft::PeerId peer_bak; + st = storage->get_term_and_votedfor(&term_bak, &peer_bak, v_group_id); + ASSERT_TRUE(st.ok()); + ASSERT_EQ(term, term_bak); + ASSERT_EQ("2.2.2.2:2000:0", peer_bak.to_string()); + } + { + // _merged_impl already catch up data after Mixed first load + int64_t term_bak; + braft::PeerId peer_bak; + st = tmp->_merged_impl->get_term_and_votedfor(&term_bak, &peer_bak, v_group_id); + ASSERT_TRUE(st.ok()); + ASSERT_EQ(term, term_bak); + ASSERT_EQ("2.2.2.2:2000:0", peer_bak.to_string()); } + // test double write + { + braft::PeerId candidate3; + term = 12; + ASSERT_EQ(0, candidate3.parse("3.3.3.3:3000:3")); + st = storage->set_term_and_votedfor(term, candidate3, v_group_id); + ASSERT_TRUE(st.ok()); + + int64_t term_bak; + braft::PeerId peer_bak; + st = tmp->_single_impl->get_term_and_votedfor(&term_bak, &peer_bak, v_group_id); + ASSERT_TRUE(st.ok()); + ASSERT_EQ(term, term_bak); + ASSERT_EQ("3.3.3.3:3000:3", peer_bak.to_string()); + + term_bak = 0; + peer_bak.reset(); + st = tmp->_merged_impl->get_term_and_votedfor(&term_bak, &peer_bak, v_group_id); + ASSERT_TRUE(st.ok()); + ASSERT_EQ(term, term_bak); + ASSERT_EQ("3.3.3.3:3000:3", peer_bak.to_string()); + } delete storage; - storage = new braft::LocalRaftMetaStorage("./data"); - ASSERT_EQ(0, storage->init()); + + // test change type of stable storage + // stage2: use merged stable storage + + // test init state with only merged + storage = braft::RaftMetaStorage::create(uri_merged); + { + ASSERT_TRUE(storage->init().ok()); + braft::KVBasedMergedMetaStorage* tmp = + dynamic_cast(storage); + ASSERT_TRUE(tmp); + } + + // test reload with only merged stable storage { - int64_t term = storage->get_term(); - ASSERT_EQ(term, 11); + int64_t term_bak; + braft::PeerId peer_bak; + st = storage->get_term_and_votedfor(&term_bak, &peer_bak, v_group_id); + ASSERT_TRUE(st.ok()); + ASSERT_EQ(term, term_bak); + ASSERT_EQ("3.3.3.3:3000:3", peer_bak.to_string()); + } + // test merged stable storage alone + { + braft::PeerId candidate4; + term = 13; + ASSERT_EQ(0, candidate4.parse("4.4.4.4:4000:4")); + st = storage->set_term_and_votedfor(term, candidate4, v_group_id); + ASSERT_TRUE(st.ok()); + + int64_t term_bak; + braft::PeerId peer_bak; + st = storage->get_term_and_votedfor(&term_bak, &peer_bak, v_group_id); + ASSERT_TRUE(st.ok()); + ASSERT_EQ(term, term_bak); + ASSERT_EQ("4.4.4.4:4000:4", peer_bak.to_string()); + } + delete storage; +} + +TEST_F(TestUsageSuits, mixed_stable_storage_downgrade) { + const std::string uri_single = "local://./disk1/replica_pool_ssd_0_0/stable"; + const std::string uri_mixed = "local-mixed://merged_path=./disk1/merged_stable" + "&&single_path=./disk1/replica_pool_ssd_0_0/stable"; + const std::string uri_merged = "local-merged://./disk1/merged_stable"; + // group_id = "pool_ssd_0", index = 0 + std::string v_group_id = "pool_ssd_0_0"; + braft::RaftMetaStorage::destroy(uri_merged, v_group_id); + system("rm -rf ./disk1"); + + // check init with only merged_stable_storage + braft::RaftMetaStorage* storage = braft::RaftMetaStorage::create(uri_merged); + { + ASSERT_TRUE(storage->init().ok()); + braft::KVBasedMergedMetaStorage* tmp = + dynamic_cast(storage); + ASSERT_TRUE(tmp); + } + int64_t term; + braft::PeerId any_peer; + butil::Status st; + + // test default value + { + int64_t term_bak; + braft::PeerId peer_bak; + st = storage->get_term_and_votedfor(&term_bak, &peer_bak, v_group_id); + ASSERT_TRUE(st.ok()); + ASSERT_EQ(1, term_bak); + ASSERT_EQ(braft::ANY_PEER, peer_bak); + } + + // test merged stable storage alone + { + term = 10; + st = storage->set_term_and_votedfor(term, any_peer, v_group_id); + ASSERT_TRUE(st.ok()); + int64_t term_bak; + braft::PeerId peer_bak; + st = storage->get_term_and_votedfor(&term_bak, &peer_bak, v_group_id); + ASSERT_TRUE(st.ok()); + ASSERT_EQ(10, term_bak); + ASSERT_EQ(braft::ANY_PEER, peer_bak); + + braft::PeerId candidate; + ASSERT_EQ(0, candidate.parse("1.1.1.1:1000:0")); + ASSERT_TRUE(storage-> + set_term_and_votedfor(term, candidate, v_group_id).ok()); + ASSERT_TRUE(storage-> + get_term_and_votedfor(&term_bak, &peer_bak, v_group_id).ok()); + ASSERT_EQ(peer_bak.addr, candidate.addr); + ASSERT_EQ(peer_bak.idx, candidate.idx); + + term = 11; braft::PeerId candidate2; - ASSERT_EQ(0, storage->get_votedfor(&candidate2)); + ASSERT_EQ(0, candidate2.parse("2.2.2.2:2000:0")); + ASSERT_TRUE(storage-> + set_term_and_votedfor(term, candidate2, v_group_id).ok()); + } + delete storage; + + // test reload with only merged stable storage + storage = braft::RaftMetaStorage::create(uri_merged); + ASSERT_TRUE(storage->init().ok()); + { + int64_t term_bak; + braft::PeerId peer_bak; + st = storage->get_term_and_votedfor(&term_bak, &peer_bak, v_group_id); + ASSERT_TRUE(st.ok()); + ASSERT_EQ(term, term_bak); + butil::ip_t ip; butil::str2ip("2.2.2.2", &ip); - ASSERT_EQ(candidate2.addr.ip, ip); - ASSERT_EQ(candidate2.addr.port, 2000); - ASSERT_EQ(candidate2.idx, 0); + ASSERT_EQ(peer_bak.addr.ip, ip); + ASSERT_EQ(peer_bak.addr.port, 2000); + ASSERT_EQ(peer_bak.idx, 0); } delete storage; + + // test downgrade stable storage from Merged to Single + // stage1: use mixed stable storage + + // test init state with both + storage = braft::RaftMetaStorage::create(uri_mixed); + ASSERT_TRUE(storage->init().ok()); + braft::MixedMetaStorage* tmp = + dynamic_cast(storage); + ASSERT_TRUE(tmp); + ASSERT_FALSE(tmp->is_bad()); + ASSERT_TRUE(tmp->_single_impl); + ASSERT_TRUE(tmp->_merged_impl); + + // test _single_impl catch up data + { + // initial data of _single_impl + int64_t term_bak = 0; + braft::PeerId peer_bak; + st = tmp->_single_impl->get_term_and_votedfor(&term_bak, &peer_bak, v_group_id); + ASSERT_TRUE(st.ok()); + ASSERT_EQ(1, term_bak); + ASSERT_EQ(braft::ANY_PEER, peer_bak); + } + { + // _single_impl catch up data when Mixed first load + int64_t term_bak; + braft::PeerId peer_bak; + st = storage->get_term_and_votedfor(&term_bak, &peer_bak, v_group_id); + ASSERT_TRUE(st.ok()); + ASSERT_EQ(term, term_bak); + ASSERT_EQ("2.2.2.2:2000:0", peer_bak.to_string()); + } + { + // _single_impl already catch up data after Mixed first load + int64_t term_bak; + braft::PeerId peer_bak; + st = tmp->_single_impl->get_term_and_votedfor(&term_bak, &peer_bak, v_group_id); + ASSERT_TRUE(st.ok()); + ASSERT_EQ(term, term_bak); + ASSERT_EQ("2.2.2.2:2000:0", peer_bak.to_string()); + } + + // test double write + { + braft::PeerId candidate3; + term = 12; + ASSERT_EQ(0, candidate3.parse("3.3.3.3:3000:3")); + st = storage->set_term_and_votedfor(term, candidate3, v_group_id); + ASSERT_TRUE(st.ok()); + + int64_t term_bak; + braft::PeerId peer_bak; + st = tmp->_single_impl->get_term_and_votedfor(&term_bak, &peer_bak, v_group_id); + ASSERT_TRUE(st.ok()); + ASSERT_EQ(term, term_bak); + ASSERT_EQ("3.3.3.3:3000:3", peer_bak.to_string()); + + term_bak = 0; + peer_bak.reset(); + st = tmp->_merged_impl->get_term_and_votedfor(&term_bak, &peer_bak, v_group_id); + ASSERT_TRUE(st.ok()); + ASSERT_EQ(term, term_bak); + ASSERT_EQ("3.3.3.3:3000:3", peer_bak.to_string()); + } + delete storage; + + + // test change type of stable storage + // stage2: use single stable storage + + // test init state with only single + storage = braft::RaftMetaStorage::create(uri_single); + { + ASSERT_TRUE(storage->init().ok()); + braft::FileBasedSingleMetaStorage* tmp = + dynamic_cast(storage); + ASSERT_TRUE(tmp); + } + + // test reload with only single stable storage + { + int64_t term_bak; + braft::PeerId peer_bak; + st = storage->get_term_and_votedfor(&term_bak, &peer_bak, v_group_id); + ASSERT_TRUE(st.ok()); + ASSERT_EQ(term, term_bak); + ASSERT_EQ("3.3.3.3:3000:3", peer_bak.to_string()); + } + // test single stable storage alone + { + braft::PeerId candidate4; + term = 13; + ASSERT_EQ(0, candidate4.parse("4.4.4.4:4000:4")); + st = storage->set_term_and_votedfor(term, candidate4, v_group_id); + ASSERT_TRUE(st.ok()); + + int64_t term_bak; + braft::PeerId peer_bak; + st = storage->get_term_and_votedfor(&term_bak, &peer_bak, v_group_id); + ASSERT_TRUE(st.ok()); + ASSERT_EQ(term, term_bak); + ASSERT_EQ("4.4.4.4:4000:4", peer_bak.to_string()); + } + delete storage; } diff --git a/test/test_node.cpp b/test/test_node.cpp index 83567445..bbc2070b 100644 --- a/test/test_node.cpp +++ b/test/test_node.cpp @@ -16,12 +16,7 @@ #include #include #include -#include "braft/snapshot_throttle.h" -#include "braft/node.h" -#include "braft/enum.pb.h" -#include "braft/errno.pb.h" -#include -#include +#include "braft/test/util.h" namespace braft { extern bvar::Adder g_num_nodes; @@ -39,472 +34,12 @@ class TestEnvironment : public ::testing::Environment { } }; -bool g_dont_print_apply_log = false; -class MockFSM : public braft::StateMachine { -public: - MockFSM(const butil::EndPoint& address_) - : address(address_) - , applied_index(0) - , snapshot_index(0) - , _on_start_following_times(0) - , _on_stop_following_times(0) - , _leader_term(-1) - { - pthread_mutex_init(&mutex, NULL); - } - virtual ~MockFSM() { - pthread_mutex_destroy(&mutex); - } - - butil::EndPoint address; - std::vector logs; - pthread_mutex_t mutex; - int64_t applied_index; - int64_t snapshot_index; - int64_t _on_start_following_times; - int64_t _on_stop_following_times; - volatile int64_t _leader_term; - - void lock() { - pthread_mutex_lock(&mutex); - } - - void on_leader_start(int64_t term) { - _leader_term = term; - } - void on_leader_stop(const butil::Status&) { - _leader_term = -1; - } - - bool is_leader() { return _leader_term > 0; } - - void unlock() { - pthread_mutex_unlock(&mutex); - } - - 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(); - ::brpc::ClosureGuard guard(iter.done()); - lock(); - logs.push_back(iter.data()); - unlock(); - applied_index = iter.index(); - } - } - - virtual void on_shutdown() { - LOG(TRACE) << "addr " << address << " shutdowned"; - } - - virtual void on_snapshot_save(braft::SnapshotWriter* writer, braft::Closure* done) { - std::string file_path = writer->get_path(); - file_path.append("/data"); - brpc::ClosureGuard done_guard(done); - - LOG(NOTICE) << "on_snapshot_save to " << file_path; - - int fd = ::creat(file_path.c_str(), 0644); - if (fd < 0) { - LOG(ERROR) << "create file failed, path: " << file_path << " err: " << berror(); - done->status().set_error(EIO, "Fail to create file"); - return; - } - lock(); - // write snapshot and log to file - for (size_t i = 0; i < logs.size(); i++) { - butil::IOBuf data = logs[i]; - int len = data.size(); - int ret = write(fd, &len, sizeof(int)); - CHECK_EQ(ret, 4); - data.cut_into_file_descriptor(fd, len); - } - ::close(fd); - snapshot_index = applied_index; - unlock(); - writer->add_file("data"); - } - - virtual int on_snapshot_load(braft::SnapshotReader* reader) { - std::string file_path = reader->get_path(); - file_path.append("/data"); - - LOG(INFO) << "on_snapshot_load from " << file_path; - - int fd = ::open(file_path.c_str(), O_RDONLY); - if (fd < 0) { - LOG(ERROR) << "creat file failed, path: " << file_path << " err: " << berror(); - return EIO; - } - - lock(); - logs.clear(); - while (true) { - int len = 0; - int ret = read(fd, &len, sizeof(int)); - if (ret <= 0) { - break; - } - - butil::IOPortal data; - data.append_from_file_descriptor(fd, len); - logs.push_back(data); - } - - ::close(fd); - unlock(); - return 0; - } - - virtual void on_start_following(const braft::LeaderChangeContext& start_following_context) { - LOG(TRACE) << "address " << address << " start following new leader: " - << start_following_context; - ++_on_start_following_times; - } - - virtual void on_stop_following(const braft::LeaderChangeContext& stop_following_context) { - LOG(TRACE) << "address " << address << " stop following old leader: " - << stop_following_context; - ++_on_stop_following_times; - } - - virtual void on_configuration_committed(const ::braft::Configuration& conf, int64_t index) { - LOG(TRACE) << "address " << address << " commit conf: " << conf << " at index " << index; - } - -}; - -class ExpectClosure : public braft::Closure { -public: - void Run() { - if (_expect_err_code >= 0) { - ASSERT_EQ(status().error_code(), _expect_err_code) - << _pos << " : " << status(); - } - if (_cond) { - _cond->signal(); - } - delete this; - } -private: - ExpectClosure(bthread::CountdownEvent* cond, int expect_err_code, const char* pos) - : _cond(cond), _expect_err_code(expect_err_code), _pos(pos) {} - - ExpectClosure(bthread::CountdownEvent* cond, const char* pos) - : _cond(cond), _expect_err_code(-1), _pos(pos) {} - - bthread::CountdownEvent* _cond; - int _expect_err_code; - const char* _pos; -}; - -typedef ExpectClosure ShutdownClosure; -typedef ExpectClosure ApplyClosure; -typedef ExpectClosure AddPeerClosure; -typedef ExpectClosure RemovePeerClosure; -typedef ExpectClosure SnapshotClosure; - -#define NEW_SHUTDOWNCLOSURE(arg...) \ - (new ExpectClosure(arg, __FILE__ ":" BAIDU_SYMBOLSTR(__LINE__))) -#define NEW_APPLYCLOSURE(arg...) \ - (new ExpectClosure(arg, __FILE__ ":" BAIDU_SYMBOLSTR(__LINE__))) -#define NEW_ADDPEERCLOSURE(arg...) \ - (new ExpectClosure(arg, __FILE__ ":" BAIDU_SYMBOLSTR(__LINE__))) -#define NEW_REMOVEPEERCLOSURE(arg...) \ - (new ExpectClosure(arg, __FILE__ ":" BAIDU_SYMBOLSTR(__LINE__))) -#define NEW_SNAPSHOTCLOSURE(arg...) \ - (new ExpectClosure(arg, __FILE__ ":" BAIDU_SYMBOLSTR(__LINE__))) - -class Cluster { -public: - Cluster(const std::string& name, const std::vector& peers, - int32_t election_timeout_ms = 3000) - : _name(name), _peers(peers) - , _election_timeout_ms(election_timeout_ms) { - int64_t throttle_throughput_bytes = 10 * 1024 * 1024; - int64_t check_cycle = 10; - _throttle = new braft::ThroughputSnapshotThrottle(throttle_throughput_bytes, check_cycle); - } - ~Cluster() { - stop_all(); - } - - int start(const butil::EndPoint& listen_addr, bool empty_peers = false, - int snapshot_interval_s = 30) { - if (_server_map[listen_addr] == NULL) { - brpc::Server* server = new brpc::Server(); - if (braft::add_service(server, listen_addr) != 0 - || server->Start(listen_addr, NULL) != 0) { - LOG(ERROR) << "Fail to start raft service"; - delete server; - return -1; - } - _server_map[listen_addr] = server; - } - - braft::NodeOptions options; - options.election_timeout_ms = _election_timeout_ms; - options.snapshot_interval_s = snapshot_interval_s; - if (!empty_peers) { - options.initial_conf = braft::Configuration(_peers); - } - MockFSM* fsm = new MockFSM(listen_addr); - options.fsm = fsm; - options.node_owns_fsm = true; - butil::string_printf(&options.log_uri, "local://./data/%s/log", - butil::endpoint2str(listen_addr).c_str()); - butil::string_printf(&options.raft_meta_uri, "local://./data/%s/raft_meta", - 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); - 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 node; - return ret; - } else { - LOG(NOTICE) << "init node " << listen_addr; - } - - { - std::lock_guard guard(_mutex); - _nodes.push_back(node); - _fsms.push_back(fsm); - } - return 0; - } - - int stop(const butil::EndPoint& listen_addr) { - - bthread::CountdownEvent cond; - braft::Node* node = remove_node(listen_addr); - if (node) { - node->shutdown(NEW_SHUTDOWNCLOSURE(&cond)); - cond.wait(); - node->join(); - } - - delete _server_map[listen_addr]; - _server_map.erase(listen_addr); - delete node; - return node ? 0 : -1; - } - - void stop_all() { - std::vector addrs; - std::vector nodes; - all_nodes(&addrs); - for (auto addr : addrs) { - braft::Node* node = remove_node(addr); - node->shutdown(NULL); - nodes.push_back(node); - std::cout << "Stopping server " << addr << std::endl; - delete _server_map[addr]; - _server_map.erase(addr); - } - for (auto node : nodes) { - std::cout << "join node " << node->node_id() << std::endl; - node->join(); - delete node; - } - } - - void clean(const butil::EndPoint& listen_addr) { - std::string data_path; - butil::string_printf(&data_path, "./data/%s", - butil::endpoint2str(listen_addr).c_str()); - - if (!butil::DeleteFile(butil::FilePath(data_path), true)) { - LOG(ERROR) << "delete path failed, path: " << data_path; - } - } - - braft::Node* leader() { - std::lock_guard guard(_mutex); - braft::Node* node = NULL; - for (size_t i = 0; i < _nodes.size(); i++) { - if (_nodes[i]->is_leader() && - _fsms[i]->_leader_term == _nodes[i]->_impl->_current_term) { - node = _nodes[i]; - break; - } - } - return node; - } - - void followers(std::vector* nodes) { - nodes->clear(); - - std::lock_guard guard(_mutex); - for (size_t i = 0; i < _nodes.size(); i++) { - if (!_nodes[i]->is_leader()) { - nodes->push_back(_nodes[i]); - } - } - } - - void wait_leader() { - while (true) { - braft::Node* node = leader(); - if (node) { - return; - } else { - usleep(100 * 1000); - } - } - } - - 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); - for (size_t i = 0; i < _nodes.size(); i++) { - braft::PeerId leader_id = _nodes[i]->leader_id(); - if (leader_id.addr != expect_addr) { - goto WAIT; - } - } - - return; -WAIT: - usleep(100 * 1000); - goto CHECK; - } - - bool ensure_same(int wait_time_s = -1) { - std::unique_lock guard(_mutex); - if (_fsms.size() <= 1) { - return true; - } - LOG(INFO) << "_fsms.size()=" << _fsms.size(); - - int nround = 0; - MockFSM* first = _fsms[0]; -CHECK: - first->lock(); - for (size_t i = 1; i < _fsms.size(); i++) { - MockFSM* fsm = _fsms[i]; - fsm->lock(); - - if (first->logs.size() != 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; - } - - for (size_t j = 0; j < first->logs.size(); j++) { - butil::IOBuf& first_data = first->logs[j]; - butil::IOBuf& fsm_data = fsm->logs[j]; - 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->unlock(); - goto WAIT; - } - } - - fsm->unlock(); - } - first->unlock(); - guard.unlock(); - check_node_status(); - - return true; -WAIT: - first->unlock(); - sleep(1); - ++nround; - if (wait_time_s > 0 && nround > wait_time_s) { - return false; - } - goto CHECK; - } - -private: - void all_nodes(std::vector* addrs) { - addrs->clear(); - - std::lock_guard guard(_mutex); - for (size_t i = 0; i < _nodes.size(); i++) { - addrs->push_back(_nodes[i]->node_id().peer_id.addr); - } - } - - braft::Node* remove_node(const butil::EndPoint& addr) { - std::lock_guard guard(_mutex); - - // remove node - braft::Node* node = NULL; - std::vector new_nodes; - for (size_t i = 0; i < _nodes.size(); i++) { - if (addr.port == _nodes[i]->node_id().peer_id.addr.port) { - node = _nodes[i]; - } else { - new_nodes.push_back(_nodes[i]); - } - } - _nodes.swap(new_nodes); - - // remove fsm - std::vector new_fsms; - for (size_t i = 0; i < _fsms.size(); i++) { - if (_fsms[i]->address != addr) { - new_fsms.push_back(_fsms[i]); - } - } - _fsms.swap(new_fsms); - - return node; - } - - std::string _name; - std::vector _peers; - std::vector _nodes; - std::vector _fsms; - std::map _server_map; - int32_t _election_timeout_ms; - raft_mutex_t _mutex; - braft::SnapshotThrottle* _throttle; -}; - class NodeTest : public testing::TestWithParam { protected: void SetUp() { g_dont_print_apply_log = false; //logging::FLAGS_v = 90; + GFLAGS_NS::SetCommandLineOption("minloglevel", "1"); GFLAGS_NS::SetCommandLineOption("crash_on_fatal_log", "true"); if (GetParam() == std::string("NoReplication")) { braft::FLAGS_raft_max_parallel_append_entries_rpc_num = 1; @@ -3280,7 +2815,7 @@ TEST_P(NodeTest, change_peers_chaos_without_snapshot) { std::cout << "Stopping cluster" << std::endl; cluster.stop_all(); GFLAGS_NS::SetCommandLineOption("raft_sync", "true"); - GFLAGS_NS::SetCommandLineOption("minloglevel", "0"); + GFLAGS_NS::SetCommandLineOption("minloglevel", "1"); } class AppendEntriesSyncClosure : public google::protobuf::Closure { diff --git a/test/test_snapshot_executor.cpp b/test/test_snapshot_executor.cpp index 867156b3..7d63bb0f 100644 --- a/test/test_snapshot_executor.cpp +++ b/test/test_snapshot_executor.cpp @@ -188,6 +188,10 @@ friend class MockSnapshotCopier; return NULL; } + virtual butil::Status gc_instance(const std::string& uri) const { + return butil::Status::OK(); + } + private: std::string _path; int64_t _last_snapshot_index; diff --git a/test/test_storage.cpp b/test/test_storage.cpp index 1b13d648..c9f7aea2 100644 --- a/test/test_storage.cpp +++ b/test/test_storage.cpp @@ -34,7 +34,7 @@ TEST_F(StorageTest, sanity) { braft::RaftMetaStorage* meta_storage = braft::RaftMetaStorage::create("local://data/raft_meta"); ASSERT_TRUE(meta_storage); - ASSERT_EQ(0, meta_storage->init()); + ASSERT_TRUE(meta_storage->init().ok()); ASSERT_FALSE(braft::RaftMetaStorage::create("hdfs://data/raft_meta")); ASSERT_FALSE(braft::RaftMetaStorage::create("://data/raft_meta")); ASSERT_FALSE(braft::RaftMetaStorage::create("data/raft_meta")); @@ -64,7 +64,7 @@ TEST_F(StorageTest, extra_space_should_be_trimmed) { entry->type = braft::ENTRY_TYPE_DATA; std::vector entries; entries.push_back(entry); - ASSERT_EQ(1u, log_storage->append_entries(entries)); + ASSERT_EQ(1u, log_storage->append_entries(entries, NULL)); entry->Release(); delete log_storage;