Skip to content

Raft Coalesced Heartbeats#280

Merged
tbg merged 8 commits intocockroachdb:masterfrom
tbg:coalesced_heartbeats_rb
Feb 9, 2015
Merged

Raft Coalesced Heartbeats#280
tbg merged 8 commits intocockroachdb:masterfrom
tbg:coalesced_heartbeats_rb

Conversation

@tbg
Copy link
Copy Markdown
Member

@tbg tbg commented Jan 23, 2015

currently work in progress, see #229 for early discussions (further discussions should be held here).

  • basic prototype
  • keep group memberships on a per-node basis
  • receive heartbeats only from (supposed leader)
  • analogous mechanism for heartbeat responses
  • basic test setup
  • test w/ invalid heartbeats (no leader)
  • test w/ multiple partially-overlapping consensus groups
  • consider correctness implications/corner cases (discussion below)

closes #229

cc @cockroachdb/developers

@tbg
Copy link
Copy Markdown
Member Author

tbg commented Jan 23, 2015

More q's based on the discussion:

  • we could theoretically have a situation in which the leader of a certain raft group deadlocks their goroutine (or something like that) while coalesced heartbeats keep going happily. Is that something we need to watch out for or should we simply assume that as long as multiraft looks ok, all raft groups are going strong? A coalesced heartbeat trigger could also check that groups we think are leading on our node actually send heartbeats (which we then discard).

  • @bdarnell said:

    Good point; we need to track leader status so we don't interfere with followers' candidate timers. I'm not as concerned about the cost of sending heartbeats when we're not the leader, but we should avoid fanning-out heartbeat responses when we're not the leader.`

    I'm interpreting this as "heartbeats are fanned out only to groups which believe that the sending node is the leader", which is what I've implemented in the next commit (The leader ID was already tracked in the group, I just needed to use it).

    Are there any subtle things that can go wrong still? Since now group members get heartbeats based on who they think their boss is, and not based on who actually is. For instance, once a node becomes a candidate it doesn't think of anyone as their leader, and consequently won't receive any heartbeats. Hence, one code path that leads to a Candidate->Follower transition is completely disabled, and while I think that one's not critical (might just cost some time), I wonder what else is in the bushes.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@bdarnell looked for this in the raft paper and in the protobufs but no dice. Is there an explanation of what this does anywhere?

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm not sure. This may be for updating the cc.Context field (which we don't use). I don't think we need to support this.

@bdarnell
Copy link
Copy Markdown
Contributor

We need similar coalescing/fan-out for MsgHeartbeatResp as well. This will be similar on the receiving sides (except that we fan out heartbeat responses to the groups that the receiving node is leader of instead of those that the sending node leads). For sending, we should just keep track of which nodes we have sent MsgHeartbeatResp and not send to the same node twice (per call to handleRaftReady).

@tbg
Copy link
Copy Markdown
Member Author

tbg commented Jan 25, 2015

In the works.

For testing, I would replace the local transport's client by one that supports intercepting (and holding back) outgoing messages, and probably add a demux for consuming it during tests (akin to the eventsDemux). With that we should be able to get pretty good testability.
Good idea? Am I overcomplicating things?

@bdarnell
Copy link
Copy Markdown
Contributor

Yeah, we should test this with a custom transport (see also blockingStorage in storage_test.go). I'm not sure whether the eventsDemux pattern makes sense here but if it turns out to be useful in your tests go for it.

@tbg tbg force-pushed the coalesced_heartbeats_rb branch from a8b39fd to 2f262eb Compare January 25, 2015 15:51
@tbg
Copy link
Copy Markdown
Member Author

tbg commented Jan 25, 2015

Ok. I'll continue development in this PR, as I wouldn't want to merge anything before I can test it anyways.
Last commit contains my take at the heartbeat responses plus their fanout.

@tbg tbg force-pushed the coalesced_heartbeats_rb branch 4 times, most recently from 93c0ec4 to 645fbe9 Compare January 28, 2015 07:52
@tbg
Copy link
Copy Markdown
Member Author

tbg commented Jan 28, 2015

@bdarnell Can you take another look?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I looked into combining this with fanoutHeartbeatRequest, but in the end I liked it better as is.

@tbg tbg force-pushed the coalesced_heartbeats_rb branch from 645fbe9 to 8aa4329 Compare January 28, 2015 18:25
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think we should build this into localRPCTransport. There should be a separate interceptingTransport type that either wraps an existing Transport or simply is a transport itself (I think the latter is probably better - you don't need the RPC machinery, just have Go call the ServerInterface method directly). Then this can all go in transport_test.go unless and until we have a use for it outside this package.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ok, will go for the thin version without RPC then. Can you explain what bothers you about the current modifications? I thought about creating an extra client but then it just seemed silly to write the same thing twice with a feature that the other one might also want to use at times.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

My concern is mainly with the dramatically different implementation (including a change to blocking behavior) depending on whether EnableEvents has been called. Since EnableEvents can only be called immediately after creation it's effectively a different type anyway.

@tbg
Copy link
Copy Markdown
Member Author

tbg commented Jan 30, 2015

@bdarnell did you see this above:

Are there any subtle things that can go wrong still? Since now group members get heartbeats based on who they think their boss is, and not based on who actually is. For instance, once a node becomes a candidate it doesn't think of anyone as their leader, and consequently won't receive any heartbeats. Hence, one code path that leads to a Candidate->Follower transition is completely disabled, and while I think that one's not critical (might just cost some time), I wonder what else is in the bushes.

@bdarnell
Copy link
Copy Markdown
Contributor

bdarnell commented Feb 1, 2015

I had missed that note the first time it appeared. It's good to be concerned about this; I think there are some scenarios in which these limited-information heartbeats could cause problems. For example, consider a group containing nodes A, B, and C:

  • A is initially the leader.
  • Node C loses connectivity to the other nodes and after a timeout declares itself a candidate.
  • The partition partially heals; C can talk to A but not B.
  • C is elected the new leader; A steps down.
  • B sees that A has been up continuously and so it still considers A the leader.

We might be able to solve this on a case-by-case basis (e.g. I think two-round elections would help in the above scenario), but to be safe we should probably add a payload to the coalesced heartbeat messages to indicate which groups the sending node considers itself to be leader of.

This thread on raft-dev discusses some related issues: https://groups.google.com/forum/?fromgroups#!topic/raft-dev/VgF47vIsezg. The term and log index is occasionally useful (but not, I think, strictly required. Etcd is not currently sending this information with their heartbeats).

@tbg tbg force-pushed the coalesced_heartbeats_rb branch from 7b3de30 to bc34681 Compare February 4, 2015 21:01
@tbg
Copy link
Copy Markdown
Member Author

tbg commented Feb 4, 2015

@bdarnell new transport, refactored test inside. not complete yet but that's what i would build on for the rest of the tests.

edit: will also take a look at your last comment above while doing that. probably some progress this WE.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If heartbeatCountMap contained pointers instead of struct values then you could mutate them without going through this read-then-write pattern.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It's convenient though because I can avoid the usual v, ok := ...; if !ok { m[key] = ... } code.

@bdarnell
Copy link
Copy Markdown
Contributor

bdarnell commented Feb 4, 2015

LGTM

bdarnell added a commit to bdarnell/cockroach that referenced this pull request Feb 5, 2015
Extracted from PR cockroachdb#280; I also need this for another change.
@tbg
Copy link
Copy Markdown
Member Author

tbg commented Feb 6, 2015

cc @cockroachdb/developers

@bdarnell
Copy link
Copy Markdown
Contributor

bdarnell commented Feb 9, 2015

The new test LGTM. It's complicated, but I don't have any ideas for simplifying it.

tbg added 7 commits February 9, 2015 19:50
* outgoing heartbeats originating from raft are ignored
* instead, each node will periodically send coalesced heartbeats
  to each of the other nodes.
* upon receipt of a heartbeat (which is coalesced, according to the above),
  individual heartbeats are fed into raft for each group that the sending
  node belongs to and follows a leader from the origin node.
* this necessitated adding a list of groupIDs to the internal `node` type.
* heartbeat responses: Raft's (potentially many) generated responses
  are bundled up and only a single response sent.
* fan out heartbeat responses to all groups considering themselves leader of
  a group of the sending node
* elect a leader and have it tick a number of times
* count and compare the heartbeats sent and responses received

TODO: add test that has followers tick as well; a closer interplay
between ticks and acknowledged messages would be needed to prevent
followers from turning into candidates.
currently this still tests only one group, but it sets a good baseline
for basic sanity. Even writing this test took a lot of care; next step
will be unittests and some multi-group tests with fairly explicit (i.e.
unparametrized) setups.
@tbg
Copy link
Copy Markdown
Member Author

tbg commented Feb 9, 2015

@bdarnell I've added an extra ping and rebased. Do we want to merge it and tackle the safety considerations above separately? I have a feeling that sending around the groups could be tricky as well - thinking of millions of ranges...

@tbg tbg force-pushed the coalesced_heartbeats_rb branch from d628f27 to c8dec08 Compare February 9, 2015 19:18
@bdarnell
Copy link
Copy Markdown
Contributor

bdarnell commented Feb 9, 2015

Yes, let's go ahead and merge.

tbg added a commit that referenced this pull request Feb 9, 2015
@tbg tbg merged commit 4c15e96 into cockroachdb:master Feb 9, 2015
@tbg tbg deleted the coalesced_heartbeats_rb branch February 9, 2015 20:13
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

Multiraft: coalesced heartbeats

3 participants