Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

raft: add raft learner #8751

Merged
merged 1 commit into from
Nov 12, 2017
Merged

Conversation

siddontang
Copy link
Contributor

This is a derived PR from #8605 which I think we can close now.

I just cache the voter count in this PR.

Using two processes may be easy, but I tried and found that many codes need to be changed, seem no big benefit now.

PTAL @xiang90

raft/progress.go Outdated
@@ -76,6 +77,8 @@ type Progress struct {
// be freed by calling inflights.freeTo with the index of the last
// received entry.
ins *inflights

isLearner bool
Copy link
Contributor Author

Choose a reason for hiding this comment

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

do we need to export this, I guess maybe user may need it later

/cc @xiang90

Copy link
Contributor

Choose a reason for hiding this comment

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

seems fine to me.

raft/raft.go Outdated
@@ -281,6 +281,9 @@ type raft struct {
tick func()
step stepFunc

// voterCount caches the total voter which can vote.
voterCount int
Copy link
Contributor

Choose a reason for hiding this comment

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

i still want to try to separate the learners out of the prs. so we can keep the old logic without adding this "cache". i feel the code can be cleaner overall too.

can you give it a try at least?

@xiang90
Copy link
Contributor

xiang90 commented Oct 25, 2017

it looks good to me overall. just want to another approach to keep track of learners.

@siddontang
Copy link
Contributor Author

I have separated the learner progress, but the code seems not be simpler.

PTAL @xiang90

raft/raft.go Outdated
for id := range r.prs {
if id == r.id {
continue
f := func(prs map[uint64]*Progress) {
Copy link
Contributor

Choose a reason for hiding this comment

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

make this a method forEachProgress(prs [], prs[], f())? so we do not need to repeat this quite a few times.

@xiang90
Copy link
Contributor

xiang90 commented Oct 26, 2017

thanks i will give this a closer look tomorrow.

raft/progress.go Outdated
@@ -76,6 +77,9 @@ type Progress struct {
// be freed by calling inflights.freeTo with the index of the last
// received entry.
ins *inflights

// IsLearner is true if the the follower is a learner.
Copy link
Contributor

Choose a reason for hiding this comment

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

if this progress is tracked for a learner.

raft/raft.go Outdated
@@ -116,6 +116,9 @@ type Config struct {
// used for testing right now.
peers []uint64

// learners can not promote or vote.
Copy link
Contributor

Choose a reason for hiding this comment

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

// learners contains the IDs of all leaner nodes (including self if the local node is a leaner) in the raft cluster.
// learners only receives entries from the leader node. It does not vote or promote itself.

raft/raft.go Outdated

state StateType

isLearner bool
Copy link
Contributor

Choose a reason for hiding this comment

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

// ieLearner is true if the local raft node is a learner.

raft/raft.go Outdated
@@ -316,6 +326,16 @@ func newRaft(c *Config) *raft {
for _, p := range peers {
r.prs[p] = &Progress{Next: 1, ins: newInflights(r.maxInflight)}
}
for _, p := range learners {
if _, ok := r.prs[p]; ok {
panic(fmt.Sprintf("cannot specify both Voter and Learner for node: %x", p))
Copy link
Contributor

Choose a reason for hiding this comment

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

node %x is in both learner and peer list.

raft/raft.go Outdated
}

// maybeCommit attempts to advance the commit index. Returns true if
// the commit index changed (in which case the caller should call
// r.bcastAppend).
func (r *raft) maybeCommit() bool {
// TODO(bmizerany): optimize.. Currently naive
mis := make(uint64Slice, 0, len(r.prs))
Copy link
Contributor

Choose a reason for hiding this comment

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

is this change really needed in this PR?

@@ -890,8 +939,8 @@ func stepLeader(r *raft, m pb.Message) {
}

// All other message types require a progress for m.From (pr).
pr, prOk := r.prs[m.From]
if !prOk {
pr := r.getProgress(m.From)
Copy link
Contributor

Choose a reason for hiding this comment

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

getProgress can return *p,ok, just like what the previous map struct does. it minimizes code changes. but not a big problem anyway.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

If here returns *Progress, bool, I will check the ok boolean in the places which originally use getProgess().xxx to replace prs[id].xxx, and this may cause code complex.

Another way is to add a mustGetProgress function, but I think it is not necessary.

raft/raft.go Outdated
@@ -990,6 +1039,10 @@ func stepLeader(r *raft, m pb.Message) {
}
r.logger.Debugf("%x failed to send message to %x because it is unreachable [%s]", r.id, m.From, pr)
case pb.MsgTransferLeader:
if pr.IsLearner {
r.logger.Debugf("%x is Learner. Ignored transferring leadership", r.id)
Copy link
Contributor

Choose a reason for hiding this comment

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

is learner

raft/raft.go Outdated
if pr != nil {
if isLearner && !pr.IsLearner {
// can only change Learner to Voter
r.logger.Infof("%x ignore addLearner for existing node %x [%s]", r.id, id, pr)
Copy link
Contributor

Choose a reason for hiding this comment

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

r.id ignored addLeaner: do not support changing from raft peer to learner.

raft/raft.go Outdated
pr.IsLearner = false
r.prs[id] = pr
} else {
r.setProgress(id, 0, r.raftLog.lastIndex()+1, isLearner)
Copy link
Contributor

Choose a reason for hiding this comment

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

we should always put the short case first in if...else... statement for readability. so move this case first, then change the pr exist case.

raft/raft.go Outdated
func (r *raft) setProgress(id, match, next uint64, isLearner bool) {
if isLearner {
if _, ok := r.prs[id]; ok {
panic(fmt.Sprintf("%x can't change voter to learner for %x", r.id, id))
Copy link
Contributor

Choose a reason for hiding this comment

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

unexpected changing from x to x.

raft/raft.go Outdated
return
}

delete(r.learnerPrs, id)
r.prs[id] = &Progress{Next: next, Match: match, ins: newInflights(r.maxInflight)}
Copy link
Contributor

Choose a reason for hiding this comment

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

i would handle not learner case first, since it is easier. so it improves readability.

@siddontang
Copy link
Contributor Author

PTAL @xiang90

@@ -348,6 +348,54 @@ func testLeaderElection(t *testing.T, preVote bool) {
}
}

func TestLearnerElectionTimeout(t *testing.T) {
Copy link
Contributor

Choose a reason for hiding this comment

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

we need to add a description for this test.


nt := newNetwork(n1, n2)

// Learner can't start election
Copy link
Contributor

Choose a reason for hiding this comment

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

n2 is learner. Learner should not start election even when times out.

for i := 0; i < n2.electionTimeout; i++ {
n2.tick()
}
if n1.state != StateFollower {
Copy link
Contributor

Choose a reason for hiding this comment

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

no need to check the state of n1 here.

t.Errorf("peer 2 state: %s, want %s", n2.state, StateFollower)
}

// n1 should become leader
Copy link
Contributor

Choose a reason for hiding this comment

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

check n1 is not leader here.


nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgBeat})

n1.addNode(2)
Copy link
Contributor

Choose a reason for hiding this comment

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

well... this tests that a learner can be promoted to a normal node. we should separate this into another test.

}

// n1 should become leader
nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
Copy link
Contributor

Choose a reason for hiding this comment

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

why we do use tick to trigger timeout above, but use msghub here?

@@ -2326,6 +2374,47 @@ func TestRestore(t *testing.T) {
}
}

func TestRestoreWithLearner(t *testing.T) {
Copy link
Contributor

Choose a reason for hiding this comment

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

add test description.

t.Errorf("nodes = %v, want %v", nodes, wnodes)
}
if !r.learnerPrs[2].IsLearner {
t.Fatalf("node 2 has suffrage %t, want %t", r.prs[2].IsLearner, true)
Copy link
Contributor

Choose a reason for hiding this comment

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

use t.errorf. rename suffrage to learner.

}

if ok := sm.restore(s); ok {
t.Fatal("restore succeed, want fail")
Copy link
Contributor

Choose a reason for hiding this comment

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

use t.errorf here for checking purpose. also doc string about why do we expect this to fail.

@xiang90
Copy link
Contributor

xiang90 commented Nov 1, 2017

Here are a minimal list of tests I want to see initially:

  • learner will not be promoted to candidate even if times out
  • learner will not vote even if it receives a valid vote request
  • learner will receive new entries from the leader
  • learner will receive snapshot from the leader
  • learner can be added
  • learner can be removed
  • learner can be promoted to normal node
  • normal node cannot be demoted to learner
  • learner configuration will be snapshotted
  • learner can be restored from a snapshot
  • node status reports learner progress correctly

@xiang90 xiang90 mentioned this pull request Nov 1, 2017
@xiang90
Copy link
Contributor

xiang90 commented Nov 10, 2017

@siddontang lgtm.

there are a couple of things left though, but we can do them in a follow up PR:

  1. test status has learner info included
  2. update documentation to include the learner feature

@siddontang
Copy link
Contributor Author

@xiang90

Please file some issues so I can't forget it. I will try to use this feature in TiKV at first.

@siddontang
Copy link
Contributor Author

Done @xiang90

@siddontang
Copy link
Contributor Author

siddontang commented Nov 10, 2017

@xiang90

The go 1.8 and 1.9 have different format rule, how to fix it?

/cc @gyuho

@gyuho
Copy link
Contributor

gyuho commented Nov 10, 2017

Yeah could you fix this line https://travis-ci.org/coreos/etcd/jobs/299973342#L841?
etcd CI only uses Go 1.9+.

@siddontang
Copy link
Contributor Author

Hi @gyuho

I update the format but still find https://travis-ci.org/coreos/etcd/jobs/300033398 failed, but I don't know why.

@gyuho
Copy link
Contributor

gyuho commented Nov 11, 2017

@siddontang Oh that was fixed just today. If you rebase from current master, it should pass.

Thanks!

@siddontang
Copy link
Contributor Author

@gyuho

I rebase the master but the CI still failed.

@@ -3338,10 +3627,19 @@ func newNetworkWithConfig(configFunc func(*Config), peers ...stateMachine) *netw
sm := newRaft(cfg)
npeers[id] = sm
case *raft:
learners := make(map[uint64]bool, 0)
Copy link
Contributor

@gyuho gyuho Nov 11, 2017

Choose a reason for hiding this comment

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

learners := make(map[uint64]bool, len(v.learnerPrs))? Our static analysis tool complains about this :0

Copy link
Contributor Author

Choose a reason for hiding this comment

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

oh, got it.

@siddontang
Copy link
Contributor Author

@gyuho

Jenkins failed but the failure seems irrelevant to Raft learner.

@xiang90
Copy link
Contributor

xiang90 commented Nov 12, 2017

CI failures are not relevant. Merging.

@absolute8511
Copy link
Contributor

I reviewed some codes for this pr and found there may be an issue which can affect the
ApplyConfChange in the raft node.

	// ApplyConfChange applies config change to the local node.
	// Returns an opaque ConfState protobuf which must be recorded
	// in snapshots. Will never return nil; it returns a pointer only
	// to match MemoryStorage.Compact.
	ApplyConfChange(cc pb.ConfChange) *pb.ConfState

Since we add new Learners to pb.ConfState, we should return the pb.ConfState with Learners in the raft.

		case cc := <-n.confc:
			if cc.NodeID == None {
				r.resetPendingConf()
				select {
				case n.confstatec <- pb.ConfState{Nodes: r.nodes()}:
				case <-n.done:
				}
				break
			}
			switch cc.Type {
			case pb.ConfChangeAddNode:
				r.addNode(cc.NodeID)
			case pb.ConfChangeAddLearnerNode:
				r.addLearner(cc.NodeID)
			case pb.ConfChangeRemoveNode:
				// block incoming proposal when local node is
				// removed
				if cc.NodeID == r.id {
					propc = nil
				}
				r.removeNode(cc.NodeID)
			case pb.ConfChangeUpdateNode:
				r.resetPendingConf()
			default:
				panic("unexpected conf type")
			}
			select {
			case n.confstatec <- pb.ConfState{Nodes: r.nodes()}:
			case <-n.done:
			}

However, from the code above, we return all the nodes and learners written into the Nodes and leave the Learners empty. Is that by designed or a potential bug?

@xiang90
Copy link
Contributor

xiang90 commented Dec 28, 2017 via email

tbg added a commit to tbg/etcd that referenced this pull request Jun 28, 2019
At the time of writing, we don't allow configuration changes to change
voters to learners directly, but note that a snapshot may compress
multiple changes to the configuration into one: the voter could have
been removed, then readded as a learner and the snapshot reflects both
changes. In that case, a voter receives a snapshot telling it that it is
now a learner. In fact, the node has to accept that snapshot, or it is
permanently cut off from the Raft log.

I think this just wasn't realized in the original work, but this is just
my guess since there generally is very little rationale on the various
decisions made. I also generally haven't been able to figure out whether
the decision to prevent voters from becoming learners without first
having been removed was motivated by some particular concern, or if it
just wasn't deemed necessary. I suspect it is the latter because
demoting a voter seems perfectly safe.

See etcd-io#8751 (comment).
tbg added a commit to tbg/etcd that referenced this pull request Jul 3, 2019
At the time of writing, we don't allow configuration changes to change
voters to learners directly, but note that a snapshot may compress
multiple changes to the configuration into one: the voter could have
been removed, then readded as a learner and the snapshot reflects both
changes. In that case, a voter receives a snapshot telling it that it is
now a learner. In fact, the node has to accept that snapshot, or it is
permanently cut off from the Raft log.

I think this just wasn't realized in the original work, but this is just
my guess since there generally is very little rationale on the various
decisions made. I also generally haven't been able to figure out whether
the decision to prevent voters from becoming learners without first
having been removed was motivated by some particular concern, or if it
just wasn't deemed necessary. I suspect it is the latter because
demoting a voter seems perfectly safe.

See etcd-io#8751 (comment).
tbg added a commit to tbg/etcd that referenced this pull request Jul 8, 2019
At the time of writing, we don't allow configuration changes to change
voters to learners directly, but note that a snapshot may compress
multiple changes to the configuration into one: the voter could have
been removed, then readded as a learner and the snapshot reflects both
changes. In that case, a voter receives a snapshot telling it that it is
now a learner. In fact, the node has to accept that snapshot, or it is
permanently cut off from the Raft log.

I think this just wasn't realized in the original work, but this is just
my guess since there generally is very little rationale on the various
decisions made. I also generally haven't been able to figure out whether
the decision to prevent voters from becoming learners without first
having been removed was motivated by some particular concern, or if it
just wasn't deemed necessary. I suspect it is the latter because
demoting a voter seems perfectly safe.

See etcd-io#8751 (comment).
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Development

Successfully merging this pull request may close these issues.

4 participants