go.etcd.io/etcd@v3.3.27+incompatible/raft/rafttest/node_test.go (about) 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 15 package rafttest 16 17 import ( 18 "context" 19 "testing" 20 "time" 21 22 "github.com/coreos/etcd/raft" 23 ) 24 25 func TestBasicProgress(t *testing.T) { 26 peers := []raft.Peer{{1, nil}, {2, nil}, {3, nil}, {4, nil}, {5, nil}} 27 nt := newRaftNetwork(1, 2, 3, 4, 5) 28 29 nodes := make([]*node, 0) 30 31 for i := 1; i <= 5; i++ { 32 n := startNode(uint64(i), peers, nt.nodeNetwork(uint64(i))) 33 nodes = append(nodes, n) 34 } 35 36 waitLeader(nodes) 37 38 for i := 0; i < 100; i++ { 39 nodes[0].Propose(context.TODO(), []byte("somedata")) 40 } 41 42 if !waitCommitConverge(nodes, 100) { 43 t.Errorf("commits failed to converge!") 44 } 45 46 for _, n := range nodes { 47 n.stop() 48 } 49 } 50 51 func TestRestart(t *testing.T) { 52 peers := []raft.Peer{{1, nil}, {2, nil}, {3, nil}, {4, nil}, {5, nil}} 53 nt := newRaftNetwork(1, 2, 3, 4, 5) 54 55 nodes := make([]*node, 0) 56 57 for i := 1; i <= 5; i++ { 58 n := startNode(uint64(i), peers, nt.nodeNetwork(uint64(i))) 59 nodes = append(nodes, n) 60 } 61 62 l := waitLeader(nodes) 63 k1, k2 := (l+1)%5, (l+2)%5 64 65 for i := 0; i < 30; i++ { 66 nodes[l].Propose(context.TODO(), []byte("somedata")) 67 } 68 nodes[k1].stop() 69 for i := 0; i < 30; i++ { 70 nodes[(l+3)%5].Propose(context.TODO(), []byte("somedata")) 71 } 72 nodes[k2].stop() 73 for i := 0; i < 30; i++ { 74 nodes[(l+4)%5].Propose(context.TODO(), []byte("somedata")) 75 } 76 nodes[k2].restart() 77 for i := 0; i < 30; i++ { 78 nodes[l].Propose(context.TODO(), []byte("somedata")) 79 } 80 nodes[k1].restart() 81 82 if !waitCommitConverge(nodes, 120) { 83 t.Errorf("commits failed to converge!") 84 } 85 86 for _, n := range nodes { 87 n.stop() 88 } 89 } 90 91 func TestPause(t *testing.T) { 92 peers := []raft.Peer{{1, nil}, {2, nil}, {3, nil}, {4, nil}, {5, nil}} 93 nt := newRaftNetwork(1, 2, 3, 4, 5) 94 95 nodes := make([]*node, 0) 96 97 for i := 1; i <= 5; i++ { 98 n := startNode(uint64(i), peers, nt.nodeNetwork(uint64(i))) 99 nodes = append(nodes, n) 100 } 101 102 waitLeader(nodes) 103 104 for i := 0; i < 30; i++ { 105 nodes[0].Propose(context.TODO(), []byte("somedata")) 106 } 107 nodes[1].pause() 108 for i := 0; i < 30; i++ { 109 nodes[0].Propose(context.TODO(), []byte("somedata")) 110 } 111 nodes[2].pause() 112 for i := 0; i < 30; i++ { 113 nodes[0].Propose(context.TODO(), []byte("somedata")) 114 } 115 nodes[2].resume() 116 for i := 0; i < 30; i++ { 117 nodes[0].Propose(context.TODO(), []byte("somedata")) 118 } 119 nodes[1].resume() 120 121 if !waitCommitConverge(nodes, 120) { 122 t.Errorf("commits failed to converge!") 123 } 124 125 for _, n := range nodes { 126 n.stop() 127 } 128 } 129 130 func waitLeader(ns []*node) int { 131 var l map[uint64]struct{} 132 var lindex int 133 134 for { 135 l = make(map[uint64]struct{}) 136 137 for i, n := range ns { 138 lead := n.Status().SoftState.Lead 139 if lead != 0 { 140 l[lead] = struct{}{} 141 if n.id == lead { 142 lindex = i 143 } 144 } 145 } 146 147 if len(l) == 1 { 148 return lindex 149 } 150 } 151 } 152 153 func waitCommitConverge(ns []*node, target uint64) bool { 154 var c map[uint64]struct{} 155 156 for i := 0; i < 50; i++ { 157 c = make(map[uint64]struct{}) 158 var good int 159 160 for _, n := range ns { 161 commit := n.Node.Status().HardState.Commit 162 c[commit] = struct{}{} 163 if commit > target { 164 good++ 165 } 166 } 167 168 if len(c) == 1 && good == len(ns) { 169 return true 170 } 171 time.Sleep(100 * time.Millisecond) 172 } 173 174 return false 175 }