bitswap.go 10.4 KB
Newer Older
Brian Tiger Chow's avatar
Brian Tiger Chow committed
1 2
// package bitswap implements the IPFS Exchange interface with the BitSwap
// bilateral exchange protocol.
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
3 4 5
package bitswap

import (
6
	"sync"
Jeromy's avatar
Jeromy committed
7 8
	"time"

9
	context "github.com/jbenet/go-ipfs/Godeps/_workspace/src/code.google.com/p/go.net/context"
Brian Tiger Chow's avatar
Brian Tiger Chow committed
10

Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
11
	blocks "github.com/jbenet/go-ipfs/blocks"
12
	blockstore "github.com/jbenet/go-ipfs/blocks/blockstore"
13 14 15 16 17
	exchange "github.com/jbenet/go-ipfs/exchange"
	bsmsg "github.com/jbenet/go-ipfs/exchange/bitswap/message"
	bsnet "github.com/jbenet/go-ipfs/exchange/bitswap/network"
	notifications "github.com/jbenet/go-ipfs/exchange/bitswap/notifications"
	strategy "github.com/jbenet/go-ipfs/exchange/bitswap/strategy"
18
	peer "github.com/jbenet/go-ipfs/peer"
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
19
	u "github.com/jbenet/go-ipfs/util"
Jeromy's avatar
Jeromy committed
20
	eventlog "github.com/jbenet/go-ipfs/util/eventlog"
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
21 22
)

23
var log = eventlog.Logger("bitswap")
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
24

Jeromy's avatar
Jeromy committed
25
// Number of providers to request for sending a wantlist to
Jeromy's avatar
Jeromy committed
26 27
// TODO: if a 'non-nice' strategy is implemented, consider increasing this value
const maxProvidersPerRequest = 3
Jeromy's avatar
Jeromy committed
28

29 30 31
// New initializes a BitSwap instance that communicates over the
// provided BitSwapNetwork. This function registers the returned instance as
// the network delegate.
32
// Runs until context is cancelled
33
func New(parent context.Context, p peer.Peer, network bsnet.BitSwapNetwork, routing bsnet.Routing,
34
	bstore blockstore.Blockstore, nice bool) exchange.Interface {
35

36 37
	ctx, cancelFunc := context.WithCancel(parent)

38 39
	notif := notifications.New()
	go func() {
Brian Tiger Chow's avatar
Brian Tiger Chow committed
40 41
		<-ctx.Done()
		notif.Shutdown()
42 43
	}()

44
	bs := &bitswap{
45
		blockstore:    bstore,
46
		cancelFunc:    cancelFunc,
47
		notifications: notif,
48
		strategy:      strategy.New(nice),
49
		routing:       routing,
Brian Tiger Chow's avatar
Brian Tiger Chow committed
50
		sender:        network,
51
		wantlist:      u.NewKeySet(),
52
		batchRequests: make(chan []u.Key, 32),
53
	}
Brian Tiger Chow's avatar
Brian Tiger Chow committed
54
	network.SetDelegate(bs)
Brian Tiger Chow's avatar
Brian Tiger Chow committed
55
	go bs.loop(ctx)
56 57 58 59

	return bs
}

60 61
// bitswap instances implement the bitswap protocol.
type bitswap struct {
62

63
	// sender delivers messages on behalf of the session
64
	sender bsnet.BitSwapNetwork
65

66
	// blockstore is the local database
67
	// NB: ensure threadsafety
68
	blockstore blockstore.Blockstore
69

70
	// routing interface for communication
71
	routing bsnet.Routing
72

73
	notifications notifications.PubSub
74

75 76 77 78
	// Requests for a set of related blocks
	// the assumption is made that the same peer is likely to
	// have more than a single block in the set
	batchRequests chan []u.Key
Jeromy's avatar
Jeromy committed
79

80
	// strategy listens to network traffic and makes decisions about how to
81
	// interact with partners.
82 83
	// TODO(brian): save the strategy's state to the datastore
	strategy strategy.Strategy
84

85
	wantlist u.KeySet
86 87 88

	// cancelFunc signals cancellation to the bitswap event loop
	cancelFunc func()
89 90
}

91
// GetBlock attempts to retrieve a particular block from peers within the
92
// deadline enforced by the context.
Jeromy's avatar
Jeromy committed
93
func (bs *bitswap) GetBlock(parent context.Context, k u.Key) (*blocks.Block, error) {
94

95 96 97 98
	// Any async work initiated by this function must end when this function
	// returns. To ensure this, derive a new context. Note that it is okay to
	// listen on parent in this scope, but NOT okay to pass |parent| to
	// functions called by this one. Otherwise those functions won't return
99 100
	// when this context's cancel func is executed. This is difficult to
	// enforce. May this comment keep you safe.
101

102
	ctx, cancelFunc := context.WithCancel(parent)
103

Jeromy's avatar
Jeromy committed
104
	ctx = eventlog.ContextWithLoggable(ctx, eventlog.Uuid("GetBlockRequest"))
105
	log.Event(ctx, "GetBlockRequestBegin", &k)
106 107 108 109 110

	defer func() {
		cancelFunc()
		log.Event(ctx, "GetBlockRequestEnd", &k)
	}()
111

112
	promise, err := bs.GetBlocks(ctx, []u.Key{k})
113 114
	if err != nil {
		return nil, err
Jeromy's avatar
Jeromy committed
115
	}
116 117

	select {
118
	case block := <-promise:
Jeromy's avatar
Jeromy committed
119
		return block, nil
120 121
	case <-parent.Done():
		return nil, parent.Err()
122
	}
123

124 125
}

126 127 128 129 130 131 132
// GetBlocks returns a channel where the caller may receive blocks that
// correspond to the provided |keys|. Returns an error if BitSwap is unable to
// begin this request within the deadline enforced by the context.
//
// NB: Your request remains open until the context expires. To conserve
// resources, provide a context with a reasonably short deadline (ie. not one
// that lasts throughout the lifetime of the server)
Jeromy's avatar
Jeromy committed
133
func (bs *bitswap) GetBlocks(ctx context.Context, keys []u.Key) (<-chan *blocks.Block, error) {
134 135 136 137 138
	// TODO log the request

	promise := bs.notifications.Subscribe(ctx, keys...)
	select {
	case bs.batchRequests <- keys:
139
		return promise, nil
140 141 142
	case <-ctx.Done():
		return nil, ctx.Err()
	}
Jeromy's avatar
Jeromy committed
143 144 145
}

func (bs *bitswap) sendWantListTo(ctx context.Context, peers <-chan peer.Peer) error {
Jeromy's avatar
Jeromy committed
146 147 148
	if peers == nil {
		panic("Cant send wantlist to nil peerchan")
	}
Jeromy's avatar
Jeromy committed
149 150 151 152 153
	message := bsmsg.New()
	for _, wanted := range bs.wantlist.Keys() {
		message.AddWanted(wanted)
	}
	for peerToQuery := range peers {
Brian Tiger Chow's avatar
Brian Tiger Chow committed
154
		log.Event(ctx, "PeerToQuery", peerToQuery)
Jeromy's avatar
Jeromy committed
155 156
		go func(p peer.Peer) {

Brian Tiger Chow's avatar
Brian Tiger Chow committed
157
			log.Event(ctx, "DialPeer", p)
Jeromy's avatar
Jeromy committed
158 159
			err := bs.sender.DialPeer(ctx, p)
			if err != nil {
160
				log.Errorf("Error sender.DialPeer(%s): %s", p, err)
Jeromy's avatar
Jeromy committed
161 162 163 164 165 166 167 168 169 170 171 172 173 174 175 176 177 178 179 180 181 182
				return
			}

			response, err := bs.sender.SendRequest(ctx, p, message)
			if err != nil {
				log.Errorf("Error sender.SendRequest(%s) = %s", p, err)
				return
			}
			// FIXME ensure accounting is handled correctly when
			// communication fails. May require slightly different API to
			// get better guarantees. May need shared sequence numbers.
			bs.strategy.MessageSent(p, message)

			if response == nil {
				return
			}
			bs.ReceiveMessage(ctx, p, response)
		}(peerToQuery)
	}
	return nil
}

Jeromy's avatar
Jeromy committed
183
func (bs *bitswap) sendWantlistToProviders(ctx context.Context, ks []u.Key) {
184
	wg := sync.WaitGroup{}
Jeromy's avatar
Jeromy committed
185
	for _, k := range ks {
186
		wg.Add(1)
Jeromy's avatar
Jeromy committed
187 188 189 190 191 192 193
		go func(k u.Key) {
			providers := bs.routing.FindProvidersAsync(ctx, k, maxProvidersPerRequest)

			err := bs.sendWantListTo(ctx, providers)
			if err != nil {
				log.Errorf("error sending wantlist: %s", err)
			}
194
			wg.Done()
Jeromy's avatar
Jeromy committed
195 196
		}(k)
	}
197
	wg.Wait()
Jeromy's avatar
Jeromy committed
198 199
}

200
// TODO ensure only one active request per key
201 202 203
func (bs *bitswap) loop(parent context.Context) {

	ctx, cancel := context.WithCancel(parent)
Jeromy's avatar
Jeromy committed
204

205
	broadcastSignal := time.NewTicker(bs.strategy.GetRebroadcastDelay())
206 207 208 209
	defer func() {
		cancel() // signal to derived async functions
		broadcastSignal.Stop()
	}()
Brian Tiger Chow's avatar
Brian Tiger Chow committed
210

Jeromy's avatar
Jeromy committed
211 212
	for {
		select {
213
		case <-broadcastSignal.C:
Jeromy's avatar
Jeromy committed
214
			// Resend unfulfilled wantlist keys
Jeromy's avatar
Jeromy committed
215
			bs.sendWantlistToProviders(ctx, bs.wantlist.Keys())
216
		case ks := <-bs.batchRequests:
217
			// TODO: implement batching on len(ks) > X for some X
Jeromy's avatar
Jeromy committed
218 219 220
			//		i.e. if given 20 keys, fetch first five, then next
			//		five, and so on, so we are more likely to be able to
			//		effectively stream the data
221 222 223 224
			if len(ks) == 0 {
				log.Warning("Received batch request for zero blocks")
				continue
			}
Jeromy's avatar
Jeromy committed
225 226
			for _, k := range ks {
				bs.wantlist.Add(k)
227
			}
Jeromy's avatar
Jeromy committed
228 229 230 231 232 233 234
			// NB: send want list to providers for the first peer in this list.
			//		the assumption is made that the providers of the first key in
			//		the set are likely to have others as well.
			//		This currently holds true in most every situation, since when
			//		pinning a file, you store and provide all blocks associated with
			//		it. Later, this assumption may not hold as true if we implement
			//		newer bitswap strategies.
235 236 237 238 239
			providers := bs.routing.FindProvidersAsync(ctx, ks[0], maxProvidersPerRequest)

			err := bs.sendWantListTo(ctx, providers)
			if err != nil {
				log.Errorf("error sending wantlist: %s", err)
Jeromy's avatar
Jeromy committed
240
			}
241
		case <-parent.Done():
Jeromy's avatar
Jeromy committed
242 243 244 245 246
			return
		}
	}
}

247 248
// HasBlock announces the existance of a block to this bitswap service. The
// service will potentially notify its peers.
Jeromy's avatar
Jeromy committed
249
func (bs *bitswap) HasBlock(ctx context.Context, blk *blocks.Block) error {
250
	// TODO check all errors
251
	log.Debugf("Has Block %s", blk.Key())
252
	bs.wantlist.Remove(blk.Key())
253
	bs.notifications.Publish(blk)
254
	bs.sendToPeersThatWant(ctx, blk)
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
255
	return bs.routing.Provide(ctx, blk.Key())
256 257
}

258
// receiveBlock handles storing the block in the blockstore and calling HasBlock
Jeromy's avatar
Jeromy committed
259 260 261 262 263 264 265 266 267 268 269 270
func (bs *bitswap) receiveBlock(ctx context.Context, block *blocks.Block) {
	// TODO verify blocks?
	if err := bs.blockstore.Put(block); err != nil {
		log.Criticalf("error putting block: %s", err)
		return
	}
	err := bs.HasBlock(ctx, block)
	if err != nil {
		log.Warningf("HasBlock errored: %s", err)
	}
}

271
// TODO(brian): handle errors
272 273
func (bs *bitswap) ReceiveMessage(ctx context.Context, p peer.Peer, incoming bsmsg.BitSwapMessage) (
	peer.Peer, bsmsg.BitSwapMessage) {
Jeromy's avatar
Jeromy committed
274
	log.Debugf("ReceiveMessage from %s", p)
275

276
	if p == nil {
277
		log.Error("Received message from nil peer!")
278 279
		// TODO propagate the error upward
		return nil, nil
280 281
	}
	if incoming == nil {
282
		log.Error("Got nil bitswap message!")
283 284
		// TODO propagate the error upward
		return nil, nil
285
	}
286

287 288 289
	// Record message bytes in ledger
	// TODO: this is bad, and could be easily abused.
	// Should only track *useful* messages in ledger
Jeromy's avatar
Jeromy committed
290 291 292
	// This call records changes to wantlists, blocks received,
	// and number of bytes transfered.
	bs.strategy.MessageReceived(p, incoming)
293

Jeromy's avatar
Jeromy committed
294 295 296 297 298
	go func() {
		for _, block := range incoming.Blocks() {
			bs.receiveBlock(ctx, block)
		}
	}()
299

300 301
	for _, key := range incoming.Wantlist() {
		if bs.strategy.ShouldSendBlockToPeer(key, p) {
302 303 304
			if block, errBlockNotFound := bs.blockstore.Get(key); errBlockNotFound != nil {
				continue
			} else {
305 306 307
				// Create a separate message to send this block in
				blkmsg := bsmsg.New()

308
				// TODO: only send this the first time
Jeromy's avatar
Jeromy committed
309 310
				//		no sense in sending our wantlist to the
				//		same peer multiple times
311 312
				for _, k := range bs.wantlist.Keys() {
					blkmsg.AddWanted(k)
313 314 315 316
				}

				blkmsg.AddBlock(block)
				bs.send(ctx, p, blkmsg)
317
			}
318 319
		}
	}
320

Jeromy's avatar
Jeromy committed
321
	// TODO: consider changing this function to not return anything
322
	return nil, nil
323 324 325
}

func (bs *bitswap) ReceiveError(err error) {
326
	log.Errorf("Bitswap ReceiveError: %s", err)
327 328
	// TODO log the network error
	// TODO bubble the network error up to the parent context/error logger
329
}
330

331 332
// send strives to ensure that accounting is always performed when a message is
// sent
333
func (bs *bitswap) send(ctx context.Context, p peer.Peer, m bsmsg.BitSwapMessage) {
Brian Tiger Chow's avatar
Brian Tiger Chow committed
334
	bs.sender.SendMessage(ctx, p, m)
335
	bs.strategy.MessageSent(p, m)
336 337
}

Jeromy's avatar
Jeromy committed
338
func (bs *bitswap) sendToPeersThatWant(ctx context.Context, block *blocks.Block) {
Jeromy's avatar
Jeromy committed
339
	log.Debugf("Sending %s to peers that want it", block)
340

341 342
	for _, p := range bs.strategy.Peers() {
		if bs.strategy.BlockIsWantedByPeer(block.Key(), p) {
343
			log.Debugf("%v wants %v", p, block.Key())
344 345
			if bs.strategy.ShouldSendBlockToPeer(block.Key(), p) {
				message := bsmsg.New()
346
				message.AddBlock(block)
347
				for _, wanted := range bs.wantlist.Keys() {
348
					message.AddWanted(wanted)
349
				}
350
				bs.send(ctx, p, message)
351 352 353 354
			}
		}
	}
}
355 356 357 358 359

func (bs *bitswap) Close() error {
	bs.cancelFunc()
	return nil // to conform to Closer interface
}