1// Copyright 2015 The etcd Authors
2//
3// Licensed under the Apache License, Version 2.0 (the "License");
4// you may not use this file except in compliance with the License.
5// You may obtain a copy of the License at
6//
7//     http://www.apache.org/licenses/LICENSE-2.0
8//
9// Unless required by applicable law or agreed to in writing, software
10// distributed under the License is distributed on an "AS IS" BASIS,
11// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12// See the License for the specific language governing permissions and
13// limitations under the License.
14
15package raft
16
17import (
18	"errors"
19
20	pb "github.com/coreos/etcd/raft/raftpb"
21)
22
23// ErrStepLocalMsg is returned when try to step a local raft message
24var ErrStepLocalMsg = errors.New("raft: cannot step raft local message")
25
26// ErrStepPeerNotFound is returned when try to step a response message
27// but there is no peer found in raft.prs for that node.
28var ErrStepPeerNotFound = errors.New("raft: cannot step as peer not found")
29
30// RawNode is a thread-unsafe Node.
31// The methods of this struct correspond to the methods of Node and are described
32// more fully there.
33type RawNode struct {
34	raft       *raft
35	prevSoftSt *SoftState
36	prevHardSt pb.HardState
37}
38
39func (rn *RawNode) newReady() Ready {
40	return newReady(rn.raft, rn.prevSoftSt, rn.prevHardSt)
41}
42
43func (rn *RawNode) commitReady(rd Ready) {
44	if rd.SoftState != nil {
45		rn.prevSoftSt = rd.SoftState
46	}
47	if !IsEmptyHardState(rd.HardState) {
48		rn.prevHardSt = rd.HardState
49	}
50	if rn.prevHardSt.Commit != 0 {
51		// In most cases, prevHardSt and rd.HardState will be the same
52		// because when there are new entries to apply we just sent a
53		// HardState with an updated Commit value. However, on initial
54		// startup the two are different because we don't send a HardState
55		// until something changes, but we do send any un-applied but
56		// committed entries (and previously-committed entries may be
57		// incorporated into the snapshot, even if rd.CommittedEntries is
58		// empty). Therefore we mark all committed entries as applied
59		// whether they were included in rd.HardState or not.
60		rn.raft.raftLog.appliedTo(rn.prevHardSt.Commit)
61	}
62	if len(rd.Entries) > 0 {
63		e := rd.Entries[len(rd.Entries)-1]
64		rn.raft.raftLog.stableTo(e.Index, e.Term)
65	}
66	if !IsEmptySnap(rd.Snapshot) {
67		rn.raft.raftLog.stableSnapTo(rd.Snapshot.Metadata.Index)
68	}
69	if len(rd.ReadStates) != 0 {
70		rn.raft.readStates = nil
71	}
72}
73
74// NewRawNode returns a new RawNode given configuration and a list of raft peers.
75func NewRawNode(config *Config, peers []Peer) (*RawNode, error) {
76	if config.ID == 0 {
77		panic("config.ID must not be zero")
78	}
79	r := newRaft(config)
80	rn := &RawNode{
81		raft: r,
82	}
83	lastIndex, err := config.Storage.LastIndex()
84	if err != nil {
85		panic(err) // TODO(bdarnell)
86	}
87	// If the log is empty, this is a new RawNode (like StartNode); otherwise it's
88	// restoring an existing RawNode (like RestartNode).
89	// TODO(bdarnell): rethink RawNode initialization and whether the application needs
90	// to be able to tell us when it expects the RawNode to exist.
91	if lastIndex == 0 {
92		r.becomeFollower(1, None)
93		ents := make([]pb.Entry, len(peers))
94		for i, peer := range peers {
95			cc := pb.ConfChange{Type: pb.ConfChangeAddNode, NodeID: peer.ID, Context: peer.Context}
96			data, err := cc.Marshal()
97			if err != nil {
98				panic("unexpected marshal error")
99			}
100
101			ents[i] = pb.Entry{Type: pb.EntryConfChange, Term: 1, Index: uint64(i + 1), Data: data}
102		}
103		r.raftLog.append(ents...)
104		r.raftLog.committed = uint64(len(ents))
105		for _, peer := range peers {
106			r.addNode(peer.ID)
107		}
108	}
109
110	// Set the initial hard and soft states after performing all initialization.
111	rn.prevSoftSt = r.softState()
112	if lastIndex == 0 {
113		rn.prevHardSt = emptyState
114	} else {
115		rn.prevHardSt = r.hardState()
116	}
117
118	return rn, nil
119}
120
121// Tick advances the internal logical clock by a single tick.
122func (rn *RawNode) Tick() {
123	rn.raft.tick()
124}
125
126// TickQuiesced advances the internal logical clock by a single tick without
127// performing any other state machine processing. It allows the caller to avoid
128// periodic heartbeats and elections when all of the peers in a Raft group are
129// known to be at the same state. Expected usage is to periodically invoke Tick
130// or TickQuiesced depending on whether the group is "active" or "quiesced".
131//
132// WARNING: Be very careful about using this method as it subverts the Raft
133// state machine. You should probably be using Tick instead.
134func (rn *RawNode) TickQuiesced() {
135	rn.raft.electionElapsed++
136}
137
138// Campaign causes this RawNode to transition to candidate state.
139func (rn *RawNode) Campaign() error {
140	return rn.raft.Step(pb.Message{
141		Type: pb.MsgHup,
142	})
143}
144
145// Propose proposes data be appended to the raft log.
146func (rn *RawNode) Propose(data []byte) error {
147	return rn.raft.Step(pb.Message{
148		Type: pb.MsgProp,
149		From: rn.raft.id,
150		Entries: []pb.Entry{
151			{Data: data},
152		}})
153}
154
155// ProposeConfChange proposes a config change.
156func (rn *RawNode) ProposeConfChange(cc pb.ConfChange) error {
157	data, err := cc.Marshal()
158	if err != nil {
159		return err
160	}
161	return rn.raft.Step(pb.Message{
162		Type: pb.MsgProp,
163		Entries: []pb.Entry{
164			{Type: pb.EntryConfChange, Data: data},
165		},
166	})
167}
168
169// ApplyConfChange applies a config change to the local node.
170func (rn *RawNode) ApplyConfChange(cc pb.ConfChange) *pb.ConfState {
171	if cc.NodeID == None {
172		rn.raft.resetPendingConf()
173		return &pb.ConfState{Nodes: rn.raft.nodes()}
174	}
175	switch cc.Type {
176	case pb.ConfChangeAddNode:
177		rn.raft.addNode(cc.NodeID)
178	case pb.ConfChangeAddLearnerNode:
179		rn.raft.addLearner(cc.NodeID)
180	case pb.ConfChangeRemoveNode:
181		rn.raft.removeNode(cc.NodeID)
182	case pb.ConfChangeUpdateNode:
183		rn.raft.resetPendingConf()
184	default:
185		panic("unexpected conf type")
186	}
187	return &pb.ConfState{Nodes: rn.raft.nodes()}
188}
189
190// Step advances the state machine using the given message.
191func (rn *RawNode) Step(m pb.Message) error {
192	// ignore unexpected local messages receiving over network
193	if IsLocalMsg(m.Type) {
194		return ErrStepLocalMsg
195	}
196	if pr := rn.raft.getProgress(m.From); pr != nil || !IsResponseMsg(m.Type) {
197		return rn.raft.Step(m)
198	}
199	return ErrStepPeerNotFound
200}
201
202// Ready returns the current point-in-time state of this RawNode.
203func (rn *RawNode) Ready() Ready {
204	rd := rn.newReady()
205	rn.raft.msgs = nil
206	return rd
207}
208
209// HasReady called when RawNode user need to check if any Ready pending.
210// Checking logic in this method should be consistent with Ready.containsUpdates().
211func (rn *RawNode) HasReady() bool {
212	r := rn.raft
213	if !r.softState().equal(rn.prevSoftSt) {
214		return true
215	}
216	if hardSt := r.hardState(); !IsEmptyHardState(hardSt) && !isHardStateEqual(hardSt, rn.prevHardSt) {
217		return true
218	}
219	if r.raftLog.unstable.snapshot != nil && !IsEmptySnap(*r.raftLog.unstable.snapshot) {
220		return true
221	}
222	if len(r.msgs) > 0 || len(r.raftLog.unstableEntries()) > 0 || r.raftLog.hasNextEnts() {
223		return true
224	}
225	if len(r.readStates) != 0 {
226		return true
227	}
228	return false
229}
230
231// Advance notifies the RawNode that the application has applied and saved progress in the
232// last Ready results.
233func (rn *RawNode) Advance(rd Ready) {
234	rn.commitReady(rd)
235}
236
237// Status returns the current status of the given group.
238func (rn *RawNode) Status() *Status {
239	status := getStatus(rn.raft)
240	return &status
241}
242
243// ReportUnreachable reports the given node is not reachable for the last send.
244func (rn *RawNode) ReportUnreachable(id uint64) {
245	_ = rn.raft.Step(pb.Message{Type: pb.MsgUnreachable, From: id})
246}
247
248// ReportSnapshot reports the status of the sent snapshot.
249func (rn *RawNode) ReportSnapshot(id uint64, status SnapshotStatus) {
250	rej := status == SnapshotFailure
251
252	_ = rn.raft.Step(pb.Message{Type: pb.MsgSnapStatus, From: id, Reject: rej})
253}
254
255// TransferLeader tries to transfer leadership to the given transferee.
256func (rn *RawNode) TransferLeader(transferee uint64) {
257	_ = rn.raft.Step(pb.Message{Type: pb.MsgTransferLeader, From: transferee})
258}
259
260// ReadIndex requests a read state. The read state will be set in ready.
261// Read State has a read index. Once the application advances further than the read
262// index, any linearizable read requests issued before the read request can be
263// processed safely. The read state will have the same rctx attached.
264func (rn *RawNode) ReadIndex(rctx []byte) {
265	_ = rn.raft.Step(pb.Message{Type: pb.MsgReadIndex, Entries: []pb.Entry{{Data: rctx}}})
266}
267