mirror of
https://github.com/kaspanet/kaspad.git
synced 2025-06-06 14:16:43 +00:00
[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:
parent
4886425caf
commit
7f899b0d09
@ -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",
|
||||
|
@ -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",
|
||||
|
@ -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,
|
||||
}
|
||||
}
|
||||
|
@ -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)
|
||||
|
@ -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.
|
||||
|
@ -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{}
|
||||
}
|
@ -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)
|
||||
}
|
||||
}
|
@ -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,
|
||||
}
|
||||
}
|
@ -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)
|
||||
}
|
||||
}
|
@ -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,
|
||||
}
|
||||
|
@ -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)
|
||||
|
@ -41,8 +41,6 @@ type GetConnectedPeerInfoMessage struct {
|
||||
ID string
|
||||
Address string
|
||||
LastPingDuration int64
|
||||
SelectedTipHash string
|
||||
IsSyncNode bool
|
||||
IsOutbound bool
|
||||
TimeOffset int64
|
||||
UserAgent string
|
||||
|
@ -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")
|
||||
}
|
||||
|
@ -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),
|
||||
|
@ -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
|
||||
}
|
@ -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)
|
||||
|
||||
|
@ -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
|
||||
}
|
||||
|
||||
|
29
app/protocol/flows/blockrelay/block_locator.go
Normal file
29
app/protocol/flows/blockrelay/block_locator.go
Normal 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
|
||||
}
|
@ -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)
|
@ -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)
|
||||
|
@ -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...)
|
||||
}
|
||||
|
@ -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 {
|
@ -1,4 +1,4 @@
|
||||
package ibd
|
||||
package blockrelay
|
||||
|
||||
import (
|
||||
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
|
@ -1,4 +1,4 @@
|
||||
package ibd
|
||||
package blockrelay
|
||||
|
||||
import (
|
||||
"github.com/kaspanet/kaspad/domain/consensus/utils/testutils"
|
@ -1,4 +1,4 @@
|
||||
package ibd
|
||||
package blockrelay
|
||||
|
||||
import (
|
||||
"errors"
|
@ -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
|
||||
}
|
@ -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
|
||||
}
|
||||
|
||||
|
@ -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
|
||||
}
|
||||
|
@ -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)
|
@ -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)
|
||||
}
|
@ -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
|
||||
}
|
@ -1,4 +1,4 @@
|
||||
package relaytransactions
|
||||
package transactionrelay
|
||||
|
||||
import (
|
||||
"github.com/kaspanet/kaspad/app/appmessage"
|
@ -1,4 +1,4 @@
|
||||
package relaytransactions
|
||||
package transactionrelay
|
||||
|
||||
import (
|
||||
"github.com/kaspanet/kaspad/app/appmessage"
|
@ -1,4 +1,4 @@
|
||||
package relaytransactions
|
||||
package transactionrelay
|
||||
|
||||
import (
|
||||
"sync"
|
@ -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)
|
||||
|
@ -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()
|
||||
|
@ -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)
|
||||
},
|
||||
),
|
||||
}
|
||||
|
@ -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(),
|
||||
|
@ -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) {
|
||||
|
@ -248,7 +248,7 @@ func (f *factory) NewConsensus(dagParams *dagconfig.Params, db infrastructuredat
|
||||
)
|
||||
|
||||
blockProcessor := blockprocessor.New(
|
||||
dagParams,
|
||||
genesisHash,
|
||||
dbManager,
|
||||
consensusStateManager,
|
||||
pruningManager,
|
||||
|
@ -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)
|
||||
}
|
||||
|
@ -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)
|
||||
|
@ -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)
|
||||
|
@ -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,
|
||||
|
@ -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
|
||||
|
@ -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)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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
|
||||
}
|
||||
}
|
||||
|
@ -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)
|
||||
|
@ -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
|
||||
}
|
||||
|
||||
|
@ -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
|
||||
}
|
||||
|
@ -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()
|
||||
}
|
||||
|
@ -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) {
|
||||
|
@ -228,8 +228,6 @@ var (
|
||||
ErrSubnetworksDisabled = newRuleError("ErrSubnetworksDisabled")
|
||||
ErrBadPruningPointUTXOSet = newRuleError("ErrBadPruningPointUTXOSet")
|
||||
|
||||
ErrMissingBlockHeaderInIBD = newRuleError("ErrMissingBlockHeaderInIBD")
|
||||
|
||||
ErrMalformedUTXO = newRuleError("ErrMalformedUTXO")
|
||||
|
||||
ErrWrongPruningPointHash = newRuleError("ErrWrongPruningPointHash")
|
||||
|
@ -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,
|
||||
}
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -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;
|
||||
|
@ -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
|
||||
}
|
||||
|
@ -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),
|
||||
}
|
||||
|
@ -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
|
||||
}
|
@ -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
|
||||
}
|
@ -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),
|
||||
}
|
||||
|
@ -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,
|
||||
|
@ -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)
|
||||
|
@ -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,
|
||||
})
|
||||
|
@ -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)
|
||||
|
Loading…
x
Reference in New Issue
Block a user