mirror of
https://github.com/etcd-io/etcd.git
synced 2024-09-27 06:25:44 +00:00
raft: introduce Node interface
This commit is contained in:
69
raft/node.go
69
raft/node.go
@@ -72,28 +72,37 @@ func (rd Ready) containsUpdates() bool {
|
||||
len(rd.Entries) > 0 || len(rd.CommittedEntries) > 0 || len(rd.Messages) > 0
|
||||
}
|
||||
|
||||
type Node struct {
|
||||
propc chan pb.Message
|
||||
recvc chan pb.Message
|
||||
compactc chan []byte
|
||||
readyc chan Ready
|
||||
tickc chan struct{}
|
||||
done chan struct{}
|
||||
type Node interface {
|
||||
// Tick increments the internal logical clock for the Node by a single tick. Election
|
||||
// timeouts and heartbeat timeouts are in units of ticks.
|
||||
Tick()
|
||||
// Campaign causes the Node to transition to candidate state and start campaigning to become leader
|
||||
Campaign(ctx context.Context) error
|
||||
// Propose proposes that data be appended to the log.
|
||||
Propose(ctx context.Context, data []byte) error
|
||||
// Step advances the state machine using the given message. ctx.Err() will be returned, if any.
|
||||
Step(ctx context.Context, msg pb.Message) error
|
||||
// Ready returns a channel that returns the current point-in-time state
|
||||
Ready() <-chan Ready
|
||||
// Stop performs any necessary termination of the Node
|
||||
Stop()
|
||||
// Compact
|
||||
Compact(d []byte)
|
||||
}
|
||||
|
||||
// Start returns a new Node given a unique raft id, a list of raft peers, and
|
||||
// StartNode returns a new Node given a unique raft id, a list of raft peers, and
|
||||
// the election and heartbeat timeouts in units of ticks.
|
||||
func Start(id int64, peers []int64, election, heartbeat int) Node {
|
||||
func StartNode(id int64, peers []int64, election, heartbeat int) Node {
|
||||
n := newNode()
|
||||
r := newRaft(id, peers, election, heartbeat)
|
||||
go n.run(r)
|
||||
return n
|
||||
return &n
|
||||
}
|
||||
|
||||
// Restart is identical to Start but takes an initial State and a slice of
|
||||
// entries. Generally this is used when restarting from a stable storage
|
||||
// RestartNode is identical to StartNode but takes an initial State and a slice
|
||||
// of entries. Generally this is used when restarting from a stable storage
|
||||
// log.
|
||||
func Restart(id int64, peers []int64, election, heartbeat int, snapshot *pb.Snapshot, st pb.HardState, ents []pb.Entry) Node {
|
||||
func RestartNode(id int64, peers []int64, election, heartbeat int, snapshot *pb.Snapshot, st pb.HardState, ents []pb.Entry) Node {
|
||||
n := newNode()
|
||||
r := newRaft(id, peers, election, heartbeat)
|
||||
if snapshot != nil {
|
||||
@@ -102,11 +111,21 @@ func Restart(id int64, peers []int64, election, heartbeat int, snapshot *pb.Snap
|
||||
r.loadState(st)
|
||||
r.loadEnts(ents)
|
||||
go n.run(r)
|
||||
return n
|
||||
return &n
|
||||
}
|
||||
|
||||
func newNode() Node {
|
||||
return Node{
|
||||
// node is the canonical implementation of the Node interface
|
||||
type node struct {
|
||||
propc chan pb.Message
|
||||
recvc chan pb.Message
|
||||
compactc chan []byte
|
||||
readyc chan Ready
|
||||
tickc chan struct{}
|
||||
done chan struct{}
|
||||
}
|
||||
|
||||
func newNode() node {
|
||||
return node{
|
||||
propc: make(chan pb.Message),
|
||||
recvc: make(chan pb.Message),
|
||||
compactc: make(chan []byte),
|
||||
@@ -116,11 +135,11 @@ func newNode() Node {
|
||||
}
|
||||
}
|
||||
|
||||
func (n *Node) Stop() {
|
||||
func (n *node) Stop() {
|
||||
close(n.done)
|
||||
}
|
||||
|
||||
func (n *Node) run(r *raft) {
|
||||
func (n *node) run(r *raft) {
|
||||
var propc chan pb.Message
|
||||
var readyc chan Ready
|
||||
|
||||
@@ -178,25 +197,24 @@ func (n *Node) run(r *raft) {
|
||||
|
||||
// Tick increments the internal logical clock for this Node. Election timeouts
|
||||
// and heartbeat timeouts are in units of ticks.
|
||||
func (n *Node) Tick() {
|
||||
func (n *node) Tick() {
|
||||
select {
|
||||
case n.tickc <- struct{}{}:
|
||||
case <-n.done:
|
||||
}
|
||||
}
|
||||
|
||||
func (n *Node) Campaign(ctx context.Context) error {
|
||||
func (n *node) Campaign(ctx context.Context) error {
|
||||
return n.Step(ctx, pb.Message{Type: msgHup})
|
||||
}
|
||||
|
||||
// Propose proposes data be appended to the log.
|
||||
func (n *Node) Propose(ctx context.Context, data []byte) error {
|
||||
func (n *node) Propose(ctx context.Context, data []byte) error {
|
||||
return n.Step(ctx, pb.Message{Type: msgProp, Entries: []pb.Entry{{Data: data}}})
|
||||
}
|
||||
|
||||
// Step advances the state machine using msgs. The ctx.Err() will be returned,
|
||||
// if any.
|
||||
func (n *Node) Step(ctx context.Context, m pb.Message) error {
|
||||
func (n *node) Step(ctx context.Context, m pb.Message) error {
|
||||
ch := n.recvc
|
||||
if m.Type == msgProp {
|
||||
ch = n.propc
|
||||
@@ -212,12 +230,11 @@ func (n *Node) Step(ctx context.Context, m pb.Message) error {
|
||||
}
|
||||
}
|
||||
|
||||
// ReadState returns the current point-in-time state.
|
||||
func (n *Node) Ready() <-chan Ready {
|
||||
func (n *node) Ready() <-chan Ready {
|
||||
return n.readyc
|
||||
}
|
||||
|
||||
func (n *Node) Compact(d []byte) {
|
||||
func (n *node) Compact(d []byte) {
|
||||
select {
|
||||
case n.compactc <- d:
|
||||
case <-n.done:
|
||||
|
||||
Reference in New Issue
Block a user