Skip to content

Commit

Permalink
issue #200: update leader timeouts after disk write
Browse files Browse the repository at this point in the history
in cases where disk writes can take longer than the leader election,
a follower can call for an election when one is not necessary. this
change resets the election related timers after all disk writes.
  • Loading branch information
Nate Hardt committed Dec 7, 2015
1 parent e9e5c50 commit 77b0470
Show file tree
Hide file tree
Showing 2 changed files with 14 additions and 2 deletions.
4 changes: 4 additions & 0 deletions RELEASES.md
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,10 @@ See [RELEASE-PROCESS.md](RELEASE-PROCESS.md).
Version 1.2.0-alpha.0 (In Development)
======================================

Bug fixes (low severity):

- #200: reset leader election timeout in follower after disk io completes

New backwards-compatible changes:

- Added new API getConfiguration2, which behaves as getConfiguration
Expand Down
12 changes: 10 additions & 2 deletions Server/RaftConsensus.cc
Original file line number Diff line number Diff line change
@@ -1,5 +1,6 @@
/* Copyright (c) 2012 Stanford University
* Copyright (c) 2015 Diego Ongaro
* Copyright (c) 2015 Scale Computing
*
* Permission to use, copy, modify, and distribute this software for any
* purpose with or without fee is hereby granted, provided that the above
Expand Down Expand Up @@ -1298,8 +1299,10 @@ RaftConsensus::handleAppendEntries(
// 'response' accordingly.
response.set_term(request.term());
}
// This request is a sign of life from the current leader. Update our term
// and convert to follower if necessary; reset the election timer.
// This request is a sign of life from the current leader. Update
// our term and convert to follower if necessary; reset the
// election timer. set it here in case request we exit the
// function early, we will set it again after the write.
stepDown(request.term());
setElectionTimer();
withholdVotesUntil = Clock::now() + ELECTION_TIMEOUT;
Expand Down Expand Up @@ -1416,6 +1419,11 @@ RaftConsensus::handleAppendEntries(
stateChanged.notify_all();
VERBOSE("New commitIndex: %lu", commitIndex);
}

// reset election timer to avoid punishing the leader for our own
// long disk writes
setElectionTimer();
withholdVotesUntil = Clock::now() + ELECTION_TIMEOUT;
}

void
Expand Down

0 comments on commit 77b0470

Please sign in to comment.