Does a heartbeat has an additional purpose?

160 views
Skip to first unread message

Gerry

unread,
Jan 26, 2015, 6:38:13 AM1/26/15
to raft...@googlegroups.com
I tried to implement the RAFT algorithm. As a template I started with the pseudocode from Diego, but I have been running into problems.

His receiver pseudo code of an append entry message looks like this:
if currentTerm < m.term:
| stepDown(m.term)
if currentTerm > m.term:
| reply{ term: currentTerm,
| success : False }
else :
| leader = peer
| state = FOLLOWER
| electionAlarm = now() + rand(1.0, 2.0) * ELECTION_TIMEOUT
| success : = (m.prevIndex == 0 or
| (m.prevIndex <= len(log) and
| log[m.prevIndex].term == m.prevTerm))
| index : = 0
| if success :
| | for j : = 1..len(m.entries) :
| | | index = m.prevIndex + j
| | | if getTerm(index) != m.entries[j].term :
| | | | log = log[1..(index - 1)] + m.entries[j]
| | commitIndex = min(m.commitIndex, index)

| reply{ term: currentTerm,
| success : success,
| matchIndex : index }

If just a heartbeat is getting processed then this routine will return the currentTerm, success and a possibly wrong matchIndex of 0 (I tried initializing index : = m.prevIndex instead).

And the sender will handle the response like this:
if currentTerm < m.term:
| stepDown(m.term)
elif state == LEADER and currentTerm == m.term :
| if m.success :
| | matchIndex[peer] = m.matchIndex
| | nextIndex[peer] += 1
| else :
| | nextIndex[peer] = max(1, nextIndex[peer] - 1)

But her I have the problem that nextIndex is getting changed beyond a value that makes sense (I did limit this with a min() statement). 

1. So as discussed elsewhere does it really make sense to handle plain heartbeats the same way like AppendEntry instructions? Or wouldn’t it be semantically cleaner to separate them?
2. Is there any other purpose of the heartbeat than resetting the election counter of the peers?
3. Does the peer after having received a heartbeat have to send a response or would it be sufficient to just reset the election timer (My implementation doesn’t use RPC calls. It is just sending messages forth and back!) 

Or is there something I didn't understand yet?

Diego Ongaro

unread,
Jan 27, 2015, 8:47:12 PM1/27/15
to Gerry, raft...@googlegroups.com
Hey Gerry,

Yeah, that doesn't look right. In case the rest of y'all are
wondering, the pseudocode comes from here:
https://github.com/ongardie/raft-pseudocode . There's a very relevant
warning on it: "warning: might not be entirely correct, check with
TLA+ spec". The story behind that pseudocode is that we were exploring
the possibility of using it as a substitute for Figure 2 (the
"cheatsheet") in the Raft paper, but we decided against it.

If you're trying to implement the algorithm and are worried about
correctness, you might want to try the TLA+ spec (latest version at
https://github.com/ongardie/raft.tla ), which I did try to get right.

> 1. So as discussed elsewhere does it really make sense to handle plain heartbeats the same way like AppendEntry instructions? Or wouldn’t it be semantically cleaner to separate them?

As you're aware, we've discussed this before. I'm pretty sure some
implementations do separate heartbeats, and it's not crazy. On the
other hand, sending and processing heartbeats the same as (empty)
AppendEntries is convenient, in my opinion.

> 2. Is there any other purpose of the heartbeat than resetting the election counter of the peers?

It gets followers the leader's latest commit index. Empty
AppendEntries are also typically used to determine where the leader
and follower's logs diverge (setting 'nextIndex'). It lets the
follower know who the leader is. It makes sure everyone's cool with
the current term.

> 3. Does the peer after having received a heartbeat have to send a response or would it be sufficient to just reset the election timer (My implementation doesn’t use RPC calls. It is just sending messages forth and back!)

The reply is useful in a few cases. If the follower's term is higher,
it gets the deposed leader to step down. If nextIndex is out of place,
the reply helps to adjust it. In some implementations (like LogCabin),
read-only requests are blocked at the leader until a round of
heartbeats complete with a majority of the cluster; without a reply,
you can't know when they complete.

One more that I can think of: you don't want client requests to block
at a partitioned leader forever. You can have clients enforce a
timeout on their requests, but maybe the leader is in a better
position to know when the client should give up (the leader knows when
it's able to make forward progress; the client doesn't). So in
LogCabin, a leader steps down if it can't get heartbeats out to a
majority of the cluster within an election timeout.

-Diego
> --
> You received this message because you are subscribed to the Google Groups
> "raft-dev" group.
> To unsubscribe from this group and stop receiving emails from it, send an
> email to raft-dev+u...@googlegroups.com.
> For more options, visit https://groups.google.com/d/optout.

Jordan Halterman (kuujo)

unread,
Jan 27, 2015, 9:23:57 PM1/27/15
to raft...@googlegroups.com
Copycat impliments a separate RPC for heartbeats, but only sends them if no AppendEntries RPCs with the follower are in progress at the time. This is really just a cleanliness thing, and since the request contains nearly everything but entries it still informs the follower of the current commit index and gives the follower a chance to inform the leader of a new term or inconsistent log.

Also, Copycat uses the AppendEntries reply to allow the follower to explicitly indicate it's last log index to the leader, thus speeding up the process of resolving the the nextIndex.

Gerry

unread,
Feb 1, 2015, 11:46:38 AM2/1/15
to raft...@googlegroups.com, gerry...@gmail.com
Hi Diego,

Thank you for answering everything in great detail! Raft has really some nice properties. Ok, I rely on the TLA+ spec now (but the pseudo code has been very helpful to implement a first version).
I’am using separate heartbeats now, just because it’s easier to debug (I’am merging the log files of all the server nodes to understand what’s going on in the cluster. And it’s handy to see if there was just a heartbeat or an AppendEntry message.)

So if I have understood everything correctly, No responses has to be sent for heartbeats(the protocol would still work properly) but it could have some advantages. 

-Gerry

Gerry

unread,
Feb 1, 2015, 12:05:08 PM2/1/15
to raft...@googlegroups.com
Thank you for the insight into the Copycat implementation. For resolving the nextIndex and synchronizing the followers log with the leaders log I'am using a special message type. This is sent as soon as a node got leader and since I'am using the TCP/IP protocol for the moment it could be done too when a connection from a peer got established/reestablished. For normal logs the advantage seems to be that I could use a binary search to find the match Index if a first check did fail and I don't have to send the big payload's when decrementing the matchIndex in order to find a match.   

Diego Ongaro

unread,
Feb 3, 2015, 6:31:54 PM2/3/15
to Gerry, raft...@googlegroups.com
On Sun, Feb 1, 2015 at 8:46 AM, Gerry <gerry...@gmail.com> wrote:
> Hi Diego,
>
> Thank you for answering everything in great detail! Raft has really some
> nice properties. Ok, I rely on the TLA+ spec now (but the pseudo code has
> been very helpful to implement a first version).

Cool, feel free to send me a PR on the pseudocode if you think it's
worth it. (I don't plan to make it a priority.)

> [...snip...]

> So if I have understood everything correctly, No responses has to be sent
> for heartbeats(the protocol would still work properly) but it could have
> some advantages.

Yep, I guess that's right.

Gerry

unread,
Feb 4, 2015, 9:11:21 AM2/4/15
to raft...@googlegroups.com, gerry...@gmail.com
Two fixes of the original pseudo code I remember (see below). 
The pseudo code was really great to have something to start with (The target implementation is done in C++ with LMDB as a key-value store).
At the moment I started experimenting and testing. I try to use the log as the primary store of the data stored in the cluster, where the state machine does the log compaction with the help of a functional unique key. 

#*\codetitle{on AppendEntries request from peer}*\#
if currentTerm < m.term:
| stepDown(m.term)
if currentTerm > m.term:
| reply {term: currentTerm,
|        success: False}
else:
| leader = peer
| state = FOLLOWER
| electionAlarm = now() + rand(1.0, 2.0) * ELECTION_TIMEOUT
| success := (m.prevIndex == 0 or
|   (m.prevIndex <= len(log) and
|    log[m.prevIndex].term == m.prevTerm))
| index := m.prevIndex + 1 // FIXED
| if success:
| | for j := 1..len(m.entries):
| | | index = m.prevIndex + j
| | | if getTerm(index) != m.entries[j].term:
| | | | log = log[1..(index-1)] + m.entries[j]
| | commitIndex = min(m.commitIndex, index)
| reply {term: currentTerm,
|        success: success,
|        matchIndex: index}


#*\codetitle{on AppendEntries response from peer}*\#
if currentTerm < m.term:
| stepDown(m.term)
elif state == LEADER and currentTerm == m.term:
| if m.success:
| | matchIndex[peer] = m.matchIndex
| | nextIndex[peer]  = m.matchIndex + 1 // FIXED
Reply all
Reply to author
Forward
0 new messages