go.etcd.io/etcd@v3.3.27+incompatible/raft/rafttest/network.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 "math/rand" 19 "sync" 20 "time" 21 22 "github.com/coreos/etcd/raft/raftpb" 23 ) 24 25 // a network interface 26 type iface interface { 27 send(m raftpb.Message) 28 recv() chan raftpb.Message 29 disconnect() 30 connect() 31 } 32 33 // a network 34 type network interface { 35 // drop message at given rate (1.0 drops all messages) 36 drop(from, to uint64, rate float64) 37 // delay message for (0, d] randomly at given rate (1.0 delay all messages) 38 // do we need rate here? 39 delay(from, to uint64, d time.Duration, rate float64) 40 disconnect(id uint64) 41 connect(id uint64) 42 // heal heals the network 43 heal() 44 } 45 46 type raftNetwork struct { 47 mu sync.Mutex 48 disconnected map[uint64]bool 49 dropmap map[conn]float64 50 delaymap map[conn]delay 51 recvQueues map[uint64]chan raftpb.Message 52 } 53 54 type conn struct { 55 from, to uint64 56 } 57 58 type delay struct { 59 d time.Duration 60 rate float64 61 } 62 63 func newRaftNetwork(nodes ...uint64) *raftNetwork { 64 pn := &raftNetwork{ 65 recvQueues: make(map[uint64]chan raftpb.Message), 66 dropmap: make(map[conn]float64), 67 delaymap: make(map[conn]delay), 68 disconnected: make(map[uint64]bool), 69 } 70 71 for _, n := range nodes { 72 pn.recvQueues[n] = make(chan raftpb.Message, 1024) 73 } 74 return pn 75 } 76 77 func (rn *raftNetwork) nodeNetwork(id uint64) iface { 78 return &nodeNetwork{id: id, raftNetwork: rn} 79 } 80 81 func (rn *raftNetwork) send(m raftpb.Message) { 82 rn.mu.Lock() 83 to := rn.recvQueues[m.To] 84 if rn.disconnected[m.To] { 85 to = nil 86 } 87 drop := rn.dropmap[conn{m.From, m.To}] 88 dl := rn.delaymap[conn{m.From, m.To}] 89 rn.mu.Unlock() 90 91 if to == nil { 92 return 93 } 94 if drop != 0 && rand.Float64() < drop { 95 return 96 } 97 // TODO: shall we dl without blocking the send call? 98 if dl.d != 0 && rand.Float64() < dl.rate { 99 rd := rand.Int63n(int64(dl.d)) 100 time.Sleep(time.Duration(rd)) 101 } 102 103 // use marshal/unmarshal to copy message to avoid data race. 104 b, err := m.Marshal() 105 if err != nil { 106 panic(err) 107 } 108 109 var cm raftpb.Message 110 err = cm.Unmarshal(b) 111 if err != nil { 112 panic(err) 113 } 114 115 select { 116 case to <- cm: 117 default: 118 // drop messages when the receiver queue is full. 119 } 120 } 121 122 func (rn *raftNetwork) recvFrom(from uint64) chan raftpb.Message { 123 rn.mu.Lock() 124 fromc := rn.recvQueues[from] 125 if rn.disconnected[from] { 126 fromc = nil 127 } 128 rn.mu.Unlock() 129 130 return fromc 131 } 132 133 func (rn *raftNetwork) drop(from, to uint64, rate float64) { 134 rn.mu.Lock() 135 defer rn.mu.Unlock() 136 rn.dropmap[conn{from, to}] = rate 137 } 138 139 func (rn *raftNetwork) delay(from, to uint64, d time.Duration, rate float64) { 140 rn.mu.Lock() 141 defer rn.mu.Unlock() 142 rn.delaymap[conn{from, to}] = delay{d, rate} 143 } 144 145 func (rn *raftNetwork) heal() { 146 rn.mu.Lock() 147 defer rn.mu.Unlock() 148 rn.dropmap = make(map[conn]float64) 149 rn.delaymap = make(map[conn]delay) 150 } 151 152 func (rn *raftNetwork) disconnect(id uint64) { 153 rn.mu.Lock() 154 defer rn.mu.Unlock() 155 rn.disconnected[id] = true 156 } 157 158 func (rn *raftNetwork) connect(id uint64) { 159 rn.mu.Lock() 160 defer rn.mu.Unlock() 161 rn.disconnected[id] = false 162 } 163 164 type nodeNetwork struct { 165 id uint64 166 *raftNetwork 167 } 168 169 func (nt *nodeNetwork) connect() { 170 nt.raftNetwork.connect(nt.id) 171 } 172 173 func (nt *nodeNetwork) disconnect() { 174 nt.raftNetwork.disconnect(nt.id) 175 } 176 177 func (nt *nodeNetwork) send(m raftpb.Message) { 178 nt.raftNetwork.send(m) 179 } 180 181 func (nt *nodeNetwork) recv() chan raftpb.Message { 182 return nt.recvFrom(nt.id) 183 }