etcd/raft/rawnode.go

283 lines
8.7 KiB
Go
Raw Normal View History

2016-05-13 06:49:15 +03:00
// Copyright 2015 The etcd Authors
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package raft
import (
"errors"
pb "go.etcd.io/etcd/raft/raftpb"
"go.etcd.io/etcd/raft/tracker"
)
// ErrStepLocalMsg is returned when try to step a local raft message
var ErrStepLocalMsg = errors.New("raft: cannot step raft local message")
// ErrStepPeerNotFound is returned when try to step a response message
// but there is no peer found in raft.prs for that node.
var ErrStepPeerNotFound = errors.New("raft: cannot step as peer not found")
// RawNode is a thread-unsafe Node.
// The methods of this struct correspond to the methods of Node and are described
// more fully there.
type RawNode struct {
raft *raft
prevSoftSt *SoftState
prevHardSt pb.HardState
}
func (rn *RawNode) newReady() Ready {
return newReady(rn.raft, rn.prevSoftSt, rn.prevHardSt)
}
func (rn *RawNode) commitReady(rd Ready) {
if rd.SoftState != nil {
rn.prevSoftSt = rd.SoftState
}
if !IsEmptyHardState(rd.HardState) {
rn.prevHardSt = rd.HardState
}
// If entries were applied (or a snapshot), update our cursor for
// the next Ready. Note that if the current HardState contains a
// new Commit index, this does not mean that we're also applying
// all of the new entries due to commit pagination by size.
if index := rd.appliedCursor(); index > 0 {
rn.raft.raftLog.appliedTo(index)
}
if len(rd.Entries) > 0 {
e := rd.Entries[len(rd.Entries)-1]
rn.raft.raftLog.stableTo(e.Index, e.Term)
}
if !IsEmptySnap(rd.Snapshot) {
rn.raft.raftLog.stableSnapTo(rd.Snapshot.Metadata.Index)
}
2016-09-10 10:51:49 +03:00
if len(rd.ReadStates) != 0 {
rn.raft.readStates = nil
}
}
// NewRawNode returns a new RawNode given configuration and a list of raft peers.
func NewRawNode(config *Config, peers []Peer) (*RawNode, error) {
if config.ID == 0 {
panic("config.ID must not be zero")
}
r := newRaft(config)
rn := &RawNode{
raft: r,
}
lastIndex, err := config.Storage.LastIndex()
if err != nil {
panic(err) // TODO(bdarnell)
}
// If the log is empty, this is a new RawNode (like StartNode); otherwise it's
// restoring an existing RawNode (like RestartNode).
// TODO(bdarnell): rethink RawNode initialization and whether the application needs
// to be able to tell us when it expects the RawNode to exist.
if lastIndex == 0 {
r.becomeFollower(1, None)
ents := make([]pb.Entry, len(peers))
for i, peer := range peers {
cc := pb.ConfChange{Type: pb.ConfChangeAddNode, NodeID: peer.ID, Context: peer.Context}
data, err := cc.Marshal()
if err != nil {
panic("unexpected marshal error")
}
ents[i] = pb.Entry{Type: pb.EntryConfChange, Term: 1, Index: uint64(i + 1), Data: data}
}
r.raftLog.append(ents...)
r.raftLog.committed = uint64(len(ents))
for _, peer := range peers {
r.applyConfChange(pb.ConfChange{NodeID: peer.ID, Type: pb.ConfChangeAddNode})
}
}
// Set the initial hard and soft states after performing all initialization.
rn.prevSoftSt = r.softState()
if lastIndex == 0 {
rn.prevHardSt = emptyState
} else {
rn.prevHardSt = r.hardState()
}
return rn, nil
}
// Tick advances the internal logical clock by a single tick.
func (rn *RawNode) Tick() {
rn.raft.tick()
}
// TickQuiesced advances the internal logical clock by a single tick without
// performing any other state machine processing. It allows the caller to avoid
// periodic heartbeats and elections when all of the peers in a Raft group are
// known to be at the same state. Expected usage is to periodically invoke Tick
// or TickQuiesced depending on whether the group is "active" or "quiesced".
//
// WARNING: Be very careful about using this method as it subverts the Raft
// state machine. You should probably be using Tick instead.
func (rn *RawNode) TickQuiesced() {
rn.raft.electionElapsed++
}
// Campaign causes this RawNode to transition to candidate state.
func (rn *RawNode) Campaign() error {
return rn.raft.Step(pb.Message{
Type: pb.MsgHup,
})
}
// Propose proposes data be appended to the raft log.
func (rn *RawNode) Propose(data []byte) error {
return rn.raft.Step(pb.Message{
Type: pb.MsgProp,
From: rn.raft.id,
Entries: []pb.Entry{
{Data: data},
}})
}
// ProposeConfChange proposes a config change.
func (rn *RawNode) ProposeConfChange(cc pb.ConfChange) error {
data, err := cc.Marshal()
if err != nil {
return err
}
return rn.raft.Step(pb.Message{
Type: pb.MsgProp,
Entries: []pb.Entry{
{Type: pb.EntryConfChange, Data: data},
},
})
}
// ApplyConfChange applies a config change to the local node.
func (rn *RawNode) ApplyConfChange(cc pb.ConfChange) *pb.ConfState {
cs := rn.raft.applyConfChange(cc)
return &cs
}
// Step advances the state machine using the given message.
func (rn *RawNode) Step(m pb.Message) error {
// ignore unexpected local messages receiving over network
2016-03-24 14:59:41 +03:00
if IsLocalMsg(m.Type) {
return ErrStepLocalMsg
}
if pr := rn.raft.prs.Progress[m.From]; pr != nil || !IsResponseMsg(m.Type) {
return rn.raft.Step(m)
}
return ErrStepPeerNotFound
}
// Ready returns the current point-in-time state of this RawNode.
func (rn *RawNode) Ready() Ready {
rd := rn.newReady()
rn.raft.msgs = nil
rn.raft.reduceUncommittedSize(rd.CommittedEntries)
return rd
}
// HasReady called when RawNode user need to check if any Ready pending.
// Checking logic in this method should be consistent with Ready.containsUpdates().
func (rn *RawNode) HasReady() bool {
r := rn.raft
if !r.softState().equal(rn.prevSoftSt) {
return true
}
if hardSt := r.hardState(); !IsEmptyHardState(hardSt) && !isHardStateEqual(hardSt, rn.prevHardSt) {
return true
}
if r.raftLog.unstable.snapshot != nil && !IsEmptySnap(*r.raftLog.unstable.snapshot) {
return true
}
if len(r.msgs) > 0 || len(r.raftLog.unstableEntries()) > 0 || r.raftLog.hasNextEnts() {
return true
}
2016-09-10 10:51:49 +03:00
if len(r.readStates) != 0 {
return true
}
return false
}
// Advance notifies the RawNode that the application has applied and saved progress in the
// last Ready results.
func (rn *RawNode) Advance(rd Ready) {
rn.commitReady(rd)
}
// Status returns the current status of the given group.
func (rn *RawNode) Status() *Status {
status := getStatus(rn.raft)
return &status
}
raft: add (*RawNode).WithProgress Calls to Status can be frequent and currently incur three heap allocations, but often the caller has no intention to hold on to the returned status. Add StatusWithoutProgress and WithProgress to allow avoiding heap allocations altogether. StatusWithoutProgress does what's on the tin and additionally returns a value (instead of a pointer) to avoid the associated heap allocation. By not returning a Progress map, it avoids all other allocations that Status incurs. To still introspect the Progress map, add WithProgress, which uses a simple visitor pattern. Add benchmarks to verify that this is indeed allocation free. ``` BenchmarkStatusProgress/members=1/Status-8 5000000 353 ns/op 784 B/op 3 allocs/op BenchmarkStatusProgress/members=1/Status-example-8 5000000 372 ns/op 784 B/op 3 allocs/op BenchmarkStatusProgress/members=1/StatusWithoutProgress-8 100000000 17.6 ns/op 0 B/op 0 allocs/op BenchmarkStatusProgress/members=1/WithProgress-8 30000000 48.6 ns/op 0 B/op 0 allocs/op BenchmarkStatusProgress/members=1/WithProgress-example-8 30000000 42.9 ns/op 0 B/op 0 allocs/op BenchmarkStatusProgress/members=3/Status-8 5000000 395 ns/op 784 B/op 3 allocs/op BenchmarkStatusProgress/members=3/Status-example-8 3000000 449 ns/op 784 B/op 3 allocs/op BenchmarkStatusProgress/members=3/StatusWithoutProgress-8 100000000 18.7 ns/op 0 B/op 0 allocs/op BenchmarkStatusProgress/members=3/WithProgress-8 20000000 78.1 ns/op 0 B/op 0 allocs/op BenchmarkStatusProgress/members=3/WithProgress-example-8 20000000 70.7 ns/op 0 B/op 0 allocs/op BenchmarkStatusProgress/members=5/Status-8 3000000 470 ns/op 784 B/op 3 allocs/op BenchmarkStatusProgress/members=5/Status-example-8 3000000 544 ns/op 784 B/op 3 allocs/op BenchmarkStatusProgress/members=5/StatusWithoutProgress-8 100000000 19.7 ns/op 0 B/op 0 allocs/op BenchmarkStatusProgress/members=5/WithProgress-8 20000000 105 ns/op 0 B/op 0 allocs/op BenchmarkStatusProgress/members=5/WithProgress-example-8 20000000 94.0 ns/op 0 B/op 0 allocs/op BenchmarkStatusProgress/members=100/Status-8 100000 11903 ns/op 22663 B/op 12 allocs/op BenchmarkStatusProgress/members=100/Status-example-8 100000 13330 ns/op 22669 B/op 12 allocs/op BenchmarkStatusProgress/members=100/StatusWithoutProgress-8 50000000 20.9 ns/op 0 B/op 0 allocs/op BenchmarkStatusProgress/members=100/WithProgress-8 1000000 1731 ns/op 0 B/op 0 allocs/op BenchmarkStatusProgress/members=100/WithProgress-example-8 1000000 1571 ns/op 0 B/op 0 allocs/op ```
2018-12-06 21:02:47 +03:00
// StatusWithoutProgress returns a Status without populating the Progress field
// (and returns the Status as a value to avoid forcing it onto the heap). This
// is more performant if the Progress is not required. See WithProgress for an
// allocation-free way to introspect the Progress.
func (rn *RawNode) StatusWithoutProgress() Status {
return getStatusWithoutProgress(rn.raft)
}
// ProgressType indicates the type of replica a Progress corresponds to.
type ProgressType byte
const (
// ProgressTypePeer accompanies a Progress for a regular peer replica.
ProgressTypePeer ProgressType = iota
// ProgressTypeLearner accompanies a Progress for a learner replica.
ProgressTypeLearner
)
// WithProgress is a helper to introspect the Progress for this node and its
// peers.
func (rn *RawNode) WithProgress(visitor func(id uint64, typ ProgressType, pr tracker.Progress)) {
rn.raft.prs.Visit(func(id uint64, pr *tracker.Progress) {
typ := ProgressTypePeer
if pr.IsLearner {
typ = ProgressTypeLearner
}
p := *pr
p.Inflights = nil
visitor(id, typ, p)
})
raft: add (*RawNode).WithProgress Calls to Status can be frequent and currently incur three heap allocations, but often the caller has no intention to hold on to the returned status. Add StatusWithoutProgress and WithProgress to allow avoiding heap allocations altogether. StatusWithoutProgress does what's on the tin and additionally returns a value (instead of a pointer) to avoid the associated heap allocation. By not returning a Progress map, it avoids all other allocations that Status incurs. To still introspect the Progress map, add WithProgress, which uses a simple visitor pattern. Add benchmarks to verify that this is indeed allocation free. ``` BenchmarkStatusProgress/members=1/Status-8 5000000 353 ns/op 784 B/op 3 allocs/op BenchmarkStatusProgress/members=1/Status-example-8 5000000 372 ns/op 784 B/op 3 allocs/op BenchmarkStatusProgress/members=1/StatusWithoutProgress-8 100000000 17.6 ns/op 0 B/op 0 allocs/op BenchmarkStatusProgress/members=1/WithProgress-8 30000000 48.6 ns/op 0 B/op 0 allocs/op BenchmarkStatusProgress/members=1/WithProgress-example-8 30000000 42.9 ns/op 0 B/op 0 allocs/op BenchmarkStatusProgress/members=3/Status-8 5000000 395 ns/op 784 B/op 3 allocs/op BenchmarkStatusProgress/members=3/Status-example-8 3000000 449 ns/op 784 B/op 3 allocs/op BenchmarkStatusProgress/members=3/StatusWithoutProgress-8 100000000 18.7 ns/op 0 B/op 0 allocs/op BenchmarkStatusProgress/members=3/WithProgress-8 20000000 78.1 ns/op 0 B/op 0 allocs/op BenchmarkStatusProgress/members=3/WithProgress-example-8 20000000 70.7 ns/op 0 B/op 0 allocs/op BenchmarkStatusProgress/members=5/Status-8 3000000 470 ns/op 784 B/op 3 allocs/op BenchmarkStatusProgress/members=5/Status-example-8 3000000 544 ns/op 784 B/op 3 allocs/op BenchmarkStatusProgress/members=5/StatusWithoutProgress-8 100000000 19.7 ns/op 0 B/op 0 allocs/op BenchmarkStatusProgress/members=5/WithProgress-8 20000000 105 ns/op 0 B/op 0 allocs/op BenchmarkStatusProgress/members=5/WithProgress-example-8 20000000 94.0 ns/op 0 B/op 0 allocs/op BenchmarkStatusProgress/members=100/Status-8 100000 11903 ns/op 22663 B/op 12 allocs/op BenchmarkStatusProgress/members=100/Status-example-8 100000 13330 ns/op 22669 B/op 12 allocs/op BenchmarkStatusProgress/members=100/StatusWithoutProgress-8 50000000 20.9 ns/op 0 B/op 0 allocs/op BenchmarkStatusProgress/members=100/WithProgress-8 1000000 1731 ns/op 0 B/op 0 allocs/op BenchmarkStatusProgress/members=100/WithProgress-example-8 1000000 1571 ns/op 0 B/op 0 allocs/op ```
2018-12-06 21:02:47 +03:00
}
// ReportUnreachable reports the given node is not reachable for the last send.
func (rn *RawNode) ReportUnreachable(id uint64) {
_ = rn.raft.Step(pb.Message{Type: pb.MsgUnreachable, From: id})
}
2016-01-06 11:17:02 +03:00
// ReportSnapshot reports the status of the sent snapshot.
func (rn *RawNode) ReportSnapshot(id uint64, status SnapshotStatus) {
rej := status == SnapshotFailure
_ = rn.raft.Step(pb.Message{Type: pb.MsgSnapStatus, From: id, Reject: rej})
}
2016-03-24 14:59:41 +03:00
// TransferLeader tries to transfer leadership to the given transferee.
func (rn *RawNode) TransferLeader(transferee uint64) {
_ = rn.raft.Step(pb.Message{Type: pb.MsgTransferLeader, From: transferee})
}
2016-09-10 10:51:49 +03:00
// ReadIndex requests a read state. The read state will be set in ready.
// Read State has a read index. Once the application advances further than the read
// index, any linearizable read requests issued before the read request can be
// processed safely. The read state will have the same rctx attached.
func (rn *RawNode) ReadIndex(rctx []byte) {
_ = rn.raft.Step(pb.Message{Type: pb.MsgReadIndex, Entries: []pb.Entry{{Data: rctx}}})
}