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	"bytes"
19	"context"
20	"fmt"
21	"math"
22	"reflect"
23	"testing"
24
25	"go.etcd.io/etcd/raft/quorum"
26	pb "go.etcd.io/etcd/raft/raftpb"
27	"go.etcd.io/etcd/raft/tracker"
28)
29
30// rawNodeAdapter is essentially a lint that makes sure that RawNode implements
31// "most of" Node. The exceptions (some of which are easy to fix) are listed
32// below.
33type rawNodeAdapter struct {
34	*RawNode
35}
36
37var _ Node = (*rawNodeAdapter)(nil)
38
39// Node specifies lead, which is pointless, can just be filled in.
40func (a *rawNodeAdapter) TransferLeadership(ctx context.Context, lead, transferee uint64) {
41	a.RawNode.TransferLeader(transferee)
42}
43
44// Node has a goroutine, RawNode doesn't need this.
45func (a *rawNodeAdapter) Stop() {}
46
47// RawNode returns a *Status.
48func (a *rawNodeAdapter) Status() Status { return a.RawNode.Status() }
49
50// RawNode takes a Ready. It doesn't really have to do that I think? It can hold on
51// to it internally. But maybe that approach is frail.
52func (a *rawNodeAdapter) Advance() { a.RawNode.Advance(Ready{}) }
53
54// RawNode returns a Ready, not a chan of one.
55func (a *rawNodeAdapter) Ready() <-chan Ready { return nil }
56
57// Node takes more contexts. Easy enough to fix.
58
59func (a *rawNodeAdapter) Campaign(context.Context) error { return a.RawNode.Campaign() }
60func (a *rawNodeAdapter) ReadIndex(_ context.Context, rctx []byte) error {
61	a.RawNode.ReadIndex(rctx)
62	// RawNode swallowed the error in ReadIndex, it probably should not do that.
63	return nil
64}
65func (a *rawNodeAdapter) Step(_ context.Context, m pb.Message) error   { return a.RawNode.Step(m) }
66func (a *rawNodeAdapter) Propose(_ context.Context, data []byte) error { return a.RawNode.Propose(data) }
67func (a *rawNodeAdapter) ProposeConfChange(_ context.Context, cc pb.ConfChangeI) error {
68	return a.RawNode.ProposeConfChange(cc)
69}
70
71// TestRawNodeStep ensures that RawNode.Step ignore local message.
72func TestRawNodeStep(t *testing.T) {
73	for i, msgn := range pb.MessageType_name {
74		t.Run(msgn, func(t *testing.T) {
75			s := NewMemoryStorage()
76			s.SetHardState(pb.HardState{Term: 1, Commit: 1})
77			s.Append([]pb.Entry{{Term: 1, Index: 1}})
78			if err := s.ApplySnapshot(pb.Snapshot{Metadata: pb.SnapshotMetadata{
79				ConfState: pb.ConfState{
80					Voters: []uint64{1},
81				},
82				Index: 1,
83				Term:  1,
84			}}); err != nil {
85				t.Fatal(err)
86			}
87			// Append an empty entry to make sure the non-local messages (like
88			// vote requests) are ignored and don't trigger assertions.
89			rawNode, err := NewRawNode(newTestConfig(1, nil, 10, 1, s))
90			if err != nil {
91				t.Fatal(err)
92			}
93			msgt := pb.MessageType(i)
94			err = rawNode.Step(pb.Message{Type: msgt})
95			// LocalMsg should be ignored.
96			if IsLocalMsg(msgt) {
97				if err != ErrStepLocalMsg {
98					t.Errorf("%d: step should ignore %s", msgt, msgn)
99				}
100			}
101		})
102	}
103}
104
105// TestNodeStepUnblock from node_test.go has no equivalent in rawNode because there is
106// no goroutine in RawNode.
107
108// TestRawNodeProposeAndConfChange tests the configuration change mechanism. Each
109// test case sends a configuration change which is either simple or joint, verifies
110// that it applies and that the resulting ConfState matches expectations, and for
111// joint configurations makes sure that they are exited successfully.
112func TestRawNodeProposeAndConfChange(t *testing.T) {
113	testCases := []struct {
114		cc   pb.ConfChangeI
115		exp  pb.ConfState
116		exp2 *pb.ConfState
117	}{
118		// V1 config change.
119		{
120			pb.ConfChange{Type: pb.ConfChangeAddNode, NodeID: 2},
121			pb.ConfState{Voters: []uint64{1, 2}},
122			nil,
123		},
124		// Proposing the same as a V2 change works just the same, without entering
125		// a joint config.
126		{
127			pb.ConfChangeV2{Changes: []pb.ConfChangeSingle{
128				{Type: pb.ConfChangeAddNode, NodeID: 2},
129			},
130			},
131			pb.ConfState{Voters: []uint64{1, 2}},
132			nil,
133		},
134		// Ditto if we add it as a learner instead.
135		{
136			pb.ConfChangeV2{Changes: []pb.ConfChangeSingle{
137				{Type: pb.ConfChangeAddLearnerNode, NodeID: 2},
138			},
139			},
140			pb.ConfState{Voters: []uint64{1}, Learners: []uint64{2}},
141			nil,
142		},
143		// We can ask explicitly for joint consensus if we want it.
144		{
145			pb.ConfChangeV2{Changes: []pb.ConfChangeSingle{
146				{Type: pb.ConfChangeAddLearnerNode, NodeID: 2},
147			},
148				Transition: pb.ConfChangeTransitionJointExplicit,
149			},
150			pb.ConfState{Voters: []uint64{1}, VotersOutgoing: []uint64{1}, Learners: []uint64{2}},
151			&pb.ConfState{Voters: []uint64{1}, Learners: []uint64{2}},
152		},
153		// Ditto, but with implicit transition (the harness checks this).
154		{
155			pb.ConfChangeV2{Changes: []pb.ConfChangeSingle{
156				{Type: pb.ConfChangeAddLearnerNode, NodeID: 2},
157			},
158				Transition: pb.ConfChangeTransitionJointImplicit,
159			},
160			pb.ConfState{
161				Voters: []uint64{1}, VotersOutgoing: []uint64{1}, Learners: []uint64{2},
162				AutoLeave: true,
163			},
164			&pb.ConfState{Voters: []uint64{1}, Learners: []uint64{2}},
165		},
166		// Add a new node and demote n1. This exercises the interesting case in
167		// which we really need joint config changes and also need LearnersNext.
168		{
169			pb.ConfChangeV2{Changes: []pb.ConfChangeSingle{
170				{NodeID: 2, Type: pb.ConfChangeAddNode},
171				{NodeID: 1, Type: pb.ConfChangeAddLearnerNode},
172				{NodeID: 3, Type: pb.ConfChangeAddLearnerNode},
173			},
174			},
175			pb.ConfState{
176				Voters:         []uint64{2},
177				VotersOutgoing: []uint64{1},
178				Learners:       []uint64{3},
179				LearnersNext:   []uint64{1},
180				AutoLeave:      true,
181			},
182			&pb.ConfState{Voters: []uint64{2}, Learners: []uint64{1, 3}},
183		},
184		// Ditto explicit.
185		{
186			pb.ConfChangeV2{Changes: []pb.ConfChangeSingle{
187				{NodeID: 2, Type: pb.ConfChangeAddNode},
188				{NodeID: 1, Type: pb.ConfChangeAddLearnerNode},
189				{NodeID: 3, Type: pb.ConfChangeAddLearnerNode},
190			},
191				Transition: pb.ConfChangeTransitionJointExplicit,
192			},
193			pb.ConfState{
194				Voters:         []uint64{2},
195				VotersOutgoing: []uint64{1},
196				Learners:       []uint64{3},
197				LearnersNext:   []uint64{1},
198			},
199			&pb.ConfState{Voters: []uint64{2}, Learners: []uint64{1, 3}},
200		},
201		// Ditto implicit.
202		{
203			pb.ConfChangeV2{
204				Changes: []pb.ConfChangeSingle{
205					{NodeID: 2, Type: pb.ConfChangeAddNode},
206					{NodeID: 1, Type: pb.ConfChangeAddLearnerNode},
207					{NodeID: 3, Type: pb.ConfChangeAddLearnerNode},
208				},
209				Transition: pb.ConfChangeTransitionJointImplicit,
210			},
211			pb.ConfState{
212				Voters:         []uint64{2},
213				VotersOutgoing: []uint64{1},
214				Learners:       []uint64{3},
215				LearnersNext:   []uint64{1},
216				AutoLeave:      true,
217			},
218			&pb.ConfState{Voters: []uint64{2}, Learners: []uint64{1, 3}},
219		},
220	}
221
222	for _, tc := range testCases {
223		t.Run("", func(t *testing.T) {
224			s := NewMemoryStorage()
225			rawNode, err := NewRawNode(newTestConfig(1, []uint64{1}, 10, 1, s))
226			if err != nil {
227				t.Fatal(err)
228			}
229
230			rawNode.Campaign()
231			proposed := false
232			var (
233				lastIndex uint64
234				ccdata    []byte
235			)
236			// Propose the ConfChange, wait until it applies, save the resulting
237			// ConfState.
238			var cs *pb.ConfState
239			for cs == nil {
240				rd := rawNode.Ready()
241				s.Append(rd.Entries)
242				for _, ent := range rd.CommittedEntries {
243					var cc pb.ConfChangeI
244					if ent.Type == pb.EntryConfChange {
245						var ccc pb.ConfChange
246						if err = ccc.Unmarshal(ent.Data); err != nil {
247							t.Fatal(err)
248						}
249						cc = ccc
250					} else if ent.Type == pb.EntryConfChangeV2 {
251						var ccc pb.ConfChangeV2
252						if err = ccc.Unmarshal(ent.Data); err != nil {
253							t.Fatal(err)
254						}
255						cc = ccc
256					}
257					if cc != nil {
258						cs = rawNode.ApplyConfChange(cc)
259					}
260				}
261				rawNode.Advance(rd)
262				// Once we are the leader, propose a command and a ConfChange.
263				if !proposed && rd.SoftState.Lead == rawNode.raft.id {
264					if err = rawNode.Propose([]byte("somedata")); err != nil {
265						t.Fatal(err)
266					}
267					if ccv1, ok := tc.cc.AsV1(); ok {
268						ccdata, err = ccv1.Marshal()
269						if err != nil {
270							t.Fatal(err)
271						}
272						rawNode.ProposeConfChange(ccv1)
273					} else {
274						ccv2 := tc.cc.AsV2()
275						ccdata, err = ccv2.Marshal()
276						if err != nil {
277							t.Fatal(err)
278						}
279						rawNode.ProposeConfChange(ccv2)
280					}
281					proposed = true
282				}
283			}
284
285			// Check that the last index is exactly the conf change we put in,
286			// down to the bits. Note that this comes from the Storage, which
287			// will not reflect any unstable entries that we'll only be presented
288			// with in the next Ready.
289			lastIndex, err = s.LastIndex()
290			if err != nil {
291				t.Fatal(err)
292			}
293
294			entries, err := s.Entries(lastIndex-1, lastIndex+1, noLimit)
295			if err != nil {
296				t.Fatal(err)
297			}
298			if len(entries) != 2 {
299				t.Fatalf("len(entries) = %d, want %d", len(entries), 2)
300			}
301			if !bytes.Equal(entries[0].Data, []byte("somedata")) {
302				t.Errorf("entries[0].Data = %v, want %v", entries[0].Data, []byte("somedata"))
303			}
304			typ := pb.EntryConfChange
305			if _, ok := tc.cc.AsV1(); !ok {
306				typ = pb.EntryConfChangeV2
307			}
308			if entries[1].Type != typ {
309				t.Fatalf("type = %v, want %v", entries[1].Type, typ)
310			}
311			if !bytes.Equal(entries[1].Data, ccdata) {
312				t.Errorf("data = %v, want %v", entries[1].Data, ccdata)
313			}
314
315			if exp := &tc.exp; !reflect.DeepEqual(exp, cs) {
316				t.Fatalf("exp:\n%+v\nact:\n%+v", exp, cs)
317			}
318
319			var maybePlusOne uint64
320			if autoLeave, ok := tc.cc.AsV2().EnterJoint(); ok && autoLeave {
321				// If this is an auto-leaving joint conf change, it will have
322				// appended the entry that auto-leaves, so add one to the last
323				// index that forms the basis of our expectations on
324				// pendingConfIndex. (Recall that lastIndex was taken from stable
325				// storage, but this auto-leaving entry isn't on stable storage
326				// yet).
327				maybePlusOne = 1
328			}
329			if exp, act := lastIndex+maybePlusOne, rawNode.raft.pendingConfIndex; exp != act {
330				t.Fatalf("pendingConfIndex: expected %d, got %d", exp, act)
331			}
332
333			// Move the RawNode along. If the ConfChange was simple, nothing else
334			// should happen. Otherwise, we're in a joint state, which is either
335			// left automatically or not. If not, we add the proposal that leaves
336			// it manually.
337			rd := rawNode.Ready()
338			var context []byte
339			if !tc.exp.AutoLeave {
340				if len(rd.Entries) > 0 {
341					t.Fatal("expected no more entries")
342				}
343				if tc.exp2 == nil {
344					return
345				}
346				context = []byte("manual")
347				t.Log("leaving joint state manually")
348				if err := rawNode.ProposeConfChange(pb.ConfChangeV2{Context: context}); err != nil {
349					t.Fatal(err)
350				}
351				rd = rawNode.Ready()
352			}
353
354			// Check that the right ConfChange comes out.
355			if len(rd.Entries) != 1 || rd.Entries[0].Type != pb.EntryConfChangeV2 {
356				t.Fatalf("expected exactly one more entry, got %+v", rd)
357			}
358			var cc pb.ConfChangeV2
359			if err := cc.Unmarshal(rd.Entries[0].Data); err != nil {
360				t.Fatal(err)
361			}
362			if !reflect.DeepEqual(cc, pb.ConfChangeV2{Context: context}) {
363				t.Fatalf("expected zero ConfChangeV2, got %+v", cc)
364			}
365			// Lie and pretend the ConfChange applied. It won't do so because now
366			// we require the joint quorum and we're only running one node.
367			cs = rawNode.ApplyConfChange(cc)
368			if exp := tc.exp2; !reflect.DeepEqual(exp, cs) {
369				t.Fatalf("exp:\n%+v\nact:\n%+v", exp, cs)
370			}
371		})
372	}
373}
374
375// TestRawNodeProposeAddDuplicateNode ensures that two proposes to add the same node should
376// not affect the later propose to add new node.
377func TestRawNodeProposeAddDuplicateNode(t *testing.T) {
378	s := NewMemoryStorage()
379	rawNode, err := NewRawNode(newTestConfig(1, []uint64{1}, 10, 1, s))
380	if err != nil {
381		t.Fatal(err)
382	}
383	rd := rawNode.Ready()
384	s.Append(rd.Entries)
385	rawNode.Advance(rd)
386
387	rawNode.Campaign()
388	for {
389		rd = rawNode.Ready()
390		s.Append(rd.Entries)
391		if rd.SoftState.Lead == rawNode.raft.id {
392			rawNode.Advance(rd)
393			break
394		}
395		rawNode.Advance(rd)
396	}
397
398	proposeConfChangeAndApply := func(cc pb.ConfChange) {
399		rawNode.ProposeConfChange(cc)
400		rd = rawNode.Ready()
401		s.Append(rd.Entries)
402		for _, entry := range rd.CommittedEntries {
403			if entry.Type == pb.EntryConfChange {
404				var cc pb.ConfChange
405				cc.Unmarshal(entry.Data)
406				rawNode.ApplyConfChange(cc)
407			}
408		}
409		rawNode.Advance(rd)
410	}
411
412	cc1 := pb.ConfChange{Type: pb.ConfChangeAddNode, NodeID: 1}
413	ccdata1, err := cc1.Marshal()
414	if err != nil {
415		t.Fatal(err)
416	}
417	proposeConfChangeAndApply(cc1)
418
419	// try to add the same node again
420	proposeConfChangeAndApply(cc1)
421
422	// the new node join should be ok
423	cc2 := pb.ConfChange{Type: pb.ConfChangeAddNode, NodeID: 2}
424	ccdata2, err := cc2.Marshal()
425	if err != nil {
426		t.Fatal(err)
427	}
428	proposeConfChangeAndApply(cc2)
429
430	lastIndex, err := s.LastIndex()
431	if err != nil {
432		t.Fatal(err)
433	}
434
435	// the last three entries should be: ConfChange cc1, cc1, cc2
436	entries, err := s.Entries(lastIndex-2, lastIndex+1, noLimit)
437	if err != nil {
438		t.Fatal(err)
439	}
440	if len(entries) != 3 {
441		t.Fatalf("len(entries) = %d, want %d", len(entries), 3)
442	}
443	if !bytes.Equal(entries[0].Data, ccdata1) {
444		t.Errorf("entries[0].Data = %v, want %v", entries[0].Data, ccdata1)
445	}
446	if !bytes.Equal(entries[2].Data, ccdata2) {
447		t.Errorf("entries[2].Data = %v, want %v", entries[2].Data, ccdata2)
448	}
449}
450
451// TestRawNodeReadIndex ensures that Rawnode.ReadIndex sends the MsgReadIndex message
452// to the underlying raft. It also ensures that ReadState can be read out.
453func TestRawNodeReadIndex(t *testing.T) {
454	msgs := []pb.Message{}
455	appendStep := func(r *raft, m pb.Message) error {
456		msgs = append(msgs, m)
457		return nil
458	}
459	wrs := []ReadState{{Index: uint64(1), RequestCtx: []byte("somedata")}}
460
461	s := NewMemoryStorage()
462	c := newTestConfig(1, []uint64{1}, 10, 1, s)
463	rawNode, err := NewRawNode(c)
464	if err != nil {
465		t.Fatal(err)
466	}
467	rawNode.raft.readStates = wrs
468	// ensure the ReadStates can be read out
469	hasReady := rawNode.HasReady()
470	if !hasReady {
471		t.Errorf("HasReady() returns %t, want %t", hasReady, true)
472	}
473	rd := rawNode.Ready()
474	if !reflect.DeepEqual(rd.ReadStates, wrs) {
475		t.Errorf("ReadStates = %d, want %d", rd.ReadStates, wrs)
476	}
477	s.Append(rd.Entries)
478	rawNode.Advance(rd)
479	// ensure raft.readStates is reset after advance
480	if rawNode.raft.readStates != nil {
481		t.Errorf("readStates = %v, want %v", rawNode.raft.readStates, nil)
482	}
483
484	wrequestCtx := []byte("somedata2")
485	rawNode.Campaign()
486	for {
487		rd = rawNode.Ready()
488		s.Append(rd.Entries)
489
490		if rd.SoftState.Lead == rawNode.raft.id {
491			rawNode.Advance(rd)
492
493			// Once we are the leader, issue a ReadIndex request
494			rawNode.raft.step = appendStep
495			rawNode.ReadIndex(wrequestCtx)
496			break
497		}
498		rawNode.Advance(rd)
499	}
500	// ensure that MsgReadIndex message is sent to the underlying raft
501	if len(msgs) != 1 {
502		t.Fatalf("len(msgs) = %d, want %d", len(msgs), 1)
503	}
504	if msgs[0].Type != pb.MsgReadIndex {
505		t.Errorf("msg type = %d, want %d", msgs[0].Type, pb.MsgReadIndex)
506	}
507	if !bytes.Equal(msgs[0].Entries[0].Data, wrequestCtx) {
508		t.Errorf("data = %v, want %v", msgs[0].Entries[0].Data, wrequestCtx)
509	}
510}
511
512// TestBlockProposal from node_test.go has no equivalent in rawNode because there is
513// no leader check in RawNode.
514
515// TestNodeTick from node_test.go has no equivalent in rawNode because
516// it reaches into the raft object which is not exposed.
517
518// TestNodeStop from node_test.go has no equivalent in rawNode because there is
519// no goroutine in RawNode.
520
521// TestRawNodeStart ensures that a node can be started correctly. Note that RawNode
522// requires the application to bootstrap the state, i.e. it does not accept peers
523// and will not create faux configuration change entries.
524func TestRawNodeStart(t *testing.T) {
525	want := Ready{
526		SoftState: &SoftState{Lead: 1, RaftState: StateLeader},
527		HardState: pb.HardState{Term: 1, Commit: 3, Vote: 1},
528		Entries: []pb.Entry{
529			{Term: 1, Index: 2, Data: nil},           // empty entry
530			{Term: 1, Index: 3, Data: []byte("foo")}, // empty entry
531		},
532		CommittedEntries: []pb.Entry{
533			{Term: 1, Index: 2, Data: nil},           // empty entry
534			{Term: 1, Index: 3, Data: []byte("foo")}, // empty entry
535		},
536		MustSync: true,
537	}
538
539	storage := NewMemoryStorage()
540	storage.ents[0].Index = 1
541
542	// TODO(tbg): this is a first prototype of what bootstrapping could look
543	// like (without the annoying faux ConfChanges). We want to persist a
544	// ConfState at some index and make sure that this index can't be reached
545	// from log position 1, so that followers are forced to pick up the
546	// ConfState in order to move away from log position 1 (unless they got
547	// bootstrapped in the same way already). Failing to do so would mean that
548	// followers diverge from the bootstrapped nodes and don't learn about the
549	// initial config.
550	//
551	// NB: this is exactly what CockroachDB does. The Raft log really begins at
552	// index 10, so empty followers (at index 1) always need a snapshot first.
553	type appenderStorage interface {
554		Storage
555		ApplySnapshot(pb.Snapshot) error
556	}
557	bootstrap := func(storage appenderStorage, cs pb.ConfState) error {
558		if len(cs.Voters) == 0 {
559			return fmt.Errorf("no voters specified")
560		}
561		fi, err := storage.FirstIndex()
562		if err != nil {
563			return err
564		}
565		if fi < 2 {
566			return fmt.Errorf("FirstIndex >= 2 is prerequisite for bootstrap")
567		}
568		if _, err = storage.Entries(fi, fi, math.MaxUint64); err == nil {
569			// TODO(tbg): match exact error
570			return fmt.Errorf("should not have been able to load first index")
571		}
572		li, err := storage.LastIndex()
573		if err != nil {
574			return err
575		}
576		if _, err = storage.Entries(li, li, math.MaxUint64); err == nil {
577			return fmt.Errorf("should not have been able to load last index")
578		}
579		hs, ics, err := storage.InitialState()
580		if err != nil {
581			return err
582		}
583		if !IsEmptyHardState(hs) {
584			return fmt.Errorf("HardState not empty")
585		}
586		if len(ics.Voters) != 0 {
587			return fmt.Errorf("ConfState not empty")
588		}
589
590		meta := pb.SnapshotMetadata{
591			Index:     1,
592			Term:      0,
593			ConfState: cs,
594		}
595		snap := pb.Snapshot{Metadata: meta}
596		return storage.ApplySnapshot(snap)
597	}
598
599	if err := bootstrap(storage, pb.ConfState{Voters: []uint64{1}}); err != nil {
600		t.Fatal(err)
601	}
602
603	rawNode, err := NewRawNode(newTestConfig(1, nil, 10, 1, storage))
604	if err != nil {
605		t.Fatal(err)
606	}
607	if rawNode.HasReady() {
608		t.Fatalf("unexpected ready: %+v", rawNode.Ready())
609	}
610	rawNode.Campaign()
611	rawNode.Propose([]byte("foo"))
612	if !rawNode.HasReady() {
613		t.Fatal("expected a Ready")
614	}
615	rd := rawNode.Ready()
616	storage.Append(rd.Entries)
617	rawNode.Advance(rd)
618
619	rd.SoftState, want.SoftState = nil, nil
620
621	if !reflect.DeepEqual(rd, want) {
622		t.Fatalf("unexpected Ready:\n%+v\nvs\n%+v", rd, want)
623	}
624
625	if rawNode.HasReady() {
626		t.Errorf("unexpected Ready: %+v", rawNode.Ready())
627	}
628}
629
630func TestRawNodeRestart(t *testing.T) {
631	entries := []pb.Entry{
632		{Term: 1, Index: 1},
633		{Term: 1, Index: 2, Data: []byte("foo")},
634	}
635	st := pb.HardState{Term: 1, Commit: 1}
636
637	want := Ready{
638		HardState: emptyState,
639		// commit up to commit index in st
640		CommittedEntries: entries[:st.Commit],
641		MustSync:         false,
642	}
643
644	storage := NewMemoryStorage()
645	storage.SetHardState(st)
646	storage.Append(entries)
647	rawNode, err := NewRawNode(newTestConfig(1, []uint64{1}, 10, 1, storage))
648	if err != nil {
649		t.Fatal(err)
650	}
651	rd := rawNode.Ready()
652	if !reflect.DeepEqual(rd, want) {
653		t.Errorf("g = %+v,\n             w   %+v", rd, want)
654	}
655	rawNode.Advance(rd)
656	if rawNode.HasReady() {
657		t.Errorf("unexpected Ready: %+v", rawNode.Ready())
658	}
659}
660
661func TestRawNodeRestartFromSnapshot(t *testing.T) {
662	snap := pb.Snapshot{
663		Metadata: pb.SnapshotMetadata{
664			ConfState: pb.ConfState{Voters: []uint64{1, 2}},
665			Index:     2,
666			Term:      1,
667		},
668	}
669	entries := []pb.Entry{
670		{Term: 1, Index: 3, Data: []byte("foo")},
671	}
672	st := pb.HardState{Term: 1, Commit: 3}
673
674	want := Ready{
675		HardState: emptyState,
676		// commit up to commit index in st
677		CommittedEntries: entries,
678		MustSync:         false,
679	}
680
681	s := NewMemoryStorage()
682	s.SetHardState(st)
683	s.ApplySnapshot(snap)
684	s.Append(entries)
685	rawNode, err := NewRawNode(newTestConfig(1, nil, 10, 1, s))
686	if err != nil {
687		t.Fatal(err)
688	}
689	if rd := rawNode.Ready(); !reflect.DeepEqual(rd, want) {
690		t.Errorf("g = %+v,\n             w   %+v", rd, want)
691	} else {
692		rawNode.Advance(rd)
693	}
694	if rawNode.HasReady() {
695		t.Errorf("unexpected Ready: %+v", rawNode.HasReady())
696	}
697}
698
699// TestNodeAdvance from node_test.go has no equivalent in rawNode because there is
700// no dependency check between Ready() and Advance()
701
702func TestRawNodeStatus(t *testing.T) {
703	s := NewMemoryStorage()
704	rn, err := NewRawNode(newTestConfig(1, []uint64{1}, 10, 1, s))
705	if err != nil {
706		t.Fatal(err)
707	}
708	if status := rn.Status(); status.Progress != nil {
709		t.Fatalf("expected no Progress because not leader: %+v", status.Progress)
710	}
711	if err := rn.Campaign(); err != nil {
712		t.Fatal(err)
713	}
714	status := rn.Status()
715	if status.Lead != 1 {
716		t.Fatal("not lead")
717	}
718	if status.RaftState != StateLeader {
719		t.Fatal("not leader")
720	}
721	if exp, act := *rn.raft.prs.Progress[1], status.Progress[1]; !reflect.DeepEqual(exp, act) {
722		t.Fatalf("want: %+v\ngot:  %+v", exp, act)
723	}
724	expCfg := tracker.Config{Voters: quorum.JointConfig{
725		quorum.MajorityConfig{1: {}},
726		nil,
727	}}
728	if !reflect.DeepEqual(expCfg, status.Config) {
729		t.Fatalf("want: %+v\ngot:  %+v", expCfg, status.Config)
730	}
731}
732
733// TestRawNodeCommitPaginationAfterRestart is the RawNode version of
734// TestNodeCommitPaginationAfterRestart. The anomaly here was even worse as the
735// Raft group would forget to apply entries:
736//
737// - node learns that index 11 is committed
738// - nextEnts returns index 1..10 in CommittedEntries (but index 10 already
739//   exceeds maxBytes), which isn't noticed internally by Raft
740// - Commit index gets bumped to 10
741// - the node persists the HardState, but crashes before applying the entries
742// - upon restart, the storage returns the same entries, but `slice` takes a
743//   different code path and removes the last entry.
744// - Raft does not emit a HardState, but when the app calls Advance(), it bumps
745//   its internal applied index cursor to 10 (when it should be 9)
746// - the next Ready asks the app to apply index 11 (omitting index 10), losing a
747//    write.
748func TestRawNodeCommitPaginationAfterRestart(t *testing.T) {
749	s := &ignoreSizeHintMemStorage{
750		MemoryStorage: NewMemoryStorage(),
751	}
752	persistedHardState := pb.HardState{
753		Term:   1,
754		Vote:   1,
755		Commit: 10,
756	}
757
758	s.hardState = persistedHardState
759	s.ents = make([]pb.Entry, 10)
760	var size uint64
761	for i := range s.ents {
762		ent := pb.Entry{
763			Term:  1,
764			Index: uint64(i + 1),
765			Type:  pb.EntryNormal,
766			Data:  []byte("a"),
767		}
768
769		s.ents[i] = ent
770		size += uint64(ent.Size())
771	}
772
773	cfg := newTestConfig(1, []uint64{1}, 10, 1, s)
774	// Set a MaxSizePerMsg that would suggest to Raft that the last committed entry should
775	// not be included in the initial rd.CommittedEntries. However, our storage will ignore
776	// this and *will* return it (which is how the Commit index ended up being 10 initially).
777	cfg.MaxSizePerMsg = size - uint64(s.ents[len(s.ents)-1].Size()) - 1
778
779	s.ents = append(s.ents, pb.Entry{
780		Term:  1,
781		Index: uint64(11),
782		Type:  pb.EntryNormal,
783		Data:  []byte("boom"),
784	})
785
786	rawNode, err := NewRawNode(cfg)
787	if err != nil {
788		t.Fatal(err)
789	}
790
791	for highestApplied := uint64(0); highestApplied != 11; {
792		rd := rawNode.Ready()
793		n := len(rd.CommittedEntries)
794		if n == 0 {
795			t.Fatalf("stopped applying entries at index %d", highestApplied)
796		}
797		if next := rd.CommittedEntries[0].Index; highestApplied != 0 && highestApplied+1 != next {
798			t.Fatalf("attempting to apply index %d after index %d, leaving a gap", next, highestApplied)
799		}
800		highestApplied = rd.CommittedEntries[n-1].Index
801		rawNode.Advance(rd)
802		rawNode.Step(pb.Message{
803			Type:   pb.MsgHeartbeat,
804			To:     1,
805			From:   1, // illegal, but we get away with it
806			Term:   1,
807			Commit: 11,
808		})
809	}
810}
811
812// TestRawNodeBoundedLogGrowthWithPartition tests a scenario where a leader is
813// partitioned from a quorum of nodes. It verifies that the leader's log is
814// protected from unbounded growth even as new entries continue to be proposed.
815// This protection is provided by the MaxUncommittedEntriesSize configuration.
816func TestRawNodeBoundedLogGrowthWithPartition(t *testing.T) {
817	const maxEntries = 16
818	data := []byte("testdata")
819	testEntry := pb.Entry{Data: data}
820	maxEntrySize := uint64(maxEntries * PayloadSize(testEntry))
821
822	s := NewMemoryStorage()
823	cfg := newTestConfig(1, []uint64{1}, 10, 1, s)
824	cfg.MaxUncommittedEntriesSize = maxEntrySize
825	rawNode, err := NewRawNode(cfg)
826	if err != nil {
827		t.Fatal(err)
828	}
829	rd := rawNode.Ready()
830	s.Append(rd.Entries)
831	rawNode.Advance(rd)
832
833	// Become the leader.
834	rawNode.Campaign()
835	for {
836		rd = rawNode.Ready()
837		s.Append(rd.Entries)
838		if rd.SoftState.Lead == rawNode.raft.id {
839			rawNode.Advance(rd)
840			break
841		}
842		rawNode.Advance(rd)
843	}
844
845	// Simulate a network partition while we make our proposals by never
846	// committing anything. These proposals should not cause the leader's
847	// log to grow indefinitely.
848	for i := 0; i < 1024; i++ {
849		rawNode.Propose(data)
850	}
851
852	// Check the size of leader's uncommitted log tail. It should not exceed the
853	// MaxUncommittedEntriesSize limit.
854	checkUncommitted := func(exp uint64) {
855		t.Helper()
856		if a := rawNode.raft.uncommittedSize; exp != a {
857			t.Fatalf("expected %d uncommitted entry bytes, found %d", exp, a)
858		}
859	}
860	checkUncommitted(maxEntrySize)
861
862	// Recover from the partition. The uncommitted tail of the Raft log should
863	// disappear as entries are committed.
864	rd = rawNode.Ready()
865	if len(rd.CommittedEntries) != maxEntries {
866		t.Fatalf("expected %d entries, got %d", maxEntries, len(rd.CommittedEntries))
867	}
868	s.Append(rd.Entries)
869	rawNode.Advance(rd)
870	checkUncommitted(0)
871}
872
873func BenchmarkStatus(b *testing.B) {
874	setup := func(members int) *RawNode {
875		peers := make([]uint64, members)
876		for i := range peers {
877			peers[i] = uint64(i + 1)
878		}
879		cfg := newTestConfig(1, peers, 3, 1, NewMemoryStorage())
880		cfg.Logger = discardLogger
881		r := newRaft(cfg)
882		r.becomeFollower(1, 1)
883		r.becomeCandidate()
884		r.becomeLeader()
885		return &RawNode{raft: r}
886	}
887
888	for _, members := range []int{1, 3, 5, 100} {
889		b.Run(fmt.Sprintf("members=%d", members), func(b *testing.B) {
890			rn := setup(members)
891
892			b.Run("Status", func(b *testing.B) {
893				b.ReportAllocs()
894				for i := 0; i < b.N; i++ {
895					_ = rn.Status()
896				}
897			})
898
899			b.Run("Status-example", func(b *testing.B) {
900				b.ReportAllocs()
901				for i := 0; i < b.N; i++ {
902					s := rn.Status()
903					var n uint64
904					for _, pr := range s.Progress {
905						n += pr.Match
906					}
907					_ = n
908				}
909			})
910
911			b.Run("BasicStatus", func(b *testing.B) {
912				b.ReportAllocs()
913				for i := 0; i < b.N; i++ {
914					_ = rn.BasicStatus()
915				}
916			})
917
918			b.Run("WithProgress", func(b *testing.B) {
919				b.ReportAllocs()
920				visit := func(uint64, ProgressType, tracker.Progress) {}
921
922				for i := 0; i < b.N; i++ {
923					rn.WithProgress(visit)
924				}
925			})
926			b.Run("WithProgress-example", func(b *testing.B) {
927				b.ReportAllocs()
928				for i := 0; i < b.N; i++ {
929					var n uint64
930					visit := func(_ uint64, _ ProgressType, pr tracker.Progress) {
931						n += pr.Match
932					}
933					rn.WithProgress(visit)
934					_ = n
935				}
936			})
937		})
938	}
939}
940
941func TestRawNodeConsumeReady(t *testing.T) {
942	// Check that readyWithoutAccept() does not call acceptReady (which resets
943	// the messages) but Ready() does.
944	s := NewMemoryStorage()
945	rn := newTestRawNode(1, []uint64{1}, 3, 1, s)
946	m1 := pb.Message{Context: []byte("foo")}
947	m2 := pb.Message{Context: []byte("bar")}
948
949	// Inject first message, make sure it's visible via readyWithoutAccept.
950	rn.raft.msgs = append(rn.raft.msgs, m1)
951	rd := rn.readyWithoutAccept()
952	if len(rd.Messages) != 1 || !reflect.DeepEqual(rd.Messages[0], m1) {
953		t.Fatalf("expected only m1 sent, got %+v", rd.Messages)
954	}
955	if len(rn.raft.msgs) != 1 || !reflect.DeepEqual(rn.raft.msgs[0], m1) {
956		t.Fatalf("expected only m1 in raft.msgs, got %+v", rn.raft.msgs)
957	}
958	// Now call Ready() which should move the message into the Ready (as opposed
959	// to leaving it in both places).
960	rd = rn.Ready()
961	if len(rn.raft.msgs) > 0 {
962		t.Fatalf("messages not reset: %+v", rn.raft.msgs)
963	}
964	if len(rd.Messages) != 1 || !reflect.DeepEqual(rd.Messages[0], m1) {
965		t.Fatalf("expected only m1 sent, got %+v", rd.Messages)
966	}
967	// Add a message to raft to make sure that Advance() doesn't drop it.
968	rn.raft.msgs = append(rn.raft.msgs, m2)
969	rn.Advance(rd)
970	if len(rn.raft.msgs) != 1 || !reflect.DeepEqual(rn.raft.msgs[0], m2) {
971		t.Fatalf("expected only m2 in raft.msgs, got %+v", rn.raft.msgs)
972	}
973}
974