Commit a008c82e authored by Georgios Bitzes's avatar Georgios Bitzes
Browse files

Add protection for 1-way network partitions

parent ad223512
Pipeline #473471 passed with stages
in 31 minutes and 45 seconds
# Changelog
All notable changes to this project will be documented in this file.
## Unreleased
### Added
- Protection against 1-way network partitions, in which a cluster node
is able to establish TCP connections to others, but the rest cannot do the same.
This resulted in cluster disruption as the affected node would not be receiving
heartbeats, but could still repeatedly attempt to get elected.
From now on, a node which has been vetoed will abstain from starting election
rounds until it has received fresh heartbeats since receiving that veto.
## 0.3.1 (2018-08-03)
### Added
- Command `hclone` for creating identical copies of entire hashes.
......
......@@ -26,7 +26,8 @@
#include "RaftReplicator.hh"
#include "RaftLease.hh"
#include "../Dispatcher.hh"
using namespace quarkdb;
namespace quarkdb {
RaftDirector::RaftDirector(RaftJournal &jour, StateMachine &sm, RaftState &st, RaftLease &ls, RaftCommitTracker &ct, RaftClock &rc, RaftWriteTracker &wt, ShardDirectory &sharddir, RaftConfig &conf, RaftReplicator &rep, const RaftContactDetails &cd)
: journal(jour), stateMachine(sm), state(st), raftClock(rc), lease(ls), commitTracker(ct), writeTracker(wt), shardDirectory(sharddir), config(conf), replicator(rep), contactDetails(cd) {
......@@ -82,6 +83,10 @@ void RaftDirector::leaderLoop(RaftStateSnapshotPtr &snapshot) {
}
void RaftDirector::runForLeader() {
// If we get vetoed, this ensures we stop election attempts up until the
// point we receive a fresh heartbeat.
std::chrono::steady_clock::time_point lastHeartbeat = raftClock.getLastHeartbeat();
// don't reuse the snapshot from the main loop,
// it could have changed in-between
RaftStateSnapshotPtr snapshot = state.getSnapshot();
......@@ -100,9 +105,16 @@ void RaftDirector::runForLeader() {
return;
}
if(RaftElection::perform(votereq, state, lease, contactDetails) != ElectionOutcome::kElected) {
ElectionOutcome electionOutcome = RaftElection::perform(votereq, state, lease, contactDetails);
if(electionOutcome != ElectionOutcome::kElected) {
state.dropOut(snapshot->term+1);
}
if(electionOutcome == ElectionOutcome::kVetoed) {
lastHeartbeatBeforeVeto = lastHeartbeat;
qdb_info("Election round for term " << snapshot->term + 1 << " resulted in a veto. This means, the next leader of this cluster cannot be me. Stopping election attempts until I receive a heartbeat.");
}
}
void RaftDirector::followerLoop(RaftStateSnapshotPtr &snapshot) {
......@@ -114,7 +126,22 @@ void RaftDirector::followerLoop(RaftStateSnapshotPtr &snapshot) {
writeTracker.flushQueues(Formatter::err("unavailable"));
state.wait(randomTimeout);
if(raftClock.timeout()) {
if(raftClock.getLastHeartbeat() == lastHeartbeatBeforeVeto) {
// I've been vetoed during my last election attempt, and no heartbeat has
// appeared since then.
//
// It could be a network connectivity issue, where I'm able to establish
// TCP connections to other nodes (and thus disrupt them), but they cannot
// send me heartbeats.
//
// It could also be that I'm not a full member of this cluster, but I
// don't know it yet, and I'm being disruptive to the other nodes.
//
// Since a veto means the next cluster leader cannot be me, completely
// abstain from starting elections until we receive a heartbeat.
}
else if(raftClock.timeout()) {
if(contains(journal.getMembership().nodes, state.getMyself())) {
qdb_event(state.getMyself().toString() << ": TIMEOUT after " << randomTimeout.count() << "ms, I am not receiving heartbeats. Attempting to start election.");
runForLeader();
......@@ -124,3 +151,5 @@ void RaftDirector::followerLoop(RaftStateSnapshotPtr &snapshot) {
}
}
}
}
......@@ -63,6 +63,7 @@ private:
RaftReplicator &replicator;
const RaftContactDetails &contactDetails;
std::chrono::steady_clock::time_point lastHeartbeatBeforeVeto;
std::thread mainThread;
};
......
......@@ -134,3 +134,8 @@ milliseconds RaftClock::refreshRandomTimeout() {
randomTimeout = timeouts.getRandom();
return randomTimeout;
}
std::chrono::steady_clock::time_point RaftClock::getLastHeartbeat() {
std::lock_guard<std::mutex> lock(lastHeartbeatMutex);
return lastHeartbeat;
}
......@@ -72,6 +72,8 @@ public:
void heartbeat();
bool timeout();
std::chrono::steady_clock::time_point getLastHeartbeat();
RaftTimeouts getTimeouts() { return timeouts; }
milliseconds getRandomTimeout();
void triggerTimeout();
......
Markdown is supported
0% or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment