forked from FCAI-Lab/etcd-io_raft
-
Notifications
You must be signed in to change notification settings - Fork 0
/
Copy pathinteraction_env_handler_deliver_msgs.go
104 lines (96 loc) · 2.72 KB
/
interaction_env_handler_deliver_msgs.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
// Copyright 2019 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 rafttest
import (
"fmt"
"strconv"
"testing"
"github.com/cockroachdb/datadriven"
"go.etcd.io/raft/v3"
"go.etcd.io/raft/v3/raftpb"
)
func (env *InteractionEnv) handleDeliverMsgs(t *testing.T, d datadriven.TestData) error {
var typ raftpb.MessageType = -1
var rs []Recipient
for _, arg := range d.CmdArgs {
if len(arg.Vals) == 0 {
id, err := strconv.ParseUint(arg.Key, 10, 64)
if err != nil {
t.Fatal(err)
}
rs = append(rs, Recipient{ID: id})
}
for i := range arg.Vals {
switch arg.Key {
case "drop":
var id uint64
arg.Scan(t, i, &id)
var found bool
for _, r := range rs {
if r.ID == id {
found = true
}
}
if found {
t.Fatalf("can't both deliver and drop msgs to %d", id)
}
rs = append(rs, Recipient{ID: id, Drop: true})
case "type":
var s string
arg.Scan(t, i, &s)
v, ok := raftpb.MessageType_value[s]
if !ok {
t.Fatalf("unknown message type %s", s)
}
typ = raftpb.MessageType(v)
}
}
}
if n := env.DeliverMsgs(typ, rs...); n == 0 {
env.Output.WriteString("no messages\n")
}
return nil
}
type Recipient struct {
ID uint64
Drop bool
}
// DeliverMsgs goes through env.Messages and, depending on the Drop flag,
// delivers or drops messages to the specified Recipients. Only messages of type
// typ are delivered (-1 for all types). Returns the number of messages handled
// (i.e. delivered or dropped). A handled message is removed from env.Messages.
func (env *InteractionEnv) DeliverMsgs(typ raftpb.MessageType, rs ...Recipient) int {
var n int
for _, r := range rs {
var msgs []raftpb.Message
msgs, env.Messages = splitMsgs(env.Messages, r.ID, typ, r.Drop)
n += len(msgs)
for _, msg := range msgs {
if r.Drop {
fmt.Fprint(env.Output, "dropped: ")
}
fmt.Fprintln(env.Output, raft.DescribeMessage(msg, defaultEntryFormatter))
if r.Drop {
// NB: it's allowed to drop messages to nodes that haven't been instantiated yet,
// we haven't used msg.To yet.
continue
}
toIdx := int(msg.To - 1)
if err := env.Nodes[toIdx].Step(msg); err != nil {
fmt.Fprintln(env.Output, err)
}
}
}
return n
}