mirror of
https://github.com/kaspanet/kaspad.git
synced 2025-06-05 13:46:42 +00:00
Extract IBD management from invs relay flow to a new separated flow (#1930)
* Separate IBD to a new flow (so now invs are handled concurrently and no route capacity errors) * Invs messages should be queued while waiting for BlockLocator msg * Close IBD channel so that HandleIBDFlow exits too * Apply flow separation to p2p protocol v4 * Manage the IBDRequestChannel through the Peer struct * Some IBDs take a little longer
This commit is contained in:
parent
b1b179c105
commit
4855d845b3
@ -13,15 +13,21 @@ func (flow *handleRelayInvsFlow) sendGetBlockLocator(highHash *externalapi.Domai
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) receiveBlockLocator() (blockLocatorHashes []*externalapi.DomainHash, err error) {
|
||||
message, err := flow.dequeueIncomingMessageAndSkipInvs(common.DefaultTimeout)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
for {
|
||||
message, err := flow.incomingRoute.DequeueWithTimeout(common.DefaultTimeout)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
switch message := message.(type) {
|
||||
case *appmessage.MsgInvRelayBlock:
|
||||
flow.invsQueue = append(flow.invsQueue, message)
|
||||
case *appmessage.MsgBlockLocator:
|
||||
return message.BlockLocatorHashes, nil
|
||||
default:
|
||||
return nil,
|
||||
protocolerrors.Errorf(true, "received unexpected message type. "+
|
||||
"expected: %s, got: %s", appmessage.CmdBlockLocator, message.Command())
|
||||
}
|
||||
}
|
||||
msgBlockLocator, ok := message.(*appmessage.MsgBlockLocator)
|
||||
if !ok {
|
||||
return nil,
|
||||
protocolerrors.Errorf(true, "received unexpected message type. "+
|
||||
"expected: %s, got: %s", appmessage.CmdBlockLocator, message.Command())
|
||||
}
|
||||
return msgBlockLocator.BlockLocatorHashes, nil
|
||||
}
|
||||
|
@ -33,8 +33,6 @@ type RelayInvsContext interface {
|
||||
GetOrphanRoots(orphanHash *externalapi.DomainHash) ([]*externalapi.DomainHash, bool, error)
|
||||
IsOrphan(blockHash *externalapi.DomainHash) bool
|
||||
IsIBDRunning() bool
|
||||
TrySetIBDRunning(ibdPeer *peerpkg.Peer) bool
|
||||
UnsetIBDRunning()
|
||||
IsRecoverableError(err error) bool
|
||||
}
|
||||
|
||||
@ -57,7 +55,10 @@ func HandleRelayInvs(context RelayInvsContext, incomingRoute *router.Route, outg
|
||||
peer: peer,
|
||||
invsQueue: make([]*appmessage.MsgInvRelayBlock, 0),
|
||||
}
|
||||
return flow.start()
|
||||
err := flow.start()
|
||||
// Currently, HandleRelayInvs flow is the only place where IBD is triggered, so the channel can be closed now
|
||||
close(peer.IBDRequestChannel())
|
||||
return err
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) start() error {
|
||||
@ -306,7 +307,14 @@ func (flow *handleRelayInvsFlow) processOrphan(block *externalapi.DomainBlock) e
|
||||
// Start IBD unless we already are in IBD
|
||||
log.Debugf("Block %s is out of orphan resolution range. "+
|
||||
"Attempting to start IBD against it.", blockHash)
|
||||
return flow.runIBDIfNotRunning(block)
|
||||
|
||||
// Send the block to IBD flow via the IBDRequestChannel.
|
||||
// Note that this is a non-blocking send, since if IBD is already running, there is no need to trigger it
|
||||
select {
|
||||
case flow.peer.IBDRequestChannel() <- block:
|
||||
default:
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) isGenesisVirtualSelectedParent() (bool, error) {
|
||||
|
@ -1,22 +1,69 @@
|
||||
package blockrelay
|
||||
|
||||
import (
|
||||
"time"
|
||||
|
||||
"github.com/kaspanet/kaspad/infrastructure/logger"
|
||||
|
||||
"github.com/kaspanet/kaspad/domain/consensus/model"
|
||||
|
||||
"github.com/kaspanet/kaspad/app/appmessage"
|
||||
"github.com/kaspanet/kaspad/app/protocol/common"
|
||||
peerpkg "github.com/kaspanet/kaspad/app/protocol/peer"
|
||||
"github.com/kaspanet/kaspad/app/protocol/protocolerrors"
|
||||
"github.com/kaspanet/kaspad/domain"
|
||||
"github.com/kaspanet/kaspad/domain/consensus/model"
|
||||
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
|
||||
"github.com/kaspanet/kaspad/domain/consensus/ruleerrors"
|
||||
"github.com/kaspanet/kaspad/domain/consensus/utils/consensushashing"
|
||||
"github.com/kaspanet/kaspad/infrastructure/config"
|
||||
"github.com/kaspanet/kaspad/infrastructure/logger"
|
||||
"github.com/kaspanet/kaspad/infrastructure/network/netadapter/router"
|
||||
"github.com/pkg/errors"
|
||||
"time"
|
||||
)
|
||||
|
||||
func (flow *handleRelayInvsFlow) runIBDIfNotRunning(block *externalapi.DomainBlock) error {
|
||||
// IBDContext is the interface for the context needed for the HandleIBD flow.
|
||||
type IBDContext interface {
|
||||
Domain() domain.Domain
|
||||
Config() *config.Config
|
||||
OnNewBlock(block *externalapi.DomainBlock, virtualChangeSet *externalapi.VirtualChangeSet) error
|
||||
OnVirtualChange(virtualChangeSet *externalapi.VirtualChangeSet) error
|
||||
OnPruningPointUTXOSetOverride() error
|
||||
IsIBDRunning() bool
|
||||
TrySetIBDRunning(ibdPeer *peerpkg.Peer) bool
|
||||
UnsetIBDRunning()
|
||||
IsRecoverableError(err error) bool
|
||||
}
|
||||
|
||||
type handleIBDFlow struct {
|
||||
IBDContext
|
||||
incomingRoute, outgoingRoute *router.Route
|
||||
peer *peerpkg.Peer
|
||||
}
|
||||
|
||||
// HandleIBD handles IBD
|
||||
func HandleIBD(context IBDContext, incomingRoute *router.Route, outgoingRoute *router.Route,
|
||||
peer *peerpkg.Peer) error {
|
||||
|
||||
flow := &handleIBDFlow{
|
||||
IBDContext: context,
|
||||
incomingRoute: incomingRoute,
|
||||
outgoingRoute: outgoingRoute,
|
||||
peer: peer,
|
||||
}
|
||||
return flow.start()
|
||||
}
|
||||
|
||||
func (flow *handleIBDFlow) start() error {
|
||||
for {
|
||||
// Wait for IBD requests triggered by other flows
|
||||
block, ok := <-flow.peer.IBDRequestChannel()
|
||||
if !ok {
|
||||
return nil
|
||||
}
|
||||
err := flow.runIBDIfNotRunning(block)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (flow *handleIBDFlow) runIBDIfNotRunning(block *externalapi.DomainBlock) error {
|
||||
wasIBDNotRunning := flow.TrySetIBDRunning(flow.peer)
|
||||
if !wasIBDNotRunning {
|
||||
log.Debugf("IBD is already running")
|
||||
@ -84,7 +131,16 @@ func (flow *handleRelayInvsFlow) runIBDIfNotRunning(block *externalapi.DomainBlo
|
||||
return nil
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) logIBDFinished(isFinishedSuccessfully bool) {
|
||||
func (flow *handleIBDFlow) isGenesisVirtualSelectedParent() (bool, error) {
|
||||
virtualSelectedParent, err := flow.Domain().Consensus().GetVirtualSelectedParent()
|
||||
if err != nil {
|
||||
return false, err
|
||||
}
|
||||
|
||||
return virtualSelectedParent.Equal(flow.Config().NetParams().GenesisHash), nil
|
||||
}
|
||||
|
||||
func (flow *handleIBDFlow) logIBDFinished(isFinishedSuccessfully bool) {
|
||||
successString := "successfully"
|
||||
if !isFinishedSuccessfully {
|
||||
successString = "(interrupted)"
|
||||
@ -95,7 +151,7 @@ func (flow *handleRelayInvsFlow) logIBDFinished(isFinishedSuccessfully bool) {
|
||||
// findHighestSharedBlock attempts to find the highest shared block between the peer
|
||||
// and this node. This method may fail because the peer and us have conflicting pruning
|
||||
// points. In that case we return (nil, false, nil) so that we may stop IBD gracefully.
|
||||
func (flow *handleRelayInvsFlow) findHighestSharedBlockHash(
|
||||
func (flow *handleIBDFlow) findHighestSharedBlockHash(
|
||||
targetHash *externalapi.DomainHash) (*externalapi.DomainHash, bool, error) {
|
||||
|
||||
log.Debugf("Sending a blockLocator to %s between pruning point and headers selected tip", flow.peer)
|
||||
@ -138,7 +194,7 @@ func (flow *handleRelayInvsFlow) findHighestSharedBlockHash(
|
||||
}
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) nextBlockLocator(lowHash, highHash *externalapi.DomainHash) (externalapi.BlockLocator, error) {
|
||||
func (flow *handleIBDFlow) nextBlockLocator(lowHash, highHash *externalapi.DomainHash) (externalapi.BlockLocator, error) {
|
||||
log.Debugf("Sending a blockLocator to %s between %s and %s", flow.peer, lowHash, highHash)
|
||||
blockLocator, err := flow.Domain().Consensus().CreateHeadersSelectedChainBlockLocator(lowHash, highHash)
|
||||
if err != nil {
|
||||
@ -156,7 +212,7 @@ func (flow *handleRelayInvsFlow) nextBlockLocator(lowHash, highHash *externalapi
|
||||
return blockLocator, nil
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) findHighestHashIndex(
|
||||
func (flow *handleIBDFlow) findHighestHashIndex(
|
||||
highestHash *externalapi.DomainHash, blockLocator externalapi.BlockLocator) (int, error) {
|
||||
|
||||
highestHashIndex := 0
|
||||
@ -181,7 +237,7 @@ func (flow *handleRelayInvsFlow) findHighestHashIndex(
|
||||
// fetchHighestHash attempts to fetch the highest hash the peer knows amongst the given
|
||||
// blockLocator. This method may fail because the peer and us have conflicting pruning
|
||||
// points. In that case we return (nil, false, nil) so that we may stop IBD gracefully.
|
||||
func (flow *handleRelayInvsFlow) fetchHighestHash(
|
||||
func (flow *handleIBDFlow) fetchHighestHash(
|
||||
targetHash *externalapi.DomainHash, blockLocator externalapi.BlockLocator) (*externalapi.DomainHash, bool, error) {
|
||||
|
||||
ibdBlockLocatorMessage := appmessage.NewMsgIBDBlockLocator(targetHash, blockLocator)
|
||||
@ -189,7 +245,7 @@ func (flow *handleRelayInvsFlow) fetchHighestHash(
|
||||
if err != nil {
|
||||
return nil, false, err
|
||||
}
|
||||
message, err := flow.dequeueIncomingMessageAndSkipInvs(common.DefaultTimeout)
|
||||
message, err := flow.incomingRoute.DequeueWithTimeout(common.DefaultTimeout)
|
||||
if err != nil {
|
||||
return nil, false, err
|
||||
}
|
||||
@ -209,7 +265,7 @@ func (flow *handleRelayInvsFlow) fetchHighestHash(
|
||||
}
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) syncPruningPointFutureHeaders(consensus externalapi.Consensus, highestSharedBlockHash *externalapi.DomainHash,
|
||||
func (flow *handleIBDFlow) syncPruningPointFutureHeaders(consensus externalapi.Consensus, highestSharedBlockHash *externalapi.DomainHash,
|
||||
highHash *externalapi.DomainHash) error {
|
||||
|
||||
log.Infof("Downloading headers from %s", flow.peer)
|
||||
@ -273,15 +329,15 @@ func (flow *handleRelayInvsFlow) syncPruningPointFutureHeaders(consensus externa
|
||||
}
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) sendRequestHeaders(highestSharedBlockHash *externalapi.DomainHash,
|
||||
func (flow *handleIBDFlow) sendRequestHeaders(highestSharedBlockHash *externalapi.DomainHash,
|
||||
peerSelectedTipHash *externalapi.DomainHash) error {
|
||||
|
||||
msgGetBlockInvs := appmessage.NewMsgRequstHeaders(highestSharedBlockHash, peerSelectedTipHash)
|
||||
return flow.outgoingRoute.Enqueue(msgGetBlockInvs)
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) receiveHeaders() (msgIBDBlock *appmessage.BlockHeadersMessage, doneHeaders bool, err error) {
|
||||
message, err := flow.dequeueIncomingMessageAndSkipInvs(common.DefaultTimeout)
|
||||
func (flow *handleIBDFlow) receiveHeaders() (msgIBDBlock *appmessage.BlockHeadersMessage, doneHeaders bool, err error) {
|
||||
message, err := flow.incomingRoute.DequeueWithTimeout(common.DefaultTimeout)
|
||||
if err != nil {
|
||||
return nil, false, err
|
||||
}
|
||||
@ -300,7 +356,7 @@ func (flow *handleRelayInvsFlow) receiveHeaders() (msgIBDBlock *appmessage.Block
|
||||
}
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) processHeader(consensus externalapi.Consensus, msgBlockHeader *appmessage.MsgBlockHeader) error {
|
||||
func (flow *handleIBDFlow) processHeader(consensus externalapi.Consensus, msgBlockHeader *appmessage.MsgBlockHeader) error {
|
||||
header := appmessage.BlockHeaderToDomainBlockHeader(msgBlockHeader)
|
||||
block := &externalapi.DomainBlock{
|
||||
Header: header,
|
||||
@ -333,7 +389,7 @@ func (flow *handleRelayInvsFlow) processHeader(consensus externalapi.Consensus,
|
||||
return nil
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) validatePruningPointFutureHeaderTimestamps() error {
|
||||
func (flow *handleIBDFlow) validatePruningPointFutureHeaderTimestamps() error {
|
||||
headerSelectedTipHash, err := flow.Domain().StagingConsensus().GetHeadersSelectedTip()
|
||||
if err != nil {
|
||||
return err
|
||||
@ -367,7 +423,7 @@ func (flow *handleRelayInvsFlow) validatePruningPointFutureHeaderTimestamps() er
|
||||
return nil
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) receiveAndInsertPruningPointUTXOSet(
|
||||
func (flow *handleIBDFlow) receiveAndInsertPruningPointUTXOSet(
|
||||
consensus externalapi.Consensus, pruningPointHash *externalapi.DomainHash) (bool, error) {
|
||||
|
||||
onEnd := logger.LogAndMeasureExecutionTime(log, "receiveAndInsertPruningPointUTXOSet")
|
||||
@ -376,7 +432,7 @@ func (flow *handleRelayInvsFlow) receiveAndInsertPruningPointUTXOSet(
|
||||
receivedChunkCount := 0
|
||||
receivedUTXOCount := 0
|
||||
for {
|
||||
message, err := flow.dequeueIncomingMessageAndSkipInvs(common.DefaultTimeout)
|
||||
message, err := flow.incomingRoute.DequeueWithTimeout(common.DefaultTimeout)
|
||||
if err != nil {
|
||||
return false, err
|
||||
}
|
||||
@ -422,7 +478,7 @@ func (flow *handleRelayInvsFlow) receiveAndInsertPruningPointUTXOSet(
|
||||
}
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) syncMissingBlockBodies(highHash *externalapi.DomainHash) error {
|
||||
func (flow *handleIBDFlow) syncMissingBlockBodies(highHash *externalapi.DomainHash) error {
|
||||
hashes, err := flow.Domain().Consensus().GetMissingBlockBodyHashes(highHash)
|
||||
if err != nil {
|
||||
return err
|
||||
@ -449,7 +505,7 @@ func (flow *handleRelayInvsFlow) syncMissingBlockBodies(highHash *externalapi.Do
|
||||
}
|
||||
|
||||
for _, expectedHash := range hashesToRequest {
|
||||
message, err := flow.dequeueIncomingMessageAndSkipInvs(common.DefaultTimeout)
|
||||
message, err := flow.incomingRoute.DequeueWithTimeout(common.DefaultTimeout)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -489,7 +545,16 @@ func (flow *handleRelayInvsFlow) syncMissingBlockBodies(highHash *externalapi.Do
|
||||
return flow.resolveVirtual()
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) resolveVirtual() error {
|
||||
func (flow *handleIBDFlow) banIfBlockIsHeaderOnly(block *externalapi.DomainBlock) error {
|
||||
if len(block.Transactions) == 0 {
|
||||
return protocolerrors.Errorf(true, "sent header of %s block where expected block with body",
|
||||
consensushashing.BlockHash(block))
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (flow *handleIBDFlow) resolveVirtual() error {
|
||||
for i := 0; ; i++ {
|
||||
if i%10 == 0 {
|
||||
log.Infof("Resolving virtual. This may take some time...")
|
||||
@ -510,18 +575,3 @@ func (flow *handleRelayInvsFlow) resolveVirtual() error {
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// dequeueIncomingMessageAndSkipInvs is a convenience method to be used during
|
||||
// IBD. Inv messages are expected to arrive at any given moment, but should be
|
||||
// ignored while we're in IBD
|
||||
func (flow *handleRelayInvsFlow) dequeueIncomingMessageAndSkipInvs(timeout time.Duration) (appmessage.Message, error) {
|
||||
for {
|
||||
message, err := flow.incomingRoute.DequeueWithTimeout(timeout)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if _, ok := message.(*appmessage.MsgInvRelayBlock); !ok {
|
||||
return message, nil
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -11,7 +11,7 @@ import (
|
||||
"github.com/pkg/errors"
|
||||
)
|
||||
|
||||
func (flow *handleRelayInvsFlow) ibdWithHeadersProof(highHash *externalapi.DomainHash) error {
|
||||
func (flow *handleIBDFlow) ibdWithHeadersProof(highHash *externalapi.DomainHash) error {
|
||||
err := flow.Domain().InitStagingConsensus()
|
||||
if err != nil {
|
||||
return err
|
||||
@ -44,7 +44,7 @@ func (flow *handleRelayInvsFlow) ibdWithHeadersProof(highHash *externalapi.Domai
|
||||
return nil
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) shouldSyncAndShouldDownloadHeadersProof(highBlock *externalapi.DomainBlock,
|
||||
func (flow *handleIBDFlow) shouldSyncAndShouldDownloadHeadersProof(highBlock *externalapi.DomainBlock,
|
||||
highestSharedBlockFound bool) (shouldDownload, shouldSync bool, err error) {
|
||||
|
||||
if !highestSharedBlockFound {
|
||||
@ -63,7 +63,7 @@ func (flow *handleRelayInvsFlow) shouldSyncAndShouldDownloadHeadersProof(highBlo
|
||||
return false, true, nil
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) checkIfHighHashHasMoreBlueWorkThanSelectedTipAndPruningDepthMoreBlueScore(highBlock *externalapi.DomainBlock) (bool, error) {
|
||||
func (flow *handleIBDFlow) checkIfHighHashHasMoreBlueWorkThanSelectedTipAndPruningDepthMoreBlueScore(highBlock *externalapi.DomainBlock) (bool, error) {
|
||||
headersSelectedTip, err := flow.Domain().Consensus().GetHeadersSelectedTip()
|
||||
if err != nil {
|
||||
return false, err
|
||||
@ -81,13 +81,13 @@ func (flow *handleRelayInvsFlow) checkIfHighHashHasMoreBlueWorkThanSelectedTipAn
|
||||
return highBlock.Header.BlueWork().Cmp(headersSelectedTipInfo.BlueWork) > 0, nil
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) syncAndValidatePruningPointProof() (*externalapi.DomainHash, error) {
|
||||
func (flow *handleIBDFlow) syncAndValidatePruningPointProof() (*externalapi.DomainHash, error) {
|
||||
log.Infof("Downloading the pruning point proof from %s", flow.peer)
|
||||
err := flow.outgoingRoute.Enqueue(appmessage.NewMsgRequestPruningPointProof())
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
message, err := flow.dequeueIncomingMessageAndSkipInvs(common.DefaultTimeout)
|
||||
message, err := flow.incomingRoute.DequeueWithTimeout(common.DefaultTimeout)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@ -113,7 +113,7 @@ func (flow *handleRelayInvsFlow) syncAndValidatePruningPointProof() (*externalap
|
||||
return consensushashing.HeaderHash(pruningPointProof.Headers[0][len(pruningPointProof.Headers[0])-1]), nil
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) downloadHeadersAndPruningUTXOSet(highHash *externalapi.DomainHash) error {
|
||||
func (flow *handleIBDFlow) downloadHeadersAndPruningUTXOSet(highHash *externalapi.DomainHash) error {
|
||||
proofPruningPoint, err := flow.syncAndValidatePruningPointProof()
|
||||
if err != nil {
|
||||
return err
|
||||
@ -164,7 +164,7 @@ func (flow *handleRelayInvsFlow) downloadHeadersAndPruningUTXOSet(highHash *exte
|
||||
return nil
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) syncPruningPointsAndPruningPointAnticone(proofPruningPoint *externalapi.DomainHash) error {
|
||||
func (flow *handleIBDFlow) syncPruningPointsAndPruningPointAnticone(proofPruningPoint *externalapi.DomainHash) error {
|
||||
log.Infof("Downloading the past pruning points and the pruning point anticone from %s", flow.peer)
|
||||
err := flow.outgoingRoute.Enqueue(appmessage.NewMsgRequestPruningPointAndItsAnticone())
|
||||
if err != nil {
|
||||
@ -214,15 +214,15 @@ func (flow *handleRelayInvsFlow) syncPruningPointsAndPruningPointAnticone(proofP
|
||||
return nil
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) processBlockWithTrustedData(
|
||||
func (flow *handleIBDFlow) processBlockWithTrustedData(
|
||||
consensus externalapi.Consensus, block *appmessage.MsgBlockWithTrustedData) error {
|
||||
|
||||
_, err := consensus.ValidateAndInsertBlockWithTrustedData(appmessage.BlockWithTrustedDataToDomainBlockWithTrustedData(block), false)
|
||||
return err
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) receiveBlockWithTrustedData() (*appmessage.MsgBlockWithTrustedData, bool, error) {
|
||||
message, err := flow.dequeueIncomingMessageAndSkipInvs(common.DefaultTimeout)
|
||||
func (flow *handleIBDFlow) receiveBlockWithTrustedData() (*appmessage.MsgBlockWithTrustedData, bool, error) {
|
||||
message, err := flow.incomingRoute.DequeueWithTimeout(common.DefaultTimeout)
|
||||
if err != nil {
|
||||
return nil, false, err
|
||||
}
|
||||
@ -242,8 +242,8 @@ func (flow *handleRelayInvsFlow) receiveBlockWithTrustedData() (*appmessage.MsgB
|
||||
}
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) receivePruningPoints() (*appmessage.MsgPruningPoints, error) {
|
||||
message, err := flow.dequeueIncomingMessageAndSkipInvs(common.DefaultTimeout)
|
||||
func (flow *handleIBDFlow) receivePruningPoints() (*appmessage.MsgPruningPoints, error) {
|
||||
message, err := flow.incomingRoute.DequeueWithTimeout(common.DefaultTimeout)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@ -258,7 +258,7 @@ func (flow *handleRelayInvsFlow) receivePruningPoints() (*appmessage.MsgPruningP
|
||||
return msgPruningPoints, nil
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) validateAndInsertPruningPoints(proofPruningPoint *externalapi.DomainHash) error {
|
||||
func (flow *handleIBDFlow) validateAndInsertPruningPoints(proofPruningPoint *externalapi.DomainHash) error {
|
||||
currentPruningPoint, err := flow.Domain().Consensus().PruningPoint()
|
||||
if err != nil {
|
||||
return err
|
||||
@ -302,7 +302,7 @@ func (flow *handleRelayInvsFlow) validateAndInsertPruningPoints(proofPruningPoin
|
||||
return nil
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) syncPruningPointUTXOSet(consensus externalapi.Consensus,
|
||||
func (flow *handleIBDFlow) syncPruningPointUTXOSet(consensus externalapi.Consensus,
|
||||
pruningPoint *externalapi.DomainHash) (bool, error) {
|
||||
|
||||
log.Infof("Checking if the suggested pruning point %s is compatible to the node DAG", pruningPoint)
|
||||
@ -330,7 +330,7 @@ func (flow *handleRelayInvsFlow) syncPruningPointUTXOSet(consensus externalapi.C
|
||||
return true, nil
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) fetchMissingUTXOSet(consensus externalapi.Consensus, pruningPointHash *externalapi.DomainHash) (succeed bool, err error) {
|
||||
func (flow *handleIBDFlow) fetchMissingUTXOSet(consensus externalapi.Consensus, pruningPointHash *externalapi.DomainHash) (succeed bool, err error) {
|
||||
defer func() {
|
||||
err := flow.Domain().StagingConsensus().ClearImportedPruningPointData()
|
||||
if err != nil {
|
||||
|
@ -58,6 +58,14 @@ func registerBlockRelayFlows(m protocolManager, router *routerpkg.Router, isStop
|
||||
|
||||
m.RegisterFlow("HandleRelayInvs", router, []appmessage.MessageCommand{
|
||||
appmessage.CmdInvRelayBlock, appmessage.CmdBlock, appmessage.CmdBlockLocator,
|
||||
},
|
||||
isStopping, errChan, func(incomingRoute *routerpkg.Route, peer *peerpkg.Peer) error {
|
||||
return blockrelay.HandleRelayInvs(m.Context(), incomingRoute,
|
||||
outgoingRoute, peer)
|
||||
},
|
||||
),
|
||||
|
||||
m.RegisterFlow("HandleIBD", router, []appmessage.MessageCommand{
|
||||
appmessage.CmdDoneHeaders, appmessage.CmdUnexpectedPruningPoint, appmessage.CmdPruningPointUTXOSetChunk,
|
||||
appmessage.CmdBlockHeaders, appmessage.CmdIBDBlockLocatorHighestHash, appmessage.CmdBlockWithTrustedData,
|
||||
appmessage.CmdDoneBlocksWithTrustedData, appmessage.CmdIBDBlockLocatorHighestHashNotFound,
|
||||
@ -65,7 +73,7 @@ func registerBlockRelayFlows(m protocolManager, router *routerpkg.Router, isStop
|
||||
appmessage.CmdPruningPointProof,
|
||||
},
|
||||
isStopping, errChan, func(incomingRoute *routerpkg.Route, peer *peerpkg.Peer) error {
|
||||
return blockrelay.HandleRelayInvs(m.Context(), incomingRoute,
|
||||
return blockrelay.HandleIBD(m.Context(), incomingRoute,
|
||||
outgoingRoute, peer)
|
||||
},
|
||||
),
|
||||
|
@ -13,15 +13,21 @@ func (flow *handleRelayInvsFlow) sendGetBlockLocator(highHash *externalapi.Domai
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) receiveBlockLocator() (blockLocatorHashes []*externalapi.DomainHash, err error) {
|
||||
message, err := flow.dequeueIncomingMessageAndSkipInvs(common.DefaultTimeout)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
for {
|
||||
message, err := flow.incomingRoute.DequeueWithTimeout(common.DefaultTimeout)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
switch message := message.(type) {
|
||||
case *appmessage.MsgInvRelayBlock:
|
||||
flow.invsQueue = append(flow.invsQueue, message)
|
||||
case *appmessage.MsgBlockLocator:
|
||||
return message.BlockLocatorHashes, nil
|
||||
default:
|
||||
return nil,
|
||||
protocolerrors.Errorf(true, "received unexpected message type. "+
|
||||
"expected: %s, got: %s", appmessage.CmdBlockLocator, message.Command())
|
||||
}
|
||||
}
|
||||
msgBlockLocator, ok := message.(*appmessage.MsgBlockLocator)
|
||||
if !ok {
|
||||
return nil,
|
||||
protocolerrors.Errorf(true, "received unexpected message type. "+
|
||||
"expected: %s, got: %s", appmessage.CmdBlockLocator, message.Command())
|
||||
}
|
||||
return msgBlockLocator.BlockLocatorHashes, nil
|
||||
}
|
||||
|
@ -33,8 +33,6 @@ type RelayInvsContext interface {
|
||||
GetOrphanRoots(orphanHash *externalapi.DomainHash) ([]*externalapi.DomainHash, bool, error)
|
||||
IsOrphan(blockHash *externalapi.DomainHash) bool
|
||||
IsIBDRunning() bool
|
||||
TrySetIBDRunning(ibdPeer *peerpkg.Peer) bool
|
||||
UnsetIBDRunning()
|
||||
IsRecoverableError(err error) bool
|
||||
}
|
||||
|
||||
@ -57,7 +55,10 @@ func HandleRelayInvs(context RelayInvsContext, incomingRoute *router.Route, outg
|
||||
peer: peer,
|
||||
invsQueue: make([]*appmessage.MsgInvRelayBlock, 0),
|
||||
}
|
||||
return flow.start()
|
||||
err := flow.start()
|
||||
// Currently, HandleRelayInvs flow is the only place where IBD is triggered, so the channel can be closed now
|
||||
close(peer.IBDRequestChannel())
|
||||
return err
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) start() error {
|
||||
@ -306,7 +307,14 @@ func (flow *handleRelayInvsFlow) processOrphan(block *externalapi.DomainBlock) e
|
||||
// Start IBD unless we already are in IBD
|
||||
log.Debugf("Block %s is out of orphan resolution range. "+
|
||||
"Attempting to start IBD against it.", blockHash)
|
||||
return flow.runIBDIfNotRunning(block)
|
||||
|
||||
// Send the block to IBD flow via the IBDRequestChannel.
|
||||
// Note that this is a non-blocking send, since if IBD is already running, there is no need to trigger it
|
||||
select {
|
||||
case flow.peer.IBDRequestChannel() <- block:
|
||||
default:
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) isGenesisVirtualSelectedParent() (bool, error) {
|
||||
|
@ -1,22 +1,69 @@
|
||||
package blockrelay
|
||||
|
||||
import (
|
||||
"time"
|
||||
|
||||
"github.com/kaspanet/kaspad/infrastructure/logger"
|
||||
|
||||
"github.com/kaspanet/kaspad/domain/consensus/model"
|
||||
|
||||
"github.com/kaspanet/kaspad/app/appmessage"
|
||||
"github.com/kaspanet/kaspad/app/protocol/common"
|
||||
peerpkg "github.com/kaspanet/kaspad/app/protocol/peer"
|
||||
"github.com/kaspanet/kaspad/app/protocol/protocolerrors"
|
||||
"github.com/kaspanet/kaspad/domain"
|
||||
"github.com/kaspanet/kaspad/domain/consensus/model"
|
||||
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
|
||||
"github.com/kaspanet/kaspad/domain/consensus/ruleerrors"
|
||||
"github.com/kaspanet/kaspad/domain/consensus/utils/consensushashing"
|
||||
"github.com/kaspanet/kaspad/infrastructure/config"
|
||||
"github.com/kaspanet/kaspad/infrastructure/logger"
|
||||
"github.com/kaspanet/kaspad/infrastructure/network/netadapter/router"
|
||||
"github.com/pkg/errors"
|
||||
"time"
|
||||
)
|
||||
|
||||
func (flow *handleRelayInvsFlow) runIBDIfNotRunning(block *externalapi.DomainBlock) error {
|
||||
// IBDContext is the interface for the context needed for the HandleIBD flow.
|
||||
type IBDContext interface {
|
||||
Domain() domain.Domain
|
||||
Config() *config.Config
|
||||
OnNewBlock(block *externalapi.DomainBlock, virtualChangeSet *externalapi.VirtualChangeSet) error
|
||||
OnVirtualChange(virtualChangeSet *externalapi.VirtualChangeSet) error
|
||||
OnPruningPointUTXOSetOverride() error
|
||||
IsIBDRunning() bool
|
||||
TrySetIBDRunning(ibdPeer *peerpkg.Peer) bool
|
||||
UnsetIBDRunning()
|
||||
IsRecoverableError(err error) bool
|
||||
}
|
||||
|
||||
type handleIBDFlow struct {
|
||||
IBDContext
|
||||
incomingRoute, outgoingRoute *router.Route
|
||||
peer *peerpkg.Peer
|
||||
}
|
||||
|
||||
// HandleIBD handles IBD
|
||||
func HandleIBD(context IBDContext, incomingRoute *router.Route, outgoingRoute *router.Route,
|
||||
peer *peerpkg.Peer) error {
|
||||
|
||||
flow := &handleIBDFlow{
|
||||
IBDContext: context,
|
||||
incomingRoute: incomingRoute,
|
||||
outgoingRoute: outgoingRoute,
|
||||
peer: peer,
|
||||
}
|
||||
return flow.start()
|
||||
}
|
||||
|
||||
func (flow *handleIBDFlow) start() error {
|
||||
for {
|
||||
// Wait for IBD requests triggered by other flows
|
||||
block, ok := <-flow.peer.IBDRequestChannel()
|
||||
if !ok {
|
||||
return nil
|
||||
}
|
||||
err := flow.runIBDIfNotRunning(block)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (flow *handleIBDFlow) runIBDIfNotRunning(block *externalapi.DomainBlock) error {
|
||||
wasIBDNotRunning := flow.TrySetIBDRunning(flow.peer)
|
||||
if !wasIBDNotRunning {
|
||||
log.Debugf("IBD is already running")
|
||||
@ -84,7 +131,16 @@ func (flow *handleRelayInvsFlow) runIBDIfNotRunning(block *externalapi.DomainBlo
|
||||
return nil
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) logIBDFinished(isFinishedSuccessfully bool) {
|
||||
func (flow *handleIBDFlow) isGenesisVirtualSelectedParent() (bool, error) {
|
||||
virtualSelectedParent, err := flow.Domain().Consensus().GetVirtualSelectedParent()
|
||||
if err != nil {
|
||||
return false, err
|
||||
}
|
||||
|
||||
return virtualSelectedParent.Equal(flow.Config().NetParams().GenesisHash), nil
|
||||
}
|
||||
|
||||
func (flow *handleIBDFlow) logIBDFinished(isFinishedSuccessfully bool) {
|
||||
successString := "successfully"
|
||||
if !isFinishedSuccessfully {
|
||||
successString = "(interrupted)"
|
||||
@ -95,7 +151,7 @@ func (flow *handleRelayInvsFlow) logIBDFinished(isFinishedSuccessfully bool) {
|
||||
// findHighestSharedBlock attempts to find the highest shared block between the peer
|
||||
// and this node. This method may fail because the peer and us have conflicting pruning
|
||||
// points. In that case we return (nil, false, nil) so that we may stop IBD gracefully.
|
||||
func (flow *handleRelayInvsFlow) findHighestSharedBlockHash(
|
||||
func (flow *handleIBDFlow) findHighestSharedBlockHash(
|
||||
targetHash *externalapi.DomainHash) (*externalapi.DomainHash, bool, error) {
|
||||
|
||||
log.Debugf("Sending a blockLocator to %s between pruning point and headers selected tip", flow.peer)
|
||||
@ -138,7 +194,7 @@ func (flow *handleRelayInvsFlow) findHighestSharedBlockHash(
|
||||
}
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) nextBlockLocator(lowHash, highHash *externalapi.DomainHash) (externalapi.BlockLocator, error) {
|
||||
func (flow *handleIBDFlow) nextBlockLocator(lowHash, highHash *externalapi.DomainHash) (externalapi.BlockLocator, error) {
|
||||
log.Debugf("Sending a blockLocator to %s between %s and %s", flow.peer, lowHash, highHash)
|
||||
blockLocator, err := flow.Domain().Consensus().CreateHeadersSelectedChainBlockLocator(lowHash, highHash)
|
||||
if err != nil {
|
||||
@ -156,7 +212,7 @@ func (flow *handleRelayInvsFlow) nextBlockLocator(lowHash, highHash *externalapi
|
||||
return blockLocator, nil
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) findHighestHashIndex(
|
||||
func (flow *handleIBDFlow) findHighestHashIndex(
|
||||
highestHash *externalapi.DomainHash, blockLocator externalapi.BlockLocator) (int, error) {
|
||||
|
||||
highestHashIndex := 0
|
||||
@ -181,7 +237,7 @@ func (flow *handleRelayInvsFlow) findHighestHashIndex(
|
||||
// fetchHighestHash attempts to fetch the highest hash the peer knows amongst the given
|
||||
// blockLocator. This method may fail because the peer and us have conflicting pruning
|
||||
// points. In that case we return (nil, false, nil) so that we may stop IBD gracefully.
|
||||
func (flow *handleRelayInvsFlow) fetchHighestHash(
|
||||
func (flow *handleIBDFlow) fetchHighestHash(
|
||||
targetHash *externalapi.DomainHash, blockLocator externalapi.BlockLocator) (*externalapi.DomainHash, bool, error) {
|
||||
|
||||
ibdBlockLocatorMessage := appmessage.NewMsgIBDBlockLocator(targetHash, blockLocator)
|
||||
@ -189,7 +245,7 @@ func (flow *handleRelayInvsFlow) fetchHighestHash(
|
||||
if err != nil {
|
||||
return nil, false, err
|
||||
}
|
||||
message, err := flow.dequeueIncomingMessageAndSkipInvs(common.DefaultTimeout)
|
||||
message, err := flow.incomingRoute.DequeueWithTimeout(common.DefaultTimeout)
|
||||
if err != nil {
|
||||
return nil, false, err
|
||||
}
|
||||
@ -209,7 +265,7 @@ func (flow *handleRelayInvsFlow) fetchHighestHash(
|
||||
}
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) syncPruningPointFutureHeaders(consensus externalapi.Consensus, highestSharedBlockHash *externalapi.DomainHash,
|
||||
func (flow *handleIBDFlow) syncPruningPointFutureHeaders(consensus externalapi.Consensus, highestSharedBlockHash *externalapi.DomainHash,
|
||||
highHash *externalapi.DomainHash) error {
|
||||
|
||||
log.Infof("Downloading headers from %s", flow.peer)
|
||||
@ -273,15 +329,15 @@ func (flow *handleRelayInvsFlow) syncPruningPointFutureHeaders(consensus externa
|
||||
}
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) sendRequestHeaders(highestSharedBlockHash *externalapi.DomainHash,
|
||||
func (flow *handleIBDFlow) sendRequestHeaders(highestSharedBlockHash *externalapi.DomainHash,
|
||||
peerSelectedTipHash *externalapi.DomainHash) error {
|
||||
|
||||
msgGetBlockInvs := appmessage.NewMsgRequstHeaders(highestSharedBlockHash, peerSelectedTipHash)
|
||||
return flow.outgoingRoute.Enqueue(msgGetBlockInvs)
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) receiveHeaders() (msgIBDBlock *appmessage.BlockHeadersMessage, doneHeaders bool, err error) {
|
||||
message, err := flow.dequeueIncomingMessageAndSkipInvs(common.DefaultTimeout)
|
||||
func (flow *handleIBDFlow) receiveHeaders() (msgIBDBlock *appmessage.BlockHeadersMessage, doneHeaders bool, err error) {
|
||||
message, err := flow.incomingRoute.DequeueWithTimeout(common.DefaultTimeout)
|
||||
if err != nil {
|
||||
return nil, false, err
|
||||
}
|
||||
@ -300,7 +356,7 @@ func (flow *handleRelayInvsFlow) receiveHeaders() (msgIBDBlock *appmessage.Block
|
||||
}
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) processHeader(consensus externalapi.Consensus, msgBlockHeader *appmessage.MsgBlockHeader) error {
|
||||
func (flow *handleIBDFlow) processHeader(consensus externalapi.Consensus, msgBlockHeader *appmessage.MsgBlockHeader) error {
|
||||
header := appmessage.BlockHeaderToDomainBlockHeader(msgBlockHeader)
|
||||
block := &externalapi.DomainBlock{
|
||||
Header: header,
|
||||
@ -333,7 +389,7 @@ func (flow *handleRelayInvsFlow) processHeader(consensus externalapi.Consensus,
|
||||
return nil
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) validatePruningPointFutureHeaderTimestamps() error {
|
||||
func (flow *handleIBDFlow) validatePruningPointFutureHeaderTimestamps() error {
|
||||
headerSelectedTipHash, err := flow.Domain().StagingConsensus().GetHeadersSelectedTip()
|
||||
if err != nil {
|
||||
return err
|
||||
@ -367,7 +423,7 @@ func (flow *handleRelayInvsFlow) validatePruningPointFutureHeaderTimestamps() er
|
||||
return nil
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) receiveAndInsertPruningPointUTXOSet(
|
||||
func (flow *handleIBDFlow) receiveAndInsertPruningPointUTXOSet(
|
||||
consensus externalapi.Consensus, pruningPointHash *externalapi.DomainHash) (bool, error) {
|
||||
|
||||
onEnd := logger.LogAndMeasureExecutionTime(log, "receiveAndInsertPruningPointUTXOSet")
|
||||
@ -376,7 +432,7 @@ func (flow *handleRelayInvsFlow) receiveAndInsertPruningPointUTXOSet(
|
||||
receivedChunkCount := 0
|
||||
receivedUTXOCount := 0
|
||||
for {
|
||||
message, err := flow.dequeueIncomingMessageAndSkipInvs(common.DefaultTimeout)
|
||||
message, err := flow.incomingRoute.DequeueWithTimeout(common.DefaultTimeout)
|
||||
if err != nil {
|
||||
return false, err
|
||||
}
|
||||
@ -422,7 +478,7 @@ func (flow *handleRelayInvsFlow) receiveAndInsertPruningPointUTXOSet(
|
||||
}
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) syncMissingBlockBodies(highHash *externalapi.DomainHash) error {
|
||||
func (flow *handleIBDFlow) syncMissingBlockBodies(highHash *externalapi.DomainHash) error {
|
||||
hashes, err := flow.Domain().Consensus().GetMissingBlockBodyHashes(highHash)
|
||||
if err != nil {
|
||||
return err
|
||||
@ -449,7 +505,7 @@ func (flow *handleRelayInvsFlow) syncMissingBlockBodies(highHash *externalapi.Do
|
||||
}
|
||||
|
||||
for _, expectedHash := range hashesToRequest {
|
||||
message, err := flow.dequeueIncomingMessageAndSkipInvs(common.DefaultTimeout)
|
||||
message, err := flow.incomingRoute.DequeueWithTimeout(common.DefaultTimeout)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -489,7 +545,16 @@ func (flow *handleRelayInvsFlow) syncMissingBlockBodies(highHash *externalapi.Do
|
||||
return flow.resolveVirtual()
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) resolveVirtual() error {
|
||||
func (flow *handleIBDFlow) banIfBlockIsHeaderOnly(block *externalapi.DomainBlock) error {
|
||||
if len(block.Transactions) == 0 {
|
||||
return protocolerrors.Errorf(true, "sent header of %s block where expected block with body",
|
||||
consensushashing.BlockHash(block))
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (flow *handleIBDFlow) resolveVirtual() error {
|
||||
for i := 0; ; i++ {
|
||||
if i%10 == 0 {
|
||||
log.Infof("Resolving virtual. This may take some time...")
|
||||
@ -510,18 +575,3 @@ func (flow *handleRelayInvsFlow) resolveVirtual() error {
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// dequeueIncomingMessageAndSkipInvs is a convenience method to be used during
|
||||
// IBD. Inv messages are expected to arrive at any given moment, but should be
|
||||
// ignored while we're in IBD
|
||||
func (flow *handleRelayInvsFlow) dequeueIncomingMessageAndSkipInvs(timeout time.Duration) (appmessage.Message, error) {
|
||||
for {
|
||||
message, err := flow.incomingRoute.DequeueWithTimeout(timeout)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if _, ok := message.(*appmessage.MsgInvRelayBlock); !ok {
|
||||
return message, nil
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -11,7 +11,7 @@ import (
|
||||
"github.com/pkg/errors"
|
||||
)
|
||||
|
||||
func (flow *handleRelayInvsFlow) ibdWithHeadersProof(highHash *externalapi.DomainHash) error {
|
||||
func (flow *handleIBDFlow) ibdWithHeadersProof(highHash *externalapi.DomainHash) error {
|
||||
err := flow.Domain().InitStagingConsensus()
|
||||
if err != nil {
|
||||
return err
|
||||
@ -44,7 +44,7 @@ func (flow *handleRelayInvsFlow) ibdWithHeadersProof(highHash *externalapi.Domai
|
||||
return nil
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) shouldSyncAndShouldDownloadHeadersProof(highBlock *externalapi.DomainBlock,
|
||||
func (flow *handleIBDFlow) shouldSyncAndShouldDownloadHeadersProof(highBlock *externalapi.DomainBlock,
|
||||
highestSharedBlockFound bool) (shouldDownload, shouldSync bool, err error) {
|
||||
|
||||
if !highestSharedBlockFound {
|
||||
@ -63,7 +63,7 @@ func (flow *handleRelayInvsFlow) shouldSyncAndShouldDownloadHeadersProof(highBlo
|
||||
return false, true, nil
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) checkIfHighHashHasMoreBlueWorkThanSelectedTipAndPruningDepthMoreBlueScore(highBlock *externalapi.DomainBlock) (bool, error) {
|
||||
func (flow *handleIBDFlow) checkIfHighHashHasMoreBlueWorkThanSelectedTipAndPruningDepthMoreBlueScore(highBlock *externalapi.DomainBlock) (bool, error) {
|
||||
headersSelectedTip, err := flow.Domain().Consensus().GetHeadersSelectedTip()
|
||||
if err != nil {
|
||||
return false, err
|
||||
@ -81,13 +81,13 @@ func (flow *handleRelayInvsFlow) checkIfHighHashHasMoreBlueWorkThanSelectedTipAn
|
||||
return highBlock.Header.BlueWork().Cmp(headersSelectedTipInfo.BlueWork) > 0, nil
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) syncAndValidatePruningPointProof() (*externalapi.DomainHash, error) {
|
||||
func (flow *handleIBDFlow) syncAndValidatePruningPointProof() (*externalapi.DomainHash, error) {
|
||||
log.Infof("Downloading the pruning point proof from %s", flow.peer)
|
||||
err := flow.outgoingRoute.Enqueue(appmessage.NewMsgRequestPruningPointProof())
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
message, err := flow.dequeueIncomingMessageAndSkipInvs(common.DefaultTimeout)
|
||||
message, err := flow.incomingRoute.DequeueWithTimeout(common.DefaultTimeout)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@ -113,7 +113,7 @@ func (flow *handleRelayInvsFlow) syncAndValidatePruningPointProof() (*externalap
|
||||
return consensushashing.HeaderHash(pruningPointProof.Headers[0][len(pruningPointProof.Headers[0])-1]), nil
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) downloadHeadersAndPruningUTXOSet(highHash *externalapi.DomainHash) error {
|
||||
func (flow *handleIBDFlow) downloadHeadersAndPruningUTXOSet(highHash *externalapi.DomainHash) error {
|
||||
proofPruningPoint, err := flow.syncAndValidatePruningPointProof()
|
||||
if err != nil {
|
||||
return err
|
||||
@ -164,7 +164,7 @@ func (flow *handleRelayInvsFlow) downloadHeadersAndPruningUTXOSet(highHash *exte
|
||||
return nil
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) syncPruningPointsAndPruningPointAnticone(proofPruningPoint *externalapi.DomainHash) error {
|
||||
func (flow *handleIBDFlow) syncPruningPointsAndPruningPointAnticone(proofPruningPoint *externalapi.DomainHash) error {
|
||||
log.Infof("Downloading the past pruning points and the pruning point anticone from %s", flow.peer)
|
||||
err := flow.outgoingRoute.Enqueue(appmessage.NewMsgRequestPruningPointAndItsAnticone())
|
||||
if err != nil {
|
||||
@ -176,7 +176,7 @@ func (flow *handleRelayInvsFlow) syncPruningPointsAndPruningPointAnticone(proofP
|
||||
return err
|
||||
}
|
||||
|
||||
message, err := flow.dequeueIncomingMessageAndSkipInvs(common.DefaultTimeout)
|
||||
message, err := flow.incomingRoute.DequeueWithTimeout(common.DefaultTimeout)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -225,7 +225,7 @@ func (flow *handleRelayInvsFlow) syncPruningPointsAndPruningPointAnticone(proofP
|
||||
return nil
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) processBlockWithTrustedData(
|
||||
func (flow *handleIBDFlow) processBlockWithTrustedData(
|
||||
consensus externalapi.Consensus, block *appmessage.MsgBlockWithTrustedDataV4, data *appmessage.MsgTrustedData) error {
|
||||
|
||||
blockWithTrustedData := &externalapi.BlockWithTrustedData{
|
||||
@ -246,8 +246,8 @@ func (flow *handleRelayInvsFlow) processBlockWithTrustedData(
|
||||
return err
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) receiveBlockWithTrustedData() (*appmessage.MsgBlockWithTrustedDataV4, bool, error) {
|
||||
message, err := flow.dequeueIncomingMessageAndSkipInvs(common.DefaultTimeout)
|
||||
func (flow *handleIBDFlow) receiveBlockWithTrustedData() (*appmessage.MsgBlockWithTrustedDataV4, bool, error) {
|
||||
message, err := flow.incomingRoute.DequeueWithTimeout(common.DefaultTimeout)
|
||||
if err != nil {
|
||||
return nil, false, err
|
||||
}
|
||||
@ -267,8 +267,8 @@ func (flow *handleRelayInvsFlow) receiveBlockWithTrustedData() (*appmessage.MsgB
|
||||
}
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) receivePruningPoints() (*appmessage.MsgPruningPoints, error) {
|
||||
message, err := flow.dequeueIncomingMessageAndSkipInvs(common.DefaultTimeout)
|
||||
func (flow *handleIBDFlow) receivePruningPoints() (*appmessage.MsgPruningPoints, error) {
|
||||
message, err := flow.incomingRoute.DequeueWithTimeout(common.DefaultTimeout)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@ -283,7 +283,7 @@ func (flow *handleRelayInvsFlow) receivePruningPoints() (*appmessage.MsgPruningP
|
||||
return msgPruningPoints, nil
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) validateAndInsertPruningPoints(proofPruningPoint *externalapi.DomainHash) error {
|
||||
func (flow *handleIBDFlow) validateAndInsertPruningPoints(proofPruningPoint *externalapi.DomainHash) error {
|
||||
currentPruningPoint, err := flow.Domain().Consensus().PruningPoint()
|
||||
if err != nil {
|
||||
return err
|
||||
@ -327,7 +327,7 @@ func (flow *handleRelayInvsFlow) validateAndInsertPruningPoints(proofPruningPoin
|
||||
return nil
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) syncPruningPointUTXOSet(consensus externalapi.Consensus,
|
||||
func (flow *handleIBDFlow) syncPruningPointUTXOSet(consensus externalapi.Consensus,
|
||||
pruningPoint *externalapi.DomainHash) (bool, error) {
|
||||
|
||||
log.Infof("Checking if the suggested pruning point %s is compatible to the node DAG", pruningPoint)
|
||||
@ -355,7 +355,7 @@ func (flow *handleRelayInvsFlow) syncPruningPointUTXOSet(consensus externalapi.C
|
||||
return true, nil
|
||||
}
|
||||
|
||||
func (flow *handleRelayInvsFlow) fetchMissingUTXOSet(consensus externalapi.Consensus, pruningPointHash *externalapi.DomainHash) (succeed bool, err error) {
|
||||
func (flow *handleIBDFlow) fetchMissingUTXOSet(consensus externalapi.Consensus, pruningPointHash *externalapi.DomainHash) (succeed bool, err error) {
|
||||
defer func() {
|
||||
err := flow.Domain().StagingConsensus().ClearImportedPruningPointData()
|
||||
if err != nil {
|
||||
|
@ -66,6 +66,14 @@ func registerBlockRelayFlows(m protocolManager, router *routerpkg.Router, isStop
|
||||
|
||||
m.RegisterFlow("HandleRelayInvs", router, []appmessage.MessageCommand{
|
||||
appmessage.CmdInvRelayBlock, appmessage.CmdBlock, appmessage.CmdBlockLocator,
|
||||
},
|
||||
isStopping, errChan, func(incomingRoute *routerpkg.Route, peer *peerpkg.Peer) error {
|
||||
return blockrelay.HandleRelayInvs(m.Context(), incomingRoute,
|
||||
outgoingRoute, peer)
|
||||
},
|
||||
),
|
||||
|
||||
m.RegisterFlow("HandleIBD", router, []appmessage.MessageCommand{
|
||||
appmessage.CmdDoneHeaders, appmessage.CmdUnexpectedPruningPoint, appmessage.CmdPruningPointUTXOSetChunk,
|
||||
appmessage.CmdBlockHeaders, appmessage.CmdIBDBlockLocatorHighestHash, appmessage.CmdBlockWithTrustedDataV4,
|
||||
appmessage.CmdDoneBlocksWithTrustedData, appmessage.CmdIBDBlockLocatorHighestHashNotFound,
|
||||
@ -74,7 +82,7 @@ func registerBlockRelayFlows(m protocolManager, router *routerpkg.Router, isStop
|
||||
appmessage.CmdTrustedData,
|
||||
},
|
||||
isStopping, errChan, func(incomingRoute *routerpkg.Route, peer *peerpkg.Peer) error {
|
||||
return blockrelay.HandleRelayInvs(m.Context(), incomingRoute,
|
||||
return blockrelay.HandleIBD(m.Context(), incomingRoute,
|
||||
outgoingRoute, peer)
|
||||
},
|
||||
),
|
||||
|
@ -31,6 +31,8 @@ type Peer struct {
|
||||
lastPingNonce uint64 // The nonce of the last ping we sent
|
||||
lastPingTime time.Time // Time we sent last ping
|
||||
lastPingDuration time.Duration // Time for last ping to return
|
||||
|
||||
ibdRequestChannel chan *externalapi.DomainBlock // A channel used to communicate IBD requests between flows
|
||||
}
|
||||
|
||||
// New returns a new Peer
|
||||
@ -38,6 +40,7 @@ func New(connection *netadapter.NetConnection) *Peer {
|
||||
return &Peer{
|
||||
connection: connection,
|
||||
connectionStarted: time.Now(),
|
||||
ibdRequestChannel: make(chan *externalapi.DomainBlock),
|
||||
}
|
||||
}
|
||||
|
||||
@ -143,3 +146,8 @@ func (p *Peer) LastPingDuration() time.Duration {
|
||||
|
||||
return p.lastPingDuration
|
||||
}
|
||||
|
||||
// IBDRequestChannel returns the channel used in order to communicate an IBD request between peer flows
|
||||
func (p *Peer) IBDRequestChannel() chan *externalapi.DomainBlock {
|
||||
return p.ibdRequestChannel
|
||||
}
|
||||
|
@ -47,6 +47,8 @@ func TestIBD(t *testing.T) {
|
||||
}
|
||||
|
||||
disableOnBlockAddedHandler = true
|
||||
// Wait for syncee to exit IBD
|
||||
time.Sleep(time.Second)
|
||||
// This should trigger resolving the syncee virtual
|
||||
mineNextBlock(t, syncer)
|
||||
time.Sleep(time.Second)
|
||||
@ -87,7 +89,7 @@ func TestIBDWithPruning(t *testing.T) {
|
||||
|
||||
start := time.Now()
|
||||
for range ticker.C {
|
||||
if time.Since(start) > defaultTimeout {
|
||||
if time.Since(start) > 2*defaultTimeout {
|
||||
t.Fatalf("Timeout waiting for IBD to finish.")
|
||||
}
|
||||
|
||||
@ -121,6 +123,7 @@ func TestIBDWithPruning(t *testing.T) {
|
||||
// This should trigger resolving the syncee virtual
|
||||
syncerTip := mineNextBlockWithMockTimestamps(t, syncer, rand.New(rand.NewSource(time.Now().UnixNano())))
|
||||
time.Sleep(time.Second)
|
||||
|
||||
synceeSelectedTip, err := syncee.rpcClient.GetSelectedTipHash()
|
||||
if err != nil {
|
||||
t.Fatalf("Error getting tip for syncee")
|
||||
|
Loading…
x
Reference in New Issue
Block a user