Move the heavy lifting in BlockLocator from the syncer to the syncee (#1324)

* Add a new message: BlockLocatorHighestHash.

* Add a new message: IBDBlockLocator.

* Implement HandleIBDBlockLocator.

* Reimplement findHighestSharedBlockHash.

* Make HandleIBDBlockLocator only return hashes that are in the selected parent chain of the target hash.

* Increase the cache sizes of blockRelationStore, reachabilityDataStore, and ghostdagDataStore.

* Fix wrong initial highHash in findHighestSharedBlockHash.

* Make go vet happy.

* Protect against receiving wrong messages when expecting MsgIBDBlockLocatorHighestHash.
This commit is contained in:
stasatdaglabs 2020-12-30 15:44:14 +02:00 committed by GitHub
parent 6172e48adc
commit 7dd0188838
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
19 changed files with 1678 additions and 1229 deletions

View File

@ -57,6 +57,8 @@ const (
CmdIBDRootNotFound CmdIBDRootNotFound
CmdRequestIBDRootHash CmdRequestIBDRootHash
CmdIBDRootHash CmdIBDRootHash
CmdIBDBlockLocator
CmdIBDBlockLocatorHighestHash
// rpc // rpc
CmdGetCurrentNetworkRequestMessage CmdGetCurrentNetworkRequestMessage
@ -149,6 +151,8 @@ var ProtocolMessageCommandToString = map[MessageCommand]string{
CmdIBDRootNotFound: "IBDRootNotFound", CmdIBDRootNotFound: "IBDRootNotFound",
CmdRequestIBDRootHash: "IBDRequestIBDRootHash", CmdRequestIBDRootHash: "IBDRequestIBDRootHash",
CmdIBDRootHash: "IBDIBDRootHash", CmdIBDRootHash: "IBDIBDRootHash",
CmdIBDBlockLocator: "IBDBlockLocator",
CmdIBDBlockLocatorHighestHash: "IBDBlockLocatorHighestHash",
} }
// RPCMessageCommandToString maps all MessageCommands to their string representation // RPCMessageCommandToString maps all MessageCommands to their string representation

View File

@ -0,0 +1,27 @@
package appmessage
import (
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
)
// MsgIBDBlockLocator represents a kaspa ibdBlockLocator message
type MsgIBDBlockLocator struct {
baseMessage
TargetHash *externalapi.DomainHash
BlockLocatorHashes []*externalapi.DomainHash
}
// Command returns the protocol command string for the message
func (msg *MsgIBDBlockLocator) Command() MessageCommand {
return CmdIBDBlockLocator
}
// NewMsgIBDBlockLocator returns a new kaspa ibdBlockLocator message
func NewMsgIBDBlockLocator(targetHash *externalapi.DomainHash,
blockLocatorHashes []*externalapi.DomainHash) *MsgIBDBlockLocator {
return &MsgIBDBlockLocator{
TargetHash: targetHash,
BlockLocatorHashes: blockLocatorHashes,
}
}

View File

@ -0,0 +1,23 @@
package appmessage
import (
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
)
// MsgIBDBlockLocatorHighestHash represents a kaspa BlockLocatorHighestHash message
type MsgIBDBlockLocatorHighestHash struct {
baseMessage
HighestHash *externalapi.DomainHash
}
// Command returns the protocol command string for the message
func (msg *MsgIBDBlockLocatorHighestHash) Command() MessageCommand {
return CmdIBDBlockLocatorHighestHash
}
// NewMsgIBDBlockLocatorHighestHash returns a new BlockLocatorHighestHash message
func NewMsgIBDBlockLocatorHighestHash(highestHash *externalapi.DomainHash) *MsgIBDBlockLocatorHighestHash {
return &MsgIBDBlockLocatorHighestHash{
HighestHash: highestHash,
}
}

View File

@ -4,23 +4,23 @@ import (
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi" "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
) )
// MsgIBDRootHash implements the Message interface and represents a kaspa // MsgIBDRootHashMessage implements the Message interface and represents a kaspa
// IBDRootHash message. It is used as a reply to IBD root hash requests. // IBDRootHash message. It is used as a reply to IBD root hash requests.
type MsgIBDRootHash struct { type MsgIBDRootHashMessage struct {
baseMessage baseMessage
Hash *externalapi.DomainHash Hash *externalapi.DomainHash
} }
// Command returns the protocol command string for the message. This is part // Command returns the protocol command string for the message. This is part
// of the Message interface implementation. // of the Message interface implementation.
func (msg *MsgIBDRootHash) Command() MessageCommand { func (msg *MsgIBDRootHashMessage) Command() MessageCommand {
return CmdIBDRootHash return CmdIBDRootHash
} }
// NewMsgIBDRootHash returns a new kaspa IBDRootHash message that conforms to // NewMsgIBDRootHashMessage returns a new kaspa IBDRootHash message that conforms to
// the Message interface. See MsgIBDRootHash for details. // the Message interface. See MsgIBDRootHashMessage for details.
func NewMsgIBDRootHash(hash *externalapi.DomainHash) *MsgIBDRootHash { func NewMsgIBDRootHashMessage(hash *externalapi.DomainHash) *MsgIBDRootHashMessage {
return &MsgIBDRootHash{ return &MsgIBDRootHashMessage{
Hash: hash, Hash: hash,
} }
} }

View File

@ -1,22 +1,22 @@
package appmessage package appmessage
// MsgRequestIBDRootHash implements the Message interface and represents a kaspa // MsgRequestIBDRootHashMessage implements the Message interface and represents a kaspa
// MsgRequestIBDRootHash message. It is used to request the IBD root hash // MsgRequestIBDRootHashMessage message. It is used to request the IBD root hash
// from a peer during IBD. // from a peer during IBD.
// //
// This message has no payload. // This message has no payload.
type MsgRequestIBDRootHash struct { type MsgRequestIBDRootHashMessage struct {
baseMessage baseMessage
} }
// Command returns the protocol command string for the message. This is part // Command returns the protocol command string for the message. This is part
// of the Message interface implementation. // of the Message interface implementation.
func (msg *MsgRequestIBDRootHash) Command() MessageCommand { func (msg *MsgRequestIBDRootHashMessage) Command() MessageCommand {
return CmdRequestIBDRootHash return CmdRequestIBDRootHash
} }
// NewMsgRequestIBDRootHash returns a new kaspa RequestIBDRootHash message that conforms to the // NewMsgRequestIBDRootHashMessage returns a new kaspa RequestIBDRootHash message that conforms to the
// Message interface. // Message interface.
func NewMsgRequestIBDRootHash() *MsgRequestIBDRootHash { func NewMsgRequestIBDRootHashMessage() *MsgRequestIBDRootHashMessage {
return &MsgRequestIBDRootHash{} return &MsgRequestIBDRootHashMessage{}
} }

View File

@ -0,0 +1,77 @@
package blockrelay
import (
"github.com/kaspanet/kaspad/app/appmessage"
"github.com/kaspanet/kaspad/app/protocol/peer"
"github.com/kaspanet/kaspad/app/protocol/protocolerrors"
"github.com/kaspanet/kaspad/domain"
"github.com/kaspanet/kaspad/infrastructure/network/netadapter/router"
)
// HandleIBDBlockLocatorContext is the interface for the context needed for the HandleIBDBlockLocator flow.
type HandleIBDBlockLocatorContext interface {
Domain() domain.Domain
}
// HandleIBDBlockLocator listens to appmessage.MsgIBDBlockLocator messages and sends
// the highest known block that's in the selected parent chain of `targetHash` to the
// requesting peer.
func HandleIBDBlockLocator(context HandleIBDBlockLocatorContext, incomingRoute *router.Route,
outgoingRoute *router.Route, peer *peer.Peer) error {
for {
message, err := incomingRoute.Dequeue()
if err != nil {
return err
}
ibdBlockLocatorMessage := message.(*appmessage.MsgIBDBlockLocator)
targetHash := ibdBlockLocatorMessage.TargetHash
log.Debugf("Received IBDBlockLocator from %s with targetHash %s", peer, targetHash)
blockInfo, err := context.Domain().Consensus().GetBlockInfo(targetHash)
if err != nil {
return err
}
if !blockInfo.Exists {
return protocolerrors.Errorf(true, "received IBDBlockLocator "+
"with an unknown targetHash %s", targetHash)
}
foundHighestHashInTheSelectedParentChainOfTargetHash := false
for _, blockLocatorHash := range ibdBlockLocatorMessage.BlockLocatorHashes {
blockInfo, err := context.Domain().Consensus().GetBlockInfo(blockLocatorHash)
if err != nil {
return err
}
if !blockInfo.Exists {
continue
}
isBlockLocatorHashInSelectedParentChainOfHighHash, err :=
context.Domain().Consensus().IsInSelectedParentChainOf(blockLocatorHash, targetHash)
if err != nil {
return err
}
if !isBlockLocatorHashInSelectedParentChainOfHighHash {
continue
}
foundHighestHashInTheSelectedParentChainOfTargetHash = true
log.Debugf("Found a known hash %s amongst peer %s's "+
"blockLocator that's in the selected parent chain of targetHash %s", blockLocatorHash, peer, targetHash)
ibdBlockLocatorHighestHashMessage := appmessage.NewMsgIBDBlockLocatorHighestHash(blockLocatorHash)
err = outgoingRoute.Enqueue(ibdBlockLocatorHighestHashMessage)
if err != nil {
return err
}
break
}
if !foundHighestHashInTheSelectedParentChainOfTargetHash {
return protocolerrors.Errorf(true, "no hash was found in the blockLocator "+
"that was in the selected parent chain of targetHash %s", targetHash)
}
}
}

View File

@ -16,7 +16,7 @@ type handleIBDRootHashRequestsFlow struct {
incomingRoute, outgoingRoute *router.Route incomingRoute, outgoingRoute *router.Route
} }
// HandleIBDRootHashRequests listens to appmessage.MsgRequestIBDRootHash messages and sends // HandleIBDRootHashRequests listens to appmessage.MsgRequestIBDRootHashMessage messages and sends
// the IBD root hash as response. // the IBD root hash as response.
func HandleIBDRootHashRequests(context HandleIBDRootHashRequestsFlowContext, incomingRoute, func HandleIBDRootHashRequests(context HandleIBDRootHashRequestsFlowContext, incomingRoute,
outgoingRoute *router.Route) error { outgoingRoute *router.Route) error {
@ -41,7 +41,7 @@ func (flow *handleIBDRootHashRequestsFlow) start() error {
return err return err
} }
err = flow.outgoingRoute.Enqueue(appmessage.NewMsgIBDRootHash(pruningPoint)) err = flow.outgoingRoute.Enqueue(appmessage.NewMsgIBDRootHashMessage(pruningPoint))
if err != nil { if err != nil {
return err return err
} }

View File

@ -31,7 +31,7 @@ func (flow *handleRelayInvsFlow) runIBDIfNotRunning(highHash *externalapi.Domain
// Fetch the UTXO set if we don't already have it // Fetch the UTXO set if we don't already have it
log.Debugf("Checking if there's a new pruning point under %s", highHash) log.Debugf("Checking if there's a new pruning point under %s", highHash)
err = flow.outgoingRoute.Enqueue(appmessage.NewMsgRequestIBDRootHash()) err = flow.outgoingRoute.Enqueue(appmessage.NewMsgRequestIBDRootHashMessage())
if err != nil { if err != nil {
return err return err
} }
@ -41,7 +41,7 @@ func (flow *handleRelayInvsFlow) runIBDIfNotRunning(highHash *externalapi.Domain
return err return err
} }
msgIBDRootHash, ok := message.(*appmessage.MsgIBDRootHash) msgIBDRootHash, ok := message.(*appmessage.MsgIBDRootHashMessage)
if !ok { if !ok {
return protocolerrors.Errorf(true, "received unexpected message type. "+ return protocolerrors.Errorf(true, "received unexpected message type. "+
"expected: %s, got: %s", appmessage.CmdIBDRootHash, message.Command()) "expected: %s, got: %s", appmessage.CmdIBDRootHash, message.Command())
@ -118,40 +118,61 @@ func (flow *handleRelayInvsFlow) syncHeaders(highHash *externalapi.DomainHash) e
return nil return nil
} }
func (flow *handleRelayInvsFlow) findHighestSharedBlockHash(highHash *externalapi.DomainHash) ( func (flow *handleRelayInvsFlow) findHighestSharedBlockHash(targetHash *externalapi.DomainHash) (*externalapi.DomainHash, error) {
lowHash *externalapi.DomainHash, err error) { lowHash := flow.Config().ActiveNetParams.GenesisHash
highHash, err := flow.Domain().Consensus().GetHeadersSelectedTip()
lowHash = flow.Config().ActiveNetParams.GenesisHash if err != nil {
currentHighHash := highHash return nil, err
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
}
} }
for !lowHash.Equal(highHash) {
log.Debugf("Sending a blockLocator to %s between %s and %s", flow.peer, lowHash, highHash)
blockLocator, err := flow.Domain().Consensus().CreateBlockLocator(lowHash, highHash, 0)
if err != nil {
return nil, err
}
ibdBlockLocatorMessage := appmessage.NewMsgIBDBlockLocator(targetHash, blockLocator)
err = flow.outgoingRoute.Enqueue(ibdBlockLocatorMessage)
if err != nil {
return nil, err
}
message, err := flow.dequeueIncomingMessageAndSkipInvs(common.DefaultTimeout)
if err != nil {
return nil, err
}
ibdBlockLocatorHighestHashMessage, ok := message.(*appmessage.MsgIBDBlockLocatorHighestHash)
if !ok {
return nil, protocolerrors.Errorf(true, "received unexpected message type. "+
"expected: %s, got: %s", appmessage.CmdIBDBlockLocatorHighestHash, message.Command())
}
highestHash := ibdBlockLocatorHighestHashMessage.HighestHash
log.Debugf("The highest hash the peer %s knows is %s", flow.peer, highestHash)
highestHashIndex := 0
highestHashIndexFound := false
for i, blockLocatorHash := range blockLocator {
if highestHash.Equal(blockLocatorHash) {
highestHashIndex = i
highestHashIndexFound = true
break
}
}
if !highestHashIndexFound {
return nil, protocolerrors.Errorf(true, "highest hash %s "+
"returned from peer %s is not in the original blockLocator", highestHash, flow.peer)
}
log.Debugf("The index of the highest hash in the original "+
"blockLocator sent to %s is %d", flow.peer, highestHashIndex)
locatorHashAboveHighestHash := highestHash
if highestHashIndex > 0 {
locatorHashAboveHighestHash = blockLocator[highestHashIndex-1]
}
highHash = locatorHashAboveHighestHash
lowHash = highestHash
}
return highHash, nil
} }
func (flow *handleRelayInvsFlow) downloadHeaders(highestSharedBlockHash *externalapi.DomainHash, func (flow *handleRelayInvsFlow) downloadHeaders(highestSharedBlockHash *externalapi.DomainHash,

View File

@ -136,8 +136,8 @@ func (m *Manager) registerBlockRelayFlows(router *routerpkg.Router, isStopping *
m.registerFlow("HandleRelayInvs", router, []appmessage.MessageCommand{ m.registerFlow("HandleRelayInvs", router, []appmessage.MessageCommand{
appmessage.CmdInvRelayBlock, appmessage.CmdBlock, appmessage.CmdBlockLocator, appmessage.CmdIBDBlock, appmessage.CmdInvRelayBlock, appmessage.CmdBlock, appmessage.CmdBlockLocator, appmessage.CmdIBDBlock,
appmessage.CmdDoneHeaders, appmessage.CmdIBDRootNotFound, appmessage.CmdIBDRootUTXOSetAndBlock, appmessage.CmdDoneHeaders, appmessage.CmdIBDRootNotFound, appmessage.CmdIBDRootUTXOSetAndBlock,
appmessage.CmdHeader, appmessage.CmdIBDRootHash}, isStopping, errChan, appmessage.CmdHeader, appmessage.CmdIBDRootHash, appmessage.CmdIBDBlockLocatorHighestHash},
func(incomingRoute *routerpkg.Route, peer *peerpkg.Peer) error { isStopping, errChan, func(incomingRoute *routerpkg.Route, peer *peerpkg.Peer) error {
return blockrelay.HandleRelayInvs(m.context, incomingRoute, return blockrelay.HandleRelayInvs(m.context, incomingRoute,
outgoingRoute, peer) outgoingRoute, peer)
}, },
@ -183,6 +183,13 @@ func (m *Manager) registerBlockRelayFlows(router *routerpkg.Router, isStopping *
return blockrelay.HandleIBDRootHashRequests(m.context, incomingRoute, outgoingRoute) return blockrelay.HandleIBDRootHashRequests(m.context, incomingRoute, outgoingRoute)
}, },
), ),
m.registerFlow("HandleIBDBlockLocator", router,
[]appmessage.MessageCommand{appmessage.CmdIBDBlockLocator}, isStopping, errChan,
func(incomingRoute *routerpkg.Route, peer *peerpkg.Peer) error {
return blockrelay.HandleIBDBlockLocator(m.context, incomingRoute, outgoingRoute, peer)
},
),
} }
} }

View File

@ -344,3 +344,26 @@ func (s *consensus) validateBlockHashExists(blockHash *externalapi.DomainHash) e
} }
return nil return nil
} }
func (s *consensus) IsInSelectedParentChainOf(blockHashA *externalapi.DomainHash, blockHashB *externalapi.DomainHash) (bool, error) {
s.lock.Lock()
defer s.lock.Unlock()
err := s.validateBlockHashExists(blockHashA)
if err != nil {
return false, err
}
err = s.validateBlockHashExists(blockHashB)
if err != nil {
return false, err
}
return s.dagTopologyManager.IsInSelectedParentChainOf(blockHashA, blockHashB)
}
func (s *consensus) GetHeadersSelectedTip() (*externalapi.DomainHash, error) {
s.lock.Lock()
defer s.lock.Unlock()
return s.headersSelectedTipStore.HeadersSelectedTip(s.databaseContext)
}

View File

@ -64,6 +64,8 @@ func NewFactory() Factory {
func (f *factory) NewConsensus(dagParams *dagconfig.Params, db infrastructuredatabase.Database) (externalapi.Consensus, error) { func (f *factory) NewConsensus(dagParams *dagconfig.Params, db infrastructuredatabase.Database) (externalapi.Consensus, error) {
dbManager := consensusdatabase.New(db) dbManager := consensusdatabase.New(db)
pruningWindowSizeForCaches := int(dagParams.PruningDepth())
// Data Structures // Data Structures
acceptanceDataStore := acceptancedatastore.New(200) acceptanceDataStore := acceptancedatastore.New(200)
blockStore, err := blockstore.New(dbManager, 200) blockStore, err := blockstore.New(dbManager, 200)
@ -74,14 +76,14 @@ func (f *factory) NewConsensus(dagParams *dagconfig.Params, db infrastructuredat
if err != nil { if err != nil {
return nil, err return nil, err
} }
blockRelationStore := blockrelationstore.New(10_000) blockRelationStore := blockrelationstore.New(pruningWindowSizeForCaches)
blockStatusStore := blockstatusstore.New(200) blockStatusStore := blockstatusstore.New(200)
multisetStore := multisetstore.New(200) multisetStore := multisetstore.New(200)
pruningStore := pruningstore.New() pruningStore := pruningstore.New()
reachabilityDataStore := reachabilitydatastore.New(10_000) reachabilityDataStore := reachabilitydatastore.New(pruningWindowSizeForCaches)
utxoDiffStore := utxodiffstore.New(200) utxoDiffStore := utxodiffstore.New(200)
consensusStateStore := consensusstatestore.New() consensusStateStore := consensusstatestore.New()
ghostdagDataStore := ghostdagdatastore.New(10_000) ghostdagDataStore := ghostdagdatastore.New(pruningWindowSizeForCaches)
headersSelectedTipStore := headersselectedtipstore.New() headersSelectedTipStore := headersselectedtipstore.New()
finalityStore := finalitystore.New(200) finalityStore := finalitystore.New(200)

View File

@ -24,4 +24,6 @@ type Consensus interface {
GetVirtualInfo() (*VirtualInfo, error) GetVirtualInfo() (*VirtualInfo, error)
IsValidPruningPoint(blockHash *DomainHash) (bool, error) IsValidPruningPoint(blockHash *DomainHash) (bool, error)
GetVirtualSelectedParentChainFromBlock(blockHash *DomainHash) (*SelectedParentChainChanges, error) GetVirtualSelectedParentChainFromBlock(blockHash *DomainHash) (*SelectedParentChainChanges, error)
IsInSelectedParentChainOf(blockHashA *DomainHash, blockHashB *DomainHash) (bool, error)
GetHeadersSelectedTip() (*DomainHash, error)
} }

View File

@ -30,8 +30,10 @@ message KaspadMessage {
IBDRootUTXOSetAndBlockMessage ibdRootUTXOSetAndBlock = 25; IBDRootUTXOSetAndBlockMessage ibdRootUTXOSetAndBlock = 25;
RequestIBDBlocksMessage requestIBDBlocks = 26; RequestIBDBlocksMessage requestIBDBlocks = 26;
IBDRootNotFoundMessage ibdRootNotFound = 27; IBDRootNotFoundMessage ibdRootNotFound = 27;
RequestIBDRootHash requestIBDRootHash = 28; RequestIBDRootHashMessage requestIBDRootHash = 28;
IBDRootHash ibdRootHash = 29; IBDRootHashMessage ibdRootHash = 29;
IbdBlockLocatorMessage ibdBlockLocator = 30;
IbdBlockLocatorHighestHashMessage ibdBlockLocatorHighestHash = 31;
GetCurrentNetworkRequestMessage getCurrentNetworkRequest = 1001; GetCurrentNetworkRequestMessage getCurrentNetworkRequest = 1001;
GetCurrentNetworkResponseMessage getCurrentNetworkResponse = 1002; GetCurrentNetworkResponseMessage getCurrentNetworkResponse = 1002;
@ -299,16 +301,29 @@ message IBDRootNotFoundMessage{
} }
// IBDRootNotFoundMessage end // IBDRootNotFoundMessage end
// RequestIBDRootHash start // RequestIBDRootHashMessage start
message RequestIBDRootHash{ message RequestIBDRootHashMessage{
} }
// RequestIBDRootHash end // RequestIBDRootHashMessage end
// IBDRootHash start // IBDRootHashMessage start
message IBDRootHash{ message IBDRootHashMessage{
Hash hash = 1; Hash hash = 1;
} }
// IBDRootHash end // IBDRootHashMessage end
// IbdBlockLocatorMessage start
message IbdBlockLocatorMessage {
Hash targetHash = 1;
repeated Hash blockLocatorHashes = 2;
}
// IbdBlockLocatorMessage end
// IbdBlockLocatorHighestHashMessage start
message IbdBlockLocatorHighestHashMessage {
Hash highestHash = 1;
}
// IbdBlockLocatorHighestHashMessage end
service P2P { service P2P {
rpc MessageStream (stream KaspadMessage) returns (stream KaspadMessage) {} rpc MessageStream (stream KaspadMessage) returns (stream KaspadMessage) {}

View File

@ -0,0 +1,28 @@
package protowire
import (
"github.com/kaspanet/kaspad/app/appmessage"
)
func (x *KaspadMessage_IbdBlockLocator) toAppMessage() (appmessage.Message, error) {
targetHash, err := x.IbdBlockLocator.TargetHash.toDomain()
if err != nil {
return nil, err
}
blockLocatorHash, err := protoHashesToDomain(x.IbdBlockLocator.BlockLocatorHashes)
if err != nil {
return nil, err
}
return &appmessage.MsgIBDBlockLocator{
TargetHash: targetHash,
BlockLocatorHashes: blockLocatorHash,
}, nil
}
func (x *KaspadMessage_IbdBlockLocator) fromAppMessage(message *appmessage.MsgIBDBlockLocator) error {
x.IbdBlockLocator = &IbdBlockLocatorMessage{
TargetHash: domainHashToProto(message.TargetHash),
BlockLocatorHashes: domainHashesToProto(message.BlockLocatorHashes),
}
return nil
}

View File

@ -0,0 +1,21 @@
package protowire
import "github.com/kaspanet/kaspad/app/appmessage"
func (x *KaspadMessage_IbdBlockLocatorHighestHash) toAppMessage() (appmessage.Message, error) {
highestHash, err := x.IbdBlockLocatorHighestHash.HighestHash.toDomain()
if err != nil {
return nil, err
}
return &appmessage.MsgIBDBlockLocatorHighestHash{
HighestHash: highestHash,
}, nil
}
func (x *KaspadMessage_IbdBlockLocatorHighestHash) fromAppMessage(message *appmessage.MsgIBDBlockLocatorHighestHash) error {
x.IbdBlockLocatorHighestHash = &IbdBlockLocatorHighestHashMessage{
HighestHash: domainHashToProto(message.HighestHash),
}
return nil
}

View File

@ -8,11 +8,11 @@ func (x *KaspadMessage_IbdRootHash) toAppMessage() (appmessage.Message, error) {
return nil, err return nil, err
} }
return &appmessage.MsgIBDRootHash{Hash: hash}, nil return &appmessage.MsgIBDRootHashMessage{Hash: hash}, nil
} }
func (x *KaspadMessage_IbdRootHash) fromAppMessage(msgIBDRootHash *appmessage.MsgIBDRootHash) error { func (x *KaspadMessage_IbdRootHash) fromAppMessage(msgIBDRootHash *appmessage.MsgIBDRootHashMessage) error {
x.IbdRootHash = &IBDRootHash{ x.IbdRootHash = &IBDRootHashMessage{
Hash: domainHashToProto(msgIBDRootHash.Hash), Hash: domainHashToProto(msgIBDRootHash.Hash),
} }
return nil return nil

View File

@ -3,9 +3,9 @@ package protowire
import "github.com/kaspanet/kaspad/app/appmessage" import "github.com/kaspanet/kaspad/app/appmessage"
func (x *KaspadMessage_RequestIBDRootHash) toAppMessage() (appmessage.Message, error) { func (x *KaspadMessage_RequestIBDRootHash) toAppMessage() (appmessage.Message, error) {
return &appmessage.MsgRequestIBDRootHash{}, nil return &appmessage.MsgRequestIBDRootHashMessage{}, nil
} }
func (x *KaspadMessage_RequestIBDRootHash) fromAppMessage(_ *appmessage.MsgRequestIBDRootHash) error { func (x *KaspadMessage_RequestIBDRootHash) fromAppMessage(_ *appmessage.MsgRequestIBDRootHashMessage) error {
return nil return nil
} }

View File

@ -195,7 +195,6 @@ func toP2PPayload(message appmessage.Message) (isKaspadMessage_Payload, error) {
return nil, err return nil, err
} }
return payload, nil return payload, nil
case *appmessage.MsgBlockHeader: case *appmessage.MsgBlockHeader:
payload := new(KaspadMessage_BlockHeader) payload := new(KaspadMessage_BlockHeader)
err := payload.fromAppMessage(message) err := payload.fromAppMessage(message)
@ -203,7 +202,6 @@ func toP2PPayload(message appmessage.Message) (isKaspadMessage_Payload, error) {
return nil, err return nil, err
} }
return payload, nil return payload, nil
case *appmessage.MsgRequestIBDRootUTXOSetAndBlock: case *appmessage.MsgRequestIBDRootUTXOSetAndBlock:
payload := new(KaspadMessage_RequestIBDRootUTXOSetAndBlock) payload := new(KaspadMessage_RequestIBDRootUTXOSetAndBlock)
err := payload.fromAppMessage(message) err := payload.fromAppMessage(message)
@ -211,7 +209,6 @@ func toP2PPayload(message appmessage.Message) (isKaspadMessage_Payload, error) {
return nil, err return nil, err
} }
return payload, nil return payload, nil
case *appmessage.MsgIBDRootUTXOSetAndBlock: case *appmessage.MsgIBDRootUTXOSetAndBlock:
payload := new(KaspadMessage_IbdRootUTXOSetAndBlock) payload := new(KaspadMessage_IbdRootUTXOSetAndBlock)
err := payload.fromAppMessage(message) err := payload.fromAppMessage(message)
@ -219,7 +216,6 @@ func toP2PPayload(message appmessage.Message) (isKaspadMessage_Payload, error) {
return nil, err return nil, err
} }
return payload, nil return payload, nil
case *appmessage.MsgRequestHeaders: case *appmessage.MsgRequestHeaders:
payload := new(KaspadMessage_RequestHeaders) payload := new(KaspadMessage_RequestHeaders)
err := payload.fromAppMessage(message) err := payload.fromAppMessage(message)
@ -227,7 +223,6 @@ func toP2PPayload(message appmessage.Message) (isKaspadMessage_Payload, error) {
return nil, err return nil, err
} }
return payload, nil return payload, nil
case *appmessage.MsgIBDRootNotFound: case *appmessage.MsgIBDRootNotFound:
payload := new(KaspadMessage_IbdRootNotFound) payload := new(KaspadMessage_IbdRootNotFound)
err := payload.fromAppMessage(message) err := payload.fromAppMessage(message)
@ -235,23 +230,34 @@ func toP2PPayload(message appmessage.Message) (isKaspadMessage_Payload, error) {
return nil, err return nil, err
} }
return payload, nil return payload, nil
case *appmessage.MsgRequestIBDRootHashMessage:
case *appmessage.MsgRequestIBDRootHash:
payload := new(KaspadMessage_RequestIBDRootHash) payload := new(KaspadMessage_RequestIBDRootHash)
err := payload.fromAppMessage(message) err := payload.fromAppMessage(message)
if err != nil { if err != nil {
return nil, err return nil, err
} }
return payload, nil return payload, nil
case *appmessage.MsgIBDRootHashMessage:
case *appmessage.MsgIBDRootHash:
payload := new(KaspadMessage_IbdRootHash) payload := new(KaspadMessage_IbdRootHash)
err := payload.fromAppMessage(message) err := payload.fromAppMessage(message)
if err != nil { if err != nil {
return nil, err return nil, err
} }
return payload, nil return payload, nil
case *appmessage.MsgIBDBlockLocator:
payload := new(KaspadMessage_IbdBlockLocator)
err := payload.fromAppMessage(message)
if err != nil {
return nil, err
}
return payload, nil
case *appmessage.MsgIBDBlockLocatorHighestHash:
payload := new(KaspadMessage_IbdBlockLocatorHighestHash)
err := payload.fromAppMessage(message)
if err != nil {
return nil, err
}
return payload, nil
default: default:
return nil, nil return nil, nil
} }