go.etcd.io/etcd@v3.3.27+incompatible/raft/rawnode_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 raft 16 17 import ( 18 "bytes" 19 "reflect" 20 "testing" 21 22 "github.com/coreos/etcd/raft/raftpb" 23 ) 24 25 // TestRawNodeStep ensures that RawNode.Step ignore local message. 26 func TestRawNodeStep(t *testing.T) { 27 for i, msgn := range raftpb.MessageType_name { 28 s := NewMemoryStorage() 29 rawNode, err := NewRawNode(newTestConfig(1, nil, 10, 1, s), []Peer{{ID: 1}}) 30 if err != nil { 31 t.Fatal(err) 32 } 33 msgt := raftpb.MessageType(i) 34 err = rawNode.Step(raftpb.Message{Type: msgt}) 35 // LocalMsg should be ignored. 36 if IsLocalMsg(msgt) { 37 if err != ErrStepLocalMsg { 38 t.Errorf("%d: step should ignore %s", msgt, msgn) 39 } 40 } 41 } 42 } 43 44 // TestNodeStepUnblock from node_test.go has no equivalent in rawNode because there is 45 // no goroutine in RawNode. 46 47 // TestRawNodeProposeAndConfChange ensures that RawNode.Propose and RawNode.ProposeConfChange 48 // send the given proposal and ConfChange to the underlying raft. 49 func TestRawNodeProposeAndConfChange(t *testing.T) { 50 s := NewMemoryStorage() 51 var err error 52 rawNode, err := NewRawNode(newTestConfig(1, nil, 10, 1, s), []Peer{{ID: 1}}) 53 if err != nil { 54 t.Fatal(err) 55 } 56 rd := rawNode.Ready() 57 s.Append(rd.Entries) 58 rawNode.Advance(rd) 59 60 rawNode.Campaign() 61 proposed := false 62 var ( 63 lastIndex uint64 64 ccdata []byte 65 ) 66 for { 67 rd = rawNode.Ready() 68 s.Append(rd.Entries) 69 // Once we are the leader, propose a command and a ConfChange. 70 if !proposed && rd.SoftState.Lead == rawNode.raft.id { 71 rawNode.Propose([]byte("somedata")) 72 73 cc := raftpb.ConfChange{Type: raftpb.ConfChangeAddNode, NodeID: 1} 74 ccdata, err = cc.Marshal() 75 if err != nil { 76 t.Fatal(err) 77 } 78 rawNode.ProposeConfChange(cc) 79 80 proposed = true 81 } 82 rawNode.Advance(rd) 83 84 // Exit when we have four entries: one ConfChange, one no-op for the election, 85 // our proposed command and proposed ConfChange. 86 lastIndex, err = s.LastIndex() 87 if err != nil { 88 t.Fatal(err) 89 } 90 if lastIndex >= 4 { 91 break 92 } 93 } 94 95 entries, err := s.Entries(lastIndex-1, lastIndex+1, noLimit) 96 if err != nil { 97 t.Fatal(err) 98 } 99 if len(entries) != 2 { 100 t.Fatalf("len(entries) = %d, want %d", len(entries), 2) 101 } 102 if !bytes.Equal(entries[0].Data, []byte("somedata")) { 103 t.Errorf("entries[0].Data = %v, want %v", entries[0].Data, []byte("somedata")) 104 } 105 if entries[1].Type != raftpb.EntryConfChange { 106 t.Fatalf("type = %v, want %v", entries[1].Type, raftpb.EntryConfChange) 107 } 108 if !bytes.Equal(entries[1].Data, ccdata) { 109 t.Errorf("data = %v, want %v", entries[1].Data, ccdata) 110 } 111 } 112 113 // TestRawNodeProposeAddDuplicateNode ensures that two proposes to add the same node should 114 // not affect the later propose to add new node. 115 func TestRawNodeProposeAddDuplicateNode(t *testing.T) { 116 s := NewMemoryStorage() 117 rawNode, err := NewRawNode(newTestConfig(1, nil, 10, 1, s), []Peer{{ID: 1}}) 118 if err != nil { 119 t.Fatal(err) 120 } 121 rd := rawNode.Ready() 122 s.Append(rd.Entries) 123 rawNode.Advance(rd) 124 125 rawNode.Campaign() 126 for { 127 rd = rawNode.Ready() 128 s.Append(rd.Entries) 129 if rd.SoftState.Lead == rawNode.raft.id { 130 rawNode.Advance(rd) 131 break 132 } 133 rawNode.Advance(rd) 134 } 135 136 proposeConfChangeAndApply := func(cc raftpb.ConfChange) { 137 rawNode.ProposeConfChange(cc) 138 rd = rawNode.Ready() 139 s.Append(rd.Entries) 140 for _, entry := range rd.CommittedEntries { 141 if entry.Type == raftpb.EntryConfChange { 142 var cc raftpb.ConfChange 143 cc.Unmarshal(entry.Data) 144 rawNode.ApplyConfChange(cc) 145 } 146 } 147 rawNode.Advance(rd) 148 } 149 150 cc1 := raftpb.ConfChange{Type: raftpb.ConfChangeAddNode, NodeID: 1} 151 ccdata1, err := cc1.Marshal() 152 if err != nil { 153 t.Fatal(err) 154 } 155 proposeConfChangeAndApply(cc1) 156 157 // try to add the same node again 158 proposeConfChangeAndApply(cc1) 159 160 // the new node join should be ok 161 cc2 := raftpb.ConfChange{Type: raftpb.ConfChangeAddNode, NodeID: 2} 162 ccdata2, err := cc2.Marshal() 163 if err != nil { 164 t.Fatal(err) 165 } 166 proposeConfChangeAndApply(cc2) 167 168 lastIndex, err := s.LastIndex() 169 if err != nil { 170 t.Fatal(err) 171 } 172 173 // the last three entries should be: ConfChange cc1, cc1, cc2 174 entries, err := s.Entries(lastIndex-2, lastIndex+1, noLimit) 175 if err != nil { 176 t.Fatal(err) 177 } 178 if len(entries) != 3 { 179 t.Fatalf("len(entries) = %d, want %d", len(entries), 3) 180 } 181 if !bytes.Equal(entries[0].Data, ccdata1) { 182 t.Errorf("entries[0].Data = %v, want %v", entries[0].Data, ccdata1) 183 } 184 if !bytes.Equal(entries[2].Data, ccdata2) { 185 t.Errorf("entries[2].Data = %v, want %v", entries[2].Data, ccdata2) 186 } 187 } 188 189 // TestRawNodeReadIndex ensures that Rawnode.ReadIndex sends the MsgReadIndex message 190 // to the underlying raft. It also ensures that ReadState can be read out. 191 func TestRawNodeReadIndex(t *testing.T) { 192 msgs := []raftpb.Message{} 193 appendStep := func(r *raft, m raftpb.Message) { 194 msgs = append(msgs, m) 195 } 196 wrs := []ReadState{{Index: uint64(1), RequestCtx: []byte("somedata")}} 197 198 s := NewMemoryStorage() 199 c := newTestConfig(1, nil, 10, 1, s) 200 rawNode, err := NewRawNode(c, []Peer{{ID: 1}}) 201 if err != nil { 202 t.Fatal(err) 203 } 204 rawNode.raft.readStates = wrs 205 // ensure the ReadStates can be read out 206 hasReady := rawNode.HasReady() 207 if !hasReady { 208 t.Errorf("HasReady() returns %t, want %t", hasReady, true) 209 } 210 rd := rawNode.Ready() 211 if !reflect.DeepEqual(rd.ReadStates, wrs) { 212 t.Errorf("ReadStates = %d, want %d", rd.ReadStates, wrs) 213 } 214 s.Append(rd.Entries) 215 rawNode.Advance(rd) 216 // ensure raft.readStates is reset after advance 217 if rawNode.raft.readStates != nil { 218 t.Errorf("readStates = %v, want %v", rawNode.raft.readStates, nil) 219 } 220 221 wrequestCtx := []byte("somedata2") 222 rawNode.Campaign() 223 for { 224 rd = rawNode.Ready() 225 s.Append(rd.Entries) 226 227 if rd.SoftState.Lead == rawNode.raft.id { 228 rawNode.Advance(rd) 229 230 // Once we are the leader, issue a ReadIndex request 231 rawNode.raft.step = appendStep 232 rawNode.ReadIndex(wrequestCtx) 233 break 234 } 235 rawNode.Advance(rd) 236 } 237 // ensure that MsgReadIndex message is sent to the underlying raft 238 if len(msgs) != 1 { 239 t.Fatalf("len(msgs) = %d, want %d", len(msgs), 1) 240 } 241 if msgs[0].Type != raftpb.MsgReadIndex { 242 t.Errorf("msg type = %d, want %d", msgs[0].Type, raftpb.MsgReadIndex) 243 } 244 if !bytes.Equal(msgs[0].Entries[0].Data, wrequestCtx) { 245 t.Errorf("data = %v, want %v", msgs[0].Entries[0].Data, wrequestCtx) 246 } 247 } 248 249 // TestBlockProposal from node_test.go has no equivalent in rawNode because there is 250 // no leader check in RawNode. 251 252 // TestNodeTick from node_test.go has no equivalent in rawNode because 253 // it reaches into the raft object which is not exposed. 254 255 // TestNodeStop from node_test.go has no equivalent in rawNode because there is 256 // no goroutine in RawNode. 257 258 // TestRawNodeStart ensures that a node can be started correctly. The node should 259 // start with correct configuration change entries, and can accept and commit 260 // proposals. 261 func TestRawNodeStart(t *testing.T) { 262 cc := raftpb.ConfChange{Type: raftpb.ConfChangeAddNode, NodeID: 1} 263 ccdata, err := cc.Marshal() 264 if err != nil { 265 t.Fatalf("unexpected marshal error: %v", err) 266 } 267 wants := []Ready{ 268 { 269 HardState: raftpb.HardState{Term: 1, Commit: 1, Vote: 0}, 270 Entries: []raftpb.Entry{ 271 {Type: raftpb.EntryConfChange, Term: 1, Index: 1, Data: ccdata}, 272 }, 273 CommittedEntries: []raftpb.Entry{ 274 {Type: raftpb.EntryConfChange, Term: 1, Index: 1, Data: ccdata}, 275 }, 276 MustSync: true, 277 }, 278 { 279 HardState: raftpb.HardState{Term: 2, Commit: 3, Vote: 1}, 280 Entries: []raftpb.Entry{{Term: 2, Index: 3, Data: []byte("foo")}}, 281 CommittedEntries: []raftpb.Entry{{Term: 2, Index: 3, Data: []byte("foo")}}, 282 MustSync: true, 283 }, 284 } 285 286 storage := NewMemoryStorage() 287 rawNode, err := NewRawNode(newTestConfig(1, nil, 10, 1, storage), []Peer{{ID: 1}}) 288 if err != nil { 289 t.Fatal(err) 290 } 291 rd := rawNode.Ready() 292 t.Logf("rd %v", rd) 293 if !reflect.DeepEqual(rd, wants[0]) { 294 t.Fatalf("#%d: g = %+v,\n w %+v", 1, rd, wants[0]) 295 } else { 296 storage.Append(rd.Entries) 297 rawNode.Advance(rd) 298 } 299 storage.Append(rd.Entries) 300 rawNode.Advance(rd) 301 302 rawNode.Campaign() 303 rd = rawNode.Ready() 304 storage.Append(rd.Entries) 305 rawNode.Advance(rd) 306 307 rawNode.Propose([]byte("foo")) 308 if rd = rawNode.Ready(); !reflect.DeepEqual(rd, wants[1]) { 309 t.Errorf("#%d: g = %+v,\n w %+v", 2, rd, wants[1]) 310 } else { 311 storage.Append(rd.Entries) 312 rawNode.Advance(rd) 313 } 314 315 if rawNode.HasReady() { 316 t.Errorf("unexpected Ready: %+v", rawNode.Ready()) 317 } 318 } 319 320 func TestRawNodeRestart(t *testing.T) { 321 entries := []raftpb.Entry{ 322 {Term: 1, Index: 1}, 323 {Term: 1, Index: 2, Data: []byte("foo")}, 324 } 325 st := raftpb.HardState{Term: 1, Commit: 1} 326 327 want := Ready{ 328 HardState: emptyState, 329 // commit up to commit index in st 330 CommittedEntries: entries[:st.Commit], 331 MustSync: true, 332 } 333 334 storage := NewMemoryStorage() 335 storage.SetHardState(st) 336 storage.Append(entries) 337 rawNode, err := NewRawNode(newTestConfig(1, nil, 10, 1, storage), nil) 338 if err != nil { 339 t.Fatal(err) 340 } 341 rd := rawNode.Ready() 342 if !reflect.DeepEqual(rd, want) { 343 t.Errorf("g = %+v,\n w %+v", rd, want) 344 } 345 rawNode.Advance(rd) 346 if rawNode.HasReady() { 347 t.Errorf("unexpected Ready: %+v", rawNode.Ready()) 348 } 349 } 350 351 func TestRawNodeRestartFromSnapshot(t *testing.T) { 352 snap := raftpb.Snapshot{ 353 Metadata: raftpb.SnapshotMetadata{ 354 ConfState: raftpb.ConfState{Nodes: []uint64{1, 2}}, 355 Index: 2, 356 Term: 1, 357 }, 358 } 359 entries := []raftpb.Entry{ 360 {Term: 1, Index: 3, Data: []byte("foo")}, 361 } 362 st := raftpb.HardState{Term: 1, Commit: 3} 363 364 want := Ready{ 365 HardState: emptyState, 366 // commit up to commit index in st 367 CommittedEntries: entries, 368 MustSync: true, 369 } 370 371 s := NewMemoryStorage() 372 s.SetHardState(st) 373 s.ApplySnapshot(snap) 374 s.Append(entries) 375 rawNode, err := NewRawNode(newTestConfig(1, nil, 10, 1, s), nil) 376 if err != nil { 377 t.Fatal(err) 378 } 379 if rd := rawNode.Ready(); !reflect.DeepEqual(rd, want) { 380 t.Errorf("g = %+v,\n w %+v", rd, want) 381 } else { 382 rawNode.Advance(rd) 383 } 384 if rawNode.HasReady() { 385 t.Errorf("unexpected Ready: %+v", rawNode.HasReady()) 386 } 387 } 388 389 // TestNodeAdvance from node_test.go has no equivalent in rawNode because there is 390 // no dependency check between Ready() and Advance() 391 392 func TestRawNodeStatus(t *testing.T) { 393 storage := NewMemoryStorage() 394 rawNode, err := NewRawNode(newTestConfig(1, nil, 10, 1, storage), []Peer{{ID: 1}}) 395 if err != nil { 396 t.Fatal(err) 397 } 398 status := rawNode.Status() 399 if status == nil { 400 t.Errorf("expected status struct, got nil") 401 } 402 }