From 0830589b0adb09d4c671d22674810107a5c92ba4 Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Sun, 9 May 2021 14:14:11 +1000 Subject: [PATCH 01/59] Pass tests with new full-client request strategy implementation --- client.go | 5 +- config.go | 7 +- peer-impl.go | 1 + peerconn.go | 178 +++--------------- piece.go | 18 +- request-strategy-defaults.go | 44 ----- request-strategy.go | 347 +++++++++++++++-------------------- torrent.go | 64 +++---- webseed-peer.go | 5 +- 9 files changed, 222 insertions(+), 447 deletions(-) diff --git a/client.go b/client.go index cab4c234..b0f537ca 100644 --- a/client.go +++ b/client.go @@ -81,6 +81,8 @@ type Client struct { websocketTrackers websocketTrackers activeAnnounceLimiter limiter.Instance + + clientPieceRequestOrder } type ipStr string @@ -293,6 +295,8 @@ func NewClient(cfg *ClientConfig) (cl *Client, err error) { }, } + go cl.requester() + return } @@ -1139,7 +1143,6 @@ func (cl *Client) newTorrent(ih metainfo.Hash, specStorage storage.ClientImpl) ( webSeeds: make(map[string]*Peer), } t._pendingPieces.NewSet = priorityBitmapStableNewSet - t.requestStrategy = cl.config.DefaultRequestStrategy(t.requestStrategyCallbacks(), &cl._mu) t.logger = cl.logger.WithContextValue(t) t.setChunkSize(defaultChunkSize) return diff --git a/config.go b/config.go index 373ce6fe..5aeef38b 100644 --- a/config.go +++ b/config.go @@ -137,8 +137,6 @@ type ClientConfig struct { // OnQuery hook func DHTOnQuery func(query *krpc.Msg, source net.Addr) (propagate bool) - DefaultRequestStrategy requestStrategyMaker - Extensions PeerExtensionBits DisableWebtorrent bool @@ -185,10 +183,7 @@ func NewDefaultClientConfig() *ClientConfig { CryptoSelector: mse.DefaultCryptoSelector, CryptoProvides: mse.AllSupportedCrypto, ListenPort: 42069, - - DefaultRequestStrategy: RequestStrategyDuplicateRequestTimeout(5 * time.Second), - - Extensions: defaultPeerExtensionBytes(), + Extensions: defaultPeerExtensionBytes(), } //cc.ConnTracker.SetNoMaxEntries() //cc.ConnTracker.Timeout = func(conntrack.Entry) time.Duration { return 0 } diff --git a/peer-impl.go b/peer-impl.go index a04a160b..f4ad12a2 100644 --- a/peer-impl.go +++ b/peer-impl.go @@ -20,4 +20,5 @@ type peerImpl interface { drop() String() string connStatusString() string + writeBufferFull() bool } diff --git a/peerconn.go b/peerconn.go index b26f4398..9522adf3 100644 --- a/peerconn.go +++ b/peerconn.go @@ -337,16 +337,16 @@ func (cn *Peer) writeStatus(w io.Writer, t *Torrent) { cn.statusFlags(), cn.downloadRate()/(1<<10), ) - fmt.Fprintf(w, " next pieces: %v%s\n", - iter.ToSlice(iter.Head(10, cn.iterPendingPiecesUntyped)), - func() string { - if cn == t.fastestPeer { - return " (fastest)" - } else { - return "" - } - }(), - ) + //fmt.Fprintf(w, " next pieces: %v%s\n", + // iter.ToSlice(iter.Head(10, cn.iterPendingPiecesUntyped)), + // func() string { + // if cn == t.fastestPeer { + // return " (fastest)" + // } else { + // return "" + // } + // }(), + //) } func (cn *Peer) close() { @@ -402,7 +402,12 @@ func (cn *PeerConn) write(msg pp.Message) bool { cn.wroteMsg(&msg) cn.writeBuffer.Write(msg.MustMarshalBinary()) torrent.Add(fmt.Sprintf("messages filled of type %s", msg.Type.String()), 1) - return cn.writeBuffer.Len() < writeBufferHighWaterLen + cn.tickleWriter() + return !cn.writeBufferFull() +} + +func (cn *PeerConn) writeBufferFull() bool { + return cn.writeBuffer.Len() >= writeBufferHighWaterLen } func (cn *PeerConn) requestMetadataPiece(index int) { @@ -440,11 +445,7 @@ func (cn *PeerConn) requestedMetadataPiece(index int) bool { // The actual value to use as the maximum outbound requests. func (cn *Peer) nominalMaxRequests() (ret int) { - return int(clamp( - 1, - int64(cn.PeerMaxRequests), - int64(cn.t.requestStrategy.nominalMaxRequests(cn.requestStrategyConnection())), - )) + return cn.PeerMaxRequests } func (cn *Peer) totalExpectingTime() (ret time.Duration) { @@ -528,12 +529,12 @@ func (pc *PeerConn) writeInterested(interested bool) bool { // are okay. type messageWriter func(pp.Message) bool -func (cn *Peer) request(r Request) bool { +func (cn *Peer) request(r Request) (more bool, err error) { if _, ok := cn.requests[r]; ok { - panic("chunk already requested") + return true, nil } if !cn.peerHasPiece(pieceIndex(r.Index)) { - panic("requesting piece peer doesn't have") + return true, errors.New("requesting piece peer doesn't have") } if !cn.t.peerIsActive(cn) { panic("requesting but not in active conns") @@ -545,7 +546,7 @@ func (cn *Peer) request(r Request) bool { if cn.peerAllowedFast.Get(int(r.Index)) { torrent.Add("allowed fast requests sent", 1) } else { - panic("requesting while choking and not allowed fast") + return cn.setInterested(true), errors.New("requesting while choked and not allowed fast") } } if cn.t.hashingPiece(pieceIndex(r.Index)) { @@ -563,12 +564,11 @@ func (cn *Peer) request(r Request) bool { } cn.validReceiveChunks[r]++ cn.t.pendingRequests[r]++ - cn.t.requestStrategy.hooks().sentRequest(r) cn.updateExpectingChunks() for _, f := range cn.callbacks.SentRequest { f(PeerRequestEvent{cn, r}) } - return cn.peerImpl.request(r) + return cn.peerImpl.request(r), nil } func (me *PeerConn) request(r Request) bool { @@ -584,64 +584,7 @@ func (me *PeerConn) cancel(r Request) bool { return me.write(makeCancelMessage(r)) } -func (cn *Peer) doRequestState() bool { - if !cn.t.networkingEnabled || cn.t.dataDownloadDisallowed { - if !cn.setInterested(false) { - return false - } - if len(cn.requests) != 0 { - for r := range cn.requests { - cn.deleteRequest(r) - // log.Printf("%p: cancelling request: %v", cn, r) - if !cn.peerImpl.cancel(r) { - return false - } - } - } - } else if len(cn.requests) <= cn.requestsLowWater { - filledBuffer := false - cn.iterPendingPieces(func(pieceIndex pieceIndex) bool { - cn.iterPendingRequests(pieceIndex, func(r Request) bool { - if !cn.setInterested(true) { - filledBuffer = true - return false - } - if len(cn.requests) >= cn.nominalMaxRequests() { - return false - } - // Choking is looked at here because our interest is dependent - // on whether we'd make requests in its absence. - if cn.peerChoking { - if !cn.peerAllowedFast.Get(bitmap.BitIndex(r.Index)) { - return false - } - } - if _, ok := cn.requests[r]; ok { - return true - } - filledBuffer = !cn.request(r) - return !filledBuffer - }) - return !filledBuffer - }) - if filledBuffer { - // If we didn't completely top up the requests, we shouldn't mark - // the low water, since we'll want to top up the requests as soon - // as we have more write buffer space. - return false - } - cn.requestsLowWater = len(cn.requests) / 2 - if len(cn.requests) == 0 { - return cn.setInterested(false) - } - } - return true -} - func (cn *PeerConn) fillWriteBuffer() { - if !cn.doRequestState() { - return - } if cn.pex.IsEnabled() { if flow := cn.pex.Share(cn.write); !flow { return @@ -743,10 +686,13 @@ func (cn *PeerConn) updateRequests() { func iterBitmapsDistinct(skip *bitmap.Bitmap, bms ...bitmap.Bitmap) iter.Func { return func(cb iter.Callback) { for _, bm := range bms { - bm.Sub(*skip) if !iter.All( - func(i interface{}) bool { - skip.Add(i.(int)) + func(_i interface{}) bool { + i := _i.(int) + if skip.Contains(i) { + return true + } + skip.Add(i) return cb(i) }, bm.Iter, @@ -757,62 +703,6 @@ func iterBitmapsDistinct(skip *bitmap.Bitmap, bms ...bitmap.Bitmap) iter.Func { } } -func iterUnbiasedPieceRequestOrder(cn requestStrategyConnection, f func(piece pieceIndex) bool) bool { - now, readahead := cn.torrent().readerPiecePriorities() - skip := bitmap.Flip(cn.peerPieces(), 0, cn.torrent().numPieces()) - skip.Union(cn.torrent().ignorePieces()) - // Return an iterator over the different priority classes, minus the skip pieces. - return iter.All( - func(_piece interface{}) bool { - return f(pieceIndex(_piece.(bitmap.BitIndex))) - }, - iterBitmapsDistinct(&skip, now, readahead), - // We have to iterate _pendingPieces separately because it isn't a Bitmap. - func(cb iter.Callback) { - cn.torrent().pendingPieces().IterTyped(func(piece int) bool { - if skip.Contains(piece) { - return true - } - more := cb(piece) - skip.Add(piece) - return more - }) - }, - ) -} - -// The connection should download highest priority pieces first, without any inclination toward -// avoiding wastage. Generally we might do this if there's a single connection, or this is the -// fastest connection, and we have active readers that signal an ordering preference. It's -// conceivable that the best connection should do this, since it's least likely to waste our time if -// assigned to the highest priority pieces, and assigning more than one this role would cause -// significant wasted bandwidth. -func (cn *Peer) shouldRequestWithoutBias() bool { - return cn.t.requestStrategy.shouldRequestWithoutBias(cn.requestStrategyConnection()) -} - -func (cn *Peer) iterPendingPieces(f func(pieceIndex) bool) { - if !cn.t.haveInfo() { - return - } - if cn.closed.IsSet() { - return - } - cn.t.requestStrategy.iterPendingPieces(cn, f) -} -func (cn *Peer) iterPendingPiecesUntyped(f iter.Callback) { - cn.iterPendingPieces(func(i pieceIndex) bool { return f(i) }) -} - -func (cn *Peer) iterPendingRequests(piece pieceIndex, f func(Request) bool) bool { - return cn.t.requestStrategy.iterUndirtiedChunks( - cn.t.piece(piece).requestStrategyPiece(), - func(cs ChunkSpec) bool { - return f(Request{pp.Integer(piece), cs}) - }, - ) -} - // check callers updaterequests func (cn *Peer) stopRequestingPiece(piece pieceIndex) bool { return cn._pieceRequestOrder.Remove(bitmap.BitIndex(piece)) @@ -831,8 +721,7 @@ func (cn *Peer) updatePiecePriority(piece pieceIndex) bool { return cn.stopRequestingPiece(piece) } prio := cn.getPieceInclination()[piece] - prio = cn.t.requestStrategy.piecePriority(cn, piece, tpp, prio) - return cn._pieceRequestOrder.Set(bitmap.BitIndex(piece), prio) || cn.shouldRequestWithoutBias() + return cn._pieceRequestOrder.Set(bitmap.BitIndex(piece), prio) } func (cn *Peer) getPieceInclination() []int { @@ -1571,7 +1460,6 @@ func (c *Peer) deleteRequest(r Request) bool { f(PeerRequestEvent{c, r}) } c.updateExpectingChunks() - c.t.requestStrategy.hooks().deletedRequest(r) pr := c.t.pendingRequests pr[r]-- n := pr[r] @@ -1722,10 +1610,6 @@ func (l connectionTrust) Less(r connectionTrust) bool { return multiless.New().Bool(l.Implicit, r.Implicit).Int64(l.NetGoodPiecesDirted, r.NetGoodPiecesDirted).Less() } -func (cn *Peer) requestStrategyConnection() requestStrategyConnection { - return cn -} - func (cn *Peer) chunksReceivedWhileExpecting() int64 { return cn._chunksReceivedWhileExpecting } @@ -1761,10 +1645,6 @@ func (cn *Peer) stats() *ConnStats { return &cn._stats } -func (cn *Peer) torrent() requestStrategyTorrent { - return cn.t.requestStrategyTorrent() -} - func (p *Peer) TryAsPeerConn() (*PeerConn, bool) { pc, ok := p.peerImpl.(*PeerConn) return pc, ok diff --git a/piece.go b/piece.go index fb9c8056..248832d0 100644 --- a/piece.go +++ b/piece.go @@ -144,7 +144,7 @@ func (p *Piece) chunkIndexSpec(chunk pp.Integer) ChunkSpec { func (p *Piece) chunkIndexRequest(chunkIndex pp.Integer) Request { return Request{ pp.Integer(p.index), - chunkIndexSpec(chunkIndex, p.length(), p.chunkSize()), + p.chunkIndexSpec(chunkIndex), } } @@ -259,10 +259,6 @@ func (p *Piece) allChunksDirty() bool { return p._dirtyChunks.Len() == int(p.numChunks()) } -func (p *Piece) requestStrategyPiece() requestStrategyPiece { - return p -} - func (p *Piece) dirtyChunks() bitmap.Bitmap { return p._dirtyChunks } @@ -270,3 +266,15 @@ func (p *Piece) dirtyChunks() bitmap.Bitmap { func (p *Piece) State() PieceState { return p.t.PieceState(p.index) } + +func (p *Piece) iterUndirtiedChunks(f func(ChunkSpec) bool) bool { + for i := pp.Integer(0); i < p.numChunks(); i++ { + if p.chunkIndexDirty(i) { + continue + } + if !f(p.chunkIndexSpec(i)) { + return false + } + } + return true +} diff --git a/request-strategy-defaults.go b/request-strategy-defaults.go index 1ece5fc6..10cbafc7 100644 --- a/request-strategy-defaults.go +++ b/request-strategy-defaults.go @@ -1,45 +1 @@ package torrent - -import ( - "github.com/anacrolix/missinggo/iter" - "github.com/anacrolix/missinggo/v2/bitmap" - pp "github.com/anacrolix/torrent/peer_protocol" -) - -// Provides default implementations for requestStrategy methods. Could be embedded, or delegated to. -type requestStrategyDefaults struct{} - -func (requestStrategyDefaults) hooks() requestStrategyHooks { - return requestStrategyHooks{ - sentRequest: func(Request) {}, - deletedRequest: func(Request) {}, - } -} - -func (requestStrategyDefaults) iterUndirtiedChunks(p requestStrategyPiece, f func(ChunkSpec) bool) bool { - chunkIndices := p.dirtyChunks().Copy() - chunkIndices.FlipRange(0, bitmap.BitIndex(p.numChunks())) - return iter.ForPerm(chunkIndices.Len(), func(i int) bool { - ci, err := chunkIndices.RB.Select(uint32(i)) - if err != nil { - panic(err) - } - return f(p.chunkIndexRequest(pp.Integer(ci)).ChunkSpec) - }) -} - -func (requestStrategyDefaults) nominalMaxRequests(cn requestStrategyConnection) int { - return int( - max( - 64, - cn.stats().ChunksReadUseful.Int64()- - (cn.stats().ChunksRead.Int64()-cn.stats().ChunksReadUseful.Int64()))) -} - -func (requestStrategyDefaults) piecePriority(cn requestStrategyConnection, piece pieceIndex, tpp piecePriority, prio int) int { - return prio -} - -func (requestStrategyDefaults) shouldRequestWithoutBias(cn requestStrategyConnection) bool { - return false -} diff --git a/request-strategy.go b/request-strategy.go index 75cff963..62e7174f 100644 --- a/request-strategy.go +++ b/request-strategy.go @@ -1,223 +1,162 @@ package torrent import ( - "math" - "sync" + "sort" "time" - "github.com/anacrolix/missinggo/v2/bitmap" - "github.com/anacrolix/missinggo/v2/prioritybitmap" - + "github.com/anacrolix/log" + "github.com/anacrolix/multiless" pp "github.com/anacrolix/torrent/peer_protocol" + "github.com/bradfitz/iter" ) -type requestStrategyPiece interface { - numChunks() pp.Integer - dirtyChunks() bitmap.Bitmap - chunkIndexRequest(i pp.Integer) Request +type clientPieceRequestOrder struct { + pieces []pieceRequestOrderPiece } -type requestStrategyTorrent interface { - numConns() int - numReaders() int - numPieces() int - readerPiecePriorities() (now, readahead bitmap.Bitmap) - ignorePieces() bitmap.Bitmap - pendingPieces() *prioritybitmap.PriorityBitmap +type pieceRequestOrderPiece struct { + t *Torrent + index pieceIndex + prio piecePriority + partial bool + availability int } -type requestStrategyConnection interface { - torrent() requestStrategyTorrent - peerPieces() bitmap.Bitmap - pieceRequestOrder() *prioritybitmap.PriorityBitmap - fastest() bool - stats() *ConnStats - totalExpectingTime() time.Duration - peerMaxRequests() int - chunksReceivedWhileExpecting() int64 -} - -type requestStrategy interface { - iterPendingPieces(requestStrategyConnection, func(pieceIndex) bool) bool - iterUndirtiedChunks(requestStrategyPiece, func(ChunkSpec) bool) bool - nominalMaxRequests(requestStrategyConnection) int - shouldRequestWithoutBias(requestStrategyConnection) bool - piecePriority(requestStrategyConnection, pieceIndex, piecePriority, int) int - hooks() requestStrategyHooks -} - -type requestStrategyHooks struct { - sentRequest func(Request) - deletedRequest func(Request) -} - -type requestStrategyCallbacks interface { - requestTimedOut(Request) -} - -type requestStrategyFuzzing struct { - requestStrategyDefaults -} - -type requestStrategyFastest struct { - requestStrategyDefaults -} - -func newRequestStrategyMaker(rs requestStrategy) requestStrategyMaker { - return func(requestStrategyCallbacks, sync.Locker) requestStrategy { - return rs - } -} - -// The fastest connection downloads strictly in order of priority, while all others adhere to their -// piece inclinations. -func RequestStrategyFastest() requestStrategyMaker { - return newRequestStrategyMaker(requestStrategyFastest{}) -} - -// Favour higher priority pieces with some fuzzing to reduce overlaps and wastage across -// connections. -func RequestStrategyFuzzing() requestStrategyMaker { - return newRequestStrategyMaker(requestStrategyFuzzing{}) -} - -func (requestStrategyFastest) shouldRequestWithoutBias(cn requestStrategyConnection) bool { - if cn.torrent().numReaders() == 0 { - return false - } - if cn.torrent().numConns() == 1 { - return true - } - if cn.fastest() { - return true - } - return false -} - -type requestStrategyDuplicateRequestTimeout struct { - requestStrategyDefaults - // How long to avoid duplicating a pending request. - duplicateRequestTimeout time.Duration - - callbacks requestStrategyCallbacks - - // The last time we requested a chunk. Deleting the request from any connection will clear this - // value. - lastRequested map[Request]*time.Timer - // The lock to take when running a request timeout handler. - timeoutLocker sync.Locker -} - -// Generates a request strategy instance for a given torrent. callbacks are probably specific to the torrent. -type requestStrategyMaker func(callbacks requestStrategyCallbacks, clientLocker sync.Locker) requestStrategy - -// Requests are strictly by piece priority, and not duplicated until duplicateRequestTimeout is -// reached. -func RequestStrategyDuplicateRequestTimeout(duplicateRequestTimeout time.Duration) requestStrategyMaker { - return func(callbacks requestStrategyCallbacks, clientLocker sync.Locker) requestStrategy { - return requestStrategyDuplicateRequestTimeout{ - duplicateRequestTimeout: duplicateRequestTimeout, - callbacks: callbacks, - lastRequested: make(map[Request]*time.Timer), - timeoutLocker: clientLocker, - } - } -} - -func (rs requestStrategyDuplicateRequestTimeout) hooks() requestStrategyHooks { - return requestStrategyHooks{ - deletedRequest: func(r Request) { - if t, ok := rs.lastRequested[r]; ok { - t.Stop() - delete(rs.lastRequested, r) - } - }, - sentRequest: rs.onSentRequest, - } -} - -func (rs requestStrategyDuplicateRequestTimeout) iterUndirtiedChunks(p requestStrategyPiece, f func(ChunkSpec) bool) bool { - for i := pp.Integer(0); i < pp.Integer(p.numChunks()); i++ { - if p.dirtyChunks().Get(bitmap.BitIndex(i)) { - continue - } - r := p.chunkIndexRequest(i) - if rs.wouldDuplicateRecent(r) { - continue - } - if !f(r.ChunkSpec) { - return false - } - } - return true -} - -func (requestStrategyFuzzing) piecePriority(cn requestStrategyConnection, piece pieceIndex, tpp piecePriority, prio int) int { - switch tpp { - case PiecePriorityNormal: - case PiecePriorityReadahead: - prio -= int(cn.torrent().numPieces()) - case PiecePriorityNext, PiecePriorityNow: - prio -= 2 * int(cn.torrent().numPieces()) - default: - panic(tpp) - } - prio += int(piece / 3) - return prio -} - -func (requestStrategyDuplicateRequestTimeout) iterPendingPieces(cn requestStrategyConnection, f func(pieceIndex) bool) bool { - return iterUnbiasedPieceRequestOrder(cn, f) -} -func defaultIterPendingPieces(rs requestStrategy, cn requestStrategyConnection, f func(pieceIndex) bool) bool { - if rs.shouldRequestWithoutBias(cn) { - return iterUnbiasedPieceRequestOrder(cn, f) - } else { - return cn.pieceRequestOrder().IterTyped(func(i int) bool { - return f(pieceIndex(i)) +func (me *clientPieceRequestOrder) addPieces(t *Torrent, numPieces pieceIndex) { + for i := range iter.N(numPieces) { + me.pieces = append(me.pieces, pieceRequestOrderPiece{ + t: t, + index: i, }) } } -func (rs requestStrategyFuzzing) iterPendingPieces(cn requestStrategyConnection, cb func(pieceIndex) bool) bool { - return defaultIterPendingPieces(rs, cn, cb) -} -func (rs requestStrategyFastest) iterPendingPieces(cn requestStrategyConnection, cb func(pieceIndex) bool) bool { - return defaultIterPendingPieces(rs, cn, cb) -} -func (rs requestStrategyDuplicateRequestTimeout) onSentRequest(r Request) { - rs.lastRequested[r] = time.AfterFunc(rs.duplicateRequestTimeout, func() { - rs.timeoutLocker.Lock() - delete(rs.lastRequested, r) - rs.timeoutLocker.Unlock() - rs.callbacks.requestTimedOut(r) - }) -} - -// The actual value to use as the maximum outbound requests. -func (rs requestStrategyDuplicateRequestTimeout) nominalMaxRequests(cn requestStrategyConnection) (ret int) { - expectingTime := int64(cn.totalExpectingTime()) - if expectingTime == 0 { - expectingTime = math.MaxInt64 - } else { - expectingTime *= 2 +func (me *clientPieceRequestOrder) removePieces(t *Torrent) { + newPieces := make([]pieceRequestOrderPiece, 0, len(me.pieces)-t.numPieces()) + for _, p := range me.pieces { + if p.t != t { + newPieces = append(newPieces, p) + } } - return int(clamp( - 1, - int64(cn.peerMaxRequests()), - max( - // It makes sense to always pipeline at least one connection, since latency must be - // non-zero. - 2, - // Request only as many as we expect to receive in the duplicateRequestTimeout - // window. We are trying to avoid having to duplicate requests. - cn.chunksReceivedWhileExpecting()*int64(rs.duplicateRequestTimeout)/expectingTime, - ), - )) + me.pieces = newPieces } -func (rs requestStrategyDuplicateRequestTimeout) wouldDuplicateRecent(r Request) bool { - // This piece has been requested on another connection, and the duplicate request timer is still - // running. - _, ok := rs.lastRequested[r] - return ok + +func (me clientPieceRequestOrder) sort() { + sort.SliceStable(me.pieces, me.less) } + +func (me clientPieceRequestOrder) update() { + for i := range me.pieces { + p := &me.pieces[i] + p.prio = p.t.piece(p.index).uncachedPriority() + p.partial = p.t.piecePartiallyDownloaded(p.index) + p.availability = p.t.pieceAvailability(p.index) + } +} + +func (me clientPieceRequestOrder) less(_i, _j int) bool { + i := me.pieces[_i] + j := me.pieces[_j] + ml := multiless.New() + ml.Int(int(j.prio), int(i.prio)) + ml.Bool(j.partial, i.partial) + ml.Int(i.availability, j.availability) + return ml.Less() +} + +func (cl *Client) requester() { + for { + func() { + cl.lock() + defer cl.unlock() + cl.doRequests() + }() + select { + case <-cl.closed.LockedChan(cl.locker()): + return + case <-time.After(10 * time.Millisecond): + } + } +} + +func (cl *Client) doRequests() { + requestOrder := clientPieceRequestOrder{} + allPeers := make(map[*Torrent][]*Peer) + storageCapacity := make(map[*Torrent]*int64) + for _, t := range cl.torrents { + // TODO: We could do metainfo requests here. + if t.haveInfo() { + value := int64(t.usualPieceSize()) + storageCapacity[t] = &value + requestOrder.addPieces(t, t.numPieces()) + } + var peers []*Peer + t.iterPeers(func(p *Peer) { + peers = append(peers, p) + }) + allPeers[t] = peers + } + requestOrder.update() + requestOrder.sort() + for _, p := range requestOrder.pieces { + if p.t.ignorePieceForRequests(p.index) { + continue + } + peers := allPeers[p.t] + torrentPiece := p.t.piece(p.index) + if left := storageCapacity[p.t]; left != nil { + if *left < int64(torrentPiece.length()) { + continue + } + *left -= int64(torrentPiece.length()) + } + p.t.piece(p.index).iterUndirtiedChunks(func(chunk ChunkSpec) bool { + for _, peer := range peers { + req := Request{pp.Integer(p.index), chunk} + _, err := peer.request(req) + if err == nil { + log.Printf("requested %v", req) + break + } + } + return true + }) + } + for _, t := range cl.torrents { + t.iterPeers(func(p *Peer) { + if !p.peerChoking && p.numLocalRequests() == 0 && !p.writeBufferFull() { + p.setInterested(false) + } + }) + } +} + +//func (requestStrategyDefaults) iterUndirtiedChunks(p requestStrategyPiece, f func(ChunkSpec) bool) bool { +// chunkIndices := p.dirtyChunks().Copy() +// chunkIndices.FlipRange(0, bitmap.BitIndex(p.numChunks())) +// return iter.ForPerm(chunkIndices.Len(), func(i int) bool { +// ci, err := chunkIndices.RB.Select(uint32(i)) +// if err != nil { +// panic(err) +// } +// return f(p.chunkIndexRequest(pp.Integer(ci)).ChunkSpec) +// }) +//} + +// +//func iterUnbiasedPieceRequestOrder( +// cn requestStrategyConnection, +// f func(piece pieceIndex) bool, +// pieceRequestOrder []pieceIndex, +//) bool { +// cn.torrent().sortPieceRequestOrder(pieceRequestOrder) +// for _, i := range pieceRequestOrder { +// if !cn.peerHasPiece(i) || cn.torrent().ignorePieceForRequests(i) { +// continue +// } +// if !f(i) { +// return false +// } +// } +// return true +//} diff --git a/torrent.go b/torrent.go index e3876474..e8cf36e4 100644 --- a/torrent.go +++ b/torrent.go @@ -55,9 +55,6 @@ type Torrent struct { dataUploadDisallowed bool userOnWriteChunkErr func(error) - // Determines what chunks to request from peers. - requestStrategy requestStrategy - closed missinggo.Event infoHash metainfo.Hash pieces []Piece @@ -150,6 +147,29 @@ type Torrent struct { pex pexState } +func (t *Torrent) pieceAvailability(i pieceIndex) (count int) { + t.iterPeers(func(peer *Peer) { + if peer.peerHasPiece(i) { + count++ + } + }) + return +} + +func (t *Torrent) sortPieceRequestOrder(sl []pieceIndex) { + if len(sl) != t.numPieces() { + panic(len(sl)) + } + availability := make([]int, len(sl)) + t.iterPeers(func(peer *Peer) { + for i := range availability { + if peer.peerHasPiece(i) { + availability[i]++ + } + } + }) +} + func (t *Torrent) numConns() int { return len(t.conns) } @@ -166,15 +186,8 @@ func (t *Torrent) readerReadaheadPieces() bitmap.Bitmap { return t._readerReadaheadPieces } -func (t *Torrent) ignorePieces() bitmap.Bitmap { - ret := t._completedPieces.Copy() - ret.Union(t.piecesQueuedForHash) - for i := 0; i < t.numPieces(); i++ { - if t.piece(i).hashing { - ret.Set(i, true) - } - } - return ret +func (t *Torrent) ignorePieceForRequests(i pieceIndex) bool { + return !t.wantPieceIndex(i) } func (t *Torrent) pendingPieces() *prioritybitmap.PriorityBitmap { @@ -413,6 +426,7 @@ func (t *Torrent) setInfo(info *metainfo.Info) error { // This seems to be all the follow-up tasks after info is set, that can't fail. func (t *Torrent) onSetInfo() { + t.cl.clientPieceRequestOrder.addPieces(t, t.numPieces()) t.iterPeers(func(p *Peer) { p.onGotInfo(t.info) }) @@ -2026,30 +2040,6 @@ func (t *Torrent) piece(i int) *Piece { return &t.pieces[i] } -func (t *Torrent) requestStrategyTorrent() requestStrategyTorrent { - return t -} - -type torrentRequestStrategyCallbacks struct { - t *Torrent -} - -func (cb torrentRequestStrategyCallbacks) requestTimedOut(r Request) { - torrent.Add("Request timeouts", 1) - cb.t.cl.lock() - defer cb.t.cl.unlock() - cb.t.iterPeers(func(cn *Peer) { - if cn.peerHasPiece(pieceIndex(r.Index)) { - cn.updateRequests() - } - }) - -} - -func (t *Torrent) requestStrategyCallbacks() requestStrategyCallbacks { - return torrentRequestStrategyCallbacks{t} -} - func (t *Torrent) onWriteChunkErr(err error) { if t.userOnWriteChunkErr != nil { go t.userOnWriteChunkErr(err) @@ -2111,7 +2101,7 @@ func (t *Torrent) SetOnWriteChunkError(f func(error)) { t.userOnWriteChunkErr = f } -func (t *Torrent) iterPeers(f func(*Peer)) { +func (t *Torrent) iterPeers(f func(p *Peer)) { for pc := range t.conns { f(&pc.Peer) } diff --git a/webseed-peer.go b/webseed-peer.go index e2df582e..9fa77a28 100644 --- a/webseed-peer.go +++ b/webseed-peer.go @@ -24,6 +24,10 @@ type webseedPeer struct { var _ peerImpl = (*webseedPeer)(nil) +func (me *webseedPeer) writeBufferFull() bool { + return false +} + func (me *webseedPeer) connStatusString() string { return me.client.Url } @@ -99,7 +103,6 @@ func (ws *webseedPeer) connectionFlags() string { func (ws *webseedPeer) drop() {} func (ws *webseedPeer) updateRequests() { - ws.peer.doRequestState() } func (ws *webseedPeer) onClose() { From b2c68b314bb62ef831a83e4fefbf21bb8bddaa1e Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Sun, 9 May 2021 14:41:03 +1000 Subject: [PATCH 02/59] Limit outstanding requests --- peerconn.go | 3 +++ request-strategy.go | 3 +-- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/peerconn.go b/peerconn.go index 9522adf3..1ce06994 100644 --- a/peerconn.go +++ b/peerconn.go @@ -533,6 +533,9 @@ func (cn *Peer) request(r Request) (more bool, err error) { if _, ok := cn.requests[r]; ok { return true, nil } + if cn.numLocalRequests() >= cn.nominalMaxRequests() { + return true, errors.New("too many outstanding requests") + } if !cn.peerHasPiece(pieceIndex(r.Index)) { return true, errors.New("requesting piece peer doesn't have") } diff --git a/request-strategy.go b/request-strategy.go index 62e7174f..d762d0f5 100644 --- a/request-strategy.go +++ b/request-strategy.go @@ -4,7 +4,6 @@ import ( "sort" "time" - "github.com/anacrolix/log" "github.com/anacrolix/multiless" pp "github.com/anacrolix/torrent/peer_protocol" "github.com/bradfitz/iter" @@ -115,7 +114,7 @@ func (cl *Client) doRequests() { req := Request{pp.Integer(p.index), chunk} _, err := peer.request(req) if err == nil { - log.Printf("requested %v", req) + //log.Printf("requested %v", req) break } } From 5f437e6b7f2703efefd2cc99830e0e7f053cb80d Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Sun, 9 May 2021 23:34:56 +1000 Subject: [PATCH 03/59] Remove empty file --- request-strategy-defaults.go | 1 - 1 file changed, 1 deletion(-) delete mode 100644 request-strategy-defaults.go diff --git a/request-strategy-defaults.go b/request-strategy-defaults.go deleted file mode 100644 index 10cbafc7..00000000 --- a/request-strategy-defaults.go +++ /dev/null @@ -1 +0,0 @@ -package torrent From 6e97ce952f26a41e30c2a0049a9073d39659110d Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Sun, 9 May 2021 23:38:38 +1000 Subject: [PATCH 04/59] Tidy up request and cancel --- peer-impl.go | 10 ++++--- peerconn.go | 76 ++++++++++++++++++------------------------------- webseed-peer.go | 15 +++------- 3 files changed, 37 insertions(+), 64 deletions(-) diff --git a/peer-impl.go b/peer-impl.go index f4ad12a2..880b8f35 100644 --- a/peer-impl.go +++ b/peer-impl.go @@ -10,12 +10,14 @@ import ( type peerImpl interface { updateRequests() writeInterested(interested bool) bool - cancel(Request) bool - // Return true if there's room for more activity. - request(Request) bool + + // Neither of these return buffer room anymore, because they're currently both posted. There's + // also PeerConn.writeBufferFull for when/where it matters. + _cancel(Request) + _request(Request) + connectionFlags() string onClose() - _postCancel(Request) onGotInfo(*metainfo.Info) drop() String() string diff --git a/peerconn.go b/peerconn.go index 1ce06994..aa31eec8 100644 --- a/peerconn.go +++ b/peerconn.go @@ -529,15 +529,15 @@ func (pc *PeerConn) writeInterested(interested bool) bool { // are okay. type messageWriter func(pp.Message) bool -func (cn *Peer) request(r Request) (more bool, err error) { +func (cn *Peer) request(r Request) error { if _, ok := cn.requests[r]; ok { - return true, nil + return nil } if cn.numLocalRequests() >= cn.nominalMaxRequests() { - return true, errors.New("too many outstanding requests") + return errors.New("too many outstanding requests") } if !cn.peerHasPiece(pieceIndex(r.Index)) { - return true, errors.New("requesting piece peer doesn't have") + return errors.New("requesting piece peer doesn't have") } if !cn.t.peerIsActive(cn) { panic("requesting but not in active conns") @@ -545,19 +545,22 @@ func (cn *Peer) request(r Request) (more bool, err error) { if cn.closed.IsSet() { panic("requesting when connection is closed") } - if cn.peerChoking { - if cn.peerAllowedFast.Get(int(r.Index)) { - torrent.Add("allowed fast requests sent", 1) - } else { - return cn.setInterested(true), errors.New("requesting while choked and not allowed fast") - } - } if cn.t.hashingPiece(pieceIndex(r.Index)) { panic("piece is being hashed") } if cn.t.pieceQueuedForHash(pieceIndex(r.Index)) { panic("piece is queued for hash") } + if !cn.setInterested(true) { + return errors.New("write buffer full after expressing interest") + } + if cn.peerChoking { + if cn.peerAllowedFast.Get(int(r.Index)) { + torrent.Add("allowed fast requests sent", 1) + } else { + errors.New("peer choking and piece not in allowed fast set") + } + } if cn.requests == nil { cn.requests = make(map[Request]struct{}) } @@ -571,11 +574,12 @@ func (cn *Peer) request(r Request) (more bool, err error) { for _, f := range cn.callbacks.SentRequest { f(PeerRequestEvent{cn, r}) } - return cn.peerImpl.request(r), nil + cn.peerImpl._request(r) + return nil } -func (me *PeerConn) request(r Request) bool { - return me.write(pp.Message{ +func (me *PeerConn) _request(r Request) { + me.write(pp.Message{ Type: pp.Request, Index: r.Index, Begin: r.Begin, @@ -583,8 +587,14 @@ func (me *PeerConn) request(r Request) bool { }) } -func (me *PeerConn) cancel(r Request) bool { - return me.write(makeCancelMessage(r)) +func (me *Peer) cancel(r Request) { + if me.deleteRequest(r) { + me.peerImpl._cancel(r) + } +} + +func (me *PeerConn) _cancel(r Request) { + me.write(makeCancelMessage(r)) } func (cn *PeerConn) fillWriteBuffer() { @@ -1317,7 +1327,7 @@ func (c *Peer) receiveChunk(msg *pp.Message) error { if p == c { return } - p.postCancel(req) + p.cancel(req) }) err := func() error { @@ -1472,26 +1482,6 @@ func (c *Peer) deleteRequest(r Request) bool { if n < 0 { panic(n) } - // If a request fails, updating the requests for the current peer first may miss the opportunity - // to try other peers for that request instead, depending on the request strategy. This might - // only affect webseed peers though, since they synchronously issue new requests: PeerConns do - // it in the writer routine. - const updateCurrentConnRequestsFirst = false - if updateCurrentConnRequestsFirst { - c.updateRequests() - } - // Give other conns a chance to pick up the request. - c.t.iterPeers(func(_c *Peer) { - // We previously checked that the peer wasn't interested to to only wake connections that - // were unable to issue requests due to starvation by the request strategy. There could be - // performance ramifications. - if _c != c && c.peerHasPiece(pieceIndex(r.Index)) { - _c.updateRequests() - } - }) - if !updateCurrentConnRequestsFirst { - c.updateRequests() - } return true } @@ -1513,18 +1503,6 @@ func (c *PeerConn) tickleWriter() { c.writerCond.Broadcast() } -func (c *Peer) postCancel(r Request) bool { - if !c.deleteRequest(r) { - return false - } - c.peerImpl._postCancel(r) - return true -} - -func (c *PeerConn) _postCancel(r Request) { - c.post(makeCancelMessage(r)) -} - func (c *PeerConn) sendChunk(r Request, msg func(pp.Message) bool, state *peerRequestState) (more bool) { c.lastChunkSent = time.Now() return msg(pp.Message{ diff --git a/webseed-peer.go b/webseed-peer.go index 9fa77a28..5f2980c3 100644 --- a/webseed-peer.go +++ b/webseed-peer.go @@ -41,30 +41,23 @@ func (ws *webseedPeer) onGotInfo(info *metainfo.Info) { ws.client.Info = info } -func (ws *webseedPeer) _postCancel(r Request) { - ws.cancel(r) -} - func (ws *webseedPeer) writeInterested(interested bool) bool { return true } -func (ws *webseedPeer) cancel(r Request) bool { +func (ws *webseedPeer) _cancel(r Request) { active, ok := ws.activeRequests[r] - if !ok { - return false + if ok { + active.Cancel() } - active.Cancel() - return true } func (ws *webseedPeer) intoSpec(r Request) webseed.RequestSpec { return webseed.RequestSpec{ws.peer.t.requestOffset(r), int64(r.Length)} } -func (ws *webseedPeer) request(r Request) bool { +func (ws *webseedPeer) _request(r Request) { ws.requesterCond.Signal() - return true } func (ws *webseedPeer) doRequest(r Request) { From 5f8471e21b69e6a632ec2d74183c30c88ab6d294 Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Sun, 9 May 2021 23:40:44 +1000 Subject: [PATCH 05/59] Rework storage.TorrentImpl to support shared capacity key --- bad_storage.go | 8 +++----- peerconn_test.go | 2 +- storage/bolt.go | 6 +++++- storage/file.go | 17 ++++++++++++----- storage/interface.go | 8 +++++--- storage/mmap.go | 6 +++--- storage/piece-resource.go | 6 ++++-- storage/sqlite/direct.go | 19 ++++++++++++++++++- test/issue377_test.go | 2 +- test/transfer_test.go | 38 ++++++++++++++++---------------------- torrent.go | 4 +++- 11 files changed, 71 insertions(+), 45 deletions(-) diff --git a/bad_storage.go b/bad_storage.go index f984c639..fc15beb9 100644 --- a/bad_storage.go +++ b/bad_storage.go @@ -15,11 +15,9 @@ type badStorage struct{} var _ storage.ClientImpl = badStorage{} func (bs badStorage) OpenTorrent(*metainfo.Info, metainfo.Hash) (storage.TorrentImpl, error) { - return bs, nil -} - -func (bs badStorage) Close() error { - return nil + return storage.TorrentImpl{ + Piece: bs.Piece, + }, nil } func (bs badStorage) Piece(p metainfo.Piece) storage.PieceImpl { diff --git a/peerconn_test.go b/peerconn_test.go index 7057e77d..18fc98ef 100644 --- a/peerconn_test.go +++ b/peerconn_test.go @@ -98,7 +98,7 @@ func BenchmarkConnectionMainReadLoop(b *testing.B) { ts := &torrentStorage{} t := &Torrent{ cl: cl, - storage: &storage.Torrent{TorrentImpl: ts}, + storage: &storage.Torrent{TorrentImpl: storage.TorrentImpl{Piece: ts.Piece, Close: ts.Close}}, pieceStateChanges: pubsub.NewPubSub(), } require.NoError(b, t.setInfo(&metainfo.Info{ diff --git a/storage/bolt.go b/storage/bolt.go index 5104c682..025c5665 100644 --- a/storage/bolt.go +++ b/storage/bolt.go @@ -43,7 +43,11 @@ func (me *boltClient) Close() error { } func (me *boltClient) OpenTorrent(info *metainfo.Info, infoHash metainfo.Hash) (TorrentImpl, error) { - return &boltTorrent{me, infoHash}, nil + t := &boltTorrent{me, infoHash} + return TorrentImpl{ + Piece: t.Piece, + Close: t.Close, + }, nil } func (me *boltTorrent) Piece(p metainfo.Piece) PieceImpl { diff --git a/storage/file.go b/storage/file.go index 1a273a34..d51a7f36 100644 --- a/storage/file.go +++ b/storage/file.go @@ -67,14 +67,16 @@ func (me *fileClientImpl) Close() error { return me.pc.Close() } -func (fs *fileClientImpl) OpenTorrent(info *metainfo.Info, infoHash metainfo.Hash) (TorrentImpl, error) { +func (fs *fileClientImpl) OpenTorrent(info *metainfo.Info, infoHash metainfo.Hash) (_ TorrentImpl, err error) { dir := fs.pathMaker(fs.baseDir, info, infoHash) upvertedFiles := info.UpvertedFiles() files := make([]file, 0, len(upvertedFiles)) for i, fileInfo := range upvertedFiles { - s, err := ToSafeFilePath(append([]string{info.Name}, fileInfo.Path...)...) + var s string + s, err = ToSafeFilePath(append([]string{info.Name}, fileInfo.Path...)...) if err != nil { - return nil, fmt.Errorf("file %v has unsafe path %q: %w", i, fileInfo.Path, err) + err = fmt.Errorf("file %v has unsafe path %q: %w", i, fileInfo.Path, err) + return } f := file{ path: filepath.Join(dir, s), @@ -83,16 +85,21 @@ func (fs *fileClientImpl) OpenTorrent(info *metainfo.Info, infoHash metainfo.Has if f.length == 0 { err = CreateNativeZeroLengthFile(f.path) if err != nil { - return nil, fmt.Errorf("creating zero length file: %w", err) + err = fmt.Errorf("creating zero length file: %w", err) + return } } files = append(files, f) } - return &fileTorrentImpl{ + t := &fileTorrentImpl{ files, segments.NewIndex(common.LengthIterFromUpvertedFiles(upvertedFiles)), infoHash, fs.pc, + } + return TorrentImpl{ + Piece: t.Piece, + Close: t.Close, }, nil } diff --git a/storage/interface.go b/storage/interface.go index 869556f8..c48c6e34 100644 --- a/storage/interface.go +++ b/storage/interface.go @@ -17,9 +17,11 @@ type ClientImpl interface { } // Data storage bound to a torrent. -type TorrentImpl interface { - Piece(metainfo.Piece) PieceImpl - Close() error +type TorrentImpl struct { + Piece func(metainfo.Piece) PieceImpl + Close func() error + // Storages that share the same value, will provide a pointer to the same function. + Capacity *func() *int64 } // Interacts with torrent piece data. Optional interfaces to implement include io.WriterTo, such as diff --git a/storage/mmap.go b/storage/mmap.go index c4e5b09e..3d996d9e 100644 --- a/storage/mmap.go +++ b/storage/mmap.go @@ -30,14 +30,14 @@ func NewMMapWithCompletion(baseDir string, completion PieceCompletion) *mmapClie } } -func (s *mmapClientImpl) OpenTorrent(info *metainfo.Info, infoHash metainfo.Hash) (t TorrentImpl, err error) { +func (s *mmapClientImpl) OpenTorrent(info *metainfo.Info, infoHash metainfo.Hash) (_ TorrentImpl, err error) { span, err := mMapTorrent(info, s.baseDir) - t = &mmapTorrentStorage{ + t := &mmapTorrentStorage{ infoHash: infoHash, span: span, pc: s.pc, } - return + return TorrentImpl{Piece: t.Piece, Close: t.Close}, err } func (s *mmapClientImpl) Close() error { diff --git a/storage/piece-resource.go b/storage/piece-resource.go index d56280bc..ec3848df 100644 --- a/storage/piece-resource.go +++ b/storage/piece-resource.go @@ -26,6 +26,7 @@ type ResourcePiecesOpts struct { // Sized puts require being able to stream from a statement executed on another connection. // Without them, we buffer the entire read and then put that. NoSizedPuts bool + Capacity *int64 } func NewResourcePieces(p PieceProvider) ClientImpl { @@ -49,10 +50,11 @@ func (piecePerResourceTorrentImpl) Close() error { } func (s piecePerResource) OpenTorrent(info *metainfo.Info, infoHash metainfo.Hash) (TorrentImpl, error) { - return piecePerResourceTorrentImpl{ + t := piecePerResourceTorrentImpl{ s, make([]sync.RWMutex, info.NumPieces()), - }, nil + } + return TorrentImpl{Piece: t.Piece, Close: t.Close}, nil } func (s piecePerResourceTorrentImpl) Piece(p metainfo.Piece) PieceImpl { diff --git a/storage/sqlite/direct.go b/storage/sqlite/direct.go index d51e1321..7748f921 100644 --- a/storage/sqlite/direct.go +++ b/storage/sqlite/direct.go @@ -61,9 +61,24 @@ func NewDirectStorage(opts NewDirectStorageOpts) (_ storage.ClientImplCloser, er if opts.BlobFlushInterval != 0 { cl.blobFlusher = time.AfterFunc(opts.BlobFlushInterval, cl.blobFlusherFunc) } + cl.capacity = cl.getCapacity return cl, nil } +func (cl *client) getCapacity() (ret *int64) { + cl.l.Lock() + defer cl.l.Unlock() + err := sqlitex.Exec(cl.conn, "select value from setting where name='capacity'", func(stmt *sqlite.Stmt) error { + ret = new(int64) + *ret = stmt.ColumnInt64(0) + return nil + }) + if err != nil { + panic(err) + } + return +} + type client struct { l sync.Mutex conn conn @@ -71,6 +86,7 @@ type client struct { blobFlusher *time.Timer opts NewDirectStorageOpts closed bool + capacity func() *int64 } func (c *client) blobFlusherFunc() { @@ -91,7 +107,8 @@ func (c *client) flushBlobs() { } func (c *client) OpenTorrent(info *metainfo.Info, infoHash metainfo.Hash) (storage.TorrentImpl, error) { - return torrent{c}, nil + t := torrent{c} + return storage.TorrentImpl{Piece: t.Piece, Close: t.Close, Capacity: &c.capacity}, nil } func (c *client) Close() error { diff --git a/test/issue377_test.go b/test/issue377_test.go index bd8c4357..7456e9c4 100644 --- a/test/issue377_test.go +++ b/test/issue377_test.go @@ -124,7 +124,7 @@ func (me *diskFullStorage) Close() error { } func (d *diskFullStorage) OpenTorrent(info *metainfo.Info, infoHash metainfo.Hash) (storage.TorrentImpl, error) { - return d, nil + return storage.TorrentImpl{Piece: d.Piece, Close: d.Close}, nil } type pieceImpl struct { diff --git a/test/transfer_test.go b/test/transfer_test.go index e58bb53b..857cd44f 100644 --- a/test/transfer_test.go +++ b/test/transfer_test.go @@ -192,7 +192,6 @@ func testClientTransfer(t *testing.T, ps testClientTransferParams) { type fileCacheClientStorageFactoryParams struct { Capacity int64 SetCapacity bool - Wrapper func(*filecache.Cache) storage.ClientImplCloser } func newFileCacheClientStorageFactory(ps fileCacheClientStorageFactoryParams) storageFactory { @@ -201,10 +200,22 @@ func newFileCacheClientStorageFactory(ps fileCacheClientStorageFactoryParams) st if err != nil { panic(err) } + var sharedCapacity *int64 if ps.SetCapacity { + sharedCapacity = &ps.Capacity fc.SetCapacity(ps.Capacity) } - return ps.Wrapper(fc) + return struct { + storage.ClientImpl + io.Closer + }{ + storage.NewResourcePiecesOpts( + fc.AsResourceProvider(), + storage.ResourcePiecesOpts{ + Capacity: sharedCapacity, + }), + ioutil.NopCloser(nil), + } } } @@ -212,17 +223,13 @@ type storageFactory func(string) storage.ClientImplCloser func TestClientTransferDefault(t *testing.T) { testClientTransfer(t, testClientTransferParams{ - LeecherStorage: newFileCacheClientStorageFactory(fileCacheClientStorageFactoryParams{ - Wrapper: fileCachePieceResourceStorage, - }), + LeecherStorage: newFileCacheClientStorageFactory(fileCacheClientStorageFactoryParams{}), }) } func TestClientTransferDefaultNoMetadata(t *testing.T) { testClientTransfer(t, testClientTransferParams{ - LeecherStorage: newFileCacheClientStorageFactory(fileCacheClientStorageFactoryParams{ - Wrapper: fileCachePieceResourceStorage, - }), + LeecherStorage: newFileCacheClientStorageFactory(fileCacheClientStorageFactoryParams{}), LeecherStartsWithoutMetadata: true, }) } @@ -244,16 +251,6 @@ func TestClientTransferRateLimitedDownload(t *testing.T) { }) } -func fileCachePieceResourceStorage(fc *filecache.Cache) storage.ClientImplCloser { - return struct { - storage.ClientImpl - io.Closer - }{ - storage.NewResourcePieces(fc.AsResourceProvider()), - ioutil.NopCloser(nil), - } -} - func testClientTransferSmallCache(t *testing.T, setReadahead bool, readahead int64) { testClientTransfer(t, testClientTransferParams{ LeecherStorage: newFileCacheClientStorageFactory(fileCacheClientStorageFactoryParams{ @@ -261,7 +258,6 @@ func testClientTransferSmallCache(t *testing.T, setReadahead bool, readahead int // Going below the piece length means it can't complete a piece so // that it can be hashed. Capacity: 5, - Wrapper: fileCachePieceResourceStorage, }), SetReadahead: setReadahead, // Can't readahead too far or the cache will thrash and drop data we @@ -324,9 +320,7 @@ func sqliteLeecherStorageTestCase(numConns int) leecherStorageTestCase { func TestClientTransferVarious(t *testing.T) { // Leecher storage for _, ls := range []leecherStorageTestCase{ - {"Filecache", newFileCacheClientStorageFactory(fileCacheClientStorageFactoryParams{ - Wrapper: fileCachePieceResourceStorage, - }), 0}, + {"Filecache", newFileCacheClientStorageFactory(fileCacheClientStorageFactoryParams{}), 0}, {"Boltdb", storage.NewBoltDB, 0}, {"SqliteDirect", func(s string) storage.ClientImplCloser { path := filepath.Join(s, "sqlite3.db") diff --git a/torrent.go b/torrent.go index e8cf36e4..b6a6f2b7 100644 --- a/torrent.go +++ b/torrent.go @@ -760,7 +760,9 @@ func (t *Torrent) close() (err error) { func() { t.storageLock.Lock() defer t.storageLock.Unlock() - t.storage.Close() + if f := t.storage.Close; f != nil { + f() + } }() } t.iterPeers(func(p *Peer) { From 56e2a8a3a6e5fbfd27a09e685c8524b3c43dc7cd Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Mon, 10 May 2021 00:53:32 +1000 Subject: [PATCH 06/59] Fix download rate, status output --- client.go | 11 +++--- conn_stats.go | 7 ++-- peerconn.go | 88 ++++++++++++++++++++++++++------------------- request-strategy.go | 77 ++++++++++++++++++++++++++++++--------- torrent.go | 29 ++++++--------- 5 files changed, 134 insertions(+), 78 deletions(-) diff --git a/client.go b/client.go index b0f537ca..dc850a9e 100644 --- a/client.go +++ b/client.go @@ -968,6 +968,10 @@ func (cl *Client) runHandshookConn(c *PeerConn, t *Torrent) error { return nil } +// If peer requests are buffered on read, this instructs the amount of memory that might be used to +// cache pending writes. Assuming 512KiB cached for sending, for 16KiB chunks. +const localClientReqq = 1 << 5 + // See the order given in Transmission's tr_peerMsgsNew. func (cl *Client) sendInitialMessages(conn *PeerConn, torrent *Torrent) { if conn.PeerExtensionBytes.SupportsExtended() && cl.config.Extensions.SupportsExtended() { @@ -979,11 +983,8 @@ func (cl *Client) sendInitialMessages(conn *PeerConn, torrent *Torrent) { M: map[pp.ExtensionName]pp.ExtensionNumber{ pp.ExtensionNameMetadata: metadataExtendedId, }, - V: cl.config.ExtendedHandshakeClientVersion, - // If peer requests are buffered on read, this instructs the amount of memory - // that might be used to cache pending writes. Assuming 512KiB cached for - // sending, for 16KiB chunks. - Reqq: 1 << 5, + V: cl.config.ExtendedHandshakeClientVersion, + Reqq: localClientReqq, YourIp: pp.CompactIp(conn.remoteIp()), Encryption: cl.config.HeaderObfuscationPolicy.Preferred || !cl.config.HeaderObfuscationPolicy.RequirePreferred, Port: cl.incomingPeerPort(), diff --git a/conn_stats.go b/conn_stats.go index 3fbc00d8..d2d52003 100644 --- a/conn_stats.go +++ b/conn_stats.go @@ -20,9 +20,10 @@ type ConnStats struct { BytesWritten Count BytesWrittenData Count - BytesRead Count - BytesReadData Count - BytesReadUsefulData Count + BytesRead Count + BytesReadData Count + BytesReadUsefulData Count + BytesReadUsefulIntendedData Count ChunksWritten Count diff --git a/peerconn.go b/peerconn.go index aa31eec8..48b305a2 100644 --- a/peerconn.go +++ b/peerconn.go @@ -167,7 +167,21 @@ func (cn *Peer) updateExpectingChunks() { } func (cn *Peer) expectingChunks() bool { - return len(cn.requests) != 0 && !cn.peerChoking + if len(cn.requests) == 0 { + return false + } + if !cn.interested { + return false + } + if !cn.peerChoking { + return true + } + for r := range cn.requests { + if cn.peerAllowedFast.Contains(bitmap.BitIndex(r.Index)) { + return true + } + } + return false } // Returns true if the connection is over IPv6. @@ -300,14 +314,20 @@ func (cn *Peer) statusFlags() (ret string) { return } -// func (cn *connection) String() string { -// var buf bytes.Buffer -// cn.writeStatus(&buf, nil) -// return buf.String() -// } - func (cn *Peer) downloadRate() float64 { - return float64(cn._stats.BytesReadUsefulData.Int64()) / cn.cumInterest().Seconds() + num := cn._stats.BytesReadUsefulData.Int64() + if num == 0 { + return 0 + } + return float64(num) / cn.totalExpectingTime().Seconds() +} + +func (cn *Peer) numRequestsByPiece() (ret map[pieceIndex]int) { + ret = make(map[pieceIndex]int) + for r := range cn.requests { + ret[pieceIndex(r.Index)]++ + } + return } func (cn *Peer) writeStatus(w io.Writer, t *Torrent) { @@ -316,6 +336,12 @@ func (cn *Peer) writeStatus(w io.Writer, t *Torrent) { fmt.Fprint(w, "CLOSED: ") } fmt.Fprintln(w, cn.connStatusString()) + prio, err := cn.peerPriority() + prioStr := fmt.Sprintf("%08x", prio) + if err != nil { + prioStr += ": " + err.Error() + } + fmt.Fprintf(w, " bep40-prio: %v\n", prioStr) fmt.Fprintf(w, " last msg: %s, connected: %s, last helpful: %s, itime: %s, etime: %s\n", eventAgeString(cn.lastMessageReceived), eventAgeString(cn.completedHandshake), @@ -324,29 +350,25 @@ func (cn *Peer) writeStatus(w io.Writer, t *Torrent) { cn.totalExpectingTime(), ) fmt.Fprintf(w, - " %s completed, %d pieces touched, good chunks: %v/%v-%v reqq: (%d,%d,%d]-%d, flags: %s, dr: %.1f KiB/s\n", + " %s completed, %d pieces touched, good chunks: %v/%v-%v reqq: %d/(%d/%d)-%d/%d, flags: %s, dr: %.1f KiB/s\n", cn.completedString(), len(cn.peerTouchedPieces), &cn._stats.ChunksReadUseful, &cn._stats.ChunksRead, &cn._stats.ChunksWritten, - cn.requestsLowWater, cn.numLocalRequests(), cn.nominalMaxRequests(), + cn.PeerMaxRequests, len(cn.peerRequests), + localClientReqq, cn.statusFlags(), cn.downloadRate()/(1<<10), ) - //fmt.Fprintf(w, " next pieces: %v%s\n", - // iter.ToSlice(iter.Head(10, cn.iterPendingPiecesUntyped)), - // func() string { - // if cn == t.fastestPeer { - // return " (fastest)" - // } else { - // return "" - // } - // }(), - //) + fmt.Fprintf(w, " requested pieces:") + for piece, count := range cn.numRequestsByPiece() { + fmt.Fprintf(w, " %v (%v)", piece, count) + } + fmt.Fprintf(w, "\n") } func (cn *Peer) close() { @@ -445,7 +467,7 @@ func (cn *PeerConn) requestedMetadataPiece(index int) bool { // The actual value to use as the maximum outbound requests. func (cn *Peer) nominalMaxRequests() (ret int) { - return cn.PeerMaxRequests + return int(clamp(1, int64(cn.PeerMaxRequests), 64)) } func (cn *Peer) totalExpectingTime() (ret time.Duration) { @@ -1277,8 +1299,10 @@ func (c *Peer) receiveChunk(msg *pp.Message) error { chunksReceived.Add("due to allowed fast", 1) } - // TODO: This needs to happen immediately, to prevent cancels occurring asynchronously when have - // actually already received the piece, while we have the Client unlocked to write the data out. + // The request needs to be deleted immediately to prevent cancels occurring asynchronously when + // have actually already received the piece, while we have the Client unlocked to write the data + // out. + deletedRequest := false { if _, ok := c.requests[req]; ok { for _, f := range c.callbacks.ReceivedRequested { @@ -1287,7 +1311,8 @@ func (c *Peer) receiveChunk(msg *pp.Message) error { } // Request has been satisfied. if c.deleteRequest(req) { - if c.expectingChunks() { + deletedRequest = true + if !c.peerChoking { c._chunksReceivedWhileExpecting++ } } else { @@ -1306,14 +1331,13 @@ func (c *Peer) receiveChunk(msg *pp.Message) error { c.allStats(add(1, func(cs *ConnStats) *Count { return &cs.ChunksReadUseful })) c.allStats(add(int64(len(msg.Piece)), func(cs *ConnStats) *Count { return &cs.BytesReadUsefulData })) + if deletedRequest { + c.allStats(add(int64(len(msg.Piece)), func(cs *ConnStats) *Count { return &cs.BytesReadUsefulIntendedData })) + } for _, f := range c.t.cl.config.Callbacks.ReceivedUsefulData { f(ReceivedUsefulDataEvent{c, msg}) } c.lastUsefulChunkReceived = time.Now() - // if t.fastestPeer != c { - // log.Printf("setting fastest connection %p", c) - // } - t.fastestPeer = c // Need to record that it hasn't been written yet, before we attempt to do // anything with it. @@ -1591,14 +1615,6 @@ func (l connectionTrust) Less(r connectionTrust) bool { return multiless.New().Bool(l.Implicit, r.Implicit).Int64(l.NetGoodPiecesDirted, r.NetGoodPiecesDirted).Less() } -func (cn *Peer) chunksReceivedWhileExpecting() int64 { - return cn._chunksReceivedWhileExpecting -} - -func (cn *Peer) fastest() bool { - return cn == cn.t.fastestPeer -} - func (cn *Peer) peerMaxRequests() int { return cn.PeerMaxRequests } diff --git a/request-strategy.go b/request-strategy.go index d762d0f5..e7349aef 100644 --- a/request-strategy.go +++ b/request-strategy.go @@ -3,6 +3,7 @@ package torrent import ( "sort" "time" + "unsafe" "github.com/anacrolix/multiless" pp "github.com/anacrolix/torrent/peer_protocol" @@ -56,11 +57,13 @@ func (me clientPieceRequestOrder) update() { func (me clientPieceRequestOrder) less(_i, _j int) bool { i := me.pieces[_i] j := me.pieces[_j] - ml := multiless.New() - ml.Int(int(j.prio), int(i.prio)) - ml.Bool(j.partial, i.partial) - ml.Int(i.availability, j.availability) - return ml.Less() + return multiless.New().Int( + int(j.prio), int(i.prio), + ).Bool( + j.partial, i.partial, + ).Int( + i.availability, j.availability, + ).Less() } func (cl *Client) requester() { @@ -81,45 +84,87 @@ func (cl *Client) requester() { func (cl *Client) doRequests() { requestOrder := clientPieceRequestOrder{} allPeers := make(map[*Torrent][]*Peer) - storageCapacity := make(map[*Torrent]*int64) + // Storage capacity left for this run, keyed by the storage capacity pointer on the storage + // TorrentImpl. + storageLeft := make(map[*func() *int64]*int64) for _, t := range cl.torrents { // TODO: We could do metainfo requests here. if t.haveInfo() { - value := int64(t.usualPieceSize()) - storageCapacity[t] = &value + if t.storage.Capacity != nil { + if _, ok := storageLeft[t.storage.Capacity]; !ok { + storageLeft[t.storage.Capacity] = (*t.storage.Capacity)() + } + } requestOrder.addPieces(t, t.numPieces()) } var peers []*Peer t.iterPeers(func(p *Peer) { - peers = append(peers, p) + if !p.closed.IsSet() { + peers = append(peers, p) + } + }) + // Sort in *desc* order, approximately the reverse of worseConn where appropriate. + sort.Slice(peers, func(i, j int) bool { + return multiless.New().Float64( + peers[j].downloadRate(), peers[i].downloadRate(), + ).Uintptr( + uintptr(unsafe.Pointer(peers[j])), uintptr(unsafe.Pointer(peers[i]))).Less() }) allPeers[t] = peers } requestOrder.update() requestOrder.sort() + // For a given piece, the set of allPeers indices that absorbed requests for the piece. + contributed := make(map[int]struct{}) for _, p := range requestOrder.pieces { if p.t.ignorePieceForRequests(p.index) { continue } peers := allPeers[p.t] torrentPiece := p.t.piece(p.index) - if left := storageCapacity[p.t]; left != nil { + if left := storageLeft[p.t.storage.Capacity]; left != nil { if *left < int64(torrentPiece.length()) { continue } *left -= int64(torrentPiece.length()) } p.t.piece(p.index).iterUndirtiedChunks(func(chunk ChunkSpec) bool { - for _, peer := range peers { - req := Request{pp.Integer(p.index), chunk} - _, err := peer.request(req) - if err == nil { - //log.Printf("requested %v", req) - break + req := Request{pp.Integer(p.index), chunk} + const skipAlreadyRequested = false + if skipAlreadyRequested { + alreadyRequested := false + p.t.iterPeers(func(p *Peer) { + if _, ok := p.requests[req]; ok { + alreadyRequested = true + } + }) + if alreadyRequested { + return true + } + } + alreadyRequested := false + for peerIndex, peer := range peers { + if alreadyRequested { + // Cancel all requests from "slower" peers after the one that requested it. + peer.cancel(req) + } else { + err := peer.request(req) + if err == nil { + contributed[peerIndex] = struct{}{} + alreadyRequested = true + //log.Printf("requested %v", req) + } } } return true }) + // Move requestees for this piece to the back. + lastIndex := len(peers) - 1 + for peerIndex := range contributed { + peers[peerIndex], peers[lastIndex] = peers[lastIndex], peers[peerIndex] + delete(contributed, peerIndex) + lastIndex-- + } } for _, t := range cl.torrents { t.iterPeers(func(p *Peer) { diff --git a/torrent.go b/torrent.go index b6a6f2b7..6eab0d69 100644 --- a/torrent.go +++ b/torrent.go @@ -17,6 +17,7 @@ import ( "time" "unsafe" + "github.com/anacrolix/multiless" "github.com/anacrolix/torrent/common" "github.com/anacrolix/torrent/segments" "github.com/anacrolix/torrent/webseed" @@ -91,8 +92,7 @@ type Torrent struct { maxEstablishedConns int // Set of addrs to which we're attempting to connect. Connections are // half-open until all handshakes are completed. - halfOpen map[string]PeerInfo - fastestPeer *Peer + halfOpen map[string]PeerInfo // Reserve of peers to connect to. A peer can be both here and in the // active connections if were told about the peer after connecting with @@ -156,20 +156,6 @@ func (t *Torrent) pieceAvailability(i pieceIndex) (count int) { return } -func (t *Torrent) sortPieceRequestOrder(sl []pieceIndex) { - if len(sl) != t.numPieces() { - panic(len(sl)) - } - availability := make([]int, len(sl)) - t.iterPeers(func(peer *Peer) { - for i := range availability { - if peer.peerHasPiece(i) { - availability[i]++ - } - } - }) -} - func (t *Torrent) numConns() int { return len(t.conns) } @@ -665,8 +651,15 @@ func (t *Torrent) writeStatus(w io.Writer) { spew.Fdump(w, t.statsLocked()) peers := t.peersAsSlice() - sort.Slice(peers, func(i, j int) bool { - return worseConn(peers[i], peers[j]) + sort.Slice(peers, func(_i, _j int) bool { + i := peers[_i] + j := peers[_j] + if less, ok := multiless.New().EagerSameLess( + i.downloadRate() == j.downloadRate(), i.downloadRate() < j.downloadRate(), + ).LessOk(); ok { + return less + } + return worseConn(i, j) }) for i, c := range peers { fmt.Fprintf(w, "%2d. ", i+1) From 9c9ba1aeac88d219da409e6b66bbd4bfe9d17d40 Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Mon, 10 May 2021 12:29:47 +1000 Subject: [PATCH 07/59] PeerConn.post becomes PeerConn.write --- client.go | 12 ++++++------ peerconn.go | 30 ++++++++++-------------------- 2 files changed, 16 insertions(+), 26 deletions(-) diff --git a/client.go b/client.go index dc850a9e..bb0f4f4c 100644 --- a/client.go +++ b/client.go @@ -975,7 +975,7 @@ const localClientReqq = 1 << 5 // See the order given in Transmission's tr_peerMsgsNew. func (cl *Client) sendInitialMessages(conn *PeerConn, torrent *Torrent) { if conn.PeerExtensionBytes.SupportsExtended() && cl.config.Extensions.SupportsExtended() { - conn.post(pp.Message{ + conn.write(pp.Message{ Type: pp.Extended, ExtendedID: pp.HandshakeExtendedID, ExtendedPayload: func() []byte { @@ -1004,11 +1004,11 @@ func (cl *Client) sendInitialMessages(conn *PeerConn, torrent *Torrent) { func() { if conn.fastEnabled() { if torrent.haveAllPieces() { - conn.post(pp.Message{Type: pp.HaveAll}) + conn.write(pp.Message{Type: pp.HaveAll}) conn.sentHaves.AddRange(0, bitmap.BitIndex(conn.t.NumPieces())) return } else if !torrent.haveAnyPieces() { - conn.post(pp.Message{Type: pp.HaveNone}) + conn.write(pp.Message{Type: pp.HaveNone}) conn.sentHaves.Clear() return } @@ -1016,7 +1016,7 @@ func (cl *Client) sendInitialMessages(conn *PeerConn, torrent *Torrent) { conn.postBitfield() }() if conn.PeerExtensionBytes.SupportsDHT() && cl.config.Extensions.SupportsDHT() && cl.haveDhtServer() { - conn.post(pp.Message{ + conn.write(pp.Message{ Type: pp.Port, Port: cl.dhtPort(), }) @@ -1074,12 +1074,12 @@ func (cl *Client) gotMetadataExtensionMsg(payload []byte, t *Torrent, c *PeerCon return err case pp.RequestMetadataExtensionMsgType: if !t.haveMetadataPiece(piece) { - c.post(t.newMetadataExtensionMessage(c, pp.RejectMetadataExtensionMsgType, d["piece"], nil)) + c.write(t.newMetadataExtensionMessage(c, pp.RejectMetadataExtensionMsgType, d["piece"], nil)) return nil } start := (1 << 14) * piece c.logger.WithDefaultLevel(log.Debug).Printf("sending metadata piece %d", piece) - c.post(t.newMetadataExtensionMessage(c, pp.DataMetadataExtensionMsgType, piece, t.metadataBytes[start:start+t.metadataPieceSize(piece)])) + c.write(t.newMetadataExtensionMessage(c, pp.DataMetadataExtensionMsgType, piece, t.metadataBytes[start:start+t.metadataPieceSize(piece)])) return nil case pp.RejectMetadataExtensionMsgType: return nil diff --git a/peerconn.go b/peerconn.go index 48b305a2..7fd819be 100644 --- a/peerconn.go +++ b/peerconn.go @@ -404,27 +404,17 @@ func (cn *Peer) peerHasPiece(piece pieceIndex) bool { // https://github.com/pion/datachannel/issues/59 is fixed. const writeBufferHighWaterLen = 1 << 15 -// Writes a message into the write buffer. Returns whether it's okay to keep writing. Posting is -// done asynchronously, so it may be that we're not able to honour backpressure from this method. It -// might be possible to merge this with PeerConn.write down the track? They seem to be very similar. -func (cn *PeerConn) post(msg pp.Message) bool { - torrent.Add(fmt.Sprintf("messages posted of type %s", msg.Type.String()), 1) +// Writes a message into the write buffer. Returns whether it's okay to keep writing. Writing is +// done asynchronously, so it may be that we're not able to honour backpressure from this method. +func (cn *PeerConn) write(msg pp.Message) bool { + torrent.Add(fmt.Sprintf("messages written of type %s", msg.Type.String()), 1) // We don't need to track bytes here because a connection.w Writer wrapper takes care of that // (although there's some delay between us recording the message, and the connection writer // flushing it out.). cn.writeBuffer.Write(msg.MustMarshalBinary()) - // Last I checked only Piece messages affect stats, and we don't post those. + // Last I checked only Piece messages affect stats, and we don't write those. cn.wroteMsg(&msg) cn.tickleWriter() - return cn.writeBuffer.Len() < writeBufferHighWaterLen -} - -// Returns true if there's room to write more. -func (cn *PeerConn) write(msg pp.Message) bool { - cn.wroteMsg(&msg) - cn.writeBuffer.Write(msg.MustMarshalBinary()) - torrent.Add(fmt.Sprintf("messages filled of type %s", msg.Type.String()), 1) - cn.tickleWriter() return !cn.writeBufferFull() } @@ -441,7 +431,7 @@ func (cn *PeerConn) requestMetadataPiece(index int) { return } cn.logger.WithDefaultLevel(log.Debug).Printf("requesting metadata piece %d", index) - cn.post(pp.Message{ + cn.write(pp.Message{ Type: pp.Extended, ExtendedID: eID, ExtendedPayload: func() []byte { @@ -689,7 +679,7 @@ func (cn *PeerConn) have(piece pieceIndex) { if cn.sentHaves.Get(bitmap.BitIndex(piece)) { return } - cn.post(pp.Message{ + cn.write(pp.Message{ Type: pp.Have, Index: pp.Integer(piece), }) @@ -703,7 +693,7 @@ func (cn *PeerConn) postBitfield() { if !cn.t.haveAnyPieces() { return } - cn.post(pp.Message{ + cn.write(pp.Message{ Type: pp.Bitfield, Bitfield: cn.t.bitfield(), }) @@ -946,7 +936,7 @@ func (c *PeerConn) reject(r Request) { if !c.fastEnabled() { panic("fast not enabled") } - c.post(r.ToMsg(pp.Reject)) + c.write(r.ToMsg(pp.Reject)) delete(c.peerRequests, r) } @@ -1028,7 +1018,7 @@ func (c *PeerConn) peerRequestDataReadFailed(err error, r Request) { if c.choking { c.logger.WithDefaultLevel(log.Warning).Printf("already choking peer, requests might not be rejected correctly") } - c.choke(c.post) + c.choke(c.write) } func readPeerRequestData(r Request, c *PeerConn) ([]byte, error) { From 977a3ec00f0d8fc322584929268c0443ca69675a Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Mon, 10 May 2021 12:30:44 +1000 Subject: [PATCH 08/59] Use multiless with Float64 --- go.mod | 2 +- go.sum | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/go.mod b/go.mod index 30b55a09..51d7b435 100644 --- a/go.mod +++ b/go.mod @@ -13,7 +13,7 @@ require ( github.com/anacrolix/missinggo v1.2.1 github.com/anacrolix/missinggo/perf v1.0.0 github.com/anacrolix/missinggo/v2 v2.5.0 - github.com/anacrolix/multiless v0.1.0 + github.com/anacrolix/multiless v0.1.1-0.20210510014912-3f17cb19bda9 github.com/anacrolix/sync v0.2.0 github.com/anacrolix/tagflag v1.3.0 github.com/anacrolix/upnp v0.1.2-0.20200416075019-5e9378ed1425 diff --git a/go.sum b/go.sum index 7cbf393e..d5c8a813 100644 --- a/go.sum +++ b/go.sum @@ -116,6 +116,8 @@ github.com/anacrolix/multiless v0.0.0-20200413040533-acfd16f65d5d/go.mod h1:TrCL github.com/anacrolix/multiless v0.0.0-20210222022749-ef43011a77ec/go.mod h1:TrCLEZfIDbMVfLoQt5tOoiBS/uq4y8+ojuEVVvTNPX4= github.com/anacrolix/multiless v0.1.0 h1:gjR3SdJ+E0avnmEoAV/7K7n2kILZhVu/M6aQEtz8H3s= github.com/anacrolix/multiless v0.1.0/go.mod h1:TrCLEZfIDbMVfLoQt5tOoiBS/uq4y8+ojuEVVvTNPX4= +github.com/anacrolix/multiless v0.1.1-0.20210510014912-3f17cb19bda9 h1:fmNDxh5ysBPinRq249xYYYOLQ/h95DoyGE9e9Gp9xqo= +github.com/anacrolix/multiless v0.1.1-0.20210510014912-3f17cb19bda9/go.mod h1:TrCLEZfIDbMVfLoQt5tOoiBS/uq4y8+ojuEVVvTNPX4= github.com/anacrolix/stm v0.1.0/go.mod h1:ZKz7e7ERWvP0KgL7WXfRjBXHNRhlVRlbBQecqFtPq+A= github.com/anacrolix/stm v0.1.1-0.20191106051447-e749ba3531cf/go.mod h1:zoVQRvSiGjGoTmbM0vSLIiaKjWtNPeTvXUSdJQA4hsg= github.com/anacrolix/stm v0.2.0/go.mod h1:zoVQRvSiGjGoTmbM0vSLIiaKjWtNPeTvXUSdJQA4hsg= From 52e6979457ed1a7eb03260e91fd2c68b25a97acb Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Mon, 10 May 2021 12:40:42 +1000 Subject: [PATCH 09/59] go mod tidy --- go.sum | 284 --------------------------------------------------------- 1 file changed, 284 deletions(-) diff --git a/go.sum b/go.sum index d5c8a813..fa66c663 100644 --- a/go.sum +++ b/go.sum @@ -4,25 +4,17 @@ bazil.org/fuse v0.0.0-20200407214033-5883e5a4b512/go.mod h1:FbcW6z/2VytnFDhZfumh cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.31.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= -cloud.google.com/go v0.37.0 h1:69FNAINiZfsEuwH3fKq8QrAAnHz+2m4XL4kVYi5BX0Q= cloud.google.com/go v0.37.0/go.mod h1:TS1dMSSfndXH133OKGwekG838Om/cQT0BUHV3HcBgoo= crawshaw.io/iox v0.0.0-20181124134642-c51c3df30797 h1:yDf7ARQc637HoxDho7xjqdvO5ZA2Yb+xzv/fOnnvZzw= crawshaw.io/iox v0.0.0-20181124134642-c51c3df30797/go.mod h1:sXBiorCo8c46JlQV3oXPKINnZ8mcqnye1EkVkqsectk= crawshaw.io/sqlite v0.3.3-0.20210127221821-98b1f83c5508 h1:fILCBBFnjnrQ0whVJlGhfv1E/QiaFDNtGFBObEVRnYg= crawshaw.io/sqlite v0.3.3-0.20210127221821-98b1f83c5508/go.mod h1:igAO5JulrQ1DbdZdtVq48mnZUBAPOeFzer7VhDWNtW4= -dmitri.shuralyov.com/app/changes v0.0.0-20180602232624-0a106ad413e3 h1:hJiie5Bf3QucGRa4ymsAUOxyhYwGEz1xrsVk0P8erlw= dmitri.shuralyov.com/app/changes v0.0.0-20180602232624-0a106ad413e3/go.mod h1:Yl+fi1br7+Rr3LqpNJf1/uxUdtRUV+Tnj0o93V2B9MU= -dmitri.shuralyov.com/html/belt v0.0.0-20180602232347-f7d459c86be0 h1:SPOUaucgtVls75mg+X7CXigS71EnsfVUK/2CgVrwqgw= dmitri.shuralyov.com/html/belt v0.0.0-20180602232347-f7d459c86be0/go.mod h1:JLBrvjyP0v+ecvNYvCpyZgu5/xkfAUhi6wJj28eUfSU= -dmitri.shuralyov.com/service/change v0.0.0-20181023043359-a85b471d5412 h1:GvWw74lx5noHocd+f6HBMXK6DuggBB1dhVkuGZbv7qM= dmitri.shuralyov.com/service/change v0.0.0-20181023043359-a85b471d5412/go.mod h1:a1inKt/atXimZ4Mv927x+r7UpyzRUf4emIoiiSC2TN4= -dmitri.shuralyov.com/state v0.0.0-20180228185332-28bcc343414c h1:ivON6cwHK1OH26MZyWDCnbTRZZf0IhNsENoNAKFS1g4= dmitri.shuralyov.com/state v0.0.0-20180228185332-28bcc343414c/go.mod h1:0PRwlb0D6DFvNNtx+9ybjezNCa8XF0xaYcETyp6rHWU= -git.apache.org/thrift.git v0.0.0-20180902110319-2566ecd5d999 h1:OR8VhtwhcAI3U48/rzBsVOuHi0zDPzYI1xASVcdSgR8= git.apache.org/thrift.git v0.0.0-20180902110319-2566ecd5d999/go.mod h1:fPE2ZNJGynbRyZ4dJvy6G277gSllfV2HJqblrnkyeyg= -github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= -github.com/Knetic/govaluate v3.0.1-0.20171022003610-9aa49832a739+incompatible h1:1G1pk05UrOh0NlF1oeaaix1x8XzrfjIDK47TY0Zehcw= github.com/Knetic/govaluate v3.0.1-0.20171022003610-9aa49832a739+incompatible/go.mod h1:r7JcOSlj0wfOMncg0iLm8Leh48TZaKVeNIfJntJ2wa0= github.com/RoaringBitmap/roaring v0.4.7/go.mod h1:8khRDP4HmeXns4xIj9oGrKSz7XTQiJx2zgh7AcNke4w= github.com/RoaringBitmap/roaring v0.4.17/go.mod h1:D3qVegWTmfCaX4Bl5CrBE9hfrSrrXIr8KVNvRsDi1NI= @@ -32,22 +24,15 @@ github.com/RoaringBitmap/roaring v0.4.23/go.mod h1:D0gp8kJQgE1A4LQ5wFLggQEyvDi06 github.com/RoaringBitmap/roaring v0.5.5/go.mod h1:puNo5VdzwbaIQxSiDIwfXl4Hnc+fbovcX4IW/dSTtUk= github.com/RoaringBitmap/roaring v0.6.0 h1:tZcn2nJpUrZf+xQY8x+9QY7BxSETMjkdNG4Ts5zahyU= github.com/RoaringBitmap/roaring v0.6.0/go.mod h1:WZ83fjBF/7uBHi6QoFyfGL4+xuV4Qn+xFkm4+vSzrhE= -github.com/Shopify/sarama v1.19.0 h1:9oksLxC6uxVPHPVYUmq6xhr1BOF/hHobWH2UzO67z1s= github.com/Shopify/sarama v1.19.0/go.mod h1:FVkBWblsNy7DGZRfXLU0O9RCGt5g3g3yEuWXgklEdEo= -github.com/Shopify/toxiproxy v2.1.4+incompatible h1:TKdv8HiTLgE5wdJuEML90aBgNWsokNbMijUGhmcoBJc= github.com/Shopify/toxiproxy v2.1.4+incompatible/go.mod h1:OXgGpZ6Cli1/URJOF1DMxUHB2q5Ap20/P/eIdh4G0pI= -github.com/VividCortex/gohistogram v1.0.0 h1:6+hBz+qvs0JOrrNhhmR7lFxo5sINxBCGXrdtl/UvroE= github.com/VividCortex/gohistogram v1.0.0/go.mod h1:Pf5mBqqDxYaXu3hDrrU+w6nw50o/4+TcAqDqk/vUH7g= -github.com/afex/hystrix-go v0.0.0-20180502004556-fa1af6a1f4f5 h1:rFw4nCn9iMW+Vajsk51NtYIcwSTkXr+JGrMd36kTDJw= github.com/afex/hystrix-go v0.0.0-20180502004556-fa1af6a1f4f5/go.mod h1:SkGFH1ia65gfNATL8TAiHDNxPzPdmEL5uirI2Uyuz6c= -github.com/alangpierce/go-forceexport v0.0.0-20160317203124-8f1d6941cd75 h1:3ILjVyslFbc4jl1w5TWuvvslFD/nDfR2H8tVaMVLrEY= github.com/alangpierce/go-forceexport v0.0.0-20160317203124-8f1d6941cd75/go.mod h1:uAXEEpARkRhCZfEvy/y0Jcc888f9tHCc1W7/UeEtreE= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= -github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751 h1:JYp7IbQjafoB+tBA3gMyHYHrpOtNuDiK/uB5uXxq5wM= github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= -github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d h1:UQZhZ2O0vMHr2cI+DC1Mbh0TJxzA3RcLoMsFw+aXw7E= github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk541a8SKzHPHnH3zbiI+7dagKZ0cgpgrD7Fyho= github.com/alexflint/go-arg v1.1.0/go.mod h1:3Rj4baqzWaGGmZA2+bVTV8zQOZEjBQAPBnL5xLT+ftY= github.com/alexflint/go-arg v1.2.0/go.mod h1:3Rj4baqzWaGGmZA2+bVTV8zQOZEjBQAPBnL5xLT+ftY= @@ -114,8 +99,6 @@ github.com/anacrolix/mmsg v1.0.0/go.mod h1:x8kRaJY/dCrY9Al0PEcj1mb/uFHwP6GCJ9fLl github.com/anacrolix/multiless v0.0.0-20191223025854-070b7994e841/go.mod h1:TrCLEZfIDbMVfLoQt5tOoiBS/uq4y8+ojuEVVvTNPX4= github.com/anacrolix/multiless v0.0.0-20200413040533-acfd16f65d5d/go.mod h1:TrCLEZfIDbMVfLoQt5tOoiBS/uq4y8+ojuEVVvTNPX4= github.com/anacrolix/multiless v0.0.0-20210222022749-ef43011a77ec/go.mod h1:TrCLEZfIDbMVfLoQt5tOoiBS/uq4y8+ojuEVVvTNPX4= -github.com/anacrolix/multiless v0.1.0 h1:gjR3SdJ+E0avnmEoAV/7K7n2kILZhVu/M6aQEtz8H3s= -github.com/anacrolix/multiless v0.1.0/go.mod h1:TrCLEZfIDbMVfLoQt5tOoiBS/uq4y8+ojuEVVvTNPX4= github.com/anacrolix/multiless v0.1.1-0.20210510014912-3f17cb19bda9 h1:fmNDxh5ysBPinRq249xYYYOLQ/h95DoyGE9e9Gp9xqo= github.com/anacrolix/multiless v0.1.1-0.20210510014912-3f17cb19bda9/go.mod h1:TrCLEZfIDbMVfLoQt5tOoiBS/uq4y8+ojuEVVvTNPX4= github.com/anacrolix/stm v0.1.0/go.mod h1:ZKz7e7ERWvP0KgL7WXfRjBXHNRhlVRlbBQecqFtPq+A= @@ -137,7 +120,6 @@ github.com/anacrolix/tagflag v1.0.0/go.mod h1:1m2U/K6ZT+JZG0+bdMK6qauP49QT4wE5pm github.com/anacrolix/tagflag v1.0.1/go.mod h1:gb0fiMQ02qU1djCSqaxGmruMvZGrMwSReidMB0zjdxo= github.com/anacrolix/tagflag v1.1.0/go.mod h1:Scxs9CV10NQatSmbyjqmqmeQNwGzlNe0CMUMIxqHIG8= github.com/anacrolix/tagflag v1.1.1-0.20200411025953-9bb5209d56c2/go.mod h1:Scxs9CV10NQatSmbyjqmqmeQNwGzlNe0CMUMIxqHIG8= -github.com/anacrolix/tagflag v1.2.0 h1:WdSv10SpxOI97++f5FUKnKPFkVGMiPlpYm52XPaMkp4= github.com/anacrolix/tagflag v1.2.0/go.mod h1:Scxs9CV10NQatSmbyjqmqmeQNwGzlNe0CMUMIxqHIG8= github.com/anacrolix/tagflag v1.3.0 h1:5NI+9CniDnEH0BWA4UcQbERyFPjKJqZnVkItGVIDy/s= github.com/anacrolix/tagflag v1.3.0/go.mod h1:Scxs9CV10NQatSmbyjqmqmeQNwGzlNe0CMUMIxqHIG8= @@ -156,89 +138,56 @@ github.com/anacrolix/upnp v0.1.2-0.20200416075019-5e9378ed1425/go.mod h1:Pz94W3k github.com/anacrolix/utp v0.0.0-20180219060659-9e0e1d1d0572/go.mod h1:MDwc+vsGEq7RMw6lr2GKOEqjWny5hO5OZXRVNaBJ2Dk= github.com/anacrolix/utp v0.1.0 h1:FOpQOmIwYsnENnz7tAGohA+r6iXpRjrq8ssKSre2Cp4= github.com/anacrolix/utp v0.1.0/go.mod h1:MDwc+vsGEq7RMw6lr2GKOEqjWny5hO5OZXRVNaBJ2Dk= -github.com/anmitsu/go-shlex v0.0.0-20161002113705-648efa622239 h1:kFOfPq6dUM1hTo4JG6LR5AXSUEsOjtdm0kw0FtQtMJA= github.com/anmitsu/go-shlex v0.0.0-20161002113705-648efa622239/go.mod h1:2FmKhYUyUczH0OGQWaF5ceTx0UBShxjsH6f8oGKYe2c= github.com/apache/thrift v0.12.0/go.mod h1:cp2SuWMxlEZw2r+iP2GNCdIi4C1qmUzdZFSVb+bacwQ= -github.com/apache/thrift v0.13.0 h1:5hryIiq9gtn+MiLVn0wP37kb/uTeRZgN08WoCsAhIhI= github.com/apache/thrift v0.13.0/go.mod h1:cp2SuWMxlEZw2r+iP2GNCdIi4C1qmUzdZFSVb+bacwQ= -github.com/armon/circbuf v0.0.0-20150827004946-bbbad097214e h1:QEF07wC0T1rKkctt1RINW/+RMTVmiwxETico2l3gxJA= github.com/armon/circbuf v0.0.0-20150827004946-bbbad097214e/go.mod h1:3U/XgcO3hCbHZ8TKRvWD2dDTCfh9M9ya+I9JpbB7O8o= -github.com/armon/go-metrics v0.0.0-20180917152333-f0300d1749da h1:8GUt8eRujhVEGZFFEjBj46YV4rDjvGrNxb0KMWYkL2I= github.com/armon/go-metrics v0.0.0-20180917152333-f0300d1749da/go.mod h1:Q73ZrmVTwzkszR9V5SSuryQ31EELlFMUz1kKyl939pY= -github.com/armon/go-radix v0.0.0-20180808171621-7fddfc383310 h1:BUAU3CGlLvorLI26FmByPp2eC2qla6E1Tw+scpcg/to= github.com/armon/go-radix v0.0.0-20180808171621-7fddfc383310/go.mod h1:ufUuZ+zHj4x4TnLV4JWEpy2hxWSpsRywHrMgIH9cCH8= -github.com/aryann/difflib v0.0.0-20170710044230-e206f873d14a h1:pv34s756C4pEXnjgPfGYgdhg/ZdajGhyOvzx8k+23nw= github.com/aryann/difflib v0.0.0-20170710044230-e206f873d14a/go.mod h1:DAHtR1m6lCRdSC2Tm3DSWRPvIPr6xNKyeHdqDQSQT+A= -github.com/aws/aws-lambda-go v1.13.3 h1:SuCy7H3NLyp+1Mrfp+m80jcbi9KYWAs9/BXwppwRDzY= github.com/aws/aws-lambda-go v1.13.3/go.mod h1:4UKl9IzQMoD+QF79YdCuzCwp8VbmG4VAQwij/eHl5CU= -github.com/aws/aws-sdk-go v1.27.0 h1:0xphMHGMLBrPMfxR2AmVjZKcMEESEgWF8Kru94BNByk= github.com/aws/aws-sdk-go v1.27.0/go.mod h1:KmX6BPdI08NWTb3/sm4ZGu5ShLoqVDhKgpiN924inxo= -github.com/aws/aws-sdk-go-v2 v0.18.0 h1:qZ+woO4SamnH/eEbjM2IDLhRNwIwND/RQyVlBLp3Jqg= github.com/aws/aws-sdk-go-v2 v0.18.0/go.mod h1:JWVYvqSMppoMJC0x5wdwiImzgXTI9FuZwxzkQq9wy+g= github.com/benbjohnson/immutable v0.2.0/go.mod h1:uc6OHo6PN2++n98KHLxW8ef4W42ylHiQSENghE1ezxI= github.com/benbjohnson/immutable v0.3.0 h1:TVRhuZx2wG9SZ0LRdqlbs9S5BZ6Y24hJEHTCgWHZEIw= github.com/benbjohnson/immutable v0.3.0/go.mod h1:uc6OHo6PN2++n98KHLxW8ef4W42ylHiQSENghE1ezxI= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= -github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= -github.com/bgentry/speakeasy v0.1.0 h1:ByYyxL9InA1OWqxJqqp2A5pYHUrCiAL6K3J+LKSsQkY= github.com/bgentry/speakeasy v0.1.0/go.mod h1:+zsyZBPWlz7T6j88CTgSN5bM796AkVf0kBD4zp0CCIs= -github.com/boltdb/bolt v1.3.1 h1:JQmyP4ZBrce+ZQu0dY660FMfatumYDLun9hBCUVIkF4= github.com/boltdb/bolt v1.3.1/go.mod h1:clJnj/oiGkjum5o1McbSZDSLxVThjynRyGBgiAx27Ps= -github.com/bradfitz/go-smtpd v0.0.0-20170404230938-deb6d6237625 h1:ckJgFhFWywOx+YLEMIJsTb+NV6NexWICk5+AMSuz3ss= github.com/bradfitz/go-smtpd v0.0.0-20170404230938-deb6d6237625/go.mod h1:HYsPBTaaSFSlLx/70C2HPIMNZpVV8+vt/A+FMnYP11g= github.com/bradfitz/iter v0.0.0-20140124041915-454541ec3da2/go.mod h1:PyRFw1Lt2wKX4ZVSQ2mk+PeDa1rxyObEDlApuIsUKuo= github.com/bradfitz/iter v0.0.0-20190303215204-33e6a9893b0c/go.mod h1:PyRFw1Lt2wKX4ZVSQ2mk+PeDa1rxyObEDlApuIsUKuo= github.com/bradfitz/iter v0.0.0-20191230175014-e8f45d346db8 h1:GKTyiRCL6zVf5wWaqKnf+7Qs6GbEPfd4iMOitWzXJx8= github.com/bradfitz/iter v0.0.0-20191230175014-e8f45d346db8/go.mod h1:spo1JLcs67NmW1aVLEgtA8Yy1elc+X8y5SRW1sFW4Og= -github.com/buger/jsonparser v0.0.0-20181115193947-bf1c66bbce23 h1:D21IyuvjDCshj1/qq+pCNd3VZOAEI9jy6Bi131YlXgI= github.com/buger/jsonparser v0.0.0-20181115193947-bf1c66bbce23/go.mod h1:bbYlZJ7hK1yFx9hf58LP0zeX7UjIGs20ufpu3evjr+s= -github.com/casbin/casbin/v2 v2.1.2 h1:bTwon/ECRx9dwBy2ewRVr5OiqjeXSGiTUY74sDPQi/g= github.com/casbin/casbin/v2 v2.1.2/go.mod h1:YcPU1XXisHhLzuxH9coDNf2FbKpjGlbCg3n9yuLkIJQ= -github.com/cenkalti/backoff v2.2.1+incompatible h1:tNowT99t7UNflLxfYYSlKYsBpXdEet03Pg2g16Swow4= github.com/cenkalti/backoff v2.2.1+incompatible/go.mod h1:90ReRw6GdpyfrHakVjL/QHaoyV4aDUVVkXQJJJ3NXXM= -github.com/census-instrumentation/opencensus-proto v0.2.1 h1:glEXhBS5PSLLv4IXzLA5yPRVX4bilULVyxxbrfOtDAk= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= -github.com/cespare/xxhash/v2 v2.1.1 h1:6MnRN8NT7+YBpUIWxHtefFZOKTAPgGjpQSxqLNn0+qY= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= -github.com/cheekybits/genny v1.0.0 h1:uGGa4nei+j20rOSeDeP5Of12XVm7TGUd4dJA9RDitfE= github.com/cheekybits/genny v1.0.0/go.mod h1:+tQajlRqAUrPI7DOSpB0XAqZYtQakVtB7wXkRAgjxjQ= -github.com/clbanning/x2j v0.0.0-20191024224557-825249438eec h1:EdRZT3IeKQmfCSrgo8SZ8V3MEnskuJP0wCYNpe+aiXo= github.com/clbanning/x2j v0.0.0-20191024224557-825249438eec/go.mod h1:jMjuTZXRI4dUb/I5gc9Hdhagfvm9+RyrPryS/auMzxE= -github.com/client9/misspell v0.3.4 h1:ta993UF76GwbvJcIo3Y68y/M3WxlpEHPWIGDkJYwzJI= github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= -github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa h1:OaNxuTZr7kxeODyLWsRMC+OD03aFUH+mW6r2d+MWa5Y= github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa/go.mod h1:zn76sxSg3SzpJ0PPJaLDCu+Bu0Lg3sKTORVIj19EIF8= -github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd h1:qMd81Ts1T2OTKmB4acZcyKaMtRnY5Y44NuXGX2GFJ1w= github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd/go.mod h1:sE/e/2PUdi/liOCUjSTXgM1o87ZssimdTWN964YiIeI= -github.com/coreos/go-semver v0.2.0 h1:3Jm3tLmsgAYcjC+4Up7hJrFBPr+n7rAqYeSw/SZazuY= github.com/coreos/go-semver v0.2.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= github.com/coreos/go-systemd v0.0.0-20180511133405-39ca1b05acc7/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= -github.com/coreos/go-systemd v0.0.0-20181012123002-c6f51f82210d h1:t5Wuyh53qYyg9eqn4BbnlIT+vmhyww0TatL+zT3uWgI= github.com/coreos/go-systemd v0.0.0-20181012123002-c6f51f82210d/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= -github.com/coreos/pkg v0.0.0-20160727233714-3ac0863d7acf h1:CAKfRE2YtTUIjjh1bkBtyYFaUT/WmOqsJjgtihT0vMI= github.com/coreos/pkg v0.0.0-20160727233714-3ac0863d7acf/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA= -github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d h1:U+s90UTSYgptZMwQh2aRr3LuazLJIa+Pg3Kc1ylSYVY= github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= -github.com/creack/pty v1.1.7 h1:6pwm8kMQKCmgUg0ZHTm5+/YvRK0s3THD/28+T6/kk4A= github.com/creack/pty v1.1.7/go.mod h1:lj5s0c3V2DBrqTV7llrYr5NG6My20zk30Fl46Y7DoTY= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= -github.com/dgrijalva/jwt-go v3.2.0+incompatible h1:7qlOGliEKZXTDg6OTjfoBKDXWrumCAMpl/TFQ4/5kLM= github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ= -github.com/docopt/docopt-go v0.0.0-20180111231733-ee0de3bc6815 h1:bWDMxwH3px2JBh6AyO7hdCn/PkvCZXii8TGj7sbtEbQ= github.com/docopt/docopt-go v0.0.0-20180111231733-ee0de3bc6815/go.mod h1:WwZ+bS3ebgob9U8Nd0kOddGdZWjyMGR8Wziv+TBNwSE= github.com/dustin/go-humanize v0.0.0-20171111073723-bb3d318650d4/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= github.com/dustin/go-humanize v0.0.0-20180421182945-02af3965c54e/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= github.com/dustin/go-humanize v1.0.0 h1:VSnTsYCnlFHaM2/igO1h6X3HA71jcobQuxemgkq4zYo= github.com/dustin/go-humanize v1.0.0/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= -github.com/eapache/go-resiliency v1.1.0 h1:1NtRmCAqadE2FN4ZcN6g90TP3uk8cg9rn9eNK2197aU= github.com/eapache/go-resiliency v1.1.0/go.mod h1:kFI+JgMyC7bLPUVY133qvEBtVayf5mFgVsvEsIPBvNs= -github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21 h1:YEetp8/yCZMuEPMUDHG0CW/brkkEp8mzqk2+ODEitlw= github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21/go.mod h1:+020luEh2TKB4/GOp8oxxtq0Daoen/Cii55CzbTV6DU= -github.com/eapache/queue v1.1.0 h1:YOEu7KNc61ntiQlcEeUIoDTJ2o8mQznoNvUhiigpIqc= github.com/eapache/queue v1.1.0/go.mod h1:6eCeP0CKFpHLu8blIFXhExK/dRa7WDZfr6jVFPTqq+I= github.com/edsrzf/mmap-go v0.0.0-20170320065105-0bce6a688712/go.mod h1:YO35OhQPt3KJa3ryjFM5Bs14WD66h8eGKpfaBNrHW5M= github.com/edsrzf/mmap-go v1.0.0 h1:CEBF7HpRnUCSJgGUb5h1Gm7e3VkmVDrR8lvWVLtrOFw= @@ -248,21 +197,13 @@ github.com/elliotchance/orderedmap v1.3.0/go.mod h1:8hdSl6jmveQw8ScByd3AaNHNk51R github.com/elliotchance/orderedmap v1.4.0 h1:wZtfeEONCbx6in1CZyE6bELEt/vFayMvsxqI5SgsR+A= github.com/elliotchance/orderedmap v1.4.0/go.mod h1:wsDwEaX5jEoyhbs7x93zk2H/qv0zwuhg4inXhDkYqys= github.com/envoyproxy/go-control-plane v0.6.9/go.mod h1:SBwIajubJHhxtWwsL9s8ss4safvEdbitLhGGK48rN6g= -github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473 h1:4cmBvAEBNJaGARUEs3/suWRyfyBfhf7I60WBZq+bv2w= github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= -github.com/envoyproxy/protoc-gen-validate v0.1.0 h1:EQciDnbrYxy13PgWoY8AqoxGiPrpgBZ1R8UNe3ddc+A= github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= -github.com/etcd-io/bbolt v1.3.3 h1:gSJmxrs37LgTqR/oyJBWok6k6SvXEUerFTbltIhXkBM= github.com/etcd-io/bbolt v1.3.3/go.mod h1:ZF2nL25h33cCyBtcyWeZ2/I3HQOfTP+0PIEvHjkjCrw= -github.com/fatih/color v1.7.0 h1:DkWD4oS2D8LGGgTQ6IvwJJXSL5Vp2ffcQg58nFV38Ys= github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4= -github.com/flynn/go-shlex v0.0.0-20150515145356-3f9db97f8568 h1:BHsljHzVlRcyQhjrss6TZTdY2VfCqZPbv5k3iBFa2ZQ= github.com/flynn/go-shlex v0.0.0-20150515145356-3f9db97f8568/go.mod h1:xEzjJPgXI435gkrCt3MPfRiAkVrwSbHsst4LCFVfpJc= -github.com/francoispqt/gojay v1.2.13 h1:d2m3sFjloqoIUQU3TsHBgj6qg/BVGlTBeHDUmyJnXKk= github.com/francoispqt/gojay v1.2.13/go.mod h1:ehT5mTG4ua4581f1++1WLG0vPdaA9HaiDsoyrBGkyDY= -github.com/franela/goblin v0.0.0-20200105215937-c9ffbefa60db h1:gb2Z18BhTPJPpLQWj4T+rfKHYCHxRHCtRxhKKjRidVw= github.com/franela/goblin v0.0.0-20200105215937-c9ffbefa60db/go.mod h1:7dvUGVsVBjqR7JHJk0brhHOZYGmfBYOrK0ZhYMEtBr4= -github.com/franela/goreq v0.0.0-20171204163338-bcd34c9993f8 h1:a9ENSRDFBUPkJ5lCgVZh26+ZbGyoVJG7yb5SSzF5H54= github.com/franela/goreq v0.0.0-20171204163338-bcd34c9993f8/go.mod h1:ZhphrRTfi2rbfLwlschooIH4+wKKDR4Pdxhh+TRoA20= github.com/frankban/quicktest v1.9.0/go.mod h1:ui7WezCLWMWxVWr1GETZY3smRy0G4KWq9vcPtJmFl7Y= github.com/frankban/quicktest v1.11.3 h1:8sXhOn0uLys67V8EsXLc6eszDs8VXWxL3iRvebPhedY= @@ -270,56 +211,38 @@ github.com/frankban/quicktest v1.11.3/go.mod h1:wRf/ReqHper53s+kmmSZizM8NamnL3IM github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= github.com/fsnotify/fsnotify v1.4.9 h1:hsms1Qyu0jgnwNXIxa+/V/PDsU6CfLf6CNO8H7IWoS4= github.com/fsnotify/fsnotify v1.4.9/go.mod h1:znqG4EE+3YCdAaPaxE2ZRY/06pZUdp0tY4IgpuI1SZQ= -github.com/getlantern/sqlite v0.3.3-0.20210215090556-4f83cf7731f0 h1:zvFSvII5rTbMZ3idAqSUjUCDgZFbWMKzxQot3/Y7nzA= -github.com/getlantern/sqlite v0.3.3-0.20210215090556-4f83cf7731f0/go.mod h1:igAO5JulrQ1DbdZdtVq48mnZUBAPOeFzer7VhDWNtW4= -github.com/ghodss/yaml v1.0.0 h1:wQHKEahhL6wmXdzwWG11gIVCkOv05bNOh+Rxn0yngAk= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= -github.com/gliderlabs/ssh v0.1.1 h1:j3L6gSLQalDETeEg/Jg0mGY0/y/N6zI2xX1978P0Uqw= github.com/gliderlabs/ssh v0.1.1/go.mod h1:U7qILu1NlMHj9FlMhZLlkCdDnU1DBEAqr0aevW3Awn0= github.com/glycerine/go-unsnap-stream v0.0.0-20180323001048-9f0cb55181dd/go.mod h1:/20jfyN9Y5QPEAprSgKAUr+glWDY39ZiUEAYOEv5dsE= github.com/glycerine/go-unsnap-stream v0.0.0-20181221182339-f9677308dec2/go.mod h1:/20jfyN9Y5QPEAprSgKAUr+glWDY39ZiUEAYOEv5dsE= github.com/glycerine/go-unsnap-stream v0.0.0-20190901134440-81cf024a9e0a/go.mod h1:/20jfyN9Y5QPEAprSgKAUr+glWDY39ZiUEAYOEv5dsE= -github.com/glycerine/go-unsnap-stream v0.0.0-20210130063903-47dfef350d96 h1:rCXyLrgJ598XNj7KTqPzAvwTzlyvI+clqasoNfLQStE= github.com/glycerine/go-unsnap-stream v0.0.0-20210130063903-47dfef350d96/go.mod h1:/20jfyN9Y5QPEAprSgKAUr+glWDY39ZiUEAYOEv5dsE= github.com/glycerine/goconvey v0.0.0-20180728074245-46e3a41ad493/go.mod h1:Ogl1Tioa0aV7gstGFO7KhffUsb9M4ydbEbbxpcEDc24= github.com/glycerine/goconvey v0.0.0-20190315024820-982ee783a72e/go.mod h1:Ogl1Tioa0aV7gstGFO7KhffUsb9M4ydbEbbxpcEDc24= -github.com/glycerine/goconvey v0.0.0-20190410193231-58a59202ab31 h1:gclg6gY70GLy3PbkQ1AERPfmLMMagS60DKF78eWwLn8= github.com/glycerine/goconvey v0.0.0-20190410193231-58a59202ab31/go.mod h1:Ogl1Tioa0aV7gstGFO7KhffUsb9M4ydbEbbxpcEDc24= -github.com/go-errors/errors v1.0.1 h1:LUHzmkK3GUKUrL/1gfBUxAHzcev3apQlezX/+O7ma6w= github.com/go-errors/errors v1.0.1/go.mod h1:f4zRHt4oKfwPJE5k8C9vpYG+aDHdBFUsgrm6/TyX73Q= github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= -github.com/go-kit/kit v0.10.0 h1:dXFJfIHVvUcpSgDOV+Ne6t7jXri8Tfv2uOLHUZ2XNuo= github.com/go-kit/kit v0.10.0/go.mod h1:xUsJbQ/Fp4kEt7AFgCuvyX4a71u8h9jB8tj/ORgOZ7o= github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= -github.com/go-logfmt/logfmt v0.5.0 h1:TrB8swr/68K7m9CcGut2g3UOihhbcbiMAYiuTXdEih4= github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= -github.com/go-sql-driver/mysql v1.4.0 h1:7LxgVwFb2hIQtMm87NdgAVfXjnt4OePseqT1tKx+opk= github.com/go-sql-driver/mysql v1.4.0/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w= -github.com/go-stack/stack v1.8.0 h1:5SgMzNM5HxrEjV0ww2lTmX6E2Izsfxas4+YHWRs3Lsk= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= -github.com/go-task/slim-sprig v0.0.0-20210107165309-348f09dbbbc0 h1:p104kn46Q8WdvHunIJ9dAyjPVtrBPhSr3KT2yUst43I= github.com/go-task/slim-sprig v0.0.0-20210107165309-348f09dbbbc0/go.mod h1:fyg7847qk6SyHyPtNmDHnmrv/HOrqktSC+C9fM+CJOE= -github.com/gogo/googleapis v1.1.0 h1:kFkMAZBNAn4j7K0GiZr8cRYzejq68VbheufiV3YuyFI= github.com/gogo/googleapis v1.1.0/go.mod h1:gf4bu3Q80BeJ6H1S1vYPm8/ELATdvryBaNFGgqEef3s= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.2.0/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= -github.com/gogo/protobuf v1.2.1 h1:/s5zKNz0uPFCZ5hddgPdo2TK2TVrUNMn0OOX8/aZMTE= github.com/gogo/protobuf v1.2.1/go.mod h1:hp+jE20tsWTFYpLwKvXlhS1hjn+gTNwPg2I6zVXpSg4= -github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b h1:VKtxabqXZkF25pY9ekfRL6a582T4P37/31XEstQ5p58= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= github.com/golang/groupcache v0.0.0-20160516000752-02826c3e7903/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20191027212112-611e8accdfc9/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= -github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e h1:1r7pUrabqp18hOBcwBwiTsbnFeTZHV9eER/QT5JVZxY= github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= -github.com/golang/lint v0.0.0-20180702182130-06c8688daad7 h1:2hRPrmiwPrp3fQX967rNJIhQPtiGXdlQWAxKbKw3VHA= github.com/golang/lint v0.0.0-20180702182130-06c8688daad7/go.mod h1:tluoj9z5200jBnyusfRPU2LqT6J+DAorxEvtC7LHB+E= github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/mock v1.2.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/mock v1.4.0/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= -github.com/golang/mock v1.4.4 h1:l75CXGRSwbaYNpl/Z2X1XIIAMSCquvXgpVZDhwEIJsc= github.com/golang/mock v1.4.4/go.mod h1:l3mdAwkq5BuhzHwde/uurv3sEJeZMXNpwsxVWU71h+4= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= @@ -331,11 +254,9 @@ github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:W github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0= github.com/golang/protobuf v1.4.1/go.mod h1:U8fpvMrcmy5pZrNK1lt4xCsGvpyWQ/VVv6QDs8UjoX8= github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= -github.com/golang/protobuf v1.4.3 h1:JjCZWpVbqXDqFVmTfYWEVTMIYrL/NPdPSCHPJ0T/raM= github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= github.com/golang/snappy v0.0.0-20180518054509-2e65f85255db/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= -github.com/golang/snappy v0.0.2 h1:aeE13tS0IiQgFjYdoL8qN3K1N2bXXtI6Vi51/y7BpMw= github.com/golang/snappy v0.0.2/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/google/btree v0.0.0-20180124185431-e89373fe6b4a/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= @@ -349,95 +270,59 @@ github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/ github.com/google/go-cmp v0.5.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.4 h1:L8R9j+yAqZuZjsqh/z+F1NCffTKKLShY6zXTItVIZ8M= github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-github v17.0.0+incompatible h1:N0LgJ1j65A7kfXrZnUDaYCs/Sf4rEjNlfyDHW9dolSY= github.com/google/go-github v17.0.0+incompatible/go.mod h1:zLgOLi98H3fifZn+44m+umXrS52loVEgC2AApnigrVQ= -github.com/google/go-querystring v1.0.0 h1:Xkwi/a1rcvNg1PPYe5vI8GbeBY/jrVuDX5ASuANWTrk= github.com/google/go-querystring v1.0.0/go.mod h1:odCYkC5MyYFN7vkCjXpyrEuKhc/BUO6wN/zVPAxq5ck= -github.com/google/gofuzz v1.0.0 h1:A8PeW59pxE9IoFRqBp37U+mSNaQoZ46F1f0f863XSXw= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= -github.com/google/martian v2.1.0+incompatible h1:/CP5g8u/VJHijgedC/Legn3BAbAaWPgecwXBIDzw5no= github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs= -github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57 h1:eqyIo2HjKhKe/mJzTG8n4VqvLXIOEG+SLdDqX7xGtkY= github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= -github.com/google/renameio v0.1.0 h1:GOZbcHa3HfsPKPlmyPyN2KEohoMXOhdMbHrvbpl2QaA= github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.1.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.1.5/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.2.0 h1:qJYtXnJRWmpe7m/3XlyhrsLrEURqHRM2kxzoxXqyUDs= github.com/google/uuid v1.2.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= -github.com/googleapis/gax-go v2.0.0+incompatible h1:j0GKcs05QVmm7yesiZq2+9cxHkNK9YM6zKx4D2qucQU= github.com/googleapis/gax-go v2.0.0+incompatible/go.mod h1:SFVmujtThgffbyetf+mdk2eWhX2bMyUtNHzFKcPA9HY= -github.com/googleapis/gax-go/v2 v2.0.3 h1:siORttZ36U2R/WjiJuDz8znElWBiAlO9rVt+mqJt0Cc= github.com/googleapis/gax-go/v2 v2.0.3/go.mod h1:LLvjysVCY1JZeum8Z6l8qUty8fiNwE08qbEPm1M08qg= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= github.com/gopherjs/gopherjs v0.0.0-20181103185306-d547d1d9531e/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= github.com/gopherjs/gopherjs v0.0.0-20190309154008-847fc94819f9/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= -github.com/gopherjs/gopherjs v0.0.0-20190910122728-9d188e94fb99 h1:twflg0XRTjwKpxb/jFExr4HGq6on2dEOmnL6FV+fgPw= github.com/gopherjs/gopherjs v0.0.0-20190910122728-9d188e94fb99/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= -github.com/gorilla/context v1.1.1 h1:AWwleXJkX/nhcU9bZSnZoi3h/qGYqQAGhq6zZe/aQW8= github.com/gorilla/context v1.1.1/go.mod h1:kBGZzfjB9CEq2AlWe17Uuf7NDRt0dE0s8S51q0aT7Yg= github.com/gorilla/mux v1.6.2/go.mod h1:1lud6UwP+6orDFRuTfBEV8e9/aOM/c4fVVCaMa2zaAs= -github.com/gorilla/mux v1.7.3 h1:gnP5JzjVOuiZD07fKKToCAOjS0yOpj/qPETTXCCS6hw= github.com/gorilla/mux v1.7.3/go.mod h1:1lud6UwP+6orDFRuTfBEV8e9/aOM/c4fVVCaMa2zaAs= github.com/gorilla/websocket v0.0.0-20170926233335-4201258b820c/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoAMk2YaljkQ= github.com/gorilla/websocket v1.4.2 h1:+/TMaTYc4QFitKJxsQ7Yye35DkWvkdLcvGKqM+x0Ufc= github.com/gorilla/websocket v1.4.2/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE= github.com/gosuri/uilive v0.0.0-20170323041506-ac356e6e42cd/go.mod h1:qkLSc0A5EXSP6B04TrN4oQoxqFI7A8XvoXSlJi8cwk8= -github.com/gosuri/uilive v0.0.3 h1:kvo6aB3pez9Wbudij8srWo4iY6SFTTxTKOkb+uRCE8I= github.com/gosuri/uilive v0.0.3/go.mod h1:qkLSc0A5EXSP6B04TrN4oQoxqFI7A8XvoXSlJi8cwk8= github.com/gosuri/uiprogress v0.0.0-20170224063937-d0567a9d84a1/go.mod h1:C1RTYn4Sc7iEyf6j8ft5dyoZ4212h8G1ol9QQluh5+0= -github.com/gosuri/uiprogress v0.0.1 h1:0kpv/XY/qTmFWl/SkaJykZXrBBzwwadmW8fRb7RJSxw= github.com/gosuri/uiprogress v0.0.1/go.mod h1:C1RTYn4Sc7iEyf6j8ft5dyoZ4212h8G1ol9QQluh5+0= -github.com/gregjones/httpcache v0.0.0-20180305231024-9cad4c3443a7 h1:pdN6V1QBWetyv/0+wjACpqVH+eVULgEjkurDLq3goeM= github.com/gregjones/httpcache v0.0.0-20180305231024-9cad4c3443a7/go.mod h1:FecbI9+v66THATjSRHfNgh1IVFe/9kFxbXtjV0ctIMA= -github.com/grpc-ecosystem/go-grpc-middleware v1.0.1-0.20190118093823-f849b5445de4 h1:z53tR0945TRRQO/fLEVPI6SMv7ZflF0TEaTAoU7tOzg= github.com/grpc-ecosystem/go-grpc-middleware v1.0.1-0.20190118093823-f849b5445de4/go.mod h1:FiyG127CGDf3tlThmgyCl78X/SZQqEOJBCDaAfeWzPs= -github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 h1:Ovs26xHkKqVztRpIrF/92BcuyuQ/YW4NSIpoGtfXNho= github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgfV/d3M/q6VIi02HzZEHgUlZvzk= github.com/grpc-ecosystem/grpc-gateway v1.5.0/go.mod h1:RSKVYQBd5MCa4OVpNdGskqpgL2+G+NZTnrVHpWWfpdw= -github.com/grpc-ecosystem/grpc-gateway v1.9.5 h1:UImYN5qQ8tuGpGE16ZmjvcTtTw24zw1QAp/SlnNrZhI= github.com/grpc-ecosystem/grpc-gateway v1.9.5/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY= -github.com/hashicorp/consul/api v1.3.0 h1:HXNYlRkkM/t+Y/Yhxtwcy02dlYwIaoxzvxPnS+cqy78= github.com/hashicorp/consul/api v1.3.0/go.mod h1:MmDNSzIMUjNpY/mQ398R4bk2FnqQLoPndWW5VkKPlCE= -github.com/hashicorp/consul/sdk v0.3.0 h1:UOxjlb4xVNF93jak1mzzoBatyFju9nrkxpVwIp/QqxQ= github.com/hashicorp/consul/sdk v0.3.0/go.mod h1:VKf9jXwCTEY1QZP2MOLRhb5i/I/ssyNV1vwHyQBF0x8= -github.com/hashicorp/errwrap v1.0.0 h1:hLrqtEDnRye3+sgx6z4qVLNuviH3MR5aQ0ykNJa/UYA= github.com/hashicorp/errwrap v1.0.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= -github.com/hashicorp/go-cleanhttp v0.5.1 h1:dH3aiDG9Jvb5r5+bYHsikaOUIpcM0xvgMXVoDkXMzJM= github.com/hashicorp/go-cleanhttp v0.5.1/go.mod h1:JpRdi6/HCYpAwUzNwuwqhbovhLtngrth3wmdIIUrZ80= -github.com/hashicorp/go-immutable-radix v1.0.0 h1:AKDB1HM5PWEA7i4nhcpwOrO2byshxBjXVn/J/3+z5/0= github.com/hashicorp/go-immutable-radix v1.0.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60= -github.com/hashicorp/go-msgpack v0.5.3 h1:zKjpN5BK/P5lMYrLmBHdBULWbJ0XpYR+7NGzqkZzoD4= github.com/hashicorp/go-msgpack v0.5.3/go.mod h1:ahLV/dePpqEmjfWmKiqvPkv/twdG7iPBM1vqhUKIvfM= -github.com/hashicorp/go-multierror v1.0.0 h1:iVjPR7a6H0tWELX5NxNe7bYopibicUzc7uPribsnS6o= github.com/hashicorp/go-multierror v1.0.0/go.mod h1:dHtQlpGsu+cZNNAkkCN/P3hoUDHhCYQXV3UM06sGGrk= -github.com/hashicorp/go-rootcerts v1.0.0 h1:Rqb66Oo1X/eSV1x66xbDccZjhJigjg0+e82kpwzSwCI= github.com/hashicorp/go-rootcerts v1.0.0/go.mod h1:K6zTfqpRlCUIjkwsN4Z+hiSfzSTQa6eBIzfwKfwNnHU= -github.com/hashicorp/go-sockaddr v1.0.0 h1:GeH6tui99pF4NJgfnhp+L6+FfobzVW3Ah46sLo0ICXs= github.com/hashicorp/go-sockaddr v1.0.0/go.mod h1:7Xibr9yA9JjQq1JpNB2Vw7kxv8xerXegt+ozgdvDeDU= -github.com/hashicorp/go-syslog v1.0.0 h1:KaodqZuhUoZereWVIYmpUgZysurB1kBLX2j0MwMrUAE= github.com/hashicorp/go-syslog v1.0.0/go.mod h1:qPfqrKkXGihmCqbJM2mZgkZGvKG1dFdvsLplgctolz4= github.com/hashicorp/go-uuid v1.0.0/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= -github.com/hashicorp/go-uuid v1.0.1 h1:fv1ep09latC32wFoVwnqcnKJGnMSdBanPczbHAYm1BE= github.com/hashicorp/go-uuid v1.0.1/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= -github.com/hashicorp/go-version v1.2.0 h1:3vNe/fWF5CBgRIguda1meWhsZHy3m8gCJ5wx+dIzX/E= github.com/hashicorp/go-version v1.2.0/go.mod h1:fltr4n8CU8Ke44wwGCBoEymUuxUHl09ZGVZPK5anwXA= -github.com/hashicorp/go.net v0.0.1 h1:sNCoNyDEvN1xa+X0baata4RdcpKwcMS6DH+xwfqPgjw= github.com/hashicorp/go.net v0.0.1/go.mod h1:hjKkEWcCURg++eb33jQU7oqQcI9XDCnUzHA0oac0k90= github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= -github.com/hashicorp/golang-lru v0.5.4 h1:YDjusn29QI/Das2iO9M0BHnIbxPeyuCHsjMW+lJfyTc= github.com/hashicorp/golang-lru v0.5.4/go.mod h1:iADmTwqILo4mZ8BN3D2Q6+9jd8WM5uGBxy+E8yxSoD4= -github.com/hashicorp/logutils v1.0.0 h1:dLEQVugN8vlakKOUE3ihGLTZJRB4j+M2cdTm/ORI65Y= github.com/hashicorp/logutils v1.0.0/go.mod h1:QIAnNjmIWmVIIkWDTG1z5v++HQmx9WQRO+LraFDTW64= -github.com/hashicorp/mdns v1.0.0 h1:WhIgCr5a7AaVH6jPUwjtRuuE7/RDufnUvzIr48smyxs= github.com/hashicorp/mdns v1.0.0/go.mod h1:tL+uN++7HEJ6SQLQ2/p+z2pH24WQKWjBPkE0mNTz8vQ= -github.com/hashicorp/memberlist v0.1.3 h1:EmmoJme1matNzb+hMpDuR/0sbJSUisxyqBGG676r31M= github.com/hashicorp/memberlist v0.1.3/go.mod h1:ajVTdAv/9Im8oMAAj5G31PhhMCZJV2pPBoIllUwCN7I= -github.com/hashicorp/serf v0.8.2 h1:YZ7UKsJv+hKjqGVUUbtE3HNj79Eln2oQ75tniF6iPt0= github.com/hashicorp/serf v0.8.2/go.mod h1:6hOLApaqBFA1NXqRQAsxw9QxuDEvNxSQRwA/JwenrHc= -github.com/hpcloud/tail v1.0.0 h1:nfCOvKYfkgYP8hkirhJocXT2+zOD8yUNjXaWfTlyFKI= github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= github.com/huandu/xstrings v1.0.0/go.mod h1:4qWG/gcEcfX4z/mBDHJ++3ReCw9ibxbsNJbcucJdbSo= github.com/huandu/xstrings v1.2.0/go.mod h1:DvyZB1rfVYsBIigL8HwpZgxHwXozlTgGqn63UyNX5k4= @@ -446,191 +331,124 @@ github.com/huandu/xstrings v1.3.0/go.mod h1:y5/lhBue+AyNmUVz9RLU9xbLR0o4KIIExikq github.com/huandu/xstrings v1.3.1/go.mod h1:y5/lhBue+AyNmUVz9RLU9xbLR0o4KIIExikq4ovT0aE= github.com/huandu/xstrings v1.3.2 h1:L18LIDzqlW6xN2rEkpdV8+oL/IXWJ1APd+vsdYy4Wdw= github.com/huandu/xstrings v1.3.2/go.mod h1:y5/lhBue+AyNmUVz9RLU9xbLR0o4KIIExikq4ovT0aE= -github.com/hudl/fargo v1.3.0 h1:0U6+BtN6LhaYuTnIJq4Wyq5cpn6O2kWrxAtcqBmYY6w= github.com/hudl/fargo v1.3.0/go.mod h1:y3CKSmjA+wD2gak7sUSXTAoopbhU08POFhmITJgmKTg= -github.com/inconshreveable/mousetrap v1.0.0 h1:Z8tu5sraLXCXIcARxBp/8cbvlwVa7Z1NHg9XEKhtSvM= github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= -github.com/influxdata/influxdb1-client v0.0.0-20191209144304-8bf82d3c094d h1:/WZQPMZNsjZ7IlCpsLGdQBINg5bxKQ1K1sh6awxLtkA= github.com/influxdata/influxdb1-client v0.0.0-20191209144304-8bf82d3c094d/go.mod h1:qj24IKcXYK6Iy9ceXlo3Tc+vtHo9lIhSX5JddghvEPo= -github.com/jellevandenhooff/dkim v0.0.0-20150330215556-f50fe3d243e1 h1:ujPKutqRlJtcfWk6toYVYagwra7HQHbXOaS171b4Tg8= github.com/jellevandenhooff/dkim v0.0.0-20150330215556-f50fe3d243e1/go.mod h1:E0B/fFc00Y+Rasa88328GlI/XbtyysCtTHZS8h7IrBU= github.com/jessevdk/go-flags v1.4.0 h1:4IU2WS7AumrZ/40jfhf4QVDMsQwqA7VEHozFRrGARJA= github.com/jessevdk/go-flags v1.4.0/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= -github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af h1:pmfjZENx5imkbgOkpRUYLnmbU7UEFbjtDA2hxJ1ichM= github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af/go.mod h1:Nht3zPeWKUH0NzdCt2Blrr5ys8VGpn0CEB0cQHVjt7k= -github.com/jonboulle/clockwork v0.1.0 h1:VKV+ZcuP6l3yW9doeqz6ziZGgcynBVQO+obU0+0hcPo= github.com/jonboulle/clockwork v0.1.0/go.mod h1:Ii8DK3G1RaLaWxj9trq07+26W01tbo22gdxWY5EU2bo= -github.com/jpillora/backoff v1.0.0 h1:uvFg412JmmHBHw7iwprIxkPMI+sGQ4kzOWsMeHnm2EA= github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4= github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= github.com/json-iterator/go v1.1.7/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/json-iterator/go v1.1.8/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/json-iterator/go v1.1.9/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= -github.com/json-iterator/go v1.1.10 h1:Kz6Cvnvv2wGdaG/V8yMvfkmNiXq9Ya2KUv4rouJJr68= github.com/json-iterator/go v1.1.10/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= -github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024 h1:rBMNdlhTLzJjJSDIjNEXX1Pz3Hmwmz91v+zycvx9PJc= github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= github.com/jtolds/gls v4.2.1+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= -github.com/jtolds/gls v4.20.0+incompatible h1:xdiiI2gbIgH/gLH7ADydsJ1uDOEzR8yvV7C0MuV77Wo= github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= -github.com/julienschmidt/httprouter v1.3.0 h1:U0609e9tgbseu3rBINet9P48AI/D3oJs4dN7jwJOQ1U= github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8IZAc4RVcycCCAKdM= -github.com/kisielk/errcheck v1.1.0 h1:ZqfnKyx9KGpRcW04j5nnPDgRgoXUeLh2YFBeFzphcA0= github.com/kisielk/errcheck v1.1.0/go.mod h1:EZBBE59ingxPouuu3KfxchcWSUPOHkagtvWXihfKN4Q= -github.com/kisielk/gotool v1.0.0 h1:AV2c/EiW3KqPNT9ZKl07ehoAGi4C5/01Cfbblndcapg= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= -github.com/konsorten/go-windows-terminal-sequences v1.0.3 h1:CE8S1cTafDpPvMhIxNJKvHsGVBgn1xWYf1NbHQhywc8= github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= -github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515 h1:T+h1c/A9Gawja4Y9mFVWj2vyii2bbUNDw3kt9VxK2EY= github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= github.com/kr/pretty v0.2.0/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= github.com/kr/pretty v0.2.1 h1:Fmg33tUaq4/8ym9TJN1x7sLJnHVwhP33CNkpYV/7rwI= github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= -github.com/kr/pty v1.1.3 h1:/Um6a/ZmD5tF7peoOJ5oN5KMQ0DrGVQSXLNwyckutPk= github.com/kr/pty v1.1.3/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0 h1:45sCR5RtlFHMR4UwH9sdQ5TC8v0qDQCHnXt+kaKSTVE= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= -github.com/lightstep/lightstep-tracer-common/golang/gogo v0.0.0-20190605223551-bc2310a04743 h1:143Bb8f8DuGWck/xpNUOckBVYfFbBTnLevfRZ1aVVqo= github.com/lightstep/lightstep-tracer-common/golang/gogo v0.0.0-20190605223551-bc2310a04743/go.mod h1:qklhhLq1aX+mtWk9cPHPzaBjWImj5ULL6C7HFJtXQMM= -github.com/lightstep/lightstep-tracer-go v0.18.1 h1:vi1F1IQ8N7hNWytK9DpJsUfQhGuNSc19z330K6vl4zk= github.com/lightstep/lightstep-tracer-go v0.18.1/go.mod h1:jlF1pusYV4pidLvZ+XD0UBX0ZE6WURAspgAczcDHrL4= github.com/lucas-clemente/quic-go v0.7.1-0.20190401152353-907071221cf9/go.mod h1:PpMmPfPKO9nKJ/psF49ESTAGQSdfXxlg1otPbEB2nOw= github.com/lucas-clemente/quic-go v0.18.0/go.mod h1:yXttHsSNxQi8AWijC/vLP+OJczXqzHSOcJrM5ITUlCg= -github.com/lucas-clemente/quic-go v0.19.3 h1:eCDQqvGBB+kCTkA0XrAFtNe81FMa0/fn4QSoeAbmiF4= github.com/lucas-clemente/quic-go v0.19.3/go.mod h1:ADXpNbTQjq1hIzCpB+y/k5iz4n4z4IwqoLb94Kh5Hu8= -github.com/lukechampine/stm v0.0.0-20191022212748-05486c32d236 h1:kYGljja/OYeRs672gWqkHNMDVAtLN8/0UZpJNDCQ3s4= github.com/lukechampine/stm v0.0.0-20191022212748-05486c32d236/go.mod h1:wTLsd5FC9rts7GkMpsPGk64CIuea+03yaLAp19Jmlg8= -github.com/lunixbochs/vtclean v1.0.0 h1:xu2sLAri4lGiovBDQKxl5mrXyESr3gUr5m5SM5+LVb8= github.com/lunixbochs/vtclean v1.0.0/go.mod h1:pHhQNgMf3btfWnGBVipUOjRYhoOsdGqdm/+2c2E2WMI= -github.com/lyft/protoc-gen-validate v0.0.13 h1:KNt/RhmQTOLr7Aj8PsJ7mTronaFyx80mRTT9qF261dA= github.com/lyft/protoc-gen-validate v0.0.13/go.mod h1:XbGvPuh87YZc5TdIa2/I4pLk0QoUACkjt2znoq26NVQ= -github.com/mailru/easyjson v0.0.0-20190312143242-1de009706dbe h1:W/GaMY0y69G4cFlmsC6B9sbuo2fP8OFP1ABjt4kPz+w= github.com/mailru/easyjson v0.0.0-20190312143242-1de009706dbe/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= github.com/marten-seemann/qpack v0.2.0/go.mod h1:F7Gl5L1jIgN1D11ucXefiuJS9UMVP2opoCp2jDKb7wc= -github.com/marten-seemann/qpack v0.2.1 h1:jvTsT/HpCn2UZJdP+UUB53FfUUgeOyG5K1ns0OJOGVs= github.com/marten-seemann/qpack v0.2.1/go.mod h1:F7Gl5L1jIgN1D11ucXefiuJS9UMVP2opoCp2jDKb7wc= github.com/marten-seemann/qtls v0.2.3/go.mod h1:xzjG7avBwGGbdZ8dTGxlBnLArsVKLvwmjgmPuiQEcYk= -github.com/marten-seemann/qtls v0.10.0 h1:ECsuYUKalRL240rRD4Ri33ISb7kAQ3qGDlrrl55b2pc= github.com/marten-seemann/qtls v0.10.0/go.mod h1:UvMd1oaYDACI99/oZUYLzMCkBXQVT0aGm99sJhbT8hs= github.com/marten-seemann/qtls-go1-15 v0.1.0/go.mod h1:GyFwywLKkRt+6mfU99csTEY1joMZz5vmB1WNZH3P81I= -github.com/marten-seemann/qtls-go1-15 v0.1.1 h1:LIH6K34bPVttyXnUWixk0bzH6/N07VxbSabxn5A5gZQ= github.com/marten-seemann/qtls-go1-15 v0.1.1/go.mod h1:GyFwywLKkRt+6mfU99csTEY1joMZz5vmB1WNZH3P81I= -github.com/mattn/go-colorable v0.0.9 h1:UVL0vNpWh04HeJXV0KLcaT7r06gOH2l4OW6ddYRUIY4= github.com/mattn/go-colorable v0.0.9/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU= github.com/mattn/go-isatty v0.0.3/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4= github.com/mattn/go-isatty v0.0.4/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4= github.com/mattn/go-isatty v0.0.7/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s= -github.com/mattn/go-isatty v0.0.10 h1:qxFzApOv4WsAL965uUPIsXzAKCZxN2p9UqdhFS4ZW10= github.com/mattn/go-isatty v0.0.10/go.mod h1:qgIWMr58cqv1PHHyhnkY9lrL7etaEgOFcMEpPG5Rm84= -github.com/mattn/go-runewidth v0.0.2 h1:UnlwIPBGaTZfPQ6T1IGzPI0EkYAQmT9fAEJ/poFC63o= github.com/mattn/go-runewidth v0.0.2/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU= github.com/mattn/go-sqlite3 v1.7.0/go.mod h1:FPy6KqzDD04eiIsT53CuJW3U88zkxoIYsOqkbpncsNc= github.com/mattn/go-sqlite3 v1.10.0/go.mod h1:FPy6KqzDD04eiIsT53CuJW3U88zkxoIYsOqkbpncsNc= github.com/mattn/go-sqlite3 v1.13.0/go.mod h1:FPy6KqzDD04eiIsT53CuJW3U88zkxoIYsOqkbpncsNc= -github.com/mattn/go-sqlite3 v2.0.2+incompatible h1:qzw9c2GNT8UFrgWNDhCTqRqYUSmu/Dav/9Z58LGpk7U= github.com/mattn/go-sqlite3 v2.0.2+incompatible/go.mod h1:FPy6KqzDD04eiIsT53CuJW3U88zkxoIYsOqkbpncsNc= -github.com/matttproud/golang_protobuf_extensions v1.0.1 h1:4hp9jkHxhMHkqkrB3Ix0jegS5sx/RkqARlsWZ6pIwiU= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= -github.com/microcosm-cc/bluemonday v1.0.1 h1:SIYunPjnlXcW+gVfvm0IlSeR5U3WZUOLfVmqg85Go44= github.com/microcosm-cc/bluemonday v1.0.1/go.mod h1:hsXNsILzKxV+sX77C5b8FSuKF00vh2OMYv+xgHpAMF4= -github.com/miekg/dns v1.0.14 h1:9jZdLNd/P4+SfEJ0TNyxYpsK8N4GtfylBLqtbYN1sbA= github.com/miekg/dns v1.0.14/go.mod h1:W1PPwlIAgtquWBMBEV9nkV9Cazfe8ScdGz/Lj7v3Nrg= -github.com/mitchellh/cli v1.0.0 h1:iGBIsUe3+HZ/AD/Vd7DErOt5sU9fa8Uj7A2s1aggv1Y= github.com/mitchellh/cli v1.0.0/go.mod h1:hNIlj7HEI86fIcpObd7a0FcrxTWetlwJDGcceTlRvqc= -github.com/mitchellh/go-homedir v1.0.0 h1:vKb8ShqSby24Yrqr/yDYkuFz8d0WUjys40rvnGC8aR0= github.com/mitchellh/go-homedir v1.0.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= -github.com/mitchellh/go-testing-interface v1.0.0 h1:fzU/JVNcaqHQEcVFAKeR41fkiLdIPrefOvVG1VZ96U0= github.com/mitchellh/go-testing-interface v1.0.0/go.mod h1:kRemZodwjscx+RGhAo8eIhFbs2+BFgRtFPeD/KE+zxI= -github.com/mitchellh/gox v0.4.0 h1:lfGJxY7ToLJQjHHwi0EX6uYBdK78egf954SQl13PQJc= github.com/mitchellh/gox v0.4.0/go.mod h1:Sd9lOJ0+aimLBi73mGofS1ycjY8lL3uZM3JPS42BGNg= -github.com/mitchellh/iochan v1.0.0 h1:C+X3KsSTLFVBr/tK1eYN/vs4rJcvsiLU338UhYPJWeY= github.com/mitchellh/iochan v1.0.0/go.mod h1:JwYml1nuB7xOzsp52dPpHFffvOCDupsG0QubkSMEySY= github.com/mitchellh/mapstructure v0.0.0-20160808181253-ca63d7c062ee/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= -github.com/mitchellh/mapstructure v1.1.2 h1:fmNYVwqnSfB9mZU6OS2O6GsXM+wcskZDuKQzvN1EDeE= github.com/mitchellh/mapstructure v1.1.2/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= -github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= -github.com/modern-go/reflect2 v1.0.1 h1:9f412s+6RmYXLWZSEzVVgPGK7C2PphHj5RJrvfx9AWI= github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/mschoch/smat v0.0.0-20160514031455-90eadee771ae/go.mod h1:qAyveg+e4CE+eKJXWVjKXM4ck2QobLqTDytGJbLLhJg= github.com/mschoch/smat v0.2.0 h1:8imxQsjDm8yFEAVBe7azKmKSgzSkZXDuKkSq9374khM= github.com/mschoch/smat v0.2.0/go.mod h1:kc9mz7DoBKqDyiRL7VZN8KvXQMWeTaVnttLRXOlotKw= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= -github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f h1:KUppIJq7/+SVif2QVs3tOP0zanoHgBEVAwHxUSIzRqU= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/nats-io/jwt v0.3.0/go.mod h1:fRYCDE99xlTsqUzISS1Bi75UBJ6ljOJQOAAu5VglpSg= -github.com/nats-io/jwt v0.3.2 h1:+RB5hMpXUUA2dfxuhBTEkMOrYmM+gKIZYS1KjSostMI= github.com/nats-io/jwt v0.3.2/go.mod h1:/euKqTS1ZD+zzjYrY7pseZrTtWQSjujC7xjPc8wL6eU= -github.com/nats-io/nats-server/v2 v2.1.2 h1:i2Ly0B+1+rzNZHHWtD4ZwKi+OU5l+uQo1iDHZ2PmiIc= github.com/nats-io/nats-server/v2 v2.1.2/go.mod h1:Afk+wRZqkMQs/p45uXdrVLuab3gwv3Z8C4HTBu8GD/k= -github.com/nats-io/nats.go v1.9.1 h1:ik3HbLhZ0YABLto7iX80pZLPw/6dx3T+++MZJwLnMrQ= github.com/nats-io/nats.go v1.9.1/go.mod h1:ZjDU1L/7fJ09jvUSRVBR2e7+RnLiiIQyqyzEE/Zbp4w= github.com/nats-io/nkeys v0.1.0/go.mod h1:xpnFELMwJABBLVhffcfd1MZx6VsNRFpEugbxziKVo7w= -github.com/nats-io/nkeys v0.1.3 h1:6JrEfig+HzTH85yxzhSVbjHRJv9cn0p6n3IngIcM5/k= github.com/nats-io/nkeys v0.1.3/go.mod h1:xpnFELMwJABBLVhffcfd1MZx6VsNRFpEugbxziKVo7w= -github.com/nats-io/nuid v1.0.1 h1:5iA8DT8V7q8WK2EScv2padNa/rTESc1KdnPw4TC2paw= github.com/nats-io/nuid v1.0.1/go.mod h1:19wcPz3Ph3q0Jbyiqsd0kePYG7A95tJPxeL+1OSON2c= -github.com/neelance/astrewrite v0.0.0-20160511093645-99348263ae86 h1:D6paGObi5Wud7xg83MaEFyjxQB1W5bz5d0IFppr+ymk= github.com/neelance/astrewrite v0.0.0-20160511093645-99348263ae86/go.mod h1:kHJEU3ofeGjhHklVoIGuVj85JJwZ6kWPaJwCIxgnFmo= -github.com/neelance/sourcemap v0.0.0-20151028013722-8c68805598ab h1:eFXv9Nu1lGbrNbj619aWwZfVF5HBrm9Plte8aNptuTI= github.com/neelance/sourcemap v0.0.0-20151028013722-8c68805598ab/go.mod h1:Qr6/a/Q4r9LP1IltGz7tA7iOK1WonHEYhu1HRBA7ZiM= github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A= -github.com/nxadm/tail v1.4.8 h1:nPr65rt6Y5JFSKQO7qToXr7pePgD6Gwiw05lkbyAQTE= github.com/nxadm/tail v1.4.8/go.mod h1:+ncqLTQzXmGhMZNUePPaPqPvBxHAIsmXswZKocGu+AU= -github.com/oklog/oklog v0.3.2 h1:wVfs8F+in6nTBMkA7CbRw+zZMIB7nNM825cM1wuzoTk= github.com/oklog/oklog v0.3.2/go.mod h1:FCV+B7mhrz4o+ueLpx+KqkyXRGMWOYEvfiXtdGtbWGs= -github.com/oklog/run v1.0.0 h1:Ru7dDtJNOyC66gQ5dQmaCa0qIsAUFY3sFpK1Xk8igrw= github.com/oklog/run v1.0.0/go.mod h1:dlhp/R75TPv97u0XWUtDeV/lRKWPKSdTuV0TZvrmrQA= -github.com/olekukonko/tablewriter v0.0.0-20170122224234-a0225b3f23b5 h1:58+kh9C6jJVXYjt8IE48G2eWl6BjwU5Gj0gqY84fy78= github.com/olekukonko/tablewriter v0.0.0-20170122224234-a0225b3f23b5/go.mod h1:vsDQFd/mU46D+Z4whnwzcISnGGzXWMclvtLoiIKAKIo= github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v1.7.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v1.12.1/go.mod h1:zj2OWP4+oCPe1qIXoGWkgMRwljMUYCdkwsT2108oapk= github.com/onsi/ginkgo v1.14.0/go.mod h1:iSB4RoI2tjJc9BBv4NKIKWKya62Rps+oPG/Lv9klQyY= github.com/onsi/ginkgo v1.14.2/go.mod h1:iSB4RoI2tjJc9BBv4NKIKWKya62Rps+oPG/Lv9klQyY= -github.com/onsi/ginkgo v1.16.1 h1:foqVmeWDD6yYpK+Yz3fHyNIxFYNxswxqNFjSKe+vI54= github.com/onsi/ginkgo v1.16.1/go.mod h1:CObGmKUOKaSC0RjmoAK7tKyn4Azo5P2IWuoMnvwxz1E= github.com/onsi/gomega v1.4.3/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY= github.com/onsi/gomega v1.7.1/go.mod h1:XdKZgCCFLUoM/7CFJVPcG8C1xQ1AJ0vpAezJrB7JYyY= github.com/onsi/gomega v1.10.1/go.mod h1:iN09h71vgCQne3DLsj+A5owkum+a2tYe+TOCB1ybHNo= github.com/onsi/gomega v1.10.3/go.mod h1:V9xEwhxec5O8UDM77eCW8vLymOMltsqPVYWrpDsH8xc= -github.com/onsi/gomega v1.11.0 h1:+CqWgvj0OZycCaqclBD1pxKHAU+tOkHmQIWvDHq2aug= github.com/onsi/gomega v1.11.0/go.mod h1:azGKhqFUon9Vuj0YmTfLSmx0FUwqXYSTl5re8lQLTUg= -github.com/op/go-logging v0.0.0-20160315200505-970db520ece7 h1:lDH9UUVJtmYCjyT0CI4q8xvlXPxeZ0gYCVvWbmPlp88= github.com/op/go-logging v0.0.0-20160315200505-970db520ece7/go.mod h1:HzydrMdWErDVzsI23lYNej1Htcns9BCg93Dk0bBINWk= -github.com/opentracing-contrib/go-observer v0.0.0-20170622124052-a52f23424492 h1:lM6RxxfUMrYL/f8bWEUqdXrANWtrL7Nndbm9iFN0DlU= github.com/opentracing-contrib/go-observer v0.0.0-20170622124052-a52f23424492/go.mod h1:Ngi6UdF0k5OKD5t5wlmGhe/EDKPoUM3BXZSSfIuJbis= -github.com/opentracing/basictracer-go v1.0.0 h1:YyUAhaEfjoWXclZVJ9sGoNct7j4TVk7lZWlQw5UXuoo= github.com/opentracing/basictracer-go v1.0.0/go.mod h1:QfBfYuafItcjQuMwinw9GhYKwFXS9KnPs5lxoYwgW74= github.com/opentracing/opentracing-go v1.0.2/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= -github.com/opentracing/opentracing-go v1.1.0 h1:pWlfV3Bxv7k65HYwkikxat0+s3pV4bsqf19k25Ur8rU= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= -github.com/openzipkin-contrib/zipkin-go-opentracing v0.4.5 h1:ZCnq+JUrvXcDVhX/xRolRBZifmabN1HcS1wrPSvxhrU= github.com/openzipkin-contrib/zipkin-go-opentracing v0.4.5/go.mod h1:/wsWhb9smxSfWAKL3wpBW7V8scJMt8N8gnaMCS9E/cA= github.com/openzipkin/zipkin-go v0.1.1/go.mod h1:NtoC/o8u3JlF1lSlyPNswIbeQH9bJTmOf0Erfk+hxe8= github.com/openzipkin/zipkin-go v0.1.6/go.mod h1:QgAqvLzwWbR/WpD4A3cGpPtJrZXNIiJc5AZX7/PBEpw= github.com/openzipkin/zipkin-go v0.2.1/go.mod h1:NaW6tEwdmWMaCDZzg8sh+IBNOxHMPnhQw8ySjnjRyN4= -github.com/openzipkin/zipkin-go v0.2.2 h1:nY8Hti+WKaP0cRsSeQ026wU03QsM762XBeCXBb9NAWI= github.com/openzipkin/zipkin-go v0.2.2/go.mod h1:NaW6tEwdmWMaCDZzg8sh+IBNOxHMPnhQw8ySjnjRyN4= -github.com/pact-foundation/pact-go v1.0.4 h1:OYkFijGHoZAYbOIb1LWXrwKQbMMRUv1oQ89blD2Mh2Q= github.com/pact-foundation/pact-go v1.0.4/go.mod h1:uExwJY4kCzNPcHRj+hCR/HBbOOIwwtUjcrb0b5/5kLM= -github.com/pascaldekloe/goe v0.0.0-20180627143212-57f6aae5913c h1:Lgl0gzECD8GnQ5QCWA8o6BtfL6mDH5rQgM4/fX3avOs= github.com/pascaldekloe/goe v0.0.0-20180627143212-57f6aae5913c/go.mod h1:lzWF7FIEvWOWxwDKqyGYQf6ZUaNfKdP144TG7ZOy1lc= -github.com/pborman/uuid v1.2.0 h1:J7Q5mO4ysT1dv8hyrUGHb9+ooztCXu1D8MY8DZYsu3g= github.com/pborman/uuid v1.2.0/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= -github.com/performancecopilot/speed v3.0.0+incompatible h1:2WnRzIquHa5QxaJKShDkLM+sc0JPuwhXzK8OYOyt3Vg= github.com/performancecopilot/speed v3.0.0+incompatible/go.mod h1:/CLtqpZ5gBg1M9iaPbIdPPGyKcA8hKdoy6hAWba7Yac= github.com/philhofer/fwd v1.0.0/go.mod h1:gk3iGcWd9+svBvR0sR+KPcfE+RNWozjowpeBVG3ZVNU= -github.com/philhofer/fwd v1.1.1 h1:GdGcTjf5RNAxwS4QLsiMzJYj5KEvPJD3Abr261yRQXQ= github.com/philhofer/fwd v1.1.1/go.mod h1:gk3iGcWd9+svBvR0sR+KPcfE+RNWozjowpeBVG3ZVNU= github.com/pierrec/lz4 v1.0.2-0.20190131084431-473cd7ce01a1/go.mod h1:3/3N9NVKO0jef7pBehbT1qWhCMrIgbYNnFAZCqQ5LRc= -github.com/pierrec/lz4 v2.0.5+incompatible h1:2xWsjqPFWcplujydGg4WmhC/6fZqK42wMM8aXeqhl0I= github.com/pierrec/lz4 v2.0.5+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= github.com/pion/datachannel v1.4.21 h1:3ZvhNyfmxsAqltQrApLPQMhSFNA+aT87RqyCq4OXmf0= github.com/pion/datachannel v1.4.21/go.mod h1:oiNyP4gHx2DIwRzX/MFyH0Rz/Gz05OgBlayAI2hAWjg= @@ -654,7 +472,6 @@ github.com/pion/mdns v0.0.4/go.mod h1:R1sL0p50l42S5lJs91oNdUL58nm0QHrhxnSegr++qC github.com/pion/mdns v0.0.5 h1:Q2oj/JB3NqfzY9xGZ1fPzZzK7sDSD8rZPOvcIQ10BCw= github.com/pion/mdns v0.0.5/go.mod h1:UgssrvdD3mxpi8tMxAXbsppL3vJ4Jipw1mTCW+al01g= github.com/pion/quic v0.1.1/go.mod h1:zEU51v7ru8Mp4AUBJvj6psrSth5eEFNnVQK5K48oV3k= -github.com/pion/quic v0.1.4 h1:bNz9sCJjlM3GqMdq7Fne57FiWfdyiJ++yHVbuqeoD3Y= github.com/pion/quic v0.1.4/go.mod h1:dBhNvkLoQqRwfi6h3Vqj3IcPLgiW7rkZxBbRdp7Vzvk= github.com/pion/randutil v0.0.0/go.mod h1:XcJrSMMbbMRhASFVOlj/5hQial/Y8oH/HVo7TBZq+j8= github.com/pion/randutil v0.1.0 h1:CFG1UdESneORglEsnimhUjf33Rwjubwj6xfiOXBa3mA= @@ -672,7 +489,6 @@ github.com/pion/sctp v1.7.10/go.mod h1:EhpTUQu1/lcK3xI+eriS6/96fWetHGCvBi9MSsnaB github.com/pion/sctp v1.7.11/go.mod h1:EhpTUQu1/lcK3xI+eriS6/96fWetHGCvBi9MSsnaBN0= github.com/pion/sctp v1.7.12 h1:GsatLufywVruXbZZT1CKg+Jr8ZTkwiPnmUC/oO9+uuY= github.com/pion/sctp v1.7.12/go.mod h1:xFe9cLMZ5Vj6eOzpyiKjT9SwGM4KpK/8Jbw5//jc+0s= -github.com/pion/sdp/v2 v2.4.0 h1:luUtaETR5x2KNNpvEMv/r4Y+/kzImzbz4Lm1z8eQNQI= github.com/pion/sdp/v2 v2.4.0/go.mod h1:L2LxrOpSTJbAns244vfPChbciR/ReU1KWfG04OpkR7E= github.com/pion/sdp/v3 v3.0.4 h1:2Kf+dgrzJflNCSw3TV5v2VLeI0s/qkzy2r5jlR0wzf8= github.com/pion/sdp/v3 v3.0.4/go.mod h1:bNiSknmJE0HYBprTHXKPQ3+JjacTv5uap92ueJZKsRk= @@ -698,7 +514,6 @@ github.com/pion/turn/v2 v2.0.5/go.mod h1:APg43CFyt/14Uy7heYUOGWdkem/Wu4PhCO/bjyr github.com/pion/udp v0.1.0/go.mod h1:BPELIjbwE9PRbd/zxI/KYBnbo7B6+oA6YuEaNE8lths= github.com/pion/udp v0.1.1 h1:8UAPvyqmsxK8oOjloDk4wUt63TzFe9WEJkg5lChlj7o= github.com/pion/udp v0.1.1/go.mod h1:6AFo+CMdKQm7UiA0eUPA8/eVCTx8jBIITLZHc9DWX5M= -github.com/pion/webrtc/v2 v2.2.26 h1:01hWE26pL3LgqfxvQ1fr6O4ZtyRFFJmQEZK39pHWfFc= github.com/pion/webrtc/v2 v2.2.26/go.mod h1:XMZbZRNHyPDe1gzTIHFcQu02283YO45CbiwFgKvXnmc= github.com/pion/webrtc/v3 v3.0.11/go.mod h1:WEvXneGTeqNmiR59v5jTsxMc4yXQyOQcRsrdAbNwSEU= github.com/pion/webrtc/v3 v3.0.27 h1:cPQEFNFrRSMT11j9c9aTmXzL3ikKAFPE2kR0ZrQcviw= @@ -707,11 +522,9 @@ github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINE github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= -github.com/pkg/profile v1.2.1 h1:F++O52m40owAmADcojzM+9gyjmMOY/T4oYJkgFDH8RE= github.com/pkg/profile v1.2.1/go.mod h1:hJw3o1OdXxsrSjjVksARp5W95eeEaEfptyVZyv6JUPA= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= -github.com/posener/complete v1.1.1 h1:ccV59UEOTzVDnDUEFdT95ZzHVZ+5+158q8+SJb2QV5w= github.com/posener/complete v1.1.1/go.mod h1:em0nMJCgc9GFtwrmVmEMR/ZL6WyhyjMBndrE9hABlRI= github.com/prometheus/client_golang v0.8.0/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= @@ -720,14 +533,12 @@ github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5Fsn github.com/prometheus/client_golang v1.3.0/go.mod h1:hJaj2vgQTGQmVCsAACORcieXFeDPbaTKGT+JTgUa3og= github.com/prometheus/client_golang v1.5.1/go.mod h1:e9GMxYsXl05ICDXkRhurwBS4Q3OK1iX/F2sw+iXX5zU= github.com/prometheus/client_golang v1.7.1/go.mod h1:PY5Wy2awLA44sXw4AOSfFBetzPP4j5+D6mVACh+pe2M= -github.com/prometheus/client_golang v1.9.0 h1:Rrch9mh17XcxvEu9D9DEpb4isxjGBtcevQjKvxPRQIU= github.com/prometheus/client_golang v1.9.0/go.mod h1:FqZLKOZnGdFAhOK4nqGHa7D66IdsO+O441Eve7ptJDU= github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= github.com/prometheus/client_model v0.0.0-20190115171406-56726106282f/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.1.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/client_model v0.2.0 h1:uq5h0d+GuxiXLJLNABMgp2qUWDPiLvgCzz2dUR+/W/M= github.com/prometheus/client_model v0.2.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/common v0.0.0-20180801064454-c7de2306084e/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro= github.com/prometheus/common v0.2.0/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= @@ -735,7 +546,6 @@ github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y8 github.com/prometheus/common v0.7.0/go.mod h1:DjGbpBbp5NYNiECxcL/VnbXCCaQpKd3tt26CguLLsqA= github.com/prometheus/common v0.9.1/go.mod h1:yhUN8i9wzaXS3w1O07YhxHEBxD+W35wd8bs7vj7HSQ4= github.com/prometheus/common v0.10.0/go.mod h1:Tlit/dnDKsSWFlCLTWaA1cyBgKHSMdTB80sz/V91rCo= -github.com/prometheus/common v0.15.0 h1:4fgOnadei3EZvgRwxJ7RMpG1k1pOZth5Pc13tyspaKM= github.com/prometheus/common v0.15.0/go.mod h1:U+gB1OBLb1lF3O42bTCL+FK18tX9Oar16Clt/msog/s= github.com/prometheus/procfs v0.0.0-20180725123919-05ee40e3a273/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= @@ -745,111 +555,66 @@ github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+Gx github.com/prometheus/procfs v0.0.11/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= github.com/prometheus/procfs v0.1.3/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= github.com/prometheus/procfs v0.2.0/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= -github.com/prometheus/procfs v0.3.0 h1:Uehi/mxLK0eiUc0H0++5tpMGTexB8wZ598MIgU8VpDM= github.com/prometheus/procfs v0.3.0/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= -github.com/rcrowley/go-metrics v0.0.0-20181016184325-3113b8401b8a h1:9ZKAASQSHhDYGoxY8uLVpewe1GDZ2vu2Tr/vTdVAkFQ= github.com/rcrowley/go-metrics v0.0.0-20181016184325-3113b8401b8a/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= -github.com/rogpeppe/fastuuid v0.0.0-20150106093220-6724a57986af h1:gu+uRPtBe88sKxUCEXRoeCvVG90TJmwhiqRpvdhQFng= github.com/rogpeppe/fastuuid v0.0.0-20150106093220-6724a57986af/go.mod h1:XWv6SoW27p1b0cqNHllgS5HIMJraePCO15w5zCzIWYg= -github.com/rogpeppe/go-internal v1.3.0 h1:RR9dF3JtopPvtkroDZuVD7qquD0bnHlKSqaQhgwt8yk= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/rs/dnscache v0.0.0-20190621150935-06bb5526f76b/go.mod h1:qe5TWALJ8/a1Lqznoc5BDHpYX/8HU60Hm2AwRmqzxqA= github.com/rs/dnscache v0.0.0-20210201191234-295bba877686 h1:IJ6Df0uxPDtNoByV0KkzVKNseWvZFCNM/S9UoyOMCSI= github.com/rs/dnscache v0.0.0-20210201191234-295bba877686/go.mod h1:qe5TWALJ8/a1Lqznoc5BDHpYX/8HU60Hm2AwRmqzxqA= -github.com/russross/blackfriday v1.5.2 h1:HyvC0ARfnZBqnXwABFeSZHpKvJHJJfPz81GNueLj0oo= github.com/russross/blackfriday v1.5.2/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR/rfWxYHBV53g= -github.com/russross/blackfriday/v2 v2.0.1 h1:lPqVAte+HuHNfhJ/0LC98ESWRz8afy9tM/0RK8m9o+Q= github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= -github.com/ryanuber/columnize v0.0.0-20160712163229-9b3edd62028f h1:UFr9zpz4xgTnIE5yIMtWAMngCdZ9p/+q6lTbgelo80M= github.com/ryanuber/columnize v0.0.0-20160712163229-9b3edd62028f/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts= github.com/ryszard/goskiplist v0.0.0-20150312221310-2dfbae5fcf46 h1:GHRpF1pTW19a8tTFrMLUcfWwyC0pnifVo2ClaLq+hP8= github.com/ryszard/goskiplist v0.0.0-20150312221310-2dfbae5fcf46/go.mod h1:uAQ5PCi+MFsC7HjREoAz1BU+Mq60+05gifQSsHSDG/8= -github.com/samuel/go-zookeeper v0.0.0-20190923202752-2cc03de413da h1:p3Vo3i64TCLY7gIfzeQaUJ+kppEO5WQG3cL8iE8tGHU= github.com/samuel/go-zookeeper v0.0.0-20190923202752-2cc03de413da/go.mod h1:gi+0XIa01GRL2eRQVjQkKGqKF3SF9vZR/HnPullcV2E= -github.com/sclevine/agouti v3.0.0+incompatible h1:8IBJS6PWz3uTlMP3YBIR5f+KAldcGuOeFkFbUWfBgK4= github.com/sclevine/agouti v3.0.0+incompatible/go.mod h1:b4WX9W9L1sfQKXeJf1mUTLZKJ48R1S7H23Ji7oFO5Bw= -github.com/sean-/seed v0.0.0-20170313163322-e2103e2c3529 h1:nn5Wsu0esKSJiIVhscUtVbo7ada43DJhG55ua/hjS5I= github.com/sean-/seed v0.0.0-20170313163322-e2103e2c3529/go.mod h1:DxrIzT+xaE7yg65j358z/aeFdxmN0P9QXhEzd20vsDc= -github.com/sergi/go-diff v1.0.0 h1:Kpca3qRNrduNnOQeazBd0ysaKrUJiIuISHxogkT9RPQ= github.com/sergi/go-diff v1.0.0/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo= -github.com/shurcooL/component v0.0.0-20170202220835-f88ec8f54cc4 h1:Fth6mevc5rX7glNLpbAMJnqKlfIkcTjZCSHEeqvKbcI= github.com/shurcooL/component v0.0.0-20170202220835-f88ec8f54cc4/go.mod h1:XhFIlyj5a1fBNx5aJTbKoIq0mNaPvOagO+HjB3EtxrY= -github.com/shurcooL/events v0.0.0-20181021180414-410e4ca65f48 h1:vabduItPAIz9px5iryD5peyx7O3Ya8TBThapgXim98o= github.com/shurcooL/events v0.0.0-20181021180414-410e4ca65f48/go.mod h1:5u70Mqkb5O5cxEA8nxTsgrgLehJeAw6Oc4Ab1c/P1HM= -github.com/shurcooL/github_flavored_markdown v0.0.0-20181002035957-2122de532470 h1:qb9IthCFBmROJ6YBS31BEMeSYjOscSiG+EO+JVNTz64= github.com/shurcooL/github_flavored_markdown v0.0.0-20181002035957-2122de532470/go.mod h1:2dOwnU2uBioM+SGy2aZoq1f/Sd1l9OkAeAUvjSyvgU0= -github.com/shurcooL/go v0.0.0-20180423040247-9e1955d9fb6e h1:MZM7FHLqUHYI0Y/mQAt3d2aYa0SiNms/hFqC9qJYolM= github.com/shurcooL/go v0.0.0-20180423040247-9e1955d9fb6e/go.mod h1:TDJrrUr11Vxrven61rcy3hJMUqaf/CLWYhHNPmT14Lk= -github.com/shurcooL/go-goon v0.0.0-20170922171312-37c2f522c041 h1:llrF3Fs4018ePo4+G/HV/uQUqEI1HMDjCeOf2V6puPc= github.com/shurcooL/go-goon v0.0.0-20170922171312-37c2f522c041/go.mod h1:N5mDOmsrJOB+vfqUK+7DmDyjhSLIIBnXo9lvZJj3MWQ= -github.com/shurcooL/gofontwoff v0.0.0-20180329035133-29b52fc0a18d h1:Yoy/IzG4lULT6qZg62sVC+qyBL8DQkmD2zv6i7OImrc= github.com/shurcooL/gofontwoff v0.0.0-20180329035133-29b52fc0a18d/go.mod h1:05UtEgK5zq39gLST6uB0cf3NEHjETfB4Fgr3Gx5R9Vw= -github.com/shurcooL/gopherjslib v0.0.0-20160914041154-feb6d3990c2c h1:UOk+nlt1BJtTcH15CT7iNO7YVWTfTv/DNwEAQHLIaDQ= github.com/shurcooL/gopherjslib v0.0.0-20160914041154-feb6d3990c2c/go.mod h1:8d3azKNyqcHP1GaQE/c6dDgjkgSx2BZ4IoEi4F1reUI= -github.com/shurcooL/highlight_diff v0.0.0-20170515013008-09bb4053de1b h1:vYEG87HxbU6dXj5npkeulCS96Dtz5xg3jcfCgpcvbIw= github.com/shurcooL/highlight_diff v0.0.0-20170515013008-09bb4053de1b/go.mod h1:ZpfEhSmds4ytuByIcDnOLkTHGUI6KNqRNPDLHDk+mUU= -github.com/shurcooL/highlight_go v0.0.0-20181028180052-98c3abbbae20 h1:7pDq9pAMCQgRohFmd25X8hIH8VxmT3TaDm+r9LHxgBk= github.com/shurcooL/highlight_go v0.0.0-20181028180052-98c3abbbae20/go.mod h1:UDKB5a1T23gOMUJrI+uSuH0VRDStOiUVSjBTRDVBVag= -github.com/shurcooL/home v0.0.0-20181020052607-80b7ffcb30f9 h1:MPblCbqA5+z6XARjScMfz1TqtJC7TuTRj0U9VqIBs6k= github.com/shurcooL/home v0.0.0-20181020052607-80b7ffcb30f9/go.mod h1:+rgNQw2P9ARFAs37qieuu7ohDNQ3gds9msbT2yn85sg= -github.com/shurcooL/htmlg v0.0.0-20170918183704-d01228ac9e50 h1:crYRwvwjdVh1biHzzciFHe8DrZcYrVcZFlJtykhRctg= github.com/shurcooL/htmlg v0.0.0-20170918183704-d01228ac9e50/go.mod h1:zPn1wHpTIePGnXSHpsVPWEktKXHr6+SS6x/IKRb7cpw= -github.com/shurcooL/httperror v0.0.0-20170206035902-86b7830d14cc h1:eHRtZoIi6n9Wo1uR+RU44C247msLWwyA89hVKwRLkMk= github.com/shurcooL/httperror v0.0.0-20170206035902-86b7830d14cc/go.mod h1:aYMfkZ6DWSJPJ6c4Wwz3QtW22G7mf/PEgaB9k/ik5+Y= -github.com/shurcooL/httpfs v0.0.0-20171119174359-809beceb2371 h1:SWV2fHctRpRrp49VXJ6UZja7gU9QLHwRpIPBN89SKEo= github.com/shurcooL/httpfs v0.0.0-20171119174359-809beceb2371/go.mod h1:ZY1cvUeJuFPAdZ/B6v7RHavJWZn2YPVFQ1OSXhCGOkg= -github.com/shurcooL/httpgzip v0.0.0-20180522190206-b1c53ac65af9 h1:fxoFD0in0/CBzXoyNhMTjvBZYW6ilSnTw7N7y/8vkmM= github.com/shurcooL/httpgzip v0.0.0-20180522190206-b1c53ac65af9/go.mod h1:919LwcH0M7/W4fcZ0/jy0qGght1GIhqyS/EgWGH2j5Q= -github.com/shurcooL/issues v0.0.0-20181008053335-6292fdc1e191 h1:T4wuULTrzCKMFlg3HmKHgXAF8oStFb/+lOIupLV2v+o= github.com/shurcooL/issues v0.0.0-20181008053335-6292fdc1e191/go.mod h1:e2qWDig5bLteJ4fwvDAc2NHzqFEthkqn7aOZAOpj+PQ= -github.com/shurcooL/issuesapp v0.0.0-20180602232740-048589ce2241 h1:Y+TeIabU8sJD10Qwd/zMty2/LEaT9GNDaA6nyZf+jgo= github.com/shurcooL/issuesapp v0.0.0-20180602232740-048589ce2241/go.mod h1:NPpHK2TI7iSaM0buivtFUc9offApnI0Alt/K8hcHy0I= -github.com/shurcooL/notifications v0.0.0-20181007000457-627ab5aea122 h1:TQVQrsyNaimGwF7bIhzoVC9QkKm4KsWd8cECGzFx8gI= github.com/shurcooL/notifications v0.0.0-20181007000457-627ab5aea122/go.mod h1:b5uSkrEVM1jQUspwbixRBhaIjIzL2xazXp6kntxYle0= -github.com/shurcooL/octicon v0.0.0-20181028054416-fa4f57f9efb2 h1:bu666BQci+y4S0tVRVjsHUeRon6vUXmsGBwdowgMrg4= github.com/shurcooL/octicon v0.0.0-20181028054416-fa4f57f9efb2/go.mod h1:eWdoE5JD4R5UVWDucdOPg1g2fqQRq78IQa9zlOV1vpQ= -github.com/shurcooL/reactions v0.0.0-20181006231557-f2e0b4ca5b82 h1:LneqU9PHDsg/AkPDU3AkqMxnMYL+imaqkpflHu73us8= github.com/shurcooL/reactions v0.0.0-20181006231557-f2e0b4ca5b82/go.mod h1:TCR1lToEk4d2s07G3XGfz2QrgHXg4RJBvjrOozvoWfk= github.com/shurcooL/sanitized_anchor_name v0.0.0-20170918181015-86672fcb3f95/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= -github.com/shurcooL/sanitized_anchor_name v1.0.0 h1:PdmoCO6wvbs+7yrJyMORt4/BmY5IYyJwS/kOiWx8mHo= github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= -github.com/shurcooL/users v0.0.0-20180125191416-49c67e49c537 h1:YGaxtkYjb8mnTvtufv2LKLwCQu2/C7qFB7UtrOlTWOY= github.com/shurcooL/users v0.0.0-20180125191416-49c67e49c537/go.mod h1:QJTqeLYEDaXHZDBsXlPCDqdhQuJkuw4NOtaxYe3xii4= -github.com/shurcooL/webdavfs v0.0.0-20170829043945-18c3829fa133 h1:JtcyT0rk/9PKOdnKQzuDR+FSjh7SGtJwpgVpfZBRKlQ= github.com/shurcooL/webdavfs v0.0.0-20170829043945-18c3829fa133/go.mod h1:hKmq5kWdCj2z2KEozexVbfEZIWiTjhE0+UjmZgPqehw= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= -github.com/sirupsen/logrus v1.6.0 h1:UBcNElsrwanuuMsnGSlYmtmgbb23qDR5dG+6X6Oo89I= github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= -github.com/smartystreets/assertions v0.0.0-20190215210624-980c5ac6f3ac h1:wbW+Bybf9pXxnCFAOWZTqkRjAc7rAIwo2e1ArUhiHxg= github.com/smartystreets/assertions v0.0.0-20190215210624-980c5ac6f3ac/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= github.com/smartystreets/goconvey v0.0.0-20181108003508-044398e4856c/go.mod h1:XDJAKZRPZ1CvBcN2aX5YOUTYGHki24fSF0Iv48Ibg0s= github.com/smartystreets/goconvey v0.0.0-20190306220146-200a235640ff/go.mod h1:KSQcGKpxUMHk3nbYzs/tIBAM2iDooCn0BmttHOJEbLs= -github.com/smartystreets/goconvey v1.6.4 h1:fv0U8FUIMPNf1L9lnHLvLhgicrIVChEkdzIKYqbNC9s= github.com/smartystreets/goconvey v1.6.4/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= -github.com/soheilhy/cmux v0.1.4 h1:0HKaf1o97UwFjHH9o5XsHUOF+tqmdA7KEzXLpiyaw0E= github.com/soheilhy/cmux v0.1.4/go.mod h1:IM3LyeVVIOuxMH7sFAkER9+bJ4dT7Ms6E4xg4kGIyLM= -github.com/sony/gobreaker v0.4.1 h1:oMnRNZXX5j85zso6xCPRNPtmAycat+WcoKbklScLDgQ= github.com/sony/gobreaker v0.4.1/go.mod h1:ZKptC7FHNvhBz7dN2LGjPVBz2sZJmc0/PkyDJOjmxWY= -github.com/sourcegraph/annotate v0.0.0-20160123013949-f4cad6c6324d h1:yKm7XZV6j9Ev6lojP2XaIshpT4ymkqhMeSghO5Ps00E= github.com/sourcegraph/annotate v0.0.0-20160123013949-f4cad6c6324d/go.mod h1:UdhH50NIW0fCiwBSr0co2m7BnFLdv4fQTgdqdJTHFeE= -github.com/sourcegraph/syntaxhighlight v0.0.0-20170531221838-bd320f5d308e h1:qpG93cPwA5f7s/ZPBJnGOYQNK/vKsaDaseuKT5Asee8= github.com/sourcegraph/syntaxhighlight v0.0.0-20170531221838-bd320f5d308e/go.mod h1:HuIsMU8RRBOtsCgI77wP899iHVBQpCmg4ErYMZB+2IA= github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= github.com/spaolacci/murmur3 v1.1.0 h1:7c1g84S4BPRrfL5Xrdp6fOJ206sU9y293DDHaoy0bLI= github.com/spaolacci/murmur3 v1.1.0/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= -github.com/spf13/cobra v0.0.3 h1:ZlrZ4XsMRm04Fr5pSFxBgfND2EBVa1nLpiy1stUsX/8= github.com/spf13/cobra v0.0.3/go.mod h1:1l0Ry5zgKvJasoi3XT1TypsSe7PqH0Sj9dhYf7v3XqQ= -github.com/spf13/pflag v1.0.1 h1:aCvUg6QPl3ibpQUxyLkrEkCHtPqYJL4x9AuhqVqFis4= github.com/spf13/pflag v1.0.1/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= github.com/streadway/amqp v0.0.0-20190404075320-75d898a42a94/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw= -github.com/streadway/amqp v0.0.0-20190827072141-edfb9018d271 h1:WhxRHzgeVGETMlmVfqhRn8RIeeNoPr2Czh33I4Zdccw= github.com/streadway/amqp v0.0.0-20190827072141-edfb9018d271/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw= -github.com/streadway/handy v0.0.0-20190108123426-d5acb3125c2a h1:AhmOdSHeswKHBjhsLs/7+1voOxT+LLrSk/Nxvk35fug= github.com/streadway/handy v0.0.0-20190108123426-d5acb3125c2a/go.mod h1:qNTQ5P5JnDBl6z3cMAg/SywNDC5ABu5ApDIw6lUbRmI= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= -github.com/stretchr/objx v0.1.1 h1:2vfRuCMp5sSVIDSqO8oNnWJq7mPa6KVP3iPIwFBuy8A= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/testify v1.2.1/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= @@ -859,28 +624,20 @@ github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5 github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.7.0 h1:nwc3DEeHmmLAfoZucVR881uASk0Mfjw8xYJ99tb5CcY= github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= -github.com/syncthing/syncthing v0.14.48-rc.4 h1:9w607+diOQw1niYrydPEKbI50Q5oZ9rneSn8Twqjqlo= github.com/syncthing/syncthing v0.14.48-rc.4/go.mod h1:nw3siZwHPA6M8iSfjDCWQ402eqvEIasMQOE8nFOxy7M= -github.com/tarm/serial v0.0.0-20180830185346-98f6abe2eb07 h1:UyzmZLoiDWMRywV4DUYb9Fbt8uiOSooupjTq10vpvnU= github.com/tarm/serial v0.0.0-20180830185346-98f6abe2eb07/go.mod h1:kDXzergiv9cbyO7IOYJZWg1U88JhDg3PB6klq9Hg2pA= github.com/tinylib/msgp v1.0.2/go.mod h1:+d+yLhGm8mzTaHzB+wgMYrodPfmZrzkirds8fDWklFE= github.com/tinylib/msgp v1.1.0/go.mod h1:+d+yLhGm8mzTaHzB+wgMYrodPfmZrzkirds8fDWklFE= github.com/tinylib/msgp v1.1.1/go.mod h1:+d+yLhGm8mzTaHzB+wgMYrodPfmZrzkirds8fDWklFE= github.com/tinylib/msgp v1.1.2/go.mod h1:+d+yLhGm8mzTaHzB+wgMYrodPfmZrzkirds8fDWklFE= -github.com/tinylib/msgp v1.1.5 h1:2gXmtWueD2HefZHQe1QOy9HVzmFrLOVvsXwXBQ0ayy0= github.com/tinylib/msgp v1.1.5/go.mod h1:eQsjooMTnV42mHu917E26IogZ2930nFyBQdofk10Udg= -github.com/tmc/grpc-websocket-proxy v0.0.0-20170815181823-89b8d40f7ca8 h1:ndzgwNDnKIqyCvHTXaCqh9KlOWKvBry6nuXMJmonVsE= github.com/tmc/grpc-websocket-proxy v0.0.0-20170815181823-89b8d40f7ca8/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= -github.com/ttacon/chalk v0.0.0-20160626202418-22c06c80ed31 h1:OXcKh35JaYsGMRzpvFkLv/MEyPuL49CThT1pZ8aSml4= github.com/ttacon/chalk v0.0.0-20160626202418-22c06c80ed31/go.mod h1:onvgF043R+lC5RZ8IT9rBXDaEDnpnw/Cl+HFiw+v/7Q= github.com/tv42/httpunix v0.0.0-20191220191345-2ba4b9c3382c h1:u6SKchux2yDvFQnDHS3lPnIRmfVJ5Sxy3ao2SIdysLQ= github.com/tv42/httpunix v0.0.0-20191220191345-2ba4b9c3382c/go.mod h1:hzIxponao9Kjc7aWznkXaL4U4TWaDSs8zcsY4Ka08nM= github.com/urfave/cli v1.20.0/go.mod h1:70zkFmudgCuE/ngEzBv17Jvp/497gISqfk5gWijbERA= -github.com/urfave/cli v1.22.1 h1:+mkCCcOFKPnCmVYVcURKps1Xe+3zP90gSYGNfRkjoIY= github.com/urfave/cli v1.22.1/go.mod h1:Gos4lmkARVdJ6EkW0WaNv/tZAAMe9V7XWyB60NtXRu0= -github.com/viant/assertly v0.4.8 h1:5x1GzBaRteIwTr5RAGFVG14uNeRFxVNbXPWrK2qAgpc= github.com/viant/assertly v0.4.8/go.mod h1:aGifi++jvCrUaklKEKT0BU95igDNaqkvz+49uaYMPRU= -github.com/viant/toolbox v0.24.0 h1:6TteTDQ68CjgcCe8wH3D3ZhUQQOJXMTbj/D9rkk2a1k= github.com/viant/toolbox v0.24.0/go.mod h1:OxMCG57V0PXuIP2HNQrtJf2CjqdmbrOx5EkMILuUhzM= github.com/willf/bitset v1.1.3/go.mod h1:RjeCKbqT1RxIR/KWY6phxZiaY1IyutSBfGjNPySAYV4= github.com/willf/bitset v1.1.9/go.mod h1:RjeCKbqT1RxIR/KWY6phxZiaY1IyutSBfGjNPySAYV4= @@ -890,36 +647,26 @@ github.com/willf/bitset v1.1.11/go.mod h1:83CECat5yLh5zVOf4P1ErAgKA5UDvKtgyUABdr github.com/willf/bloom v0.0.0-20170505221640-54e3b963ee16/go.mod h1:MmAltL9pDMNTrvUkxdg0k0q5I0suxmuwp3KbyrZLOZ8= github.com/willf/bloom v2.0.3+incompatible h1:QDacWdqcAUI1MPOwIQZRy9kOR7yxfyEmxX8Wdm2/JPA= github.com/willf/bloom v2.0.3+incompatible/go.mod h1:MmAltL9pDMNTrvUkxdg0k0q5I0suxmuwp3KbyrZLOZ8= -github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2 h1:eY9dn8+vbi4tKz5Qo6v2eYzo7kUS51QINcR5jNpbZS8= github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2/go.mod h1:UETIi67q53MR2AWcXfiuqkDkRtnGDLqkBTpCHuJHxtU= -github.com/yuin/goldmark v1.2.1 h1:ruQGxdhGHe7FWOJPT0mKs5+pD2Xs1Bm/kdGlHO04FmM= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= go.etcd.io/bbolt v1.3.3/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= go.etcd.io/bbolt v1.3.4/go.mod h1:G5EMThwa9y8QZGBClrRx5EY+Yw9kAhnjy3bSjsnlVTQ= go.etcd.io/bbolt v1.3.5 h1:XAzx9gjCb0Rxj7EoqcClPD1d5ZBxZJk0jbuoPHenBt0= go.etcd.io/bbolt v1.3.5/go.mod h1:G5EMThwa9y8QZGBClrRx5EY+Yw9kAhnjy3bSjsnlVTQ= -go.etcd.io/etcd v0.0.0-20191023171146-3cf2f69b5738 h1:VcrIfasaLFkyjk6KNlXQSzO+B0fZcnECiDrKJsfxka0= go.etcd.io/etcd v0.0.0-20191023171146-3cf2f69b5738/go.mod h1:dnLIgRNXwCJa5e+c6mIZCrds/GIG4ncV9HhK5PX7jPg= go.opencensus.io v0.18.0/go.mod h1:vKdFvxhtzZ9onBp9VKHK8z/sRpBMnKAsufL7wlDrCOA= go.opencensus.io v0.20.1/go.mod h1:6WKK9ahsWS3RSO+PY9ZHZUfv2irvY6gN279GOPZjmmk= go.opencensus.io v0.20.2/go.mod h1:6WKK9ahsWS3RSO+PY9ZHZUfv2irvY6gN279GOPZjmmk= go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= -go.opencensus.io v0.22.3 h1:8sGtKOrtQqkN1bp2AtX+misvLIlOmsEsNd+9NIcPEm8= go.opencensus.io v0.22.3/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= go.uber.org/atomic v1.3.2/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= -go.uber.org/atomic v1.5.0 h1:OI5t8sDa1Or+q8AeE+yKeB/SDYioSHAgcVljj9JIETY= go.uber.org/atomic v1.5.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0= -go.uber.org/multierr v1.3.0 h1:sFPn2GLc3poCkfrpIXGhBD2X0CMIo4Q/zSULXrj/+uc= go.uber.org/multierr v1.3.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4= -go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee h1:0mgffUl7nfd+FpvXMVz4IDEaUSmT1ysygQC7qYo7sG4= go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA= go.uber.org/zap v1.10.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= -go.uber.org/zap v1.13.0 h1:nR6NoDBgAf67s68NhaXbsojM+2gxp3S1hWkHDl27pVU= go.uber.org/zap v1.13.0/go.mod h1:zwrFLgMcdUuIBviXEYEH1YKNaOBnKXsx2IPda5bBwHM= -go4.org v0.0.0-20180809161055-417644f6feb5 h1:+hE86LblG4AyDgwMCLTE6FOlM9+qjHSYS+rKqxUVdsM= go4.org v0.0.0-20180809161055-417644f6feb5/go.mod h1:MkTOUMDaeVYJUOUsaDXIhWPZYa1yOyC1qaOBpL57BhE= -golang.org/x/build v0.0.0-20190111050920-041ab4dc3f9d h1:E2M5QgjZ/Jg+ObCQAudsXxuTsLj7Nl5RV/lZcQZmKSo= golang.org/x/build v0.0.0-20190111050920-041ab4dc3f9d/go.mod h1:OWs+y06UdEOHN4y+MfF/py+xQ/tYqIWW03b70/CG9Rw= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20181029021203-45a5f77698d3/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= @@ -939,7 +686,6 @@ golang.org/x/crypto v0.0.0-20201221181555-eec23a3978ad/go.mod h1:jdWPYTVW3xRLrWP golang.org/x/crypto v0.0.0-20210322153248-0c34fe9e7dc2/go.mod h1:T9bdIzuCu7OtxOm1hfPfRQxPLYneinmdGuTeoZ9dtd4= golang.org/x/crypto v0.0.0-20210421170649-83a5a9bb288b h1:7mWr3k41Qtv8XlltBkDkl8LoP3mpSgBW8BUoxtEdbXg= golang.org/x/crypto v0.0.0-20210421170649-83a5a9bb288b/go.mod h1:T9bdIzuCu7OtxOm1hfPfRQxPLYneinmdGuTeoZ9dtd4= -golang.org/x/exp v0.0.0-20190121172915-509febef88a4 h1:c2HOrn5iMezYjSlGPncknSEr/8x5LELb/ilJbXi9DEA= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/lint v0.0.0-20180702182130-06c8688daad7/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= @@ -947,11 +693,9 @@ golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvx golang.org/x/lint v0.0.0-20190301231843-5614ed5bae6f/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= -golang.org/x/lint v0.0.0-20200302205851-738671d3881b h1:Wh+f8QHJXR411sJR8/vRBTZ7YapZaRvUcLFFJhusH0k= golang.org/x/lint v0.0.0-20200302205851-738671d3881b/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= -golang.org/x/mod v0.3.0 h1:RM4zey1++hCTbCVQfnWeKs9/IEsaBLA8vTkd0WVtmH4= golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/net v0.0.0-20180524181706-dfa909b99c79/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -998,9 +742,7 @@ golang.org/x/net v0.0.0-20210427231257-85d9c07bbe3a/go.mod h1:OJAsFXCWl8Ukc7SiCT golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20181017192945-9dcd33a902f4/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20181203162652-d668ce993890/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= -golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421 h1:Wo7BWFiOk0QRFMLYMqJGFMd9CgUAcGx7V+qEg/h5IBI= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= -golang.org/x/perf v0.0.0-20180704124530-6e6d33e29852 h1:xYq6+9AtI+xP3M4r0N1hCkHrInHDBohhquRgx9Kk6gI= golang.org/x/perf v0.0.0-20180704124530-6e6d33e29852/go.mod h1:JLpeXjPJfIyPr5TlbXLkXWLhP8nz10XfvxElABhCtcw= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -1064,14 +806,12 @@ golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210426230700-d19ff857e887 h1:dXfMednGJh/SUUFjTLsWJz3P+TQt9qnR11GgeI3vWKs= golang.org/x/sys v0.0.0-20210426230700-d19ff857e887/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/term v0.0.0-20201117132131-f5c789dd3221/go.mod h1:Nr5EML6q2oocZ2LXRh80K7BxOlk5/8JxuGnuhpl+muw= -golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1 h1:v+OssWQX+hTHEmOBgwxdZxK4zHq3yOs8F9J7mk0PY8E= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.3.6 h1:aRYxNxv6iGQlyVaZmk6ZgYEDa+Jg18DxebPSrd6bg1M= golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= @@ -1099,7 +839,6 @@ golang.org/x/tools v0.0.0-20200103221440-774c71fcf114/go.mod h1:TB2adYChydJhpapK golang.org/x/tools v0.0.0-20200130002326-2f3ba24bd6e7/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= golang.org/x/tools v0.0.0-20200928182047-19e03678916f/go.mod h1:z6u4i615ZeAfBE4XtMziQW1fSVJXACjjbWkB/mvPzlU= golang.org/x/tools v0.0.0-20201022035929-9cf592e881e9/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= -golang.org/x/tools v0.0.0-20201224043029-2b0845dc783e h1:4nW4NLDYnU28ojHaHO8OVxFHk/aQ33U01a9cjED+pzE= golang.org/x/tools v0.0.0-20201224043029-2b0845dc783e/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -1109,12 +848,10 @@ golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8T google.golang.org/api v0.0.0-20180910000450-7ca32eb868bf/go.mod h1:4mhQ8q/RsB7i+udVvVy5NUi08OU8ZlA0gRVgrF7VFY0= google.golang.org/api v0.0.0-20181030000543-1d582fd0359e/go.mod h1:4mhQ8q/RsB7i+udVvVy5NUi08OU8ZlA0gRVgrF7VFY0= google.golang.org/api v0.1.0/go.mod h1:UGEZY7KEX120AnNLIHFMKIo4obdJhkp2tPbaPlQx13Y= -google.golang.org/api v0.3.1 h1:oJra/lMfmtm13/rgY/8i3MzjFWYXvQIAKjQ3HqofMk8= google.golang.org/api v0.3.1/go.mod h1:6wY9I6uQWHQ8EM57III9mq/AjF+i8G65rmVagqKMtkk= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.2.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.3.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= -google.golang.org/appengine v1.4.0 h1:/wp5JvzpHIxhs/dumFmF7BXTf3Z+dd4uXta4kVyO508= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20180831171423-11092d34479b/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= @@ -1125,7 +862,6 @@ google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRn google.golang.org/genproto v0.0.0-20190425155659-357c62f0e4bb/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= google.golang.org/genproto v0.0.0-20190530194941-fb225487d101/go.mod h1:z3L6/3dTEVtUr6QSP8miRzeRqwQOioJ9I66odjN4I7s= google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= -google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013 h1:+kGHl1aib/qcwaRi1CbqBZ1rk19r85MNUf8HaBghugY= google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo= google.golang.org/grpc v1.14.0/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= google.golang.org/grpc v1.16.0/go.mod h1:0JHn/cJsOMiMfNA9+DeHDlAU7KAAB5GDlYFpa9MZMio= @@ -1138,7 +874,6 @@ google.golang.org/grpc v1.22.1/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyac google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= google.golang.org/grpc v1.23.1/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= google.golang.org/grpc v1.26.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= -google.golang.org/grpc v1.27.0 h1:rRYRFMVgRv6E0D70Skyfsr28tDXIuuPZyWGMPdMcnXg= google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= @@ -1148,29 +883,19 @@ google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzi google.golang.org/protobuf v1.22.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= google.golang.org/protobuf v1.23.1-0.20200526195155-81db48ad09cc/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= -google.golang.org/protobuf v1.25.0 h1:Ejskq+SyPohKW+1uil0JJMtmHCgJPJ/qWTxr8qp+R4c= google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= -gopkg.in/alecthomas/kingpin.v2 v2.2.6 h1:jMFz6MfLP0/4fUyZle81rXUoxOBFi19VUFKVDOQfozc= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15 h1:YR8cESwS4TdDjEe65xsg0ogRM/Nc3DYOhEAlW+xobZo= gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/cheggaaa/pb.v1 v1.0.25 h1:Ev7yu1/f6+d+b3pi5vPdRPc6nNtP1umSfcWiEfRqv6I= gopkg.in/cheggaaa/pb.v1 v1.0.25/go.mod h1:V/YB90LKu/1FcN3WVnfiiE5oMCibMjukxqG/qStrOgw= -gopkg.in/errgo.v2 v2.1.0 h1:0vLT13EuvQ0hNvakwLuFZ/jYrLp5F3kcWHXdRggjCE8= gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= -gopkg.in/fsnotify.v1 v1.4.7 h1:xOHLXZwVvI9hhs+cLKq5+I5onOuwQLhQwiu63xxlHs4= gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys= -gopkg.in/gcfg.v1 v1.2.3 h1:m8OOJ4ccYHnx2f4gQwpno8nAX5OGOh7RLaaz0pj3Ogs= gopkg.in/gcfg.v1 v1.2.3/go.mod h1:yesOnuUOFQAhST5vPY4nbZsb/huCgGGXlipJsBn0b3o= -gopkg.in/inf.v0 v0.9.1 h1:73M5CoZyi3ZLMOyDlQh031Cx6N9NDJ2Vvfl76EDAgDc= gopkg.in/inf.v0 v0.9.1/go.mod h1:cWUDdTG/fYaXco+Dcufb5Vnc6Gp2YChqWtbxRZE0mXw= -gopkg.in/resty.v1 v1.12.0 h1:CuXP0Pjfw9rOuY6EP+UvtNvt5DSqHpIxILZKT/quCZI= gopkg.in/resty.v1 v1.12.0/go.mod h1:mDo4pnntr5jdWRML875a/NmxYqAlA73dVijT2AXvQQo= -gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 h1:uRGJdciOHaEIrze2W8Q3AKkepLTh2hOroT7a+7czfdQ= gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw= -gopkg.in/warnings.v0 v0.1.2 h1:wFXVbFY8DY5/xOe1ECiWdKCzZlxgshcYVNkBHstARME= gopkg.in/warnings.v0 v0.1.2/go.mod h1:jksf8JmL6Qr/oQM2OXTHunEvvTAsrWBLb6OOjuVWRNI= gopkg.in/yaml.v2 v2.0.0-20170812160011-eb3733d160e7/go.mod h1:JAlM8MvJe8wmxCU4Bli9HhUf9+ttbYbLASfIpnQbh74= gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= @@ -1179,27 +904,18 @@ gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.5/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.7/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY= gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c h1:dUUwHk2QECo/6vqA44rthZ8ie2QXMNeKRTHCNY2nXvo= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -grpc.go4.org v0.0.0-20170609214715-11d0a25b4919 h1:tmXTu+dfa+d9Evp8NpJdgOy6+rt8/x4yG7qPBrtNfLY= grpc.go4.org v0.0.0-20170609214715-11d0a25b4919/go.mod h1:77eQGdRu53HpSqPFJFmuJdjuHRquDANNeA4x7B8WQ9o= honnef.co/go/tools v0.0.0-20180728063816-88497007e858/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -honnef.co/go/tools v0.0.1-2019.2.3 h1:3JgtbtFHMiCmsznwGVTUWbgGov+pVqnlf1dEJTNAXeM= honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= -rsc.io/quote/v3 v3.1.0 h1:9JKUTTIUgS6kzR9mK1YuGKv6Nl+DijDNIc0ghT58FaY= rsc.io/quote/v3 v3.1.0/go.mod h1:yEA65RcK8LyAZtP9Kv3t0HmxON59tX3rD+tICJqUlj0= -rsc.io/sampler v1.3.0 h1:7uVkIFmeBqHfdjD+gZwtXXI+RODJ2Wc4O7MPEh/QiW4= rsc.io/sampler v1.3.0/go.mod h1:T1hPZKmBbMNahiBKFy5HrXp6adAjACjK9JXDnKaTXpA= -sigs.k8s.io/yaml v1.1.0 h1:4A07+ZFc2wgJwo8YNlQpr1rVlgUDlxXHhPJciaPY5gs= sigs.k8s.io/yaml v1.1.0/go.mod h1:UJmg0vDUVViEyp3mgSv9WPwZCDxu4rQW1olrI1uml+o= -sourcegraph.com/sourcegraph/appdash v0.0.0-20190731080439-ebfcffb1b5c0 h1:ucqkfpjg9WzSUubAO62csmucvxl4/JeW3F4I4909XkM= sourcegraph.com/sourcegraph/appdash v0.0.0-20190731080439-ebfcffb1b5c0/go.mod h1:hI742Nqp5OhwiqlzhgfbWU4mW4yO10fP+LoT9WOswdU= -sourcegraph.com/sourcegraph/go-diff v0.5.0 h1:eTiIR0CoWjGzJcnQ3OkhIl/b9GJovq4lSAVRt0ZFEG8= sourcegraph.com/sourcegraph/go-diff v0.5.0/go.mod h1:kuch7UrkMzY0X+p9CRK03kfuPQ2zzQcaEFbx8wA8rck= -sourcegraph.com/sqs/pbtypes v0.0.0-20180604144634-d3ebe8f20ae4 h1:JPJh2pk3+X4lXAkZIk2RuE/7/FoK9maXw+TNPJhVS/c= sourcegraph.com/sqs/pbtypes v0.0.0-20180604144634-d3ebe8f20ae4/go.mod h1:ketZ/q3QxT9HOBeFhu6RdvsftgpsbFHBF5Cas6cDKZ0= From 2d2456249a26580c3d1b3ad53da37e77a0f9274f Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Mon, 10 May 2021 13:35:24 +1000 Subject: [PATCH 10/59] Remove unused pieceRequestOrder in Client --- client.go | 2 -- torrent.go | 1 - 2 files changed, 3 deletions(-) diff --git a/client.go b/client.go index bb0f4f4c..3e899549 100644 --- a/client.go +++ b/client.go @@ -81,8 +81,6 @@ type Client struct { websocketTrackers websocketTrackers activeAnnounceLimiter limiter.Instance - - clientPieceRequestOrder } type ipStr string diff --git a/torrent.go b/torrent.go index 6eab0d69..c56ddbd1 100644 --- a/torrent.go +++ b/torrent.go @@ -412,7 +412,6 @@ func (t *Torrent) setInfo(info *metainfo.Info) error { // This seems to be all the follow-up tasks after info is set, that can't fail. func (t *Torrent) onSetInfo() { - t.cl.clientPieceRequestOrder.addPieces(t, t.numPieces()) t.iterPeers(func(p *Peer) { p.onGotInfo(t.info) }) From b508877d828ace979685ce6e827544860387dfc8 Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Mon, 10 May 2021 17:02:17 +1000 Subject: [PATCH 11/59] Track peer availability at the Torrent-level --- peerconn.go | 56 +++++++++++++++++++++++++++++++++------------ piece.go | 1 + request-strategy.go | 11 +++++---- 3 files changed, 48 insertions(+), 20 deletions(-) diff --git a/peerconn.go b/peerconn.go index 7fd819be..7e20ba3b 100644 --- a/peerconn.go +++ b/peerconn.go @@ -371,15 +371,16 @@ func (cn *Peer) writeStatus(w io.Writer, t *Torrent) { fmt.Fprintf(w, "\n") } -func (cn *Peer) close() { - if !cn.closed.Set() { +func (p *Peer) close() { + if !p.closed.Set() { return } - cn.discardPieceInclination() - cn._pieceRequestOrder.Clear() - cn.peerImpl.onClose() - for _, f := range cn.callbacks.PeerClosed { - f(cn) + p.discardPieceInclination() + p._pieceRequestOrder.Clear() + p.peerImpl.onClose() + p.t.decPeerPieceAvailability(p) + for _, f := range p.callbacks.PeerClosed { + f(p) } } @@ -764,7 +765,7 @@ func (cn *Peer) discardPieceInclination() { cn.pieceInclination = nil } -func (cn *PeerConn) peerPiecesChanged() { +func (cn *Peer) peerPiecesChanged() { if cn.t.haveInfo() { prioritiesChanged := false for i := pieceIndex(0); i < cn.t.numPieces(); i++ { @@ -776,7 +777,7 @@ func (cn *PeerConn) peerPiecesChanged() { cn.updateRequests() } } - cn.t.maybeDropMutuallyCompletePeer(&cn.Peer) + cn.t.maybeDropMutuallyCompletePeer(cn) } func (cn *PeerConn) raisePeerMinPieces(newMin pieceIndex) { @@ -793,6 +794,9 @@ func (cn *PeerConn) peerSentHave(piece pieceIndex) error { return nil } cn.raisePeerMinPieces(piece + 1) + if !cn.peerHasPiece(piece) { + cn.t.incPieceAvailability(piece) + } cn._peerPieces.Set(bitmap.BitIndex(piece), true) cn.t.maybeDropMutuallyCompletePeer(&cn.Peer) if cn.updatePiecePriority(piece) { @@ -802,20 +806,27 @@ func (cn *PeerConn) peerSentHave(piece pieceIndex) error { } func (cn *PeerConn) peerSentBitfield(bf []bool) error { - cn.peerSentHaveAll = false if len(bf)%8 != 0 { panic("expected bitfield length divisible by 8") } - // We know that the last byte means that at most the last 7 bits are - // wasted. + // We know that the last byte means that at most the last 7 bits are wasted. cn.raisePeerMinPieces(pieceIndex(len(bf) - 7)) if cn.t.haveInfo() && len(bf) > int(cn.t.numPieces()) { // Ignore known excess pieces. bf = bf[:cn.t.numPieces()] } + pp := cn.newPeerPieces() + cn.peerSentHaveAll = false for i, have := range bf { if have { cn.raisePeerMinPieces(pieceIndex(i) + 1) + if !pp.Contains(i) { + cn.t.incPieceAvailability(i) + } + } else { + if pp.Contains(i) { + cn.t.decPieceAvailability(i) + } } cn._peerPieces.Set(i, have) } @@ -823,14 +834,28 @@ func (cn *PeerConn) peerSentBitfield(bf []bool) error { return nil } -func (cn *PeerConn) onPeerSentHaveAll() error { +func (cn *Peer) onPeerHasAllPieces() { + t := cn.t + if t.haveInfo() { + pp := cn.newPeerPieces() + for i := range iter.N(t.numPieces()) { + if !pp.Contains(i) { + t.incPieceAvailability(i) + } + } + } cn.peerSentHaveAll = true cn._peerPieces.Clear() cn.peerPiecesChanged() +} + +func (cn *PeerConn) onPeerSentHaveAll() error { + cn.onPeerHasAllPieces() return nil } func (cn *PeerConn) peerSentHaveNone() error { + cn.t.decPeerPieceAvailability(&cn.Peer) cn._peerPieces.Clear() cn.peerSentHaveAll = false cn.peerPiecesChanged() @@ -1613,10 +1638,11 @@ func (cn *Peer) peerMaxRequests() int { func (cn *PeerConn) PeerPieces() bitmap.Bitmap { cn.locker().RLock() defer cn.locker().RUnlock() - return cn.peerPieces() + return cn.newPeerPieces() } -func (cn *Peer) peerPieces() bitmap.Bitmap { +// Returns a new Bitmap that includes bits for all pieces we have. +func (cn *Peer) newPeerPieces() bitmap.Bitmap { ret := cn._peerPieces.Copy() if cn.peerSentHaveAll { ret.AddRange(0, cn.t.numPieces()) diff --git a/piece.go b/piece.go index 248832d0..cee6b7d0 100644 --- a/piece.go +++ b/piece.go @@ -55,6 +55,7 @@ type Piece struct { publicPieceState PieceState priority piecePriority + availability int64 // This can be locked when the Client lock is taken, but probably not vice versa. pendingWritesMutex sync.Mutex diff --git a/request-strategy.go b/request-strategy.go index e7349aef..9369905b 100644 --- a/request-strategy.go +++ b/request-strategy.go @@ -19,7 +19,7 @@ type pieceRequestOrderPiece struct { index pieceIndex prio piecePriority partial bool - availability int + availability int64 } func (me *clientPieceRequestOrder) addPieces(t *Torrent, numPieces pieceIndex) { @@ -45,12 +45,13 @@ func (me clientPieceRequestOrder) sort() { sort.SliceStable(me.pieces, me.less) } -func (me clientPieceRequestOrder) update() { +func (me *clientPieceRequestOrder) update() { for i := range me.pieces { p := &me.pieces[i] - p.prio = p.t.piece(p.index).uncachedPriority() + tp := p.t.piece(p.index) + p.prio = tp.uncachedPriority() p.partial = p.t.piecePartiallyDownloaded(p.index) - p.availability = p.t.pieceAvailability(p.index) + p.availability = tp.availability } } @@ -61,7 +62,7 @@ func (me clientPieceRequestOrder) less(_i, _j int) bool { int(j.prio), int(i.prio), ).Bool( j.partial, i.partial, - ).Int( + ).Int64( i.availability, j.availability, ).Less() } From 6d25c99ac65ed32dc1f550df5dc5adf774944690 Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Mon, 10 May 2021 17:03:56 +1000 Subject: [PATCH 12/59] Ignore pieces for request after accounting for storage --- request-strategy.go | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/request-strategy.go b/request-strategy.go index 9369905b..8f3811d4 100644 --- a/request-strategy.go +++ b/request-strategy.go @@ -31,6 +31,10 @@ func (me *clientPieceRequestOrder) addPieces(t *Torrent, numPieces pieceIndex) { } } +func (me *clientPieceRequestOrder) Len() int { + return len(me.pieces) +} + func (me *clientPieceRequestOrder) removePieces(t *Torrent) { newPieces := make([]pieceRequestOrderPiece, 0, len(me.pieces)-t.numPieces()) for _, p := range me.pieces { @@ -118,9 +122,6 @@ func (cl *Client) doRequests() { // For a given piece, the set of allPeers indices that absorbed requests for the piece. contributed := make(map[int]struct{}) for _, p := range requestOrder.pieces { - if p.t.ignorePieceForRequests(p.index) { - continue - } peers := allPeers[p.t] torrentPiece := p.t.piece(p.index) if left := storageLeft[p.t.storage.Capacity]; left != nil { @@ -129,6 +130,9 @@ func (cl *Client) doRequests() { } *left -= int64(torrentPiece.length()) } + if p.t.ignorePieceForRequests(p.index) { + continue + } p.t.piece(p.index).iterUndirtiedChunks(func(chunk ChunkSpec) bool { req := Request{pp.Integer(p.index), chunk} const skipAlreadyRequested = false From d9172ef2778b7a1405a8e360c8b1beda829dc774 Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Mon, 10 May 2021 17:04:27 +1000 Subject: [PATCH 13/59] Comments --- request-strategy.go | 31 +------------------------------ 1 file changed, 1 insertion(+), 30 deletions(-) diff --git a/request-strategy.go b/request-strategy.go index 8f3811d4..c373be7a 100644 --- a/request-strategy.go +++ b/request-strategy.go @@ -165,6 +165,7 @@ func (cl *Client) doRequests() { }) // Move requestees for this piece to the back. lastIndex := len(peers) - 1 + // Probably should sort the contributees, to make the ordering more deterministic. for peerIndex := range contributed { peers[peerIndex], peers[lastIndex] = peers[lastIndex], peers[peerIndex] delete(contributed, peerIndex) @@ -179,33 +180,3 @@ func (cl *Client) doRequests() { }) } } - -//func (requestStrategyDefaults) iterUndirtiedChunks(p requestStrategyPiece, f func(ChunkSpec) bool) bool { -// chunkIndices := p.dirtyChunks().Copy() -// chunkIndices.FlipRange(0, bitmap.BitIndex(p.numChunks())) -// return iter.ForPerm(chunkIndices.Len(), func(i int) bool { -// ci, err := chunkIndices.RB.Select(uint32(i)) -// if err != nil { -// panic(err) -// } -// return f(p.chunkIndexRequest(pp.Integer(ci)).ChunkSpec) -// }) -//} - -// -//func iterUnbiasedPieceRequestOrder( -// cn requestStrategyConnection, -// f func(piece pieceIndex) bool, -// pieceRequestOrder []pieceIndex, -//) bool { -// cn.torrent().sortPieceRequestOrder(pieceRequestOrder) -// for _, i := range pieceRequestOrder { -// if !cn.peerHasPiece(i) || cn.torrent().ignorePieceForRequests(i) { -// continue -// } -// if !f(i) { -// return false -// } -// } -// return true -//} From 99a7cb9291b0495369c0b688443175873caf25ed Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Mon, 10 May 2021 17:05:08 +1000 Subject: [PATCH 14/59] Missed piece availability code --- torrent.go | 78 +++++++++++++++++++++++++++++++++++++++++++----------- 1 file changed, 63 insertions(+), 15 deletions(-) diff --git a/torrent.go b/torrent.go index c56ddbd1..ae78625f 100644 --- a/torrent.go +++ b/torrent.go @@ -147,7 +147,7 @@ type Torrent struct { pex pexState } -func (t *Torrent) pieceAvailability(i pieceIndex) (count int) { +func (t *Torrent) pieceAvailabilityFromPeers(i pieceIndex) (count int) { t.iterPeers(func(peer *Peer) { if peer.peerHasPiece(i) { count++ @@ -156,6 +156,19 @@ func (t *Torrent) pieceAvailability(i pieceIndex) (count int) { return } +func (t *Torrent) decPieceAvailability(i pieceIndex) { + p := t.piece(i) + if p.availability <= 0 { + panic(p.availability) + } + p.availability-- +} + +func (t *Torrent) incPieceAvailability(i pieceIndex) { + p := t.piece(i) + p.availability++ +} + func (t *Torrent) numConns() int { return len(t.conns) } @@ -422,6 +435,10 @@ func (t *Torrent) onSetInfo() { // t.logger.Printf("piece %s completion unknown, queueing check", p) t.queuePieceCheck(pieceIndex(i)) } + if p.availability != 0 { + panic(p.availability) + } + p.availability = int64(t.pieceAvailabilityFromPeers(i)) } t.cl.event.Broadcast() t.gotMetainfo.Set() @@ -535,6 +552,22 @@ func (t *Torrent) newMetadataExtensionMessage(c *PeerConn, msgType int, piece in } } +type pieceAvailabilityRun struct { + availability int64 + count pieceIndex +} + +func (t *Torrent) pieceAvailabilityRuns() (ret []pieceAvailabilityRun) { + rle := missinggo.NewRunLengthEncoder(func(el interface{}, count uint64) { + ret = append(ret, pieceAvailabilityRun{el.(int64), int(count)}) + }) + for _, p := range t.pieces { + rle.Append(p.availability, 1) + } + rle.Flush() + return +} + func (t *Torrent) pieceStateRuns() (ret PieceStateRuns) { rle := missinggo.NewRunLengthEncoder(func(el interface{}, count uint64) { ret = append(ret, PieceStateRun{ @@ -605,17 +638,21 @@ func (t *Torrent) writeStatus(w io.Writer) { } fmt.Fprintln(w) } - fmt.Fprintf(w, "Piece length: %s\n", func() string { - if t.haveInfo() { - return fmt.Sprint(t.usualPieceSize()) - } else { - return "?" - } - }()) + fmt.Fprintf(w, "Piece length: %s\n", + func() string { + if t.haveInfo() { + return fmt.Sprintf("%v (%v chunks)", + t.usualPieceSize(), + float64(t.usualPieceSize())/float64(t.chunkSize)) + } else { + return "no info" + } + }(), + ) if t.info != nil { fmt.Fprintf(w, "Num Pieces: %d (%d completed)\n", t.numPieces(), t.numPiecesCompleted()) - fmt.Fprintf(w, "Piece States: %s", t.pieceStateRuns()) - fmt.Fprintln(w) + fmt.Fprintf(w, "Piece States: %s\n", t.pieceStateRuns()) + fmt.Fprintf(w, "Piece availability: %v\n", t.pieceAvailabilityRuns()) } fmt.Fprintf(w, "Reader Pieces:") t.forReaderOffsetPieces(func(begin, end pieceIndex) (again bool) { @@ -888,11 +925,14 @@ func chunkIndex(cs ChunkSpec, chunkSize pp.Integer) int { } func (t *Torrent) wantPieceIndex(index pieceIndex) bool { - if !t.haveInfo() { - return false - } - if index < 0 || index >= t.numPieces() { - return false + // TODO: Are these overly conservative, should we be guarding this here? + { + if !t.haveInfo() { + return false + } + if index < 0 || index >= t.numPieces() { + return false + } } p := &t.pieces[index] if p.queuedForHash() { @@ -1317,6 +1357,13 @@ func (t *Torrent) deletePeerConn(c *PeerConn) (ret bool) { return } +func (t *Torrent) decPeerPieceAvailability(p *Peer) { + p.newPeerPieces().IterTyped(func(i int) bool { + p.t.decPieceAvailability(i) + return true + }) +} + func (t *Torrent) numActivePeers() (num int) { t.iterPeers(func(*Peer) { num++ @@ -2154,6 +2201,7 @@ func (t *Torrent) addWebSeed(url string) { ws.onGotInfo(t.info) } t.webSeeds[url] = &ws.peer + ws.peer.onPeerHasAllPieces() } func (t *Torrent) peerIsActive(p *Peer) (active bool) { From 875ead3582234dbc10877eb27649e2d3fab208af Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Mon, 10 May 2021 17:42:51 +1000 Subject: [PATCH 15/59] Reuse pieceRequestOrder slice --- client.go | 2 ++ request-strategy.go | 3 ++- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/client.go b/client.go index 3e899549..e5915c1f 100644 --- a/client.go +++ b/client.go @@ -81,6 +81,8 @@ type Client struct { websocketTrackers websocketTrackers activeAnnounceLimiter limiter.Instance + + pieceRequestOrder clientPieceRequestOrder } type ipStr string diff --git a/request-strategy.go b/request-strategy.go index c373be7a..4bce3321 100644 --- a/request-strategy.go +++ b/request-strategy.go @@ -87,7 +87,8 @@ func (cl *Client) requester() { } func (cl *Client) doRequests() { - requestOrder := clientPieceRequestOrder{} + requestOrder := &cl.pieceRequestOrder + requestOrder.pieces = requestOrder.pieces[:0] allPeers := make(map[*Torrent][]*Peer) // Storage capacity left for this run, keyed by the storage capacity pointer on the storage // TorrentImpl. From fc0690901465800de093f41ab2139fd68db0c735 Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Mon, 10 May 2021 17:43:10 +1000 Subject: [PATCH 16/59] Improve naming on storage key --- request-strategy.go | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/request-strategy.go b/request-strategy.go index 4bce3321..3c9df28f 100644 --- a/request-strategy.go +++ b/request-strategy.go @@ -96,9 +96,10 @@ func (cl *Client) doRequests() { for _, t := range cl.torrents { // TODO: We could do metainfo requests here. if t.haveInfo() { - if t.storage.Capacity != nil { - if _, ok := storageLeft[t.storage.Capacity]; !ok { - storageLeft[t.storage.Capacity] = (*t.storage.Capacity)() + key := t.storage.Capacity + if key != nil { + if _, ok := storageLeft[key]; !ok { + storageLeft[key] = (*key)() } } requestOrder.addPieces(t, t.numPieces()) From a2fcce5374b4ed8a1b5d418a611ae4d7846d73dd Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Mon, 10 May 2021 17:43:23 +1000 Subject: [PATCH 17/59] Don't need stable sort --- request-strategy.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/request-strategy.go b/request-strategy.go index 3c9df28f..dfdfc5ab 100644 --- a/request-strategy.go +++ b/request-strategy.go @@ -46,7 +46,7 @@ func (me *clientPieceRequestOrder) removePieces(t *Torrent) { } func (me clientPieceRequestOrder) sort() { - sort.SliceStable(me.pieces, me.less) + sort.Slice(me.pieces, me.less) } func (me *clientPieceRequestOrder) update() { From 16eb8a96931e8dd3c6ca8ecded0bff4ad3f3db81 Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Mon, 10 May 2021 17:43:34 +1000 Subject: [PATCH 18/59] Do requests every 100ms --- request-strategy.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/request-strategy.go b/request-strategy.go index dfdfc5ab..1f906510 100644 --- a/request-strategy.go +++ b/request-strategy.go @@ -81,7 +81,7 @@ func (cl *Client) requester() { select { case <-cl.closed.LockedChan(cl.locker()): return - case <-time.After(10 * time.Millisecond): + case <-time.After(100 * time.Millisecond): } } } From 4577e8a3c121925912483d9c6dff96fefa5e8d9b Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Tue, 11 May 2021 17:14:56 +1000 Subject: [PATCH 19/59] Forgot to remove initial have all state for webseeds Later when the webseed was closed, it was assumed that it had contributed to the piece availability count. --- torrent.go | 1 - 1 file changed, 1 deletion(-) diff --git a/torrent.go b/torrent.go index ae78625f..405748c9 100644 --- a/torrent.go +++ b/torrent.go @@ -2175,7 +2175,6 @@ func (t *Torrent) addWebSeed(url string) { outgoing: true, Network: "http", reconciledHandshakeStats: true, - peerSentHaveAll: true, // TODO: Raise this limit, and instead limit concurrent fetches. PeerMaxRequests: 32, RemoteAddr: remoteAddrFromUrl(url), From 09823b9e361741cfc88a5ce088294ec6253a17d3 Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Wed, 12 May 2021 17:45:36 +1000 Subject: [PATCH 20/59] Rework to improve work stealing and try to thread peers through all request pieces --- misc.go | 10 ++ peerconn.go | 36 +++--- piece.go | 12 +- request-strategy.go | 263 +++++++++++++++++++++++++++++--------------- 4 files changed, 210 insertions(+), 111 deletions(-) diff --git a/misc.go b/misc.go index 5ad01d85..0cd85a3f 100644 --- a/misc.go +++ b/misc.go @@ -151,6 +151,16 @@ func min(as ...int64) int64 { return ret } +func minInt(as ...int) int { + ret := as[0] + for _, a := range as[1:] { + if a < ret { + ret = a + } + } + return ret +} + var unlimited = rate.NewLimiter(rate.Inf, 0) type ( diff --git a/peerconn.go b/peerconn.go index 7e20ba3b..1721663c 100644 --- a/peerconn.go +++ b/peerconn.go @@ -173,17 +173,18 @@ func (cn *Peer) expectingChunks() bool { if !cn.interested { return false } - if !cn.peerChoking { - return true - } for r := range cn.requests { - if cn.peerAllowedFast.Contains(bitmap.BitIndex(r.Index)) { + if !cn.remoteChokingPiece(r.Index.Int()) { return true } } return false } +func (cn *Peer) remoteChokingPiece(piece pieceIndex) bool { + return cn.peerChoking && !cn.peerAllowedFast.Contains(bitmap.BitIndex(piece)) +} + // Returns true if the connection is over IPv6. func (cn *PeerConn) ipv6() bool { ip := cn.remoteIp() @@ -542,13 +543,7 @@ func (pc *PeerConn) writeInterested(interested bool) bool { // are okay. type messageWriter func(pp.Message) bool -func (cn *Peer) request(r Request) error { - if _, ok := cn.requests[r]; ok { - return nil - } - if cn.numLocalRequests() >= cn.nominalMaxRequests() { - return errors.New("too many outstanding requests") - } +func (cn *Peer) shouldRequest(r Request) error { if !cn.peerHasPiece(pieceIndex(r.Index)) { return errors.New("requesting piece peer doesn't have") } @@ -564,15 +559,18 @@ func (cn *Peer) request(r Request) error { if cn.t.pieceQueuedForHash(pieceIndex(r.Index)) { panic("piece is queued for hash") } - if !cn.setInterested(true) { - return errors.New("write buffer full after expressing interest") + return nil +} + +func (cn *Peer) request(r Request) error { + if err := cn.shouldRequest(r); err != nil { + panic(err) } - if cn.peerChoking { - if cn.peerAllowedFast.Get(int(r.Index)) { - torrent.Add("allowed fast requests sent", 1) - } else { - errors.New("peer choking and piece not in allowed fast set") - } + if _, ok := cn.requests[r]; ok { + return nil + } + if cn.numLocalRequests() >= cn.nominalMaxRequests() { + return errors.New("too many outstanding requests") } if cn.requests == nil { cn.requests = make(map[Request]struct{}) diff --git a/piece.go b/piece.go index cee6b7d0..af9607c4 100644 --- a/piece.go +++ b/piece.go @@ -222,10 +222,7 @@ func (p *Piece) SetPriority(prio piecePriority) { p.t.updatePiecePriority(p.index) } -func (p *Piece) uncachedPriority() (ret piecePriority) { - if p.t.pieceComplete(p.index) || p.t.pieceQueuedForHash(p.index) || p.t.hashingPiece(p.index) { - return PiecePriorityNone - } +func (p *Piece) purePriority() (ret piecePriority) { for _, f := range p.files { ret.Raise(f.prio) } @@ -242,6 +239,13 @@ func (p *Piece) uncachedPriority() (ret piecePriority) { return } +func (p *Piece) uncachedPriority() (ret piecePriority) { + if p.t.pieceComplete(p.index) || p.t.pieceQueuedForHash(p.index) || p.t.hashingPiece(p.index) { + return PiecePriorityNone + } + return p.purePriority() +} + // Tells the Client to refetch the completion status from storage, updating priority etc. if // necessary. Might be useful if you know the state of the piece data has changed externally. func (p *Piece) UpdateCompletion() { diff --git a/request-strategy.go b/request-strategy.go index 1f906510..5daa7de1 100644 --- a/request-strategy.go +++ b/request-strategy.go @@ -20,45 +20,17 @@ type pieceRequestOrderPiece struct { prio piecePriority partial bool availability int64 -} - -func (me *clientPieceRequestOrder) addPieces(t *Torrent, numPieces pieceIndex) { - for i := range iter.N(numPieces) { - me.pieces = append(me.pieces, pieceRequestOrderPiece{ - t: t, - index: i, - }) - } + request bool } func (me *clientPieceRequestOrder) Len() int { return len(me.pieces) } -func (me *clientPieceRequestOrder) removePieces(t *Torrent) { - newPieces := make([]pieceRequestOrderPiece, 0, len(me.pieces)-t.numPieces()) - for _, p := range me.pieces { - if p.t != t { - newPieces = append(newPieces, p) - } - } - me.pieces = newPieces -} - func (me clientPieceRequestOrder) sort() { sort.Slice(me.pieces, me.less) } -func (me *clientPieceRequestOrder) update() { - for i := range me.pieces { - p := &me.pieces[i] - tp := p.t.piece(p.index) - p.prio = tp.uncachedPriority() - p.partial = p.t.piecePartiallyDownloaded(p.index) - p.availability = tp.availability - } -} - func (me clientPieceRequestOrder) less(_i, _j int) bool { i := me.pieces[_i] j := me.pieces[_j] @@ -66,9 +38,7 @@ func (me clientPieceRequestOrder) less(_i, _j int) bool { int(j.prio), int(i.prio), ).Bool( j.partial, i.partial, - ).Int64( - i.availability, j.availability, - ).Less() + ).Int64(i.availability, j.availability).Int(i.index, j.index).Less() } func (cl *Client) requester() { @@ -86,45 +56,111 @@ func (cl *Client) requester() { } } +type requestsPeer struct { + cur *Peer + nextRequests map[Request]struct{} + nextInterest bool + requestablePiecesRemaining int +} + +func (rp *requestsPeer) canRequestPiece(p pieceIndex) bool { + return rp.hasPiece(p) && (!rp.choking() || rp.pieceAllowedFast(p)) +} + +func (rp *requestsPeer) hasPiece(i pieceIndex) bool { + return rp.cur.peerHasPiece(i) +} + +func (rp *requestsPeer) pieceAllowedFast(p pieceIndex) bool { + return rp.cur.peerAllowedFast.Contains(p) +} + +func (rp *requestsPeer) choking() bool { + return rp.cur.peerChoking +} + +func (rp *requestsPeer) hasExistingRequest(r Request) bool { + _, ok := rp.cur.requests[r] + return ok +} + +func (rp *requestsPeer) canFitRequest() bool { + return len(rp.nextRequests) < rp.cur.nominalMaxRequests() +} + +// Returns true if it is added and wasn't there before. +func (rp *requestsPeer) addNextRequest(r Request) bool { + _, ok := rp.nextRequests[r] + if ok { + return false + } + rp.nextRequests[r] = struct{}{} + return true +} + +type peersForPieceRequests struct { + requestsInPiece int + *requestsPeer +} + +func (me *peersForPieceRequests) addNextRequest(r Request) { + if me.requestsPeer.addNextRequest(r) { + return + me.requestsInPiece++ + } +} + func (cl *Client) doRequests() { requestOrder := &cl.pieceRequestOrder requestOrder.pieces = requestOrder.pieces[:0] - allPeers := make(map[*Torrent][]*Peer) + allPeers := make(map[*Torrent][]*requestsPeer) // Storage capacity left for this run, keyed by the storage capacity pointer on the storage // TorrentImpl. storageLeft := make(map[*func() *int64]*int64) for _, t := range cl.torrents { // TODO: We could do metainfo requests here. - if t.haveInfo() { - key := t.storage.Capacity - if key != nil { - if _, ok := storageLeft[key]; !ok { - storageLeft[key] = (*key)() - } - } - requestOrder.addPieces(t, t.numPieces()) + if !t.haveInfo() { + continue } - var peers []*Peer + key := t.storage.Capacity + if key != nil { + if _, ok := storageLeft[key]; !ok { + storageLeft[key] = (*key)() + } + } + var peers []*requestsPeer t.iterPeers(func(p *Peer) { if !p.closed.IsSet() { - peers = append(peers, p) + peers = append(peers, &requestsPeer{ + cur: p, + nextRequests: make(map[Request]struct{}), + }) } }) - // Sort in *desc* order, approximately the reverse of worseConn where appropriate. - sort.Slice(peers, func(i, j int) bool { - return multiless.New().Float64( - peers[j].downloadRate(), peers[i].downloadRate(), - ).Uintptr( - uintptr(unsafe.Pointer(peers[j])), uintptr(unsafe.Pointer(peers[i]))).Less() - }) + for i := range iter.N(t.numPieces()) { + tp := t.piece(i) + pp := tp.purePriority() + request := !t.ignorePieceForRequests(i) + requestOrder.pieces = append(requestOrder.pieces, pieceRequestOrderPiece{ + t: t, + index: i, + prio: pp, + partial: t.piecePartiallyDownloaded(i), + availability: tp.availability, + request: request, + }) + if request { + for _, p := range peers { + if p.canRequestPiece(i) { + p.requestablePiecesRemaining++ + } + } + } + } allPeers[t] = peers } - requestOrder.update() requestOrder.sort() - // For a given piece, the set of allPeers indices that absorbed requests for the piece. - contributed := make(map[int]struct{}) for _, p := range requestOrder.pieces { - peers := allPeers[p.t] torrentPiece := p.t.piece(p.index) if left := storageLeft[p.t.storage.Capacity]; left != nil { if *left < int64(torrentPiece.length()) { @@ -132,53 +168,104 @@ func (cl *Client) doRequests() { } *left -= int64(torrentPiece.length()) } - if p.t.ignorePieceForRequests(p.index) { + if !p.request { continue } - p.t.piece(p.index).iterUndirtiedChunks(func(chunk ChunkSpec) bool { + peersForPiece := make([]*peersForPieceRequests, 0, len(allPeers[p.t])) + for _, peer := range allPeers[p.t] { + peersForPiece = append(peersForPiece, &peersForPieceRequests{ + requestsInPiece: 0, + requestsPeer: peer, + }) + } + sortPeersForPiece := func() { + sort.Slice(peersForPiece, func(i, j int) bool { + return multiless.New().Bool( + peersForPiece[j].canFitRequest(), + peersForPiece[i].canFitRequest(), + ).Int( + peersForPiece[i].requestsInPiece, + peersForPiece[j].requestsInPiece, + ).Int( + peersForPiece[i].requestablePiecesRemaining, + peersForPiece[j].requestablePiecesRemaining, + ).Float64( + peersForPiece[j].cur.downloadRate(), + peersForPiece[i].cur.downloadRate(), + ).EagerSameLess( + peersForPiece[i].cur.completedHandshake.Equal(peersForPiece[j].cur.completedHandshake), + peersForPiece[i].cur.completedHandshake.Before(peersForPiece[j].cur.completedHandshake), + // TODO: Probably peer priority can come next + ).Uintptr( + uintptr(unsafe.Pointer(peersForPiece[j].cur)), + uintptr(unsafe.Pointer(peersForPiece[i].cur)), + ).Less() + }) + } + pendingChunksRemaining := int(p.t.pieceNumPendingChunks(p.index)) + torrentPiece.iterUndirtiedChunks(func(chunk ChunkSpec) bool { req := Request{pp.Integer(p.index), chunk} - const skipAlreadyRequested = false - if skipAlreadyRequested { - alreadyRequested := false - p.t.iterPeers(func(p *Peer) { - if _, ok := p.requests[req]; ok { - alreadyRequested = true - } - }) - if alreadyRequested { + pendingChunksRemaining-- + sortPeersForPiece() + for i, peer := range peersForPiece { + if i > pendingChunksRemaining { + break + } + if peer.hasExistingRequest(req) && peer.canFitRequest() { + peer.addNextRequest(req) return true } } - alreadyRequested := false - for peerIndex, peer := range peers { - if alreadyRequested { - // Cancel all requests from "slower" peers after the one that requested it. - peer.cancel(req) - } else { - err := peer.request(req) - if err == nil { - contributed[peerIndex] = struct{}{} - alreadyRequested = true - //log.Printf("requested %v", req) + for _, peer := range peersForPiece { + if !peer.canFitRequest() { + continue + } + if !peer.hasPiece(p.index) { + continue + } + if !peer.pieceAllowedFast(p.index) { + // TODO: Verify that's okay to stay uninterested if we request allowed fast + // pieces. + peer.nextInterest = true + if peer.choking() { + continue } } + peer.addNextRequest(req) + return true } return true }) - // Move requestees for this piece to the back. - lastIndex := len(peers) - 1 - // Probably should sort the contributees, to make the ordering more deterministic. - for peerIndex := range contributed { - peers[peerIndex], peers[lastIndex] = peers[lastIndex], peers[peerIndex] - delete(contributed, peerIndex) - lastIndex-- + for _, peer := range peersForPiece { + if peer.canRequestPiece(p.index) { + peer.requestablePiecesRemaining-- + } } } - for _, t := range cl.torrents { - t.iterPeers(func(p *Peer) { - if !p.peerChoking && p.numLocalRequests() == 0 && !p.writeBufferFull() { - p.setInterested(false) + for _, peers := range allPeers { + for _, rp := range peers { + if rp.requestablePiecesRemaining != 0 { + panic(rp.requestablePiecesRemaining) } - }) + applyPeerNextRequests(rp) + } + } +} + +func applyPeerNextRequests(rp *requestsPeer) { + p := rp.cur + p.setInterested(rp.nextInterest) + for req := range p.requests { + if _, ok := rp.nextRequests[req]; !ok { + p.cancel(req) + } + } + for req := range rp.nextRequests { + err := p.request(req) + if err != nil { + panic(err) + } else { + //log.Print(req) + } } } From ef489f4a4417d1f40016fa32e53a153f4deded7d Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Wed, 12 May 2021 21:47:35 +1000 Subject: [PATCH 21/59] Fixes to reassigning chunks in requests --- request-strategy.go | 31 ++++++++++++++++++++++--------- 1 file changed, 22 insertions(+), 9 deletions(-) diff --git a/request-strategy.go b/request-strategy.go index 5daa7de1..4682720a 100644 --- a/request-strategy.go +++ b/request-strategy.go @@ -180,10 +180,7 @@ func (cl *Client) doRequests() { } sortPeersForPiece := func() { sort.Slice(peersForPiece, func(i, j int) bool { - return multiless.New().Bool( - peersForPiece[j].canFitRequest(), - peersForPiece[i].canFitRequest(), - ).Int( + return multiless.New().Int( peersForPiece[i].requestsInPiece, peersForPiece[j].requestsInPiece, ).Int( @@ -207,14 +204,27 @@ func (cl *Client) doRequests() { req := Request{pp.Integer(p.index), chunk} pendingChunksRemaining-- sortPeersForPiece() - for i, peer := range peersForPiece { - if i > pendingChunksRemaining { + skipped := 0 + // Try up to the number of peers that could legitimately receive the request equal to + // the number of chunks left. This should ensure that only the best peers serve the last + // few chunks in a piece. + for _, peer := range peersForPiece { + if !peer.canFitRequest() || !peer.hasPiece(p.index) || (!peer.pieceAllowedFast(p.index) && peer.choking()) { + continue + } + if skipped > pendingChunksRemaining { break } - if peer.hasExistingRequest(req) && peer.canFitRequest() { - peer.addNextRequest(req) - return true + if !peer.hasExistingRequest(req) { + skipped++ + continue } + if !peer.pieceAllowedFast(p.index) { + // We must stay interested for this. + peer.nextInterest = true + } + peer.addNextRequest(req) + return true } for _, peer := range peersForPiece { if !peer.canFitRequest() { @@ -236,6 +246,9 @@ func (cl *Client) doRequests() { } return true }) + if pendingChunksRemaining != 0 { + panic(pendingChunksRemaining) + } for _, peer := range peersForPiece { if peer.canRequestPiece(p.index) { peer.requestablePiecesRemaining-- From f19564d5970522fcd0b901bb86fc7824737c8263 Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Wed, 12 May 2021 21:48:11 +1000 Subject: [PATCH 22/59] Improve piece availability status output --- torrent.go | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/torrent.go b/torrent.go index 405748c9..05598b3c 100644 --- a/torrent.go +++ b/torrent.go @@ -12,6 +12,7 @@ import ( "net/http" "net/url" "sort" + "strings" "sync" "text/tabwriter" "time" @@ -553,13 +554,17 @@ func (t *Torrent) newMetadataExtensionMessage(c *PeerConn, msgType int, piece in } type pieceAvailabilityRun struct { - availability int64 count pieceIndex + availability int64 +} + +func (me pieceAvailabilityRun) String() string { + return fmt.Sprintf("%v(%v)", me.count, me.availability) } func (t *Torrent) pieceAvailabilityRuns() (ret []pieceAvailabilityRun) { rle := missinggo.NewRunLengthEncoder(func(el interface{}, count uint64) { - ret = append(ret, pieceAvailabilityRun{el.(int64), int(count)}) + ret = append(ret, pieceAvailabilityRun{availability: el.(int64), count: int(count)}) }) for _, p := range t.pieces { rle.Append(p.availability, 1) @@ -652,7 +657,12 @@ func (t *Torrent) writeStatus(w io.Writer) { if t.info != nil { fmt.Fprintf(w, "Num Pieces: %d (%d completed)\n", t.numPieces(), t.numPiecesCompleted()) fmt.Fprintf(w, "Piece States: %s\n", t.pieceStateRuns()) - fmt.Fprintf(w, "Piece availability: %v\n", t.pieceAvailabilityRuns()) + fmt.Fprintf(w, "Piece availability: %v\n", strings.Join(func() (ret []string) { + for _, run := range t.pieceAvailabilityRuns() { + ret = append(ret, run.String()) + } + return + }(), " ")) } fmt.Fprintf(w, "Reader Pieces:") t.forReaderOffsetPieces(func(begin, end pieceIndex) (again bool) { From 233135493fcbef0d27e1de6e63f3eeba9097b9eb Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Wed, 12 May 2021 21:48:23 +1000 Subject: [PATCH 23/59] Improve num requests per piece status output --- peerconn.go | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/peerconn.go b/peerconn.go index 1721663c..a5d078e0 100644 --- a/peerconn.go +++ b/peerconn.go @@ -8,6 +8,7 @@ import ( "io" "math/rand" "net" + "sort" "strconv" "strings" "sync" @@ -366,8 +367,19 @@ func (cn *Peer) writeStatus(w io.Writer, t *Torrent) { cn.downloadRate()/(1<<10), ) fmt.Fprintf(w, " requested pieces:") + type pieceNumRequestsType struct { + piece pieceIndex + numRequests int + } + var pieceNumRequests []pieceNumRequestsType for piece, count := range cn.numRequestsByPiece() { - fmt.Fprintf(w, " %v (%v)", piece, count) + pieceNumRequests = append(pieceNumRequests, pieceNumRequestsType{piece, count}) + } + sort.Slice(pieceNumRequests, func(i, j int) bool { + return pieceNumRequests[i].piece < pieceNumRequests[j].piece + }) + for _, elem := range pieceNumRequests { + fmt.Fprintf(w, " %v(%v)", elem.piece, elem.numRequests) } fmt.Fprintf(w, "\n") } From 0d4e566fc03447ed94988886141f45ee3ba51d6d Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Thu, 13 May 2021 09:56:58 +1000 Subject: [PATCH 24/59] Extract request strategy stuff into a separate module --- client.go | 3 +- client_test.go | 2 + misc.go | 30 ++-- piece.go | 29 +--- request-strategy.go | 289 ++++++--------------------------- request-strategy/order.go | 226 ++++++++++++++++++++++++++ request-strategy/order_test.go | 1 + request-strategy/peer.go | 29 ++++ request-strategy/piece.go | 15 ++ types/types.go | 52 ++++++ 10 files changed, 392 insertions(+), 284 deletions(-) create mode 100644 request-strategy/order.go create mode 100644 request-strategy/order_test.go create mode 100644 request-strategy/peer.go create mode 100644 request-strategy/piece.go create mode 100644 types/types.go diff --git a/client.go b/client.go index e5915c1f..8544d4e3 100644 --- a/client.go +++ b/client.go @@ -25,6 +25,7 @@ import ( "github.com/anacrolix/missinggo/v2/pproffd" "github.com/anacrolix/sync" "github.com/anacrolix/torrent/internal/limiter" + request_strategy "github.com/anacrolix/torrent/request-strategy" "github.com/anacrolix/torrent/tracker" "github.com/anacrolix/torrent/webtorrent" "github.com/davecgh/go-spew/spew" @@ -82,7 +83,7 @@ type Client struct { activeAnnounceLimiter limiter.Instance - pieceRequestOrder clientPieceRequestOrder + pieceRequestOrder request_strategy.ClientPieceOrder } type ipStr string diff --git a/client_test.go b/client_test.go index adcd0c72..e8f652eb 100644 --- a/client_test.go +++ b/client_test.go @@ -554,6 +554,8 @@ func TestPeerInvalidHave(t *testing.T) { t: tt, }} cn.peerImpl = cn + cl.lock() + defer cl.unlock() assert.NoError(t, cn.peerSentHave(0)) assert.Error(t, cn.peerSentHave(1)) } diff --git a/misc.go b/misc.go index 0cd85a3f..508f0a63 100644 --- a/misc.go +++ b/misc.go @@ -5,29 +5,27 @@ import ( "net" "github.com/anacrolix/missinggo/v2" + "github.com/anacrolix/torrent/types" "golang.org/x/time/rate" "github.com/anacrolix/torrent/metainfo" pp "github.com/anacrolix/torrent/peer_protocol" ) -type ChunkSpec struct { - Begin, Length pp.Integer -} +type ( + Request = types.Request + ChunkSpec = types.ChunkSpec + piecePriority = types.PiecePriority +) -type Request struct { - Index pp.Integer - ChunkSpec -} - -func (r Request) ToMsg(mt pp.MessageType) pp.Message { - return pp.Message{ - Type: mt, - Index: r.Index, - Begin: r.Begin, - Length: r.Length, - } -} +const ( + PiecePriorityNormal = types.PiecePriorityNormal + PiecePriorityNone = types.PiecePriorityNone + PiecePriorityNow = types.PiecePriorityNow + PiecePriorityReadahead = types.PiecePriorityReadahead + PiecePriorityNext = types.PiecePriorityNext + PiecePriorityHigh = types.PiecePriorityHigh +) func newRequest(index, begin, length pp.Integer) Request { return Request{index, ChunkSpec{begin, length}} diff --git a/piece.go b/piece.go index af9607c4..3ee648d3 100644 --- a/piece.go +++ b/piece.go @@ -11,33 +11,6 @@ import ( "github.com/anacrolix/torrent/storage" ) -// Describes the importance of obtaining a particular piece. -type piecePriority byte - -func (pp *piecePriority) Raise(maybe piecePriority) bool { - if maybe > *pp { - *pp = maybe - return true - } - return false -} - -// Priority for use in PriorityBitmap -func (me piecePriority) BitmapPriority() int { - return -int(me) -} - -const ( - PiecePriorityNone piecePriority = iota // Not wanted. Must be the zero value. - PiecePriorityNormal // Wanted. - PiecePriorityHigh // Wanted a lot. - PiecePriorityReadahead // May be required soon. - // Succeeds a piece where a read occurred. Currently the same as Now, - // apparently due to issues with caching. - PiecePriorityNext - PiecePriorityNow // A Reader is reading in this piece. Highest urgency. -) - type Piece struct { // The completed piece SHA1 hash, from the metainfo "pieces" field. hash *metainfo.Hash @@ -272,7 +245,7 @@ func (p *Piece) State() PieceState { return p.t.PieceState(p.index) } -func (p *Piece) iterUndirtiedChunks(f func(ChunkSpec) bool) bool { +func (p *Piece) iterUndirtiedChunks(f func(cs ChunkSpec) bool) bool { for i := pp.Integer(0); i < p.numChunks(); i++ { if p.chunkIndexDirty(i) { continue diff --git a/request-strategy.go b/request-strategy.go index 4682720a..a1221ad6 100644 --- a/request-strategy.go +++ b/request-strategy.go @@ -1,46 +1,14 @@ package torrent import ( - "sort" + "log" "time" "unsafe" - "github.com/anacrolix/multiless" - pp "github.com/anacrolix/torrent/peer_protocol" - "github.com/bradfitz/iter" + request_strategy "github.com/anacrolix/torrent/request-strategy" + "github.com/anacrolix/torrent/types" ) -type clientPieceRequestOrder struct { - pieces []pieceRequestOrderPiece -} - -type pieceRequestOrderPiece struct { - t *Torrent - index pieceIndex - prio piecePriority - partial bool - availability int64 - request bool -} - -func (me *clientPieceRequestOrder) Len() int { - return len(me.pieces) -} - -func (me clientPieceRequestOrder) sort() { - sort.Slice(me.pieces, me.less) -} - -func (me clientPieceRequestOrder) less(_i, _j int) bool { - i := me.pieces[_i] - j := me.pieces[_j] - return multiless.New().Int( - int(j.prio), int(i.prio), - ).Bool( - j.partial, i.partial, - ).Int64(i.availability, j.availability).Int(i.index, j.index).Less() -} - func (cl *Client) requester() { for { func() { @@ -56,229 +24,72 @@ func (cl *Client) requester() { } } -type requestsPeer struct { - cur *Peer - nextRequests map[Request]struct{} - nextInterest bool - requestablePiecesRemaining int -} - -func (rp *requestsPeer) canRequestPiece(p pieceIndex) bool { - return rp.hasPiece(p) && (!rp.choking() || rp.pieceAllowedFast(p)) -} - -func (rp *requestsPeer) hasPiece(i pieceIndex) bool { - return rp.cur.peerHasPiece(i) -} - -func (rp *requestsPeer) pieceAllowedFast(p pieceIndex) bool { - return rp.cur.peerAllowedFast.Contains(p) -} - -func (rp *requestsPeer) choking() bool { - return rp.cur.peerChoking -} - -func (rp *requestsPeer) hasExistingRequest(r Request) bool { - _, ok := rp.cur.requests[r] - return ok -} - -func (rp *requestsPeer) canFitRequest() bool { - return len(rp.nextRequests) < rp.cur.nominalMaxRequests() -} - -// Returns true if it is added and wasn't there before. -func (rp *requestsPeer) addNextRequest(r Request) bool { - _, ok := rp.nextRequests[r] - if ok { - return false - } - rp.nextRequests[r] = struct{}{} - return true -} - -type peersForPieceRequests struct { - requestsInPiece int - *requestsPeer -} - -func (me *peersForPieceRequests) addNextRequest(r Request) { - if me.requestsPeer.addNextRequest(r) { - return - me.requestsInPiece++ - } -} - func (cl *Client) doRequests() { - requestOrder := &cl.pieceRequestOrder - requestOrder.pieces = requestOrder.pieces[:0] - allPeers := make(map[*Torrent][]*requestsPeer) - // Storage capacity left for this run, keyed by the storage capacity pointer on the storage - // TorrentImpl. - storageLeft := make(map[*func() *int64]*int64) + ts := make([]*request_strategy.Torrent, 0, len(cl.torrents)) for _, t := range cl.torrents { - // TODO: We could do metainfo requests here. - if !t.haveInfo() { - continue + rst := &request_strategy.Torrent{} + if t.storage != nil { + rst.Capacity = t.storage.Capacity } - key := t.storage.Capacity - if key != nil { - if _, ok := storageLeft[key]; !ok { - storageLeft[key] = (*key)() - } + for i := range t.pieces { + p := &t.pieces[i] + rst.Pieces = append(rst.Pieces, request_strategy.Piece{ + Request: !t.ignorePieceForRequests(i), + Priority: p.purePriority(), + Partial: t.piecePartiallyDownloaded(i), + Availability: p.availability, + Length: int64(p.length()), + NumPendingChunks: int(t.pieceNumPendingChunks(i)), + IterPendingChunks: func(f func(types.ChunkSpec)) { + p.iterUndirtiedChunks(func(cs ChunkSpec) bool { + f(cs) + return true + }) + }, + }) } - var peers []*requestsPeer t.iterPeers(func(p *Peer) { - if !p.closed.IsSet() { - peers = append(peers, &requestsPeer{ - cur: p, - nextRequests: make(map[Request]struct{}), - }) + if p.closed.IsSet() { + return } + rst.Peers = append(rst.Peers, &request_strategy.Peer{ + HasPiece: p.peerHasPiece, + MaxRequests: p.nominalMaxRequests, + HasExistingRequest: func(r request_strategy.Request) bool { + _, ok := p.requests[r] + return ok + }, + Choking: p.peerChoking, + PieceAllowedFast: func(i pieceIndex) bool { + return p.peerAllowedFast.Contains(i) + }, + DownloadRate: p.downloadRate(), + Age: time.Since(p.completedHandshake), + Id: unsafe.Pointer(p), + }) }) - for i := range iter.N(t.numPieces()) { - tp := t.piece(i) - pp := tp.purePriority() - request := !t.ignorePieceForRequests(i) - requestOrder.pieces = append(requestOrder.pieces, pieceRequestOrderPiece{ - t: t, - index: i, - prio: pp, - partial: t.piecePartiallyDownloaded(i), - availability: tp.availability, - request: request, - }) - if request { - for _, p := range peers { - if p.canRequestPiece(i) { - p.requestablePiecesRemaining++ - } - } - } - } - allPeers[t] = peers + ts = append(ts, rst) } - requestOrder.sort() - for _, p := range requestOrder.pieces { - torrentPiece := p.t.piece(p.index) - if left := storageLeft[p.t.storage.Capacity]; left != nil { - if *left < int64(torrentPiece.length()) { - continue - } - *left -= int64(torrentPiece.length()) - } - if !p.request { - continue - } - peersForPiece := make([]*peersForPieceRequests, 0, len(allPeers[p.t])) - for _, peer := range allPeers[p.t] { - peersForPiece = append(peersForPiece, &peersForPieceRequests{ - requestsInPiece: 0, - requestsPeer: peer, - }) - } - sortPeersForPiece := func() { - sort.Slice(peersForPiece, func(i, j int) bool { - return multiless.New().Int( - peersForPiece[i].requestsInPiece, - peersForPiece[j].requestsInPiece, - ).Int( - peersForPiece[i].requestablePiecesRemaining, - peersForPiece[j].requestablePiecesRemaining, - ).Float64( - peersForPiece[j].cur.downloadRate(), - peersForPiece[i].cur.downloadRate(), - ).EagerSameLess( - peersForPiece[i].cur.completedHandshake.Equal(peersForPiece[j].cur.completedHandshake), - peersForPiece[i].cur.completedHandshake.Before(peersForPiece[j].cur.completedHandshake), - // TODO: Probably peer priority can come next - ).Uintptr( - uintptr(unsafe.Pointer(peersForPiece[j].cur)), - uintptr(unsafe.Pointer(peersForPiece[i].cur)), - ).Less() - }) - } - pendingChunksRemaining := int(p.t.pieceNumPendingChunks(p.index)) - torrentPiece.iterUndirtiedChunks(func(chunk ChunkSpec) bool { - req := Request{pp.Integer(p.index), chunk} - pendingChunksRemaining-- - sortPeersForPiece() - skipped := 0 - // Try up to the number of peers that could legitimately receive the request equal to - // the number of chunks left. This should ensure that only the best peers serve the last - // few chunks in a piece. - for _, peer := range peersForPiece { - if !peer.canFitRequest() || !peer.hasPiece(p.index) || (!peer.pieceAllowedFast(p.index) && peer.choking()) { - continue - } - if skipped > pendingChunksRemaining { - break - } - if !peer.hasExistingRequest(req) { - skipped++ - continue - } - if !peer.pieceAllowedFast(p.index) { - // We must stay interested for this. - peer.nextInterest = true - } - peer.addNextRequest(req) - return true - } - for _, peer := range peersForPiece { - if !peer.canFitRequest() { - continue - } - if !peer.hasPiece(p.index) { - continue - } - if !peer.pieceAllowedFast(p.index) { - // TODO: Verify that's okay to stay uninterested if we request allowed fast - // pieces. - peer.nextInterest = true - if peer.choking() { - continue - } - } - peer.addNextRequest(req) - return true - } - return true - }) - if pendingChunksRemaining != 0 { - panic(pendingChunksRemaining) - } - for _, peer := range peersForPiece { - if peer.canRequestPiece(p.index) { - peer.requestablePiecesRemaining-- - } - } - } - for _, peers := range allPeers { - for _, rp := range peers { - if rp.requestablePiecesRemaining != 0 { - panic(rp.requestablePiecesRemaining) - } - applyPeerNextRequests(rp) - } + nextPeerStates := cl.pieceRequestOrder.DoRequests(ts) + for p, state := range nextPeerStates { + applyPeerNextRequestState(p, state) } } -func applyPeerNextRequests(rp *requestsPeer) { - p := rp.cur - p.setInterested(rp.nextInterest) +func applyPeerNextRequestState(_p request_strategy.PeerPointer, rp request_strategy.PeerNextRequestState) { + p := (*Peer)(_p) + p.setInterested(rp.Interested) for req := range p.requests { - if _, ok := rp.nextRequests[req]; !ok { + if _, ok := rp.Requests[req]; !ok { p.cancel(req) } } - for req := range rp.nextRequests { + for req := range rp.Requests { err := p.request(req) if err != nil { panic(err) } else { - //log.Print(req) + log.Print(req) } } } diff --git a/request-strategy/order.go b/request-strategy/order.go new file mode 100644 index 00000000..3c7d82ab --- /dev/null +++ b/request-strategy/order.go @@ -0,0 +1,226 @@ +package request_strategy + +import ( + "sort" + + "github.com/anacrolix/multiless" + pp "github.com/anacrolix/torrent/peer_protocol" + "github.com/anacrolix/torrent/types" +) + +type ( + Request = types.Request + pieceIndex = types.PieceIndex + piecePriority = types.PiecePriority +) + +type ClientPieceOrder struct { + pieces []pieceRequestOrderPiece +} + +type pieceRequestOrderPiece struct { + t *Torrent + index pieceIndex + Piece +} + +func (me *ClientPieceOrder) Len() int { + return len(me.pieces) +} + +func (me ClientPieceOrder) sort() { + sort.Slice(me.pieces, me.less) +} + +func (me ClientPieceOrder) less(_i, _j int) bool { + i := me.pieces[_i] + j := me.pieces[_j] + return multiless.New().Int( + int(j.Priority), int(i.Priority), + ).Bool( + j.Partial, i.Partial, + ).Int64(i.Availability, j.Availability).Int(i.index, j.index).Less() +} + +type requestsPeer struct { + *Peer + nextState PeerNextRequestState + requestablePiecesRemaining int +} + +func (rp *requestsPeer) canFitRequest() bool { + return len(rp.nextState.Requests) < rp.MaxRequests() +} + +// Returns true if it is added and wasn't there before. +func (rp *requestsPeer) addNextRequest(r Request) bool { + _, ok := rp.nextState.Requests[r] + if ok { + return false + } + rp.nextState.Requests[r] = struct{}{} + return true +} + +type peersForPieceRequests struct { + requestsInPiece int + *requestsPeer +} + +func (me *peersForPieceRequests) addNextRequest(r Request) { + if me.requestsPeer.addNextRequest(r) { + return + me.requestsInPiece++ + } +} + +type Torrent struct { + Pieces []Piece + Capacity *func() *int64 + Peers []*Peer // not closed. +} + +func (requestOrder *ClientPieceOrder) DoRequests(torrents []*Torrent) map[PeerPointer]PeerNextRequestState { + requestOrder.pieces = requestOrder.pieces[:0] + allPeers := make(map[*Torrent][]*requestsPeer) + // Storage capacity left for this run, keyed by the storage capacity pointer on the storage + // TorrentImpl. + storageLeft := make(map[*func() *int64]*int64) + for _, t := range torrents { + // TODO: We could do metainfo requests here. + key := t.Capacity + if key != nil { + if _, ok := storageLeft[key]; !ok { + storageLeft[key] = (*key)() + } + } + var peers []*requestsPeer + for _, p := range t.Peers { + peers = append(peers, &requestsPeer{ + Peer: p, + nextState: PeerNextRequestState{ + Requests: make(map[Request]struct{}), + }, + }) + } + for i, tp := range t.Pieces { + requestOrder.pieces = append(requestOrder.pieces, pieceRequestOrderPiece{ + t: t, + index: i, + Piece: tp, + }) + if tp.Request { + for _, p := range peers { + if p.canRequestPiece(i) { + p.requestablePiecesRemaining++ + } + } + } + } + allPeers[t] = peers + } + requestOrder.sort() + for _, p := range requestOrder.pieces { + torrentPiece := p + if left := storageLeft[p.t.Capacity]; left != nil { + if *left < int64(torrentPiece.Length) { + continue + } + *left -= int64(torrentPiece.Length) + } + if !p.Request { + continue + } + peersForPiece := make([]*peersForPieceRequests, 0, len(allPeers[p.t])) + for _, peer := range allPeers[p.t] { + peersForPiece = append(peersForPiece, &peersForPieceRequests{ + requestsInPiece: 0, + requestsPeer: peer, + }) + } + sortPeersForPiece := func() { + sort.Slice(peersForPiece, func(i, j int) bool { + return multiless.New().Int( + peersForPiece[i].requestsInPiece, + peersForPiece[j].requestsInPiece, + ).Int( + peersForPiece[i].requestablePiecesRemaining, + peersForPiece[j].requestablePiecesRemaining, + ).Float64( + peersForPiece[j].DownloadRate, + peersForPiece[i].DownloadRate, + ).Int64( + int64(peersForPiece[j].Age), int64(peersForPiece[i].Age), + // TODO: Probably peer priority can come next + ).Uintptr( + uintptr(peersForPiece[j].Id), + uintptr(peersForPiece[i].Id), + ).MustLess() + }) + } + pendingChunksRemaining := int(p.NumPendingChunks) + torrentPiece.IterPendingChunks(func(chunk types.ChunkSpec) { + req := Request{pp.Integer(p.index), chunk} + pendingChunksRemaining-- + sortPeersForPiece() + skipped := 0 + // Try up to the number of peers that could legitimately receive the request equal to + // the number of chunks left. This should ensure that only the best peers serve the last + // few chunks in a piece. + for _, peer := range peersForPiece { + if !peer.canFitRequest() || !peer.HasPiece(p.index) || (!peer.PieceAllowedFast(p.index) && peer.Choking) { + continue + } + if skipped > pendingChunksRemaining { + break + } + if !peer.HasExistingRequest(req) { + skipped++ + continue + } + if !peer.PieceAllowedFast(p.index) { + // We must stay interested for this. + peer.nextState.Interested = true + } + peer.addNextRequest(req) + return + } + for _, peer := range peersForPiece { + if !peer.canFitRequest() { + continue + } + if !peer.HasPiece(p.index) { + continue + } + if !peer.PieceAllowedFast(p.index) { + // TODO: Verify that's okay to stay uninterested if we request allowed fast + // pieces. + peer.nextState.Interested = true + if peer.Choking { + continue + } + } + peer.addNextRequest(req) + return + } + }) + if pendingChunksRemaining != 0 { + panic(pendingChunksRemaining) + } + for _, peer := range peersForPiece { + if peer.canRequestPiece(p.index) { + peer.requestablePiecesRemaining-- + } + } + } + ret := make(map[PeerPointer]PeerNextRequestState) + for _, peers := range allPeers { + for _, rp := range peers { + if rp.requestablePiecesRemaining != 0 { + panic(rp.requestablePiecesRemaining) + } + ret[rp.Id] = rp.nextState + } + } + return ret +} diff --git a/request-strategy/order_test.go b/request-strategy/order_test.go new file mode 100644 index 00000000..e014624e --- /dev/null +++ b/request-strategy/order_test.go @@ -0,0 +1 @@ +package request_strategy diff --git a/request-strategy/peer.go b/request-strategy/peer.go new file mode 100644 index 00000000..4a3d0689 --- /dev/null +++ b/request-strategy/peer.go @@ -0,0 +1,29 @@ +package request_strategy + +import ( + "time" + "unsafe" +) + +type PeerNextRequestState struct { + Interested bool + Requests map[Request]struct{} +} + +type PeerPointer = unsafe.Pointer + +type Peer struct { + HasPiece func(pieceIndex) bool + MaxRequests func() int + HasExistingRequest func(Request) bool + Choking bool + PieceAllowedFast func(pieceIndex) bool + DownloadRate float64 + Age time.Duration + Id PeerPointer +} + +// TODO: This might be used in more places I think. +func (p *Peer) canRequestPiece(i pieceIndex) bool { + return p.HasPiece(i) && (!p.Choking || p.PieceAllowedFast(i)) +} diff --git a/request-strategy/piece.go b/request-strategy/piece.go new file mode 100644 index 00000000..ec778a8c --- /dev/null +++ b/request-strategy/piece.go @@ -0,0 +1,15 @@ +package request_strategy + +import ( + "github.com/anacrolix/torrent/types" +) + +type Piece struct { + Request bool + Priority piecePriority + Partial bool + Availability int64 + Length int64 + NumPendingChunks int + IterPendingChunks func(func(types.ChunkSpec)) +} diff --git a/types/types.go b/types/types.go new file mode 100644 index 00000000..a06f7e6a --- /dev/null +++ b/types/types.go @@ -0,0 +1,52 @@ +package types + +import ( + pp "github.com/anacrolix/torrent/peer_protocol" +) + +type PieceIndex = int + +type ChunkSpec struct { + Begin, Length pp.Integer +} + +type Request struct { + Index pp.Integer + ChunkSpec +} + +func (r Request) ToMsg(mt pp.MessageType) pp.Message { + return pp.Message{ + Type: mt, + Index: r.Index, + Begin: r.Begin, + Length: r.Length, + } +} + +// Describes the importance of obtaining a particular piece. +type PiecePriority byte + +func (pp *PiecePriority) Raise(maybe PiecePriority) bool { + if maybe > *pp { + *pp = maybe + return true + } + return false +} + +// Priority for use in PriorityBitmap +func (me PiecePriority) BitmapPriority() int { + return -int(me) +} + +const ( + PiecePriorityNone PiecePriority = iota // Not wanted. Must be the zero value. + PiecePriorityNormal // Wanted. + PiecePriorityHigh // Wanted a lot. + PiecePriorityReadahead // May be required soon. + // Succeeds a piece where a read occurred. Currently the same as Now, + // apparently due to issues with caching. + PiecePriorityNext + PiecePriorityNow // A Reader is reading in this piece. Highest urgency. +) From 26837ee73c872a1a8131f778c780c3a0f84d1bc7 Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Thu, 13 May 2021 11:26:22 +1000 Subject: [PATCH 25/59] Add a working request strategy test --- request-strategy.go | 16 +++-- request-strategy/order.go | 105 +++++++++++++++++---------------- request-strategy/order_test.go | 81 +++++++++++++++++++++++++ request-strategy/peer.go | 23 +++++--- 4 files changed, 163 insertions(+), 62 deletions(-) diff --git a/request-strategy.go b/request-strategy.go index a1221ad6..cb30d39a 100644 --- a/request-strategy.go +++ b/request-strategy.go @@ -52,9 +52,9 @@ func (cl *Client) doRequests() { if p.closed.IsSet() { return } - rst.Peers = append(rst.Peers, &request_strategy.Peer{ + rst.Peers = append(rst.Peers, request_strategy.Peer{ HasPiece: p.peerHasPiece, - MaxRequests: p.nominalMaxRequests, + MaxRequests: p.nominalMaxRequests(), HasExistingRequest: func(r request_strategy.Request) bool { _, ok := p.requests[r] return ok @@ -65,7 +65,7 @@ func (cl *Client) doRequests() { }, DownloadRate: p.downloadRate(), Age: time.Since(p.completedHandshake), - Id: unsafe.Pointer(p), + Id: (*peerId)(p), }) }) ts = append(ts, rst) @@ -76,8 +76,14 @@ func (cl *Client) doRequests() { } } -func applyPeerNextRequestState(_p request_strategy.PeerPointer, rp request_strategy.PeerNextRequestState) { - p := (*Peer)(_p) +type peerId Peer + +func (p *peerId) Uintptr() uintptr { + return uintptr(unsafe.Pointer(p)) +} + +func applyPeerNextRequestState(_p request_strategy.PeerId, rp request_strategy.PeerNextRequestState) { + p := (*Peer)(_p.(*peerId)) p.setInterested(rp.Interested) for req := range p.requests { if _, ok := rp.Requests[req]; !ok { diff --git a/request-strategy/order.go b/request-strategy/order.go index 3c7d82ab..f0a5eaef 100644 --- a/request-strategy/order.go +++ b/request-strategy/order.go @@ -6,12 +6,15 @@ import ( "github.com/anacrolix/multiless" pp "github.com/anacrolix/torrent/peer_protocol" "github.com/anacrolix/torrent/types" + "github.com/davecgh/go-spew/spew" ) type ( Request = types.Request pieceIndex = types.PieceIndex piecePriority = types.PiecePriority + // This can be made into a type-param later, will be great for testing. + ChunkSpec = types.ChunkSpec ) type ClientPieceOrder struct { @@ -43,13 +46,13 @@ func (me ClientPieceOrder) less(_i, _j int) bool { } type requestsPeer struct { - *Peer + Peer nextState PeerNextRequestState requestablePiecesRemaining int } func (rp *requestsPeer) canFitRequest() bool { - return len(rp.nextState.Requests) < rp.MaxRequests() + return len(rp.nextState.Requests) < rp.MaxRequests } // Returns true if it is added and wasn't there before. @@ -69,7 +72,6 @@ type peersForPieceRequests struct { func (me *peersForPieceRequests) addNextRequest(r Request) { if me.requestsPeer.addNextRequest(r) { - return me.requestsInPiece++ } } @@ -77,10 +79,10 @@ func (me *peersForPieceRequests) addNextRequest(r Request) { type Torrent struct { Pieces []Piece Capacity *func() *int64 - Peers []*Peer // not closed. + Peers []Peer // not closed. } -func (requestOrder *ClientPieceOrder) DoRequests(torrents []*Torrent) map[PeerPointer]PeerNextRequestState { +func (requestOrder *ClientPieceOrder) DoRequests(torrents []*Torrent) map[PeerId]PeerNextRequestState { requestOrder.pieces = requestOrder.pieces[:0] allPeers := make(map[*Torrent][]*requestsPeer) // Storage capacity left for this run, keyed by the storage capacity pointer on the storage @@ -153,57 +155,60 @@ func (requestOrder *ClientPieceOrder) DoRequests(torrents []*Torrent) map[PeerPo int64(peersForPiece[j].Age), int64(peersForPiece[i].Age), // TODO: Probably peer priority can come next ).Uintptr( - uintptr(peersForPiece[j].Id), - uintptr(peersForPiece[i].Id), + peersForPiece[i].Id.Uintptr(), + peersForPiece[j].Id.Uintptr(), ).MustLess() }) } pendingChunksRemaining := int(p.NumPendingChunks) - torrentPiece.IterPendingChunks(func(chunk types.ChunkSpec) { - req := Request{pp.Integer(p.index), chunk} - pendingChunksRemaining-- - sortPeersForPiece() - skipped := 0 - // Try up to the number of peers that could legitimately receive the request equal to - // the number of chunks left. This should ensure that only the best peers serve the last - // few chunks in a piece. - for _, peer := range peersForPiece { - if !peer.canFitRequest() || !peer.HasPiece(p.index) || (!peer.PieceAllowedFast(p.index) && peer.Choking) { - continue - } - if skipped > pendingChunksRemaining { - break - } - if !peer.HasExistingRequest(req) { - skipped++ - continue - } - if !peer.PieceAllowedFast(p.index) { - // We must stay interested for this. - peer.nextState.Interested = true - } - peer.addNextRequest(req) - return - } - for _, peer := range peersForPiece { - if !peer.canFitRequest() { - continue - } - if !peer.HasPiece(p.index) { - continue - } - if !peer.PieceAllowedFast(p.index) { - // TODO: Verify that's okay to stay uninterested if we request allowed fast - // pieces. - peer.nextState.Interested = true - if peer.Choking { + if f := torrentPiece.IterPendingChunks; f != nil { + f(func(chunk types.ChunkSpec) { + req := Request{pp.Integer(p.index), chunk} + pendingChunksRemaining-- + sortPeersForPiece() + spew.Dump(peersForPiece) + skipped := 0 + // Try up to the number of peers that could legitimately receive the request equal to + // the number of chunks left. This should ensure that only the best peers serve the last + // few chunks in a piece. + for _, peer := range peersForPiece { + if !peer.canFitRequest() || !peer.HasPiece(p.index) || (!peer.pieceAllowedFastOrDefault(p.index) && peer.Choking) { continue } + if skipped >= pendingChunksRemaining { + break + } + if f := peer.HasExistingRequest; f == nil || !f(req) { + skipped++ + continue + } + if !peer.pieceAllowedFastOrDefault(p.index) { + // We must stay interested for this. + peer.nextState.Interested = true + } + peer.addNextRequest(req) + return } - peer.addNextRequest(req) - return - } - }) + for _, peer := range peersForPiece { + if !peer.canFitRequest() { + continue + } + if !peer.HasPiece(p.index) { + continue + } + if !peer.pieceAllowedFastOrDefault(p.index) { + // TODO: Verify that's okay to stay uninterested if we request allowed fast + // pieces. + peer.nextState.Interested = true + if peer.Choking { + continue + } + } + peer.addNextRequest(req) + return + } + }) + } if pendingChunksRemaining != 0 { panic(pendingChunksRemaining) } @@ -213,7 +218,7 @@ func (requestOrder *ClientPieceOrder) DoRequests(torrents []*Torrent) map[PeerPo } } } - ret := make(map[PeerPointer]PeerNextRequestState) + ret := make(map[PeerId]PeerNextRequestState) for _, peers := range allPeers { for _, rp := range peers { if rp.requestablePiecesRemaining != 0 { diff --git a/request-strategy/order_test.go b/request-strategy/order_test.go index e014624e..c972b301 100644 --- a/request-strategy/order_test.go +++ b/request-strategy/order_test.go @@ -1 +1,82 @@ package request_strategy + +import ( + "math" + "testing" + + pp "github.com/anacrolix/torrent/peer_protocol" + qt "github.com/frankban/quicktest" +) + +func r(i pieceIndex, begin int) Request { + return Request{pp.Integer(i), ChunkSpec{pp.Integer(begin), 1}} +} + +func chunkIter(offsets ...int) func(func(ChunkSpec)) { + return func(f func(ChunkSpec)) { + for _, offset := range offsets { + f(ChunkSpec{pp.Integer(offset), 1}) + } + } +} + +func requestSetFromSlice(rs ...Request) (ret map[Request]struct{}) { + ret = make(map[Request]struct{}, len(rs)) + for _, r := range rs { + ret[r] = struct{}{} + } + return +} + +type intPeerId int + +func (i intPeerId) Uintptr() uintptr { + return uintptr(i) +} + +func TestStealingFromSlowerPeers(t *testing.T) { + c := qt.New(t) + order := ClientPieceOrder{} + basePeer := Peer{ + HasPiece: func(i pieceIndex) bool { + return true + }, + MaxRequests: math.MaxInt16, + DownloadRate: 2, + } + stealee := basePeer + stealee.DownloadRate = 1 + stealee.HasExistingRequest = func(r Request) bool { + return true + } + stealee.Id = intPeerId(1) + firstStealer := basePeer + firstStealer.Id = intPeerId(2) + secondStealer := basePeer + secondStealer.Id = intPeerId(3) + c.Assert(order.DoRequests([]*Torrent{{ + Pieces: []Piece{{ + Request: true, + NumPendingChunks: 2, + IterPendingChunks: chunkIter(0, 1), + }}, + Peers: []Peer{ + stealee, + firstStealer, + secondStealer, + }, + }}), qt.ContentEquals, map[PeerId]PeerNextRequestState{ + intPeerId(2): { + Interested: true, + Requests: requestSetFromSlice(r(0, 0)), + }, + intPeerId(3): { + Interested: true, + Requests: requestSetFromSlice(r(0, 1)), + }, + stealee.Id: { + Interested: false, + Requests: requestSetFromSlice(), + }, + }) +} diff --git a/request-strategy/peer.go b/request-strategy/peer.go index 4a3d0689..21ef0d2e 100644 --- a/request-strategy/peer.go +++ b/request-strategy/peer.go @@ -2,7 +2,6 @@ package request_strategy import ( "time" - "unsafe" ) type PeerNextRequestState struct { @@ -10,20 +9,30 @@ type PeerNextRequestState struct { Requests map[Request]struct{} } -type PeerPointer = unsafe.Pointer +type PeerId interface { + Uintptr() uintptr +} type Peer struct { - HasPiece func(pieceIndex) bool - MaxRequests func() int - HasExistingRequest func(Request) bool + HasPiece func(i pieceIndex) bool + MaxRequests int + HasExistingRequest func(r Request) bool Choking bool PieceAllowedFast func(pieceIndex) bool DownloadRate float64 Age time.Duration - Id PeerPointer + // This is passed back out at the end, so must support equality. Could be a type-param later. + Id PeerId +} + +func (p *Peer) pieceAllowedFastOrDefault(i pieceIndex) bool { + if f := p.PieceAllowedFast; f != nil { + return f(i) + } + return false } // TODO: This might be used in more places I think. func (p *Peer) canRequestPiece(i pieceIndex) bool { - return p.HasPiece(i) && (!p.Choking || p.PieceAllowedFast(i)) + return p.HasPiece(i) && (!p.Choking || (p.PieceAllowedFast != nil && p.PieceAllowedFast(i))) } From e7c5ed679f9655249e0bd4d242626e4726d28c72 Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Thu, 13 May 2021 13:49:57 +1000 Subject: [PATCH 26/59] Don't log each applied request --- request-strategy.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/request-strategy.go b/request-strategy.go index cb30d39a..21596a23 100644 --- a/request-strategy.go +++ b/request-strategy.go @@ -1,7 +1,6 @@ package torrent import ( - "log" "time" "unsafe" @@ -95,7 +94,7 @@ func applyPeerNextRequestState(_p request_strategy.PeerId, rp request_strategy.P if err != nil { panic(err) } else { - log.Print(req) + //log.Print(req) } } } From 07ba6e921028c5eb4186439f983997b8f8cfba79 Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Thu, 13 May 2021 13:50:41 +1000 Subject: [PATCH 27/59] New tests and fixes for them Not complete. There's still a request-stealing balancing issue, but it's functional for now. --- request-strategy/order.go | 12 +++-- request-strategy/order_test.go | 92 +++++++++++++++++++++++++++++++++- 2 files changed, 99 insertions(+), 5 deletions(-) diff --git a/request-strategy/order.go b/request-strategy/order.go index f0a5eaef..307e47ef 100644 --- a/request-strategy/order.go +++ b/request-strategy/order.go @@ -1,12 +1,12 @@ package request_strategy import ( + "math" "sort" "github.com/anacrolix/multiless" pp "github.com/anacrolix/torrent/peer_protocol" "github.com/anacrolix/torrent/types" - "github.com/davecgh/go-spew/spew" ) type ( @@ -164,24 +164,28 @@ func (requestOrder *ClientPieceOrder) DoRequests(torrents []*Torrent) map[PeerId if f := torrentPiece.IterPendingChunks; f != nil { f(func(chunk types.ChunkSpec) { req := Request{pp.Integer(p.index), chunk} - pendingChunksRemaining-- + defer func() { pendingChunksRemaining-- }() sortPeersForPiece() - spew.Dump(peersForPiece) skipped := 0 // Try up to the number of peers that could legitimately receive the request equal to // the number of chunks left. This should ensure that only the best peers serve the last // few chunks in a piece. + lowestNumRequestsInPiece := math.MaxInt16 for _, peer := range peersForPiece { if !peer.canFitRequest() || !peer.HasPiece(p.index) || (!peer.pieceAllowedFastOrDefault(p.index) && peer.Choking) { continue } - if skipped >= pendingChunksRemaining { + if skipped+1 >= pendingChunksRemaining { break } if f := peer.HasExistingRequest; f == nil || !f(req) { skipped++ + lowestNumRequestsInPiece = peer.requestsInPiece continue } + if peer.requestsInPiece > lowestNumRequestsInPiece { + break + } if !peer.pieceAllowedFastOrDefault(p.index) { // We must stay interested for this. peer.nextState.Interested = true diff --git a/request-strategy/order_test.go b/request-strategy/order_test.go index c972b301..20961403 100644 --- a/request-strategy/order_test.go +++ b/request-strategy/order_test.go @@ -34,7 +34,50 @@ func (i intPeerId) Uintptr() uintptr { return uintptr(i) } -func TestStealingFromSlowerPeers(t *testing.T) { +func TestStealingFromSlowerPeer(t *testing.T) { + c := qt.New(t) + order := ClientPieceOrder{} + basePeer := Peer{ + HasPiece: func(i pieceIndex) bool { + return true + }, + MaxRequests: math.MaxInt16, + DownloadRate: 2, + } + // Slower than the stealers, but has all requests already. + stealee := basePeer + stealee.DownloadRate = 1 + stealee.HasExistingRequest = func(r Request) bool { + return true + } + stealee.Id = intPeerId(1) + firstStealer := basePeer + firstStealer.Id = intPeerId(2) + secondStealer := basePeer + secondStealer.Id = intPeerId(3) + results := order.DoRequests([]*Torrent{{ + Pieces: []Piece{{ + Request: true, + NumPendingChunks: 5, + IterPendingChunks: chunkIter(0, 1, 2, 3, 4), + }}, + Peers: []Peer{ + stealee, + firstStealer, + secondStealer, + }, + }}) + c.Assert(results, qt.HasLen, 3) + check := func(p PeerId, l int) { + c.Check(results[p].Requests, qt.HasLen, l) + c.Check(results[p].Interested, qt.Equals, l > 0) + } + check(stealee.Id, 1) + check(firstStealer.Id, 2) + check(secondStealer.Id, 2) +} + +func TestStealingFromSlowerPeersBasic(t *testing.T) { c := qt.New(t) order := ClientPieceOrder{} basePeer := Peer{ @@ -80,3 +123,50 @@ func TestStealingFromSlowerPeers(t *testing.T) { }, }) } + +func TestPeerKeepsExistingIfReasonable(t *testing.T) { + c := qt.New(t) + order := ClientPieceOrder{} + basePeer := Peer{ + HasPiece: func(i pieceIndex) bool { + return true + }, + MaxRequests: math.MaxInt16, + DownloadRate: 2, + } + // Slower than the stealers, but has all requests already. + stealee := basePeer + stealee.DownloadRate = 1 + keepReq := r(0, 0) + stealee.HasExistingRequest = func(r Request) bool { + return r == keepReq + } + stealee.Id = intPeerId(1) + firstStealer := basePeer + firstStealer.Id = intPeerId(2) + secondStealer := basePeer + secondStealer.Id = intPeerId(3) + results := order.DoRequests([]*Torrent{{ + Pieces: []Piece{{ + Request: true, + NumPendingChunks: 4, + IterPendingChunks: chunkIter(0, 1, 3, 4), + }}, + Peers: []Peer{ + stealee, + firstStealer, + secondStealer, + }, + }}) + c.Assert(results, qt.HasLen, 3) + check := func(p PeerId, l int) { + c.Check(results[p].Requests, qt.HasLen, l) + c.Check(results[p].Interested, qt.Equals, l > 0) + } + check(firstStealer.Id, 2) + check(secondStealer.Id, 1) + c.Check(results[stealee.Id], qt.ContentEquals, PeerNextRequestState{ + Interested: true, + Requests: requestSetFromSlice(keepReq), + }) +} From 4e9f707aeb9b1c37bb0a0481cc6ef017d43f557a Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Thu, 13 May 2021 18:35:49 +1000 Subject: [PATCH 28/59] Move piece allocation to its own func --- request-strategy/order.go | 180 +++++++++++++++++++------------------- 1 file changed, 92 insertions(+), 88 deletions(-) diff --git a/request-strategy/order.go b/request-strategy/order.go index 307e47ef..da6890e5 100644 --- a/request-strategy/order.go +++ b/request-strategy/order.go @@ -133,94 +133,7 @@ func (requestOrder *ClientPieceOrder) DoRequests(torrents []*Torrent) map[PeerId if !p.Request { continue } - peersForPiece := make([]*peersForPieceRequests, 0, len(allPeers[p.t])) - for _, peer := range allPeers[p.t] { - peersForPiece = append(peersForPiece, &peersForPieceRequests{ - requestsInPiece: 0, - requestsPeer: peer, - }) - } - sortPeersForPiece := func() { - sort.Slice(peersForPiece, func(i, j int) bool { - return multiless.New().Int( - peersForPiece[i].requestsInPiece, - peersForPiece[j].requestsInPiece, - ).Int( - peersForPiece[i].requestablePiecesRemaining, - peersForPiece[j].requestablePiecesRemaining, - ).Float64( - peersForPiece[j].DownloadRate, - peersForPiece[i].DownloadRate, - ).Int64( - int64(peersForPiece[j].Age), int64(peersForPiece[i].Age), - // TODO: Probably peer priority can come next - ).Uintptr( - peersForPiece[i].Id.Uintptr(), - peersForPiece[j].Id.Uintptr(), - ).MustLess() - }) - } - pendingChunksRemaining := int(p.NumPendingChunks) - if f := torrentPiece.IterPendingChunks; f != nil { - f(func(chunk types.ChunkSpec) { - req := Request{pp.Integer(p.index), chunk} - defer func() { pendingChunksRemaining-- }() - sortPeersForPiece() - skipped := 0 - // Try up to the number of peers that could legitimately receive the request equal to - // the number of chunks left. This should ensure that only the best peers serve the last - // few chunks in a piece. - lowestNumRequestsInPiece := math.MaxInt16 - for _, peer := range peersForPiece { - if !peer.canFitRequest() || !peer.HasPiece(p.index) || (!peer.pieceAllowedFastOrDefault(p.index) && peer.Choking) { - continue - } - if skipped+1 >= pendingChunksRemaining { - break - } - if f := peer.HasExistingRequest; f == nil || !f(req) { - skipped++ - lowestNumRequestsInPiece = peer.requestsInPiece - continue - } - if peer.requestsInPiece > lowestNumRequestsInPiece { - break - } - if !peer.pieceAllowedFastOrDefault(p.index) { - // We must stay interested for this. - peer.nextState.Interested = true - } - peer.addNextRequest(req) - return - } - for _, peer := range peersForPiece { - if !peer.canFitRequest() { - continue - } - if !peer.HasPiece(p.index) { - continue - } - if !peer.pieceAllowedFastOrDefault(p.index) { - // TODO: Verify that's okay to stay uninterested if we request allowed fast - // pieces. - peer.nextState.Interested = true - if peer.Choking { - continue - } - } - peer.addNextRequest(req) - return - } - }) - } - if pendingChunksRemaining != 0 { - panic(pendingChunksRemaining) - } - for _, peer := range peersForPiece { - if peer.canRequestPiece(p.index) { - peer.requestablePiecesRemaining-- - } - } + allocatePendingChunks(p, allPeers[p.t]) } ret := make(map[PeerId]PeerNextRequestState) for _, peers := range allPeers { @@ -233,3 +146,94 @@ func (requestOrder *ClientPieceOrder) DoRequests(torrents []*Torrent) map[PeerId } return ret } + +func allocatePendingChunks(p pieceRequestOrderPiece, peers []*requestsPeer) { + peersForPiece := make([]*peersForPieceRequests, 0, len(peers)) + for _, peer := range peers { + peersForPiece = append(peersForPiece, &peersForPieceRequests{ + requestsInPiece: 0, + requestsPeer: peer, + }) + } + sortPeersForPiece := func() { + sort.Slice(peersForPiece, func(i, j int) bool { + return multiless.New().Int( + peersForPiece[i].requestsInPiece, + peersForPiece[j].requestsInPiece, + ).Int( + peersForPiece[i].requestablePiecesRemaining, + peersForPiece[j].requestablePiecesRemaining, + ).Float64( + peersForPiece[j].DownloadRate, + peersForPiece[i].DownloadRate, + ).Int64( + int64(peersForPiece[j].Age), int64(peersForPiece[i].Age), + // TODO: Probably peer priority can come next + ).Uintptr( + peersForPiece[i].Id.Uintptr(), + peersForPiece[j].Id.Uintptr(), + ).MustLess() + }) + } + pendingChunksRemaining := int(p.NumPendingChunks) + if f := p.IterPendingChunks; f != nil { + f(func(chunk types.ChunkSpec) { + req := Request{pp.Integer(p.index), chunk} + defer func() { pendingChunksRemaining-- }() + sortPeersForPiece() + skipped := 0 + // Try up to the number of peers that could legitimately receive the request equal to + // the number of chunks left. This should ensure that only the best peers serve the last + // few chunks in a piece. + lowestNumRequestsInPiece := math.MaxInt16 + for _, peer := range peersForPiece { + if !peer.canFitRequest() || !peer.HasPiece(p.index) || (!peer.pieceAllowedFastOrDefault(p.index) && peer.Choking) { + continue + } + if skipped+1 >= pendingChunksRemaining { + break + } + if f := peer.HasExistingRequest; f == nil || !f(req) { + skipped++ + lowestNumRequestsInPiece = peer.requestsInPiece + continue + } + if peer.requestsInPiece > lowestNumRequestsInPiece { + break + } + if !peer.pieceAllowedFastOrDefault(p.index) { + // We must stay interested for this. + peer.nextState.Interested = true + } + peer.addNextRequest(req) + return + } + for _, peer := range peersForPiece { + if !peer.canFitRequest() { + continue + } + if !peer.HasPiece(p.index) { + continue + } + if !peer.pieceAllowedFastOrDefault(p.index) { + // TODO: Verify that's okay to stay uninterested if we request allowed fast + // pieces. + peer.nextState.Interested = true + if peer.Choking { + continue + } + } + peer.addNextRequest(req) + return + } + }) + } + if pendingChunksRemaining != 0 { + panic(pendingChunksRemaining) + } + for _, peer := range peersForPiece { + if peer.canRequestPiece(p.index) { + peer.requestablePiecesRemaining-- + } + } +} From b80e2b08b5b846882798cd4981f0e9b28a17fb40 Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Thu, 13 May 2021 20:56:12 +1000 Subject: [PATCH 29/59] Get request rebalancing working optimally! --- request-strategy/order.go | 142 ++++++++++++++++++--------------- request-strategy/order_test.go | 59 +++++++++++++- request-strategy/piece.go | 7 ++ 3 files changed, 142 insertions(+), 66 deletions(-) diff --git a/request-strategy/order.go b/request-strategy/order.go index da6890e5..c16cb3ea 100644 --- a/request-strategy/order.go +++ b/request-strategy/order.go @@ -1,7 +1,6 @@ package request_strategy import ( - "math" "sort" "github.com/anacrolix/multiless" @@ -55,14 +54,12 @@ func (rp *requestsPeer) canFitRequest() bool { return len(rp.nextState.Requests) < rp.MaxRequests } -// Returns true if it is added and wasn't there before. -func (rp *requestsPeer) addNextRequest(r Request) bool { +func (rp *requestsPeer) addNextRequest(r Request) { _, ok := rp.nextState.Requests[r] if ok { - return false + panic("should only add once") } rp.nextState.Requests[r] = struct{}{} - return true } type peersForPieceRequests struct { @@ -71,9 +68,8 @@ type peersForPieceRequests struct { } func (me *peersForPieceRequests) addNextRequest(r Request) { - if me.requestsPeer.addNextRequest(r) { - me.requestsInPiece++ - } + me.requestsPeer.addNextRequest(r) + me.requestsInPiece++ } type Torrent struct { @@ -155,9 +151,16 @@ func allocatePendingChunks(p pieceRequestOrderPiece, peers []*requestsPeer) { requestsPeer: peer, }) } - sortPeersForPiece := func() { + defer func() { + for _, peer := range peersForPiece { + if peer.canRequestPiece(p.index) { + peer.requestablePiecesRemaining-- + } + } + }() + sortPeersForPiece := func(byHasRequest *Request) { sort.Slice(peersForPiece, func(i, j int) bool { - return multiless.New().Int( + ml := multiless.New().Int( peersForPiece[i].requestsInPiece, peersForPiece[j].requestsInPiece, ).Int( @@ -166,7 +169,13 @@ func allocatePendingChunks(p pieceRequestOrderPiece, peers []*requestsPeer) { ).Float64( peersForPiece[j].DownloadRate, peersForPiece[i].DownloadRate, - ).Int64( + ) + if byHasRequest != nil { + _, iHas := peersForPiece[i].nextState.Requests[*byHasRequest] + _, jHas := peersForPiece[j].nextState.Requests[*byHasRequest] + ml = ml.Bool(jHas, iHas) + } + return ml.Int64( int64(peersForPiece[j].Age), int64(peersForPiece[i].Age), // TODO: Probably peer priority can come next ).Uintptr( @@ -175,65 +184,68 @@ func allocatePendingChunks(p pieceRequestOrderPiece, peers []*requestsPeer) { ).MustLess() }) } + preallocated := make(map[ChunkSpec]*peersForPieceRequests, p.NumPendingChunks) + p.iterPendingChunksWrapper(func(spec ChunkSpec) { + req := Request{pp.Integer(p.index), spec} + for _, p := range peersForPiece { + if h := p.HasExistingRequest; h != nil && h(req) { + preallocated[spec] = p + p.addNextRequest(req) + } + } + }) pendingChunksRemaining := int(p.NumPendingChunks) - if f := p.IterPendingChunks; f != nil { - f(func(chunk types.ChunkSpec) { - req := Request{pp.Integer(p.index), chunk} - defer func() { pendingChunksRemaining-- }() - sortPeersForPiece() - skipped := 0 - // Try up to the number of peers that could legitimately receive the request equal to - // the number of chunks left. This should ensure that only the best peers serve the last - // few chunks in a piece. - lowestNumRequestsInPiece := math.MaxInt16 - for _, peer := range peersForPiece { - if !peer.canFitRequest() || !peer.HasPiece(p.index) || (!peer.pieceAllowedFastOrDefault(p.index) && peer.Choking) { - continue - } - if skipped+1 >= pendingChunksRemaining { - break - } - if f := peer.HasExistingRequest; f == nil || !f(req) { - skipped++ - lowestNumRequestsInPiece = peer.requestsInPiece - continue - } - if peer.requestsInPiece > lowestNumRequestsInPiece { - break - } - if !peer.pieceAllowedFastOrDefault(p.index) { - // We must stay interested for this. - peer.nextState.Interested = true - } - peer.addNextRequest(req) - return + p.iterPendingChunksWrapper(func(chunk types.ChunkSpec) { + if _, ok := preallocated[chunk]; ok { + return + } + req := Request{pp.Integer(p.index), chunk} + defer func() { pendingChunksRemaining-- }() + sortPeersForPiece(nil) + for _, peer := range peersForPiece { + if !peer.canFitRequest() { + continue } - for _, peer := range peersForPiece { - if !peer.canFitRequest() { - continue - } - if !peer.HasPiece(p.index) { - continue - } - if !peer.pieceAllowedFastOrDefault(p.index) { - // TODO: Verify that's okay to stay uninterested if we request allowed fast - // pieces. - peer.nextState.Interested = true - if peer.Choking { - continue - } - } - peer.addNextRequest(req) - return + if !peer.HasPiece(p.index) { + continue } - }) + if !peer.pieceAllowedFastOrDefault(p.index) { + // TODO: Verify that's okay to stay uninterested if we request allowed fast pieces. + peer.nextState.Interested = true + if peer.Choking { + continue + } + } + peer.addNextRequest(req) + return + } + }) +chunk: + for chunk, prePeer := range preallocated { + req := Request{pp.Integer(p.index), chunk} + prePeer.requestsInPiece-- + sortPeersForPiece(&req) + delete(prePeer.nextState.Requests, req) + for _, peer := range peersForPiece { + if !peer.canFitRequest() { + continue + } + if !peer.HasPiece(p.index) { + continue + } + if !peer.pieceAllowedFastOrDefault(p.index) { + // TODO: Verify that's okay to stay uninterested if we request allowed fast pieces. + peer.nextState.Interested = true + if peer.Choking { + continue + } + } + pendingChunksRemaining-- + peer.addNextRequest(req) + continue chunk + } } if pendingChunksRemaining != 0 { panic(pendingChunksRemaining) } - for _, peer := range peersForPiece { - if peer.canRequestPiece(p.index) { - peer.requestablePiecesRemaining-- - } - } } diff --git a/request-strategy/order_test.go b/request-strategy/order_test.go index 20961403..213b6f52 100644 --- a/request-strategy/order_test.go +++ b/request-strategy/order_test.go @@ -5,6 +5,7 @@ import ( "testing" pp "github.com/anacrolix/torrent/peer_protocol" + "github.com/bradfitz/iter" qt "github.com/frankban/quicktest" ) @@ -12,6 +13,14 @@ func r(i pieceIndex, begin int) Request { return Request{pp.Integer(i), ChunkSpec{pp.Integer(begin), 1}} } +func chunkIterRange(end int) func(func(ChunkSpec)) { + return func(f func(ChunkSpec)) { + for offset := range iter.N(end) { + f(ChunkSpec{pp.Integer(offset), 1}) + } + } +} + func chunkIter(offsets ...int) func(func(ChunkSpec)) { return func(f func(ChunkSpec)) { for _, offset := range offsets { @@ -59,7 +68,7 @@ func TestStealingFromSlowerPeer(t *testing.T) { Pieces: []Piece{{ Request: true, NumPendingChunks: 5, - IterPendingChunks: chunkIter(0, 1, 2, 3, 4), + IterPendingChunks: chunkIterRange(5), }}, Peers: []Peer{ stealee, @@ -170,3 +179,51 @@ func TestPeerKeepsExistingIfReasonable(t *testing.T) { Requests: requestSetFromSlice(keepReq), }) } + +func TestDontStealUnnecessarily(t *testing.T) { + c := qt.New(t) + order := ClientPieceOrder{} + basePeer := Peer{ + HasPiece: func(i pieceIndex) bool { + return true + }, + MaxRequests: math.MaxInt16, + DownloadRate: 2, + } + // Slower than the stealers, but has all requests already. + stealee := basePeer + stealee.DownloadRate = 1 + keepReqs := requestSetFromSlice(r(0, 0), r(0, 1), r(0, 2)) + stealee.HasExistingRequest = func(r Request) bool { + _, ok := keepReqs[r] + return ok + } + stealee.Id = intPeerId(1) + firstStealer := basePeer + firstStealer.Id = intPeerId(2) + secondStealer := basePeer + secondStealer.Id = intPeerId(3) + results := order.DoRequests([]*Torrent{{ + Pieces: []Piece{{ + Request: true, + NumPendingChunks: 9, + IterPendingChunks: chunkIterRange(9), + }}, + Peers: []Peer{ + firstStealer, + stealee, + secondStealer, + }, + }}) + c.Assert(results, qt.HasLen, 3) + check := func(p PeerId, l int) { + c.Check(results[p].Requests, qt.HasLen, l) + c.Check(results[p].Interested, qt.Equals, l > 0) + } + check(firstStealer.Id, 3) + check(secondStealer.Id, 3) + c.Check(results[stealee.Id], qt.ContentEquals, PeerNextRequestState{ + Interested: true, + Requests: keepReqs, + }) +} diff --git a/request-strategy/piece.go b/request-strategy/piece.go index ec778a8c..508ed829 100644 --- a/request-strategy/piece.go +++ b/request-strategy/piece.go @@ -13,3 +13,10 @@ type Piece struct { NumPendingChunks int IterPendingChunks func(func(types.ChunkSpec)) } + +func (p *Piece) iterPendingChunksWrapper(f func(ChunkSpec)) { + i := p.IterPendingChunks + if i != nil { + i(f) + } +} From 9dd85e5bd273bb888c66ef49588e06ae811ffa79 Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Fri, 14 May 2021 10:23:18 +1000 Subject: [PATCH 30/59] Fix non-deterministic test --- request-strategy/order_test.go | 25 ++++++++++--------------- 1 file changed, 10 insertions(+), 15 deletions(-) diff --git a/request-strategy/order_test.go b/request-strategy/order_test.go index 213b6f52..8c518b50 100644 --- a/request-strategy/order_test.go +++ b/request-strategy/order_test.go @@ -86,6 +86,11 @@ func TestStealingFromSlowerPeer(t *testing.T) { check(secondStealer.Id, 2) } +func checkNumRequestsAndInterest(c *qt.C, next PeerNextRequestState, num int, interest bool) { + c.Check(next.Requests, qt.HasLen, num) + c.Check(next.Interested, qt.Equals, interest) +} + func TestStealingFromSlowerPeersBasic(t *testing.T) { c := qt.New(t) order := ClientPieceOrder{} @@ -106,7 +111,7 @@ func TestStealingFromSlowerPeersBasic(t *testing.T) { firstStealer.Id = intPeerId(2) secondStealer := basePeer secondStealer.Id = intPeerId(3) - c.Assert(order.DoRequests([]*Torrent{{ + results := order.DoRequests([]*Torrent{{ Pieces: []Piece{{ Request: true, NumPendingChunks: 2, @@ -117,20 +122,10 @@ func TestStealingFromSlowerPeersBasic(t *testing.T) { firstStealer, secondStealer, }, - }}), qt.ContentEquals, map[PeerId]PeerNextRequestState{ - intPeerId(2): { - Interested: true, - Requests: requestSetFromSlice(r(0, 0)), - }, - intPeerId(3): { - Interested: true, - Requests: requestSetFromSlice(r(0, 1)), - }, - stealee.Id: { - Interested: false, - Requests: requestSetFromSlice(), - }, - }) + }}) + checkNumRequestsAndInterest(c, results[firstStealer.Id], 1, true) + checkNumRequestsAndInterest(c, results[secondStealer.Id], 1, true) + checkNumRequestsAndInterest(c, results[stealee.Id], 0, false) } func TestPeerKeepsExistingIfReasonable(t *testing.T) { From 8e9cb9f2be69498a95cd784df5f0a1f6f6664c09 Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Fri, 14 May 2021 10:24:50 +1000 Subject: [PATCH 31/59] Do checks for preallocated requests too Otherwise we reserve requests with the assumption that they can be assigned later, and they actually might not be. --- request-strategy/order.go | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/request-strategy/order.go b/request-strategy/order.go index c16cb3ea..56002594 100644 --- a/request-strategy/order.go +++ b/request-strategy/order.go @@ -187,11 +187,18 @@ func allocatePendingChunks(p pieceRequestOrderPiece, peers []*requestsPeer) { preallocated := make(map[ChunkSpec]*peersForPieceRequests, p.NumPendingChunks) p.iterPendingChunksWrapper(func(spec ChunkSpec) { req := Request{pp.Integer(p.index), spec} - for _, p := range peersForPiece { - if h := p.HasExistingRequest; h != nil && h(req) { - preallocated[spec] = p - p.addNextRequest(req) + for _, peer := range peersForPiece { + if h := peer.HasExistingRequest; h == nil || !h(req) { + continue } + if !peer.canFitRequest() { + continue + } + if !peer.canRequestPiece(p.index) { + continue + } + preallocated[spec] = peer + peer.addNextRequest(req) } }) pendingChunksRemaining := int(p.NumPendingChunks) From 307d6d178fc6502a800380a642460b6cf4f75e94 Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Fri, 14 May 2021 11:50:41 +1000 Subject: [PATCH 32/59] Prepare to implement max unverified bytes --- request-strategy.go | 5 ++- request-strategy/order.go | 61 ++++++++++++++++++++++++------------- request-strategy/torrent.go | 11 +++++++ 3 files changed, 55 insertions(+), 22 deletions(-) create mode 100644 request-strategy/torrent.go diff --git a/request-strategy.go b/request-strategy.go index 21596a23..84473b40 100644 --- a/request-strategy.go +++ b/request-strategy.go @@ -26,7 +26,10 @@ func (cl *Client) requester() { func (cl *Client) doRequests() { ts := make([]*request_strategy.Torrent, 0, len(cl.torrents)) for _, t := range cl.torrents { - rst := &request_strategy.Torrent{} + rst := &request_strategy.Torrent{ + StableId: uintptr(unsafe.Pointer(t)), + //MaxUnverifiedBytes: 1 << 20, + } if t.storage != nil { rst.Capacity = t.storage.Capacity } diff --git a/request-strategy/order.go b/request-strategy/order.go index 56002594..4742254d 100644 --- a/request-strategy/order.go +++ b/request-strategy/order.go @@ -20,8 +20,16 @@ type ClientPieceOrder struct { pieces []pieceRequestOrderPiece } +type orderTorrent struct { + *Torrent + unverifiedBytes int64 + // Potentially shared with other torrents. + storageLeft *int64 + peers []*requestsPeer +} + type pieceRequestOrderPiece struct { - t *Torrent + t *orderTorrent index pieceIndex Piece } @@ -41,7 +49,13 @@ func (me ClientPieceOrder) less(_i, _j int) bool { int(j.Priority), int(i.Priority), ).Bool( j.Partial, i.Partial, - ).Int64(i.Availability, j.Availability).Int(i.index, j.index).Less() + ).Int64( + i.Availability, j.Availability, + ).Int( + i.index, j.index, + ).Uintptr( + i.t.StableId, j.t.StableId, + ).MustLess() } type requestsPeer struct { @@ -72,25 +86,24 @@ func (me *peersForPieceRequests) addNextRequest(r Request) { me.requestsInPiece++ } -type Torrent struct { - Pieces []Piece - Capacity *func() *int64 - Peers []Peer // not closed. -} - func (requestOrder *ClientPieceOrder) DoRequests(torrents []*Torrent) map[PeerId]PeerNextRequestState { requestOrder.pieces = requestOrder.pieces[:0] - allPeers := make(map[*Torrent][]*requestsPeer) // Storage capacity left for this run, keyed by the storage capacity pointer on the storage // TorrentImpl. storageLeft := make(map[*func() *int64]*int64) - for _, t := range torrents { + orderTorrents := make([]*orderTorrent, 0, len(torrents)) + for _, _t := range torrents { // TODO: We could do metainfo requests here. + t := &orderTorrent{ + Torrent: _t, + unverifiedBytes: 0, + } key := t.Capacity if key != nil { if _, ok := storageLeft[key]; !ok { storageLeft[key] = (*key)() } + t.storageLeft = storageLeft[key] } var peers []*requestsPeer for _, p := range t.Peers { @@ -107,7 +120,7 @@ func (requestOrder *ClientPieceOrder) DoRequests(torrents []*Torrent) map[PeerId index: i, Piece: tp, }) - if tp.Request { + if tp.Request && tp.NumPendingChunks != 0 { for _, p := range peers { if p.canRequestPiece(i) { p.requestablePiecesRemaining++ @@ -115,25 +128,31 @@ func (requestOrder *ClientPieceOrder) DoRequests(torrents []*Torrent) map[PeerId } } } - allPeers[t] = peers + t.peers = peers + orderTorrents = append(orderTorrents, t) } requestOrder.sort() - for _, p := range requestOrder.pieces { - torrentPiece := p - if left := storageLeft[p.t.Capacity]; left != nil { - if *left < int64(torrentPiece.Length) { + for _, piece := range requestOrder.pieces { + if left := piece.t.storageLeft; left != nil { + if *left < int64(piece.Length) { continue } - *left -= int64(torrentPiece.Length) + *left -= int64(piece.Length) } - if !p.Request { + if !piece.Request || piece.NumPendingChunks == 0 { continue } - allocatePendingChunks(p, allPeers[p.t]) + if piece.t.MaxUnverifiedBytes != 0 && piece.t.unverifiedBytes+piece.Length > piece.t.MaxUnverifiedBytes { + //log.Print("skipping piece") + continue + } + allocatePendingChunks(piece, piece.t.peers) + piece.t.unverifiedBytes += piece.Length + //log.Print(piece.t.unverifiedBytes) } ret := make(map[PeerId]PeerNextRequestState) - for _, peers := range allPeers { - for _, rp := range peers { + for _, ots := range orderTorrents { + for _, rp := range ots.peers { if rp.requestablePiecesRemaining != 0 { panic(rp.requestablePiecesRemaining) } diff --git a/request-strategy/torrent.go b/request-strategy/torrent.go new file mode 100644 index 00000000..2090c7a8 --- /dev/null +++ b/request-strategy/torrent.go @@ -0,0 +1,11 @@ +package request_strategy + +type Torrent struct { + Pieces []Piece + Capacity *func() *int64 + Peers []Peer // not closed. + // Some value that's unique and stable between runs. Could even use the infohash? + StableId uintptr + + MaxUnverifiedBytes int64 +} From 63b0e42731c61b34d2172354424cd7892a836320 Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Fri, 14 May 2021 13:06:12 +1000 Subject: [PATCH 33/59] Get max unverified bytes working --- request-strategy.go | 8 +- request-strategy/order.go | 149 ++++++++++++++++++--------------- request-strategy/order_test.go | 8 +- request-strategy/piece.go | 6 +- 4 files changed, 92 insertions(+), 79 deletions(-) diff --git a/request-strategy.go b/request-strategy.go index 84473b40..8d8b3b23 100644 --- a/request-strategy.go +++ b/request-strategy.go @@ -24,11 +24,11 @@ func (cl *Client) requester() { } func (cl *Client) doRequests() { - ts := make([]*request_strategy.Torrent, 0, len(cl.torrents)) + ts := make([]request_strategy.Torrent, 0, len(cl.torrents)) for _, t := range cl.torrents { - rst := &request_strategy.Torrent{ - StableId: uintptr(unsafe.Pointer(t)), - //MaxUnverifiedBytes: 1 << 20, + rst := request_strategy.Torrent{ + StableId: uintptr(unsafe.Pointer(t)), + MaxUnverifiedBytes: 10 << 20, } if t.storage != nil { rst.Capacity = t.storage.Capacity diff --git a/request-strategy/order.go b/request-strategy/order.go index 4742254d..9052fb0c 100644 --- a/request-strategy/order.go +++ b/request-strategy/order.go @@ -16,46 +16,31 @@ type ( ChunkSpec = types.ChunkSpec ) -type ClientPieceOrder struct { - pieces []pieceRequestOrderPiece -} +type ClientPieceOrder struct{} -type orderTorrent struct { - *Torrent +type filterTorrent struct { + Torrent unverifiedBytes int64 // Potentially shared with other torrents. storageLeft *int64 - peers []*requestsPeer } -type pieceRequestOrderPiece struct { - t *orderTorrent - index pieceIndex - Piece -} - -func (me *ClientPieceOrder) Len() int { - return len(me.pieces) -} - -func (me ClientPieceOrder) sort() { - sort.Slice(me.pieces, me.less) -} - -func (me ClientPieceOrder) less(_i, _j int) bool { - i := me.pieces[_i] - j := me.pieces[_j] - return multiless.New().Int( - int(j.Priority), int(i.Priority), - ).Bool( - j.Partial, i.Partial, - ).Int64( - i.Availability, j.Availability, - ).Int( - i.index, j.index, - ).Uintptr( - i.t.StableId, j.t.StableId, - ).MustLess() +func sortFilterPieces(pieces []filterPiece) { + sort.Slice(pieces, func(_i, _j int) bool { + i := pieces[_i] + j := pieces[_j] + return multiless.New().Int( + int(j.Priority), int(i.Priority), + ).Bool( + j.Partial, i.Partial, + ).Int64( + i.Availability, j.Availability, + ).Int( + i.index, j.index, + ).Uintptr( + i.t.StableId, j.t.StableId, + ).MustLess() + }) } type requestsPeer struct { @@ -86,15 +71,27 @@ func (me *peersForPieceRequests) addNextRequest(r Request) { me.requestsInPiece++ } -func (requestOrder *ClientPieceOrder) DoRequests(torrents []*Torrent) map[PeerId]PeerNextRequestState { - requestOrder.pieces = requestOrder.pieces[:0] +type requestablePiece struct { + index pieceIndex + t Torrent + NumPendingChunks int + IterPendingChunks ChunksIter +} + +type filterPiece struct { + t *filterTorrent + index pieceIndex + Piece +} + +func getRequestablePieces(torrents []Torrent) (ret []requestablePiece) { // Storage capacity left for this run, keyed by the storage capacity pointer on the storage // TorrentImpl. storageLeft := make(map[*func() *int64]*int64) - orderTorrents := make([]*orderTorrent, 0, len(torrents)) + var pieces []filterPiece for _, _t := range torrents { // TODO: We could do metainfo requests here. - t := &orderTorrent{ + t := &filterTorrent{ Torrent: _t, unverifiedBytes: 0, } @@ -105,34 +102,16 @@ func (requestOrder *ClientPieceOrder) DoRequests(torrents []*Torrent) map[PeerId } t.storageLeft = storageLeft[key] } - var peers []*requestsPeer - for _, p := range t.Peers { - peers = append(peers, &requestsPeer{ - Peer: p, - nextState: PeerNextRequestState{ - Requests: make(map[Request]struct{}), - }, - }) - } for i, tp := range t.Pieces { - requestOrder.pieces = append(requestOrder.pieces, pieceRequestOrderPiece{ + pieces = append(pieces, filterPiece{ t: t, index: i, Piece: tp, }) - if tp.Request && tp.NumPendingChunks != 0 { - for _, p := range peers { - if p.canRequestPiece(i) { - p.requestablePiecesRemaining++ - } - } - } } - t.peers = peers - orderTorrents = append(orderTorrents, t) } - requestOrder.sort() - for _, piece := range requestOrder.pieces { + sortFilterPieces(pieces) + for _, piece := range pieces { if left := piece.t.storageLeft; left != nil { if *left < int64(piece.Length) { continue @@ -143,16 +122,48 @@ func (requestOrder *ClientPieceOrder) DoRequests(torrents []*Torrent) map[PeerId continue } if piece.t.MaxUnverifiedBytes != 0 && piece.t.unverifiedBytes+piece.Length > piece.t.MaxUnverifiedBytes { - //log.Print("skipping piece") continue } - allocatePendingChunks(piece, piece.t.peers) piece.t.unverifiedBytes += piece.Length - //log.Print(piece.t.unverifiedBytes) + ret = append(ret, requestablePiece{ + index: piece.index, + t: piece.t.Torrent, + NumPendingChunks: piece.NumPendingChunks, + IterPendingChunks: piece.iterPendingChunksWrapper, + }) + } + return +} + +// TODO: We could do metainfo requests here. +func (requestOrder *ClientPieceOrder) DoRequests(torrents []Torrent) map[PeerId]PeerNextRequestState { + requestPieces := getRequestablePieces(torrents) + allPeers := make(map[uintptr][]*requestsPeer, len(torrents)) + for _, t := range torrents { + peers := make([]*requestsPeer, 0, len(t.Peers)) + for _, p := range t.Peers { + peers = append(peers, &requestsPeer{ + Peer: p, + nextState: PeerNextRequestState{ + Requests: make(map[Request]struct{}), + }, + }) + } + allPeers[t.StableId] = peers + } + for _, piece := range requestPieces { + for _, peer := range allPeers[piece.t.StableId] { + if peer.canRequestPiece(piece.index) { + peer.requestablePiecesRemaining++ + } + } + } + for _, piece := range requestPieces { + allocatePendingChunks(piece, allPeers[piece.t.StableId]) } ret := make(map[PeerId]PeerNextRequestState) - for _, ots := range orderTorrents { - for _, rp := range ots.peers { + for _, peers := range allPeers { + for _, rp := range peers { if rp.requestablePiecesRemaining != 0 { panic(rp.requestablePiecesRemaining) } @@ -162,7 +173,7 @@ func (requestOrder *ClientPieceOrder) DoRequests(torrents []*Torrent) map[PeerId return ret } -func allocatePendingChunks(p pieceRequestOrderPiece, peers []*requestsPeer) { +func allocatePendingChunks(p requestablePiece, peers []*requestsPeer) { peersForPiece := make([]*peersForPieceRequests, 0, len(peers)) for _, peer := range peers { peersForPiece = append(peersForPiece, &peersForPieceRequests{ @@ -204,7 +215,7 @@ func allocatePendingChunks(p pieceRequestOrderPiece, peers []*requestsPeer) { }) } preallocated := make(map[ChunkSpec]*peersForPieceRequests, p.NumPendingChunks) - p.iterPendingChunksWrapper(func(spec ChunkSpec) { + p.IterPendingChunks(func(spec ChunkSpec) { req := Request{pp.Integer(p.index), spec} for _, peer := range peersForPiece { if h := peer.HasExistingRequest; h == nil || !h(req) { @@ -221,7 +232,7 @@ func allocatePendingChunks(p pieceRequestOrderPiece, peers []*requestsPeer) { } }) pendingChunksRemaining := int(p.NumPendingChunks) - p.iterPendingChunksWrapper(func(chunk types.ChunkSpec) { + p.IterPendingChunks(func(chunk types.ChunkSpec) { if _, ok := preallocated[chunk]; ok { return } @@ -243,11 +254,12 @@ func allocatePendingChunks(p pieceRequestOrderPiece, peers []*requestsPeer) { } } peer.addNextRequest(req) - return + break } }) chunk: for chunk, prePeer := range preallocated { + pendingChunksRemaining-- req := Request{pp.Integer(p.index), chunk} prePeer.requestsInPiece-- sortPeersForPiece(&req) @@ -266,7 +278,6 @@ chunk: continue } } - pendingChunksRemaining-- peer.addNextRequest(req) continue chunk } diff --git a/request-strategy/order_test.go b/request-strategy/order_test.go index 8c518b50..fd8b53f0 100644 --- a/request-strategy/order_test.go +++ b/request-strategy/order_test.go @@ -64,7 +64,7 @@ func TestStealingFromSlowerPeer(t *testing.T) { firstStealer.Id = intPeerId(2) secondStealer := basePeer secondStealer.Id = intPeerId(3) - results := order.DoRequests([]*Torrent{{ + results := order.DoRequests([]Torrent{{ Pieces: []Piece{{ Request: true, NumPendingChunks: 5, @@ -111,7 +111,7 @@ func TestStealingFromSlowerPeersBasic(t *testing.T) { firstStealer.Id = intPeerId(2) secondStealer := basePeer secondStealer.Id = intPeerId(3) - results := order.DoRequests([]*Torrent{{ + results := order.DoRequests([]Torrent{{ Pieces: []Piece{{ Request: true, NumPendingChunks: 2, @@ -150,7 +150,7 @@ func TestPeerKeepsExistingIfReasonable(t *testing.T) { firstStealer.Id = intPeerId(2) secondStealer := basePeer secondStealer.Id = intPeerId(3) - results := order.DoRequests([]*Torrent{{ + results := order.DoRequests([]Torrent{{ Pieces: []Piece{{ Request: true, NumPendingChunks: 4, @@ -198,7 +198,7 @@ func TestDontStealUnnecessarily(t *testing.T) { firstStealer.Id = intPeerId(2) secondStealer := basePeer secondStealer.Id = intPeerId(3) - results := order.DoRequests([]*Torrent{{ + results := order.DoRequests([]Torrent{{ Pieces: []Piece{{ Request: true, NumPendingChunks: 9, diff --git a/request-strategy/piece.go b/request-strategy/piece.go index 508ed829..bc59c052 100644 --- a/request-strategy/piece.go +++ b/request-strategy/piece.go @@ -4,6 +4,8 @@ import ( "github.com/anacrolix/torrent/types" ) +type ChunksIter func(func(types.ChunkSpec)) + type Piece struct { Request bool Priority piecePriority @@ -11,10 +13,10 @@ type Piece struct { Availability int64 Length int64 NumPendingChunks int - IterPendingChunks func(func(types.ChunkSpec)) + IterPendingChunks ChunksIter } -func (p *Piece) iterPendingChunksWrapper(f func(ChunkSpec)) { +func (p Piece) iterPendingChunksWrapper(f func(ChunkSpec)) { i := p.IterPendingChunks if i != nil { i(f) From ad298364aa93175dfa86e08bc5614ddf6ed530a4 Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Fri, 14 May 2021 13:40:09 +1000 Subject: [PATCH 34/59] Add client-level max unverified bytes --- cmd/torrent/main.go | 28 +++++++++++++++------------- config.go | 2 ++ request-strategy.go | 8 +++++--- request-strategy/order.go | 21 +++++++++++++++++---- request-strategy/order_test.go | 24 ++++++++++++------------ 5 files changed, 51 insertions(+), 32 deletions(-) diff --git a/cmd/torrent/main.go b/cmd/torrent/main.go index 02020e77..374f8b1a 100644 --- a/cmd/torrent/main.go +++ b/cmd/torrent/main.go @@ -169,19 +169,20 @@ var flags struct { type SpewBencodingCmd struct{} type DownloadCmd struct { - Mmap bool `help:"memory-map torrent data"` - TestPeer []string `help:"addresses of some starting peers"` - Seed bool `help:"seed after download is complete"` - Addr string `help:"network listen addr"` - UploadRate *tagflag.Bytes `help:"max piece bytes to send per second"` - DownloadRate *tagflag.Bytes `help:"max bytes per second down from peers"` - PackedBlocklist string - PublicIP net.IP - Progress bool `default:"true"` - PieceStates bool - Quiet bool `help:"discard client logging"` - Stats *bool `help:"print stats at termination"` - Dht bool `default:"true"` + Mmap bool `help:"memory-map torrent data"` + TestPeer []string `help:"addresses of some starting peers"` + Seed bool `help:"seed after download is complete"` + Addr string `help:"network listen addr"` + MaxUnverifiedBytes tagflag.Bytes `help:"maximum number bytes to have pending verification"` + UploadRate *tagflag.Bytes `help:"max piece bytes to send per second"` + DownloadRate *tagflag.Bytes `help:"max bytes per second down from peers"` + PackedBlocklist string + PublicIP net.IP + Progress bool `default:"true"` + PieceStates bool + Quiet bool `help:"discard client logging"` + Stats *bool `help:"print stats at termination"` + Dht bool `default:"true"` TcpPeers bool `default:"true"` UtpPeers bool `default:"true"` @@ -311,6 +312,7 @@ func downloadErr() error { if flags.Quiet { clientConfig.Logger = log.Discard } + clientConfig.MaxUnverifiedBytes = flags.MaxUnverifiedBytes.Int64() var stop missinggo.SynchronizedEvent defer func() { diff --git a/config.go b/config.go index 5aeef38b..37b2b714 100644 --- a/config.go +++ b/config.go @@ -59,6 +59,8 @@ type ClientConfig struct { // (~4096), and the requested chunk size (~16KiB, see // TorrentSpec.ChunkSize). DownloadRateLimiter *rate.Limiter + // Maximum unverified bytes across all torrents. Not used if zero. + MaxUnverifiedBytes int64 // User-provided Client peer ID. If not present, one is generated automatically. PeerID string diff --git a/request-strategy.go b/request-strategy.go index 8d8b3b23..7c7660bc 100644 --- a/request-strategy.go +++ b/request-strategy.go @@ -27,8 +27,7 @@ func (cl *Client) doRequests() { ts := make([]request_strategy.Torrent, 0, len(cl.torrents)) for _, t := range cl.torrents { rst := request_strategy.Torrent{ - StableId: uintptr(unsafe.Pointer(t)), - MaxUnverifiedBytes: 10 << 20, + StableId: uintptr(unsafe.Pointer(t)), } if t.storage != nil { rst.Capacity = t.storage.Capacity @@ -72,7 +71,10 @@ func (cl *Client) doRequests() { }) ts = append(ts, rst) } - nextPeerStates := cl.pieceRequestOrder.DoRequests(ts) + nextPeerStates := request_strategy.Run(request_strategy.Input{ + Torrents: ts, + MaxUnverifiedBytes: cl.config.MaxUnverifiedBytes, + }) for p, state := range nextPeerStates { applyPeerNextRequestState(p, state) } diff --git a/request-strategy/order.go b/request-strategy/order.go index 9052fb0c..54bb5285 100644 --- a/request-strategy/order.go +++ b/request-strategy/order.go @@ -84,12 +84,12 @@ type filterPiece struct { Piece } -func getRequestablePieces(torrents []Torrent) (ret []requestablePiece) { +func getRequestablePieces(input Input) (ret []requestablePiece) { // Storage capacity left for this run, keyed by the storage capacity pointer on the storage // TorrentImpl. storageLeft := make(map[*func() *int64]*int64) var pieces []filterPiece - for _, _t := range torrents { + for _, _t := range input.Torrents { // TODO: We could do metainfo requests here. t := &filterTorrent{ Torrent: _t, @@ -111,6 +111,7 @@ func getRequestablePieces(torrents []Torrent) (ret []requestablePiece) { } } sortFilterPieces(pieces) + var allTorrentsUnverifiedBytes int64 for _, piece := range pieces { if left := piece.t.storageLeft; left != nil { if *left < int64(piece.Length) { @@ -119,12 +120,18 @@ func getRequestablePieces(torrents []Torrent) (ret []requestablePiece) { *left -= int64(piece.Length) } if !piece.Request || piece.NumPendingChunks == 0 { + // TODO: Clarify exactly what is verified. Stuff that's being hashed should be + // considered unverified and hold up further requests. continue } if piece.t.MaxUnverifiedBytes != 0 && piece.t.unverifiedBytes+piece.Length > piece.t.MaxUnverifiedBytes { continue } + if input.MaxUnverifiedBytes != 0 && allTorrentsUnverifiedBytes+piece.Length > input.MaxUnverifiedBytes { + continue + } piece.t.unverifiedBytes += piece.Length + allTorrentsUnverifiedBytes += piece.Length ret = append(ret, requestablePiece{ index: piece.index, t: piece.t.Torrent, @@ -135,9 +142,15 @@ func getRequestablePieces(torrents []Torrent) (ret []requestablePiece) { return } +type Input struct { + Torrents []Torrent + MaxUnverifiedBytes int64 +} + // TODO: We could do metainfo requests here. -func (requestOrder *ClientPieceOrder) DoRequests(torrents []Torrent) map[PeerId]PeerNextRequestState { - requestPieces := getRequestablePieces(torrents) +func Run(input Input) map[PeerId]PeerNextRequestState { + requestPieces := getRequestablePieces(input) + torrents := input.Torrents allPeers := make(map[uintptr][]*requestsPeer, len(torrents)) for _, t := range torrents { peers := make([]*requestsPeer, 0, len(t.Peers)) diff --git a/request-strategy/order_test.go b/request-strategy/order_test.go index fd8b53f0..adc0c478 100644 --- a/request-strategy/order_test.go +++ b/request-strategy/order_test.go @@ -45,7 +45,6 @@ func (i intPeerId) Uintptr() uintptr { func TestStealingFromSlowerPeer(t *testing.T) { c := qt.New(t) - order := ClientPieceOrder{} basePeer := Peer{ HasPiece: func(i pieceIndex) bool { return true @@ -64,7 +63,7 @@ func TestStealingFromSlowerPeer(t *testing.T) { firstStealer.Id = intPeerId(2) secondStealer := basePeer secondStealer.Id = intPeerId(3) - results := order.DoRequests([]Torrent{{ + results := Run(Input{Torrents: []Torrent{{ Pieces: []Piece{{ Request: true, NumPendingChunks: 5, @@ -75,7 +74,8 @@ func TestStealingFromSlowerPeer(t *testing.T) { firstStealer, secondStealer, }, - }}) + }}}) + c.Assert(results, qt.HasLen, 3) check := func(p PeerId, l int) { c.Check(results[p].Requests, qt.HasLen, l) @@ -93,7 +93,6 @@ func checkNumRequestsAndInterest(c *qt.C, next PeerNextRequestState, num int, in func TestStealingFromSlowerPeersBasic(t *testing.T) { c := qt.New(t) - order := ClientPieceOrder{} basePeer := Peer{ HasPiece: func(i pieceIndex) bool { return true @@ -111,7 +110,7 @@ func TestStealingFromSlowerPeersBasic(t *testing.T) { firstStealer.Id = intPeerId(2) secondStealer := basePeer secondStealer.Id = intPeerId(3) - results := order.DoRequests([]Torrent{{ + results := Run(Input{Torrents: []Torrent{{ Pieces: []Piece{{ Request: true, NumPendingChunks: 2, @@ -122,7 +121,8 @@ func TestStealingFromSlowerPeersBasic(t *testing.T) { firstStealer, secondStealer, }, - }}) + }}}) + checkNumRequestsAndInterest(c, results[firstStealer.Id], 1, true) checkNumRequestsAndInterest(c, results[secondStealer.Id], 1, true) checkNumRequestsAndInterest(c, results[stealee.Id], 0, false) @@ -130,7 +130,6 @@ func TestStealingFromSlowerPeersBasic(t *testing.T) { func TestPeerKeepsExistingIfReasonable(t *testing.T) { c := qt.New(t) - order := ClientPieceOrder{} basePeer := Peer{ HasPiece: func(i pieceIndex) bool { return true @@ -150,7 +149,7 @@ func TestPeerKeepsExistingIfReasonable(t *testing.T) { firstStealer.Id = intPeerId(2) secondStealer := basePeer secondStealer.Id = intPeerId(3) - results := order.DoRequests([]Torrent{{ + results := Run(Input{Torrents: []Torrent{{ Pieces: []Piece{{ Request: true, NumPendingChunks: 4, @@ -161,7 +160,8 @@ func TestPeerKeepsExistingIfReasonable(t *testing.T) { firstStealer, secondStealer, }, - }}) + }}}) + c.Assert(results, qt.HasLen, 3) check := func(p PeerId, l int) { c.Check(results[p].Requests, qt.HasLen, l) @@ -177,7 +177,6 @@ func TestPeerKeepsExistingIfReasonable(t *testing.T) { func TestDontStealUnnecessarily(t *testing.T) { c := qt.New(t) - order := ClientPieceOrder{} basePeer := Peer{ HasPiece: func(i pieceIndex) bool { return true @@ -198,7 +197,7 @@ func TestDontStealUnnecessarily(t *testing.T) { firstStealer.Id = intPeerId(2) secondStealer := basePeer secondStealer.Id = intPeerId(3) - results := order.DoRequests([]Torrent{{ + results := Run(Input{Torrents: []Torrent{{ Pieces: []Piece{{ Request: true, NumPendingChunks: 9, @@ -209,7 +208,8 @@ func TestDontStealUnnecessarily(t *testing.T) { stealee, secondStealer, }, - }}) + }}}) + c.Assert(results, qt.HasLen, 3) check := func(p PeerId, l int) { c.Check(results[p].Requests, qt.HasLen, l) From fcc4ef2b958bac16ece1dcbc795e300eed6f978d Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Fri, 14 May 2021 15:19:49 +1000 Subject: [PATCH 35/59] Fix sqlite piece completion --- torrent.go | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/torrent.go b/torrent.go index 05598b3c..70e4ecd8 100644 --- a/torrent.go +++ b/torrent.go @@ -166,8 +166,11 @@ func (t *Torrent) decPieceAvailability(i pieceIndex) { } func (t *Torrent) incPieceAvailability(i pieceIndex) { - p := t.piece(i) - p.availability++ + // If we don't the info, this should be reconciled when we do. + if t.haveInfo() { + p := t.piece(i) + p.availability++ + } } func (t *Torrent) numConns() int { From 42bf6b58a6caa7225fcedc2c4799d5f945b344c2 Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Mon, 17 May 2021 09:08:39 +1000 Subject: [PATCH 36/59] Fix in decPieceAvailability when we don't have torrent info Happens when we close a Peer before getting Torrent info. --- torrent.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/torrent.go b/torrent.go index 70e4ecd8..da71ba93 100644 --- a/torrent.go +++ b/torrent.go @@ -158,6 +158,9 @@ func (t *Torrent) pieceAvailabilityFromPeers(i pieceIndex) (count int) { } func (t *Torrent) decPieceAvailability(i pieceIndex) { + if !t.haveInfo() { + return + } p := t.piece(i) if p.availability <= 0 { panic(p.availability) From 86524214f56cbc3c446cb5e0d585844c3dbf3903 Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Thu, 20 May 2021 08:34:20 +1000 Subject: [PATCH 37/59] Merge maxRequests into localClientReqq --- client.go | 5 +++-- global.go | 1 - peerconn.go | 5 +++-- 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/client.go b/client.go index 8544d4e3..9300b24b 100644 --- a/client.go +++ b/client.go @@ -969,8 +969,9 @@ func (cl *Client) runHandshookConn(c *PeerConn, t *Torrent) error { return nil } -// If peer requests are buffered on read, this instructs the amount of memory that might be used to -// cache pending writes. Assuming 512KiB cached for sending, for 16KiB chunks. +// Maximum pending requests we allow peers to send us. If peer requests are buffered on read, this +// instructs the amount of memory that might be used to cache pending writes. Assuming 512KiB +// (1<<19) cached for sending, for 16KiB (1<<14) chunks. const localClientReqq = 1 << 5 // See the order given in Transmission's tr_peerMsgsNew. diff --git a/global.go b/global.go index 1a09b06e..e06d9323 100644 --- a/global.go +++ b/global.go @@ -9,7 +9,6 @@ import ( const ( pieceHash = crypto.SHA1 - maxRequests = 250 // Maximum pending requests we allow peers to send us. defaultChunkSize = 0x4000 // 16KiB ) diff --git a/peerconn.go b/peerconn.go index a5d078e0..71ac5f3d 100644 --- a/peerconn.go +++ b/peerconn.go @@ -989,7 +989,8 @@ func (c *PeerConn) onReadRequest(r Request) error { } return nil } - if len(c.peerRequests) >= maxRequests { + // TODO: What if they've already requested this? + if len(c.peerRequests) >= localClientReqq { torrent.Add("requests received while queue full", 1) if c.fastEnabled() { c.reject(r) @@ -1010,7 +1011,7 @@ func (c *PeerConn) onReadRequest(r Request) error { return errors.New("bad Request") } if c.peerRequests == nil { - c.peerRequests = make(map[Request]*peerRequestState, maxRequests) + c.peerRequests = make(map[Request]*peerRequestState, localClientReqq) } value := &peerRequestState{} c.peerRequests[r] = value From a26e7f9933ca2ff86e7a9e19f75207fad86be610 Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Thu, 20 May 2021 08:56:53 +1000 Subject: [PATCH 38/59] Determine peer max requests based on receive speed --- peerconn.go | 18 ++++++++++++------ request-strategy.go | 4 ++++ 2 files changed, 16 insertions(+), 6 deletions(-) diff --git a/peerconn.go b/peerconn.go index 71ac5f3d..f3680e18 100644 --- a/peerconn.go +++ b/peerconn.go @@ -47,6 +47,10 @@ type PeerRemoteAddr interface { String() string } +// Since we have to store all the requests in memory, we can't reasonably exceed what would be +// indexable with the memory space available. +type maxRequests = int + type Peer struct { // First to ensure 64-bit alignment for atomics. See #262. _stats ConnStats @@ -83,9 +87,10 @@ type Peer struct { cumulativeExpectedToReceiveChunks time.Duration _chunksReceivedWhileExpecting int64 - choking bool - requests map[Request]struct{} - requestsLowWater int + choking bool + requests map[Request]struct{} + piecesReceivedSinceLastRequestUpdate maxRequests + maxPiecesReceivedBetweenRequestUpdates maxRequests // Chunks that we might reasonably expect to receive from the peer. Due to // latency, buffering, and implementation differences, we may receive // chunks that are no longer in the set of requests actually want. @@ -114,7 +119,7 @@ type Peer struct { peerTouchedPieces map[pieceIndex]struct{} peerAllowedFast bitmap.Bitmap - PeerMaxRequests int // Maximum pending requests the peer allows. + PeerMaxRequests maxRequests // Maximum pending requests the peer allows. PeerExtensionIDs map[pp.ExtensionName]pp.ExtensionNumber PeerClientName string @@ -470,8 +475,8 @@ func (cn *PeerConn) requestedMetadataPiece(index int) bool { } // The actual value to use as the maximum outbound requests. -func (cn *Peer) nominalMaxRequests() (ret int) { - return int(clamp(1, int64(cn.PeerMaxRequests), 64)) +func (cn *Peer) nominalMaxRequests() (ret maxRequests) { + return int(clamp(1, 2*int64(cn.maxPiecesReceivedBetweenRequestUpdates), int64(cn.PeerMaxRequests))) } func (cn *Peer) totalExpectingTime() (ret time.Duration) { @@ -1358,6 +1363,7 @@ func (c *Peer) receiveChunk(msg *pp.Message) error { c.allStats(add(1, func(cs *ConnStats) *Count { return &cs.ChunksReadUseful })) c.allStats(add(int64(len(msg.Piece)), func(cs *ConnStats) *Count { return &cs.BytesReadUsefulData })) if deletedRequest { + c.piecesReceivedSinceLastRequestUpdate++ c.allStats(add(int64(len(msg.Piece)), func(cs *ConnStats) *Count { return &cs.BytesReadUsefulIntendedData })) } for _, f := range c.t.cl.config.Callbacks.ReceivedUsefulData { diff --git a/request-strategy.go b/request-strategy.go index 7c7660bc..f0470007 100644 --- a/request-strategy.go +++ b/request-strategy.go @@ -53,6 +53,10 @@ func (cl *Client) doRequests() { if p.closed.IsSet() { return } + if p.piecesReceivedSinceLastRequestUpdate > p.maxPiecesReceivedBetweenRequestUpdates { + p.maxPiecesReceivedBetweenRequestUpdates = p.piecesReceivedSinceLastRequestUpdate + } + p.piecesReceivedSinceLastRequestUpdate = 0 rst.Peers = append(rst.Peers, request_strategy.Peer{ HasPiece: p.peerHasPiece, MaxRequests: p.nominalMaxRequests(), From 95d808d3c5f318b8c09c233b021fb8a8f542c415 Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Thu, 20 May 2021 11:16:54 +1000 Subject: [PATCH 39/59] Use rejiggered missinggo bitmap --- client.go | 4 ++-- file.go | 12 ++++++------ peerconn.go | 29 +++++++++++++++-------------- piece.go | 10 +++++----- request-strategy.go | 3 ++- test/transfer_test.go | 3 ++- torrent.go | 22 +++++++++++----------- torrent_test.go | 5 +++-- 8 files changed, 46 insertions(+), 42 deletions(-) diff --git a/client.go b/client.go index 9300b24b..98bf0892 100644 --- a/client.go +++ b/client.go @@ -18,10 +18,10 @@ import ( "github.com/anacrolix/dht/v2" "github.com/anacrolix/dht/v2/krpc" "github.com/anacrolix/log" - "github.com/anacrolix/missinggo/bitmap" "github.com/anacrolix/missinggo/perf" "github.com/anacrolix/missinggo/pubsub" "github.com/anacrolix/missinggo/slices" + "github.com/anacrolix/missinggo/v2/bitmap" "github.com/anacrolix/missinggo/v2/pproffd" "github.com/anacrolix/sync" "github.com/anacrolix/torrent/internal/limiter" @@ -1007,7 +1007,7 @@ func (cl *Client) sendInitialMessages(conn *PeerConn, torrent *Torrent) { if conn.fastEnabled() { if torrent.haveAllPieces() { conn.write(pp.Message{Type: pp.HaveAll}) - conn.sentHaves.AddRange(0, bitmap.BitIndex(conn.t.NumPieces())) + conn.sentHaves.AddRange(0, bitmap.BitRange(conn.t.NumPieces())) return } else if !torrent.haveAnyPieces() { conn.write(pp.Message{Type: pp.HaveNone}) diff --git a/file.go b/file.go index e2ebcf4d..37c185c1 100644 --- a/file.go +++ b/file.go @@ -66,20 +66,20 @@ func fileBytesLeft( switch numPiecesSpanned { case 0: case 1: - if !torrentCompletedPieces.Get(fileFirstPieceIndex) { + if !torrentCompletedPieces.Get(bitmap.BitIndex(fileFirstPieceIndex)) { left += fileLength } default: - if !torrentCompletedPieces.Get(fileFirstPieceIndex) { + if !torrentCompletedPieces.Get(bitmap.BitIndex(fileFirstPieceIndex)) { left += torrentUsualPieceSize - (fileTorrentOffset % torrentUsualPieceSize) } - if !torrentCompletedPieces.Get(fileEndPieceIndex - 1) { + if !torrentCompletedPieces.Get(bitmap.BitIndex(fileEndPieceIndex - 1)) { left += fileTorrentOffset + fileLength - int64(fileEndPieceIndex-1)*torrentUsualPieceSize } completedMiddlePieces := torrentCompletedPieces.Copy() - completedMiddlePieces.RemoveRange(0, fileFirstPieceIndex+1) - completedMiddlePieces.RemoveRange(fileEndPieceIndex-1, bitmap.ToEnd) - left += int64(numPiecesSpanned-2-completedMiddlePieces.Len()) * torrentUsualPieceSize + completedMiddlePieces.RemoveRange(0, bitmap.BitRange(fileFirstPieceIndex+1)) + completedMiddlePieces.RemoveRange(bitmap.BitRange(fileEndPieceIndex-1), bitmap.ToEnd) + left += int64(numPiecesSpanned-2-pieceIndex(completedMiddlePieces.Len())) * torrentUsualPieceSize } return } diff --git a/peerconn.go b/peerconn.go index f3680e18..976081ab 100644 --- a/peerconn.go +++ b/peerconn.go @@ -232,7 +232,7 @@ func (cn *Peer) peerHasAllPieces() (all bool, known bool) { if !cn.t.haveInfo() { return false, false } - return bitmap.Flip(cn._peerPieces, 0, bitmap.BitIndex(cn.t.numPieces())).IsEmpty(), true + return bitmap.Flip(cn._peerPieces, 0, bitmap.BitRange(cn.t.numPieces())).IsEmpty(), true } func (cn *PeerConn) locker() *lockWithDeferreds { @@ -267,7 +267,7 @@ func (cn *PeerConn) onGotInfo(info *metainfo.Info) { // Correct the PeerPieces slice length. Return false if the existing slice is invalid, such as by // receiving badly sized BITFIELD, or invalid HAVE messages. func (cn *PeerConn) setNumPieces(num pieceIndex) { - cn._peerPieces.RemoveRange(bitmap.BitIndex(num), bitmap.ToEnd) + cn._peerPieces.RemoveRange(bitmap.BitRange(num), bitmap.ToEnd) cn.peerPiecesChanged() } @@ -730,10 +730,10 @@ func iterBitmapsDistinct(skip *bitmap.Bitmap, bms ...bitmap.Bitmap) iter.Func { if !iter.All( func(_i interface{}) bool { i := _i.(int) - if skip.Contains(i) { + if skip.Contains(bitmap.BitIndex(i)) { return true } - skip.Add(i) + skip.Add(bitmap.BitIndex(i)) return cb(i) }, bm.Iter, @@ -746,7 +746,7 @@ func iterBitmapsDistinct(skip *bitmap.Bitmap, bms ...bitmap.Bitmap) iter.Func { // check callers updaterequests func (cn *Peer) stopRequestingPiece(piece pieceIndex) bool { - return cn._pieceRequestOrder.Remove(bitmap.BitIndex(piece)) + return cn._pieceRequestOrder.Remove(piece) } // This is distinct from Torrent piece priority, which is the user's @@ -762,7 +762,7 @@ func (cn *Peer) updatePiecePriority(piece pieceIndex) bool { return cn.stopRequestingPiece(piece) } prio := cn.getPieceInclination()[piece] - return cn._pieceRequestOrder.Set(bitmap.BitIndex(piece), prio) + return cn._pieceRequestOrder.Set(piece, prio) } func (cn *Peer) getPieceInclination() []int { @@ -835,15 +835,15 @@ func (cn *PeerConn) peerSentBitfield(bf []bool) error { for i, have := range bf { if have { cn.raisePeerMinPieces(pieceIndex(i) + 1) - if !pp.Contains(i) { + if !pp.Contains(bitmap.BitIndex(i)) { cn.t.incPieceAvailability(i) } } else { - if pp.Contains(i) { + if pp.Contains(bitmap.BitIndex(i)) { cn.t.decPieceAvailability(i) } } - cn._peerPieces.Set(i, have) + cn._peerPieces.Set(bitmap.BitIndex(i), have) } cn.peerPiecesChanged() return nil @@ -854,7 +854,7 @@ func (cn *Peer) onPeerHasAllPieces() { if t.haveInfo() { pp := cn.newPeerPieces() for i := range iter.N(t.numPieces()) { - if !pp.Contains(i) { + if !pp.Contains(bitmap.BitIndex(i)) { t.incPieceAvailability(i) } } @@ -1198,7 +1198,7 @@ func (c *PeerConn) mainReadLoop() (err error) { case pp.AllowedFast: torrent.Add("allowed fasts received", 1) log.Fmsg("peer allowed fast: %d", msg.Index).AddValues(c).SetLevel(log.Debug).Log(c.t.logger) - c.peerAllowedFast.Add(int(msg.Index)) + c.peerAllowedFast.Add(bitmap.BitIndex(msg.Index)) c.updateRequests() case pp.Extended: err = c.onReadExtendedMsg(msg.ExtendedID, msg.ExtendedPayload) @@ -1326,7 +1326,7 @@ func (c *Peer) receiveChunk(msg *pp.Message) error { } c.decExpectedChunkReceive(req) - if c.peerChoking && c.peerAllowedFast.Get(int(req.Index)) { + if c.peerChoking && c.peerAllowedFast.Get(bitmap.BitIndex(req.Index)) { chunksReceived.Add("due to allowed fast", 1) } @@ -1658,11 +1658,12 @@ func (cn *PeerConn) PeerPieces() bitmap.Bitmap { return cn.newPeerPieces() } -// Returns a new Bitmap that includes bits for all pieces we have. +// Returns a new Bitmap that includes bits for all pieces the peer claims to have. func (cn *Peer) newPeerPieces() bitmap.Bitmap { ret := cn._peerPieces.Copy() if cn.peerSentHaveAll { - ret.AddRange(0, cn.t.numPieces()) + + ret.AddRange(0, bitmap.BitRange(cn.t.numPieces())) } return ret } diff --git a/piece.go b/piece.go index 3ee648d3..45b37c0a 100644 --- a/piece.go +++ b/piece.go @@ -53,7 +53,7 @@ func (p *Piece) Storage() storage.Piece { } func (p *Piece) pendingChunkIndex(chunkIndex int) bool { - return !p._dirtyChunks.Contains(chunkIndex) + return !p._dirtyChunks.Contains(bitmap.BitIndex(chunkIndex)) } func (p *Piece) pendingChunk(cs ChunkSpec, chunkSize pp.Integer) bool { @@ -69,12 +69,12 @@ func (p *Piece) numDirtyChunks() pp.Integer { } func (p *Piece) unpendChunkIndex(i int) { - p._dirtyChunks.Add(i) + p._dirtyChunks.Add(bitmap.BitIndex(i)) p.t.tickleReaders() } func (p *Piece) pendChunkIndex(i int) { - p._dirtyChunks.Remove(i) + p._dirtyChunks.Remove(bitmap.BitIndex(i)) } func (p *Piece) numChunks() pp.Integer { @@ -199,7 +199,7 @@ func (p *Piece) purePriority() (ret piecePriority) { for _, f := range p.files { ret.Raise(f.prio) } - if p.t.readerNowPieces().Contains(int(p.index)) { + if p.t.readerNowPieces().Contains(bitmap.BitIndex(p.index)) { ret.Raise(PiecePriorityNow) } // if t._readerNowPieces.Contains(piece - 1) { @@ -234,7 +234,7 @@ func (p *Piece) completion() (ret storage.Completion) { } func (p *Piece) allChunksDirty() bool { - return p._dirtyChunks.Len() == int(p.numChunks()) + return p._dirtyChunks.Len() == bitmap.BitRange(p.numChunks()) } func (p *Piece) dirtyChunks() bitmap.Bitmap { diff --git a/request-strategy.go b/request-strategy.go index f0470007..72f7f3d4 100644 --- a/request-strategy.go +++ b/request-strategy.go @@ -4,6 +4,7 @@ import ( "time" "unsafe" + "github.com/anacrolix/missinggo/v2/bitmap" request_strategy "github.com/anacrolix/torrent/request-strategy" "github.com/anacrolix/torrent/types" ) @@ -66,7 +67,7 @@ func (cl *Client) doRequests() { }, Choking: p.peerChoking, PieceAllowedFast: func(i pieceIndex) bool { - return p.peerAllowedFast.Contains(i) + return p.peerAllowedFast.Contains(bitmap.BitIndex(i)) }, DownloadRate: p.downloadRate(), Age: time.Since(p.completedHandshake), diff --git a/test/transfer_test.go b/test/transfer_test.go index 857cd44f..b3187c0b 100644 --- a/test/transfer_test.go +++ b/test/transfer_test.go @@ -12,6 +12,7 @@ import ( "testing/iotest" "time" + "github.com/anacrolix/missinggo/v2/bitmap" "github.com/anacrolix/missinggo/v2/filecache" "github.com/anacrolix/torrent" "github.com/anacrolix/torrent/internal/testutil" @@ -168,7 +169,7 @@ func testClientTransfer(t *testing.T, ps testClientTransferParams) { for _, pc := range leecherPeerConns { completed := pc.PeerPieces().Len() t.Logf("peer conn %v has %v completed pieces", pc, completed) - if completed == leecherTorrent.Info().NumPieces() { + if completed == bitmap.BitRange(leecherTorrent.Info().NumPieces()) { foundSeeder = true } } diff --git a/torrent.go b/torrent.go index da71ba93..5190b65e 100644 --- a/torrent.go +++ b/torrent.go @@ -759,7 +759,7 @@ func (t *Torrent) bytesMissingLocked() int64 { } func (t *Torrent) bytesLeft() (left int64) { - bitmap.Flip(t._completedPieces, 0, bitmap.BitIndex(t.numPieces())).IterTyped(func(piece int) bool { + bitmap.Flip(t._completedPieces, 0, bitmap.BitRange(t.numPieces())).IterTyped(func(piece int) bool { p := &t.pieces[piece] left += int64(p.length() - p.numDirtyBytes()) return true @@ -794,8 +794,8 @@ func (t *Torrent) numPieces() pieceIndex { return pieceIndex(t.info.NumPieces()) } -func (t *Torrent) numPiecesCompleted() (num int) { - return t._completedPieces.Len() +func (t *Torrent) numPiecesCompleted() (num pieceIndex) { + return pieceIndex(t._completedPieces.Len()) } func (t *Torrent) close() (err error) { @@ -895,7 +895,7 @@ func (t *Torrent) haveAllPieces() bool { if !t.haveInfo() { return false } - return t._completedPieces.Len() == bitmap.BitIndex(t.numPieces()) + return t._completedPieces.Len() == bitmap.BitRange(t.numPieces()) } func (t *Torrent) havePiece(index pieceIndex) bool { @@ -960,7 +960,7 @@ func (t *Torrent) wantPieceIndex(index pieceIndex) bool { if t.pieceComplete(index) { return false } - if t._pendingPieces.Contains(bitmap.BitIndex(index)) { + if t._pendingPieces.Contains(int(index)) { return true } // t.logger.Printf("piece %d not pending", index) @@ -1019,7 +1019,7 @@ func (t *Torrent) pieceNumPendingChunks(piece pieceIndex) pp.Integer { } func (t *Torrent) pieceAllDirty(piece pieceIndex) bool { - return t.pieces[piece]._dirtyChunks.Len() == int(t.pieceNumChunks(piece)) + return t.pieces[piece]._dirtyChunks.Len() == bitmap.BitRange(t.pieceNumChunks(piece)) } func (t *Torrent) readersChanged() { @@ -1078,11 +1078,11 @@ func (t *Torrent) updatePiecePriority(piece pieceIndex) { newPrio := p.uncachedPriority() // t.logger.Printf("torrent %p: piece %d: uncached priority: %v", t, piece, newPrio) if newPrio == PiecePriorityNone { - if !t._pendingPieces.Remove(bitmap.BitIndex(piece)) { + if !t._pendingPieces.Remove(int(piece)) { return } } else { - if !t._pendingPieces.Set(bitmap.BitIndex(piece), newPrio.BitmapPriority()) { + if !t._pendingPieces.Set(int(piece), newPrio.BitmapPriority()) { return } } @@ -1138,7 +1138,7 @@ func (t *Torrent) forReaderOffsetPieces(f func(begin, end pieceIndex) (more bool } func (t *Torrent) piecePriority(piece pieceIndex) piecePriority { - prio, ok := t._pendingPieces.GetPriority(bitmap.BitIndex(piece)) + prio, ok := t._pendingPieces.GetPriority(piece) if !ok { return PiecePriorityNone } @@ -1286,7 +1286,7 @@ func (t *Torrent) readerPiecePriorities() (now, readahead bitmap.Bitmap) { t.forReaderOffsetPieces(func(begin, end pieceIndex) bool { if end > begin { now.Add(bitmap.BitIndex(begin)) - readahead.AddRange(bitmap.BitIndex(begin)+1, bitmap.BitIndex(end)) + readahead.AddRange(bitmap.BitRange(begin)+1, bitmap.BitRange(end)) } return true }) @@ -1966,7 +1966,7 @@ func (t *Torrent) tryCreatePieceHasher() bool { return false } p := t.piece(pi) - t.piecesQueuedForHash.Remove(pi) + t.piecesQueuedForHash.Remove(bitmap.BitIndex(pi)) p.hashing = true t.publishPieceChange(pi) t.updatePiecePriority(pi) diff --git a/torrent_test.go b/torrent_test.go index d53aa693..ceff1fee 100644 --- a/torrent_test.go +++ b/torrent_test.go @@ -9,6 +9,7 @@ import ( "testing" "github.com/anacrolix/missinggo" + "github.com/anacrolix/missinggo/v2/bitmap" "github.com/bradfitz/iter" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -96,8 +97,8 @@ func BenchmarkUpdatePiecePriorities(b *testing.B) { r.Seek(3500000, io.SeekStart) } assert.Len(b, t.readers, 7) - for i := 0; i < int(t.numPieces()); i += 3 { - t._completedPieces.Set(i, true) + for i := 0; i < t.numPieces(); i += 3 { + t._completedPieces.Set(bitmap.BitIndex(i), true) } t.DownloadPieces(0, t.numPieces()) for range iter.N(b.N) { From 1a27e1439130bbd6586a9bdb2e47cf6ee513b268 Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Thu, 20 May 2021 11:26:08 +1000 Subject: [PATCH 40/59] Don't always force piece reallocation The balancing for unforced pieces may not be exactly what we want yet. --- request-strategy/order.go | 41 +++++++++++++++++++++------- request-strategy/order_test.go | 49 +++++++++++++++++++++++++++------- 2 files changed, 71 insertions(+), 19 deletions(-) diff --git a/request-strategy/order.go b/request-strategy/order.go index 54bb5285..2ac04ab4 100644 --- a/request-strategy/order.go +++ b/request-strategy/order.go @@ -74,6 +74,7 @@ func (me *peersForPieceRequests) addNextRequest(r Request) { type requestablePiece struct { index pieceIndex t Torrent + alwaysReallocate bool NumPendingChunks int IterPendingChunks ChunksIter } @@ -137,6 +138,7 @@ func getRequestablePieces(input Input) (ret []requestablePiece) { t: piece.t.Torrent, NumPendingChunks: piece.NumPendingChunks, IterPendingChunks: piece.iterPendingChunksWrapper, + alwaysReallocate: piece.Priority >= types.PiecePriorityNext, }) } return @@ -201,23 +203,42 @@ func allocatePendingChunks(p requestablePiece, peers []*requestsPeer) { } } }() - sortPeersForPiece := func(byHasRequest *Request) { + sortPeersForPiece := func(req *Request) { sort.Slice(peersForPiece, func(i, j int) bool { - ml := multiless.New().Int( - peersForPiece[i].requestsInPiece, - peersForPiece[j].requestsInPiece, - ).Int( + byHasRequest := func() multiless.Computation { + ml := multiless.New() + if req != nil { + _, iHas := peersForPiece[i].nextState.Requests[*req] + _, jHas := peersForPiece[j].nextState.Requests[*req] + ml = ml.Bool(jHas, iHas) + } + return ml + }() + ml := multiless.New() + // We always "reallocate", that is force even striping amongst peers that are either on + // the last piece they can contribute too, or for pieces marked for this behaviour. + // Striping prevents starving peers of requests, and will always re-balance to the + // fastest known peers. + if !p.alwaysReallocate { + ml = ml.Bool( + peersForPiece[j].requestablePiecesRemaining == 1, + peersForPiece[i].requestablePiecesRemaining == 1) + } + if p.alwaysReallocate || peersForPiece[j].requestablePiecesRemaining == 1 { + ml = ml.Int( + peersForPiece[i].requestsInPiece, + peersForPiece[j].requestsInPiece) + } else { + ml = ml.AndThen(byHasRequest) + } + ml = ml.Int( peersForPiece[i].requestablePiecesRemaining, peersForPiece[j].requestablePiecesRemaining, ).Float64( peersForPiece[j].DownloadRate, peersForPiece[i].DownloadRate, ) - if byHasRequest != nil { - _, iHas := peersForPiece[i].nextState.Requests[*byHasRequest] - _, jHas := peersForPiece[j].nextState.Requests[*byHasRequest] - ml = ml.Bool(jHas, iHas) - } + ml = ml.AndThen(byHasRequest) return ml.Int64( int64(peersForPiece[j].Age), int64(peersForPiece[i].Age), // TODO: Probably peer priority can come next diff --git a/request-strategy/order_test.go b/request-strategy/order_test.go index adc0c478..a448ac37 100644 --- a/request-strategy/order_test.go +++ b/request-strategy/order_test.go @@ -187,7 +187,9 @@ func TestDontStealUnnecessarily(t *testing.T) { // Slower than the stealers, but has all requests already. stealee := basePeer stealee.DownloadRate = 1 - keepReqs := requestSetFromSlice(r(0, 0), r(0, 1), r(0, 2)) + keepReqs := requestSetFromSlice( + r(3, 2), r(3, 4), r(3, 6), r(3, 8), + r(4, 0), r(4, 1), r(4, 7), r(4, 8)) stealee.HasExistingRequest = func(r Request) bool { _, ok := keepReqs[r] return ok @@ -197,12 +199,41 @@ func TestDontStealUnnecessarily(t *testing.T) { firstStealer.Id = intPeerId(2) secondStealer := basePeer secondStealer.Id = intPeerId(3) + secondStealer.HasPiece = func(i pieceIndex) bool { + switch i { + case 1, 3: + return true + default: + return false + } + } results := Run(Input{Torrents: []Torrent{{ - Pieces: []Piece{{ - Request: true, - NumPendingChunks: 9, - IterPendingChunks: chunkIterRange(9), - }}, + Pieces: []Piece{ + { + Request: true, + NumPendingChunks: 0, + IterPendingChunks: chunkIterRange(9), + }, + { + Request: true, + NumPendingChunks: 7, + IterPendingChunks: chunkIterRange(7), + }, + { + Request: true, + NumPendingChunks: 0, + IterPendingChunks: chunkIterRange(0), + }, + { + Request: true, + NumPendingChunks: 9, + IterPendingChunks: chunkIterRange(9), + }, + { + Request: true, + NumPendingChunks: 9, + IterPendingChunks: chunkIterRange(9), + }}, Peers: []Peer{ firstStealer, stealee, @@ -215,10 +246,10 @@ func TestDontStealUnnecessarily(t *testing.T) { c.Check(results[p].Requests, qt.HasLen, l) c.Check(results[p].Interested, qt.Equals, l > 0) } - check(firstStealer.Id, 3) - check(secondStealer.Id, 3) + check(firstStealer.Id, 5) + check(secondStealer.Id, 7+9) c.Check(results[stealee.Id], qt.ContentEquals, PeerNextRequestState{ Interested: true, - Requests: keepReqs, + Requests: requestSetFromSlice(r(4, 0), r(4, 1), r(4, 7), r(4, 8)), }) } From b43987fcc8fb738e7fe36f52c0b40c250b741f10 Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Thu, 20 May 2021 14:01:31 +1000 Subject: [PATCH 41/59] Fix up some crashes around piece availability --- peerconn.go | 16 +++++++++++----- torrent.go | 15 ++++++++++----- 2 files changed, 21 insertions(+), 10 deletions(-) diff --git a/peerconn.go b/peerconn.go index 976081ab..8e3f5443 100644 --- a/peerconn.go +++ b/peerconn.go @@ -396,7 +396,9 @@ func (p *Peer) close() { p.discardPieceInclination() p._pieceRequestOrder.Clear() p.peerImpl.onClose() - p.t.decPeerPieceAvailability(p) + if p.t != nil { + p.t.decPeerPieceAvailability(p) + } for _, f := range p.callbacks.PeerClosed { f(p) } @@ -1651,19 +1653,23 @@ func (cn *Peer) peerMaxRequests() int { return cn.PeerMaxRequests } -// Returns the pieces the peer has claimed to have. +// Returns the pieces the peer could have based on their claims. If we don't know how many pieces +// are in the torrent, it could be a very large range the peer has sent HaveAll. func (cn *PeerConn) PeerPieces() bitmap.Bitmap { cn.locker().RLock() defer cn.locker().RUnlock() return cn.newPeerPieces() } -// Returns a new Bitmap that includes bits for all pieces the peer claims to have. +// Returns a new Bitmap that includes bits for all pieces the peer could have based on their claims. func (cn *Peer) newPeerPieces() bitmap.Bitmap { ret := cn._peerPieces.Copy() if cn.peerSentHaveAll { - - ret.AddRange(0, bitmap.BitRange(cn.t.numPieces())) + if cn.t.haveInfo() { + ret.AddRange(0, bitmap.BitRange(cn.t.numPieces())) + } else { + ret.AddRange(0, bitmap.ToEnd) + } } return ret } diff --git a/torrent.go b/torrent.go index 5190b65e..e500a47d 100644 --- a/torrent.go +++ b/torrent.go @@ -436,16 +436,18 @@ func (t *Torrent) onSetInfo() { p.onGotInfo(t.info) }) for i := range t.pieces { - t.updatePieceCompletion(pieceIndex(i)) p := &t.pieces[i] - if !p.storageCompletionOk { - // t.logger.Printf("piece %s completion unknown, queueing check", p) - t.queuePieceCheck(pieceIndex(i)) - } + // Need to add availability before updating piece completion, as that may result in conns + // being dropped. if p.availability != 0 { panic(p.availability) } p.availability = int64(t.pieceAvailabilityFromPeers(i)) + t.updatePieceCompletion(pieceIndex(i)) + if !p.storageCompletionOk { + // t.logger.Printf("piece %s completion unknown, queueing check", p) + t.queuePieceCheck(pieceIndex(i)) + } } t.cl.event.Broadcast() t.gotMetainfo.Set() @@ -1374,6 +1376,9 @@ func (t *Torrent) deletePeerConn(c *PeerConn) (ret bool) { } func (t *Torrent) decPeerPieceAvailability(p *Peer) { + if !t.haveInfo() { + return + } p.newPeerPieces().IterTyped(func(i int) bool { p.t.decPieceAvailability(i) return true From 925f5d147d86bc81bf8c58b0afa9e7d026916480 Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Thu, 20 May 2021 14:08:38 +1000 Subject: [PATCH 42/59] Resolve local replaces --- go.mod | 4 +- go.sum | 284 +++++++++++++++++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 286 insertions(+), 2 deletions(-) diff --git a/go.mod b/go.mod index 51d7b435..7a1e19b3 100644 --- a/go.mod +++ b/go.mod @@ -12,8 +12,8 @@ require ( github.com/anacrolix/log v0.9.0 github.com/anacrolix/missinggo v1.2.1 github.com/anacrolix/missinggo/perf v1.0.0 - github.com/anacrolix/missinggo/v2 v2.5.0 - github.com/anacrolix/multiless v0.1.1-0.20210510014912-3f17cb19bda9 + github.com/anacrolix/missinggo/v2 v2.5.1-0.20210520011502-b3d95d6b1d02 + github.com/anacrolix/multiless v0.1.1-0.20210520040635-10ee7b5f3cff github.com/anacrolix/sync v0.2.0 github.com/anacrolix/tagflag v1.3.0 github.com/anacrolix/upnp v0.1.2-0.20200416075019-5e9378ed1425 diff --git a/go.sum b/go.sum index fa66c663..8c49d1e3 100644 --- a/go.sum +++ b/go.sum @@ -4,17 +4,26 @@ bazil.org/fuse v0.0.0-20200407214033-5883e5a4b512/go.mod h1:FbcW6z/2VytnFDhZfumh cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.31.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= +cloud.google.com/go v0.37.0 h1:69FNAINiZfsEuwH3fKq8QrAAnHz+2m4XL4kVYi5BX0Q= cloud.google.com/go v0.37.0/go.mod h1:TS1dMSSfndXH133OKGwekG838Om/cQT0BUHV3HcBgoo= crawshaw.io/iox v0.0.0-20181124134642-c51c3df30797 h1:yDf7ARQc637HoxDho7xjqdvO5ZA2Yb+xzv/fOnnvZzw= crawshaw.io/iox v0.0.0-20181124134642-c51c3df30797/go.mod h1:sXBiorCo8c46JlQV3oXPKINnZ8mcqnye1EkVkqsectk= +crawshaw.io/sqlite v0.3.2/go.mod h1:igAO5JulrQ1DbdZdtVq48mnZUBAPOeFzer7VhDWNtW4= crawshaw.io/sqlite v0.3.3-0.20210127221821-98b1f83c5508 h1:fILCBBFnjnrQ0whVJlGhfv1E/QiaFDNtGFBObEVRnYg= crawshaw.io/sqlite v0.3.3-0.20210127221821-98b1f83c5508/go.mod h1:igAO5JulrQ1DbdZdtVq48mnZUBAPOeFzer7VhDWNtW4= +dmitri.shuralyov.com/app/changes v0.0.0-20180602232624-0a106ad413e3 h1:hJiie5Bf3QucGRa4ymsAUOxyhYwGEz1xrsVk0P8erlw= dmitri.shuralyov.com/app/changes v0.0.0-20180602232624-0a106ad413e3/go.mod h1:Yl+fi1br7+Rr3LqpNJf1/uxUdtRUV+Tnj0o93V2B9MU= +dmitri.shuralyov.com/html/belt v0.0.0-20180602232347-f7d459c86be0 h1:SPOUaucgtVls75mg+X7CXigS71EnsfVUK/2CgVrwqgw= dmitri.shuralyov.com/html/belt v0.0.0-20180602232347-f7d459c86be0/go.mod h1:JLBrvjyP0v+ecvNYvCpyZgu5/xkfAUhi6wJj28eUfSU= +dmitri.shuralyov.com/service/change v0.0.0-20181023043359-a85b471d5412 h1:GvWw74lx5noHocd+f6HBMXK6DuggBB1dhVkuGZbv7qM= dmitri.shuralyov.com/service/change v0.0.0-20181023043359-a85b471d5412/go.mod h1:a1inKt/atXimZ4Mv927x+r7UpyzRUf4emIoiiSC2TN4= +dmitri.shuralyov.com/state v0.0.0-20180228185332-28bcc343414c h1:ivON6cwHK1OH26MZyWDCnbTRZZf0IhNsENoNAKFS1g4= dmitri.shuralyov.com/state v0.0.0-20180228185332-28bcc343414c/go.mod h1:0PRwlb0D6DFvNNtx+9ybjezNCa8XF0xaYcETyp6rHWU= +git.apache.org/thrift.git v0.0.0-20180902110319-2566ecd5d999 h1:OR8VhtwhcAI3U48/rzBsVOuHi0zDPzYI1xASVcdSgR8= git.apache.org/thrift.git v0.0.0-20180902110319-2566ecd5d999/go.mod h1:fPE2ZNJGynbRyZ4dJvy6G277gSllfV2HJqblrnkyeyg= +github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= +github.com/Knetic/govaluate v3.0.1-0.20171022003610-9aa49832a739+incompatible h1:1G1pk05UrOh0NlF1oeaaix1x8XzrfjIDK47TY0Zehcw= github.com/Knetic/govaluate v3.0.1-0.20171022003610-9aa49832a739+incompatible/go.mod h1:r7JcOSlj0wfOMncg0iLm8Leh48TZaKVeNIfJntJ2wa0= github.com/RoaringBitmap/roaring v0.4.7/go.mod h1:8khRDP4HmeXns4xIj9oGrKSz7XTQiJx2zgh7AcNke4w= github.com/RoaringBitmap/roaring v0.4.17/go.mod h1:D3qVegWTmfCaX4Bl5CrBE9hfrSrrXIr8KVNvRsDi1NI= @@ -24,15 +33,22 @@ github.com/RoaringBitmap/roaring v0.4.23/go.mod h1:D0gp8kJQgE1A4LQ5wFLggQEyvDi06 github.com/RoaringBitmap/roaring v0.5.5/go.mod h1:puNo5VdzwbaIQxSiDIwfXl4Hnc+fbovcX4IW/dSTtUk= github.com/RoaringBitmap/roaring v0.6.0 h1:tZcn2nJpUrZf+xQY8x+9QY7BxSETMjkdNG4Ts5zahyU= github.com/RoaringBitmap/roaring v0.6.0/go.mod h1:WZ83fjBF/7uBHi6QoFyfGL4+xuV4Qn+xFkm4+vSzrhE= +github.com/Shopify/sarama v1.19.0 h1:9oksLxC6uxVPHPVYUmq6xhr1BOF/hHobWH2UzO67z1s= github.com/Shopify/sarama v1.19.0/go.mod h1:FVkBWblsNy7DGZRfXLU0O9RCGt5g3g3yEuWXgklEdEo= +github.com/Shopify/toxiproxy v2.1.4+incompatible h1:TKdv8HiTLgE5wdJuEML90aBgNWsokNbMijUGhmcoBJc= github.com/Shopify/toxiproxy v2.1.4+incompatible/go.mod h1:OXgGpZ6Cli1/URJOF1DMxUHB2q5Ap20/P/eIdh4G0pI= +github.com/VividCortex/gohistogram v1.0.0 h1:6+hBz+qvs0JOrrNhhmR7lFxo5sINxBCGXrdtl/UvroE= github.com/VividCortex/gohistogram v1.0.0/go.mod h1:Pf5mBqqDxYaXu3hDrrU+w6nw50o/4+TcAqDqk/vUH7g= +github.com/afex/hystrix-go v0.0.0-20180502004556-fa1af6a1f4f5 h1:rFw4nCn9iMW+Vajsk51NtYIcwSTkXr+JGrMd36kTDJw= github.com/afex/hystrix-go v0.0.0-20180502004556-fa1af6a1f4f5/go.mod h1:SkGFH1ia65gfNATL8TAiHDNxPzPdmEL5uirI2Uyuz6c= +github.com/alangpierce/go-forceexport v0.0.0-20160317203124-8f1d6941cd75 h1:3ILjVyslFbc4jl1w5TWuvvslFD/nDfR2H8tVaMVLrEY= github.com/alangpierce/go-forceexport v0.0.0-20160317203124-8f1d6941cd75/go.mod h1:uAXEEpARkRhCZfEvy/y0Jcc888f9tHCc1W7/UeEtreE= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= +github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751 h1:JYp7IbQjafoB+tBA3gMyHYHrpOtNuDiK/uB5uXxq5wM= github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= +github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d h1:UQZhZ2O0vMHr2cI+DC1Mbh0TJxzA3RcLoMsFw+aXw7E= github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk541a8SKzHPHnH3zbiI+7dagKZ0cgpgrD7Fyho= github.com/alexflint/go-arg v1.1.0/go.mod h1:3Rj4baqzWaGGmZA2+bVTV8zQOZEjBQAPBnL5xLT+ftY= github.com/alexflint/go-arg v1.2.0/go.mod h1:3Rj4baqzWaGGmZA2+bVTV8zQOZEjBQAPBnL5xLT+ftY= @@ -93,6 +109,8 @@ github.com/anacrolix/missinggo/v2 v2.3.1/go.mod h1:3XNH0OEmyMUZuvXmYdl+FDfXd0vvS github.com/anacrolix/missinggo/v2 v2.4.1-0.20200227072623-f02f6484f997/go.mod h1:KY+ij+mWvwGuqSuecLjjPv5LFw5ICUc1UvRems3VAZE= github.com/anacrolix/missinggo/v2 v2.5.0 h1:75aciOVrzVV1bTH9rl8tYLbXO9A7HXFtHexTChawe/U= github.com/anacrolix/missinggo/v2 v2.5.0/go.mod h1:HYuCbwvJXY3XbcmcIcTgZXHleoDXawxPWx/YiPzFzV0= +github.com/anacrolix/missinggo/v2 v2.5.1-0.20210520011502-b3d95d6b1d02 h1:wf3HKUunewks4FdGJqkViby+vr3n5/IFpPsyEMokxYE= +github.com/anacrolix/missinggo/v2 v2.5.1-0.20210520011502-b3d95d6b1d02/go.mod h1:WEjqh2rmKECd0t1VhQkLGTdIWXO6f6NLjp5GlMZ+6FA= github.com/anacrolix/mmsg v0.0.0-20180515031531-a4a3ba1fc8bb/go.mod h1:x2/ErsYUmT77kezS63+wzZp8E3byYB0gzirM/WMBLfw= github.com/anacrolix/mmsg v1.0.0 h1:btC7YLjOn29aTUAExJiVUhQOuf/8rhm+/nWCMAnL3Hg= github.com/anacrolix/mmsg v1.0.0/go.mod h1:x8kRaJY/dCrY9Al0PEcj1mb/uFHwP6GCJ9fLl4thEPc= @@ -101,6 +119,8 @@ github.com/anacrolix/multiless v0.0.0-20200413040533-acfd16f65d5d/go.mod h1:TrCL github.com/anacrolix/multiless v0.0.0-20210222022749-ef43011a77ec/go.mod h1:TrCLEZfIDbMVfLoQt5tOoiBS/uq4y8+ojuEVVvTNPX4= github.com/anacrolix/multiless v0.1.1-0.20210510014912-3f17cb19bda9 h1:fmNDxh5ysBPinRq249xYYYOLQ/h95DoyGE9e9Gp9xqo= github.com/anacrolix/multiless v0.1.1-0.20210510014912-3f17cb19bda9/go.mod h1:TrCLEZfIDbMVfLoQt5tOoiBS/uq4y8+ojuEVVvTNPX4= +github.com/anacrolix/multiless v0.1.1-0.20210520040635-10ee7b5f3cff h1:ve99yq2FjiN3OANMjRz2rjCX4f2PSKMf3NeDFnmcs8s= +github.com/anacrolix/multiless v0.1.1-0.20210520040635-10ee7b5f3cff/go.mod h1:TrCLEZfIDbMVfLoQt5tOoiBS/uq4y8+ojuEVVvTNPX4= github.com/anacrolix/stm v0.1.0/go.mod h1:ZKz7e7ERWvP0KgL7WXfRjBXHNRhlVRlbBQecqFtPq+A= github.com/anacrolix/stm v0.1.1-0.20191106051447-e749ba3531cf/go.mod h1:zoVQRvSiGjGoTmbM0vSLIiaKjWtNPeTvXUSdJQA4hsg= github.com/anacrolix/stm v0.2.0/go.mod h1:zoVQRvSiGjGoTmbM0vSLIiaKjWtNPeTvXUSdJQA4hsg= @@ -138,56 +158,89 @@ github.com/anacrolix/upnp v0.1.2-0.20200416075019-5e9378ed1425/go.mod h1:Pz94W3k github.com/anacrolix/utp v0.0.0-20180219060659-9e0e1d1d0572/go.mod h1:MDwc+vsGEq7RMw6lr2GKOEqjWny5hO5OZXRVNaBJ2Dk= github.com/anacrolix/utp v0.1.0 h1:FOpQOmIwYsnENnz7tAGohA+r6iXpRjrq8ssKSre2Cp4= github.com/anacrolix/utp v0.1.0/go.mod h1:MDwc+vsGEq7RMw6lr2GKOEqjWny5hO5OZXRVNaBJ2Dk= +github.com/anmitsu/go-shlex v0.0.0-20161002113705-648efa622239 h1:kFOfPq6dUM1hTo4JG6LR5AXSUEsOjtdm0kw0FtQtMJA= github.com/anmitsu/go-shlex v0.0.0-20161002113705-648efa622239/go.mod h1:2FmKhYUyUczH0OGQWaF5ceTx0UBShxjsH6f8oGKYe2c= github.com/apache/thrift v0.12.0/go.mod h1:cp2SuWMxlEZw2r+iP2GNCdIi4C1qmUzdZFSVb+bacwQ= +github.com/apache/thrift v0.13.0 h1:5hryIiq9gtn+MiLVn0wP37kb/uTeRZgN08WoCsAhIhI= github.com/apache/thrift v0.13.0/go.mod h1:cp2SuWMxlEZw2r+iP2GNCdIi4C1qmUzdZFSVb+bacwQ= +github.com/armon/circbuf v0.0.0-20150827004946-bbbad097214e h1:QEF07wC0T1rKkctt1RINW/+RMTVmiwxETico2l3gxJA= github.com/armon/circbuf v0.0.0-20150827004946-bbbad097214e/go.mod h1:3U/XgcO3hCbHZ8TKRvWD2dDTCfh9M9ya+I9JpbB7O8o= +github.com/armon/go-metrics v0.0.0-20180917152333-f0300d1749da h1:8GUt8eRujhVEGZFFEjBj46YV4rDjvGrNxb0KMWYkL2I= github.com/armon/go-metrics v0.0.0-20180917152333-f0300d1749da/go.mod h1:Q73ZrmVTwzkszR9V5SSuryQ31EELlFMUz1kKyl939pY= +github.com/armon/go-radix v0.0.0-20180808171621-7fddfc383310 h1:BUAU3CGlLvorLI26FmByPp2eC2qla6E1Tw+scpcg/to= github.com/armon/go-radix v0.0.0-20180808171621-7fddfc383310/go.mod h1:ufUuZ+zHj4x4TnLV4JWEpy2hxWSpsRywHrMgIH9cCH8= +github.com/aryann/difflib v0.0.0-20170710044230-e206f873d14a h1:pv34s756C4pEXnjgPfGYgdhg/ZdajGhyOvzx8k+23nw= github.com/aryann/difflib v0.0.0-20170710044230-e206f873d14a/go.mod h1:DAHtR1m6lCRdSC2Tm3DSWRPvIPr6xNKyeHdqDQSQT+A= +github.com/aws/aws-lambda-go v1.13.3 h1:SuCy7H3NLyp+1Mrfp+m80jcbi9KYWAs9/BXwppwRDzY= github.com/aws/aws-lambda-go v1.13.3/go.mod h1:4UKl9IzQMoD+QF79YdCuzCwp8VbmG4VAQwij/eHl5CU= +github.com/aws/aws-sdk-go v1.27.0 h1:0xphMHGMLBrPMfxR2AmVjZKcMEESEgWF8Kru94BNByk= github.com/aws/aws-sdk-go v1.27.0/go.mod h1:KmX6BPdI08NWTb3/sm4ZGu5ShLoqVDhKgpiN924inxo= +github.com/aws/aws-sdk-go-v2 v0.18.0 h1:qZ+woO4SamnH/eEbjM2IDLhRNwIwND/RQyVlBLp3Jqg= github.com/aws/aws-sdk-go-v2 v0.18.0/go.mod h1:JWVYvqSMppoMJC0x5wdwiImzgXTI9FuZwxzkQq9wy+g= github.com/benbjohnson/immutable v0.2.0/go.mod h1:uc6OHo6PN2++n98KHLxW8ef4W42ylHiQSENghE1ezxI= github.com/benbjohnson/immutable v0.3.0 h1:TVRhuZx2wG9SZ0LRdqlbs9S5BZ6Y24hJEHTCgWHZEIw= github.com/benbjohnson/immutable v0.3.0/go.mod h1:uc6OHo6PN2++n98KHLxW8ef4W42ylHiQSENghE1ezxI= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= +github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= +github.com/bgentry/speakeasy v0.1.0 h1:ByYyxL9InA1OWqxJqqp2A5pYHUrCiAL6K3J+LKSsQkY= github.com/bgentry/speakeasy v0.1.0/go.mod h1:+zsyZBPWlz7T6j88CTgSN5bM796AkVf0kBD4zp0CCIs= +github.com/boltdb/bolt v1.3.1 h1:JQmyP4ZBrce+ZQu0dY660FMfatumYDLun9hBCUVIkF4= github.com/boltdb/bolt v1.3.1/go.mod h1:clJnj/oiGkjum5o1McbSZDSLxVThjynRyGBgiAx27Ps= +github.com/bradfitz/go-smtpd v0.0.0-20170404230938-deb6d6237625 h1:ckJgFhFWywOx+YLEMIJsTb+NV6NexWICk5+AMSuz3ss= github.com/bradfitz/go-smtpd v0.0.0-20170404230938-deb6d6237625/go.mod h1:HYsPBTaaSFSlLx/70C2HPIMNZpVV8+vt/A+FMnYP11g= github.com/bradfitz/iter v0.0.0-20140124041915-454541ec3da2/go.mod h1:PyRFw1Lt2wKX4ZVSQ2mk+PeDa1rxyObEDlApuIsUKuo= github.com/bradfitz/iter v0.0.0-20190303215204-33e6a9893b0c/go.mod h1:PyRFw1Lt2wKX4ZVSQ2mk+PeDa1rxyObEDlApuIsUKuo= github.com/bradfitz/iter v0.0.0-20191230175014-e8f45d346db8 h1:GKTyiRCL6zVf5wWaqKnf+7Qs6GbEPfd4iMOitWzXJx8= github.com/bradfitz/iter v0.0.0-20191230175014-e8f45d346db8/go.mod h1:spo1JLcs67NmW1aVLEgtA8Yy1elc+X8y5SRW1sFW4Og= +github.com/buger/jsonparser v0.0.0-20181115193947-bf1c66bbce23 h1:D21IyuvjDCshj1/qq+pCNd3VZOAEI9jy6Bi131YlXgI= github.com/buger/jsonparser v0.0.0-20181115193947-bf1c66bbce23/go.mod h1:bbYlZJ7hK1yFx9hf58LP0zeX7UjIGs20ufpu3evjr+s= +github.com/casbin/casbin/v2 v2.1.2 h1:bTwon/ECRx9dwBy2ewRVr5OiqjeXSGiTUY74sDPQi/g= github.com/casbin/casbin/v2 v2.1.2/go.mod h1:YcPU1XXisHhLzuxH9coDNf2FbKpjGlbCg3n9yuLkIJQ= +github.com/cenkalti/backoff v2.2.1+incompatible h1:tNowT99t7UNflLxfYYSlKYsBpXdEet03Pg2g16Swow4= github.com/cenkalti/backoff v2.2.1+incompatible/go.mod h1:90ReRw6GdpyfrHakVjL/QHaoyV4aDUVVkXQJJJ3NXXM= +github.com/census-instrumentation/opencensus-proto v0.2.1 h1:glEXhBS5PSLLv4IXzLA5yPRVX4bilULVyxxbrfOtDAk= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= +github.com/cespare/xxhash/v2 v2.1.1 h1:6MnRN8NT7+YBpUIWxHtefFZOKTAPgGjpQSxqLNn0+qY= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/cheekybits/genny v1.0.0 h1:uGGa4nei+j20rOSeDeP5Of12XVm7TGUd4dJA9RDitfE= github.com/cheekybits/genny v1.0.0/go.mod h1:+tQajlRqAUrPI7DOSpB0XAqZYtQakVtB7wXkRAgjxjQ= +github.com/clbanning/x2j v0.0.0-20191024224557-825249438eec h1:EdRZT3IeKQmfCSrgo8SZ8V3MEnskuJP0wCYNpe+aiXo= github.com/clbanning/x2j v0.0.0-20191024224557-825249438eec/go.mod h1:jMjuTZXRI4dUb/I5gc9Hdhagfvm9+RyrPryS/auMzxE= +github.com/client9/misspell v0.3.4 h1:ta993UF76GwbvJcIo3Y68y/M3WxlpEHPWIGDkJYwzJI= github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= +github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa h1:OaNxuTZr7kxeODyLWsRMC+OD03aFUH+mW6r2d+MWa5Y= github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa/go.mod h1:zn76sxSg3SzpJ0PPJaLDCu+Bu0Lg3sKTORVIj19EIF8= +github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd h1:qMd81Ts1T2OTKmB4acZcyKaMtRnY5Y44NuXGX2GFJ1w= github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd/go.mod h1:sE/e/2PUdi/liOCUjSTXgM1o87ZssimdTWN964YiIeI= +github.com/coreos/go-semver v0.2.0 h1:3Jm3tLmsgAYcjC+4Up7hJrFBPr+n7rAqYeSw/SZazuY= github.com/coreos/go-semver v0.2.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= github.com/coreos/go-systemd v0.0.0-20180511133405-39ca1b05acc7/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= +github.com/coreos/go-systemd v0.0.0-20181012123002-c6f51f82210d h1:t5Wuyh53qYyg9eqn4BbnlIT+vmhyww0TatL+zT3uWgI= github.com/coreos/go-systemd v0.0.0-20181012123002-c6f51f82210d/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= +github.com/coreos/pkg v0.0.0-20160727233714-3ac0863d7acf h1:CAKfRE2YtTUIjjh1bkBtyYFaUT/WmOqsJjgtihT0vMI= github.com/coreos/pkg v0.0.0-20160727233714-3ac0863d7acf/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA= +github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d h1:U+s90UTSYgptZMwQh2aRr3LuazLJIa+Pg3Kc1ylSYVY= github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= +github.com/creack/pty v1.1.7 h1:6pwm8kMQKCmgUg0ZHTm5+/YvRK0s3THD/28+T6/kk4A= github.com/creack/pty v1.1.7/go.mod h1:lj5s0c3V2DBrqTV7llrYr5NG6My20zk30Fl46Y7DoTY= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/dgrijalva/jwt-go v3.2.0+incompatible h1:7qlOGliEKZXTDg6OTjfoBKDXWrumCAMpl/TFQ4/5kLM= github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ= +github.com/docopt/docopt-go v0.0.0-20180111231733-ee0de3bc6815 h1:bWDMxwH3px2JBh6AyO7hdCn/PkvCZXii8TGj7sbtEbQ= github.com/docopt/docopt-go v0.0.0-20180111231733-ee0de3bc6815/go.mod h1:WwZ+bS3ebgob9U8Nd0kOddGdZWjyMGR8Wziv+TBNwSE= github.com/dustin/go-humanize v0.0.0-20171111073723-bb3d318650d4/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= github.com/dustin/go-humanize v0.0.0-20180421182945-02af3965c54e/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= github.com/dustin/go-humanize v1.0.0 h1:VSnTsYCnlFHaM2/igO1h6X3HA71jcobQuxemgkq4zYo= github.com/dustin/go-humanize v1.0.0/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= +github.com/eapache/go-resiliency v1.1.0 h1:1NtRmCAqadE2FN4ZcN6g90TP3uk8cg9rn9eNK2197aU= github.com/eapache/go-resiliency v1.1.0/go.mod h1:kFI+JgMyC7bLPUVY133qvEBtVayf5mFgVsvEsIPBvNs= +github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21 h1:YEetp8/yCZMuEPMUDHG0CW/brkkEp8mzqk2+ODEitlw= github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21/go.mod h1:+020luEh2TKB4/GOp8oxxtq0Daoen/Cii55CzbTV6DU= +github.com/eapache/queue v1.1.0 h1:YOEu7KNc61ntiQlcEeUIoDTJ2o8mQznoNvUhiigpIqc= github.com/eapache/queue v1.1.0/go.mod h1:6eCeP0CKFpHLu8blIFXhExK/dRa7WDZfr6jVFPTqq+I= github.com/edsrzf/mmap-go v0.0.0-20170320065105-0bce6a688712/go.mod h1:YO35OhQPt3KJa3ryjFM5Bs14WD66h8eGKpfaBNrHW5M= github.com/edsrzf/mmap-go v1.0.0 h1:CEBF7HpRnUCSJgGUb5h1Gm7e3VkmVDrR8lvWVLtrOFw= @@ -197,13 +250,21 @@ github.com/elliotchance/orderedmap v1.3.0/go.mod h1:8hdSl6jmveQw8ScByd3AaNHNk51R github.com/elliotchance/orderedmap v1.4.0 h1:wZtfeEONCbx6in1CZyE6bELEt/vFayMvsxqI5SgsR+A= github.com/elliotchance/orderedmap v1.4.0/go.mod h1:wsDwEaX5jEoyhbs7x93zk2H/qv0zwuhg4inXhDkYqys= github.com/envoyproxy/go-control-plane v0.6.9/go.mod h1:SBwIajubJHhxtWwsL9s8ss4safvEdbitLhGGK48rN6g= +github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473 h1:4cmBvAEBNJaGARUEs3/suWRyfyBfhf7I60WBZq+bv2w= github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= +github.com/envoyproxy/protoc-gen-validate v0.1.0 h1:EQciDnbrYxy13PgWoY8AqoxGiPrpgBZ1R8UNe3ddc+A= github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= +github.com/etcd-io/bbolt v1.3.3 h1:gSJmxrs37LgTqR/oyJBWok6k6SvXEUerFTbltIhXkBM= github.com/etcd-io/bbolt v1.3.3/go.mod h1:ZF2nL25h33cCyBtcyWeZ2/I3HQOfTP+0PIEvHjkjCrw= +github.com/fatih/color v1.7.0 h1:DkWD4oS2D8LGGgTQ6IvwJJXSL5Vp2ffcQg58nFV38Ys= github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4= +github.com/flynn/go-shlex v0.0.0-20150515145356-3f9db97f8568 h1:BHsljHzVlRcyQhjrss6TZTdY2VfCqZPbv5k3iBFa2ZQ= github.com/flynn/go-shlex v0.0.0-20150515145356-3f9db97f8568/go.mod h1:xEzjJPgXI435gkrCt3MPfRiAkVrwSbHsst4LCFVfpJc= +github.com/francoispqt/gojay v1.2.13 h1:d2m3sFjloqoIUQU3TsHBgj6qg/BVGlTBeHDUmyJnXKk= github.com/francoispqt/gojay v1.2.13/go.mod h1:ehT5mTG4ua4581f1++1WLG0vPdaA9HaiDsoyrBGkyDY= +github.com/franela/goblin v0.0.0-20200105215937-c9ffbefa60db h1:gb2Z18BhTPJPpLQWj4T+rfKHYCHxRHCtRxhKKjRidVw= github.com/franela/goblin v0.0.0-20200105215937-c9ffbefa60db/go.mod h1:7dvUGVsVBjqR7JHJk0brhHOZYGmfBYOrK0ZhYMEtBr4= +github.com/franela/goreq v0.0.0-20171204163338-bcd34c9993f8 h1:a9ENSRDFBUPkJ5lCgVZh26+ZbGyoVJG7yb5SSzF5H54= github.com/franela/goreq v0.0.0-20171204163338-bcd34c9993f8/go.mod h1:ZhphrRTfi2rbfLwlschooIH4+wKKDR4Pdxhh+TRoA20= github.com/frankban/quicktest v1.9.0/go.mod h1:ui7WezCLWMWxVWr1GETZY3smRy0G4KWq9vcPtJmFl7Y= github.com/frankban/quicktest v1.11.3 h1:8sXhOn0uLys67V8EsXLc6eszDs8VXWxL3iRvebPhedY= @@ -211,38 +272,54 @@ github.com/frankban/quicktest v1.11.3/go.mod h1:wRf/ReqHper53s+kmmSZizM8NamnL3IM github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= github.com/fsnotify/fsnotify v1.4.9 h1:hsms1Qyu0jgnwNXIxa+/V/PDsU6CfLf6CNO8H7IWoS4= github.com/fsnotify/fsnotify v1.4.9/go.mod h1:znqG4EE+3YCdAaPaxE2ZRY/06pZUdp0tY4IgpuI1SZQ= +github.com/ghodss/yaml v1.0.0 h1:wQHKEahhL6wmXdzwWG11gIVCkOv05bNOh+Rxn0yngAk= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= +github.com/gliderlabs/ssh v0.1.1 h1:j3L6gSLQalDETeEg/Jg0mGY0/y/N6zI2xX1978P0Uqw= github.com/gliderlabs/ssh v0.1.1/go.mod h1:U7qILu1NlMHj9FlMhZLlkCdDnU1DBEAqr0aevW3Awn0= github.com/glycerine/go-unsnap-stream v0.0.0-20180323001048-9f0cb55181dd/go.mod h1:/20jfyN9Y5QPEAprSgKAUr+glWDY39ZiUEAYOEv5dsE= github.com/glycerine/go-unsnap-stream v0.0.0-20181221182339-f9677308dec2/go.mod h1:/20jfyN9Y5QPEAprSgKAUr+glWDY39ZiUEAYOEv5dsE= github.com/glycerine/go-unsnap-stream v0.0.0-20190901134440-81cf024a9e0a/go.mod h1:/20jfyN9Y5QPEAprSgKAUr+glWDY39ZiUEAYOEv5dsE= +github.com/glycerine/go-unsnap-stream v0.0.0-20210130063903-47dfef350d96 h1:rCXyLrgJ598XNj7KTqPzAvwTzlyvI+clqasoNfLQStE= github.com/glycerine/go-unsnap-stream v0.0.0-20210130063903-47dfef350d96/go.mod h1:/20jfyN9Y5QPEAprSgKAUr+glWDY39ZiUEAYOEv5dsE= github.com/glycerine/goconvey v0.0.0-20180728074245-46e3a41ad493/go.mod h1:Ogl1Tioa0aV7gstGFO7KhffUsb9M4ydbEbbxpcEDc24= github.com/glycerine/goconvey v0.0.0-20190315024820-982ee783a72e/go.mod h1:Ogl1Tioa0aV7gstGFO7KhffUsb9M4ydbEbbxpcEDc24= +github.com/glycerine/goconvey v0.0.0-20190410193231-58a59202ab31 h1:gclg6gY70GLy3PbkQ1AERPfmLMMagS60DKF78eWwLn8= github.com/glycerine/goconvey v0.0.0-20190410193231-58a59202ab31/go.mod h1:Ogl1Tioa0aV7gstGFO7KhffUsb9M4ydbEbbxpcEDc24= +github.com/go-errors/errors v1.0.1 h1:LUHzmkK3GUKUrL/1gfBUxAHzcev3apQlezX/+O7ma6w= github.com/go-errors/errors v1.0.1/go.mod h1:f4zRHt4oKfwPJE5k8C9vpYG+aDHdBFUsgrm6/TyX73Q= github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= +github.com/go-kit/kit v0.10.0 h1:dXFJfIHVvUcpSgDOV+Ne6t7jXri8Tfv2uOLHUZ2XNuo= github.com/go-kit/kit v0.10.0/go.mod h1:xUsJbQ/Fp4kEt7AFgCuvyX4a71u8h9jB8tj/ORgOZ7o= github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= +github.com/go-logfmt/logfmt v0.5.0 h1:TrB8swr/68K7m9CcGut2g3UOihhbcbiMAYiuTXdEih4= github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= +github.com/go-sql-driver/mysql v1.4.0 h1:7LxgVwFb2hIQtMm87NdgAVfXjnt4OePseqT1tKx+opk= github.com/go-sql-driver/mysql v1.4.0/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w= +github.com/go-stack/stack v1.8.0 h1:5SgMzNM5HxrEjV0ww2lTmX6E2Izsfxas4+YHWRs3Lsk= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= +github.com/go-task/slim-sprig v0.0.0-20210107165309-348f09dbbbc0 h1:p104kn46Q8WdvHunIJ9dAyjPVtrBPhSr3KT2yUst43I= github.com/go-task/slim-sprig v0.0.0-20210107165309-348f09dbbbc0/go.mod h1:fyg7847qk6SyHyPtNmDHnmrv/HOrqktSC+C9fM+CJOE= +github.com/gogo/googleapis v1.1.0 h1:kFkMAZBNAn4j7K0GiZr8cRYzejq68VbheufiV3YuyFI= github.com/gogo/googleapis v1.1.0/go.mod h1:gf4bu3Q80BeJ6H1S1vYPm8/ELATdvryBaNFGgqEef3s= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.2.0/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= +github.com/gogo/protobuf v1.2.1 h1:/s5zKNz0uPFCZ5hddgPdo2TK2TVrUNMn0OOX8/aZMTE= github.com/gogo/protobuf v1.2.1/go.mod h1:hp+jE20tsWTFYpLwKvXlhS1hjn+gTNwPg2I6zVXpSg4= +github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b h1:VKtxabqXZkF25pY9ekfRL6a582T4P37/31XEstQ5p58= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= github.com/golang/groupcache v0.0.0-20160516000752-02826c3e7903/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20191027212112-611e8accdfc9/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e h1:1r7pUrabqp18hOBcwBwiTsbnFeTZHV9eER/QT5JVZxY= github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/lint v0.0.0-20180702182130-06c8688daad7 h1:2hRPrmiwPrp3fQX967rNJIhQPtiGXdlQWAxKbKw3VHA= github.com/golang/lint v0.0.0-20180702182130-06c8688daad7/go.mod h1:tluoj9z5200jBnyusfRPU2LqT6J+DAorxEvtC7LHB+E= github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/mock v1.2.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/mock v1.4.0/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= +github.com/golang/mock v1.4.4 h1:l75CXGRSwbaYNpl/Z2X1XIIAMSCquvXgpVZDhwEIJsc= github.com/golang/mock v1.4.4/go.mod h1:l3mdAwkq5BuhzHwde/uurv3sEJeZMXNpwsxVWU71h+4= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= @@ -254,9 +331,11 @@ github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:W github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0= github.com/golang/protobuf v1.4.1/go.mod h1:U8fpvMrcmy5pZrNK1lt4xCsGvpyWQ/VVv6QDs8UjoX8= github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= +github.com/golang/protobuf v1.4.3 h1:JjCZWpVbqXDqFVmTfYWEVTMIYrL/NPdPSCHPJ0T/raM= github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= github.com/golang/snappy v0.0.0-20180518054509-2e65f85255db/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= +github.com/golang/snappy v0.0.2 h1:aeE13tS0IiQgFjYdoL8qN3K1N2bXXtI6Vi51/y7BpMw= github.com/golang/snappy v0.0.2/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/google/btree v0.0.0-20180124185431-e89373fe6b4a/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= @@ -270,59 +349,95 @@ github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/ github.com/google/go-cmp v0.5.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.4 h1:L8R9j+yAqZuZjsqh/z+F1NCffTKKLShY6zXTItVIZ8M= github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-github v17.0.0+incompatible h1:N0LgJ1j65A7kfXrZnUDaYCs/Sf4rEjNlfyDHW9dolSY= github.com/google/go-github v17.0.0+incompatible/go.mod h1:zLgOLi98H3fifZn+44m+umXrS52loVEgC2AApnigrVQ= +github.com/google/go-querystring v1.0.0 h1:Xkwi/a1rcvNg1PPYe5vI8GbeBY/jrVuDX5ASuANWTrk= github.com/google/go-querystring v1.0.0/go.mod h1:odCYkC5MyYFN7vkCjXpyrEuKhc/BUO6wN/zVPAxq5ck= +github.com/google/gofuzz v1.0.0 h1:A8PeW59pxE9IoFRqBp37U+mSNaQoZ46F1f0f863XSXw= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= +github.com/google/martian v2.1.0+incompatible h1:/CP5g8u/VJHijgedC/Legn3BAbAaWPgecwXBIDzw5no= github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs= +github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57 h1:eqyIo2HjKhKe/mJzTG8n4VqvLXIOEG+SLdDqX7xGtkY= github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= +github.com/google/renameio v0.1.0 h1:GOZbcHa3HfsPKPlmyPyN2KEohoMXOhdMbHrvbpl2QaA= github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.1.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.1.5/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.2.0 h1:qJYtXnJRWmpe7m/3XlyhrsLrEURqHRM2kxzoxXqyUDs= github.com/google/uuid v1.2.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +github.com/googleapis/gax-go v2.0.0+incompatible h1:j0GKcs05QVmm7yesiZq2+9cxHkNK9YM6zKx4D2qucQU= github.com/googleapis/gax-go v2.0.0+incompatible/go.mod h1:SFVmujtThgffbyetf+mdk2eWhX2bMyUtNHzFKcPA9HY= +github.com/googleapis/gax-go/v2 v2.0.3 h1:siORttZ36U2R/WjiJuDz8znElWBiAlO9rVt+mqJt0Cc= github.com/googleapis/gax-go/v2 v2.0.3/go.mod h1:LLvjysVCY1JZeum8Z6l8qUty8fiNwE08qbEPm1M08qg= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= github.com/gopherjs/gopherjs v0.0.0-20181103185306-d547d1d9531e/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= github.com/gopherjs/gopherjs v0.0.0-20190309154008-847fc94819f9/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= +github.com/gopherjs/gopherjs v0.0.0-20190910122728-9d188e94fb99 h1:twflg0XRTjwKpxb/jFExr4HGq6on2dEOmnL6FV+fgPw= github.com/gopherjs/gopherjs v0.0.0-20190910122728-9d188e94fb99/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= +github.com/gorilla/context v1.1.1 h1:AWwleXJkX/nhcU9bZSnZoi3h/qGYqQAGhq6zZe/aQW8= github.com/gorilla/context v1.1.1/go.mod h1:kBGZzfjB9CEq2AlWe17Uuf7NDRt0dE0s8S51q0aT7Yg= github.com/gorilla/mux v1.6.2/go.mod h1:1lud6UwP+6orDFRuTfBEV8e9/aOM/c4fVVCaMa2zaAs= +github.com/gorilla/mux v1.7.3 h1:gnP5JzjVOuiZD07fKKToCAOjS0yOpj/qPETTXCCS6hw= github.com/gorilla/mux v1.7.3/go.mod h1:1lud6UwP+6orDFRuTfBEV8e9/aOM/c4fVVCaMa2zaAs= github.com/gorilla/websocket v0.0.0-20170926233335-4201258b820c/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoAMk2YaljkQ= github.com/gorilla/websocket v1.4.2 h1:+/TMaTYc4QFitKJxsQ7Yye35DkWvkdLcvGKqM+x0Ufc= github.com/gorilla/websocket v1.4.2/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE= github.com/gosuri/uilive v0.0.0-20170323041506-ac356e6e42cd/go.mod h1:qkLSc0A5EXSP6B04TrN4oQoxqFI7A8XvoXSlJi8cwk8= +github.com/gosuri/uilive v0.0.3 h1:kvo6aB3pez9Wbudij8srWo4iY6SFTTxTKOkb+uRCE8I= github.com/gosuri/uilive v0.0.3/go.mod h1:qkLSc0A5EXSP6B04TrN4oQoxqFI7A8XvoXSlJi8cwk8= github.com/gosuri/uiprogress v0.0.0-20170224063937-d0567a9d84a1/go.mod h1:C1RTYn4Sc7iEyf6j8ft5dyoZ4212h8G1ol9QQluh5+0= +github.com/gosuri/uiprogress v0.0.1 h1:0kpv/XY/qTmFWl/SkaJykZXrBBzwwadmW8fRb7RJSxw= github.com/gosuri/uiprogress v0.0.1/go.mod h1:C1RTYn4Sc7iEyf6j8ft5dyoZ4212h8G1ol9QQluh5+0= +github.com/gregjones/httpcache v0.0.0-20180305231024-9cad4c3443a7 h1:pdN6V1QBWetyv/0+wjACpqVH+eVULgEjkurDLq3goeM= github.com/gregjones/httpcache v0.0.0-20180305231024-9cad4c3443a7/go.mod h1:FecbI9+v66THATjSRHfNgh1IVFe/9kFxbXtjV0ctIMA= +github.com/grpc-ecosystem/go-grpc-middleware v1.0.1-0.20190118093823-f849b5445de4 h1:z53tR0945TRRQO/fLEVPI6SMv7ZflF0TEaTAoU7tOzg= github.com/grpc-ecosystem/go-grpc-middleware v1.0.1-0.20190118093823-f849b5445de4/go.mod h1:FiyG127CGDf3tlThmgyCl78X/SZQqEOJBCDaAfeWzPs= +github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 h1:Ovs26xHkKqVztRpIrF/92BcuyuQ/YW4NSIpoGtfXNho= github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgfV/d3M/q6VIi02HzZEHgUlZvzk= github.com/grpc-ecosystem/grpc-gateway v1.5.0/go.mod h1:RSKVYQBd5MCa4OVpNdGskqpgL2+G+NZTnrVHpWWfpdw= +github.com/grpc-ecosystem/grpc-gateway v1.9.5 h1:UImYN5qQ8tuGpGE16ZmjvcTtTw24zw1QAp/SlnNrZhI= github.com/grpc-ecosystem/grpc-gateway v1.9.5/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY= +github.com/hashicorp/consul/api v1.3.0 h1:HXNYlRkkM/t+Y/Yhxtwcy02dlYwIaoxzvxPnS+cqy78= github.com/hashicorp/consul/api v1.3.0/go.mod h1:MmDNSzIMUjNpY/mQ398R4bk2FnqQLoPndWW5VkKPlCE= +github.com/hashicorp/consul/sdk v0.3.0 h1:UOxjlb4xVNF93jak1mzzoBatyFju9nrkxpVwIp/QqxQ= github.com/hashicorp/consul/sdk v0.3.0/go.mod h1:VKf9jXwCTEY1QZP2MOLRhb5i/I/ssyNV1vwHyQBF0x8= +github.com/hashicorp/errwrap v1.0.0 h1:hLrqtEDnRye3+sgx6z4qVLNuviH3MR5aQ0ykNJa/UYA= github.com/hashicorp/errwrap v1.0.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= +github.com/hashicorp/go-cleanhttp v0.5.1 h1:dH3aiDG9Jvb5r5+bYHsikaOUIpcM0xvgMXVoDkXMzJM= github.com/hashicorp/go-cleanhttp v0.5.1/go.mod h1:JpRdi6/HCYpAwUzNwuwqhbovhLtngrth3wmdIIUrZ80= +github.com/hashicorp/go-immutable-radix v1.0.0 h1:AKDB1HM5PWEA7i4nhcpwOrO2byshxBjXVn/J/3+z5/0= github.com/hashicorp/go-immutable-radix v1.0.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60= +github.com/hashicorp/go-msgpack v0.5.3 h1:zKjpN5BK/P5lMYrLmBHdBULWbJ0XpYR+7NGzqkZzoD4= github.com/hashicorp/go-msgpack v0.5.3/go.mod h1:ahLV/dePpqEmjfWmKiqvPkv/twdG7iPBM1vqhUKIvfM= +github.com/hashicorp/go-multierror v1.0.0 h1:iVjPR7a6H0tWELX5NxNe7bYopibicUzc7uPribsnS6o= github.com/hashicorp/go-multierror v1.0.0/go.mod h1:dHtQlpGsu+cZNNAkkCN/P3hoUDHhCYQXV3UM06sGGrk= +github.com/hashicorp/go-rootcerts v1.0.0 h1:Rqb66Oo1X/eSV1x66xbDccZjhJigjg0+e82kpwzSwCI= github.com/hashicorp/go-rootcerts v1.0.0/go.mod h1:K6zTfqpRlCUIjkwsN4Z+hiSfzSTQa6eBIzfwKfwNnHU= +github.com/hashicorp/go-sockaddr v1.0.0 h1:GeH6tui99pF4NJgfnhp+L6+FfobzVW3Ah46sLo0ICXs= github.com/hashicorp/go-sockaddr v1.0.0/go.mod h1:7Xibr9yA9JjQq1JpNB2Vw7kxv8xerXegt+ozgdvDeDU= +github.com/hashicorp/go-syslog v1.0.0 h1:KaodqZuhUoZereWVIYmpUgZysurB1kBLX2j0MwMrUAE= github.com/hashicorp/go-syslog v1.0.0/go.mod h1:qPfqrKkXGihmCqbJM2mZgkZGvKG1dFdvsLplgctolz4= github.com/hashicorp/go-uuid v1.0.0/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= +github.com/hashicorp/go-uuid v1.0.1 h1:fv1ep09latC32wFoVwnqcnKJGnMSdBanPczbHAYm1BE= github.com/hashicorp/go-uuid v1.0.1/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= +github.com/hashicorp/go-version v1.2.0 h1:3vNe/fWF5CBgRIguda1meWhsZHy3m8gCJ5wx+dIzX/E= github.com/hashicorp/go-version v1.2.0/go.mod h1:fltr4n8CU8Ke44wwGCBoEymUuxUHl09ZGVZPK5anwXA= +github.com/hashicorp/go.net v0.0.1 h1:sNCoNyDEvN1xa+X0baata4RdcpKwcMS6DH+xwfqPgjw= github.com/hashicorp/go.net v0.0.1/go.mod h1:hjKkEWcCURg++eb33jQU7oqQcI9XDCnUzHA0oac0k90= github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= +github.com/hashicorp/golang-lru v0.5.1 h1:0hERBMJE1eitiLkihrMvRVBYAkpHzc/J3QdDN+dAcgU= github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= github.com/hashicorp/golang-lru v0.5.4/go.mod h1:iADmTwqILo4mZ8BN3D2Q6+9jd8WM5uGBxy+E8yxSoD4= +github.com/hashicorp/logutils v1.0.0 h1:dLEQVugN8vlakKOUE3ihGLTZJRB4j+M2cdTm/ORI65Y= github.com/hashicorp/logutils v1.0.0/go.mod h1:QIAnNjmIWmVIIkWDTG1z5v++HQmx9WQRO+LraFDTW64= +github.com/hashicorp/mdns v1.0.0 h1:WhIgCr5a7AaVH6jPUwjtRuuE7/RDufnUvzIr48smyxs= github.com/hashicorp/mdns v1.0.0/go.mod h1:tL+uN++7HEJ6SQLQ2/p+z2pH24WQKWjBPkE0mNTz8vQ= +github.com/hashicorp/memberlist v0.1.3 h1:EmmoJme1matNzb+hMpDuR/0sbJSUisxyqBGG676r31M= github.com/hashicorp/memberlist v0.1.3/go.mod h1:ajVTdAv/9Im8oMAAj5G31PhhMCZJV2pPBoIllUwCN7I= +github.com/hashicorp/serf v0.8.2 h1:YZ7UKsJv+hKjqGVUUbtE3HNj79Eln2oQ75tniF6iPt0= github.com/hashicorp/serf v0.8.2/go.mod h1:6hOLApaqBFA1NXqRQAsxw9QxuDEvNxSQRwA/JwenrHc= +github.com/hpcloud/tail v1.0.0 h1:nfCOvKYfkgYP8hkirhJocXT2+zOD8yUNjXaWfTlyFKI= github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= github.com/huandu/xstrings v1.0.0/go.mod h1:4qWG/gcEcfX4z/mBDHJ++3ReCw9ibxbsNJbcucJdbSo= github.com/huandu/xstrings v1.2.0/go.mod h1:DvyZB1rfVYsBIigL8HwpZgxHwXozlTgGqn63UyNX5k4= @@ -331,124 +446,191 @@ github.com/huandu/xstrings v1.3.0/go.mod h1:y5/lhBue+AyNmUVz9RLU9xbLR0o4KIIExikq github.com/huandu/xstrings v1.3.1/go.mod h1:y5/lhBue+AyNmUVz9RLU9xbLR0o4KIIExikq4ovT0aE= github.com/huandu/xstrings v1.3.2 h1:L18LIDzqlW6xN2rEkpdV8+oL/IXWJ1APd+vsdYy4Wdw= github.com/huandu/xstrings v1.3.2/go.mod h1:y5/lhBue+AyNmUVz9RLU9xbLR0o4KIIExikq4ovT0aE= +github.com/hudl/fargo v1.3.0 h1:0U6+BtN6LhaYuTnIJq4Wyq5cpn6O2kWrxAtcqBmYY6w= github.com/hudl/fargo v1.3.0/go.mod h1:y3CKSmjA+wD2gak7sUSXTAoopbhU08POFhmITJgmKTg= +github.com/inconshreveable/mousetrap v1.0.0 h1:Z8tu5sraLXCXIcARxBp/8cbvlwVa7Z1NHg9XEKhtSvM= github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= +github.com/influxdata/influxdb1-client v0.0.0-20191209144304-8bf82d3c094d h1:/WZQPMZNsjZ7IlCpsLGdQBINg5bxKQ1K1sh6awxLtkA= github.com/influxdata/influxdb1-client v0.0.0-20191209144304-8bf82d3c094d/go.mod h1:qj24IKcXYK6Iy9ceXlo3Tc+vtHo9lIhSX5JddghvEPo= +github.com/jellevandenhooff/dkim v0.0.0-20150330215556-f50fe3d243e1 h1:ujPKutqRlJtcfWk6toYVYagwra7HQHbXOaS171b4Tg8= github.com/jellevandenhooff/dkim v0.0.0-20150330215556-f50fe3d243e1/go.mod h1:E0B/fFc00Y+Rasa88328GlI/XbtyysCtTHZS8h7IrBU= github.com/jessevdk/go-flags v1.4.0 h1:4IU2WS7AumrZ/40jfhf4QVDMsQwqA7VEHozFRrGARJA= github.com/jessevdk/go-flags v1.4.0/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= +github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af h1:pmfjZENx5imkbgOkpRUYLnmbU7UEFbjtDA2hxJ1ichM= github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af/go.mod h1:Nht3zPeWKUH0NzdCt2Blrr5ys8VGpn0CEB0cQHVjt7k= +github.com/jonboulle/clockwork v0.1.0 h1:VKV+ZcuP6l3yW9doeqz6ziZGgcynBVQO+obU0+0hcPo= github.com/jonboulle/clockwork v0.1.0/go.mod h1:Ii8DK3G1RaLaWxj9trq07+26W01tbo22gdxWY5EU2bo= +github.com/jpillora/backoff v1.0.0 h1:uvFg412JmmHBHw7iwprIxkPMI+sGQ4kzOWsMeHnm2EA= github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4= github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= github.com/json-iterator/go v1.1.7/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/json-iterator/go v1.1.8/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/json-iterator/go v1.1.9/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= +github.com/json-iterator/go v1.1.10 h1:Kz6Cvnvv2wGdaG/V8yMvfkmNiXq9Ya2KUv4rouJJr68= github.com/json-iterator/go v1.1.10/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= +github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024 h1:rBMNdlhTLzJjJSDIjNEXX1Pz3Hmwmz91v+zycvx9PJc= github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= github.com/jtolds/gls v4.2.1+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= +github.com/jtolds/gls v4.20.0+incompatible h1:xdiiI2gbIgH/gLH7ADydsJ1uDOEzR8yvV7C0MuV77Wo= github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= +github.com/julienschmidt/httprouter v1.3.0 h1:U0609e9tgbseu3rBINet9P48AI/D3oJs4dN7jwJOQ1U= github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8IZAc4RVcycCCAKdM= +github.com/kisielk/errcheck v1.1.0 h1:ZqfnKyx9KGpRcW04j5nnPDgRgoXUeLh2YFBeFzphcA0= github.com/kisielk/errcheck v1.1.0/go.mod h1:EZBBE59ingxPouuu3KfxchcWSUPOHkagtvWXihfKN4Q= +github.com/kisielk/gotool v1.0.0 h1:AV2c/EiW3KqPNT9ZKl07ehoAGi4C5/01Cfbblndcapg= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= +github.com/konsorten/go-windows-terminal-sequences v1.0.3 h1:CE8S1cTafDpPvMhIxNJKvHsGVBgn1xWYf1NbHQhywc8= github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= +github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515 h1:T+h1c/A9Gawja4Y9mFVWj2vyii2bbUNDw3kt9VxK2EY= github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= github.com/kr/pretty v0.2.0/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= github.com/kr/pretty v0.2.1 h1:Fmg33tUaq4/8ym9TJN1x7sLJnHVwhP33CNkpYV/7rwI= github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= +github.com/kr/pty v1.1.3 h1:/Um6a/ZmD5tF7peoOJ5oN5KMQ0DrGVQSXLNwyckutPk= github.com/kr/pty v1.1.3/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0 h1:45sCR5RtlFHMR4UwH9sdQ5TC8v0qDQCHnXt+kaKSTVE= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= +github.com/lightstep/lightstep-tracer-common/golang/gogo v0.0.0-20190605223551-bc2310a04743 h1:143Bb8f8DuGWck/xpNUOckBVYfFbBTnLevfRZ1aVVqo= github.com/lightstep/lightstep-tracer-common/golang/gogo v0.0.0-20190605223551-bc2310a04743/go.mod h1:qklhhLq1aX+mtWk9cPHPzaBjWImj5ULL6C7HFJtXQMM= +github.com/lightstep/lightstep-tracer-go v0.18.1 h1:vi1F1IQ8N7hNWytK9DpJsUfQhGuNSc19z330K6vl4zk= github.com/lightstep/lightstep-tracer-go v0.18.1/go.mod h1:jlF1pusYV4pidLvZ+XD0UBX0ZE6WURAspgAczcDHrL4= github.com/lucas-clemente/quic-go v0.7.1-0.20190401152353-907071221cf9/go.mod h1:PpMmPfPKO9nKJ/psF49ESTAGQSdfXxlg1otPbEB2nOw= github.com/lucas-clemente/quic-go v0.18.0/go.mod h1:yXttHsSNxQi8AWijC/vLP+OJczXqzHSOcJrM5ITUlCg= +github.com/lucas-clemente/quic-go v0.19.3 h1:eCDQqvGBB+kCTkA0XrAFtNe81FMa0/fn4QSoeAbmiF4= github.com/lucas-clemente/quic-go v0.19.3/go.mod h1:ADXpNbTQjq1hIzCpB+y/k5iz4n4z4IwqoLb94Kh5Hu8= +github.com/lukechampine/stm v0.0.0-20191022212748-05486c32d236 h1:kYGljja/OYeRs672gWqkHNMDVAtLN8/0UZpJNDCQ3s4= github.com/lukechampine/stm v0.0.0-20191022212748-05486c32d236/go.mod h1:wTLsd5FC9rts7GkMpsPGk64CIuea+03yaLAp19Jmlg8= +github.com/lunixbochs/vtclean v1.0.0 h1:xu2sLAri4lGiovBDQKxl5mrXyESr3gUr5m5SM5+LVb8= github.com/lunixbochs/vtclean v1.0.0/go.mod h1:pHhQNgMf3btfWnGBVipUOjRYhoOsdGqdm/+2c2E2WMI= +github.com/lyft/protoc-gen-validate v0.0.13 h1:KNt/RhmQTOLr7Aj8PsJ7mTronaFyx80mRTT9qF261dA= github.com/lyft/protoc-gen-validate v0.0.13/go.mod h1:XbGvPuh87YZc5TdIa2/I4pLk0QoUACkjt2znoq26NVQ= +github.com/mailru/easyjson v0.0.0-20190312143242-1de009706dbe h1:W/GaMY0y69G4cFlmsC6B9sbuo2fP8OFP1ABjt4kPz+w= github.com/mailru/easyjson v0.0.0-20190312143242-1de009706dbe/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= github.com/marten-seemann/qpack v0.2.0/go.mod h1:F7Gl5L1jIgN1D11ucXefiuJS9UMVP2opoCp2jDKb7wc= +github.com/marten-seemann/qpack v0.2.1 h1:jvTsT/HpCn2UZJdP+UUB53FfUUgeOyG5K1ns0OJOGVs= github.com/marten-seemann/qpack v0.2.1/go.mod h1:F7Gl5L1jIgN1D11ucXefiuJS9UMVP2opoCp2jDKb7wc= github.com/marten-seemann/qtls v0.2.3/go.mod h1:xzjG7avBwGGbdZ8dTGxlBnLArsVKLvwmjgmPuiQEcYk= +github.com/marten-seemann/qtls v0.10.0 h1:ECsuYUKalRL240rRD4Ri33ISb7kAQ3qGDlrrl55b2pc= github.com/marten-seemann/qtls v0.10.0/go.mod h1:UvMd1oaYDACI99/oZUYLzMCkBXQVT0aGm99sJhbT8hs= github.com/marten-seemann/qtls-go1-15 v0.1.0/go.mod h1:GyFwywLKkRt+6mfU99csTEY1joMZz5vmB1WNZH3P81I= +github.com/marten-seemann/qtls-go1-15 v0.1.1 h1:LIH6K34bPVttyXnUWixk0bzH6/N07VxbSabxn5A5gZQ= github.com/marten-seemann/qtls-go1-15 v0.1.1/go.mod h1:GyFwywLKkRt+6mfU99csTEY1joMZz5vmB1WNZH3P81I= +github.com/mattn/go-colorable v0.0.9 h1:UVL0vNpWh04HeJXV0KLcaT7r06gOH2l4OW6ddYRUIY4= github.com/mattn/go-colorable v0.0.9/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU= github.com/mattn/go-isatty v0.0.3/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4= github.com/mattn/go-isatty v0.0.4/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4= github.com/mattn/go-isatty v0.0.7/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s= +github.com/mattn/go-isatty v0.0.10 h1:qxFzApOv4WsAL965uUPIsXzAKCZxN2p9UqdhFS4ZW10= github.com/mattn/go-isatty v0.0.10/go.mod h1:qgIWMr58cqv1PHHyhnkY9lrL7etaEgOFcMEpPG5Rm84= +github.com/mattn/go-runewidth v0.0.2 h1:UnlwIPBGaTZfPQ6T1IGzPI0EkYAQmT9fAEJ/poFC63o= github.com/mattn/go-runewidth v0.0.2/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU= github.com/mattn/go-sqlite3 v1.7.0/go.mod h1:FPy6KqzDD04eiIsT53CuJW3U88zkxoIYsOqkbpncsNc= github.com/mattn/go-sqlite3 v1.10.0/go.mod h1:FPy6KqzDD04eiIsT53CuJW3U88zkxoIYsOqkbpncsNc= github.com/mattn/go-sqlite3 v1.13.0/go.mod h1:FPy6KqzDD04eiIsT53CuJW3U88zkxoIYsOqkbpncsNc= +github.com/mattn/go-sqlite3 v2.0.2+incompatible h1:qzw9c2GNT8UFrgWNDhCTqRqYUSmu/Dav/9Z58LGpk7U= github.com/mattn/go-sqlite3 v2.0.2+incompatible/go.mod h1:FPy6KqzDD04eiIsT53CuJW3U88zkxoIYsOqkbpncsNc= +github.com/matttproud/golang_protobuf_extensions v1.0.1 h1:4hp9jkHxhMHkqkrB3Ix0jegS5sx/RkqARlsWZ6pIwiU= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= +github.com/microcosm-cc/bluemonday v1.0.1 h1:SIYunPjnlXcW+gVfvm0IlSeR5U3WZUOLfVmqg85Go44= github.com/microcosm-cc/bluemonday v1.0.1/go.mod h1:hsXNsILzKxV+sX77C5b8FSuKF00vh2OMYv+xgHpAMF4= +github.com/miekg/dns v1.0.14 h1:9jZdLNd/P4+SfEJ0TNyxYpsK8N4GtfylBLqtbYN1sbA= github.com/miekg/dns v1.0.14/go.mod h1:W1PPwlIAgtquWBMBEV9nkV9Cazfe8ScdGz/Lj7v3Nrg= +github.com/mitchellh/cli v1.0.0 h1:iGBIsUe3+HZ/AD/Vd7DErOt5sU9fa8Uj7A2s1aggv1Y= github.com/mitchellh/cli v1.0.0/go.mod h1:hNIlj7HEI86fIcpObd7a0FcrxTWetlwJDGcceTlRvqc= +github.com/mitchellh/go-homedir v1.0.0 h1:vKb8ShqSby24Yrqr/yDYkuFz8d0WUjys40rvnGC8aR0= github.com/mitchellh/go-homedir v1.0.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= +github.com/mitchellh/go-testing-interface v1.0.0 h1:fzU/JVNcaqHQEcVFAKeR41fkiLdIPrefOvVG1VZ96U0= github.com/mitchellh/go-testing-interface v1.0.0/go.mod h1:kRemZodwjscx+RGhAo8eIhFbs2+BFgRtFPeD/KE+zxI= +github.com/mitchellh/gox v0.4.0 h1:lfGJxY7ToLJQjHHwi0EX6uYBdK78egf954SQl13PQJc= github.com/mitchellh/gox v0.4.0/go.mod h1:Sd9lOJ0+aimLBi73mGofS1ycjY8lL3uZM3JPS42BGNg= +github.com/mitchellh/iochan v1.0.0 h1:C+X3KsSTLFVBr/tK1eYN/vs4rJcvsiLU338UhYPJWeY= github.com/mitchellh/iochan v1.0.0/go.mod h1:JwYml1nuB7xOzsp52dPpHFffvOCDupsG0QubkSMEySY= github.com/mitchellh/mapstructure v0.0.0-20160808181253-ca63d7c062ee/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= +github.com/mitchellh/mapstructure v1.1.2 h1:fmNYVwqnSfB9mZU6OS2O6GsXM+wcskZDuKQzvN1EDeE= github.com/mitchellh/mapstructure v1.1.2/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= +github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= +github.com/modern-go/reflect2 v1.0.1 h1:9f412s+6RmYXLWZSEzVVgPGK7C2PphHj5RJrvfx9AWI= github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/mschoch/smat v0.0.0-20160514031455-90eadee771ae/go.mod h1:qAyveg+e4CE+eKJXWVjKXM4ck2QobLqTDytGJbLLhJg= github.com/mschoch/smat v0.2.0 h1:8imxQsjDm8yFEAVBe7azKmKSgzSkZXDuKkSq9374khM= github.com/mschoch/smat v0.2.0/go.mod h1:kc9mz7DoBKqDyiRL7VZN8KvXQMWeTaVnttLRXOlotKw= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= +github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f h1:KUppIJq7/+SVif2QVs3tOP0zanoHgBEVAwHxUSIzRqU= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/nats-io/jwt v0.3.0/go.mod h1:fRYCDE99xlTsqUzISS1Bi75UBJ6ljOJQOAAu5VglpSg= +github.com/nats-io/jwt v0.3.2 h1:+RB5hMpXUUA2dfxuhBTEkMOrYmM+gKIZYS1KjSostMI= github.com/nats-io/jwt v0.3.2/go.mod h1:/euKqTS1ZD+zzjYrY7pseZrTtWQSjujC7xjPc8wL6eU= +github.com/nats-io/nats-server/v2 v2.1.2 h1:i2Ly0B+1+rzNZHHWtD4ZwKi+OU5l+uQo1iDHZ2PmiIc= github.com/nats-io/nats-server/v2 v2.1.2/go.mod h1:Afk+wRZqkMQs/p45uXdrVLuab3gwv3Z8C4HTBu8GD/k= +github.com/nats-io/nats.go v1.9.1 h1:ik3HbLhZ0YABLto7iX80pZLPw/6dx3T+++MZJwLnMrQ= github.com/nats-io/nats.go v1.9.1/go.mod h1:ZjDU1L/7fJ09jvUSRVBR2e7+RnLiiIQyqyzEE/Zbp4w= github.com/nats-io/nkeys v0.1.0/go.mod h1:xpnFELMwJABBLVhffcfd1MZx6VsNRFpEugbxziKVo7w= +github.com/nats-io/nkeys v0.1.3 h1:6JrEfig+HzTH85yxzhSVbjHRJv9cn0p6n3IngIcM5/k= github.com/nats-io/nkeys v0.1.3/go.mod h1:xpnFELMwJABBLVhffcfd1MZx6VsNRFpEugbxziKVo7w= +github.com/nats-io/nuid v1.0.1 h1:5iA8DT8V7q8WK2EScv2padNa/rTESc1KdnPw4TC2paw= github.com/nats-io/nuid v1.0.1/go.mod h1:19wcPz3Ph3q0Jbyiqsd0kePYG7A95tJPxeL+1OSON2c= +github.com/neelance/astrewrite v0.0.0-20160511093645-99348263ae86 h1:D6paGObi5Wud7xg83MaEFyjxQB1W5bz5d0IFppr+ymk= github.com/neelance/astrewrite v0.0.0-20160511093645-99348263ae86/go.mod h1:kHJEU3ofeGjhHklVoIGuVj85JJwZ6kWPaJwCIxgnFmo= +github.com/neelance/sourcemap v0.0.0-20151028013722-8c68805598ab h1:eFXv9Nu1lGbrNbj619aWwZfVF5HBrm9Plte8aNptuTI= github.com/neelance/sourcemap v0.0.0-20151028013722-8c68805598ab/go.mod h1:Qr6/a/Q4r9LP1IltGz7tA7iOK1WonHEYhu1HRBA7ZiM= github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A= +github.com/nxadm/tail v1.4.8 h1:nPr65rt6Y5JFSKQO7qToXr7pePgD6Gwiw05lkbyAQTE= github.com/nxadm/tail v1.4.8/go.mod h1:+ncqLTQzXmGhMZNUePPaPqPvBxHAIsmXswZKocGu+AU= +github.com/oklog/oklog v0.3.2 h1:wVfs8F+in6nTBMkA7CbRw+zZMIB7nNM825cM1wuzoTk= github.com/oklog/oklog v0.3.2/go.mod h1:FCV+B7mhrz4o+ueLpx+KqkyXRGMWOYEvfiXtdGtbWGs= +github.com/oklog/run v1.0.0 h1:Ru7dDtJNOyC66gQ5dQmaCa0qIsAUFY3sFpK1Xk8igrw= github.com/oklog/run v1.0.0/go.mod h1:dlhp/R75TPv97u0XWUtDeV/lRKWPKSdTuV0TZvrmrQA= +github.com/olekukonko/tablewriter v0.0.0-20170122224234-a0225b3f23b5 h1:58+kh9C6jJVXYjt8IE48G2eWl6BjwU5Gj0gqY84fy78= github.com/olekukonko/tablewriter v0.0.0-20170122224234-a0225b3f23b5/go.mod h1:vsDQFd/mU46D+Z4whnwzcISnGGzXWMclvtLoiIKAKIo= github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v1.7.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v1.12.1/go.mod h1:zj2OWP4+oCPe1qIXoGWkgMRwljMUYCdkwsT2108oapk= github.com/onsi/ginkgo v1.14.0/go.mod h1:iSB4RoI2tjJc9BBv4NKIKWKya62Rps+oPG/Lv9klQyY= github.com/onsi/ginkgo v1.14.2/go.mod h1:iSB4RoI2tjJc9BBv4NKIKWKya62Rps+oPG/Lv9klQyY= +github.com/onsi/ginkgo v1.16.1 h1:foqVmeWDD6yYpK+Yz3fHyNIxFYNxswxqNFjSKe+vI54= github.com/onsi/ginkgo v1.16.1/go.mod h1:CObGmKUOKaSC0RjmoAK7tKyn4Azo5P2IWuoMnvwxz1E= github.com/onsi/gomega v1.4.3/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY= github.com/onsi/gomega v1.7.1/go.mod h1:XdKZgCCFLUoM/7CFJVPcG8C1xQ1AJ0vpAezJrB7JYyY= github.com/onsi/gomega v1.10.1/go.mod h1:iN09h71vgCQne3DLsj+A5owkum+a2tYe+TOCB1ybHNo= github.com/onsi/gomega v1.10.3/go.mod h1:V9xEwhxec5O8UDM77eCW8vLymOMltsqPVYWrpDsH8xc= +github.com/onsi/gomega v1.11.0 h1:+CqWgvj0OZycCaqclBD1pxKHAU+tOkHmQIWvDHq2aug= github.com/onsi/gomega v1.11.0/go.mod h1:azGKhqFUon9Vuj0YmTfLSmx0FUwqXYSTl5re8lQLTUg= +github.com/op/go-logging v0.0.0-20160315200505-970db520ece7 h1:lDH9UUVJtmYCjyT0CI4q8xvlXPxeZ0gYCVvWbmPlp88= github.com/op/go-logging v0.0.0-20160315200505-970db520ece7/go.mod h1:HzydrMdWErDVzsI23lYNej1Htcns9BCg93Dk0bBINWk= +github.com/opentracing-contrib/go-observer v0.0.0-20170622124052-a52f23424492 h1:lM6RxxfUMrYL/f8bWEUqdXrANWtrL7Nndbm9iFN0DlU= github.com/opentracing-contrib/go-observer v0.0.0-20170622124052-a52f23424492/go.mod h1:Ngi6UdF0k5OKD5t5wlmGhe/EDKPoUM3BXZSSfIuJbis= +github.com/opentracing/basictracer-go v1.0.0 h1:YyUAhaEfjoWXclZVJ9sGoNct7j4TVk7lZWlQw5UXuoo= github.com/opentracing/basictracer-go v1.0.0/go.mod h1:QfBfYuafItcjQuMwinw9GhYKwFXS9KnPs5lxoYwgW74= github.com/opentracing/opentracing-go v1.0.2/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= +github.com/opentracing/opentracing-go v1.1.0 h1:pWlfV3Bxv7k65HYwkikxat0+s3pV4bsqf19k25Ur8rU= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= +github.com/openzipkin-contrib/zipkin-go-opentracing v0.4.5 h1:ZCnq+JUrvXcDVhX/xRolRBZifmabN1HcS1wrPSvxhrU= github.com/openzipkin-contrib/zipkin-go-opentracing v0.4.5/go.mod h1:/wsWhb9smxSfWAKL3wpBW7V8scJMt8N8gnaMCS9E/cA= github.com/openzipkin/zipkin-go v0.1.1/go.mod h1:NtoC/o8u3JlF1lSlyPNswIbeQH9bJTmOf0Erfk+hxe8= github.com/openzipkin/zipkin-go v0.1.6/go.mod h1:QgAqvLzwWbR/WpD4A3cGpPtJrZXNIiJc5AZX7/PBEpw= github.com/openzipkin/zipkin-go v0.2.1/go.mod h1:NaW6tEwdmWMaCDZzg8sh+IBNOxHMPnhQw8ySjnjRyN4= +github.com/openzipkin/zipkin-go v0.2.2 h1:nY8Hti+WKaP0cRsSeQ026wU03QsM762XBeCXBb9NAWI= github.com/openzipkin/zipkin-go v0.2.2/go.mod h1:NaW6tEwdmWMaCDZzg8sh+IBNOxHMPnhQw8ySjnjRyN4= +github.com/pact-foundation/pact-go v1.0.4 h1:OYkFijGHoZAYbOIb1LWXrwKQbMMRUv1oQ89blD2Mh2Q= github.com/pact-foundation/pact-go v1.0.4/go.mod h1:uExwJY4kCzNPcHRj+hCR/HBbOOIwwtUjcrb0b5/5kLM= +github.com/pascaldekloe/goe v0.0.0-20180627143212-57f6aae5913c h1:Lgl0gzECD8GnQ5QCWA8o6BtfL6mDH5rQgM4/fX3avOs= github.com/pascaldekloe/goe v0.0.0-20180627143212-57f6aae5913c/go.mod h1:lzWF7FIEvWOWxwDKqyGYQf6ZUaNfKdP144TG7ZOy1lc= +github.com/pborman/uuid v1.2.0 h1:J7Q5mO4ysT1dv8hyrUGHb9+ooztCXu1D8MY8DZYsu3g= github.com/pborman/uuid v1.2.0/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= +github.com/performancecopilot/speed v3.0.0+incompatible h1:2WnRzIquHa5QxaJKShDkLM+sc0JPuwhXzK8OYOyt3Vg= github.com/performancecopilot/speed v3.0.0+incompatible/go.mod h1:/CLtqpZ5gBg1M9iaPbIdPPGyKcA8hKdoy6hAWba7Yac= github.com/philhofer/fwd v1.0.0/go.mod h1:gk3iGcWd9+svBvR0sR+KPcfE+RNWozjowpeBVG3ZVNU= +github.com/philhofer/fwd v1.1.1 h1:GdGcTjf5RNAxwS4QLsiMzJYj5KEvPJD3Abr261yRQXQ= github.com/philhofer/fwd v1.1.1/go.mod h1:gk3iGcWd9+svBvR0sR+KPcfE+RNWozjowpeBVG3ZVNU= github.com/pierrec/lz4 v1.0.2-0.20190131084431-473cd7ce01a1/go.mod h1:3/3N9NVKO0jef7pBehbT1qWhCMrIgbYNnFAZCqQ5LRc= +github.com/pierrec/lz4 v2.0.5+incompatible h1:2xWsjqPFWcplujydGg4WmhC/6fZqK42wMM8aXeqhl0I= github.com/pierrec/lz4 v2.0.5+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= github.com/pion/datachannel v1.4.21 h1:3ZvhNyfmxsAqltQrApLPQMhSFNA+aT87RqyCq4OXmf0= github.com/pion/datachannel v1.4.21/go.mod h1:oiNyP4gHx2DIwRzX/MFyH0Rz/Gz05OgBlayAI2hAWjg= @@ -472,6 +654,7 @@ github.com/pion/mdns v0.0.4/go.mod h1:R1sL0p50l42S5lJs91oNdUL58nm0QHrhxnSegr++qC github.com/pion/mdns v0.0.5 h1:Q2oj/JB3NqfzY9xGZ1fPzZzK7sDSD8rZPOvcIQ10BCw= github.com/pion/mdns v0.0.5/go.mod h1:UgssrvdD3mxpi8tMxAXbsppL3vJ4Jipw1mTCW+al01g= github.com/pion/quic v0.1.1/go.mod h1:zEU51v7ru8Mp4AUBJvj6psrSth5eEFNnVQK5K48oV3k= +github.com/pion/quic v0.1.4 h1:bNz9sCJjlM3GqMdq7Fne57FiWfdyiJ++yHVbuqeoD3Y= github.com/pion/quic v0.1.4/go.mod h1:dBhNvkLoQqRwfi6h3Vqj3IcPLgiW7rkZxBbRdp7Vzvk= github.com/pion/randutil v0.0.0/go.mod h1:XcJrSMMbbMRhASFVOlj/5hQial/Y8oH/HVo7TBZq+j8= github.com/pion/randutil v0.1.0 h1:CFG1UdESneORglEsnimhUjf33Rwjubwj6xfiOXBa3mA= @@ -489,6 +672,7 @@ github.com/pion/sctp v1.7.10/go.mod h1:EhpTUQu1/lcK3xI+eriS6/96fWetHGCvBi9MSsnaB github.com/pion/sctp v1.7.11/go.mod h1:EhpTUQu1/lcK3xI+eriS6/96fWetHGCvBi9MSsnaBN0= github.com/pion/sctp v1.7.12 h1:GsatLufywVruXbZZT1CKg+Jr8ZTkwiPnmUC/oO9+uuY= github.com/pion/sctp v1.7.12/go.mod h1:xFe9cLMZ5Vj6eOzpyiKjT9SwGM4KpK/8Jbw5//jc+0s= +github.com/pion/sdp/v2 v2.4.0 h1:luUtaETR5x2KNNpvEMv/r4Y+/kzImzbz4Lm1z8eQNQI= github.com/pion/sdp/v2 v2.4.0/go.mod h1:L2LxrOpSTJbAns244vfPChbciR/ReU1KWfG04OpkR7E= github.com/pion/sdp/v3 v3.0.4 h1:2Kf+dgrzJflNCSw3TV5v2VLeI0s/qkzy2r5jlR0wzf8= github.com/pion/sdp/v3 v3.0.4/go.mod h1:bNiSknmJE0HYBprTHXKPQ3+JjacTv5uap92ueJZKsRk= @@ -514,6 +698,7 @@ github.com/pion/turn/v2 v2.0.5/go.mod h1:APg43CFyt/14Uy7heYUOGWdkem/Wu4PhCO/bjyr github.com/pion/udp v0.1.0/go.mod h1:BPELIjbwE9PRbd/zxI/KYBnbo7B6+oA6YuEaNE8lths= github.com/pion/udp v0.1.1 h1:8UAPvyqmsxK8oOjloDk4wUt63TzFe9WEJkg5lChlj7o= github.com/pion/udp v0.1.1/go.mod h1:6AFo+CMdKQm7UiA0eUPA8/eVCTx8jBIITLZHc9DWX5M= +github.com/pion/webrtc/v2 v2.2.26 h1:01hWE26pL3LgqfxvQ1fr6O4ZtyRFFJmQEZK39pHWfFc= github.com/pion/webrtc/v2 v2.2.26/go.mod h1:XMZbZRNHyPDe1gzTIHFcQu02283YO45CbiwFgKvXnmc= github.com/pion/webrtc/v3 v3.0.11/go.mod h1:WEvXneGTeqNmiR59v5jTsxMc4yXQyOQcRsrdAbNwSEU= github.com/pion/webrtc/v3 v3.0.27 h1:cPQEFNFrRSMT11j9c9aTmXzL3ikKAFPE2kR0ZrQcviw= @@ -522,9 +707,11 @@ github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINE github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= +github.com/pkg/profile v1.2.1 h1:F++O52m40owAmADcojzM+9gyjmMOY/T4oYJkgFDH8RE= github.com/pkg/profile v1.2.1/go.mod h1:hJw3o1OdXxsrSjjVksARp5W95eeEaEfptyVZyv6JUPA= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= +github.com/posener/complete v1.1.1 h1:ccV59UEOTzVDnDUEFdT95ZzHVZ+5+158q8+SJb2QV5w= github.com/posener/complete v1.1.1/go.mod h1:em0nMJCgc9GFtwrmVmEMR/ZL6WyhyjMBndrE9hABlRI= github.com/prometheus/client_golang v0.8.0/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= @@ -533,12 +720,14 @@ github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5Fsn github.com/prometheus/client_golang v1.3.0/go.mod h1:hJaj2vgQTGQmVCsAACORcieXFeDPbaTKGT+JTgUa3og= github.com/prometheus/client_golang v1.5.1/go.mod h1:e9GMxYsXl05ICDXkRhurwBS4Q3OK1iX/F2sw+iXX5zU= github.com/prometheus/client_golang v1.7.1/go.mod h1:PY5Wy2awLA44sXw4AOSfFBetzPP4j5+D6mVACh+pe2M= +github.com/prometheus/client_golang v1.9.0 h1:Rrch9mh17XcxvEu9D9DEpb4isxjGBtcevQjKvxPRQIU= github.com/prometheus/client_golang v1.9.0/go.mod h1:FqZLKOZnGdFAhOK4nqGHa7D66IdsO+O441Eve7ptJDU= github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= github.com/prometheus/client_model v0.0.0-20190115171406-56726106282f/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.1.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= +github.com/prometheus/client_model v0.2.0 h1:uq5h0d+GuxiXLJLNABMgp2qUWDPiLvgCzz2dUR+/W/M= github.com/prometheus/client_model v0.2.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/common v0.0.0-20180801064454-c7de2306084e/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro= github.com/prometheus/common v0.2.0/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= @@ -546,6 +735,7 @@ github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y8 github.com/prometheus/common v0.7.0/go.mod h1:DjGbpBbp5NYNiECxcL/VnbXCCaQpKd3tt26CguLLsqA= github.com/prometheus/common v0.9.1/go.mod h1:yhUN8i9wzaXS3w1O07YhxHEBxD+W35wd8bs7vj7HSQ4= github.com/prometheus/common v0.10.0/go.mod h1:Tlit/dnDKsSWFlCLTWaA1cyBgKHSMdTB80sz/V91rCo= +github.com/prometheus/common v0.15.0 h1:4fgOnadei3EZvgRwxJ7RMpG1k1pOZth5Pc13tyspaKM= github.com/prometheus/common v0.15.0/go.mod h1:U+gB1OBLb1lF3O42bTCL+FK18tX9Oar16Clt/msog/s= github.com/prometheus/procfs v0.0.0-20180725123919-05ee40e3a273/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= @@ -555,66 +745,111 @@ github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+Gx github.com/prometheus/procfs v0.0.11/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= github.com/prometheus/procfs v0.1.3/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= github.com/prometheus/procfs v0.2.0/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= +github.com/prometheus/procfs v0.3.0 h1:Uehi/mxLK0eiUc0H0++5tpMGTexB8wZ598MIgU8VpDM= github.com/prometheus/procfs v0.3.0/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= +github.com/rcrowley/go-metrics v0.0.0-20181016184325-3113b8401b8a h1:9ZKAASQSHhDYGoxY8uLVpewe1GDZ2vu2Tr/vTdVAkFQ= github.com/rcrowley/go-metrics v0.0.0-20181016184325-3113b8401b8a/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= +github.com/rogpeppe/fastuuid v0.0.0-20150106093220-6724a57986af h1:gu+uRPtBe88sKxUCEXRoeCvVG90TJmwhiqRpvdhQFng= github.com/rogpeppe/fastuuid v0.0.0-20150106093220-6724a57986af/go.mod h1:XWv6SoW27p1b0cqNHllgS5HIMJraePCO15w5zCzIWYg= +github.com/rogpeppe/go-internal v1.3.0 h1:RR9dF3JtopPvtkroDZuVD7qquD0bnHlKSqaQhgwt8yk= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/rs/dnscache v0.0.0-20190621150935-06bb5526f76b/go.mod h1:qe5TWALJ8/a1Lqznoc5BDHpYX/8HU60Hm2AwRmqzxqA= github.com/rs/dnscache v0.0.0-20210201191234-295bba877686 h1:IJ6Df0uxPDtNoByV0KkzVKNseWvZFCNM/S9UoyOMCSI= github.com/rs/dnscache v0.0.0-20210201191234-295bba877686/go.mod h1:qe5TWALJ8/a1Lqznoc5BDHpYX/8HU60Hm2AwRmqzxqA= +github.com/russross/blackfriday v1.5.2 h1:HyvC0ARfnZBqnXwABFeSZHpKvJHJJfPz81GNueLj0oo= github.com/russross/blackfriday v1.5.2/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR/rfWxYHBV53g= +github.com/russross/blackfriday/v2 v2.0.1 h1:lPqVAte+HuHNfhJ/0LC98ESWRz8afy9tM/0RK8m9o+Q= github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= +github.com/ryanuber/columnize v0.0.0-20160712163229-9b3edd62028f h1:UFr9zpz4xgTnIE5yIMtWAMngCdZ9p/+q6lTbgelo80M= github.com/ryanuber/columnize v0.0.0-20160712163229-9b3edd62028f/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts= github.com/ryszard/goskiplist v0.0.0-20150312221310-2dfbae5fcf46 h1:GHRpF1pTW19a8tTFrMLUcfWwyC0pnifVo2ClaLq+hP8= github.com/ryszard/goskiplist v0.0.0-20150312221310-2dfbae5fcf46/go.mod h1:uAQ5PCi+MFsC7HjREoAz1BU+Mq60+05gifQSsHSDG/8= +github.com/samuel/go-zookeeper v0.0.0-20190923202752-2cc03de413da h1:p3Vo3i64TCLY7gIfzeQaUJ+kppEO5WQG3cL8iE8tGHU= github.com/samuel/go-zookeeper v0.0.0-20190923202752-2cc03de413da/go.mod h1:gi+0XIa01GRL2eRQVjQkKGqKF3SF9vZR/HnPullcV2E= +github.com/sclevine/agouti v3.0.0+incompatible h1:8IBJS6PWz3uTlMP3YBIR5f+KAldcGuOeFkFbUWfBgK4= github.com/sclevine/agouti v3.0.0+incompatible/go.mod h1:b4WX9W9L1sfQKXeJf1mUTLZKJ48R1S7H23Ji7oFO5Bw= +github.com/sean-/seed v0.0.0-20170313163322-e2103e2c3529 h1:nn5Wsu0esKSJiIVhscUtVbo7ada43DJhG55ua/hjS5I= github.com/sean-/seed v0.0.0-20170313163322-e2103e2c3529/go.mod h1:DxrIzT+xaE7yg65j358z/aeFdxmN0P9QXhEzd20vsDc= +github.com/sergi/go-diff v1.0.0 h1:Kpca3qRNrduNnOQeazBd0ysaKrUJiIuISHxogkT9RPQ= github.com/sergi/go-diff v1.0.0/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo= +github.com/shurcooL/component v0.0.0-20170202220835-f88ec8f54cc4 h1:Fth6mevc5rX7glNLpbAMJnqKlfIkcTjZCSHEeqvKbcI= github.com/shurcooL/component v0.0.0-20170202220835-f88ec8f54cc4/go.mod h1:XhFIlyj5a1fBNx5aJTbKoIq0mNaPvOagO+HjB3EtxrY= +github.com/shurcooL/events v0.0.0-20181021180414-410e4ca65f48 h1:vabduItPAIz9px5iryD5peyx7O3Ya8TBThapgXim98o= github.com/shurcooL/events v0.0.0-20181021180414-410e4ca65f48/go.mod h1:5u70Mqkb5O5cxEA8nxTsgrgLehJeAw6Oc4Ab1c/P1HM= +github.com/shurcooL/github_flavored_markdown v0.0.0-20181002035957-2122de532470 h1:qb9IthCFBmROJ6YBS31BEMeSYjOscSiG+EO+JVNTz64= github.com/shurcooL/github_flavored_markdown v0.0.0-20181002035957-2122de532470/go.mod h1:2dOwnU2uBioM+SGy2aZoq1f/Sd1l9OkAeAUvjSyvgU0= +github.com/shurcooL/go v0.0.0-20180423040247-9e1955d9fb6e h1:MZM7FHLqUHYI0Y/mQAt3d2aYa0SiNms/hFqC9qJYolM= github.com/shurcooL/go v0.0.0-20180423040247-9e1955d9fb6e/go.mod h1:TDJrrUr11Vxrven61rcy3hJMUqaf/CLWYhHNPmT14Lk= +github.com/shurcooL/go-goon v0.0.0-20170922171312-37c2f522c041 h1:llrF3Fs4018ePo4+G/HV/uQUqEI1HMDjCeOf2V6puPc= github.com/shurcooL/go-goon v0.0.0-20170922171312-37c2f522c041/go.mod h1:N5mDOmsrJOB+vfqUK+7DmDyjhSLIIBnXo9lvZJj3MWQ= +github.com/shurcooL/gofontwoff v0.0.0-20180329035133-29b52fc0a18d h1:Yoy/IzG4lULT6qZg62sVC+qyBL8DQkmD2zv6i7OImrc= github.com/shurcooL/gofontwoff v0.0.0-20180329035133-29b52fc0a18d/go.mod h1:05UtEgK5zq39gLST6uB0cf3NEHjETfB4Fgr3Gx5R9Vw= +github.com/shurcooL/gopherjslib v0.0.0-20160914041154-feb6d3990c2c h1:UOk+nlt1BJtTcH15CT7iNO7YVWTfTv/DNwEAQHLIaDQ= github.com/shurcooL/gopherjslib v0.0.0-20160914041154-feb6d3990c2c/go.mod h1:8d3azKNyqcHP1GaQE/c6dDgjkgSx2BZ4IoEi4F1reUI= +github.com/shurcooL/highlight_diff v0.0.0-20170515013008-09bb4053de1b h1:vYEG87HxbU6dXj5npkeulCS96Dtz5xg3jcfCgpcvbIw= github.com/shurcooL/highlight_diff v0.0.0-20170515013008-09bb4053de1b/go.mod h1:ZpfEhSmds4ytuByIcDnOLkTHGUI6KNqRNPDLHDk+mUU= +github.com/shurcooL/highlight_go v0.0.0-20181028180052-98c3abbbae20 h1:7pDq9pAMCQgRohFmd25X8hIH8VxmT3TaDm+r9LHxgBk= github.com/shurcooL/highlight_go v0.0.0-20181028180052-98c3abbbae20/go.mod h1:UDKB5a1T23gOMUJrI+uSuH0VRDStOiUVSjBTRDVBVag= +github.com/shurcooL/home v0.0.0-20181020052607-80b7ffcb30f9 h1:MPblCbqA5+z6XARjScMfz1TqtJC7TuTRj0U9VqIBs6k= github.com/shurcooL/home v0.0.0-20181020052607-80b7ffcb30f9/go.mod h1:+rgNQw2P9ARFAs37qieuu7ohDNQ3gds9msbT2yn85sg= +github.com/shurcooL/htmlg v0.0.0-20170918183704-d01228ac9e50 h1:crYRwvwjdVh1biHzzciFHe8DrZcYrVcZFlJtykhRctg= github.com/shurcooL/htmlg v0.0.0-20170918183704-d01228ac9e50/go.mod h1:zPn1wHpTIePGnXSHpsVPWEktKXHr6+SS6x/IKRb7cpw= +github.com/shurcooL/httperror v0.0.0-20170206035902-86b7830d14cc h1:eHRtZoIi6n9Wo1uR+RU44C247msLWwyA89hVKwRLkMk= github.com/shurcooL/httperror v0.0.0-20170206035902-86b7830d14cc/go.mod h1:aYMfkZ6DWSJPJ6c4Wwz3QtW22G7mf/PEgaB9k/ik5+Y= +github.com/shurcooL/httpfs v0.0.0-20171119174359-809beceb2371 h1:SWV2fHctRpRrp49VXJ6UZja7gU9QLHwRpIPBN89SKEo= github.com/shurcooL/httpfs v0.0.0-20171119174359-809beceb2371/go.mod h1:ZY1cvUeJuFPAdZ/B6v7RHavJWZn2YPVFQ1OSXhCGOkg= +github.com/shurcooL/httpgzip v0.0.0-20180522190206-b1c53ac65af9 h1:fxoFD0in0/CBzXoyNhMTjvBZYW6ilSnTw7N7y/8vkmM= github.com/shurcooL/httpgzip v0.0.0-20180522190206-b1c53ac65af9/go.mod h1:919LwcH0M7/W4fcZ0/jy0qGght1GIhqyS/EgWGH2j5Q= +github.com/shurcooL/issues v0.0.0-20181008053335-6292fdc1e191 h1:T4wuULTrzCKMFlg3HmKHgXAF8oStFb/+lOIupLV2v+o= github.com/shurcooL/issues v0.0.0-20181008053335-6292fdc1e191/go.mod h1:e2qWDig5bLteJ4fwvDAc2NHzqFEthkqn7aOZAOpj+PQ= +github.com/shurcooL/issuesapp v0.0.0-20180602232740-048589ce2241 h1:Y+TeIabU8sJD10Qwd/zMty2/LEaT9GNDaA6nyZf+jgo= github.com/shurcooL/issuesapp v0.0.0-20180602232740-048589ce2241/go.mod h1:NPpHK2TI7iSaM0buivtFUc9offApnI0Alt/K8hcHy0I= +github.com/shurcooL/notifications v0.0.0-20181007000457-627ab5aea122 h1:TQVQrsyNaimGwF7bIhzoVC9QkKm4KsWd8cECGzFx8gI= github.com/shurcooL/notifications v0.0.0-20181007000457-627ab5aea122/go.mod h1:b5uSkrEVM1jQUspwbixRBhaIjIzL2xazXp6kntxYle0= +github.com/shurcooL/octicon v0.0.0-20181028054416-fa4f57f9efb2 h1:bu666BQci+y4S0tVRVjsHUeRon6vUXmsGBwdowgMrg4= github.com/shurcooL/octicon v0.0.0-20181028054416-fa4f57f9efb2/go.mod h1:eWdoE5JD4R5UVWDucdOPg1g2fqQRq78IQa9zlOV1vpQ= +github.com/shurcooL/reactions v0.0.0-20181006231557-f2e0b4ca5b82 h1:LneqU9PHDsg/AkPDU3AkqMxnMYL+imaqkpflHu73us8= github.com/shurcooL/reactions v0.0.0-20181006231557-f2e0b4ca5b82/go.mod h1:TCR1lToEk4d2s07G3XGfz2QrgHXg4RJBvjrOozvoWfk= github.com/shurcooL/sanitized_anchor_name v0.0.0-20170918181015-86672fcb3f95/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= +github.com/shurcooL/sanitized_anchor_name v1.0.0 h1:PdmoCO6wvbs+7yrJyMORt4/BmY5IYyJwS/kOiWx8mHo= github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= +github.com/shurcooL/users v0.0.0-20180125191416-49c67e49c537 h1:YGaxtkYjb8mnTvtufv2LKLwCQu2/C7qFB7UtrOlTWOY= github.com/shurcooL/users v0.0.0-20180125191416-49c67e49c537/go.mod h1:QJTqeLYEDaXHZDBsXlPCDqdhQuJkuw4NOtaxYe3xii4= +github.com/shurcooL/webdavfs v0.0.0-20170829043945-18c3829fa133 h1:JtcyT0rk/9PKOdnKQzuDR+FSjh7SGtJwpgVpfZBRKlQ= github.com/shurcooL/webdavfs v0.0.0-20170829043945-18c3829fa133/go.mod h1:hKmq5kWdCj2z2KEozexVbfEZIWiTjhE0+UjmZgPqehw= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= +github.com/sirupsen/logrus v1.6.0 h1:UBcNElsrwanuuMsnGSlYmtmgbb23qDR5dG+6X6Oo89I= github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= +github.com/smartystreets/assertions v0.0.0-20190215210624-980c5ac6f3ac h1:wbW+Bybf9pXxnCFAOWZTqkRjAc7rAIwo2e1ArUhiHxg= github.com/smartystreets/assertions v0.0.0-20190215210624-980c5ac6f3ac/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= github.com/smartystreets/goconvey v0.0.0-20181108003508-044398e4856c/go.mod h1:XDJAKZRPZ1CvBcN2aX5YOUTYGHki24fSF0Iv48Ibg0s= github.com/smartystreets/goconvey v0.0.0-20190306220146-200a235640ff/go.mod h1:KSQcGKpxUMHk3nbYzs/tIBAM2iDooCn0BmttHOJEbLs= +github.com/smartystreets/goconvey v1.6.4 h1:fv0U8FUIMPNf1L9lnHLvLhgicrIVChEkdzIKYqbNC9s= github.com/smartystreets/goconvey v1.6.4/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= +github.com/soheilhy/cmux v0.1.4 h1:0HKaf1o97UwFjHH9o5XsHUOF+tqmdA7KEzXLpiyaw0E= github.com/soheilhy/cmux v0.1.4/go.mod h1:IM3LyeVVIOuxMH7sFAkER9+bJ4dT7Ms6E4xg4kGIyLM= +github.com/sony/gobreaker v0.4.1 h1:oMnRNZXX5j85zso6xCPRNPtmAycat+WcoKbklScLDgQ= github.com/sony/gobreaker v0.4.1/go.mod h1:ZKptC7FHNvhBz7dN2LGjPVBz2sZJmc0/PkyDJOjmxWY= +github.com/sourcegraph/annotate v0.0.0-20160123013949-f4cad6c6324d h1:yKm7XZV6j9Ev6lojP2XaIshpT4ymkqhMeSghO5Ps00E= github.com/sourcegraph/annotate v0.0.0-20160123013949-f4cad6c6324d/go.mod h1:UdhH50NIW0fCiwBSr0co2m7BnFLdv4fQTgdqdJTHFeE= +github.com/sourcegraph/syntaxhighlight v0.0.0-20170531221838-bd320f5d308e h1:qpG93cPwA5f7s/ZPBJnGOYQNK/vKsaDaseuKT5Asee8= github.com/sourcegraph/syntaxhighlight v0.0.0-20170531221838-bd320f5d308e/go.mod h1:HuIsMU8RRBOtsCgI77wP899iHVBQpCmg4ErYMZB+2IA= github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= github.com/spaolacci/murmur3 v1.1.0 h1:7c1g84S4BPRrfL5Xrdp6fOJ206sU9y293DDHaoy0bLI= github.com/spaolacci/murmur3 v1.1.0/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= +github.com/spf13/cobra v0.0.3 h1:ZlrZ4XsMRm04Fr5pSFxBgfND2EBVa1nLpiy1stUsX/8= github.com/spf13/cobra v0.0.3/go.mod h1:1l0Ry5zgKvJasoi3XT1TypsSe7PqH0Sj9dhYf7v3XqQ= +github.com/spf13/pflag v1.0.1 h1:aCvUg6QPl3ibpQUxyLkrEkCHtPqYJL4x9AuhqVqFis4= github.com/spf13/pflag v1.0.1/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= github.com/streadway/amqp v0.0.0-20190404075320-75d898a42a94/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw= +github.com/streadway/amqp v0.0.0-20190827072141-edfb9018d271 h1:WhxRHzgeVGETMlmVfqhRn8RIeeNoPr2Czh33I4Zdccw= github.com/streadway/amqp v0.0.0-20190827072141-edfb9018d271/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw= +github.com/streadway/handy v0.0.0-20190108123426-d5acb3125c2a h1:AhmOdSHeswKHBjhsLs/7+1voOxT+LLrSk/Nxvk35fug= github.com/streadway/handy v0.0.0-20190108123426-d5acb3125c2a/go.mod h1:qNTQ5P5JnDBl6z3cMAg/SywNDC5ABu5ApDIw6lUbRmI= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/objx v0.1.1 h1:2vfRuCMp5sSVIDSqO8oNnWJq7mPa6KVP3iPIwFBuy8A= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/testify v1.2.1/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= @@ -624,20 +859,28 @@ github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5 github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.7.0 h1:nwc3DEeHmmLAfoZucVR881uASk0Mfjw8xYJ99tb5CcY= github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= +github.com/syncthing/syncthing v0.14.48-rc.4 h1:9w607+diOQw1niYrydPEKbI50Q5oZ9rneSn8Twqjqlo= github.com/syncthing/syncthing v0.14.48-rc.4/go.mod h1:nw3siZwHPA6M8iSfjDCWQ402eqvEIasMQOE8nFOxy7M= +github.com/tarm/serial v0.0.0-20180830185346-98f6abe2eb07 h1:UyzmZLoiDWMRywV4DUYb9Fbt8uiOSooupjTq10vpvnU= github.com/tarm/serial v0.0.0-20180830185346-98f6abe2eb07/go.mod h1:kDXzergiv9cbyO7IOYJZWg1U88JhDg3PB6klq9Hg2pA= github.com/tinylib/msgp v1.0.2/go.mod h1:+d+yLhGm8mzTaHzB+wgMYrodPfmZrzkirds8fDWklFE= github.com/tinylib/msgp v1.1.0/go.mod h1:+d+yLhGm8mzTaHzB+wgMYrodPfmZrzkirds8fDWklFE= github.com/tinylib/msgp v1.1.1/go.mod h1:+d+yLhGm8mzTaHzB+wgMYrodPfmZrzkirds8fDWklFE= github.com/tinylib/msgp v1.1.2/go.mod h1:+d+yLhGm8mzTaHzB+wgMYrodPfmZrzkirds8fDWklFE= +github.com/tinylib/msgp v1.1.5 h1:2gXmtWueD2HefZHQe1QOy9HVzmFrLOVvsXwXBQ0ayy0= github.com/tinylib/msgp v1.1.5/go.mod h1:eQsjooMTnV42mHu917E26IogZ2930nFyBQdofk10Udg= +github.com/tmc/grpc-websocket-proxy v0.0.0-20170815181823-89b8d40f7ca8 h1:ndzgwNDnKIqyCvHTXaCqh9KlOWKvBry6nuXMJmonVsE= github.com/tmc/grpc-websocket-proxy v0.0.0-20170815181823-89b8d40f7ca8/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= +github.com/ttacon/chalk v0.0.0-20160626202418-22c06c80ed31 h1:OXcKh35JaYsGMRzpvFkLv/MEyPuL49CThT1pZ8aSml4= github.com/ttacon/chalk v0.0.0-20160626202418-22c06c80ed31/go.mod h1:onvgF043R+lC5RZ8IT9rBXDaEDnpnw/Cl+HFiw+v/7Q= github.com/tv42/httpunix v0.0.0-20191220191345-2ba4b9c3382c h1:u6SKchux2yDvFQnDHS3lPnIRmfVJ5Sxy3ao2SIdysLQ= github.com/tv42/httpunix v0.0.0-20191220191345-2ba4b9c3382c/go.mod h1:hzIxponao9Kjc7aWznkXaL4U4TWaDSs8zcsY4Ka08nM= github.com/urfave/cli v1.20.0/go.mod h1:70zkFmudgCuE/ngEzBv17Jvp/497gISqfk5gWijbERA= +github.com/urfave/cli v1.22.1 h1:+mkCCcOFKPnCmVYVcURKps1Xe+3zP90gSYGNfRkjoIY= github.com/urfave/cli v1.22.1/go.mod h1:Gos4lmkARVdJ6EkW0WaNv/tZAAMe9V7XWyB60NtXRu0= +github.com/viant/assertly v0.4.8 h1:5x1GzBaRteIwTr5RAGFVG14uNeRFxVNbXPWrK2qAgpc= github.com/viant/assertly v0.4.8/go.mod h1:aGifi++jvCrUaklKEKT0BU95igDNaqkvz+49uaYMPRU= +github.com/viant/toolbox v0.24.0 h1:6TteTDQ68CjgcCe8wH3D3ZhUQQOJXMTbj/D9rkk2a1k= github.com/viant/toolbox v0.24.0/go.mod h1:OxMCG57V0PXuIP2HNQrtJf2CjqdmbrOx5EkMILuUhzM= github.com/willf/bitset v1.1.3/go.mod h1:RjeCKbqT1RxIR/KWY6phxZiaY1IyutSBfGjNPySAYV4= github.com/willf/bitset v1.1.9/go.mod h1:RjeCKbqT1RxIR/KWY6phxZiaY1IyutSBfGjNPySAYV4= @@ -647,26 +890,36 @@ github.com/willf/bitset v1.1.11/go.mod h1:83CECat5yLh5zVOf4P1ErAgKA5UDvKtgyUABdr github.com/willf/bloom v0.0.0-20170505221640-54e3b963ee16/go.mod h1:MmAltL9pDMNTrvUkxdg0k0q5I0suxmuwp3KbyrZLOZ8= github.com/willf/bloom v2.0.3+incompatible h1:QDacWdqcAUI1MPOwIQZRy9kOR7yxfyEmxX8Wdm2/JPA= github.com/willf/bloom v2.0.3+incompatible/go.mod h1:MmAltL9pDMNTrvUkxdg0k0q5I0suxmuwp3KbyrZLOZ8= +github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2 h1:eY9dn8+vbi4tKz5Qo6v2eYzo7kUS51QINcR5jNpbZS8= github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2/go.mod h1:UETIi67q53MR2AWcXfiuqkDkRtnGDLqkBTpCHuJHxtU= +github.com/yuin/goldmark v1.2.1 h1:ruQGxdhGHe7FWOJPT0mKs5+pD2Xs1Bm/kdGlHO04FmM= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= go.etcd.io/bbolt v1.3.3/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= go.etcd.io/bbolt v1.3.4/go.mod h1:G5EMThwa9y8QZGBClrRx5EY+Yw9kAhnjy3bSjsnlVTQ= go.etcd.io/bbolt v1.3.5 h1:XAzx9gjCb0Rxj7EoqcClPD1d5ZBxZJk0jbuoPHenBt0= go.etcd.io/bbolt v1.3.5/go.mod h1:G5EMThwa9y8QZGBClrRx5EY+Yw9kAhnjy3bSjsnlVTQ= +go.etcd.io/etcd v0.0.0-20191023171146-3cf2f69b5738 h1:VcrIfasaLFkyjk6KNlXQSzO+B0fZcnECiDrKJsfxka0= go.etcd.io/etcd v0.0.0-20191023171146-3cf2f69b5738/go.mod h1:dnLIgRNXwCJa5e+c6mIZCrds/GIG4ncV9HhK5PX7jPg= go.opencensus.io v0.18.0/go.mod h1:vKdFvxhtzZ9onBp9VKHK8z/sRpBMnKAsufL7wlDrCOA= go.opencensus.io v0.20.1/go.mod h1:6WKK9ahsWS3RSO+PY9ZHZUfv2irvY6gN279GOPZjmmk= go.opencensus.io v0.20.2/go.mod h1:6WKK9ahsWS3RSO+PY9ZHZUfv2irvY6gN279GOPZjmmk= go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= +go.opencensus.io v0.22.3 h1:8sGtKOrtQqkN1bp2AtX+misvLIlOmsEsNd+9NIcPEm8= go.opencensus.io v0.22.3/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= go.uber.org/atomic v1.3.2/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= +go.uber.org/atomic v1.5.0 h1:OI5t8sDa1Or+q8AeE+yKeB/SDYioSHAgcVljj9JIETY= go.uber.org/atomic v1.5.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0= +go.uber.org/multierr v1.3.0 h1:sFPn2GLc3poCkfrpIXGhBD2X0CMIo4Q/zSULXrj/+uc= go.uber.org/multierr v1.3.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4= +go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee h1:0mgffUl7nfd+FpvXMVz4IDEaUSmT1ysygQC7qYo7sG4= go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA= go.uber.org/zap v1.10.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= +go.uber.org/zap v1.13.0 h1:nR6NoDBgAf67s68NhaXbsojM+2gxp3S1hWkHDl27pVU= go.uber.org/zap v1.13.0/go.mod h1:zwrFLgMcdUuIBviXEYEH1YKNaOBnKXsx2IPda5bBwHM= +go4.org v0.0.0-20180809161055-417644f6feb5 h1:+hE86LblG4AyDgwMCLTE6FOlM9+qjHSYS+rKqxUVdsM= go4.org v0.0.0-20180809161055-417644f6feb5/go.mod h1:MkTOUMDaeVYJUOUsaDXIhWPZYa1yOyC1qaOBpL57BhE= +golang.org/x/build v0.0.0-20190111050920-041ab4dc3f9d h1:E2M5QgjZ/Jg+ObCQAudsXxuTsLj7Nl5RV/lZcQZmKSo= golang.org/x/build v0.0.0-20190111050920-041ab4dc3f9d/go.mod h1:OWs+y06UdEOHN4y+MfF/py+xQ/tYqIWW03b70/CG9Rw= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20181029021203-45a5f77698d3/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= @@ -686,6 +939,7 @@ golang.org/x/crypto v0.0.0-20201221181555-eec23a3978ad/go.mod h1:jdWPYTVW3xRLrWP golang.org/x/crypto v0.0.0-20210322153248-0c34fe9e7dc2/go.mod h1:T9bdIzuCu7OtxOm1hfPfRQxPLYneinmdGuTeoZ9dtd4= golang.org/x/crypto v0.0.0-20210421170649-83a5a9bb288b h1:7mWr3k41Qtv8XlltBkDkl8LoP3mpSgBW8BUoxtEdbXg= golang.org/x/crypto v0.0.0-20210421170649-83a5a9bb288b/go.mod h1:T9bdIzuCu7OtxOm1hfPfRQxPLYneinmdGuTeoZ9dtd4= +golang.org/x/exp v0.0.0-20190121172915-509febef88a4 h1:c2HOrn5iMezYjSlGPncknSEr/8x5LELb/ilJbXi9DEA= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/lint v0.0.0-20180702182130-06c8688daad7/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= @@ -693,9 +947,11 @@ golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvx golang.org/x/lint v0.0.0-20190301231843-5614ed5bae6f/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20200302205851-738671d3881b h1:Wh+f8QHJXR411sJR8/vRBTZ7YapZaRvUcLFFJhusH0k= golang.org/x/lint v0.0.0-20200302205851-738671d3881b/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= +golang.org/x/mod v0.3.0 h1:RM4zey1++hCTbCVQfnWeKs9/IEsaBLA8vTkd0WVtmH4= golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/net v0.0.0-20180524181706-dfa909b99c79/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -742,7 +998,9 @@ golang.org/x/net v0.0.0-20210427231257-85d9c07bbe3a/go.mod h1:OJAsFXCWl8Ukc7SiCT golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20181017192945-9dcd33a902f4/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20181203162652-d668ce993890/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= +golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421 h1:Wo7BWFiOk0QRFMLYMqJGFMd9CgUAcGx7V+qEg/h5IBI= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/perf v0.0.0-20180704124530-6e6d33e29852 h1:xYq6+9AtI+xP3M4r0N1hCkHrInHDBohhquRgx9Kk6gI= golang.org/x/perf v0.0.0-20180704124530-6e6d33e29852/go.mod h1:JLpeXjPJfIyPr5TlbXLkXWLhP8nz10XfvxElABhCtcw= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -806,12 +1064,14 @@ golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210426230700-d19ff857e887 h1:dXfMednGJh/SUUFjTLsWJz3P+TQt9qnR11GgeI3vWKs= golang.org/x/sys v0.0.0-20210426230700-d19ff857e887/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/term v0.0.0-20201117132131-f5c789dd3221/go.mod h1:Nr5EML6q2oocZ2LXRh80K7BxOlk5/8JxuGnuhpl+muw= +golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1 h1:v+OssWQX+hTHEmOBgwxdZxK4zHq3yOs8F9J7mk0PY8E= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= +golang.org/x/text v0.3.6 h1:aRYxNxv6iGQlyVaZmk6ZgYEDa+Jg18DxebPSrd6bg1M= golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= @@ -839,6 +1099,7 @@ golang.org/x/tools v0.0.0-20200103221440-774c71fcf114/go.mod h1:TB2adYChydJhpapK golang.org/x/tools v0.0.0-20200130002326-2f3ba24bd6e7/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= golang.org/x/tools v0.0.0-20200928182047-19e03678916f/go.mod h1:z6u4i615ZeAfBE4XtMziQW1fSVJXACjjbWkB/mvPzlU= golang.org/x/tools v0.0.0-20201022035929-9cf592e881e9/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= +golang.org/x/tools v0.0.0-20201224043029-2b0845dc783e h1:4nW4NLDYnU28ojHaHO8OVxFHk/aQ33U01a9cjED+pzE= golang.org/x/tools v0.0.0-20201224043029-2b0845dc783e/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -848,10 +1109,12 @@ golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8T google.golang.org/api v0.0.0-20180910000450-7ca32eb868bf/go.mod h1:4mhQ8q/RsB7i+udVvVy5NUi08OU8ZlA0gRVgrF7VFY0= google.golang.org/api v0.0.0-20181030000543-1d582fd0359e/go.mod h1:4mhQ8q/RsB7i+udVvVy5NUi08OU8ZlA0gRVgrF7VFY0= google.golang.org/api v0.1.0/go.mod h1:UGEZY7KEX120AnNLIHFMKIo4obdJhkp2tPbaPlQx13Y= +google.golang.org/api v0.3.1 h1:oJra/lMfmtm13/rgY/8i3MzjFWYXvQIAKjQ3HqofMk8= google.golang.org/api v0.3.1/go.mod h1:6wY9I6uQWHQ8EM57III9mq/AjF+i8G65rmVagqKMtkk= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.2.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.3.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= +google.golang.org/appengine v1.4.0 h1:/wp5JvzpHIxhs/dumFmF7BXTf3Z+dd4uXta4kVyO508= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20180831171423-11092d34479b/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= @@ -862,6 +1125,7 @@ google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRn google.golang.org/genproto v0.0.0-20190425155659-357c62f0e4bb/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= google.golang.org/genproto v0.0.0-20190530194941-fb225487d101/go.mod h1:z3L6/3dTEVtUr6QSP8miRzeRqwQOioJ9I66odjN4I7s= google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= +google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013 h1:+kGHl1aib/qcwaRi1CbqBZ1rk19r85MNUf8HaBghugY= google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo= google.golang.org/grpc v1.14.0/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= google.golang.org/grpc v1.16.0/go.mod h1:0JHn/cJsOMiMfNA9+DeHDlAU7KAAB5GDlYFpa9MZMio= @@ -874,6 +1138,7 @@ google.golang.org/grpc v1.22.1/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyac google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= google.golang.org/grpc v1.23.1/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= google.golang.org/grpc v1.26.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= +google.golang.org/grpc v1.27.0 h1:rRYRFMVgRv6E0D70Skyfsr28tDXIuuPZyWGMPdMcnXg= google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= @@ -883,19 +1148,29 @@ google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzi google.golang.org/protobuf v1.22.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= google.golang.org/protobuf v1.23.1-0.20200526195155-81db48ad09cc/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= +google.golang.org/protobuf v1.25.0 h1:Ejskq+SyPohKW+1uil0JJMtmHCgJPJ/qWTxr8qp+R4c= google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= +gopkg.in/alecthomas/kingpin.v2 v2.2.6 h1:jMFz6MfLP0/4fUyZle81rXUoxOBFi19VUFKVDOQfozc= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15 h1:YR8cESwS4TdDjEe65xsg0ogRM/Nc3DYOhEAlW+xobZo= gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/cheggaaa/pb.v1 v1.0.25 h1:Ev7yu1/f6+d+b3pi5vPdRPc6nNtP1umSfcWiEfRqv6I= gopkg.in/cheggaaa/pb.v1 v1.0.25/go.mod h1:V/YB90LKu/1FcN3WVnfiiE5oMCibMjukxqG/qStrOgw= +gopkg.in/errgo.v2 v2.1.0 h1:0vLT13EuvQ0hNvakwLuFZ/jYrLp5F3kcWHXdRggjCE8= gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= +gopkg.in/fsnotify.v1 v1.4.7 h1:xOHLXZwVvI9hhs+cLKq5+I5onOuwQLhQwiu63xxlHs4= gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys= +gopkg.in/gcfg.v1 v1.2.3 h1:m8OOJ4ccYHnx2f4gQwpno8nAX5OGOh7RLaaz0pj3Ogs= gopkg.in/gcfg.v1 v1.2.3/go.mod h1:yesOnuUOFQAhST5vPY4nbZsb/huCgGGXlipJsBn0b3o= +gopkg.in/inf.v0 v0.9.1 h1:73M5CoZyi3ZLMOyDlQh031Cx6N9NDJ2Vvfl76EDAgDc= gopkg.in/inf.v0 v0.9.1/go.mod h1:cWUDdTG/fYaXco+Dcufb5Vnc6Gp2YChqWtbxRZE0mXw= +gopkg.in/resty.v1 v1.12.0 h1:CuXP0Pjfw9rOuY6EP+UvtNvt5DSqHpIxILZKT/quCZI= gopkg.in/resty.v1 v1.12.0/go.mod h1:mDo4pnntr5jdWRML875a/NmxYqAlA73dVijT2AXvQQo= +gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 h1:uRGJdciOHaEIrze2W8Q3AKkepLTh2hOroT7a+7czfdQ= gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw= +gopkg.in/warnings.v0 v0.1.2 h1:wFXVbFY8DY5/xOe1ECiWdKCzZlxgshcYVNkBHstARME= gopkg.in/warnings.v0 v0.1.2/go.mod h1:jksf8JmL6Qr/oQM2OXTHunEvvTAsrWBLb6OOjuVWRNI= gopkg.in/yaml.v2 v2.0.0-20170812160011-eb3733d160e7/go.mod h1:JAlM8MvJe8wmxCU4Bli9HhUf9+ttbYbLASfIpnQbh74= gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= @@ -904,18 +1179,27 @@ gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.5/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.7/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY= gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c h1:dUUwHk2QECo/6vqA44rthZ8ie2QXMNeKRTHCNY2nXvo= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +grpc.go4.org v0.0.0-20170609214715-11d0a25b4919 h1:tmXTu+dfa+d9Evp8NpJdgOy6+rt8/x4yG7qPBrtNfLY= grpc.go4.org v0.0.0-20170609214715-11d0a25b4919/go.mod h1:77eQGdRu53HpSqPFJFmuJdjuHRquDANNeA4x7B8WQ9o= honnef.co/go/tools v0.0.0-20180728063816-88497007e858/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.1-2019.2.3 h1:3JgtbtFHMiCmsznwGVTUWbgGov+pVqnlf1dEJTNAXeM= honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= +rsc.io/quote/v3 v3.1.0 h1:9JKUTTIUgS6kzR9mK1YuGKv6Nl+DijDNIc0ghT58FaY= rsc.io/quote/v3 v3.1.0/go.mod h1:yEA65RcK8LyAZtP9Kv3t0HmxON59tX3rD+tICJqUlj0= +rsc.io/sampler v1.3.0 h1:7uVkIFmeBqHfdjD+gZwtXXI+RODJ2Wc4O7MPEh/QiW4= rsc.io/sampler v1.3.0/go.mod h1:T1hPZKmBbMNahiBKFy5HrXp6adAjACjK9JXDnKaTXpA= +sigs.k8s.io/yaml v1.1.0 h1:4A07+ZFc2wgJwo8YNlQpr1rVlgUDlxXHhPJciaPY5gs= sigs.k8s.io/yaml v1.1.0/go.mod h1:UJmg0vDUVViEyp3mgSv9WPwZCDxu4rQW1olrI1uml+o= +sourcegraph.com/sourcegraph/appdash v0.0.0-20190731080439-ebfcffb1b5c0 h1:ucqkfpjg9WzSUubAO62csmucvxl4/JeW3F4I4909XkM= sourcegraph.com/sourcegraph/appdash v0.0.0-20190731080439-ebfcffb1b5c0/go.mod h1:hI742Nqp5OhwiqlzhgfbWU4mW4yO10fP+LoT9WOswdU= +sourcegraph.com/sourcegraph/go-diff v0.5.0 h1:eTiIR0CoWjGzJcnQ3OkhIl/b9GJovq4lSAVRt0ZFEG8= sourcegraph.com/sourcegraph/go-diff v0.5.0/go.mod h1:kuch7UrkMzY0X+p9CRK03kfuPQ2zzQcaEFbx8wA8rck= +sourcegraph.com/sqs/pbtypes v0.0.0-20180604144634-d3ebe8f20ae4 h1:JPJh2pk3+X4lXAkZIk2RuE/7/FoK9maXw+TNPJhVS/c= sourcegraph.com/sqs/pbtypes v0.0.0-20180604144634-d3ebe8f20ae4/go.mod h1:ketZ/q3QxT9HOBeFhu6RdvsftgpsbFHBF5Cas6cDKZ0= From 24ceed61dabd4f404963be9d4343d17422ebeab8 Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Thu, 20 May 2021 18:51:08 +1000 Subject: [PATCH 43/59] Break out peerConnWriter --- client.go | 34 +++++++++-- go.mod | 2 +- go.sum | 5 ++ internal/chansync/broadcast-cond.go.go | 32 ++++++++++ internal/chansync/set-once.go | 41 +++++++++++++ peerconn.go | 83 +++++++++++++++++--------- peerconn_test.go | 3 +- pexconn_test.go | 4 +- 8 files changed, 167 insertions(+), 37 deletions(-) create mode 100644 internal/chansync/broadcast-cond.go.go create mode 100644 internal/chansync/set-once.go diff --git a/client.go b/client.go index 98bf0892..31dbf4ce 100644 --- a/client.go +++ b/client.go @@ -960,7 +960,7 @@ func (cl *Client) runHandshookConn(c *PeerConn, t *Torrent) error { return fmt.Errorf("adding connection: %w", err) } defer t.dropConnection(c) - go c.writer(time.Minute) + c.startWriter() cl.sendInitialMessages(c, t) err := c.mainReadLoop() if err != nil { @@ -969,6 +969,32 @@ func (cl *Client) runHandshookConn(c *PeerConn, t *Torrent) error { return nil } +func (pc *PeerConn) startWriter() { + w := &pc.messageWriter + *w = peerConnWriter{ + fillWriteBuffer: func() { + pc.locker().Lock() + defer pc.locker().Unlock() + pc.fillWriteBuffer() + }, + closed: &pc.closed, + logger: pc.logger, + w: pc.w, + keepAlive: func() bool { + pc.locker().Lock() + defer pc.locker().Unlock() + return pc.useful() + }, + writeBuffer: new(bytes.Buffer), + } + go func() { + defer pc.locker().Unlock() + defer pc.close() + defer pc.locker().Lock() + pc.messageWriter.run(time.Minute) + }() +} + // Maximum pending requests we allow peers to send us. If peer requests are buffered on read, this // instructs the amount of memory that might be used to cache pending writes. Assuming 512KiB // (1<<19) cached for sending, for 16KiB (1<<14) chunks. @@ -1409,13 +1435,11 @@ func (cl *Client) newConnection(nc net.Conn, outgoing bool, remoteAddr PeerRemot Network: network, callbacks: &cl.config.Callbacks, }, - connString: connString, - conn: nc, - writeBuffer: new(bytes.Buffer), + connString: connString, + conn: nc, } c.peerImpl = c c.logger = cl.logger.WithDefaultLevel(log.Warning).WithContextValue(c) - c.writerCond.L = cl.locker() c.setRW(connStatsReadWriter{nc, c}) c.r = &rateLimitedReader{ l: cl.config.DownloadRateLimiter, diff --git a/go.mod b/go.mod index 7a1e19b3..88326bfe 100644 --- a/go.mod +++ b/go.mod @@ -14,7 +14,7 @@ require ( github.com/anacrolix/missinggo/perf v1.0.0 github.com/anacrolix/missinggo/v2 v2.5.1-0.20210520011502-b3d95d6b1d02 github.com/anacrolix/multiless v0.1.1-0.20210520040635-10ee7b5f3cff - github.com/anacrolix/sync v0.2.0 + github.com/anacrolix/sync v0.3.0 github.com/anacrolix/tagflag v1.3.0 github.com/anacrolix/upnp v0.1.2-0.20200416075019-5e9378ed1425 github.com/anacrolix/utp v0.1.0 diff --git a/go.sum b/go.sum index 8c49d1e3..c12955bc 100644 --- a/go.sum +++ b/go.sum @@ -133,6 +133,10 @@ github.com/anacrolix/sync v0.0.0-20180611022320-3c4cb11f5a01/go.mod h1:+u91KiUuf github.com/anacrolix/sync v0.0.0-20180808010631-44578de4e778/go.mod h1:s735Etp3joe/voe2sdaXLcqDdJSay1O0OPnM0ystjqk= github.com/anacrolix/sync v0.2.0 h1:oRe22/ZB+v7v/5Mbc4d2zE0AXEZy0trKyKLjqYOt6tY= github.com/anacrolix/sync v0.2.0/go.mod h1:BbecHL6jDSExojhNtgTFSBcdGerzNc64tz3DCOj/I0g= +github.com/anacrolix/sync v0.2.1-0.20210520084835-26aa6614542f h1:7KqmZoEOIXa0UbR2WQ/YPF4H+MPV6rhWk4E4tcv5eDg= +github.com/anacrolix/sync v0.2.1-0.20210520084835-26aa6614542f/go.mod h1:BbecHL6jDSExojhNtgTFSBcdGerzNc64tz3DCOj/I0g= +github.com/anacrolix/sync v0.3.0 h1:ZPjTrkqQWEfnYVGTQHh5qNjokWaXnjsyXTJSMsKY0TA= +github.com/anacrolix/sync v0.3.0/go.mod h1:BbecHL6jDSExojhNtgTFSBcdGerzNc64tz3DCOj/I0g= github.com/anacrolix/tagflag v0.0.0-20180109131632-2146c8d41bf0/go.mod h1:1m2U/K6ZT+JZG0+bdMK6qauP49QT4wE5pmhJXOKKCHw= github.com/anacrolix/tagflag v0.0.0-20180605133421-f477c8c2f14c/go.mod h1:1m2U/K6ZT+JZG0+bdMK6qauP49QT4wE5pmhJXOKKCHw= github.com/anacrolix/tagflag v0.0.0-20180803105420-3a8ff5428f76/go.mod h1:1m2U/K6ZT+JZG0+bdMK6qauP49QT4wE5pmhJXOKKCHw= @@ -428,6 +432,7 @@ github.com/hashicorp/go.net v0.0.1/go.mod h1:hjKkEWcCURg++eb33jQU7oqQcI9XDCnUzHA github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= github.com/hashicorp/golang-lru v0.5.1 h1:0hERBMJE1eitiLkihrMvRVBYAkpHzc/J3QdDN+dAcgU= github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= +github.com/hashicorp/golang-lru v0.5.4 h1:YDjusn29QI/Das2iO9M0BHnIbxPeyuCHsjMW+lJfyTc= github.com/hashicorp/golang-lru v0.5.4/go.mod h1:iADmTwqILo4mZ8BN3D2Q6+9jd8WM5uGBxy+E8yxSoD4= github.com/hashicorp/logutils v1.0.0 h1:dLEQVugN8vlakKOUE3ihGLTZJRB4j+M2cdTm/ORI65Y= github.com/hashicorp/logutils v1.0.0/go.mod h1:QIAnNjmIWmVIIkWDTG1z5v++HQmx9WQRO+LraFDTW64= diff --git a/internal/chansync/broadcast-cond.go.go b/internal/chansync/broadcast-cond.go.go new file mode 100644 index 00000000..9b890692 --- /dev/null +++ b/internal/chansync/broadcast-cond.go.go @@ -0,0 +1,32 @@ +package chansync + +import ( + "github.com/anacrolix/sync" +) + +// Can be used as zero-value. Due to the caller needing to bring their own synchronization, an +// eqiuvalent to "sync".Cond.Signal is not provided. BroadcastCond is intended to be selected on +// with other channels. +type BroadcastCond struct { + mu sync.Mutex + ch chan struct{} +} + +func (me *BroadcastCond) Broadcast() { + me.mu.Lock() + defer me.mu.Unlock() + if me.ch != nil { + close(me.ch) + me.ch = nil + } +} + +// Should be called before releasing locks on resources that might trigger subsequent Broadcasts. +func (me *BroadcastCond) WaitChan() <-chan struct{} { + me.mu.Lock() + defer me.mu.Unlock() + if me.ch == nil { + me.ch = make(chan struct{}) + } + return me.ch +} diff --git a/internal/chansync/set-once.go b/internal/chansync/set-once.go new file mode 100644 index 00000000..523e5eaf --- /dev/null +++ b/internal/chansync/set-once.go @@ -0,0 +1,41 @@ +package chansync + +import "sync" + +// SetOnce is a boolean value that can only be flipped from false to true. +type SetOnce struct { + ch chan struct{} + initOnce sync.Once + closeOnce sync.Once +} + +func (me *SetOnce) Chan() <-chan struct{} { + me.init() + return me.ch +} + +func (me *SetOnce) init() { + me.initOnce.Do(func() { + me.ch = make(chan struct{}) + }) +} + +// Set only returns true the first time it is called. +func (me *SetOnce) Set() (first bool) { + me.closeOnce.Do(func() { + me.init() + first = true + close(me.ch) + }) + return +} + +func (me *SetOnce) IsSet() bool { + me.init() + select { + case <-me.ch: + return true + default: + return false + } +} diff --git a/peerconn.go b/peerconn.go index 8e3f5443..9cddf7d7 100644 --- a/peerconn.go +++ b/peerconn.go @@ -11,18 +11,18 @@ import ( "sort" "strconv" "strings" - "sync" "time" "github.com/anacrolix/log" - "github.com/anacrolix/missinggo" "github.com/anacrolix/missinggo/iter" "github.com/anacrolix/missinggo/v2/bitmap" "github.com/anacrolix/missinggo/v2/prioritybitmap" "github.com/anacrolix/multiless" - "github.com/anacrolix/torrent/metainfo" + "github.com/anacrolix/sync" "github.com/anacrolix/torrent/bencode" + "github.com/anacrolix/torrent/internal/chansync" + "github.com/anacrolix/torrent/metainfo" "github.com/anacrolix/torrent/mse" pp "github.com/anacrolix/torrent/peer_protocol" ) @@ -68,7 +68,7 @@ type Peer struct { cryptoMethod mse.CryptoMethod Discovery PeerSource trusted bool - closed missinggo.Event + closed chansync.SetOnce // Set true after we've added our ConnStats generated during handshake to // other ConnStat instances as determined when the *Torrent became known. reconciledHandshakeStats bool @@ -148,11 +148,10 @@ type PeerConn struct { w io.Writer r io.Reader - writeBuffer *bytes.Buffer - uploadTimer *time.Timer - writerCond sync.Cond + messageWriter peerConnWriter - pex pexConnState + uploadTimer *time.Timer + pex pexConnState } func (cn *PeerConn) connStatusString() string { @@ -429,17 +428,25 @@ const writeBufferHighWaterLen = 1 << 15 // done asynchronously, so it may be that we're not able to honour backpressure from this method. func (cn *PeerConn) write(msg pp.Message) bool { torrent.Add(fmt.Sprintf("messages written of type %s", msg.Type.String()), 1) - // We don't need to track bytes here because a connection.w Writer wrapper takes care of that - // (although there's some delay between us recording the message, and the connection writer + // We don't need to track bytes here because the connection's Writer has that behaviour injected + // (although there's some delay between us buffering the message, and the connection writer // flushing it out.). - cn.writeBuffer.Write(msg.MustMarshalBinary()) + notFull := cn.messageWriter.write(msg) // Last I checked only Piece messages affect stats, and we don't write those. cn.wroteMsg(&msg) cn.tickleWriter() + return notFull +} + +func (cn *peerConnWriter) write(msg pp.Message) bool { + cn.mu.Lock() + defer cn.mu.Unlock() + cn.writeBuffer.Write(msg.MustMarshalBinary()) + cn.writeCond.Broadcast() return !cn.writeBufferFull() } -func (cn *PeerConn) writeBufferFull() bool { +func (cn *peerConnWriter) writeBufferFull() bool { return cn.writeBuffer.Len() >= writeBufferHighWaterLen } @@ -636,25 +643,38 @@ func (cn *PeerConn) fillWriteBuffer() { cn.upload(cn.write) } +type peerConnWriter struct { + // Must not be called with the local mutex held, as it will call back into the write method. + fillWriteBuffer func() + closed *chansync.SetOnce + logger log.Logger + w io.Writer + keepAlive func() bool + + mu sync.Mutex + writeCond chansync.BroadcastCond + // Pointer so we can swap with the "front buffer". + writeBuffer *bytes.Buffer +} + // Routine that writes to the peer. Some of what to write is buffered by // activity elsewhere in the Client, and some is determined locally when the // connection is writable. -func (cn *PeerConn) writer(keepAliveTimeout time.Duration) { +func (cn *peerConnWriter) run(keepAliveTimeout time.Duration) { var ( lastWrite time.Time = time.Now() keepAliveTimer *time.Timer ) keepAliveTimer = time.AfterFunc(keepAliveTimeout, func() { - cn.locker().Lock() - defer cn.locker().Unlock() + cn.mu.Lock() + defer cn.mu.Unlock() if time.Since(lastWrite) >= keepAliveTimeout { - cn.tickleWriter() + cn.writeCond.Broadcast() } keepAliveTimer.Reset(keepAliveTimeout) }) - cn.locker().Lock() - defer cn.locker().Unlock() - defer cn.close() + cn.mu.Lock() + defer cn.mu.Unlock() defer keepAliveTimer.Stop() frontBuf := new(bytes.Buffer) for { @@ -662,22 +682,31 @@ func (cn *PeerConn) writer(keepAliveTimeout time.Duration) { return } if cn.writeBuffer.Len() == 0 { - cn.fillWriteBuffer() + func() { + cn.mu.Unlock() + defer cn.mu.Lock() + cn.fillWriteBuffer() + }() } - if cn.writeBuffer.Len() == 0 && time.Since(lastWrite) >= keepAliveTimeout && cn.useful() { + if cn.writeBuffer.Len() == 0 && time.Since(lastWrite) >= keepAliveTimeout && cn.keepAlive() { cn.writeBuffer.Write(pp.Message{Keepalive: true}.MustMarshalBinary()) torrent.Add("written keepalives", 1) } if cn.writeBuffer.Len() == 0 { - // TODO: Minimize wakeups.... - cn.writerCond.Wait() + writeCond := cn.writeCond.WaitChan() + cn.mu.Unlock() + select { + case <-cn.closed.Chan(): + case <-writeCond: + } + cn.mu.Lock() continue } // Flip the buffers. frontBuf, cn.writeBuffer = cn.writeBuffer, frontBuf - cn.locker().Unlock() + cn.mu.Unlock() n, err := cn.w.Write(frontBuf.Bytes()) - cn.locker().Lock() + cn.mu.Lock() if n != 0 { lastWrite = time.Now() keepAliveTimer.Reset(keepAliveTimeout) @@ -1463,7 +1492,7 @@ func (c *PeerConn) uploadAllowed() bool { func (c *PeerConn) setRetryUploadTimer(delay time.Duration) { if c.uploadTimer == nil { - c.uploadTimer = time.AfterFunc(delay, c.writerCond.Broadcast) + c.uploadTimer = time.AfterFunc(delay, c.tickleWriter) } else { c.uploadTimer.Reset(delay) } @@ -1558,7 +1587,7 @@ func (c *Peer) deleteAllRequests() { // This is called when something has changed that should wake the writer, such as putting stuff into // the writeBuffer, or changing some state that the writer can act on. func (c *PeerConn) tickleWriter() { - c.writerCond.Broadcast() + c.messageWriter.writeCond.Broadcast() } func (c *PeerConn) sendChunk(r Request, msg func(pp.Message) bool, state *peerRequestState) (more bool) { diff --git a/peerconn_test.go b/peerconn_test.go index 18fc98ef..c28bc632 100644 --- a/peerconn_test.go +++ b/peerconn_test.go @@ -5,7 +5,6 @@ import ( "net" "sync" "testing" - "time" "github.com/anacrolix/missinggo/pubsub" "github.com/bradfitz/iter" @@ -32,7 +31,7 @@ func TestSendBitfieldThenHave(t *testing.T) { r, w := io.Pipe() //c.r = r c.w = w - go c.writer(time.Minute) + c.startWriter() c.locker().Lock() c.t._completedPieces.Add(1) c.postBitfield( /*[]bool{false, true, false}*/ ) diff --git a/pexconn_test.go b/pexconn_test.go index a3ff1d02..df9aa44e 100644 --- a/pexconn_test.go +++ b/pexconn_test.go @@ -21,7 +21,7 @@ func TestPexConnState(t *testing.T) { c := cl.newConnection(nil, false, addr, addr.Network(), "") c.PeerExtensionIDs = make(map[pp.ExtensionName]pp.ExtensionNumber) c.PeerExtensionIDs[pp.ExtensionNamePex] = pexExtendedId - c.writerCond.L.Lock() + c.messageWriter.mu.Lock() c.setTorrent(torrent) torrent.addPeerConn(c) @@ -36,7 +36,7 @@ func TestPexConnState(t *testing.T) { out = m return true } - c.writerCond.Wait() + <-c.messageWriter.writeCond.WaitChan() c.pex.Share(testWriter) require.True(t, writerCalled) require.EqualValues(t, pp.Extended, out.Type) From d37dea1f61a9c83c05f32a302e1e40da6d853505 Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Thu, 20 May 2021 18:55:23 +1000 Subject: [PATCH 44/59] Move peerConnMsgWriter into its own file --- client.go | 40 ++++--------------------- peerconn.go | 86 ++--------------------------------------------------- 2 files changed, 9 insertions(+), 117 deletions(-) diff --git a/client.go b/client.go index 31dbf4ce..7ac6b264 100644 --- a/client.go +++ b/client.go @@ -2,7 +2,6 @@ package torrent import ( "bufio" - "bytes" "context" "crypto/rand" "encoding/binary" @@ -21,13 +20,11 @@ import ( "github.com/anacrolix/missinggo/perf" "github.com/anacrolix/missinggo/pubsub" "github.com/anacrolix/missinggo/slices" + "github.com/anacrolix/missinggo/v2" "github.com/anacrolix/missinggo/v2/bitmap" + "github.com/anacrolix/missinggo/v2/conntrack" "github.com/anacrolix/missinggo/v2/pproffd" "github.com/anacrolix/sync" - "github.com/anacrolix/torrent/internal/limiter" - request_strategy "github.com/anacrolix/torrent/request-strategy" - "github.com/anacrolix/torrent/tracker" - "github.com/anacrolix/torrent/webtorrent" "github.com/davecgh/go-spew/spew" "github.com/dustin/go-humanize" "github.com/google/btree" @@ -35,15 +32,16 @@ import ( "golang.org/x/time/rate" "golang.org/x/xerrors" - "github.com/anacrolix/missinggo/v2" - "github.com/anacrolix/missinggo/v2/conntrack" - "github.com/anacrolix/torrent/bencode" + "github.com/anacrolix/torrent/internal/limiter" "github.com/anacrolix/torrent/iplist" "github.com/anacrolix/torrent/metainfo" "github.com/anacrolix/torrent/mse" pp "github.com/anacrolix/torrent/peer_protocol" + request_strategy "github.com/anacrolix/torrent/request-strategy" "github.com/anacrolix/torrent/storage" + "github.com/anacrolix/torrent/tracker" + "github.com/anacrolix/torrent/webtorrent" ) // Clients contain zero or more Torrents. A Client manages a blocklist, the @@ -969,32 +967,6 @@ func (cl *Client) runHandshookConn(c *PeerConn, t *Torrent) error { return nil } -func (pc *PeerConn) startWriter() { - w := &pc.messageWriter - *w = peerConnWriter{ - fillWriteBuffer: func() { - pc.locker().Lock() - defer pc.locker().Unlock() - pc.fillWriteBuffer() - }, - closed: &pc.closed, - logger: pc.logger, - w: pc.w, - keepAlive: func() bool { - pc.locker().Lock() - defer pc.locker().Unlock() - return pc.useful() - }, - writeBuffer: new(bytes.Buffer), - } - go func() { - defer pc.locker().Unlock() - defer pc.close() - defer pc.locker().Lock() - pc.messageWriter.run(time.Minute) - }() -} - // Maximum pending requests we allow peers to send us. If peer requests are buffered on read, this // instructs the amount of memory that might be used to cache pending writes. Assuming 512KiB // (1<<19) cached for sending, for 16KiB (1<<14) chunks. diff --git a/peerconn.go b/peerconn.go index 9cddf7d7..b2753d6a 100644 --- a/peerconn.go +++ b/peerconn.go @@ -18,7 +18,6 @@ import ( "github.com/anacrolix/missinggo/v2/bitmap" "github.com/anacrolix/missinggo/v2/prioritybitmap" "github.com/anacrolix/multiless" - "github.com/anacrolix/sync" "github.com/anacrolix/torrent/bencode" "github.com/anacrolix/torrent/internal/chansync" @@ -148,7 +147,7 @@ type PeerConn struct { w io.Writer r io.Reader - messageWriter peerConnWriter + messageWriter peerConnMsgWriter uploadTimer *time.Timer pex pexConnState @@ -438,7 +437,7 @@ func (cn *PeerConn) write(msg pp.Message) bool { return notFull } -func (cn *peerConnWriter) write(msg pp.Message) bool { +func (cn *peerConnMsgWriter) write(msg pp.Message) bool { cn.mu.Lock() defer cn.mu.Unlock() cn.writeBuffer.Write(msg.MustMarshalBinary()) @@ -446,7 +445,7 @@ func (cn *peerConnWriter) write(msg pp.Message) bool { return !cn.writeBufferFull() } -func (cn *peerConnWriter) writeBufferFull() bool { +func (cn *peerConnMsgWriter) writeBufferFull() bool { return cn.writeBuffer.Len() >= writeBufferHighWaterLen } @@ -643,85 +642,6 @@ func (cn *PeerConn) fillWriteBuffer() { cn.upload(cn.write) } -type peerConnWriter struct { - // Must not be called with the local mutex held, as it will call back into the write method. - fillWriteBuffer func() - closed *chansync.SetOnce - logger log.Logger - w io.Writer - keepAlive func() bool - - mu sync.Mutex - writeCond chansync.BroadcastCond - // Pointer so we can swap with the "front buffer". - writeBuffer *bytes.Buffer -} - -// Routine that writes to the peer. Some of what to write is buffered by -// activity elsewhere in the Client, and some is determined locally when the -// connection is writable. -func (cn *peerConnWriter) run(keepAliveTimeout time.Duration) { - var ( - lastWrite time.Time = time.Now() - keepAliveTimer *time.Timer - ) - keepAliveTimer = time.AfterFunc(keepAliveTimeout, func() { - cn.mu.Lock() - defer cn.mu.Unlock() - if time.Since(lastWrite) >= keepAliveTimeout { - cn.writeCond.Broadcast() - } - keepAliveTimer.Reset(keepAliveTimeout) - }) - cn.mu.Lock() - defer cn.mu.Unlock() - defer keepAliveTimer.Stop() - frontBuf := new(bytes.Buffer) - for { - if cn.closed.IsSet() { - return - } - if cn.writeBuffer.Len() == 0 { - func() { - cn.mu.Unlock() - defer cn.mu.Lock() - cn.fillWriteBuffer() - }() - } - if cn.writeBuffer.Len() == 0 && time.Since(lastWrite) >= keepAliveTimeout && cn.keepAlive() { - cn.writeBuffer.Write(pp.Message{Keepalive: true}.MustMarshalBinary()) - torrent.Add("written keepalives", 1) - } - if cn.writeBuffer.Len() == 0 { - writeCond := cn.writeCond.WaitChan() - cn.mu.Unlock() - select { - case <-cn.closed.Chan(): - case <-writeCond: - } - cn.mu.Lock() - continue - } - // Flip the buffers. - frontBuf, cn.writeBuffer = cn.writeBuffer, frontBuf - cn.mu.Unlock() - n, err := cn.w.Write(frontBuf.Bytes()) - cn.mu.Lock() - if n != 0 { - lastWrite = time.Now() - keepAliveTimer.Reset(keepAliveTimeout) - } - if err != nil { - cn.logger.WithDefaultLevel(log.Debug).Printf("error writing: %v", err) - return - } - if n != frontBuf.Len() { - panic("short write") - } - frontBuf.Reset() - } -} - func (cn *PeerConn) have(piece pieceIndex) { if cn.sentHaves.Get(bitmap.BitIndex(piece)) { return From 36f52d7a145142e682b79563c84b229159fdcb58 Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Thu, 20 May 2021 20:23:45 +1000 Subject: [PATCH 45/59] Apply next request state asynchronously --- client.go | 4 +- internal/chansync/broadcast-cond.go.go | 3 +- internal/chansync/set-once.go | 3 +- peer-impl.go | 5 +- peerconn.go | 94 +++++++++++++------------- pexconn_test.go | 2 +- request-strategy.go | 46 ++++++++++--- webseed-peer.go | 12 +++- 8 files changed, 102 insertions(+), 67 deletions(-) diff --git a/client.go b/client.go index 7ac6b264..1858b848 100644 --- a/client.go +++ b/client.go @@ -33,12 +33,12 @@ import ( "golang.org/x/xerrors" "github.com/anacrolix/torrent/bencode" + "github.com/anacrolix/torrent/internal/chansync" "github.com/anacrolix/torrent/internal/limiter" "github.com/anacrolix/torrent/iplist" "github.com/anacrolix/torrent/metainfo" "github.com/anacrolix/torrent/mse" pp "github.com/anacrolix/torrent/peer_protocol" - request_strategy "github.com/anacrolix/torrent/request-strategy" "github.com/anacrolix/torrent/storage" "github.com/anacrolix/torrent/tracker" "github.com/anacrolix/torrent/webtorrent" @@ -81,7 +81,7 @@ type Client struct { activeAnnounceLimiter limiter.Instance - pieceRequestOrder request_strategy.ClientPieceOrder + updateRequests chansync.BroadcastCond } type ipStr string diff --git a/internal/chansync/broadcast-cond.go.go b/internal/chansync/broadcast-cond.go.go index 9b890692..6d96d3c4 100644 --- a/internal/chansync/broadcast-cond.go.go +++ b/internal/chansync/broadcast-cond.go.go @@ -22,7 +22,8 @@ func (me *BroadcastCond) Broadcast() { } // Should be called before releasing locks on resources that might trigger subsequent Broadcasts. -func (me *BroadcastCond) WaitChan() <-chan struct{} { +// The channel is closed when the condition changes. +func (me *BroadcastCond) Signaled() Signaled { me.mu.Lock() defer me.mu.Unlock() if me.ch == nil { diff --git a/internal/chansync/set-once.go b/internal/chansync/set-once.go index 523e5eaf..db0e6e89 100644 --- a/internal/chansync/set-once.go +++ b/internal/chansync/set-once.go @@ -9,7 +9,8 @@ type SetOnce struct { closeOnce sync.Once } -func (me *SetOnce) Chan() <-chan struct{} { +// Returns a channel that is closed when the event is flagged. +func (me *SetOnce) Done() Done { me.init() return me.ch } diff --git a/peer-impl.go b/peer-impl.go index 880b8f35..23c0fbb9 100644 --- a/peer-impl.go +++ b/peer-impl.go @@ -8,13 +8,14 @@ import ( // BitTorrent protocol connections. Some methods are underlined so as to avoid collisions with // legacy PeerConn methods. type peerImpl interface { + onNextRequestStateChanged() updateRequests() writeInterested(interested bool) bool // Neither of these return buffer room anymore, because they're currently both posted. There's // also PeerConn.writeBufferFull for when/where it matters. - _cancel(Request) - _request(Request) + _cancel(Request) bool + _request(Request) bool connectionFlags() string onClose() diff --git a/peerconn.go b/peerconn.go index b2753d6a..40848e37 100644 --- a/peerconn.go +++ b/peerconn.go @@ -24,6 +24,7 @@ import ( "github.com/anacrolix/torrent/metainfo" "github.com/anacrolix/torrent/mse" pp "github.com/anacrolix/torrent/peer_protocol" + request_strategy "github.com/anacrolix/torrent/request-strategy" ) type PeerSource string @@ -48,7 +49,10 @@ type PeerRemoteAddr interface { // Since we have to store all the requests in memory, we can't reasonably exceed what would be // indexable with the memory space available. -type maxRequests = int +type ( + maxRequests = int + requestState = request_strategy.PeerNextRequestState +) type Peer struct { // First to ensure 64-bit alignment for atomics. See #262. @@ -78,7 +82,8 @@ type Peer struct { lastChunkSent time.Time // Stuff controlled by the local peer. - interested bool + nextRequestState requestState + actualRequestState requestState lastBecameInterested time.Time priorInterest time.Duration @@ -87,7 +92,6 @@ type Peer struct { _chunksReceivedWhileExpecting int64 choking bool - requests map[Request]struct{} piecesReceivedSinceLastRequestUpdate maxRequests maxPiecesReceivedBetweenRequestUpdates maxRequests // Chunks that we might reasonably expect to receive from the peer. Due to @@ -171,13 +175,13 @@ func (cn *Peer) updateExpectingChunks() { } func (cn *Peer) expectingChunks() bool { - if len(cn.requests) == 0 { + if len(cn.actualRequestState.Requests) == 0 { return false } - if !cn.interested { + if !cn.actualRequestState.Interested { return false } - for r := range cn.requests { + for r := range cn.actualRequestState.Requests { if !cn.remoteChokingPiece(r.Index.Int()) { return true } @@ -217,7 +221,7 @@ func (l *PeerConn) hasPreferredNetworkOver(r *PeerConn) (left, ok bool) { func (cn *Peer) cumInterest() time.Duration { ret := cn.priorInterest - if cn.interested { + if cn.actualRequestState.Interested { ret += time.Since(cn.lastBecameInterested) } return ret @@ -301,7 +305,7 @@ func (cn *Peer) statusFlags() (ret string) { c := func(b byte) { ret += string([]byte{b}) } - if cn.interested { + if cn.actualRequestState.Interested { c('i') } if cn.choking { @@ -329,7 +333,7 @@ func (cn *Peer) downloadRate() float64 { func (cn *Peer) numRequestsByPiece() (ret map[pieceIndex]int) { ret = make(map[pieceIndex]int) - for r := range cn.requests { + for r := range cn.actualRequestState.Requests { ret[pieceIndex(r.Index)]++ } return @@ -437,18 +441,6 @@ func (cn *PeerConn) write(msg pp.Message) bool { return notFull } -func (cn *peerConnMsgWriter) write(msg pp.Message) bool { - cn.mu.Lock() - defer cn.mu.Unlock() - cn.writeBuffer.Write(msg.MustMarshalBinary()) - cn.writeCond.Broadcast() - return !cn.writeBufferFull() -} - -func (cn *peerConnMsgWriter) writeBufferFull() bool { - return cn.writeBuffer.Len() >= writeBufferHighWaterLen -} - func (cn *PeerConn) requestMetadataPiece(index int) { eID := cn.PeerExtensionIDs[pp.ExtensionNameMetadata] if eID == 0 { @@ -538,10 +530,10 @@ func (cn *PeerConn) unchoke(msg func(pp.Message) bool) bool { } func (cn *Peer) setInterested(interested bool) bool { - if cn.interested == interested { + if cn.actualRequestState.Interested == interested { return true } - cn.interested = interested + cn.actualRequestState.Interested = interested if interested { cn.lastBecameInterested = time.Now() } else if !cn.lastBecameInterested.IsZero() { @@ -587,20 +579,20 @@ func (cn *Peer) shouldRequest(r Request) error { return nil } -func (cn *Peer) request(r Request) error { +func (cn *Peer) request(r Request) (more bool, err error) { if err := cn.shouldRequest(r); err != nil { panic(err) } - if _, ok := cn.requests[r]; ok { - return nil + if _, ok := cn.actualRequestState.Requests[r]; ok { + return true, nil } if cn.numLocalRequests() >= cn.nominalMaxRequests() { - return errors.New("too many outstanding requests") + return true, errors.New("too many outstanding requests") } - if cn.requests == nil { - cn.requests = make(map[Request]struct{}) + if cn.actualRequestState.Requests == nil { + cn.actualRequestState.Requests = make(map[Request]struct{}) } - cn.requests[r] = struct{}{} + cn.actualRequestState.Requests[r] = struct{}{} if cn.validReceiveChunks == nil { cn.validReceiveChunks = make(map[Request]int) } @@ -610,12 +602,11 @@ func (cn *Peer) request(r Request) error { for _, f := range cn.callbacks.SentRequest { f(PeerRequestEvent{cn, r}) } - cn.peerImpl._request(r) - return nil + return cn.peerImpl._request(r), nil } -func (me *PeerConn) _request(r Request) { - me.write(pp.Message{ +func (me *PeerConn) _request(r Request) bool { + return me.write(pp.Message{ Type: pp.Request, Index: r.Index, Begin: r.Begin, @@ -623,17 +614,21 @@ func (me *PeerConn) _request(r Request) { }) } -func (me *Peer) cancel(r Request) { +func (me *Peer) cancel(r Request) bool { if me.deleteRequest(r) { - me.peerImpl._cancel(r) + return me.peerImpl._cancel(r) } + return true } -func (me *PeerConn) _cancel(r Request) { - me.write(makeCancelMessage(r)) +func (me *PeerConn) _cancel(r Request) bool { + return me.write(makeCancelMessage(r)) } func (cn *PeerConn) fillWriteBuffer() { + if !cn.applyNextRequestState() { + return + } if cn.pex.IsEnabled() { if flow := cn.pex.Share(cn.write); !flow { return @@ -668,8 +663,7 @@ func (cn *PeerConn) postBitfield() { } func (cn *PeerConn) updateRequests() { - // log.Print("update requests") - cn.tickleWriter() + cn.t.cl.tickleRequester() } // Emits the indices in the Bitmaps bms in order, never repeating any index. @@ -1286,7 +1280,7 @@ func (c *Peer) receiveChunk(msg *pp.Message) error { // out. deletedRequest := false { - if _, ok := c.requests[req]; ok { + if _, ok := c.actualRequestState.Requests[req]; ok { for _, f := range c.callbacks.ReceivedRequested { f(PeerMessageEvent{c, msg}) } @@ -1468,14 +1462,15 @@ func (c *Peer) peerHasWantedPieces() bool { } func (c *Peer) numLocalRequests() int { - return len(c.requests) + return len(c.actualRequestState.Requests) } func (c *Peer) deleteRequest(r Request) bool { - if _, ok := c.requests[r]; !ok { + delete(c.nextRequestState.Requests, r) + if _, ok := c.actualRequestState.Requests[r]; !ok { return false } - delete(c.requests, r) + delete(c.actualRequestState.Requests, r) for _, f := range c.callbacks.DeletedRequest { f(PeerRequestEvent{c, r}) } @@ -1493,12 +1488,13 @@ func (c *Peer) deleteRequest(r Request) bool { } func (c *Peer) deleteAllRequests() { - for r := range c.requests { + for r := range c.actualRequestState.Requests { c.deleteRequest(r) } - if len(c.requests) != 0 { - panic(len(c.requests)) + if l := len(c.actualRequestState.Requests); l != 0 { + panic(l) } + c.nextRequestState.Requests = nil // for c := range c.t.conns { // c.tickleWriter() // } @@ -1635,3 +1631,7 @@ func (p *Peer) TryAsPeerConn() (*PeerConn, bool) { pc, ok := p.peerImpl.(*PeerConn) return pc, ok } + +func (p *PeerConn) onNextRequestStateChanged() { + p.tickleWriter() +} diff --git a/pexconn_test.go b/pexconn_test.go index df9aa44e..7bb61ecd 100644 --- a/pexconn_test.go +++ b/pexconn_test.go @@ -36,7 +36,7 @@ func TestPexConnState(t *testing.T) { out = m return true } - <-c.messageWriter.writeCond.WaitChan() + <-c.messageWriter.writeCond.Signaled() c.pex.Share(testWriter) require.True(t, writerCalled) require.EqualValues(t, pp.Extended, out.Type) diff --git a/request-strategy.go b/request-strategy.go index 72f7f3d4..7313c84a 100644 --- a/request-strategy.go +++ b/request-strategy.go @@ -5,25 +5,36 @@ import ( "unsafe" "github.com/anacrolix/missinggo/v2/bitmap" + + "github.com/anacrolix/torrent/internal/chansync" request_strategy "github.com/anacrolix/torrent/request-strategy" "github.com/anacrolix/torrent/types" ) func (cl *Client) requester() { for { - func() { + update := func() chansync.Signaled { cl.lock() defer cl.unlock() cl.doRequests() + return cl.updateRequests.Signaled() }() + // We can probably tune how often to heed this signal. TODO: Currently disabled to retain + // existing behaviour, while the signalling is worked out. + update = nil select { case <-cl.closed.LockedChan(cl.locker()): return + case <-update: case <-time.After(100 * time.Millisecond): } } } +func (cl *Client) tickleRequester() { + cl.updateRequests.Broadcast() +} + func (cl *Client) doRequests() { ts := make([]request_strategy.Torrent, 0, len(cl.torrents)) for _, t := range cl.torrents { @@ -62,7 +73,7 @@ func (cl *Client) doRequests() { HasPiece: p.peerHasPiece, MaxRequests: p.nominalMaxRequests(), HasExistingRequest: func(r request_strategy.Request) bool { - _, ok := p.requests[r] + _, ok := p.actualRequestState.Requests[r] return ok }, Choking: p.peerChoking, @@ -81,7 +92,7 @@ func (cl *Client) doRequests() { MaxUnverifiedBytes: cl.config.MaxUnverifiedBytes, }) for p, state := range nextPeerStates { - applyPeerNextRequestState(p, state) + setPeerNextRequestState(p, state) } } @@ -91,20 +102,35 @@ func (p *peerId) Uintptr() uintptr { return uintptr(unsafe.Pointer(p)) } -func applyPeerNextRequestState(_p request_strategy.PeerId, rp request_strategy.PeerNextRequestState) { +func setPeerNextRequestState(_p request_strategy.PeerId, rp request_strategy.PeerNextRequestState) { p := (*Peer)(_p.(*peerId)) - p.setInterested(rp.Interested) - for req := range p.requests { - if _, ok := rp.Requests[req]; !ok { - p.cancel(req) + p.nextRequestState = rp + p.onNextRequestStateChanged() +} + +func (p *Peer) applyNextRequestState() bool { + next := p.nextRequestState + current := p.actualRequestState + if !p.setInterested(next.Interested) { + return false + } + for req := range current.Requests { + if _, ok := next.Requests[req]; !ok { + if !p.cancel(req) { + return false + } } } - for req := range rp.Requests { - err := p.request(req) + for req := range next.Requests { + more, err := p.request(req) if err != nil { panic(err) } else { //log.Print(req) } + if !more { + return false + } } + return true } diff --git a/webseed-peer.go b/webseed-peer.go index 5f2980c3..7af892f0 100644 --- a/webseed-peer.go +++ b/webseed-peer.go @@ -45,19 +45,21 @@ func (ws *webseedPeer) writeInterested(interested bool) bool { return true } -func (ws *webseedPeer) _cancel(r Request) { +func (ws *webseedPeer) _cancel(r Request) bool { active, ok := ws.activeRequests[r] if ok { active.Cancel() } + return true } func (ws *webseedPeer) intoSpec(r Request) webseed.RequestSpec { return webseed.RequestSpec{ws.peer.t.requestOffset(r), int64(r.Length)} } -func (ws *webseedPeer) _request(r Request) { +func (ws *webseedPeer) _request(r Request) bool { ws.requesterCond.Signal() + return true } func (ws *webseedPeer) doRequest(r Request) { @@ -76,7 +78,7 @@ func (ws *webseedPeer) requester() { defer ws.requesterCond.L.Unlock() start: for !ws.peer.closed.IsSet() { - for r := range ws.peer.requests { + for r := range ws.peer.actualRequestState.Requests { if _, ok := ws.activeRequests[r]; ok { continue } @@ -142,3 +144,7 @@ func (ws *webseedPeer) requestResultHandler(r Request, webseedRequest webseed.Re } } } + +func (me *webseedPeer) onNextRequestStateChanged() { + me.peer.applyNextRequestState() +} From 47c49735fc7cee62bd28454aa2827db34b798d29 Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Thu, 20 May 2021 20:24:34 +1000 Subject: [PATCH 46/59] Fix race in piece availability runs --- torrent.go | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/torrent.go b/torrent.go index e500a47d..41460412 100644 --- a/torrent.go +++ b/torrent.go @@ -18,13 +18,6 @@ import ( "time" "unsafe" - "github.com/anacrolix/multiless" - "github.com/anacrolix/torrent/common" - "github.com/anacrolix/torrent/segments" - "github.com/anacrolix/torrent/webseed" - "github.com/davecgh/go-spew/spew" - "github.com/pion/datachannel" - "github.com/anacrolix/dht/v2" "github.com/anacrolix/log" "github.com/anacrolix/missinggo" @@ -34,12 +27,18 @@ import ( "github.com/anacrolix/missinggo/slices" "github.com/anacrolix/missinggo/v2/bitmap" "github.com/anacrolix/missinggo/v2/prioritybitmap" + "github.com/anacrolix/multiless" + "github.com/davecgh/go-spew/spew" + "github.com/pion/datachannel" "github.com/anacrolix/torrent/bencode" + "github.com/anacrolix/torrent/common" "github.com/anacrolix/torrent/metainfo" pp "github.com/anacrolix/torrent/peer_protocol" + "github.com/anacrolix/torrent/segments" "github.com/anacrolix/torrent/storage" "github.com/anacrolix/torrent/tracker" + "github.com/anacrolix/torrent/webseed" "github.com/anacrolix/torrent/webtorrent" ) @@ -574,8 +573,8 @@ func (t *Torrent) pieceAvailabilityRuns() (ret []pieceAvailabilityRun) { rle := missinggo.NewRunLengthEncoder(func(el interface{}, count uint64) { ret = append(ret, pieceAvailabilityRun{availability: el.(int64), count: int(count)}) }) - for _, p := range t.pieces { - rle.Append(p.availability, 1) + for i := range t.pieces { + rle.Append(t.pieces[i].availability, 1) } rle.Flush() return From a2dbec3b4a02b781ab9a6afc527ef5aab371b49a Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Thu, 20 May 2021 20:27:54 +1000 Subject: [PATCH 47/59] Add chansync interfaces --- internal/chansync/interfaces.go | 8 ++++++++ 1 file changed, 8 insertions(+) create mode 100644 internal/chansync/interfaces.go diff --git a/internal/chansync/interfaces.go b/internal/chansync/interfaces.go new file mode 100644 index 00000000..751ba0ef --- /dev/null +++ b/internal/chansync/interfaces.go @@ -0,0 +1,8 @@ +package chansync + +// Here we'll strongly-type channels to assist correct usage, if possible. + +type ( + Signaled <-chan struct{} + Done <-chan struct{} +) From 906db030f3c9aa8cdcaf14975f5d875d5d62579c Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Thu, 20 May 2021 20:28:23 +1000 Subject: [PATCH 48/59] Add peerConnMsgWriter file Forgot this earlier, new to Sublime Merge. --- peer-conn-msg-writer.go | 133 ++++++++++++++++++++++++++++++++++++++++ 1 file changed, 133 insertions(+) create mode 100644 peer-conn-msg-writer.go diff --git a/peer-conn-msg-writer.go b/peer-conn-msg-writer.go new file mode 100644 index 00000000..809d85fc --- /dev/null +++ b/peer-conn-msg-writer.go @@ -0,0 +1,133 @@ +package torrent + +import ( + "bytes" + "io" + "time" + + "github.com/anacrolix/log" + "github.com/anacrolix/sync" + + "github.com/anacrolix/torrent/internal/chansync" + pp "github.com/anacrolix/torrent/peer_protocol" +) + +func (pc *PeerConn) startWriter() { + w := &pc.messageWriter + *w = peerConnMsgWriter{ + fillWriteBuffer: func() { + pc.locker().Lock() + defer pc.locker().Unlock() + if pc.closed.IsSet() { + return + } + pc.fillWriteBuffer() + }, + closed: &pc.closed, + logger: pc.logger, + w: pc.w, + keepAlive: func() bool { + pc.locker().Lock() + defer pc.locker().Unlock() + return pc.useful() + }, + writeBuffer: new(bytes.Buffer), + } + go func() { + defer pc.locker().Unlock() + defer pc.close() + defer pc.locker().Lock() + pc.messageWriter.run(time.Minute) + }() +} + +type peerConnMsgWriter struct { + // Must not be called with the local mutex held, as it will call back into the write method. + fillWriteBuffer func() + closed *chansync.SetOnce + logger log.Logger + w io.Writer + keepAlive func() bool + + mu sync.Mutex + writeCond chansync.BroadcastCond + // Pointer so we can swap with the "front buffer". + writeBuffer *bytes.Buffer +} + +// Routine that writes to the peer. Some of what to write is buffered by +// activity elsewhere in the Client, and some is determined locally when the +// connection is writable. +func (cn *peerConnMsgWriter) run(keepAliveTimeout time.Duration) { + var ( + lastWrite time.Time = time.Now() + keepAliveTimer *time.Timer + ) + keepAliveTimer = time.AfterFunc(keepAliveTimeout, func() { + cn.mu.Lock() + defer cn.mu.Unlock() + if time.Since(lastWrite) >= keepAliveTimeout { + cn.writeCond.Broadcast() + } + keepAliveTimer.Reset(keepAliveTimeout) + }) + cn.mu.Lock() + defer cn.mu.Unlock() + defer keepAliveTimer.Stop() + frontBuf := new(bytes.Buffer) + for { + if cn.closed.IsSet() { + return + } + if cn.writeBuffer.Len() == 0 { + func() { + cn.mu.Unlock() + defer cn.mu.Lock() + cn.fillWriteBuffer() + }() + } + if cn.writeBuffer.Len() == 0 && time.Since(lastWrite) >= keepAliveTimeout && cn.keepAlive() { + cn.writeBuffer.Write(pp.Message{Keepalive: true}.MustMarshalBinary()) + torrent.Add("written keepalives", 1) + } + if cn.writeBuffer.Len() == 0 { + writeCond := cn.writeCond.Signaled() + cn.mu.Unlock() + select { + case <-cn.closed.Done(): + case <-writeCond: + } + cn.mu.Lock() + continue + } + // Flip the buffers. + frontBuf, cn.writeBuffer = cn.writeBuffer, frontBuf + cn.mu.Unlock() + n, err := cn.w.Write(frontBuf.Bytes()) + cn.mu.Lock() + if n != 0 { + lastWrite = time.Now() + keepAliveTimer.Reset(keepAliveTimeout) + } + if err != nil { + cn.logger.WithDefaultLevel(log.Debug).Printf("error writing: %v", err) + return + } + if n != frontBuf.Len() { + panic("short write") + } + frontBuf.Reset() + } +} + +func (cn *peerConnMsgWriter) write(msg pp.Message) bool { + cn.mu.Lock() + defer cn.mu.Unlock() + cn.writeBuffer.Write(msg.MustMarshalBinary()) + cn.writeCond.Broadcast() + return !cn.writeBufferFull() +} + +func (cn *peerConnMsgWriter) writeBufferFull() bool { + return cn.writeBuffer.Len() >= writeBufferHighWaterLen +} From 33d3a75cd33569a4d19e878101b65fa79017b265 Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Thu, 20 May 2021 20:33:17 +1000 Subject: [PATCH 49/59] Rename file --- request-strategy.go => requesting.go | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename request-strategy.go => requesting.go (100%) diff --git a/request-strategy.go b/requesting.go similarity index 100% rename from request-strategy.go rename to requesting.go From c895a21a165090d8a37ac43350e46eb67a00dd4d Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Fri, 21 May 2021 11:49:57 +1000 Subject: [PATCH 50/59] Track chunks read for webseed peers too --- conn_stats.go | 11 +++-------- peerconn.go | 15 ++++++++------- webseed-peer.go | 4 ++++ 3 files changed, 15 insertions(+), 15 deletions(-) diff --git a/conn_stats.go b/conn_stats.go index d2d52003..0c5bfc78 100644 --- a/conn_stats.go +++ b/conn_stats.go @@ -79,14 +79,9 @@ func (cs *ConnStats) wroteMsg(msg *pp.Message) { } } -func (cs *ConnStats) readMsg(msg *pp.Message) { - // We want to also handle extended metadata pieces here, but we wouldn't - // have decoded the extended payload yet. - switch msg.Type { - case pp.Piece: - cs.ChunksRead.Add(1) - cs.BytesReadData.Add(int64(len(msg.Piece))) - } +func (cs *ConnStats) receivedChunk(size int64) { + cs.ChunksRead.Add(1) + cs.BytesReadData.Add(size) } func (cs *ConnStats) incrementPiecesDirtiedGood() { diff --git a/peerconn.go b/peerconn.go index 40848e37..46756320 100644 --- a/peerconn.go +++ b/peerconn.go @@ -856,10 +856,6 @@ func (cn *PeerConn) wroteMsg(msg *pp.Message) { cn.allStats(func(cs *ConnStats) { cs.wroteMsg(msg) }) } -func (cn *PeerConn) readMsg(msg *pp.Message) { - cn.allStats(func(cs *ConnStats) { cs.readMsg(msg) }) -} - // After handshake, we know what Torrent and Client stats to include for a // connection. func (cn *Peer) postHandshakeStats(f func(*ConnStats)) { @@ -1065,7 +1061,6 @@ func (c *PeerConn) mainReadLoop() (err error) { if err != nil { return err } - c.readMsg(&msg) c.lastMessageReceived = time.Now() if msg.Keepalive { receivedKeepalives.Add(1) @@ -1105,6 +1100,7 @@ func (c *PeerConn) mainReadLoop() (err error) { r := newRequestFromMessage(&msg) err = c.onReadRequest(r) case pp.Piece: + c.doChunkReadStats(int64(len(msg.Piece))) err = c.receiveChunk(&msg) if len(msg.Piece) == int(t.chunkSize) { t.chunkPool.Put(&msg.Piece) @@ -1253,10 +1249,12 @@ func (cn *PeerConn) rw() io.ReadWriter { }{cn.r, cn.w} } +func (c *Peer) doChunkReadStats(size int64) { + c.allStats(func(cs *ConnStats) { cs.receivedChunk(size) }) +} + // Handle a received chunk from a peer. func (c *Peer) receiveChunk(msg *pp.Message) error { - t := c.t - cl := t.cl chunksReceived.Add("total", 1) req := newRequestFromMessage(msg) @@ -1296,6 +1294,9 @@ func (c *Peer) receiveChunk(msg *pp.Message) error { } } + t := c.t + cl := t.cl + // Do we actually want this chunk? if t.haveChunk(req) { chunksReceived.Add("wasted", 1) diff --git a/webseed-peer.go b/webseed-peer.go index 7af892f0..f2ccbff5 100644 --- a/webseed-peer.go +++ b/webseed-peer.go @@ -110,6 +110,10 @@ func (ws *webseedPeer) onClose() { func (ws *webseedPeer) requestResultHandler(r Request, webseedRequest webseed.Request) { result := <-webseedRequest.Result + // We do this here rather than inside receiveChunk, since we want to count errors too. I'm not + // sure if we can divine which errors indicate cancellation on our end without hitting the + // network though. + ws.peer.doChunkReadStats(int64(len(result.Bytes))) ws.peer.t.cl.lock() defer ws.peer.t.cl.unlock() if result.Err != nil { From e0a4831f27a1715f0814eeb86ca8e9fca2ac812d Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Fri, 21 May 2021 14:01:41 +1000 Subject: [PATCH 51/59] Fix bitmap-related benchmark --- misc_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/misc_test.go b/misc_test.go index f3a37683..5190e4f4 100644 --- a/misc_test.go +++ b/misc_test.go @@ -35,7 +35,7 @@ func BenchmarkIterBitmapsDistinct(t *testing.B) { output := iter.ToSlice(iterBitmapsDistinct(&skipCopy, first, second)) t.StopTimer() assert.Equal(t, []interface{}{0, 3, 2}, output) - assert.Equal(t, []int{1}, skip.ToSortedSlice()) + assert.Equal(t, []bitmap.BitIndex{1}, skip.ToSortedSlice()) } } From f50b8fc2fa6d6069ae461860af5a0f209f05b3c9 Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Fri, 21 May 2021 14:02:45 +1000 Subject: [PATCH 52/59] Fix issue with duplicate preallocated requests Likely introduced by allowing actual and next request states to differ. --- request-strategy/order.go | 43 ++++++++++++++++++++++++++------- request-strategy/order_test.go | 44 +++++++++++++++++++++++++++++++++- 2 files changed, 78 insertions(+), 9 deletions(-) diff --git a/request-strategy/order.go b/request-strategy/order.go index 2ac04ab4..2a364eb8 100644 --- a/request-strategy/order.go +++ b/request-strategy/order.go @@ -1,9 +1,11 @@ package request_strategy import ( + "fmt" "sort" "github.com/anacrolix/multiless" + pp "github.com/anacrolix/torrent/peer_protocol" "github.com/anacrolix/torrent/types" ) @@ -160,7 +162,7 @@ func Run(input Input) map[PeerId]PeerNextRequestState { peers = append(peers, &requestsPeer{ Peer: p, nextState: PeerNextRequestState{ - Requests: make(map[Request]struct{}), + Requests: make(map[Request]struct{}, p.MaxRequests), }, }) } @@ -182,12 +184,29 @@ func Run(input Input) map[PeerId]PeerNextRequestState { if rp.requestablePiecesRemaining != 0 { panic(rp.requestablePiecesRemaining) } + if _, ok := ret[rp.Id]; ok { + panic(fmt.Sprintf("duplicate peer id: %v", rp.Id)) + } ret[rp.Id] = rp.nextState } } return ret } +// Checks that a sorted peersForPiece slice makes sense. +func ensureValidSortedPeersForPieceRequests(peers []*peersForPieceRequests, sortLess func(_, _ int) bool) { + if !sort.SliceIsSorted(peers, sortLess) { + panic("not sorted") + } + peerMap := make(map[*peersForPieceRequests]struct{}, len(peers)) + for _, p := range peers { + if _, ok := peerMap[p]; ok { + panic(p) + } + peerMap[p] = struct{}{} + } +} + func allocatePendingChunks(p requestablePiece, peers []*requestsPeer) { peersForPiece := make([]*peersForPieceRequests, 0, len(peers)) for _, peer := range peers { @@ -204,7 +223,7 @@ func allocatePendingChunks(p requestablePiece, peers []*requestsPeer) { } }() sortPeersForPiece := func(req *Request) { - sort.Slice(peersForPiece, func(i, j int) bool { + less := func(i, j int) bool { byHasRequest := func() multiless.Computation { ml := multiless.New() if req != nil { @@ -246,9 +265,13 @@ func allocatePendingChunks(p requestablePiece, peers []*requestsPeer) { peersForPiece[i].Id.Uintptr(), peersForPiece[j].Id.Uintptr(), ).MustLess() - }) + } + sort.Slice(peersForPiece, less) + ensureValidSortedPeersForPieceRequests(peersForPiece, less) } - preallocated := make(map[ChunkSpec]*peersForPieceRequests, p.NumPendingChunks) + // Chunks can be preassigned several times, if peers haven't been able to update their "actual" + // with "next" request state before another request strategy run occurs. + preallocated := make(map[ChunkSpec][]*peersForPieceRequests, p.NumPendingChunks) p.IterPendingChunks(func(spec ChunkSpec) { req := Request{pp.Integer(p.index), spec} for _, peer := range peersForPiece { @@ -261,7 +284,7 @@ func allocatePendingChunks(p requestablePiece, peers []*requestsPeer) { if !peer.canRequestPiece(p.index) { continue } - preallocated[spec] = peer + preallocated[spec] = append(preallocated[spec], peer) peer.addNextRequest(req) } }) @@ -292,12 +315,16 @@ func allocatePendingChunks(p requestablePiece, peers []*requestsPeer) { } }) chunk: - for chunk, prePeer := range preallocated { + for chunk, prePeers := range preallocated { pendingChunksRemaining-- req := Request{pp.Integer(p.index), chunk} - prePeer.requestsInPiece-- + for _, pp := range prePeers { + pp.requestsInPiece-- + } sortPeersForPiece(&req) - delete(prePeer.nextState.Requests, req) + for _, pp := range prePeers { + delete(pp.nextState.Requests, req) + } for _, peer := range peersForPiece { if !peer.canFitRequest() { continue diff --git a/request-strategy/order_test.go b/request-strategy/order_test.go index a448ac37..d37b0423 100644 --- a/request-strategy/order_test.go +++ b/request-strategy/order_test.go @@ -4,9 +4,10 @@ import ( "math" "testing" - pp "github.com/anacrolix/torrent/peer_protocol" "github.com/bradfitz/iter" qt "github.com/frankban/quicktest" + + pp "github.com/anacrolix/torrent/peer_protocol" ) func r(i pieceIndex, begin int) Request { @@ -253,3 +254,44 @@ func TestDontStealUnnecessarily(t *testing.T) { Requests: requestSetFromSlice(r(4, 0), r(4, 1), r(4, 7), r(4, 8)), }) } + +// This tests a situation where multiple peers had the same existing request, due to "actual" and +// "next" request states being out of sync. This reasonable occurs when a peer hasn't fully updated +// its actual request state since the last request strategy run. +func TestDuplicatePreallocations(t *testing.T) { + peer := func(id int, downloadRate float64) Peer { + return Peer{ + HasExistingRequest: func(r Request) bool { + return true + }, + MaxRequests: 2, + HasPiece: func(i pieceIndex) bool { + return true + }, + Id: intPeerId(id), + DownloadRate: downloadRate, + } + } + results := Run(Input{ + Torrents: []Torrent{{ + Pieces: []Piece{{ + Request: true, + NumPendingChunks: 1, + IterPendingChunks: chunkIterRange(1), + }, { + Request: true, + NumPendingChunks: 1, + IterPendingChunks: chunkIterRange(1), + }}, + Peers: []Peer{ + // The second peer was be marked as the preallocation, clobbering the first. The + // first peer is preferred, and the piece isn't striped, so it gets preallocated a + // request, and then gets reallocated from the peer the same request. + peer(1, 2), + peer(2, 1), + }, + }}, + }) + c := qt.New(t) + c.Assert(2, qt.Equals, len(results[intPeerId(1)].Requests)+len(results[intPeerId(2)].Requests)) +} From 259356ccd6d1f921f50400b41bfbd477a8757df8 Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Fri, 21 May 2021 23:50:29 +1000 Subject: [PATCH 53/59] Rework bencode decoding so it might support embedded structs --- bencode/decode.go | 103 +++++++++++++++++++++-------------------- bencode/decode_test.go | 3 +- bencode/tags.go | 3 ++ 3 files changed, 58 insertions(+), 51 deletions(-) diff --git a/bencode/decode.go b/bencode/decode.go index 8b22fa73..51804614 100644 --- a/bencode/decode.go +++ b/bencode/decode.go @@ -205,50 +205,40 @@ func (d *Decoder) parseString(v reflect.Value) error { // Info for parsing a dict value. type dictField struct { - Value reflect.Value // Storage for the parsed value. - // True if field value should be parsed into Value. If false, the value - // should be parsed and discarded. - Ok bool - Set func() // Call this after parsing into Value. - IgnoreUnmarshalTypeError bool + Type reflect.Type + Get func(value reflect.Value) func(reflect.Value) + Tags tag } // Returns specifics for parsing a dict field value. -func getDictField(dict reflect.Value, key string) dictField { +func getDictField(dict reflect.Type, key string) dictField { // get valuev as a map value or as a struct field switch dict.Kind() { case reflect.Map: - value := reflect.New(dict.Type().Elem()).Elem() return dictField{ - Value: value, - Ok: true, - Set: func() { - if dict.IsNil() { - dict.Set(reflect.MakeMap(dict.Type())) + Type: dict.Elem(), + Get: func(mapValue reflect.Value) func(reflect.Value) { + return func(value reflect.Value) { + if mapValue.IsNil() { + mapValue.Set(reflect.MakeMap(dict)) + } + // Assigns the value into the map. + //log.Printf("map type: %v", mapValue.Type()) + mapValue.SetMapIndex(reflect.ValueOf(key).Convert(dict.Key()), value) } - // Assigns the value into the map. - dict.SetMapIndex(reflect.ValueOf(key).Convert(dict.Type().Key()), value) }, } case reflect.Struct: - sf, ok := getStructFieldForKey(dict.Type(), key) - if !ok { - return dictField{} - } - if sf.r.PkgPath != "" { - panic(&UnmarshalFieldError{ - Key: key, - Type: dict.Type(), - Field: sf.r, - }) - } - return dictField{ - Value: dict.FieldByIndex(sf.r.Index), - Ok: true, - Set: func() {}, - IgnoreUnmarshalTypeError: sf.tag.IgnoreUnmarshalTypeError(), - } + return getStructFieldForKey(dict, key) + //if sf.r.PkgPath != "" { + // panic(&UnmarshalFieldError{ + // Key: key, + // Type: dict.Type(), + // Field: sf.r, + // }) + //} default: + panic("unimplemented") return dictField{} } } @@ -260,11 +250,12 @@ type structField struct { var ( structFieldsMu sync.Mutex - structFields = map[reflect.Type]map[string]structField{} + structFields = map[reflect.Type]map[string]dictField{} ) -func parseStructFields(struct_ reflect.Type, each func(string, structField)) { - for i, n := 0, struct_.NumField(); i < n; i++ { +func parseStructFields(struct_ reflect.Type, each func(string, dictField)) { + for _i, n := 0, struct_.NumField(); _i < n; _i++ { + i := _i f := struct_.Field(i) if f.Anonymous { continue @@ -278,25 +269,35 @@ func parseStructFields(struct_ reflect.Type, each func(string, structField)) { if key == "" { key = f.Name } - each(key, structField{f, tag}) + each(key, dictField{f.Type, func(value reflect.Value) func(reflect.Value) { + return value.Field(i).Set + }, tag}) } } func saveStructFields(struct_ reflect.Type) { - m := make(map[string]structField) - parseStructFields(struct_, func(key string, sf structField) { + m := make(map[string]dictField) + parseStructFields(struct_, func(key string, sf dictField) { m[key] = sf }) structFields[struct_] = m } -func getStructFieldForKey(struct_ reflect.Type, key string) (f structField, ok bool) { +func getStructFieldForKey(struct_ reflect.Type, key string) (f dictField) { structFieldsMu.Lock() if _, ok := structFields[struct_]; !ok { saveStructFields(struct_) } - f, ok = structFields[struct_][key] + f, ok := structFields[struct_][key] structFieldsMu.Unlock() + if !ok { + var discard interface{} + return dictField{ + Type: reflect.TypeOf(discard), + Get: func(reflect.Value) func(reflect.Value) { return func(reflect.Value) {} }, + Tags: nil, + } + } return } @@ -314,31 +315,33 @@ func (d *Decoder) parseDict(v reflect.Value) error { return nil } - df := getDictField(v, keyStr) + df := getDictField(v.Type(), keyStr) // now we need to actually parse it - if df.Ok { - // log.Printf("parsing ok struct field for key %q", keyStr) - ok, err = d.parseValue(df.Value) - } else { + if df.Type == nil { // Discard the value, there's nowhere to put it. var if_ interface{} if_, ok = d.parseValueInterface() if if_ == nil { - err = fmt.Errorf("error parsing value for key %q", keyStr) + return fmt.Errorf("error parsing value for key %q", keyStr) } + if !ok { + return fmt.Errorf("missing value for key %q", keyStr) + } + continue } + setValue := reflect.New(df.Type).Elem() + //log.Printf("parsing into %v", setValue.Type()) + ok, err = d.parseValue(setValue) if err != nil { - if _, ok := err.(*UnmarshalTypeError); !ok || !df.IgnoreUnmarshalTypeError { + if _, ok := err.(*UnmarshalTypeError); !ok || !df.Tags.IgnoreUnmarshalTypeError() { return fmt.Errorf("parsing value for key %q: %s", keyStr, err) } } if !ok { return fmt.Errorf("missing value for key %q", keyStr) } - if df.Ok { - df.Set() - } + df.Get(v)(setValue) } } diff --git a/bencode/decode_test.go b/bencode/decode_test.go index 4b72edbb..056a399a 100644 --- a/bencode/decode_test.go +++ b/bencode/decode_test.go @@ -7,6 +7,7 @@ import ( "reflect" "testing" + qt "github.com/frankban/quicktest" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) @@ -144,7 +145,7 @@ func TestIgnoreUnmarshalTypeError(t *testing.T) { }{} require.Error(t, Unmarshal([]byte("d6:Normal5:helloe"), &s)) assert.NoError(t, Unmarshal([]byte("d6:Ignore5:helloe"), &s)) - require.Nil(t, Unmarshal([]byte("d6:Ignorei42ee"), &s)) + qt.Assert(t, Unmarshal([]byte("d6:Ignorei42ee"), &s), qt.IsNil) assert.EqualValues(t, 42, s.Ignore) } diff --git a/bencode/tags.go b/bencode/tags.go index 50bdc72b..d4adeb24 100644 --- a/bencode/tags.go +++ b/bencode/tags.go @@ -24,6 +24,9 @@ func (me tag) Key() string { } func (me tag) HasOpt(opt string) bool { + if len(me) < 1 { + return false + } for _, s := range me[1:] { if s == opt { return true From 25d2eea12d9d5302a216d58fa32032447dd1abb5 Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Sat, 22 May 2021 11:02:39 +1000 Subject: [PATCH 54/59] bencode: Support anonymous embedded struct pointers More to come if this line of improvement is retained. --- bencode/decode.go | 18 ++++++++++------ bencode/encode.go | 52 +++++++++++++++++++++++++++++++---------------- 2 files changed, 47 insertions(+), 23 deletions(-) diff --git a/bencode/decode.go b/bencode/decode.go index 51804614..e891f5be 100644 --- a/bencode/decode.go +++ b/bencode/decode.go @@ -243,21 +243,27 @@ func getDictField(dict reflect.Type, key string) dictField { } } -type structField struct { - r reflect.StructField - tag tag -} - var ( structFieldsMu sync.Mutex structFields = map[reflect.Type]map[string]dictField{} ) -func parseStructFields(struct_ reflect.Type, each func(string, dictField)) { +func parseStructFields(struct_ reflect.Type, each func(key string, df dictField)) { for _i, n := 0, struct_.NumField(); _i < n; _i++ { i := _i f := struct_.Field(i) if f.Anonymous { + parseStructFields(f.Type.Elem(), func(key string, df dictField) { + innerGet := df.Get + df.Get = func(value reflect.Value) func(reflect.Value) { + anonPtr := value.Field(i) + if anonPtr.IsNil() { + anonPtr.Set(reflect.New(f.Type.Elem())) + } + return innerGet(anonPtr.Elem()) + } + each(key, df) + }) continue } tagStr := f.Tag.Get("bencode") diff --git a/bencode/encode.go b/bencode/encode.go index 443c11e7..f25cfef8 100644 --- a/bencode/encode.go +++ b/bencode/encode.go @@ -133,13 +133,16 @@ func (e *Encoder) reflectValue(v reflect.Value) { e.reflectString(v.String()) case reflect.Struct: e.writeString("d") - for _, ef := range encodeFields(v.Type()) { - field_value := v.Field(ef.i) - if ef.omit_empty && isEmptyValue(field_value) { + for _, ef := range getEncodeFields(v.Type()) { + fieldValue := ef.i(v) + if !fieldValue.IsValid() { + continue + } + if ef.omitEmpty && isEmptyValue(fieldValue) { continue } e.reflectString(ef.tag) - e.reflectValue(field_value) + e.reflectValue(fieldValue) } e.writeString("e") case reflect.Map: @@ -190,9 +193,9 @@ func (e *Encoder) reflectValue(v reflect.Value) { } type encodeField struct { - i int - tag string - omit_empty bool + i func(v reflect.Value) reflect.Value + tag string + omitEmpty bool } type encodeFieldsSortType []encodeField @@ -206,31 +209,47 @@ var ( encodeFieldsCache = make(map[reflect.Type][]encodeField) ) -func encodeFields(t reflect.Type) []encodeField { +func getEncodeFields(t reflect.Type) []encodeField { typeCacheLock.RLock() fs, ok := encodeFieldsCache[t] typeCacheLock.RUnlock() if ok { return fs } - + fs = makeEncodeFields(t) typeCacheLock.Lock() defer typeCacheLock.Unlock() - fs, ok = encodeFieldsCache[t] - if ok { - return fs - } + encodeFieldsCache[t] = fs + return fs +} - for i, n := 0, t.NumField(); i < n; i++ { +func makeEncodeFields(t reflect.Type) (fs []encodeField) { + for _i, n := 0, t.NumField(); _i < n; _i++ { + i := _i f := t.Field(i) if f.PkgPath != "" { continue } if f.Anonymous { + anonEFs := makeEncodeFields(f.Type.Elem()) + for aefi := range anonEFs { + anonEF := anonEFs[aefi] + bottomField := anonEF + bottomField.i = func(v reflect.Value) reflect.Value { + v = v.Field(i) + if v.IsNil() { + return reflect.Value{} + } + return anonEF.i(v.Elem()) + } + fs = append(fs, bottomField) + } continue } var ef encodeField - ef.i = i + ef.i = func(v reflect.Value) reflect.Value { + return v.Field(i) + } ef.tag = f.Name tv := getTag(f.Tag) @@ -240,11 +259,10 @@ func encodeFields(t reflect.Type) []encodeField { if tv.Key() != "" { ef.tag = tv.Key() } - ef.omit_empty = tv.OmitEmpty() + ef.omitEmpty = tv.OmitEmpty() fs = append(fs, ef) } fss := encodeFieldsSortType(fs) sort.Sort(fss) - encodeFieldsCache[t] = fs return fs } From 047cdbae0d95dbef453c17aec7d48d6ef7b79cc9 Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Mon, 24 May 2021 17:31:52 +1000 Subject: [PATCH 55/59] bencode: Improve support for embedded structs --- bencode/decode.go | 11 ++++++++--- bencode/encode.go | 16 ++++++++++++---- 2 files changed, 20 insertions(+), 7 deletions(-) diff --git a/bencode/decode.go b/bencode/decode.go index e891f5be..43bb9ce6 100644 --- a/bencode/decode.go +++ b/bencode/decode.go @@ -253,14 +253,19 @@ func parseStructFields(struct_ reflect.Type, each func(key string, df dictField) i := _i f := struct_.Field(i) if f.Anonymous { - parseStructFields(f.Type.Elem(), func(key string, df dictField) { + t := f.Type + if t.Kind() == reflect.Ptr { + t = t.Elem() + } + parseStructFields(t, func(key string, df dictField) { innerGet := df.Get df.Get = func(value reflect.Value) func(reflect.Value) { anonPtr := value.Field(i) - if anonPtr.IsNil() { + if anonPtr.Kind() == reflect.Ptr && anonPtr.IsNil() { anonPtr.Set(reflect.New(f.Type.Elem())) + anonPtr = anonPtr.Elem() } - return innerGet(anonPtr.Elem()) + return innerGet(anonPtr) } each(key, df) }) diff --git a/bencode/encode.go b/bencode/encode.go index f25cfef8..05153a8c 100644 --- a/bencode/encode.go +++ b/bencode/encode.go @@ -231,16 +231,24 @@ func makeEncodeFields(t reflect.Type) (fs []encodeField) { continue } if f.Anonymous { - anonEFs := makeEncodeFields(f.Type.Elem()) + t := f.Type + if t.Kind() == reflect.Ptr { + t = t.Elem() + } + anonEFs := makeEncodeFields(t) for aefi := range anonEFs { anonEF := anonEFs[aefi] bottomField := anonEF bottomField.i = func(v reflect.Value) reflect.Value { v = v.Field(i) - if v.IsNil() { - return reflect.Value{} + if v.Kind() == reflect.Ptr { + if v.IsNil() { + // This will skip serializing this value. + return reflect.Value{} + } + v = v.Elem() } - return anonEF.i(v.Elem()) + return anonEF.i(v) } fs = append(fs, bottomField) } From 7d9e2b18d69878d5cb4e6e276ceedeeae5598911 Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Mon, 24 May 2021 17:36:39 +1000 Subject: [PATCH 56/59] Extract chansync to github.com/anacrolix/chansync --- client.go | 9 ++++-- go.mod | 1 + go.sum | 2 ++ internal/chansync/broadcast-cond.go.go | 33 -------------------- internal/chansync/interfaces.go | 8 ----- internal/chansync/set-once.go | 42 -------------------------- peer-conn-msg-writer.go | 2 +- peerconn.go | 2 +- requesting.go | 2 +- 9 files changed, 13 insertions(+), 88 deletions(-) delete mode 100644 internal/chansync/broadcast-cond.go.go delete mode 100644 internal/chansync/interfaces.go delete mode 100644 internal/chansync/set-once.go diff --git a/client.go b/client.go index 1858b848..2920c263 100644 --- a/client.go +++ b/client.go @@ -32,8 +32,9 @@ import ( "golang.org/x/time/rate" "golang.org/x/xerrors" + "github.com/anacrolix/chansync" + "github.com/anacrolix/torrent/bencode" - "github.com/anacrolix/torrent/internal/chansync" "github.com/anacrolix/torrent/internal/limiter" "github.com/anacrolix/torrent/iplist" "github.com/anacrolix/torrent/metainfo" @@ -260,7 +261,7 @@ func NewClient(cfg *ClientConfig) (cl *Client, err error) { if err != nil { panic(err) } - cl.dhtServers = append(cl.dhtServers, anacrolixDhtServerWrapper{ds}) + cl.dhtServers = append(cl.dhtServers, AnacrolixDhtServerWrapper{ds}) cl.onClose = append(cl.onClose, func() { ds.Close() }) } } @@ -314,6 +315,10 @@ func (cl *Client) AddDialer(d Dialer) { } } +func (cl *Client) Listeners() []Listener { + return cl.listeners +} + // Registers a Listener, and starts Accepting on it. You must Close Listeners provided this way // yourself. func (cl *Client) AddListener(l Listener) { diff --git a/go.mod b/go.mod index 88326bfe..b5e8dead 100644 --- a/go.mod +++ b/go.mod @@ -5,6 +5,7 @@ require ( crawshaw.io/sqlite v0.3.3-0.20210127221821-98b1f83c5508 github.com/RoaringBitmap/roaring v0.6.0 // indirect github.com/alexflint/go-arg v1.3.0 + github.com/anacrolix/chansync v0.0.0-20210524073341-a336ebc2de92 // indirect github.com/anacrolix/confluence v1.7.1-0.20210311004351-d642adb8546c // indirect github.com/anacrolix/dht/v2 v2.9.1 github.com/anacrolix/envpprof v1.1.1 diff --git a/go.sum b/go.sum index c12955bc..f5eb4a2b 100644 --- a/go.sum +++ b/go.sum @@ -56,6 +56,8 @@ github.com/alexflint/go-arg v1.3.0 h1:UfldqSdFWeLtoOuVRosqofU4nmhI1pYEbT4ZFS34Bd github.com/alexflint/go-arg v1.3.0/go.mod h1:9iRbDxne7LcR/GSvEr7ma++GLpdIU1zrghf2y2768kM= github.com/alexflint/go-scalar v1.0.0 h1:NGupf1XV/Xb04wXskDFzS0KWOLH632W/EO4fAFi+A70= github.com/alexflint/go-scalar v1.0.0/go.mod h1:GpHzbCOZXEKMEcygYQ5n/aa4Aq84zbxjy3MxYW0gjYw= +github.com/anacrolix/chansync v0.0.0-20210524073341-a336ebc2de92 h1:WGk37RyXPWcIALJxTkTNrXN3yLQp7hSFa3x5GkrK/Rs= +github.com/anacrolix/chansync v0.0.0-20210524073341-a336ebc2de92/go.mod h1:DZsatdsdXxD0WiwcGl0nJVwyjCKMDv+knl1q2iBjA2k= github.com/anacrolix/confluence v1.7.1-0.20210221224747-9cb14aa2c53a/go.mod h1:T0JHvSaf9UfoiUdCtCOUuRroHm/tauUJTbLc6/vd5YA= github.com/anacrolix/confluence v1.7.1-0.20210221225853-90405640e928/go.mod h1:NoLcfoRet+kYttjLXJRmh4qBVrylJsfIItik5GGj21A= github.com/anacrolix/confluence v1.7.1-0.20210311004351-d642adb8546c h1:HfbeiZS/0hwdotwtQhllrd3PagmuLgCN9O8CHJgzPGQ= diff --git a/internal/chansync/broadcast-cond.go.go b/internal/chansync/broadcast-cond.go.go deleted file mode 100644 index 6d96d3c4..00000000 --- a/internal/chansync/broadcast-cond.go.go +++ /dev/null @@ -1,33 +0,0 @@ -package chansync - -import ( - "github.com/anacrolix/sync" -) - -// Can be used as zero-value. Due to the caller needing to bring their own synchronization, an -// eqiuvalent to "sync".Cond.Signal is not provided. BroadcastCond is intended to be selected on -// with other channels. -type BroadcastCond struct { - mu sync.Mutex - ch chan struct{} -} - -func (me *BroadcastCond) Broadcast() { - me.mu.Lock() - defer me.mu.Unlock() - if me.ch != nil { - close(me.ch) - me.ch = nil - } -} - -// Should be called before releasing locks on resources that might trigger subsequent Broadcasts. -// The channel is closed when the condition changes. -func (me *BroadcastCond) Signaled() Signaled { - me.mu.Lock() - defer me.mu.Unlock() - if me.ch == nil { - me.ch = make(chan struct{}) - } - return me.ch -} diff --git a/internal/chansync/interfaces.go b/internal/chansync/interfaces.go deleted file mode 100644 index 751ba0ef..00000000 --- a/internal/chansync/interfaces.go +++ /dev/null @@ -1,8 +0,0 @@ -package chansync - -// Here we'll strongly-type channels to assist correct usage, if possible. - -type ( - Signaled <-chan struct{} - Done <-chan struct{} -) diff --git a/internal/chansync/set-once.go b/internal/chansync/set-once.go deleted file mode 100644 index db0e6e89..00000000 --- a/internal/chansync/set-once.go +++ /dev/null @@ -1,42 +0,0 @@ -package chansync - -import "sync" - -// SetOnce is a boolean value that can only be flipped from false to true. -type SetOnce struct { - ch chan struct{} - initOnce sync.Once - closeOnce sync.Once -} - -// Returns a channel that is closed when the event is flagged. -func (me *SetOnce) Done() Done { - me.init() - return me.ch -} - -func (me *SetOnce) init() { - me.initOnce.Do(func() { - me.ch = make(chan struct{}) - }) -} - -// Set only returns true the first time it is called. -func (me *SetOnce) Set() (first bool) { - me.closeOnce.Do(func() { - me.init() - first = true - close(me.ch) - }) - return -} - -func (me *SetOnce) IsSet() bool { - me.init() - select { - case <-me.ch: - return true - default: - return false - } -} diff --git a/peer-conn-msg-writer.go b/peer-conn-msg-writer.go index 809d85fc..40fc27ef 100644 --- a/peer-conn-msg-writer.go +++ b/peer-conn-msg-writer.go @@ -5,10 +5,10 @@ import ( "io" "time" + "github.com/anacrolix/chansync" "github.com/anacrolix/log" "github.com/anacrolix/sync" - "github.com/anacrolix/torrent/internal/chansync" pp "github.com/anacrolix/torrent/peer_protocol" ) diff --git a/peerconn.go b/peerconn.go index 46756320..b35564ac 100644 --- a/peerconn.go +++ b/peerconn.go @@ -19,8 +19,8 @@ import ( "github.com/anacrolix/missinggo/v2/prioritybitmap" "github.com/anacrolix/multiless" + "github.com/anacrolix/chansync" "github.com/anacrolix/torrent/bencode" - "github.com/anacrolix/torrent/internal/chansync" "github.com/anacrolix/torrent/metainfo" "github.com/anacrolix/torrent/mse" pp "github.com/anacrolix/torrent/peer_protocol" diff --git a/requesting.go b/requesting.go index 7313c84a..f622b043 100644 --- a/requesting.go +++ b/requesting.go @@ -6,7 +6,7 @@ import ( "github.com/anacrolix/missinggo/v2/bitmap" - "github.com/anacrolix/torrent/internal/chansync" + "github.com/anacrolix/chansync" request_strategy "github.com/anacrolix/torrent/request-strategy" "github.com/anacrolix/torrent/types" ) From bc186ac2114b21b2115ae5737fc4a081fbe247da Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Mon, 24 May 2021 17:37:04 +1000 Subject: [PATCH 57/59] Expose AnacrolixDhtServerWrapper --- client_test.go | 2 +- dht.go | 10 +++++----- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/client_test.go b/client_test.go index e8f652eb..84a99f46 100644 --- a/client_test.go +++ b/client_test.go @@ -322,7 +322,7 @@ func TestDhtInheritBlocklist(t *testing.T) { numServers := 0 cl.eachDhtServer(func(s DhtServer) { t.Log(s) - assert.Equal(t, ipl, s.(anacrolixDhtServerWrapper).Server.IPBlocklist()) + assert.Equal(t, ipl, s.(AnacrolixDhtServerWrapper).Server.IPBlocklist()) numServers++ }) assert.EqualValues(t, 2, numServers) diff --git a/dht.go b/dht.go index 8b0e8471..464090f9 100644 --- a/dht.go +++ b/dht.go @@ -29,11 +29,11 @@ type DhtAnnounce interface { Peers() <-chan dht.PeersValues } -type anacrolixDhtServerWrapper struct { +type AnacrolixDhtServerWrapper struct { *dht.Server } -func (me anacrolixDhtServerWrapper) Stats() interface{} { +func (me AnacrolixDhtServerWrapper) Stats() interface{} { return me.Server.Stats() } @@ -45,13 +45,13 @@ func (me anacrolixDhtAnnounceWrapper) Peers() <-chan dht.PeersValues { return me.Announce.Peers } -func (me anacrolixDhtServerWrapper) Announce(hash [20]byte, port int, impliedPort bool) (DhtAnnounce, error) { +func (me AnacrolixDhtServerWrapper) Announce(hash [20]byte, port int, impliedPort bool) (DhtAnnounce, error) { ann, err := me.Server.Announce(hash, port, impliedPort) return anacrolixDhtAnnounceWrapper{ann}, err } -func (me anacrolixDhtServerWrapper) Ping(addr *net.UDPAddr) { +func (me AnacrolixDhtServerWrapper) Ping(addr *net.UDPAddr) { me.Server.Ping(addr) } -var _ DhtServer = anacrolixDhtServerWrapper{} +var _ DhtServer = AnacrolixDhtServerWrapper{} From ef39f408fe7cb1fa4f5b38b9166da18a2282b066 Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Mon, 24 May 2021 17:38:09 +1000 Subject: [PATCH 58/59] Add storage/disabled The default storage will create empty files on torrent open, which is undesirable in some circumstances. This storage implementation is explicit about not storing anything. --- storage/disabled/disabled.go | 58 ++++++++++++++++++++++++++++++++++++ 1 file changed, 58 insertions(+) create mode 100644 storage/disabled/disabled.go diff --git a/storage/disabled/disabled.go b/storage/disabled/disabled.go new file mode 100644 index 00000000..22d51434 --- /dev/null +++ b/storage/disabled/disabled.go @@ -0,0 +1,58 @@ +package disabled + +import ( + "errors" + + "github.com/anacrolix/torrent/metainfo" + "github.com/anacrolix/torrent/storage" +) + +type Client struct{} + +var capacity int64 + +func (c Client) OpenTorrent(info *metainfo.Info, infoHash metainfo.Hash) (storage.TorrentImpl, error) { + capFunc := func() *int64 { + return &capacity + } + return storage.TorrentImpl{ + Piece: func(piece metainfo.Piece) storage.PieceImpl { + return Piece{} + }, + Close: func() error { + return nil + }, + Capacity: &capFunc, + }, nil +} + +func (c Client) capacity() *int64 { + return &capacity +} + +type Piece struct{} + +func (Piece) ReadAt(p []byte, off int64) (n int, err error) { + err = errors.New("disabled") + return +} + +func (Piece) WriteAt(p []byte, off int64) (n int, err error) { + err = errors.New("disabled") + return +} + +func (Piece) MarkComplete() error { + return errors.New("disabled") +} + +func (Piece) MarkNotComplete() error { + return errors.New("disabled") +} + +func (Piece) Completion() storage.Completion { + return storage.Completion{ + Complete: false, + Ok: true, + } +} From f183acefe2c5e69a2f2d8efd7c3d0a41ad1acdb3 Mon Sep 17 00:00:00 2001 From: Matt Joiner Date: Mon, 24 May 2021 18:06:42 +1000 Subject: [PATCH 59/59] Add default param name in TorrentImpl.Piece func --- storage/interface.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/storage/interface.go b/storage/interface.go index c48c6e34..271bc7e6 100644 --- a/storage/interface.go +++ b/storage/interface.go @@ -18,7 +18,7 @@ type ClientImpl interface { // Data storage bound to a torrent. type TorrentImpl struct { - Piece func(metainfo.Piece) PieceImpl + Piece func(p metainfo.Piece) PieceImpl Close func() error // Storages that share the same value, will provide a pointer to the same function. Capacity *func() *int64