etcd/raft/rawnode.go

242 lines
7.9 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/v3/raft/raftpb"
"go.etcd.io/etcd/v3/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
}
raft: clean up bootstrap This is the first (maybe not last) step in cleaning up the bootstrap code around StartNode. Initializing a Raft group for the first time is awkward, since a configuration has to be pulled from thin air. The way this is solved today is unclean: The app is supposed to pass peers to StartNode(), we add configuration changes for them to the log, immediately pretend that they are applied, but actually leave them unapplied (to give the app a chance to observe them, though if the app did decide to not apply them things would really go off the rails), and then return control to the app. The app will then process the initial Readys and as a result the configuration will be persisted to disk; restarts of the node then use RestartNode which doesn't take any peers. The code that did this lived awkwardly in two places fairly deep down the callstack, though it was really only necessary in StartNode(). This commit refactors things to make this more obvious: only StartNode does this dance now. In particular, RawNode does not support this at all any more; it expects the app to set up its Storage correctly. Future work may provide helpers to make this "preseeding" of the Storage more user-friendly. It isn't entirely straightforward to do so since the Storage interface doesn't provide the right accessors for this purpose. Briefly speaking, we want to make sure that a non-bootstrapped node can never catch up via the log so that we can implicitly use one of the "skipped" log entries to represent the configuration change into the bootstrap configuration. This is an invasive change that affects all consumers of raft, and it is of lower urgency since the code (post this commit) already encapsulates the complexity sufficiently.
2019-07-18 14:41:52 +03:00
// NewRawNode instantiates a RawNode from the given configuration.
//
// See Bootstrap() for bootstrapping an initial state; this replaces the former
// 'peers' argument to this method (with identical behavior). However, It is
// recommended that instead of calling Bootstrap, applications bootstrap their
// state manually by setting up a Storage that has a first index > 1 and which
// stores the desired ConfState as its InitialState.
raft: clean up bootstrap This is the first (maybe not last) step in cleaning up the bootstrap code around StartNode. Initializing a Raft group for the first time is awkward, since a configuration has to be pulled from thin air. The way this is solved today is unclean: The app is supposed to pass peers to StartNode(), we add configuration changes for them to the log, immediately pretend that they are applied, but actually leave them unapplied (to give the app a chance to observe them, though if the app did decide to not apply them things would really go off the rails), and then return control to the app. The app will then process the initial Readys and as a result the configuration will be persisted to disk; restarts of the node then use RestartNode which doesn't take any peers. The code that did this lived awkwardly in two places fairly deep down the callstack, though it was really only necessary in StartNode(). This commit refactors things to make this more obvious: only StartNode does this dance now. In particular, RawNode does not support this at all any more; it expects the app to set up its Storage correctly. Future work may provide helpers to make this "preseeding" of the Storage more user-friendly. It isn't entirely straightforward to do so since the Storage interface doesn't provide the right accessors for this purpose. Briefly speaking, we want to make sure that a non-bootstrapped node can never catch up via the log so that we can implicitly use one of the "skipped" log entries to represent the configuration change into the bootstrap configuration. This is an invasive change that affects all consumers of raft, and it is of lower urgency since the code (post this commit) already encapsulates the complexity sufficiently.
2019-07-18 14:41:52 +03:00
func NewRawNode(config *Config) (*RawNode, error) {
r := newRaft(config)
rn := &RawNode{
raft: r,
}
rn.prevSoftSt = r.softState()
raft: clean up bootstrap This is the first (maybe not last) step in cleaning up the bootstrap code around StartNode. Initializing a Raft group for the first time is awkward, since a configuration has to be pulled from thin air. The way this is solved today is unclean: The app is supposed to pass peers to StartNode(), we add configuration changes for them to the log, immediately pretend that they are applied, but actually leave them unapplied (to give the app a chance to observe them, though if the app did decide to not apply them things would really go off the rails), and then return control to the app. The app will then process the initial Readys and as a result the configuration will be persisted to disk; restarts of the node then use RestartNode which doesn't take any peers. The code that did this lived awkwardly in two places fairly deep down the callstack, though it was really only necessary in StartNode(). This commit refactors things to make this more obvious: only StartNode does this dance now. In particular, RawNode does not support this at all any more; it expects the app to set up its Storage correctly. Future work may provide helpers to make this "preseeding" of the Storage more user-friendly. It isn't entirely straightforward to do so since the Storage interface doesn't provide the right accessors for this purpose. Briefly speaking, we want to make sure that a non-bootstrapped node can never catch up via the log so that we can implicitly use one of the "skipped" log entries to represent the configuration change into the bootstrap configuration. This is an invasive change that affects all consumers of raft, and it is of lower urgency since the code (post this commit) already encapsulates the complexity sufficiently.
2019-07-18 14:41:52 +03:00
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. See (Node).ProposeConfChange for
// details.
func (rn *RawNode) ProposeConfChange(cc pb.ConfChangeI) error {
m, err := confChangeToMsg(cc)
if err != nil {
return err
}
return rn.raft.Step(m)
}
// ApplyConfChange applies a config change to the local node. The app must call
// this when it applies a configuration change, except when it decides to reject
// the configuration change, in which case no call must take place.
func (rn *RawNode) ApplyConfChange(cc pb.ConfChangeI) *pb.ConfState {
cs := rn.raft.applyConfChange(cc.AsV2())
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 outstanding work that the application needs to handle. This
// includes appending and applying entries or a snapshot, updating the HardState,
// and sending messages. The returned Ready() *must* be handled and subsequently
// passed back via Advance().
func (rn *RawNode) Ready() Ready {
rd := rn.readyWithoutAccept()
rn.acceptReady(rd)
return rd
}
// readyWithoutAccept returns a Ready. This is a read-only operation, i.e. there
// is no obligation that the Ready must be handled.
func (rn *RawNode) readyWithoutAccept() Ready {
return newReady(rn.raft, rn.prevSoftSt, rn.prevHardSt)
}
// acceptReady is called when the consumer of the RawNode has decided to go
// ahead and handle a Ready. Nothing must alter the state of the RawNode between
// this call and the prior call to Ready().
func (rn *RawNode) acceptReady(rd Ready) {
if rd.SoftState != nil {
rn.prevSoftSt = rd.SoftState
}
if len(rd.ReadStates) != 0 {
rn.raft.readStates = nil
}
rn.raft.msgs = nil
}
// 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.hasPendingSnapshot() {
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) {
if !IsEmptyHardState(rd.HardState) {
rn.prevHardSt = rd.HardState
}
rn.raft.advance(rd)
}
// Status returns the current status of the given group. This allocates, see
// BasicStatus and WithProgress for allocation-friendlier choices.
func (rn *RawNode) Status() Status {
status := getStatus(rn.raft)
return status
}
// BasicStatus returns a BasicStatus. Notably this does not contain the
// Progress map; see WithProgress for an allocation-free way to inspect it.
func (rn *RawNode) BasicStatus() BasicStatus {
return getBasicStatus(rn.raft)
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
}
// 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}}})
}