swarm: network rewrite merge

This commit is contained in:
ethersphere
2018-06-20 14:06:27 +02:00
parent 574378edb5
commit e187711c65
201 changed files with 39605 additions and 9921 deletions

152
swarm/network/README.md Normal file
View File

@@ -0,0 +1,152 @@
## Streaming
Streaming is a new protocol of the swarm bzz bundle of protocols.
This protocol provides the basic logic for chunk-based data flow.
It implements simple retrieve requests and delivery using priority queue.
A data exchange stream is a directional flow of chunks between peers.
The source of datachunks is the upstream, the receiver is called the
downstream peer. Each streaming protocol defines an outgoing streamer
and an incoming streamer, the former installing on the upstream,
the latter on the downstream peer.
Subscribe on StreamerPeer launches an incoming streamer that sends
a subscribe msg upstream. The streamer on the upstream peer
handles the subscribe msg by installing the relevant outgoing streamer
. The modules now engage in a process of upstream sending a sequence of hashes of
chunks downstream (OfferedHashesMsg). The downstream peer evaluates which hashes are needed
and get it delivered by sending back a msg (WantedHashesMsg).
Historical syncing is supported - currently not the right abstraction --
state kept across sessions by saving a series of intervals after their last
batch actually arrived.
Live streaming is also supported, by starting session from the first item
after the subscription.
Provable data exchange. In case a stream represents a swarm document's data layer
or higher level chunks, streaming up to a certain index is always provable. It saves on
sending intermediate chunks.
Using the streamer logic, various stream types are easy to implement:
* light node requests:
* url lookup with offset
* document download
* document upload
* syncing
* live session syncing
* historical syncing
* simple retrieve requests and deliveries
* mutable resource updates streams
* receipting for finger pointing
## Syncing
Syncing is the process that makes sure storer nodes end up storing all and only the chunks that are requested from them.
### Requirements
- eventual consistency: so each chunk historical should be syncable
- since the same chunk can and will arrive from many peers, (network traffic should be
optimised, only one transfer of data per chunk)
- explicit request deliveries should be prioritised higher than recent chunks received
during the ongoing session which in turn should be higher than historical chunks.
- insured chunks should get receipted for finger pointing litigation, the receipts storage
should be organised efficiently, upstream peer should also be able to find these
receipts for a deleted chunk easily to refute their challenge.
- syncing should be resilient to cut connections, metadata should be persisted that
keep track of syncing state across sessions, historical syncing state should survive restart
- extra data structures to support syncing should be kept at minimum
- syncing is organized separately for chunk types (resource update v content chunk)
- various types of streams should have common logic abstracted
Syncing is now entirely mediated by the localstore, ie., no processes or memory leaks due to network contention.
When a new chunk is stored, its chunk hash is index by proximity bin
peers syncronise by getting the chunks closer to the downstream peer than to the upstream one.
Consequently peers just sync all stored items for the kad bin the receiving peer falls into.
The special case of nearest neighbour sets is handled by the downstream peer
indicating they want to sync all kademlia bins with proximity equal to or higher
than their depth.
This sync state represents the initial state of a sync connection session.
Retrieval is dictated by downstream peers simply using a special streamer protocol.
Syncing chunks created during the session by the upstream peer is called live session syncing
while syncing of earlier chunks is historical syncing.
Once the relevant chunk is retrieved, downstream peer looks up all hash segments in its localstore
and sends to the upstream peer a message with a a bitvector to indicate
missing chunks (e.g., for chunk `k`, hash with chunk internal index which case )
new items. In turn upstream peer sends the relevant chunk data alongside their index.
On sending chunks there is a priority queue system. If during looking up hashes in its localstore,
downstream peer hits on an open request then a retrieve request is sent immediately to the upstream peer indicating
that no extra round of checks is needed. If another peers syncer hits the same open request, it is slightly unsafe to not ask
that peer too: if the first one disconnects before delivering or fails to deliver and therefore gets
disconnected, we should still be able to continue with the other. The minimum redundant traffic coming from such simultaneous
eventualities should be sufficiently rare not to warrant more complex treatment.
Session syncing involves downstream peer to request a new state on a bin from upstream.
using the new state, the range (of chunks) between the previous state and the new one are retrieved
and chunks are requested identical to the historical case. After receiving all the missing chunks
from the new hashes, downstream peer will request a new range. If this happens before upstream peer updates a new state,
we say that session syncing is live or the two peers are in sync. In general the time interval passed since downstream peer request up to the current session cursor is a good indication of a permanent (probably increasing) lag.
If there is no historical backlog, and downstream peer has an acceptable 'last synced' tag, then it is said to be fully synced with the upstream peer.
If a peer is fully synced with all its storer peers, it can advertise itself as globally fully synced.
The downstream peer persists the record of the last synced offset. When the two peers disconnect and
reconnect syncing can start from there.
This situation however can also happen while historical syncing is not yet complete.
Effectively this means that the peer needs to persist a record of an arbitrary array of offset ranges covered.
### Delivery requests
once the appropriate ranges of the hashstream are retrieved and buffered, downstream peer just scans the hashes, looks them up in localstore, if not found, create a request entry.
The range is referenced by the chunk index. Alongside the name (indicating the stream, e.g., content chunks for bin 6) and the range
downstream peer sends a 128 long bitvector indicating which chunks are needed.
Newly created requests are satisfied bound together in a waitgroup which when done, will promptt sending the next one.
to be able to do check and storage concurrently, we keep a buffer of one, we start with two batches of hashes.
If there is nothing to give, upstream peers SetNextBatch is blocking. Subscription ends with an unsubscribe. which removes the syncer from the map.
Canceling requests (for instance the late chunks of an erasure batch) should be a chan closed
on the request
Simple request is also a subscribe
different streaming protocols are different p2p protocols with same message types.
the constructor is the Run function itself. which takes a streamerpeer as argument
### provable streams
The swarm hash over the hash stream has many advantages. It implements a provable data transfer
and provide efficient storage for receipts in the form of inclusion proofs useable for finger pointing litigation.
When challenged on a missing chunk, upstream peer will provide an inclusion proof of a chunk hash against the state of the
sync stream. In order to be able to generate such an inclusion proof, upstream peer needs to store the hash index (counting consecutive hash-size segments) alongside the chunk data and preserve it even when the chunk data is deleted until the chunk is no longer insured.
if there is no valid insurance on the files the entry may be deleted.
As long as the chunk is preserved, no takeover proof will be needed since the node can respond to any challenge.
However, once the node needs to delete an insured chunk for capacity reasons, a receipt should be available to
refute the challenge by finger pointing to a downstream peer.
As part of the deletion protocol then, hashes of insured chunks to be removed are pushed to an infinite stream for every bin.
Downstream peer on the other hand needs to make sure that they can only be finger pointed about a chunk they did receive and store.
For this the check of a state should be exhaustive. If historical syncing finishes on one state, all hashes before are covered, no
surprises. In other words historical syncing this process is self verifying. With session syncing however, it is not enough to check going back covering the range from old offset to new. Continuity (i.e., that the new state is extension of the old) needs to be verified: after downstream peer reads the range into a buffer, it appends the buffer the last known state at the last known offset and verifies the resulting hash matches
the latest state. Past intervals of historical syncing are checked via the the session root.
Upstream peer signs the states, downstream peers can use as handover proofs.
Downstream peers sign off on a state together with an initial offset.
Once historical syncing is complete and the session does not lag, downstream peer only preserves the latest upstream state and store the signed version.
Upstream peer needs to keep the latest takeover states: each deleted chunk's hash should be covered by takeover proof of at least one peer. If historical syncing is complete, upstream peer typically will store only the latest takeover proof from downstream peer.
Crucially, the structure is totally independent of the number of peers in the bin, so it scales extremely well.
## implementation
The simplest protocol just involves upstream peer to prefix the key with the kademlia proximity order (say 0-15 or 0-31)
and simply iterate on index per bin when syncing with a peer.
priority queues are used for sending chunks so that user triggered requests should be responded to first, session syncing second, and historical with lower priority.
The request on chunks remains implemented as a dataless entry in the memory store.
The lifecycle of this object should be more carefully thought through, ie., when it fails to retrieve it should be removed.

View File

@@ -0,0 +1,66 @@
// Copyright 2018 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package bitvector
import (
"errors"
)
var errInvalidLength = errors.New("invalid length")
type BitVector struct {
len int
b []byte
}
func New(l int) (bv *BitVector, err error) {
return NewFromBytes(make([]byte, l/8+1), l)
}
func NewFromBytes(b []byte, l int) (bv *BitVector, err error) {
if l <= 0 {
return nil, errInvalidLength
}
if len(b)*8 < l {
return nil, errInvalidLength
}
return &BitVector{
len: l,
b: b,
}, nil
}
func (bv *BitVector) Get(i int) bool {
bi := i / 8
return bv.b[bi]&(0x1<<uint(i%8)) != 0
}
func (bv *BitVector) Set(i int, v bool) {
bi := i / 8
cv := bv.Get(i)
if cv != v {
bv.b[bi] ^= 0x1 << uint8(i%8)
}
}
func (bv *BitVector) Bytes() []byte {
return bv.b
}
func (bv *BitVector) Length() int {
return bv.len
}

View File

@@ -0,0 +1,104 @@
// Copyright 2018 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package bitvector
import "testing"
func TestBitvectorNew(t *testing.T) {
_, err := New(0)
if err != errInvalidLength {
t.Errorf("expected err %v, got %v", errInvalidLength, err)
}
_, err = NewFromBytes(nil, 0)
if err != errInvalidLength {
t.Errorf("expected err %v, got %v", errInvalidLength, err)
}
_, err = NewFromBytes([]byte{0}, 9)
if err != errInvalidLength {
t.Errorf("expected err %v, got %v", errInvalidLength, err)
}
_, err = NewFromBytes(make([]byte, 8), 8)
if err != nil {
t.Error(err)
}
}
func TestBitvectorGetSet(t *testing.T) {
for _, length := range []int{
1,
2,
4,
8,
9,
15,
16,
} {
bv, err := New(length)
if err != nil {
t.Errorf("error for length %v: %v", length, err)
}
for i := 0; i < length; i++ {
if bv.Get(i) {
t.Errorf("expected false for element on index %v", i)
}
}
func() {
defer func() {
if err := recover(); err == nil {
t.Errorf("expecting panic")
}
}()
bv.Get(length + 8)
}()
for i := 0; i < length; i++ {
bv.Set(i, true)
for j := 0; j < length; j++ {
if j == i {
if !bv.Get(j) {
t.Errorf("element on index %v is not set to true", i)
}
} else {
if bv.Get(j) {
t.Errorf("element on index %v is not false", i)
}
}
}
bv.Set(i, false)
if bv.Get(i) {
t.Errorf("element on index %v is not set to false", i)
}
}
}
}
func TestBitvectorNewFromBytesGet(t *testing.T) {
bv, err := NewFromBytes([]byte{8}, 8)
if err != nil {
t.Error(err)
}
if !bv.Get(3) {
t.Fatalf("element 3 is not set to true: state %08b", bv.b[0])
}
}

30
swarm/network/common.go Normal file
View File

@@ -0,0 +1,30 @@
// Copyright 2018 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package network
import (
"fmt"
"strings"
)
func LogAddrs(nns [][]byte) string {
var nnsa []string
for _, nn := range nns {
nnsa = append(nnsa, fmt.Sprintf("%08x", nn[:4]))
}
return strings.Join(nnsa, ", ")
}

View File

@@ -1,232 +0,0 @@
// Copyright 2016 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package network
import (
"bytes"
"encoding/binary"
"fmt"
"time"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/metrics"
"github.com/ethereum/go-ethereum/swarm/storage"
)
//metrics variables
var (
syncReceiveCount = metrics.NewRegisteredCounter("network.sync.recv.count", nil)
syncReceiveIgnore = metrics.NewRegisteredCounter("network.sync.recv.ignore", nil)
syncSendCount = metrics.NewRegisteredCounter("network.sync.send.count", nil)
syncSendRefused = metrics.NewRegisteredCounter("network.sync.send.refused", nil)
syncSendNotFound = metrics.NewRegisteredCounter("network.sync.send.notfound", nil)
)
// Handler for storage/retrieval related protocol requests
// implements the StorageHandler interface used by the bzz protocol
type Depo struct {
hashfunc storage.SwarmHasher
localStore storage.ChunkStore
netStore storage.ChunkStore
}
func NewDepo(hash storage.SwarmHasher, localStore, remoteStore storage.ChunkStore) *Depo {
return &Depo{
hashfunc: hash,
localStore: localStore,
netStore: remoteStore, // entrypoint internal
}
}
// Handles UnsyncedKeysMsg after msg decoding - unsynced hashes upto sync state
// * the remote sync state is just stored and handled in protocol
// * filters through the new syncRequests and send the ones missing
// * back immediately as a deliveryRequest message
// * empty message just pings back for more (is this needed?)
// * strict signed sync states may be needed.
func (self *Depo) HandleUnsyncedKeysMsg(req *unsyncedKeysMsgData, p *peer) error {
unsynced := req.Unsynced
var missing []*syncRequest
var chunk *storage.Chunk
var err error
for _, req := range unsynced {
// skip keys that are found,
chunk, err = self.localStore.Get(req.Key[:])
if err != nil || chunk.SData == nil {
missing = append(missing, req)
}
}
log.Debug(fmt.Sprintf("Depo.HandleUnsyncedKeysMsg: received %v unsynced keys: %v missing. new state: %v", len(unsynced), len(missing), req.State))
log.Trace(fmt.Sprintf("Depo.HandleUnsyncedKeysMsg: received %v", unsynced))
// send delivery request with missing keys
err = p.deliveryRequest(missing)
if err != nil {
return err
}
// set peers state to persist
p.syncState = req.State
return nil
}
// Handles deliveryRequestMsg
// * serves actual chunks asked by the remote peer
// by pushing to the delivery queue (sync db) of the correct priority
// (remote peer is free to reprioritize)
// * the message implies remote peer wants more, so trigger for
// * new outgoing unsynced keys message is fired
func (self *Depo) HandleDeliveryRequestMsg(req *deliveryRequestMsgData, p *peer) error {
deliver := req.Deliver
// queue the actual delivery of a chunk ()
log.Trace(fmt.Sprintf("Depo.HandleDeliveryRequestMsg: received %v delivery requests: %v", len(deliver), deliver))
for _, sreq := range deliver {
// TODO: look up in cache here or in deliveries
// priorities are taken from the message so the remote party can
// reprioritise to at their leisure
// r = self.pullCached(sreq.Key) // pulls and deletes from cache
Push(p, sreq.Key, sreq.Priority)
}
// sends it out as unsyncedKeysMsg
p.syncer.sendUnsyncedKeys()
return nil
}
// the entrypoint for store requests coming from the bzz wire protocol
// if key found locally, return. otherwise
// remote is untrusted, so hash is verified and chunk passed on to NetStore
func (self *Depo) HandleStoreRequestMsg(req *storeRequestMsgData, p *peer) {
var islocal bool
req.from = p
chunk, err := self.localStore.Get(req.Key)
switch {
case err != nil:
log.Trace(fmt.Sprintf("Depo.handleStoreRequest: %v not found locally. create new chunk/request", req.Key))
// not found in memory cache, ie., a genuine store request
// create chunk
syncReceiveCount.Inc(1)
chunk = storage.NewChunk(req.Key, nil)
case chunk.SData == nil:
// found chunk in memory store, needs the data, validate now
log.Trace(fmt.Sprintf("Depo.HandleStoreRequest: %v. request entry found", req))
default:
// data is found, store request ignored
// this should update access count?
syncReceiveIgnore.Inc(1)
log.Trace(fmt.Sprintf("Depo.HandleStoreRequest: %v found locally. ignore.", req))
islocal = true
//return
}
hasher := self.hashfunc()
hasher.Write(req.SData)
if !bytes.Equal(hasher.Sum(nil), req.Key) {
// data does not validate, ignore
// TODO: peer should be penalised/dropped?
log.Warn(fmt.Sprintf("Depo.HandleStoreRequest: chunk invalid. store request ignored: %v", req))
return
}
if islocal {
return
}
// update chunk with size and data
chunk.SData = req.SData // protocol validates that SData is minimum 9 bytes long (int64 size + at least one byte of data)
chunk.Size = int64(binary.LittleEndian.Uint64(req.SData[0:8]))
log.Trace(fmt.Sprintf("delivery of %v from %v", chunk, p))
chunk.Source = p
self.netStore.Put(chunk)
}
// entrypoint for retrieve requests coming from the bzz wire protocol
// checks swap balance - return if peer has no credit
func (self *Depo) HandleRetrieveRequestMsg(req *retrieveRequestMsgData, p *peer) {
req.from = p
// swap - record credit for 1 request
// note that only charge actual reqsearches
var err error
if p.swap != nil {
err = p.swap.Add(1)
}
if err != nil {
log.Warn(fmt.Sprintf("Depo.HandleRetrieveRequest: %v - cannot process request: %v", req.Key.Log(), err))
return
}
// call storage.NetStore#Get which
// blocks until local retrieval finished
// launches cloud retrieval
chunk, _ := self.netStore.Get(req.Key)
req = self.strategyUpdateRequest(chunk.Req, req)
// check if we can immediately deliver
if chunk.SData != nil {
log.Trace(fmt.Sprintf("Depo.HandleRetrieveRequest: %v - content found, delivering...", req.Key.Log()))
if req.MaxSize == 0 || int64(req.MaxSize) >= chunk.Size {
sreq := &storeRequestMsgData{
Id: req.Id,
Key: chunk.Key,
SData: chunk.SData,
requestTimeout: req.timeout, //
}
syncSendCount.Inc(1)
p.syncer.addRequest(sreq, DeliverReq)
} else {
syncSendRefused.Inc(1)
log.Trace(fmt.Sprintf("Depo.HandleRetrieveRequest: %v - content found, not wanted", req.Key.Log()))
}
} else {
syncSendNotFound.Inc(1)
log.Trace(fmt.Sprintf("Depo.HandleRetrieveRequest: %v - content not found locally. asked swarm for help. will get back", req.Key.Log()))
}
}
// add peer request the chunk and decides the timeout for the response if still searching
func (self *Depo) strategyUpdateRequest(rs *storage.RequestStatus, origReq *retrieveRequestMsgData) (req *retrieveRequestMsgData) {
log.Trace(fmt.Sprintf("Depo.strategyUpdateRequest: key %v", origReq.Key.Log()))
// we do not create an alternative one
req = origReq
if rs != nil {
self.addRequester(rs, req)
req.setTimeout(self.searchTimeout(rs, req))
}
return
}
// decides the timeout promise sent with the immediate peers response to a retrieve request
// if timeout is explicitly set and expired
func (self *Depo) searchTimeout(rs *storage.RequestStatus, req *retrieveRequestMsgData) (timeout *time.Time) {
reqt := req.getTimeout()
t := time.Now().Add(searchTimeout)
if reqt != nil && reqt.Before(t) {
return reqt
} else {
return &t
}
}
/*
adds a new peer to an existing open request
only add if less than requesterCount peers forwarded the same request id so far
note this is done irrespective of status (searching or found)
*/
func (self *Depo) addRequester(rs *storage.RequestStatus, req *retrieveRequestMsgData) {
log.Trace(fmt.Sprintf("Depo.addRequester: key %v - add peer to req.Id %v", req.Key.Log(), req.Id))
list := rs.Requesters[req.Id]
rs.Requesters[req.Id] = append(list, req)
}

210
swarm/network/discovery.go Normal file
View File

@@ -0,0 +1,210 @@
// Copyright 2016 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package network
import (
"fmt"
"sync"
"github.com/ethereum/go-ethereum/swarm/pot"
)
// discovery bzz extension for requesting and relaying node address records
// discPeer wraps BzzPeer and embeds an Overlay connectivity driver
type discPeer struct {
*BzzPeer
overlay Overlay
sentPeers bool // whether we already sent peer closer to this address
mtx sync.RWMutex
peers map[string]bool // tracks node records sent to the peer
depth uint8 // the proximity order advertised by remote as depth of saturation
}
// NewDiscovery constructs a discovery peer
func newDiscovery(p *BzzPeer, o Overlay) *discPeer {
d := &discPeer{
overlay: o,
BzzPeer: p,
peers: make(map[string]bool),
}
// record remote as seen so we never send a peer its own record
d.seen(d)
return d
}
// HandleMsg is the message handler that delegates incoming messages
func (d *discPeer) HandleMsg(msg interface{}) error {
switch msg := msg.(type) {
case *peersMsg:
return d.handlePeersMsg(msg)
case *subPeersMsg:
return d.handleSubPeersMsg(msg)
default:
return fmt.Errorf("unknown message type: %T", msg)
}
}
// NotifyDepth sends a message to all connections if depth of saturation is changed
func NotifyDepth(depth uint8, h Overlay) {
f := func(val OverlayConn, po int, _ bool) bool {
dp, ok := val.(*discPeer)
if ok {
dp.NotifyDepth(depth)
}
return true
}
h.EachConn(nil, 255, f)
}
// NotifyPeer informs all peers about a newly added node
func NotifyPeer(p OverlayAddr, k Overlay) {
f := func(val OverlayConn, po int, _ bool) bool {
dp, ok := val.(*discPeer)
if ok {
dp.NotifyPeer(p, uint8(po))
}
return true
}
k.EachConn(p.Address(), 255, f)
}
// NotifyPeer notifies the remote node (recipient) about a peer if
// the peer's PO is within the recipients advertised depth
// OR the peer is closer to the recipient than self
// unless already notified during the connection session
func (d *discPeer) NotifyPeer(a OverlayAddr, po uint8) {
// immediately return
if (po < d.getDepth() && pot.ProxCmp(d.localAddr, d, a) != 1) || d.seen(a) {
return
}
// log.Trace(fmt.Sprintf("%08x peer %08x notified of peer %08x", d.localAddr.Over()[:4], d.Address()[:4], a.Address()[:4]))
resp := &peersMsg{
Peers: []*BzzAddr{ToAddr(a)},
}
go d.Send(resp)
}
// NotifyDepth sends a subPeers Msg to the receiver notifying them about
// a change in the depth of saturation
func (d *discPeer) NotifyDepth(po uint8) {
// log.Trace(fmt.Sprintf("%08x peer %08x notified of new depth %v", d.localAddr.Over()[:4], d.Address()[:4], po))
go d.Send(&subPeersMsg{Depth: po})
}
/*
peersMsg is the message to pass peer information
It is always a response to a peersRequestMsg
The encoding of a peer address is identical the devp2p base protocol peers
messages: [IP, Port, NodeID],
Note that a node's FileStore address is not the NodeID but the hash of the NodeID.
TODO:
To mitigate against spurious peers messages, requests should be remembered
and correctness of responses should be checked
If the proxBin of peers in the response is incorrect the sender should be
disconnected
*/
// peersMsg encapsulates an array of peer addresses
// used for communicating about known peers
// relevant for bootstrapping connectivity and updating peersets
type peersMsg struct {
Peers []*BzzAddr
}
// String pretty prints a peersMsg
func (msg peersMsg) String() string {
return fmt.Sprintf("%T: %v", msg, msg.Peers)
}
// handlePeersMsg called by the protocol when receiving peerset (for target address)
// list of nodes ([]PeerAddr in peersMsg) is added to the overlay db using the
// Register interface method
func (d *discPeer) handlePeersMsg(msg *peersMsg) error {
// register all addresses
if len(msg.Peers) == 0 {
return nil
}
for _, a := range msg.Peers {
d.seen(a)
NotifyPeer(a, d.overlay)
}
return d.overlay.Register(toOverlayAddrs(msg.Peers...))
}
// subPeers msg is communicating the depth/sharpness/focus of the overlay table of a peer
type subPeersMsg struct {
Depth uint8
}
// String returns the pretty printer
func (msg subPeersMsg) String() string {
return fmt.Sprintf("%T: request peers > PO%02d. ", msg, msg.Depth)
}
func (d *discPeer) handleSubPeersMsg(msg *subPeersMsg) error {
if !d.sentPeers {
d.setDepth(msg.Depth)
var peers []*BzzAddr
d.overlay.EachConn(d.Over(), 255, func(p OverlayConn, po int, isproxbin bool) bool {
if pob, _ := pof(d, d.localAddr, 0); pob > po {
return false
}
if !d.seen(p) {
peers = append(peers, ToAddr(p.Off()))
}
return true
})
if len(peers) > 0 {
// log.Debug(fmt.Sprintf("%08x: %v peers sent to %v", d.overlay.BaseAddr(), len(peers), d))
go d.Send(&peersMsg{Peers: peers})
}
}
d.sentPeers = true
return nil
}
// seen takes an Overlay peer and checks if it was sent to a peer already
// if not, marks the peer as sent
func (d *discPeer) seen(p OverlayPeer) bool {
d.mtx.Lock()
defer d.mtx.Unlock()
k := string(p.Address())
if d.peers[k] {
return true
}
d.peers[k] = true
return false
}
func (d *discPeer) getDepth() uint8 {
d.mtx.RLock()
defer d.mtx.RUnlock()
return d.depth
}
func (d *discPeer) setDepth(depth uint8) {
d.mtx.Lock()
defer d.mtx.Unlock()
d.depth = depth
}

View File

@@ -0,0 +1,57 @@
// Copyright 2016 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package network
import (
"testing"
p2ptest "github.com/ethereum/go-ethereum/p2p/testing"
)
/***
*
* - after connect, that outgoing subpeersmsg is sent
*
*/
func TestDiscovery(t *testing.T) {
params := NewHiveParams()
s, pp := newHiveTester(t, params, 1, nil)
id := s.IDs[0]
raddr := NewAddrFromNodeID(id)
pp.Register([]OverlayAddr{OverlayAddr(raddr)})
// start the hive and wait for the connection
pp.Start(s.Server)
defer pp.Stop()
// send subPeersMsg to the peer
err := s.TestExchanges(p2ptest.Exchange{
Label: "outgoing subPeersMsg",
Expects: []p2ptest.Expect{
{
Code: 1,
Msg: &subPeersMsg{Depth: 0},
Peer: id,
},
},
})
if err != nil {
t.Fatal(err)
}
}

View File

@@ -1,150 +0,0 @@
// Copyright 2016 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package network
import (
"fmt"
"math/rand"
"time"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/swarm/storage"
)
const requesterCount = 3
/*
forwarder implements the CloudStore interface (use by storage.NetStore)
and serves as the cloud store backend orchestrating storage/retrieval/delivery
via the native bzz protocol
which uses an MSB logarithmic distance-based semi-permanent Kademlia table for
* recursive forwarding style routing for retrieval
* smart syncronisation
*/
type forwarder struct {
hive *Hive
}
func NewForwarder(hive *Hive) *forwarder {
return &forwarder{hive: hive}
}
// generate a unique id uint64
func generateId() uint64 {
r := rand.New(rand.NewSource(time.Now().UnixNano()))
return uint64(r.Int63())
}
var searchTimeout = 3 * time.Second
// forwarding logic
// logic propagating retrieve requests to peers given by the kademlia hive
func (self *forwarder) Retrieve(chunk *storage.Chunk) {
peers := self.hive.getPeers(chunk.Key, 0)
log.Trace(fmt.Sprintf("forwarder.Retrieve: %v - received %d peers from KΛÐΞMLIΛ...", chunk.Key.Log(), len(peers)))
OUT:
for _, p := range peers {
log.Trace(fmt.Sprintf("forwarder.Retrieve: sending retrieveRequest %v to peer [%v]", chunk.Key.Log(), p))
for _, recipients := range chunk.Req.Requesters {
for _, recipient := range recipients {
req := recipient.(*retrieveRequestMsgData)
if req.from.Addr() == p.Addr() {
continue OUT
}
}
}
req := &retrieveRequestMsgData{
Key: chunk.Key,
Id: generateId(),
}
var err error
if p.swap != nil {
err = p.swap.Add(-1)
}
if err == nil {
p.retrieve(req)
break OUT
}
log.Warn(fmt.Sprintf("forwarder.Retrieve: unable to send retrieveRequest to peer [%v]: %v", chunk.Key.Log(), err))
}
}
// requests to specific peers given by the kademlia hive
// except for peers that the store request came from (if any)
// delivery queueing taken care of by syncer
func (self *forwarder) Store(chunk *storage.Chunk) {
var n int
msg := &storeRequestMsgData{
Key: chunk.Key,
SData: chunk.SData,
}
var source *peer
if chunk.Source != nil {
source = chunk.Source.(*peer)
}
for _, p := range self.hive.getPeers(chunk.Key, 0) {
log.Trace(fmt.Sprintf("forwarder.Store: %v %v", p, chunk))
if p.syncer != nil && (source == nil || p.Addr() != source.Addr()) {
n++
Deliver(p, msg, PropagateReq)
}
}
log.Trace(fmt.Sprintf("forwarder.Store: sent to %v peers (chunk = %v)", n, chunk))
}
// once a chunk is found deliver it to its requesters unless timed out
func (self *forwarder) Deliver(chunk *storage.Chunk) {
// iterate over request entries
for id, requesters := range chunk.Req.Requesters {
counter := requesterCount
msg := &storeRequestMsgData{
Key: chunk.Key,
SData: chunk.SData,
}
var n int
var req *retrieveRequestMsgData
// iterate over requesters with the same id
for id, r := range requesters {
req = r.(*retrieveRequestMsgData)
if req.timeout == nil || req.timeout.After(time.Now()) {
log.Trace(fmt.Sprintf("forwarder.Deliver: %v -> %v", req.Id, req.from))
msg.Id = uint64(id)
Deliver(req.from, msg, DeliverReq)
n++
counter--
if counter <= 0 {
break
}
}
}
log.Trace(fmt.Sprintf("forwarder.Deliver: submit chunk %v (request id %v) for delivery to %v peers", chunk.Key.Log(), id, n))
}
}
// initiate delivery of a chunk to a particular peer via syncer#addRequest
// depending on syncer mode and priority settings and sync request type
// this either goes via confirmation roundtrip or queued or pushed directly
func Deliver(p *peer, req interface{}, ty int) {
p.syncer.addRequest(req, ty)
}
// push chunk over to peer
func Push(p *peer, key storage.Key, priority uint) {
p.syncer.doDelivery(key, priority, p.syncer.quit)
}

View File

@@ -18,386 +18,244 @@ package network
import (
"fmt"
"math/rand"
"path/filepath"
"sync"
"time"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/metrics"
"github.com/ethereum/go-ethereum/common/hexutil"
"github.com/ethereum/go-ethereum/p2p"
"github.com/ethereum/go-ethereum/p2p/discover"
"github.com/ethereum/go-ethereum/p2p/netutil"
"github.com/ethereum/go-ethereum/swarm/network/kademlia"
"github.com/ethereum/go-ethereum/swarm/storage"
"github.com/ethereum/go-ethereum/swarm/log"
"github.com/ethereum/go-ethereum/swarm/state"
)
// Hive is the logistic manager of the swarm
// it uses a generic kademlia nodetable to find best peer list
// for any target
// this is used by the netstore to search for content in the swarm
// the bzz protocol peersMsgData exchange is relayed to Kademlia
// for db storage and filtering
// connections and disconnections are reported and relayed
// to keep the nodetable uptodate
/*
Hive is the logistic manager of the swarm
var (
peersNumGauge = metrics.NewRegisteredGauge("network.peers.num", nil)
addPeerCounter = metrics.NewRegisteredCounter("network.addpeer.count", nil)
removePeerCounter = metrics.NewRegisteredCounter("network.removepeer.count", nil)
)
When the hive is started, a forever loop is launched that
asks the Overlay Topology driver (e.g., generic kademlia nodetable)
to suggest peers to bootstrap connectivity
*/
type Hive struct {
listenAddr func() string
callInterval uint64
id discover.NodeID
addr kademlia.Address
kad *kademlia.Kademlia
path string
quit chan bool
toggle chan bool
more chan bool
// for testing only
swapEnabled bool
syncEnabled bool
blockRead bool
blockWrite bool
// Overlay is the interface for kademlia (or other topology drivers)
type Overlay interface {
// suggest peers to connect to
SuggestPeer() (OverlayAddr, int, bool)
// register and deregister peer connections
On(OverlayConn) (depth uint8, changed bool)
Off(OverlayConn)
// register peer addresses
Register([]OverlayAddr) error
// iterate over connected peers
EachConn([]byte, int, func(OverlayConn, int, bool) bool)
// iterate over known peers (address records)
EachAddr([]byte, int, func(OverlayAddr, int, bool) bool)
// pretty print the connectivity
String() string
// base Overlay address of the node itself
BaseAddr() []byte
// connectivity health check used for testing
Healthy(*PeerPot) *Health
}
const (
callInterval = 3000000000
// bucketSize = 3
// maxProx = 8
// proxBinSize = 4
)
// HiveParams holds the config options to hive
type HiveParams struct {
CallInterval uint64
KadDbPath string
*kademlia.KadParams
Discovery bool // if want discovery of not
PeersBroadcastSetSize uint8 // how many peers to use when relaying
MaxPeersPerRequest uint8 // max size for peer address batches
KeepAliveInterval time.Duration
}
//create default params
func NewDefaultHiveParams() *HiveParams {
kad := kademlia.NewDefaultKadParams()
// kad.BucketSize = bucketSize
// kad.MaxProx = maxProx
// kad.ProxBinSize = proxBinSize
// NewHiveParams returns hive config with only the
func NewHiveParams() *HiveParams {
return &HiveParams{
CallInterval: callInterval,
KadParams: kad,
Discovery: true,
PeersBroadcastSetSize: 3,
MaxPeersPerRequest: 5,
KeepAliveInterval: 500 * time.Millisecond,
}
}
//this can only finally be set after all config options (file, cmd line, env vars)
//have been evaluated
func (self *HiveParams) Init(path string) {
self.KadDbPath = filepath.Join(path, "bzz-peers.json")
// Hive manages network connections of the swarm node
type Hive struct {
*HiveParams // settings
Overlay // the overlay connectiviy driver
Store state.Store // storage interface to save peers across sessions
addPeer func(*discover.Node) // server callback to connect to a peer
// bookkeeping
lock sync.Mutex
ticker *time.Ticker
}
func NewHive(addr common.Hash, params *HiveParams, swapEnabled, syncEnabled bool) *Hive {
kad := kademlia.New(kademlia.Address(addr), params.KadParams)
// NewHive constructs a new hive
// HiveParams: config parameters
// Overlay: connectivity driver using a network topology
// StateStore: to save peers across sessions
func NewHive(params *HiveParams, overlay Overlay, store state.Store) *Hive {
return &Hive{
callInterval: params.CallInterval,
kad: kad,
addr: kad.Addr(),
path: params.KadDbPath,
swapEnabled: swapEnabled,
syncEnabled: syncEnabled,
HiveParams: params,
Overlay: overlay,
Store: store,
}
}
func (self *Hive) SyncEnabled(on bool) {
self.syncEnabled = on
}
func (self *Hive) SwapEnabled(on bool) {
self.swapEnabled = on
}
func (self *Hive) BlockNetworkRead(on bool) {
self.blockRead = on
}
func (self *Hive) BlockNetworkWrite(on bool) {
self.blockWrite = on
}
// public accessor to the hive base address
func (self *Hive) Addr() kademlia.Address {
return self.addr
}
// Start receives network info only at startup
// listedAddr is a function to retrieve listening address to advertise to peers
// connectPeer is a function to connect to a peer based on its NodeID or enode URL
// there are called on the p2p.Server which runs on the node
func (self *Hive) Start(id discover.NodeID, listenAddr func() string, connectPeer func(string) error) (err error) {
self.toggle = make(chan bool)
self.more = make(chan bool)
self.quit = make(chan bool)
self.id = id
self.listenAddr = listenAddr
err = self.kad.Load(self.path, nil)
if err != nil {
log.Warn(fmt.Sprintf("Warning: error reading kaddb '%s' (skipping): %v", self.path, err))
err = nil
// Start stars the hive, receives p2p.Server only at startup
// server is used to connect to a peer based on its NodeID or enode URL
// these are called on the p2p.Server which runs on the node
func (h *Hive) Start(server *p2p.Server) error {
log.Info(fmt.Sprintf("%08x hive starting", h.BaseAddr()[:4]))
// if state store is specified, load peers to prepopulate the overlay address book
if h.Store != nil {
log.Info("detected an existing store. trying to load peers")
if err := h.loadPeers(); err != nil {
log.Error(fmt.Sprintf("%08x hive encoutered an error trying to load peers", h.BaseAddr()[:4]))
return err
}
}
// assigns the p2p.Server#AddPeer function to connect to peers
h.addPeer = server.AddPeer
// ticker to keep the hive alive
h.ticker = time.NewTicker(h.KeepAliveInterval)
// this loop is doing bootstrapping and maintains a healthy table
go self.keepAlive()
go func() {
// whenever toggled ask kademlia about most preferred peer
for alive := range self.more {
if !alive {
// receiving false closes the loop while allowing parallel routines
// to attempt to write to more (remove Peer when shutting down)
return
}
node, need, proxLimit := self.kad.Suggest()
if node != nil && len(node.Url) > 0 {
log.Trace(fmt.Sprintf("call known bee %v", node.Url))
// enode or any lower level connection address is unnecessary in future
// discovery table is used to look it up.
connectPeer(node.Url)
}
if need {
// a random peer is taken from the table
peers := self.kad.FindClosest(kademlia.RandomAddressAt(self.addr, rand.Intn(self.kad.MaxProx)), 1)
if len(peers) > 0 {
// a random address at prox bin 0 is sent for lookup
randAddr := kademlia.RandomAddressAt(self.addr, proxLimit)
req := &retrieveRequestMsgData{
Key: storage.Key(randAddr[:]),
}
log.Trace(fmt.Sprintf("call any bee near %v (PO%03d) - messenger bee: %v", randAddr, proxLimit, peers[0]))
peers[0].(*peer).retrieve(req)
} else {
log.Warn(fmt.Sprintf("no peer"))
}
log.Trace(fmt.Sprintf("buzz kept alive"))
} else {
log.Info(fmt.Sprintf("no need for more bees"))
}
select {
case self.toggle <- need:
case <-self.quit:
return
}
log.Debug(fmt.Sprintf("queen's address: %v, population: %d (%d)", self.addr, self.kad.Count(), self.kad.DBCount()))
}
}()
return
}
// keepAlive is a forever loop
// in its awake state it periodically triggers connection attempts
// by writing to self.more until Kademlia Table is saturated
// wake state is toggled by writing to self.toggle
// it restarts if the table becomes non-full again due to disconnections
func (self *Hive) keepAlive() {
alarm := time.NewTicker(time.Duration(self.callInterval)).C
for {
peersNumGauge.Update(int64(self.kad.Count()))
select {
case <-alarm:
if self.kad.DBCount() > 0 {
select {
case self.more <- true:
log.Debug(fmt.Sprintf("buzz wakeup"))
default:
}
}
case need := <-self.toggle:
if alarm == nil && need {
alarm = time.NewTicker(time.Duration(self.callInterval)).C
}
if alarm != nil && !need {
alarm = nil
}
case <-self.quit:
return
}
}
}
func (self *Hive) Stop() error {
// closing toggle channel quits the updateloop
close(self.quit)
return self.kad.Save(self.path, saveSync)
}
// called at the end of a successful protocol handshake
func (self *Hive) addPeer(p *peer) error {
addPeerCounter.Inc(1)
defer func() {
select {
case self.more <- true:
default:
}
}()
log.Trace(fmt.Sprintf("hi new bee %v", p))
err := self.kad.On(p, loadSync)
if err != nil {
return err
}
// self lookup (can be encoded as nil/zero key since peers addr known) + no id ()
// the most common way of saying hi in bzz is initiation of gossip
// let me know about anyone new from my hood , here is the storageradius
// to send the 6 byte self lookup
// we do not record as request or forward it, just reply with peers
p.retrieve(&retrieveRequestMsgData{})
log.Trace(fmt.Sprintf("'whatsup wheresdaparty' sent to %v", p))
go h.connect()
return nil
}
// called after peer disconnected
func (self *Hive) removePeer(p *peer) {
removePeerCounter.Inc(1)
log.Debug(fmt.Sprintf("bee %v removed", p))
self.kad.Off(p, saveSync)
select {
case self.more <- true:
default:
// Stop terminates the updateloop and saves the peers
func (h *Hive) Stop() error {
log.Info(fmt.Sprintf("%08x hive stopping, saving peers", h.BaseAddr()[:4]))
h.ticker.Stop()
if h.Store != nil {
if err := h.savePeers(); err != nil {
return fmt.Errorf("could not save peers to persistence store: %v", err)
}
if err := h.Store.Close(); err != nil {
return fmt.Errorf("could not close file handle to persistence store: %v", err)
}
}
if self.kad.Count() == 0 {
log.Debug(fmt.Sprintf("empty, all bees gone"))
log.Info(fmt.Sprintf("%08x hive stopped, dropping peers", h.BaseAddr()[:4]))
h.EachConn(nil, 255, func(p OverlayConn, _ int, _ bool) bool {
log.Info(fmt.Sprintf("%08x dropping peer %08x", h.BaseAddr()[:4], p.Address()[:4]))
p.Drop(nil)
return true
})
log.Info(fmt.Sprintf("%08x all peers dropped", h.BaseAddr()[:4]))
return nil
}
// connect is a forever loop
// at each iteration, ask the overlay driver to suggest the most preferred peer to connect to
// as well as advertises saturation depth if needed
func (h *Hive) connect() {
for range h.ticker.C {
addr, depth, changed := h.SuggestPeer()
if h.Discovery && changed {
NotifyDepth(uint8(depth), h)
}
if addr == nil {
continue
}
log.Trace(fmt.Sprintf("%08x hive connect() suggested %08x", h.BaseAddr()[:4], addr.Address()[:4]))
under, err := discover.ParseNode(string(addr.(Addr).Under()))
if err != nil {
log.Warn(fmt.Sprintf("%08x unable to connect to bee %08x: invalid node URL: %v", h.BaseAddr()[:4], addr.Address()[:4], err))
continue
}
log.Trace(fmt.Sprintf("%08x attempt to connect to bee %08x", h.BaseAddr()[:4], addr.Address()[:4]))
h.addPeer(under)
}
}
// Retrieve a list of live peers that are closer to target than us
func (self *Hive) getPeers(target storage.Key, max int) (peers []*peer) {
var addr kademlia.Address
copy(addr[:], target[:])
for _, node := range self.kad.FindClosest(addr, max) {
peers = append(peers, node.(*peer))
// Run protocol run function
func (h *Hive) Run(p *BzzPeer) error {
dp := newDiscovery(p, h)
depth, changed := h.On(dp)
// if we want discovery, advertise change of depth
if h.Discovery {
if changed {
// if depth changed, send to all peers
NotifyDepth(depth, h)
} else {
// otherwise just send depth to new peer
dp.NotifyDepth(depth)
}
}
NotifyPeer(p.Off(), h)
defer h.Off(dp)
return dp.Run(dp.HandleMsg)
}
// NodeInfo function is used by the p2p.server RPC interface to display
// protocol specific node information
func (h *Hive) NodeInfo() interface{} {
return h.String()
}
// PeerInfo function is used by the p2p.server RPC interface to display
// protocol specific information any connected peer referred to by their NodeID
func (h *Hive) PeerInfo(id discover.NodeID) interface{} {
addr := NewAddrFromNodeID(id)
return struct {
OAddr hexutil.Bytes
UAddr hexutil.Bytes
}{
OAddr: addr.OAddr,
UAddr: addr.UAddr,
}
}
// ToAddr returns the serialisable version of u
func ToAddr(pa OverlayPeer) *BzzAddr {
if addr, ok := pa.(*BzzAddr); ok {
return addr
}
if p, ok := pa.(*discPeer); ok {
return p.BzzAddr
}
return pa.(*BzzPeer).BzzAddr
}
// loadPeers, savePeer implement persistence callback/
func (h *Hive) loadPeers() error {
var as []*BzzAddr
err := h.Store.Get("peers", &as)
if err != nil {
if err == state.ErrNotFound {
log.Info(fmt.Sprintf("hive %08x: no persisted peers found", h.BaseAddr()[:4]))
return nil
}
return err
}
log.Info(fmt.Sprintf("hive %08x: peers loaded", h.BaseAddr()[:4]))
return h.Register(toOverlayAddrs(as...))
}
// toOverlayAddrs transforms an array of BzzAddr to OverlayAddr
func toOverlayAddrs(as ...*BzzAddr) (oas []OverlayAddr) {
for _, a := range as {
oas = append(oas, OverlayAddr(a))
}
return
}
// disconnects all the peers
func (self *Hive) DropAll() {
log.Info(fmt.Sprintf("dropping all bees"))
for _, node := range self.kad.FindClosest(kademlia.Address{}, 0) {
node.Drop()
}
}
// contructor for kademlia.NodeRecord based on peer address alone
// TODO: should go away and only addr passed to kademlia
func newNodeRecord(addr *peerAddr) *kademlia.NodeRecord {
now := time.Now()
return &kademlia.NodeRecord{
Addr: addr.Addr,
Url: addr.String(),
Seen: now,
After: now,
}
}
// called by the protocol when receiving peerset (for target address)
// peersMsgData is converted to a slice of NodeRecords for Kademlia
// this is to store all thats needed
func (self *Hive) HandlePeersMsg(req *peersMsgData, from *peer) {
var nrs []*kademlia.NodeRecord
for _, p := range req.Peers {
if err := netutil.CheckRelayIP(from.remoteAddr.IP, p.IP); err != nil {
log.Trace(fmt.Sprintf("invalid peer IP %v from %v: %v", from.remoteAddr.IP, p.IP, err))
continue
// savePeers, savePeer implement persistence callback/
func (h *Hive) savePeers() error {
var peers []*BzzAddr
h.Overlay.EachAddr(nil, 256, func(pa OverlayAddr, i int, _ bool) bool {
if pa == nil {
log.Warn(fmt.Sprintf("empty addr: %v", i))
return true
}
nrs = append(nrs, newNodeRecord(p))
apa := ToAddr(pa)
log.Trace("saving peer", "peer", apa)
peers = append(peers, apa)
return true
})
if err := h.Store.Put("peers", peers); err != nil {
return fmt.Errorf("could not save peers: %v", err)
}
self.kad.Add(nrs)
}
// peer wraps the protocol instance to represent a connected peer
// it implements kademlia.Node interface
type peer struct {
*bzz // protocol instance running on peer connection
}
// protocol instance implements kademlia.Node interface (embedded peer)
func (self *peer) Addr() kademlia.Address {
return self.remoteAddr.Addr
}
func (self *peer) Url() string {
return self.remoteAddr.String()
}
// TODO take into account traffic
func (self *peer) LastActive() time.Time {
return self.lastActive
}
// reads the serialised form of sync state persisted as the 'Meta' attribute
// and sets the decoded syncState on the online node
func loadSync(record *kademlia.NodeRecord, node kademlia.Node) error {
p, ok := node.(*peer)
if !ok {
return fmt.Errorf("invalid type")
}
if record.Meta == nil {
log.Debug(fmt.Sprintf("no sync state for node record %v setting default", record))
p.syncState = &syncState{DbSyncState: &storage.DbSyncState{}}
return nil
}
state, err := decodeSync(record.Meta)
if err != nil {
return fmt.Errorf("error decoding kddb record meta info into a sync state: %v", err)
}
log.Trace(fmt.Sprintf("sync state for node record %v read from Meta: %s", record, string(*(record.Meta))))
p.syncState = state
return err
}
// callback when saving a sync state
func saveSync(record *kademlia.NodeRecord, node kademlia.Node) {
if p, ok := node.(*peer); ok {
meta, err := encodeSync(p.syncState)
if err != nil {
log.Warn(fmt.Sprintf("error saving sync state for %v: %v", node, err))
return
}
log.Trace(fmt.Sprintf("saved sync state for %v: %s", node, string(*meta)))
record.Meta = meta
}
}
// the immediate response to a retrieve request,
// sends relevant peer data given by the kademlia hive to the requester
// TODO: remember peers sent for duration of the session, only new peers sent
func (self *Hive) peers(req *retrieveRequestMsgData) {
if req != nil {
var addrs []*peerAddr
if req.timeout == nil || time.Now().Before(*(req.timeout)) {
key := req.Key
// self lookup from remote peer
if storage.IsZeroKey(key) {
addr := req.from.Addr()
key = storage.Key(addr[:])
req.Key = nil
}
// get peer addresses from hive
for _, peer := range self.getPeers(key, int(req.MaxPeers)) {
addrs = append(addrs, peer.remoteAddr)
}
log.Debug(fmt.Sprintf("Hive sending %d peer addresses to %v. req.Id: %v, req.Key: %v", len(addrs), req.from, req.Id, req.Key.Log()))
peersData := &peersMsgData{
Peers: addrs,
Key: req.Key,
Id: req.Id,
}
peersData.setTimeout(req.timeout)
req.from.peers(peersData)
}
}
}
func (self *Hive) String() string {
return self.kad.String()
return nil
}

108
swarm/network/hive_test.go Normal file
View File

@@ -0,0 +1,108 @@
// Copyright 2016 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package network
import (
"io/ioutil"
"log"
"os"
"testing"
p2ptest "github.com/ethereum/go-ethereum/p2p/testing"
"github.com/ethereum/go-ethereum/swarm/state"
)
func newHiveTester(t *testing.T, params *HiveParams, n int, store state.Store) (*bzzTester, *Hive) {
// setup
addr := RandomAddr() // tested peers peer address
to := NewKademlia(addr.OAddr, NewKadParams())
pp := NewHive(params, to, store) // hive
return newBzzBaseTester(t, n, addr, DiscoverySpec, pp.Run), pp
}
func TestRegisterAndConnect(t *testing.T) {
params := NewHiveParams()
s, pp := newHiveTester(t, params, 1, nil)
id := s.IDs[0]
raddr := NewAddrFromNodeID(id)
pp.Register([]OverlayAddr{OverlayAddr(raddr)})
// start the hive and wait for the connection
err := pp.Start(s.Server)
if err != nil {
t.Fatal(err)
}
defer pp.Stop()
// retrieve and broadcast
err = s.TestDisconnected(&p2ptest.Disconnect{
Peer: s.IDs[0],
Error: nil,
})
if err == nil || err.Error() != "timed out waiting for peers to disconnect" {
t.Fatalf("expected peer to connect")
}
}
func TestHiveStatePersistance(t *testing.T) {
log.SetOutput(os.Stdout)
dir, err := ioutil.TempDir("", "hive_test_store")
if err != nil {
panic(err)
}
defer os.RemoveAll(dir)
store, err := state.NewDBStore(dir) //start the hive with an empty dbstore
params := NewHiveParams()
s, pp := newHiveTester(t, params, 5, store)
peers := make(map[string]bool)
for _, id := range s.IDs {
raddr := NewAddrFromNodeID(id)
pp.Register([]OverlayAddr{OverlayAddr(raddr)})
peers[raddr.String()] = true
}
// start the hive and wait for the connection
err = pp.Start(s.Server)
if err != nil {
t.Fatal(err)
}
pp.Stop()
store.Close()
persistedStore, err := state.NewDBStore(dir) //start the hive with an empty dbstore
s1, pp := newHiveTester(t, params, 1, persistedStore)
//start the hive and wait for the connection
pp.Start(s1.Server)
i := 0
pp.Overlay.EachAddr(nil, 256, func(addr OverlayAddr, po int, nn bool) bool {
delete(peers, addr.(*BzzAddr).String())
i++
return true
})
if len(peers) != 0 || i != 5 {
t.Fatalf("invalid peers loaded")
}
}

765
swarm/network/kademlia.go Normal file
View File

@@ -0,0 +1,765 @@
// Copyright 2017 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package network
import (
"bytes"
"fmt"
"math/rand"
"strings"
"sync"
"time"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/swarm/log"
"github.com/ethereum/go-ethereum/swarm/pot"
)
/*
Taking the proximity order relative to a fix point x classifies the points in
the space (n byte long byte sequences) into bins. Items in each are at
most half as distant from x as items in the previous bin. Given a sample of
uniformly distributed items (a hash function over arbitrary sequence) the
proximity scale maps onto series of subsets with cardinalities on a negative
exponential scale.
It also has the property that any two item belonging to the same bin are at
most half as distant from each other as they are from x.
If we think of random sample of items in the bins as connections in a network of
interconnected nodes then relative proximity can serve as the basis for local
decisions for graph traversal where the task is to find a route between two
points. Since in every hop, the finite distance halves, there is
a guaranteed constant maximum limit on the number of hops needed to reach one
node from the other.
*/
var pof = pot.DefaultPof(256)
// KadParams holds the config params for Kademlia
type KadParams struct {
// adjustable parameters
MaxProxDisplay int // number of rows the table shows
MinProxBinSize int // nearest neighbour core minimum cardinality
MinBinSize int // minimum number of peers in a row
MaxBinSize int // maximum number of peers in a row before pruning
RetryInterval int64 // initial interval before a peer is first redialed
RetryExponent int // exponent to multiply retry intervals with
MaxRetries int // maximum number of redial attempts
// function to sanction or prevent suggesting a peer
Reachable func(OverlayAddr) bool
}
// NewKadParams returns a params struct with default values
func NewKadParams() *KadParams {
return &KadParams{
MaxProxDisplay: 16,
MinProxBinSize: 2,
MinBinSize: 2,
MaxBinSize: 4,
RetryInterval: 4200000000, // 4.2 sec
MaxRetries: 42,
RetryExponent: 2,
}
}
// Kademlia is a table of live peers and a db of known peers (node records)
type Kademlia struct {
lock sync.RWMutex
*KadParams // Kademlia configuration parameters
base []byte // immutable baseaddress of the table
addrs *pot.Pot // pots container for known peer addresses
conns *pot.Pot // pots container for live peer connections
depth uint8 // stores the last current depth of saturation
nDepth int // stores the last neighbourhood depth
nDepthC chan int // returned by DepthC function to signal neighbourhood depth change
addrCountC chan int // returned by AddrCountC function to signal peer count change
}
// NewKademlia creates a Kademlia table for base address addr
// with parameters as in params
// if params is nil, it uses default values
func NewKademlia(addr []byte, params *KadParams) *Kademlia {
if params == nil {
params = NewKadParams()
}
return &Kademlia{
base: addr,
KadParams: params,
addrs: pot.NewPot(nil, 0),
conns: pot.NewPot(nil, 0),
}
}
// OverlayPeer interface captures the common aspect of view of a peer from the Overlay
// topology driver
type OverlayPeer interface {
Address() []byte
}
// OverlayConn represents a connected peer
type OverlayConn interface {
OverlayPeer
Drop(error) // call to indicate a peer should be expunged
Off() OverlayAddr // call to return a persitent OverlayAddr
}
// OverlayAddr represents a kademlia peer record
type OverlayAddr interface {
OverlayPeer
Update(OverlayAddr) OverlayAddr // returns the updated version of the original
}
// entry represents a Kademlia table entry (an extension of OverlayPeer)
type entry struct {
OverlayPeer
seenAt time.Time
retries int
}
// newEntry creates a kademlia peer from an OverlayPeer interface
func newEntry(p OverlayPeer) *entry {
return &entry{
OverlayPeer: p,
seenAt: time.Now(),
}
}
// Bin is the binary (bitvector) serialisation of the entry address
func (e *entry) Bin() string {
return pot.ToBin(e.addr().Address())
}
// Label is a short tag for the entry for debug
func Label(e *entry) string {
return fmt.Sprintf("%s (%d)", e.Hex()[:4], e.retries)
}
// Hex is the hexadecimal serialisation of the entry address
func (e *entry) Hex() string {
return fmt.Sprintf("%x", e.addr().Address())
}
// String is the short tag for the entry
func (e *entry) String() string {
return fmt.Sprintf("%s (%d)", e.Hex()[:8], e.retries)
}
// addr returns the kad peer record (OverlayAddr) corresponding to the entry
func (e *entry) addr() OverlayAddr {
a, _ := e.OverlayPeer.(OverlayAddr)
return a
}
// conn returns the connected peer (OverlayPeer) corresponding to the entry
func (e *entry) conn() OverlayConn {
c, _ := e.OverlayPeer.(OverlayConn)
return c
}
// Register enters each OverlayAddr as kademlia peer record into the
// database of known peer addresses
func (k *Kademlia) Register(peers []OverlayAddr) error {
k.lock.Lock()
defer k.lock.Unlock()
var known, size int
for _, p := range peers {
// error if self received, peer should know better
// and should be punished for this
if bytes.Equal(p.Address(), k.base) {
return fmt.Errorf("add peers: %x is self", k.base)
}
var found bool
k.addrs, _, found, _ = pot.Swap(k.addrs, p, pof, func(v pot.Val) pot.Val {
// if not found
if v == nil {
// insert new offline peer into conns
return newEntry(p)
}
// found among known peers, do nothing
return v
})
if found {
known++
}
size++
}
// send new address count value only if there are new addresses
if k.addrCountC != nil && size-known > 0 {
k.addrCountC <- k.addrs.Size()
}
// log.Trace(fmt.Sprintf("%x registered %v peers, %v known, total: %v", k.BaseAddr()[:4], size, known, k.addrs.Size()))
k.sendNeighbourhoodDepthChange()
return nil
}
// SuggestPeer returns a known peer for the lowest proximity bin for the
// lowest bincount below depth
// naturally if there is an empty row it returns a peer for that
func (k *Kademlia) SuggestPeer() (a OverlayAddr, o int, want bool) {
k.lock.Lock()
defer k.lock.Unlock()
minsize := k.MinBinSize
depth := k.neighbourhoodDepth()
// if there is a callable neighbour within the current proxBin, connect
// this makes sure nearest neighbour set is fully connected
var ppo int
k.addrs.EachNeighbour(k.base, pof, func(val pot.Val, po int) bool {
if po < depth {
return false
}
a = k.callable(val)
ppo = po
return a == nil
})
if a != nil {
log.Trace(fmt.Sprintf("%08x candidate nearest neighbour found: %v (%v)", k.BaseAddr()[:4], a, ppo))
return a, 0, false
}
// log.Trace(fmt.Sprintf("%08x no candidate nearest neighbours to connect to (Depth: %v, minProxSize: %v) %#v", k.BaseAddr()[:4], depth, k.MinProxBinSize, a))
var bpo []int
prev := -1
k.conns.EachBin(k.base, pof, 0, func(po, size int, f func(func(val pot.Val, i int) bool) bool) bool {
prev++
for ; prev < po; prev++ {
bpo = append(bpo, prev)
minsize = 0
}
if size < minsize {
bpo = append(bpo, po)
minsize = size
}
return size > 0 && po < depth
})
// all buckets are full, ie., minsize == k.MinBinSize
if len(bpo) == 0 {
// log.Debug(fmt.Sprintf("%08x: all bins saturated", k.BaseAddr()[:4]))
return nil, 0, false
}
// as long as we got candidate peers to connect to
// dont ask for new peers (want = false)
// try to select a candidate peer
// find the first callable peer
nxt := bpo[0]
k.addrs.EachBin(k.base, pof, nxt, func(po, _ int, f func(func(pot.Val, int) bool) bool) bool {
// for each bin (up until depth) we find callable candidate peers
if po >= depth {
return false
}
return f(func(val pot.Val, _ int) bool {
a = k.callable(val)
return a == nil
})
})
// found a candidate
if a != nil {
return a, 0, false
}
// no candidate peer found, request for the short bin
var changed bool
if uint8(nxt) < k.depth {
k.depth = uint8(nxt)
changed = true
}
return a, nxt, changed
}
// On inserts the peer as a kademlia peer into the live peers
func (k *Kademlia) On(p OverlayConn) (uint8, bool) {
k.lock.Lock()
defer k.lock.Unlock()
e := newEntry(p)
var ins bool
k.conns, _, _, _ = pot.Swap(k.conns, p, pof, func(v pot.Val) pot.Val {
// if not found live
if v == nil {
ins = true
// insert new online peer into conns
return e
}
// found among live peers, do nothing
return v
})
if ins {
// insert new online peer into addrs
k.addrs, _, _, _ = pot.Swap(k.addrs, p, pof, func(v pot.Val) pot.Val {
return e
})
// send new address count value only if the peer is inserted
if k.addrCountC != nil {
k.addrCountC <- k.addrs.Size()
}
}
log.Trace(k.string())
// calculate if depth of saturation changed
depth := uint8(k.saturation(k.MinBinSize))
var changed bool
if depth != k.depth {
changed = true
k.depth = depth
}
k.sendNeighbourhoodDepthChange()
return k.depth, changed
}
// NeighbourhoodDepthC returns the channel that sends a new kademlia
// neighbourhood depth on each change.
// Not receiving from the returned channel will block On function
// when the neighbourhood depth is changed.
func (k *Kademlia) NeighbourhoodDepthC() <-chan int {
if k.nDepthC == nil {
k.nDepthC = make(chan int)
}
return k.nDepthC
}
// sendNeighbourhoodDepthChange sends new neighbourhood depth to k.nDepth channel
// if it is initialized.
func (k *Kademlia) sendNeighbourhoodDepthChange() {
// nDepthC is initialized when NeighbourhoodDepthC is called and returned by it.
// It provides signaling of neighbourhood depth change.
// This part of the code is sending new neighbourhood depth to nDepthC if that condition is met.
if k.nDepthC != nil {
nDepth := k.neighbourhoodDepth()
if nDepth != k.nDepth {
k.nDepth = nDepth
k.nDepthC <- nDepth
}
}
}
// AddrCountC returns the channel that sends a new
// address count value on each change.
// Not receiving from the returned channel will block Register function
// when address count value changes.
func (k *Kademlia) AddrCountC() <-chan int {
if k.addrCountC == nil {
k.addrCountC = make(chan int)
}
return k.addrCountC
}
// Off removes a peer from among live peers
func (k *Kademlia) Off(p OverlayConn) {
k.lock.Lock()
defer k.lock.Unlock()
var del bool
k.addrs, _, _, _ = pot.Swap(k.addrs, p, pof, func(v pot.Val) pot.Val {
// v cannot be nil, must check otherwise we overwrite entry
if v == nil {
panic(fmt.Sprintf("connected peer not found %v", p))
}
del = true
return newEntry(p.Off())
})
if del {
k.conns, _, _, _ = pot.Swap(k.conns, p, pof, func(_ pot.Val) pot.Val {
// v cannot be nil, but no need to check
return nil
})
// send new address count value only if the peer is deleted
if k.addrCountC != nil {
k.addrCountC <- k.addrs.Size()
}
k.sendNeighbourhoodDepthChange()
}
}
func (k *Kademlia) EachBin(base []byte, pof pot.Pof, o int, eachBinFunc func(conn OverlayConn, po int) bool) {
k.lock.RLock()
defer k.lock.RUnlock()
var startPo int
var endPo int
kadDepth := k.neighbourhoodDepth()
k.conns.EachBin(base, pof, o, func(po, size int, f func(func(val pot.Val, i int) bool) bool) bool {
if startPo > 0 && endPo != k.MaxProxDisplay {
startPo = endPo + 1
}
if po < kadDepth {
endPo = po
} else {
endPo = k.MaxProxDisplay
}
for bin := startPo; bin <= endPo; bin++ {
f(func(val pot.Val, _ int) bool {
return eachBinFunc(val.(*entry).conn(), bin)
})
}
return true
})
}
// EachConn is an iterator with args (base, po, f) applies f to each live peer
// that has proximity order po or less as measured from the base
// if base is nil, kademlia base address is used
func (k *Kademlia) EachConn(base []byte, o int, f func(OverlayConn, int, bool) bool) {
k.lock.RLock()
defer k.lock.RUnlock()
k.eachConn(base, o, f)
}
func (k *Kademlia) eachConn(base []byte, o int, f func(OverlayConn, int, bool) bool) {
if len(base) == 0 {
base = k.base
}
depth := k.neighbourhoodDepth()
k.conns.EachNeighbour(base, pof, func(val pot.Val, po int) bool {
if po > o {
return true
}
return f(val.(*entry).conn(), po, po >= depth)
})
}
// EachAddr called with (base, po, f) is an iterator applying f to each known peer
// that has proximity order po or less as measured from the base
// if base is nil, kademlia base address is used
func (k *Kademlia) EachAddr(base []byte, o int, f func(OverlayAddr, int, bool) bool) {
k.lock.RLock()
defer k.lock.RUnlock()
k.eachAddr(base, o, f)
}
func (k *Kademlia) eachAddr(base []byte, o int, f func(OverlayAddr, int, bool) bool) {
if len(base) == 0 {
base = k.base
}
depth := k.neighbourhoodDepth()
k.addrs.EachNeighbour(base, pof, func(val pot.Val, po int) bool {
if po > o {
return true
}
return f(val.(*entry).addr(), po, po >= depth)
})
}
// neighbourhoodDepth returns the proximity order that defines the distance of
// the nearest neighbour set with cardinality >= MinProxBinSize
// if there is altogether less than MinProxBinSize peers it returns 0
// caller must hold the lock
func (k *Kademlia) neighbourhoodDepth() (depth int) {
if k.conns.Size() < k.MinProxBinSize {
return 0
}
var size int
f := func(v pot.Val, i int) bool {
size++
depth = i
return size < k.MinProxBinSize
}
k.conns.EachNeighbour(k.base, pof, f)
return depth
}
// callable when called with val,
func (k *Kademlia) callable(val pot.Val) OverlayAddr {
e := val.(*entry)
// not callable if peer is live or exceeded maxRetries
if e.conn() != nil || e.retries > k.MaxRetries {
return nil
}
// calculate the allowed number of retries based on time lapsed since last seen
timeAgo := int64(time.Since(e.seenAt))
div := int64(k.RetryExponent)
div += (150000 - rand.Int63n(300000)) * div / 1000000
var retries int
for delta := timeAgo; delta > k.RetryInterval; delta /= div {
retries++
}
// this is never called concurrently, so safe to increment
// peer can be retried again
if retries < e.retries {
log.Trace(fmt.Sprintf("%08x: %v long time since last try (at %v) needed before retry %v, wait only warrants %v", k.BaseAddr()[:4], e, timeAgo, e.retries, retries))
return nil
}
// function to sanction or prevent suggesting a peer
if k.Reachable != nil && !k.Reachable(e.addr()) {
log.Trace(fmt.Sprintf("%08x: peer %v is temporarily not callable", k.BaseAddr()[:4], e))
return nil
}
e.retries++
log.Trace(fmt.Sprintf("%08x: peer %v is callable", k.BaseAddr()[:4], e))
return e.addr()
}
// BaseAddr return the kademlia base address
func (k *Kademlia) BaseAddr() []byte {
return k.base
}
// String returns kademlia table + kaddb table displayed with ascii
func (k *Kademlia) String() string {
k.lock.RLock()
defer k.lock.RUnlock()
return k.string()
}
// String returns kademlia table + kaddb table displayed with ascii
func (k *Kademlia) string() string {
wsrow := " "
var rows []string
rows = append(rows, "=========================================================================")
rows = append(rows, fmt.Sprintf("%v KΛÐΞMLIΛ hive: queen's address: %x", time.Now().UTC().Format(time.UnixDate), k.BaseAddr()[:3]))
rows = append(rows, fmt.Sprintf("population: %d (%d), MinProxBinSize: %d, MinBinSize: %d, MaxBinSize: %d", k.conns.Size(), k.addrs.Size(), k.MinProxBinSize, k.MinBinSize, k.MaxBinSize))
liverows := make([]string, k.MaxProxDisplay)
peersrows := make([]string, k.MaxProxDisplay)
depth := k.neighbourhoodDepth()
rest := k.conns.Size()
k.conns.EachBin(k.base, pof, 0, func(po, size int, f func(func(val pot.Val, i int) bool) bool) bool {
var rowlen int
if po >= k.MaxProxDisplay {
po = k.MaxProxDisplay - 1
}
row := []string{fmt.Sprintf("%2d", size)}
rest -= size
f(func(val pot.Val, vpo int) bool {
e := val.(*entry)
row = append(row, fmt.Sprintf("%x", e.Address()[:2]))
rowlen++
return rowlen < 4
})
r := strings.Join(row, " ")
r = r + wsrow
liverows[po] = r[:31]
return true
})
k.addrs.EachBin(k.base, pof, 0, func(po, size int, f func(func(val pot.Val, i int) bool) bool) bool {
var rowlen int
if po >= k.MaxProxDisplay {
po = k.MaxProxDisplay - 1
}
if size < 0 {
panic("wtf")
}
row := []string{fmt.Sprintf("%2d", size)}
// we are displaying live peers too
f(func(val pot.Val, vpo int) bool {
e := val.(*entry)
row = append(row, Label(e))
rowlen++
return rowlen < 4
})
peersrows[po] = strings.Join(row, " ")
return true
})
for i := 0; i < k.MaxProxDisplay; i++ {
if i == depth {
rows = append(rows, fmt.Sprintf("============ DEPTH: %d ==========================================", i))
}
left := liverows[i]
right := peersrows[i]
if len(left) == 0 {
left = " 0 "
}
if len(right) == 0 {
right = " 0"
}
rows = append(rows, fmt.Sprintf("%03d %v | %v", i, left, right))
}
rows = append(rows, "=========================================================================")
return "\n" + strings.Join(rows, "\n")
}
// PeerPot keeps info about expected nearest neighbours and empty bins
// used for testing only
type PeerPot struct {
NNSet [][]byte
EmptyBins []int
}
// NewPeerPotMap creates a map of pot record of OverlayAddr with keys
// as hexadecimal representations of the address.
func NewPeerPotMap(kadMinProxSize int, addrs [][]byte) map[string]*PeerPot {
// create a table of all nodes for health check
np := pot.NewPot(nil, 0)
for _, addr := range addrs {
np, _, _ = pot.Add(np, addr, pof)
}
ppmap := make(map[string]*PeerPot)
for i, a := range addrs {
pl := 256
prev := 256
var emptyBins []int
var nns [][]byte
np.EachNeighbour(addrs[i], pof, func(val pot.Val, po int) bool {
a := val.([]byte)
if po == 256 {
return true
}
if pl == 256 || pl == po {
nns = append(nns, a)
}
if pl == 256 && len(nns) >= kadMinProxSize {
pl = po
prev = po
}
if prev < pl {
for j := prev; j > po; j-- {
emptyBins = append(emptyBins, j)
}
}
prev = po - 1
return true
})
for j := prev; j >= 0; j-- {
emptyBins = append(emptyBins, j)
}
log.Trace(fmt.Sprintf("%x NNS: %s", addrs[i][:4], LogAddrs(nns)))
ppmap[common.Bytes2Hex(a)] = &PeerPot{nns, emptyBins}
}
return ppmap
}
// saturation returns the lowest proximity order that the bin for that order
// has less than n peers
func (k *Kademlia) saturation(n int) int {
prev := -1
k.addrs.EachBin(k.base, pof, 0, func(po, size int, f func(func(val pot.Val, i int) bool) bool) bool {
prev++
return prev == po && size >= n
})
depth := k.neighbourhoodDepth()
if depth < prev {
return depth
}
return prev
}
// full returns true if all required bins have connected peers.
// It is used in Healthy function.
func (k *Kademlia) full(emptyBins []int) (full bool) {
prev := 0
e := len(emptyBins)
ok := true
depth := k.neighbourhoodDepth()
k.conns.EachBin(k.base, pof, 0, func(po, _ int, _ func(func(val pot.Val, i int) bool) bool) bool {
if prev == depth+1 {
return true
}
for i := prev; i < po; i++ {
e--
if e < 0 {
ok = false
return false
}
if emptyBins[e] != i {
log.Trace(fmt.Sprintf("%08x po: %d, i: %d, e: %d, emptybins: %v", k.BaseAddr()[:4], po, i, e, logEmptyBins(emptyBins)))
if emptyBins[e] < i {
panic("incorrect peerpot")
}
ok = false
return false
}
}
prev = po + 1
return true
})
if !ok {
return false
}
return e == 0
}
func (k *Kademlia) knowNearestNeighbours(peers [][]byte) bool {
pm := make(map[string]bool)
k.eachAddr(nil, 255, func(p OverlayAddr, po int, nn bool) bool {
if !nn {
return false
}
pk := fmt.Sprintf("%x", p.Address())
pm[pk] = true
return true
})
for _, p := range peers {
pk := fmt.Sprintf("%x", p)
if !pm[pk] {
log.Trace(fmt.Sprintf("%08x: known nearest neighbour %s not found", k.BaseAddr()[:4], pk[:8]))
return false
}
}
return true
}
func (k *Kademlia) gotNearestNeighbours(peers [][]byte) (got bool, n int, missing [][]byte) {
pm := make(map[string]bool)
k.eachConn(nil, 255, func(p OverlayConn, po int, nn bool) bool {
if !nn {
return false
}
pk := fmt.Sprintf("%x", p.Address())
pm[pk] = true
return true
})
var gots int
var culprits [][]byte
for _, p := range peers {
pk := fmt.Sprintf("%x", p)
if pm[pk] {
gots++
} else {
log.Trace(fmt.Sprintf("%08x: ExpNN: %s not found", k.BaseAddr()[:4], pk[:8]))
culprits = append(culprits, p)
}
}
return gots == len(peers), gots, culprits
}
// Health state of the Kademlia
type Health struct {
KnowNN bool // whether node knows all its nearest neighbours
GotNN bool // whether node is connected to all its nearest neighbours
CountNN int // amount of nearest neighbors connected to
CulpritsNN [][]byte // which known NNs are missing
Full bool // whether node has a peer in each kademlia bin (where there is such a peer)
Hive string
}
// Healthy reports the health state of the kademlia connectivity
// returns a Health struct
func (k *Kademlia) Healthy(pp *PeerPot) *Health {
k.lock.RLock()
defer k.lock.RUnlock()
gotnn, countnn, culpritsnn := k.gotNearestNeighbours(pp.NNSet)
knownn := k.knowNearestNeighbours(pp.NNSet)
full := k.full(pp.EmptyBins)
log.Trace(fmt.Sprintf("%08x: healthy: knowNNs: %v, gotNNs: %v, full: %v\n", k.BaseAddr()[:4], knownn, gotnn, full))
return &Health{knownn, gotnn, countnn, culpritsnn, full, k.string()}
}
func logEmptyBins(ebs []int) string {
var ebss []string
for _, eb := range ebs {
ebss = append(ebss, fmt.Sprintf("%d", eb))
}
return strings.Join(ebss, ", ")
}

View File

@@ -1,173 +0,0 @@
// Copyright 2016 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package kademlia
import (
"fmt"
"math/rand"
"strings"
"github.com/ethereum/go-ethereum/common"
)
type Address common.Hash
func (a Address) String() string {
return fmt.Sprintf("%x", a[:])
}
func (a *Address) MarshalJSON() (out []byte, err error) {
return []byte(`"` + a.String() + `"`), nil
}
func (a *Address) UnmarshalJSON(value []byte) error {
*a = Address(common.HexToHash(string(value[1 : len(value)-1])))
return nil
}
// the string form of the binary representation of an address (only first 8 bits)
func (a Address) Bin() string {
var bs []string
for _, b := range a[:] {
bs = append(bs, fmt.Sprintf("%08b", b))
}
return strings.Join(bs, "")
}
/*
Proximity(x, y) returns the proximity order of the MSB distance between x and y
The distance metric MSB(x, y) of two equal length byte sequences x and y is the
value of the binary integer cast of the x^y, ie., x and y bitwise xor-ed.
the binary cast is big endian: most significant bit first (=MSB).
Proximity(x, y) is a discrete logarithmic scaling of the MSB distance.
It is defined as the reverse rank of the integer part of the base 2
logarithm of the distance.
It is calculated by counting the number of common leading zeros in the (MSB)
binary representation of the x^y.
(0 farthest, 255 closest, 256 self)
*/
func proximity(one, other Address) (ret int) {
for i := 0; i < len(one); i++ {
oxo := one[i] ^ other[i]
for j := 0; j < 8; j++ {
if (oxo>>uint8(7-j))&0x01 != 0 {
return i*8 + j
}
}
}
return len(one) * 8
}
// Address.ProxCmp compares the distances a->target and b->target.
// Returns -1 if a is closer to target, 1 if b is closer to target
// and 0 if they are equal.
func (target Address) ProxCmp(a, b Address) int {
for i := range target {
da := a[i] ^ target[i]
db := b[i] ^ target[i]
if da > db {
return 1
} else if da < db {
return -1
}
}
return 0
}
// randomAddressAt(address, prox) generates a random address
// at proximity order prox relative to address
// if prox is negative a random address is generated
func RandomAddressAt(self Address, prox int) (addr Address) {
addr = self
var pos int
if prox >= 0 {
pos = prox / 8
trans := prox % 8
transbytea := byte(0)
for j := 0; j <= trans; j++ {
transbytea |= 1 << uint8(7-j)
}
flipbyte := byte(1 << uint8(7-trans))
transbyteb := transbytea ^ byte(255)
randbyte := byte(rand.Intn(255))
addr[pos] = ((addr[pos] & transbytea) ^ flipbyte) | randbyte&transbyteb
}
for i := pos + 1; i < len(addr); i++ {
addr[i] = byte(rand.Intn(255))
}
return
}
// KeyRange(a0, a1, proxLimit) returns the address inclusive address
// range that contain addresses closer to one than other
func KeyRange(one, other Address, proxLimit int) (start, stop Address) {
prox := proximity(one, other)
if prox >= proxLimit {
prox = proxLimit
}
start = CommonBitsAddrByte(one, other, byte(0x00), prox)
stop = CommonBitsAddrByte(one, other, byte(0xff), prox)
return
}
func CommonBitsAddrF(self, other Address, f func() byte, p int) (addr Address) {
prox := proximity(self, other)
var pos int
if p <= prox {
prox = p
}
pos = prox / 8
addr = self
trans := byte(prox % 8)
var transbytea byte
if p > prox {
transbytea = byte(0x7f)
} else {
transbytea = byte(0xff)
}
transbytea >>= trans
transbyteb := transbytea ^ byte(0xff)
addrpos := addr[pos]
addrpos &= transbyteb
if p > prox {
addrpos ^= byte(0x80 >> trans)
}
addrpos |= transbytea & f()
addr[pos] = addrpos
for i := pos + 1; i < len(addr); i++ {
addr[i] = f()
}
return
}
func CommonBitsAddr(self, other Address, prox int) (addr Address) {
return CommonBitsAddrF(self, other, func() byte { return byte(rand.Intn(255)) }, prox)
}
func CommonBitsAddrByte(self, other Address, b byte, prox int) (addr Address) {
return CommonBitsAddrF(self, other, func() byte { return b }, prox)
}
// randomAddressAt() generates a random address
func RandomAddress() Address {
return RandomAddressAt(Address{}, -1)
}

View File

@@ -1,96 +0,0 @@
// Copyright 2016 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package kademlia
import (
"math/rand"
"reflect"
"testing"
"github.com/ethereum/go-ethereum/common"
)
func (Address) Generate(rand *rand.Rand, size int) reflect.Value {
var id Address
for i := 0; i < len(id); i++ {
id[i] = byte(uint8(rand.Intn(255)))
}
return reflect.ValueOf(id)
}
func TestCommonBitsAddrF(t *testing.T) {
a := Address(common.HexToHash("0x0123456789abcdef0123456789abcdef0123456789abcdef0123456789abcdef"))
b := Address(common.HexToHash("0x8123456789abcdef0123456789abcdef0123456789abcdef0123456789abcdef"))
c := Address(common.HexToHash("0x4123456789abcdef0123456789abcdef0123456789abcdef0123456789abcdef"))
d := Address(common.HexToHash("0x0023456789abcdef0123456789abcdef0123456789abcdef0123456789abcdef"))
e := Address(common.HexToHash("0x01A3456789abcdef0123456789abcdef0123456789abcdef0123456789abcdef"))
ab := CommonBitsAddrF(a, b, func() byte { return byte(0x00) }, 10)
expab := Address(common.HexToHash("0x8000000000000000000000000000000000000000000000000000000000000000"))
if ab != expab {
t.Fatalf("%v != %v", ab, expab)
}
ac := CommonBitsAddrF(a, c, func() byte { return byte(0x00) }, 10)
expac := Address(common.HexToHash("0x4000000000000000000000000000000000000000000000000000000000000000"))
if ac != expac {
t.Fatalf("%v != %v", ac, expac)
}
ad := CommonBitsAddrF(a, d, func() byte { return byte(0x00) }, 10)
expad := Address(common.HexToHash("0x0000000000000000000000000000000000000000000000000000000000000000"))
if ad != expad {
t.Fatalf("%v != %v", ad, expad)
}
ae := CommonBitsAddrF(a, e, func() byte { return byte(0x00) }, 10)
expae := Address(common.HexToHash("0x0180000000000000000000000000000000000000000000000000000000000000"))
if ae != expae {
t.Fatalf("%v != %v", ae, expae)
}
acf := CommonBitsAddrF(a, c, func() byte { return byte(0xff) }, 10)
expacf := Address(common.HexToHash("0x7fffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff"))
if acf != expacf {
t.Fatalf("%v != %v", acf, expacf)
}
aeo := CommonBitsAddrF(a, e, func() byte { return byte(0x00) }, 2)
expaeo := Address(common.HexToHash("0x0000000000000000000000000000000000000000000000000000000000000000"))
if aeo != expaeo {
t.Fatalf("%v != %v", aeo, expaeo)
}
aep := CommonBitsAddrF(a, e, func() byte { return byte(0xff) }, 2)
expaep := Address(common.HexToHash("0x3fffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff"))
if aep != expaep {
t.Fatalf("%v != %v", aep, expaep)
}
}
func TestRandomAddressAt(t *testing.T) {
var a Address
for i := 0; i < 100; i++ {
a = RandomAddress()
prox := rand.Intn(255)
b := RandomAddressAt(a, prox)
if proximity(a, b) != prox {
t.Fatalf("incorrect address prox(%v, %v) == %v (expected %v)", a, b, proximity(a, b), prox)
}
}
}

View File

@@ -1,350 +0,0 @@
// Copyright 2016 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package kademlia
import (
"encoding/json"
"fmt"
"io/ioutil"
"os"
"sync"
"time"
"github.com/ethereum/go-ethereum/log"
)
type NodeData interface {
json.Marshaler
json.Unmarshaler
}
// allow inactive peers under
type NodeRecord struct {
Addr Address // address of node
Url string // Url, used to connect to node
After time.Time // next call after time
Seen time.Time // last connected at time
Meta *json.RawMessage // arbitrary metadata saved for a peer
node Node
}
func (self *NodeRecord) setSeen() {
t := time.Now()
self.Seen = t
self.After = t
}
func (self *NodeRecord) String() string {
return fmt.Sprintf("<%v>", self.Addr)
}
// persisted node record database ()
type KadDb struct {
Address Address
Nodes [][]*NodeRecord
index map[Address]*NodeRecord
cursors []int
lock sync.RWMutex
purgeInterval time.Duration
initialRetryInterval time.Duration
connRetryExp int
}
func newKadDb(addr Address, params *KadParams) *KadDb {
return &KadDb{
Address: addr,
Nodes: make([][]*NodeRecord, params.MaxProx+1), // overwritten by load
cursors: make([]int, params.MaxProx+1),
index: make(map[Address]*NodeRecord),
purgeInterval: params.PurgeInterval,
initialRetryInterval: params.InitialRetryInterval,
connRetryExp: params.ConnRetryExp,
}
}
func (self *KadDb) findOrCreate(index int, a Address, url string) *NodeRecord {
defer self.lock.Unlock()
self.lock.Lock()
record, found := self.index[a]
if !found {
record = &NodeRecord{
Addr: a,
Url: url,
}
log.Info(fmt.Sprintf("add new record %v to kaddb", record))
// insert in kaddb
self.index[a] = record
self.Nodes[index] = append(self.Nodes[index], record)
} else {
log.Info(fmt.Sprintf("found record %v in kaddb", record))
}
// update last seen time
record.setSeen()
// update with url in case IP/port changes
record.Url = url
return record
}
// add adds node records to kaddb (persisted node record db)
func (self *KadDb) add(nrs []*NodeRecord, proximityBin func(Address) int) {
defer self.lock.Unlock()
self.lock.Lock()
var n int
var nodes []*NodeRecord
for _, node := range nrs {
_, found := self.index[node.Addr]
if !found && node.Addr != self.Address {
node.setSeen()
self.index[node.Addr] = node
index := proximityBin(node.Addr)
dbcursor := self.cursors[index]
nodes = self.Nodes[index]
// this is inefficient for allocation, need to just append then shift
newnodes := make([]*NodeRecord, len(nodes)+1)
copy(newnodes[:], nodes[:dbcursor])
newnodes[dbcursor] = node
copy(newnodes[dbcursor+1:], nodes[dbcursor:])
log.Trace(fmt.Sprintf("new nodes: %v, nodes: %v", newnodes, nodes))
self.Nodes[index] = newnodes
n++
}
}
if n > 0 {
log.Debug(fmt.Sprintf("%d/%d node records (new/known)", n, len(nrs)))
}
}
/*
next return one node record with the highest priority for desired
connection.
This is used to pick candidates for live nodes that are most wanted for
a higly connected low centrality network structure for Swarm which best suits
for a Kademlia-style routing.
* Starting as naive node with empty db, this implements Kademlia bootstrapping
* As a mature node, it fills short lines. All on demand.
The candidate is chosen using the following strategy:
We check for missing online nodes in the buckets for 1 upto Max BucketSize rounds.
On each round we proceed from the low to high proximity order buckets.
If the number of active nodes (=connected peers) is < rounds, then start looking
for a known candidate. To determine if there is a candidate to recommend the
kaddb node record database row corresponding to the bucket is checked.
If the row cursor is on position i, the ith element in the row is chosen.
If the record is scheduled not to be retried before NOW, the next element is taken.
If the record is scheduled to be retried, it is set as checked, scheduled for
checking and is returned. The time of the next check is in X (duration) such that
X = ConnRetryExp * delta where delta is the time past since the last check and
ConnRetryExp is constant obsoletion factor. (Note that when node records are added
from peer messages, they are marked as checked and placed at the cursor, ie.
given priority over older entries). Entries which were checked more than
purgeInterval ago are deleted from the kaddb row. If no candidate is found after
a full round of checking the next bucket up is considered. If no candidate is
found when we reach the maximum-proximity bucket, the next round starts.
node record a is more favoured to b a > b iff a is a passive node (record of
offline past peer)
|proxBin(a)| < |proxBin(b)|
|| (proxBin(a) < proxBin(b) && |proxBin(a)| == |proxBin(b)|)
|| (proxBin(a) == proxBin(b) && lastChecked(a) < lastChecked(b))
The second argument returned names the first missing slot found
*/
func (self *KadDb) findBest(maxBinSize int, binSize func(int) int) (node *NodeRecord, need bool, proxLimit int) {
// return nil, proxLimit indicates that all buckets are filled
defer self.lock.Unlock()
self.lock.Lock()
var interval time.Duration
var found bool
var purge []bool
var delta time.Duration
var cursor int
var count int
var after time.Time
// iterate over columns maximum bucketsize times
for rounds := 1; rounds <= maxBinSize; rounds++ {
ROUND:
// iterate over rows from PO 0 upto MaxProx
for po, dbrow := range self.Nodes {
// if row has rounds connected peers, then take the next
if binSize(po) >= rounds {
continue ROUND
}
if !need {
// set proxlimit to the PO where the first missing slot is found
proxLimit = po
need = true
}
purge = make([]bool, len(dbrow))
// there is a missing slot - finding a node to connect to
// select a node record from the relavant kaddb row (of identical prox order)
ROW:
for cursor = self.cursors[po]; !found && count < len(dbrow); cursor = (cursor + 1) % len(dbrow) {
count++
node = dbrow[cursor]
// skip already connected nodes
if node.node != nil {
log.Debug(fmt.Sprintf("kaddb record %v (PO%03d:%d/%d) already connected", node.Addr, po, cursor, len(dbrow)))
continue ROW
}
// if node is scheduled to connect
if node.After.After(time.Now()) {
log.Debug(fmt.Sprintf("kaddb record %v (PO%03d:%d) skipped. seen at %v (%v ago), scheduled at %v", node.Addr, po, cursor, node.Seen, delta, node.After))
continue ROW
}
delta = time.Since(node.Seen)
if delta < self.initialRetryInterval {
delta = self.initialRetryInterval
}
if delta > self.purgeInterval {
// remove node
purge[cursor] = true
log.Debug(fmt.Sprintf("kaddb record %v (PO%03d:%d) unreachable since %v. Removed", node.Addr, po, cursor, node.Seen))
continue ROW
}
log.Debug(fmt.Sprintf("kaddb record %v (PO%03d:%d) ready to be tried. seen at %v (%v ago), scheduled at %v", node.Addr, po, cursor, node.Seen, delta, node.After))
// scheduling next check
interval = delta * time.Duration(self.connRetryExp)
after = time.Now().Add(interval)
log.Debug(fmt.Sprintf("kaddb record %v (PO%03d:%d) selected as candidate connection %v. seen at %v (%v ago), selectable since %v, retry after %v (in %v)", node.Addr, po, cursor, rounds, node.Seen, delta, node.After, after, interval))
node.After = after
found = true
} // ROW
self.cursors[po] = cursor
self.delete(po, purge)
if found {
return node, need, proxLimit
}
} // ROUND
} // ROUNDS
return nil, need, proxLimit
}
// deletes the noderecords of a kaddb row corresponding to the indexes
// caller must hold the dblock
// the call is unsafe, no index checks
func (self *KadDb) delete(row int, purge []bool) {
var nodes []*NodeRecord
dbrow := self.Nodes[row]
for i, del := range purge {
if i == self.cursors[row] {
//reset cursor
self.cursors[row] = len(nodes)
}
// delete the entry to be purged
if del {
delete(self.index, dbrow[i].Addr)
continue
}
// otherwise append to new list
nodes = append(nodes, dbrow[i])
}
self.Nodes[row] = nodes
}
// save persists kaddb on disk (written to file on path in json format.
func (self *KadDb) save(path string, cb func(*NodeRecord, Node)) error {
defer self.lock.Unlock()
self.lock.Lock()
var n int
for _, b := range self.Nodes {
for _, node := range b {
n++
node.After = time.Now()
node.Seen = time.Now()
if cb != nil {
cb(node, node.node)
}
}
}
data, err := json.MarshalIndent(self, "", " ")
if err != nil {
return err
}
err = ioutil.WriteFile(path, data, os.ModePerm)
if err != nil {
log.Warn(fmt.Sprintf("unable to save kaddb with %v nodes to %v: %v", n, path, err))
} else {
log.Info(fmt.Sprintf("saved kaddb with %v nodes to %v", n, path))
}
return err
}
// Load(path) loads the node record database (kaddb) from file on path.
func (self *KadDb) load(path string, cb func(*NodeRecord, Node) error) (err error) {
defer self.lock.Unlock()
self.lock.Lock()
var data []byte
data, err = ioutil.ReadFile(path)
if err != nil {
return
}
err = json.Unmarshal(data, self)
if err != nil {
return
}
var n int
var purge []bool
for po, b := range self.Nodes {
purge = make([]bool, len(b))
ROW:
for i, node := range b {
if cb != nil {
err = cb(node, node.node)
if err != nil {
purge[i] = true
continue ROW
}
}
n++
if node.After.IsZero() {
node.After = time.Now()
}
self.index[node.Addr] = node
}
self.delete(po, purge)
}
log.Info(fmt.Sprintf("loaded kaddb with %v nodes from %v", n, path))
return
}
// accessor for KAD offline db count
func (self *KadDb) count() int {
defer self.lock.Unlock()
self.lock.Lock()
return len(self.index)
}

View File

@@ -1,454 +0,0 @@
// Copyright 2016 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package kademlia
import (
"fmt"
"sort"
"strings"
"sync"
"time"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/metrics"
)
//metrics variables
//For metrics, we want to count how many times peers are added/removed
//at a certain index. Thus we do that with an array of counters with
//entry for each index
var (
bucketAddIndexCount []metrics.Counter
bucketRmIndexCount []metrics.Counter
)
const (
bucketSize = 4
proxBinSize = 2
maxProx = 8
connRetryExp = 2
maxPeers = 100
)
var (
purgeInterval = 42 * time.Hour
initialRetryInterval = 42 * time.Millisecond
maxIdleInterval = 42 * 1000 * time.Millisecond
// maxIdleInterval = 42 * 10 0 * time.Millisecond
)
type KadParams struct {
// adjustable parameters
MaxProx int
ProxBinSize int
BucketSize int
PurgeInterval time.Duration
InitialRetryInterval time.Duration
MaxIdleInterval time.Duration
ConnRetryExp int
}
func NewDefaultKadParams() *KadParams {
return &KadParams{
MaxProx: maxProx,
ProxBinSize: proxBinSize,
BucketSize: bucketSize,
PurgeInterval: purgeInterval,
InitialRetryInterval: initialRetryInterval,
MaxIdleInterval: maxIdleInterval,
ConnRetryExp: connRetryExp,
}
}
// Kademlia is a table of active nodes
type Kademlia struct {
addr Address // immutable baseaddress of the table
*KadParams // Kademlia configuration parameters
proxLimit int // state, the PO of the first row of the most proximate bin
proxSize int // state, the number of peers in the most proximate bin
count int // number of active peers (w live connection)
buckets [][]Node // the actual bins
db *KadDb // kaddb, node record database
lock sync.RWMutex // mutex to access buckets
}
type Node interface {
Addr() Address
Url() string
LastActive() time.Time
Drop()
}
// public constructor
// add is the base address of the table
// params is KadParams configuration
func New(addr Address, params *KadParams) *Kademlia {
buckets := make([][]Node, params.MaxProx+1)
kad := &Kademlia{
addr: addr,
KadParams: params,
buckets: buckets,
db: newKadDb(addr, params),
}
kad.initMetricsVariables()
return kad
}
// accessor for KAD base address
func (self *Kademlia) Addr() Address {
return self.addr
}
// accessor for KAD active node count
func (self *Kademlia) Count() int {
defer self.lock.Unlock()
self.lock.Lock()
return self.count
}
// accessor for KAD active node count
func (self *Kademlia) DBCount() int {
return self.db.count()
}
// On is the entry point called when a new nodes is added
// unsafe in that node is not checked to be already active node (to be called once)
func (self *Kademlia) On(node Node, cb func(*NodeRecord, Node) error) (err error) {
log.Debug(fmt.Sprintf("%v", self))
defer self.lock.Unlock()
self.lock.Lock()
index := self.proximityBin(node.Addr())
record := self.db.findOrCreate(index, node.Addr(), node.Url())
if cb != nil {
err = cb(record, node)
log.Trace(fmt.Sprintf("cb(%v, %v) ->%v", record, node, err))
if err != nil {
return fmt.Errorf("unable to add node %v, callback error: %v", node.Addr(), err)
}
log.Debug(fmt.Sprintf("add node record %v with node %v", record, node))
}
// insert in kademlia table of active nodes
bucket := self.buckets[index]
// if bucket is full insertion replaces the worst node
// TODO: give priority to peers with active traffic
if len(bucket) < self.BucketSize { // >= allows us to add peers beyond the bucketsize limitation
self.buckets[index] = append(bucket, node)
bucketAddIndexCount[index].Inc(1)
log.Debug(fmt.Sprintf("add node %v to table", node))
self.setProxLimit(index, true)
record.node = node
self.count++
return nil
}
// always rotate peers
idle := self.MaxIdleInterval
var pos int
var replaced Node
for i, p := range bucket {
idleInt := time.Since(p.LastActive())
if idleInt > idle {
idle = idleInt
pos = i
replaced = p
}
}
if replaced == nil {
log.Debug(fmt.Sprintf("all peers wanted, PO%03d bucket full", index))
return fmt.Errorf("bucket full")
}
log.Debug(fmt.Sprintf("node %v replaced by %v (idle for %v > %v)", replaced, node, idle, self.MaxIdleInterval))
replaced.Drop()
// actually replace in the row. When off(node) is called, the peer is no longer in the row
bucket[pos] = node
// there is no change in bucket cardinalities so no prox limit adjustment is needed
record.node = node
self.count++
return nil
}
// Off is the called when a node is taken offline (from the protocol main loop exit)
func (self *Kademlia) Off(node Node, cb func(*NodeRecord, Node)) (err error) {
self.lock.Lock()
defer self.lock.Unlock()
index := self.proximityBin(node.Addr())
bucketRmIndexCount[index].Inc(1)
bucket := self.buckets[index]
for i := 0; i < len(bucket); i++ {
if node.Addr() == bucket[i].Addr() {
self.buckets[index] = append(bucket[:i], bucket[(i+1):]...)
self.setProxLimit(index, false)
break
}
}
record := self.db.index[node.Addr()]
// callback on remove
if cb != nil {
cb(record, record.node)
}
record.node = nil
self.count--
log.Debug(fmt.Sprintf("remove node %v from table, population now is %v", node, self.count))
return
}
// proxLimit is dynamically adjusted so that
// 1) there is no empty buckets in bin < proxLimit and
// 2) the sum of all items are the minimum possible but higher than ProxBinSize
// adjust Prox (proxLimit and proxSize after an insertion/removal of nodes)
// caller holds the lock
func (self *Kademlia) setProxLimit(r int, on bool) {
// if the change is outside the core (PO lower)
// and the change does not leave a bucket empty then
// no adjustment needed
if r < self.proxLimit && len(self.buckets[r]) > 0 {
return
}
// if on=a node was added, then r must be within prox limit so increment cardinality
if on {
self.proxSize++
curr := len(self.buckets[self.proxLimit])
// if now core is big enough without the furthest bucket, then contract
// this can result in more than one bucket change
for self.proxSize >= self.ProxBinSize+curr && curr > 0 {
self.proxSize -= curr
self.proxLimit++
curr = len(self.buckets[self.proxLimit])
log.Trace(fmt.Sprintf("proxbin contraction (size: %v, limit: %v, bin: %v)", self.proxSize, self.proxLimit, r))
}
return
}
// otherwise
if r >= self.proxLimit {
self.proxSize--
}
// expand core by lowering prox limit until hit zero or cover the empty bucket or reached target cardinality
for (self.proxSize < self.ProxBinSize || r < self.proxLimit) &&
self.proxLimit > 0 {
//
self.proxLimit--
self.proxSize += len(self.buckets[self.proxLimit])
log.Trace(fmt.Sprintf("proxbin expansion (size: %v, limit: %v, bin: %v)", self.proxSize, self.proxLimit, r))
}
}
/*
returns the list of nodes belonging to the same proximity bin
as the target. The most proximate bin will be the union of the bins between
proxLimit and MaxProx.
*/
func (self *Kademlia) FindClosest(target Address, max int) []Node {
self.lock.Lock()
defer self.lock.Unlock()
r := nodesByDistance{
target: target,
}
po := self.proximityBin(target)
index := po
step := 1
log.Trace(fmt.Sprintf("serving %v nodes at %v (PO%02d)", max, index, po))
// if max is set to 0, just want a full bucket, dynamic number
min := max
// set limit to max
limit := max
if max == 0 {
min = 1
limit = maxPeers
}
var n int
for index >= 0 {
// add entire bucket
for _, p := range self.buckets[index] {
r.push(p, limit)
n++
}
// terminate if index reached the bottom or enough peers > min
log.Trace(fmt.Sprintf("add %v -> %v (PO%02d, PO%03d)", len(self.buckets[index]), n, index, po))
if n >= min && (step < 0 || max == 0) {
break
}
// reach top most non-empty PO bucket, turn around
if index == self.MaxProx {
index = po
step = -1
}
index += step
}
log.Trace(fmt.Sprintf("serve %d (<=%d) nodes for target lookup %v (PO%03d)", n, max, target, po))
return r.nodes
}
func (self *Kademlia) Suggest() (*NodeRecord, bool, int) {
defer self.lock.RUnlock()
self.lock.RLock()
return self.db.findBest(self.BucketSize, func(i int) int { return len(self.buckets[i]) })
}
// adds node records to kaddb (persisted node record db)
func (self *Kademlia) Add(nrs []*NodeRecord) {
self.db.add(nrs, self.proximityBin)
}
// nodesByDistance is a list of nodes, ordered by distance to target.
type nodesByDistance struct {
nodes []Node
target Address
}
func sortedByDistanceTo(target Address, slice []Node) bool {
var last Address
for i, node := range slice {
if i > 0 {
if target.ProxCmp(node.Addr(), last) < 0 {
return false
}
}
last = node.Addr()
}
return true
}
// push(node, max) adds the given node to the list, keeping the total size
// below max elements.
func (h *nodesByDistance) push(node Node, max int) {
// returns the firt index ix such that func(i) returns true
ix := sort.Search(len(h.nodes), func(i int) bool {
return h.target.ProxCmp(h.nodes[i].Addr(), node.Addr()) >= 0
})
if len(h.nodes) < max {
h.nodes = append(h.nodes, node)
}
if ix < len(h.nodes) {
copy(h.nodes[ix+1:], h.nodes[ix:])
h.nodes[ix] = node
}
}
/*
Taking the proximity order relative to a fix point x classifies the points in
the space (n byte long byte sequences) into bins. Items in each are at
most half as distant from x as items in the previous bin. Given a sample of
uniformly distributed items (a hash function over arbitrary sequence) the
proximity scale maps onto series of subsets with cardinalities on a negative
exponential scale.
It also has the property that any two item belonging to the same bin are at
most half as distant from each other as they are from x.
If we think of random sample of items in the bins as connections in a network of interconnected nodes than relative proximity can serve as the basis for local
decisions for graph traversal where the task is to find a route between two
points. Since in every hop, the finite distance halves, there is
a guaranteed constant maximum limit on the number of hops needed to reach one
node from the other.
*/
func (self *Kademlia) proximityBin(other Address) (ret int) {
ret = proximity(self.addr, other)
if ret > self.MaxProx {
ret = self.MaxProx
}
return
}
// provides keyrange for chunk db iteration
func (self *Kademlia) KeyRange(other Address) (start, stop Address) {
defer self.lock.RUnlock()
self.lock.RLock()
return KeyRange(self.addr, other, self.proxLimit)
}
// save persists kaddb on disk (written to file on path in json format.
func (self *Kademlia) Save(path string, cb func(*NodeRecord, Node)) error {
return self.db.save(path, cb)
}
// Load(path) loads the node record database (kaddb) from file on path.
func (self *Kademlia) Load(path string, cb func(*NodeRecord, Node) error) (err error) {
return self.db.load(path, cb)
}
// kademlia table + kaddb table displayed with ascii
func (self *Kademlia) String() string {
defer self.lock.RUnlock()
self.lock.RLock()
defer self.db.lock.RUnlock()
self.db.lock.RLock()
var rows []string
rows = append(rows, "=========================================================================")
rows = append(rows, fmt.Sprintf("%v KΛÐΞMLIΛ hive: queen's address: %v", time.Now().UTC().Format(time.UnixDate), self.addr.String()[:6]))
rows = append(rows, fmt.Sprintf("population: %d (%d), proxLimit: %d, proxSize: %d", self.count, len(self.db.index), self.proxLimit, self.proxSize))
rows = append(rows, fmt.Sprintf("MaxProx: %d, ProxBinSize: %d, BucketSize: %d", self.MaxProx, self.ProxBinSize, self.BucketSize))
for i, bucket := range self.buckets {
if i == self.proxLimit {
rows = append(rows, fmt.Sprintf("============ PROX LIMIT: %d ==========================================", i))
}
row := []string{fmt.Sprintf("%03d", i), fmt.Sprintf("%2d", len(bucket))}
var k int
c := self.db.cursors[i]
for ; k < len(bucket); k++ {
p := bucket[(c+k)%len(bucket)]
row = append(row, p.Addr().String()[:6])
if k == 4 {
break
}
}
for ; k < 4; k++ {
row = append(row, " ")
}
row = append(row, fmt.Sprintf("| %2d %2d", len(self.db.Nodes[i]), self.db.cursors[i]))
for j, p := range self.db.Nodes[i] {
row = append(row, p.Addr.String()[:6])
if j == 3 {
break
}
}
rows = append(rows, strings.Join(row, " "))
if i == self.MaxProx {
}
}
rows = append(rows, "=========================================================================")
return strings.Join(rows, "\n")
}
//We have to build up the array of counters for each index
func (self *Kademlia) initMetricsVariables() {
//create the arrays
bucketAddIndexCount = make([]metrics.Counter, self.MaxProx+1)
bucketRmIndexCount = make([]metrics.Counter, self.MaxProx+1)
//at each index create a metrics counter
for i := 0; i < (self.KadParams.MaxProx + 1); i++ {
bucketAddIndexCount[i] = metrics.NewRegisteredCounter(fmt.Sprintf("network.kademlia.bucket.add.%d.index", i), nil)
bucketRmIndexCount[i] = metrics.NewRegisteredCounter(fmt.Sprintf("network.kademlia.bucket.rm.%d.index", i), nil)
}
}

View File

@@ -1,392 +0,0 @@
// Copyright 2016 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package kademlia
import (
"fmt"
"math"
"math/rand"
"os"
"path/filepath"
"reflect"
"testing"
"testing/quick"
"time"
)
var (
quickrand = rand.New(rand.NewSource(time.Now().Unix()))
quickcfgFindClosest = &quick.Config{MaxCount: 50, Rand: quickrand}
quickcfgBootStrap = &quick.Config{MaxCount: 100, Rand: quickrand}
)
type testNode struct {
addr Address
}
func (n *testNode) String() string {
return fmt.Sprintf("%x", n.addr[:])
}
func (n *testNode) Addr() Address {
return n.addr
}
func (n *testNode) Drop() {
}
func (n *testNode) Url() string {
return ""
}
func (n *testNode) LastActive() time.Time {
return time.Now()
}
func TestOn(t *testing.T) {
addr, ok1 := gen(Address{}, quickrand).(Address)
other, ok2 := gen(Address{}, quickrand).(Address)
if !ok1 || !ok2 {
t.Errorf("oops")
}
kad := New(addr, NewDefaultKadParams())
err := kad.On(&testNode{addr: other}, nil)
_ = err
}
func TestBootstrap(t *testing.T) {
test := func(test *bootstrapTest) bool {
// for any node kad.le, Target and N
params := NewDefaultKadParams()
params.MaxProx = test.MaxProx
params.BucketSize = test.BucketSize
params.ProxBinSize = test.BucketSize
kad := New(test.Self, params)
var err error
for p := 0; p < 9; p++ {
var nrs []*NodeRecord
n := math.Pow(float64(2), float64(7-p))
for i := 0; i < int(n); i++ {
addr := RandomAddressAt(test.Self, p)
nrs = append(nrs, &NodeRecord{
Addr: addr,
})
}
kad.Add(nrs)
}
node := &testNode{test.Self}
n := 0
for n < 100 {
err = kad.On(node, nil)
if err != nil {
t.Fatalf("backend not accepting node: %v", err)
}
record, need, _ := kad.Suggest()
if !need {
break
}
n++
if record == nil {
continue
}
node = &testNode{record.Addr}
}
exp := test.BucketSize * (test.MaxProx + 1)
if kad.Count() != exp {
t.Errorf("incorrect number of peers, expected %d, got %d\n%v", exp, kad.Count(), kad)
return false
}
return true
}
if err := quick.Check(test, quickcfgBootStrap); err != nil {
t.Error(err)
}
}
func TestFindClosest(t *testing.T) {
test := func(test *FindClosestTest) bool {
// for any node kad.le, Target and N
params := NewDefaultKadParams()
params.MaxProx = 7
kad := New(test.Self, params)
var err error
for _, node := range test.All {
err = kad.On(node, nil)
if err != nil && err.Error() != "bucket full" {
t.Fatalf("backend not accepting node: %v", err)
}
}
if len(test.All) == 0 || test.N == 0 {
return true
}
nodes := kad.FindClosest(test.Target, test.N)
// check that the number of results is min(N, kad.len)
wantN := test.N
if tlen := kad.Count(); tlen < test.N {
wantN = tlen
}
if len(nodes) != wantN {
t.Errorf("wrong number of nodes: got %d, want %d", len(nodes), wantN)
return false
}
if hasDuplicates(nodes) {
t.Errorf("result contains duplicates")
return false
}
if !sortedByDistanceTo(test.Target, nodes) {
t.Errorf("result is not sorted by distance to target")
return false
}
// check that the result nodes have minimum distance to target.
farthestResult := nodes[len(nodes)-1].Addr()
for i, b := range kad.buckets {
for j, n := range b {
if contains(nodes, n.Addr()) {
continue // don't run the check below for nodes in result
}
if test.Target.ProxCmp(n.Addr(), farthestResult) < 0 {
_ = i * j
t.Errorf("kad.le contains node that is closer to target but it's not in result")
return false
}
}
}
return true
}
if err := quick.Check(test, quickcfgFindClosest); err != nil {
t.Error(err)
}
}
type proxTest struct {
add bool
index int
addr Address
}
var (
addresses []Address
)
func TestProxAdjust(t *testing.T) {
r := rand.New(rand.NewSource(time.Now().UnixNano()))
self := gen(Address{}, r).(Address)
params := NewDefaultKadParams()
params.MaxProx = 7
kad := New(self, params)
var err error
for i := 0; i < 100; i++ {
a := gen(Address{}, r).(Address)
addresses = append(addresses, a)
err = kad.On(&testNode{addr: a}, nil)
if err != nil && err.Error() != "bucket full" {
t.Fatalf("backend not accepting node: %v", err)
}
if !kad.proxCheck(t) {
return
}
}
test := func(test *proxTest) bool {
node := &testNode{test.addr}
if test.add {
kad.On(node, nil)
} else {
kad.Off(node, nil)
}
return kad.proxCheck(t)
}
if err := quick.Check(test, quickcfgFindClosest); err != nil {
t.Error(err)
}
}
func TestSaveLoad(t *testing.T) {
r := rand.New(rand.NewSource(time.Now().UnixNano()))
addresses := gen([]Address{}, r).([]Address)
self := RandomAddress()
params := NewDefaultKadParams()
params.MaxProx = 7
kad := New(self, params)
var err error
for _, a := range addresses {
err = kad.On(&testNode{addr: a}, nil)
if err != nil && err.Error() != "bucket full" {
t.Fatalf("backend not accepting node: %v", err)
}
}
nodes := kad.FindClosest(self, 100)
path := filepath.Join(os.TempDir(), "bzz-kad-test-save-load.peers")
err = kad.Save(path, nil)
if err != nil && err.Error() != "bucket full" {
t.Fatalf("unepected error saving kaddb: %v", err)
}
kad = New(self, params)
err = kad.Load(path, nil)
if err != nil && err.Error() != "bucket full" {
t.Fatalf("unepected error loading kaddb: %v", err)
}
for _, b := range kad.db.Nodes {
for _, node := range b {
err = kad.On(&testNode{node.Addr}, nil)
if err != nil && err.Error() != "bucket full" {
t.Fatalf("backend not accepting node: %v", err)
}
}
}
loadednodes := kad.FindClosest(self, 100)
for i, node := range loadednodes {
if nodes[i].Addr() != node.Addr() {
t.Errorf("node mismatch at %d/%d: %v != %v", i, len(nodes), nodes[i].Addr(), node.Addr())
}
}
}
func (self *Kademlia) proxCheck(t *testing.T) bool {
var sum int
for i, b := range self.buckets {
l := len(b)
// if we are in the high prox multibucket
if i >= self.proxLimit {
sum += l
} else if l == 0 {
t.Errorf("bucket %d empty, yet proxLimit is %d\n%v", len(b), self.proxLimit, self)
return false
}
}
// check if merged high prox bucket does not exceed size
if sum > 0 {
if sum != self.proxSize {
t.Errorf("proxSize incorrect, expected %v, got %v", sum, self.proxSize)
return false
}
last := len(self.buckets[self.proxLimit])
if last > 0 && sum >= self.ProxBinSize+last {
t.Errorf("proxLimit %v incorrect, redundant non-empty bucket %d added to proxBin with %v (target %v)\n%v", self.proxLimit, last, sum-last, self.ProxBinSize, self)
return false
}
if self.proxLimit > 0 && sum < self.ProxBinSize {
t.Errorf("proxLimit %v incorrect. proxSize %v is less than target %v, yet there is more peers", self.proxLimit, sum, self.ProxBinSize)
return false
}
}
return true
}
type bootstrapTest struct {
MaxProx int
BucketSize int
Self Address
}
func (*bootstrapTest) Generate(rand *rand.Rand, size int) reflect.Value {
t := &bootstrapTest{
Self: gen(Address{}, rand).(Address),
MaxProx: 5 + rand.Intn(2),
BucketSize: rand.Intn(3) + 1,
}
return reflect.ValueOf(t)
}
type FindClosestTest struct {
Self Address
Target Address
All []Node
N int
}
func (c FindClosestTest) String() string {
return fmt.Sprintf("A: %064x\nT: %064x\n(%d)\n", c.Self[:], c.Target[:], c.N)
}
func (*FindClosestTest) Generate(rand *rand.Rand, size int) reflect.Value {
t := &FindClosestTest{
Self: gen(Address{}, rand).(Address),
Target: gen(Address{}, rand).(Address),
N: rand.Intn(bucketSize),
}
for _, a := range gen([]Address{}, rand).([]Address) {
t.All = append(t.All, &testNode{addr: a})
}
return reflect.ValueOf(t)
}
func (*proxTest) Generate(rand *rand.Rand, size int) reflect.Value {
var add bool
if rand.Intn(1) == 0 {
add = true
}
var t *proxTest
if add {
t = &proxTest{
addr: gen(Address{}, rand).(Address),
add: add,
}
} else {
t = &proxTest{
index: rand.Intn(len(addresses)),
add: add,
}
}
return reflect.ValueOf(t)
}
func hasDuplicates(slice []Node) bool {
seen := make(map[Address]bool)
for _, node := range slice {
if seen[node.Addr()] {
return true
}
seen[node.Addr()] = true
}
return false
}
func contains(nodes []Node, addr Address) bool {
for _, n := range nodes {
if n.Addr() == addr {
return true
}
}
return false
}
// gen wraps quick.Value so it's easier to use.
// it generates a random value of the given value's type.
func gen(typ interface{}, rand *rand.Rand) interface{} {
v, ok := quick.Value(reflect.TypeOf(typ), rand)
if !ok {
panic(fmt.Sprintf("couldn't generate random value of type %T", typ))
}
return v.Interface()
}

View File

@@ -0,0 +1,623 @@
// Copyright 2017 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package network
import (
"bytes"
"fmt"
"os"
"testing"
"time"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/swarm/pot"
)
func init() {
h := log.LvlFilterHandler(log.LvlWarn, log.StreamHandler(os.Stderr, log.TerminalFormat(true)))
log.Root().SetHandler(h)
}
func testKadPeerAddr(s string) *BzzAddr {
a := pot.NewAddressFromString(s)
return &BzzAddr{OAddr: a, UAddr: a}
}
type testDropPeer struct {
Peer
dropc chan error
}
type dropError struct {
error
addr string
}
func (d *testDropPeer) Drop(err error) {
err2 := &dropError{err, binStr(d)}
d.dropc <- err2
}
type testKademlia struct {
*Kademlia
Discovery bool
dropc chan error
}
func newTestKademlia(b string) *testKademlia {
params := NewKadParams()
params.MinBinSize = 1
params.MinProxBinSize = 2
base := pot.NewAddressFromString(b)
return &testKademlia{
NewKademlia(base, params),
false,
make(chan error),
}
}
func (k *testKademlia) newTestKadPeer(s string) Peer {
return &testDropPeer{&BzzPeer{BzzAddr: testKadPeerAddr(s)}, k.dropc}
}
func (k *testKademlia) On(ons ...string) *testKademlia {
for _, s := range ons {
k.Kademlia.On(k.newTestKadPeer(s).(OverlayConn))
}
return k
}
func (k *testKademlia) Off(offs ...string) *testKademlia {
for _, s := range offs {
k.Kademlia.Off(k.newTestKadPeer(s).(OverlayConn))
}
return k
}
func (k *testKademlia) Register(regs ...string) *testKademlia {
var as []OverlayAddr
for _, s := range regs {
as = append(as, testKadPeerAddr(s))
}
err := k.Kademlia.Register(as)
if err != nil {
panic(err.Error())
}
return k
}
func testSuggestPeer(t *testing.T, k *testKademlia, expAddr string, expPo int, expWant bool) error {
addr, o, want := k.SuggestPeer()
if binStr(addr) != expAddr {
return fmt.Errorf("incorrect peer address suggested. expected %v, got %v", expAddr, binStr(addr))
}
if o != expPo {
return fmt.Errorf("incorrect prox order suggested. expected %v, got %v", expPo, o)
}
if want != expWant {
return fmt.Errorf("expected SuggestPeer to want peers: %v", expWant)
}
return nil
}
func binStr(a OverlayPeer) string {
if a == nil {
return "<nil>"
}
return pot.ToBin(a.Address())[:8]
}
func TestSuggestPeerBug(t *testing.T) {
// 2 row gap, unsaturated proxbin, no callables -> want PO 0
k := newTestKademlia("00000000").On(
"10000000", "11000000",
"01000000",
"00010000", "00011000",
).Off(
"01000000",
)
err := testSuggestPeer(t, k, "01000000", 0, false)
if err != nil {
t.Fatal(err.Error())
}
}
func TestSuggestPeerFindPeers(t *testing.T) {
// 2 row gap, unsaturated proxbin, no callables -> want PO 0
k := newTestKademlia("00000000").On("00100000")
err := testSuggestPeer(t, k, "<nil>", 0, false)
if err != nil {
t.Fatal(err.Error())
}
// 2 row gap, saturated proxbin, no callables -> want PO 0
k.On("00010000")
err = testSuggestPeer(t, k, "<nil>", 0, false)
if err != nil {
t.Fatal(err.Error())
}
// 1 row gap (1 less), saturated proxbin, no callables -> want PO 1
k.On("10000000")
err = testSuggestPeer(t, k, "<nil>", 1, false)
if err != nil {
t.Fatal(err.Error())
}
// no gap (1 less), saturated proxbin, no callables -> do not want more
k.On("01000000", "00100001")
err = testSuggestPeer(t, k, "<nil>", 0, false)
if err != nil {
t.Fatal(err.Error())
}
// oversaturated proxbin, > do not want more
k.On("00100001")
err = testSuggestPeer(t, k, "<nil>", 0, false)
if err != nil {
t.Fatal(err.Error())
}
// reintroduce gap, disconnected peer callable
// log.Info(k.String())
k.Off("01000000")
err = testSuggestPeer(t, k, "01000000", 0, false)
if err != nil {
t.Fatal(err.Error())
}
// second time disconnected peer not callable
// with reasonably set Interval
err = testSuggestPeer(t, k, "<nil>", 1, true)
if err != nil {
t.Fatal(err.Error())
}
// on and off again, peer callable again
k.On("01000000")
k.Off("01000000")
err = testSuggestPeer(t, k, "01000000", 0, false)
if err != nil {
t.Fatal(err.Error())
}
k.On("01000000")
// new closer peer appears, it is immediately wanted
k.Register("00010001")
err = testSuggestPeer(t, k, "00010001", 0, false)
if err != nil {
t.Fatal(err.Error())
}
// PO1 disconnects
k.On("00010001")
log.Info(k.String())
k.Off("01000000")
log.Info(k.String())
// second time, gap filling
err = testSuggestPeer(t, k, "01000000", 0, false)
if err != nil {
t.Fatal(err.Error())
}
k.On("01000000")
err = testSuggestPeer(t, k, "<nil>", 0, false)
if err != nil {
t.Fatal(err.Error())
}
k.MinBinSize = 2
err = testSuggestPeer(t, k, "<nil>", 0, true)
if err != nil {
t.Fatal(err.Error())
}
k.Register("01000001")
err = testSuggestPeer(t, k, "01000001", 0, false)
if err != nil {
t.Fatal(err.Error())
}
k.On("10000001")
log.Trace(fmt.Sprintf("Kad:\n%v", k.String()))
err = testSuggestPeer(t, k, "<nil>", 1, true)
if err != nil {
t.Fatal(err.Error())
}
k.On("01000001")
err = testSuggestPeer(t, k, "<nil>", 0, false)
if err != nil {
t.Fatal(err.Error())
}
k.MinBinSize = 3
k.Register("10000010")
err = testSuggestPeer(t, k, "10000010", 0, false)
if err != nil {
t.Fatal(err.Error())
}
k.On("10000010")
err = testSuggestPeer(t, k, "<nil>", 1, false)
if err != nil {
t.Fatal(err.Error())
}
k.On("01000010")
err = testSuggestPeer(t, k, "<nil>", 2, false)
if err != nil {
t.Fatal(err.Error())
}
k.On("00100010")
err = testSuggestPeer(t, k, "<nil>", 3, false)
if err != nil {
t.Fatal(err.Error())
}
k.On("00010010")
err = testSuggestPeer(t, k, "<nil>", 0, false)
if err != nil {
t.Fatal(err.Error())
}
}
func TestSuggestPeerRetries(t *testing.T) {
t.Skip("Test is disabled, because it is flaky. It fails with kademlia_test.go:346: incorrect peer address suggested. expected <nil>, got 01000000")
// 2 row gap, unsaturated proxbin, no callables -> want PO 0
k := newTestKademlia("00000000")
k.RetryInterval = int64(100 * time.Millisecond) // cycle
k.MaxRetries = 50
k.RetryExponent = 2
sleep := func(n int) {
ts := k.RetryInterval
for i := 1; i < n; i++ {
ts *= int64(k.RetryExponent)
}
time.Sleep(time.Duration(ts))
}
k.Register("01000000")
k.On("00000001", "00000010")
err := testSuggestPeer(t, k, "01000000", 0, false)
if err != nil {
t.Fatal(err.Error())
}
err = testSuggestPeer(t, k, "<nil>", 0, false)
if err != nil {
t.Fatal(err.Error())
}
sleep(1)
err = testSuggestPeer(t, k, "01000000", 0, false)
if err != nil {
t.Fatal(err.Error())
}
err = testSuggestPeer(t, k, "<nil>", 0, false)
if err != nil {
t.Fatal(err.Error())
}
sleep(1)
err = testSuggestPeer(t, k, "01000000", 0, false)
if err != nil {
t.Fatal(err.Error())
}
err = testSuggestPeer(t, k, "<nil>", 0, false)
if err != nil {
t.Fatal(err.Error())
}
sleep(2)
err = testSuggestPeer(t, k, "01000000", 0, false)
if err != nil {
t.Fatal(err.Error())
}
err = testSuggestPeer(t, k, "<nil>", 0, false)
if err != nil {
t.Fatal(err.Error())
}
sleep(2)
err = testSuggestPeer(t, k, "<nil>", 0, false)
if err != nil {
t.Fatal(err.Error())
}
}
func TestKademliaHiveString(t *testing.T) {
k := newTestKademlia("00000000").On("01000000", "00100000").Register("10000000", "10000001")
k.MaxProxDisplay = 8
h := k.String()
expH := "\n=========================================================================\nMon Feb 27 12:10:28 UTC 2017 KΛÐΞMLIΛ hive: queen's address: 000000\npopulation: 2 (4), MinProxBinSize: 2, MinBinSize: 1, MaxBinSize: 4\n000 0 | 2 8100 (0) 8000 (0)\n============ DEPTH: 1 ==========================================\n001 1 4000 | 1 4000 (0)\n002 1 2000 | 1 2000 (0)\n003 0 | 0\n004 0 | 0\n005 0 | 0\n006 0 | 0\n007 0 | 0\n========================================================================="
if expH[104:] != h[104:] {
t.Fatalf("incorrect hive output. expected %v, got %v", expH, h)
}
}
// testKademliaCase constructs the kademlia and PeerPot map to validate
// the SuggestPeer and Healthy methods for provided hex-encoded addresses.
// Argument pivotAddr is the address of the kademlia.
func testKademliaCase(t *testing.T, pivotAddr string, addrs ...string) {
addr := common.FromHex(pivotAddr)
addrs = append(addrs, pivotAddr)
k := NewKademlia(addr, NewKadParams())
as := make([][]byte, len(addrs))
for i, a := range addrs {
as[i] = common.FromHex(a)
}
for _, a := range as {
if bytes.Equal(a, addr) {
continue
}
p := &BzzAddr{OAddr: a, UAddr: a}
if err := k.Register([]OverlayAddr{p}); err != nil {
t.Fatal(err)
}
}
ppmap := NewPeerPotMap(2, as)
pp := ppmap[pivotAddr]
for {
a, _, _ := k.SuggestPeer()
if a == nil {
break
}
k.On(&BzzPeer{BzzAddr: a.(*BzzAddr)})
}
h := k.Healthy(pp)
if !(h.GotNN && h.KnowNN && h.Full) {
t.Error("not healthy")
}
}
/*
The regression test for the following invalid kademlia edge case.
Addresses used in this test are discovered as part of the simulation network
in higher level tests for streaming. They were generated randomly.
=========================================================================
Mon Apr 9 12:18:24 UTC 2018 KΛÐΞMLIΛ hive: queen's address: 7efef1
population: 9 (49), MinProxBinSize: 2, MinBinSize: 2, MaxBinSize: 4
000 2 d7e5 ec56 | 18 ec56 (0) d7e5 (0) d9e0 (0) c735 (0)
001 2 18f1 3176 | 14 18f1 (0) 10bb (0) 10d1 (0) 0421 (0)
002 2 52aa 47cd | 11 52aa (0) 51d9 (0) 5161 (0) 5130 (0)
003 1 646e | 1 646e (0)
004 0 | 3 769c (0) 76d1 (0) 7656 (0)
============ DEPTH: 5 ==========================================
005 1 7a48 | 1 7a48 (0)
006 1 7cbd | 1 7cbd (0)
007 0 | 0
008 0 | 0
009 0 | 0
010 0 | 0
011 0 | 0
012 0 | 0
013 0 | 0
014 0 | 0
015 0 | 0
=========================================================================
*/
func TestKademliaCase1(t *testing.T) {
testKademliaCase(t,
"7efef1c41d77f843ad167be95f6660567eb8a4a59f39240000cce2e0d65baf8e",
"ec560e6a4806aa37f147ee83687f3cf044d9953e61eedb8c34b6d50d9e2c5623",
"646e9540c84f6a2f9cf6585d45a4c219573b4fd1b64a3c9a1386fc5cf98c0d4d",
"18f13c5fba653781019025ab10e8d2fdc916d6448729268afe9e928ffcdbb8e8",
"317617acf99b4ffddda8a736f8fc6c6ede0bf690bc23d834123823e6d03e2f69",
"d7e52d9647a5d1c27a68c3ee65d543be3947ae4b68537b236d71ef9cb15fb9ab",
"7a48f75f8ca60487ae42d6f92b785581b40b91f2da551ae73d5eae46640e02e8",
"7cbd42350bde8e18ae5b955b5450f8e2cef3419f92fbf5598160c60fd78619f0",
"52aa3ddec61f4d48dd505a2385403c634f6ad06ee1d99c5c90a5ba6006f9af9c",
"47cdb6fa93eeb8bc91a417ff4e3b14a9c2ea85137462e2f575fae97f0c4be60d",
"5161943eb42e2a03e715fe8afa1009ff5200060c870ead6ab103f63f26cb107f",
"a38eaa1255f76bf883ca0830c86e8c4bb7eed259a8348aae9b03f21f90105bee",
"b2522bdf1ab26f324e75424fdf6e493b47e8a27687fe76347607b344fc010075",
"5bd7213964efb2580b91d02ac31ef126838abeba342f5dbdbe8d4d03562671a2",
"0b531adb82744768b694d7f94f73d4f0c9de591266108daeb8c74066bfc9c9ca",
"28501f59f70e888d399570145ed884353e017443c675aa12731ada7c87ea14f7",
"4a45f1fc63e1a9cb9dfa44c98da2f3d20c2923e5d75ff60b2db9d1bdb0c54d51",
"b193431ee35cd32de95805e7c1c749450c47486595aae7195ea6b6019a64fd61",
"baebf36a1e35a7ed834e1c72faf44ba16c159fa47d3289ceb3ca35fefa8739b5",
"a3659bd32e05fa36c8d20dbaaed8362bf1a8a7bd116aed62d8a43a2efbdf513f",
"10d1b50881a4770ebebdd0a75589dabb931e6716747b0f65fd6b080b88c4fdb6",
"3c76b8ca5c7ce6a03320646826213f59229626bf5b9d25da0c3ec0662dcb8ff3",
"4d72a04ddeb851a68cd197ef9a92a3e2ff01fbbff638e64929dd1a9c2e150112",
"c7353d320987956075b5bc1668571c7a36c800d5598fdc4832ec6569561e15d1",
"d9e0c7c90878c20ab7639d5954756f54775404b3483407fe1b483635182734f6",
"8fca67216b7939c0824fb06c5279901a94da41da9482b000f56df9906736ee75",
"460719d7f7aa7d7438f0eaf30333484fa3bd0f233632c10ba89e6e46dd3604be",
"0421d92c8a1c79ed5d01305a3d25aaf22a8f5f9e3d4bc80da47ee16ce20465fe",
"3441d9d9c0f05820a1bb6459fc7d8ef266a1bd929e7db939a10f544efe8261ea",
"ab198a66c293586746758468c610e5d3914d4ce629147eff6dd55a31f863ff8f",
"3a1c8c16b0763f3d2c35269f454ff779d1255e954d2deaf6c040fb3f0bcdc945",
"5561c0ea3b203e173b11e6aa9d0e621a4e10b1d8b178b8fe375220806557b823",
"7656caccdc79cd8d7ce66d415cc96a718e8271c62fb35746bfc2b49faf3eebf3",
"5130594fd54c1652cf2debde2c4204573ed76555d1e26757fe345b409af1544a",
"76d1e83c71ca246d042e37ff1db181f2776265fbcfdc890ce230bfa617c9c2f0",
"89580231962624c53968c1b0095b4a2732b2a2640a19fdd7d21fd064fcc0a5ef",
"3d10d001fff44680c7417dd66ecf2e984f0baa20a9bbcea348583ba5ff210c4f",
"43754e323f0f3a1155b1852bd6edd55da86b8c4cfe3df8b33733fca50fc202b8",
"a9e7b1bb763ae6452ddcacd174993f82977d81a85206bb2ae3c842e2d8e19b4c",
"10bb07da7bc7c7757f74149eff167d528a94a253cdc694a863f4d50054c00b6d",
"28f0bc1b44658548d6e05dd16d4c2fe77f1da5d48b6774bc4263b045725d0c19",
"835fbbf1d16ba7347b6e2fc552d6e982148d29c624ea20383850df3c810fa8fc",
"8e236c56a77d7f46e41e80f7092b1a68cd8e92f6156365f41813ad1ca2c6b6f3",
"51d9c857e9238c49186e37b4eccf17a82de3d5739f026f6043798ab531456e73",
"bbddf7db6a682225301f36a9fd5b0d0121d2951753e1681295f3465352ad511f",
"2690a910c33ee37b91eb6c4e0731d1d345e2dc3b46d308503a6e85bbc242c69e",
"769ce86aa90b518b7ed382f9fdacfbed93574e18dc98fe6c342e4f9f409c2d5a",
"ba3bebec689ce51d3e12776c45f80d25164fdfb694a8122d908081aaa2e7122c",
"3a51f4146ea90a815d0d283d1ceb20b928d8b4d45875e892696986a3c0d8fb9b",
"81968a2d8fb39114342ee1da85254ec51e0608d7f0f6997c2a8354c260a71009",
)
}
/*
The regression test for the following invalid kademlia edge case.
Addresses used in this test are discovered as part of the simulation network
in higher level tests for streaming. They were generated randomly.
=========================================================================
Mon Apr 9 18:43:48 UTC 2018 KΛÐΞMLIΛ hive: queen's address: bc7f3b
population: 9 (49), MinProxBinSize: 2, MinBinSize: 2, MaxBinSize: 4
000 2 0f49 67ff | 28 0f49 (0) 0211 (0) 07b2 (0) 0703 (0)
001 2 e84b f3a4 | 13 f3a4 (0) e84b (0) e58b (0) e60b (0)
002 1 8dba | 1 8dba (0)
003 2 a008 ad72 | 2 ad72 (0) a008 (0)
004 0 | 3 b61f (0) b27f (0) b027 (0)
============ DEPTH: 5 ==========================================
005 1 ba19 | 1 ba19 (0)
006 0 | 0
007 1 bdd6 | 1 bdd6 (0)
008 0 | 0
009 0 | 0
010 0 | 0
011 0 | 0
012 0 | 0
013 0 | 0
014 0 | 0
015 0 | 0
=========================================================================
*/
func TestKademliaCase2(t *testing.T) {
testKademliaCase(t,
"bc7f3b6a4a7e3c91b100ca6680b6c06ff407972b88956324ca853295893e0237", "67ffb61d3aa27449d277016188f35f19e2321fbda5008c68cf6303faa080534f", "600cd54c842eadac1729c04abfc369bc244572ca76117105b9dd910283b82730", "d955a05409650de151218557425105a8aa2867bb6a0e0462fa1cf90abcf87ad6", "7a6b726de45abdf7bb3e5fd9fb0dc8932270ca4dedef92238c80c05bcdb570e3", "263e99424ebfdb652adb4e3dcd27d59e11bb7ae1c057b3ef6f390d0228006254", "ba195d1a53aafde68e661c64d39db8c2a73505bf336125c15c3560de3b48b7ed", "3458c762169937115f67cabc35a6c384ed70293a8aec37b077a6c1b8e02d510e", "4ef4dc2e28ac6efdba57e134ac24dd4e0be68b9d54f7006515eb9509105f700c", "2a8782b79b0c24b9714dfd2c8ff1932bebc08aa6520b4eaeaa59ff781238890c", "625d02e960506f4524e9cdeac85b33faf3ea437fceadbd478b62b78720cf24fc", "e051a36a8c8637f520ba259c9ed3fadaf740dadc6a04c3f0e21778ebd4cd6ac4", "e34bc014fa2504f707bb3d904872b56c2fa250bee3cb19a147a0418541f1bd90", "28036dc79add95799916893890add5d8972f3b95325a509d6ded3d448f4dc652", "1b013c407794fa2e4c955d8f51cbc6bd78588a174b6548246b291281304b5409", "34f71b68698e1534095ff23ee9c35bf64c7f12b8463e7c6f6b19c25cf03928b4", "c712c6e9bbb7076832972a95890e340b94ed735935c3c0bb788e61f011b59479", "a008d5becdcda4b9dbfdaafc3cec586cf61dcf2d4b713b6168fff02e3b9f0b08", "29de15555cdbebaab214009e416ee92f947dcec5dab9894129f50f1b17138f34", "5df9449f700bd4b5a23688b68b293f2e92fa6ca524c93bc6bb9936efba9d9ada", "3ab0168a5f87fedc6a39b53c628256ac87a98670d8691bbdaaecec22418d13a2", "1ee299b2d2a74a568494130e6869e66d57982d345c482a0e0eeb285ac219ae3b", "e0e0e3b860cea9b7a74cf1b0675cc632dc64e80a02f20bbc5e96e2e8bb670606", "dc1ba6f169b0fcdcca021dcebaf39fe5d4875e7e69b854fad65687c1d7719ec0", "d321f73e42fcfb1d3a303eddf018ca5dffdcfd5567cd5ec1212f045f6a07e47d", "070320c3da7b542e5ca8aaf6a0a53d2bb5113ed264ab1db2dceee17c729edcb1", "17d314d65fdd136b50d182d2c8f5edf16e7838c2be8cf2c00abe4b406dbcd1d8", "e60b99e0a06f7d2d99d84085f67cdf8cc22a9ae22c339365d80f90289834a2b4", "02115771e18932e1f67a45f11f5bf743c5dae97fbc477d34d35c996012420eac", "3102a40eb2e5060353dd19bf61eeec8782dd1bebfcb57f4c796912252b591827", "8dbaf231062f2dc7ddaba5f9c7761b0c21292be51bf8c2ef503f31d4a2f63f79", "b02787b713c83a9f9183216310f04251994e04c2763a9024731562e8978e7cc4", "b27fe6cd33989e10909ce794c4b0b88feae286b614a59d49a3444c1a7b51ea82", "07b2d2c94fdc6fd148fe23be2ed9eff54f5e12548f29ed8416e6860fc894466f", "e58bf9f451ef62ac44ff0a9bb0610ec0fd14d423235954f0d3695e83017cbfc4", "bdd600b91bb79d1ee0053b854de308cfaa7e2abce575ea6815a0a7b3449609c2", "0f49c93c1edc7999920b21977cedd51a763940dac32e319feb9c1df2da0f3071", "7cbf0297cd41acf655cd6f960d7aaf61479edb4189d5c001cbc730861f0deb41", "79265193778d87ad626a5f59397bc075872d7302a12634ce2451a767d0a82da2", "2fe7d705f7c370b9243dbaafe007d555ff58d218822fca49d347b12a0282457c", "e84bc0c83d05e55a0080eed41dda5a795da4b9313a4da697142e69a65834cbb3", "cc4d278bd9aa0e9fb3cd8d2e0d68fb791aab5de4b120b845c409effbed47a180", "1a2317a8646cd4b6d3c4aa4cc25f676533abb689cf180787db216880a1239ad8", "cbafd6568cf8e99076208e6b6843f5808a7087897c67aad0c54694669398f889", "7b7c8357255fc37b4dae0e1af61589035fd39ff627e0938c6b3da8b4e4ec5d23", "2b8d782c1f5bac46c922cf439f6aa79f91e9ba5ffc0020d58455188a2075b334", "b61f45af2306705740742e76197a119235584ced01ef3f7cf3d4370f6c557cd1", "2775612e7cdae2780bf494c370bdcbe69c55e4a1363b1dc79ea0135e61221cce", "f3a49bb22f40885e961299abfa697a7df690a79f067bf3a4847a3ad48d826c9f", "ad724ac218dc133c0aadf4618eae21fdd0c2f3787af279846b49e2b4f97ff167",
)
}
/*
The regression test for the following invalid kademlia edge case.
Addresses used in this test are discovered as part of the simulation network
in higher level tests for streaming. They were generated randomly.
=========================================================================
Mon Apr 9 19:04:35 UTC 2018 KΛÐΞMLIΛ hive: queen's address: b4822e
population: 8 (49), MinProxBinSize: 2, MinBinSize: 2, MaxBinSize: 4
000 2 786c 774b | 29 774b (0) 786c (0) 7a79 (0) 7d2f (0)
001 2 d9de cf19 | 10 cf19 (0) d9de (0) d2ff (0) d2a2 (0)
002 2 8ca1 8d74 | 5 8d74 (0) 8ca1 (0) 9793 (0) 9f51 (0)
003 0 | 0
004 0 | 3 bfac (0) bcbb (0) bde9 (0)
005 0 | 0
============ DEPTH: 6 ==========================================
006 1 b660 | 1 b660 (0)
007 0 | 0
008 1 b450 | 1 b450 (0)
009 0 | 0
010 0 | 0
011 0 | 0
012 0 | 0
013 0 | 0
014 0 | 0
015 0 | 0
=========================================================================
*/
func TestKademliaCase3(t *testing.T) {
testKademliaCase(t,
"b4822e874a01b94ac3a35c821e6db131e785c2fcbb3556e84b36102caf09b091", "2ecf54ea38d58f9cfc3862e54e5854a7c506fbc640e0b38e46d7d45a19794999", "442374092be50fc7392e8dd3f6fab3158ff7f14f26ff98060aed9b2eecf0b97d", "b450a4a67fcfa3b976cf023d8f1f15052b727f712198ce901630efe2f95db191", "9a7291638eb1c989a6dd6661a42c735b23ac6605b5d3e428aa5ffe650e892c85", "67f62eeab9804cfcac02b25ebeab9113d1b9d03dd5200b1c5a324cc0163e722f", "2e4a0e4b53bca4a9d7e2734150e9f579f29a255ade18a268461b20d026c9ee90", "30dd79c5fcdaa1b106f6960c45c9fde7c046aa3d931088d98c52ab759d0b2ac4", "97936fb5a581e59753c54fa5feec493714f2218245f61f97a62eafd4699433e4", "3a2899b6e129e3e193f6e2aefb82589c948c246d2ec1d4272af32ef3b2660f44", "f0e2a8aa88e67269e9952431ef12e5b29b7f41a1871fbfc38567fad95655d607", "7fa12b3f3c5f8383bfc644b958f72a486969733fa097d8952b3eb4f7b4f73192", "360c167aad5fc992656d6010ec45fdce5bcd492ad9608bc515e2be70d4e430c1", "fe21bc969b3d8e5a64a6484a829c1e04208f26f3cd4de6afcbc172a5bd17f1f1", "b660a1f40141d7ccd282fe5bd9838744119bd1cb3780498b5173578cc5ad308f", "44dcb3370e76680e2fba8cd986ad45ff0b77ca45680ee8d950e47922c4af6226", "8ca126923d17fccb689647307b89f38aa14e2a7b9ebcf3c1e31ccf3d2291a3bc", "f0ae19ae9ce6329327cbf42baf090e084c196b0877d8c7b69997e0123be23ef8", "d2a2a217385158e3e1e348883a14bc423e57daa12077e8c49797d16121ea0810", "f5467ccd85bb4ebe768527db520a210459969a5f1fae6e07b43f519799f0b224", "68be5fd9f9d142a5099e3609011fe3bab7bb992c595999e31e0b3d1668dfb3cf", "4d49a8a476e4934afc6b5c36db9bece3ed1804f20b952da5a21b2b0de766aa73", "ea7155745ef3fb2d099513887a2ba279333ced65c65facbd890ce58bd3fce772", "cf19f51f4e848053d289ac95a9138cdd23fc3077ae913cd58cda8cc7a521b2e1", "590b1cd41c7e6144e76b5cd515a3a4d0a4317624620a3f1685f43ae68bdcd890", "d2ffe0626b5f94a7e00fa0b506e7455a3d9399c15800db108d5e715ef5f6e346", "69630878c50a91f6c2edd23a706bfa0b50bd5661672a37d67bab38e6bca3b698", "445e9067079899bb5faafaca915ae6c0f6b1b730a5a628835dd827636f7feb1e", "6461c77491f1c4825958949f23c153e6e1759a5be53abbcee17c9da3867f3141", "23a235f4083771ccc207771daceda700b525a59ab586788d4f6892e69e34a6e2", "bde99f79ef41a81607ddcf92b9f95dcbc6c3537e91e8bf740e193dc73b19485e", "177957c0e5f0fbd12b88022a91768095d193830986caec8d888097d3ff4310b8", "bcbbdbaa4cdf8352422072f332e05111b732354a35c4d7c617ce1fc3b8b42a5a", "774b6717fdfb0d1629fb9d4c04a9ca40079ae2955d7f82e897477055ed017abb", "16443bf625be6d39ecaa6f114e5d2c1d47a64bfd3c13808d94b55b6b6acef2ee", "8d7495d9008066505ed00ce8198af82bfa5a6b4c08768b4c9fb3aa4eb0b0cca2", "15800849a53349508cb382959527f6c3cf1a46158ff1e6e2316b7dea7967e35f", "7a792f0f4a2b731781d1b244b2a57947f1a2e32900a1c0793449f9f7ae18a7b7", "5e517c2832c9deaa7df77c7bad4d20fd6eda2b7815e155e68bc48238fac1416f", "9f51a14f0019c72bd1d472706d8c80a18c1873c6a0663e754b60eae8094483d7", "7d2fabb565122521d22ba99fed9e5be6a458fbc93156d54db27d97a00b8c3a97", "786c9e412a7db4ec278891fa534caa9a1d1a028c631c6f3aeb9c4d96ad895c36", "3bd6341d40641c2632a5a0cd7a63553a04e251efd7195897a1d27e02a7a8bfde", "31efd1f5fb57b8cff0318d77a1a9e8d67e1d1c8d18ce90f99c3a240dff48cdc8", "d9de3e1156ce1380150948acbcfecd99c96e7f4b0bc97745f4681593d017f74f", "427a2201e09f9583cd990c03b81b58148c297d474a3b50f498d83b1c7a9414cd", "bfaca11596d3dec406a9fcf5d97536516dfe7f0e3b12078428a7e1700e25218a", "351c4770a097248a650008152d0cab5825d048bef770da7f3364f59d1e721bc0", "ee00f205d1486b2be7381d962bd2867263758e880529e4e2bfedfa613bbc0e71", "6aa3b6418d89e3348e4859c823ef4d6d7cd46aa7f7e77aba586c4214d760d8f8",
)
}
/*
The regression test for the following invalid kademlia edge case.
Addresses used in this test are discovered as part of the simulation network
in higher level tests for streaming. They were generated randomly.
=========================================================================
Mon Apr 9 19:16:25 UTC 2018 KΛÐΞMLIΛ hive: queen's address: 9a90fe
population: 8 (49), MinProxBinSize: 2, MinBinSize: 2, MaxBinSize: 4
000 2 72ef 4e6c | 24 0b1e (0) 0d66 (0) 17f5 (0) 17e8 (0)
001 2 fc2b fa47 | 13 fa47 (0) fc2b (0) fffd (0) ecef (0)
002 2 b847 afa8 | 6 afa8 (0) ad77 (0) bb7c (0) b847 (0)
003 0 | 0
004 0 | 4 91fc (0) 957d (0) 9482 (0) 949a (0)
============ DEPTH: 5 ==========================================
005 1 9ccf | 1 9ccf (0)
006 0 | 0
007 1 9bb2 | 1 9bb2 (0)
008 0 | 0
009 0 | 0
010 0 | 0
011 0 | 0
012 0 | 0
013 0 | 0
014 0 | 0
015 0 | 0
=========================================================================
*/
func TestKademliaCase4(t *testing.T) {
testKademliaCase(t,
"9a90fe3506277244549064b8c3276abb06284a199d9063a97331947f2b7da7f4",
"c19359eddef24b7be1a833b4475f212cd944263627a53f9ef4837d106c247730", "fc2b6fef99ef947f7e57c3df376891769e2a2fd83d2b8e634e0fc1e91eaa080c", "ecefc0e1a8ea7bb4b48c469e077401fce175dd75294255b96c4e54f6a2950a55", "bb7ce598efc056bba343cc2614aa3f67a575557561290b44c73a63f8f433f9f7", "55fbee6ca52dfd7f0be0db969ee8e524b654ab4f0cce7c05d83887d7d2a15460", "afa852b6b319998c6a283cc0c82d2f5b8e9410075d7700f3012761f1cfbd0f76", "36c370cfb63f2087971ba6e58d7585b04e16b8f0da335efb91554c2dd8fe191c", "6be41e029985edebc901fb77fc4fb65516b6d85086e2a98bfa3159c99391e585", "dd3cfc72ea553e7d2b28f0037a65646b30955b929d29ba4c40f4a2a811248e77", "da3a8f18e09c7b0ca235c4e33e1441a5188f1df023138bf207753ee63e768f7d", "de9e3ab4dc572d54a2d4b878329fd832bb51a149f4ce167316eeb177b61e7e01", "4e6c1ecde6ed917706257fe020a1d02d2e9d87fca4c85f0f7b132491008c5032", "72ef04b77a070e13463b3529dd312bcacfb7a12d20dc597f5ec3de0501e9b834", "3fef57186675d524ab8bb1f54ba8cb68610babca1247c0c46dbb60aed003c69d", "1d8e6b71f7a052865d6558d4ba44ad5fab7b908cc1badf5766822e1c20d0d823", "6be2f2b4ffa173014d4ec7df157d289744a2bda54bb876b264ccfa898a0da315", "b0ba3fff8643f9985c744327b0c4c869763509fd5da2de9a80a4a0a082021255", "9ccf40b9406ba2e6567101fb9b4e5334a9ec74263eff47267da266ba45e6c158", "d7347f02c180a448e60f73931845062ce00048750b584790278e9c93ef31ad81", "b68c6359a22b3bee6fecb8804311cfd816648ea31d530c9fb48e477e029d707a", "0d668a18ad7c2820214df6df95a6c855ce19fb1cb765f8ca620e45db76686d37", "3fbd2663bff65533246f1fabb9f38086854c6218aeb3dc9ac6ac73d4f0988f91", "949aa5719ca846052bfaa1b38c97b6eca3df3e24c0e0630042c6bccafbb4cdb5", "77b8a2b917bef5d54f3792183b014cca7798f713ff14fe0b2ac79b4c9f6f996d", "17e853cbd8dc00cba3cd9ffeb36f26a9f41a0eb92f80b62c2cda16771c935388", "5f682ed7a8cf2f98387c3def7c97f9f05ae39e39d393eeca3cf621268d6347f8", "ad77487eaf11fd8084ba4517a51766eb0e5b77dd3492dfa79aa3a2802fb29d20", "d247cfcacf9a8200ebaddf639f8c926ab0a001abe682f40df3785e80ed124e91", "195589442e11907eede1ee6524157f1125f68399f3170c835ff81c603b069f6c", "5b5ca0a67f3c54e7d3a6a862ef56168ec9ed1f4945e6c24de6d336b2be2e6f8c", "56430e4caa253015f1f998dce4a48a88af1953f68e94eca14f53074ae9c3e467", "0b1eed6a5bf612d1d8e08f5c546f3d12e838568fd3aa43ed4c537f10c65545d6", "7058db19a56dfff01988ac4a62e1310597f9c8d7ebde6890dadabf047d722d39", "b847380d6888ff7cd11402d086b19eccc40950b52c9d67e73cb4f8462f5df078", "df6c048419a2290ab546d527e9eeba349e7f7e1759bafe4adac507ce60ef9670", "91fc5b4b24fc3fbfea7f9a3d0f0437cb5733c0c2345d8bdffd7048d6e3b8a37b", "957d8ea51b37523952b6f5ae95462fcd4aed1483ef32cc80b69580aaeee03606", "efa82e4e91ad9ab781977400e9ac0bb9de7389aaedebdae979b73d1d3b8d72b0", "7400c9f3f3fc0cc6fe8cc37ab24b9771f44e9f78be913f73cd35fc4be030d6bd", "9bb28f4122d61f7bb56fe27ef706159fb802fef0f5de9dfa32c9c5b3183235f1", "40a8de6e98953498b806614532ea4abf8b99ad7f9719fb68203a6eae2efa5b2a", "412de0b218b8f7dcacc9205cd16ffb4eca5b838f46a2f4f9f534026061a47308", "17f56ecad51075080680ad9faa0fd8946b824d3296ddb20be07f9809fe8d1c5a", "fffd4e7ae885a41948a342b6647955a7ec8a8039039f510cff467ef597675457", "35e78e11b5ac46a29dd04ab0043136c3291f4ca56cb949ace33111ed56395463", "94824fc80230af82077c83bfc01dc9675b1f9d3d538b1e5f41c21ac753598691", "fa470ae314ca3fce493f21b423eef2a49522e09126f6f2326fa3c9cac0b344f7", "7078860b5b621b21ac7b95f9fc4739c8235ce5066a8b9bd7d938146a34fa88ec", "eea53560f0428bfd2eca4f86a5ce9dec5ff1309129a975d73465c1c9e9da71d1",
)
}
/*
The regression test for the following invalid kademlia edge case.
Addresses used in this test are discovered as part of the simulation network
in higher level tests for streaming. They were generated randomly.
=========================================================================
Mon Apr 9 19:25:18 UTC 2018 KΛÐΞMLIΛ hive: queen's address: 5dd5c7
population: 13 (49), MinProxBinSize: 2, MinBinSize: 2, MaxBinSize: 4
000 2 e528 fad0 | 22 fad0 (0) e528 (0) e3bb (0) ed13 (0)
001 3 3f30 18e0 1dd3 | 7 3f30 (0) 23db (0) 10b6 (0) 18e0 (0)
002 4 7c54 7804 61e4 60f9 | 10 61e4 (0) 60f9 (0) 636c (0) 7186 (0)
003 2 40ae 4bae | 5 4bae (0) 4d5c (0) 403a (0) 40ae (0)
004 0 | 0
005 0 | 3 5808 (0) 5a0e (0) 5bdb (0)
============ DEPTH: 6 ==========================================
006 2 5f14 5f61 | 2 5f14 (0) 5f61 (0)
007 0 | 0
008 0 | 0
009 0 | 0
010 0 | 0
011 0 | 0
012 0 | 0
013 0 | 0
014 0 | 0
015 0 | 0
=========================================================================
*/
func TestKademliaCase5(t *testing.T) {
testKademliaCase(t,
"5dd5c77dd9006a800478fcebb02d48d4036389e7d3c8f6a83b97dbad13f4c0a9",
"78fafa0809929a1279ece089a51d12457c2d8416dff859aeb2ccc24bb50df5ec", "1dd39b1257e745f147cbbc3cadd609ccd6207c41056dbc4254bba5d2527d3ee5", "5f61dd66d4d94aec8fcc3ce0e7885c7edf30c43143fa730e2841c5d28e3cd081", "8aa8b0472cb351d967e575ad05c4b9f393e76c4b01ef4b3a54aac5283b78abc9", "4502f385152a915b438a6726ce3ea9342e7a6db91a23c2f6bee83a885ed7eb82", "718677a504249db47525e959ef1784bed167e1c46f1e0275b9c7b588e28a3758", "7c54c6ed1f8376323896ed3a4e048866410de189e9599dd89bf312ca4adb96b5", "18e03bd3378126c09e799a497150da5c24c895aedc84b6f0dbae41fc4bac081a", "23db76ac9e6e58d9f5395ca78252513a7b4118b4155f8462d3d5eec62486cadc", "40ae0e8f065e96c7adb7fa39505136401f01780481e678d718b7f6dbb2c906ec", "c1539998b8bae19d339d6bbb691f4e9daeb0e86847545229e80fe0dffe716e92", "ed139d73a2699e205574c08722ca9f030ad2d866c662f1112a276b91421c3cb9", "5bdb19584b7a36d09ca689422ef7e6bb681b8f2558a6b2177a8f7c812f631022", "636c9de7fe234ffc15d67a504c69702c719f626c17461d3f2918e924cd9d69e2", "de4455413ff9335c440d52458c6544191bd58a16d85f700c1de53b62773064ea", "de1963310849527acabc7885b6e345a56406a8f23e35e436b6d9725e69a79a83", "a80a50a467f561210a114cba6c7fb1489ed43a14d61a9edd70e2eb15c31f074d", "7804f12b8d8e6e4b375b242058242068a3809385e05df0e64973cde805cf729c", "60f9aa320c02c6f2e6370aa740cf7cea38083fa95fca8c99552cda52935c1520", "d8da963602390f6c002c00ce62a84b514edfce9ebde035b277a957264bb54d21", "8463d93256e026fe436abad44697152b9a56ac8e06a0583d318e9571b83d073c", "9a3f78fcefb9a05e40a23de55f6153d7a8b9d973ede43a380bf46bb3b3847de1", "e3bb576f4b3760b9ca6bff59326f4ebfc4a669d263fb7d67ab9797adea54ed13", "4d5cdbd6dcca5bdf819a0fe8d175dc55cc96f088d37462acd5ea14bc6296bdbe", "5a0ed28de7b5258c727cb85447071c74c00a5fbba9e6bc0393bc51944d04ab2a", "61e4ddb479c283c638f4edec24353b6cc7a3a13b930824aad016b0996ca93c47", "7e3610868acf714836cafaaa7b8c009a9ac6e3a6d443e5586cf661530a204ee2", "d74b244d4345d2c86e30a097105e4fb133d53c578320285132a952cdaa64416e", "cfeed57d0f935bfab89e3f630a7c97e0b1605f0724d85a008bbfb92cb47863a8", "580837af95055670e20d494978f60c7f1458dc4b9e389fc7aa4982b2aca3bce3", "df55c0c49e6c8a83d82dfa1c307d3bf6a20e18721c80d8ec4f1f68dc0a137ced", "5f149c51ce581ba32a285439a806c063ced01ccd4211cd024e6a615b8f216f95", "1eb76b00aeb127b10dd1b7cd4c3edeb4d812b5a658f0feb13e85c4d2b7c6fe06", "7a56ba7c3fb7cbfb5561a46a75d95d7722096b45771ec16e6fa7bbfab0b35dfe", "4bae85ad88c28470f0015246d530adc0cd1778bdd5145c3c6b538ee50c4e04bd", "afd1892e2a7145c99ec0ebe9ded0d3fec21089b277a68d47f45961ec5e39e7e0", "953138885d7b36b0ef79e46030f8e61fd7037fbe5ce9e0a94d728e8c8d7eab86", "de761613ef305e4f628cb6bf97d7b7dc69a9d513dc233630792de97bcda777a6", "3f3087280063d09504c084bbf7fdf984347a72b50d097fd5b086ffabb5b3fb4c", "7d18a94bb1ebfdef4d3e454d2db8cb772f30ca57920dd1e402184a9e598581a0", "a7d6fbdc9126d9f10d10617f49fb9f5474ffe1b229f76b7dd27cebba30eccb5d", "fad0246303618353d1387ec10c09ee991eb6180697ed3470ed9a6b377695203d", "1cf66e09ea51ee5c23df26615a9e7420be2ac8063f28f60a3bc86020e94fe6f3", "8269cdaa153da7c358b0b940791af74d7c651cd4d3f5ed13acfe6d0f2c539e7f", "90d52eaaa60e74bf1c79106113f2599471a902d7b1c39ac1f55b20604f453c09", "9788fd0c09190a3f3d0541f68073a2f44c2fcc45bb97558a7c319f36c25a75b3", "10b68fc44157ecfdae238ee6c1ce0333f906ad04d1a4cb1505c8e35c3c87fbb0", "e5284117fdf3757920475c786e0004cb00ba0932163659a89b36651a01e57394", "403ad51d911e113dcd5f9ff58c94f6d278886a2a4da64c3ceca2083282c92de3",
)
}

View File

@@ -1,308 +0,0 @@
// Copyright 2016 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package network
import (
"fmt"
"net"
"time"
"github.com/ethereum/go-ethereum/contracts/chequebook"
"github.com/ethereum/go-ethereum/p2p/discover"
"github.com/ethereum/go-ethereum/swarm/network/kademlia"
"github.com/ethereum/go-ethereum/swarm/services/swap"
"github.com/ethereum/go-ethereum/swarm/storage"
)
/*
BZZ protocol Message Types and Message Data Types
*/
// bzz protocol message codes
const (
statusMsg = iota // 0x01
storeRequestMsg // 0x02
retrieveRequestMsg // 0x03
peersMsg // 0x04
syncRequestMsg // 0x05
deliveryRequestMsg // 0x06
unsyncedKeysMsg // 0x07
paymentMsg // 0x08
)
/*
Handshake
* Version: 8 byte integer version of the protocol
* ID: arbitrary byte sequence client identifier human readable
* Addr: the address advertised by the node, format similar to DEVp2p wire protocol
* Swap: info for the swarm accounting protocol
* NetworkID: 8 byte integer network identifier
* Caps: swarm-specific capabilities, format identical to devp2p
* SyncState: syncronisation state (db iterator key and address space etc) persisted about the peer
*/
type statusMsgData struct {
Version uint64
ID string
Addr *peerAddr
Swap *swap.SwapProfile
NetworkId uint64
}
func (self *statusMsgData) String() string {
return fmt.Sprintf("Status: Version: %v, ID: %v, Addr: %v, Swap: %v, NetworkId: %v", self.Version, self.ID, self.Addr, self.Swap, self.NetworkId)
}
/*
store requests are forwarded to the peers in their kademlia proximity bin
if they are distant
if they are within our storage radius or have any incentive to store it
then attach your nodeID to the metadata
if the storage request is sufficiently close (within our proxLimit, i. e., the
last row of the routing table)
*/
type storeRequestMsgData struct {
Key storage.Key // hash of datasize | data
SData []byte // the actual chunk Data
// optional
Id uint64 // request ID. if delivery, the ID is retrieve request ID
requestTimeout *time.Time // expiry for forwarding - [not serialised][not currently used]
storageTimeout *time.Time // expiry of content - [not serialised][not currently used]
from *peer // [not serialised] protocol registers the requester
}
func (self storeRequestMsgData) String() string {
var from string
if self.from == nil {
from = "self"
} else {
from = self.from.Addr().String()
}
end := len(self.SData)
if len(self.SData) > 10 {
end = 10
}
return fmt.Sprintf("from: %v, Key: %v; ID: %v, requestTimeout: %v, storageTimeout: %v, SData %x", from, self.Key, self.Id, self.requestTimeout, self.storageTimeout, self.SData[:end])
}
/*
Retrieve request
Timeout in milliseconds. Note that zero timeout retrieval requests do not request forwarding, but prompt for a peers message response. therefore they serve also
as messages to retrieve peers.
MaxSize specifies the maximum size that the peer will accept. This is useful in
particular if we allow storage and delivery of multichunk payload representing
the entire or partial subtree unfolding from the requested root key.
So when only interested in limited part of a stream (infinite trees) or only
testing chunk availability etc etc, we can indicate it by limiting the size here.
Request ID can be newly generated or kept from the request originator.
If request ID Is missing or zero, the request is handled as a lookup only
prompting a peers response but not launching a search. Lookup requests are meant
to be used to bootstrap kademlia tables.
In the special case that the key is the zero value as well, the remote peer's
address is assumed (the message is to be handled as a self lookup request).
The response is a PeersMsg with the peers in the kademlia proximity bin
corresponding to the address.
*/
type retrieveRequestMsgData struct {
Key storage.Key // target Key address of chunk to be retrieved
Id uint64 // request id, request is a lookup if missing or zero
MaxSize uint64 // maximum size of delivery accepted
MaxPeers uint64 // maximum number of peers returned
Timeout uint64 // the longest time we are expecting a response
timeout *time.Time // [not serialied]
from *peer //
}
func (self *retrieveRequestMsgData) String() string {
var from string
if self.from == nil {
from = "ourselves"
} else {
from = self.from.Addr().String()
}
var target []byte
if len(self.Key) > 3 {
target = self.Key[:4]
}
return fmt.Sprintf("from: %v, Key: %x; ID: %v, MaxSize: %v, MaxPeers: %d", from, target, self.Id, self.MaxSize, self.MaxPeers)
}
// lookups are encoded by missing request ID
func (self *retrieveRequestMsgData) isLookup() bool {
return self.Id == 0
}
// sets timeout fields
func (self *retrieveRequestMsgData) setTimeout(t *time.Time) {
self.timeout = t
if t != nil {
self.Timeout = uint64(t.UnixNano())
} else {
self.Timeout = 0
}
}
func (self *retrieveRequestMsgData) getTimeout() (t *time.Time) {
if self.Timeout > 0 && self.timeout == nil {
timeout := time.Unix(int64(self.Timeout), 0)
t = &timeout
self.timeout = t
}
return
}
// peerAddr is sent in StatusMsg as part of the handshake
type peerAddr struct {
IP net.IP
Port uint16
ID []byte // the 64 byte NodeID (ECDSA Public Key)
Addr kademlia.Address
}
// peerAddr pretty prints as enode
func (self *peerAddr) String() string {
var nodeid discover.NodeID
copy(nodeid[:], self.ID)
return discover.NewNode(nodeid, self.IP, 0, self.Port).String()
}
/*
peers Msg is one response to retrieval; it is always encouraged after a retrieval
request to respond with a list of peers in the same kademlia proximity bin.
The encoding of a peer is identical to that in the devp2p base protocol peers
messages: [IP, Port, NodeID]
note that a node's DPA address is not the NodeID but the hash of the NodeID.
Timeout serves to indicate whether the responder is forwarding the query within
the timeout or not.
NodeID serves as the owner of payment contracts and signer of proofs of transfer.
The Key is the target (if response to a retrieval request) or missing (zero value)
peers address (hash of NodeID) if retrieval request was a self lookup.
Peers message is requested by retrieval requests with a missing or zero value request ID
*/
type peersMsgData struct {
Peers []*peerAddr //
Timeout uint64 //
timeout *time.Time // indicate whether responder is expected to deliver content
Key storage.Key // present if a response to a retrieval request
Id uint64 // present if a response to a retrieval request
from *peer
}
// peers msg pretty printer
func (self *peersMsgData) String() string {
var from string
if self.from == nil {
from = "ourselves"
} else {
from = self.from.Addr().String()
}
var target []byte
if len(self.Key) > 3 {
target = self.Key[:4]
}
return fmt.Sprintf("from: %v, Key: %x; ID: %v, Peers: %v", from, target, self.Id, self.Peers)
}
func (self *peersMsgData) setTimeout(t *time.Time) {
self.timeout = t
if t != nil {
self.Timeout = uint64(t.UnixNano())
} else {
self.Timeout = 0
}
}
/*
syncRequest
is sent after the handshake to initiate syncing
the syncState of the remote node is persisted in kaddb and set on the
peer/protocol instance when the node is registered by hive as online{
*/
type syncRequestMsgData struct {
SyncState *syncState `rlp:"nil"`
}
func (self *syncRequestMsgData) String() string {
return fmt.Sprintf("%v", self.SyncState)
}
/*
deliveryRequest
is sent once a batch of sync keys is filtered. The ones not found are
sent as a list of syncReuest (hash, priority) in the Deliver field.
When the source receives the sync request it continues to iterate
and fetch at most N items as yet unsynced.
At the same time responds with deliveries of the items.
*/
type deliveryRequestMsgData struct {
Deliver []*syncRequest
}
func (self *deliveryRequestMsgData) String() string {
return fmt.Sprintf("sync request for new chunks\ndelivery request for %v chunks", len(self.Deliver))
}
/*
unsyncedKeys
is sent first after the handshake if SyncState iterator brings up hundreds, thousands?
and subsequently sent as a response to deliveryRequestMsgData.
Syncing is the iterative process of exchanging unsyncedKeys and deliveryRequestMsgs
both ways.
State contains the sync state sent by the source. When the source receives the
sync state it continues to iterate and fetch at most N items as yet unsynced.
At the same time responds with deliveries of the items.
*/
type unsyncedKeysMsgData struct {
Unsynced []*syncRequest
State *syncState
}
func (self *unsyncedKeysMsgData) String() string {
return fmt.Sprintf("sync: keys of %d new chunks (state %v) => synced: %v", len(self.Unsynced), self.State, self.State.Synced)
}
/*
payment
is sent when the swap balance is tilted in favour of the remote peer
and in absolute units exceeds the PayAt parameter in the remote peer's profile
*/
type paymentMsgData struct {
Units uint // units actually paid for (checked against amount by swap)
Promise *chequebook.Cheque // payment with cheque
}
func (self *paymentMsgData) String() string {
return fmt.Sprintf("payment for %d units: %v", self.Units, self.Promise)
}

View File

@@ -0,0 +1,111 @@
// Copyright 2018 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
// package priority_queue implement a channel based priority queue
// over arbitrary types. It provides an
// an autopop loop applying a function to the items always respecting
// their priority. The structure is only quasi consistent ie., if a lower
// priority item is autopopped, it is guaranteed that there was a point
// when no higher priority item was present, ie. it is not guaranteed
// that there was any point where the lower priority item was present
// but the higher was not
package priorityqueue
import (
"context"
"errors"
)
var (
errContention = errors.New("queue contention")
errBadPriority = errors.New("bad priority")
wakey = struct{}{}
)
// PriorityQueue is the basic structure
type PriorityQueue struct {
queues []chan interface{}
wakeup chan struct{}
}
// New is the constructor for PriorityQueue
func New(n int, l int) *PriorityQueue {
var queues = make([]chan interface{}, n)
for i := range queues {
queues[i] = make(chan interface{}, l)
}
return &PriorityQueue{
queues: queues,
wakeup: make(chan struct{}, 1),
}
}
// Run is a forever loop popping items from the queues
func (pq *PriorityQueue) Run(ctx context.Context, f func(interface{})) {
top := len(pq.queues) - 1
p := top
READ:
for {
q := pq.queues[p]
select {
case <-ctx.Done():
return
case x := <-q:
f(x)
p = top
default:
if p > 0 {
p--
continue READ
}
p = top
select {
case <-ctx.Done():
return
case <-pq.wakeup:
}
}
}
}
// Push pushes an item to the appropriate queue specified in the priority argument
// if context is given it waits until either the item is pushed or the Context aborts
// otherwise returns errContention if the queue is full
func (pq *PriorityQueue) Push(ctx context.Context, x interface{}, p int) error {
if p < 0 || p >= len(pq.queues) {
return errBadPriority
}
if ctx == nil {
select {
case pq.queues[p] <- x:
default:
return errContention
}
} else {
select {
case pq.queues[p] <- x:
case <-ctx.Done():
return ctx.Err()
}
}
select {
case pq.wakeup <- wakey:
default:
}
return nil
}

View File

@@ -0,0 +1,97 @@
// Copyright 2018 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package priorityqueue
import (
"context"
"sync"
"testing"
)
func TestPriorityQueue(t *testing.T) {
var results []string
wg := sync.WaitGroup{}
pq := New(3, 2)
wg.Add(1)
go pq.Run(context.Background(), func(v interface{}) {
results = append(results, v.(string))
wg.Done()
})
pq.Push(context.Background(), "2.0", 2)
wg.Wait()
if results[0] != "2.0" {
t.Errorf("expected first result %q, got %q", "2.0", results[0])
}
Loop:
for i, tc := range []struct {
priorities []int
values []string
results []string
errors []error
}{
{
priorities: []int{0},
values: []string{""},
results: []string{""},
},
{
priorities: []int{0, 1},
values: []string{"0.0", "1.0"},
results: []string{"1.0", "0.0"},
},
{
priorities: []int{1, 0},
values: []string{"1.0", "0.0"},
results: []string{"1.0", "0.0"},
},
{
priorities: []int{0, 1, 1},
values: []string{"0.0", "1.0", "1.1"},
results: []string{"1.0", "1.1", "0.0"},
},
{
priorities: []int{0, 0, 0},
values: []string{"0.0", "0.0", "0.1"},
errors: []error{nil, nil, errContention},
},
} {
var results []string
wg := sync.WaitGroup{}
pq := New(3, 2)
wg.Add(len(tc.values))
for j, value := range tc.values {
err := pq.Push(nil, value, tc.priorities[j])
if tc.errors != nil && err != tc.errors[j] {
t.Errorf("expected push error %v, got %v", tc.errors[j], err)
continue Loop
}
if err != nil {
continue Loop
}
}
go pq.Run(context.Background(), func(v interface{}) {
results = append(results, v.(string))
wg.Done()
})
wg.Wait()
for k, result := range tc.results {
if results[k] != result {
t.Errorf("test case %v: expected %v element %q, got %q", i, k, result, results[k])
}
}
}
}

View File

@@ -16,519 +16,414 @@
package network
/*
bzz implements the swarm wire protocol [bzz] (sister of eth and shh)
the protocol instance is launched on each peer by the network layer if the
bzz protocol handler is registered on the p2p server.
The bzz protocol component speaks the bzz protocol
* handle the protocol handshake
* register peers in the KΛÐΞMLIΛ table via the hive logistic manager
* dispatch to hive for handling the DHT logic
* encode and decode requests for storage and retrieval
* handle sync protocol messages via the syncer
* talks the SWAP payment protocol (swap accounting is done within NetStore)
*/
import (
"context"
"errors"
"fmt"
"net"
"strconv"
"sync"
"time"
"github.com/ethereum/go-ethereum/contracts/chequebook"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/metrics"
"github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/p2p"
bzzswap "github.com/ethereum/go-ethereum/swarm/services/swap"
"github.com/ethereum/go-ethereum/swarm/services/swap/swap"
"github.com/ethereum/go-ethereum/swarm/storage"
)
//metrics variables
var (
storeRequestMsgCounter = metrics.NewRegisteredCounter("network.protocol.msg.storerequest.count", nil)
retrieveRequestMsgCounter = metrics.NewRegisteredCounter("network.protocol.msg.retrieverequest.count", nil)
peersMsgCounter = metrics.NewRegisteredCounter("network.protocol.msg.peers.count", nil)
syncRequestMsgCounter = metrics.NewRegisteredCounter("network.protocol.msg.syncrequest.count", nil)
unsyncedKeysMsgCounter = metrics.NewRegisteredCounter("network.protocol.msg.unsyncedkeys.count", nil)
deliverRequestMsgCounter = metrics.NewRegisteredCounter("network.protocol.msg.deliverrequest.count", nil)
paymentMsgCounter = metrics.NewRegisteredCounter("network.protocol.msg.payment.count", nil)
invalidMsgCounter = metrics.NewRegisteredCounter("network.protocol.msg.invalid.count", nil)
handleStatusMsgCounter = metrics.NewRegisteredCounter("network.protocol.msg.handlestatus.count", nil)
"github.com/ethereum/go-ethereum/p2p/discover"
"github.com/ethereum/go-ethereum/p2p/protocols"
"github.com/ethereum/go-ethereum/rpc"
"github.com/ethereum/go-ethereum/swarm/log"
"github.com/ethereum/go-ethereum/swarm/state"
)
const (
Version = 0
ProtocolLength = uint64(8)
DefaultNetworkID = 3
// ProtocolMaxMsgSize maximum allowed message size
ProtocolMaxMsgSize = 10 * 1024 * 1024
NetworkId = 3
// timeout for waiting
bzzHandshakeTimeout = 3000 * time.Millisecond
)
// bzz represents the swarm wire protocol
// an instance is running on each peer
type bzz struct {
storage StorageHandler // handler storage/retrieval related requests coming via the bzz wire protocol
hive *Hive // the logistic manager, peerPool, routing service and peer handler
dbAccess *DbAccess // access to db storage counter and iterator for syncing
requestDb *storage.LDBDatabase // db to persist backlog of deliveries to aid syncing
remoteAddr *peerAddr // remote peers address
peer *p2p.Peer // the p2p peer object
rw p2p.MsgReadWriter // messageReadWriter to send messages to
backend chequebook.Backend
lastActive time.Time
NetworkId uint64
swap *swap.Swap // swap instance for the peer connection
swapParams *bzzswap.SwapParams // swap settings both local and remote
swapEnabled bool // flag to enable SWAP (will be set via Caps in handshake)
syncEnabled bool // flag to enable SYNC (will be set via Caps in handshake)
syncer *syncer // syncer instance for the peer connection
syncParams *SyncParams // syncer params
syncState *syncState // outgoing syncronisation state (contains reference to remote peers db counter)
// BzzSpec is the spec of the generic swarm handshake
var BzzSpec = &protocols.Spec{
Name: "bzz",
Version: 4,
MaxMsgSize: 10 * 1024 * 1024,
Messages: []interface{}{
HandshakeMsg{},
},
}
// interface type for handler of storage/retrieval related requests coming
// via the bzz wire protocol
// messages: UnsyncedKeys, DeliveryRequest, StoreRequest, RetrieveRequest
type StorageHandler interface {
HandleUnsyncedKeysMsg(req *unsyncedKeysMsgData, p *peer) error
HandleDeliveryRequestMsg(req *deliveryRequestMsgData, p *peer) error
HandleStoreRequestMsg(req *storeRequestMsgData, p *peer)
HandleRetrieveRequestMsg(req *retrieveRequestMsgData, p *peer)
// DiscoverySpec is the spec for the bzz discovery subprotocols
var DiscoverySpec = &protocols.Spec{
Name: "hive",
Version: 4,
MaxMsgSize: 10 * 1024 * 1024,
Messages: []interface{}{
peersMsg{},
subPeersMsg{},
},
}
/*
main entrypoint, wrappers starting a server that will run the bzz protocol
use this constructor to attach the protocol ("class") to server caps
This is done by node.Node#Register(func(node.ServiceContext) (Service, error))
Service implements Protocols() which is an array of protocol constructors
at node startup the protocols are initialised
the Dev p2p layer then calls Run(p *p2p.Peer, rw p2p.MsgReadWriter) error
on each peer connection
The Run function of the Bzz protocol class creates a bzz instance
which will represent the peer for the swarm hive and all peer-aware components
*/
func Bzz(cloud StorageHandler, backend chequebook.Backend, hive *Hive, dbaccess *DbAccess, sp *bzzswap.SwapParams, sy *SyncParams, networkId uint64) (p2p.Protocol, error) {
// Addr interface that peerPool needs
type Addr interface {
OverlayPeer
Over() []byte
Under() []byte
String() string
Update(OverlayAddr) OverlayAddr
}
// a single global request db is created for all peer connections
// this is to persist delivery backlog and aid syncronisation
requestDb, err := storage.NewLDBDatabase(sy.RequestDbPath)
if err != nil {
return p2p.Protocol{}, fmt.Errorf("error setting up request db: %v", err)
// Peer interface represents an live peer connection
type Peer interface {
Addr // the address of a peer
Conn // the live connection (protocols.Peer)
LastActive() time.Time // last time active
}
// Conn interface represents an live peer connection
type Conn interface {
ID() discover.NodeID // the key that uniquely identifies the Node for the peerPool
Handshake(context.Context, interface{}, func(interface{}) error) (interface{}, error) // can send messages
Send(interface{}) error // can send messages
Drop(error) // disconnect this peer
Run(func(interface{}) error) error // the run function to run a protocol
Off() OverlayAddr
}
// BzzConfig captures the config params used by the hive
type BzzConfig struct {
OverlayAddr []byte // base address of the overlay network
UnderlayAddr []byte // node's underlay address
HiveParams *HiveParams
NetworkID uint64
}
// Bzz is the swarm protocol bundle
type Bzz struct {
*Hive
NetworkID uint64
localAddr *BzzAddr
mtx sync.Mutex
handshakes map[discover.NodeID]*HandshakeMsg
streamerSpec *protocols.Spec
streamerRun func(*BzzPeer) error
}
// NewBzz is the swarm protocol constructor
// arguments
// * bzz config
// * overlay driver
// * peer store
func NewBzz(config *BzzConfig, kad Overlay, store state.Store, streamerSpec *protocols.Spec, streamerRun func(*BzzPeer) error) *Bzz {
return &Bzz{
Hive: NewHive(config.HiveParams, kad, store),
NetworkID: config.NetworkID,
localAddr: &BzzAddr{config.OverlayAddr, config.UnderlayAddr},
handshakes: make(map[discover.NodeID]*HandshakeMsg),
streamerRun: streamerRun,
streamerSpec: streamerSpec,
}
if networkId == 0 {
networkId = NetworkId
}
return p2p.Protocol{
Name: "bzz",
Version: Version,
Length: ProtocolLength,
Run: func(p *p2p.Peer, rw p2p.MsgReadWriter) error {
return run(requestDb, cloud, backend, hive, dbaccess, sp, sy, networkId, p, rw)
}
// UpdateLocalAddr updates underlayaddress of the running node
func (b *Bzz) UpdateLocalAddr(byteaddr []byte) *BzzAddr {
b.localAddr = b.localAddr.Update(&BzzAddr{
UAddr: byteaddr,
OAddr: b.localAddr.OAddr,
}).(*BzzAddr)
return b.localAddr
}
// NodeInfo returns the node's overlay address
func (b *Bzz) NodeInfo() interface{} {
return b.localAddr.Address()
}
// Protocols return the protocols swarm offers
// Bzz implements the node.Service interface
// * handshake/hive
// * discovery
func (b *Bzz) Protocols() []p2p.Protocol {
protocol := []p2p.Protocol{
{
Name: BzzSpec.Name,
Version: BzzSpec.Version,
Length: BzzSpec.Length(),
Run: b.runBzz,
NodeInfo: b.NodeInfo,
},
}, nil
{
Name: DiscoverySpec.Name,
Version: DiscoverySpec.Version,
Length: DiscoverySpec.Length(),
Run: b.RunProtocol(DiscoverySpec, b.Hive.Run),
NodeInfo: b.Hive.NodeInfo,
PeerInfo: b.Hive.PeerInfo,
},
}
if b.streamerSpec != nil && b.streamerRun != nil {
protocol = append(protocol, p2p.Protocol{
Name: b.streamerSpec.Name,
Version: b.streamerSpec.Version,
Length: b.streamerSpec.Length(),
Run: b.RunProtocol(b.streamerSpec, b.streamerRun),
})
}
return protocol
}
/*
the main protocol loop that
* does the handshake by exchanging statusMsg
* if peer is valid and accepted, registers with the hive
* then enters into a forever loop handling incoming messages
* storage and retrieval related queries coming via bzz are dispatched to StorageHandler
* peer-related messages are dispatched to the hive
* payment related messages are relayed to SWAP service
* on disconnect, unregister the peer in the hive (note RemovePeer in the post-disconnect hook)
* whenever the loop terminates, the peer will disconnect with Subprotocol error
* whenever handlers return an error the loop terminates
*/
func run(requestDb *storage.LDBDatabase, depo StorageHandler, backend chequebook.Backend, hive *Hive, dbaccess *DbAccess, sp *bzzswap.SwapParams, sy *SyncParams, networkId uint64, p *p2p.Peer, rw p2p.MsgReadWriter) (err error) {
// APIs returns the APIs offered by bzz
// * hive
// Bzz implements the node.Service interface
func (b *Bzz) APIs() []rpc.API {
return []rpc.API{{
Namespace: "hive",
Version: "3.0",
Service: b.Hive,
}}
}
self := &bzz{
storage: depo,
backend: backend,
hive: hive,
dbAccess: dbaccess,
requestDb: requestDb,
peer: p,
rw: rw,
swapParams: sp,
syncParams: sy,
swapEnabled: hive.swapEnabled,
syncEnabled: true,
NetworkId: networkId,
// RunProtocol is a wrapper for swarm subprotocols
// returns a p2p protocol run function that can be assigned to p2p.Protocol#Run field
// arguments:
// * p2p protocol spec
// * run function taking BzzPeer as argument
// this run function is meant to block for the duration of the protocol session
// on return the session is terminated and the peer is disconnected
// the protocol waits for the bzz handshake is negotiated
// the overlay address on the BzzPeer is set from the remote handshake
func (b *Bzz) RunProtocol(spec *protocols.Spec, run func(*BzzPeer) error) func(*p2p.Peer, p2p.MsgReadWriter) error {
return func(p *p2p.Peer, rw p2p.MsgReadWriter) error {
// wait for the bzz protocol to perform the handshake
handshake, _ := b.GetHandshake(p.ID())
defer b.removeHandshake(p.ID())
select {
case <-handshake.done:
case <-time.After(bzzHandshakeTimeout):
return fmt.Errorf("%08x: %s protocol timeout waiting for handshake on %08x", b.BaseAddr()[:4], spec.Name, p.ID().Bytes()[:4])
}
if handshake.err != nil {
return fmt.Errorf("%08x: %s protocol closed: %v", b.BaseAddr()[:4], spec.Name, handshake.err)
}
// the handshake has succeeded so construct the BzzPeer and run the protocol
peer := &BzzPeer{
Peer: protocols.NewPeer(p, rw, spec),
localAddr: b.localAddr,
BzzAddr: handshake.peerAddr,
lastActive: time.Now(),
}
return run(peer)
}
}
// handle handshake
err = self.handleStatus()
if err != nil {
return err
}
// performHandshake implements the negotiation of the bzz handshake
// shared among swarm subprotocols
func (b *Bzz) performHandshake(p *protocols.Peer, handshake *HandshakeMsg) error {
ctx, cancel := context.WithTimeout(context.Background(), bzzHandshakeTimeout)
defer func() {
// if the handler loop exits, the peer is disconnecting
// deregister the peer in the hive
self.hive.removePeer(&peer{bzz: self})
if self.syncer != nil {
self.syncer.stop() // quits request db and delivery loops, save requests
}
if self.swap != nil {
self.swap.Stop() // quits chequebox autocash etc
}
close(handshake.done)
cancel()
}()
// the main forever loop that handles incoming requests
for {
if self.hive.blockRead {
log.Warn(fmt.Sprintf("Cannot read network"))
time.Sleep(100 * time.Millisecond)
continue
}
err = self.handle()
if err != nil {
return
}
rsh, err := p.Handshake(ctx, handshake, b.checkHandshake)
if err != nil {
handshake.err = err
return err
}
handshake.peerAddr = rsh.(*HandshakeMsg).Addr
return nil
}
// TODO: may need to implement protocol drop only? don't want to kick off the peer
// if they are useful for other protocols
func (self *bzz) Drop() {
self.peer.Disconnect(p2p.DiscSubprotocolError)
}
// runBzz is the p2p protocol run function for the bzz base protocol
// that negotiates the bzz handshake
func (b *Bzz) runBzz(p *p2p.Peer, rw p2p.MsgReadWriter) error {
handshake, _ := b.GetHandshake(p.ID())
if !<-handshake.init {
return fmt.Errorf("%08x: bzz already started on peer %08x", b.localAddr.Over()[:4], ToOverlayAddr(p.ID().Bytes())[:4])
}
close(handshake.init)
defer b.removeHandshake(p.ID())
peer := protocols.NewPeer(p, rw, BzzSpec)
err := b.performHandshake(peer, handshake)
if err != nil {
log.Warn(fmt.Sprintf("%08x: handshake failed with remote peer %08x: %v", b.localAddr.Over()[:4], ToOverlayAddr(p.ID().Bytes())[:4], err))
// one cycle of the main forever loop that handles and dispatches incoming messages
func (self *bzz) handle() error {
msg, err := self.rw.ReadMsg()
log.Debug(fmt.Sprintf("<- %v", msg))
return err
}
// fail if we get another handshake
msg, err := rw.ReadMsg()
if err != nil {
return err
}
if msg.Size > ProtocolMaxMsgSize {
return fmt.Errorf("message too long: %v > %v", msg.Size, ProtocolMaxMsgSize)
msg.Discard()
return errors.New("received multiple handshakes")
}
// BzzPeer is the bzz protocol view of a protocols.Peer (itself an extension of p2p.Peer)
// implements the Peer interface and all interfaces Peer implements: Addr, OverlayPeer
type BzzPeer struct {
*protocols.Peer // represents the connection for online peers
localAddr *BzzAddr // local Peers address
*BzzAddr // remote address -> implements Addr interface = protocols.Peer
lastActive time.Time // time is updated whenever mutexes are releasing
}
func NewBzzTestPeer(p *protocols.Peer, addr *BzzAddr) *BzzPeer {
return &BzzPeer{
Peer: p,
localAddr: addr,
BzzAddr: NewAddrFromNodeID(p.ID()),
}
// make sure that the payload has been fully consumed
defer msg.Discard()
}
switch msg.Code {
// Off returns the overlay peer record for offline persistence
func (p *BzzPeer) Off() OverlayAddr {
return p.BzzAddr
}
case statusMsg:
// no extra status message allowed. The one needed already handled by
// handleStatus
log.Debug(fmt.Sprintf("Status message: %v", msg))
return errors.New("extra status message")
// LastActive returns the time the peer was last active
func (p *BzzPeer) LastActive() time.Time {
return p.lastActive
}
case storeRequestMsg:
// store requests are dispatched to netStore
storeRequestMsgCounter.Inc(1)
var req storeRequestMsgData
if err := msg.Decode(&req); err != nil {
return fmt.Errorf("<- %v: %v", msg, err)
}
if n := len(req.SData); n < 9 {
return fmt.Errorf("<- %v: Data too short (%v)", msg, n)
}
// last Active time is set only when receiving chunks
self.lastActive = time.Now()
log.Trace(fmt.Sprintf("incoming store request: %s", req.String()))
// swap accounting is done within forwarding
self.storage.HandleStoreRequestMsg(&req, &peer{bzz: self})
/*
Handshake
case retrieveRequestMsg:
// retrieve Requests are dispatched to netStore
retrieveRequestMsgCounter.Inc(1)
var req retrieveRequestMsgData
if err := msg.Decode(&req); err != nil {
return fmt.Errorf("<- %v: %v", msg, err)
}
req.from = &peer{bzz: self}
// if request is lookup and not to be delivered
if req.isLookup() {
log.Trace(fmt.Sprintf("self lookup for %v: responding with peers only...", req.from))
} else if req.Key == nil {
return fmt.Errorf("protocol handler: req.Key == nil || req.Timeout == nil")
} else {
// swap accounting is done within netStore
self.storage.HandleRetrieveRequestMsg(&req, &peer{bzz: self})
}
// direct response with peers, TODO: sort this out
self.hive.peers(&req)
* Version: 8 byte integer version of the protocol
* NetworkID: 8 byte integer network identifier
* Addr: the address advertised by the node including underlay and overlay connecctions
*/
type HandshakeMsg struct {
Version uint64
NetworkID uint64
Addr *BzzAddr
case peersMsg:
// response to lookups and immediate response to retrieve requests
// dispatches new peer data to the hive that adds them to KADDB
peersMsgCounter.Inc(1)
var req peersMsgData
if err := msg.Decode(&req); err != nil {
return fmt.Errorf("<- %v: %v", msg, err)
}
req.from = &peer{bzz: self}
log.Trace(fmt.Sprintf("<- peer addresses: %v", req))
self.hive.HandlePeersMsg(&req, &peer{bzz: self})
// peerAddr is the address received in the peer handshake
peerAddr *BzzAddr
case syncRequestMsg:
syncRequestMsgCounter.Inc(1)
var req syncRequestMsgData
if err := msg.Decode(&req); err != nil {
return fmt.Errorf("<- %v: %v", msg, err)
}
log.Debug(fmt.Sprintf("<- sync request: %v", req))
self.lastActive = time.Now()
self.sync(req.SyncState)
init chan bool
done chan struct{}
err error
}
case unsyncedKeysMsg:
// coming from parent node offering
unsyncedKeysMsgCounter.Inc(1)
var req unsyncedKeysMsgData
if err := msg.Decode(&req); err != nil {
return fmt.Errorf("<- %v: %v", msg, err)
}
log.Debug(fmt.Sprintf("<- unsynced keys : %s", req.String()))
err := self.storage.HandleUnsyncedKeysMsg(&req, &peer{bzz: self})
self.lastActive = time.Now()
if err != nil {
return fmt.Errorf("<- %v: %v", msg, err)
}
// String pretty prints the handshake
func (bh *HandshakeMsg) String() string {
return fmt.Sprintf("Handshake: Version: %v, NetworkID: %v, Addr: %v", bh.Version, bh.NetworkID, bh.Addr)
}
case deliveryRequestMsg:
// response to syncKeysMsg hashes filtered not existing in db
// also relays the last synced state to the source
deliverRequestMsgCounter.Inc(1)
var req deliveryRequestMsgData
if err := msg.Decode(&req); err != nil {
return fmt.Errorf("<-msg %v: %v", msg, err)
}
log.Debug(fmt.Sprintf("<- delivery request: %s", req.String()))
err := self.storage.HandleDeliveryRequestMsg(&req, &peer{bzz: self})
self.lastActive = time.Now()
if err != nil {
return fmt.Errorf("<- %v: %v", msg, err)
}
case paymentMsg:
// swap protocol message for payment, Units paid for, Cheque paid with
paymentMsgCounter.Inc(1)
if self.swapEnabled {
var req paymentMsgData
if err := msg.Decode(&req); err != nil {
return fmt.Errorf("<- %v: %v", msg, err)
}
log.Debug(fmt.Sprintf("<- payment: %s", req.String()))
self.swap.Receive(int(req.Units), req.Promise)
}
default:
// no other message is allowed
invalidMsgCounter.Inc(1)
return fmt.Errorf("invalid message code: %v", msg.Code)
// Perform initiates the handshake and validates the remote handshake message
func (b *Bzz) checkHandshake(hs interface{}) error {
rhs := hs.(*HandshakeMsg)
if rhs.NetworkID != b.NetworkID {
return fmt.Errorf("network id mismatch %d (!= %d)", rhs.NetworkID, b.NetworkID)
}
if rhs.Version != uint64(BzzSpec.Version) {
return fmt.Errorf("version mismatch %d (!= %d)", rhs.Version, BzzSpec.Version)
}
return nil
}
func (self *bzz) handleStatus() (err error) {
// removeHandshake removes handshake for peer with peerID
// from the bzz handshake store
func (b *Bzz) removeHandshake(peerID discover.NodeID) {
b.mtx.Lock()
defer b.mtx.Unlock()
delete(b.handshakes, peerID)
}
handshake := &statusMsgData{
Version: uint64(Version),
ID: "honey",
Addr: self.selfAddr(),
NetworkId: self.NetworkId,
Swap: &bzzswap.SwapProfile{
Profile: self.swapParams.Profile,
PayProfile: self.swapParams.PayProfile,
},
}
err = p2p.Send(self.rw, statusMsg, handshake)
if err != nil {
return err
}
// read and handle remote status
var msg p2p.Msg
msg, err = self.rw.ReadMsg()
if err != nil {
return err
}
if msg.Code != statusMsg {
return fmt.Errorf("first msg has code %x (!= %x)", msg.Code, statusMsg)
}
handleStatusMsgCounter.Inc(1)
if msg.Size > ProtocolMaxMsgSize {
return fmt.Errorf("message too long: %v > %v", msg.Size, ProtocolMaxMsgSize)
}
var status statusMsgData
if err := msg.Decode(&status); err != nil {
return fmt.Errorf("<- %v: %v", msg, err)
}
if status.NetworkId != self.NetworkId {
return fmt.Errorf("network id mismatch: %d (!= %d)", status.NetworkId, self.NetworkId)
}
if Version != status.Version {
return fmt.Errorf("protocol version mismatch: %d (!= %d)", status.Version, Version)
}
self.remoteAddr = self.peerAddr(status.Addr)
log.Trace(fmt.Sprintf("self: advertised IP: %v, peer advertised: %v, local address: %v\npeer: advertised IP: %v, remote address: %v\n", self.selfAddr(), self.remoteAddr, self.peer.LocalAddr(), status.Addr.IP, self.peer.RemoteAddr()))
if self.swapEnabled {
// set remote profile for accounting
self.swap, err = bzzswap.NewSwap(self.swapParams, status.Swap, self.backend, self)
if err != nil {
return err
// GetHandshake returns the bzz handhake that the remote peer with peerID sent
func (b *Bzz) GetHandshake(peerID discover.NodeID) (*HandshakeMsg, bool) {
b.mtx.Lock()
defer b.mtx.Unlock()
handshake, found := b.handshakes[peerID]
if !found {
handshake = &HandshakeMsg{
Version: uint64(BzzSpec.Version),
NetworkID: b.NetworkID,
Addr: b.localAddr,
init: make(chan bool, 1),
done: make(chan struct{}),
}
// when handhsake is first created for a remote peer
// it is initialised with the init
handshake.init <- true
b.handshakes[peerID] = handshake
}
log.Info(fmt.Sprintf("Peer %08x is capable (%d/%d)", self.remoteAddr.Addr[:4], status.Version, status.NetworkId))
err = self.hive.addPeer(&peer{bzz: self})
return handshake, found
}
// BzzAddr implements the PeerAddr interface
type BzzAddr struct {
OAddr []byte
UAddr []byte
}
// Address implements OverlayPeer interface to be used in Overlay
func (a *BzzAddr) Address() []byte {
return a.OAddr
}
// Over returns the overlay address
func (a *BzzAddr) Over() []byte {
return a.OAddr
}
// Under returns the underlay address
func (a *BzzAddr) Under() []byte {
return a.UAddr
}
// ID returns the nodeID from the underlay enode address
func (a *BzzAddr) ID() discover.NodeID {
return discover.MustParseNode(string(a.UAddr)).ID
}
// Update updates the underlay address of a peer record
func (a *BzzAddr) Update(na OverlayAddr) OverlayAddr {
return &BzzAddr{a.OAddr, na.(Addr).Under()}
}
// String pretty prints the address
func (a *BzzAddr) String() string {
return fmt.Sprintf("%x <%s>", a.OAddr, a.UAddr)
}
// RandomAddr is a utility method generating an address from a public key
func RandomAddr() *BzzAddr {
key, err := crypto.GenerateKey()
if err != nil {
return err
panic("unable to generate key")
}
// hive sets syncstate so sync should start after node added
log.Info(fmt.Sprintf("syncronisation request sent with %v", self.syncState))
self.syncRequest()
return nil
pubkey := crypto.FromECDSAPub(&key.PublicKey)
var id discover.NodeID
copy(id[:], pubkey[1:])
return NewAddrFromNodeID(id)
}
func (self *bzz) sync(state *syncState) error {
// syncer setup
if self.syncer != nil {
return errors.New("sync request can only be sent once")
// NewNodeIDFromAddr transforms the underlay address to an adapters.NodeID
func NewNodeIDFromAddr(addr Addr) discover.NodeID {
log.Info(fmt.Sprintf("uaddr=%s", string(addr.Under())))
node := discover.MustParseNode(string(addr.Under()))
return node.ID
}
// NewAddrFromNodeID constucts a BzzAddr from a discover.NodeID
// the overlay address is derived as the hash of the nodeID
func NewAddrFromNodeID(id discover.NodeID) *BzzAddr {
return &BzzAddr{
OAddr: ToOverlayAddr(id.Bytes()),
UAddr: []byte(discover.NewNode(id, net.IP{127, 0, 0, 1}, 30303, 30303).String()),
}
}
cnt := self.dbAccess.counter()
remoteaddr := self.remoteAddr.Addr
start, stop := self.hive.kad.KeyRange(remoteaddr)
// an explicitly received nil syncstate disables syncronisation
if state == nil {
self.syncEnabled = false
log.Warn(fmt.Sprintf("syncronisation disabled for peer %v", self))
state = &syncState{DbSyncState: &storage.DbSyncState{}, Synced: true}
} else {
state.synced = make(chan bool)
state.SessionAt = cnt
if storage.IsZeroKey(state.Stop) && state.Synced {
state.Start = storage.Key(start[:])
state.Stop = storage.Key(stop[:])
}
log.Debug(fmt.Sprintf("syncronisation requested by peer %v at state %v", self, state))
// NewAddrFromNodeIDAndPort constucts a BzzAddr from a discover.NodeID and port uint16
// the overlay address is derived as the hash of the nodeID
func NewAddrFromNodeIDAndPort(id discover.NodeID, host net.IP, port uint16) *BzzAddr {
return &BzzAddr{
OAddr: ToOverlayAddr(id.Bytes()),
UAddr: []byte(discover.NewNode(id, host, port, port).String()),
}
var err error
self.syncer, err = newSyncer(
self.requestDb,
storage.Key(remoteaddr[:]),
self.dbAccess,
self.unsyncedKeys, self.store,
self.syncParams, state, func() bool { return self.syncEnabled },
)
if err != nil {
return nil
}
log.Trace(fmt.Sprintf("syncer set for peer %v", self))
return nil
}
func (self *bzz) String() string {
return self.remoteAddr.String()
}
// repair reported address if IP missing
func (self *bzz) peerAddr(base *peerAddr) *peerAddr {
if base.IP.IsUnspecified() {
host, _, _ := net.SplitHostPort(self.peer.RemoteAddr().String())
base.IP = net.ParseIP(host)
}
return base
}
// returns self advertised node connection info (listening address w enodes)
// IP will get repaired on the other end if missing
// or resolved via ID by discovery at dialout
func (self *bzz) selfAddr() *peerAddr {
id := self.hive.id
host, port, _ := net.SplitHostPort(self.hive.listenAddr())
intport, _ := strconv.Atoi(port)
addr := &peerAddr{
Addr: self.hive.addr,
ID: id[:],
IP: net.ParseIP(host),
Port: uint16(intport),
}
return addr
}
// outgoing messages
// send retrieveRequestMsg
func (self *bzz) retrieve(req *retrieveRequestMsgData) error {
return self.send(retrieveRequestMsg, req)
}
// send storeRequestMsg
func (self *bzz) store(req *storeRequestMsgData) error {
return self.send(storeRequestMsg, req)
}
func (self *bzz) syncRequest() error {
req := &syncRequestMsgData{}
if self.hive.syncEnabled {
log.Debug(fmt.Sprintf("syncronisation request to peer %v at state %v", self, self.syncState))
req.SyncState = self.syncState
}
if self.syncState == nil {
log.Warn(fmt.Sprintf("syncronisation disabled for peer %v at state %v", self, self.syncState))
}
return self.send(syncRequestMsg, req)
}
// queue storeRequestMsg in request db
func (self *bzz) deliveryRequest(reqs []*syncRequest) error {
req := &deliveryRequestMsgData{
Deliver: reqs,
}
return self.send(deliveryRequestMsg, req)
}
// batch of syncRequests to send off
func (self *bzz) unsyncedKeys(reqs []*syncRequest, state *syncState) error {
req := &unsyncedKeysMsgData{
Unsynced: reqs,
State: state,
}
return self.send(unsyncedKeysMsg, req)
}
// send paymentMsg
func (self *bzz) Pay(units int, promise swap.Promise) {
req := &paymentMsgData{uint(units), promise.(*chequebook.Cheque)}
self.payment(req)
}
// send paymentMsg
func (self *bzz) payment(req *paymentMsgData) error {
return self.send(paymentMsg, req)
}
// sends peersMsg
func (self *bzz) peers(req *peersMsgData) error {
return self.send(peersMsg, req)
}
func (self *bzz) send(msg uint64, data interface{}) error {
if self.hive.blockWrite {
return fmt.Errorf("network write blocked")
}
log.Trace(fmt.Sprintf("-> %v: %v (%T) to %v", msg, data, data, self))
err := p2p.Send(self.rw, msg, data)
if err != nil {
self.Drop()
}
return err
// ToOverlayAddr creates an overlayaddress from a byte slice
func ToOverlayAddr(id []byte) []byte {
return crypto.Keccak256(id)
}

View File

@@ -1,4 +1,4 @@
// Copyright 2014 The go-ethereum Authors
// Copyright 2016 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
@@ -15,3 +15,226 @@
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package network
import (
"flag"
"fmt"
"os"
"sync"
"testing"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/p2p"
"github.com/ethereum/go-ethereum/p2p/discover"
"github.com/ethereum/go-ethereum/p2p/protocols"
p2ptest "github.com/ethereum/go-ethereum/p2p/testing"
)
var (
loglevel = flag.Int("loglevel", 2, "verbosity of logs")
)
func init() {
flag.Parse()
log.Root().SetHandler(log.LvlFilterHandler(log.Lvl(*loglevel), log.StreamHandler(os.Stderr, log.TerminalFormat(true))))
}
type testStore struct {
sync.Mutex
values map[string][]byte
}
func newTestStore() *testStore {
return &testStore{values: make(map[string][]byte)}
}
func (t *testStore) Load(key string) ([]byte, error) {
t.Lock()
defer t.Unlock()
v, ok := t.values[key]
if !ok {
return nil, fmt.Errorf("key not found: %s", key)
}
return v, nil
}
func (t *testStore) Save(key string, v []byte) error {
t.Lock()
defer t.Unlock()
t.values[key] = v
return nil
}
func HandshakeMsgExchange(lhs, rhs *HandshakeMsg, id discover.NodeID) []p2ptest.Exchange {
return []p2ptest.Exchange{
{
Expects: []p2ptest.Expect{
{
Code: 0,
Msg: lhs,
Peer: id,
},
},
},
{
Triggers: []p2ptest.Trigger{
{
Code: 0,
Msg: rhs,
Peer: id,
},
},
},
}
}
func newBzzBaseTester(t *testing.T, n int, addr *BzzAddr, spec *protocols.Spec, run func(*BzzPeer) error) *bzzTester {
cs := make(map[string]chan bool)
srv := func(p *BzzPeer) error {
defer func() {
if cs[p.ID().String()] != nil {
close(cs[p.ID().String()])
}
}()
return run(p)
}
protocol := func(p *p2p.Peer, rw p2p.MsgReadWriter) error {
return srv(&BzzPeer{
Peer: protocols.NewPeer(p, rw, spec),
localAddr: addr,
BzzAddr: NewAddrFromNodeID(p.ID()),
})
}
s := p2ptest.NewProtocolTester(t, NewNodeIDFromAddr(addr), n, protocol)
for _, id := range s.IDs {
cs[id.String()] = make(chan bool)
}
return &bzzTester{
addr: addr,
ProtocolTester: s,
cs: cs,
}
}
type bzzTester struct {
*p2ptest.ProtocolTester
addr *BzzAddr
cs map[string]chan bool
}
func newBzzHandshakeTester(t *testing.T, n int, addr *BzzAddr) *bzzTester {
config := &BzzConfig{
OverlayAddr: addr.Over(),
UnderlayAddr: addr.Under(),
HiveParams: NewHiveParams(),
NetworkID: DefaultNetworkID,
}
kad := NewKademlia(addr.OAddr, NewKadParams())
bzz := NewBzz(config, kad, nil, nil, nil)
s := p2ptest.NewProtocolTester(t, NewNodeIDFromAddr(addr), 1, bzz.runBzz)
return &bzzTester{
addr: addr,
ProtocolTester: s,
}
}
// should test handshakes in one exchange? parallelisation
func (s *bzzTester) testHandshake(lhs, rhs *HandshakeMsg, disconnects ...*p2ptest.Disconnect) error {
var peers []discover.NodeID
id := NewNodeIDFromAddr(rhs.Addr)
if len(disconnects) > 0 {
for _, d := range disconnects {
peers = append(peers, d.Peer)
}
} else {
peers = []discover.NodeID{id}
}
if err := s.TestExchanges(HandshakeMsgExchange(lhs, rhs, id)...); err != nil {
return err
}
if len(disconnects) > 0 {
return s.TestDisconnected(disconnects...)
}
// If we don't expect disconnect, ensure peers remain connected
err := s.TestDisconnected(&p2ptest.Disconnect{
Peer: s.IDs[0],
Error: nil,
})
if err == nil {
return fmt.Errorf("Unexpected peer disconnect")
}
if err.Error() != "timed out waiting for peers to disconnect" {
return err
}
return nil
}
func correctBzzHandshake(addr *BzzAddr) *HandshakeMsg {
return &HandshakeMsg{
Version: 4,
NetworkID: DefaultNetworkID,
Addr: addr,
}
}
func TestBzzHandshakeNetworkIDMismatch(t *testing.T) {
addr := RandomAddr()
s := newBzzHandshakeTester(t, 1, addr)
id := s.IDs[0]
err := s.testHandshake(
correctBzzHandshake(addr),
&HandshakeMsg{Version: 4, NetworkID: 321, Addr: NewAddrFromNodeID(id)},
&p2ptest.Disconnect{Peer: id, Error: fmt.Errorf("Handshake error: Message handler error: (msg code 0): network id mismatch 321 (!= 3)")},
)
if err != nil {
t.Fatal(err)
}
}
func TestBzzHandshakeVersionMismatch(t *testing.T) {
addr := RandomAddr()
s := newBzzHandshakeTester(t, 1, addr)
id := s.IDs[0]
err := s.testHandshake(
correctBzzHandshake(addr),
&HandshakeMsg{Version: 0, NetworkID: 3, Addr: NewAddrFromNodeID(id)},
&p2ptest.Disconnect{Peer: id, Error: fmt.Errorf("Handshake error: Message handler error: (msg code 0): version mismatch 0 (!= 4)")},
)
if err != nil {
t.Fatal(err)
}
}
func TestBzzHandshakeSuccess(t *testing.T) {
addr := RandomAddr()
s := newBzzHandshakeTester(t, 1, addr)
id := s.IDs[0]
err := s.testHandshake(
correctBzzHandshake(addr),
&HandshakeMsg{Version: 4, NetworkID: 3, Addr: NewAddrFromNodeID(id)},
)
if err != nil {
t.Fatal(err)
}
}

View File

@@ -0,0 +1,17 @@
// Copyright 2018 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package discovery

View File

@@ -0,0 +1,586 @@
// Copyright 2018 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package discovery
import (
"context"
"encoding/json"
"errors"
"flag"
"fmt"
"io/ioutil"
"math/rand"
"os"
"path"
"strings"
"sync"
"testing"
"time"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/node"
"github.com/ethereum/go-ethereum/p2p"
"github.com/ethereum/go-ethereum/p2p/discover"
"github.com/ethereum/go-ethereum/p2p/simulations"
"github.com/ethereum/go-ethereum/p2p/simulations/adapters"
"github.com/ethereum/go-ethereum/swarm/network"
"github.com/ethereum/go-ethereum/swarm/state"
colorable "github.com/mattn/go-colorable"
)
// serviceName is used with the exec adapter so the exec'd binary knows which
// service to execute
const serviceName = "discovery"
const testMinProxBinSize = 2
const discoveryPersistenceDatadir = "discovery_persistence_test_store"
var discoveryPersistencePath = path.Join(os.TempDir(), discoveryPersistenceDatadir)
var discoveryEnabled = true
var persistenceEnabled = false
var services = adapters.Services{
serviceName: newService,
}
func cleanDbStores() error {
entries, err := ioutil.ReadDir(os.TempDir())
if err != nil {
return err
}
for _, f := range entries {
if strings.HasPrefix(f.Name(), discoveryPersistenceDatadir) {
os.RemoveAll(path.Join(os.TempDir(), f.Name()))
}
}
return nil
}
func getDbStore(nodeID string) (*state.DBStore, error) {
if _, err := os.Stat(discoveryPersistencePath + "_" + nodeID); os.IsNotExist(err) {
log.Info(fmt.Sprintf("directory for nodeID %s does not exist. creating...", nodeID))
ioutil.TempDir("", discoveryPersistencePath+"_"+nodeID)
}
log.Info(fmt.Sprintf("opening storage directory for nodeID %s", nodeID))
store, err := state.NewDBStore(discoveryPersistencePath + "_" + nodeID)
if err != nil {
return nil, err
}
return store, nil
}
var (
nodeCount = flag.Int("nodes", 10, "number of nodes to create (default 10)")
initCount = flag.Int("conns", 1, "number of originally connected peers (default 1)")
snapshotFile = flag.String("snapshot", "", "create snapshot")
loglevel = flag.Int("loglevel", 3, "verbosity of logs")
rawlog = flag.Bool("rawlog", false, "remove terminal formatting from logs")
)
func init() {
flag.Parse()
// register the discovery service which will run as a devp2p
// protocol when using the exec adapter
adapters.RegisterServices(services)
log.PrintOrigins(true)
log.Root().SetHandler(log.LvlFilterHandler(log.Lvl(*loglevel), log.StreamHandler(colorable.NewColorableStderr(), log.TerminalFormat(!*rawlog))))
}
// Benchmarks to test the average time it takes for an N-node ring
// to full a healthy kademlia topology
func BenchmarkDiscovery_8_1(b *testing.B) { benchmarkDiscovery(b, 8, 1) }
func BenchmarkDiscovery_16_1(b *testing.B) { benchmarkDiscovery(b, 16, 1) }
func BenchmarkDiscovery_32_1(b *testing.B) { benchmarkDiscovery(b, 32, 1) }
func BenchmarkDiscovery_64_1(b *testing.B) { benchmarkDiscovery(b, 64, 1) }
func BenchmarkDiscovery_128_1(b *testing.B) { benchmarkDiscovery(b, 128, 1) }
func BenchmarkDiscovery_256_1(b *testing.B) { benchmarkDiscovery(b, 256, 1) }
func BenchmarkDiscovery_8_2(b *testing.B) { benchmarkDiscovery(b, 8, 2) }
func BenchmarkDiscovery_16_2(b *testing.B) { benchmarkDiscovery(b, 16, 2) }
func BenchmarkDiscovery_32_2(b *testing.B) { benchmarkDiscovery(b, 32, 2) }
func BenchmarkDiscovery_64_2(b *testing.B) { benchmarkDiscovery(b, 64, 2) }
func BenchmarkDiscovery_128_2(b *testing.B) { benchmarkDiscovery(b, 128, 2) }
func BenchmarkDiscovery_256_2(b *testing.B) { benchmarkDiscovery(b, 256, 2) }
func BenchmarkDiscovery_8_4(b *testing.B) { benchmarkDiscovery(b, 8, 4) }
func BenchmarkDiscovery_16_4(b *testing.B) { benchmarkDiscovery(b, 16, 4) }
func BenchmarkDiscovery_32_4(b *testing.B) { benchmarkDiscovery(b, 32, 4) }
func BenchmarkDiscovery_64_4(b *testing.B) { benchmarkDiscovery(b, 64, 4) }
func BenchmarkDiscovery_128_4(b *testing.B) { benchmarkDiscovery(b, 128, 4) }
func BenchmarkDiscovery_256_4(b *testing.B) { benchmarkDiscovery(b, 256, 4) }
func TestDiscoverySimulationDockerAdapter(t *testing.T) {
testDiscoverySimulationDockerAdapter(t, *nodeCount, *initCount)
}
func testDiscoverySimulationDockerAdapter(t *testing.T, nodes, conns int) {
adapter, err := adapters.NewDockerAdapter()
if err != nil {
if err == adapters.ErrLinuxOnly {
t.Skip(err)
} else {
t.Fatal(err)
}
}
testDiscoverySimulation(t, nodes, conns, adapter)
}
func TestDiscoverySimulationExecAdapter(t *testing.T) {
testDiscoverySimulationExecAdapter(t, *nodeCount, *initCount)
}
func testDiscoverySimulationExecAdapter(t *testing.T, nodes, conns int) {
baseDir, err := ioutil.TempDir("", "swarm-test")
if err != nil {
t.Fatal(err)
}
defer os.RemoveAll(baseDir)
testDiscoverySimulation(t, nodes, conns, adapters.NewExecAdapter(baseDir))
}
func TestDiscoverySimulationSimAdapter(t *testing.T) {
testDiscoverySimulationSimAdapter(t, *nodeCount, *initCount)
}
func TestDiscoveryPersistenceSimulationSimAdapter(t *testing.T) {
testDiscoveryPersistenceSimulationSimAdapter(t, *nodeCount, *initCount)
}
func testDiscoveryPersistenceSimulationSimAdapter(t *testing.T, nodes, conns int) {
testDiscoveryPersistenceSimulation(t, nodes, conns, adapters.NewSimAdapter(services))
}
func testDiscoverySimulationSimAdapter(t *testing.T, nodes, conns int) {
testDiscoverySimulation(t, nodes, conns, adapters.NewSimAdapter(services))
}
func testDiscoverySimulation(t *testing.T, nodes, conns int, adapter adapters.NodeAdapter) {
startedAt := time.Now()
result, err := discoverySimulation(nodes, conns, adapter)
if err != nil {
t.Fatalf("Setting up simulation failed: %v", err)
}
if result.Error != nil {
t.Fatalf("Simulation failed: %s", result.Error)
}
t.Logf("Simulation with %d nodes passed in %s", nodes, result.FinishedAt.Sub(result.StartedAt))
var min, max time.Duration
var sum int
for _, pass := range result.Passes {
duration := pass.Sub(result.StartedAt)
if sum == 0 || duration < min {
min = duration
}
if duration > max {
max = duration
}
sum += int(duration.Nanoseconds())
}
t.Logf("Min: %s, Max: %s, Average: %s", min, max, time.Duration(sum/len(result.Passes))*time.Nanosecond)
finishedAt := time.Now()
t.Logf("Setup: %s, shutdown: %s", result.StartedAt.Sub(startedAt), finishedAt.Sub(result.FinishedAt))
}
func testDiscoveryPersistenceSimulation(t *testing.T, nodes, conns int, adapter adapters.NodeAdapter) map[int][]byte {
persistenceEnabled = true
discoveryEnabled = true
result, err := discoveryPersistenceSimulation(nodes, conns, adapter)
if err != nil {
t.Fatalf("Setting up simulation failed: %v", err)
}
if result.Error != nil {
t.Fatalf("Simulation failed: %s", result.Error)
}
t.Logf("Simulation with %d nodes passed in %s", nodes, result.FinishedAt.Sub(result.StartedAt))
// set the discovery and persistence flags again to default so other
// tests will not be affected
discoveryEnabled = true
persistenceEnabled = false
return nil
}
func benchmarkDiscovery(b *testing.B, nodes, conns int) {
for i := 0; i < b.N; i++ {
result, err := discoverySimulation(nodes, conns, adapters.NewSimAdapter(services))
if err != nil {
b.Fatalf("setting up simulation failed: %v", err)
}
if result.Error != nil {
b.Logf("simulation failed: %s", result.Error)
}
}
}
func discoverySimulation(nodes, conns int, adapter adapters.NodeAdapter) (*simulations.StepResult, error) {
// create network
net := simulations.NewNetwork(adapter, &simulations.NetworkConfig{
ID: "0",
DefaultService: serviceName,
})
defer net.Shutdown()
trigger := make(chan discover.NodeID)
ids := make([]discover.NodeID, nodes)
for i := 0; i < nodes; i++ {
conf := adapters.RandomNodeConfig()
node, err := net.NewNodeWithConfig(conf)
if err != nil {
return nil, fmt.Errorf("error starting node: %s", err)
}
if err := net.Start(node.ID()); err != nil {
return nil, fmt.Errorf("error starting node %s: %s", node.ID().TerminalString(), err)
}
if err := triggerChecks(trigger, net, node.ID()); err != nil {
return nil, fmt.Errorf("error triggering checks for node %s: %s", node.ID().TerminalString(), err)
}
ids[i] = node.ID()
}
// run a simulation which connects the 10 nodes in a ring and waits
// for full peer discovery
var addrs [][]byte
action := func(ctx context.Context) error {
return nil
}
wg := sync.WaitGroup{}
for i := range ids {
// collect the overlay addresses, to
addrs = append(addrs, network.ToOverlayAddr(ids[i].Bytes()))
for j := 0; j < conns; j++ {
var k int
if j == 0 {
k = (i + 1) % len(ids)
} else {
k = rand.Intn(len(ids))
}
wg.Add(1)
go func(i, k int) {
defer wg.Done()
net.Connect(ids[i], ids[k])
}(i, k)
}
}
wg.Wait()
log.Debug(fmt.Sprintf("nodes: %v", len(addrs)))
// construct the peer pot, so that kademlia health can be checked
ppmap := network.NewPeerPotMap(testMinProxBinSize, addrs)
check := func(ctx context.Context, id discover.NodeID) (bool, error) {
select {
case <-ctx.Done():
return false, ctx.Err()
default:
}
node := net.GetNode(id)
if node == nil {
return false, fmt.Errorf("unknown node: %s", id)
}
client, err := node.Client()
if err != nil {
return false, fmt.Errorf("error getting node client: %s", err)
}
healthy := &network.Health{}
addr := common.Bytes2Hex(network.ToOverlayAddr(id.Bytes()))
if err := client.Call(&healthy, "hive_healthy", ppmap[addr]); err != nil {
return false, fmt.Errorf("error getting node health: %s", err)
}
log.Debug(fmt.Sprintf("node %4s healthy: got nearest neighbours: %v, know nearest neighbours: %v, saturated: %v\n%v", id, healthy.GotNN, healthy.KnowNN, healthy.Full, healthy.Hive))
return healthy.KnowNN && healthy.GotNN && healthy.Full, nil
}
// 64 nodes ~ 1min
// 128 nodes ~
timeout := 300 * time.Second
ctx, cancel := context.WithTimeout(context.Background(), timeout)
defer cancel()
result := simulations.NewSimulation(net).Run(ctx, &simulations.Step{
Action: action,
Trigger: trigger,
Expect: &simulations.Expectation{
Nodes: ids,
Check: check,
},
})
if result.Error != nil {
return result, nil
}
if *snapshotFile != "" {
snap, err := net.Snapshot()
if err != nil {
return nil, errors.New("no shapshot dude")
}
jsonsnapshot, err := json.Marshal(snap)
if err != nil {
return nil, fmt.Errorf("corrupt json snapshot: %v", err)
}
log.Info("writing snapshot", "file", *snapshotFile)
err = ioutil.WriteFile(*snapshotFile, jsonsnapshot, 0755)
if err != nil {
return nil, err
}
}
return result, nil
}
func discoveryPersistenceSimulation(nodes, conns int, adapter adapters.NodeAdapter) (*simulations.StepResult, error) {
cleanDbStores()
defer cleanDbStores()
// create network
net := simulations.NewNetwork(adapter, &simulations.NetworkConfig{
ID: "0",
DefaultService: serviceName,
})
defer net.Shutdown()
trigger := make(chan discover.NodeID)
ids := make([]discover.NodeID, nodes)
var addrs [][]byte
for i := 0; i < nodes; i++ {
conf := adapters.RandomNodeConfig()
node, err := net.NewNodeWithConfig(conf)
if err != nil {
panic(err)
}
if err != nil {
return nil, fmt.Errorf("error starting node: %s", err)
}
if err := net.Start(node.ID()); err != nil {
return nil, fmt.Errorf("error starting node %s: %s", node.ID().TerminalString(), err)
}
if err := triggerChecks(trigger, net, node.ID()); err != nil {
return nil, fmt.Errorf("error triggering checks for node %s: %s", node.ID().TerminalString(), err)
}
ids[i] = node.ID()
a := network.ToOverlayAddr(ids[i].Bytes())
addrs = append(addrs, a)
}
// run a simulation which connects the 10 nodes in a ring and waits
// for full peer discovery
ppmap := network.NewPeerPotMap(testMinProxBinSize, addrs)
var restartTime time.Time
action := func(ctx context.Context) error {
ticker := time.NewTicker(500 * time.Millisecond)
for range ticker.C {
isHealthy := true
for _, id := range ids {
//call Healthy RPC
node := net.GetNode(id)
if node == nil {
return fmt.Errorf("unknown node: %s", id)
}
client, err := node.Client()
if err != nil {
return fmt.Errorf("error getting node client: %s", err)
}
healthy := &network.Health{}
addr := common.Bytes2Hex(network.ToOverlayAddr(id.Bytes()))
if err := client.Call(&healthy, "hive_healthy", ppmap[addr]); err != nil {
return fmt.Errorf("error getting node health: %s", err)
}
log.Info(fmt.Sprintf("NODE: %s, IS HEALTHY: %t", id.String(), healthy.GotNN && healthy.KnowNN && healthy.Full))
if !healthy.GotNN || !healthy.Full {
isHealthy = false
break
}
}
if isHealthy {
break
}
}
ticker.Stop()
log.Info("reached healthy kademlia. starting to shutdown nodes.")
shutdownStarted := time.Now()
// stop all ids, then start them again
for _, id := range ids {
node := net.GetNode(id)
if err := net.Stop(node.ID()); err != nil {
return fmt.Errorf("error stopping node %s: %s", node.ID().TerminalString(), err)
}
}
log.Info(fmt.Sprintf("shutting down nodes took: %s", time.Since(shutdownStarted)))
persistenceEnabled = true
discoveryEnabled = false
restartTime = time.Now()
for _, id := range ids {
node := net.GetNode(id)
if err := net.Start(node.ID()); err != nil {
return fmt.Errorf("error starting node %s: %s", node.ID().TerminalString(), err)
}
if err := triggerChecks(trigger, net, node.ID()); err != nil {
return fmt.Errorf("error triggering checks for node %s: %s", node.ID().TerminalString(), err)
}
}
log.Info(fmt.Sprintf("restarting nodes took: %s", time.Since(restartTime)))
return nil
}
//connects in a chain
wg := sync.WaitGroup{}
//connects in a ring
for i := range ids {
for j := 1; j <= conns; j++ {
k := (i + j) % len(ids)
if k == i {
k = (k + 1) % len(ids)
}
wg.Add(1)
go func(i, k int) {
defer wg.Done()
net.Connect(ids[i], ids[k])
}(i, k)
}
}
wg.Wait()
log.Debug(fmt.Sprintf("nodes: %v", len(addrs)))
// construct the peer pot, so that kademlia health can be checked
check := func(ctx context.Context, id discover.NodeID) (bool, error) {
select {
case <-ctx.Done():
return false, ctx.Err()
default:
}
node := net.GetNode(id)
if node == nil {
return false, fmt.Errorf("unknown node: %s", id)
}
client, err := node.Client()
if err != nil {
return false, fmt.Errorf("error getting node client: %s", err)
}
healthy := &network.Health{}
addr := common.Bytes2Hex(network.ToOverlayAddr(id.Bytes()))
if err := client.Call(&healthy, "hive_healthy", ppmap[addr]); err != nil {
return false, fmt.Errorf("error getting node health: %s", err)
}
log.Info(fmt.Sprintf("node %4s healthy: got nearest neighbours: %v, know nearest neighbours: %v, saturated: %v", id, healthy.GotNN, healthy.KnowNN, healthy.Full))
return healthy.KnowNN && healthy.GotNN && healthy.Full, nil
}
// 64 nodes ~ 1min
// 128 nodes ~
timeout := 300 * time.Second
ctx, cancel := context.WithTimeout(context.Background(), timeout)
defer cancel()
result := simulations.NewSimulation(net).Run(ctx, &simulations.Step{
Action: action,
Trigger: trigger,
Expect: &simulations.Expectation{
Nodes: ids,
Check: check,
},
})
if result.Error != nil {
return result, nil
}
return result, nil
}
// triggerChecks triggers a simulation step check whenever a peer is added or
// removed from the given node, and also every second to avoid a race between
// peer events and kademlia becoming healthy
func triggerChecks(trigger chan discover.NodeID, net *simulations.Network, id discover.NodeID) error {
node := net.GetNode(id)
if node == nil {
return fmt.Errorf("unknown node: %s", id)
}
client, err := node.Client()
if err != nil {
return err
}
events := make(chan *p2p.PeerEvent)
sub, err := client.Subscribe(context.Background(), "admin", events, "peerEvents")
if err != nil {
return fmt.Errorf("error getting peer events for node %v: %s", id, err)
}
go func() {
defer sub.Unsubscribe()
tick := time.NewTicker(time.Second)
defer tick.Stop()
for {
select {
case <-events:
trigger <- id
case <-tick.C:
trigger <- id
case err := <-sub.Err():
if err != nil {
log.Error(fmt.Sprintf("error getting peer events for node %v", id), "err", err)
}
return
}
}
}()
return nil
}
func newService(ctx *adapters.ServiceContext) (node.Service, error) {
host := adapters.ExternalIP()
addr := network.NewAddrFromNodeIDAndPort(ctx.Config.ID, host, ctx.Config.Port)
kp := network.NewKadParams()
kp.MinProxBinSize = testMinProxBinSize
if ctx.Config.Reachable != nil {
kp.Reachable = func(o network.OverlayAddr) bool {
return ctx.Config.Reachable(o.(*network.BzzAddr).ID())
}
}
kad := network.NewKademlia(addr.Over(), kp)
hp := network.NewHiveParams()
hp.KeepAliveInterval = time.Duration(200) * time.Millisecond
hp.Discovery = discoveryEnabled
log.Info(fmt.Sprintf("discovery for nodeID %s is %t", ctx.Config.ID.String(), hp.Discovery))
config := &network.BzzConfig{
OverlayAddr: addr.Over(),
UnderlayAddr: addr.Under(),
HiveParams: hp,
}
if persistenceEnabled {
log.Info(fmt.Sprintf("persistence enabled for nodeID %s", ctx.Config.ID.String()))
store, err := getDbStore(ctx.Config.ID.String())
if err != nil {
return nil, err
}
return network.NewBzz(config, kad, store, nil, nil), nil
}
return network.NewBzz(config, kad, nil, nil, nil), nil
}

File diff suppressed because one or more lines are too long

View File

@@ -0,0 +1,144 @@
// Copyright 2018 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
// You can run this simulation using
//
// go run ./swarm/network/simulations/overlay.go
package main
import (
"flag"
"fmt"
"net/http"
"runtime"
"sync"
"time"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/node"
"github.com/ethereum/go-ethereum/p2p/discover"
"github.com/ethereum/go-ethereum/p2p/simulations"
"github.com/ethereum/go-ethereum/p2p/simulations/adapters"
"github.com/ethereum/go-ethereum/swarm/network"
"github.com/ethereum/go-ethereum/swarm/state"
colorable "github.com/mattn/go-colorable"
)
var (
noDiscovery = flag.Bool("no-discovery", false, "disable discovery (useful if you want to load a snapshot)")
vmodule = flag.String("vmodule", "", "log filters for logger via Vmodule")
verbosity = flag.Int("verbosity", 0, "log filters for logger via Vmodule")
httpSimPort = 8888
)
func init() {
flag.Parse()
//initialize the logger
//this is a demonstration on how to use Vmodule for filtering logs
//provide -vmodule as param, and comma-separated values, e.g.:
//-vmodule overlay_test.go=4,simulations=3
//above examples sets overlay_test.go logs to level 4, while packages ending with "simulations" to 3
if *vmodule != "" {
//only enable the pattern matching handler if the flag has been provided
glogger := log.NewGlogHandler(log.StreamHandler(colorable.NewColorableStderr(), log.TerminalFormat(true)))
if *verbosity > 0 {
glogger.Verbosity(log.Lvl(*verbosity))
}
glogger.Vmodule(*vmodule)
log.Root().SetHandler(glogger)
}
}
type Simulation struct {
mtx sync.Mutex
stores map[discover.NodeID]*state.InmemoryStore
}
func NewSimulation() *Simulation {
return &Simulation{
stores: make(map[discover.NodeID]*state.InmemoryStore),
}
}
func (s *Simulation) NewService(ctx *adapters.ServiceContext) (node.Service, error) {
id := ctx.Config.ID
s.mtx.Lock()
store, ok := s.stores[id]
if !ok {
store = state.NewInmemoryStore()
s.stores[id] = store
}
s.mtx.Unlock()
addr := network.NewAddrFromNodeID(id)
kp := network.NewKadParams()
kp.MinProxBinSize = 2
kp.MaxBinSize = 4
kp.MinBinSize = 1
kp.MaxRetries = 1000
kp.RetryExponent = 2
kp.RetryInterval = 1000000
kad := network.NewKademlia(addr.Over(), kp)
hp := network.NewHiveParams()
hp.Discovery = !*noDiscovery
hp.KeepAliveInterval = 300 * time.Millisecond
config := &network.BzzConfig{
OverlayAddr: addr.Over(),
UnderlayAddr: addr.Under(),
HiveParams: hp,
}
return network.NewBzz(config, kad, store, nil, nil), nil
}
//create the simulation network
func newSimulationNetwork() *simulations.Network {
s := NewSimulation()
services := adapters.Services{
"overlay": s.NewService,
}
adapter := adapters.NewSimAdapter(services)
simNetwork := simulations.NewNetwork(adapter, &simulations.NetworkConfig{
DefaultService: "overlay",
})
return simNetwork
}
//return a new http server
func newOverlaySim(sim *simulations.Network) *simulations.Server {
return simulations.NewServer(sim)
}
// var server
func main() {
//cpu optimization
runtime.GOMAXPROCS(runtime.NumCPU())
//run the sim
runOverlaySim()
}
func runOverlaySim() {
//create the simulation network
net := newSimulationNetwork()
//create a http server with it
sim := newOverlaySim(net)
log.Info(fmt.Sprintf("starting simulation server on 0.0.0.0:%d...", httpSimPort))
//start the HTTP server
http.ListenAndServe(fmt.Sprintf(":%d", httpSimPort), sim)
}

View File

@@ -0,0 +1,195 @@
// Copyright 2018 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package main
import (
"context"
"encoding/json"
"fmt"
"io/ioutil"
"net/http"
"net/http/httptest"
"net/url"
"testing"
"time"
"github.com/ethereum/go-ethereum/p2p/discover"
"github.com/ethereum/go-ethereum/p2p/simulations"
"github.com/ethereum/go-ethereum/swarm/log"
)
var (
nodeCount = 16
)
//This test is used to test the overlay simulation.
//As the simulation is executed via a main, it is easily missed on changes
//An automated test will prevent that
//The test just connects to the simulations, starts the network,
//starts the mocker, gets the number of nodes, and stops it again.
//It also provides a documentation on the steps needed by frontends
//to use the simulations
func TestOverlaySim(t *testing.T) {
t.Skip("Test is flaky, see: https://github.com/ethersphere/go-ethereum/issues/592")
//start the simulation
log.Info("Start simulation backend")
//get the simulation networ; needed to subscribe for up events
net := newSimulationNetwork()
//create the overlay simulation
sim := newOverlaySim(net)
//create a http test server with it
srv := httptest.NewServer(sim)
defer srv.Close()
log.Debug("Http simulation server started. Start simulation network")
//start the simulation network (initialization of simulation)
resp, err := http.Post(srv.URL+"/start", "application/json", nil)
if err != nil {
t.Fatal(err)
}
defer resp.Body.Close()
if resp.StatusCode != http.StatusOK {
t.Fatalf("Expected Status Code %d, got %d", http.StatusOK, resp.StatusCode)
}
log.Debug("Start mocker")
//start the mocker, needs a node count and an ID
resp, err = http.PostForm(srv.URL+"/mocker/start",
url.Values{
"node-count": {fmt.Sprintf("%d", nodeCount)},
"mocker-type": {simulations.GetMockerList()[0]},
})
if err != nil {
t.Fatal(err)
}
defer resp.Body.Close()
if resp.StatusCode != http.StatusOK {
reason, err := ioutil.ReadAll(resp.Body)
if err != nil {
t.Fatal(err)
}
t.Fatalf("Expected Status Code %d, got %d, response body %s", http.StatusOK, resp.StatusCode, string(reason))
}
//variables needed to wait for nodes being up
var upCount int
trigger := make(chan discover.NodeID)
//wait for all nodes to be up
ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
defer cancel()
//start watching node up events...
go watchSimEvents(net, ctx, trigger)
//...and wait until all expected up events (nodeCount) have been received
LOOP:
for {
select {
case <-trigger:
//new node up event received, increase counter
upCount++
//all expected node up events received
if upCount == nodeCount {
break LOOP
}
case <-ctx.Done():
t.Fatalf("Timed out waiting for up events")
}
}
//at this point we can query the server
log.Info("Get number of nodes")
//get the number of nodes
resp, err = http.Get(srv.URL + "/nodes")
if err != nil {
t.Fatal(err)
}
defer resp.Body.Close()
if resp.StatusCode != http.StatusOK {
t.Fatalf("err %s", resp.Status)
}
b, err := ioutil.ReadAll(resp.Body)
if err != nil {
t.Fatal(err)
}
//unmarshal number of nodes from JSON response
var nodesArr []simulations.Node
err = json.Unmarshal(b, &nodesArr)
if err != nil {
t.Fatal(err)
}
//check if number of nodes received is same as sent
if len(nodesArr) != nodeCount {
t.Fatal(fmt.Errorf("Expected %d number of nodes, got %d", nodeCount, len(nodesArr)))
}
//need to let it run for a little while, otherwise stopping it immediately can crash due running nodes
//wanting to connect to already stopped nodes
time.Sleep(1 * time.Second)
log.Info("Stop the network")
//stop the network
resp, err = http.Post(srv.URL+"/stop", "application/json", nil)
if err != nil {
t.Fatal(err)
}
defer resp.Body.Close()
if resp.StatusCode != http.StatusOK {
t.Fatalf("err %s", resp.Status)
}
log.Info("Reset the network")
//reset the network (removes all nodes and connections)
resp, err = http.Post(srv.URL+"/reset", "application/json", nil)
if err != nil {
t.Fatal(err)
}
defer resp.Body.Close()
if resp.StatusCode != http.StatusOK {
t.Fatalf("err %s", resp.Status)
}
}
//watch for events so we know when all nodes are up
func watchSimEvents(net *simulations.Network, ctx context.Context, trigger chan discover.NodeID) {
events := make(chan *simulations.Event)
sub := net.Events().Subscribe(events)
defer sub.Unsubscribe()
for {
select {
case ev := <-events:
//only catch node up events
if ev.Type == simulations.EventTypeNode {
if ev.Node.Up {
log.Debug("got node up event", "event", ev, "node", ev.Node.Config.ID)
select {
case trigger <- ev.Node.Config.ID:
case <-ctx.Done():
return
}
}
}
case <-ctx.Done():
return
}
}
}

View File

@@ -0,0 +1,449 @@
// Copyright 2018 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package stream
import (
"context"
"encoding/binary"
"errors"
"flag"
"fmt"
"io"
"io/ioutil"
"os"
"sync/atomic"
"testing"
"time"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/node"
"github.com/ethereum/go-ethereum/p2p"
"github.com/ethereum/go-ethereum/p2p/discover"
"github.com/ethereum/go-ethereum/p2p/simulations/adapters"
p2ptest "github.com/ethereum/go-ethereum/p2p/testing"
"github.com/ethereum/go-ethereum/rpc"
"github.com/ethereum/go-ethereum/swarm/network"
"github.com/ethereum/go-ethereum/swarm/state"
"github.com/ethereum/go-ethereum/swarm/storage"
"github.com/ethereum/go-ethereum/swarm/storage/mock"
"github.com/ethereum/go-ethereum/swarm/storage/mock/db"
colorable "github.com/mattn/go-colorable"
)
var (
deliveries map[discover.NodeID]*Delivery
stores map[discover.NodeID]storage.ChunkStore
toAddr func(discover.NodeID) *network.BzzAddr
peerCount func(discover.NodeID) int
adapter = flag.String("adapter", "sim", "type of simulation: sim|exec|docker")
loglevel = flag.Int("loglevel", 2, "verbosity of logs")
nodes = flag.Int("nodes", 0, "number of nodes")
chunks = flag.Int("chunks", 0, "number of chunks")
useMockStore = flag.Bool("mockstore", false, "disabled mock store (default: enabled)")
)
var (
defaultSkipCheck bool
waitPeerErrC chan error
chunkSize = 4096
registries map[discover.NodeID]*TestRegistry
createStoreFunc func(id discover.NodeID, addr *network.BzzAddr) (storage.ChunkStore, error)
getRetrieveFunc = defaultRetrieveFunc
subscriptionCount = 0
globalStore mock.GlobalStorer
globalStoreDir string
)
var services = adapters.Services{
"streamer": NewStreamerService,
"intervalsStreamer": newIntervalsStreamerService,
}
func init() {
flag.Parse()
// register the Delivery service which will run as a devp2p
// protocol when using the exec adapter
adapters.RegisterServices(services)
log.PrintOrigins(true)
log.Root().SetHandler(log.LvlFilterHandler(log.Lvl(*loglevel), log.StreamHandler(colorable.NewColorableStderr(), log.TerminalFormat(true))))
}
func createGlobalStore() {
var err error
globalStoreDir, err = ioutil.TempDir("", "global.store")
if err != nil {
log.Error("Error initiating global store temp directory!", "err", err)
return
}
globalStore, err = db.NewGlobalStore(globalStoreDir)
if err != nil {
log.Error("Error initiating global store!", "err", err)
}
}
// NewStreamerService
func NewStreamerService(ctx *adapters.ServiceContext) (node.Service, error) {
var err error
id := ctx.Config.ID
addr := toAddr(id)
kad := network.NewKademlia(addr.Over(), network.NewKadParams())
stores[id], err = createStoreFunc(id, addr)
if err != nil {
return nil, err
}
store := stores[id].(*storage.LocalStore)
db := storage.NewDBAPI(store)
delivery := NewDelivery(kad, db)
deliveries[id] = delivery
r := NewRegistry(addr, delivery, db, state.NewInmemoryStore(), &RegistryOptions{
SkipCheck: defaultSkipCheck,
DoRetrieve: false,
})
RegisterSwarmSyncerServer(r, db)
RegisterSwarmSyncerClient(r, db)
go func() {
waitPeerErrC <- waitForPeers(r, 1*time.Second, peerCount(id))
}()
fileStore := storage.NewFileStore(storage.NewNetStore(store, getRetrieveFunc(id)), storage.NewFileStoreParams())
testRegistry := &TestRegistry{Registry: r, fileStore: fileStore}
registries[id] = testRegistry
return testRegistry, nil
}
func defaultRetrieveFunc(id discover.NodeID) func(chunk *storage.Chunk) error {
return nil
}
func datadirsCleanup() {
for _, id := range ids {
os.RemoveAll(datadirs[id])
}
if globalStoreDir != "" {
os.RemoveAll(globalStoreDir)
}
}
//local stores need to be cleaned up after the sim is done
func localStoreCleanup() {
log.Info("Cleaning up...")
for _, id := range ids {
registries[id].Close()
stores[id].Close()
}
log.Info("Local store cleanup done")
}
func newStreamerTester(t *testing.T) (*p2ptest.ProtocolTester, *Registry, *storage.LocalStore, func(), error) {
// setup
addr := network.RandomAddr() // tested peers peer address
to := network.NewKademlia(addr.OAddr, network.NewKadParams())
// temp datadir
datadir, err := ioutil.TempDir("", "streamer")
if err != nil {
return nil, nil, nil, func() {}, err
}
removeDataDir := func() {
os.RemoveAll(datadir)
}
params := storage.NewDefaultLocalStoreParams()
params.Init(datadir)
params.BaseKey = addr.Over()
localStore, err := storage.NewTestLocalStoreForAddr(params)
if err != nil {
return nil, nil, nil, removeDataDir, err
}
db := storage.NewDBAPI(localStore)
delivery := NewDelivery(to, db)
streamer := NewRegistry(addr, delivery, db, state.NewInmemoryStore(), &RegistryOptions{
SkipCheck: defaultSkipCheck,
})
teardown := func() {
streamer.Close()
removeDataDir()
}
protocolTester := p2ptest.NewProtocolTester(t, network.NewNodeIDFromAddr(addr), 1, streamer.runProtocol)
err = waitForPeers(streamer, 1*time.Second, 1)
if err != nil {
return nil, nil, nil, nil, errors.New("timeout: peer is not created")
}
return protocolTester, streamer, localStore, teardown, nil
}
func waitForPeers(streamer *Registry, timeout time.Duration, expectedPeers int) error {
ticker := time.NewTicker(10 * time.Millisecond)
timeoutTimer := time.NewTimer(timeout)
for {
select {
case <-ticker.C:
if streamer.peersCount() >= expectedPeers {
return nil
}
case <-timeoutTimer.C:
return errors.New("timeout")
}
}
}
type roundRobinStore struct {
index uint32
stores []storage.ChunkStore
}
func newRoundRobinStore(stores ...storage.ChunkStore) *roundRobinStore {
return &roundRobinStore{
stores: stores,
}
}
func (rrs *roundRobinStore) Get(addr storage.Address) (*storage.Chunk, error) {
return nil, errors.New("get not well defined on round robin store")
}
func (rrs *roundRobinStore) Put(chunk *storage.Chunk) {
i := atomic.AddUint32(&rrs.index, 1)
idx := int(i) % len(rrs.stores)
rrs.stores[idx].Put(chunk)
}
func (rrs *roundRobinStore) Close() {
for _, store := range rrs.stores {
store.Close()
}
}
type TestRegistry struct {
*Registry
fileStore *storage.FileStore
}
func (r *TestRegistry) APIs() []rpc.API {
a := r.Registry.APIs()
a = append(a, rpc.API{
Namespace: "stream",
Version: "3.0",
Service: r,
Public: true,
})
return a
}
func readAll(fileStore *storage.FileStore, hash []byte) (int64, error) {
r, _ := fileStore.Retrieve(hash)
buf := make([]byte, 1024)
var n int
var total int64
var err error
for (total == 0 || n > 0) && err == nil {
n, err = r.ReadAt(buf, total)
total += int64(n)
}
if err != nil && err != io.EOF {
return total, err
}
return total, nil
}
func (r *TestRegistry) ReadAll(hash common.Hash) (int64, error) {
return readAll(r.fileStore, hash[:])
}
func (r *TestRegistry) Start(server *p2p.Server) error {
return r.Registry.Start(server)
}
func (r *TestRegistry) Stop() error {
return r.Registry.Stop()
}
type TestExternalRegistry struct {
*Registry
}
func (r *TestExternalRegistry) APIs() []rpc.API {
a := r.Registry.APIs()
a = append(a, rpc.API{
Namespace: "stream",
Version: "3.0",
Service: r,
Public: true,
})
return a
}
func (r *TestExternalRegistry) GetHashes(ctx context.Context, peerId discover.NodeID, s Stream) (*rpc.Subscription, error) {
peer := r.getPeer(peerId)
client, err := peer.getClient(ctx, s)
if err != nil {
return nil, err
}
c := client.Client.(*testExternalClient)
notifier, supported := rpc.NotifierFromContext(ctx)
if !supported {
return nil, fmt.Errorf("Subscribe not supported")
}
sub := notifier.CreateSubscription()
go func() {
// if we begin sending event immediately some events
// will probably be dropped since the subscription ID might not be send to
// the client.
// ref: rpc/subscription_test.go#L65
time.Sleep(1 * time.Second)
for {
select {
case h := <-c.hashes:
<-c.enableNotificationsC // wait for notification subscription to complete
if err := notifier.Notify(sub.ID, h); err != nil {
log.Warn(fmt.Sprintf("rpc sub notifier notify stream %s: %v", s, err))
}
case err := <-sub.Err():
if err != nil {
log.Warn(fmt.Sprintf("caught subscription error in stream %s: %v", s, err))
}
case <-notifier.Closed():
log.Trace(fmt.Sprintf("rpc sub notifier closed"))
return
}
}
}()
return sub, nil
}
func (r *TestExternalRegistry) EnableNotifications(peerId discover.NodeID, s Stream) error {
peer := r.getPeer(peerId)
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
defer cancel()
client, err := peer.getClient(ctx, s)
if err != nil {
return err
}
close(client.Client.(*testExternalClient).enableNotificationsC)
return nil
}
// TODO: merge functionalities of testExternalClient and testExternalServer
// with testClient and testServer.
type testExternalClient struct {
hashes chan []byte
db *storage.DBAPI
enableNotificationsC chan struct{}
}
func newTestExternalClient(db *storage.DBAPI) *testExternalClient {
return &testExternalClient{
hashes: make(chan []byte),
db: db,
enableNotificationsC: make(chan struct{}),
}
}
func (c *testExternalClient) NeedData(hash []byte) func() {
chunk, _ := c.db.GetOrCreateRequest(hash)
if chunk.ReqC == nil {
return nil
}
c.hashes <- hash
return func() {
chunk.WaitToStore()
}
}
func (c *testExternalClient) BatchDone(Stream, uint64, []byte, []byte) func() (*TakeoverProof, error) {
return nil
}
func (c *testExternalClient) Close() {}
const testExternalServerBatchSize = 10
type testExternalServer struct {
t string
keyFunc func(key []byte, index uint64)
sessionAt uint64
maxKeys uint64
streamer *TestExternalRegistry
}
func newTestExternalServer(t string, sessionAt, maxKeys uint64, keyFunc func(key []byte, index uint64)) *testExternalServer {
if keyFunc == nil {
keyFunc = binary.BigEndian.PutUint64
}
return &testExternalServer{
t: t,
keyFunc: keyFunc,
sessionAt: sessionAt,
maxKeys: maxKeys,
}
}
func (s *testExternalServer) SetNextBatch(from uint64, to uint64) ([]byte, uint64, uint64, *HandoverProof, error) {
if from == 0 && to == 0 {
from = s.sessionAt
to = s.sessionAt + testExternalServerBatchSize
}
if to-from > testExternalServerBatchSize {
to = from + testExternalServerBatchSize - 1
}
if from >= s.maxKeys && to > s.maxKeys {
return nil, 0, 0, nil, io.EOF
}
if to > s.maxKeys {
to = s.maxKeys
}
b := make([]byte, HashSize*(to-from+1))
for i := from; i <= to; i++ {
s.keyFunc(b[(i-from)*HashSize:(i-from+1)*HashSize], i)
}
return b, from, to, nil, nil
}
func (s *testExternalServer) GetData([]byte) ([]byte, error) {
return make([]byte, 4096), nil
}
func (s *testExternalServer) Close() {}
// Sets the global value defaultSkipCheck.
// It should be used in test function defer to reset the global value
// to the original value.
//
// defer setDefaultSkipCheck(defaultSkipCheck)
// defaultSkipCheck = skipCheck
//
// This works as defer function arguments evaluations are evaluated as ususal,
// but only the function body invocation is deferred.
func setDefaultSkipCheck(skipCheck bool) {
defaultSkipCheck = skipCheck
}

View File

@@ -0,0 +1,272 @@
// Copyright 2018 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package stream
import (
"errors"
"time"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/metrics"
"github.com/ethereum/go-ethereum/p2p/discover"
"github.com/ethereum/go-ethereum/swarm/log"
"github.com/ethereum/go-ethereum/swarm/network"
"github.com/ethereum/go-ethereum/swarm/storage"
)
const (
swarmChunkServerStreamName = "RETRIEVE_REQUEST"
deliveryCap = 32
)
var (
processReceivedChunksCount = metrics.NewRegisteredCounter("network.stream.received_chunks.count", nil)
handleRetrieveRequestMsgCount = metrics.NewRegisteredCounter("network.stream.handle_retrieve_request_msg.count", nil)
requestFromPeersCount = metrics.NewRegisteredCounter("network.stream.request_from_peers.count", nil)
requestFromPeersEachCount = metrics.NewRegisteredCounter("network.stream.request_from_peers_each.count", nil)
)
type Delivery struct {
db *storage.DBAPI
overlay network.Overlay
receiveC chan *ChunkDeliveryMsg
getPeer func(discover.NodeID) *Peer
}
func NewDelivery(overlay network.Overlay, db *storage.DBAPI) *Delivery {
d := &Delivery{
db: db,
overlay: overlay,
receiveC: make(chan *ChunkDeliveryMsg, deliveryCap),
}
go d.processReceivedChunks()
return d
}
// SwarmChunkServer implements Server
type SwarmChunkServer struct {
deliveryC chan []byte
batchC chan []byte
db *storage.DBAPI
currentLen uint64
quit chan struct{}
}
// NewSwarmChunkServer is SwarmChunkServer constructor
func NewSwarmChunkServer(db *storage.DBAPI) *SwarmChunkServer {
s := &SwarmChunkServer{
deliveryC: make(chan []byte, deliveryCap),
batchC: make(chan []byte),
db: db,
quit: make(chan struct{}),
}
go s.processDeliveries()
return s
}
// processDeliveries handles delivered chunk hashes
func (s *SwarmChunkServer) processDeliveries() {
var hashes []byte
var batchC chan []byte
for {
select {
case <-s.quit:
return
case hash := <-s.deliveryC:
hashes = append(hashes, hash...)
batchC = s.batchC
case batchC <- hashes:
hashes = nil
batchC = nil
}
}
}
// SetNextBatch
func (s *SwarmChunkServer) SetNextBatch(_, _ uint64) (hashes []byte, from uint64, to uint64, proof *HandoverProof, err error) {
select {
case hashes = <-s.batchC:
case <-s.quit:
return
}
from = s.currentLen
s.currentLen += uint64(len(hashes))
to = s.currentLen
return
}
// Close needs to be called on a stream server
func (s *SwarmChunkServer) Close() {
close(s.quit)
}
// GetData retrives chunk data from db store
func (s *SwarmChunkServer) GetData(key []byte) ([]byte, error) {
chunk, err := s.db.Get(storage.Address(key))
if err == storage.ErrFetching {
<-chunk.ReqC
} else if err != nil {
return nil, err
}
return chunk.SData, nil
}
// RetrieveRequestMsg is the protocol msg for chunk retrieve requests
type RetrieveRequestMsg struct {
Addr storage.Address
SkipCheck bool
}
func (d *Delivery) handleRetrieveRequestMsg(sp *Peer, req *RetrieveRequestMsg) error {
log.Trace("received request", "peer", sp.ID(), "hash", req.Addr)
handleRetrieveRequestMsgCount.Inc(1)
s, err := sp.getServer(NewStream(swarmChunkServerStreamName, "", false))
if err != nil {
return err
}
streamer := s.Server.(*SwarmChunkServer)
chunk, created := d.db.GetOrCreateRequest(req.Addr)
if chunk.ReqC != nil {
if created {
if err := d.RequestFromPeers(chunk.Addr[:], true, sp.ID()); err != nil {
log.Warn("unable to forward chunk request", "peer", sp.ID(), "key", chunk.Addr, "err", err)
chunk.SetErrored(storage.ErrChunkForward)
return nil
}
}
go func() {
t := time.NewTimer(10 * time.Minute)
defer t.Stop()
log.Debug("waiting delivery", "peer", sp.ID(), "hash", req.Addr, "node", common.Bytes2Hex(d.overlay.BaseAddr()), "created", created)
start := time.Now()
select {
case <-chunk.ReqC:
log.Debug("retrieve request ReqC closed", "peer", sp.ID(), "hash", req.Addr, "time", time.Since(start))
case <-t.C:
log.Debug("retrieve request timeout", "peer", sp.ID(), "hash", req.Addr)
chunk.SetErrored(storage.ErrChunkTimeout)
return
}
chunk.SetErrored(nil)
if req.SkipCheck {
err := sp.Deliver(chunk, s.priority)
if err != nil {
log.Warn("ERROR in handleRetrieveRequestMsg, DROPPING peer!", "err", err)
sp.Drop(err)
}
}
streamer.deliveryC <- chunk.Addr[:]
}()
return nil
}
// TODO: call the retrieve function of the outgoing syncer
if req.SkipCheck {
log.Trace("deliver", "peer", sp.ID(), "hash", chunk.Addr)
if length := len(chunk.SData); length < 9 {
log.Error("Chunk.SData to deliver is too short", "len(chunk.SData)", length, "address", chunk.Addr)
}
return sp.Deliver(chunk, s.priority)
}
streamer.deliveryC <- chunk.Addr[:]
return nil
}
type ChunkDeliveryMsg struct {
Addr storage.Address
SData []byte // the stored chunk Data (incl size)
peer *Peer // set in handleChunkDeliveryMsg
}
func (d *Delivery) handleChunkDeliveryMsg(sp *Peer, req *ChunkDeliveryMsg) error {
req.peer = sp
d.receiveC <- req
return nil
}
func (d *Delivery) processReceivedChunks() {
R:
for req := range d.receiveC {
processReceivedChunksCount.Inc(1)
// this should be has locally
chunk, err := d.db.Get(req.Addr)
if err == nil {
continue R
}
if err != storage.ErrFetching {
log.Error("processReceivedChunks db error", "addr", req.Addr, "err", err, "chunk", chunk)
continue R
}
select {
case <-chunk.ReqC:
log.Error("someone else delivered?", "hash", chunk.Addr.Hex())
continue R
default:
}
chunk.SData = req.SData
d.db.Put(chunk)
go func(req *ChunkDeliveryMsg) {
err := chunk.WaitToStore()
if err == storage.ErrChunkInvalid {
req.peer.Drop(err)
}
}(req)
}
}
// RequestFromPeers sends a chunk retrieve request to
func (d *Delivery) RequestFromPeers(hash []byte, skipCheck bool, peersToSkip ...discover.NodeID) error {
var success bool
var err error
requestFromPeersCount.Inc(1)
d.overlay.EachConn(hash, 255, func(p network.OverlayConn, po int, nn bool) bool {
spId := p.(network.Peer).ID()
for _, p := range peersToSkip {
if p == spId {
log.Trace("Delivery.RequestFromPeers: skip peer", "peer", spId)
return true
}
}
sp := d.getPeer(spId)
if sp == nil {
log.Warn("Delivery.RequestFromPeers: peer not found", "id", spId)
return true
}
// TODO: skip light nodes that do not accept retrieve requests
err = sp.SendPriority(&RetrieveRequestMsg{
Addr: hash,
SkipCheck: skipCheck,
}, Top)
if err != nil {
return true
}
requestFromPeersEachCount.Inc(1)
success = true
return false
})
if success {
return nil
}
return errors.New("no peer found")
}

View File

@@ -0,0 +1,699 @@
// Copyright 2018 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package stream
import (
"bytes"
"context"
crand "crypto/rand"
"fmt"
"io"
"sync"
"testing"
"time"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/p2p/discover"
"github.com/ethereum/go-ethereum/p2p/simulations"
p2ptest "github.com/ethereum/go-ethereum/p2p/testing"
"github.com/ethereum/go-ethereum/rpc"
"github.com/ethereum/go-ethereum/swarm/log"
"github.com/ethereum/go-ethereum/swarm/network"
streamTesting "github.com/ethereum/go-ethereum/swarm/network/stream/testing"
"github.com/ethereum/go-ethereum/swarm/storage"
)
func TestStreamerRetrieveRequest(t *testing.T) {
tester, streamer, _, teardown, err := newStreamerTester(t)
defer teardown()
if err != nil {
t.Fatal(err)
}
peerID := tester.IDs[0]
streamer.delivery.RequestFromPeers(hash0[:], true)
err = tester.TestExchanges(p2ptest.Exchange{
Label: "RetrieveRequestMsg",
Expects: []p2ptest.Expect{
{
Code: 5,
Msg: &RetrieveRequestMsg{
Addr: hash0[:],
SkipCheck: true,
},
Peer: peerID,
},
},
})
if err != nil {
t.Fatalf("Expected no error, got %v", err)
}
}
func TestStreamerUpstreamRetrieveRequestMsgExchangeWithoutStore(t *testing.T) {
tester, streamer, _, teardown, err := newStreamerTester(t)
defer teardown()
if err != nil {
t.Fatal(err)
}
peerID := tester.IDs[0]
chunk := storage.NewChunk(storage.Address(hash0[:]), nil)
peer := streamer.getPeer(peerID)
peer.handleSubscribeMsg(&SubscribeMsg{
Stream: NewStream(swarmChunkServerStreamName, "", false),
History: nil,
Priority: Top,
})
err = tester.TestExchanges(p2ptest.Exchange{
Label: "RetrieveRequestMsg",
Triggers: []p2ptest.Trigger{
{
Code: 5,
Msg: &RetrieveRequestMsg{
Addr: chunk.Addr[:],
},
Peer: peerID,
},
},
Expects: []p2ptest.Expect{
{
Code: 1,
Msg: &OfferedHashesMsg{
HandoverProof: nil,
Hashes: nil,
From: 0,
To: 0,
},
Peer: peerID,
},
},
})
expectedError := `exchange #0 "RetrieveRequestMsg": timed out`
if err == nil || err.Error() != expectedError {
t.Fatalf("Expected error %v, got %v", expectedError, err)
}
}
// upstream request server receives a retrieve Request and responds with
// offered hashes or delivery if skipHash is set to true
func TestStreamerUpstreamRetrieveRequestMsgExchange(t *testing.T) {
tester, streamer, localStore, teardown, err := newStreamerTester(t)
defer teardown()
if err != nil {
t.Fatal(err)
}
peerID := tester.IDs[0]
peer := streamer.getPeer(peerID)
stream := NewStream(swarmChunkServerStreamName, "", false)
peer.handleSubscribeMsg(&SubscribeMsg{
Stream: stream,
History: nil,
Priority: Top,
})
hash := storage.Address(hash0[:])
chunk := storage.NewChunk(hash, nil)
chunk.SData = hash
localStore.Put(chunk)
chunk.WaitToStore()
err = tester.TestExchanges(p2ptest.Exchange{
Label: "RetrieveRequestMsg",
Triggers: []p2ptest.Trigger{
{
Code: 5,
Msg: &RetrieveRequestMsg{
Addr: hash,
},
Peer: peerID,
},
},
Expects: []p2ptest.Expect{
{
Code: 1,
Msg: &OfferedHashesMsg{
HandoverProof: &HandoverProof{
Handover: &Handover{},
},
Hashes: hash,
From: 0,
// TODO: why is this 32???
To: 32,
Stream: stream,
},
Peer: peerID,
},
},
})
if err != nil {
t.Fatal(err)
}
hash = storage.Address(hash1[:])
chunk = storage.NewChunk(hash, nil)
chunk.SData = hash1[:]
localStore.Put(chunk)
chunk.WaitToStore()
err = tester.TestExchanges(p2ptest.Exchange{
Label: "RetrieveRequestMsg",
Triggers: []p2ptest.Trigger{
{
Code: 5,
Msg: &RetrieveRequestMsg{
Addr: hash,
SkipCheck: true,
},
Peer: peerID,
},
},
Expects: []p2ptest.Expect{
{
Code: 6,
Msg: &ChunkDeliveryMsg{
Addr: hash,
SData: hash,
},
Peer: peerID,
},
},
})
if err != nil {
t.Fatal(err)
}
}
func TestStreamerDownstreamChunkDeliveryMsgExchange(t *testing.T) {
tester, streamer, localStore, teardown, err := newStreamerTester(t)
defer teardown()
if err != nil {
t.Fatal(err)
}
streamer.RegisterClientFunc("foo", func(p *Peer, t string, live bool) (Client, error) {
return &testClient{
t: t,
}, nil
})
peerID := tester.IDs[0]
stream := NewStream("foo", "", true)
err = streamer.Subscribe(peerID, stream, NewRange(5, 8), Top)
if err != nil {
t.Fatalf("Expected no error, got %v", err)
}
chunkKey := hash0[:]
chunkData := hash1[:]
chunk, created := localStore.GetOrCreateRequest(chunkKey)
if !created {
t.Fatal("chunk already exists")
}
select {
case <-chunk.ReqC:
t.Fatal("chunk is already received")
default:
}
err = tester.TestExchanges(p2ptest.Exchange{
Label: "Subscribe message",
Expects: []p2ptest.Expect{
{
Code: 4,
Msg: &SubscribeMsg{
Stream: stream,
History: NewRange(5, 8),
Priority: Top,
},
Peer: peerID,
},
},
},
p2ptest.Exchange{
Label: "ChunkDeliveryRequest message",
Triggers: []p2ptest.Trigger{
{
Code: 6,
Msg: &ChunkDeliveryMsg{
Addr: chunkKey,
SData: chunkData,
},
Peer: peerID,
},
},
})
if err != nil {
t.Fatalf("Expected no error, got %v", err)
}
timeout := time.NewTimer(1 * time.Second)
select {
case <-timeout.C:
t.Fatal("timeout receiving chunk")
case <-chunk.ReqC:
}
storedChunk, err := localStore.Get(chunkKey)
if err != nil {
t.Fatalf("Expected no error, got %v", err)
}
if !bytes.Equal(storedChunk.SData, chunkData) {
t.Fatal("Retrieved chunk has different data than original")
}
}
func TestDeliveryFromNodes(t *testing.T) {
testDeliveryFromNodes(t, 2, 1, dataChunkCount, true)
testDeliveryFromNodes(t, 2, 1, dataChunkCount, false)
testDeliveryFromNodes(t, 4, 1, dataChunkCount, true)
testDeliveryFromNodes(t, 4, 1, dataChunkCount, false)
testDeliveryFromNodes(t, 8, 1, dataChunkCount, true)
testDeliveryFromNodes(t, 8, 1, dataChunkCount, false)
testDeliveryFromNodes(t, 16, 1, dataChunkCount, true)
testDeliveryFromNodes(t, 16, 1, dataChunkCount, false)
}
func testDeliveryFromNodes(t *testing.T, nodes, conns, chunkCount int, skipCheck bool) {
defaultSkipCheck = skipCheck
toAddr = network.NewAddrFromNodeID
createStoreFunc = createTestLocalStorageFromSim
conf := &streamTesting.RunConfig{
Adapter: *adapter,
NodeCount: nodes,
ConnLevel: conns,
ToAddr: toAddr,
Services: services,
EnableMsgEvents: false,
}
sim, teardown, err := streamTesting.NewSimulation(conf)
var rpcSubscriptionsWg sync.WaitGroup
defer func() {
rpcSubscriptionsWg.Wait()
teardown()
}()
if err != nil {
t.Fatal(err.Error())
}
stores = make(map[discover.NodeID]storage.ChunkStore)
for i, id := range sim.IDs {
stores[id] = sim.Stores[i]
}
registries = make(map[discover.NodeID]*TestRegistry)
deliveries = make(map[discover.NodeID]*Delivery)
peerCount = func(id discover.NodeID) int {
if sim.IDs[0] == id || sim.IDs[nodes-1] == id {
return 1
}
return 2
}
// here we distribute chunks of a random file into Stores of nodes 1 to nodes
rrFileStore := storage.NewFileStore(newRoundRobinStore(sim.Stores[1:]...), storage.NewFileStoreParams())
size := chunkCount * chunkSize
fileHash, wait, err := rrFileStore.Store(io.LimitReader(crand.Reader, int64(size)), int64(size), false)
// wait until all chunks stored
wait()
if err != nil {
t.Fatal(err.Error())
}
errc := make(chan error, 1)
waitPeerErrC = make(chan error)
quitC := make(chan struct{})
defer close(quitC)
action := func(ctx context.Context) error {
// each node Subscribes to each other's swarmChunkServerStreamName
// need to wait till an aynchronous process registers the peers in streamer.peers
// that is used by Subscribe
// using a global err channel to share betweem action and node service
i := 0
for err := range waitPeerErrC {
if err != nil {
return fmt.Errorf("error waiting for peers: %s", err)
}
i++
if i == nodes {
break
}
}
// each node subscribes to the upstream swarm chunk server stream
// which responds to chunk retrieve requests all but the last node in the chain does not
for j := 0; j < nodes-1; j++ {
id := sim.IDs[j]
err := sim.CallClient(id, func(client *rpc.Client) error {
doneC, err := streamTesting.WatchDisconnections(id, client, errc, quitC)
if err != nil {
return err
}
rpcSubscriptionsWg.Add(1)
go func() {
<-doneC
rpcSubscriptionsWg.Done()
}()
ctx, cancel := context.WithTimeout(ctx, 1*time.Second)
defer cancel()
sid := sim.IDs[j+1]
return client.CallContext(ctx, nil, "stream_subscribeStream", sid, NewStream(swarmChunkServerStreamName, "", false), NewRange(0, 0), Top)
})
if err != nil {
return err
}
}
// create a retriever FileStore for the pivot node
delivery := deliveries[sim.IDs[0]]
retrieveFunc := func(chunk *storage.Chunk) error {
return delivery.RequestFromPeers(chunk.Addr[:], skipCheck)
}
netStore := storage.NewNetStore(sim.Stores[0].(*storage.LocalStore), retrieveFunc)
fileStore := storage.NewFileStore(netStore, storage.NewFileStoreParams())
go func() {
// start the retrieval on the pivot node - this will spawn retrieve requests for missing chunks
// we must wait for the peer connections to have started before requesting
n, err := readAll(fileStore, fileHash)
log.Info(fmt.Sprintf("retrieved %v", fileHash), "read", n, "err", err)
if err != nil {
errc <- fmt.Errorf("requesting chunks action error: %v", err)
}
}()
return nil
}
check := func(ctx context.Context, id discover.NodeID) (bool, error) {
select {
case err := <-errc:
return false, err
case <-ctx.Done():
return false, ctx.Err()
default:
}
var total int64
err := sim.CallClient(id, func(client *rpc.Client) error {
ctx, cancel := context.WithTimeout(ctx, 5*time.Second)
defer cancel()
return client.CallContext(ctx, &total, "stream_readAll", common.BytesToHash(fileHash))
})
log.Info(fmt.Sprintf("check if %08x is available locally: number of bytes read %v/%v (error: %v)", fileHash, total, size, err))
if err != nil || total != int64(size) {
return false, nil
}
return true, nil
}
conf.Step = &simulations.Step{
Action: action,
Trigger: streamTesting.Trigger(10*time.Millisecond, quitC, sim.IDs[0]),
// we are only testing the pivot node (net.Nodes[0])
Expect: &simulations.Expectation{
Nodes: sim.IDs[0:1],
Check: check,
},
}
startedAt := time.Now()
timeout := 300 * time.Second
ctx, cancel := context.WithTimeout(context.Background(), timeout)
defer cancel()
result, err := sim.Run(ctx, conf)
finishedAt := time.Now()
if err != nil {
t.Fatalf("Setting up simulation failed: %v", err)
}
if result.Error != nil {
t.Fatalf("Simulation failed: %s", result.Error)
}
streamTesting.CheckResult(t, result, startedAt, finishedAt)
}
func BenchmarkDeliveryFromNodesWithoutCheck(b *testing.B) {
for chunks := 32; chunks <= 128; chunks *= 2 {
for i := 2; i < 32; i *= 2 {
b.Run(
fmt.Sprintf("nodes=%v,chunks=%v", i, chunks),
func(b *testing.B) {
benchmarkDeliveryFromNodes(b, i, 1, chunks, true)
},
)
}
}
}
func BenchmarkDeliveryFromNodesWithCheck(b *testing.B) {
for chunks := 32; chunks <= 128; chunks *= 2 {
for i := 2; i < 32; i *= 2 {
b.Run(
fmt.Sprintf("nodes=%v,chunks=%v", i, chunks),
func(b *testing.B) {
benchmarkDeliveryFromNodes(b, i, 1, chunks, false)
},
)
}
}
}
func benchmarkDeliveryFromNodes(b *testing.B, nodes, conns, chunkCount int, skipCheck bool) {
defaultSkipCheck = skipCheck
toAddr = network.NewAddrFromNodeID
createStoreFunc = createTestLocalStorageFromSim
registries = make(map[discover.NodeID]*TestRegistry)
timeout := 300 * time.Second
ctx, cancel := context.WithTimeout(context.Background(), timeout)
defer cancel()
conf := &streamTesting.RunConfig{
Adapter: *adapter,
NodeCount: nodes,
ConnLevel: conns,
ToAddr: toAddr,
Services: services,
EnableMsgEvents: false,
}
sim, teardown, err := streamTesting.NewSimulation(conf)
var rpcSubscriptionsWg sync.WaitGroup
defer func() {
rpcSubscriptionsWg.Wait()
teardown()
}()
if err != nil {
b.Fatal(err.Error())
}
stores = make(map[discover.NodeID]storage.ChunkStore)
deliveries = make(map[discover.NodeID]*Delivery)
for i, id := range sim.IDs {
stores[id] = sim.Stores[i]
}
peerCount = func(id discover.NodeID) int {
if sim.IDs[0] == id || sim.IDs[nodes-1] == id {
return 1
}
return 2
}
// wait channel for all nodes all peer connections to set up
waitPeerErrC = make(chan error)
// create a FileStore for the last node in the chain which we are gonna write to
remoteFileStore := storage.NewFileStore(sim.Stores[nodes-1], storage.NewFileStoreParams())
// channel to signal simulation initialisation with action call complete
// or node disconnections
disconnectC := make(chan error)
quitC := make(chan struct{})
initC := make(chan error)
action := func(ctx context.Context) error {
// each node Subscribes to each other's swarmChunkServerStreamName
// need to wait till an aynchronous process registers the peers in streamer.peers
// that is used by Subscribe
// waitPeerErrC using a global err channel to share betweem action and node service
i := 0
for err := range waitPeerErrC {
if err != nil {
return fmt.Errorf("error waiting for peers: %s", err)
}
i++
if i == nodes {
break
}
}
var err error
// each node except the last one subscribes to the upstream swarm chunk server stream
// which responds to chunk retrieve requests
for j := 0; j < nodes-1; j++ {
id := sim.IDs[j]
err = sim.CallClient(id, func(client *rpc.Client) error {
doneC, err := streamTesting.WatchDisconnections(id, client, disconnectC, quitC)
if err != nil {
return err
}
rpcSubscriptionsWg.Add(1)
go func() {
<-doneC
rpcSubscriptionsWg.Done()
}()
ctx, cancel := context.WithTimeout(ctx, 1*time.Second)
defer cancel()
sid := sim.IDs[j+1] // the upstream peer's id
return client.CallContext(ctx, nil, "stream_subscribeStream", sid, NewStream(swarmChunkServerStreamName, "", false), NewRange(0, 0), Top)
})
if err != nil {
break
}
}
initC <- err
return nil
}
// the check function is only triggered when the benchmark finishes
trigger := make(chan discover.NodeID)
check := func(ctx context.Context, id discover.NodeID) (_ bool, err error) {
return true, nil
}
conf.Step = &simulations.Step{
Action: action,
Trigger: trigger,
// we are only testing the pivot node (net.Nodes[0])
Expect: &simulations.Expectation{
Nodes: sim.IDs[0:1],
Check: check,
},
}
// run the simulation in the background
errc := make(chan error)
go func() {
_, err := sim.Run(ctx, conf)
close(quitC)
errc <- err
}()
// wait for simulation action to complete stream subscriptions
err = <-initC
if err != nil {
b.Fatalf("simulation failed to initialise. expected no error. got %v", err)
}
// create a retriever FileStore for the pivot node
// by now deliveries are set for each node by the streamer service
delivery := deliveries[sim.IDs[0]]
retrieveFunc := func(chunk *storage.Chunk) error {
return delivery.RequestFromPeers(chunk.Addr[:], skipCheck)
}
netStore := storage.NewNetStore(sim.Stores[0].(*storage.LocalStore), retrieveFunc)
// benchmark loop
b.ResetTimer()
b.StopTimer()
Loop:
for i := 0; i < b.N; i++ {
// uploading chunkCount random chunks to the last node
hashes := make([]storage.Address, chunkCount)
for i := 0; i < chunkCount; i++ {
// create actual size real chunks
hash, wait, err := remoteFileStore.Store(io.LimitReader(crand.Reader, int64(chunkSize)), int64(chunkSize), false)
// wait until all chunks stored
wait()
if err != nil {
b.Fatalf("expected no error. got %v", err)
}
// collect the hashes
hashes[i] = hash
}
// now benchmark the actual retrieval
// netstore.Get is called for each hash in a go routine and errors are collected
b.StartTimer()
errs := make(chan error)
for _, hash := range hashes {
go func(h storage.Address) {
_, err := netStore.Get(h)
log.Warn("test check netstore get", "hash", h, "err", err)
errs <- err
}(hash)
}
// count and report retrieval errors
// if there are misses then chunk timeout is too low for the distance and volume (?)
var total, misses int
for err := range errs {
if err != nil {
log.Warn(err.Error())
misses++
}
total++
if total == chunkCount {
break
}
}
b.StopTimer()
select {
case err = <-disconnectC:
if err != nil {
break Loop
}
default:
}
if misses > 0 {
err = fmt.Errorf("%v chunk not found out of %v", misses, total)
break Loop
}
}
select {
case <-quitC:
case trigger <- sim.IDs[0]:
}
if err == nil {
err = <-errc
} else {
if e := <-errc; e != nil {
b.Errorf("sim.Run function error: %v", e)
}
}
// benchmark over, trigger the check function to conclude the simulation
if err != nil {
b.Fatalf("expected no error. got %v", err)
}
}
func createTestLocalStorageFromSim(id discover.NodeID, addr *network.BzzAddr) (storage.ChunkStore, error) {
return stores[id], nil
}

View File

@@ -0,0 +1,42 @@
// Copyright 2018 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package intervals
import (
"io/ioutil"
"os"
"testing"
"github.com/ethereum/go-ethereum/swarm/state"
)
// TestDBStore tests basic functionality of DBStore.
func TestDBStore(t *testing.T) {
dir, err := ioutil.TempDir("", "intervals_test_db_store")
if err != nil {
panic(err)
}
defer os.RemoveAll(dir)
store, err := state.NewDBStore(dir)
if err != nil {
t.Fatal(err)
}
defer store.Close()
testStore(t, store)
}

View File

@@ -0,0 +1,206 @@
// Copyright 2018 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package intervals
import (
"bytes"
"fmt"
"strconv"
"sync"
)
// Intervals store a list of intervals. Its purpose is to provide
// methods to add new intervals and retrieve missing intervals that
// need to be added.
// It may be used in synchronization of streaming data to persist
// retrieved data ranges between sessions.
type Intervals struct {
start uint64
ranges [][2]uint64
mu sync.RWMutex
}
// New creates a new instance of Intervals.
// Start argument limits the lower bound of intervals.
// No range bellow start bound will be added by Add method or
// returned by Next method. This limit may be used for
// tracking "live" synchronization, where the sync session
// starts from a specific value, and if "live" sync intervals
// need to be merged with historical ones, it can be safely done.
func NewIntervals(start uint64) *Intervals {
return &Intervals{
start: start,
}
}
// Add adds a new range to intervals. Range start and end are values
// are both inclusive.
func (i *Intervals) Add(start, end uint64) {
i.mu.Lock()
defer i.mu.Unlock()
i.add(start, end)
}
func (i *Intervals) add(start, end uint64) {
if start < i.start {
start = i.start
}
if end < i.start {
return
}
minStartJ := -1
maxEndJ := -1
j := 0
for ; j < len(i.ranges); j++ {
if minStartJ < 0 {
if (start <= i.ranges[j][0] && end+1 >= i.ranges[j][0]) || (start <= i.ranges[j][1]+1 && end+1 >= i.ranges[j][1]) {
if i.ranges[j][0] < start {
start = i.ranges[j][0]
}
minStartJ = j
}
}
if (start <= i.ranges[j][1] && end+1 >= i.ranges[j][1]) || (start <= i.ranges[j][0] && end+1 >= i.ranges[j][0]) {
if i.ranges[j][1] > end {
end = i.ranges[j][1]
}
maxEndJ = j
}
if end+1 <= i.ranges[j][0] {
break
}
}
if minStartJ < 0 && maxEndJ < 0 {
i.ranges = append(i.ranges[:j], append([][2]uint64{{start, end}}, i.ranges[j:]...)...)
return
}
if minStartJ >= 0 {
i.ranges[minStartJ][0] = start
}
if maxEndJ >= 0 {
i.ranges[maxEndJ][1] = end
}
if minStartJ >= 0 && maxEndJ >= 0 && minStartJ != maxEndJ {
i.ranges[maxEndJ][0] = start
i.ranges = append(i.ranges[:minStartJ], i.ranges[maxEndJ:]...)
}
}
// Merge adds all the intervals from the the m Interval to current one.
func (i *Intervals) Merge(m *Intervals) {
m.mu.RLock()
defer m.mu.RUnlock()
i.mu.Lock()
defer i.mu.Unlock()
for _, r := range m.ranges {
i.add(r[0], r[1])
}
}
// Next returns the first range interval that is not fulfilled. Returned
// start and end values are both inclusive, meaning that the whole range
// including start and end need to be added in order to full the gap
// in intervals.
// Returned value for end is 0 if the next interval is after the whole
// range that is stored in Intervals. Zero end value represents no limit
// on the next interval length.
func (i *Intervals) Next() (start, end uint64) {
i.mu.RLock()
defer i.mu.RUnlock()
l := len(i.ranges)
if l == 0 {
return i.start, 0
}
if i.ranges[0][0] != i.start {
return i.start, i.ranges[0][0] - 1
}
if l == 1 {
return i.ranges[0][1] + 1, 0
}
return i.ranges[0][1] + 1, i.ranges[1][0] - 1
}
// Last returns the value that is at the end of the last interval.
func (i *Intervals) Last() (end uint64) {
i.mu.RLock()
defer i.mu.RUnlock()
l := len(i.ranges)
if l == 0 {
return 0
}
return i.ranges[l-1][1]
}
// String returns a descriptive representation of range intervals
// in [] notation, as a list of two element vectors.
func (i *Intervals) String() string {
return fmt.Sprint(i.ranges)
}
// MarshalBinary encodes Intervals parameters into a semicolon separated list.
// The first element in the list is base36-encoded start value. The following
// elements are two base36-encoded value ranges separated by comma.
func (i *Intervals) MarshalBinary() (data []byte, err error) {
d := make([][]byte, len(i.ranges)+1)
d[0] = []byte(strconv.FormatUint(i.start, 36))
for j := range i.ranges {
r := i.ranges[j]
d[j+1] = []byte(strconv.FormatUint(r[0], 36) + "," + strconv.FormatUint(r[1], 36))
}
return bytes.Join(d, []byte(";")), nil
}
// UnmarshalBinary decodes data according to the Intervals.MarshalBinary format.
func (i *Intervals) UnmarshalBinary(data []byte) (err error) {
d := bytes.Split(data, []byte(";"))
l := len(d)
if l == 0 {
return nil
}
if l >= 1 {
i.start, err = strconv.ParseUint(string(d[0]), 36, 64)
if err != nil {
return err
}
}
if l == 1 {
return nil
}
i.ranges = make([][2]uint64, 0, l-1)
for j := 1; j < l; j++ {
r := bytes.SplitN(d[j], []byte(","), 2)
if len(r) < 2 {
return fmt.Errorf("range %d has less then 2 elements", j)
}
start, err := strconv.ParseUint(string(r[0]), 36, 64)
if err != nil {
return fmt.Errorf("parsing the first element in range %d: %v", j, err)
}
end, err := strconv.ParseUint(string(r[1]), 36, 64)
if err != nil {
return fmt.Errorf("parsing the second element in range %d: %v", j, err)
}
i.ranges = append(i.ranges, [2]uint64{start, end})
}
return nil
}

View File

@@ -0,0 +1,395 @@
// Copyright 2018 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package intervals
import "testing"
// Test tests Interval methods Add, Next and Last for various
// initial state.
func Test(t *testing.T) {
for i, tc := range []struct {
startLimit uint64
initial [][2]uint64
start uint64
end uint64
expected string
nextStart uint64
nextEnd uint64
last uint64
}{
{
initial: nil,
start: 0,
end: 0,
expected: "[[0 0]]",
nextStart: 1,
nextEnd: 0,
last: 0,
},
{
initial: nil,
start: 0,
end: 10,
expected: "[[0 10]]",
nextStart: 11,
nextEnd: 0,
last: 10,
},
{
initial: nil,
start: 5,
end: 15,
expected: "[[5 15]]",
nextStart: 0,
nextEnd: 4,
last: 15,
},
{
initial: [][2]uint64{{0, 0}},
start: 0,
end: 0,
expected: "[[0 0]]",
nextStart: 1,
nextEnd: 0,
last: 0,
},
{
initial: [][2]uint64{{0, 0}},
start: 5,
end: 15,
expected: "[[0 0] [5 15]]",
nextStart: 1,
nextEnd: 4,
last: 15,
},
{
initial: [][2]uint64{{5, 15}},
start: 5,
end: 15,
expected: "[[5 15]]",
nextStart: 0,
nextEnd: 4,
last: 15,
},
{
initial: [][2]uint64{{5, 15}},
start: 5,
end: 20,
expected: "[[5 20]]",
nextStart: 0,
nextEnd: 4,
last: 20,
},
{
initial: [][2]uint64{{5, 15}},
start: 10,
end: 20,
expected: "[[5 20]]",
nextStart: 0,
nextEnd: 4,
last: 20,
},
{
initial: [][2]uint64{{5, 15}},
start: 0,
end: 20,
expected: "[[0 20]]",
nextStart: 21,
nextEnd: 0,
last: 20,
},
{
initial: [][2]uint64{{5, 15}},
start: 2,
end: 10,
expected: "[[2 15]]",
nextStart: 0,
nextEnd: 1,
last: 15,
},
{
initial: [][2]uint64{{5, 15}},
start: 2,
end: 4,
expected: "[[2 15]]",
nextStart: 0,
nextEnd: 1,
last: 15,
},
{
initial: [][2]uint64{{5, 15}},
start: 2,
end: 5,
expected: "[[2 15]]",
nextStart: 0,
nextEnd: 1,
last: 15,
},
{
initial: [][2]uint64{{5, 15}},
start: 2,
end: 3,
expected: "[[2 3] [5 15]]",
nextStart: 0,
nextEnd: 1,
last: 15,
},
{
initial: [][2]uint64{{5, 15}},
start: 2,
end: 4,
expected: "[[2 15]]",
nextStart: 0,
nextEnd: 1,
last: 15,
},
{
initial: [][2]uint64{{0, 1}, {5, 15}},
start: 2,
end: 4,
expected: "[[0 15]]",
nextStart: 16,
nextEnd: 0,
last: 15,
},
{
initial: [][2]uint64{{0, 5}, {15, 20}},
start: 2,
end: 10,
expected: "[[0 10] [15 20]]",
nextStart: 11,
nextEnd: 14,
last: 20,
},
{
initial: [][2]uint64{{0, 5}, {15, 20}},
start: 8,
end: 18,
expected: "[[0 5] [8 20]]",
nextStart: 6,
nextEnd: 7,
last: 20,
},
{
initial: [][2]uint64{{0, 5}, {15, 20}},
start: 2,
end: 17,
expected: "[[0 20]]",
nextStart: 21,
nextEnd: 0,
last: 20,
},
{
initial: [][2]uint64{{0, 5}, {15, 20}},
start: 2,
end: 25,
expected: "[[0 25]]",
nextStart: 26,
nextEnd: 0,
last: 25,
},
{
initial: [][2]uint64{{0, 5}, {15, 20}},
start: 5,
end: 14,
expected: "[[0 20]]",
nextStart: 21,
nextEnd: 0,
last: 20,
},
{
initial: [][2]uint64{{0, 5}, {15, 20}},
start: 6,
end: 14,
expected: "[[0 20]]",
nextStart: 21,
nextEnd: 0,
last: 20,
},
{
initial: [][2]uint64{{0, 5}, {15, 20}, {30, 40}},
start: 6,
end: 29,
expected: "[[0 40]]",
nextStart: 41,
nextEnd: 0,
last: 40,
},
{
initial: [][2]uint64{{0, 5}, {15, 20}, {30, 40}, {50, 60}},
start: 3,
end: 55,
expected: "[[0 60]]",
nextStart: 61,
nextEnd: 0,
last: 60,
},
{
initial: [][2]uint64{{0, 5}, {15, 20}, {30, 40}, {50, 60}},
start: 21,
end: 49,
expected: "[[0 5] [15 60]]",
nextStart: 6,
nextEnd: 14,
last: 60,
},
{
initial: [][2]uint64{{0, 5}, {15, 20}, {30, 40}, {50, 60}},
start: 0,
end: 100,
expected: "[[0 100]]",
nextStart: 101,
nextEnd: 0,
last: 100,
},
{
startLimit: 100,
initial: nil,
start: 0,
end: 0,
expected: "[]",
nextStart: 100,
nextEnd: 0,
last: 0,
},
{
startLimit: 100,
initial: nil,
start: 20,
end: 30,
expected: "[]",
nextStart: 100,
nextEnd: 0,
last: 0,
},
{
startLimit: 100,
initial: nil,
start: 50,
end: 100,
expected: "[[100 100]]",
nextStart: 101,
nextEnd: 0,
last: 100,
},
{
startLimit: 100,
initial: nil,
start: 50,
end: 110,
expected: "[[100 110]]",
nextStart: 111,
nextEnd: 0,
last: 110,
},
{
startLimit: 100,
initial: nil,
start: 120,
end: 130,
expected: "[[120 130]]",
nextStart: 100,
nextEnd: 119,
last: 130,
},
{
startLimit: 100,
initial: nil,
start: 120,
end: 130,
expected: "[[120 130]]",
nextStart: 100,
nextEnd: 119,
last: 130,
},
} {
intervals := NewIntervals(tc.startLimit)
intervals.ranges = tc.initial
intervals.Add(tc.start, tc.end)
got := intervals.String()
if got != tc.expected {
t.Errorf("interval #%d: expected %s, got %s", i, tc.expected, got)
}
nextStart, nextEnd := intervals.Next()
if nextStart != tc.nextStart {
t.Errorf("interval #%d, expected next start %d, got %d", i, tc.nextStart, nextStart)
}
if nextEnd != tc.nextEnd {
t.Errorf("interval #%d, expected next end %d, got %d", i, tc.nextEnd, nextEnd)
}
last := intervals.Last()
if last != tc.last {
t.Errorf("interval #%d, expected last %d, got %d", i, tc.last, last)
}
}
}
func TestMerge(t *testing.T) {
for i, tc := range []struct {
initial [][2]uint64
merge [][2]uint64
expected string
}{
{
initial: nil,
merge: nil,
expected: "[]",
},
{
initial: [][2]uint64{{10, 20}},
merge: nil,
expected: "[[10 20]]",
},
{
initial: nil,
merge: [][2]uint64{{15, 25}},
expected: "[[15 25]]",
},
{
initial: [][2]uint64{{0, 100}},
merge: [][2]uint64{{150, 250}},
expected: "[[0 100] [150 250]]",
},
{
initial: [][2]uint64{{0, 100}},
merge: [][2]uint64{{101, 250}},
expected: "[[0 250]]",
},
{
initial: [][2]uint64{{0, 10}, {30, 40}},
merge: [][2]uint64{{20, 25}, {41, 50}},
expected: "[[0 10] [20 25] [30 50]]",
},
{
initial: [][2]uint64{{0, 5}, {15, 20}, {30, 40}, {50, 60}},
merge: [][2]uint64{{6, 25}},
expected: "[[0 25] [30 40] [50 60]]",
},
} {
intervals := NewIntervals(0)
intervals.ranges = tc.initial
m := NewIntervals(0)
m.ranges = tc.merge
intervals.Merge(m)
got := intervals.String()
if got != tc.expected {
t.Errorf("interval #%d: expected %s, got %s", i, tc.expected, got)
}
}
}

View File

@@ -0,0 +1,80 @@
// Copyright 2018 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package intervals
import (
"errors"
"testing"
"github.com/ethereum/go-ethereum/swarm/state"
)
var ErrNotFound = errors.New("not found")
// TestInmemoryStore tests basic functionality of InmemoryStore.
func TestInmemoryStore(t *testing.T) {
testStore(t, state.NewInmemoryStore())
}
// testStore is a helper function to test various Store implementations.
func testStore(t *testing.T, s state.Store) {
key1 := "key1"
i1 := NewIntervals(0)
i1.Add(10, 20)
if err := s.Put(key1, i1); err != nil {
t.Fatal(err)
}
i := &Intervals{}
err := s.Get(key1, i)
if err != nil {
t.Fatal(err)
}
if i.String() != i1.String() {
t.Errorf("expected interval %s, got %s", i1, i)
}
key2 := "key2"
i2 := NewIntervals(0)
i2.Add(10, 20)
if err := s.Put(key2, i2); err != nil {
t.Fatal(err)
}
err = s.Get(key2, i)
if err != nil {
t.Fatal(err)
}
if i.String() != i2.String() {
t.Errorf("expected interval %s, got %s", i2, i)
}
if err := s.Delete(key1); err != nil {
t.Fatal(err)
}
if err := s.Get(key1, i); err != state.ErrNotFound {
t.Errorf("expected error %v, got %s", state.ErrNotFound, err)
}
if err := s.Get(key2, i); err != nil {
t.Errorf("expected error %v, got %s", nil, err)
}
if err := s.Delete(key2); err != nil {
t.Fatal(err)
}
if err := s.Get(key2, i); err != state.ErrNotFound {
t.Errorf("expected error %v, got %s", state.ErrNotFound, err)
}
}

View File

@@ -0,0 +1,313 @@
// Copyright 2018 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package stream
import (
"context"
crand "crypto/rand"
"encoding/binary"
"fmt"
"io"
"sync"
"testing"
"time"
"github.com/ethereum/go-ethereum/node"
"github.com/ethereum/go-ethereum/p2p/discover"
"github.com/ethereum/go-ethereum/p2p/simulations"
"github.com/ethereum/go-ethereum/p2p/simulations/adapters"
"github.com/ethereum/go-ethereum/rpc"
"github.com/ethereum/go-ethereum/swarm/network"
streamTesting "github.com/ethereum/go-ethereum/swarm/network/stream/testing"
"github.com/ethereum/go-ethereum/swarm/state"
"github.com/ethereum/go-ethereum/swarm/storage"
)
var (
externalStreamName = "externalStream"
externalStreamSessionAt uint64 = 50
externalStreamMaxKeys uint64 = 100
)
func newIntervalsStreamerService(ctx *adapters.ServiceContext) (node.Service, error) {
id := ctx.Config.ID
addr := toAddr(id)
kad := network.NewKademlia(addr.Over(), network.NewKadParams())
store := stores[id].(*storage.LocalStore)
db := storage.NewDBAPI(store)
delivery := NewDelivery(kad, db)
deliveries[id] = delivery
r := NewRegistry(addr, delivery, db, state.NewInmemoryStore(), &RegistryOptions{
SkipCheck: defaultSkipCheck,
})
r.RegisterClientFunc(externalStreamName, func(p *Peer, t string, live bool) (Client, error) {
return newTestExternalClient(db), nil
})
r.RegisterServerFunc(externalStreamName, func(p *Peer, t string, live bool) (Server, error) {
return newTestExternalServer(t, externalStreamSessionAt, externalStreamMaxKeys, nil), nil
})
go func() {
waitPeerErrC <- waitForPeers(r, 1*time.Second, peerCount(id))
}()
return &TestExternalRegistry{r}, nil
}
func TestIntervals(t *testing.T) {
testIntervals(t, true, nil, false)
testIntervals(t, false, NewRange(9, 26), false)
testIntervals(t, true, NewRange(9, 26), false)
testIntervals(t, true, nil, true)
testIntervals(t, false, NewRange(9, 26), true)
testIntervals(t, true, NewRange(9, 26), true)
}
func testIntervals(t *testing.T, live bool, history *Range, skipCheck bool) {
nodes := 2
chunkCount := dataChunkCount
defer setDefaultSkipCheck(defaultSkipCheck)
defaultSkipCheck = skipCheck
toAddr = network.NewAddrFromNodeID
conf := &streamTesting.RunConfig{
Adapter: *adapter,
NodeCount: nodes,
ConnLevel: 1,
ToAddr: toAddr,
Services: services,
DefaultService: "intervalsStreamer",
}
sim, teardown, err := streamTesting.NewSimulation(conf)
var rpcSubscriptionsWg sync.WaitGroup
defer func() {
rpcSubscriptionsWg.Wait()
teardown()
}()
if err != nil {
t.Fatal(err)
}
stores = make(map[discover.NodeID]storage.ChunkStore)
deliveries = make(map[discover.NodeID]*Delivery)
for i, id := range sim.IDs {
stores[id] = sim.Stores[i]
}
peerCount = func(id discover.NodeID) int {
return 1
}
fileStore := storage.NewFileStore(sim.Stores[0], storage.NewFileStoreParams())
size := chunkCount * chunkSize
_, wait, err := fileStore.Store(io.LimitReader(crand.Reader, int64(size)), int64(size), false)
wait()
if err != nil {
t.Fatal(err)
}
errc := make(chan error, 1)
waitPeerErrC = make(chan error)
quitC := make(chan struct{})
defer close(quitC)
action := func(ctx context.Context) error {
i := 0
for err := range waitPeerErrC {
if err != nil {
return fmt.Errorf("error waiting for peers: %s", err)
}
i++
if i == nodes {
break
}
}
id := sim.IDs[1]
err := sim.CallClient(id, func(client *rpc.Client) error {
sid := sim.IDs[0]
doneC, err := streamTesting.WatchDisconnections(id, client, errc, quitC)
if err != nil {
return err
}
rpcSubscriptionsWg.Add(1)
go func() {
<-doneC
rpcSubscriptionsWg.Done()
}()
ctx, cancel := context.WithTimeout(ctx, 100*time.Second)
defer cancel()
err = client.CallContext(ctx, nil, "stream_subscribeStream", sid, NewStream(externalStreamName, "", live), history, Top)
if err != nil {
return err
}
liveErrC := make(chan error)
historyErrC := make(chan error)
go func() {
if !live {
close(liveErrC)
return
}
var err error
defer func() {
liveErrC <- err
}()
// live stream
liveHashesChan := make(chan []byte)
liveSubscription, err := client.Subscribe(ctx, "stream", liveHashesChan, "getHashes", sid, NewStream(externalStreamName, "", true))
if err != nil {
return
}
defer liveSubscription.Unsubscribe()
i := externalStreamSessionAt
// we have subscribed, enable notifications
err = client.CallContext(ctx, nil, "stream_enableNotifications", sid, NewStream(externalStreamName, "", true))
if err != nil {
return
}
for {
select {
case hash := <-liveHashesChan:
h := binary.BigEndian.Uint64(hash)
if h != i {
err = fmt.Errorf("expected live hash %d, got %d", i, h)
return
}
i++
if i > externalStreamMaxKeys {
return
}
case err = <-liveSubscription.Err():
return
case <-ctx.Done():
return
}
}
}()
go func() {
if live && history == nil {
close(historyErrC)
return
}
var err error
defer func() {
historyErrC <- err
}()
// history stream
historyHashesChan := make(chan []byte)
historySubscription, err := client.Subscribe(ctx, "stream", historyHashesChan, "getHashes", sid, NewStream(externalStreamName, "", false))
if err != nil {
return
}
defer historySubscription.Unsubscribe()
var i uint64
historyTo := externalStreamMaxKeys
if history != nil {
i = history.From
if history.To != 0 {
historyTo = history.To
}
}
// we have subscribed, enable notifications
err = client.CallContext(ctx, nil, "stream_enableNotifications", sid, NewStream(externalStreamName, "", false))
if err != nil {
return
}
for {
select {
case hash := <-historyHashesChan:
h := binary.BigEndian.Uint64(hash)
if h != i {
err = fmt.Errorf("expected history hash %d, got %d", i, h)
return
}
i++
if i > historyTo {
return
}
case err = <-historySubscription.Err():
return
case <-ctx.Done():
return
}
}
}()
if err := <-liveErrC; err != nil {
return err
}
if err := <-historyErrC; err != nil {
return err
}
return nil
})
return err
}
check := func(ctx context.Context, id discover.NodeID) (bool, error) {
select {
case err := <-errc:
return false, err
case <-ctx.Done():
return false, ctx.Err()
default:
}
return true, nil
}
conf.Step = &simulations.Step{
Action: action,
Trigger: streamTesting.Trigger(10*time.Millisecond, quitC, sim.IDs[0]),
Expect: &simulations.Expectation{
Nodes: sim.IDs[1:1],
Check: check,
},
}
startedAt := time.Now()
timeout := 300 * time.Second
ctx, cancel := context.WithTimeout(context.Background(), timeout)
defer cancel()
result, err := sim.Run(ctx, conf)
finishedAt := time.Now()
if err != nil {
t.Fatalf("Setting up simulation failed: %v", err)
}
if result.Error != nil {
t.Fatalf("Simulation failed: %s", result.Error)
}
streamTesting.CheckResult(t, result, startedAt, finishedAt)
}

View File

@@ -0,0 +1,370 @@
// Copyright 2018 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package stream
import (
"errors"
"fmt"
"sync"
"time"
"github.com/ethereum/go-ethereum/metrics"
"github.com/ethereum/go-ethereum/swarm/log"
bv "github.com/ethereum/go-ethereum/swarm/network/bitvector"
"github.com/ethereum/go-ethereum/swarm/storage"
)
// Stream defines a unique stream identifier.
type Stream struct {
// Name is used for Client and Server functions identification.
Name string
// Key is the name of specific stream data.
Key string
// Live defines whether the stream delivers only new data
// for the specific stream.
Live bool
}
func NewStream(name string, key string, live bool) Stream {
return Stream{
Name: name,
Key: key,
Live: live,
}
}
// String return a stream id based on all Stream fields.
func (s Stream) String() string {
t := "h"
if s.Live {
t = "l"
}
return fmt.Sprintf("%s|%s|%s", s.Name, s.Key, t)
}
// SubcribeMsg is the protocol msg for requesting a stream(section)
type SubscribeMsg struct {
Stream Stream
History *Range `rlp:"nil"`
Priority uint8 // delivered on priority channel
}
// RequestSubscriptionMsg is the protocol msg for a node to request subscription to a
// specific stream
type RequestSubscriptionMsg struct {
Stream Stream
History *Range `rlp:"nil"`
Priority uint8 // delivered on priority channel
}
func (p *Peer) handleRequestSubscription(req *RequestSubscriptionMsg) (err error) {
log.Debug(fmt.Sprintf("handleRequestSubscription: streamer %s to subscribe to %s with stream %s", p.streamer.addr.ID(), p.ID(), req.Stream))
return p.streamer.Subscribe(p.ID(), req.Stream, req.History, req.Priority)
}
func (p *Peer) handleSubscribeMsg(req *SubscribeMsg) (err error) {
metrics.GetOrRegisterCounter("peer.handlesubscribemsg", nil).Inc(1)
defer func() {
if err != nil {
if e := p.Send(SubscribeErrorMsg{
Error: err.Error(),
}); e != nil {
log.Error("send stream subscribe error message", "err", err)
}
}
}()
log.Debug("received subscription", "from", p.streamer.addr.ID(), "peer", p.ID(), "stream", req.Stream, "history", req.History)
f, err := p.streamer.GetServerFunc(req.Stream.Name)
if err != nil {
return err
}
s, err := f(p, req.Stream.Key, req.Stream.Live)
if err != nil {
return err
}
os, err := p.setServer(req.Stream, s, req.Priority)
if err != nil {
return err
}
var from uint64
var to uint64
if !req.Stream.Live && req.History != nil {
from = req.History.From
to = req.History.To
}
go func() {
if err := p.SendOfferedHashes(os, from, to); err != nil {
log.Warn("SendOfferedHashes dropping peer", "err", err)
p.Drop(err)
}
}()
if req.Stream.Live && req.History != nil {
// subscribe to the history stream
s, err := f(p, req.Stream.Key, false)
if err != nil {
return err
}
os, err := p.setServer(getHistoryStream(req.Stream), s, getHistoryPriority(req.Priority))
if err != nil {
return err
}
go func() {
if err := p.SendOfferedHashes(os, req.History.From, req.History.To); err != nil {
log.Warn("SendOfferedHashes dropping peer", "err", err)
p.Drop(err)
}
}()
}
return nil
}
type SubscribeErrorMsg struct {
Error string
}
func (p *Peer) handleSubscribeErrorMsg(req *SubscribeErrorMsg) (err error) {
return fmt.Errorf("subscribe to peer %s: %v", p.ID(), req.Error)
}
type UnsubscribeMsg struct {
Stream Stream
}
func (p *Peer) handleUnsubscribeMsg(req *UnsubscribeMsg) error {
return p.removeServer(req.Stream)
}
type QuitMsg struct {
Stream Stream
}
func (p *Peer) handleQuitMsg(req *QuitMsg) error {
return p.removeClient(req.Stream)
}
// OfferedHashesMsg is the protocol msg for offering to hand over a
// stream section
type OfferedHashesMsg struct {
Stream Stream // name of Stream
From, To uint64 // peer and db-specific entry count
Hashes []byte // stream of hashes (128)
*HandoverProof // HandoverProof
}
// String pretty prints OfferedHashesMsg
func (m OfferedHashesMsg) String() string {
return fmt.Sprintf("Stream '%v' [%v-%v] (%v)", m.Stream, m.From, m.To, len(m.Hashes)/HashSize)
}
// handleOfferedHashesMsg protocol msg handler calls the incoming streamer interface
// Filter method
func (p *Peer) handleOfferedHashesMsg(req *OfferedHashesMsg) error {
metrics.GetOrRegisterCounter("peer.handleofferedhashes", nil).Inc(1)
c, _, err := p.getOrSetClient(req.Stream, req.From, req.To)
if err != nil {
return err
}
hashes := req.Hashes
want, err := bv.New(len(hashes) / HashSize)
if err != nil {
return fmt.Errorf("error initiaising bitvector of length %v: %v", len(hashes)/HashSize, err)
}
wg := sync.WaitGroup{}
for i := 0; i < len(hashes); i += HashSize {
hash := hashes[i : i+HashSize]
if wait := c.NeedData(hash); wait != nil {
want.Set(i/HashSize, true)
wg.Add(1)
// create request and wait until the chunk data arrives and is stored
go func(w func()) {
w()
wg.Done()
}(wait)
}
}
// done := make(chan bool)
// go func() {
// wg.Wait()
// close(done)
// }()
// go func() {
// select {
// case <-done:
// s.next <- s.batchDone(p, req, hashes)
// case <-time.After(1 * time.Second):
// p.Drop(errors.New("timeout waiting for batch to be delivered"))
// }
// }()
go func() {
wg.Wait()
select {
case c.next <- c.batchDone(p, req, hashes):
case <-c.quit:
}
}()
// only send wantedKeysMsg if all missing chunks of the previous batch arrived
// except
if c.stream.Live {
c.sessionAt = req.From
}
from, to := c.nextBatch(req.To + 1)
log.Trace("received offered batch", "peer", p.ID(), "stream", req.Stream, "from", req.From, "to", req.To)
if from == to {
return nil
}
msg := &WantedHashesMsg{
Stream: req.Stream,
Want: want.Bytes(),
From: from,
To: to,
}
go func() {
select {
case <-time.After(120 * time.Second):
log.Warn("handleOfferedHashesMsg timeout, so dropping peer")
p.Drop(errors.New("handle offered hashes timeout"))
return
case err := <-c.next:
if err != nil {
log.Warn("c.next dropping peer", "err", err)
p.Drop(err)
return
}
case <-c.quit:
return
}
log.Trace("sending want batch", "peer", p.ID(), "stream", msg.Stream, "from", msg.From, "to", msg.To)
err := p.SendPriority(msg, c.priority)
if err != nil {
log.Warn("SendPriority err, so dropping peer", "err", err)
p.Drop(err)
}
}()
return nil
}
// WantedHashesMsg is the protocol msg data for signaling which hashes
// offered in OfferedHashesMsg downstream peer actually wants sent over
type WantedHashesMsg struct {
Stream Stream
Want []byte // bitvector indicating which keys of the batch needed
From, To uint64 // next interval offset - empty if not to be continued
}
// String pretty prints WantedHashesMsg
func (m WantedHashesMsg) String() string {
return fmt.Sprintf("Stream '%v', Want: %x, Next: [%v-%v]", m.Stream, m.Want, m.From, m.To)
}
// handleWantedHashesMsg protocol msg handler
// * sends the next batch of unsynced keys
// * sends the actual data chunks as per WantedHashesMsg
func (p *Peer) handleWantedHashesMsg(req *WantedHashesMsg) error {
metrics.GetOrRegisterCounter("peer.handlewantedhashesmsg", nil).Inc(1)
log.Trace("received wanted batch", "peer", p.ID(), "stream", req.Stream, "from", req.From, "to", req.To)
s, err := p.getServer(req.Stream)
if err != nil {
return err
}
hashes := s.currentBatch
// launch in go routine since GetBatch blocks until new hashes arrive
go func() {
if err := p.SendOfferedHashes(s, req.From, req.To); err != nil {
log.Warn("SendOfferedHashes dropping peer", "err", err)
p.Drop(err)
}
}()
// go p.SendOfferedHashes(s, req.From, req.To)
l := len(hashes) / HashSize
log.Trace("wanted batch length", "peer", p.ID(), "stream", req.Stream, "from", req.From, "to", req.To, "lenhashes", len(hashes), "l", l)
want, err := bv.NewFromBytes(req.Want, l)
if err != nil {
return fmt.Errorf("error initiaising bitvector of length %v: %v", l, err)
}
for i := 0; i < l; i++ {
if want.Get(i) {
metrics.GetOrRegisterCounter("peer.handlewantedhashesmsg.actualget", nil).Inc(1)
hash := hashes[i*HashSize : (i+1)*HashSize]
data, err := s.GetData(hash)
if err != nil {
return fmt.Errorf("handleWantedHashesMsg get data %x: %v", hash, err)
}
chunk := storage.NewChunk(hash, nil)
chunk.SData = data
if length := len(chunk.SData); length < 9 {
log.Error("Chunk.SData to sync is too short", "len(chunk.SData)", length, "address", chunk.Addr)
}
if err := p.Deliver(chunk, s.priority); err != nil {
return err
}
}
}
return nil
}
// Handover represents a statement that the upstream peer hands over the stream section
type Handover struct {
Stream Stream // name of stream
Start, End uint64 // index of hashes
Root []byte // Root hash for indexed segment inclusion proofs
}
// HandoverProof represents a signed statement that the upstream peer handed over the stream section
type HandoverProof struct {
Sig []byte // Sign(Hash(Serialisation(Handover)))
*Handover
}
// Takeover represents a statement that downstream peer took over (stored all data)
// handed over
type Takeover Handover
// TakeoverProof represents a signed statement that the downstream peer took over
// the stream section
type TakeoverProof struct {
Sig []byte // Sign(Hash(Serialisation(Takeover)))
*Takeover
}
// TakeoverProofMsg is the protocol msg sent by downstream peer
type TakeoverProofMsg TakeoverProof
// String pretty prints TakeoverProofMsg
func (m TakeoverProofMsg) String() string {
return fmt.Sprintf("Stream: '%v' [%v-%v], Root: %x, Sig: %x", m.Stream, m.Start, m.End, m.Root, m.Sig)
}
func (p *Peer) handleTakeoverProofMsg(req *TakeoverProofMsg) error {
_, err := p.getServer(req.Stream)
// store the strongest takeoverproof for the stream in streamer
return err
}

View File

@@ -0,0 +1,328 @@
// Copyright 2018 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package stream
import (
"context"
"fmt"
"sync"
"time"
"github.com/ethereum/go-ethereum/metrics"
"github.com/ethereum/go-ethereum/p2p/protocols"
"github.com/ethereum/go-ethereum/swarm/log"
pq "github.com/ethereum/go-ethereum/swarm/network/priorityqueue"
"github.com/ethereum/go-ethereum/swarm/network/stream/intervals"
"github.com/ethereum/go-ethereum/swarm/state"
"github.com/ethereum/go-ethereum/swarm/storage"
)
var sendTimeout = 30 * time.Second
type notFoundError struct {
t string
s Stream
}
func newNotFoundError(t string, s Stream) *notFoundError {
return &notFoundError{t: t, s: s}
}
func (e *notFoundError) Error() string {
return fmt.Sprintf("%s not found for stream %q", e.t, e.s)
}
// Peer is the Peer extension for the streaming protocol
type Peer struct {
*protocols.Peer
streamer *Registry
pq *pq.PriorityQueue
serverMu sync.RWMutex
clientMu sync.RWMutex // protects both clients and clientParams
servers map[Stream]*server
clients map[Stream]*client
// clientParams map keeps required client arguments
// that are set on Registry.Subscribe and used
// on creating a new client in offered hashes handler.
clientParams map[Stream]*clientParams
quit chan struct{}
}
// NewPeer is the constructor for Peer
func NewPeer(peer *protocols.Peer, streamer *Registry) *Peer {
p := &Peer{
Peer: peer,
pq: pq.New(int(PriorityQueue), PriorityQueueCap),
streamer: streamer,
servers: make(map[Stream]*server),
clients: make(map[Stream]*client),
clientParams: make(map[Stream]*clientParams),
quit: make(chan struct{}),
}
ctx, cancel := context.WithCancel(context.Background())
go p.pq.Run(ctx, func(i interface{}) { p.Send(i) })
go func() {
<-p.quit
cancel()
}()
return p
}
// Deliver sends a storeRequestMsg protocol message to the peer
func (p *Peer) Deliver(chunk *storage.Chunk, priority uint8) error {
msg := &ChunkDeliveryMsg{
Addr: chunk.Addr,
SData: chunk.SData,
}
return p.SendPriority(msg, priority)
}
// SendPriority sends message to the peer using the outgoing priority queue
func (p *Peer) SendPriority(msg interface{}, priority uint8) error {
defer metrics.GetOrRegisterResettingTimer(fmt.Sprintf("peer.sendpriority_t.%d", priority), nil).UpdateSince(time.Now())
metrics.GetOrRegisterCounter(fmt.Sprintf("peer.sendpriority.%d", priority), nil).Inc(1)
ctx, cancel := context.WithTimeout(context.Background(), sendTimeout)
defer cancel()
return p.pq.Push(ctx, msg, int(priority))
}
// SendOfferedHashes sends OfferedHashesMsg protocol msg
func (p *Peer) SendOfferedHashes(s *server, f, t uint64) error {
hashes, from, to, proof, err := s.SetNextBatch(f, t)
if err != nil {
return err
}
// true only when quiting
if len(hashes) == 0 {
return nil
}
if proof == nil {
proof = &HandoverProof{
Handover: &Handover{},
}
}
s.currentBatch = hashes
msg := &OfferedHashesMsg{
HandoverProof: proof,
Hashes: hashes,
From: from,
To: to,
Stream: s.stream,
}
log.Trace("Swarm syncer offer batch", "peer", p.ID(), "stream", s.stream, "len", len(hashes), "from", from, "to", to)
return p.SendPriority(msg, s.priority)
}
func (p *Peer) getServer(s Stream) (*server, error) {
p.serverMu.RLock()
defer p.serverMu.RUnlock()
server := p.servers[s]
if server == nil {
return nil, newNotFoundError("server", s)
}
return server, nil
}
func (p *Peer) setServer(s Stream, o Server, priority uint8) (*server, error) {
p.serverMu.Lock()
defer p.serverMu.Unlock()
if p.servers[s] != nil {
return nil, fmt.Errorf("server %s already registered", s)
}
os := &server{
Server: o,
stream: s,
priority: priority,
}
p.servers[s] = os
return os, nil
}
func (p *Peer) removeServer(s Stream) error {
p.serverMu.Lock()
defer p.serverMu.Unlock()
server, ok := p.servers[s]
if !ok {
return newNotFoundError("server", s)
}
server.Close()
delete(p.servers, s)
return nil
}
func (p *Peer) getClient(ctx context.Context, s Stream) (c *client, err error) {
var params *clientParams
func() {
p.clientMu.RLock()
defer p.clientMu.RUnlock()
c = p.clients[s]
if c != nil {
return
}
params = p.clientParams[s]
}()
if c != nil {
return c, nil
}
if params != nil {
//debug.PrintStack()
if err := params.waitClient(ctx); err != nil {
return nil, err
}
}
p.clientMu.RLock()
defer p.clientMu.RUnlock()
c = p.clients[s]
if c != nil {
return c, nil
}
return nil, newNotFoundError("client", s)
}
func (p *Peer) getOrSetClient(s Stream, from, to uint64) (c *client, created bool, err error) {
p.clientMu.Lock()
defer p.clientMu.Unlock()
c = p.clients[s]
if c != nil {
return c, false, nil
}
f, err := p.streamer.GetClientFunc(s.Name)
if err != nil {
return nil, false, err
}
is, err := f(p, s.Key, s.Live)
if err != nil {
return nil, false, err
}
cp, err := p.getClientParams(s)
if err != nil {
return nil, false, err
}
defer func() {
if err == nil {
if err := p.removeClientParams(s); err != nil {
log.Error("stream set client: remove client params", "stream", s, "peer", p, "err", err)
}
}
}()
intervalsKey := peerStreamIntervalsKey(p, s)
if s.Live {
// try to find previous history and live intervals and merge live into history
historyKey := peerStreamIntervalsKey(p, NewStream(s.Name, s.Key, false))
historyIntervals := &intervals.Intervals{}
err := p.streamer.intervalsStore.Get(historyKey, historyIntervals)
switch err {
case nil:
liveIntervals := &intervals.Intervals{}
err := p.streamer.intervalsStore.Get(intervalsKey, liveIntervals)
switch err {
case nil:
historyIntervals.Merge(liveIntervals)
if err := p.streamer.intervalsStore.Put(historyKey, historyIntervals); err != nil {
log.Error("stream set client: put history intervals", "stream", s, "peer", p, "err", err)
}
case state.ErrNotFound:
default:
log.Error("stream set client: get live intervals", "stream", s, "peer", p, "err", err)
}
case state.ErrNotFound:
default:
log.Error("stream set client: get history intervals", "stream", s, "peer", p, "err", err)
}
}
if err := p.streamer.intervalsStore.Put(intervalsKey, intervals.NewIntervals(from)); err != nil {
return nil, false, err
}
next := make(chan error, 1)
c = &client{
Client: is,
stream: s,
priority: cp.priority,
to: cp.to,
next: next,
quit: make(chan struct{}),
intervalsStore: p.streamer.intervalsStore,
intervalsKey: intervalsKey,
}
p.clients[s] = c
cp.clientCreated() // unblock all possible getClient calls that are waiting
next <- nil // this is to allow wantedKeysMsg before first batch arrives
return c, true, nil
}
func (p *Peer) removeClient(s Stream) error {
p.clientMu.Lock()
defer p.clientMu.Unlock()
client, ok := p.clients[s]
if !ok {
return newNotFoundError("client", s)
}
client.close()
return nil
}
func (p *Peer) setClientParams(s Stream, params *clientParams) error {
p.clientMu.Lock()
defer p.clientMu.Unlock()
if p.clients[s] != nil {
return fmt.Errorf("client %s already exists", s)
}
if p.clientParams[s] != nil {
return fmt.Errorf("client params %s already set", s)
}
p.clientParams[s] = params
return nil
}
func (p *Peer) getClientParams(s Stream) (*clientParams, error) {
params := p.clientParams[s]
if params == nil {
return nil, fmt.Errorf("client params '%v' not provided to peer %v", s, p.ID())
}
return params, nil
}
func (p *Peer) removeClientParams(s Stream) error {
_, ok := p.clientParams[s]
if !ok {
return newNotFoundError("client params", s)
}
delete(p.clientParams, s)
return nil
}
func (p *Peer) close() {
for _, s := range p.servers {
s.Close()
}
}

View File

@@ -0,0 +1,791 @@
// Copyright 2018 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package stream
import (
"context"
crand "crypto/rand"
"fmt"
"math/rand"
"strings"
"sync"
"testing"
"time"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/p2p/discover"
"github.com/ethereum/go-ethereum/p2p/simulations"
"github.com/ethereum/go-ethereum/swarm/log"
"github.com/ethereum/go-ethereum/swarm/network"
streamTesting "github.com/ethereum/go-ethereum/swarm/network/stream/testing"
"github.com/ethereum/go-ethereum/swarm/storage"
)
//constants for random file generation
const (
minFileSize = 2
maxFileSize = 40
)
func initRetrievalTest() {
//global func to get overlay address from discover ID
toAddr = func(id discover.NodeID) *network.BzzAddr {
addr := network.NewAddrFromNodeID(id)
return addr
}
//global func to create local store
createStoreFunc = createTestLocalStorageForId
//local stores
stores = make(map[discover.NodeID]storage.ChunkStore)
//data directories for each node and store
datadirs = make(map[discover.NodeID]string)
//deliveries for each node
deliveries = make(map[discover.NodeID]*Delivery)
//global retrieve func
getRetrieveFunc = func(id discover.NodeID) func(chunk *storage.Chunk) error {
return func(chunk *storage.Chunk) error {
skipCheck := true
return deliveries[id].RequestFromPeers(chunk.Addr[:], skipCheck)
}
}
//registries, map of discover.NodeID to its streamer
registries = make(map[discover.NodeID]*TestRegistry)
//not needed for this test but required from common_test for NewStreamService
waitPeerErrC = make(chan error)
//also not needed for this test but required for NewStreamService
peerCount = func(id discover.NodeID) int {
if ids[0] == id || ids[len(ids)-1] == id {
return 1
}
return 2
}
}
//This test is a retrieval test for nodes.
//A configurable number of nodes can be
//provided to the test.
//Files are uploaded to nodes, other nodes try to retrieve the file
//Number of nodes can be provided via commandline too.
func TestFileRetrieval(t *testing.T) {
if *nodes != 0 {
fileRetrievalTest(t, *nodes)
} else {
nodeCnt := []int{16}
//if the `longrunning` flag has been provided
//run more test combinations
if *longrunning {
nodeCnt = append(nodeCnt, 32, 64, 128)
}
for _, n := range nodeCnt {
fileRetrievalTest(t, n)
}
}
}
//This test is a retrieval test for nodes.
//One node is randomly selected to be the pivot node.
//A configurable number of chunks and nodes can be
//provided to the test, the number of chunks is uploaded
//to the pivot node and other nodes try to retrieve the chunk(s).
//Number of chunks and nodes can be provided via commandline too.
func TestRetrieval(t *testing.T) {
//if nodes/chunks have been provided via commandline,
//run the tests with these values
if *nodes != 0 && *chunks != 0 {
retrievalTest(t, *chunks, *nodes)
} else {
var nodeCnt []int
var chnkCnt []int
//if the `longrunning` flag has been provided
//run more test combinations
if *longrunning {
nodeCnt = []int{16, 32, 128}
chnkCnt = []int{4, 32, 256}
} else {
//default test
nodeCnt = []int{16}
chnkCnt = []int{32}
}
for _, n := range nodeCnt {
for _, c := range chnkCnt {
retrievalTest(t, c, n)
}
}
}
}
//Every test runs 3 times, a live, a history, and a live AND history
func fileRetrievalTest(t *testing.T, nodeCount int) {
//test live and NO history
log.Info("Testing live and no history", "nodeCount", nodeCount)
live = true
history = false
err := runFileRetrievalTest(nodeCount)
if err != nil {
t.Fatal(err)
}
//test history only
log.Info("Testing history only", "nodeCount", nodeCount)
live = false
history = true
err = runFileRetrievalTest(nodeCount)
if err != nil {
t.Fatal(err)
}
//finally test live and history
log.Info("Testing live and history", "nodeCount", nodeCount)
live = true
err = runFileRetrievalTest(nodeCount)
if err != nil {
t.Fatal(err)
}
}
//Every test runs 3 times, a live, a history, and a live AND history
func retrievalTest(t *testing.T, chunkCount int, nodeCount int) {
//test live and NO history
log.Info("Testing live and no history", "chunkCount", chunkCount, "nodeCount", nodeCount)
live = true
history = false
err := runRetrievalTest(chunkCount, nodeCount)
if err != nil {
t.Fatal(err)
}
//test history only
log.Info("Testing history only", "chunkCount", chunkCount, "nodeCount", nodeCount)
live = false
history = true
err = runRetrievalTest(chunkCount, nodeCount)
if err != nil {
t.Fatal(err)
}
//finally test live and history
log.Info("Testing live and history", "chunkCount", chunkCount, "nodeCount", nodeCount)
live = true
err = runRetrievalTest(chunkCount, nodeCount)
if err != nil {
t.Fatal(err)
}
}
/*
The upload is done by dependency to the global
`live` and `history` variables;
If `live` is set, first stream subscriptions are established,
then files are uploaded to nodes.
If `history` is enabled, first upload files, then build up subscriptions.
The test loads a snapshot file to construct the swarm network,
assuming that the snapshot file identifies a healthy
kademlia network. Nevertheless a health check runs in the
simulation's `action` function.
The snapshot should have 'streamer' in its service list.
*/
func runFileRetrievalTest(nodeCount int) error {
//for every run (live, history), int the variables
initRetrievalTest()
//the ids of the snapshot nodes, initiate only now as we need nodeCount
ids = make([]discover.NodeID, nodeCount)
//channel to check for disconnection errors
disconnectC := make(chan error)
//channel to close disconnection watcher routine
quitC := make(chan struct{})
//the test conf (using same as in `snapshot_sync_test`
conf = &synctestConfig{}
//map of overlay address to discover ID
conf.addrToIdMap = make(map[string]discover.NodeID)
//array where the generated chunk hashes will be stored
conf.hashes = make([]storage.Address, 0)
//load nodes from the snapshot file
net, err := initNetWithSnapshot(nodeCount)
if err != nil {
return err
}
var rpcSubscriptionsWg sync.WaitGroup
//do cleanup after test is terminated
defer func() {
//shutdown the snapshot network
net.Shutdown()
//after the test, clean up local stores initialized with createLocalStoreForId
localStoreCleanup()
//finally clear all data directories
datadirsCleanup()
}()
//get the nodes of the network
nodes := net.GetNodes()
//iterate over all nodes...
for c := 0; c < len(nodes); c++ {
//create an array of discovery nodeIDS
ids[c] = nodes[c].ID()
a := network.ToOverlayAddr(ids[c].Bytes())
//append it to the array of all overlay addresses
conf.addrs = append(conf.addrs, a)
conf.addrToIdMap[string(a)] = ids[c]
}
//needed for healthy call
ppmap = network.NewPeerPotMap(testMinProxBinSize, conf.addrs)
//an array for the random files
var randomFiles []string
//channel to signal when the upload has finished
uploadFinished := make(chan struct{})
//channel to trigger new node checks
trigger := make(chan discover.NodeID)
//simulation action
action := func(ctx context.Context) error {
//first run the health check on all nodes,
//wait until nodes are all healthy
ticker := time.NewTicker(200 * time.Millisecond)
defer ticker.Stop()
for range ticker.C {
healthy := true
for _, id := range ids {
r := registries[id]
//PeerPot for this node
addr := common.Bytes2Hex(r.addr.OAddr)
pp := ppmap[addr]
//call Healthy RPC
h := r.delivery.overlay.Healthy(pp)
//print info
log.Debug(r.delivery.overlay.String())
log.Debug(fmt.Sprintf("IS HEALTHY: %t", h.GotNN && h.KnowNN && h.Full))
if !h.GotNN || !h.Full {
healthy = false
break
}
}
if healthy {
break
}
}
if history {
log.Info("Uploading for history")
//If testing only history, we upload the chunk(s) first
conf.hashes, randomFiles, err = uploadFilesToNodes(nodes)
if err != nil {
return err
}
}
//variables needed to wait for all subscriptions established before uploading
errc := make(chan error)
//now setup and start event watching in order to know when we can upload
ctx, watchCancel := context.WithTimeout(context.Background(), MaxTimeout*time.Second)
defer watchCancel()
log.Info("Setting up stream subscription")
//We need two iterations, one to subscribe to the subscription events
//(so we know when setup phase is finished), and one to
//actually run the stream subscriptions. We can't do it in the same iteration,
//because while the first nodes in the loop are setting up subscriptions,
//the latter ones have not subscribed to listen to peer events yet,
//and then we miss events.
//first iteration: setup disconnection watcher and subscribe to peer events
for j, id := range ids {
log.Trace(fmt.Sprintf("Subscribe to subscription events: %d", j))
client, err := net.GetNode(id).Client()
if err != nil {
return err
}
wsDoneC := watchSubscriptionEvents(ctx, id, client, errc, quitC)
// doneC is nil, the error happened which is sent to errc channel, already
if wsDoneC == nil {
continue
}
rpcSubscriptionsWg.Add(1)
go func() {
<-wsDoneC
rpcSubscriptionsWg.Done()
}()
//watch for peers disconnecting
wdDoneC, err := streamTesting.WatchDisconnections(id, client, disconnectC, quitC)
if err != nil {
return err
}
rpcSubscriptionsWg.Add(1)
go func() {
<-wdDoneC
rpcSubscriptionsWg.Done()
}()
}
//second iteration: start syncing and setup stream subscriptions
for j, id := range ids {
log.Trace(fmt.Sprintf("Start syncing and stream subscriptions: %d", j))
client, err := net.GetNode(id).Client()
if err != nil {
return err
}
//start syncing!
var cnt int
err = client.CallContext(ctx, &cnt, "stream_startSyncing")
if err != nil {
return err
}
//increment the number of subscriptions we need to wait for
//by the count returned from startSyncing (SYNC subscriptions)
subscriptionCount += cnt
//now also add the number of RETRIEVAL_REQUEST subscriptions
for snid := range registries[id].peers {
subscriptionCount++
err = client.CallContext(ctx, nil, "stream_subscribeStream", snid, NewStream(swarmChunkServerStreamName, "", false), nil, Top)
if err != nil {
return err
}
}
}
//now wait until the number of expected subscriptions has been finished
//`watchSubscriptionEvents` will write with a `nil` value to errc
//every time a `SubscriptionMsg` has been received
for err := range errc {
if err != nil {
return err
}
//`nil` received, decrement count
subscriptionCount--
//all subscriptions received
if subscriptionCount == 0 {
break
}
}
log.Info("Stream subscriptions successfully requested, action terminated")
if live {
//upload generated files to nodes
var hashes []storage.Address
var rfiles []string
hashes, rfiles, err = uploadFilesToNodes(nodes)
if err != nil {
return err
}
conf.hashes = append(conf.hashes, hashes...)
randomFiles = append(randomFiles, rfiles...)
//signal to the trigger loop that the upload has finished
uploadFinished <- struct{}{}
}
return nil
}
//check defines what will be checked during the test
check := func(ctx context.Context, id discover.NodeID) (bool, error) {
select {
case <-ctx.Done():
return false, ctx.Err()
case e := <-disconnectC:
log.Error(e.Error())
return false, fmt.Errorf("Disconnect event detected, network unhealthy")
default:
}
log.Trace(fmt.Sprintf("Checking node: %s", id))
//if there are more than one chunk, test only succeeds if all expected chunks are found
allSuccess := true
//check on the node's FileStore (netstore)
fileStore := registries[id].fileStore
//check all chunks
for i, hash := range conf.hashes {
reader, _ := fileStore.Retrieve(hash)
//check that we can read the file size and that it corresponds to the generated file size
if s, err := reader.Size(nil); err != nil || s != int64(len(randomFiles[i])) {
allSuccess = false
log.Warn("Retrieve error", "err", err, "hash", hash, "nodeId", id)
} else {
log.Debug(fmt.Sprintf("File with root hash %x successfully retrieved", hash))
}
}
return allSuccess, nil
}
//for each tick, run the checks on all nodes
timingTicker := time.NewTicker(5 * time.Second)
defer timingTicker.Stop()
go func() {
//for live upload, we should wait for uploads to have finished
//before starting to trigger the checks, due to file size
if live {
<-uploadFinished
}
for range timingTicker.C {
for i := 0; i < len(ids); i++ {
log.Trace(fmt.Sprintf("triggering step %d, id %s", i, ids[i]))
trigger <- ids[i]
}
}
}()
log.Info("Starting simulation run...")
timeout := MaxTimeout * time.Second
ctx, cancel := context.WithTimeout(context.Background(), timeout)
defer cancel()
//run the simulation
result := simulations.NewSimulation(net).Run(ctx, &simulations.Step{
Action: action,
Trigger: trigger,
Expect: &simulations.Expectation{
Nodes: ids,
Check: check,
},
})
if result.Error != nil {
return result.Error
}
return nil
}
/*
The test generates the given number of chunks.
The upload is done by dependency to the global
`live` and `history` variables;
If `live` is set, first stream subscriptions are established, then
upload to a random node.
If `history` is enabled, first upload then build up subscriptions.
The test loads a snapshot file to construct the swarm network,
assuming that the snapshot file identifies a healthy
kademlia network. Nevertheless a health check runs in the
simulation's `action` function.
The snapshot should have 'streamer' in its service list.
*/
func runRetrievalTest(chunkCount int, nodeCount int) error {
//for every run (live, history), int the variables
initRetrievalTest()
//the ids of the snapshot nodes, initiate only now as we need nodeCount
ids = make([]discover.NodeID, nodeCount)
//channel to check for disconnection errors
disconnectC := make(chan error)
//channel to close disconnection watcher routine
quitC := make(chan struct{})
//the test conf (using same as in `snapshot_sync_test`
conf = &synctestConfig{}
//map of overlay address to discover ID
conf.addrToIdMap = make(map[string]discover.NodeID)
//array where the generated chunk hashes will be stored
conf.hashes = make([]storage.Address, 0)
//load nodes from the snapshot file
net, err := initNetWithSnapshot(nodeCount)
if err != nil {
return err
}
var rpcSubscriptionsWg sync.WaitGroup
//do cleanup after test is terminated
defer func() {
//shutdown the snapshot network
net.Shutdown()
//after the test, clean up local stores initialized with createLocalStoreForId
localStoreCleanup()
//finally clear all data directories
datadirsCleanup()
}()
//get the nodes of the network
nodes := net.GetNodes()
//select one index at random...
idx := rand.Intn(len(nodes))
//...and get the the node at that index
//this is the node selected for upload
uploadNode := nodes[idx]
//iterate over all nodes...
for c := 0; c < len(nodes); c++ {
//create an array of discovery nodeIDS
ids[c] = nodes[c].ID()
a := network.ToOverlayAddr(ids[c].Bytes())
//append it to the array of all overlay addresses
conf.addrs = append(conf.addrs, a)
conf.addrToIdMap[string(a)] = ids[c]
}
//needed for healthy call
ppmap = network.NewPeerPotMap(testMinProxBinSize, conf.addrs)
trigger := make(chan discover.NodeID)
//simulation action
action := func(ctx context.Context) error {
//first run the health check on all nodes,
//wait until nodes are all healthy
ticker := time.NewTicker(200 * time.Millisecond)
defer ticker.Stop()
for range ticker.C {
healthy := true
for _, id := range ids {
r := registries[id]
//PeerPot for this node
addr := common.Bytes2Hex(network.ToOverlayAddr(id.Bytes()))
pp := ppmap[addr]
//call Healthy RPC
h := r.delivery.overlay.Healthy(pp)
//print info
log.Debug(r.delivery.overlay.String())
log.Debug(fmt.Sprintf("IS HEALTHY: %t", h.GotNN && h.KnowNN && h.Full))
if !h.GotNN || !h.Full {
healthy = false
break
}
}
if healthy {
break
}
}
if history {
log.Info("Uploading for history")
//If testing only history, we upload the chunk(s) first
conf.hashes, err = uploadFileToSingleNodeStore(uploadNode.ID(), chunkCount)
if err != nil {
return err
}
}
//variables needed to wait for all subscriptions established before uploading
errc := make(chan error)
//now setup and start event watching in order to know when we can upload
ctx, watchCancel := context.WithTimeout(context.Background(), MaxTimeout*time.Second)
defer watchCancel()
log.Info("Setting up stream subscription")
//We need two iterations, one to subscribe to the subscription events
//(so we know when setup phase is finished), and one to
//actually run the stream subscriptions. We can't do it in the same iteration,
//because while the first nodes in the loop are setting up subscriptions,
//the latter ones have not subscribed to listen to peer events yet,
//and then we miss events.
//first iteration: setup disconnection watcher and subscribe to peer events
for j, id := range ids {
log.Trace(fmt.Sprintf("Subscribe to subscription events: %d", j))
client, err := net.GetNode(id).Client()
if err != nil {
return err
}
//check for `SubscribeMsg` events to know when setup phase is complete
wsDoneC := watchSubscriptionEvents(ctx, id, client, errc, quitC)
// doneC is nil, the error happened which is sent to errc channel, already
if wsDoneC == nil {
continue
}
rpcSubscriptionsWg.Add(1)
go func() {
<-wsDoneC
rpcSubscriptionsWg.Done()
}()
//watch for peers disconnecting
wdDoneC, err := streamTesting.WatchDisconnections(id, client, disconnectC, quitC)
if err != nil {
return err
}
rpcSubscriptionsWg.Add(1)
go func() {
<-wdDoneC
rpcSubscriptionsWg.Done()
}()
}
//second iteration: start syncing and setup stream subscriptions
for j, id := range ids {
log.Trace(fmt.Sprintf("Start syncing and stream subscriptions: %d", j))
client, err := net.GetNode(id).Client()
if err != nil {
return err
}
//start syncing!
var cnt int
err = client.CallContext(ctx, &cnt, "stream_startSyncing")
if err != nil {
return err
}
//increment the number of subscriptions we need to wait for
//by the count returned from startSyncing (SYNC subscriptions)
subscriptionCount += cnt
//now also add the number of RETRIEVAL_REQUEST subscriptions
for snid := range registries[id].peers {
subscriptionCount++
err = client.CallContext(ctx, nil, "stream_subscribeStream", snid, NewStream(swarmChunkServerStreamName, "", false), nil, Top)
if err != nil {
return err
}
}
}
//now wait until the number of expected subscriptions has been finished
//`watchSubscriptionEvents` will write with a `nil` value to errc
//every time a `SubscriptionMsg` has been received
for err := range errc {
if err != nil {
return err
}
//`nil` received, decrement count
subscriptionCount--
//all subscriptions received
if subscriptionCount == 0 {
break
}
}
log.Info("Stream subscriptions successfully requested, action terminated")
if live {
//now upload the chunks to the selected random single node
chnks, err := uploadFileToSingleNodeStore(uploadNode.ID(), chunkCount)
if err != nil {
return err
}
conf.hashes = append(conf.hashes, chnks...)
}
return nil
}
chunkSize := storage.DefaultChunkSize
//check defines what will be checked during the test
check := func(ctx context.Context, id discover.NodeID) (bool, error) {
//don't check the uploader node
if id == uploadNode.ID() {
return true, nil
}
select {
case <-ctx.Done():
return false, ctx.Err()
case e := <-disconnectC:
log.Error(e.Error())
return false, fmt.Errorf("Disconnect event detected, network unhealthy")
default:
}
log.Trace(fmt.Sprintf("Checking node: %s", id))
//if there are more than one chunk, test only succeeds if all expected chunks are found
allSuccess := true
//check on the node's FileStore (netstore)
fileStore := registries[id].fileStore
//check all chunks
for _, chnk := range conf.hashes {
reader, _ := fileStore.Retrieve(chnk)
//assuming that reading the Size of the chunk is enough to know we found it
if s, err := reader.Size(nil); err != nil || s != chunkSize {
allSuccess = false
log.Warn("Retrieve error", "err", err, "chunk", chnk, "nodeId", id)
} else {
log.Debug(fmt.Sprintf("Chunk %x found", chnk))
}
}
return allSuccess, nil
}
//for each tick, run the checks on all nodes
timingTicker := time.NewTicker(5 * time.Second)
defer timingTicker.Stop()
go func() {
for range timingTicker.C {
for i := 0; i < len(ids); i++ {
log.Trace(fmt.Sprintf("triggering step %d, id %s", i, ids[i]))
trigger <- ids[i]
}
}
}()
log.Info("Starting simulation run...")
timeout := MaxTimeout * time.Second
ctx, cancel := context.WithTimeout(context.Background(), timeout)
defer cancel()
//run the simulation
result := simulations.NewSimulation(net).Run(ctx, &simulations.Step{
Action: action,
Trigger: trigger,
Expect: &simulations.Expectation{
Nodes: ids,
Check: check,
},
})
if result.Error != nil {
return result.Error
}
return nil
}
//upload generated files to nodes
//every node gets one file uploaded
func uploadFilesToNodes(nodes []*simulations.Node) ([]storage.Address, []string, error) {
nodeCnt := len(nodes)
log.Debug(fmt.Sprintf("Uploading %d files to nodes", nodeCnt))
//array holding generated files
rfiles := make([]string, nodeCnt)
//array holding the root hashes of the files
rootAddrs := make([]storage.Address, nodeCnt)
var err error
//for every node, generate a file and upload
for i, n := range nodes {
id := n.ID()
fileStore := registries[id].fileStore
//generate a file
rfiles[i], err = generateRandomFile()
if err != nil {
return nil, nil, err
}
//store it (upload it) on the FileStore
rk, wait, err := fileStore.Store(strings.NewReader(rfiles[i]), int64(len(rfiles[i])), false)
log.Debug("Uploaded random string file to node")
wait()
if err != nil {
return nil, nil, err
}
rootAddrs[i] = rk
}
return rootAddrs, rfiles, nil
}
//generate a random file (string)
func generateRandomFile() (string, error) {
//generate a random file size between minFileSize and maxFileSize
fileSize := rand.Intn(maxFileSize-minFileSize) + minFileSize
log.Debug(fmt.Sprintf("Generated file with filesize %d kB", fileSize))
b := make([]byte, fileSize*1024)
_, err := crand.Read(b)
if err != nil {
log.Error("Error generating random file.", "err", err)
return "", err
}
return string(b), nil
}

View File

@@ -0,0 +1,719 @@
// Copyright 2018 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package stream
import (
"context"
crand "crypto/rand"
"encoding/json"
"flag"
"fmt"
"io"
"io/ioutil"
"math/rand"
"os"
"sync"
"testing"
"time"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/p2p"
"github.com/ethereum/go-ethereum/p2p/discover"
"github.com/ethereum/go-ethereum/p2p/simulations"
"github.com/ethereum/go-ethereum/p2p/simulations/adapters"
"github.com/ethereum/go-ethereum/rpc"
"github.com/ethereum/go-ethereum/swarm/network"
streamTesting "github.com/ethereum/go-ethereum/swarm/network/stream/testing"
"github.com/ethereum/go-ethereum/swarm/pot"
"github.com/ethereum/go-ethereum/swarm/storage"
)
const testMinProxBinSize = 2
const MaxTimeout = 600
var (
pof = pot.DefaultPof(256)
conf *synctestConfig
ids []discover.NodeID
datadirs map[discover.NodeID]string
ppmap map[string]*network.PeerPot
live bool
history bool
longrunning = flag.Bool("longrunning", false, "do run long-running tests")
)
type synctestConfig struct {
addrs [][]byte
hashes []storage.Address
idToChunksMap map[discover.NodeID][]int
chunksToNodesMap map[string][]int
addrToIdMap map[string]discover.NodeID
}
func init() {
rand.Seed(time.Now().Unix())
}
//common_test needs to initialize the test in a init() func
//in order for adapters to register the NewStreamerService;
//this service is dependent on some global variables
//we thus need to initialize first as init() as well.
func initSyncTest() {
//assign the toAddr func so NewStreamerService can build the addr
toAddr = func(id discover.NodeID) *network.BzzAddr {
addr := network.NewAddrFromNodeID(id)
return addr
}
//global func to create local store
if *useMockStore {
createStoreFunc = createMockStore
} else {
createStoreFunc = createTestLocalStorageForId
}
//local stores
stores = make(map[discover.NodeID]storage.ChunkStore)
//data directories for each node and store
datadirs = make(map[discover.NodeID]string)
//deliveries for each node
deliveries = make(map[discover.NodeID]*Delivery)
//registries, map of discover.NodeID to its streamer
registries = make(map[discover.NodeID]*TestRegistry)
//not needed for this test but required from common_test for NewStreamService
waitPeerErrC = make(chan error)
//also not needed for this test but required for NewStreamService
peerCount = func(id discover.NodeID) int {
if ids[0] == id || ids[len(ids)-1] == id {
return 1
}
return 2
}
if *useMockStore {
createGlobalStore()
}
}
//This test is a syncing test for nodes.
//One node is randomly selected to be the pivot node.
//A configurable number of chunks and nodes can be
//provided to the test, the number of chunks is uploaded
//to the pivot node, and we check that nodes get the chunks
//they are expected to store based on the syncing protocol.
//Number of chunks and nodes can be provided via commandline too.
func TestSyncing(t *testing.T) {
//if nodes/chunks have been provided via commandline,
//run the tests with these values
if *nodes != 0 && *chunks != 0 {
log.Info(fmt.Sprintf("Running test with %d chunks and %d nodes...", *chunks, *nodes))
testSyncing(t, *chunks, *nodes)
} else {
var nodeCnt []int
var chnkCnt []int
//if the `longrunning` flag has been provided
//run more test combinations
if *longrunning {
chnkCnt = []int{1, 8, 32, 256, 1024}
nodeCnt = []int{16, 32, 64, 128, 256}
} else {
//default test
chnkCnt = []int{4, 32}
nodeCnt = []int{32, 16}
}
for _, chnk := range chnkCnt {
for _, n := range nodeCnt {
log.Info(fmt.Sprintf("Long running test with %d chunks and %d nodes...", chnk, n))
testSyncing(t, chnk, n)
}
}
}
}
//Do run the tests
//Every test runs 3 times, a live, a history, and a live AND history
func testSyncing(t *testing.T, chunkCount int, nodeCount int) {
//test live and NO history
log.Info("Testing live and no history")
live = true
history = false
err := runSyncTest(chunkCount, nodeCount, live, history)
if err != nil {
t.Fatal(err)
}
//test history only
log.Info("Testing history only")
live = false
history = true
err = runSyncTest(chunkCount, nodeCount, live, history)
if err != nil {
t.Fatal(err)
}
//finally test live and history
log.Info("Testing live and history")
live = true
err = runSyncTest(chunkCount, nodeCount, live, history)
if err != nil {
t.Fatal(err)
}
}
/*
The test generates the given number of chunks
The upload is done by dependency to the global
`live` and `history` variables;
If `live` is set, first stream subscriptions are established, then
upload to a random node.
If `history` is enabled, first upload then build up subscriptions.
For every chunk generated, the nearest node addresses
are identified, we verify that the nodes closer to the
chunk addresses actually do have the chunks in their local stores.
The test loads a snapshot file to construct the swarm network,
assuming that the snapshot file identifies a healthy
kademlia network. The snapshot should have 'streamer' in its service list.
For every test run, a series of three tests will be executed:
- a LIVE test first, where first subscriptions are established,
then a file (random chunks) is uploaded
- a HISTORY test, where the file is uploaded first, and then
the subscriptions are established
- a crude LIVE AND HISTORY test last, where (different) chunks
are uploaded twice, once before and once after subscriptions
*/
func runSyncTest(chunkCount int, nodeCount int, live bool, history bool) error {
initSyncTest()
//the ids of the snapshot nodes, initiate only now as we need nodeCount
ids = make([]discover.NodeID, nodeCount)
//initialize the test struct
conf = &synctestConfig{}
//map of discover ID to indexes of chunks expected at that ID
conf.idToChunksMap = make(map[discover.NodeID][]int)
//map of overlay address to discover ID
conf.addrToIdMap = make(map[string]discover.NodeID)
//array where the generated chunk hashes will be stored
conf.hashes = make([]storage.Address, 0)
//channel to trigger node checks in the simulation
trigger := make(chan discover.NodeID)
//channel to check for disconnection errors
disconnectC := make(chan error)
//channel to close disconnection watcher routine
quitC := make(chan struct{})
//load nodes from the snapshot file
net, err := initNetWithSnapshot(nodeCount)
if err != nil {
return err
}
var rpcSubscriptionsWg sync.WaitGroup
//do cleanup after test is terminated
defer func() {
// close quitC channel to signall all goroutines to clanup
// before calling simulation network shutdown.
close(quitC)
//wait for all rpc subscriptions to unsubscribe
rpcSubscriptionsWg.Wait()
//shutdown the snapshot network
net.Shutdown()
//after the test, clean up local stores initialized with createLocalStoreForId
localStoreCleanup()
//finally clear all data directories
datadirsCleanup()
}()
//get the nodes of the network
nodes := net.GetNodes()
//select one index at random...
idx := rand.Intn(len(nodes))
//...and get the the node at that index
//this is the node selected for upload
node := nodes[idx]
log.Info("Initializing test config")
//iterate over all nodes...
for c := 0; c < len(nodes); c++ {
//create an array of discovery node IDs
ids[c] = nodes[c].ID()
//get the kademlia overlay address from this ID
a := network.ToOverlayAddr(ids[c].Bytes())
//append it to the array of all overlay addresses
conf.addrs = append(conf.addrs, a)
//the proximity calculation is on overlay addr,
//the p2p/simulations check func triggers on discover.NodeID,
//so we need to know which overlay addr maps to which nodeID
conf.addrToIdMap[string(a)] = ids[c]
}
log.Info("Test config successfully initialized")
//only needed for healthy call when debugging
ppmap = network.NewPeerPotMap(testMinProxBinSize, conf.addrs)
//define the action to be performed before the test checks: start syncing
action := func(ctx context.Context) error {
//first run the health check on all nodes,
//wait until nodes are all healthy
ticker := time.NewTicker(200 * time.Millisecond)
defer ticker.Stop()
for range ticker.C {
healthy := true
for _, id := range ids {
r := registries[id]
//PeerPot for this node
addr := common.Bytes2Hex(network.ToOverlayAddr(id.Bytes()))
pp := ppmap[addr]
//call Healthy RPC
h := r.delivery.overlay.Healthy(pp)
//print info
log.Debug(r.delivery.overlay.String())
log.Debug(fmt.Sprintf("IS HEALTHY: %t", h.GotNN && h.KnowNN && h.Full))
if !h.GotNN || !h.Full {
healthy = false
break
}
}
if healthy {
break
}
}
if history {
log.Info("Uploading for history")
//If testing only history, we upload the chunk(s) first
chunks, err := uploadFileToSingleNodeStore(node.ID(), chunkCount)
if err != nil {
return err
}
conf.hashes = append(conf.hashes, chunks...)
//finally map chunks to the closest addresses
mapKeysToNodes(conf)
}
//variables needed to wait for all subscriptions established before uploading
errc := make(chan error)
//now setup and start event watching in order to know when we can upload
ctx, watchCancel := context.WithTimeout(context.Background(), MaxTimeout*time.Second)
defer watchCancel()
log.Info("Setting up stream subscription")
//We need two iterations, one to subscribe to the subscription events
//(so we know when setup phase is finished), and one to
//actually run the stream subscriptions. We can't do it in the same iteration,
//because while the first nodes in the loop are setting up subscriptions,
//the latter ones have not subscribed to listen to peer events yet,
//and then we miss events.
//first iteration: setup disconnection watcher and subscribe to peer events
for j, id := range ids {
log.Trace(fmt.Sprintf("Subscribe to subscription events: %d", j))
client, err := net.GetNode(id).Client()
if err != nil {
return err
}
wsDoneC := watchSubscriptionEvents(ctx, id, client, errc, quitC)
// doneC is nil, the error happened which is sent to errc channel, already
if wsDoneC == nil {
continue
}
rpcSubscriptionsWg.Add(1)
go func() {
<-wsDoneC
rpcSubscriptionsWg.Done()
}()
//watch for peers disconnecting
wdDoneC, err := streamTesting.WatchDisconnections(id, client, disconnectC, quitC)
if err != nil {
return err
}
rpcSubscriptionsWg.Add(1)
go func() {
<-wdDoneC
rpcSubscriptionsWg.Done()
}()
}
//second iteration: start syncing
for j, id := range ids {
log.Trace(fmt.Sprintf("Start syncing subscriptions: %d", j))
client, err := net.GetNode(id).Client()
if err != nil {
return err
}
//start syncing!
var cnt int
err = client.CallContext(ctx, &cnt, "stream_startSyncing")
if err != nil {
return err
}
//increment the number of subscriptions we need to wait for
//by the count returned from startSyncing (SYNC subscriptions)
subscriptionCount += cnt
}
//now wait until the number of expected subscriptions has been finished
//`watchSubscriptionEvents` will write with a `nil` value to errc
for err := range errc {
if err != nil {
return err
}
//`nil` received, decrement count
subscriptionCount--
//all subscriptions received
if subscriptionCount == 0 {
break
}
}
log.Info("Stream subscriptions successfully requested")
if live {
//now upload the chunks to the selected random single node
hashes, err := uploadFileToSingleNodeStore(node.ID(), chunkCount)
if err != nil {
return err
}
conf.hashes = append(conf.hashes, hashes...)
//finally map chunks to the closest addresses
log.Debug(fmt.Sprintf("Uploaded chunks for live syncing: %v", conf.hashes))
mapKeysToNodes(conf)
log.Info(fmt.Sprintf("Uploaded %d chunks to random single node", chunkCount))
}
log.Info("Action terminated")
return nil
}
//check defines what will be checked during the test
check := func(ctx context.Context, id discover.NodeID) (bool, error) {
select {
case <-ctx.Done():
return false, ctx.Err()
case e := <-disconnectC:
log.Error(e.Error())
return false, fmt.Errorf("Disconnect event detected, network unhealthy")
default:
}
log.Trace(fmt.Sprintf("Checking node: %s", id))
//select the local store for the given node
//if there are more than one chunk, test only succeeds if all expected chunks are found
allSuccess := true
//all the chunk indexes which are supposed to be found for this node
localChunks := conf.idToChunksMap[id]
//for each expected chunk, check if it is in the local store
for _, ch := range localChunks {
//get the real chunk by the index in the index array
chunk := conf.hashes[ch]
log.Trace(fmt.Sprintf("node has chunk: %s:", chunk))
//check if the expected chunk is indeed in the localstore
var err error
if *useMockStore {
if globalStore == nil {
return false, fmt.Errorf("Something went wrong; using mockStore enabled but globalStore is nil")
}
//use the globalStore if the mockStore should be used; in that case,
//the complete localStore stack is bypassed for getting the chunk
_, err = globalStore.Get(common.BytesToAddress(id.Bytes()), chunk)
} else {
//use the actual localstore
lstore := stores[id]
_, err = lstore.Get(chunk)
}
if err != nil {
log.Warn(fmt.Sprintf("Chunk %s NOT found for id %s", chunk, id))
allSuccess = false
} else {
log.Debug(fmt.Sprintf("Chunk %s IS FOUND for id %s", chunk, id))
}
}
return allSuccess, nil
}
//for each tick, run the checks on all nodes
timingTicker := time.NewTicker(time.Second * 1)
defer timingTicker.Stop()
go func() {
for range timingTicker.C {
for i := 0; i < len(ids); i++ {
log.Trace(fmt.Sprintf("triggering step %d, id %s", i, ids[i]))
trigger <- ids[i]
}
}
}()
log.Info("Starting simulation run...")
timeout := MaxTimeout * time.Second
ctx, cancel := context.WithTimeout(context.Background(), timeout)
defer cancel()
//run the simulation
result := simulations.NewSimulation(net).Run(ctx, &simulations.Step{
Action: action,
Trigger: trigger,
Expect: &simulations.Expectation{
Nodes: ids,
Check: check,
},
})
if result.Error != nil {
return result.Error
}
log.Info("Simulation terminated")
return nil
}
//the server func to start syncing
//issues `RequestSubscriptionMsg` to peers, based on po, by iterating over
//the kademlia's `EachBin` function.
//returns the number of subscriptions requested
func (r *TestRegistry) StartSyncing(ctx context.Context) (int, error) {
var err error
if log.Lvl(*loglevel) == log.LvlDebug {
//PeerPot for this node
addr := common.Bytes2Hex(r.addr.OAddr)
pp := ppmap[addr]
//call Healthy RPC
h := r.delivery.overlay.Healthy(pp)
//print info
log.Debug(r.delivery.overlay.String())
log.Debug(fmt.Sprintf("IS HEALTHY: %t", h.GotNN && h.KnowNN && h.Full))
}
kad, ok := r.delivery.overlay.(*network.Kademlia)
if !ok {
return 0, fmt.Errorf("Not a Kademlia!")
}
subCnt := 0
//iterate over each bin and solicit needed subscription to bins
kad.EachBin(r.addr.Over(), pof, 0, func(conn network.OverlayConn, po int) bool {
//identify begin and start index of the bin(s) we want to subscribe to
log.Debug(fmt.Sprintf("Requesting subscription by: registry %s from peer %s for bin: %d", r.addr.ID(), conf.addrToIdMap[string(conn.Address())], po))
var histRange *Range
if history {
histRange = &Range{}
}
subCnt++
err = r.RequestSubscription(conf.addrToIdMap[string(conn.Address())], NewStream("SYNC", FormatSyncBinKey(uint8(po)), live), histRange, Top)
if err != nil {
log.Error(fmt.Sprintf("Error in RequestSubsciption! %v", err))
return false
}
return true
})
return subCnt, nil
}
//map chunk keys to addresses which are responsible
func mapKeysToNodes(conf *synctestConfig) {
kmap := make(map[string][]int)
nodemap := make(map[string][]int)
//build a pot for chunk hashes
np := pot.NewPot(nil, 0)
indexmap := make(map[string]int)
for i, a := range conf.addrs {
indexmap[string(a)] = i
np, _, _ = pot.Add(np, a, pof)
}
//for each address, run EachNeighbour on the chunk hashes pot to identify closest nodes
log.Trace(fmt.Sprintf("Generated hash chunk(s): %v", conf.hashes))
for i := 0; i < len(conf.hashes); i++ {
pl := 256 //highest possible proximity
var nns []int
np.EachNeighbour([]byte(conf.hashes[i]), pof, func(val pot.Val, po int) bool {
a := val.([]byte)
if pl < 256 && pl != po {
return false
}
if pl == 256 || pl == po {
log.Trace(fmt.Sprintf("appending %s", conf.addrToIdMap[string(a)]))
nns = append(nns, indexmap[string(a)])
nodemap[string(a)] = append(nodemap[string(a)], i)
}
if pl == 256 && len(nns) >= testMinProxBinSize {
//maxProxBinSize has been reached at this po, so save it
//we will add all other nodes at the same po
pl = po
}
return true
})
kmap[string(conf.hashes[i])] = nns
}
for addr, chunks := range nodemap {
//this selects which chunks are expected to be found with the given node
conf.idToChunksMap[conf.addrToIdMap[addr]] = chunks
}
log.Debug(fmt.Sprintf("Map of expected chunks by ID: %v", conf.idToChunksMap))
conf.chunksToNodesMap = kmap
}
//upload a file(chunks) to a single local node store
func uploadFileToSingleNodeStore(id discover.NodeID, chunkCount int) ([]storage.Address, error) {
log.Debug(fmt.Sprintf("Uploading to node id: %s", id))
lstore := stores[id]
size := chunkSize
fileStore := storage.NewFileStore(lstore, storage.NewFileStoreParams())
var rootAddrs []storage.Address
for i := 0; i < chunkCount; i++ {
rk, wait, err := fileStore.Store(io.LimitReader(crand.Reader, int64(size)), int64(size), false)
wait()
if err != nil {
return nil, err
}
rootAddrs = append(rootAddrs, (rk))
}
return rootAddrs, nil
}
//initialize a network from a snapshot
func initNetWithSnapshot(nodeCount int) (*simulations.Network, error) {
var a adapters.NodeAdapter
//add the streamer service to the node adapter
if *adapter == "exec" {
dirname, err := ioutil.TempDir(".", "")
if err != nil {
return nil, err
}
a = adapters.NewExecAdapter(dirname)
} else if *adapter == "tcp" {
a = adapters.NewTCPAdapter(services)
} else if *adapter == "sim" {
a = adapters.NewSimAdapter(services)
}
log.Info("Setting up Snapshot network")
net := simulations.NewNetwork(a, &simulations.NetworkConfig{
ID: "0",
DefaultService: "streamer",
})
f, err := os.Open(fmt.Sprintf("testing/snapshot_%d.json", nodeCount))
if err != nil {
return nil, err
}
defer f.Close()
jsonbyte, err := ioutil.ReadAll(f)
if err != nil {
return nil, err
}
var snap simulations.Snapshot
err = json.Unmarshal(jsonbyte, &snap)
if err != nil {
return nil, err
}
//the snapshot probably has the property EnableMsgEvents not set
//just in case, set it to true!
//(we need this to wait for messages before uploading)
for _, n := range snap.Nodes {
n.Node.Config.EnableMsgEvents = true
}
log.Info("Waiting for p2p connections to be established...")
//now we can load the snapshot
err = net.Load(&snap)
if err != nil {
return nil, err
}
log.Info("Snapshot loaded")
return net, nil
}
//we want to wait for subscriptions to be established before uploading to test
//that live syncing is working correctly
func watchSubscriptionEvents(ctx context.Context, id discover.NodeID, client *rpc.Client, errc chan error, quitC chan struct{}) (doneC <-chan struct{}) {
events := make(chan *p2p.PeerEvent)
sub, err := client.Subscribe(context.Background(), "admin", events, "peerEvents")
if err != nil {
log.Error(err.Error())
errc <- fmt.Errorf("error getting peer events for node %v: %s", id, err)
return
}
c := make(chan struct{})
go func() {
defer func() {
log.Trace("watch subscription events: unsubscribe", "id", id)
sub.Unsubscribe()
close(c)
}()
for {
select {
case <-quitC:
return
case <-ctx.Done():
select {
case errc <- ctx.Err():
case <-quitC:
}
return
case e := <-events:
//just catch SubscribeMsg
if e.Type == p2p.PeerEventTypeMsgRecv && e.Protocol == "stream" && e.MsgCode != nil && *e.MsgCode == 4 {
errc <- nil
}
case err := <-sub.Err():
if err != nil {
select {
case errc <- fmt.Errorf("error getting peer events for node %v: %v", id, err):
case <-quitC:
}
return
}
}
}
}()
return c
}
//create a local store for the given node
func createTestLocalStorageForId(id discover.NodeID, addr *network.BzzAddr) (storage.ChunkStore, error) {
var datadir string
var err error
datadir, err = ioutil.TempDir("", fmt.Sprintf("syncer-test-%s", id.TerminalString()))
if err != nil {
return nil, err
}
datadirs[id] = datadir
var store storage.ChunkStore
params := storage.NewDefaultLocalStoreParams()
params.ChunkDbPath = datadir
params.BaseKey = addr.Over()
store, err = storage.NewTestLocalStoreForAddr(params)
if err != nil {
return nil, err
}
return store, nil
}

View File

@@ -0,0 +1,739 @@
// Copyright 2018 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package stream
import (
"context"
"fmt"
"math"
"sync"
"time"
"github.com/ethereum/go-ethereum/metrics"
"github.com/ethereum/go-ethereum/p2p"
"github.com/ethereum/go-ethereum/p2p/discover"
"github.com/ethereum/go-ethereum/p2p/protocols"
"github.com/ethereum/go-ethereum/rpc"
"github.com/ethereum/go-ethereum/swarm/log"
"github.com/ethereum/go-ethereum/swarm/network"
"github.com/ethereum/go-ethereum/swarm/network/stream/intervals"
"github.com/ethereum/go-ethereum/swarm/pot"
"github.com/ethereum/go-ethereum/swarm/state"
"github.com/ethereum/go-ethereum/swarm/storage"
)
const (
Low uint8 = iota
Mid
High
Top
PriorityQueue // number of queues
PriorityQueueCap = 32 // queue capacity
HashSize = 32
)
// Registry registry for outgoing and incoming streamer constructors
type Registry struct {
api *API
addr *network.BzzAddr
skipCheck bool
clientMu sync.RWMutex
serverMu sync.RWMutex
peersMu sync.RWMutex
serverFuncs map[string]func(*Peer, string, bool) (Server, error)
clientFuncs map[string]func(*Peer, string, bool) (Client, error)
peers map[discover.NodeID]*Peer
delivery *Delivery
intervalsStore state.Store
doRetrieve bool
}
// RegistryOptions holds optional values for NewRegistry constructor.
type RegistryOptions struct {
SkipCheck bool
DoSync bool
DoRetrieve bool
SyncUpdateDelay time.Duration
}
// NewRegistry is Streamer constructor
func NewRegistry(addr *network.BzzAddr, delivery *Delivery, db *storage.DBAPI, intervalsStore state.Store, options *RegistryOptions) *Registry {
if options == nil {
options = &RegistryOptions{}
}
if options.SyncUpdateDelay <= 0 {
options.SyncUpdateDelay = 15 * time.Second
}
streamer := &Registry{
addr: addr,
skipCheck: options.SkipCheck,
serverFuncs: make(map[string]func(*Peer, string, bool) (Server, error)),
clientFuncs: make(map[string]func(*Peer, string, bool) (Client, error)),
peers: make(map[discover.NodeID]*Peer),
delivery: delivery,
intervalsStore: intervalsStore,
doRetrieve: options.DoRetrieve,
}
streamer.api = NewAPI(streamer)
delivery.getPeer = streamer.getPeer
streamer.RegisterServerFunc(swarmChunkServerStreamName, func(_ *Peer, _ string, _ bool) (Server, error) {
return NewSwarmChunkServer(delivery.db), nil
})
streamer.RegisterClientFunc(swarmChunkServerStreamName, func(p *Peer, t string, live bool) (Client, error) {
return NewSwarmSyncerClient(p, delivery.db, false, NewStream(swarmChunkServerStreamName, t, live))
})
RegisterSwarmSyncerServer(streamer, db)
RegisterSwarmSyncerClient(streamer, db)
if options.DoSync {
// latestIntC function ensures that
// - receiving from the in chan is not blocked by processing inside the for loop
// - the latest int value is delivered to the loop after the processing is done
// In context of NeighbourhoodDepthC:
// after the syncing is done updating inside the loop, we do not need to update on the intermediate
// depth changes, only to the latest one
latestIntC := func(in <-chan int) <-chan int {
out := make(chan int, 1)
go func() {
defer close(out)
for i := range in {
select {
case <-out:
default:
}
out <- i
}
}()
return out
}
go func() {
// wait for kademlia table to be healthy
time.Sleep(options.SyncUpdateDelay)
kad := streamer.delivery.overlay.(*network.Kademlia)
depthC := latestIntC(kad.NeighbourhoodDepthC())
addressBookSizeC := latestIntC(kad.AddrCountC())
// initial requests for syncing subscription to peers
streamer.updateSyncing()
for depth := range depthC {
log.Debug("Kademlia neighbourhood depth change", "depth", depth)
// Prevent too early sync subscriptions by waiting until there are no
// new peers connecting. Sync streams updating will be done after no
// peers are connected for at least SyncUpdateDelay period.
timer := time.NewTimer(options.SyncUpdateDelay)
// Hard limit to sync update delay, preventing long delays
// on a very dynamic network
maxTimer := time.NewTimer(3 * time.Minute)
loop:
for {
select {
case <-maxTimer.C:
// force syncing update when a hard timeout is reached
log.Trace("Sync subscriptions update on hard timeout")
// request for syncing subscription to new peers
streamer.updateSyncing()
break loop
case <-timer.C:
// start syncing as no new peers has been added to kademlia
// for some time
log.Trace("Sync subscriptions update")
// request for syncing subscription to new peers
streamer.updateSyncing()
break loop
case size := <-addressBookSizeC:
log.Trace("Kademlia address book size changed on depth change", "size", size)
// new peers has been added to kademlia,
// reset the timer to prevent early sync subscriptions
if !timer.Stop() {
<-timer.C
}
timer.Reset(options.SyncUpdateDelay)
}
}
timer.Stop()
maxTimer.Stop()
}
}()
}
return streamer
}
// RegisterClient registers an incoming streamer constructor
func (r *Registry) RegisterClientFunc(stream string, f func(*Peer, string, bool) (Client, error)) {
r.clientMu.Lock()
defer r.clientMu.Unlock()
r.clientFuncs[stream] = f
}
// RegisterServer registers an outgoing streamer constructor
func (r *Registry) RegisterServerFunc(stream string, f func(*Peer, string, bool) (Server, error)) {
r.serverMu.Lock()
defer r.serverMu.Unlock()
r.serverFuncs[stream] = f
}
// GetClient accessor for incoming streamer constructors
func (r *Registry) GetClientFunc(stream string) (func(*Peer, string, bool) (Client, error), error) {
r.clientMu.RLock()
defer r.clientMu.RUnlock()
f := r.clientFuncs[stream]
if f == nil {
return nil, fmt.Errorf("stream %v not registered", stream)
}
return f, nil
}
// GetServer accessor for incoming streamer constructors
func (r *Registry) GetServerFunc(stream string) (func(*Peer, string, bool) (Server, error), error) {
r.serverMu.RLock()
defer r.serverMu.RUnlock()
f := r.serverFuncs[stream]
if f == nil {
return nil, fmt.Errorf("stream %v not registered", stream)
}
return f, nil
}
func (r *Registry) RequestSubscription(peerId discover.NodeID, s Stream, h *Range, prio uint8) error {
// check if the stream is registered
if _, err := r.GetServerFunc(s.Name); err != nil {
return err
}
peer := r.getPeer(peerId)
if peer == nil {
return fmt.Errorf("peer not found %v", peerId)
}
if _, err := peer.getServer(s); err != nil {
if e, ok := err.(*notFoundError); ok && e.t == "server" {
// request subscription only if the server for this stream is not created
log.Debug("RequestSubscription ", "peer", peerId, "stream", s, "history", h)
return peer.Send(&RequestSubscriptionMsg{
Stream: s,
History: h,
Priority: prio,
})
}
return err
}
log.Trace("RequestSubscription: already subscribed", "peer", peerId, "stream", s, "history", h)
return nil
}
// Subscribe initiates the streamer
func (r *Registry) Subscribe(peerId discover.NodeID, s Stream, h *Range, priority uint8) error {
// check if the stream is registered
if _, err := r.GetClientFunc(s.Name); err != nil {
return err
}
peer := r.getPeer(peerId)
if peer == nil {
return fmt.Errorf("peer not found %v", peerId)
}
var to uint64
if !s.Live && h != nil {
to = h.To
}
err := peer.setClientParams(s, newClientParams(priority, to))
if err != nil {
return err
}
if s.Live && h != nil {
if err := peer.setClientParams(
getHistoryStream(s),
newClientParams(getHistoryPriority(priority), h.To),
); err != nil {
return err
}
}
msg := &SubscribeMsg{
Stream: s,
History: h,
Priority: priority,
}
log.Debug("Subscribe ", "peer", peerId, "stream", s, "history", h)
return peer.SendPriority(msg, priority)
}
func (r *Registry) Unsubscribe(peerId discover.NodeID, s Stream) error {
peer := r.getPeer(peerId)
if peer == nil {
return fmt.Errorf("peer not found %v", peerId)
}
msg := &UnsubscribeMsg{
Stream: s,
}
log.Debug("Unsubscribe ", "peer", peerId, "stream", s)
if err := peer.Send(msg); err != nil {
return err
}
return peer.removeClient(s)
}
// Quit sends the QuitMsg to the peer to remove the
// stream peer client and terminate the streaming.
func (r *Registry) Quit(peerId discover.NodeID, s Stream) error {
peer := r.getPeer(peerId)
if peer == nil {
log.Debug("stream quit: peer not found", "peer", peerId, "stream", s)
// if the peer is not found, abort the request
return nil
}
msg := &QuitMsg{
Stream: s,
}
log.Debug("Quit ", "peer", peerId, "stream", s)
return peer.Send(msg)
}
func (r *Registry) Retrieve(chunk *storage.Chunk) error {
return r.delivery.RequestFromPeers(chunk.Addr[:], r.skipCheck)
}
func (r *Registry) NodeInfo() interface{} {
return nil
}
func (r *Registry) PeerInfo(id discover.NodeID) interface{} {
return nil
}
func (r *Registry) Close() error {
return r.intervalsStore.Close()
}
func (r *Registry) getPeer(peerId discover.NodeID) *Peer {
r.peersMu.RLock()
defer r.peersMu.RUnlock()
return r.peers[peerId]
}
func (r *Registry) setPeer(peer *Peer) {
r.peersMu.Lock()
r.peers[peer.ID()] = peer
metrics.GetOrRegisterGauge("registry.peers", nil).Update(int64(len(r.peers)))
r.peersMu.Unlock()
}
func (r *Registry) deletePeer(peer *Peer) {
r.peersMu.Lock()
delete(r.peers, peer.ID())
metrics.GetOrRegisterGauge("registry.peers", nil).Update(int64(len(r.peers)))
r.peersMu.Unlock()
}
func (r *Registry) peersCount() (c int) {
r.peersMu.Lock()
c = len(r.peers)
r.peersMu.Unlock()
return
}
// Run protocol run function
func (r *Registry) Run(p *network.BzzPeer) error {
sp := NewPeer(p.Peer, r)
r.setPeer(sp)
defer r.deletePeer(sp)
defer close(sp.quit)
defer sp.close()
if r.doRetrieve {
err := r.Subscribe(p.ID(), NewStream(swarmChunkServerStreamName, "", false), nil, Top)
if err != nil {
return err
}
}
return sp.Run(sp.HandleMsg)
}
// updateSyncing subscribes to SYNC streams by iterating over the
// kademlia connections and bins. If there are existing SYNC streams
// and they are no longer required after iteration, request to Quit
// them will be send to appropriate peers.
func (r *Registry) updateSyncing() {
// if overlay in not Kademlia, panic
kad := r.delivery.overlay.(*network.Kademlia)
// map of all SYNC streams for all peers
// used at the and of the function to remove servers
// that are not needed anymore
subs := make(map[discover.NodeID]map[Stream]struct{})
r.peersMu.RLock()
for id, peer := range r.peers {
peer.serverMu.RLock()
for stream := range peer.servers {
if stream.Name == "SYNC" {
if _, ok := subs[id]; !ok {
subs[id] = make(map[Stream]struct{})
}
subs[id][stream] = struct{}{}
}
}
peer.serverMu.RUnlock()
}
r.peersMu.RUnlock()
// request subscriptions for all nodes and bins
kad.EachBin(r.addr.Over(), pot.DefaultPof(256), 0, func(conn network.OverlayConn, bin int) bool {
p := conn.(network.Peer)
log.Debug(fmt.Sprintf("Requesting subscription by: registry %s from peer %s for bin: %d", r.addr.ID(), p.ID(), bin))
// bin is always less then 256 and it is safe to convert it to type uint8
stream := NewStream("SYNC", FormatSyncBinKey(uint8(bin)), true)
if streams, ok := subs[p.ID()]; ok {
// delete live and history streams from the map, so that it won't be removed with a Quit request
delete(streams, stream)
delete(streams, getHistoryStream(stream))
}
err := r.RequestSubscription(p.ID(), stream, NewRange(0, 0), High)
if err != nil {
log.Debug("Request subscription", "err", err, "peer", p.ID(), "stream", stream)
return false
}
return true
})
// remove SYNC servers that do not need to be subscribed
for id, streams := range subs {
if len(streams) == 0 {
continue
}
peer := r.getPeer(id)
if peer == nil {
continue
}
for stream := range streams {
log.Debug("Remove sync server", "peer", id, "stream", stream)
err := r.Quit(peer.ID(), stream)
if err != nil && err != p2p.ErrShuttingDown {
log.Error("quit", "err", err, "peer", peer.ID(), "stream", stream)
}
}
}
}
func (r *Registry) runProtocol(p *p2p.Peer, rw p2p.MsgReadWriter) error {
peer := protocols.NewPeer(p, rw, Spec)
bzzPeer := network.NewBzzTestPeer(peer, r.addr)
r.delivery.overlay.On(bzzPeer)
defer r.delivery.overlay.Off(bzzPeer)
return r.Run(bzzPeer)
}
// HandleMsg is the message handler that delegates incoming messages
func (p *Peer) HandleMsg(msg interface{}) error {
switch msg := msg.(type) {
case *SubscribeMsg:
return p.handleSubscribeMsg(msg)
case *SubscribeErrorMsg:
return p.handleSubscribeErrorMsg(msg)
case *UnsubscribeMsg:
return p.handleUnsubscribeMsg(msg)
case *OfferedHashesMsg:
return p.handleOfferedHashesMsg(msg)
case *TakeoverProofMsg:
return p.handleTakeoverProofMsg(msg)
case *WantedHashesMsg:
return p.handleWantedHashesMsg(msg)
case *ChunkDeliveryMsg:
return p.streamer.delivery.handleChunkDeliveryMsg(p, msg)
case *RetrieveRequestMsg:
return p.streamer.delivery.handleRetrieveRequestMsg(p, msg)
case *RequestSubscriptionMsg:
return p.handleRequestSubscription(msg)
case *QuitMsg:
return p.handleQuitMsg(msg)
default:
return fmt.Errorf("unknown message type: %T", msg)
}
}
type server struct {
Server
stream Stream
priority uint8
currentBatch []byte
}
// Server interface for outgoing peer Streamer
type Server interface {
SetNextBatch(uint64, uint64) (hashes []byte, from uint64, to uint64, proof *HandoverProof, err error)
GetData([]byte) ([]byte, error)
Close()
}
type client struct {
Client
stream Stream
priority uint8
sessionAt uint64
to uint64
next chan error
quit chan struct{}
intervalsKey string
intervalsStore state.Store
}
func peerStreamIntervalsKey(p *Peer, s Stream) string {
return p.ID().String() + s.String()
}
func (c client) AddInterval(start, end uint64) (err error) {
i := &intervals.Intervals{}
err = c.intervalsStore.Get(c.intervalsKey, i)
if err != nil {
return err
}
i.Add(start, end)
return c.intervalsStore.Put(c.intervalsKey, i)
}
func (c client) NextInterval() (start, end uint64, err error) {
i := &intervals.Intervals{}
err = c.intervalsStore.Get(c.intervalsKey, i)
if err != nil {
return 0, 0, err
}
start, end = i.Next()
return start, end, nil
}
// Client interface for incoming peer Streamer
type Client interface {
NeedData([]byte) func()
BatchDone(Stream, uint64, []byte, []byte) func() (*TakeoverProof, error)
Close()
}
func (c *client) nextBatch(from uint64) (nextFrom uint64, nextTo uint64) {
if c.to > 0 && from >= c.to {
return 0, 0
}
if c.stream.Live {
return from, 0
} else if from >= c.sessionAt {
if c.to > 0 {
return from, c.to
}
return from, math.MaxUint64
}
nextFrom, nextTo, err := c.NextInterval()
if err != nil {
log.Error("next intervals", "stream", c.stream)
return
}
if nextTo > c.to {
nextTo = c.to
}
if nextTo == 0 {
nextTo = c.sessionAt
}
return
}
func (c *client) batchDone(p *Peer, req *OfferedHashesMsg, hashes []byte) error {
if tf := c.BatchDone(req.Stream, req.From, hashes, req.Root); tf != nil {
tp, err := tf()
if err != nil {
return err
}
if err := p.SendPriority(tp, c.priority); err != nil {
return err
}
if c.to > 0 && tp.Takeover.End >= c.to {
return p.streamer.Unsubscribe(p.Peer.ID(), req.Stream)
}
return nil
}
// TODO: make a test case for testing if the interval is added when the batch is done
if err := c.AddInterval(req.From, req.To); err != nil {
return err
}
return nil
}
func (c *client) close() {
select {
case <-c.quit:
default:
close(c.quit)
}
c.Close()
}
// clientParams store parameters for the new client
// between a subscription and initial offered hashes request handling.
type clientParams struct {
priority uint8
to uint64
// signal when the client is created
clientCreatedC chan struct{}
}
func newClientParams(priority uint8, to uint64) *clientParams {
return &clientParams{
priority: priority,
to: to,
clientCreatedC: make(chan struct{}),
}
}
func (c *clientParams) waitClient(ctx context.Context) error {
select {
case <-ctx.Done():
return ctx.Err()
case <-c.clientCreatedC:
return nil
}
}
func (c *clientParams) clientCreated() {
close(c.clientCreatedC)
}
// Spec is the spec of the streamer protocol
var Spec = &protocols.Spec{
Name: "stream",
Version: 4,
MaxMsgSize: 10 * 1024 * 1024,
Messages: []interface{}{
UnsubscribeMsg{},
OfferedHashesMsg{},
WantedHashesMsg{},
TakeoverProofMsg{},
SubscribeMsg{},
RetrieveRequestMsg{},
ChunkDeliveryMsg{},
SubscribeErrorMsg{},
RequestSubscriptionMsg{},
QuitMsg{},
},
}
func (r *Registry) Protocols() []p2p.Protocol {
return []p2p.Protocol{
{
Name: Spec.Name,
Version: Spec.Version,
Length: Spec.Length(),
Run: r.runProtocol,
// NodeInfo: ,
// PeerInfo: ,
},
}
}
func (r *Registry) APIs() []rpc.API {
return []rpc.API{
{
Namespace: "stream",
Version: "3.0",
Service: r.api,
Public: true,
},
}
}
func (r *Registry) Start(server *p2p.Server) error {
log.Info("Streamer started")
return nil
}
func (r *Registry) Stop() error {
return nil
}
type Range struct {
From, To uint64
}
func NewRange(from, to uint64) *Range {
return &Range{
From: from,
To: to,
}
}
func (r *Range) String() string {
return fmt.Sprintf("%v-%v", r.From, r.To)
}
func getHistoryPriority(priority uint8) uint8 {
if priority == 0 {
return 0
}
return priority - 1
}
func getHistoryStream(s Stream) Stream {
return NewStream(s.Name, s.Key, false)
}
type API struct {
streamer *Registry
}
func NewAPI(r *Registry) *API {
return &API{
streamer: r,
}
}
func (api *API) SubscribeStream(peerId discover.NodeID, s Stream, history *Range, priority uint8) error {
return api.streamer.Subscribe(peerId, s, history, priority)
}
func (api *API) UnsubscribeStream(peerId discover.NodeID, s Stream) error {
return api.streamer.Unsubscribe(peerId, s)
}

View File

@@ -0,0 +1,684 @@
// Copyright 2018 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package stream
import (
"bytes"
"testing"
"time"
"github.com/ethereum/go-ethereum/crypto/sha3"
p2ptest "github.com/ethereum/go-ethereum/p2p/testing"
)
func TestStreamerSubscribe(t *testing.T) {
tester, streamer, _, teardown, err := newStreamerTester(t)
defer teardown()
if err != nil {
t.Fatal(err)
}
stream := NewStream("foo", "", true)
err = streamer.Subscribe(tester.IDs[0], stream, NewRange(0, 0), Top)
if err == nil || err.Error() != "stream foo not registered" {
t.Fatalf("Expected error %v, got %v", "stream foo not registered", err)
}
}
func TestStreamerRequestSubscription(t *testing.T) {
tester, streamer, _, teardown, err := newStreamerTester(t)
defer teardown()
if err != nil {
t.Fatal(err)
}
stream := NewStream("foo", "", false)
err = streamer.RequestSubscription(tester.IDs[0], stream, &Range{}, Top)
if err == nil || err.Error() != "stream foo not registered" {
t.Fatalf("Expected error %v, got %v", "stream foo not registered", err)
}
}
var (
hash0 = sha3.Sum256([]byte{0})
hash1 = sha3.Sum256([]byte{1})
hash2 = sha3.Sum256([]byte{2})
hashesTmp = append(hash0[:], hash1[:]...)
hashes = append(hashesTmp, hash2[:]...)
)
type testClient struct {
t string
wait0 chan bool
wait2 chan bool
batchDone chan bool
receivedHashes map[string][]byte
}
func newTestClient(t string) *testClient {
return &testClient{
t: t,
wait0: make(chan bool),
wait2: make(chan bool),
batchDone: make(chan bool),
receivedHashes: make(map[string][]byte),
}
}
func (self *testClient) NeedData(hash []byte) func() {
self.receivedHashes[string(hash)] = hash
if bytes.Equal(hash, hash0[:]) {
return func() {
<-self.wait0
}
} else if bytes.Equal(hash, hash2[:]) {
return func() {
<-self.wait2
}
}
return nil
}
func (self *testClient) BatchDone(Stream, uint64, []byte, []byte) func() (*TakeoverProof, error) {
close(self.batchDone)
return nil
}
func (self *testClient) Close() {}
type testServer struct {
t string
}
func newTestServer(t string) *testServer {
return &testServer{
t: t,
}
}
func (self *testServer) SetNextBatch(from uint64, to uint64) ([]byte, uint64, uint64, *HandoverProof, error) {
return make([]byte, HashSize), from + 1, to + 1, nil, nil
}
func (self *testServer) GetData([]byte) ([]byte, error) {
return nil, nil
}
func (self *testServer) Close() {
}
func TestStreamerDownstreamSubscribeUnsubscribeMsgExchange(t *testing.T) {
tester, streamer, _, teardown, err := newStreamerTester(t)
defer teardown()
if err != nil {
t.Fatal(err)
}
streamer.RegisterClientFunc("foo", func(p *Peer, t string, live bool) (Client, error) {
return newTestClient(t), nil
})
peerID := tester.IDs[0]
stream := NewStream("foo", "", true)
err = streamer.Subscribe(peerID, stream, NewRange(5, 8), Top)
if err != nil {
t.Fatalf("Expected no error, got %v", err)
}
err = tester.TestExchanges(
p2ptest.Exchange{
Label: "Subscribe message",
Expects: []p2ptest.Expect{
{
Code: 4,
Msg: &SubscribeMsg{
Stream: stream,
History: NewRange(5, 8),
Priority: Top,
},
Peer: peerID,
},
},
},
// trigger OfferedHashesMsg to actually create the client
p2ptest.Exchange{
Label: "OfferedHashes message",
Triggers: []p2ptest.Trigger{
{
Code: 1,
Msg: &OfferedHashesMsg{
HandoverProof: &HandoverProof{
Handover: &Handover{},
},
Hashes: hashes,
From: 5,
To: 8,
Stream: stream,
},
Peer: peerID,
},
},
Expects: []p2ptest.Expect{
{
Code: 2,
Msg: &WantedHashesMsg{
Stream: stream,
Want: []byte{5},
From: 9,
To: 0,
},
Peer: peerID,
},
},
},
)
if err != nil {
t.Fatal(err)
}
err = streamer.Unsubscribe(peerID, stream)
if err != nil {
t.Fatalf("Expected no error, got %v", err)
}
err = tester.TestExchanges(p2ptest.Exchange{
Label: "Unsubscribe message",
Expects: []p2ptest.Expect{
{
Code: 0,
Msg: &UnsubscribeMsg{
Stream: stream,
},
Peer: peerID,
},
},
})
if err != nil {
t.Fatal(err)
}
}
func TestStreamerUpstreamSubscribeUnsubscribeMsgExchange(t *testing.T) {
tester, streamer, _, teardown, err := newStreamerTester(t)
defer teardown()
if err != nil {
t.Fatal(err)
}
stream := NewStream("foo", "", false)
streamer.RegisterServerFunc("foo", func(p *Peer, t string, live bool) (Server, error) {
return newTestServer(t), nil
})
peerID := tester.IDs[0]
err = tester.TestExchanges(p2ptest.Exchange{
Label: "Subscribe message",
Triggers: []p2ptest.Trigger{
{
Code: 4,
Msg: &SubscribeMsg{
Stream: stream,
History: NewRange(5, 8),
Priority: Top,
},
Peer: peerID,
},
},
Expects: []p2ptest.Expect{
{
Code: 1,
Msg: &OfferedHashesMsg{
Stream: stream,
HandoverProof: &HandoverProof{
Handover: &Handover{},
},
Hashes: make([]byte, HashSize),
From: 6,
To: 9,
},
Peer: peerID,
},
},
})
if err != nil {
t.Fatal(err)
}
err = tester.TestExchanges(p2ptest.Exchange{
Label: "unsubscribe message",
Triggers: []p2ptest.Trigger{
{
Code: 0,
Msg: &UnsubscribeMsg{
Stream: stream,
},
Peer: peerID,
},
},
})
if err != nil {
t.Fatal(err)
}
}
func TestStreamerUpstreamSubscribeUnsubscribeMsgExchangeLive(t *testing.T) {
tester, streamer, _, teardown, err := newStreamerTester(t)
defer teardown()
if err != nil {
t.Fatal(err)
}
stream := NewStream("foo", "", true)
streamer.RegisterServerFunc("foo", func(p *Peer, t string, live bool) (Server, error) {
return newTestServer(t), nil
})
peerID := tester.IDs[0]
err = tester.TestExchanges(p2ptest.Exchange{
Label: "Subscribe message",
Triggers: []p2ptest.Trigger{
{
Code: 4,
Msg: &SubscribeMsg{
Stream: stream,
Priority: Top,
},
Peer: peerID,
},
},
Expects: []p2ptest.Expect{
{
Code: 1,
Msg: &OfferedHashesMsg{
Stream: stream,
HandoverProof: &HandoverProof{
Handover: &Handover{},
},
Hashes: make([]byte, HashSize),
From: 1,
To: 1,
},
Peer: peerID,
},
},
})
if err != nil {
t.Fatal(err)
}
err = tester.TestExchanges(p2ptest.Exchange{
Label: "unsubscribe message",
Triggers: []p2ptest.Trigger{
{
Code: 0,
Msg: &UnsubscribeMsg{
Stream: stream,
},
Peer: peerID,
},
},
})
if err != nil {
t.Fatal(err)
}
}
func TestStreamerUpstreamSubscribeErrorMsgExchange(t *testing.T) {
tester, streamer, _, teardown, err := newStreamerTester(t)
defer teardown()
if err != nil {
t.Fatal(err)
}
streamer.RegisterServerFunc("foo", func(p *Peer, t string, live bool) (Server, error) {
return newTestServer(t), nil
})
stream := NewStream("bar", "", true)
peerID := tester.IDs[0]
err = tester.TestExchanges(p2ptest.Exchange{
Label: "Subscribe message",
Triggers: []p2ptest.Trigger{
{
Code: 4,
Msg: &SubscribeMsg{
Stream: stream,
History: NewRange(5, 8),
Priority: Top,
},
Peer: peerID,
},
},
Expects: []p2ptest.Expect{
{
Code: 7,
Msg: &SubscribeErrorMsg{
Error: "stream bar not registered",
},
Peer: peerID,
},
},
})
if err != nil {
t.Fatal(err)
}
}
func TestStreamerUpstreamSubscribeLiveAndHistory(t *testing.T) {
tester, streamer, _, teardown, err := newStreamerTester(t)
defer teardown()
if err != nil {
t.Fatal(err)
}
stream := NewStream("foo", "", true)
streamer.RegisterServerFunc("foo", func(p *Peer, t string, live bool) (Server, error) {
return &testServer{
t: t,
}, nil
})
peerID := tester.IDs[0]
err = tester.TestExchanges(p2ptest.Exchange{
Label: "Subscribe message",
Triggers: []p2ptest.Trigger{
{
Code: 4,
Msg: &SubscribeMsg{
Stream: stream,
History: NewRange(5, 8),
Priority: Top,
},
Peer: peerID,
},
},
Expects: []p2ptest.Expect{
{
Code: 1,
Msg: &OfferedHashesMsg{
Stream: NewStream("foo", "", false),
HandoverProof: &HandoverProof{
Handover: &Handover{},
},
Hashes: make([]byte, HashSize),
From: 6,
To: 9,
},
Peer: peerID,
},
{
Code: 1,
Msg: &OfferedHashesMsg{
Stream: stream,
HandoverProof: &HandoverProof{
Handover: &Handover{},
},
From: 1,
To: 1,
Hashes: make([]byte, HashSize),
},
Peer: peerID,
},
},
})
if err != nil {
t.Fatal(err)
}
}
func TestStreamerDownstreamOfferedHashesMsgExchange(t *testing.T) {
tester, streamer, _, teardown, err := newStreamerTester(t)
defer teardown()
if err != nil {
t.Fatal(err)
}
stream := NewStream("foo", "", true)
var tc *testClient
streamer.RegisterClientFunc("foo", func(p *Peer, t string, live bool) (Client, error) {
tc = newTestClient(t)
return tc, nil
})
peerID := tester.IDs[0]
err = streamer.Subscribe(peerID, stream, NewRange(5, 8), Top)
if err != nil {
t.Fatalf("Expected no error, got %v", err)
}
err = tester.TestExchanges(p2ptest.Exchange{
Label: "Subscribe message",
Expects: []p2ptest.Expect{
{
Code: 4,
Msg: &SubscribeMsg{
Stream: stream,
History: NewRange(5, 8),
Priority: Top,
},
Peer: peerID,
},
},
},
p2ptest.Exchange{
Label: "WantedHashes message",
Triggers: []p2ptest.Trigger{
{
Code: 1,
Msg: &OfferedHashesMsg{
HandoverProof: &HandoverProof{
Handover: &Handover{},
},
Hashes: hashes,
From: 5,
To: 8,
Stream: stream,
},
Peer: peerID,
},
},
Expects: []p2ptest.Expect{
{
Code: 2,
Msg: &WantedHashesMsg{
Stream: stream,
Want: []byte{5},
From: 9,
To: 0,
},
Peer: peerID,
},
},
})
if err != nil {
t.Fatal(err)
}
if len(tc.receivedHashes) != 3 {
t.Fatalf("Expected number of received hashes %v, got %v", 3, len(tc.receivedHashes))
}
close(tc.wait0)
timeout := time.NewTimer(100 * time.Millisecond)
defer timeout.Stop()
select {
case <-tc.batchDone:
t.Fatal("batch done early")
case <-timeout.C:
}
close(tc.wait2)
timeout2 := time.NewTimer(10000 * time.Millisecond)
defer timeout2.Stop()
select {
case <-tc.batchDone:
case <-timeout2.C:
t.Fatal("timeout waiting batchdone call")
}
}
func TestStreamerRequestSubscriptionQuitMsgExchange(t *testing.T) {
tester, streamer, _, teardown, err := newStreamerTester(t)
defer teardown()
if err != nil {
t.Fatal(err)
}
streamer.RegisterServerFunc("foo", func(p *Peer, t string, live bool) (Server, error) {
return newTestServer(t), nil
})
peerID := tester.IDs[0]
stream := NewStream("foo", "", true)
err = streamer.RequestSubscription(peerID, stream, NewRange(5, 8), Top)
if err != nil {
t.Fatalf("Expected no error, got %v", err)
}
err = tester.TestExchanges(
p2ptest.Exchange{
Label: "RequestSubscription message",
Expects: []p2ptest.Expect{
{
Code: 8,
Msg: &RequestSubscriptionMsg{
Stream: stream,
History: NewRange(5, 8),
Priority: Top,
},
Peer: peerID,
},
},
},
p2ptest.Exchange{
Label: "Subscribe message",
Triggers: []p2ptest.Trigger{
{
Code: 4,
Msg: &SubscribeMsg{
Stream: stream,
History: NewRange(5, 8),
Priority: Top,
},
Peer: peerID,
},
},
Expects: []p2ptest.Expect{
{
Code: 1,
Msg: &OfferedHashesMsg{
Stream: NewStream("foo", "", false),
HandoverProof: &HandoverProof{
Handover: &Handover{},
},
Hashes: make([]byte, HashSize),
From: 6,
To: 9,
},
Peer: peerID,
},
{
Code: 1,
Msg: &OfferedHashesMsg{
Stream: stream,
HandoverProof: &HandoverProof{
Handover: &Handover{},
},
From: 1,
To: 1,
Hashes: make([]byte, HashSize),
},
Peer: peerID,
},
},
},
)
if err != nil {
t.Fatal(err)
}
err = streamer.Quit(peerID, stream)
if err != nil {
t.Fatalf("Expected no error, got %v", err)
}
err = tester.TestExchanges(p2ptest.Exchange{
Label: "Quit message",
Expects: []p2ptest.Expect{
{
Code: 9,
Msg: &QuitMsg{
Stream: stream,
},
Peer: peerID,
},
},
})
if err != nil {
t.Fatal(err)
}
historyStream := getHistoryStream(stream)
err = streamer.Quit(peerID, historyStream)
if err != nil {
t.Fatalf("Expected no error, got %v", err)
}
err = tester.TestExchanges(p2ptest.Exchange{
Label: "Quit message",
Expects: []p2ptest.Expect{
{
Code: 9,
Msg: &QuitMsg{
Stream: historyStream,
},
Peer: peerID,
},
},
})
if err != nil {
t.Fatal(err)
}
}

View File

@@ -0,0 +1,297 @@
// Copyright 2018 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package stream
import (
"math"
"strconv"
"time"
"github.com/ethereum/go-ethereum/metrics"
"github.com/ethereum/go-ethereum/swarm/log"
"github.com/ethereum/go-ethereum/swarm/storage"
)
const (
// BatchSize = 2
BatchSize = 128
)
// SwarmSyncerServer implements an Server for history syncing on bins
// offered streams:
// * live request delivery with or without checkback
// * (live/non-live historical) chunk syncing per proximity bin
type SwarmSyncerServer struct {
po uint8
db *storage.DBAPI
sessionAt uint64
start uint64
quit chan struct{}
}
// NewSwarmSyncerServer is contructor for SwarmSyncerServer
func NewSwarmSyncerServer(live bool, po uint8, db *storage.DBAPI) (*SwarmSyncerServer, error) {
sessionAt := db.CurrentBucketStorageIndex(po)
var start uint64
if live {
start = sessionAt
}
return &SwarmSyncerServer{
po: po,
db: db,
sessionAt: sessionAt,
start: start,
quit: make(chan struct{}),
}, nil
}
func RegisterSwarmSyncerServer(streamer *Registry, db *storage.DBAPI) {
streamer.RegisterServerFunc("SYNC", func(p *Peer, t string, live bool) (Server, error) {
po, err := ParseSyncBinKey(t)
if err != nil {
return nil, err
}
return NewSwarmSyncerServer(live, po, db)
})
// streamer.RegisterServerFunc(stream, func(p *Peer) (Server, error) {
// return NewOutgoingProvableSwarmSyncer(po, db)
// })
}
// Close needs to be called on a stream server
func (s *SwarmSyncerServer) Close() {
close(s.quit)
}
// GetSection retrieves the actual chunk from localstore
func (s *SwarmSyncerServer) GetData(key []byte) ([]byte, error) {
chunk, err := s.db.Get(storage.Address(key))
if err == storage.ErrFetching {
<-chunk.ReqC
} else if err != nil {
return nil, err
}
return chunk.SData, nil
}
// GetBatch retrieves the next batch of hashes from the dbstore
func (s *SwarmSyncerServer) SetNextBatch(from, to uint64) ([]byte, uint64, uint64, *HandoverProof, error) {
var batch []byte
i := 0
if from == 0 {
from = s.start
}
if to <= from || from >= s.sessionAt {
to = math.MaxUint64
}
var ticker *time.Ticker
defer func() {
if ticker != nil {
ticker.Stop()
}
}()
var wait bool
for {
if wait {
if ticker == nil {
ticker = time.NewTicker(1000 * time.Millisecond)
}
select {
case <-ticker.C:
case <-s.quit:
return nil, 0, 0, nil, nil
}
}
metrics.GetOrRegisterCounter("syncer.setnextbatch.iterator", nil).Inc(1)
err := s.db.Iterator(from, to, s.po, func(addr storage.Address, idx uint64) bool {
batch = append(batch, addr[:]...)
i++
to = idx
return i < BatchSize
})
if err != nil {
return nil, 0, 0, nil, err
}
if len(batch) > 0 {
break
}
wait = true
}
log.Trace("Swarm syncer offer batch", "po", s.po, "len", i, "from", from, "to", to, "current store count", s.db.CurrentBucketStorageIndex(s.po))
return batch, from, to, nil, nil
}
// SwarmSyncerClient
type SwarmSyncerClient struct {
sessionAt uint64
nextC chan struct{}
sessionRoot storage.Address
sessionReader storage.LazySectionReader
retrieveC chan *storage.Chunk
storeC chan *storage.Chunk
db *storage.DBAPI
// chunker storage.Chunker
currentRoot storage.Address
requestFunc func(chunk *storage.Chunk)
end, start uint64
peer *Peer
ignoreExistingRequest bool
stream Stream
}
// NewSwarmSyncerClient is a contructor for provable data exchange syncer
func NewSwarmSyncerClient(p *Peer, db *storage.DBAPI, ignoreExistingRequest bool, stream Stream) (*SwarmSyncerClient, error) {
return &SwarmSyncerClient{
db: db,
peer: p,
ignoreExistingRequest: ignoreExistingRequest,
stream: stream,
}, nil
}
// // NewIncomingProvableSwarmSyncer is a contructor for provable data exchange syncer
// func NewIncomingProvableSwarmSyncer(po int, priority int, index uint64, sessionAt uint64, intervals []uint64, sessionRoot storage.Key, chunker *storage.PyramidChunker, store storage.ChunkStore, p Peer) *SwarmSyncerClient {
// retrieveC := make(storage.Chunk, chunksCap)
// RunChunkRequestor(p, retrieveC)
// storeC := make(storage.Chunk, chunksCap)
// RunChunkStorer(store, storeC)
// s := &SwarmSyncerClient{
// po: po,
// priority: priority,
// sessionAt: sessionAt,
// start: index,
// end: index,
// nextC: make(chan struct{}, 1),
// intervals: intervals,
// sessionRoot: sessionRoot,
// sessionReader: chunker.Join(sessionRoot, retrieveC),
// retrieveC: retrieveC,
// storeC: storeC,
// }
// return s
// }
// // StartSyncing is called on the Peer to start the syncing process
// // the idea is that it is called only after kademlia is close to healthy
// func StartSyncing(s *Streamer, peerId discover.NodeID, po uint8, nn bool) {
// lastPO := po
// if nn {
// lastPO = maxPO
// }
//
// for i := po; i <= lastPO; i++ {
// s.Subscribe(peerId, "SYNC", newSyncLabel("LIVE", po), 0, 0, High, true)
// s.Subscribe(peerId, "SYNC", newSyncLabel("HISTORY", po), 0, 0, Mid, false)
// }
// }
// RegisterSwarmSyncerClient registers the client constructor function for
// to handle incoming sync streams
func RegisterSwarmSyncerClient(streamer *Registry, db *storage.DBAPI) {
streamer.RegisterClientFunc("SYNC", func(p *Peer, t string, live bool) (Client, error) {
return NewSwarmSyncerClient(p, db, true, NewStream("SYNC", t, live))
})
}
// NeedData
func (s *SwarmSyncerClient) NeedData(key []byte) (wait func()) {
chunk, _ := s.db.GetOrCreateRequest(key)
// TODO: we may want to request from this peer anyway even if the request exists
// ignoreExistingRequest is temporary commented out until its functionality is verified.
// For now, this optimization can be disabled.
if chunk.ReqC == nil { //|| (s.ignoreExistingRequest && !created) {
return nil
}
// create request and wait until the chunk data arrives and is stored
return func() {
chunk.WaitToStore()
}
}
// BatchDone
func (s *SwarmSyncerClient) BatchDone(stream Stream, from uint64, hashes []byte, root []byte) func() (*TakeoverProof, error) {
// TODO: reenable this with putter/getter refactored code
// if s.chunker != nil {
// return func() (*TakeoverProof, error) { return s.TakeoverProof(stream, from, hashes, root) }
// }
return nil
}
func (s *SwarmSyncerClient) TakeoverProof(stream Stream, from uint64, hashes []byte, root storage.Address) (*TakeoverProof, error) {
// for provable syncer currentRoot is non-zero length
// TODO: reenable this with putter/getter
// if s.chunker != nil {
// if from > s.sessionAt { // for live syncing currentRoot is always updated
// //expRoot, err := s.chunker.Append(s.currentRoot, bytes.NewReader(hashes), s.retrieveC, s.storeC)
// expRoot, _, err := s.chunker.Append(s.currentRoot, bytes.NewReader(hashes), s.retrieveC)
// if err != nil {
// return nil, err
// }
// if !bytes.Equal(root, expRoot) {
// return nil, fmt.Errorf("HandoverProof mismatch")
// }
// s.currentRoot = root
// } else {
// expHashes := make([]byte, len(hashes))
// _, err := s.sessionReader.ReadAt(expHashes, int64(s.end*HashSize))
// if err != nil && err != io.EOF {
// return nil, err
// }
// if !bytes.Equal(expHashes, hashes) {
// return nil, errors.New("invalid proof")
// }
// }
// return nil, nil
// }
s.end += uint64(len(hashes)) / HashSize
takeover := &Takeover{
Stream: stream,
Start: s.start,
End: s.end,
Root: root,
}
// serialise and sign
return &TakeoverProof{
Takeover: takeover,
Sig: nil,
}, nil
}
func (s *SwarmSyncerClient) Close() {}
// base for parsing and formating sync bin key
// it must be 2 <= base <= 36
const syncBinKeyBase = 36
// FormatSyncBinKey returns a string representation of
// Kademlia bin number to be used as key for SYNC stream.
func FormatSyncBinKey(bin uint8) string {
return strconv.FormatUint(uint64(bin), syncBinKeyBase)
}
// ParseSyncBinKey parses the string representation
// and returns the Kademlia bin number.
func ParseSyncBinKey(s string) (uint8, error) {
bin, err := strconv.ParseUint(s, syncBinKeyBase, 8)
if err != nil {
return 0, err
}
return uint8(bin), nil
}

View File

@@ -0,0 +1,264 @@
// Copyright 2018 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package stream
import (
"context"
crand "crypto/rand"
"fmt"
"io"
"io/ioutil"
"math"
"sync"
"testing"
"time"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/p2p/discover"
"github.com/ethereum/go-ethereum/p2p/simulations"
"github.com/ethereum/go-ethereum/rpc"
"github.com/ethereum/go-ethereum/swarm/log"
"github.com/ethereum/go-ethereum/swarm/network"
streamTesting "github.com/ethereum/go-ethereum/swarm/network/stream/testing"
"github.com/ethereum/go-ethereum/swarm/storage"
)
const dataChunkCount = 200
func TestSyncerSimulation(t *testing.T) {
testSyncBetweenNodes(t, 2, 1, dataChunkCount, true, 1)
testSyncBetweenNodes(t, 4, 1, dataChunkCount, true, 1)
testSyncBetweenNodes(t, 8, 1, dataChunkCount, true, 1)
testSyncBetweenNodes(t, 16, 1, dataChunkCount, true, 1)
}
func createMockStore(id discover.NodeID, addr *network.BzzAddr) (storage.ChunkStore, error) {
var err error
address := common.BytesToAddress(id.Bytes())
mockStore := globalStore.NewNodeStore(address)
params := storage.NewDefaultLocalStoreParams()
datadirs[id], err = ioutil.TempDir("", "localMockStore-"+id.TerminalString())
if err != nil {
return nil, err
}
params.Init(datadirs[id])
params.BaseKey = addr.Over()
lstore, err := storage.NewLocalStore(params, mockStore)
return lstore, nil
}
func testSyncBetweenNodes(t *testing.T, nodes, conns, chunkCount int, skipCheck bool, po uint8) {
defer setDefaultSkipCheck(defaultSkipCheck)
defaultSkipCheck = skipCheck
//data directories for each node and store
datadirs = make(map[discover.NodeID]string)
if *useMockStore {
createStoreFunc = createMockStore
createGlobalStore()
} else {
createStoreFunc = createTestLocalStorageFromSim
}
defer datadirsCleanup()
registries = make(map[discover.NodeID]*TestRegistry)
toAddr = func(id discover.NodeID) *network.BzzAddr {
addr := network.NewAddrFromNodeID(id)
//hack to put addresses in same space
addr.OAddr[0] = byte(0)
return addr
}
conf := &streamTesting.RunConfig{
Adapter: *adapter,
NodeCount: nodes,
ConnLevel: conns,
ToAddr: toAddr,
Services: services,
EnableMsgEvents: false,
}
// HACK: these are global variables in the test so that they are available for
// the service constructor function
// TODO: will this work with exec/docker adapter?
// localstore of nodes made available for action and check calls
stores = make(map[discover.NodeID]storage.ChunkStore)
deliveries = make(map[discover.NodeID]*Delivery)
// create context for simulation run
timeout := 30 * time.Second
ctx, cancel := context.WithTimeout(context.Background(), timeout)
// defer cancel should come before defer simulation teardown
defer cancel()
// create simulation network with the config
sim, teardown, err := streamTesting.NewSimulation(conf)
var rpcSubscriptionsWg sync.WaitGroup
defer func() {
rpcSubscriptionsWg.Wait()
teardown()
}()
if err != nil {
t.Fatal(err.Error())
}
nodeIndex := make(map[discover.NodeID]int)
for i, id := range sim.IDs {
nodeIndex[id] = i
if !*useMockStore {
stores[id] = sim.Stores[i]
sim.Stores[i] = stores[id]
}
}
// peerCount function gives the number of peer connections for a nodeID
// this is needed for the service run function to wait until
// each protocol instance runs and the streamer peers are available
peerCount = func(id discover.NodeID) int {
if sim.IDs[0] == id || sim.IDs[nodes-1] == id {
return 1
}
return 2
}
waitPeerErrC = make(chan error)
// create DBAPI-s for all nodes
dbs := make([]*storage.DBAPI, nodes)
for i := 0; i < nodes; i++ {
dbs[i] = storage.NewDBAPI(sim.Stores[i].(*storage.LocalStore))
}
// collect hashes in po 1 bin for each node
hashes := make([][]storage.Address, nodes)
totalHashes := 0
hashCounts := make([]int, nodes)
for i := nodes - 1; i >= 0; i-- {
if i < nodes-1 {
hashCounts[i] = hashCounts[i+1]
}
dbs[i].Iterator(0, math.MaxUint64, po, func(addr storage.Address, index uint64) bool {
hashes[i] = append(hashes[i], addr)
totalHashes++
hashCounts[i]++
return true
})
}
// errc is error channel for simulation
errc := make(chan error, 1)
quitC := make(chan struct{})
defer close(quitC)
// action is subscribe
action := func(ctx context.Context) error {
// need to wait till an aynchronous process registers the peers in streamer.peers
// that is used by Subscribe
// the global peerCount function tells how many connections each node has
// TODO: this is to be reimplemented with peerEvent watcher without global var
i := 0
for err := range waitPeerErrC {
if err != nil {
return fmt.Errorf("error waiting for peers: %s", err)
}
i++
if i == nodes {
break
}
}
// each node Subscribes to each other's swarmChunkServerStreamName
for j := 0; j < nodes-1; j++ {
id := sim.IDs[j]
sim.Stores[j] = stores[id]
err := sim.CallClient(id, func(client *rpc.Client) error {
// report disconnect events to the error channel cos peers should not disconnect
doneC, err := streamTesting.WatchDisconnections(id, client, errc, quitC)
if err != nil {
return err
}
rpcSubscriptionsWg.Add(1)
go func() {
<-doneC
rpcSubscriptionsWg.Done()
}()
ctx, cancel := context.WithTimeout(ctx, 1*time.Second)
defer cancel()
// start syncing, i.e., subscribe to upstream peers po 1 bin
sid := sim.IDs[j+1]
return client.CallContext(ctx, nil, "stream_subscribeStream", sid, NewStream("SYNC", FormatSyncBinKey(1), false), NewRange(0, 0), Top)
})
if err != nil {
return err
}
}
// here we distribute chunks of a random file into stores 1...nodes
rrFileStore := storage.NewFileStore(newRoundRobinStore(sim.Stores[1:]...), storage.NewFileStoreParams())
size := chunkCount * chunkSize
_, wait, err := rrFileStore.Store(io.LimitReader(crand.Reader, int64(size)), int64(size), false)
// need to wait cos we then immediately collect the relevant bin content
wait()
if err != nil {
t.Fatal(err.Error())
}
return nil
}
// this makes sure check is not called before the previous call finishes
check := func(ctx context.Context, id discover.NodeID) (bool, error) {
select {
case err := <-errc:
return false, err
case <-ctx.Done():
return false, ctx.Err()
default:
}
i := nodeIndex[id]
var total, found int
for j := i; j < nodes; j++ {
total += len(hashes[j])
for _, key := range hashes[j] {
chunk, err := dbs[i].Get(key)
if err == storage.ErrFetching {
<-chunk.ReqC
} else if err != nil {
continue
}
// needed for leveldb not to be closed?
// chunk.WaitToStore()
found++
}
}
log.Debug("sync check", "node", id, "index", i, "bin", po, "found", found, "total", total)
return total == found, nil
}
conf.Step = &simulations.Step{
Action: action,
Trigger: streamTesting.Trigger(500*time.Millisecond, quitC, sim.IDs[0:nodes-1]...),
Expect: &simulations.Expectation{
Nodes: sim.IDs[0:1],
Check: check,
},
}
startedAt := time.Now()
result, err := sim.Run(ctx, conf)
finishedAt := time.Now()
if err != nil {
t.Fatalf("Setting up simulation failed: %v", err)
}
if result.Error != nil {
t.Fatalf("Simulation failed: %s", result.Error)
}
streamTesting.CheckResult(t, result, startedAt, finishedAt)
}

File diff suppressed because one or more lines are too long

File diff suppressed because one or more lines are too long

File diff suppressed because one or more lines are too long

File diff suppressed because one or more lines are too long

File diff suppressed because one or more lines are too long

View File

@@ -0,0 +1,293 @@
// Copyright 2018 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package testing
import (
"context"
"errors"
"fmt"
"io/ioutil"
"math/rand"
"os"
"sync"
"testing"
"time"
"github.com/ethereum/go-ethereum/p2p"
"github.com/ethereum/go-ethereum/p2p/discover"
"github.com/ethereum/go-ethereum/p2p/simulations"
"github.com/ethereum/go-ethereum/p2p/simulations/adapters"
"github.com/ethereum/go-ethereum/rpc"
"github.com/ethereum/go-ethereum/swarm/log"
"github.com/ethereum/go-ethereum/swarm/network"
"github.com/ethereum/go-ethereum/swarm/storage"
)
type Simulation struct {
Net *simulations.Network
Stores []storage.ChunkStore
Addrs []network.Addr
IDs []discover.NodeID
}
func SetStores(addrs ...network.Addr) ([]storage.ChunkStore, func(), error) {
var datadirs []string
stores := make([]storage.ChunkStore, len(addrs))
var err error
for i, addr := range addrs {
var datadir string
datadir, err = ioutil.TempDir("", "streamer")
if err != nil {
break
}
var store storage.ChunkStore
params := storage.NewDefaultLocalStoreParams()
params.Init(datadir)
params.BaseKey = addr.Over()
store, err = storage.NewTestLocalStoreForAddr(params)
if err != nil {
break
}
datadirs = append(datadirs, datadir)
stores[i] = store
}
teardown := func() {
for i, datadir := range datadirs {
stores[i].Close()
os.RemoveAll(datadir)
}
}
return stores, teardown, err
}
func NewAdapter(adapterType string, services adapters.Services) (adapter adapters.NodeAdapter, teardown func(), err error) {
teardown = func() {}
switch adapterType {
case "sim":
adapter = adapters.NewSimAdapter(services)
case "exec":
baseDir, err0 := ioutil.TempDir("", "swarm-test")
if err0 != nil {
return nil, teardown, err0
}
teardown = func() { os.RemoveAll(baseDir) }
adapter = adapters.NewExecAdapter(baseDir)
case "docker":
adapter, err = adapters.NewDockerAdapter()
if err != nil {
return nil, teardown, err
}
default:
return nil, teardown, errors.New("adapter needs to be one of sim, exec, docker")
}
return adapter, teardown, nil
}
func CheckResult(t *testing.T, result *simulations.StepResult, startedAt, finishedAt time.Time) {
t.Logf("Simulation passed in %s", result.FinishedAt.Sub(result.StartedAt))
if len(result.Passes) > 1 {
var min, max time.Duration
var sum int
for _, pass := range result.Passes {
duration := pass.Sub(result.StartedAt)
if sum == 0 || duration < min {
min = duration
}
if duration > max {
max = duration
}
sum += int(duration.Nanoseconds())
}
t.Logf("Min: %s, Max: %s, Average: %s", min, max, time.Duration(sum/len(result.Passes))*time.Nanosecond)
}
t.Logf("Setup: %s, Shutdown: %s", result.StartedAt.Sub(startedAt), finishedAt.Sub(result.FinishedAt))
}
type RunConfig struct {
Adapter string
Step *simulations.Step
NodeCount int
ConnLevel int
ToAddr func(discover.NodeID) *network.BzzAddr
Services adapters.Services
DefaultService string
EnableMsgEvents bool
}
func NewSimulation(conf *RunConfig) (*Simulation, func(), error) {
// create network
nodes := conf.NodeCount
adapter, adapterTeardown, err := NewAdapter(conf.Adapter, conf.Services)
if err != nil {
return nil, adapterTeardown, err
}
defaultService := "streamer"
if conf.DefaultService != "" {
defaultService = conf.DefaultService
}
net := simulations.NewNetwork(adapter, &simulations.NetworkConfig{
ID: "0",
DefaultService: defaultService,
})
teardown := func() {
adapterTeardown()
net.Shutdown()
}
ids := make([]discover.NodeID, nodes)
addrs := make([]network.Addr, nodes)
// start nodes
for i := 0; i < nodes; i++ {
nodeconf := adapters.RandomNodeConfig()
nodeconf.EnableMsgEvents = conf.EnableMsgEvents
node, err := net.NewNodeWithConfig(nodeconf)
if err != nil {
return nil, teardown, fmt.Errorf("error creating node: %s", err)
}
ids[i] = node.ID()
addrs[i] = conf.ToAddr(ids[i])
}
// set nodes number of Stores available
stores, storeTeardown, err := SetStores(addrs...)
teardown = func() {
net.Shutdown()
adapterTeardown()
storeTeardown()
}
if err != nil {
return nil, teardown, err
}
s := &Simulation{
Net: net,
Stores: stores,
IDs: ids,
Addrs: addrs,
}
return s, teardown, nil
}
func (s *Simulation) Run(ctx context.Context, conf *RunConfig) (*simulations.StepResult, error) {
// bring up nodes, launch the servive
nodes := conf.NodeCount
conns := conf.ConnLevel
for i := 0; i < nodes; i++ {
if err := s.Net.Start(s.IDs[i]); err != nil {
return nil, fmt.Errorf("error starting node %s: %s", s.IDs[i].TerminalString(), err)
}
}
// run a simulation which connects the 10 nodes in a chain
wg := sync.WaitGroup{}
for i := range s.IDs {
// collect the overlay addresses, to
for j := 0; j < conns; j++ {
var k int
if j == 0 {
k = i - 1
} else {
k = rand.Intn(len(s.IDs))
}
if i > 0 {
wg.Add(1)
go func(i, k int) {
defer wg.Done()
s.Net.Connect(s.IDs[i], s.IDs[k])
}(i, k)
}
}
}
wg.Wait()
log.Info(fmt.Sprintf("simulation with %v nodes", len(s.Addrs)))
// create an only locally retrieving FileStore for the pivot node to test
// if retriee requests have arrived
result := simulations.NewSimulation(s.Net).Run(ctx, conf.Step)
return result, nil
}
// WatchDisconnections subscribes to admin peerEvents and sends peer event drop
// errors to the errc channel. Channel quitC signals the termination of the event loop.
// Returned doneC will be closed after the rpc subscription is unsubscribed,
// signaling that simulations network is safe to shutdown.
func WatchDisconnections(id discover.NodeID, client *rpc.Client, errc chan error, quitC chan struct{}) (doneC <-chan struct{}, err error) {
events := make(chan *p2p.PeerEvent)
sub, err := client.Subscribe(context.Background(), "admin", events, "peerEvents")
if err != nil {
return nil, fmt.Errorf("error getting peer events for node %v: %s", id, err)
}
c := make(chan struct{})
go func() {
defer func() {
log.Trace("watch disconnections: unsubscribe", "id", id)
sub.Unsubscribe()
close(c)
}()
for {
select {
case <-quitC:
return
case e := <-events:
if e.Type == p2p.PeerEventTypeDrop {
select {
case errc <- fmt.Errorf("peerEvent for node %v: %v", id, e):
case <-quitC:
return
}
}
case err := <-sub.Err():
if err != nil {
select {
case errc <- fmt.Errorf("error getting peer events for node %v: %v", id, err):
case <-quitC:
return
}
}
}
}
}()
return c, nil
}
func Trigger(d time.Duration, quitC chan struct{}, ids ...discover.NodeID) chan discover.NodeID {
trigger := make(chan discover.NodeID)
go func() {
defer close(trigger)
ticker := time.NewTicker(d)
defer ticker.Stop()
// we are only testing the pivot node (net.Nodes[0])
for range ticker.C {
for _, id := range ids {
select {
case trigger <- id:
case <-quitC:
return
}
}
}
}()
return trigger
}
func (sim *Simulation) CallClient(id discover.NodeID, f func(*rpc.Client) error) error {
node := sim.Net.GetNode(id)
if node == nil {
return fmt.Errorf("unknown node: %s", id)
}
client, err := node.Client()
if err != nil {
return fmt.Errorf("error getting node client: %s", err)
}
return f(client)
}

View File

@@ -1,389 +0,0 @@
// Copyright 2016 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package network
import (
"encoding/binary"
"fmt"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/swarm/storage"
"github.com/syndtr/goleveldb/leveldb"
"github.com/syndtr/goleveldb/leveldb/iterator"
)
const counterKeyPrefix = 0x01
/*
syncDb is a queueing service for outgoing deliveries.
One instance per priority queue for each peer
a syncDb instance maintains an in-memory buffer (of capacity bufferSize)
once its in-memory buffer is full it switches to persisting in db
and dbRead iterator iterates through the items keeping their order
once the db read catches up (there is no more items in the db) then
it switches back to in-memory buffer.
when syncdb is stopped all items in the buffer are saved to the db
*/
type syncDb struct {
start []byte // this syncdb starting index in requestdb
key storage.Key // remote peers address key
counterKey []byte // db key to persist counter
priority uint // priotity High|Medium|Low
buffer chan interface{} // incoming request channel
db *storage.LDBDatabase // underlying db (TODO should be interface)
done chan bool // chan to signal goroutines finished quitting
quit chan bool // chan to signal quitting to goroutines
total, dbTotal int // counts for one session
batch chan chan int // channel for batch requests
dbBatchSize uint // number of items before batch is saved
}
// constructor needs a shared request db (leveldb)
// priority is used in the index key
// uses a buffer and a leveldb for persistent storage
// bufferSize, dbBatchSize are config parameters
func newSyncDb(db *storage.LDBDatabase, key storage.Key, priority uint, bufferSize, dbBatchSize uint, deliver func(interface{}, chan bool) bool) *syncDb {
start := make([]byte, 42)
start[1] = byte(priorities - priority)
copy(start[2:34], key)
counterKey := make([]byte, 34)
counterKey[0] = counterKeyPrefix
copy(counterKey[1:], start[1:34])
syncdb := &syncDb{
start: start,
key: key,
counterKey: counterKey,
priority: priority,
buffer: make(chan interface{}, bufferSize),
db: db,
done: make(chan bool),
quit: make(chan bool),
batch: make(chan chan int),
dbBatchSize: dbBatchSize,
}
log.Trace(fmt.Sprintf("syncDb[peer: %v, priority: %v] - initialised", key.Log(), priority))
// starts the main forever loop reading from buffer
go syncdb.bufferRead(deliver)
return syncdb
}
/*
bufferRead is a forever iterator loop that takes care of delivering
outgoing store requests reads from incoming buffer
its argument is the deliver function taking the item as first argument
and a quit channel as second.
Closing of this channel is supposed to abort all waiting for delivery
(typically network write)
The iteration switches between 2 modes,
* buffer mode reads the in-memory buffer and delivers the items directly
* db mode reads from the buffer and writes to the db, parallelly another
routine is started that reads from the db and delivers items
If there is buffer contention in buffer mode (slow network, high upload volume)
syncdb switches to db mode and starts dbRead
Once db backlog is delivered, it reverts back to in-memory buffer
It is automatically started when syncdb is initialised.
It saves the buffer to db upon receiving quit signal. syncDb#stop()
*/
func (self *syncDb) bufferRead(deliver func(interface{}, chan bool) bool) {
var buffer, db chan interface{} // channels representing the two read modes
var more bool
var req interface{}
var entry *syncDbEntry
var inBatch, inDb int
batch := new(leveldb.Batch)
var dbSize chan int
quit := self.quit
counterValue := make([]byte, 8)
// counter is used for keeping the items in order, persisted to db
// start counter where db was at, 0 if not found
data, err := self.db.Get(self.counterKey)
var counter uint64
if err == nil {
counter = binary.BigEndian.Uint64(data)
log.Trace(fmt.Sprintf("syncDb[%v/%v] - counter read from db at %v", self.key.Log(), self.priority, counter))
} else {
log.Trace(fmt.Sprintf("syncDb[%v/%v] - counter starts at %v", self.key.Log(), self.priority, counter))
}
LOOP:
for {
// waiting for item next in the buffer, or quit signal or batch request
select {
// buffer only closes when writing to db
case req = <-buffer:
// deliver request : this is blocking on network write so
// it is passed the quit channel as argument, so that it returns
// if syncdb is stopped. In this case we need to save the item to the db
more = deliver(req, self.quit)
if !more {
log.Debug(fmt.Sprintf("syncDb[%v/%v] quit: switching to db. session tally (db/total): %v/%v", self.key.Log(), self.priority, self.dbTotal, self.total))
// received quit signal, save request currently waiting delivery
// by switching to db mode and closing the buffer
buffer = nil
db = self.buffer
close(db)
quit = nil // needs to block the quit case in select
break // break from select, this item will be written to the db
}
self.total++
log.Trace(fmt.Sprintf("syncDb[%v/%v] deliver (db/total): %v/%v", self.key.Log(), self.priority, self.dbTotal, self.total))
// by the time deliver returns, there were new writes to the buffer
// if buffer contention is detected, switch to db mode which drains
// the buffer so no process will block on pushing store requests
if len(buffer) == cap(buffer) {
log.Debug(fmt.Sprintf("syncDb[%v/%v] buffer full %v: switching to db. session tally (db/total): %v/%v", self.key.Log(), self.priority, cap(buffer), self.dbTotal, self.total))
buffer = nil
db = self.buffer
}
continue LOOP
// incoming entry to put into db
case req, more = <-db:
if !more {
// only if quit is called, saved all the buffer
binary.BigEndian.PutUint64(counterValue, counter)
batch.Put(self.counterKey, counterValue) // persist counter in batch
self.writeSyncBatch(batch) // save batch
log.Trace(fmt.Sprintf("syncDb[%v/%v] quitting: save current batch to db", self.key.Log(), self.priority))
break LOOP
}
self.dbTotal++
self.total++
// otherwise break after select
case dbSize = <-self.batch:
// explicit request for batch
if inBatch == 0 && quit != nil {
// there was no writes since the last batch so db depleted
// switch to buffer mode
log.Debug(fmt.Sprintf("syncDb[%v/%v] empty db: switching to buffer", self.key.Log(), self.priority))
db = nil
buffer = self.buffer
dbSize <- 0 // indicates to 'caller' that batch has been written
inDb = 0
continue LOOP
}
binary.BigEndian.PutUint64(counterValue, counter)
batch.Put(self.counterKey, counterValue)
log.Debug(fmt.Sprintf("syncDb[%v/%v] write batch %v/%v - %x - %x", self.key.Log(), self.priority, inBatch, counter, self.counterKey, counterValue))
batch = self.writeSyncBatch(batch)
dbSize <- inBatch // indicates to 'caller' that batch has been written
inBatch = 0
continue LOOP
// closing syncDb#quit channel is used to signal to all goroutines to quit
case <-quit:
// need to save backlog, so switch to db mode
db = self.buffer
buffer = nil
quit = nil
log.Trace(fmt.Sprintf("syncDb[%v/%v] quitting: save buffer to db", self.key.Log(), self.priority))
close(db)
continue LOOP
}
// only get here if we put req into db
entry, err = self.newSyncDbEntry(req, counter)
if err != nil {
log.Warn(fmt.Sprintf("syncDb[%v/%v] saving request %v (#%v/%v) failed: %v", self.key.Log(), self.priority, req, inBatch, inDb, err))
continue LOOP
}
batch.Put(entry.key, entry.val)
log.Trace(fmt.Sprintf("syncDb[%v/%v] to batch %v '%v' (#%v/%v/%v)", self.key.Log(), self.priority, req, entry, inBatch, inDb, counter))
// if just switched to db mode and not quitting, then launch dbRead
// in a parallel go routine to send deliveries from db
if inDb == 0 && quit != nil {
log.Trace(fmt.Sprintf("syncDb[%v/%v] start dbRead", self.key.Log(), self.priority))
go self.dbRead(true, counter, deliver)
}
inDb++
inBatch++
counter++
// need to save the batch if it gets too large (== dbBatchSize)
if inBatch%int(self.dbBatchSize) == 0 {
batch = self.writeSyncBatch(batch)
}
}
log.Info(fmt.Sprintf("syncDb[%v:%v]: saved %v keys (saved counter at %v)", self.key.Log(), self.priority, inBatch, counter))
close(self.done)
}
// writes the batch to the db and returns a new batch object
func (self *syncDb) writeSyncBatch(batch *leveldb.Batch) *leveldb.Batch {
err := self.db.Write(batch)
if err != nil {
log.Warn(fmt.Sprintf("syncDb[%v/%v] saving batch to db failed: %v", self.key.Log(), self.priority, err))
return batch
}
return new(leveldb.Batch)
}
// abstract type for db entries (TODO could be a feature of Receipts)
type syncDbEntry struct {
key, val []byte
}
func (self syncDbEntry) String() string {
return fmt.Sprintf("key: %x, value: %x", self.key, self.val)
}
/*
dbRead is iterating over store requests to be sent over to the peer
this is mainly to prevent crashes due to network output buffer contention (???)
as well as to make syncronisation resilient to disconnects
the messages are supposed to be sent in the p2p priority queue.
the request DB is shared between peers, but domains for each syncdb
are disjoint. dbkeys (42 bytes) are structured:
* 0: 0x00 (0x01 reserved for counter key)
* 1: priorities - priority (so that high priority can be replayed first)
* 2-33: peers address
* 34-41: syncdb counter to preserve order (this field is missing for the counter key)
values (40 bytes) are:
* 0-31: key
* 32-39: request id
dbRead needs a boolean to indicate if on first round all the historical
record is synced. Second argument to indicate current db counter
The third is the function to apply
*/
func (self *syncDb) dbRead(useBatches bool, counter uint64, fun func(interface{}, chan bool) bool) {
key := make([]byte, 42)
copy(key, self.start)
binary.BigEndian.PutUint64(key[34:], counter)
var batches, n, cnt, total int
var more bool
var entry *syncDbEntry
var it iterator.Iterator
var del *leveldb.Batch
batchSizes := make(chan int)
for {
// if useBatches is false, cnt is not set
if useBatches {
// this could be called before all cnt items sent out
// so that loop is not blocking while delivering
// only relevant if cnt is large
select {
case self.batch <- batchSizes:
case <-self.quit:
return
}
// wait for the write to finish and get the item count in the next batch
cnt = <-batchSizes
batches++
if cnt == 0 {
// empty
return
}
}
it = self.db.NewIterator()
it.Seek(key)
if !it.Valid() {
copy(key, self.start)
useBatches = true
continue
}
del = new(leveldb.Batch)
log.Trace(fmt.Sprintf("syncDb[%v/%v]: new iterator: %x (batch %v, count %v)", self.key.Log(), self.priority, key, batches, cnt))
for n = 0; !useBatches || n < cnt; it.Next() {
copy(key, it.Key())
if len(key) == 0 || key[0] != 0 {
copy(key, self.start)
useBatches = true
break
}
val := make([]byte, 40)
copy(val, it.Value())
entry = &syncDbEntry{key, val}
// log.Trace(fmt.Sprintf("syncDb[%v/%v] - %v, batches: %v, total: %v, session total from db: %v/%v", self.key.Log(), self.priority, self.key.Log(), batches, total, self.dbTotal, self.total))
more = fun(entry, self.quit)
if !more {
// quit received when waiting to deliver entry, the entry will not be deleted
log.Trace(fmt.Sprintf("syncDb[%v/%v] batch %v quit after %v/%v items", self.key.Log(), self.priority, batches, n, cnt))
break
}
// since subsequent batches of the same db session are indexed incrementally
// deleting earlier batches can be delayed and parallelised
// this could be batch delete when db is idle (but added complexity esp when quitting)
del.Delete(key)
n++
total++
}
log.Debug(fmt.Sprintf("syncDb[%v/%v] - db session closed, batches: %v, total: %v, session total from db: %v/%v", self.key.Log(), self.priority, batches, total, self.dbTotal, self.total))
self.db.Write(del) // this could be async called only when db is idle
it.Release()
}
}
//
func (self *syncDb) stop() {
close(self.quit)
<-self.done
}
// calculate a dbkey for the request, for the db to work
// see syncdb for db key structure
// polimorphic: accepted types, see syncer#addRequest
func (self *syncDb) newSyncDbEntry(req interface{}, counter uint64) (entry *syncDbEntry, err error) {
var key storage.Key
var chunk *storage.Chunk
var id uint64
var ok bool
var sreq *storeRequestMsgData
if key, ok = req.(storage.Key); ok {
id = generateId()
} else if chunk, ok = req.(*storage.Chunk); ok {
key = chunk.Key
id = generateId()
} else if sreq, ok = req.(*storeRequestMsgData); ok {
key = sreq.Key
id = sreq.Id
} else if entry, ok = req.(*syncDbEntry); !ok {
return nil, fmt.Errorf("type not allowed: %v (%T)", req, req)
}
// order by peer > priority > seqid
// value is request id if exists
if entry == nil {
dbkey := make([]byte, 42)
dbval := make([]byte, 40)
// encode key
copy(dbkey[:], self.start[:34]) // db peer
binary.BigEndian.PutUint64(dbkey[34:], counter)
// encode value
copy(dbval, key[:])
binary.BigEndian.PutUint64(dbval[32:], id)
entry = &syncDbEntry{dbkey, dbval}
}
return
}

View File

@@ -1,222 +0,0 @@
// Copyright 2016 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package network
import (
"bytes"
"fmt"
"io/ioutil"
"os"
"path/filepath"
"testing"
"time"
"github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/swarm/storage"
)
func init() {
log.Root().SetHandler(log.LvlFilterHandler(log.LvlCrit, log.StreamHandler(os.Stderr, log.TerminalFormat(false))))
}
type testSyncDb struct {
*syncDb
c int
t *testing.T
fromDb chan bool
delivered [][]byte
sent []int
dbdir string
at int
}
func newTestSyncDb(priority, bufferSize, batchSize int, dbdir string, t *testing.T) *testSyncDb {
if len(dbdir) == 0 {
tmp, err := ioutil.TempDir(os.TempDir(), "syncdb-test")
if err != nil {
t.Fatalf("unable to create temporary direcory %v: %v", tmp, err)
}
dbdir = tmp
}
db, err := storage.NewLDBDatabase(filepath.Join(dbdir, "requestdb"))
if err != nil {
t.Fatalf("unable to create db: %v", err)
}
self := &testSyncDb{
fromDb: make(chan bool),
dbdir: dbdir,
t: t,
}
h := crypto.Keccak256Hash([]byte{0})
key := storage.Key(h[:])
self.syncDb = newSyncDb(db, key, uint(priority), uint(bufferSize), uint(batchSize), self.deliver)
// kick off db iterator right away, if no items on db this will allow
// reading from the buffer
return self
}
func (self *testSyncDb) close() {
self.db.Close()
os.RemoveAll(self.dbdir)
}
func (self *testSyncDb) push(n int) {
for i := 0; i < n; i++ {
self.buffer <- storage.Key(crypto.Keccak256([]byte{byte(self.c)}))
self.sent = append(self.sent, self.c)
self.c++
}
log.Debug(fmt.Sprintf("pushed %v requests", n))
}
func (self *testSyncDb) draindb() {
it := self.db.NewIterator()
defer it.Release()
for {
it.Seek(self.start)
if !it.Valid() {
return
}
k := it.Key()
if len(k) == 0 || k[0] == 1 {
return
}
it.Release()
it = self.db.NewIterator()
}
}
func (self *testSyncDb) deliver(req interface{}, quit chan bool) bool {
_, db := req.(*syncDbEntry)
key, _, _, _, err := parseRequest(req)
if err != nil {
self.t.Fatalf("unexpected error of key %v: %v", key, err)
}
self.delivered = append(self.delivered, key)
select {
case self.fromDb <- db:
return true
case <-quit:
return false
}
}
func (self *testSyncDb) expect(n int, db bool) {
var ok bool
// for n items
for i := 0; i < n; i++ {
ok = <-self.fromDb
if self.at+1 > len(self.delivered) {
self.t.Fatalf("expected %v, got %v", self.at+1, len(self.delivered))
}
if len(self.sent) > self.at && !bytes.Equal(crypto.Keccak256([]byte{byte(self.sent[self.at])}), self.delivered[self.at]) {
self.t.Fatalf("expected delivery %v/%v/%v to be hash of %v, from db: %v = %v", i, n, self.at, self.sent[self.at], ok, db)
log.Debug(fmt.Sprintf("%v/%v/%v to be hash of %v, from db: %v = %v", i, n, self.at, self.sent[self.at], ok, db))
}
if !ok && db {
self.t.Fatalf("expected delivery %v/%v/%v from db", i, n, self.at)
}
if ok && !db {
self.t.Fatalf("expected delivery %v/%v/%v from cache", i, n, self.at)
}
self.at++
}
}
func TestSyncDb(t *testing.T) {
t.Skip("fails randomly on all platforms")
priority := High
bufferSize := 5
batchSize := 2 * bufferSize
s := newTestSyncDb(priority, bufferSize, batchSize, "", t)
defer s.close()
defer s.stop()
s.dbRead(false, 0, s.deliver)
s.draindb()
s.push(4)
s.expect(1, false)
// 3 in buffer
time.Sleep(100 * time.Millisecond)
s.push(3)
// push over limit
s.expect(1, false)
// one popped from the buffer, then contention detected
s.expect(4, true)
s.push(4)
s.expect(5, true)
// depleted db, switch back to buffer
s.draindb()
s.push(5)
s.expect(4, false)
s.push(3)
s.expect(4, false)
// buffer depleted
time.Sleep(100 * time.Millisecond)
s.push(6)
s.expect(1, false)
// push into buffer full, switch to db
s.expect(5, true)
s.draindb()
s.push(1)
s.expect(1, false)
}
func TestSaveSyncDb(t *testing.T) {
amount := 30
priority := High
bufferSize := amount
batchSize := 10
s := newTestSyncDb(priority, bufferSize, batchSize, "", t)
go s.dbRead(false, 0, s.deliver)
s.push(amount)
s.stop()
s.db.Close()
s = newTestSyncDb(priority, bufferSize, batchSize, s.dbdir, t)
go s.dbRead(false, 0, s.deliver)
s.expect(amount, true)
for i, key := range s.delivered {
expKey := crypto.Keccak256([]byte{byte(i)})
if !bytes.Equal(key, expKey) {
t.Fatalf("delivery %v expected to be key %x, got %x", i, expKey, key)
}
}
s.push(amount)
s.expect(amount, false)
for i := amount; i < 2*amount; i++ {
key := s.delivered[i]
expKey := crypto.Keccak256([]byte{byte(i - amount)})
if !bytes.Equal(key, expKey) {
t.Fatalf("delivery %v expected to be key %x, got %x", i, expKey, key)
}
}
s.stop()
s.db.Close()
s = newTestSyncDb(priority, bufferSize, batchSize, s.dbdir, t)
defer s.close()
defer s.stop()
go s.dbRead(false, 0, s.deliver)
s.push(1)
s.expect(1, false)
}

View File

@@ -1,781 +0,0 @@
// Copyright 2016 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package network
import (
"encoding/binary"
"encoding/json"
"fmt"
"path/filepath"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/swarm/storage"
)
// syncer parameters (global, not peer specific) default values
const (
requestDbBatchSize = 512 // size of batch before written to request db
keyBufferSize = 1024 // size of buffer for unsynced keys
syncBatchSize = 128 // maximum batchsize for outgoing requests
syncBufferSize = 128 // size of buffer for delivery requests
syncCacheSize = 1024 // cache capacity to store request queue in memory
)
// priorities
const (
Low = iota // 0
Medium // 1
High // 2
priorities // 3 number of priority levels
)
// request types
const (
DeliverReq = iota // 0
PushReq // 1
PropagateReq // 2
HistoryReq // 3
BacklogReq // 4
)
// json serialisable struct to record the syncronisation state between 2 peers
type syncState struct {
*storage.DbSyncState // embeds the following 4 fields:
// Start Key // lower limit of address space
// Stop Key // upper limit of address space
// First uint64 // counter taken from last sync state
// Last uint64 // counter of remote peer dbStore at the time of last connection
SessionAt uint64 // set at the time of connection
LastSeenAt uint64 // set at the time of connection
Latest storage.Key // cursor of dbstore when last (continuously set by syncer)
Synced bool // true iff Sync is done up to the last disconnect
synced chan bool // signal that sync stage finished
}
// wrapper of db-s to provide mockable custom local chunk store access to syncer
type DbAccess struct {
db *storage.DbStore
loc *storage.LocalStore
}
func NewDbAccess(loc *storage.LocalStore) *DbAccess {
return &DbAccess{loc.DbStore.(*storage.DbStore), loc}
}
// to obtain the chunks from key or request db entry only
func (self *DbAccess) get(key storage.Key) (*storage.Chunk, error) {
return self.loc.Get(key)
}
// current storage counter of chunk db
func (self *DbAccess) counter() uint64 {
return self.db.Counter()
}
// implemented by dbStoreSyncIterator
type keyIterator interface {
Next() storage.Key
}
// generator function for iteration by address range and storage counter
func (self *DbAccess) iterator(s *syncState) keyIterator {
it, err := self.db.NewSyncIterator(*(s.DbSyncState))
if err != nil {
return nil
}
return keyIterator(it)
}
func (self syncState) String() string {
if self.Synced {
return fmt.Sprintf(
"session started at: %v, last seen at: %v, latest key: %v",
self.SessionAt, self.LastSeenAt,
self.Latest.Log(),
)
} else {
return fmt.Sprintf(
"address: %v-%v, index: %v-%v, session started at: %v, last seen at: %v, latest key: %v",
self.Start.Log(), self.Stop.Log(),
self.First, self.Last,
self.SessionAt, self.LastSeenAt,
self.Latest.Log(),
)
}
}
// syncer parameters (global, not peer specific)
type SyncParams struct {
RequestDbPath string // path for request db (leveldb)
RequestDbBatchSize uint // nuber of items before batch is saved to requestdb
KeyBufferSize uint // size of key buffer
SyncBatchSize uint // maximum batchsize for outgoing requests
SyncBufferSize uint // size of buffer for
SyncCacheSize uint // cache capacity to store request queue in memory
SyncPriorities []uint // list of priority levels for req types 0-3
SyncModes []bool // list of sync modes for for req types 0-3
}
// constructor with default values
func NewDefaultSyncParams() *SyncParams {
return &SyncParams{
RequestDbBatchSize: requestDbBatchSize,
KeyBufferSize: keyBufferSize,
SyncBufferSize: syncBufferSize,
SyncBatchSize: syncBatchSize,
SyncCacheSize: syncCacheSize,
SyncPriorities: []uint{High, Medium, Medium, Low, Low},
SyncModes: []bool{true, true, true, true, false},
}
}
//this can only finally be set after all config options (file, cmd line, env vars)
//have been evaluated
func (self *SyncParams) Init(path string) {
self.RequestDbPath = filepath.Join(path, "requests")
}
// syncer is the agent that manages content distribution/storage replication/chunk storeRequest forwarding
type syncer struct {
*SyncParams // sync parameters
syncF func() bool // if syncing is needed
key storage.Key // remote peers address key
state *syncState // sync state for our dbStore
syncStates chan *syncState // different stages of sync
deliveryRequest chan bool // one of two triggers needed to send unsyncedKeys
newUnsyncedKeys chan bool // one of two triggers needed to send unsynced keys
quit chan bool // signal to quit loops
// DB related fields
dbAccess *DbAccess // access to dbStore
// native fields
queues [priorities]*syncDb // in-memory cache / queues for sync reqs
keys [priorities]chan interface{} // buffer for unsynced keys
deliveries [priorities]chan *storeRequestMsgData // delivery
// bzz protocol instance outgoing message callbacks (mockable for testing)
unsyncedKeys func([]*syncRequest, *syncState) error // send unsyncedKeysMsg
store func(*storeRequestMsgData) error // send storeRequestMsg
}
// a syncer instance is linked to each peer connection
// constructor is called from protocol after successful handshake
// the returned instance is attached to the peer and can be called
// by the forwarder
func newSyncer(
db *storage.LDBDatabase, remotekey storage.Key,
dbAccess *DbAccess,
unsyncedKeys func([]*syncRequest, *syncState) error,
store func(*storeRequestMsgData) error,
params *SyncParams,
state *syncState,
syncF func() bool,
) (*syncer, error) {
syncBufferSize := params.SyncBufferSize
keyBufferSize := params.KeyBufferSize
dbBatchSize := params.RequestDbBatchSize
self := &syncer{
syncF: syncF,
key: remotekey,
dbAccess: dbAccess,
syncStates: make(chan *syncState, 20),
deliveryRequest: make(chan bool, 1),
newUnsyncedKeys: make(chan bool, 1),
SyncParams: params,
state: state,
quit: make(chan bool),
unsyncedKeys: unsyncedKeys,
store: store,
}
// initialising
for i := 0; i < priorities; i++ {
self.keys[i] = make(chan interface{}, keyBufferSize)
self.deliveries[i] = make(chan *storeRequestMsgData)
// initialise a syncdb instance for each priority queue
self.queues[i] = newSyncDb(db, remotekey, uint(i), syncBufferSize, dbBatchSize, self.deliver(uint(i)))
}
log.Info(fmt.Sprintf("syncer started: %v", state))
// launch chunk delivery service
go self.syncDeliveries()
// launch sync task manager
if self.syncF() {
go self.sync()
}
// process unsynced keys to broadcast
go self.syncUnsyncedKeys()
return self, nil
}
// metadata serialisation
func encodeSync(state *syncState) (*json.RawMessage, error) {
data, err := json.MarshalIndent(state, "", " ")
if err != nil {
return nil, err
}
meta := json.RawMessage(data)
return &meta, nil
}
func decodeSync(meta *json.RawMessage) (*syncState, error) {
if meta == nil {
return nil, fmt.Errorf("unable to deserialise sync state from <nil>")
}
data := []byte(*(meta))
if len(data) == 0 {
return nil, fmt.Errorf("unable to deserialise sync state from <nil>")
}
state := &syncState{DbSyncState: &storage.DbSyncState{}}
err := json.Unmarshal(data, state)
return state, err
}
/*
sync implements the syncing script
* first all items left in the request Db are replayed
* type = StaleSync
* Mode: by default once again via confirmation roundtrip
* Priority: the items are replayed as the proirity specified for StaleSync
* but within the order respects earlier priority level of request
* after all items are consumed for a priority level, the the respective
queue for delivery requests is open (this way new reqs not written to db)
(TODO: this should be checked)
* the sync state provided by the remote peer is used to sync history
* all the backlog from earlier (aborted) syncing is completed starting from latest
* if Last < LastSeenAt then all items in between then process all
backlog from upto last disconnect
* if Last > 0 &&
sync is called from the syncer constructor and is not supposed to be used externally
*/
func (self *syncer) sync() {
state := self.state
// sync finished
defer close(self.syncStates)
// 0. first replay stale requests from request db
if state.SessionAt == 0 {
log.Debug(fmt.Sprintf("syncer[%v]: nothing to sync", self.key.Log()))
return
}
log.Debug(fmt.Sprintf("syncer[%v]: start replaying stale requests from request db", self.key.Log()))
for p := priorities - 1; p >= 0; p-- {
self.queues[p].dbRead(false, 0, self.replay())
}
log.Debug(fmt.Sprintf("syncer[%v]: done replaying stale requests from request db", self.key.Log()))
// unless peer is synced sync unfinished history beginning on
if !state.Synced {
start := state.Start
if !storage.IsZeroKey(state.Latest) {
// 1. there is unfinished earlier sync
state.Start = state.Latest
log.Debug(fmt.Sprintf("syncer[%v]: start syncronising backlog (unfinished sync: %v)", self.key.Log(), state))
// blocks while the entire history upto state is synced
self.syncState(state)
if state.Last < state.SessionAt {
state.First = state.Last + 1
}
}
state.Latest = storage.ZeroKey
state.Start = start
// 2. sync up to last disconnect1
if state.First < state.LastSeenAt {
state.Last = state.LastSeenAt
log.Debug(fmt.Sprintf("syncer[%v]: start syncronising history upto last disconnect at %v: %v", self.key.Log(), state.LastSeenAt, state))
self.syncState(state)
state.First = state.LastSeenAt
}
state.Latest = storage.ZeroKey
} else {
// synchronisation starts at end of last session
state.First = state.LastSeenAt
}
// 3. sync up to current session start
// if there have been new chunks since last session
if state.LastSeenAt < state.SessionAt {
state.Last = state.SessionAt
log.Debug(fmt.Sprintf("syncer[%v]: start syncronising history since last disconnect at %v up until session start at %v: %v", self.key.Log(), state.LastSeenAt, state.SessionAt, state))
// blocks until state syncing is finished
self.syncState(state)
}
log.Info(fmt.Sprintf("syncer[%v]: syncing all history complete", self.key.Log()))
}
// wait till syncronised block uptil state is synced
func (self *syncer) syncState(state *syncState) {
self.syncStates <- state
select {
case <-state.synced:
case <-self.quit:
}
}
// stop quits both request processor and saves the request cache to disk
func (self *syncer) stop() {
close(self.quit)
log.Trace(fmt.Sprintf("syncer[%v]: stop and save sync request db backlog", self.key.Log()))
for _, db := range self.queues {
db.stop()
}
}
// rlp serialisable sync request
type syncRequest struct {
Key storage.Key
Priority uint
}
func (self *syncRequest) String() string {
return fmt.Sprintf("<Key: %v, Priority: %v>", self.Key.Log(), self.Priority)
}
func (self *syncer) newSyncRequest(req interface{}, p int) (*syncRequest, error) {
key, _, _, _, err := parseRequest(req)
// TODO: if req has chunk, it should be put in a cache
// create
if err != nil {
return nil, err
}
return &syncRequest{key, uint(p)}, nil
}
// serves historical items from the DB
// * read is on demand, blocking unless history channel is read
// * accepts sync requests (syncStates) to create new db iterator
// * closes the channel one iteration finishes
func (self *syncer) syncHistory(state *syncState) chan interface{} {
var n uint
history := make(chan interface{})
log.Debug(fmt.Sprintf("syncer[%v]: syncing history between %v - %v for chunk addresses %v - %v", self.key.Log(), state.First, state.Last, state.Start, state.Stop))
it := self.dbAccess.iterator(state)
if it != nil {
go func() {
// signal end of the iteration ended
defer close(history)
IT:
for {
key := it.Next()
if key == nil {
break IT
}
select {
// blocking until history channel is read from
case history <- key:
n++
log.Trace(fmt.Sprintf("syncer[%v]: history: %v (%v keys)", self.key.Log(), key.Log(), n))
state.Latest = key
case <-self.quit:
return
}
}
log.Debug(fmt.Sprintf("syncer[%v]: finished syncing history between %v - %v for chunk addresses %v - %v (at %v) (chunks = %v)", self.key.Log(), state.First, state.Last, state.Start, state.Stop, state.Latest, n))
}()
}
return history
}
// triggers key syncronisation
func (self *syncer) sendUnsyncedKeys() {
select {
case self.deliveryRequest <- true:
default:
}
}
// assembles a new batch of unsynced keys
// * keys are drawn from the key buffers in order of priority queue
// * if the queues of priority for History (HistoryReq) or higher are depleted,
// historical data is used so historical items are lower priority within
// their priority group.
// * Order of historical data is unspecified
func (self *syncer) syncUnsyncedKeys() {
// send out new
var unsynced []*syncRequest
var more, justSynced bool
var keyCount, historyCnt int
var history chan interface{}
priority := High
keys := self.keys[priority]
var newUnsyncedKeys, deliveryRequest chan bool
keyCounts := make([]int, priorities)
histPrior := self.SyncPriorities[HistoryReq]
syncStates := self.syncStates
state := self.state
LOOP:
for {
var req interface{}
// select the highest priority channel to read from
// keys channels are buffered so the highest priority ones
// are checked first - integrity can only be guaranteed if writing
// is locked while selecting
if priority != High || len(keys) == 0 {
// selection is not needed if the High priority queue has items
keys = nil
PRIORITIES:
for priority = High; priority >= 0; priority-- {
// the first priority channel that is non-empty will be assigned to keys
if len(self.keys[priority]) > 0 {
log.Trace(fmt.Sprintf("syncer[%v]: reading request with priority %v", self.key.Log(), priority))
keys = self.keys[priority]
break PRIORITIES
}
log.Trace(fmt.Sprintf("syncer[%v/%v]: queue: [%v, %v, %v]", self.key.Log(), priority, len(self.keys[High]), len(self.keys[Medium]), len(self.keys[Low])))
// if the input queue is empty on this level, resort to history if there is any
if uint(priority) == histPrior && history != nil {
log.Trace(fmt.Sprintf("syncer[%v]: reading history for %v", self.key.Log(), self.key))
keys = history
break PRIORITIES
}
}
}
// if peer ready to receive but nothing to send
if keys == nil && deliveryRequest == nil {
// if no items left and switch to waiting mode
log.Trace(fmt.Sprintf("syncer[%v]: buffers consumed. Waiting", self.key.Log()))
newUnsyncedKeys = self.newUnsyncedKeys
}
// send msg iff
// * peer is ready to receive keys AND (
// * all queues and history are depleted OR
// * batch full OR
// * all history have been consumed, synced)
if deliveryRequest == nil &&
(justSynced ||
len(unsynced) > 0 && keys == nil ||
len(unsynced) == int(self.SyncBatchSize)) {
justSynced = false
// listen to requests
deliveryRequest = self.deliveryRequest
newUnsyncedKeys = nil // not care about data until next req comes in
// set sync to current counter
// (all nonhistorical outgoing traffic sheduled and persisted
state.LastSeenAt = self.dbAccess.counter()
state.Latest = storage.ZeroKey
log.Trace(fmt.Sprintf("syncer[%v]: sending %v", self.key.Log(), unsynced))
// send the unsynced keys
stateCopy := *state
err := self.unsyncedKeys(unsynced, &stateCopy)
if err != nil {
log.Warn(fmt.Sprintf("syncer[%v]: unable to send unsynced keys: %v", self.key.Log(), err))
}
self.state = state
log.Debug(fmt.Sprintf("syncer[%v]: --> %v keys sent: (total: %v (%v), history: %v), sent sync state: %v", self.key.Log(), len(unsynced), keyCounts, keyCount, historyCnt, stateCopy))
unsynced = nil
keys = nil
}
// process item and add it to the batch
select {
case <-self.quit:
break LOOP
case req, more = <-keys:
if keys == history && !more {
log.Trace(fmt.Sprintf("syncer[%v]: syncing history segment complete", self.key.Log()))
// history channel is closed, waiting for new state (called from sync())
syncStates = self.syncStates
state.Synced = true // this signals that the current segment is complete
select {
case state.synced <- false:
case <-self.quit:
break LOOP
}
justSynced = true
history = nil
}
case <-deliveryRequest:
log.Trace(fmt.Sprintf("syncer[%v]: peer ready to receive", self.key.Log()))
// this 1 cap channel can wake up the loop
// signaling that peer is ready to receive unsynced Keys
// the channel is set to nil any further writes will be ignored
deliveryRequest = nil
case <-newUnsyncedKeys:
log.Trace(fmt.Sprintf("syncer[%v]: new unsynced keys available", self.key.Log()))
// this 1 cap channel can wake up the loop
// signals that data is available to send if peer is ready to receive
newUnsyncedKeys = nil
keys = self.keys[High]
case state, more = <-syncStates:
// this resets the state
if !more {
state = self.state
log.Trace(fmt.Sprintf("syncer[%v]: (priority %v) syncing complete upto %v)", self.key.Log(), priority, state))
state.Synced = true
syncStates = nil
} else {
log.Trace(fmt.Sprintf("syncer[%v]: (priority %v) syncing history upto %v priority %v)", self.key.Log(), priority, state, histPrior))
state.Synced = false
history = self.syncHistory(state)
// only one history at a time, only allow another one once the
// history channel is closed
syncStates = nil
}
}
if req == nil {
continue LOOP
}
log.Trace(fmt.Sprintf("syncer[%v]: (priority %v) added to unsynced keys: %v", self.key.Log(), priority, req))
keyCounts[priority]++
keyCount++
if keys == history {
log.Trace(fmt.Sprintf("syncer[%v]: (priority %v) history item %v (synced = %v)", self.key.Log(), priority, req, state.Synced))
historyCnt++
}
if sreq, err := self.newSyncRequest(req, priority); err == nil {
// extract key from req
log.Trace(fmt.Sprintf("syncer[%v]: (priority %v): request %v (synced = %v)", self.key.Log(), priority, req, state.Synced))
unsynced = append(unsynced, sreq)
} else {
log.Warn(fmt.Sprintf("syncer[%v]: (priority %v): error creating request for %v: %v)", self.key.Log(), priority, req, err))
}
}
}
// delivery loop
// takes into account priority, send store Requests with chunk (delivery)
// idle blocking if no new deliveries in any of the queues
func (self *syncer) syncDeliveries() {
var req *storeRequestMsgData
p := High
var deliveries chan *storeRequestMsgData
var msg *storeRequestMsgData
var err error
var c = [priorities]int{}
var n = [priorities]int{}
var total, success uint
for {
deliveries = self.deliveries[p]
select {
case req = <-deliveries:
n[p]++
c[p]++
default:
if p == Low {
// blocking, depletion on all channels, no preference for priority
select {
case req = <-self.deliveries[High]:
n[High]++
case req = <-self.deliveries[Medium]:
n[Medium]++
case req = <-self.deliveries[Low]:
n[Low]++
case <-self.quit:
return
}
p = High
} else {
p--
continue
}
}
total++
msg, err = self.newStoreRequestMsgData(req)
if err != nil {
log.Warn(fmt.Sprintf("syncer[%v]: failed to create store request for %v: %v", self.key.Log(), req, err))
} else {
err = self.store(msg)
if err != nil {
log.Warn(fmt.Sprintf("syncer[%v]: failed to deliver %v: %v", self.key.Log(), req, err))
} else {
success++
log.Trace(fmt.Sprintf("syncer[%v]: %v successfully delivered", self.key.Log(), req))
}
}
if total%self.SyncBatchSize == 0 {
log.Debug(fmt.Sprintf("syncer[%v]: deliver Total: %v, Success: %v, High: %v/%v, Medium: %v/%v, Low %v/%v", self.key.Log(), total, success, c[High], n[High], c[Medium], n[Medium], c[Low], n[Low]))
}
}
}
/*
addRequest handles requests for delivery
it accepts 4 types:
* storeRequestMsgData: coming from netstore propagate response
* chunk: coming from forwarding (questionable: id?)
* key: from incoming syncRequest
* syncDbEntry: key,id encoded in db
If sync mode is on for the type of request, then
it sends the request to the keys queue of the correct priority
channel buffered with capacity (SyncBufferSize)
If sync mode is off then, requests are directly sent to deliveries
*/
func (self *syncer) addRequest(req interface{}, ty int) {
// retrieve priority for request type name int8
priority := self.SyncPriorities[ty]
// sync mode for this type ON
if self.syncF() || ty == DeliverReq {
if self.SyncModes[ty] {
self.addKey(req, priority, self.quit)
} else {
self.addDelivery(req, priority, self.quit)
}
}
}
// addKey queues sync request for sync confirmation with given priority
// ie the key will go out in an unsyncedKeys message
func (self *syncer) addKey(req interface{}, priority uint, quit chan bool) bool {
select {
case self.keys[priority] <- req:
// this wakes up the unsynced keys loop if idle
select {
case self.newUnsyncedKeys <- true:
default:
}
return true
case <-quit:
return false
}
}
// addDelivery queues delivery request for with given priority
// ie the chunk will be delivered ASAP mod priority queueing handled by syncdb
// requests are persisted across sessions for correct sync
func (self *syncer) addDelivery(req interface{}, priority uint, quit chan bool) bool {
select {
case self.queues[priority].buffer <- req:
return true
case <-quit:
return false
}
}
// doDelivery delivers the chunk for the request with given priority
// without queuing
func (self *syncer) doDelivery(req interface{}, priority uint, quit chan bool) bool {
msgdata, err := self.newStoreRequestMsgData(req)
if err != nil {
log.Warn(fmt.Sprintf("unable to deliver request %v: %v", msgdata, err))
return false
}
select {
case self.deliveries[priority] <- msgdata:
return true
case <-quit:
return false
}
}
// returns the delivery function for given priority
// passed on to syncDb
func (self *syncer) deliver(priority uint) func(req interface{}, quit chan bool) bool {
return func(req interface{}, quit chan bool) bool {
return self.doDelivery(req, priority, quit)
}
}
// returns the replay function passed on to syncDb
// depending on sync mode settings for BacklogReq,
// re play of request db backlog sends items via confirmation
// or directly delivers
func (self *syncer) replay() func(req interface{}, quit chan bool) bool {
sync := self.SyncModes[BacklogReq]
priority := self.SyncPriorities[BacklogReq]
// sync mode for this type ON
if sync {
return func(req interface{}, quit chan bool) bool {
return self.addKey(req, priority, quit)
}
} else {
return func(req interface{}, quit chan bool) bool {
return self.doDelivery(req, priority, quit)
}
}
}
// given a request, extends it to a full storeRequestMsgData
// polimorphic: see addRequest for the types accepted
func (self *syncer) newStoreRequestMsgData(req interface{}) (*storeRequestMsgData, error) {
key, id, chunk, sreq, err := parseRequest(req)
if err != nil {
return nil, err
}
if sreq == nil {
if chunk == nil {
var err error
chunk, err = self.dbAccess.get(key)
if err != nil {
return nil, err
}
}
sreq = &storeRequestMsgData{
Id: id,
Key: chunk.Key,
SData: chunk.SData,
}
}
return sreq, nil
}
// parse request types and extracts, key, id, chunk, request if available
// does not do chunk lookup !
func parseRequest(req interface{}) (storage.Key, uint64, *storage.Chunk, *storeRequestMsgData, error) {
var key storage.Key
var entry *syncDbEntry
var chunk *storage.Chunk
var id uint64
var ok bool
var sreq *storeRequestMsgData
var err error
if key, ok = req.(storage.Key); ok {
id = generateId()
} else if entry, ok = req.(*syncDbEntry); ok {
id = binary.BigEndian.Uint64(entry.val[32:])
key = storage.Key(entry.val[:32])
} else if chunk, ok = req.(*storage.Chunk); ok {
key = chunk.Key
id = generateId()
} else if sreq, ok = req.(*storeRequestMsgData); ok {
key = sreq.Key
} else {
err = fmt.Errorf("type not allowed: %v (%T)", req, req)
}
return key, id, chunk, sreq, err
}