[NOD-1579] Improve the IBD mechanism (#1174)

* [NOD-1579] Remove selected tip hash messages.

* [NOD-1579] Start moving IBD stuff into blockrelay.

* [NOD-1579] Rename relaytransactions to transactionrelay.

* [NOD-1579] Move IBD files into blockrelay.

* [NOD-1579] Remove flow stuff from ibd.go.

* [NOD-1579] Bring back IsInIBD().

* [NOD-1579] Simplify block relay flow.

* [NOD-1579] Check orphan pool for missing parents to avoid unnecessary processing.

* [NOD-1579] Implement processOrphan.

* [NOD-1579] Implement addToOrphanSetAndRequestMissingParents.

* [NOD-1579] Fix TestIBD.

* [NOD-1579] Implement isBlockInOrphanResolutionRange.

* [NOD-1579] Implement limited block locators.

* [NOD-1579] Add some comments.

* [NOD-1579] Specifically check for StatusHeaderOnly in blockrelay.

* [NOD-1579] Simplify runIBDIfNotRunning.

* [NOD-1579] Don't run IBD if it is already running.

* [NOD-1579] Fix a comment.

* [NOD-1579] Rename mode to syncInfo.

* [NOD-1579] Simplify validateAndInsertBlock.

* [NOD-1579] Fix bad SyncStateSynced condition.

* [NOD-1579] Implement validateAgainstSyncStateAndResolveInsertMode.

* [NOD-1579] Use insertModeHeader.

* [NOD-1579] Add logs to TrySetIBDRunning and UnsetIBDRunning.

* [NOD-1579] Implement and use dequeueIncomingMessageAndSkipInvs.

* [NOD-1579] Fix a log.

* [NOD-1579] Fix a bug in createBlockLocator.

* [NOD-1579] Rename a variable.

* [NOD-1579] Fix a slew of bugs in missingBlockBodyHashes and selectedChildIterator.

* [NOD-1579] Fix bad chunk size in syncMissingBlockBodies.

* [NOD-1579] Remove maxOrphanBlueScoreDiff.

* [NOD-1579] Fix merge errors.

* [NOD-1579] Remove a debug log.

* [NOD-1579] Add logs.

* [NOD-1579] Make various go quality tools happy.

* [NOD-1579] Fix a typo in a variable name.

* [NOD-1579] Fix full blocks over header-only blocks not failing the missing-parents validation.

* [NOD-1579] Add an error log about a condition that should never happen.

* [NOD-1579] Check all antiPast hashes instead of just the lowHash's anticone to filter for header-only blocks.

* [NOD-1579] Remove the nil stuff from GetBlockLocator.

* [NOD-1579] Remove superfluous condition in handleRelayInvsFlow.start().

* [NOD-1579] Return a boolean from requestBlock instead of comparing to nil.

* [NOD-1579] Fix a bad log.Debugf.

* [NOD-1579] Remove a redundant check.

* [NOD-1579] Change an info log to a warning log.

* [NOD-1579] Move OnNewBlock out of relayBlock.

* [NOD-1579] Remove redundant exists check from runIBDIfNotRunning.

* [NOD-1579] Fix bad call to OnNewBlock.

* [NOD-1579] Remove an impossible check.

* [NOD-1579] Added a log.

* [NOD-1579] Rename insertModeBlockWithoutUpdatingVirtual to insertModeBlockBody.

* [NOD-1579] Add a check for duplicate headers.

* [NOD-1579] Added a comment.

* [NOD-1579] Tighten a stop condition.

* [NOD-1579] Simplify a log.

* [NOD-1579] Clarify a log.

* [NOD-1579] Move a log.
This commit is contained in:
stasatdaglabs 2020-12-06 16:23:56 +02:00 committed by GitHub
parent 4886425caf
commit 7f899b0d09
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
65 changed files with 2217 additions and 2968 deletions

View File

@ -41,8 +41,6 @@ const (
CmdPong
CmdRequestBlockLocator
CmdBlockLocator
CmdSelectedTip
CmdRequestSelectedTip
CmdInvRelayBlock
CmdRequestRelayBlocks
CmdInvTransaction
@ -123,8 +121,6 @@ var ProtocolMessageCommandToString = map[MessageCommand]string{
CmdPong: "Pong",
CmdRequestBlockLocator: "RequestBlockLocator",
CmdBlockLocator: "BlockLocator",
CmdSelectedTip: "SelectedTip",
CmdRequestSelectedTip: "RequestSelectedTip",
CmdInvRelayBlock: "InvRelayBlock",
CmdRequestRelayBlocks: "RequestRelayBlocks",
CmdInvTransaction: "InvTransaction",

View File

@ -25,7 +25,7 @@ func TestIBDBlock(t *testing.T) {
bh := NewBlockHeader(1, parentHashes, hashMerkleRoot, acceptedIDMerkleRoot, utxoCommitment, bits, nonce)
// Ensure the command is expected value.
wantCmd := MessageCommand(17)
wantCmd := MessageCommand(15)
msg := NewMsgIBDBlock(NewMsgBlock(bh))
if cmd := msg.Command(); cmd != wantCmd {
t.Errorf("NewMsgIBDBlock: wrong command - got %v want %v",

View File

@ -5,13 +5,14 @@ import (
)
// MsgRequestBlockLocator implements the Message interface and represents a kaspa
// RequestBlockLocator message. It is used to request a block locator between high
// and low hash.
// RequestBlockLocator message. It is used to request a block locator between low
// and high hash.
// The locator is returned via a locator message (MsgBlockLocator).
type MsgRequestBlockLocator struct {
baseMessage
HighHash *externalapi.DomainHash
LowHash *externalapi.DomainHash
HighHash *externalapi.DomainHash
Limit uint32
}
// Command returns the protocol command string for the message. This is part
@ -23,9 +24,10 @@ func (msg *MsgRequestBlockLocator) Command() MessageCommand {
// NewMsgRequestBlockLocator returns a new RequestBlockLocator message that conforms to the
// Message interface using the passed parameters and defaults for the remaining
// fields.
func NewMsgRequestBlockLocator(highHash, lowHash *externalapi.DomainHash) *MsgRequestBlockLocator {
func NewMsgRequestBlockLocator(lowHash, highHash *externalapi.DomainHash, limit uint32) *MsgRequestBlockLocator {
return &MsgRequestBlockLocator{
HighHash: highHash,
LowHash: lowHash,
HighHash: highHash,
Limit: limit,
}
}

View File

@ -18,7 +18,7 @@ func TestRequestBlockLocator(t *testing.T) {
// Ensure the command is expected value.
wantCmd := MessageCommand(9)
msg := NewMsgRequestBlockLocator(highHash, &externalapi.DomainHash{})
msg := NewMsgRequestBlockLocator(highHash, &externalapi.DomainHash{}, 0)
if cmd := msg.Command(); cmd != wantCmd {
t.Errorf("NewMsgRequestBlockLocator: wrong command - got %v want %v",
cmd, wantCmd)

View File

@ -4,10 +4,6 @@ import (
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
)
// MaxRequestIBDBlocksHashes is the maximum number of hashes that can
// be in a single RequestIBDBlocks message.
const MaxRequestIBDBlocksHashes = MaxInvPerMsg
// MsgRequestIBDBlocks implements the Message interface and represents a kaspa
// RequestIBDBlocks message. It is used to request blocks as part of the IBD
// protocol.

View File

@ -1,21 +0,0 @@
package appmessage
// MsgRequestSelectedTip implements the Message interface and represents a kaspa
// RequestSelectedTip message. It is used to request the selected tip of another peer.
//
// This message has no payload.
type MsgRequestSelectedTip struct {
baseMessage
}
// Command returns the protocol command string for the message. This is part
// of the Message interface implementation.
func (msg *MsgRequestSelectedTip) Command() MessageCommand {
return CmdRequestSelectedTip
}
// NewMsgRequestSelectedTip returns a new kaspa RequestSelectedTip message that conforms to the
// Message interface.
func NewMsgRequestSelectedTip() *MsgRequestSelectedTip {
return &MsgRequestSelectedTip{}
}

View File

@ -1,20 +0,0 @@
// Copyright (c) 2013-2016 The btcsuite developers
// Use of this source code is governed by an ISC
// license that can be found in the LICENSE file.
package appmessage
import (
"testing"
)
// TestRequestSelectedTip tests the MsgRequestSelectedTip API.
func TestRequestSelectedTip(t *testing.T) {
// Ensure the command is expected value.
wantCmd := MessageCommand(12)
msg := NewMsgRequestSelectedTip()
if cmd := msg.Command(); cmd != wantCmd {
t.Errorf("NewMsgRequestSelectedTip: wrong command - got %v want %v",
cmd, wantCmd)
}
}

View File

@ -1,28 +0,0 @@
package appmessage
import (
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
)
// MsgSelectedTip implements the Message interface and represents a kaspa
// selectedtip message. It is used to answer getseltip messages and tell
// the asking peer what is the selected tip of this peer.
type MsgSelectedTip struct {
baseMessage
// The selected tip hash of the generator of the message.
SelectedTipHash *externalapi.DomainHash
}
// Command returns the protocol command string for the message. This is part
// of the Message interface implementation.
func (msg *MsgSelectedTip) Command() MessageCommand {
return CmdSelectedTip
}
// NewMsgSelectedTip returns a new kaspa selectedtip message that conforms to the
// Message interface.
func NewMsgSelectedTip(selectedTipHash *externalapi.DomainHash) *MsgSelectedTip {
return &MsgSelectedTip{
SelectedTipHash: selectedTipHash,
}
}

View File

@ -1,18 +0,0 @@
package appmessage
import (
"testing"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
)
// TestSelectedTip tests the MsgSelectedTip API.
func TestSelectedTip(t *testing.T) {
// Ensure the command is expected value.
wantCmd := MessageCommand(11)
msg := NewMsgSelectedTip(&externalapi.DomainHash{})
if cmd := msg.Command(); cmd != wantCmd {
t.Errorf("NewMsgSelectedTip: wrong command - got %v want %v",
cmd, wantCmd)
}
}

View File

@ -53,9 +53,6 @@ type MsgVersion struct {
// on the appmessage. This has a max length of MaxUserAgentLen.
UserAgent string
// The selected tip hash of the generator of the version message.
SelectedTipHash *externalapi.DomainHash
// Don't announce transactions to peer.
DisableRelayTx bool
@ -85,7 +82,7 @@ func (msg *MsgVersion) Command() MessageCommand {
// Message interface using the passed parameters and defaults for the remaining
// fields.
func NewMsgVersion(addr *NetAddress, id *id.ID, network string,
selectedTipHash *externalapi.DomainHash, subnetworkID *externalapi.DomainSubnetworkID) *MsgVersion {
subnetworkID *externalapi.DomainSubnetworkID) *MsgVersion {
// Limit the timestamp to one millisecond precision since the protocol
// doesn't support better.
@ -97,7 +94,6 @@ func NewMsgVersion(addr *NetAddress, id *id.ID, network string,
Address: addr,
ID: id,
UserAgent: DefaultUserAgent,
SelectedTipHash: selectedTipHash,
DisableRelayTx: false,
SubnetworkID: subnetworkID,
}

View File

@ -10,7 +10,6 @@ import (
"testing"
"github.com/davecgh/go-spew/spew"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/kaspanet/kaspad/infrastructure/network/netadapter/id"
)
@ -19,7 +18,6 @@ func TestVersion(t *testing.T) {
pver := ProtocolVersion
// Create version message data.
selectedTipHash := &externalapi.DomainHash{12, 34}
tcpAddrMe := &net.TCPAddr{IP: net.ParseIP("127.0.0.1"), Port: 16111}
me := NewNetAddress(tcpAddrMe, SFNodeNetwork)
generatedID, err := id.GenerateID()
@ -28,7 +26,7 @@ func TestVersion(t *testing.T) {
}
// Ensure we get the correct data back out.
msg := NewMsgVersion(me, generatedID, "mainnet", selectedTipHash, nil)
msg := NewMsgVersion(me, generatedID, "mainnet", nil)
if msg.ProtocolVersion != pver {
t.Errorf("NewMsgVersion: wrong protocol version - got %v, want %v",
msg.ProtocolVersion, pver)
@ -45,10 +43,6 @@ func TestVersion(t *testing.T) {
t.Errorf("NewMsgVersion: wrong user agent - got %v, want %v",
msg.UserAgent, DefaultUserAgent)
}
if *msg.SelectedTipHash != *selectedTipHash {
t.Errorf("NewMsgVersion: wrong selected tip hash - got %s, want %s",
msg.SelectedTipHash, selectedTipHash)
}
if msg.DisableRelayTx {
t.Errorf("NewMsgVersion: disable relay tx is not false by "+
"default - got %v, want %v", msg.DisableRelayTx, false)

View File

@ -41,8 +41,6 @@ type GetConnectedPeerInfoMessage struct {
ID string
Address string
LastPingDuration int64
SelectedTipHash string
IsSyncNode bool
IsOutbound bool
TimeOffset int64
UserAgent string

View File

@ -53,7 +53,7 @@ func (f *FlowContext) broadcastTransactionsAfterBlockAdded(
f.updateTransactionsToRebroadcast(block)
// Don't relay transactions when in IBD.
if atomic.LoadUint32(&f.isInIBD) != 0 {
if f.IsIBDRunning() {
return nil
}
@ -103,3 +103,27 @@ func (f *FlowContext) AddBlock(block *externalapi.DomainBlock) error {
}
return f.Broadcast(appmessage.NewMsgInvBlock(consensushashing.BlockHash(block)))
}
// IsIBDRunning returns true if IBD is currently marked as running
func (f *FlowContext) IsIBDRunning() bool {
return atomic.LoadUint32(&f.isInIBD) != 0
}
// TrySetIBDRunning attempts to set `isInIBD`. Returns false
// if it is already set
func (f *FlowContext) TrySetIBDRunning() bool {
succeeded := atomic.CompareAndSwapUint32(&f.isInIBD, 0, 1)
if succeeded {
log.Infof("IBD started")
}
return succeeded
}
// UnsetIBDRunning unsets isInIBD
func (f *FlowContext) UnsetIBDRunning() {
succeeded := atomic.CompareAndSwapUint32(&f.isInIBD, 1, 0)
if !succeeded {
panic("attempted to unset isInIBD when it was not set to begin with")
}
log.Infof("IBD finished")
}

View File

@ -9,7 +9,7 @@ import (
"github.com/kaspanet/kaspad/domain"
"github.com/kaspanet/kaspad/app/protocol/flows/blockrelay"
"github.com/kaspanet/kaspad/app/protocol/flows/relaytransactions"
"github.com/kaspanet/kaspad/app/protocol/flows/transactionrelay"
peerpkg "github.com/kaspanet/kaspad/app/protocol/peer"
"github.com/kaspanet/kaspad/infrastructure/config"
"github.com/kaspanet/kaspad/infrastructure/network/addressmanager"
@ -41,13 +41,11 @@ type FlowContext struct {
transactionsToRebroadcastLock sync.Mutex
transactionsToRebroadcast map[externalapi.DomainTransactionID]*externalapi.DomainTransaction
lastRebroadcastTime time.Time
sharedRequestedTransactions *relaytransactions.SharedRequestedTransactions
sharedRequestedTransactions *transactionrelay.SharedRequestedTransactions
sharedRequestedBlocks *blockrelay.SharedRequestedBlocks
isInIBD uint32
startIBDMutex sync.Mutex
ibdPeer *peerpkg.Peer
isInIBD uint32
peers map[id.ID]*peerpkg.Peer
peersMutex sync.RWMutex
@ -66,7 +64,7 @@ func New(cfg *config.Config, domain domain.Domain, addressManager *addressmanage
domain: domain,
addressManager: addressManager,
connectionManager: connectionManager,
sharedRequestedTransactions: relaytransactions.NewSharedRequestedTransactions(),
sharedRequestedTransactions: transactionrelay.NewSharedRequestedTransactions(),
sharedRequestedBlocks: blockrelay.NewSharedRequestedBlocks(),
peers: make(map[id.ID]*peerpkg.Peer),
transactionsToRebroadcast: make(map[externalapi.DomainTransactionID]*externalapi.DomainTransaction),

View File

@ -1,130 +0,0 @@
package flowcontext
import (
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"sync/atomic"
"time"
"github.com/kaspanet/kaspad/util/mstime"
peerpkg "github.com/kaspanet/kaspad/app/protocol/peer"
)
// StartIBDIfRequired selects a peer and starts IBD against it
// if required
func (f *FlowContext) StartIBDIfRequired() error {
f.startIBDMutex.Lock()
defer f.startIBDMutex.Unlock()
if f.IsInIBD() {
return nil
}
syncInfo, err := f.domain.Consensus().GetSyncInfo()
if err != nil {
return err
}
if syncInfo.State == externalapi.SyncStateRelay {
return nil
}
peer, err := f.selectPeerForIBD(syncInfo)
if err != nil {
return err
}
if peer == nil {
spawn("StartIBDIfRequired-requestSelectedTipsIfRequired", f.requestSelectedTipsIfRequired)
return nil
}
atomic.StoreUint32(&f.isInIBD, 1)
f.ibdPeer = peer
spawn("StartIBDIfRequired-peer.StartIBD", peer.StartIBD)
return nil
}
// IsInIBD is true if IBD is currently running
func (f *FlowContext) IsInIBD() bool {
return atomic.LoadUint32(&f.isInIBD) != 0
}
// selectPeerForIBD returns the first peer whose selected tip
// hash is not in our DAG
func (f *FlowContext) selectPeerForIBD(syncInfo *externalapi.SyncInfo) (*peerpkg.Peer, error) {
f.peersMutex.RLock()
defer f.peersMutex.RUnlock()
for _, peer := range f.peers {
peerSelectedTipHash := peer.SelectedTipHash()
if f.IsOrphan(peerSelectedTipHash) {
continue
}
blockInfo, err := f.domain.Consensus().GetBlockInfo(peerSelectedTipHash)
if err != nil {
return nil, err
}
if syncInfo.State == externalapi.SyncStateHeadersFirst {
if !blockInfo.Exists {
return peer, nil
}
} else {
if blockInfo.Exists && blockInfo.BlockStatus == externalapi.StatusHeaderOnly &&
blockInfo.IsBlockInHeaderPruningPointFuture {
return peer, nil
}
}
}
return nil, nil
}
func (f *FlowContext) requestSelectedTipsIfRequired() {
dagTimeCurrent, err := f.shouldRequestSelectedTips()
if err != nil {
panic(err)
}
if dagTimeCurrent {
return
}
f.requestSelectedTips()
}
func (f *FlowContext) shouldRequestSelectedTips() (bool, error) {
const minDurationToRequestSelectedTips = time.Minute
virtualSelectedParent, err := f.domain.Consensus().GetVirtualSelectedParent()
if err != nil {
return false, err
}
virtualSelectedParentTime := mstime.UnixMilliseconds(virtualSelectedParent.Header.TimeInMilliseconds)
return mstime.Now().Sub(virtualSelectedParentTime) > minDurationToRequestSelectedTips, nil
}
func (f *FlowContext) requestSelectedTips() {
f.peersMutex.RLock()
defer f.peersMutex.RUnlock()
for _, peer := range f.peers {
peer.RequestSelectedTipIfRequired()
}
}
// FinishIBD finishes the current IBD flow and starts a new one if required.
func (f *FlowContext) FinishIBD() error {
f.ibdPeer = nil
atomic.StoreUint32(&f.isInIBD, 0)
return f.StartIBDIfRequired()
}
// IBDPeer returns the currently active IBD peer.
// Returns nil if we aren't currently in IBD
func (f *FlowContext) IBDPeer() *peerpkg.Peer {
if !f.IsInIBD() {
return nil
}
return f.ibdPeer
}

View File

@ -52,7 +52,7 @@ func (f *FlowContext) UnorphanBlocks(rootBlock *externalapi.DomainBlock) ([]*ext
if err != nil {
return nil, err
}
if !orphanBlockParentInfo.Exists {
if !orphanBlockParentInfo.Exists || orphanBlockParentInfo.BlockStatus == externalapi.StatusHeaderOnly {
log.Tracef("Cannot unorphan block %s. It's missing at "+
"least the following parent: %s", orphanHash, orphanBlockParentHash)

View File

@ -4,7 +4,7 @@ import (
"time"
"github.com/kaspanet/kaspad/app/appmessage"
"github.com/kaspanet/kaspad/app/protocol/flows/relaytransactions"
"github.com/kaspanet/kaspad/app/protocol/flows/transactionrelay"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/kaspanet/kaspad/domain/consensus/utils/consensushashing"
)
@ -54,9 +54,9 @@ func (f *FlowContext) txIDsToRebroadcast() []*externalapi.DomainTransactionID {
return txIDs
}
// SharedRequestedTransactions returns a *relaytransactions.SharedRequestedTransactions for sharing
// SharedRequestedTransactions returns a *transactionrelay.SharedRequestedTransactions for sharing
// data about requested transactions between different peers.
func (f *FlowContext) SharedRequestedTransactions() *relaytransactions.SharedRequestedTransactions {
func (f *FlowContext) SharedRequestedTransactions() *transactionrelay.SharedRequestedTransactions {
return f.sharedRequestedTransactions
}

View File

@ -0,0 +1,29 @@
package blockrelay
import (
"github.com/kaspanet/kaspad/app/appmessage"
"github.com/kaspanet/kaspad/app/protocol/common"
"github.com/kaspanet/kaspad/app/protocol/protocolerrors"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
)
func (flow *handleRelayInvsFlow) sendGetBlockLocator(lowHash *externalapi.DomainHash,
highHash *externalapi.DomainHash, limit uint32) error {
msgGetBlockLocator := appmessage.NewMsgRequestBlockLocator(lowHash, highHash, limit)
return flow.outgoingRoute.Enqueue(msgGetBlockLocator)
}
func (flow *handleRelayInvsFlow) receiveBlockLocator() (blockLocatorHashes []*externalapi.DomainHash, err error) {
message, err := flow.dequeueIncomingMessageAndSkipInvs(common.DefaultTimeout)
if err != nil {
return nil, err
}
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
}

View File

@ -1,9 +1,10 @@
package ibd
package blockrelay
import (
"github.com/kaspanet/kaspad/app/appmessage"
"github.com/kaspanet/kaspad/app/protocol/protocolerrors"
"github.com/kaspanet/kaspad/domain"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/kaspanet/kaspad/infrastructure/network/netadapter/router"
"github.com/pkg/errors"
)
@ -30,7 +31,7 @@ func HandleIBDBlockRequests(context HandleIBDBlockRequestsContext, incomingRoute
if err != nil {
return err
}
if !blockInfo.Exists {
if !blockInfo.Exists || blockInfo.BlockStatus == externalapi.StatusHeaderOnly {
return protocolerrors.Errorf(true, "block %s not found", hash)
}
block, err := context.Domain().Consensus().GetBlock(hash)

View File

@ -5,6 +5,7 @@ import (
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/externalapi"
"github.com/kaspanet/kaspad/infrastructure/network/netadapter/router"
"github.com/pkg/errors"
)
@ -31,7 +32,7 @@ func HandleRelayBlockRequests(context RelayBlockRequestsContext, incomingRoute *
if err != nil {
return err
}
if !blockInfo.Exists {
if !blockInfo.Exists || blockInfo.BlockStatus == externalapi.StatusHeaderOnly {
return protocolerrors.Errorf(true, "block %s not found", hash)
}
block, err := context.Domain().Consensus().GetBlock(hash)

View File

@ -9,24 +9,30 @@ import (
"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/logger"
"github.com/kaspanet/kaspad/infrastructure/config"
"github.com/kaspanet/kaspad/infrastructure/network/netadapter"
"github.com/kaspanet/kaspad/infrastructure/network/netadapter/router"
mathUtil "github.com/kaspanet/kaspad/util/math"
"github.com/pkg/errors"
)
// orphanResolutionRange is the maximum amount of blockLocator hashes
// to search for known blocks. See isBlockInOrphanResolutionRange for
// further details
var orphanResolutionRange uint32 = 5
// RelayInvsContext is the interface for the context needed for the HandleRelayInvs flow.
type RelayInvsContext interface {
Domain() domain.Domain
Config() *config.Config
NetAdapter() *netadapter.NetAdapter
OnNewBlock(block *externalapi.DomainBlock) error
SharedRequestedBlocks() *SharedRequestedBlocks
StartIBDIfRequired() error
IsInIBD() bool
Broadcast(message appmessage.Message) error
AddOrphan(orphanBlock *externalapi.DomainBlock)
IsOrphan(blockHash *externalapi.DomainHash) bool
IsIBDRunning() bool
TrySetIBDRunning() bool
UnsetIBDRunning()
}
type handleRelayInvsFlow struct {
@ -79,24 +85,45 @@ func (flow *handleRelayInvsFlow) start() error {
continue
}
err = flow.StartIBDIfRequired()
if err != nil {
return err
}
if flow.IsInIBD() {
// Block relay is disabled during IBD
if flow.IsIBDRunning() {
log.Debugf("Got block %s while in IBD. continuing...", inv.Hash)
// Block relay is disabled during IBD
continue
}
requestQueue := newHashesQueueSet()
requestQueue.enqueueIfNotExists(inv.Hash)
log.Debugf("Requesting block %s", inv.Hash)
block, exists, err := flow.requestBlock(inv.Hash)
if err != nil {
return err
}
if exists {
log.Debugf("Aborting requesting block %s because it already exists", inv.Hash)
continue
}
for requestQueue.len() > 0 {
err := flow.requestBlocks(requestQueue)
log.Debugf("Processing block %s", inv.Hash)
missingParents, err := flow.processBlock(block)
if err != nil {
return err
}
if len(missingParents) > 0 {
log.Debugf("Block %s contains orphans: %s", inv.Hash, missingParents)
err := flow.processOrphan(block, missingParents)
if err != nil {
return err
}
continue
}
log.Debugf("Relaying block %s", inv.Hash)
err = flow.relayBlock(block)
if err != nil {
return err
}
log.Infof("Accepted block %s via relay", inv.Hash)
err = flow.OnNewBlock(block)
if err != nil {
return err
}
}
}
@ -121,73 +148,34 @@ func (flow *handleRelayInvsFlow) readInv() (*appmessage.MsgInvRelayBlock, error)
return inv, nil
}
func (flow *handleRelayInvsFlow) requestBlocks(requestQueue *hashesQueueSet) error {
onEnd := logger.LogAndMeasureExecutionTime(log, "handleRelayInvsFlow.requestBlocks")
defer onEnd()
numHashesToRequest := mathUtil.MinInt(appmessage.MaxRequestRelayBlocksHashes, requestQueue.len())
hashesToRequest := requestQueue.dequeue(numHashesToRequest)
requestedBlocksSet := map[externalapi.DomainHash]struct{}{}
var filteredHashesToRequest []*externalapi.DomainHash
for _, hash := range hashesToRequest {
exists := flow.SharedRequestedBlocks().addIfNotExists(hash)
if exists {
continue
}
// The block can become known from another peer in the process of orphan resolution
blockInfo, err := flow.Domain().Consensus().GetBlockInfo(hash)
if err != nil {
return err
}
if blockInfo.Exists {
continue
}
requestedBlocksSet[*hash] = struct{}{}
filteredHashesToRequest = append(filteredHashesToRequest, hash)
}
// Exit early if we've filtered out all the hashes
if len(filteredHashesToRequest) == 0 {
log.Debugf("requestBlocks - no hashes to request. Returning..")
return nil
func (flow *handleRelayInvsFlow) requestBlock(requestHash *externalapi.DomainHash) (*externalapi.DomainBlock, bool, error) {
exists := flow.SharedRequestedBlocks().addIfNotExists(requestHash)
if exists {
return nil, true, nil
}
// In case the function returns earlier than expected, we want to make sure flow.SharedRequestedBlocks() is
// clean from any pending blocks.
defer flow.SharedRequestedBlocks().removeSet(requestedBlocksSet)
defer flow.SharedRequestedBlocks().remove(requestHash)
getRelayBlocksMsg := appmessage.NewMsgRequestRelayBlocks(filteredHashesToRequest)
getRelayBlocksMsg := appmessage.NewMsgRequestRelayBlocks([]*externalapi.DomainHash{requestHash})
err := flow.outgoingRoute.Enqueue(getRelayBlocksMsg)
if err != nil {
return err
return nil, false, err
}
for _, expectedHash := range filteredHashesToRequest {
log.Debugf("Waiting for block %s", expectedHash)
msgBlock, err := flow.readMsgBlock()
if err != nil {
return err
}
block := appmessage.MsgBlockToDomainBlock(msgBlock)
blockHash := consensushashing.BlockHash(block)
log.Criticalf("got block %s", blockHash)
if *blockHash != *expectedHash {
return protocolerrors.Errorf(true, "expected block %s but got %s", expectedHash, blockHash)
}
err = flow.processAndRelayBlock(requestQueue, requestedBlocksSet, block)
if err != nil {
return err
}
flow.SharedRequestedBlocks().remove(blockHash)
msgBlock, err := flow.readMsgBlock()
if err != nil {
return nil, false, err
}
return nil
block := appmessage.MsgBlockToDomainBlock(msgBlock)
blockHash := consensushashing.BlockHash(block)
if *blockHash != *requestHash {
return nil, false, protocolerrors.Errorf(true, "got unrequested block %s", blockHash)
}
return block, false, nil
}
// readMsgBlock returns the next msgBlock in msgChan, and populates invsQueue with any inv messages that meanwhile arrive.
@ -211,52 +199,92 @@ func (flow *handleRelayInvsFlow) readMsgBlock() (msgBlock *appmessage.MsgBlock,
}
}
func (flow *handleRelayInvsFlow) processAndRelayBlock(requestQueue *hashesQueueSet, requestedBlocksSet map[externalapi.DomainHash]struct{}, block *externalapi.DomainBlock) error {
func (flow *handleRelayInvsFlow) processBlock(block *externalapi.DomainBlock) ([]*externalapi.DomainHash, error) {
blockHash := consensushashing.BlockHash(block)
log.Debugf("processAndRelayBlock start for block %s", blockHash)
defer log.Debugf("processAndRelayBlock end for block %s", blockHash)
err := flow.Domain().Consensus().ValidateAndInsertBlock(block)
if err != nil {
if !errors.As(err, &ruleerrors.RuleError{}) {
return errors.Wrapf(err, "failed to process block %s", blockHash)
return nil, errors.Wrapf(err, "failed to process block %s", blockHash)
}
missingParentsError := &ruleerrors.ErrMissingParents{}
if errors.As(err, missingParentsError) {
// Add the orphan to the orphan pool
flow.AddOrphan(block)
// Request the parents for the orphan block from the peer that sent it.
for _, missingAncestor := range missingParentsError.MissingParentHashes {
if _, ok := requestedBlocksSet[*missingAncestor]; ok {
log.Debugf("ancestor %s already exists in requestedBlocksSet", missingAncestor)
continue
}
requestQueue.enqueueIfNotExists(missingAncestor)
}
return nil
return missingParentsError.MissingParentHashes, nil
}
log.Infof("Rejected block %s from %s: %s", blockHash, flow.peer, err)
log.Warnf("Rejected block %s from %s: %s", blockHash, flow.peer, err)
return protocolerrors.Wrapf(true, err, "got invalid block %s from relay", blockHash)
return nil, protocolerrors.Wrapf(true, err, "got invalid block %s from relay", blockHash)
}
err = flow.Broadcast(appmessage.NewMsgInvBlock(blockHash))
if err != nil {
return err
}
log.Debugf("Accepted block %s via relay", blockHash)
err = flow.StartIBDIfRequired()
if err != nil {
return err
}
err = flow.OnNewBlock(block)
if err != nil {
return err
}
return nil
return nil, nil
}
func (flow *handleRelayInvsFlow) relayBlock(block *externalapi.DomainBlock) error {
blockHash := consensushashing.BlockHash(block)
return flow.Broadcast(appmessage.NewMsgInvBlock(blockHash))
}
func (flow *handleRelayInvsFlow) processOrphan(block *externalapi.DomainBlock, missingParents []*externalapi.DomainHash) error {
blockHash := consensushashing.BlockHash(block)
// Return if the block has been orphaned from elsewhere already
if flow.IsOrphan(blockHash) {
log.Debugf("Skipping orphan processing for block %s because it is already an orphan", blockHash)
return nil
}
// Add the block to the orphan set if it's within orphan resolution range
isBlockInOrphanResolutionRange, err := flow.isBlockInOrphanResolutionRange(blockHash)
if err != nil {
return err
}
if isBlockInOrphanResolutionRange {
log.Debugf("Block %s is within orphan resolution range. "+
"Adding it to the orphan set and requesting its missing parents", blockHash)
flow.addToOrphanSetAndRequestMissingParents(block, missingParents)
return nil
}
// 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(blockHash)
}
// isBlockInOrphanResolutionRange finds out whether the given blockHash should be
// retrieved via the unorphaning mechanism or via IBD. This method sends a
// getBlockLocator request to the peer with a limit of orphanResolutionRange.
// In the response, if we know none of the hashes, we should retrieve the given
// blockHash via IBD. Otherwise, via unorphaning.
func (flow *handleRelayInvsFlow) isBlockInOrphanResolutionRange(blockHash *externalapi.DomainHash) (bool, error) {
lowHash := flow.Config().ActiveNetParams.GenesisHash
err := flow.sendGetBlockLocator(lowHash, blockHash, orphanResolutionRange)
if err != nil {
return false, err
}
blockLocatorHashes, err := flow.receiveBlockLocator()
if err != nil {
return false, err
}
for _, blockLocatorHash := range blockLocatorHashes {
blockInfo, err := flow.Domain().Consensus().GetBlockInfo(blockLocatorHash)
if err != nil {
return false, err
}
if blockInfo.Exists && blockInfo.BlockStatus != externalapi.StatusHeaderOnly {
return true, nil
}
}
return false, nil
}
func (flow *handleRelayInvsFlow) addToOrphanSetAndRequestMissingParents(
block *externalapi.DomainBlock, missingParents []*externalapi.DomainHash) {
flow.AddOrphan(block)
invMessages := make([]*appmessage.MsgInvRelayBlock, len(missingParents))
for i, missingParent := range missingParents {
invMessages[i] = appmessage.NewMsgInvBlock(missingParent)
}
flow.invsQueue = append(invMessages, flow.invsQueue...)
}

View File

@ -1,4 +1,4 @@
package ibd
package blockrelay
import (
"github.com/kaspanet/kaspad/app/appmessage"
@ -32,13 +32,16 @@ func HandleRequestBlockLocator(context RequestBlockLocatorContext, incomingRoute
func (flow *handleRequestBlockLocatorFlow) start() error {
for {
lowHash, highHash, err := flow.receiveGetBlockLocator()
lowHash, highHash, limit, err := flow.receiveGetBlockLocator()
if err != nil {
return err
}
locator, err := flow.Domain().Consensus().CreateBlockLocator(lowHash, highHash)
locator, err := flow.Domain().Consensus().CreateBlockLocator(lowHash, highHash, limit)
if err != nil || len(locator) == 0 {
if err != nil {
log.Debugf("Received error from CreateBlockLocator: %s", err)
}
return protocolerrors.Errorf(true, "couldn't build a block "+
"locator between blocks %s and %s", lowHash, highHash)
}
@ -51,15 +54,15 @@ func (flow *handleRequestBlockLocatorFlow) start() error {
}
func (flow *handleRequestBlockLocatorFlow) receiveGetBlockLocator() (lowHash *externalapi.DomainHash,
highHash *externalapi.DomainHash, err error) {
highHash *externalapi.DomainHash, limit uint32, err error) {
message, err := flow.incomingRoute.Dequeue()
if err != nil {
return nil, nil, err
return nil, nil, 0, err
}
msgGetBlockLocator := message.(*appmessage.MsgRequestBlockLocator)
return msgGetBlockLocator.LowHash, msgGetBlockLocator.HighHash, nil
return msgGetBlockLocator.LowHash, msgGetBlockLocator.HighHash, msgGetBlockLocator.Limit, nil
}
func (flow *handleRequestBlockLocatorFlow) sendBlockLocator(locator externalapi.BlockLocator) error {

View File

@ -1,4 +1,4 @@
package ibd
package blockrelay
import (
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"

View File

@ -1,4 +1,4 @@
package ibd
package blockrelay
import (
"github.com/kaspanet/kaspad/domain/consensus/utils/testutils"

View File

@ -1,126 +1,255 @@
package ibd
package blockrelay
import (
"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/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/network/netadapter/router"
"github.com/pkg/errors"
"time"
)
// HandleIBDContext is the interface for the context needed for the HandleIBD flow.
type HandleIBDContext interface {
Domain() domain.Domain
Config() *config.Config
OnNewBlock(block *externalapi.DomainBlock) error
FinishIBD() error
}
type handleIBDFlow struct {
HandleIBDContext
incomingRoute, outgoingRoute *router.Route
peer *peerpkg.Peer
}
// HandleIBD waits for IBD start and handles it when IBD is triggered for this peer
func HandleIBD(context HandleIBDContext, incomingRoute *router.Route, outgoingRoute *router.Route,
peer *peerpkg.Peer) error {
flow := &handleIBDFlow{
HandleIBDContext: context,
incomingRoute: incomingRoute,
outgoingRoute: outgoingRoute,
peer: peer,
func (flow *handleRelayInvsFlow) runIBDIfNotRunning(highHash *externalapi.DomainHash) error {
wasIBDNotRunning := flow.TrySetIBDRunning()
if !wasIBDNotRunning {
log.Debugf("IBD is already running")
return nil
}
return flow.start()
}
defer flow.UnsetIBDRunning()
func (flow *handleIBDFlow) start() error {
for {
err := flow.runIBD()
if err != nil {
return err
}
}
}
log.Debugf("IBD started with peer %s and highHash %s", flow.peer, highHash)
func (flow *handleIBDFlow) runIBD() error {
flow.peer.WaitForIBDStart()
err := flow.ibdLoop()
// Fetch all the headers if we don't already have them
log.Debugf("Downloading headers up to %s", highHash)
err := flow.syncHeaders(highHash)
if err != nil {
finishIBDErr := flow.FinishIBD()
if finishIBDErr != nil {
return finishIBDErr
}
return err
}
return flow.FinishIBD()
}
log.Debugf("Finished downloading headers up to %s", highHash)
func (flow *handleIBDFlow) ibdLoop() error {
for {
syncInfo, err := flow.Domain().Consensus().GetSyncInfo()
// Fetch the UTXO set if we don't already have it
log.Debugf("Downloading the IBD root UTXO set under highHash %s", highHash)
syncInfo, err := flow.Domain().Consensus().GetSyncInfo()
if err != nil {
return err
}
if syncInfo.State == externalapi.SyncStateAwaitingUTXOSet {
found, err := flow.fetchMissingUTXOSet(syncInfo.IBDRootUTXOBlockHash)
if err != nil {
return err
}
switch syncInfo.State {
case externalapi.SyncStateHeadersFirst:
err := flow.syncHeaders()
if err != nil {
return err
}
case externalapi.SyncStateMissingUTXOSet:
found, err := flow.fetchMissingUTXOSet(syncInfo.IBDRootUTXOBlockHash)
if err != nil {
return err
}
if !found {
return nil
}
case externalapi.SyncStateMissingBlockBodies:
err := flow.syncMissingBlockBodies()
if err != nil {
return err
}
case externalapi.SyncStateRelay:
if !found {
log.Infof("Cannot download the IBD root UTXO set under highHash %s", highHash)
return nil
default:
return errors.Errorf("unexpected state %s", syncInfo.State)
}
}
}
log.Debugf("Finished downloading the IBD root UTXO set under highHash %s", highHash)
func (flow *handleIBDFlow) syncHeaders() error {
peerSelectedTipHash := flow.peer.SelectedTipHash()
log.Debugf("Trying to find highest shared chain block with peer %s with selected tip %s", flow.peer, peerSelectedTipHash)
highestSharedBlockHash, err := flow.findHighestSharedBlockHash(peerSelectedTipHash)
// Fetch the block bodies
log.Debugf("Downloading block bodies up to %s", highHash)
err = flow.syncMissingBlockBodies(highHash)
if err != nil {
return err
}
log.Debugf("Finished downloading block bodies up to %s", highHash)
return nil
}
func (flow *handleRelayInvsFlow) syncHeaders(highHash *externalapi.DomainHash) error {
log.Debugf("Trying to find highest shared chain block with peer %s with high hash %s", flow.peer, highHash)
highestSharedBlockHash, err := flow.findHighestSharedBlockHash(highHash)
if err != nil {
return err
}
log.Debugf("Found highest shared chain block %s with peer %s", highestSharedBlockHash, flow.peer)
return flow.downloadHeaders(highestSharedBlockHash, peerSelectedTipHash)
return flow.downloadHeaders(highestSharedBlockHash, highHash)
}
func (flow *handleIBDFlow) syncMissingBlockBodies() error {
hashes, err := flow.Domain().Consensus().GetMissingBlockBodyHashes(flow.peer.SelectedTipHash())
func (flow *handleRelayInvsFlow) findHighestSharedBlockHash(highHash *externalapi.DomainHash) (
lowHash *externalapi.DomainHash, err error) {
lowHash = flow.Config().ActiveNetParams.GenesisHash
currentHighHash := highHash
for {
err := flow.sendGetBlockLocator(lowHash, currentHighHash, 0)
if err != nil {
return nil, err
}
blockLocatorHashes, err := flow.receiveBlockLocator()
if err != nil {
return nil, err
}
// We check whether the locator's highest hash is in the local DAG.
// If it is, return it. If it isn't, we need to narrow our
// getBlockLocator request and try again.
locatorHighHash := blockLocatorHashes[0]
locatorHighHashInfo, err := flow.Domain().Consensus().GetBlockInfo(locatorHighHash)
if err != nil {
return nil, err
}
if locatorHighHashInfo.Exists {
return locatorHighHash, nil
}
lowHash, currentHighHash, err = flow.Domain().Consensus().FindNextBlockLocatorBoundaries(blockLocatorHashes)
if err != nil {
return nil, err
}
}
}
func (flow *handleRelayInvsFlow) downloadHeaders(highestSharedBlockHash *externalapi.DomainHash,
highHash *externalapi.DomainHash) error {
err := flow.sendRequestHeaders(highestSharedBlockHash, highHash)
if err != nil {
return err
}
for offset := 0; offset < len(hashes); offset += appmessage.MaxRequestIBDBlocksHashes {
blocksReceived := 0
for {
msgBlockHeader, doneIBD, err := flow.receiveHeader()
if err != nil {
return err
}
if doneIBD {
return nil
}
err = flow.processHeader(msgBlockHeader)
if err != nil {
return err
}
blocksReceived++
if blocksReceived%ibdBatchSize == 0 {
err = flow.outgoingRoute.Enqueue(appmessage.NewMsgRequestNextHeaders())
if err != nil {
return err
}
}
}
}
func (flow *handleRelayInvsFlow) sendRequestHeaders(highestSharedBlockHash *externalapi.DomainHash,
peerSelectedTipHash *externalapi.DomainHash) error {
msgGetBlockInvs := appmessage.NewMsgRequstHeaders(highestSharedBlockHash, peerSelectedTipHash)
return flow.outgoingRoute.Enqueue(msgGetBlockInvs)
}
func (flow *handleRelayInvsFlow) receiveHeader() (msgIBDBlock *appmessage.MsgBlockHeader, doneIBD bool, err error) {
message, err := flow.dequeueIncomingMessageAndSkipInvs(common.DefaultTimeout)
if err != nil {
return nil, false, err
}
switch message := message.(type) {
case *appmessage.MsgBlockHeader:
return message, false, nil
case *appmessage.MsgDoneHeaders:
return nil, true, nil
default:
return nil, false,
protocolerrors.Errorf(true, "received unexpected message type. "+
"expected: %s or %s, got: %s", appmessage.CmdHeader, appmessage.CmdDoneHeaders, message.Command())
}
}
func (flow *handleRelayInvsFlow) processHeader(msgBlockHeader *appmessage.MsgBlockHeader) error {
header := appmessage.BlockHeaderToDomainBlockHeader(msgBlockHeader)
block := &externalapi.DomainBlock{
Header: header,
Transactions: nil,
}
blockHash := consensushashing.BlockHash(block)
blockInfo, err := flow.Domain().Consensus().GetBlockInfo(blockHash)
if err != nil {
return err
}
if blockInfo.Exists {
log.Debugf("Block header %s is already in the DAG. Skipping...", blockHash)
return nil
}
err = flow.Domain().Consensus().ValidateAndInsertBlock(block)
if err != nil {
if !errors.As(err, &ruleerrors.RuleError{}) {
return errors.Wrapf(err, "failed to process header %s during IBD", blockHash)
}
log.Infof("Rejected block header %s from %s during IBD: %s", blockHash, flow.peer, err)
return protocolerrors.Wrapf(true, err, "got invalid block %s during IBD", blockHash)
}
return nil
}
func (flow *handleRelayInvsFlow) fetchMissingUTXOSet(ibdRootHash *externalapi.DomainHash) (bool, error) {
err := flow.outgoingRoute.Enqueue(appmessage.NewMsgRequestIBDRootUTXOSetAndBlock(ibdRootHash))
if err != nil {
return false, err
}
utxoSet, block, found, err := flow.receiveIBDRootUTXOSetAndBlock()
if err != nil {
return false, err
}
if !found {
return false, nil
}
err = flow.Domain().Consensus().ValidateAndInsertBlock(block)
if err != nil {
blockHash := consensushashing.BlockHash(block)
return false, protocolerrors.ConvertToBanningProtocolErrorIfRuleError(err, "got invalid block %s during IBD", blockHash)
}
err = flow.Domain().Consensus().SetPruningPointUTXOSet(utxoSet)
if err != nil {
return false, protocolerrors.ConvertToBanningProtocolErrorIfRuleError(err, "error with IBD root UTXO set")
}
return true, nil
}
func (flow *handleRelayInvsFlow) receiveIBDRootUTXOSetAndBlock() ([]byte, *externalapi.DomainBlock, bool, error) {
message, err := flow.dequeueIncomingMessageAndSkipInvs(common.DefaultTimeout)
if err != nil {
return nil, nil, false, err
}
switch message := message.(type) {
case *appmessage.MsgIBDRootUTXOSetAndBlock:
return message.UTXOSet,
appmessage.MsgBlockToDomainBlock(message.Block), true, nil
case *appmessage.MsgIBDRootNotFound:
return nil, nil, false, nil
default:
return nil, nil, false,
protocolerrors.Errorf(true, "received unexpected message type. "+
"expected: %s or %s, got: %s",
appmessage.CmdIBDRootUTXOSetAndBlock, appmessage.CmdIBDRootNotFound, message.Command(),
)
}
}
func (flow *handleRelayInvsFlow) syncMissingBlockBodies(highHash *externalapi.DomainHash) error {
hashes, err := flow.Domain().Consensus().GetMissingBlockBodyHashes(highHash)
if err != nil {
return err
}
for offset := 0; offset < len(hashes); offset += ibdBatchSize {
var hashesToRequest []*externalapi.DomainHash
if offset+appmessage.MaxRequestIBDBlocksHashes < len(hashes) {
hashesToRequest = hashes[offset : offset+appmessage.MaxRequestIBDBlocksHashes]
if offset+ibdBatchSize < len(hashes) {
hashesToRequest = hashes[offset : offset+ibdBatchSize]
} else {
hashesToRequest = hashes[offset:]
}
@ -131,7 +260,7 @@ func (flow *handleIBDFlow) syncMissingBlockBodies() error {
}
for _, expectedHash := range hashesToRequest {
message, err := flow.incomingRoute.DequeueWithTimeout(common.DefaultTimeout)
message, err := flow.dequeueIncomingMessageAndSkipInvs(common.DefaultTimeout)
if err != nil {
return err
}
@ -162,194 +291,17 @@ func (flow *handleIBDFlow) syncMissingBlockBodies() error {
return nil
}
func (flow *handleIBDFlow) fetchMissingUTXOSet(ibdRootHash *externalapi.DomainHash) (bool, error) {
err := flow.outgoingRoute.Enqueue(appmessage.NewMsgRequestIBDRootUTXOSetAndBlock(ibdRootHash))
if err != nil {
return false, err
}
utxoSet, block, found, err := flow.receiveIBDRootUTXOSetAndBlock()
if err != nil {
return false, err
}
if !found {
return false, nil
}
err = flow.Domain().Consensus().ValidateAndInsertBlock(block)
if err != nil {
blockHash := consensushashing.BlockHash(block)
return false, protocolerrors.ConvertToBanningProtocolErrorIfRuleError(err, "got invalid block %s during IBD", blockHash)
}
err = flow.Domain().Consensus().SetPruningPointUTXOSet(utxoSet)
if err != nil {
return false, protocolerrors.ConvertToBanningProtocolErrorIfRuleError(err, "error with IBD root UTXO set")
}
return true, nil
}
func (flow *handleIBDFlow) receiveIBDRootUTXOSetAndBlock() ([]byte, *externalapi.DomainBlock, bool, error) {
message, err := flow.incomingRoute.DequeueWithTimeout(common.DefaultTimeout)
if err != nil {
return nil, nil, false, err
}
switch message := message.(type) {
case *appmessage.MsgIBDRootUTXOSetAndBlock:
return message.UTXOSet,
appmessage.MsgBlockToDomainBlock(message.Block), true, nil
case *appmessage.MsgIBDRootNotFound:
return nil, nil, false, nil
default:
return nil, nil, false,
protocolerrors.Errorf(true, "received unexpected message type. "+
"expected: %s or %s, got: %s",
appmessage.CmdIBDRootUTXOSetAndBlock, appmessage.CmdIBDRootNotFound, message.Command(),
)
}
}
func (flow *handleIBDFlow) findHighestSharedBlockHash(peerSelectedTipHash *externalapi.DomainHash) (lowHash *externalapi.DomainHash,
err error) {
lowHash = flow.Config().ActiveNetParams.GenesisHash
highHash := peerSelectedTipHash
// 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 {
err := flow.sendGetBlockLocator(lowHash, highHash)
message, err := flow.incomingRoute.DequeueWithTimeout(timeout)
if err != nil {
return nil, err
}
blockLocatorHashes, err := flow.receiveBlockLocator()
if err != nil {
return nil, err
}
// We check whether the locator's highest hash is in the local DAG.
// If it is, return it. If it isn't, we need to narrow our
// getBlockLocator request and try again.
locatorHighHash := blockLocatorHashes[0]
locatorHighHashInfo, err := flow.Domain().Consensus().GetBlockInfo(locatorHighHash)
if err != nil {
return nil, err
}
if locatorHighHashInfo.Exists {
return locatorHighHash, nil
}
highHash, lowHash, err = flow.Domain().Consensus().FindNextBlockLocatorBoundaries(blockLocatorHashes)
if err != nil {
return nil, err
if _, ok := message.(*appmessage.MsgInvRelayBlock); !ok {
return message, nil
}
}
}
func (flow *handleIBDFlow) sendGetBlockLocator(lowHash *externalapi.DomainHash, highHash *externalapi.DomainHash) error {
msgGetBlockLocator := appmessage.NewMsgRequestBlockLocator(highHash, lowHash)
return flow.outgoingRoute.Enqueue(msgGetBlockLocator)
}
func (flow *handleIBDFlow) receiveBlockLocator() (blockLocatorHashes []*externalapi.DomainHash, err error) {
message, err := flow.incomingRoute.DequeueWithTimeout(common.DefaultTimeout)
if err != nil {
return nil, err
}
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
}
func (flow *handleIBDFlow) downloadHeaders(highestSharedBlockHash *externalapi.DomainHash,
peerSelectedTipHash *externalapi.DomainHash) error {
err := flow.sendRequestHeaders(highestSharedBlockHash, peerSelectedTipHash)
if err != nil {
return err
}
blocksReceived := 0
for {
msgBlockHeader, doneIBD, err := flow.receiveHeader()
if err != nil {
return err
}
if doneIBD {
return nil
}
err = flow.processHeader(msgBlockHeader)
if err != nil {
return err
}
blocksReceived++
if blocksReceived%ibdBatchSize == 0 {
err = flow.outgoingRoute.Enqueue(appmessage.NewMsgRequestNextHeaders())
if err != nil {
return err
}
}
}
}
func (flow *handleIBDFlow) sendRequestHeaders(highestSharedBlockHash *externalapi.DomainHash,
peerSelectedTipHash *externalapi.DomainHash) error {
msgGetBlockInvs := appmessage.NewMsgRequstHeaders(highestSharedBlockHash, peerSelectedTipHash)
return flow.outgoingRoute.Enqueue(msgGetBlockInvs)
}
func (flow *handleIBDFlow) receiveHeader() (msgIBDBlock *appmessage.MsgBlockHeader, doneIBD bool, err error) {
message, err := flow.incomingRoute.DequeueWithTimeout(common.DefaultTimeout)
if err != nil {
return nil, false, err
}
switch message := message.(type) {
case *appmessage.MsgBlockHeader:
return message, false, nil
case *appmessage.MsgDoneHeaders:
return nil, true, nil
default:
return nil, false,
protocolerrors.Errorf(true, "received unexpected message type. "+
"expected: %s or %s, got: %s", appmessage.CmdHeader, appmessage.CmdDoneHeaders, message.Command())
}
}
func (flow *handleIBDFlow) processHeader(msgBlockHeader *appmessage.MsgBlockHeader) error {
header := appmessage.BlockHeaderToDomainBlockHeader(msgBlockHeader)
block := &externalapi.DomainBlock{
Header: header,
Transactions: nil,
}
blockHash := consensushashing.BlockHash(block)
blockInfo, err := flow.Domain().Consensus().GetBlockInfo(blockHash)
if err != nil {
return err
}
if blockInfo.Exists {
log.Debugf("Block header %s is already in the DAG. Skipping...", blockHash)
return nil
}
err = flow.Domain().Consensus().ValidateAndInsertBlock(block)
if err != nil {
if !errors.As(err, &ruleerrors.RuleError{}) {
return errors.Wrapf(err, "failed to process header %s during IBD", blockHash)
}
log.Infof("Rejected block header %s from %s during IBD: %s", blockHash, flow.peer, err)
return protocolerrors.Wrapf(true, err, "got invalid block %s during IBD", blockHash)
}
return nil
}

View File

@ -24,7 +24,6 @@ type HandleHandshakeContext interface {
NetAdapter() *netadapter.NetAdapter
Domain() domain.Domain
AddressManager() *addressmanager.AddressManager
StartIBDIfRequired() error
AddToPeers(peer *peerpkg.Peer) error
HandleError(err error, flowName string, isStopping *uint32, errChan chan<- error)
}
@ -92,12 +91,6 @@ func HandleHandshake(context HandleHandshakeContext, netConnection *netadapter.N
if peerAddress != nil {
context.AddressManager().AddAddresses(peerAddress)
}
err = context.StartIBDIfRequired()
if err != nil {
return nil, err
}
return peer, nil
}

View File

@ -4,7 +4,6 @@ import (
"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/domain/consensus/utils/consensushashing"
"github.com/kaspanet/kaspad/infrastructure/logger"
"github.com/kaspanet/kaspad/infrastructure/network/netadapter/router"
"github.com/kaspanet/kaspad/version"
@ -52,17 +51,11 @@ func (flow *sendVersionFlow) start() error {
onEnd := logger.LogAndMeasureExecutionTime(log, "sendVersionFlow.start")
defer onEnd()
virtualSelectedParent, err := flow.Domain().Consensus().GetVirtualSelectedParent()
if err != nil {
return err
}
selectedTipHash := consensushashing.BlockHash(virtualSelectedParent)
subnetworkID := flow.Config().SubnetworkID
// Version message.
localAddress := flow.AddressManager().BestLocalAddress(flow.peer.Connection().NetAddress())
subnetworkID := flow.Config().SubnetworkID
msg := appmessage.NewMsgVersion(localAddress, flow.NetAdapter().ID(),
flow.Config().ActiveNetParams.Name, selectedTipHash, subnetworkID)
flow.Config().ActiveNetParams.Name, subnetworkID)
msg.AddUserAgent(userAgentName, userAgentVersion, flow.Config().UserAgentComments...)
// Advertise the services flag
@ -74,7 +67,7 @@ func (flow *sendVersionFlow) start() error {
// Advertise if inv messages for transactions are desired.
msg.DisableRelayTx = flow.Config().BlocksOnly
err = flow.outgoingRoute.Enqueue(msg)
err := flow.outgoingRoute.Enqueue(msg)
if err != nil {
return err
}

View File

@ -1,9 +0,0 @@
package ibd
import (
"github.com/kaspanet/kaspad/infrastructure/logger"
"github.com/kaspanet/kaspad/util/panics"
)
var log, _ = logger.Get(logger.SubsystemTags.IBDS)
var spawn = panics.GoroutineWrapperFunc(log)

View File

@ -1,66 +0,0 @@
package selectedtip
import (
"github.com/kaspanet/kaspad/app/appmessage"
"github.com/kaspanet/kaspad/domain"
"github.com/kaspanet/kaspad/domain/consensus/utils/consensushashing"
"github.com/kaspanet/kaspad/infrastructure/network/netadapter/router"
"github.com/pkg/errors"
)
// HandleRequestSelectedTipContext is the interface for the context needed for the HandleRequestSelectedTip flow.
type HandleRequestSelectedTipContext interface {
Domain() domain.Domain
}
type handleRequestSelectedTipFlow struct {
HandleRequestSelectedTipContext
incomingRoute, outgoingRoute *router.Route
}
// HandleRequestSelectedTip handles getSelectedTip messages
func HandleRequestSelectedTip(context HandleRequestSelectedTipContext, incomingRoute *router.Route, outgoingRoute *router.Route) error {
flow := &handleRequestSelectedTipFlow{
HandleRequestSelectedTipContext: context,
incomingRoute: incomingRoute,
outgoingRoute: outgoingRoute,
}
return flow.start()
}
func (flow *handleRequestSelectedTipFlow) start() error {
for {
err := flow.receiveGetSelectedTip()
if err != nil {
return err
}
err = flow.sendSelectedTipHash()
if err != nil {
return err
}
}
}
func (flow *handleRequestSelectedTipFlow) receiveGetSelectedTip() error {
message, err := flow.incomingRoute.Dequeue()
if err != nil {
return err
}
_, ok := message.(*appmessage.MsgRequestSelectedTip)
if !ok {
return errors.Errorf("received unexpected message type. "+
"expected: %s, got: %s", appmessage.CmdRequestSelectedTip, message.Command())
}
return nil
}
func (flow *handleRequestSelectedTipFlow) sendSelectedTipHash() error {
virtualSelectedParent, err := flow.Domain().Consensus().GetVirtualSelectedParent()
if err != nil {
return err
}
msgSelectedTip := appmessage.NewMsgSelectedTip(consensushashing.BlockHash(virtualSelectedParent))
return flow.outgoingRoute.Enqueue(msgSelectedTip)
}

View File

@ -1,78 +0,0 @@
package selectedtip
import (
"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/domain"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/kaspanet/kaspad/infrastructure/network/netadapter/router"
)
// RequestSelectedTipContext is the interface for the context needed for the RequestSelectedTip flow.
type RequestSelectedTipContext interface {
Domain() domain.Domain
StartIBDIfRequired() error
}
type requestSelectedTipFlow struct {
RequestSelectedTipContext
incomingRoute, outgoingRoute *router.Route
peer *peerpkg.Peer
}
// RequestSelectedTip waits for selected tip requests and handles them
func RequestSelectedTip(context RequestSelectedTipContext, incomingRoute *router.Route,
outgoingRoute *router.Route, peer *peerpkg.Peer) error {
flow := &requestSelectedTipFlow{
RequestSelectedTipContext: context,
incomingRoute: incomingRoute,
outgoingRoute: outgoingRoute,
peer: peer,
}
return flow.start()
}
func (flow *requestSelectedTipFlow) start() error {
for {
err := flow.runSelectedTipRequest()
if err != nil {
return err
}
}
}
func (flow *requestSelectedTipFlow) runSelectedTipRequest() error {
flow.peer.WaitForSelectedTipRequests()
defer flow.peer.FinishRequestingSelectedTip()
err := flow.requestSelectedTip()
if err != nil {
return err
}
peerSelectedTipHash, err := flow.receiveSelectedTip()
if err != nil {
return err
}
flow.peer.SetSelectedTipHash(peerSelectedTipHash)
return flow.StartIBDIfRequired()
}
func (flow *requestSelectedTipFlow) requestSelectedTip() error {
msgGetSelectedTip := appmessage.NewMsgRequestSelectedTip()
return flow.outgoingRoute.Enqueue(msgGetSelectedTip)
}
func (flow *requestSelectedTipFlow) receiveSelectedTip() (selectedTipHash *externalapi.DomainHash, err error) {
message, err := flow.incomingRoute.DequeueWithTimeout(common.DefaultTimeout)
if err != nil {
return nil, err
}
msgSelectedTip := message.(*appmessage.MsgSelectedTip)
return msgSelectedTip.SelectedTipHash, nil
}

View File

@ -1,4 +1,4 @@
package relaytransactions
package transactionrelay
import (
"github.com/kaspanet/kaspad/app/appmessage"

View File

@ -1,4 +1,4 @@
package relaytransactions
package transactionrelay
import (
"github.com/kaspanet/kaspad/app/appmessage"

View File

@ -1,4 +1,4 @@
package relaytransactions
package transactionrelay
import (
"sync"

View File

@ -37,12 +37,6 @@ func (m *Manager) Peers() []*peerpkg.Peer {
return m.context.Peers()
}
// IBDPeer returns the currently active IBD peer.
// Returns nil if we aren't currently in IBD
func (m *Manager) IBDPeer() *peerpkg.Peer {
return m.context.IBDPeer()
}
// AddTransaction adds transaction to the mempool and propagates it.
func (m *Manager) AddTransaction(tx *externalapi.DomainTransaction) error {
return m.context.AddTransaction(tx)

View File

@ -2,7 +2,6 @@ package peer
import (
"sync"
"sync/atomic"
"time"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
@ -22,9 +21,6 @@ const maxProtocolVersion = 1
type Peer struct {
connection *netadapter.NetConnection
selectedTipHashMtx sync.RWMutex
selectedTipHash *externalapi.DomainHash
userAgent string
services appmessage.ServiceFlag
advertisedProtocolVerion uint32 // protocol version advertised by remote
@ -39,21 +35,13 @@ 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
isSelectedTipRequested uint32
selectedTipRequestChan chan struct{}
lastSelectedTipRequest mstime.Time
ibdStartChan chan struct{}
}
// New returns a new Peer
func New(connection *netadapter.NetConnection) *Peer {
return &Peer{
connection: connection,
selectedTipRequestChan: make(chan struct{}),
ibdStartChan: make(chan struct{}),
connectionStarted: time.Now(),
connection: connection,
connectionStarted: time.Now(),
}
}
@ -62,20 +50,6 @@ func (p *Peer) Connection() *netadapter.NetConnection {
return p.connection
}
// SelectedTipHash returns the selected tip of the peer.
func (p *Peer) SelectedTipHash() *externalapi.DomainHash {
p.selectedTipHashMtx.RLock()
defer p.selectedTipHashMtx.RUnlock()
return p.selectedTipHash
}
// SetSelectedTipHash sets the selected tip of the peer.
func (p *Peer) SetSelectedTipHash(hash *externalapi.DomainHash) {
p.selectedTipHashMtx.Lock()
defer p.selectedTipHashMtx.Unlock()
p.selectedTipHash = hash
}
// SubnetworkID returns the subnetwork the peer is associated with.
// It is nil in full nodes.
func (p *Peer) SubnetworkID() *externalapi.DomainSubnetworkID {
@ -128,7 +102,6 @@ func (p *Peer) UpdateFieldsFromMsgVersion(msg *appmessage.MsgVersion) {
p.userAgent = msg.UserAgent
p.disableRelayTx = msg.DisableRelayTx
p.selectedTipHash = msg.SelectedTipHash
p.subnetworkID = msg.SubnetworkID
p.timeOffset = mstime.Since(msg.Timestamp)
@ -156,46 +129,6 @@ func (p *Peer) String() string {
return p.connection.String()
}
// RequestSelectedTipIfRequired notifies the peer that requesting
// a selected tip is required. This triggers the selected tip
// request flow.
func (p *Peer) RequestSelectedTipIfRequired() {
if atomic.SwapUint32(&p.isSelectedTipRequested, 1) != 0 {
return
}
const minGetSelectedTipInterval = time.Minute
if mstime.Since(p.lastSelectedTipRequest) < minGetSelectedTipInterval {
return
}
p.lastSelectedTipRequest = mstime.Now()
p.selectedTipRequestChan <- struct{}{}
}
// WaitForSelectedTipRequests blocks the current thread until
// a selected tip is requested from this peer
func (p *Peer) WaitForSelectedTipRequests() {
<-p.selectedTipRequestChan
}
// FinishRequestingSelectedTip finishes requesting the selected
// tip from this peer
func (p *Peer) FinishRequestingSelectedTip() {
atomic.StoreUint32(&p.isSelectedTipRequested, 0)
}
// StartIBD starts the IBD process for this peer
func (p *Peer) StartIBD() {
p.ibdStartChan <- struct{}{}
}
// WaitForIBDStart blocks the current thread until
// IBD start is requested from this peer
func (p *Peer) WaitForIBDStart() {
<-p.ibdStartChan
}
// Address returns the address associated with this connection
func (p *Peer) Address() string {
return p.connection.Address()

View File

@ -8,10 +8,8 @@ import (
"github.com/kaspanet/kaspad/app/protocol/flows/addressexchange"
"github.com/kaspanet/kaspad/app/protocol/flows/blockrelay"
"github.com/kaspanet/kaspad/app/protocol/flows/handshake"
"github.com/kaspanet/kaspad/app/protocol/flows/ibd"
"github.com/kaspanet/kaspad/app/protocol/flows/ibd/selectedtip"
"github.com/kaspanet/kaspad/app/protocol/flows/ping"
"github.com/kaspanet/kaspad/app/protocol/flows/relaytransactions"
"github.com/kaspanet/kaspad/app/protocol/flows/transactionrelay"
peerpkg "github.com/kaspanet/kaspad/app/protocol/peer"
"github.com/kaspanet/kaspad/app/protocol/protocolerrors"
"github.com/kaspanet/kaspad/infrastructure/network/addressmanager"
@ -107,7 +105,6 @@ func (m *Manager) registerFlows(router *routerpkg.Router, errChan chan error, is
flows = m.registerAddressFlows(router, isStopping, errChan)
flows = append(flows, m.registerBlockRelayFlows(router, isStopping, errChan)...)
flows = append(flows, m.registerPingFlows(router, isStopping, errChan)...)
flows = append(flows, m.registerIBDFlows(router, isStopping, errChan)...)
flows = append(flows, m.registerTransactionRelayFlow(router, isStopping, errChan)...)
flows = append(flows, m.registerRejectsFlow(router, isStopping, errChan)...)
@ -136,7 +133,10 @@ func (m *Manager) registerBlockRelayFlows(router *routerpkg.Router, isStopping *
outgoingRoute := router.OutgoingRoute()
return []*flow{
m.registerFlow("HandleRelayInvs", router, []appmessage.MessageCommand{appmessage.CmdInvRelayBlock, appmessage.CmdBlock}, isStopping, errChan,
m.registerFlow("HandleRelayInvs", router, []appmessage.MessageCommand{
appmessage.CmdInvRelayBlock, appmessage.CmdBlock, appmessage.CmdBlockLocator, appmessage.CmdIBDBlock,
appmessage.CmdDoneHeaders, appmessage.CmdIBDRootNotFound, appmessage.CmdIBDRootUTXOSetAndBlock,
appmessage.CmdHeader}, isStopping, errChan,
func(incomingRoute *routerpkg.Route, peer *peerpkg.Peer) error {
return blockrelay.HandleRelayInvs(m.context, incomingRoute,
outgoingRoute, peer)
@ -148,6 +148,34 @@ func (m *Manager) registerBlockRelayFlows(router *routerpkg.Router, isStopping *
return blockrelay.HandleRelayBlockRequests(m.context, incomingRoute, outgoingRoute, peer)
},
),
m.registerFlow("HandleRequestBlockLocator", router,
[]appmessage.MessageCommand{appmessage.CmdRequestBlockLocator}, isStopping, errChan,
func(incomingRoute *routerpkg.Route, peer *peerpkg.Peer) error {
return blockrelay.HandleRequestBlockLocator(m.context, incomingRoute, outgoingRoute)
},
),
m.registerFlow("HandleRequestHeaders", router,
[]appmessage.MessageCommand{appmessage.CmdRequestHeaders, appmessage.CmdRequestNextHeaders}, isStopping, errChan,
func(incomingRoute *routerpkg.Route, peer *peerpkg.Peer) error {
return blockrelay.HandleRequestHeaders(m.context, incomingRoute, outgoingRoute)
},
),
m.registerFlow("HandleRequestIBDRootUTXOSetAndBlock", router,
[]appmessage.MessageCommand{appmessage.CmdRequestIBDRootUTXOSetAndBlock}, isStopping, errChan,
func(incomingRoute *routerpkg.Route, peer *peerpkg.Peer) error {
return blockrelay.HandleRequestIBDRootUTXOSetAndBlock(m.context, incomingRoute, outgoingRoute)
},
),
m.registerFlow("HandleIBDBlockRequests", router,
[]appmessage.MessageCommand{appmessage.CmdRequestIBDBlocks}, isStopping, errChan,
func(incomingRoute *routerpkg.Route, peer *peerpkg.Peer) error {
return blockrelay.HandleIBDBlockRequests(m.context, incomingRoute, outgoingRoute)
},
),
}
}
@ -169,62 +197,6 @@ func (m *Manager) registerPingFlows(router *routerpkg.Router, isStopping *uint32
}
}
func (m *Manager) registerIBDFlows(router *routerpkg.Router, isStopping *uint32, errChan chan error) []*flow {
outgoingRoute := router.OutgoingRoute()
return []*flow{
m.registerFlow("HandleIBD", router, []appmessage.MessageCommand{appmessage.CmdBlockLocator, appmessage.CmdIBDBlock,
appmessage.CmdDoneHeaders, appmessage.CmdIBDRootNotFound, appmessage.CmdIBDRootUTXOSetAndBlock, appmessage.CmdHeader},
isStopping, errChan,
func(incomingRoute *routerpkg.Route, peer *peerpkg.Peer) error {
return ibd.HandleIBD(m.context, incomingRoute, outgoingRoute, peer)
},
),
m.registerFlow("RequestSelectedTip", router,
[]appmessage.MessageCommand{appmessage.CmdSelectedTip}, isStopping, errChan,
func(incomingRoute *routerpkg.Route, peer *peerpkg.Peer) error {
return selectedtip.RequestSelectedTip(m.context, incomingRoute, outgoingRoute, peer)
},
),
m.registerFlow("HandleRequestSelectedTip", router,
[]appmessage.MessageCommand{appmessage.CmdRequestSelectedTip}, isStopping, errChan,
func(incomingRoute *routerpkg.Route, peer *peerpkg.Peer) error {
return selectedtip.HandleRequestSelectedTip(m.context, incomingRoute, outgoingRoute)
},
),
m.registerFlow("HandleRequestBlockLocator", router,
[]appmessage.MessageCommand{appmessage.CmdRequestBlockLocator}, isStopping, errChan,
func(incomingRoute *routerpkg.Route, peer *peerpkg.Peer) error {
return ibd.HandleRequestBlockLocator(m.context, incomingRoute, outgoingRoute)
},
),
m.registerFlow("HandleRequestHeaders", router,
[]appmessage.MessageCommand{appmessage.CmdRequestHeaders, appmessage.CmdRequestNextHeaders}, isStopping, errChan,
func(incomingRoute *routerpkg.Route, peer *peerpkg.Peer) error {
return ibd.HandleRequestHeaders(m.context, incomingRoute, outgoingRoute)
},
),
m.registerFlow("HandleRequestIBDRootUTXOSetAndBlock", router,
[]appmessage.MessageCommand{appmessage.CmdRequestIBDRootUTXOSetAndBlock}, isStopping, errChan,
func(incomingRoute *routerpkg.Route, peer *peerpkg.Peer) error {
return ibd.HandleRequestIBDRootUTXOSetAndBlock(m.context, incomingRoute, outgoingRoute)
},
),
m.registerFlow("HandleIBDBlockRequests", router,
[]appmessage.MessageCommand{appmessage.CmdRequestIBDBlocks}, isStopping, errChan,
func(incomingRoute *routerpkg.Route, peer *peerpkg.Peer) error {
return ibd.HandleIBDBlockRequests(m.context, incomingRoute, outgoingRoute)
},
),
}
}
func (m *Manager) registerTransactionRelayFlow(router *routerpkg.Router, isStopping *uint32, errChan chan error) []*flow {
outgoingRoute := router.OutgoingRoute()
@ -232,13 +204,13 @@ func (m *Manager) registerTransactionRelayFlow(router *routerpkg.Router, isStopp
m.registerFlow("HandleRelayedTransactions", router,
[]appmessage.MessageCommand{appmessage.CmdInvTransaction, appmessage.CmdTx, appmessage.CmdTransactionNotFound}, isStopping, errChan,
func(incomingRoute *routerpkg.Route, peer *peerpkg.Peer) error {
return relaytransactions.HandleRelayedTransactions(m.context, incomingRoute, outgoingRoute)
return transactionrelay.HandleRelayedTransactions(m.context, incomingRoute, outgoingRoute)
},
),
m.registerFlow("HandleRequestTransactions", router,
[]appmessage.MessageCommand{appmessage.CmdRequestTransactions}, isStopping, errChan,
func(incomingRoute *routerpkg.Route, peer *peerpkg.Peer) error {
return relaytransactions.HandleRequestedTransactions(m.context, incomingRoute, outgoingRoute)
return transactionrelay.HandleRequestedTransactions(m.context, incomingRoute, outgoingRoute)
},
),
}

View File

@ -9,15 +9,12 @@ import (
// HandleGetConnectedPeerInfo handles the respectively named RPC command
func HandleGetConnectedPeerInfo(context *rpccontext.Context, _ *router.Router, _ appmessage.Message) (appmessage.Message, error) {
peers := context.ProtocolManager.Peers()
ibdPeer := context.ProtocolManager.IBDPeer()
infos := make([]*appmessage.GetConnectedPeerInfoMessage, 0, len(peers))
for _, peer := range peers {
info := &appmessage.GetConnectedPeerInfoMessage{
ID: peer.ID().String(),
Address: peer.Address(),
LastPingDuration: peer.LastPingDuration().Milliseconds(),
SelectedTipHash: peer.SelectedTipHash().String(),
IsSyncNode: peer == ibdPeer,
IsOutbound: peer.IsOutbound(),
TimeOffset: peer.TimeOffset().Milliseconds(),
UserAgent: peer.UserAgent(),

View File

@ -199,11 +199,11 @@ func (s *consensus) GetVirtualSelectedParent() (*externalapi.DomainBlock, error)
return s.blockStore.Block(s.databaseContext, virtualGHOSTDAGData.SelectedParent())
}
func (s *consensus) CreateBlockLocator(lowHash, highHash *externalapi.DomainHash) (externalapi.BlockLocator, error) {
func (s *consensus) CreateBlockLocator(lowHash, highHash *externalapi.DomainHash, limit uint32) (externalapi.BlockLocator, error) {
s.lock.Lock()
defer s.lock.Unlock()
return s.syncManager.CreateBlockLocator(lowHash, highHash)
return s.syncManager.CreateBlockLocator(lowHash, highHash, limit)
}
func (s *consensus) FindNextBlockLocatorBoundaries(blockLocator externalapi.BlockLocator) (lowHash, highHash *externalapi.DomainHash, err error) {

View File

@ -248,7 +248,7 @@ func (f *factory) NewConsensus(dagParams *dagconfig.Params, db infrastructuredat
)
blockProcessor := blockprocessor.New(
dagParams,
genesisHash,
dbManager,
consensusStateManager,
pruningManager,

View File

@ -15,7 +15,7 @@ type Consensus interface {
GetPruningPointUTXOSet(expectedPruningPointHash *DomainHash) ([]byte, error)
SetPruningPointUTXOSet(serializedUTXOSet []byte) error
GetVirtualSelectedParent() (*DomainBlock, error)
CreateBlockLocator(lowHash, highHash *DomainHash) (BlockLocator, error)
CreateBlockLocator(lowHash, highHash *DomainHash, limit uint32) (BlockLocator, error)
FindNextBlockLocatorBoundaries(blockLocator BlockLocator) (lowHash, highHash *DomainHash, err error)
GetSyncInfo() (*SyncInfo, error)
}

View File

@ -5,11 +5,10 @@ import "fmt"
// Each of the following represent one of the possible sync
// states of the consensus
const (
SyncStateRelay SyncState = iota
SyncStateMissingGenesis
SyncStateHeadersFirst
SyncStateMissingUTXOSet
SyncStateMissingBlockBodies
SyncStateSynced SyncState = iota
SyncStateAwaitingGenesis
SyncStateAwaitingUTXOSet
SyncStateAwaitingBlockBodies
)
// SyncState represents the current sync state of the consensus
@ -17,16 +16,14 @@ type SyncState uint8
func (s SyncState) String() string {
switch s {
case SyncStateRelay:
return "SyncStateRelay"
case SyncStateMissingGenesis:
return "SyncStateMissingGenesis"
case SyncStateHeadersFirst:
return "SyncStateHeadersFirst"
case SyncStateMissingUTXOSet:
return "SyncStateMissingUTXOSet"
case SyncStateMissingBlockBodies:
return "SyncStateMissingBlockBodies"
case SyncStateSynced:
return "SyncStateSynced"
case SyncStateAwaitingGenesis:
return "SyncStateAwaitingGenesis"
case SyncStateAwaitingUTXOSet:
return "SyncStateAwaitingUTXOSet"
case SyncStateAwaitingBlockBodies:
return "SyncStateAwaitingBlockBodies"
}
return fmt.Sprintf("<unknown state (%d)>", s)

View File

@ -6,7 +6,7 @@ import "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
type SyncManager interface {
GetHashesBetween(lowHash, highHash *externalapi.DomainHash) ([]*externalapi.DomainHash, error)
GetMissingBlockBodyHashes(highHash *externalapi.DomainHash) ([]*externalapi.DomainHash, error)
CreateBlockLocator(lowHash, highHash *externalapi.DomainHash) (externalapi.BlockLocator, error)
CreateBlockLocator(lowHash, highHash *externalapi.DomainHash, limit uint32) (externalapi.BlockLocator, error)
FindNextBlockLocatorBoundaries(blockLocator externalapi.BlockLocator) (lowHash, highHash *externalapi.DomainHash, err error)
IsBlockInHeaderPruningPointFuture(blockHash *externalapi.DomainHash) (bool, error)
GetSyncInfo() (*externalapi.SyncInfo, error)

View File

@ -3,14 +3,13 @@ package blockprocessor
import (
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/kaspanet/kaspad/domain/dagconfig"
"github.com/kaspanet/kaspad/infrastructure/logger"
)
// blockProcessor is responsible for processing incoming blocks
// and creating blocks from the current state
type blockProcessor struct {
dagParams *dagconfig.Params
genesisHash *externalapi.DomainHash
databaseContext model.DBManager
consensusStateManager model.ConsensusStateManager
@ -43,7 +42,7 @@ type blockProcessor struct {
// New instantiates a new BlockProcessor
func New(
dagParams *dagconfig.Params,
genesisHash *externalapi.DomainHash,
databaseContext model.DBManager,
consensusStateManager model.ConsensusStateManager,
pruningManager model.PruningManager,
@ -71,7 +70,7 @@ func New(
headerTipsStore model.HeaderTipsStore) model.BlockProcessor {
return &blockProcessor{
dagParams: dagParams,
genesisHash: genesisHash,
databaseContext: databaseContext,
pruningManager: pruningManager,
blockValidator: blockValidator,

View File

@ -11,88 +11,51 @@ import (
"github.com/pkg/errors"
)
type insertMode uint8
const (
insertModeGenesis insertMode = iota
insertModeHeader
insertModeBlockBody
insertModeBlock
)
func (bp *blockProcessor) validateAndInsertBlock(block *externalapi.DomainBlock) error {
hash := consensushashing.HeaderHash(block.Header)
log.Debugf("Validating block %s", hash)
blockHash := consensushashing.HeaderHash(block.Header)
log.Debugf("Validating block %s", blockHash)
syncInfo, err := bp.syncManager.GetSyncInfo()
insertMode, err := bp.validateAgainstSyncStateAndResolveInsertMode(block)
if err != nil {
return err
}
if isHeaderOnlyBlock(block) && syncInfo.State != externalapi.SyncStateRelay {
syncInfo.State = externalapi.SyncStateHeadersFirst
}
if syncInfo.State == externalapi.SyncStateMissingUTXOSet {
if isHeaderOnlyBlock(block) {
// Allow processing headers while in state SyncStateMissingUTXOSet
syncInfo.State = externalapi.SyncStateHeadersFirst
} else {
headerTipsPruningPoint, err := bp.consensusStateManager.HeaderTipsPruningPoint()
if err != nil {
return err
}
if *hash != *headerTipsPruningPoint {
return errors.Errorf("cannot insert blocks other than the header pruning point "+
"while in %s mode", syncInfo.State)
}
syncInfo.State = externalapi.SyncStateMissingBlockBodies
}
}
if syncInfo.State == externalapi.SyncStateHeadersFirst && !isHeaderOnlyBlock(block) {
syncInfo.State = externalapi.SyncStateRelay
log.Warnf("block %s contains transactions while validating in header only mode", hash)
}
if syncInfo.State == externalapi.SyncStateMissingBlockBodies {
headerTips, err := bp.headerTipsStore.Tips(bp.databaseContext)
if err != nil {
return err
}
selectedHeaderTip, err := bp.ghostdagManager.ChooseSelectedParent(headerTips...)
if err != nil {
return err
}
if *selectedHeaderTip == *hash {
syncInfo.State = externalapi.SyncStateRelay
}
}
err = bp.checkBlockStatus(hash, syncInfo)
err = bp.checkBlockStatus(blockHash, insertMode)
if err != nil {
return err
}
err = bp.validateBlock(block, syncInfo)
err = bp.validateBlock(block, insertMode)
if err != nil {
bp.discardAllChanges()
return err
}
hasHeader, err := bp.hasHeader(hash)
hasHeader, err := bp.hasHeader(blockHash)
if err != nil {
return err
}
if !hasHeader {
if syncInfo.State == externalapi.SyncStateMissingBlockBodies {
return errors.Wrapf(ruleerrors.ErrMissingBlockHeaderInIBD, "no block header is stored for block %s. "+
"Every block we get during %s mode should have a pre-stored header", syncInfo.State, hash)
}
err = bp.reachabilityManager.AddBlock(hash)
err = bp.reachabilityManager.AddBlock(blockHash)
if err != nil {
return err
}
}
if syncInfo.State == externalapi.SyncStateHeadersFirst {
bp.blockStatusStore.Stage(hash, externalapi.StatusHeaderOnly)
if insertMode == insertModeHeader {
bp.blockStatusStore.Stage(blockHash, externalapi.StatusHeaderOnly)
} else {
bp.blockStatusStore.Stage(hash, externalapi.StatusUTXOPendingVerification)
bp.blockStatusStore.Stage(blockHash, externalapi.StatusUTXOPendingVerification)
}
// Block validations passed, save whatever DAG data was
@ -102,13 +65,8 @@ func (bp *blockProcessor) validateAndInsertBlock(block *externalapi.DomainBlock)
return err
}
hasTips, err := bp.headerTipsStore.HasTips(bp.databaseContext)
if err != nil {
return err
}
var oldHeadersSelectedTip *externalapi.DomainHash
if hasTips {
if insertMode != insertModeGenesis {
var err error
oldHeadersSelectedTip, err = bp.headerTipsManager.SelectedTip()
if err != nil {
@ -116,14 +74,14 @@ func (bp *blockProcessor) validateAndInsertBlock(block *externalapi.DomainBlock)
}
}
if syncInfo.State == externalapi.SyncStateHeadersFirst {
err = bp.headerTipsManager.AddHeaderTip(hash)
if insertMode == insertModeHeader {
err = bp.headerTipsManager.AddHeaderTip(blockHash)
if err != nil {
return err
}
} else if syncInfo.State == externalapi.SyncStateRelay || syncInfo.State == externalapi.SyncStateMissingGenesis {
} else if insertMode == insertModeBlock || insertMode == insertModeGenesis {
// Attempt to add the block to the virtual
err = bp.consensusStateManager.AddBlockToVirtual(hash)
err = bp.consensusStateManager.AddBlockToVirtual(blockHash)
if err != nil {
return err
}
@ -135,14 +93,14 @@ func (bp *blockProcessor) validateAndInsertBlock(block *externalapi.DomainBlock)
bp.headerTipsStore.Stage(tips)
}
if syncInfo.State != externalapi.SyncStateMissingGenesis {
err = bp.updateReachabilityReindexRoot(oldHeadersSelectedTip)
if insertMode != insertModeGenesis {
err := bp.updateReachabilityReindexRoot(oldHeadersSelectedTip)
if err != nil {
return err
}
}
if syncInfo.State == externalapi.SyncStateRelay {
if insertMode == insertModeBlock {
// Trigger pruning, which will check if the pruning point changed and delete the data if it did.
err = bp.pruningManager.FindNextPruningPoint()
if err != nil {
@ -155,7 +113,7 @@ func (bp *blockProcessor) validateAndInsertBlock(block *externalapi.DomainBlock)
return err
}
log.Debugf("Block %s validated and inserted", hash)
log.Debugf("Block %s validated and inserted", blockHash)
var logClosureErr error
log.Debugf("%s", logger.NewLogClosure(func() string {
@ -179,6 +137,62 @@ func (bp *blockProcessor) validateAndInsertBlock(block *externalapi.DomainBlock)
return nil
}
func (bp *blockProcessor) validateAgainstSyncStateAndResolveInsertMode(block *externalapi.DomainBlock) (insertMode, error) {
syncInfo, err := bp.syncManager.GetSyncInfo()
if err != nil {
return 0, err
}
syncState := syncInfo.State
isHeaderOnlyBlock := isHeaderOnlyBlock(block)
blockHash := consensushashing.HeaderHash(block.Header)
if syncState == externalapi.SyncStateAwaitingGenesis {
if isHeaderOnlyBlock {
return 0, errors.Errorf("Got a header-only block while awaiting genesis")
}
if *blockHash != *bp.genesisHash {
return 0, errors.Errorf("Received a non-genesis block while awaiting genesis")
}
return insertModeGenesis, nil
}
if isHeaderOnlyBlock {
return insertModeHeader, nil
}
if syncState == externalapi.SyncStateAwaitingUTXOSet {
headerTipsPruningPoint, err := bp.consensusStateManager.HeaderTipsPruningPoint()
if err != nil {
return 0, err
}
if *blockHash != *headerTipsPruningPoint {
return 0, errors.Errorf("cannot insert blocks other than the header pruning point " +
"while awaiting the UTXO set")
}
return insertModeBlock, nil
}
if syncState == externalapi.SyncStateAwaitingBlockBodies {
headerTips, err := bp.headerTipsStore.Tips(bp.databaseContext)
if err != nil {
return 0, err
}
selectedHeaderTip, err := bp.ghostdagManager.ChooseSelectedParent(headerTips...)
if err != nil {
return 0, err
}
if *selectedHeaderTip != *blockHash {
return insertModeBlockBody, nil
}
}
return insertModeBlock, nil
}
func isHeaderOnlyBlock(block *externalapi.DomainBlock) bool {
return len(block.Transactions) == 0
}
func (bp *blockProcessor) updateReachabilityReindexRoot(oldHeadersSelectedTip *externalapi.DomainHash) error {
headersSelectedTip, err := bp.headerTipsManager.SelectedTip()
if err != nil {
@ -192,16 +206,11 @@ func (bp *blockProcessor) updateReachabilityReindexRoot(oldHeadersSelectedTip *e
return bp.reachabilityManager.UpdateReindexRoot(headersSelectedTip)
}
func isHeaderOnlyBlock(block *externalapi.DomainBlock) bool {
return len(block.Transactions) == 0
}
func (bp *blockProcessor) checkBlockStatus(hash *externalapi.DomainHash, mode *externalapi.SyncInfo) error {
func (bp *blockProcessor) checkBlockStatus(hash *externalapi.DomainHash, mode insertMode) error {
exists, err := bp.blockStatusStore.Exists(bp.databaseContext, hash)
if err != nil {
return err
}
if !exists {
return nil
}
@ -215,14 +224,20 @@ func (bp *blockProcessor) checkBlockStatus(hash *externalapi.DomainHash, mode *e
return errors.Wrapf(ruleerrors.ErrKnownInvalid, "block %s is a known invalid block", hash)
}
if mode.State == externalapi.SyncStateHeadersFirst || status != externalapi.StatusHeaderOnly {
isBlockBodyAfterBlockHeader := mode != insertModeHeader && status == externalapi.StatusHeaderOnly
if !isBlockBodyAfterBlockHeader {
return errors.Wrapf(ruleerrors.ErrDuplicateBlock, "block %s already exists", hash)
}
isDuplicateHeader := mode == insertModeHeader && status == externalapi.StatusHeaderOnly
if isDuplicateHeader {
return errors.Wrapf(ruleerrors.ErrDuplicateBlock, "block %s already exists", hash)
}
return nil
}
func (bp *blockProcessor) validateBlock(block *externalapi.DomainBlock, mode *externalapi.SyncInfo) error {
func (bp *blockProcessor) validateBlock(block *externalapi.DomainBlock, mode insertMode) error {
blockHash := consensushashing.HeaderHash(block.Header)
hasHeader, err := bp.hasHeader(blockHash)
if err != nil {
@ -241,7 +256,7 @@ func (bp *blockProcessor) validateBlock(block *externalapi.DomainBlock, mode *ex
return err
}
err = bp.blockValidator.ValidatePruningPointViolationAndProofOfWorkAndDifficulty(blockHash)
err = bp.validatePruningPointViolationAndProofOfWorkAndDifficulty(block, mode)
if err != nil {
return err
}
@ -283,12 +298,19 @@ func (bp *blockProcessor) validatePreProofOfWork(block *externalapi.DomainBlock)
return nil
}
func (bp *blockProcessor) validatePostProofOfWork(block *externalapi.DomainBlock, mode *externalapi.SyncInfo) error {
func (bp *blockProcessor) validatePruningPointViolationAndProofOfWorkAndDifficulty(block *externalapi.DomainBlock, mode insertMode) error {
blockHash := consensushashing.HeaderHash(block.Header)
if mode != insertModeHeader {
// We stage the block here since we need it for parent validation
bp.blockStore.Stage(blockHash, block)
}
return bp.blockValidator.ValidatePruningPointViolationAndProofOfWorkAndDifficulty(blockHash)
}
func (bp *blockProcessor) validatePostProofOfWork(block *externalapi.DomainBlock, mode insertMode) error {
blockHash := consensushashing.BlockHash(block)
if mode.State != externalapi.SyncStateHeadersFirst {
bp.blockStore.Stage(blockHash, block)
if mode != insertModeHeader {
err := bp.blockValidator.ValidateBodyInIsolation(blockHash)
if err != nil {
return err

View File

@ -15,7 +15,7 @@ func (v *blockValidator) ValidatePruningPointViolationAndProofOfWorkAndDifficult
return err
}
err = v.checkParentsExist(header)
err = v.checkParentsExist(blockHash, header)
if err != nil {
return err
}
@ -104,17 +104,32 @@ func (v *blockValidator) checkProofOfWork(header *externalapi.DomainBlockHeader)
return nil
}
func (v *blockValidator) checkParentsExist(header *externalapi.DomainBlockHeader) error {
func (v *blockValidator) checkParentsExist(blockHash *externalapi.DomainHash, header *externalapi.DomainBlockHeader) error {
missingParentHashes := []*externalapi.DomainHash{}
isFullBlock, err := v.blockStore.HasBlock(v.databaseContext, blockHash)
if err != nil {
return err
}
for _, parent := range header.ParentHashes {
exists, err := v.blockHeaderStore.HasBlockHeader(v.databaseContext, parent)
parentHeaderExists, err := v.blockHeaderStore.HasBlockHeader(v.databaseContext, parent)
if err != nil {
return err
}
if !exists {
if !parentHeaderExists {
missingParentHashes = append(missingParentHashes, parent)
continue
}
if isFullBlock {
parentStatus, err := v.blockStatusStore.Get(v.databaseContext, parent)
if err != nil {
return err
}
if parentStatus == externalapi.StatusHeaderOnly {
missingParentHashes = append(missingParentHashes, parent)
}
}
}

View File

@ -103,7 +103,6 @@ func isHashInSlice(hash *externalapi.DomainHash, hashes []*externalapi.DomainHas
}
func (dtm *dagTopologyManager) SetParents(blockHash *externalapi.DomainHash, parentHashes []*externalapi.DomainHash) error {
hasRelations, err := dtm.blockRelationStore.Has(dtm.databaseContext, blockHash)
if err != nil {
return err
@ -128,7 +127,6 @@ func (dtm *dagTopologyManager) SetParents(blockHash *externalapi.DomainHash, par
if *parentChild == *blockHash {
parentRelations.Children = append(parentRelations.Children[:i], parentRelations.Children[i+1:]...)
dtm.blockRelationStore.StageBlockRelation(currentParent, parentRelations)
break
}
}

View File

@ -13,7 +13,7 @@ type selectedChildIterator struct {
current *externalapi.DomainHash
}
func (s selectedChildIterator) Next() bool {
func (s *selectedChildIterator) Next() bool {
children, err := s.dagTopologyManager.Children(s.current)
if err != nil {
panic(err)
@ -21,7 +21,7 @@ func (s selectedChildIterator) Next() bool {
for _, child := range children {
if *child == *model.VirtualBlockHash {
break
continue
}
isChildInSelectedParentChainOfHighHash, err := s.dagTopologyManager.IsInSelectedParentChainOf(child, s.highHash)

View File

@ -110,6 +110,7 @@ func (sm *syncManager) missingBlockBodyHashes(highHash *externalapi.DomainHash)
}
lowHash := headerTipsPruningPoint
foundHeaderOnlyBlock := false
for selectedChildIterator.Next() {
selectedChild := selectedChildIterator.Get()
selectedChildStatus, err := sm.blockStatusStore.Get(sm.databaseContext, selectedChild)
@ -117,10 +118,17 @@ func (sm *syncManager) missingBlockBodyHashes(highHash *externalapi.DomainHash)
return nil, err
}
if selectedChildStatus != externalapi.StatusHeaderOnly {
lowHash = selectedChild
if selectedChildStatus == externalapi.StatusHeaderOnly {
foundHeaderOnlyBlock = true
break
}
lowHash = selectedChild
}
if !foundHeaderOnlyBlock {
// TODO: Once block children are fixed, this error
// should be returned instead of simply logged
log.Errorf("no header-only blocks between %s and %s",
lowHash, highHash)
}
hashesBetween, err := sm.antiPastHashesBetween(lowHash, highHash)
@ -128,44 +136,17 @@ func (sm *syncManager) missingBlockBodyHashes(highHash *externalapi.DomainHash)
return nil, err
}
lowHashAnticone, err := sm.dagTraversalManager.AnticoneFromContext(highHash, lowHash)
if err != nil {
return nil, err
}
blockToRemoveFromHashesBetween := hashset.New()
for _, blockHash := range lowHashAnticone {
isHeaderOnlyBlock, err := sm.isHeaderOnlyBlock(blockHash)
missingBlocks := make([]*externalapi.DomainHash, 0, len(hashesBetween))
for _, blockHash := range hashesBetween {
blockStatus, err := sm.blockStatusStore.Get(sm.databaseContext, blockHash)
if err != nil {
return nil, err
}
if !isHeaderOnlyBlock {
blockToRemoveFromHashesBetween.Add(blockHash)
if blockStatus == externalapi.StatusHeaderOnly {
missingBlocks = append(missingBlocks, blockHash)
}
}
missingBlocks := make([]*externalapi.DomainHash, 0, len(hashesBetween)-len(lowHashAnticone))
for i, blockHash := range hashesBetween {
// If blockToRemoveFromHashesBetween is empty, no more blocks should be
// filtered, so we can copy the rest of hashesBetween into missingBlocks
if blockToRemoveFromHashesBetween.Length() == 0 {
missingBlocks = append(missingBlocks, hashesBetween[i:]...)
break
}
if blockToRemoveFromHashesBetween.Contains(blockHash) {
blockToRemoveFromHashesBetween.Remove(blockHash)
continue
}
missingBlocks = append(missingBlocks, blockHash)
}
if blockToRemoveFromHashesBetween.Length() != 0 {
return nil, errors.Errorf("blockToRemoveFromHashesBetween.Length() is expected to be 0")
}
return missingBlocks, nil
}

View File

@ -7,7 +7,7 @@ import (
// createBlockLocator creates a block locator for the passed high and low hashes.
// See the BlockLocator type comments for more details.
func (sm *syncManager) createBlockLocator(lowHash, highHash *externalapi.DomainHash) (externalapi.BlockLocator, error) {
func (sm *syncManager) createBlockLocator(lowHash, highHash *externalapi.DomainHash, limit uint32) (externalapi.BlockLocator, error) {
// We use the selected parent of the high block, so that the
// block locator won't contain it.
highBlockGHOSTDAGData, err := sm.ghostdagDataStore.Get(sm.databaseContext, highHash)
@ -28,6 +28,11 @@ func (sm *syncManager) createBlockLocator(lowHash, highHash *externalapi.DomainH
for currentHash != nil {
locator = append(locator, currentHash)
// Stop if we've reached the limit (if it's set)
if limit > 0 && uint32(len(locator)) == limit {
break
}
currentBlockGHOSTDAGData, err := sm.ghostdagDataStore.Get(sm.databaseContext, currentHash)
if err != nil {
return nil, err
@ -36,7 +41,11 @@ func (sm *syncManager) createBlockLocator(lowHash, highHash *externalapi.DomainH
// Nothing more to add once the low node has been added.
if currentBlockBlueScore <= lowBlockBlueScore {
if *currentHash != *lowHash {
isCurrentHashInSelectedParentChainOfLowHash, err := sm.dagTopologyManager.IsInSelectedParentChainOf(currentHash, lowHash)
if err != nil {
return nil, err
}
if !isCurrentHashInSelectedParentChainOfLowHash {
return nil, errors.Errorf("highHash and lowHash are " +
"not in the same selected parent chain.")
}
@ -67,24 +76,24 @@ func (sm *syncManager) createBlockLocator(lowHash, highHash *externalapi.DomainH
// hash and the highest known block locator hash. This is used to create the
// next block locator to find the highest shared known chain block with a
// remote kaspad.
func (sm *syncManager) findNextBlockLocatorBoundaries(blockLocator externalapi.BlockLocator) (lowHash, highHash *externalapi.DomainHash, err error) {
func (sm *syncManager) findNextBlockLocatorBoundaries(blockLocator externalapi.BlockLocator) (
lowHash, highHash *externalapi.DomainHash, err error) {
// Find the most recent locator block hash in the DAG. In case none of
// the hashes in the locator are in the DAG, fall back to the genesis block.
lowHash = sm.genesisBlockHash
nextBlockLocatorIndex := int64(len(blockLocator) - 1)
for i, hash := range blockLocator {
highestKnownHash := sm.genesisBlockHash
lowestUnknownHash := blockLocator[len(blockLocator)-1]
for _, hash := range blockLocator {
exists, err := sm.blockStatusStore.Exists(sm.databaseContext, hash)
if err != nil {
return nil, nil, err
}
if exists {
lowHash = hash
nextBlockLocatorIndex = int64(i) - 1
if !exists {
lowestUnknownHash = hash
} else {
highestKnownHash = hash
break
}
}
if nextBlockLocatorIndex < 0 {
return nil, lowHash, nil
}
return blockLocator[nextBlockLocatorIndex], lowHash, nil
return highestKnownHash, lowestUnknownHash, nil
}

View File

@ -1,9 +1,7 @@
package syncmanager
import (
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/kaspanet/kaspad/util/mstime"
)
// areHeaderTipsSyncedMaxTimeDifference is the number of blocks from
@ -18,7 +16,7 @@ func (sm *syncManager) syncInfo() (*externalapi.SyncInfo, error) {
}
var ibdRootUTXOBlockHash *externalapi.DomainHash
if syncState == externalapi.SyncStateMissingUTXOSet {
if syncState == externalapi.SyncStateAwaitingUTXOSet {
ibdRootUTXOBlockHash, err = sm.consensusStateManager.HeaderTipsPruningPoint()
if err != nil {
return nil, err
@ -42,27 +40,19 @@ func (sm *syncManager) resolveSyncState() (externalapi.SyncState, error) {
return 0, err
}
if !hasTips {
return externalapi.SyncStateMissingGenesis, nil
return externalapi.SyncStateAwaitingGenesis, nil
}
headerVirtualSelectedParentHash, err := sm.headerVirtualSelectedParentHash()
if err != nil {
return 0, err
}
isSynced, err := sm.areHeaderTipsSynced(headerVirtualSelectedParentHash)
headerVirtualSelectedParentStatus, err := sm.blockStatusStore.Get(sm.databaseContext, headerVirtualSelectedParentHash)
if err != nil {
return 0, err
}
if !isSynced {
return externalapi.SyncStateHeadersFirst, nil
}
virtualSelectedParentHash, err := sm.virtualSelectedParentHash()
if err != nil {
return 0, err
}
if *virtualSelectedParentHash == *headerVirtualSelectedParentHash {
return externalapi.SyncStateRelay, nil
if headerVirtualSelectedParentStatus != externalapi.StatusHeaderOnly {
return externalapi.SyncStateSynced, nil
}
// Once the header tips are synced, check the status of
@ -80,18 +70,10 @@ func (sm *syncManager) resolveSyncState() (externalapi.SyncState, error) {
return 0, err
}
if headerTipsPruningPointStatus != externalapi.StatusValid {
return externalapi.SyncStateMissingUTXOSet, nil
return externalapi.SyncStateAwaitingUTXOSet, nil
}
return externalapi.SyncStateMissingBlockBodies, nil
}
func (sm *syncManager) virtualSelectedParentHash() (*externalapi.DomainHash, error) {
virtualGHOSTDAGData, err := sm.ghostdagDataStore.Get(sm.databaseContext, model.VirtualBlockHash)
if err != nil {
return nil, err
}
return virtualGHOSTDAGData.SelectedParent(), nil
return externalapi.SyncStateAwaitingBlockBodies, nil
}
func (sm *syncManager) headerVirtualSelectedParentHash() (*externalapi.DomainHash, error) {
@ -102,21 +84,6 @@ func (sm *syncManager) headerVirtualSelectedParentHash() (*externalapi.DomainHas
return sm.ghostdagManager.ChooseSelectedParent(headerTips...)
}
func (sm *syncManager) areHeaderTipsSynced(headerVirtualSelectedParentHash *externalapi.DomainHash) (bool, error) {
virtualSelectedParentHeader, err := sm.blockHeaderStore.BlockHeader(sm.databaseContext, headerVirtualSelectedParentHash)
if err != nil {
return false, err
}
virtualSelectedParentTimeInMilliseconds := virtualSelectedParentHeader.TimeInMilliseconds
nowInMilliseconds := mstime.Now().UnixMilliseconds()
timeDifference := nowInMilliseconds - virtualSelectedParentTimeInMilliseconds
maxTimeDifference := areHeaderTipsSyncedMaxTimeDifference * sm.targetTimePerBlock
return timeDifference <= maxTimeDifference, nil
}
func (sm *syncManager) getHeaderCount() uint64 {
return sm.blockHeaderStore.Count()
}

View File

@ -78,11 +78,11 @@ func (sm *syncManager) IsBlockInHeaderPruningPointFuture(blockHash *externalapi.
return sm.isBlockInHeaderPruningPointFuture(blockHash)
}
func (sm *syncManager) CreateBlockLocator(lowHash, highHash *externalapi.DomainHash) (externalapi.BlockLocator, error) {
func (sm *syncManager) CreateBlockLocator(lowHash, highHash *externalapi.DomainHash, limit uint32) (externalapi.BlockLocator, error) {
onEnd := logger.LogAndMeasureExecutionTime(log, "CreateBlockLocator")
defer onEnd()
return sm.createBlockLocator(lowHash, highHash)
return sm.createBlockLocator(lowHash, highHash, limit)
}
func (sm *syncManager) FindNextBlockLocatorBoundaries(blockLocator externalapi.BlockLocator) (lowHash, highHash *externalapi.DomainHash, err error) {

View File

@ -228,8 +228,6 @@ var (
ErrSubnetworksDisabled = newRuleError("ErrSubnetworksDisabled")
ErrBadPruningPointUTXOSet = newRuleError("ErrBadPruningPointUTXOSet")
ErrMissingBlockHeaderInIBD = newRuleError("ErrMissingBlockHeaderInIBD")
ErrMalformedUTXO = newRuleError("ErrMalformedUTXO")
ErrWrongPruningPointHash = newRuleError("ErrWrongPruningPointHash")

View File

@ -52,7 +52,6 @@ var (
ntarLog = BackendLog.Logger("NTAR")
dnssLog = BackendLog.Logger("DNSS")
snvrLog = BackendLog.Logger("SNVR")
ibdsLog = BackendLog.Logger("IBDS")
wsvcLog = BackendLog.Logger("WSVC")
reacLog = BackendLog.Logger("REAC")
)
@ -85,7 +84,6 @@ var SubsystemTags = struct {
NTAR,
DNSS,
SNVR,
IBDS,
WSVC,
REAC string
}{
@ -115,7 +113,6 @@ var SubsystemTags = struct {
NTAR: "NTAR",
DNSS: "DNSS",
SNVR: "SNVR",
IBDS: "IBDS",
WSVC: "WSVC",
REAC: "REAC",
}
@ -148,7 +145,6 @@ var subsystemLoggers = map[string]*Logger{
SubsystemTags.NTAR: ntarLog,
SubsystemTags.DNSS: dnssLog,
SubsystemTags.SNVR: snvrLog,
SubsystemTags.IBDS: ibdsLog,
SubsystemTags.WSVC: wsvcLog,
SubsystemTags.REAC: reacLog,
}

View File

@ -15,14 +15,12 @@ message KaspadMessage {
RequestNextHeadersMessage requestNextHeaders = 8;
DoneHeadersMessage DoneHeaders = 9;
RequestRelayBlocksMessage requestRelayBlocks = 10;
RequestSelectedTipMessage requestSelectedTip = 11;
RequestTransactionsMessage requestTransactions = 12;
BlockMessage ibdBlock = 13;
InvRelayBlockMessage invRelayBlock = 14;
InvTransactionsMessage invTransactions = 15;
PingMessage ping = 16;
PongMessage pong = 17;
SelectedTipMessage selectedTip = 18;
VerackMessage verack = 19;
VersionMessage version = 20;
TransactionNotFoundMessage transactionNotFound = 21;
@ -171,6 +169,7 @@ message Hash{
message RequestBlockLocatorMessage{
Hash lowHash = 1;
Hash highHash = 2;
uint32 limit = 3;
}
// GetBlockLocatorMessage end
@ -203,11 +202,6 @@ message RequestRelayBlocksMessage{
}
// RequestRelayBlocksMessage end
// GetSelectedTipMessage start
message RequestSelectedTipMessage{
}
// GetSelectedTipMessage end
// RequestTransactionsMessage start
message RequestTransactionsMessage {
repeated TransactionId ids = 1;
@ -244,12 +238,6 @@ message PongMessage{
}
// PongMessage end
// SelectedTipMessage start
message SelectedTipMessage{
Hash selectedTipHash = 1;
}
// SelectedTipMessage end
// VerackMessage start
message VerackMessage{
}
@ -263,7 +251,6 @@ message VersionMessage{
NetAddress address = 4;
bytes id = 5;
string userAgent = 6;
Hash selectedTipHash = 7;
bool disableRelayTx = 8;
SubnetworkId subnetworkId = 9;
string network = 10;
@ -408,8 +395,6 @@ message GetConnectedPeerInfoMessage{
string id = 1;
string address = 2;
int64 lastPingDuration = 3;
string selectedTipHash = 4;
bool isSyncNode = 5;
bool isOutbound = 6;
int64 timeOffset = 7;
string userAgent = 8;

View File

@ -16,6 +16,7 @@ func (x *KaspadMessage_RequestBlockLocator) toAppMessage() (appmessage.Message,
return &appmessage.MsgRequestBlockLocator{
LowHash: lowHash,
HighHash: highHash,
Limit: x.RequestBlockLocator.Limit,
}, nil
}
@ -23,6 +24,7 @@ func (x *KaspadMessage_RequestBlockLocator) fromAppMessage(msgGetBlockLocator *a
x.RequestBlockLocator = &RequestBlockLocatorMessage{
LowHash: domainHashToProto(msgGetBlockLocator.LowHash),
HighHash: domainHashToProto(msgGetBlockLocator.HighHash),
Limit: msgGetBlockLocator.Limit,
}
return nil
}

View File

@ -2,14 +2,9 @@ package protowire
import (
"github.com/kaspanet/kaspad/app/appmessage"
"github.com/pkg/errors"
)
func (x *KaspadMessage_RequestIBDBlocks) toAppMessage() (appmessage.Message, error) {
if len(x.RequestIBDBlocks.Hashes) > appmessage.MaxRequestIBDBlocksHashes {
return nil, errors.Errorf("too many hashes for message "+
"[count %d, max %d]", len(x.RequestIBDBlocks.Hashes), appmessage.MaxRequestIBDBlocksHashes)
}
hashes, err := protoHashesToDomain(x.RequestIBDBlocks.Hashes)
if err != nil {
return nil, err
@ -18,11 +13,6 @@ func (x *KaspadMessage_RequestIBDBlocks) toAppMessage() (appmessage.Message, err
}
func (x *KaspadMessage_RequestIBDBlocks) fromAppMessage(msgRequestIBDBlocks *appmessage.MsgRequestIBDBlocks) error {
if len(msgRequestIBDBlocks.Hashes) > appmessage.MaxRequestIBDBlocksHashes {
return errors.Errorf("too many hashes for message "+
"[count %d, max %d]", len(msgRequestIBDBlocks.Hashes), appmessage.MaxRequestIBDBlocksHashes)
}
x.RequestIBDBlocks = &RequestIBDBlocksMessage{
Hashes: domainHashesToProto(msgRequestIBDBlocks.Hashes),
}

View File

@ -1,11 +0,0 @@
package protowire
import "github.com/kaspanet/kaspad/app/appmessage"
func (x *KaspadMessage_RequestSelectedTip) toAppMessage() (appmessage.Message, error) {
return &appmessage.MsgRequestSelectedTip{}, nil
}
func (x *KaspadMessage_RequestSelectedTip) fromAppMessage(_ *appmessage.MsgRequestSelectedTip) error {
return nil
}

View File

@ -1,19 +0,0 @@
package protowire
import "github.com/kaspanet/kaspad/app/appmessage"
func (x *KaspadMessage_SelectedTip) toAppMessage() (appmessage.Message, error) {
hash, err := x.SelectedTip.SelectedTipHash.toDomain()
if err != nil {
return nil, err
}
return &appmessage.MsgSelectedTip{SelectedTipHash: hash}, nil
}
func (x *KaspadMessage_SelectedTip) fromAppMessage(msgSelectedTip *appmessage.MsgSelectedTip) error {
x.SelectedTip = &SelectedTipMessage{
SelectedTipHash: domainHashToProto(msgSelectedTip.SelectedTipHash),
}
return nil
}

View File

@ -17,11 +17,6 @@ func (x *KaspadMessage_Version) toAppMessage() (appmessage.Message, error) {
}
}
selectedTipHash, err := x.Version.SelectedTipHash.toDomain()
if err != nil {
return nil, err
}
subnetworkID, err := x.Version.SubnetworkId.toDomain()
if err != nil {
return nil, err
@ -40,7 +35,6 @@ func (x *KaspadMessage_Version) toAppMessage() (appmessage.Message, error) {
Address: address,
ID: id.FromBytes(x.Version.Id),
UserAgent: x.Version.UserAgent,
SelectedTipHash: selectedTipHash,
DisableRelayTx: x.Version.DisableRelayTx,
SubnetworkID: subnetworkID,
}, nil
@ -71,7 +65,6 @@ func (x *KaspadMessage_Version) fromAppMessage(msgVersion *appmessage.MsgVersion
Address: address,
Id: versionID,
UserAgent: msgVersion.UserAgent,
SelectedTipHash: domainHashToProto(msgVersion.SelectedTipHash),
DisableRelayTx: msgVersion.DisableRelayTx,
SubnetworkId: domainSubnetworkIDToProto(msgVersion.SubnetworkID),
}

View File

@ -23,8 +23,6 @@ func (x *KaspadMessage_GetConnectedPeerInfoResponse) toAppMessage() (appmessage.
ID: info.Id,
Address: info.Address,
LastPingDuration: info.LastPingDuration,
SelectedTipHash: info.SelectedTipHash,
IsSyncNode: info.IsSyncNode,
IsOutbound: info.IsOutbound,
TimeOffset: info.TimeOffset,
UserAgent: info.UserAgent,
@ -49,8 +47,6 @@ func (x *KaspadMessage_GetConnectedPeerInfoResponse) fromAppMessage(message *app
Id: info.ID,
Address: info.Address,
LastPingDuration: info.LastPingDuration,
SelectedTipHash: info.SelectedTipHash,
IsSyncNode: info.IsSyncNode,
IsOutbound: info.IsOutbound,
TimeOffset: info.TimeOffset,
UserAgent: info.UserAgent,

View File

@ -114,13 +114,6 @@ func toP2PPayload(message appmessage.Message) (isKaspadMessage_Payload, error) {
return nil, err
}
return payload, nil
case *appmessage.MsgRequestSelectedTip:
payload := new(KaspadMessage_RequestSelectedTip)
err := payload.fromAppMessage(message)
if err != nil {
return nil, err
}
return payload, nil
case *appmessage.MsgRequestTransactions:
payload := new(KaspadMessage_RequestTransactions)
err := payload.fromAppMessage(message)
@ -170,13 +163,6 @@ func toP2PPayload(message appmessage.Message) (isKaspadMessage_Payload, error) {
return nil, err
}
return payload, nil
case *appmessage.MsgSelectedTip:
payload := new(KaspadMessage_SelectedTip)
err := payload.fromAppMessage(message)
if err != nil {
return nil, err
}
return payload, nil
case *appmessage.MsgTx:
payload := new(KaspadMessage_Transaction)
err := payload.fromAppMessage(message)

View File

@ -120,7 +120,6 @@ func (mna *MinimalNetAdapter) handleHandshake(routes *Routes, ourID *id.ID) erro
Address: nil,
ID: ourID,
UserAgent: "/net-adapter-mock/",
SelectedTipHash: versionMessage.SelectedTipHash,
DisableRelayTx: true,
SubnetworkID: nil,
})

View File

@ -14,7 +14,7 @@ func TestIBD(t *testing.T) {
syncer, syncee, _, teardown := standardSetup(t)
defer teardown()
for i := 0; i < numBlocks; i++ {
for i := 0; i < numBlocks-1; i++ {
mineNextBlock(t, syncer)
}
@ -28,6 +28,9 @@ func TestIBD(t *testing.T) {
connect(t, syncer, syncee)
// We expect this to trigger IBD
mineNextBlock(t, syncer)
select {
case <-time.After(defaultTimeout):
t.Fatalf("Timeout waiting for IBD to finish. Received %d blocks out of %d", receivedBlocks, numBlocks)