bitswap.go 11.2 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
	"sync"
Jeromy's avatar
Jeromy committed
9 10
	"time"

11 12
	bssrs "github.com/ipfs/go-bitswap/sessionrequestsplitter"

Jeromy's avatar
Jeromy committed
13
	decision "github.com/ipfs/go-bitswap/decision"
14
	bsgetter "github.com/ipfs/go-bitswap/getter"
Jeromy's avatar
Jeromy committed
15
	bsmsg "github.com/ipfs/go-bitswap/message"
16
	bsmq "github.com/ipfs/go-bitswap/messagequeue"
Jeromy's avatar
Jeromy committed
17
	bsnet "github.com/ipfs/go-bitswap/network"
18
	bspm "github.com/ipfs/go-bitswap/peermanager"
19
	bspqm "github.com/ipfs/go-bitswap/providerquerymanager"
20
	bssession "github.com/ipfs/go-bitswap/session"
21
	bssm "github.com/ipfs/go-bitswap/sessionmanager"
22
	bsspm "github.com/ipfs/go-bitswap/sessionpeermanager"
23
	bswm "github.com/ipfs/go-bitswap/wantmanager"
Jeromy's avatar
Jeromy committed
24 25 26 27 28 29 30 31 32 33
	blocks "github.com/ipfs/go-block-format"
	cid "github.com/ipfs/go-cid"
	blockstore "github.com/ipfs/go-ipfs-blockstore"
	delay "github.com/ipfs/go-ipfs-delay"
	exchange "github.com/ipfs/go-ipfs-exchange-interface"
	logging "github.com/ipfs/go-log"
	metrics "github.com/ipfs/go-metrics-interface"
	process "github.com/jbenet/goprocess"
	procctx "github.com/jbenet/goprocess/context"
	peer "github.com/libp2p/go-libp2p-peer"
34 35
)

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

38 39
var _ exchange.SessionExchange = (*Bitswap)(nil)

Brian Tiger Chow's avatar
Brian Tiger Chow committed
40
const (
Brian Tiger Chow's avatar
Brian Tiger Chow committed
41 42 43
	// 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
44 45
	// TODO: if a 'non-nice' strategy is implemented, consider increasing this value
	maxProvidersPerRequest = 3
Steven Allen's avatar
Steven Allen committed
46
	findProviderDelay      = 1 * time.Second
Brian Tiger Chow's avatar
Brian Tiger Chow committed
47
	providerRequestTimeout = time.Second * 10
48 49 50
	// these requests take at _least_ two minutes at the moment.
	provideTimeout       = time.Minute * 3
	sizeBatchRequestChan = 32
Jeromy's avatar
Jeromy committed
51
)
52

Jeromy's avatar
Jeromy committed
53
var (
54 55
	HasBlockBufferSize    = 256
	provideKeysBufferSize = 2048
Steven Allen's avatar
Steven Allen committed
56
	provideWorkerMax      = 6
57 58 59

	// the 1<<18+15 is to observe old file chunks that are 1<<18 + 14 in size
	metricsBuckets = []float64{1 << 6, 1 << 10, 1 << 14, 1 << 18, 1<<18 + 15, 1 << 22}
Brian Tiger Chow's avatar
Brian Tiger Chow committed
60
)
Jeromy's avatar
Jeromy committed
61

62
var rebroadcastDelay = delay.Fixed(time.Minute)
63

Brian Tiger Chow's avatar
Brian Tiger Chow committed
64 65 66 67
// 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.
Łukasz Magiera's avatar
Łukasz Magiera committed
68 69
func New(parent context.Context, network bsnet.BitSwapNetwork,
	bstore blockstore.Blockstore) exchange.Interface {
70

71 72
	// 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
73
	// coupled to the concerns of the ipfs daemon in this way.
74 75 76 77
	//
	// 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
78
	ctx, cancelFunc := context.WithCancel(parent)
79
	ctx = metrics.CtxSubScope(ctx, "bitswap")
80
	dupHist := metrics.NewCtx(ctx, "recv_dup_blocks_bytes", "Summary of duplicate"+
81
		" data blocks recived").Histogram(metricsBuckets)
82
	allHist := metrics.NewCtx(ctx, "recv_all_blocks_bytes", "Summary of all"+
83
		" data blocks recived").Histogram(metricsBuckets)
84

85 86 87
	sentHistogram := metrics.NewCtx(ctx, "sent_all_blocks_bytes", "Histogram of blocks sent by"+
		" this bitswap").Histogram(metricsBuckets)

88 89 90 91
	px := process.WithTeardown(func() error {
		return nil
	})

92 93
	peerQueueFactory := func(ctx context.Context, p peer.ID) bspm.PeerQueue {
		return bsmq.New(ctx, p, network)
94 95
	}

96
	wm := bswm.New(ctx, bspm.New(ctx, peerQueueFactory))
97 98
	pqm := bspqm.New(ctx, network)

99 100
	sessionFactory := func(ctx context.Context, id uint64, pm bssession.PeerManager, srs bssession.RequestSplitter) bssm.Session {
		return bssession.New(ctx, id, wm, pm, srs)
101 102
	}
	sessionPeerManagerFactory := func(ctx context.Context, id uint64) bssession.PeerManager {
103
		return bsspm.New(ctx, id, network.ConnectionManager(), pqm)
104
	}
105 106 107
	sessionRequestSplitterFactory := func(ctx context.Context) bssession.RequestSplitter {
		return bssrs.New(ctx)
	}
108

109
	bs := &Bitswap{
110
		blockstore:    bstore,
111
		engine:        decision.NewEngine(ctx, bstore), // TODO close the engine with Close() method
112
		network:       network,
113
		process:       px,
114 115
		newBlocks:     make(chan cid.Cid, HasBlockBufferSize),
		provideKeys:   make(chan cid.Cid, provideKeysBufferSize),
116
		wm:            wm,
117
		pqm:           pqm,
118
		sm:            bssm.New(ctx, sessionFactory, sessionPeerManagerFactory, sessionRequestSplitterFactory),
119
		counters:      new(counters),
120 121 122
		dupMetric:     dupHist,
		allMetric:     allHist,
		sentHistogram: sentHistogram,
123
	}
124 125

	bs.wm.Startup()
126
	bs.pqm.Startup()
Brian Tiger Chow's avatar
Brian Tiger Chow committed
127
	network.SetDelegate(bs)
128

129 130
	// Start up bitswaps async worker routines
	bs.startWorkers(px, ctx)
131 132 133 134 135 136 137 138 139

	// 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

140 141 142
	return bs
}

143 144
// Bitswap instances implement the bitswap protocol.
type Bitswap struct {
145
	// the wantlist tracks global wants for bitswap
146
	wm *bswm.WantManager
147

148 149 150
	// the provider query manager manages requests to find providers
	pqm *bspqm.ProviderQueryManager

151 152
	// 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
153

154 155
	// network delivers messages on behalf of the session
	network bsnet.BitSwapNetwork
156 157 158 159 160

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

161 162 163
	// 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
164
	newBlocks chan cid.Cid
165
	// provideKeys directly feeds provide workers
166
	provideKeys chan cid.Cid
167

168 169 170
	process process.Process

	// Counters for various statistics
171 172
	counterLk sync.Mutex
	counters  *counters
173 174

	// Metrics interface metrics
175 176 177
	dupMetric     metrics.Histogram
	allMetric     metrics.Histogram
	sentHistogram metrics.Histogram
Jeromy's avatar
Jeromy committed
178

179 180
	// the sessionmanager manages tracking sessions
	sm *bssm.SessionManager
181 182
}

183 184 185 186 187 188 189 190 191 192
type counters struct {
	blocksRecvd    uint64
	dupBlocksRecvd uint64
	dupDataRecvd   uint64
	blocksSent     uint64
	dataSent       uint64
	dataRecvd      uint64
	messagesRecvd  uint64
}

193
type blockRequest struct {
194
	Cid cid.Cid
195
	Ctx context.Context
196 197
}

198
// GetBlock attempts to retrieve a particular block from peers within the
199
// deadline enforced by the context.
200
func (bs *Bitswap) GetBlock(parent context.Context, k cid.Cid) (blocks.Block, error) {
201
	return bsgetter.SyncGetBlock(parent, k, bs.GetBlocks)
202 203
}

204 205
func (bs *Bitswap) WantlistForPeer(p peer.ID) []cid.Cid {
	var out []cid.Cid
206
	for _, e := range bs.engine.WantlistForPeer(p) {
207
		out = append(out, e.Cid)
208 209 210 211
	}
	return out
}

212 213 214 215
func (bs *Bitswap) LedgerForPeer(p peer.ID) *decision.Receipt {
	return bs.engine.LedgerForPeer(p)
}

216 217 218 219 220 221 222
// 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)
223
func (bs *Bitswap) GetBlocks(ctx context.Context, keys []cid.Cid) (<-chan blocks.Block, error) {
224 225
	session := bs.sm.NewSession(ctx)
	return session.GetBlocks(ctx, keys)
Jeromy's avatar
Jeromy committed
226 227
}

Łukasz Magiera's avatar
Łukasz Magiera committed
228
// HasBlock announces the existence of a block to this bitswap service. The
229
// service will potentially notify its peers.
230
func (bs *Bitswap) HasBlock(blk blocks.Block) error {
231 232 233 234 235 236 237 238
	return bs.receiveBlockFrom(blk, "")
}

// TODO: Some of this stuff really only needs to be done when adding a block
// from the user, not when receiving it from the network.
// In case you run `git blame` on this comment, I'll save you some time: ask
// @whyrusleeping, I don't know the answers you seek.
func (bs *Bitswap) receiveBlockFrom(blk blocks.Block, from peer.ID) error {
239 240 241 242 243
	select {
	case <-bs.process.Closing():
		return errors.New("bitswap is closed")
	default:
	}
244

245
	err := bs.blockstore.Put(blk)
246 247
	if err != nil {
		log.Errorf("Error writing block to datastore: %s", err)
248 249
		return err
	}
250

251 252 253 254 255
	// 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
256

257
	bs.sm.ReceiveBlockFrom(from, blk)
258

259 260
	bs.engine.AddBlock(blk)

261
	select {
262
	case bs.newBlocks <- blk.Cid():
263
		// send block off to be reprovided
264 265
	case <-bs.process.Closing():
		return bs.process.Close()
266 267
	}
	return nil
268 269
}

270
func (bs *Bitswap) ReceiveMessage(ctx context.Context, p peer.ID, incoming bsmsg.BitSwapMessage) {
Steven Allen's avatar
Steven Allen committed
271 272 273
	bs.counterLk.Lock()
	bs.counters.messagesRecvd++
	bs.counterLk.Unlock()
Jeromy's avatar
Jeromy committed
274

Jeromy's avatar
Jeromy committed
275 276
	// This call records changes to wantlists, blocks received,
	// and number of bytes transfered.
277
	bs.engine.MessageReceived(p, incoming)
Jeromy's avatar
Jeromy committed
278 279
	// TODO: this is bad, and could be easily abused.
	// Should only track *useful* messages in ledger
280

281 282 283
	iblocks := incoming.Blocks()

	if len(iblocks) == 0 {
284 285 286
		return
	}

Jeromy's avatar
Jeromy committed
287 288
	wg := sync.WaitGroup{}
	for _, block := range iblocks {
289

Jeromy's avatar
Jeromy committed
290
		wg.Add(1)
291
		go func(b blocks.Block) { // TODO: this probably doesnt need to be a goroutine...
Jeromy's avatar
Jeromy committed
292
			defer wg.Done()
293

294
			bs.updateReceiveCounters(b)
295
			bs.sm.UpdateReceiveCounters(b)
296
			log.Debugf("got block %s from %s", b, p)
297

298
			// skip received blocks that are not in the wantlist
299
			if !bs.wm.IsWanted(b.Cid()) {
300 301 302
				return
			}

303 304
			if err := bs.receiveBlockFrom(b, p); err != nil {
				log.Warningf("ReceiveMessage recvBlockFrom error: %s", err)
Jeromy's avatar
Jeromy committed
305
			}
306
			log.Event(ctx, "Bitswap.GetBlockRequest.End", b.Cid())
Jeromy's avatar
Jeromy committed
307
		}(block)
308
	}
Jeromy's avatar
Jeromy committed
309
	wg.Wait()
310 311
}

312 313
var ErrAlreadyHaveBlock = errors.New("already have block")

314
func (bs *Bitswap) updateReceiveCounters(b blocks.Block) {
315
	blkLen := len(b.RawData())
316
	has, err := bs.blockstore.Has(b.Cid())
317 318
	if err != nil {
		log.Infof("blockstore.Has error: %s", err)
319
		return
320
	}
321 322 323

	bs.allMetric.Observe(float64(blkLen))
	if has {
324
		bs.dupMetric.Observe(float64(blkLen))
325 326
	}

327 328
	bs.counterLk.Lock()
	defer bs.counterLk.Unlock()
329
	c := bs.counters
330

331 332
	c.blocksRecvd++
	c.dataRecvd += uint64(len(b.RawData()))
333
	if has {
334 335
		c.dupBlocksRecvd++
		c.dupDataRecvd += uint64(blkLen)
336 337 338
	}
}

339
// Connected/Disconnected warns bitswap about peer connections.
340
func (bs *Bitswap) PeerConnected(p peer.ID) {
341
	bs.wm.Connected(p)
342
	bs.engine.PeerConnected(p)
343 344
}

345
// Connected/Disconnected warns bitswap about peer connections.
346
func (bs *Bitswap) PeerDisconnected(p peer.ID) {
347
	bs.wm.Disconnected(p)
348
	bs.engine.PeerDisconnected(p)
349 350
}

351
func (bs *Bitswap) ReceiveError(err error) {
352
	log.Infof("Bitswap ReceiveError: %s", err)
353 354
	// TODO log the network error
	// TODO bubble the network error up to the parent context/error logger
355 356
}

357
func (bs *Bitswap) Close() error {
358
	return bs.process.Close()
359
}
360

361
func (bs *Bitswap) GetWantlist() []cid.Cid {
362
	entries := bs.wm.CurrentWants()
363
	out := make([]cid.Cid, 0, len(entries))
364
	for _, e := range entries {
365
		out = append(out, e.Cid)
366 367 368
	}
	return out
}
369 370 371 372

func (bs *Bitswap) IsOnline() bool {
	return true
}
373 374 375 376

func (bs *Bitswap) NewSession(ctx context.Context) exchange.Fetcher {
	return bs.sm.NewSession(ctx)
}