Skip to content

Commit 8a40a51

Browse files
committedOct 5, 2016
Pull algorithm module for Gossip component
The gossip protocol disseminates messages via push and pull. This commit contains the module that runs the pull protocol which is a separate, timer-based protocol. The patch also contains some gofmt changes Change-Id: I39797631756d76ccd90951af6ac812c1e956fa6b Signed-off-by: Yacov Manevich <yacovm@il.ibm.com>
1 parent eefbf7c commit 8a40a51

File tree

5 files changed

+826
-14
lines changed

5 files changed

+826
-14
lines changed
 

‎gossip/discovery/discovery_impl.go

+4-6
Original file line numberDiff line numberDiff line change
@@ -113,7 +113,6 @@ func NewDiscoveryService(bootstrapPeers []*NetworkMember, self NetworkMember, co
113113
go d.periodicalReconnectToDead()
114114
go d.handlePresumedDeadPeers()
115115

116-
117116
return d
118117
}
119118

@@ -132,18 +131,17 @@ func (d *gossipDiscoveryImpl) InitiateSync(peerNum int) {
132131
k = n
133132
}
134133

135-
for _, i := range util.GetRandomIndices(k, n - 1) {
134+
for _, i := range util.GetRandomIndices(k, n-1) {
136135
pulledPeer := d.cachedMembership.Alive[i].Membership
137136
netMember := &NetworkMember{
138-
Id: pulledPeer.Id,
137+
Id: pulledPeer.Id,
139138
Endpoint: pulledPeer.Endpoint,
140139
Metadata: pulledPeer.Metadata,
141140
}
142141
d.comm.SendToPeer(netMember, memReq)
143142
}
144143
}
145144

146-
147145
func (d *gossipDiscoveryImpl) handlePresumedDeadPeers() {
148146
d.stopSignal.Add(1)
149147
defer d.stopSignal.Done()
@@ -231,7 +229,7 @@ func (d *gossipDiscoveryImpl) createMembershipResponse(known []string) *proto.Me
231229
defer d.lock.RUnlock()
232230

233231
alivePeers := make([]*proto.AliveMessage, 0)
234-
deadPeers := make([]*proto.AliveMessage, 0)
232+
deadPeers := make([]*proto.AliveMessage, 0)
235233

236234
for _, am := range d.cachedMembership.Alive {
237235
isKnown := false
@@ -261,7 +259,7 @@ func (d *gossipDiscoveryImpl) createMembershipResponse(known []string) *proto.Me
261259
}
262260
}
263261

264-
return &proto.MembershipResponse {
262+
return &proto.MembershipResponse{
265263
Alive: append(alivePeers, aliveMsg),
266264
Dead: deadPeers,
267265
}

‎gossip/discovery/discovery_test.go

+2-2
Original file line numberDiff line numberDiff line change
@@ -92,7 +92,7 @@ func (comm *dummyCommModule) Ping(peer *NetworkMember) bool {
9292

9393
_, alreadyExists := comm.streams[peer.Id]
9494
if !alreadyExists {
95-
newConn, err := grpc.Dial(peer.Endpoint, grpc.WithInsecure(), grpc.WithTimeout(time.Duration(500) * time.Millisecond))
95+
newConn, err := grpc.Dial(peer.Endpoint, grpc.WithInsecure(), grpc.WithTimeout(time.Duration(500)*time.Millisecond))
9696
if err != nil {
9797
//fmt.Printf("Error dialing: to %v: %v\n",peer.Endpoint, err)
9898
return false
@@ -133,7 +133,7 @@ func (comm *dummyCommModule) CloseConn(id string) {
133133
comm.streams[id].CloseSend()
134134
comm.conns[id].Close()
135135

136-
delete(comm.streams,id)
136+
delete(comm.streams, id)
137137
delete(comm.conns, id)
138138
}
139139

‎gossip/gossip/algo/pull.go

+288
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,288 @@
1+
/*
2+
Copyright IBM Corp. 2016 All Rights Reserved.
3+
4+
Licensed under the Apache License, Version 2.0 (the "License");
5+
you may not use this file except in compliance with the License.
6+
You may obtain a copy of the License at
7+
8+
http://www.apache.org/licenses/LICENSE-2.0
9+
10+
Unless required by applicable law or agreed to in writing, software
11+
distributed under the License is distributed on an "AS IS" BASIS,
12+
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13+
See the License for the specific language governing permissions and
14+
limitations under the License.
15+
*/
16+
17+
package algo
18+
19+
import (
20+
"math/rand"
21+
"sync"
22+
"sync/atomic"
23+
"time"
24+
25+
"github.com/hyperledger/fabric/gossip/util"
26+
)
27+
28+
/* PullEngine is an object that performs pull-based gossip, and maintains an internal state of items
29+
identified by uint64 numbers.
30+
The protocol is as follows:
31+
1) The Initiator sends a Hello message with a specific NONCE to a set of remote peers.
32+
2) Each remote peer responds with a digest of its messages and returns that NONCE.
33+
3) The initiator checks the validity of the NONCEs received, aggregates the digests,
34+
and crafts a request containing specific item ids it wants to receive from each remote peer and then
35+
sends each request to its corresponding peer.
36+
4) Each peer sends back the response containing the items requested, if it still holds them and the NONCE.
37+
38+
Other peer Initiator
39+
O <-------- Hello <NONCE> ------------------------- O
40+
/|\ --------- Digest <[3,5,8, 10...], NONCE> --------> /|\
41+
| <-------- Request <[3,8], NONCE> ----------------- |
42+
/ \ --------- Response <[item3, item8], NONCE>-------> / \
43+
44+
*/
45+
46+
const (
47+
DEF_DIGEST_WAIT_TIME = time.Duration(4) * time.Second
48+
DEF_REQUEST_WAIT_TIME = time.Duration(4) * time.Second
49+
DEF_RESPONSE_WAIT_TIME = time.Duration(7) * time.Second
50+
)
51+
52+
func init() {
53+
rand.Seed(42)
54+
}
55+
56+
var defaultDigestWaitTime = DEF_DIGEST_WAIT_TIME
57+
var defaultRequestWaitTime = DEF_REQUEST_WAIT_TIME
58+
var defaultResponseWaitTime = DEF_RESPONSE_WAIT_TIME
59+
60+
// PullAdapter is needed by the PullEngine in order to
61+
// send messages to the remote PullEngine instances.
62+
// The PullEngine expects to be invoked with
63+
// OnHello, OnDigest, OnReq, OnRes when the respective message arrives
64+
// from a remote PullEngine
65+
type PullAdapter interface {
66+
// SelectPeers returns a slice of peers which the engine will initiate the protocol with
67+
SelectPeers() []string
68+
69+
// Hello sends a hello message to initiate the protocol
70+
// and returns an NONCE that is expected to be returned
71+
// in the digest message.
72+
Hello(dest string, nonce uint64)
73+
74+
// SendDigest sends a digest to a remote PullEngine.
75+
// The context parameter specifies the remote engine to send to.
76+
SendDigest(digest []uint64, nonce uint64, context interface{})
77+
78+
// SendReq sends an array of items to a certain remote PullEngine identified
79+
// by a string
80+
SendReq(dest string, items []uint64, nonce uint64)
81+
82+
// SendRes sends an array of items to a remote PullEngine identified by a context.
83+
SendRes(items []uint64, context interface{}, nonce uint64)
84+
}
85+
86+
type PullEngine struct {
87+
PullAdapter
88+
stopFlag int32
89+
state *util.Set
90+
item2owners map[uint64][]string
91+
peers2nonces map[string]uint64
92+
nonces2peers map[uint64]string
93+
acceptingDigests int32
94+
acceptingResponses int32
95+
lock sync.Mutex
96+
nonces *util.Set
97+
}
98+
99+
func NewPullEngine(participant PullAdapter, sleepTime time.Duration) *PullEngine {
100+
engine := &PullEngine{
101+
PullAdapter: participant,
102+
stopFlag: int32(0),
103+
state: util.NewSet(),
104+
item2owners: make(map[uint64][]string),
105+
peers2nonces: make(map[string]uint64),
106+
nonces2peers: make(map[uint64]string),
107+
acceptingDigests: int32(0),
108+
acceptingResponses: int32(0),
109+
nonces: util.NewSet(),
110+
}
111+
112+
go func() {
113+
for !engine.toDie() {
114+
time.Sleep(sleepTime)
115+
engine.initiatePull()
116+
117+
}
118+
}()
119+
120+
return engine
121+
}
122+
123+
func (engine *PullEngine) toDie() bool {
124+
return (atomic.LoadInt32(&(engine.stopFlag)) == int32(1))
125+
}
126+
127+
func (engine *PullEngine) acceptResponses() {
128+
atomic.StoreInt32(&(engine.acceptingResponses), int32(1))
129+
}
130+
131+
func (engine *PullEngine) isAcceptingResponses() bool {
132+
return atomic.LoadInt32(&(engine.acceptingResponses)) == int32(1)
133+
}
134+
135+
func (engine *PullEngine) acceptDigests() {
136+
atomic.StoreInt32(&(engine.acceptingDigests), int32(1))
137+
}
138+
139+
func (engine *PullEngine) isAcceptingDigests() bool {
140+
return atomic.LoadInt32(&(engine.acceptingDigests)) == int32(1)
141+
}
142+
143+
func (engine *PullEngine) ignoreDigests() {
144+
atomic.StoreInt32(&(engine.acceptingDigests), int32(0))
145+
}
146+
147+
func (engine *PullEngine) Stop() {
148+
atomic.StoreInt32(&(engine.stopFlag), int32(1))
149+
}
150+
151+
func (engine *PullEngine) initiatePull() {
152+
engine.lock.Lock()
153+
defer engine.lock.Unlock()
154+
155+
engine.acceptDigests()
156+
for _, peer := range engine.SelectPeers() {
157+
nonce := engine.newNONCE()
158+
engine.nonces.Add(nonce)
159+
engine.nonces2peers[nonce] = peer
160+
engine.peers2nonces[peer] = nonce
161+
engine.Hello(peer, nonce)
162+
}
163+
164+
time.AfterFunc(defaultDigestWaitTime, func() {
165+
engine.processIncomingDigests()
166+
})
167+
}
168+
169+
func (engine *PullEngine) processIncomingDigests() {
170+
engine.ignoreDigests()
171+
172+
engine.lock.Lock()
173+
defer engine.lock.Unlock()
174+
175+
requestMapping := make(map[string][]uint64)
176+
for n, sources := range engine.item2owners {
177+
// select a random source
178+
source := sources[rand.Intn(len(sources))]
179+
if _, exists := requestMapping[source]; !exists {
180+
requestMapping[source] = make([]uint64, 0)
181+
}
182+
// append the number to that source
183+
requestMapping[source] = append(requestMapping[source], n)
184+
}
185+
186+
engine.acceptResponses()
187+
188+
for dest, seqsToReq := range requestMapping {
189+
engine.SendReq(dest, seqsToReq, engine.peers2nonces[dest])
190+
}
191+
192+
time.AfterFunc(defaultResponseWaitTime, engine.endPull)
193+
194+
}
195+
196+
func (engine *PullEngine) endPull() {
197+
engine.lock.Lock()
198+
defer engine.lock.Unlock()
199+
200+
atomic.StoreInt32(&(engine.acceptingResponses), int32(0))
201+
engine.nonces.Clear()
202+
203+
engine.item2owners = make(map[uint64][]string)
204+
engine.peers2nonces = make(map[string]uint64)
205+
engine.nonces2peers = make(map[uint64]string)
206+
}
207+
208+
func (engine *PullEngine) OnDigest(digest []uint64, nonce uint64, context interface{}) {
209+
if !engine.isAcceptingDigests() || !engine.nonces.Exists(nonce) {
210+
return
211+
}
212+
213+
engine.lock.Lock()
214+
defer engine.lock.Unlock()
215+
216+
for _, n := range digest {
217+
if engine.state.Exists(n) {
218+
continue
219+
}
220+
221+
if _, exists := engine.item2owners[n]; !exists {
222+
engine.item2owners[n] = make([]string, 0)
223+
}
224+
225+
engine.item2owners[n] = append(engine.item2owners[n], engine.nonces2peers[nonce])
226+
}
227+
}
228+
229+
func (engine *PullEngine) Add(seqs ...uint64) {
230+
for _, seq := range seqs {
231+
engine.state.Add(seq)
232+
}
233+
}
234+
235+
func (engine *PullEngine) Remove(seqs ...uint64) {
236+
for _, seq := range seqs {
237+
engine.state.Remove(seq)
238+
}
239+
}
240+
241+
func (engine *PullEngine) OnHello(nonce uint64, context interface{}) {
242+
engine.nonces.Add(nonce)
243+
time.AfterFunc(defaultRequestWaitTime, func() {
244+
engine.nonces.Remove(nonce)
245+
})
246+
247+
a := engine.state.ToArray()
248+
digest := make([]uint64, len(a))
249+
for i, item := range a {
250+
digest[i] = item.(uint64)
251+
}
252+
engine.SendDigest(digest, nonce, context)
253+
}
254+
255+
func (engine *PullEngine) OnReq(items []uint64, nonce uint64, context interface{}) {
256+
if !engine.nonces.Exists(nonce) {
257+
return
258+
}
259+
engine.lock.Lock()
260+
defer engine.lock.Unlock()
261+
262+
items2Send := make([]uint64, 0)
263+
for _, item := range items {
264+
if engine.state.Exists(item) {
265+
items2Send = append(items2Send, item)
266+
}
267+
}
268+
269+
engine.SendRes(items2Send, context, nonce)
270+
}
271+
272+
func (engine *PullEngine) OnRes(items []uint64, nonce uint64) {
273+
if !engine.nonces.Exists(nonce) || !engine.isAcceptingResponses() {
274+
return
275+
}
276+
277+
engine.Add(items...)
278+
}
279+
280+
func (engine *PullEngine) newNONCE() uint64 {
281+
n := uint64(0)
282+
for {
283+
n = uint64(rand.Int63())
284+
if !engine.nonces.Exists(n) {
285+
return n
286+
}
287+
}
288+
}

‎gossip/gossip/algo/pull_test.go

+472
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,472 @@
1+
/*
2+
Copyright IBM Corp. 2016 All Rights Reserved.
3+
4+
Licensed under the Apache License, Version 2.0 (the "License");
5+
you may not use this file except in compliance with the License.
6+
You may obtain a copy of the License at
7+
8+
http://www.apache.org/licenses/LICENSE-2.0
9+
10+
Unless required by applicable law or agreed to in writing, software
11+
distributed under the License is distributed on an "AS IS" BASIS,
12+
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13+
See the License for the specific language governing permissions and
14+
limitations under the License.
15+
*/
16+
17+
package algo
18+
19+
import (
20+
"sync"
21+
"testing"
22+
"time"
23+
24+
"github.com/hyperledger/fabric/gossip/util"
25+
"github.com/stretchr/testify/assert"
26+
"sync/atomic"
27+
)
28+
29+
func init() {
30+
defaultRequestWaitTime = time.Duration(50) * time.Millisecond
31+
defaultDigestWaitTime = time.Duration(20) * time.Millisecond
32+
defaultResponseWaitTime = time.Duration(50) * time.Millisecond
33+
34+
}
35+
36+
type messageHook func(interface{})
37+
38+
type pullTestInstance struct {
39+
msgHooks []messageHook
40+
peers map[string]*pullTestInstance
41+
name string
42+
nextPeerSelection []string
43+
msgQueue chan interface{}
44+
lock sync.Mutex
45+
stopChan chan struct{}
46+
*PullEngine
47+
}
48+
49+
type helloMsg struct {
50+
nonce uint64
51+
source string
52+
}
53+
54+
type digestMsg struct {
55+
nonce uint64
56+
digest []uint64
57+
source string
58+
}
59+
60+
type reqMsg struct {
61+
items []uint64
62+
nonce uint64
63+
source string
64+
}
65+
66+
type resMsg struct {
67+
items []uint64
68+
nonce uint64
69+
}
70+
71+
func newPushPullTestInstance(name string, peers map[string]*pullTestInstance) *pullTestInstance {
72+
inst := &pullTestInstance{
73+
msgHooks: make([]messageHook, 0),
74+
peers: peers,
75+
msgQueue: make(chan interface{}, 100),
76+
nextPeerSelection: make([]string, 0),
77+
stopChan: make(chan struct{}, 1),
78+
name: name,
79+
}
80+
81+
inst.PullEngine = NewPullEngine(inst, time.Duration(500)*time.Millisecond)
82+
83+
peers[name] = inst
84+
go func() {
85+
for {
86+
select {
87+
case <-inst.stopChan:
88+
return
89+
break
90+
case m := <-inst.msgQueue:
91+
inst.handleMessage(m)
92+
break
93+
}
94+
}
95+
}()
96+
97+
return inst
98+
}
99+
100+
// Used to test the messages one peer sends to another.
101+
// Assert statements should be passed via the messageHook f
102+
func (p *pullTestInstance) hook(f messageHook) {
103+
p.lock.Lock()
104+
defer p.lock.Unlock()
105+
p.msgHooks = append(p.msgHooks, f)
106+
}
107+
108+
func (p *pullTestInstance) handleMessage(m interface{}) {
109+
p.lock.Lock()
110+
for _, f := range p.msgHooks {
111+
f(m)
112+
}
113+
p.lock.Unlock()
114+
115+
if helloMsg, isHello := m.(*helloMsg); isHello {
116+
p.OnHello(helloMsg.nonce, helloMsg.source)
117+
return
118+
}
119+
120+
if digestMsg, isDigest := m.(*digestMsg); isDigest {
121+
p.OnDigest(digestMsg.digest, digestMsg.nonce, digestMsg.source)
122+
return
123+
}
124+
125+
if reqMsg, isReq := m.(*reqMsg); isReq {
126+
p.OnReq(reqMsg.items, reqMsg.nonce, reqMsg.source)
127+
return
128+
}
129+
130+
if resMsg, isRes := m.(*resMsg); isRes {
131+
p.OnRes(resMsg.items, resMsg.nonce)
132+
}
133+
}
134+
135+
func (p *pullTestInstance) stop() {
136+
p.stopChan <- struct{}{}
137+
p.Stop()
138+
}
139+
140+
func (p *pullTestInstance) setNextPeerSelection(selection []string) {
141+
p.lock.Lock()
142+
defer p.lock.Unlock()
143+
p.nextPeerSelection = selection
144+
}
145+
146+
func (p *pullTestInstance) SelectPeers() []string {
147+
p.lock.Lock()
148+
defer p.lock.Unlock()
149+
return p.nextPeerSelection
150+
}
151+
152+
func (p *pullTestInstance) Hello(dest string, nonce uint64) {
153+
p.peers[dest].msgQueue <- &helloMsg{nonce: nonce, source: p.name}
154+
}
155+
156+
func (p *pullTestInstance) SendDigest(digest []uint64, nonce uint64, context interface{}) {
157+
p.peers[context.(string)].msgQueue <- &digestMsg{source: p.name, nonce: nonce, digest: digest}
158+
}
159+
160+
func (p *pullTestInstance) SendReq(dest string, items []uint64, nonce uint64) {
161+
p.peers[dest].msgQueue <- &reqMsg{nonce: nonce, source: p.name, items: items}
162+
}
163+
164+
func (p *pullTestInstance) SendRes(items []uint64, context interface{}, nonce uint64) {
165+
p.peers[context.(string)].msgQueue <- &resMsg{items: items, nonce: nonce}
166+
}
167+
168+
func TestPullEngine_Add(t *testing.T) {
169+
peers := make(map[string]*pullTestInstance)
170+
inst1 := newPushPullTestInstance("p1", peers)
171+
defer inst1.Stop()
172+
inst1.Add(uint64(0))
173+
inst1.Add(uint64(0))
174+
assert.True(t, inst1.PullEngine.state.Exists(uint64(0)))
175+
}
176+
177+
func TestPullEngine_Remove(t *testing.T) {
178+
peers := make(map[string]*pullTestInstance)
179+
inst1 := newPushPullTestInstance("p1", peers)
180+
defer inst1.Stop()
181+
inst1.Add(uint64(0))
182+
assert.True(t, inst1.PullEngine.state.Exists(uint64(0)))
183+
inst1.Remove(uint64(0))
184+
assert.False(t, inst1.PullEngine.state.Exists(uint64(0)))
185+
inst1.Remove(uint64(0)) // remove twice
186+
assert.False(t, inst1.PullEngine.state.Exists(uint64(0)))
187+
}
188+
189+
func TestPullEngine_Stop(t *testing.T) {
190+
peers := make(map[string]*pullTestInstance)
191+
inst1 := newPushPullTestInstance("p1", peers)
192+
inst2 := newPushPullTestInstance("p2", peers)
193+
defer inst2.stop()
194+
inst2.setNextPeerSelection([]string{"p1"})
195+
go func() {
196+
for i := 0; i < 100; i++ {
197+
inst1.Add(uint64(i))
198+
time.Sleep(time.Duration(10) * time.Millisecond)
199+
}
200+
}()
201+
202+
time.Sleep(time.Duration(800) * time.Millisecond)
203+
len1 := len(inst2.state.ToArray())
204+
inst1.stop()
205+
time.Sleep(time.Duration(800) * time.Millisecond)
206+
len2 := len(inst2.state.ToArray())
207+
assert.Equal(t, len1, len2, "PullEngine was still active after Stop() was invoked!")
208+
}
209+
210+
func TestPullEngineSelectiveUpdates(t *testing.T) {
211+
// Scenario: inst1 has {1, 3} and inst2 has {0,1,2,3}.
212+
// inst1 initiates to inst2
213+
// Expected outcome: inst1 asks for 0,2 and inst2 sends 0,2 only
214+
peers := make(map[string]*pullTestInstance)
215+
inst1 := newPushPullTestInstance("p1", peers)
216+
inst2 := newPushPullTestInstance("p2", peers)
217+
defer inst1.stop()
218+
defer inst2.stop()
219+
220+
inst1.Add(uint64(1), uint64(3))
221+
inst2.Add(uint64(0), uint64(1), uint64(2), uint64(3))
222+
223+
// Ensure inst2 sent a proper digest to inst1
224+
inst1.hook(func(m interface{}) {
225+
if dig, isDig := m.(*digestMsg); isDig {
226+
assert.True(t, util.IndexInSlice(dig.digest, uint64(0), numericCompare) != -1)
227+
assert.True(t, util.IndexInSlice(dig.digest, uint64(1), numericCompare) != -1)
228+
assert.True(t, util.IndexInSlice(dig.digest, uint64(2), numericCompare) != -1)
229+
assert.True(t, util.IndexInSlice(dig.digest, uint64(3), numericCompare) != -1)
230+
}
231+
})
232+
233+
// Ensure inst1 requested only needed updates from inst2
234+
inst2.hook(func(m interface{}) {
235+
if req, isReq := m.(*reqMsg); isReq {
236+
assert.True(t, util.IndexInSlice(req.items, uint64(1), numericCompare) == -1)
237+
assert.True(t, util.IndexInSlice(req.items, uint64(3), numericCompare) == -1)
238+
239+
assert.True(t, util.IndexInSlice(req.items, uint64(0), numericCompare) != -1)
240+
assert.True(t, util.IndexInSlice(req.items, uint64(2), numericCompare) != -1)
241+
}
242+
})
243+
244+
// Ensure inst1 received only needed updates from inst2
245+
inst1.hook(func(m interface{}) {
246+
if res, isRes := m.(*resMsg); isRes {
247+
assert.True(t, util.IndexInSlice(res.items, uint64(1), numericCompare) == -1)
248+
assert.True(t, util.IndexInSlice(res.items, uint64(3), numericCompare) == -1)
249+
250+
assert.True(t, util.IndexInSlice(res.items, uint64(0), numericCompare) != -1)
251+
assert.True(t, util.IndexInSlice(res.items, uint64(2), numericCompare) != -1)
252+
}
253+
})
254+
255+
inst1.setNextPeerSelection([]string{"p2"})
256+
257+
time.Sleep(time.Duration(800) * time.Millisecond)
258+
assert.Equal(t, len(inst2.state.ToArray()), len(inst1.state.ToArray()))
259+
}
260+
261+
func TestByzantineResponder(t *testing.T) {
262+
// Scenario: inst1 sends hello to inst2 but inst3 is byzantine so it attempts to send a digest and a response to inst1.
263+
// expected outcome is for inst1 not to process updates from inst3.
264+
peers := make(map[string]*pullTestInstance)
265+
inst1 := newPushPullTestInstance("p1", peers)
266+
inst2 := newPushPullTestInstance("p2", peers)
267+
inst3 := newPushPullTestInstance("p3", peers)
268+
defer inst1.stop()
269+
defer inst2.stop()
270+
defer inst3.stop()
271+
272+
receivedDigestFromInst3 := int32(0)
273+
274+
inst2.Add(uint64(1), uint64(2), uint64(3))
275+
inst3.Add(uint64(5), uint64(6), uint64(7))
276+
277+
inst2.hook(func(m interface{}) {
278+
if _, isHello := m.(*helloMsg); isHello {
279+
inst3.SendDigest([]uint64{uint64(5), uint64(6), uint64(7)}, 0, "p1")
280+
}
281+
})
282+
283+
inst1.hook(func(m interface{}) {
284+
if dig, isDig := m.(*digestMsg); isDig {
285+
if dig.source == "p3" {
286+
atomic.StoreInt32(&receivedDigestFromInst3, int32(1))
287+
time.AfterFunc(time.Duration(25)*time.Millisecond, func() {
288+
inst3.SendRes([]uint64{uint64(5), uint64(6), uint64(7)}, "p1", 0)
289+
})
290+
}
291+
}
292+
293+
if res, isRes := m.(*resMsg); isRes {
294+
// the response is from p3
295+
if util.IndexInSlice(res.items, uint64(6), numericCompare) != -1 {
296+
// inst1 is currently accepting responses
297+
assert.Equal(t, int32(1), atomic.LoadInt32(&(inst1.acceptingResponses)))
298+
}
299+
}
300+
})
301+
302+
inst1.setNextPeerSelection([]string{"p2"})
303+
304+
time.Sleep(time.Duration(800) * time.Millisecond)
305+
306+
assert.Equal(t, int32(1), atomic.LoadInt32(&receivedDigestFromInst3), "inst1 hasn't received a digest from inst3")
307+
308+
assert.True(t, util.IndexInSlice(inst1.state.ToArray(), uint64(1), numericCompare) != -1)
309+
assert.True(t, util.IndexInSlice(inst1.state.ToArray(), uint64(2), numericCompare) != -1)
310+
assert.True(t, util.IndexInSlice(inst1.state.ToArray(), uint64(3), numericCompare) != -1)
311+
312+
assert.True(t, util.IndexInSlice(inst1.state.ToArray(), uint64(5), numericCompare) == -1)
313+
assert.True(t, util.IndexInSlice(inst1.state.ToArray(), uint64(6), numericCompare) == -1)
314+
assert.True(t, util.IndexInSlice(inst1.state.ToArray(), uint64(7), numericCompare) == -1)
315+
316+
}
317+
318+
func TestMultipleInitiators(t *testing.T) {
319+
// Scenario: inst1, inst2 and inst3 both start protocol with inst4 at the same time.
320+
// Expected outcome: inst4 successfully transfers state to all of them
321+
peers := make(map[string]*pullTestInstance)
322+
inst1 := newPushPullTestInstance("p1", peers)
323+
inst2 := newPushPullTestInstance("p2", peers)
324+
inst3 := newPushPullTestInstance("p3", peers)
325+
inst4 := newPushPullTestInstance("p4", peers)
326+
defer inst1.stop()
327+
defer inst2.stop()
328+
defer inst3.stop()
329+
defer inst4.stop()
330+
331+
inst4.Add(uint64(1), uint64(2), uint64(3), uint64(4))
332+
inst1.setNextPeerSelection([]string{"p4"})
333+
inst2.setNextPeerSelection([]string{"p4"})
334+
inst3.setNextPeerSelection([]string{"p4"})
335+
336+
time.Sleep(time.Duration(800) * time.Millisecond)
337+
338+
for _, inst := range []*pullTestInstance{inst1, inst2, inst3} {
339+
assert.True(t, util.IndexInSlice(inst.state.ToArray(), uint64(1), numericCompare) != -1)
340+
assert.True(t, util.IndexInSlice(inst.state.ToArray(), uint64(2), numericCompare) != -1)
341+
assert.True(t, util.IndexInSlice(inst.state.ToArray(), uint64(3), numericCompare) != -1)
342+
assert.True(t, util.IndexInSlice(inst.state.ToArray(), uint64(4), numericCompare) != -1)
343+
}
344+
345+
}
346+
347+
func TestLatePeers(t *testing.T) {
348+
// Scenario: inst1 initiates to inst2 (items: {1,2,3,4}) and inst3 (items: {5,6,7,8}),
349+
// but inst2 is too slow to respond, and all items
350+
// should be received from inst3.
351+
peers := make(map[string]*pullTestInstance)
352+
inst1 := newPushPullTestInstance("p1", peers)
353+
inst2 := newPushPullTestInstance("p2", peers)
354+
inst3 := newPushPullTestInstance("p3", peers)
355+
defer inst1.stop()
356+
defer inst2.stop()
357+
defer inst3.stop()
358+
inst2.Add(uint64(1), uint64(2), uint64(3), uint64(4))
359+
inst3.Add(uint64(5), uint64(6), uint64(7), uint64(8))
360+
inst2.hook(func(m interface{}) {
361+
time.Sleep(time.Duration(60) * time.Millisecond)
362+
})
363+
inst1.setNextPeerSelection([]string{"p2", "p3"})
364+
365+
time.Sleep(time.Duration(800) * time.Millisecond)
366+
367+
assert.True(t, util.IndexInSlice(inst1.state.ToArray(), uint64(1), numericCompare) == -1)
368+
assert.True(t, util.IndexInSlice(inst1.state.ToArray(), uint64(2), numericCompare) == -1)
369+
assert.True(t, util.IndexInSlice(inst1.state.ToArray(), uint64(3), numericCompare) == -1)
370+
assert.True(t, util.IndexInSlice(inst1.state.ToArray(), uint64(4), numericCompare) == -1)
371+
372+
assert.True(t, util.IndexInSlice(inst1.state.ToArray(), uint64(5), numericCompare) != -1)
373+
assert.True(t, util.IndexInSlice(inst1.state.ToArray(), uint64(6), numericCompare) != -1)
374+
assert.True(t, util.IndexInSlice(inst1.state.ToArray(), uint64(7), numericCompare) != -1)
375+
assert.True(t, util.IndexInSlice(inst1.state.ToArray(), uint64(8), numericCompare) != -1)
376+
377+
}
378+
379+
func TestBiDiUpdates(t *testing.T) {
380+
// Scenario: inst1 has {1, 3} and inst2 has {0,2} and both initiate to the other at the same time.
381+
// Expected outcome: both have {0,1,2,3} in the end
382+
peers := make(map[string]*pullTestInstance)
383+
inst1 := newPushPullTestInstance("p1", peers)
384+
inst2 := newPushPullTestInstance("p2", peers)
385+
defer inst1.stop()
386+
defer inst2.stop()
387+
388+
inst1.Add(uint64(1), uint64(3))
389+
inst2.Add(uint64(0), uint64(2))
390+
391+
inst1.setNextPeerSelection([]string{"p2"})
392+
inst2.setNextPeerSelection([]string{"p1"})
393+
394+
time.Sleep(time.Duration(800) * time.Millisecond)
395+
396+
assert.True(t, util.IndexInSlice(inst1.state.ToArray(), uint64(0), numericCompare) != -1)
397+
assert.True(t, util.IndexInSlice(inst1.state.ToArray(), uint64(1), numericCompare) != -1)
398+
assert.True(t, util.IndexInSlice(inst1.state.ToArray(), uint64(2), numericCompare) != -1)
399+
assert.True(t, util.IndexInSlice(inst1.state.ToArray(), uint64(3), numericCompare) != -1)
400+
401+
assert.True(t, util.IndexInSlice(inst2.state.ToArray(), uint64(0), numericCompare) != -1)
402+
assert.True(t, util.IndexInSlice(inst2.state.ToArray(), uint64(1), numericCompare) != -1)
403+
assert.True(t, util.IndexInSlice(inst2.state.ToArray(), uint64(2), numericCompare) != -1)
404+
assert.True(t, util.IndexInSlice(inst2.state.ToArray(), uint64(3), numericCompare) != -1)
405+
406+
}
407+
408+
func TestSpread(t *testing.T) {
409+
// Scenario: inst1 initiates to inst2, inst3 inst4 and each have items 0-100. inst5 also has the same items but isn't selected
410+
// Expected outcome: each responder (inst2, inst3 and inst4) is chosen at least once (the probability for not choosing each of them is slim)
411+
// inst5 isn't selected at all
412+
peers := make(map[string]*pullTestInstance)
413+
inst1 := newPushPullTestInstance("p1", peers)
414+
inst2 := newPushPullTestInstance("p2", peers)
415+
inst3 := newPushPullTestInstance("p3", peers)
416+
inst4 := newPushPullTestInstance("p4", peers)
417+
inst5 := newPushPullTestInstance("p5", peers)
418+
defer inst1.stop()
419+
defer inst2.stop()
420+
defer inst3.stop()
421+
defer inst4.stop()
422+
defer inst5.stop()
423+
424+
chooseCounters := make(map[string]int)
425+
chooseCounters["p2"] = 0
426+
chooseCounters["p3"] = 0
427+
chooseCounters["p4"] = 0
428+
chooseCounters["p5"] = 0
429+
430+
lock := &sync.Mutex{}
431+
432+
addToCounters := func(dest string) func(m interface{}) {
433+
return func(m interface{}) {
434+
if _, isReq := m.(*reqMsg); isReq {
435+
lock.Lock()
436+
chooseCounters[dest]++
437+
lock.Unlock()
438+
}
439+
}
440+
}
441+
442+
inst2.hook(addToCounters("p2"))
443+
inst3.hook(addToCounters("p3"))
444+
inst4.hook(addToCounters("p4"))
445+
inst5.hook(addToCounters("p5"))
446+
447+
for i := 0; i < 100; i++ {
448+
item := uint64(i)
449+
inst2.Add(item)
450+
inst3.Add(item)
451+
inst4.Add(item)
452+
}
453+
454+
inst1.setNextPeerSelection([]string{"p2", "p3", "p4"})
455+
456+
time.Sleep(time.Duration(800) * time.Millisecond)
457+
458+
lock.Lock()
459+
for p_i, counter := range chooseCounters {
460+
if p_i == "p5" {
461+
assert.Equal(t, 0, counter)
462+
} else {
463+
assert.True(t, counter > 0, "%s was not selected!", p_i)
464+
}
465+
}
466+
lock.Unlock()
467+
468+
}
469+
470+
func numericCompare(a interface{}, b interface{}) bool {
471+
return a.(uint64) == b.(uint64)
472+
}

‎gossip/util/misc.go

+60-6
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,7 @@ package util
1919
import (
2020
"math/rand"
2121
"reflect"
22+
"sync"
2223
)
2324

2425
type Equals func(a interface{}, b interface{}) bool
@@ -27,11 +28,6 @@ func init() {
2728
rand.Seed(42)
2829
}
2930

30-
31-
func numbericEqual(a interface{}, b interface{}) bool {
32-
return a.(int) == b.(int)
33-
}
34-
3531
func IndexInSlice(array interface{}, o interface{}, equals Equals) int {
3632
arr := reflect.ValueOf(array)
3733
for i := 0; i < arr.Len(); i++ {
@@ -42,6 +38,10 @@ func IndexInSlice(array interface{}, o interface{}, equals Equals) int {
4238
return -1
4339
}
4440

41+
func numbericEqual(a interface{}, b interface{}) bool {
42+
return a.(int) == b.(int)
43+
}
44+
4545
func GetRandomIndices(indiceCount, highestIndex int) []int {
4646
if highestIndex+1 < indiceCount {
4747
return nil
@@ -63,4 +63,58 @@ func GetRandomIndices(indiceCount, highestIndex int) []int {
6363
indices = append(indices, n)
6464
}
6565
return indices
66-
}
66+
}
67+
68+
func Abs(a, b uint64) uint64 {
69+
if a > b {
70+
return a - b
71+
} else {
72+
return b - a
73+
}
74+
}
75+
76+
type Set struct {
77+
items map[interface{}]struct{}
78+
lock *sync.RWMutex
79+
}
80+
81+
func NewSet() *Set {
82+
return &Set{lock: &sync.RWMutex{}, items: make(map[interface{}]struct{})}
83+
}
84+
85+
func (s *Set) Add(item interface{}) {
86+
s.lock.Lock()
87+
defer s.lock.Unlock()
88+
s.items[item] = struct{}{}
89+
}
90+
91+
func (s *Set) Exists(item interface{}) bool {
92+
s.lock.RLock()
93+
defer s.lock.RUnlock()
94+
_, exists := s.items[item]
95+
return exists
96+
}
97+
98+
func (s *Set) ToArray() []interface{} {
99+
s.lock.RLock()
100+
defer s.lock.RUnlock()
101+
a := make([]interface{}, len(s.items))
102+
i := 0
103+
for item := range s.items {
104+
a[i] = item
105+
i++
106+
}
107+
return a
108+
}
109+
110+
func (s *Set) Clear() {
111+
s.lock.Lock()
112+
defer s.lock.Unlock()
113+
s.items = make(map[interface{}]struct{})
114+
}
115+
116+
func (s *Set) Remove(item interface{}) {
117+
s.lock.Lock()
118+
defer s.lock.Unlock()
119+
delete(s.items, item)
120+
}

0 commit comments

Comments
 (0)
Please sign in to comment.