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

import (
6
	"context"
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
7
	"errors"
8
	"math"
9
	"sync"
Jeromy's avatar
Jeromy committed
10 11
	"time"

12 13 14 15 16 17 18
	blocks "github.com/ipfs/go-ipfs/blocks"
	blockstore "github.com/ipfs/go-ipfs/blocks/blockstore"
	exchange "github.com/ipfs/go-ipfs/exchange"
	decision "github.com/ipfs/go-ipfs/exchange/bitswap/decision"
	bsmsg "github.com/ipfs/go-ipfs/exchange/bitswap/message"
	bsnet "github.com/ipfs/go-ipfs/exchange/bitswap/network"
	notifications "github.com/ipfs/go-ipfs/exchange/bitswap/notifications"
Jeromy's avatar
Jeromy committed
19
	flags "github.com/ipfs/go-ipfs/flags"
20
	"github.com/ipfs/go-ipfs/thirdparty/delay"
Jeromy's avatar
Jeromy committed
21

22 23
	process "gx/ipfs/QmSF8fPo3jgVBAy8fpdjjYqgG87dkJgUprRBHRd2tmfgpP/goprocess"
	procctx "gx/ipfs/QmSF8fPo3jgVBAy8fpdjjYqgG87dkJgUprRBHRd2tmfgpP/goprocess/context"
Jeromy's avatar
Jeromy committed
24
	logging "gx/ipfs/QmSpJByNKFX1sCsHBEp3R73FL4NF6FnQTEGyNAXHm2GS52/go-log"
25 26 27
	loggables "gx/ipfs/QmTcfnDHimxBJqx6utpnWqVHdvyquXgkwAvYt4zMaJMKS2/go-libp2p-loggables"
	cid "gx/ipfs/QmV5gPoRsjN1Gid3LMdNZTyfCtP2DsvqEbMAmz82RmmiGk/go-cid"
	peer "gx/ipfs/QmZcUPvPhD1Xvk6mwijYF8AfR3mG31S1YsEfHG4khrFPRr/go-libp2p-peer"
28 29
)

Jeromy's avatar
Jeromy committed
30
var log = logging.Logger("bitswap")
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
31

Brian Tiger Chow's avatar
Brian Tiger Chow committed
32
const (
Brian Tiger Chow's avatar
Brian Tiger Chow committed
33 34 35
	// maxProvidersPerRequest specifies the maximum number of providers desired
	// from the network. This value is specified because the network streams
	// results.
Brian Tiger Chow's avatar
Brian Tiger Chow committed
36 37 38 39
	// TODO: if a 'non-nice' strategy is implemented, consider increasing this value
	maxProvidersPerRequest = 3
	providerRequestTimeout = time.Second * 10
	hasBlockTimeout        = time.Second * 15
40
	provideTimeout         = time.Second * 15
Brian Tiger Chow's avatar
Brian Tiger Chow committed
41
	sizeBatchRequestChan   = 32
42 43
	// kMaxPriority is the max priority as defined by the bitswap protocol
	kMaxPriority = math.MaxInt32
Jeromy's avatar
Jeromy committed
44
)
45

Jeromy's avatar
Jeromy committed
46
var (
47 48 49
	HasBlockBufferSize    = 256
	provideKeysBufferSize = 2048
	provideWorkerMax      = 512
Brian Tiger Chow's avatar
Brian Tiger Chow committed
50
)
Jeromy's avatar
Jeromy committed
51

Jeromy's avatar
Jeromy committed
52 53 54 55 56 57 58 59
func init() {
	if flags.LowMemMode {
		HasBlockBufferSize = 64
		provideKeysBufferSize = 512
		provideWorkerMax = 16
	}
}

60
var rebroadcastDelay = delay.Fixed(time.Minute)
61

Brian Tiger Chow's avatar
Brian Tiger Chow committed
62 63 64 65
// New initializes a BitSwap instance that communicates over the provided
// BitSwapNetwork. This function registers the returned instance as the network
// delegate.
// Runs until context is cancelled.
66
func New(parent context.Context, p peer.ID, network bsnet.BitSwapNetwork,
67
	bstore blockstore.Blockstore, nice bool) exchange.Interface {
68

69 70
	// important to use provided parent context (since it may include important
	// loggable data). It's probably not a good idea to allow bitswap to be
71
	// coupled to the concerns of the ipfs daemon in this way.
72 73 74 75
	//
	// FIXME(btc) Now that bitswap manages itself using a process, it probably
	// shouldn't accept a context anymore. Clients should probably use Close()
	// exclusively. We should probably find another way to share logging data
76 77
	ctx, cancelFunc := context.WithCancel(parent)

78
	notif := notifications.New()
79 80 81 82 83
	px := process.WithTeardown(func() error {
		notif.Shutdown()
		return nil
	})

84
	bs := &Bitswap{
85
		blockstore:    bstore,
86
		notifications: notif,
87
		engine:        decision.NewEngine(ctx, bstore), // TODO close the engine with Close() method
88
		network:       network,
89
		findKeys:      make(chan *blockRequest, sizeBatchRequestChan),
90
		process:       px,
91 92
		newBlocks:     make(chan *cid.Cid, HasBlockBufferSize),
		provideKeys:   make(chan *cid.Cid, provideKeysBufferSize),
93
		wm:            NewWantManager(ctx, network),
94
	}
95
	go bs.wm.Run()
Brian Tiger Chow's avatar
Brian Tiger Chow committed
96
	network.SetDelegate(bs)
97

98 99
	// Start up bitswaps async worker routines
	bs.startWorkers(px, ctx)
100 101 102 103 104 105 106 107 108

	// bind the context and process.
	// do it over here to avoid closing before all setup is done.
	go func() {
		<-px.Closing() // process closes first
		cancelFunc()
	}()
	procctx.CloseAfterContext(px, ctx) // parent cancelled first

109 110 111
	return bs
}

112 113
// Bitswap instances implement the bitswap protocol.
type Bitswap struct {
114 115 116
	// the peermanager manages sending messages to peers in a way that
	// wont block bitswap operation
	wm *WantManager
117

118 119
	// the engine is the bit of logic that decides who to send which blocks to
	engine *decision.Engine
Juan Batiz-Benet's avatar
Juan Batiz-Benet committed
120

121 122
	// network delivers messages on behalf of the session
	network bsnet.BitSwapNetwork
123 124 125 126 127

	// blockstore is the local database
	// NB: ensure threadsafety
	blockstore blockstore.Blockstore

128 129
	// notifications engine for receiving new blocks and routing them to the
	// appropriate user requests
130 131
	notifications notifications.PubSub

132
	// findKeys sends keys to a worker to find and connect to providers for them
133
	findKeys chan *blockRequest
134 135 136
	// newBlocks is a channel for newly added blocks to be provided to the
	// network.  blocks pushed down this channel get buffered and fed to the
	// provideKeys channel later on to avoid too much network activity
137
	newBlocks chan *cid.Cid
138
	// provideKeys directly feeds provide workers
139
	provideKeys chan *cid.Cid
140

141 142 143
	process process.Process

	// Counters for various statistics
Jeromy's avatar
Jeromy committed
144
	counterLk      sync.Mutex
145 146
	blocksRecvd    int
	dupBlocksRecvd int
147
	dupDataRecvd   uint64
148 149
}

150
type blockRequest struct {
151
	Cid *cid.Cid
152
	Ctx context.Context
153 154
}

155
// GetBlock attempts to retrieve a particular block from peers within the
156
// deadline enforced by the context.
157 158 159
func (bs *Bitswap) GetBlock(parent context.Context, k *cid.Cid) (blocks.Block, error) {
	if k == nil {
		log.Error("nil cid in GetBlock")
jbenet's avatar
jbenet committed
160 161
		return nil, blockstore.ErrNotFound
	}
162

163 164 165 166
	// 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
167 168
	// when this context's cancel func is executed. This is difficult to
	// enforce. May this comment keep you safe.
169
	ctx, cancelFunc := context.WithCancel(parent)
170

171 172
	// TODO: this request ID should come in from a higher layer so we can track
	// across multiple 'GetBlock' invocations
173
	ctx = logging.ContextWithLoggable(ctx, loggables.Uuid("GetBlockRequest"))
174 175
	log.Event(ctx, "Bitswap.GetBlockRequest.Start", k)
	defer log.Event(ctx, "Bitswap.GetBlockRequest.End", k)
176
	defer cancelFunc()
177

178
	promise, err := bs.GetBlocks(ctx, []*cid.Cid{k})
179 180
	if err != nil {
		return nil, err
Jeromy's avatar
Jeromy committed
181
	}
182 183

	select {
184 185 186 187 188 189 190 191 192
	case block, ok := <-promise:
		if !ok {
			select {
			case <-ctx.Done():
				return nil, ctx.Err()
			default:
				return nil, errors.New("promise channel was closed")
			}
		}
Jeromy's avatar
Jeromy committed
193
		return block, nil
194 195
	case <-parent.Done():
		return nil, parent.Err()
196 197 198
	}
}

199 200
func (bs *Bitswap) WantlistForPeer(p peer.ID) []*cid.Cid {
	var out []*cid.Cid
201
	for _, e := range bs.engine.WantlistForPeer(p) {
202
		out = append(out, e.Cid)
203 204 205 206
	}
	return out
}

207 208 209 210
func (bs *Bitswap) LedgerForPeer(p peer.ID) *decision.Receipt {
	return bs.engine.LedgerForPeer(p)
}

211 212 213 214 215 216 217
// 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)
218
func (bs *Bitswap) GetBlocks(ctx context.Context, keys []*cid.Cid) (<-chan blocks.Block, error) {
219
	if len(keys) == 0 {
220
		out := make(chan blocks.Block)
221 222 223 224
		close(out)
		return out, nil
	}

225 226 227 228 229
	select {
	case <-bs.process.Closing():
		return nil, errors.New("bitswap is closed")
	default:
	}
230
	promise := bs.notifications.Subscribe(ctx, keys...)
231

232
	for _, k := range keys {
233
		log.Event(ctx, "Bitswap.GetBlockRequest.Start", k)
234 235
	}

236
	bs.wm.WantBlocks(ctx, keys)
237

238 239 240
	// NB: Optimization. Assumes that providers of key[0] are likely to
	// be able to provide for all keys. This currently holds true in most
	// every situation. Later, this assumption may not hold as true.
241
	req := &blockRequest{
242
		Cid: keys[0],
243
		Ctx: ctx,
244
	}
245

246
	remaining := cid.NewSet()
247
	for _, k := range keys {
248
		remaining.Add(k)
249 250 251 252 253 254 255 256
	}

	out := make(chan blocks.Block)
	go func() {
		ctx, cancel := context.WithCancel(ctx)
		defer cancel()
		defer close(out)
		defer func() {
257 258
			// can't just defer this call on its own, arguments are resolved *when* the defer is created
			bs.CancelWants(remaining.Keys())
259 260 261 262 263 264 265 266
		}()
		for {
			select {
			case blk, ok := <-promise:
				if !ok {
					return
				}

267
				remaining.Remove(blk.Cid())
268 269 270 271 272 273 274 275 276 277 278
				select {
				case out <- blk:
				case <-ctx.Done():
					return
				}
			case <-ctx.Done():
				return
			}
		}
	}()

279
	select {
Jeromy's avatar
Jeromy committed
280
	case bs.findKeys <- req:
281
		return out, nil
282 283 284
	case <-ctx.Done():
		return nil, ctx.Err()
	}
Jeromy's avatar
Jeromy committed
285 286
}

287
// CancelWant removes a given key from the wantlist
288 289
func (bs *Bitswap) CancelWants(cids []*cid.Cid) {
	bs.wm.CancelWants(cids)
290 291
}

292 293
// HasBlock announces the existance of a block to this bitswap service. The
// service will potentially notify its peers.
294
func (bs *Bitswap) HasBlock(blk blocks.Block) error {
295 296 297 298 299
	select {
	case <-bs.process.Closing():
		return errors.New("bitswap is closed")
	default:
	}
300

301
	err := bs.blockstore.Put(blk)
302 303
	if err != nil {
		log.Errorf("Error writing block to datastore: %s", err)
304 305
		return err
	}
306

307 308 309 310 311
	// NOTE: There exists the possiblity for a race condition here.  If a user
	// creates a node, then adds it to the dagservice while another goroutine
	// is waiting on a GetBlock for that object, they will receive a reference
	// to the same node. We should address this soon, but i'm not going to do
	// it now as it requires more thought and isnt causing immediate problems.
Jeromy's avatar
Jeromy committed
312 313
	bs.notifications.Publish(blk)

314 315
	bs.engine.AddBlock(blk)

316
	select {
317
	case bs.newBlocks <- blk.Cid():
318
		// send block off to be reprovided
319 320
	case <-bs.process.Closing():
		return bs.process.Close()
321 322
	}
	return nil
323 324
}

325
func (bs *Bitswap) ReceiveMessage(ctx context.Context, p peer.ID, incoming bsmsg.BitSwapMessage) {
Jeromy's avatar
Jeromy committed
326 327
	// This call records changes to wantlists, blocks received,
	// and number of bytes transfered.
328
	bs.engine.MessageReceived(p, incoming)
Jeromy's avatar
Jeromy committed
329 330
	// TODO: this is bad, and could be easily abused.
	// Should only track *useful* messages in ledger
331

332 333 334
	iblocks := incoming.Blocks()

	if len(iblocks) == 0 {
335 336 337 338
		return
	}

	// quickly send out cancels, reduces chances of duplicate block receives
339
	var keys []*cid.Cid
340
	for _, block := range iblocks {
341
		if _, found := bs.wm.wl.Contains(block.Cid()); !found {
342
			log.Infof("received un-asked-for %s from %s", block, p)
343 344
			continue
		}
345
		keys = append(keys, block.Cid())
Jeromy's avatar
Jeromy committed
346 347
	}
	bs.wm.CancelWants(keys)
348

Jeromy's avatar
Jeromy committed
349 350 351
	wg := sync.WaitGroup{}
	for _, block := range iblocks {
		wg.Add(1)
352
		go func(b blocks.Block) {
Jeromy's avatar
Jeromy committed
353
			defer wg.Done()
354

355
			if err := bs.updateReceiveCounters(b); err != nil {
356
				return // ignore error, is either logged previously, or ErrAlreadyHaveBlock
Jeromy's avatar
Jeromy committed
357
			}
358

359 360
			k := b.Cid()
			log.Event(ctx, "Bitswap.GetBlockRequest.End", k)
361

362
			log.Debugf("got block %s from %s", b, p)
363
			if err := bs.HasBlock(b); err != nil {
Jeromy's avatar
Jeromy committed
364 365 366
				log.Warningf("ReceiveMessage HasBlock error: %s", err)
			}
		}(block)
367
	}
Jeromy's avatar
Jeromy committed
368
	wg.Wait()
369 370
}

371 372
var ErrAlreadyHaveBlock = errors.New("already have block")

373
func (bs *Bitswap) updateReceiveCounters(b blocks.Block) error {
374 375 376
	bs.counterLk.Lock()
	defer bs.counterLk.Unlock()
	bs.blocksRecvd++
377
	has, err := bs.blockstore.Has(b.Cid())
378 379 380 381 382 383
	if err != nil {
		log.Infof("blockstore.Has error: %s", err)
		return err
	}
	if err == nil && has {
		bs.dupBlocksRecvd++
Jeromy's avatar
Jeromy committed
384
		bs.dupDataRecvd += uint64(len(b.RawData()))
385 386 387 388 389 390 391 392
	}

	if has {
		return ErrAlreadyHaveBlock
	}
	return nil
}

393
// Connected/Disconnected warns bitswap about peer connections
394
func (bs *Bitswap) PeerConnected(p peer.ID) {
395
	bs.wm.Connected(p)
396 397 398
}

// Connected/Disconnected warns bitswap about peer connections
399
func (bs *Bitswap) PeerDisconnected(p peer.ID) {
400
	bs.wm.Disconnected(p)
401
	bs.engine.PeerDisconnected(p)
402 403
}

404
func (bs *Bitswap) ReceiveError(err error) {
405
	log.Infof("Bitswap ReceiveError: %s", err)
406 407
	// TODO log the network error
	// TODO bubble the network error up to the parent context/error logger
408 409
}

410
func (bs *Bitswap) Close() error {
411
	return bs.process.Close()
412
}
413

414 415
func (bs *Bitswap) GetWantlist() []*cid.Cid {
	var out []*cid.Cid
416
	for _, e := range bs.wm.wl.Entries() {
417
		out = append(out, e.Cid)
418 419 420
	}
	return out
}
421 422 423 424

func (bs *Bitswap) IsOnline() bool {
	return true
}