core, eth: implement eth/65 transaction fetcher
This commit is contained in:
committed by
Péter Szilágyi
parent
dcffb7777f
commit
049e17116e
363
eth/peer.go
363
eth/peer.go
@ -27,6 +27,7 @@ import (
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/core/forkid"
|
||||
"github.com/ethereum/go-ethereum/core/types"
|
||||
"github.com/ethereum/go-ethereum/eth/fetcher"
|
||||
"github.com/ethereum/go-ethereum/p2p"
|
||||
"github.com/ethereum/go-ethereum/rlp"
|
||||
)
|
||||
@ -41,24 +42,39 @@ const (
|
||||
maxKnownTxs = 32768 // Maximum transactions hashes to keep in the known list (prevent DOS)
|
||||
maxKnownBlocks = 1024 // Maximum block hashes to keep in the known list (prevent DOS)
|
||||
|
||||
// maxQueuedTxs is the maximum number of transaction lists to queue up before
|
||||
// dropping broadcasts. This is a sensitive number as a transaction list might
|
||||
// contain a single transaction, or thousands.
|
||||
maxQueuedTxs = 128
|
||||
// maxQueuedTxs is the maximum number of transactions to queue up before dropping
|
||||
// broadcasts.
|
||||
maxQueuedTxs = 4096
|
||||
|
||||
// maxQueuedProps is the maximum number of block propagations to queue up before
|
||||
// maxQueuedTxAnns is the maximum number of transaction announcements to queue up
|
||||
// before dropping broadcasts.
|
||||
maxQueuedTxAnns = 4096
|
||||
|
||||
// maxQueuedTxRetrieval is the maximum number of tx retrieval requests to queue up
|
||||
// before dropping requests.
|
||||
maxQueuedTxRetrieval = 4096
|
||||
|
||||
// maxQueuedBlocks is the maximum number of block propagations to queue up before
|
||||
// dropping broadcasts. There's not much point in queueing stale blocks, so a few
|
||||
// that might cover uncles should be enough.
|
||||
maxQueuedProps = 4
|
||||
maxQueuedBlocks = 4
|
||||
|
||||
// maxQueuedAnns is the maximum number of block announcements to queue up before
|
||||
// maxQueuedBlockAnns is the maximum number of block announcements to queue up before
|
||||
// dropping broadcasts. Similarly to block propagations, there's no point to queue
|
||||
// above some healthy uncle limit, so use that.
|
||||
maxQueuedAnns = 4
|
||||
maxQueuedBlockAnns = 4
|
||||
|
||||
handshakeTimeout = 5 * time.Second
|
||||
)
|
||||
|
||||
// max is a helper function which returns the larger of the two given integers.
|
||||
func max(a, b int) int {
|
||||
if a > b {
|
||||
return a
|
||||
}
|
||||
return b
|
||||
}
|
||||
|
||||
// PeerInfo represents a short summary of the Ethereum sub-protocol metadata known
|
||||
// about a connected peer.
|
||||
type PeerInfo struct {
|
||||
@ -86,48 +102,48 @@ type peer struct {
|
||||
td *big.Int
|
||||
lock sync.RWMutex
|
||||
|
||||
knownTxs mapset.Set // Set of transaction hashes known to be known by this peer
|
||||
knownBlocks mapset.Set // Set of block hashes known to be known by this peer
|
||||
queuedTxs chan []*types.Transaction // Queue of transactions to broadcast to the peer
|
||||
queuedProps chan *propEvent // Queue of blocks to broadcast to the peer
|
||||
queuedAnns chan *types.Block // Queue of blocks to announce to the peer
|
||||
term chan struct{} // Termination channel to stop the broadcaster
|
||||
knownTxs mapset.Set // Set of transaction hashes known to be known by this peer
|
||||
knownBlocks mapset.Set // Set of block hashes known to be known by this peer
|
||||
queuedBlocks chan *propEvent // Queue of blocks to broadcast to the peer
|
||||
queuedBlockAnns chan *types.Block // Queue of blocks to announce to the peer
|
||||
txPropagation chan []common.Hash // Channel used to queue transaction propagation requests
|
||||
txAnnounce chan []common.Hash // Channel used to queue transaction announcement requests
|
||||
txRetrieval chan []common.Hash // Channel used to queue transaction retrieval requests
|
||||
getPooledTx func(common.Hash) *types.Transaction // Callback used to retrieve transaction from txpool
|
||||
term chan struct{} // Termination channel to stop the broadcaster
|
||||
}
|
||||
|
||||
func newPeer(version int, p *p2p.Peer, rw p2p.MsgReadWriter) *peer {
|
||||
func newPeer(version int, p *p2p.Peer, rw p2p.MsgReadWriter, getPooledTx func(hash common.Hash) *types.Transaction) *peer {
|
||||
return &peer{
|
||||
Peer: p,
|
||||
rw: rw,
|
||||
version: version,
|
||||
id: fmt.Sprintf("%x", p.ID().Bytes()[:8]),
|
||||
knownTxs: mapset.NewSet(),
|
||||
knownBlocks: mapset.NewSet(),
|
||||
queuedTxs: make(chan []*types.Transaction, maxQueuedTxs),
|
||||
queuedProps: make(chan *propEvent, maxQueuedProps),
|
||||
queuedAnns: make(chan *types.Block, maxQueuedAnns),
|
||||
term: make(chan struct{}),
|
||||
Peer: p,
|
||||
rw: rw,
|
||||
version: version,
|
||||
id: fmt.Sprintf("%x", p.ID().Bytes()[:8]),
|
||||
knownTxs: mapset.NewSet(),
|
||||
knownBlocks: mapset.NewSet(),
|
||||
queuedBlocks: make(chan *propEvent, maxQueuedBlocks),
|
||||
queuedBlockAnns: make(chan *types.Block, maxQueuedBlockAnns),
|
||||
txPropagation: make(chan []common.Hash),
|
||||
txAnnounce: make(chan []common.Hash),
|
||||
txRetrieval: make(chan []common.Hash),
|
||||
getPooledTx: getPooledTx,
|
||||
term: make(chan struct{}),
|
||||
}
|
||||
}
|
||||
|
||||
// broadcast is a write loop that multiplexes block propagations, announcements
|
||||
// and transaction broadcasts into the remote peer. The goal is to have an async
|
||||
// writer that does not lock up node internals.
|
||||
func (p *peer) broadcast() {
|
||||
// broadcastBlocks is a write loop that multiplexes block propagations,
|
||||
// announcements into the remote peer. The goal is to have an async writer
|
||||
// that does not lock up node internals.
|
||||
func (p *peer) broadcastBlocks() {
|
||||
for {
|
||||
select {
|
||||
case txs := <-p.queuedTxs:
|
||||
if err := p.SendTransactions(txs); err != nil {
|
||||
return
|
||||
}
|
||||
p.Log().Trace("Broadcast transactions", "count", len(txs))
|
||||
|
||||
case prop := <-p.queuedProps:
|
||||
case prop := <-p.queuedBlocks:
|
||||
if err := p.SendNewBlock(prop.block, prop.td); err != nil {
|
||||
return
|
||||
}
|
||||
p.Log().Trace("Propagated block", "number", prop.block.Number(), "hash", prop.block.Hash(), "td", prop.td)
|
||||
|
||||
case block := <-p.queuedAnns:
|
||||
case block := <-p.queuedBlockAnns:
|
||||
if err := p.SendNewBlockHashes([]common.Hash{block.Hash()}, []uint64{block.NumberU64()}); err != nil {
|
||||
return
|
||||
}
|
||||
@ -139,6 +155,175 @@ func (p *peer) broadcast() {
|
||||
}
|
||||
}
|
||||
|
||||
// broadcastTxs is a write loop that multiplexes transaction propagations,
|
||||
// announcements into the remote peer. The goal is to have an async writer
|
||||
// that does not lock up node internals.
|
||||
func (p *peer) broadcastTxs() {
|
||||
var (
|
||||
txProps []common.Hash // Queue of transaction propagations to the peer
|
||||
txAnnos []common.Hash // Queue of transaction announcements to the peer
|
||||
done chan struct{} // Non-nil if background network sender routine is active.
|
||||
errch = make(chan error) // Channel used to receive network error
|
||||
)
|
||||
scheduleTask := func() {
|
||||
// Short circuit if there already has a inflight task.
|
||||
if done != nil {
|
||||
return
|
||||
}
|
||||
// Spin up transaction propagation task if there is any
|
||||
// queued hashes.
|
||||
if len(txProps) > 0 {
|
||||
var (
|
||||
hashes []common.Hash
|
||||
txs []*types.Transaction
|
||||
size common.StorageSize
|
||||
)
|
||||
for i := 0; i < len(txProps) && size < txsyncPackSize; i++ {
|
||||
if tx := p.getPooledTx(txProps[i]); tx != nil {
|
||||
txs = append(txs, tx)
|
||||
size += tx.Size()
|
||||
}
|
||||
hashes = append(hashes, txProps[i])
|
||||
}
|
||||
txProps = txProps[:copy(txProps, txProps[len(hashes):])]
|
||||
if len(txs) > 0 {
|
||||
done = make(chan struct{})
|
||||
go func() {
|
||||
if err := p.SendNewTransactions(txs); err != nil {
|
||||
errch <- err
|
||||
return
|
||||
}
|
||||
close(done)
|
||||
p.Log().Trace("Sent transactions", "count", len(txs))
|
||||
}()
|
||||
return
|
||||
}
|
||||
}
|
||||
// Spin up transaction announcement task if there is any
|
||||
// queued hashes.
|
||||
if len(txAnnos) > 0 {
|
||||
var (
|
||||
hashes []common.Hash
|
||||
pending []common.Hash
|
||||
size common.StorageSize
|
||||
)
|
||||
for i := 0; i < len(txAnnos) && size < txsyncPackSize; i++ {
|
||||
if tx := p.getPooledTx(txAnnos[i]); tx != nil {
|
||||
pending = append(pending, txAnnos[i])
|
||||
size += common.HashLength
|
||||
}
|
||||
hashes = append(hashes, txAnnos[i])
|
||||
}
|
||||
txAnnos = txAnnos[:copy(txAnnos, txAnnos[len(hashes):])]
|
||||
if len(pending) > 0 {
|
||||
done = make(chan struct{})
|
||||
go func() {
|
||||
if err := p.SendNewTransactionHashes(pending); err != nil {
|
||||
errch <- err
|
||||
return
|
||||
}
|
||||
close(done)
|
||||
p.Log().Trace("Sent transaction announcements", "count", len(pending))
|
||||
}()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
for {
|
||||
scheduleTask()
|
||||
select {
|
||||
case hashes := <-p.txPropagation:
|
||||
if len(txProps) == maxQueuedTxs {
|
||||
continue
|
||||
}
|
||||
if len(txProps)+len(hashes) > maxQueuedTxs {
|
||||
hashes = hashes[:maxQueuedTxs-len(txProps)]
|
||||
}
|
||||
txProps = append(txProps, hashes...)
|
||||
|
||||
case hashes := <-p.txAnnounce:
|
||||
if len(txAnnos) == maxQueuedTxAnns {
|
||||
continue
|
||||
}
|
||||
if len(txAnnos)+len(hashes) > maxQueuedTxAnns {
|
||||
hashes = hashes[:maxQueuedTxAnns-len(txAnnos)]
|
||||
}
|
||||
txAnnos = append(txAnnos, hashes...)
|
||||
|
||||
case <-done:
|
||||
done = nil
|
||||
|
||||
case <-errch:
|
||||
return
|
||||
|
||||
case <-p.term:
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// retrievalTxs is a write loop which is responsible for retrieving transaction
|
||||
// from the remote peer. The goal is to have an async writer that does not lock
|
||||
// up node internals. If there are too many requests queued, then new arrival
|
||||
// requests will be dropped silently so that we can ensure the memory assumption
|
||||
// is fixed for each peer.
|
||||
func (p *peer) retrievalTxs() {
|
||||
var (
|
||||
requests []common.Hash // Queue of transaction requests to the peer
|
||||
done chan struct{} // Non-nil if background network sender routine is active.
|
||||
errch = make(chan error) // Channel used to receive network error
|
||||
)
|
||||
// pick chooses a reasonble number of transaction hashes for retrieval.
|
||||
pick := func() []common.Hash {
|
||||
var ret []common.Hash
|
||||
if len(requests) > fetcher.MaxTransactionFetch {
|
||||
ret = requests[:fetcher.MaxTransactionFetch]
|
||||
} else {
|
||||
ret = requests[:]
|
||||
}
|
||||
requests = requests[:copy(requests, requests[len(ret):])]
|
||||
return ret
|
||||
}
|
||||
// send sends transactions retrieval request.
|
||||
send := func(hashes []common.Hash, done chan struct{}) {
|
||||
if err := p.RequestTxs(hashes); err != nil {
|
||||
errch <- err
|
||||
return
|
||||
}
|
||||
close(done)
|
||||
p.Log().Trace("Sent transaction retrieval request", "count", len(hashes))
|
||||
}
|
||||
for {
|
||||
select {
|
||||
case hashes := <-p.txRetrieval:
|
||||
if len(requests) == maxQueuedTxRetrieval {
|
||||
continue
|
||||
}
|
||||
if len(requests)+len(hashes) > maxQueuedTxRetrieval {
|
||||
hashes = hashes[:maxQueuedTxRetrieval-len(requests)]
|
||||
}
|
||||
requests = append(requests, hashes...)
|
||||
if done == nil {
|
||||
done = make(chan struct{})
|
||||
go send(pick(), done)
|
||||
}
|
||||
|
||||
case <-done:
|
||||
done = nil
|
||||
if pending := pick(); len(pending) > 0 {
|
||||
done = make(chan struct{})
|
||||
go send(pending, done)
|
||||
}
|
||||
|
||||
case <- errch:
|
||||
return
|
||||
|
||||
case <-p.term:
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// close signals the broadcast goroutine to terminate.
|
||||
func (p *peer) close() {
|
||||
close(p.term)
|
||||
@ -194,33 +379,67 @@ func (p *peer) MarkTransaction(hash common.Hash) {
|
||||
p.knownTxs.Add(hash)
|
||||
}
|
||||
|
||||
// SendTransactions sends transactions to the peer and includes the hashes
|
||||
// in its transaction hash set for future reference.
|
||||
func (p *peer) SendTransactions(txs types.Transactions) error {
|
||||
// SendNewTransactionHashes sends a batch of transaction hashes to the peer and
|
||||
// includes the hashes in its transaction hash set for future reference.
|
||||
func (p *peer) SendNewTransactionHashes(hashes []common.Hash) error {
|
||||
// Mark all the transactions as known, but ensure we don't overflow our limits
|
||||
for p.knownTxs.Cardinality() > max(0, maxKnownTxs-len(hashes)) {
|
||||
p.knownTxs.Pop()
|
||||
}
|
||||
for _, hash := range hashes {
|
||||
p.knownTxs.Add(hash)
|
||||
}
|
||||
return p2p.Send(p.rw, NewPooledTransactionHashesMsg, hashes)
|
||||
}
|
||||
|
||||
// SendNewTransactions sends transactions to the peer and includes the hashes
|
||||
// in its transaction hash set for future reference.
|
||||
func (p *peer) SendNewTransactions(txs types.Transactions) error {
|
||||
// Mark all the transactions as known, but ensure we don't overflow our limits
|
||||
for p.knownTxs.Cardinality() > max(0, maxKnownTxs-len(txs)) {
|
||||
p.knownTxs.Pop()
|
||||
}
|
||||
for _, tx := range txs {
|
||||
p.knownTxs.Add(tx.Hash())
|
||||
}
|
||||
for p.knownTxs.Cardinality() >= maxKnownTxs {
|
||||
p.knownTxs.Pop()
|
||||
}
|
||||
return p2p.Send(p.rw, TxMsg, txs)
|
||||
}
|
||||
|
||||
func (p *peer) SendTransactionRLP(txs []rlp.RawValue) error {
|
||||
return p2p.Send(p.rw, TxMsg, txs)
|
||||
}
|
||||
|
||||
// AsyncSendTransactions queues list of transactions propagation to a remote
|
||||
// peer. If the peer's broadcast queue is full, the event is silently dropped.
|
||||
func (p *peer) AsyncSendTransactions(txs []*types.Transaction) {
|
||||
func (p *peer) AsyncSendTransactions(hashes []common.Hash) {
|
||||
select {
|
||||
case p.queuedTxs <- txs:
|
||||
case p.txPropagation <- hashes:
|
||||
// Mark all the transactions as known, but ensure we don't overflow our limits
|
||||
for _, tx := range txs {
|
||||
p.knownTxs.Add(tx.Hash())
|
||||
}
|
||||
for p.knownTxs.Cardinality() >= maxKnownTxs {
|
||||
for p.knownTxs.Cardinality() > max(0, maxKnownTxs-len(hashes)) {
|
||||
p.knownTxs.Pop()
|
||||
}
|
||||
default:
|
||||
p.Log().Debug("Dropping transaction propagation", "count", len(txs))
|
||||
for _, hash := range hashes {
|
||||
p.knownTxs.Add(hash)
|
||||
}
|
||||
case <-p.term:
|
||||
p.Log().Debug("Dropping transaction propagation", "count", len(hashes))
|
||||
}
|
||||
}
|
||||
|
||||
// AsyncSendTransactions queues list of transactions propagation to a remote
|
||||
// peer. If the peer's broadcast queue is full, the event is silently dropped.
|
||||
func (p *peer) AsyncSendTransactionHashes(hashes []common.Hash) {
|
||||
select {
|
||||
case p.txAnnounce <- hashes:
|
||||
// Mark all the transactions as known, but ensure we don't overflow our limits
|
||||
for p.knownTxs.Cardinality() > max(0, maxKnownTxs-len(hashes)) {
|
||||
p.knownTxs.Pop()
|
||||
}
|
||||
for _, hash := range hashes {
|
||||
p.knownTxs.Add(hash)
|
||||
}
|
||||
case <-p.term:
|
||||
p.Log().Debug("Dropping transaction announcement", "count", len(hashes))
|
||||
}
|
||||
}
|
||||
|
||||
@ -228,12 +447,12 @@ func (p *peer) AsyncSendTransactions(txs []*types.Transaction) {
|
||||
// a hash notification.
|
||||
func (p *peer) SendNewBlockHashes(hashes []common.Hash, numbers []uint64) error {
|
||||
// Mark all the block hashes as known, but ensure we don't overflow our limits
|
||||
for p.knownBlocks.Cardinality() > max(0, maxKnownBlocks-len(hashes)) {
|
||||
p.knownBlocks.Pop()
|
||||
}
|
||||
for _, hash := range hashes {
|
||||
p.knownBlocks.Add(hash)
|
||||
}
|
||||
for p.knownBlocks.Cardinality() >= maxKnownBlocks {
|
||||
p.knownBlocks.Pop()
|
||||
}
|
||||
request := make(newBlockHashesData, len(hashes))
|
||||
for i := 0; i < len(hashes); i++ {
|
||||
request[i].Hash = hashes[i]
|
||||
@ -247,12 +466,12 @@ func (p *peer) SendNewBlockHashes(hashes []common.Hash, numbers []uint64) error
|
||||
// dropped.
|
||||
func (p *peer) AsyncSendNewBlockHash(block *types.Block) {
|
||||
select {
|
||||
case p.queuedAnns <- block:
|
||||
case p.queuedBlockAnns <- block:
|
||||
// Mark all the block hash as known, but ensure we don't overflow our limits
|
||||
p.knownBlocks.Add(block.Hash())
|
||||
for p.knownBlocks.Cardinality() >= maxKnownBlocks {
|
||||
p.knownBlocks.Pop()
|
||||
}
|
||||
p.knownBlocks.Add(block.Hash())
|
||||
default:
|
||||
p.Log().Debug("Dropping block announcement", "number", block.NumberU64(), "hash", block.Hash())
|
||||
}
|
||||
@ -261,10 +480,10 @@ func (p *peer) AsyncSendNewBlockHash(block *types.Block) {
|
||||
// SendNewBlock propagates an entire block to a remote peer.
|
||||
func (p *peer) SendNewBlock(block *types.Block, td *big.Int) error {
|
||||
// Mark all the block hash as known, but ensure we don't overflow our limits
|
||||
p.knownBlocks.Add(block.Hash())
|
||||
for p.knownBlocks.Cardinality() >= maxKnownBlocks {
|
||||
p.knownBlocks.Pop()
|
||||
}
|
||||
p.knownBlocks.Add(block.Hash())
|
||||
return p2p.Send(p.rw, NewBlockMsg, []interface{}{block, td})
|
||||
}
|
||||
|
||||
@ -272,12 +491,12 @@ func (p *peer) SendNewBlock(block *types.Block, td *big.Int) error {
|
||||
// the peer's broadcast queue is full, the event is silently dropped.
|
||||
func (p *peer) AsyncSendNewBlock(block *types.Block, td *big.Int) {
|
||||
select {
|
||||
case p.queuedProps <- &propEvent{block: block, td: td}:
|
||||
case p.queuedBlocks <- &propEvent{block: block, td: td}:
|
||||
// Mark all the block hash as known, but ensure we don't overflow our limits
|
||||
p.knownBlocks.Add(block.Hash())
|
||||
for p.knownBlocks.Cardinality() >= maxKnownBlocks {
|
||||
p.knownBlocks.Pop()
|
||||
}
|
||||
p.knownBlocks.Add(block.Hash())
|
||||
default:
|
||||
p.Log().Debug("Dropping block propagation", "number", block.NumberU64(), "hash", block.Hash())
|
||||
}
|
||||
@ -352,6 +571,22 @@ func (p *peer) RequestReceipts(hashes []common.Hash) error {
|
||||
return p2p.Send(p.rw, GetReceiptsMsg, hashes)
|
||||
}
|
||||
|
||||
// RequestTxs fetches a batch of transactions from a remote node.
|
||||
func (p *peer) RequestTxs(hashes []common.Hash) error {
|
||||
p.Log().Debug("Fetching batch of transactions", "count", len(hashes))
|
||||
return p2p.Send(p.rw, GetPooledTransactionsMsg, hashes)
|
||||
}
|
||||
|
||||
// AsyncRequestTxs queues a tx retrieval request to a remote peer. If
|
||||
// the peer's retrieval queue is full, the event is silently dropped.
|
||||
func (p *peer) AsyncRequestTxs(hashes []common.Hash) {
|
||||
select {
|
||||
case p.txRetrieval <- hashes:
|
||||
case <-p.term:
|
||||
p.Log().Debug("Dropping transaction retrieval request", "count", len(hashes))
|
||||
}
|
||||
}
|
||||
|
||||
// Handshake executes the eth protocol handshake, negotiating version number,
|
||||
// network IDs, difficulties, head and genesis blocks.
|
||||
func (p *peer) Handshake(network uint64, td *big.Int, head common.Hash, genesis common.Hash, forkID forkid.ID, forkFilter forkid.Filter) error {
|
||||
@ -372,7 +607,7 @@ func (p *peer) Handshake(network uint64, td *big.Int, head common.Hash, genesis
|
||||
CurrentBlock: head,
|
||||
GenesisBlock: genesis,
|
||||
})
|
||||
case p.version == eth64:
|
||||
case p.version >= eth64:
|
||||
errc <- p2p.Send(p.rw, StatusMsg, &statusData{
|
||||
ProtocolVersion: uint32(p.version),
|
||||
NetworkID: network,
|
||||
@ -389,7 +624,7 @@ func (p *peer) Handshake(network uint64, td *big.Int, head common.Hash, genesis
|
||||
switch {
|
||||
case p.version == eth63:
|
||||
errc <- p.readStatusLegacy(network, &status63, genesis)
|
||||
case p.version == eth64:
|
||||
case p.version >= eth64:
|
||||
errc <- p.readStatus(network, &status, genesis, forkFilter)
|
||||
default:
|
||||
panic(fmt.Sprintf("unsupported eth protocol version: %d", p.version))
|
||||
@ -410,7 +645,7 @@ func (p *peer) Handshake(network uint64, td *big.Int, head common.Hash, genesis
|
||||
switch {
|
||||
case p.version == eth63:
|
||||
p.td, p.head = status63.TD, status63.CurrentBlock
|
||||
case p.version == eth64:
|
||||
case p.version >= eth64:
|
||||
p.td, p.head = status.TD, status.Head
|
||||
default:
|
||||
panic(fmt.Sprintf("unsupported eth protocol version: %d", p.version))
|
||||
@ -511,7 +746,9 @@ func (ps *peerSet) Register(p *peer) error {
|
||||
return errAlreadyRegistered
|
||||
}
|
||||
ps.peers[p.id] = p
|
||||
go p.broadcast()
|
||||
go p.broadcastBlocks()
|
||||
go p.broadcastTxs()
|
||||
go p.retrievalTxs()
|
||||
|
||||
return nil
|
||||
}
|
||||
|
Reference in New Issue
Block a user