Filter redundant blocks from daa window (#1925)

* Copy blockrelay flows to v4

* Remove duplicate sending of the same DAA blocks

* Advance testnet version

* Renames and add comments

* Add IBD test between v3 and v4

* Fix syncee v4 p2p version

* Check if netsync finished with selected tip
This commit is contained in:
Ori Newman 2022-01-09 16:58:51 +02:00 committed by GitHub
parent d2379608ad
commit dadacdc0f4
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
41 changed files with 3686 additions and 919 deletions

View File

@ -436,10 +436,10 @@ func RPCBlockToDomainBlock(block *RPCBlock) (*externalapi.DomainBlock, error) {
// BlockWithTrustedDataToDomainBlockWithTrustedData converts *MsgBlockWithTrustedData to *externalapi.BlockWithTrustedData
func BlockWithTrustedDataToDomainBlockWithTrustedData(block *MsgBlockWithTrustedData) *externalapi.BlockWithTrustedData {
daaWindow := make([]*externalapi.TrustedDataDataDAABlock, len(block.DAAWindow))
daaWindow := make([]*externalapi.TrustedDataDataDAAHeader, len(block.DAAWindow))
for i, daaBlock := range block.DAAWindow {
daaWindow[i] = &externalapi.TrustedDataDataDAABlock{
Block: MsgBlockToDomainBlock(daaBlock.Block),
daaWindow[i] = &externalapi.TrustedDataDataDAAHeader{
Header: BlockHeaderToDomainBlockHeader(&daaBlock.Block.Header),
GHOSTDAGData: ghostdagDataToDomainGHOSTDAGData(daaBlock.GHOSTDAGData),
}
}
@ -454,12 +454,27 @@ func BlockWithTrustedDataToDomainBlockWithTrustedData(block *MsgBlockWithTrusted
return &externalapi.BlockWithTrustedData{
Block: MsgBlockToDomainBlock(block.Block),
DAAScore: block.DAAScore,
DAAWindow: daaWindow,
GHOSTDAGData: ghostdagData,
}
}
// TrustedDataDataDAABlockV4ToTrustedDataDataDAAHeader converts *TrustedDataDAAHeader to *externalapi.TrustedDataDataDAAHeader
func TrustedDataDataDAABlockV4ToTrustedDataDataDAAHeader(daaBlock *TrustedDataDAAHeader) *externalapi.TrustedDataDataDAAHeader {
return &externalapi.TrustedDataDataDAAHeader{
Header: BlockHeaderToDomainBlockHeader(daaBlock.Header),
GHOSTDAGData: ghostdagDataToDomainGHOSTDAGData(daaBlock.GHOSTDAGData),
}
}
// GHOSTDAGHashPairToDomainGHOSTDAGHashPair converts *BlockGHOSTDAGDataHashPair to *externalapi.BlockGHOSTDAGDataHashPair
func GHOSTDAGHashPairToDomainGHOSTDAGHashPair(datum *BlockGHOSTDAGDataHashPair) *externalapi.BlockGHOSTDAGDataHashPair {
return &externalapi.BlockGHOSTDAGDataHashPair{
Hash: datum.Hash,
GHOSTDAGData: ghostdagDataToDomainGHOSTDAGData(datum.GHOSTDAGData),
}
}
func ghostdagDataToDomainGHOSTDAGData(data *BlockGHOSTDAGData) *externalapi.BlockGHOSTDAGData {
bluesAnticoneSizes := make(map[externalapi.DomainHash]externalapi.KType, len(data.BluesAnticoneSizes))
for _, pair := range data.BluesAnticoneSizes {
@ -500,7 +515,9 @@ func DomainBlockWithTrustedDataToBlockWithTrustedData(block *externalapi.BlockWi
daaWindow := make([]*TrustedDataDataDAABlock, len(block.DAAWindow))
for i, daaBlock := range block.DAAWindow {
daaWindow[i] = &TrustedDataDataDAABlock{
Block: DomainBlockToMsgBlock(daaBlock.Block),
Block: &MsgBlock{
Header: *DomainBlockHeaderToBlockHeader(daaBlock.Header),
},
GHOSTDAGData: domainGHOSTDAGDataGHOSTDAGData(daaBlock.GHOSTDAGData),
}
}
@ -515,7 +532,41 @@ func DomainBlockWithTrustedDataToBlockWithTrustedData(block *externalapi.BlockWi
return &MsgBlockWithTrustedData{
Block: DomainBlockToMsgBlock(block.Block),
DAAScore: block.DAAScore,
DAAScore: block.Block.Header.DAAScore(),
DAAWindow: daaWindow,
GHOSTDAGData: ghostdagData,
}
}
// DomainBlockWithTrustedDataToBlockWithTrustedDataV4 converts a set of *externalapi.DomainBlock, daa window indices and ghostdag data indices
// to *MsgBlockWithTrustedDataV4
func DomainBlockWithTrustedDataToBlockWithTrustedDataV4(block *externalapi.DomainBlock, daaWindowIndices, ghostdagDataIndices []uint64) *MsgBlockWithTrustedDataV4 {
return &MsgBlockWithTrustedDataV4{
Block: DomainBlockToMsgBlock(block),
DAAWindowIndices: daaWindowIndices,
GHOSTDAGDataIndices: ghostdagDataIndices,
}
}
// DomainTrustedDataToTrustedData converts *externalapi.BlockWithTrustedData to *MsgBlockWithTrustedData
func DomainTrustedDataToTrustedData(domainDAAWindow []*externalapi.TrustedDataDataDAAHeader, domainGHOSTDAGData []*externalapi.BlockGHOSTDAGDataHashPair) *MsgTrustedData {
daaWindow := make([]*TrustedDataDAAHeader, len(domainDAAWindow))
for i, daaBlock := range domainDAAWindow {
daaWindow[i] = &TrustedDataDAAHeader{
Header: DomainBlockHeaderToBlockHeader(daaBlock.Header),
GHOSTDAGData: domainGHOSTDAGDataGHOSTDAGData(daaBlock.GHOSTDAGData),
}
}
ghostdagData := make([]*BlockGHOSTDAGDataHashPair, len(domainGHOSTDAGData))
for i, datum := range domainGHOSTDAGData {
ghostdagData[i] = &BlockGHOSTDAGDataHashPair{
Hash: datum.Hash,
GHOSTDAGData: domainGHOSTDAGDataGHOSTDAGData(datum.GHOSTDAGData),
}
}
return &MsgTrustedData{
DAAWindow: daaWindow,
GHOSTDAGData: ghostdagData,
}

View File

@ -67,6 +67,8 @@ const (
CmdRequestPruningPointProof
CmdPruningPointProof
CmdReady
CmdTrustedData
CmdBlockWithTrustedDataV4
// rpc
CmdGetCurrentNetworkRequestMessage
@ -189,6 +191,8 @@ var ProtocolMessageCommandToString = map[MessageCommand]string{
CmdRequestPruningPointProof: "RequestPruningPointProof",
CmdPruningPointProof: "PruningPointProof",
CmdReady: "Ready",
CmdTrustedData: "TrustedData",
CmdBlockWithTrustedDataV4: "BlockWithTrustedDataV4",
}
// RPCMessageCommandToString maps all MessageCommands to their string representation

View File

@ -0,0 +1,20 @@
package appmessage
// MsgBlockWithTrustedDataV4 represents a kaspa BlockWithTrustedDataV4 message
type MsgBlockWithTrustedDataV4 struct {
baseMessage
Block *MsgBlock
DAAWindowIndices []uint64
GHOSTDAGDataIndices []uint64
}
// Command returns the protocol command string for the message
func (msg *MsgBlockWithTrustedDataV4) Command() MessageCommand {
return CmdBlockWithTrustedDataV4
}
// NewMsgBlockWithTrustedDataV4 returns a new MsgBlockWithTrustedDataV4.
func NewMsgBlockWithTrustedDataV4() *MsgBlockWithTrustedDataV4 {
return &MsgBlockWithTrustedDataV4{}
}

View File

@ -0,0 +1,25 @@
package appmessage
// MsgTrustedData represents a kaspa TrustedData message
type MsgTrustedData struct {
baseMessage
DAAWindow []*TrustedDataDAAHeader
GHOSTDAGData []*BlockGHOSTDAGDataHashPair
}
// Command returns the protocol command string for the message
func (msg *MsgTrustedData) Command() MessageCommand {
return CmdTrustedData
}
// NewMsgTrustedData returns a new MsgTrustedData.
func NewMsgTrustedData() *MsgTrustedData {
return &MsgTrustedData{}
}
// TrustedDataDAAHeader is an appmessage representation of externalapi.TrustedDataDataDAAHeader
type TrustedDataDAAHeader struct {
Header *MsgBlockHeader
GHOSTDAGData *BlockGHOSTDAGData
}

View File

@ -0,0 +1,27 @@
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(highHash *externalapi.DomainHash, limit uint32) error {
msgGetBlockLocator := appmessage.NewMsgRequestBlockLocator(highHash, limit)
return flow.outgoingRoute.Enqueue(msgGetBlockLocator)
}
func (flow *handleRelayInvsFlow) receiveBlockLocator() (blockLocatorHashes []*externalapi.DomainHash, err error) {
message, err := flow.dequeueIncomingMessageAndSkipInvs(common.DefaultTimeout)
if err != nil {
return nil, err
}
msgBlockLocator, ok := message.(*appmessage.MsgBlockLocator)
if !ok {
return nil,
protocolerrors.Errorf(true, "received unexpected message type. "+
"expected: %s, got: %s", appmessage.CmdBlockLocator, message.Command())
}
return msgBlockLocator.BlockLocatorHashes, nil
}

View File

@ -0,0 +1,86 @@
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/domain/consensus/model/externalapi"
"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
}
// The IBD block locator is checking only existing blocks with bodies.
if !blockInfo.Exists || blockInfo.BlockStatus == externalapi.StatusHeaderOnly {
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 {
log.Warnf("no hash was found in the blockLocator "+
"that was in the selected parent chain of targetHash %s", targetHash)
ibdBlockLocatorHighestHashNotFoundMessage := appmessage.NewMsgIBDBlockLocatorHighestHashNotFound()
err = outgoingRoute.Enqueue(ibdBlockLocatorHighestHashNotFoundMessage)
if err != nil {
return err
}
}
}
}

View File

@ -0,0 +1,54 @@
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"
)
// HandleIBDBlockRequestsContext is the interface for the context needed for the HandleIBDBlockRequests flow.
type HandleIBDBlockRequestsContext interface {
Domain() domain.Domain
}
// HandleIBDBlockRequests listens to appmessage.MsgRequestRelayBlocks messages and sends
// their corresponding blocks to the requesting peer.
func HandleIBDBlockRequests(context HandleIBDBlockRequestsContext, incomingRoute *router.Route,
outgoingRoute *router.Route) error {
for {
message, err := incomingRoute.Dequeue()
if err != nil {
return err
}
msgRequestIBDBlocks := message.(*appmessage.MsgRequestIBDBlocks)
log.Debugf("Got request for %d ibd blocks", len(msgRequestIBDBlocks.Hashes))
for i, hash := range msgRequestIBDBlocks.Hashes {
// Fetch the block from the database.
blockInfo, err := context.Domain().Consensus().GetBlockInfo(hash)
if err != nil {
return err
}
if !blockInfo.Exists || blockInfo.BlockStatus == externalapi.StatusHeaderOnly {
return protocolerrors.Errorf(true, "block %s not found", hash)
}
block, err := context.Domain().Consensus().GetBlock(hash)
if err != nil {
return errors.Wrapf(err, "unable to fetch requested block hash %s", hash)
}
// TODO (Partial nodes): Convert block to partial block if needed
blockMessage := appmessage.DomainBlockToMsgBlock(block)
ibdBlockMessage := appmessage.NewMsgIBDBlock(blockMessage)
err = outgoingRoute.Enqueue(ibdBlockMessage)
if err != nil {
return err
}
log.Debugf("sent %d out of %d", i+1, len(msgRequestIBDBlocks.Hashes))
}
}
}

View File

@ -0,0 +1,145 @@
package blockrelay
import (
"github.com/kaspanet/kaspad/app/appmessage"
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/config"
"github.com/kaspanet/kaspad/infrastructure/network/netadapter/router"
"sync/atomic"
)
// PruningPointAndItsAnticoneRequestsContext is the interface for the context needed for the HandlePruningPointAndItsAnticoneRequests flow.
type PruningPointAndItsAnticoneRequestsContext interface {
Domain() domain.Domain
Config() *config.Config
}
var isBusy uint32
// HandlePruningPointAndItsAnticoneRequests listens to appmessage.MsgRequestPruningPointAndItsAnticone messages and sends
// the pruning point and its anticone to the requesting peer.
func HandlePruningPointAndItsAnticoneRequests(context PruningPointAndItsAnticoneRequestsContext, incomingRoute *router.Route,
outgoingRoute *router.Route, peer *peerpkg.Peer) error {
for {
err := func() error {
_, err := incomingRoute.Dequeue()
if err != nil {
return err
}
if !atomic.CompareAndSwapUint32(&isBusy, 0, 1) {
return protocolerrors.Errorf(false, "node is busy with other pruning point anticone requests")
}
defer atomic.StoreUint32(&isBusy, 0)
log.Debugf("Got request for pruning point and its anticone from %s", peer)
pruningPointHeaders, err := context.Domain().Consensus().PruningPointHeaders()
if err != nil {
return err
}
msgPruningPointHeaders := make([]*appmessage.MsgBlockHeader, len(pruningPointHeaders))
for i, header := range pruningPointHeaders {
msgPruningPointHeaders[i] = appmessage.DomainBlockHeaderToBlockHeader(header)
}
err = outgoingRoute.Enqueue(appmessage.NewMsgPruningPoints(msgPruningPointHeaders))
if err != nil {
return err
}
pointAndItsAnticone, err := context.Domain().Consensus().PruningPointAndItsAnticone()
if err != nil {
return err
}
windowSize := context.Config().NetParams().DifficultyAdjustmentWindowSize
daaWindowBlocks := make([]*externalapi.TrustedDataDataDAAHeader, 0, windowSize)
daaWindowHashesToIndex := make(map[externalapi.DomainHash]int, windowSize)
trustedDataDAABlockIndexes := make(map[externalapi.DomainHash][]uint64)
ghostdagData := make([]*externalapi.BlockGHOSTDAGDataHashPair, 0)
ghostdagDataHashToIndex := make(map[externalapi.DomainHash]int)
trustedDataGHOSTDAGDataIndexes := make(map[externalapi.DomainHash][]uint64)
for _, blockHash := range pointAndItsAnticone {
blockDAAWindowHashes, err := context.Domain().Consensus().BlockDAAWindowHashes(blockHash)
if err != nil {
return err
}
trustedDataDAABlockIndexes[*blockHash] = make([]uint64, 0, windowSize)
for i, daaBlockHash := range blockDAAWindowHashes {
index, exists := daaWindowHashesToIndex[*daaBlockHash]
if !exists {
trustedDataDataDAAHeader, err := context.Domain().Consensus().TrustedDataDataDAAHeader(blockHash, daaBlockHash, uint64(i))
if err != nil {
return err
}
daaWindowBlocks = append(daaWindowBlocks, trustedDataDataDAAHeader)
index = len(daaWindowBlocks) - 1
daaWindowHashesToIndex[*daaBlockHash] = index
}
trustedDataDAABlockIndexes[*blockHash] = append(trustedDataDAABlockIndexes[*blockHash], uint64(index))
}
ghostdagDataBlockHashes, err := context.Domain().Consensus().TrustedBlockAssociatedGHOSTDAGDataBlockHashes(blockHash)
if err != nil {
return err
}
trustedDataGHOSTDAGDataIndexes[*blockHash] = make([]uint64, 0, context.Config().NetParams().K)
for _, ghostdagDataBlockHash := range ghostdagDataBlockHashes {
index, exists := ghostdagDataHashToIndex[*ghostdagDataBlockHash]
if !exists {
data, err := context.Domain().Consensus().TrustedGHOSTDAGData(ghostdagDataBlockHash)
if err != nil {
return err
}
ghostdagData = append(ghostdagData, &externalapi.BlockGHOSTDAGDataHashPair{
Hash: ghostdagDataBlockHash,
GHOSTDAGData: data,
})
index = len(ghostdagData) - 1
ghostdagDataHashToIndex[*ghostdagDataBlockHash] = index
}
trustedDataGHOSTDAGDataIndexes[*blockHash] = append(trustedDataGHOSTDAGDataIndexes[*blockHash], uint64(index))
}
}
err = outgoingRoute.Enqueue(appmessage.DomainTrustedDataToTrustedData(daaWindowBlocks, ghostdagData))
if err != nil {
return err
}
for _, blockHash := range pointAndItsAnticone {
block, err := context.Domain().Consensus().GetBlock(blockHash)
if err != nil {
return err
}
err = outgoingRoute.Enqueue(appmessage.DomainBlockWithTrustedDataToBlockWithTrustedDataV4(block, trustedDataDAABlockIndexes[*blockHash], trustedDataGHOSTDAGDataIndexes[*blockHash]))
if err != nil {
return err
}
}
err = outgoingRoute.Enqueue(appmessage.NewMsgDoneBlocksWithTrustedData())
if err != nil {
return err
}
log.Debugf("Sent pruning point and its anticone to %s", peer)
return nil
}()
if err != nil {
return err
}
}
}

View File

@ -0,0 +1,40 @@
package blockrelay
import (
"github.com/kaspanet/kaspad/app/appmessage"
peerpkg "github.com/kaspanet/kaspad/app/protocol/peer"
"github.com/kaspanet/kaspad/domain"
"github.com/kaspanet/kaspad/infrastructure/network/netadapter/router"
)
// PruningPointProofRequestsContext is the interface for the context needed for the HandlePruningPointProofRequests flow.
type PruningPointProofRequestsContext interface {
Domain() domain.Domain
}
// HandlePruningPointProofRequests listens to appmessage.MsgRequestPruningPointProof messages and sends
// the pruning point proof to the requesting peer.
func HandlePruningPointProofRequests(context PruningPointProofRequestsContext, incomingRoute *router.Route,
outgoingRoute *router.Route, peer *peerpkg.Peer) error {
for {
_, err := incomingRoute.Dequeue()
if err != nil {
return err
}
log.Debugf("Got request for pruning point proof from %s", peer)
pruningPointProof, err := context.Domain().Consensus().BuildPruningPointProof()
if err != nil {
return err
}
pruningPointProofMessage := appmessage.DomainPruningPointProofToMsgPruningPointProof(pruningPointProof)
err = outgoingRoute.Enqueue(pruningPointProofMessage)
if err != nil {
return err
}
log.Debugf("Sent pruning point proof to %s", peer)
}
}

View File

@ -0,0 +1,53 @@
package blockrelay
import (
"github.com/kaspanet/kaspad/app/appmessage"
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"
)
// RelayBlockRequestsContext is the interface for the context needed for the HandleRelayBlockRequests flow.
type RelayBlockRequestsContext interface {
Domain() domain.Domain
}
// HandleRelayBlockRequests listens to appmessage.MsgRequestRelayBlocks messages and sends
// their corresponding blocks to the requesting peer.
func HandleRelayBlockRequests(context RelayBlockRequestsContext, incomingRoute *router.Route,
outgoingRoute *router.Route, peer *peerpkg.Peer) error {
for {
message, err := incomingRoute.Dequeue()
if err != nil {
return err
}
getRelayBlocksMessage := message.(*appmessage.MsgRequestRelayBlocks)
log.Debugf("Got request for relay blocks with hashes %s", getRelayBlocksMessage.Hashes)
for _, hash := range getRelayBlocksMessage.Hashes {
// Fetch the block from the database.
blockInfo, err := context.Domain().Consensus().GetBlockInfo(hash)
if err != nil {
return err
}
if !blockInfo.Exists || blockInfo.BlockStatus == externalapi.StatusHeaderOnly {
return protocolerrors.Errorf(true, "block %s not found", hash)
}
block, err := context.Domain().Consensus().GetBlock(hash)
if err != nil {
return errors.Wrapf(err, "unable to fetch requested block hash %s", hash)
}
// TODO (Partial nodes): Convert block to partial block if needed
err = outgoingRoute.Enqueue(appmessage.DomainBlockToMsgBlock(block))
if err != nil {
return err
}
log.Debugf("Relayed block with hash %s", hash)
}
}
}

View File

@ -0,0 +1,374 @@
package blockrelay
import (
"github.com/kaspanet/kaspad/app/appmessage"
"github.com/kaspanet/kaspad/app/protocol/common"
"github.com/kaspanet/kaspad/app/protocol/flowcontext"
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"
)
// 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
OnNewBlock(block *externalapi.DomainBlock, virtualChangeSet *externalapi.VirtualChangeSet) error
OnVirtualChange(virtualChangeSet *externalapi.VirtualChangeSet) error
OnPruningPointUTXOSetOverride() error
SharedRequestedBlocks() *flowcontext.SharedRequestedBlocks
Broadcast(message appmessage.Message) error
AddOrphan(orphanBlock *externalapi.DomainBlock)
GetOrphanRoots(orphanHash *externalapi.DomainHash) ([]*externalapi.DomainHash, bool, error)
IsOrphan(blockHash *externalapi.DomainHash) bool
IsIBDRunning() bool
TrySetIBDRunning(ibdPeer *peerpkg.Peer) bool
UnsetIBDRunning()
IsRecoverableError(err error) bool
}
type handleRelayInvsFlow struct {
RelayInvsContext
incomingRoute, outgoingRoute *router.Route
peer *peerpkg.Peer
invsQueue []*appmessage.MsgInvRelayBlock
}
// HandleRelayInvs listens to appmessage.MsgInvRelayBlock messages, requests their corresponding blocks if they
// are missing, adds them to the DAG and propagates them to the rest of the network.
func HandleRelayInvs(context RelayInvsContext, incomingRoute *router.Route, outgoingRoute *router.Route,
peer *peerpkg.Peer) error {
flow := &handleRelayInvsFlow{
RelayInvsContext: context,
incomingRoute: incomingRoute,
outgoingRoute: outgoingRoute,
peer: peer,
invsQueue: make([]*appmessage.MsgInvRelayBlock, 0),
}
return flow.start()
}
func (flow *handleRelayInvsFlow) start() error {
for {
log.Debugf("Waiting for inv")
inv, err := flow.readInv()
if err != nil {
return err
}
log.Debugf("Got relay inv for block %s", inv.Hash)
blockInfo, err := flow.Domain().Consensus().GetBlockInfo(inv.Hash)
if err != nil {
return err
}
if blockInfo.Exists && blockInfo.BlockStatus != externalapi.StatusHeaderOnly {
if blockInfo.BlockStatus == externalapi.StatusInvalid {
return protocolerrors.Errorf(true, "sent inv of an invalid block %s",
inv.Hash)
}
log.Debugf("Block %s already exists. continuing...", inv.Hash)
continue
}
isGenesisVirtualSelectedParent, err := flow.isGenesisVirtualSelectedParent()
if err != nil {
return err
}
if flow.IsOrphan(inv.Hash) {
if flow.Config().NetParams().DisallowDirectBlocksOnTopOfGenesis && !flow.Config().AllowSubmitBlockWhenNotSynced && isGenesisVirtualSelectedParent {
log.Infof("Cannot process orphan %s for a node with only the genesis block. The node needs to IBD "+
"to the recent pruning point before normal operation can resume.", inv.Hash)
continue
}
log.Debugf("Block %s is a known orphan. Requesting its missing ancestors", inv.Hash)
err := flow.AddOrphanRootsToQueue(inv.Hash)
if err != nil {
return err
}
continue
}
// Block relay is disabled during IBD
if flow.IsIBDRunning() {
log.Debugf("Got block %s while in IBD. continuing...", inv.Hash)
continue
}
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
}
err = flow.banIfBlockIsHeaderOnly(block)
if err != nil {
return err
}
if flow.Config().NetParams().DisallowDirectBlocksOnTopOfGenesis && !flow.Config().AllowSubmitBlockWhenNotSynced && !flow.Config().Devnet && flow.isChildOfGenesis(block) {
log.Infof("Cannot process %s because it's a direct child of genesis.", consensushashing.BlockHash(block))
continue
}
log.Debugf("Processing block %s", inv.Hash)
missingParents, virtualChangeSet, err := flow.processBlock(block)
if err != nil {
if errors.Is(err, ruleerrors.ErrPrunedBlock) {
log.Infof("Ignoring pruned block %s", inv.Hash)
continue
}
if errors.Is(err, ruleerrors.ErrDuplicateBlock) {
log.Infof("Ignoring duplicate block %s", inv.Hash)
continue
}
return err
}
if len(missingParents) > 0 {
log.Debugf("Block %s is orphan and has missing parents: %s", inv.Hash, missingParents)
err := flow.processOrphan(block)
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, virtualChangeSet)
if err != nil {
return err
}
}
}
func (flow *handleRelayInvsFlow) banIfBlockIsHeaderOnly(block *externalapi.DomainBlock) error {
if len(block.Transactions) == 0 {
return protocolerrors.Errorf(true, "sent header of %s block where expected block with body",
consensushashing.BlockHash(block))
}
return nil
}
func (flow *handleRelayInvsFlow) readInv() (*appmessage.MsgInvRelayBlock, error) {
if len(flow.invsQueue) > 0 {
var inv *appmessage.MsgInvRelayBlock
inv, flow.invsQueue = flow.invsQueue[0], flow.invsQueue[1:]
return inv, nil
}
msg, err := flow.incomingRoute.Dequeue()
if err != nil {
return nil, err
}
inv, ok := msg.(*appmessage.MsgInvRelayBlock)
if !ok {
return nil, protocolerrors.Errorf(true, "unexpected %s message in the block relay handleRelayInvsFlow while "+
"expecting an inv message", msg.Command())
}
return inv, 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().Remove(requestHash)
getRelayBlocksMsg := appmessage.NewMsgRequestRelayBlocks([]*externalapi.DomainHash{requestHash})
err := flow.outgoingRoute.Enqueue(getRelayBlocksMsg)
if err != nil {
return nil, false, err
}
msgBlock, err := flow.readMsgBlock()
if err != nil {
return nil, false, err
}
block := appmessage.MsgBlockToDomainBlock(msgBlock)
blockHash := consensushashing.BlockHash(block)
if !blockHash.Equal(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.
//
// Note: this function assumes msgChan can contain only appmessage.MsgInvRelayBlock and appmessage.MsgBlock messages.
func (flow *handleRelayInvsFlow) readMsgBlock() (msgBlock *appmessage.MsgBlock, err error) {
for {
message, err := flow.incomingRoute.DequeueWithTimeout(common.DefaultTimeout)
if err != nil {
return nil, err
}
switch message := message.(type) {
case *appmessage.MsgInvRelayBlock:
flow.invsQueue = append(flow.invsQueue, message)
case *appmessage.MsgBlock:
return message, nil
default:
return nil, errors.Errorf("unexpected message %s", message.Command())
}
}
}
func (flow *handleRelayInvsFlow) processBlock(block *externalapi.DomainBlock) ([]*externalapi.DomainHash, *externalapi.VirtualChangeSet, error) {
blockHash := consensushashing.BlockHash(block)
virtualChangeSet, err := flow.Domain().Consensus().ValidateAndInsertBlock(block, true)
if err != nil {
if !errors.As(err, &ruleerrors.RuleError{}) {
return nil, nil, errors.Wrapf(err, "failed to process block %s", blockHash)
}
missingParentsError := &ruleerrors.ErrMissingParents{}
if errors.As(err, missingParentsError) {
return missingParentsError.MissingParentHashes, nil, nil
}
log.Warnf("Rejected block %s from %s: %s", blockHash, flow.peer, err)
return nil, nil, protocolerrors.Wrapf(true, err, "got invalid block %s from relay", blockHash)
}
return nil, virtualChangeSet, 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) 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 {
if flow.Config().NetParams().DisallowDirectBlocksOnTopOfGenesis && !flow.Config().AllowSubmitBlockWhenNotSynced {
isGenesisVirtualSelectedParent, err := flow.isGenesisVirtualSelectedParent()
if err != nil {
return err
}
if isGenesisVirtualSelectedParent {
log.Infof("Cannot process orphan %s for a node with only the genesis block. The node needs to IBD "+
"to the recent pruning point before normal operation can resume.", blockHash)
return nil
}
}
log.Debugf("Block %s is within orphan resolution range. "+
"Adding it to the orphan set", blockHash)
flow.AddOrphan(block)
log.Debugf("Requesting block %s missing ancestors", blockHash)
return flow.AddOrphanRootsToQueue(blockHash)
}
// Start IBD unless we already are in IBD
log.Debugf("Block %s is out of orphan resolution range. "+
"Attempting to start IBD against it.", blockHash)
return flow.runIBDIfNotRunning(block)
}
func (flow *handleRelayInvsFlow) isGenesisVirtualSelectedParent() (bool, error) {
virtualSelectedParent, err := flow.Domain().Consensus().GetVirtualSelectedParent()
if err != nil {
return false, err
}
return virtualSelectedParent.Equal(flow.Config().NetParams().GenesisHash), nil
}
func (flow *handleRelayInvsFlow) isChildOfGenesis(block *externalapi.DomainBlock) bool {
parents := block.Header.DirectParents()
return len(parents) == 1 && parents[0].Equal(flow.Config().NetParams().GenesisHash)
}
// 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) {
err := flow.sendGetBlockLocator(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) AddOrphanRootsToQueue(orphan *externalapi.DomainHash) error {
orphanRoots, orphanExists, err := flow.GetOrphanRoots(orphan)
if err != nil {
return err
}
if !orphanExists {
log.Infof("Orphan block %s was missing from the orphan pool while requesting for its roots. This "+
"probably happened because it was randomly evicted immediately after it was added.", orphan)
}
log.Infof("Block %s has %d missing ancestors. Adding them to the invs queue...", orphan, len(orphanRoots))
invMessages := make([]*appmessage.MsgInvRelayBlock, len(orphanRoots))
for i, root := range orphanRoots {
log.Debugf("Adding block %s missing ancestor %s to the invs queue", orphan, root)
invMessages[i] = appmessage.NewMsgInvBlock(root)
}
flow.invsQueue = append(invMessages, flow.invsQueue...)
return nil
}

View File

@ -0,0 +1,75 @@
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"
)
// RequestBlockLocatorContext is the interface for the context needed for the HandleRequestBlockLocator flow.
type RequestBlockLocatorContext interface {
Domain() domain.Domain
}
type handleRequestBlockLocatorFlow struct {
RequestBlockLocatorContext
incomingRoute, outgoingRoute *router.Route
}
// HandleRequestBlockLocator handles getBlockLocator messages
func HandleRequestBlockLocator(context RequestBlockLocatorContext, incomingRoute *router.Route,
outgoingRoute *router.Route) error {
flow := &handleRequestBlockLocatorFlow{
RequestBlockLocatorContext: context,
incomingRoute: incomingRoute,
outgoingRoute: outgoingRoute,
}
return flow.start()
}
func (flow *handleRequestBlockLocatorFlow) start() error {
for {
highHash, limit, err := flow.receiveGetBlockLocator()
if err != nil {
return err
}
log.Debugf("Received getBlockLocator with highHash: %s, limit: %d", highHash, limit)
locator, err := flow.Domain().Consensus().CreateBlockLocatorFromPruningPoint(highHash, limit)
if err != nil || len(locator) == 0 {
if err != nil {
log.Debugf("Received error from CreateBlockLocatorFromPruningPoint: %s", err)
}
return protocolerrors.Errorf(true, "couldn't build a block "+
"locator between the pruning point and %s", highHash)
}
err = flow.sendBlockLocator(locator)
if err != nil {
return err
}
}
}
func (flow *handleRequestBlockLocatorFlow) receiveGetBlockLocator() (highHash *externalapi.DomainHash, limit uint32, err error) {
message, err := flow.incomingRoute.Dequeue()
if err != nil {
return nil, 0, err
}
msgGetBlockLocator := message.(*appmessage.MsgRequestBlockLocator)
return msgGetBlockLocator.HighHash, msgGetBlockLocator.Limit, nil
}
func (flow *handleRequestBlockLocatorFlow) sendBlockLocator(locator externalapi.BlockLocator) error {
msgBlockLocator := appmessage.NewMsgBlockLocator(locator)
err := flow.outgoingRoute.Enqueue(msgBlockLocator)
if err != nil {
return err
}
return nil
}

View File

@ -0,0 +1,105 @@
package blockrelay
import (
"github.com/kaspanet/kaspad/app/protocol/peer"
"github.com/kaspanet/kaspad/app/protocol/protocolerrors"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/kaspanet/kaspad/app/appmessage"
"github.com/kaspanet/kaspad/domain"
"github.com/kaspanet/kaspad/infrastructure/network/netadapter/router"
)
const ibdBatchSize = router.DefaultMaxMessages
// RequestHeadersContext is the interface for the context needed for the HandleRequestHeaders flow.
type RequestHeadersContext interface {
Domain() domain.Domain
}
type handleRequestHeadersFlow struct {
RequestHeadersContext
incomingRoute, outgoingRoute *router.Route
peer *peer.Peer
}
// HandleRequestHeaders handles RequestHeaders messages
func HandleRequestHeaders(context RequestHeadersContext, incomingRoute *router.Route,
outgoingRoute *router.Route, peer *peer.Peer) error {
flow := &handleRequestHeadersFlow{
RequestHeadersContext: context,
incomingRoute: incomingRoute,
outgoingRoute: outgoingRoute,
peer: peer,
}
return flow.start()
}
func (flow *handleRequestHeadersFlow) start() error {
for {
lowHash, highHash, err := receiveRequestHeaders(flow.incomingRoute)
if err != nil {
return err
}
log.Debugf("Recieved requestHeaders with lowHash: %s, highHash: %s", lowHash, highHash)
for !lowHash.Equal(highHash) {
log.Debugf("Getting block headers between %s and %s to %s", lowHash, highHash, flow.peer)
// GetHashesBetween is a relatively heavy operation so we limit it
// in order to avoid locking the consensus for too long
// maxBlocks MUST be >= MergeSetSizeLimit + 1
const maxBlocks = 1 << 10
blockHashes, _, err := flow.Domain().Consensus().GetHashesBetween(lowHash, highHash, maxBlocks)
if err != nil {
return err
}
log.Debugf("Got %d header hashes above lowHash %s", len(blockHashes), lowHash)
blockHeaders := make([]*appmessage.MsgBlockHeader, len(blockHashes))
for i, blockHash := range blockHashes {
blockHeader, err := flow.Domain().Consensus().GetBlockHeader(blockHash)
if err != nil {
return err
}
blockHeaders[i] = appmessage.DomainBlockHeaderToBlockHeader(blockHeader)
}
blockHeadersMessage := appmessage.NewBlockHeadersMessage(blockHeaders)
err = flow.outgoingRoute.Enqueue(blockHeadersMessage)
if err != nil {
return err
}
message, err := flow.incomingRoute.Dequeue()
if err != nil {
return err
}
if _, ok := message.(*appmessage.MsgRequestNextHeaders); !ok {
return protocolerrors.Errorf(true, "received unexpected message type. "+
"expected: %s, got: %s", appmessage.CmdRequestNextHeaders, message.Command())
}
// The next lowHash is the last element in blockHashes
lowHash = blockHashes[len(blockHashes)-1]
}
err = flow.outgoingRoute.Enqueue(appmessage.NewMsgDoneHeaders())
if err != nil {
return err
}
}
}
func receiveRequestHeaders(incomingRoute *router.Route) (lowHash *externalapi.DomainHash,
highHash *externalapi.DomainHash, err error) {
message, err := incomingRoute.Dequeue()
if err != nil {
return nil, nil, err
}
msgRequestIBDBlocks := message.(*appmessage.MsgRequestHeaders)
return msgRequestIBDBlocks.LowHash, msgRequestIBDBlocks.HighHash, nil
}

View File

@ -0,0 +1,140 @@
package blockrelay
import (
"errors"
"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"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/kaspanet/kaspad/domain/consensus/ruleerrors"
"github.com/kaspanet/kaspad/infrastructure/logger"
"github.com/kaspanet/kaspad/infrastructure/network/netadapter/router"
)
// HandleRequestPruningPointUTXOSetContext is the interface for the context needed for the HandleRequestPruningPointUTXOSet flow.
type HandleRequestPruningPointUTXOSetContext interface {
Domain() domain.Domain
}
type handleRequestPruningPointUTXOSetFlow struct {
HandleRequestPruningPointUTXOSetContext
incomingRoute, outgoingRoute *router.Route
}
// HandleRequestPruningPointUTXOSet listens to appmessage.MsgRequestPruningPointUTXOSet messages and sends
// the pruning point UTXO set and block body.
func HandleRequestPruningPointUTXOSet(context HandleRequestPruningPointUTXOSetContext, incomingRoute,
outgoingRoute *router.Route) error {
flow := &handleRequestPruningPointUTXOSetFlow{
HandleRequestPruningPointUTXOSetContext: context,
incomingRoute: incomingRoute,
outgoingRoute: outgoingRoute,
}
return flow.start()
}
func (flow *handleRequestPruningPointUTXOSetFlow) start() error {
for {
msgRequestPruningPointUTXOSet, err := flow.waitForRequestPruningPointUTXOSetMessages()
if err != nil {
return err
}
err = flow.handleRequestPruningPointUTXOSetMessage(msgRequestPruningPointUTXOSet)
if err != nil {
return err
}
}
}
func (flow *handleRequestPruningPointUTXOSetFlow) handleRequestPruningPointUTXOSetMessage(
msgRequestPruningPointUTXOSet *appmessage.MsgRequestPruningPointUTXOSet) error {
onEnd := logger.LogAndMeasureExecutionTime(log, "handleRequestPruningPointUTXOSetFlow")
defer onEnd()
log.Debugf("Got request for pruning point UTXO set")
return flow.sendPruningPointUTXOSet(msgRequestPruningPointUTXOSet)
}
func (flow *handleRequestPruningPointUTXOSetFlow) waitForRequestPruningPointUTXOSetMessages() (
*appmessage.MsgRequestPruningPointUTXOSet, error) {
message, err := flow.incomingRoute.Dequeue()
if err != nil {
return nil, err
}
msgRequestPruningPointUTXOSet, ok := message.(*appmessage.MsgRequestPruningPointUTXOSet)
if !ok {
// TODO: Change to shouldBan: true once we fix the bug of getting redundant messages
return nil, protocolerrors.Errorf(false, "received unexpected message type. "+
"expected: %s, got: %s", appmessage.CmdRequestPruningPointUTXOSet, message.Command())
}
return msgRequestPruningPointUTXOSet, nil
}
func (flow *handleRequestPruningPointUTXOSetFlow) sendPruningPointUTXOSet(
msgRequestPruningPointUTXOSet *appmessage.MsgRequestPruningPointUTXOSet) error {
// Send the UTXO set in `step`-sized chunks
const step = 1000
var fromOutpoint *externalapi.DomainOutpoint
chunksSent := 0
for {
pruningPointUTXOs, err := flow.Domain().Consensus().GetPruningPointUTXOs(
msgRequestPruningPointUTXOSet.PruningPointHash, fromOutpoint, step)
if err != nil {
if errors.Is(err, ruleerrors.ErrWrongPruningPointHash) {
return flow.outgoingRoute.Enqueue(appmessage.NewMsgUnexpectedPruningPoint())
}
}
log.Debugf("Retrieved %d UTXOs for pruning block %s",
len(pruningPointUTXOs), msgRequestPruningPointUTXOSet.PruningPointHash)
outpointAndUTXOEntryPairs :=
appmessage.DomainOutpointAndUTXOEntryPairsToOutpointAndUTXOEntryPairs(pruningPointUTXOs)
err = flow.outgoingRoute.Enqueue(appmessage.NewMsgPruningPointUTXOSetChunk(outpointAndUTXOEntryPairs))
if err != nil {
return err
}
finished := len(pruningPointUTXOs) < step
if finished && chunksSent%ibdBatchSize != 0 {
log.Debugf("Finished sending UTXOs for pruning block %s",
msgRequestPruningPointUTXOSet.PruningPointHash)
return flow.outgoingRoute.Enqueue(appmessage.NewMsgDonePruningPointUTXOSetChunks())
}
if len(pruningPointUTXOs) > 0 {
fromOutpoint = pruningPointUTXOs[len(pruningPointUTXOs)-1].Outpoint
}
chunksSent++
// Wait for the peer to request more chunks every `ibdBatchSize` chunks
if chunksSent%ibdBatchSize == 0 {
message, err := flow.incomingRoute.DequeueWithTimeout(common.DefaultTimeout)
if err != nil {
return err
}
_, ok := message.(*appmessage.MsgRequestNextPruningPointUTXOSetChunk)
if !ok {
// TODO: Change to shouldBan: true once we fix the bug of getting redundant messages
return protocolerrors.Errorf(false, "received unexpected message type. "+
"expected: %s, got: %s", appmessage.CmdRequestNextPruningPointUTXOSetChunk, message.Command())
}
if finished {
log.Debugf("Finished sending UTXOs for pruning block %s",
msgRequestPruningPointUTXOSet.PruningPointHash)
return flow.outgoingRoute.Enqueue(appmessage.NewMsgDonePruningPointUTXOSetChunks())
}
}
}
}

View File

@ -0,0 +1,527 @@
package blockrelay
import (
"time"
"github.com/kaspanet/kaspad/infrastructure/logger"
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/app/appmessage"
"github.com/kaspanet/kaspad/app/protocol/common"
"github.com/kaspanet/kaspad/app/protocol/protocolerrors"
"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/pkg/errors"
)
func (flow *handleRelayInvsFlow) runIBDIfNotRunning(block *externalapi.DomainBlock) error {
wasIBDNotRunning := flow.TrySetIBDRunning(flow.peer)
if !wasIBDNotRunning {
log.Debugf("IBD is already running")
return nil
}
isFinishedSuccessfully := false
defer func() {
flow.UnsetIBDRunning()
flow.logIBDFinished(isFinishedSuccessfully)
}()
highHash := consensushashing.BlockHash(block)
log.Debugf("IBD started with peer %s and highHash %s", flow.peer, highHash)
log.Debugf("Syncing blocks up to %s", highHash)
log.Debugf("Trying to find highest shared chain block with peer %s with high hash %s", flow.peer, highHash)
highestSharedBlockHash, highestSharedBlockFound, err := flow.findHighestSharedBlockHash(highHash)
if err != nil {
return err
}
log.Debugf("Found highest shared chain block %s with peer %s", highestSharedBlockHash, flow.peer)
shouldDownloadHeadersProof, shouldSync, err := flow.shouldSyncAndShouldDownloadHeadersProof(block, highestSharedBlockFound)
if err != nil {
return err
}
if !shouldSync {
return nil
}
if shouldDownloadHeadersProof {
log.Infof("Starting IBD with headers proof")
err := flow.ibdWithHeadersProof(highHash)
if err != nil {
return err
}
} else {
if flow.Config().NetParams().DisallowDirectBlocksOnTopOfGenesis && !flow.Config().AllowSubmitBlockWhenNotSynced {
isGenesisVirtualSelectedParent, err := flow.isGenesisVirtualSelectedParent()
if err != nil {
return err
}
if isGenesisVirtualSelectedParent {
log.Infof("Cannot IBD to %s because it won't change the pruning point. The node needs to IBD "+
"to the recent pruning point before normal operation can resume.", highHash)
return nil
}
}
err = flow.syncPruningPointFutureHeaders(flow.Domain().Consensus(), highestSharedBlockHash, highHash)
if err != nil {
return err
}
}
err = flow.syncMissingBlockBodies(highHash)
if err != nil {
return err
}
log.Debugf("Finished syncing blocks up to %s", highHash)
isFinishedSuccessfully = true
return nil
}
func (flow *handleRelayInvsFlow) logIBDFinished(isFinishedSuccessfully bool) {
successString := "successfully"
if !isFinishedSuccessfully {
successString = "(interrupted)"
}
log.Infof("IBD finished %s", successString)
}
// findHighestSharedBlock attempts to find the highest shared block between the peer
// and this node. This method may fail because the peer and us have conflicting pruning
// points. In that case we return (nil, false, nil) so that we may stop IBD gracefully.
func (flow *handleRelayInvsFlow) findHighestSharedBlockHash(
targetHash *externalapi.DomainHash) (*externalapi.DomainHash, bool, error) {
log.Debugf("Sending a blockLocator to %s between pruning point and headers selected tip", flow.peer)
blockLocator, err := flow.Domain().Consensus().CreateFullHeadersSelectedChainBlockLocator()
if err != nil {
return nil, false, err
}
for {
highestHash, highestHashFound, err := flow.fetchHighestHash(targetHash, blockLocator)
if err != nil {
return nil, false, err
}
if !highestHashFound {
return nil, false, nil
}
highestHashIndex, err := flow.findHighestHashIndex(highestHash, blockLocator)
if err != nil {
return nil, false, err
}
if highestHashIndex == 0 ||
// If the block locator contains only two adjacent chain blocks, the
// syncer will always find the same highest chain block, so to avoid
// an endless loop, we explicitly stop the loop in such situation.
(len(blockLocator) == 2 && highestHashIndex == 1) {
return highestHash, true, nil
}
locatorHashAboveHighestHash := highestHash
if highestHashIndex > 0 {
locatorHashAboveHighestHash = blockLocator[highestHashIndex-1]
}
blockLocator, err = flow.nextBlockLocator(highestHash, locatorHashAboveHighestHash)
if err != nil {
return nil, false, err
}
}
}
func (flow *handleRelayInvsFlow) nextBlockLocator(lowHash, highHash *externalapi.DomainHash) (externalapi.BlockLocator, error) {
log.Debugf("Sending a blockLocator to %s between %s and %s", flow.peer, lowHash, highHash)
blockLocator, err := flow.Domain().Consensus().CreateHeadersSelectedChainBlockLocator(lowHash, highHash)
if err != nil {
if errors.Is(model.ErrBlockNotInSelectedParentChain, err) {
return nil, err
}
log.Debugf("Headers selected parent chain moved since findHighestSharedBlockHash - " +
"restarting with full block locator")
blockLocator, err = flow.Domain().Consensus().CreateFullHeadersSelectedChainBlockLocator()
if err != nil {
return nil, err
}
}
return blockLocator, nil
}
func (flow *handleRelayInvsFlow) findHighestHashIndex(
highestHash *externalapi.DomainHash, blockLocator externalapi.BlockLocator) (int, error) {
highestHashIndex := 0
highestHashIndexFound := false
for i, blockLocatorHash := range blockLocator {
if highestHash.Equal(blockLocatorHash) {
highestHashIndex = i
highestHashIndexFound = true
break
}
}
if !highestHashIndexFound {
return 0, 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)
return highestHashIndex, nil
}
// fetchHighestHash attempts to fetch the highest hash the peer knows amongst the given
// blockLocator. This method may fail because the peer and us have conflicting pruning
// points. In that case we return (nil, false, nil) so that we may stop IBD gracefully.
func (flow *handleRelayInvsFlow) fetchHighestHash(
targetHash *externalapi.DomainHash, blockLocator externalapi.BlockLocator) (*externalapi.DomainHash, bool, error) {
ibdBlockLocatorMessage := appmessage.NewMsgIBDBlockLocator(targetHash, blockLocator)
err := flow.outgoingRoute.Enqueue(ibdBlockLocatorMessage)
if err != nil {
return nil, false, err
}
message, err := flow.dequeueIncomingMessageAndSkipInvs(common.DefaultTimeout)
if err != nil {
return nil, false, err
}
switch message := message.(type) {
case *appmessage.MsgIBDBlockLocatorHighestHash:
highestHash := message.HighestHash
log.Debugf("The highest hash the peer %s knows is %s", flow.peer, highestHash)
return highestHash, true, nil
case *appmessage.MsgIBDBlockLocatorHighestHashNotFound:
log.Debugf("Peer %s does not know any block within our blockLocator. "+
"This should only happen if there's a DAG split deeper than the pruning point.", flow.peer)
return nil, false, nil
default:
return nil, false, protocolerrors.Errorf(true, "received unexpected message type. "+
"expected: %s, got: %s", appmessage.CmdIBDBlockLocatorHighestHash, message.Command())
}
}
func (flow *handleRelayInvsFlow) syncPruningPointFutureHeaders(consensus externalapi.Consensus, highestSharedBlockHash *externalapi.DomainHash,
highHash *externalapi.DomainHash) error {
log.Infof("Downloading headers from %s", flow.peer)
err := flow.sendRequestHeaders(highestSharedBlockHash, highHash)
if err != nil {
return err
}
// Keep a short queue of BlockHeadersMessages so that there's
// never a moment when the node is not validating and inserting
// headers
blockHeadersMessageChan := make(chan *appmessage.BlockHeadersMessage, 2)
errChan := make(chan error)
spawn("handleRelayInvsFlow-syncPruningPointFutureHeaders", func() {
for {
blockHeadersMessage, doneIBD, err := flow.receiveHeaders()
if err != nil {
errChan <- err
return
}
if doneIBD {
close(blockHeadersMessageChan)
return
}
blockHeadersMessageChan <- blockHeadersMessage
err = flow.outgoingRoute.Enqueue(appmessage.NewMsgRequestNextHeaders())
if err != nil {
errChan <- err
return
}
}
})
for {
select {
case ibdBlocksMessage, ok := <-blockHeadersMessageChan:
if !ok {
// If the highHash has not been received, the peer is misbehaving
highHashBlockInfo, err := consensus.GetBlockInfo(highHash)
if err != nil {
return err
}
if !highHashBlockInfo.Exists {
return protocolerrors.Errorf(true, "did not receive "+
"highHash block %s from peer %s during block download", highHash, flow.peer)
}
return nil
}
for _, header := range ibdBlocksMessage.BlockHeaders {
err = flow.processHeader(consensus, header)
if err != nil {
return err
}
}
case err := <-errChan:
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) receiveHeaders() (msgIBDBlock *appmessage.BlockHeadersMessage, doneHeaders bool, err error) {
message, err := flow.dequeueIncomingMessageAndSkipInvs(common.DefaultTimeout)
if err != nil {
return nil, false, err
}
switch message := message.(type) {
case *appmessage.BlockHeadersMessage:
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.CmdBlockHeaders,
appmessage.CmdDoneHeaders,
message.Command())
}
}
func (flow *handleRelayInvsFlow) processHeader(consensus externalapi.Consensus, msgBlockHeader *appmessage.MsgBlockHeader) error {
header := appmessage.BlockHeaderToDomainBlockHeader(msgBlockHeader)
block := &externalapi.DomainBlock{
Header: header,
Transactions: nil,
}
blockHash := consensushashing.BlockHash(block)
blockInfo, err := 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 = consensus.ValidateAndInsertBlock(block, false)
if err != nil {
if !errors.As(err, &ruleerrors.RuleError{}) {
return errors.Wrapf(err, "failed to process header %s during IBD", blockHash)
}
if errors.Is(err, ruleerrors.ErrDuplicateBlock) {
log.Debugf("Skipping block header %s as it is a duplicate", blockHash)
} else {
log.Infof("Rejected block header %s from %s during IBD: %s", blockHash, flow.peer, err)
return protocolerrors.Wrapf(true, err, "got invalid block header %s during IBD", blockHash)
}
}
return nil
}
func (flow *handleRelayInvsFlow) validatePruningPointFutureHeaderTimestamps() error {
headerSelectedTipHash, err := flow.Domain().StagingConsensus().GetHeadersSelectedTip()
if err != nil {
return err
}
headerSelectedTipHeader, err := flow.Domain().StagingConsensus().GetBlockHeader(headerSelectedTipHash)
if err != nil {
return err
}
headerSelectedTipTimestamp := headerSelectedTipHeader.TimeInMilliseconds()
currentSelectedTipHash, err := flow.Domain().Consensus().GetHeadersSelectedTip()
if err != nil {
return err
}
currentSelectedTipHeader, err := flow.Domain().Consensus().GetBlockHeader(currentSelectedTipHash)
if err != nil {
return err
}
currentSelectedTipTimestamp := currentSelectedTipHeader.TimeInMilliseconds()
if headerSelectedTipTimestamp < currentSelectedTipTimestamp {
return protocolerrors.Errorf(false, "the timestamp of the candidate selected "+
"tip is smaller than the current selected tip")
}
minTimestampDifferenceInMilliseconds := (10 * time.Minute).Milliseconds()
if headerSelectedTipTimestamp-currentSelectedTipTimestamp < minTimestampDifferenceInMilliseconds {
return protocolerrors.Errorf(false, "difference between the timestamps of "+
"the current pruning point and the candidate pruning point is too small. Aborting IBD...")
}
return nil
}
func (flow *handleRelayInvsFlow) receiveAndInsertPruningPointUTXOSet(
consensus externalapi.Consensus, pruningPointHash *externalapi.DomainHash) (bool, error) {
onEnd := logger.LogAndMeasureExecutionTime(log, "receiveAndInsertPruningPointUTXOSet")
defer onEnd()
receivedChunkCount := 0
receivedUTXOCount := 0
for {
message, err := flow.dequeueIncomingMessageAndSkipInvs(common.DefaultTimeout)
if err != nil {
return false, err
}
switch message := message.(type) {
case *appmessage.MsgPruningPointUTXOSetChunk:
receivedUTXOCount += len(message.OutpointAndUTXOEntryPairs)
domainOutpointAndUTXOEntryPairs :=
appmessage.OutpointAndUTXOEntryPairsToDomainOutpointAndUTXOEntryPairs(message.OutpointAndUTXOEntryPairs)
err := consensus.AppendImportedPruningPointUTXOs(domainOutpointAndUTXOEntryPairs)
if err != nil {
return false, err
}
receivedChunkCount++
if receivedChunkCount%ibdBatchSize == 0 {
log.Debugf("Received %d UTXO set chunks so far, totaling in %d UTXOs",
receivedChunkCount, receivedUTXOCount)
requestNextPruningPointUTXOSetChunkMessage := appmessage.NewMsgRequestNextPruningPointUTXOSetChunk()
err := flow.outgoingRoute.Enqueue(requestNextPruningPointUTXOSetChunkMessage)
if err != nil {
return false, err
}
}
case *appmessage.MsgDonePruningPointUTXOSetChunks:
log.Infof("Finished receiving the UTXO set. Total UTXOs: %d", receivedUTXOCount)
return true, nil
case *appmessage.MsgUnexpectedPruningPoint:
log.Infof("Could not receive the next UTXO chunk because the pruning point %s "+
"is no longer the pruning point of peer %s", pruningPointHash, flow.peer)
return false, nil
default:
return false, protocolerrors.Errorf(true, "received unexpected message type. "+
"expected: %s or %s or %s, got: %s", appmessage.CmdPruningPointUTXOSetChunk,
appmessage.CmdDonePruningPointUTXOSetChunks, appmessage.CmdUnexpectedPruningPoint, message.Command(),
)
}
}
}
func (flow *handleRelayInvsFlow) syncMissingBlockBodies(highHash *externalapi.DomainHash) error {
hashes, err := flow.Domain().Consensus().GetMissingBlockBodyHashes(highHash)
if err != nil {
return err
}
if len(hashes) == 0 {
// Blocks can be inserted inside the DAG during IBD if those were requested before IBD started.
// In rare cases, all the IBD blocks might be already inserted by the time we reach this point.
// In these cases - GetMissingBlockBodyHashes would return an empty array.
log.Debugf("No missing block body hashes found.")
return nil
}
for offset := 0; offset < len(hashes); offset += ibdBatchSize {
var hashesToRequest []*externalapi.DomainHash
if offset+ibdBatchSize < len(hashes) {
hashesToRequest = hashes[offset : offset+ibdBatchSize]
} else {
hashesToRequest = hashes[offset:]
}
err := flow.outgoingRoute.Enqueue(appmessage.NewMsgRequestIBDBlocks(hashesToRequest))
if err != nil {
return err
}
for _, expectedHash := range hashesToRequest {
message, err := flow.dequeueIncomingMessageAndSkipInvs(common.DefaultTimeout)
if err != nil {
return err
}
msgIBDBlock, ok := message.(*appmessage.MsgIBDBlock)
if !ok {
return protocolerrors.Errorf(true, "received unexpected message type. "+
"expected: %s, got: %s", appmessage.CmdIBDBlock, message.Command())
}
block := appmessage.MsgBlockToDomainBlock(msgIBDBlock.MsgBlock)
blockHash := consensushashing.BlockHash(block)
if !expectedHash.Equal(blockHash) {
return protocolerrors.Errorf(true, "expected block %s but got %s", expectedHash, blockHash)
}
err = flow.banIfBlockIsHeaderOnly(block)
if err != nil {
return err
}
virtualChangeSet, err := flow.Domain().Consensus().ValidateAndInsertBlock(block, false)
if err != nil {
if errors.Is(err, ruleerrors.ErrDuplicateBlock) {
log.Debugf("Skipping IBD Block %s as it has already been added to the DAG", blockHash)
continue
}
return protocolerrors.ConvertToBanningProtocolErrorIfRuleError(err, "invalid block %s", blockHash)
}
err = flow.OnNewBlock(block, virtualChangeSet)
if err != nil {
return err
}
}
}
return flow.resolveVirtual()
}
func (flow *handleRelayInvsFlow) resolveVirtual() error {
for i := 0; ; i++ {
if i%10 == 0 {
log.Infof("Resolving virtual. This may take some time...")
}
virtualChangeSet, isCompletelyResolved, err := flow.Domain().Consensus().ResolveVirtual()
if err != nil {
return err
}
err = flow.OnVirtualChange(virtualChangeSet)
if err != nil {
return err
}
if isCompletelyResolved {
log.Infof("Resolved virtual")
return nil
}
}
}
// dequeueIncomingMessageAndSkipInvs is a convenience method to be used during
// IBD. Inv messages are expected to arrive at any given moment, but should be
// ignored while we're in IBD
func (flow *handleRelayInvsFlow) dequeueIncomingMessageAndSkipInvs(timeout time.Duration) (appmessage.Message, error) {
for {
message, err := flow.incomingRoute.DequeueWithTimeout(timeout)
if err != nil {
return nil, err
}
if _, ok := message.(*appmessage.MsgInvRelayBlock); !ok {
return message, nil
}
}
}

View File

@ -0,0 +1,389 @@
package blockrelay
import (
"fmt"
"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"
"github.com/kaspanet/kaspad/domain/consensus/ruleerrors"
"github.com/kaspanet/kaspad/domain/consensus/utils/consensushashing"
"github.com/pkg/errors"
)
func (flow *handleRelayInvsFlow) ibdWithHeadersProof(highHash *externalapi.DomainHash) error {
err := flow.Domain().InitStagingConsensus()
if err != nil {
return err
}
err = flow.downloadHeadersAndPruningUTXOSet(highHash)
if err != nil {
if !flow.IsRecoverableError(err) {
return err
}
deleteStagingConsensusErr := flow.Domain().DeleteStagingConsensus()
if deleteStagingConsensusErr != nil {
return deleteStagingConsensusErr
}
return err
}
err = flow.Domain().CommitStagingConsensus()
if err != nil {
return err
}
err = flow.OnPruningPointUTXOSetOverride()
if err != nil {
return err
}
return nil
}
func (flow *handleRelayInvsFlow) shouldSyncAndShouldDownloadHeadersProof(highBlock *externalapi.DomainBlock,
highestSharedBlockFound bool) (shouldDownload, shouldSync bool, err error) {
if !highestSharedBlockFound {
hasMoreBlueWorkThanSelectedTipAndPruningDepthMoreBlueScore, err := flow.checkIfHighHashHasMoreBlueWorkThanSelectedTipAndPruningDepthMoreBlueScore(highBlock)
if err != nil {
return false, false, err
}
if hasMoreBlueWorkThanSelectedTipAndPruningDepthMoreBlueScore {
return true, true, nil
}
return false, false, nil
}
return false, true, nil
}
func (flow *handleRelayInvsFlow) checkIfHighHashHasMoreBlueWorkThanSelectedTipAndPruningDepthMoreBlueScore(highBlock *externalapi.DomainBlock) (bool, error) {
headersSelectedTip, err := flow.Domain().Consensus().GetHeadersSelectedTip()
if err != nil {
return false, err
}
headersSelectedTipInfo, err := flow.Domain().Consensus().GetBlockInfo(headersSelectedTip)
if err != nil {
return false, err
}
if highBlock.Header.BlueScore() < headersSelectedTipInfo.BlueScore+flow.Config().NetParams().PruningDepth() {
return false, nil
}
return highBlock.Header.BlueWork().Cmp(headersSelectedTipInfo.BlueWork) > 0, nil
}
func (flow *handleRelayInvsFlow) syncAndValidatePruningPointProof() (*externalapi.DomainHash, error) {
log.Infof("Downloading the pruning point proof from %s", flow.peer)
err := flow.outgoingRoute.Enqueue(appmessage.NewMsgRequestPruningPointProof())
if err != nil {
return nil, err
}
message, err := flow.dequeueIncomingMessageAndSkipInvs(common.DefaultTimeout)
if err != nil {
return nil, err
}
pruningPointProofMessage, ok := message.(*appmessage.MsgPruningPointProof)
if !ok {
return nil, protocolerrors.Errorf(true, "received unexpected message type. "+
"expected: %s, got: %s", appmessage.CmdPruningPointProof, message.Command())
}
pruningPointProof := appmessage.MsgPruningPointProofToDomainPruningPointProof(pruningPointProofMessage)
err = flow.Domain().Consensus().ValidatePruningPointProof(pruningPointProof)
if err != nil {
if errors.As(err, &ruleerrors.RuleError{}) {
return nil, protocolerrors.Wrapf(true, err, "pruning point proof validation failed")
}
return nil, err
}
err = flow.Domain().StagingConsensus().ApplyPruningPointProof(pruningPointProof)
if err != nil {
return nil, err
}
return consensushashing.HeaderHash(pruningPointProof.Headers[0][len(pruningPointProof.Headers[0])-1]), nil
}
func (flow *handleRelayInvsFlow) downloadHeadersAndPruningUTXOSet(highHash *externalapi.DomainHash) error {
proofPruningPoint, err := flow.syncAndValidatePruningPointProof()
if err != nil {
return err
}
err = flow.syncPruningPointsAndPruningPointAnticone(proofPruningPoint)
if err != nil {
return err
}
// TODO: Remove this condition once there's more proper way to check finality violation
// in the headers proof.
if proofPruningPoint.Equal(flow.Config().NetParams().GenesisHash) {
return protocolerrors.Errorf(true, "the genesis pruning point violates finality")
}
err = flow.syncPruningPointFutureHeaders(flow.Domain().StagingConsensus(), proofPruningPoint, highHash)
if err != nil {
return err
}
log.Infof("Headers downloaded from peer %s", flow.peer)
highHashInfo, err := flow.Domain().StagingConsensus().GetBlockInfo(highHash)
if err != nil {
return err
}
if !highHashInfo.Exists {
return protocolerrors.Errorf(true, "the triggering IBD block was not sent")
}
err = flow.validatePruningPointFutureHeaderTimestamps()
if err != nil {
return err
}
log.Debugf("Syncing the current pruning point UTXO set")
syncedPruningPointUTXOSetSuccessfully, err := flow.syncPruningPointUTXOSet(flow.Domain().StagingConsensus(), proofPruningPoint)
if err != nil {
return err
}
if !syncedPruningPointUTXOSetSuccessfully {
log.Debugf("Aborting IBD because the pruning point UTXO set failed to sync")
return nil
}
log.Debugf("Finished syncing the current pruning point UTXO set")
return nil
}
func (flow *handleRelayInvsFlow) syncPruningPointsAndPruningPointAnticone(proofPruningPoint *externalapi.DomainHash) error {
log.Infof("Downloading the past pruning points and the pruning point anticone from %s", flow.peer)
err := flow.outgoingRoute.Enqueue(appmessage.NewMsgRequestPruningPointAndItsAnticone())
if err != nil {
return err
}
err = flow.validateAndInsertPruningPoints(proofPruningPoint)
if err != nil {
return err
}
message, err := flow.dequeueIncomingMessageAndSkipInvs(common.DefaultTimeout)
if err != nil {
return err
}
msgTrustedData, ok := message.(*appmessage.MsgTrustedData)
if !ok {
return protocolerrors.Errorf(true, "received unexpected message type. "+
"expected: %s, got: %s", appmessage.CmdTrustedData, message.Command())
}
pruningPointWithMetaData, done, err := flow.receiveBlockWithTrustedData()
if err != nil {
return err
}
if done {
return protocolerrors.Errorf(true, "got `done` message before receiving the pruning point")
}
if !pruningPointWithMetaData.Block.Header.BlockHash().Equal(proofPruningPoint) {
return protocolerrors.Errorf(true, "first block with trusted data is not the pruning point")
}
err = flow.processBlockWithTrustedData(flow.Domain().StagingConsensus(), pruningPointWithMetaData, msgTrustedData)
if err != nil {
return err
}
for {
blockWithTrustedData, done, err := flow.receiveBlockWithTrustedData()
if err != nil {
return err
}
if done {
break
}
err = flow.processBlockWithTrustedData(flow.Domain().StagingConsensus(), blockWithTrustedData, msgTrustedData)
if err != nil {
return err
}
}
log.Infof("Finished downloading pruning point and its anticone from %s", flow.peer)
return nil
}
func (flow *handleRelayInvsFlow) processBlockWithTrustedData(
consensus externalapi.Consensus, block *appmessage.MsgBlockWithTrustedDataV4, data *appmessage.MsgTrustedData) error {
blockWithTrustedData := &externalapi.BlockWithTrustedData{
Block: appmessage.MsgBlockToDomainBlock(block.Block),
DAAWindow: make([]*externalapi.TrustedDataDataDAAHeader, 0, len(block.DAAWindowIndices)),
GHOSTDAGData: make([]*externalapi.BlockGHOSTDAGDataHashPair, 0, len(block.GHOSTDAGDataIndices)),
}
for _, index := range block.DAAWindowIndices {
blockWithTrustedData.DAAWindow = append(blockWithTrustedData.DAAWindow, appmessage.TrustedDataDataDAABlockV4ToTrustedDataDataDAAHeader(data.DAAWindow[index]))
}
for _, index := range block.GHOSTDAGDataIndices {
blockWithTrustedData.GHOSTDAGData = append(blockWithTrustedData.GHOSTDAGData, appmessage.GHOSTDAGHashPairToDomainGHOSTDAGHashPair(data.GHOSTDAGData[index]))
}
_, err := consensus.ValidateAndInsertBlockWithTrustedData(blockWithTrustedData, false)
return err
}
func (flow *handleRelayInvsFlow) receiveBlockWithTrustedData() (*appmessage.MsgBlockWithTrustedDataV4, bool, error) {
message, err := flow.dequeueIncomingMessageAndSkipInvs(common.DefaultTimeout)
if err != nil {
return nil, false, err
}
switch downCastedMessage := message.(type) {
case *appmessage.MsgBlockWithTrustedDataV4:
return downCastedMessage, false, nil
case *appmessage.MsgDoneBlocksWithTrustedData:
return nil, true, nil
default:
return nil, false,
protocolerrors.Errorf(true, "received unexpected message type. "+
"expected: %s or %s, got: %s",
(&appmessage.MsgBlockWithTrustedData{}).Command(),
(&appmessage.MsgDoneBlocksWithTrustedData{}).Command(),
downCastedMessage.Command())
}
}
func (flow *handleRelayInvsFlow) receivePruningPoints() (*appmessage.MsgPruningPoints, error) {
message, err := flow.dequeueIncomingMessageAndSkipInvs(common.DefaultTimeout)
if err != nil {
return nil, err
}
msgPruningPoints, ok := message.(*appmessage.MsgPruningPoints)
if !ok {
return nil,
protocolerrors.Errorf(true, "received unexpected message type. "+
"expected: %s, got: %s", appmessage.CmdPruningPoints, message.Command())
}
return msgPruningPoints, nil
}
func (flow *handleRelayInvsFlow) validateAndInsertPruningPoints(proofPruningPoint *externalapi.DomainHash) error {
currentPruningPoint, err := flow.Domain().Consensus().PruningPoint()
if err != nil {
return err
}
if currentPruningPoint.Equal(proofPruningPoint) {
return protocolerrors.Errorf(true, "the proposed pruning point is the same as the current pruning point")
}
pruningPoints, err := flow.receivePruningPoints()
if err != nil {
return err
}
headers := make([]externalapi.BlockHeader, len(pruningPoints.Headers))
for i, header := range pruningPoints.Headers {
headers[i] = appmessage.BlockHeaderToDomainBlockHeader(header)
}
arePruningPointsViolatingFinality, err := flow.Domain().Consensus().ArePruningPointsViolatingFinality(headers)
if err != nil {
return err
}
if arePruningPointsViolatingFinality {
// TODO: Find a better way to deal with finality conflicts.
return protocolerrors.Errorf(false, "pruning points are violating finality")
}
lastPruningPoint := consensushashing.HeaderHash(headers[len(headers)-1])
if !lastPruningPoint.Equal(proofPruningPoint) {
return protocolerrors.Errorf(true, "the proof pruning point is not equal to the last pruning "+
"point in the list")
}
err = flow.Domain().StagingConsensus().ImportPruningPoints(headers)
if err != nil {
return err
}
return nil
}
func (flow *handleRelayInvsFlow) syncPruningPointUTXOSet(consensus externalapi.Consensus,
pruningPoint *externalapi.DomainHash) (bool, error) {
log.Infof("Checking if the suggested pruning point %s is compatible to the node DAG", pruningPoint)
isValid, err := flow.Domain().StagingConsensus().IsValidPruningPoint(pruningPoint)
if err != nil {
return false, err
}
if !isValid {
return false, protocolerrors.Errorf(true, "invalid pruning point %s", pruningPoint)
}
log.Info("Fetching the pruning point UTXO set")
isSuccessful, err := flow.fetchMissingUTXOSet(consensus, pruningPoint)
if err != nil {
return false, err
}
if !isSuccessful {
log.Infof("Couldn't successfully fetch the pruning point UTXO set. Stopping IBD.")
return false, nil
}
log.Info("Fetched the new pruning point UTXO set")
return true, nil
}
func (flow *handleRelayInvsFlow) fetchMissingUTXOSet(consensus externalapi.Consensus, pruningPointHash *externalapi.DomainHash) (succeed bool, err error) {
defer func() {
err := flow.Domain().StagingConsensus().ClearImportedPruningPointData()
if err != nil {
panic(fmt.Sprintf("failed to clear imported pruning point data: %s", err))
}
}()
err = flow.outgoingRoute.Enqueue(appmessage.NewMsgRequestPruningPointUTXOSet(pruningPointHash))
if err != nil {
return false, err
}
receivedAll, err := flow.receiveAndInsertPruningPointUTXOSet(consensus, pruningPointHash)
if err != nil {
return false, err
}
if !receivedAll {
return false, nil
}
err = flow.Domain().StagingConsensus().ValidateAndInsertImportedPruningPoint(pruningPointHash)
if err != nil {
// TODO: Find a better way to deal with finality conflicts.
if errors.Is(err, ruleerrors.ErrSuggestedPruningViolatesFinality) {
return false, nil
}
return false, protocolerrors.ConvertToBanningProtocolErrorIfRuleError(err, "error with pruning point UTXO set")
}
return true, nil
}

View File

@ -0,0 +1,9 @@
package blockrelay
import (
"github.com/kaspanet/kaspad/infrastructure/logger"
"github.com/kaspanet/kaspad/util/panics"
)
var log = logger.RegisterSubSystem("PROT")
var spawn = panics.GoroutineWrapperFunc(log)

View File

@ -0,0 +1,35 @@
package blockrelay
import (
"github.com/kaspanet/kaspad/app/appmessage"
peerpkg "github.com/kaspanet/kaspad/app/protocol/peer"
"github.com/kaspanet/kaspad/domain"
"github.com/kaspanet/kaspad/infrastructure/config"
"github.com/kaspanet/kaspad/infrastructure/network/netadapter/router"
)
// SendVirtualSelectedParentInvContext is the interface for the context needed for the SendVirtualSelectedParentInv flow.
type SendVirtualSelectedParentInvContext interface {
Domain() domain.Domain
Config() *config.Config
}
// SendVirtualSelectedParentInv sends a peer the selected parent hash of the virtual
func SendVirtualSelectedParentInv(context SendVirtualSelectedParentInvContext,
outgoingRoute *router.Route, peer *peerpkg.Peer) error {
virtualSelectedParent, err := context.Domain().Consensus().GetVirtualSelectedParent()
if err != nil {
return err
}
if virtualSelectedParent.Equal(context.Config().NetParams().GenesisHash) {
log.Debugf("Skipping sending the virtual selected parent hash to peer %s because it's the genesis", peer)
return nil
}
log.Debugf("Sending virtual selected parent hash %s to peer %s", virtualSelectedParent, peer)
virtualSelectedParentInv := appmessage.NewMsgInvBlock(virtualSelectedParent)
return outgoingRoute.Enqueue(virtualSelectedParentInv)
}

View File

@ -5,10 +5,10 @@ import (
"github.com/kaspanet/kaspad/app/protocol/common"
"github.com/kaspanet/kaspad/app/protocol/flowcontext"
"github.com/kaspanet/kaspad/app/protocol/flows/v3/addressexchange"
"github.com/kaspanet/kaspad/app/protocol/flows/v3/blockrelay"
"github.com/kaspanet/kaspad/app/protocol/flows/v3/ping"
"github.com/kaspanet/kaspad/app/protocol/flows/v3/rejects"
"github.com/kaspanet/kaspad/app/protocol/flows/v3/transactionrelay"
"github.com/kaspanet/kaspad/app/protocol/flows/v4/blockrelay"
peerpkg "github.com/kaspanet/kaspad/app/protocol/peer"
routerpkg "github.com/kaspanet/kaspad/infrastructure/network/netadapter/router"
)
@ -67,10 +67,11 @@ func registerBlockRelayFlows(m protocolManager, router *routerpkg.Router, isStop
m.RegisterFlow("HandleRelayInvs", router, []appmessage.MessageCommand{
appmessage.CmdInvRelayBlock, appmessage.CmdBlock, appmessage.CmdBlockLocator,
appmessage.CmdDoneHeaders, appmessage.CmdUnexpectedPruningPoint, appmessage.CmdPruningPointUTXOSetChunk,
appmessage.CmdBlockHeaders, appmessage.CmdIBDBlockLocatorHighestHash, appmessage.CmdBlockWithTrustedData,
appmessage.CmdBlockHeaders, appmessage.CmdIBDBlockLocatorHighestHash, appmessage.CmdBlockWithTrustedDataV4,
appmessage.CmdDoneBlocksWithTrustedData, appmessage.CmdIBDBlockLocatorHighestHashNotFound,
appmessage.CmdDonePruningPointUTXOSetChunks, appmessage.CmdIBDBlock, appmessage.CmdPruningPoints,
appmessage.CmdPruningPointProof,
appmessage.CmdTrustedData,
},
isStopping, errChan, func(incomingRoute *routerpkg.Route, peer *peerpkg.Peer) error {
return blockrelay.HandleRelayInvs(m.Context(), incomingRoute,

View File

@ -39,21 +39,22 @@ type consensus struct {
finalityManager model.FinalityManager
pruningProofManager model.PruningProofManager
acceptanceDataStore model.AcceptanceDataStore
blockStore model.BlockStore
blockHeaderStore model.BlockHeaderStore
pruningStore model.PruningStore
ghostdagDataStores []model.GHOSTDAGDataStore
blockRelationStores []model.BlockRelationStore
blockStatusStore model.BlockStatusStore
consensusStateStore model.ConsensusStateStore
headersSelectedTipStore model.HeaderSelectedTipStore
multisetStore model.MultisetStore
reachabilityDataStores []model.ReachabilityDataStore
utxoDiffStore model.UTXODiffStore
finalityStore model.FinalityStore
headersSelectedChainStore model.HeadersSelectedChainStore
daaBlocksStore model.DAABlocksStore
acceptanceDataStore model.AcceptanceDataStore
blockStore model.BlockStore
blockHeaderStore model.BlockHeaderStore
pruningStore model.PruningStore
ghostdagDataStores []model.GHOSTDAGDataStore
blockRelationStores []model.BlockRelationStore
blockStatusStore model.BlockStatusStore
consensusStateStore model.ConsensusStateStore
headersSelectedTipStore model.HeaderSelectedTipStore
multisetStore model.MultisetStore
reachabilityDataStores []model.ReachabilityDataStore
utxoDiffStore model.UTXODiffStore
finalityStore model.FinalityStore
headersSelectedChainStore model.HeadersSelectedChainStore
daaBlocksStore model.DAABlocksStore
blocksWithTrustedDataDAAWindowStore model.BlocksWithTrustedDataDAAWindowStore
}
func (s *consensus) ValidateAndInsertBlockWithTrustedData(block *externalapi.BlockWithTrustedData, validateUTXO bool) (*externalapi.VirtualChangeSet, error) {
@ -119,7 +120,6 @@ func (s *consensus) Init(skipAddingGenesis bool) error {
if !skipAddingGenesis && s.blockStore.Count(stagingArea) == 0 {
genesisWithTrustedData := &externalapi.BlockWithTrustedData{
Block: s.genesisBlock,
DAAScore: 0,
DAAWindow: nil,
GHOSTDAGData: []*externalapi.BlockGHOSTDAGDataHashPair{
{
@ -144,6 +144,7 @@ func (s *consensus) PruningPointAndItsAnticone() ([]*externalapi.DomainHash, err
return s.pruningManager.PruningPointAndItsAnticone()
}
// TODO: Remove this method once v3 is obsolete
func (s *consensus) BlockWithTrustedData(blockHash *externalapi.DomainHash) (*externalapi.BlockWithTrustedData, error) {
s.lock.Lock()
defer s.lock.Unlock()
@ -744,6 +745,9 @@ func (s *consensus) ValidatePruningPointProof(pruningPointProof *externalapi.Pru
}
func (s *consensus) ApplyPruningPointProof(pruningPointProof *externalapi.PruningPointProof) error {
s.lock.Lock()
defer s.lock.Unlock()
stagingArea := model.NewStagingArea()
err := s.pruningProofManager.ApplyPruningPointProof(stagingArea, pruningPointProof)
if err != nil {
@ -757,3 +761,66 @@ func (s *consensus) ApplyPruningPointProof(pruningPointProof *externalapi.Prunin
return nil
}
func (s *consensus) BlockDAAWindowHashes(blockHash *externalapi.DomainHash) ([]*externalapi.DomainHash, error) {
s.lock.Lock()
defer s.lock.Unlock()
stagingArea := model.NewStagingArea()
return s.dagTraversalManager.DAABlockWindow(stagingArea, blockHash)
}
func (s *consensus) TrustedDataDataDAAHeader(trustedBlockHash, daaBlockHash *externalapi.DomainHash, daaBlockWindowIndex uint64) (*externalapi.TrustedDataDataDAAHeader, error) {
s.lock.Lock()
defer s.lock.Unlock()
stagingArea := model.NewStagingArea()
header, err := s.blockHeaderStore.BlockHeader(s.databaseContext, stagingArea, daaBlockHash)
if err != nil {
return nil, err
}
ghostdagData, err := s.ghostdagDataStores[0].Get(s.databaseContext, stagingArea, daaBlockHash, false)
isNotFoundError := database.IsNotFoundError(err)
if !isNotFoundError && err != nil {
return nil, err
}
if !isNotFoundError {
return &externalapi.TrustedDataDataDAAHeader{
Header: header,
GHOSTDAGData: ghostdagData,
}, nil
}
ghostdagDataHashPair, err := s.blocksWithTrustedDataDAAWindowStore.DAAWindowBlock(s.databaseContext, stagingArea, trustedBlockHash, daaBlockWindowIndex)
if err != nil {
return nil, err
}
return &externalapi.TrustedDataDataDAAHeader{
Header: header,
GHOSTDAGData: ghostdagDataHashPair.GHOSTDAGData,
}, nil
}
func (s *consensus) TrustedBlockAssociatedGHOSTDAGDataBlockHashes(blockHash *externalapi.DomainHash) ([]*externalapi.DomainHash, error) {
s.lock.Lock()
defer s.lock.Unlock()
return s.pruningManager.TrustedBlockAssociatedGHOSTDAGDataBlockHashes(model.NewStagingArea(), blockHash)
}
func (s *consensus) TrustedGHOSTDAGData(blockHash *externalapi.DomainHash) (*externalapi.BlockGHOSTDAGData, error) {
s.lock.Lock()
defer s.lock.Unlock()
stagingArea := model.NewStagingArea()
ghostdagData, err := s.ghostdagDataStores[0].Get(s.databaseContext, stagingArea, blockHash, false)
isNotFoundError := database.IsNotFoundError(err)
if isNotFoundError || ghostdagData.SelectedParent().Equal(model.VirtualGenesisBlockHash) {
return s.ghostdagDataStores[0].Get(s.databaseContext, stagingArea, blockHash, true)
}
return ghostdagData, nil
}

View File

@ -445,21 +445,22 @@ func (f *factory) NewConsensus(config *Config, db infrastructuredatabase.Databas
finalityManager: finalityManager,
pruningProofManager: pruningProofManager,
acceptanceDataStore: acceptanceDataStore,
blockStore: blockStore,
blockHeaderStore: blockHeaderStore,
pruningStore: pruningStore,
ghostdagDataStores: ghostdagDataStores,
blockStatusStore: blockStatusStore,
blockRelationStores: blockRelationStores,
consensusStateStore: consensusStateStore,
headersSelectedTipStore: headersSelectedTipStore,
multisetStore: multisetStore,
reachabilityDataStores: reachabilityDataStores,
utxoDiffStore: utxoDiffStore,
finalityStore: finalityStore,
headersSelectedChainStore: headersSelectedChainStore,
daaBlocksStore: daaBlocksStore,
acceptanceDataStore: acceptanceDataStore,
blockStore: blockStore,
blockHeaderStore: blockHeaderStore,
pruningStore: pruningStore,
ghostdagDataStores: ghostdagDataStores,
blockStatusStore: blockStatusStore,
blockRelationStores: blockRelationStores,
consensusStateStore: consensusStateStore,
headersSelectedTipStore: headersSelectedTipStore,
multisetStore: multisetStore,
reachabilityDataStores: reachabilityDataStores,
utxoDiffStore: utxoDiffStore,
finalityStore: finalityStore,
headersSelectedChainStore: headersSelectedChainStore,
daaBlocksStore: daaBlocksStore,
blocksWithTrustedDataDAAWindowStore: daaWindowStore,
}
err = c.Init(config.SkipAddingGenesis)
@ -637,7 +638,8 @@ func (f *factory) dagProcesses(config *Config,
reachabilityDataStores[i],
ghostdagManagers[i],
daaWindowStore,
config.GenesisHash)
config.GenesisHash,
config.DifficultyAdjustmentWindowSize)
}
return reachabilityManagers, dagTopologyManagers, ghostdagManagers, dagTraversalManagers

View File

@ -6,17 +6,13 @@ package externalapi
// anticone on a pruned-headers node.
type BlockWithTrustedData struct {
Block *DomainBlock
DAAScore uint64
DAAWindow []*TrustedDataDataDAABlock
DAAWindow []*TrustedDataDataDAAHeader
GHOSTDAGData []*BlockGHOSTDAGDataHashPair
}
// TrustedDataDataDAABlock is a block that belongs to BlockWithTrustedData.DAAWindow
// TODO: Currently each trusted data block contains the entire set of blocks in its
// DAA window. There's a lot of duplications between DAA windows of trusted blocks.
// This duplication should be optimized out.
type TrustedDataDataDAABlock struct {
Block *DomainBlock
// TrustedDataDataDAAHeader is a block that belongs to BlockWithTrustedData.DAAWindow
type TrustedDataDataDAAHeader struct {
Header BlockHeader
GHOSTDAGData *BlockGHOSTDAGData
}

View File

@ -47,4 +47,8 @@ type Consensus interface {
EstimateNetworkHashesPerSecond(startHash *DomainHash, windowSize int) (uint64, error)
PopulateMass(transaction *DomainTransaction)
ResolveVirtual() (*VirtualChangeSet, bool, error)
BlockDAAWindowHashes(blockHash *DomainHash) ([]*DomainHash, error)
TrustedDataDataDAAHeader(trustedBlockHash, daaBlockHash *DomainHash, daaBlockWindowIndex uint64) (*TrustedDataDataDAAHeader, error)
TrustedBlockAssociatedGHOSTDAGDataBlockHashes(blockHash *DomainHash) ([]*DomainHash, error)
TrustedGHOSTDAGData(blockHash *DomainHash) (*BlockGHOSTDAGData, error)
}

View File

@ -14,6 +14,7 @@ type DAGTraversalManager interface {
AnticoneFromVirtualPOV(stagingArea *StagingArea, blockHash *externalapi.DomainHash) ([]*externalapi.DomainHash, error)
BlockWindow(stagingArea *StagingArea, highHash *externalapi.DomainHash, windowSize int) ([]*externalapi.DomainHash, error)
BlockWindowWithGHOSTDAGData(stagingArea *StagingArea, highHash *externalapi.DomainHash, windowSize int) ([]*externalapi.BlockGHOSTDAGDataHashPair, error)
DAABlockWindow(stagingArea *StagingArea, highHash *externalapi.DomainHash) ([]*externalapi.DomainHash, error)
NewDownHeap(stagingArea *StagingArea) BlockHeap
NewUpHeap(stagingArea *StagingArea) BlockHeap
CalculateChainPath(stagingArea *StagingArea, fromBlockHash, toBlockHash *externalapi.DomainHash) (

View File

@ -14,5 +14,6 @@ type PruningManager interface {
PruneAllBlocksBelow(stagingArea *StagingArea, pruningPointHash *externalapi.DomainHash) error
PruningPointAndItsAnticone() ([]*externalapi.DomainHash, error)
ExpectedHeaderPruningPoint(stagingArea *StagingArea, blockHash *externalapi.DomainHash) (*externalapi.DomainHash, error)
TrustedBlockAssociatedGHOSTDAGDataBlockHashes(stagingArea *StagingArea, blockHash *externalapi.DomainHash) ([]*externalapi.DomainHash, error)
BlockWithTrustedData(stagingArea *StagingArea, blockHash *externalapi.DomainHash) (*externalapi.BlockWithTrustedData, error)
}

View File

@ -12,13 +12,12 @@ func (bp *blockProcessor) validateAndInsertBlockWithTrustedData(stagingArea *mod
blockHash := consensushashing.BlockHash(block.Block)
for i, daaBlock := range block.DAAWindow {
hash := consensushashing.BlockHash(daaBlock.Block)
hash := consensushashing.HeaderHash(daaBlock.Header)
bp.blocksWithTrustedDataDAAWindowStore.Stage(stagingArea, blockHash, uint64(i), &externalapi.BlockGHOSTDAGDataHashPair{
Hash: hash,
GHOSTDAGData: daaBlock.GHOSTDAGData,
})
bp.blockStore.Stage(stagingArea, hash, daaBlock.Block)
bp.blockHeaderStore.Stage(stagingArea, hash, daaBlock.Block.Header)
bp.blockHeaderStore.Stage(stagingArea, hash, daaBlock.Header)
}
blockReplacedGHOSTDAGData, err := bp.ghostdagDataWithoutPrunedBlocks(stagingArea, block.GHOSTDAGData[0].GHOSTDAGData)
@ -31,7 +30,7 @@ func (bp *blockProcessor) validateAndInsertBlockWithTrustedData(stagingArea *mod
bp.ghostdagDataStore.Stage(stagingArea, pair.Hash, pair.GHOSTDAGData, true)
}
bp.daaBlocksStore.StageDAAScore(stagingArea, blockHash, block.DAAScore)
bp.daaBlocksStore.StageDAAScore(stagingArea, blockHash, block.Block.Header.DAAScore())
return bp.validateAndInsertBlock(stagingArea, block.Block, false, validateUTXO, true)
}

View File

@ -11,12 +11,13 @@ import (
type dagTraversalManager struct {
databaseContext model.DBReader
dagTopologyManager model.DAGTopologyManager
ghostdagManager model.GHOSTDAGManager
ghostdagDataStore model.GHOSTDAGDataStore
reachabilityDataStore model.ReachabilityDataStore
daaWindowStore model.BlocksWithTrustedDataDAAWindowStore
genesisHash *externalapi.DomainHash
dagTopologyManager model.DAGTopologyManager
ghostdagManager model.GHOSTDAGManager
ghostdagDataStore model.GHOSTDAGDataStore
reachabilityDataStore model.ReachabilityDataStore
daaWindowStore model.BlocksWithTrustedDataDAAWindowStore
genesisHash *externalapi.DomainHash
difficultyAdjustmentWindowSize int
}
// New instantiates a new DAGTraversalManager
@ -27,7 +28,8 @@ func New(
reachabilityDataStore model.ReachabilityDataStore,
ghostdagManager model.GHOSTDAGManager,
daaWindowStore model.BlocksWithTrustedDataDAAWindowStore,
genesisHash *externalapi.DomainHash) model.DAGTraversalManager {
genesisHash *externalapi.DomainHash,
difficultyAdjustmentWindowSize int) model.DAGTraversalManager {
return &dagTraversalManager{
databaseContext: databaseContext,
dagTopologyManager: dagTopologyManager,
@ -36,7 +38,8 @@ func New(
ghostdagManager: ghostdagManager,
daaWindowStore: daaWindowStore,
genesisHash: genesisHash,
genesisHash: genesisHash,
difficultyAdjustmentWindowSize: difficultyAdjustmentWindowSize,
}
}

View File

@ -6,6 +6,10 @@ import (
"github.com/kaspanet/kaspad/infrastructure/db/database"
)
func (dtm *dagTraversalManager) DAABlockWindow(stagingArea *model.StagingArea, highHash *externalapi.DomainHash) ([]*externalapi.DomainHash, error) {
return dtm.BlockWindow(stagingArea, highHash, dtm.difficultyAdjustmentWindowSize)
}
// BlockWindow returns a blockWindow of the given size that contains the
// blocks in the past of highHash, the sorting is unspecified.
// If the number of blocks in the past of startingNode is less then windowSize,

View File

@ -968,26 +968,21 @@ func (pm *pruningManager) BlockWithTrustedData(stagingArea *model.StagingArea, b
return nil, err
}
daaScore, err := pm.daaBlocksStore.DAAScore(pm.databaseContext, stagingArea, blockHash)
if err != nil {
return nil, err
}
windowSize := pm.difficultyAdjustmentWindowSize
window, err := pm.dagTraversalManager.BlockWindowWithGHOSTDAGData(stagingArea, blockHash, windowSize)
if err != nil {
return nil, err
}
windowPairs := make([]*externalapi.TrustedDataDataDAABlock, len(window))
windowPairs := make([]*externalapi.TrustedDataDataDAAHeader, len(window))
for i, daaBlock := range window {
daaDomainBlock, err := pm.blocksStore.Block(pm.databaseContext, stagingArea, daaBlock.Hash)
daaDomainBlock, err := pm.blockHeaderStore.BlockHeader(pm.databaseContext, stagingArea, daaBlock.Hash)
if err != nil {
return nil, err
}
windowPairs[i] = &externalapi.TrustedDataDataDAABlock{
Block: daaDomainBlock,
windowPairs[i] = &externalapi.TrustedDataDataDAAHeader{
Header: daaDomainBlock,
GHOSTDAGData: daaBlock.GHOSTDAGData,
}
}
@ -1027,7 +1022,6 @@ func (pm *pruningManager) BlockWithTrustedData(stagingArea *model.StagingArea, b
return &externalapi.BlockWithTrustedData{
Block: block,
DAAScore: daaScore,
DAAWindow: windowPairs,
GHOSTDAGData: ghostdagDataHashPairs,
}, nil
@ -1133,3 +1127,37 @@ func (pm *pruningManager) isPruningPointInPruningDepth(stagingArea *model.Stagin
return blockGHOSTDAGData.BlueScore() >= pruningPointHeader.BlueScore()+pm.pruningDepth, nil
}
func (pm *pruningManager) TrustedBlockAssociatedGHOSTDAGDataBlockHashes(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) ([]*externalapi.DomainHash, error) {
blockHashes := make([]*externalapi.DomainHash, 0, pm.k)
current := blockHash
isTrustedData := false
for i := externalapi.KType(0); i <= pm.k; i++ {
ghostdagData, err := pm.ghostdagDataStore.Get(pm.databaseContext, stagingArea, current, isTrustedData)
isNotFoundError := database.IsNotFoundError(err)
if !isNotFoundError && err != nil {
return nil, err
}
if isNotFoundError || ghostdagData.SelectedParent().Equal(model.VirtualGenesisBlockHash) {
isTrustedData = true
ghostdagData, err = pm.ghostdagDataStore.Get(pm.databaseContext, stagingArea, current, true)
if err != nil {
return nil, err
}
}
blockHashes = append(blockHashes, current)
if ghostdagData.SelectedParent().Equal(pm.genesisHash) {
break
}
if current.Equal(pm.genesisHash) {
break
}
current = ghostdagData.SelectedParent()
}
return blockHashes, nil
}

View File

@ -284,11 +284,11 @@ var MainnetParams = Params{
// TestnetParams defines the network parameters for the test Kaspa network.
var TestnetParams = Params{
K: defaultGHOSTDAGK,
Name: "kaspa-testnet-7",
Name: "kaspa-testnet-8",
Net: appmessage.Testnet,
RPCPort: "16210",
DefaultPort: "16211",
DNSSeeds: []string{"testnet-7-dnsseed.daglabs-dev.com"},
DNSSeeds: []string{"testnet-8-dnsseed.daglabs-dev.com"},
// DAG parameters
GenesisBlock: &testnetGenesisBlock,

View File

@ -49,7 +49,6 @@ func TestCreateStagingConsensus(t *testing.T) {
addGenesisToStagingConsensus := func() {
genesisWithTrustedData := &externalapi.BlockWithTrustedData{
Block: consensusConfig.GenesisBlock,
DAAScore: 0,
DAAWindow: nil,
GHOSTDAGData: []*externalapi.BlockGHOSTDAGDataHashPair{
{

View File

@ -45,6 +45,8 @@ message KaspadMessage {
RequestPruningPointProofMessage requestPruningPointProof = 48;
PruningPointProofMessage pruningPointProof = 49;
ReadyMessage ready = 50;
BlockWithTrustedDataV4Message blockWithTrustedDataV4 = 51;
TrustedDataMessage trustedData = 52;
GetCurrentNetworkRequestMessage getCurrentNetworkRequest = 1001;
GetCurrentNetworkResponseMessage getCurrentNetworkResponse = 1002;

View File

@ -2389,6 +2389,61 @@ func (x *DaaBlock) GetGhostdagData() *GhostdagData {
return nil
}
type DaaBlockV4 struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
Header *BlockHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
GhostdagData *GhostdagData `protobuf:"bytes,2,opt,name=ghostdagData,proto3" json:"ghostdagData,omitempty"`
}
func (x *DaaBlockV4) Reset() {
*x = DaaBlockV4{}
if protoimpl.UnsafeEnabled {
mi := &file_p2p_proto_msgTypes[44]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *DaaBlockV4) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*DaaBlockV4) ProtoMessage() {}
func (x *DaaBlockV4) ProtoReflect() protoreflect.Message {
mi := &file_p2p_proto_msgTypes[44]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use DaaBlockV4.ProtoReflect.Descriptor instead.
func (*DaaBlockV4) Descriptor() ([]byte, []int) {
return file_p2p_proto_rawDescGZIP(), []int{44}
}
func (x *DaaBlockV4) GetHeader() *BlockHeader {
if x != nil {
return x.Header
}
return nil
}
func (x *DaaBlockV4) GetGhostdagData() *GhostdagData {
if x != nil {
return x.GhostdagData
}
return nil
}
type BlockGhostdagDataHashPair struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
@ -2401,7 +2456,7 @@ type BlockGhostdagDataHashPair struct {
func (x *BlockGhostdagDataHashPair) Reset() {
*x = BlockGhostdagDataHashPair{}
if protoimpl.UnsafeEnabled {
mi := &file_p2p_proto_msgTypes[44]
mi := &file_p2p_proto_msgTypes[45]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@ -2414,7 +2469,7 @@ func (x *BlockGhostdagDataHashPair) String() string {
func (*BlockGhostdagDataHashPair) ProtoMessage() {}
func (x *BlockGhostdagDataHashPair) ProtoReflect() protoreflect.Message {
mi := &file_p2p_proto_msgTypes[44]
mi := &file_p2p_proto_msgTypes[45]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@ -2427,7 +2482,7 @@ func (x *BlockGhostdagDataHashPair) ProtoReflect() protoreflect.Message {
// Deprecated: Use BlockGhostdagDataHashPair.ProtoReflect.Descriptor instead.
func (*BlockGhostdagDataHashPair) Descriptor() ([]byte, []int) {
return file_p2p_proto_rawDescGZIP(), []int{44}
return file_p2p_proto_rawDescGZIP(), []int{45}
}
func (x *BlockGhostdagDataHashPair) GetHash() *Hash {
@ -2460,7 +2515,7 @@ type GhostdagData struct {
func (x *GhostdagData) Reset() {
*x = GhostdagData{}
if protoimpl.UnsafeEnabled {
mi := &file_p2p_proto_msgTypes[45]
mi := &file_p2p_proto_msgTypes[46]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@ -2473,7 +2528,7 @@ func (x *GhostdagData) String() string {
func (*GhostdagData) ProtoMessage() {}
func (x *GhostdagData) ProtoReflect() protoreflect.Message {
mi := &file_p2p_proto_msgTypes[45]
mi := &file_p2p_proto_msgTypes[46]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@ -2486,7 +2541,7 @@ func (x *GhostdagData) ProtoReflect() protoreflect.Message {
// Deprecated: Use GhostdagData.ProtoReflect.Descriptor instead.
func (*GhostdagData) Descriptor() ([]byte, []int) {
return file_p2p_proto_rawDescGZIP(), []int{45}
return file_p2p_proto_rawDescGZIP(), []int{46}
}
func (x *GhostdagData) GetBlueScore() uint64 {
@ -2543,7 +2598,7 @@ type BluesAnticoneSizes struct {
func (x *BluesAnticoneSizes) Reset() {
*x = BluesAnticoneSizes{}
if protoimpl.UnsafeEnabled {
mi := &file_p2p_proto_msgTypes[46]
mi := &file_p2p_proto_msgTypes[47]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@ -2556,7 +2611,7 @@ func (x *BluesAnticoneSizes) String() string {
func (*BluesAnticoneSizes) ProtoMessage() {}
func (x *BluesAnticoneSizes) ProtoReflect() protoreflect.Message {
mi := &file_p2p_proto_msgTypes[46]
mi := &file_p2p_proto_msgTypes[47]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@ -2569,7 +2624,7 @@ func (x *BluesAnticoneSizes) ProtoReflect() protoreflect.Message {
// Deprecated: Use BluesAnticoneSizes.ProtoReflect.Descriptor instead.
func (*BluesAnticoneSizes) Descriptor() ([]byte, []int) {
return file_p2p_proto_rawDescGZIP(), []int{46}
return file_p2p_proto_rawDescGZIP(), []int{47}
}
func (x *BluesAnticoneSizes) GetBlueHash() *Hash {
@ -2595,7 +2650,7 @@ type DoneBlocksWithTrustedDataMessage struct {
func (x *DoneBlocksWithTrustedDataMessage) Reset() {
*x = DoneBlocksWithTrustedDataMessage{}
if protoimpl.UnsafeEnabled {
mi := &file_p2p_proto_msgTypes[47]
mi := &file_p2p_proto_msgTypes[48]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@ -2608,7 +2663,7 @@ func (x *DoneBlocksWithTrustedDataMessage) String() string {
func (*DoneBlocksWithTrustedDataMessage) ProtoMessage() {}
func (x *DoneBlocksWithTrustedDataMessage) ProtoReflect() protoreflect.Message {
mi := &file_p2p_proto_msgTypes[47]
mi := &file_p2p_proto_msgTypes[48]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@ -2621,7 +2676,7 @@ func (x *DoneBlocksWithTrustedDataMessage) ProtoReflect() protoreflect.Message {
// Deprecated: Use DoneBlocksWithTrustedDataMessage.ProtoReflect.Descriptor instead.
func (*DoneBlocksWithTrustedDataMessage) Descriptor() ([]byte, []int) {
return file_p2p_proto_rawDescGZIP(), []int{47}
return file_p2p_proto_rawDescGZIP(), []int{48}
}
type PruningPointsMessage struct {
@ -2635,7 +2690,7 @@ type PruningPointsMessage struct {
func (x *PruningPointsMessage) Reset() {
*x = PruningPointsMessage{}
if protoimpl.UnsafeEnabled {
mi := &file_p2p_proto_msgTypes[48]
mi := &file_p2p_proto_msgTypes[49]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@ -2648,7 +2703,7 @@ func (x *PruningPointsMessage) String() string {
func (*PruningPointsMessage) ProtoMessage() {}
func (x *PruningPointsMessage) ProtoReflect() protoreflect.Message {
mi := &file_p2p_proto_msgTypes[48]
mi := &file_p2p_proto_msgTypes[49]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@ -2661,7 +2716,7 @@ func (x *PruningPointsMessage) ProtoReflect() protoreflect.Message {
// Deprecated: Use PruningPointsMessage.ProtoReflect.Descriptor instead.
func (*PruningPointsMessage) Descriptor() ([]byte, []int) {
return file_p2p_proto_rawDescGZIP(), []int{48}
return file_p2p_proto_rawDescGZIP(), []int{49}
}
func (x *PruningPointsMessage) GetHeaders() []*BlockHeader {
@ -2680,7 +2735,7 @@ type RequestPruningPointProofMessage struct {
func (x *RequestPruningPointProofMessage) Reset() {
*x = RequestPruningPointProofMessage{}
if protoimpl.UnsafeEnabled {
mi := &file_p2p_proto_msgTypes[49]
mi := &file_p2p_proto_msgTypes[50]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@ -2693,7 +2748,7 @@ func (x *RequestPruningPointProofMessage) String() string {
func (*RequestPruningPointProofMessage) ProtoMessage() {}
func (x *RequestPruningPointProofMessage) ProtoReflect() protoreflect.Message {
mi := &file_p2p_proto_msgTypes[49]
mi := &file_p2p_proto_msgTypes[50]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@ -2706,7 +2761,7 @@ func (x *RequestPruningPointProofMessage) ProtoReflect() protoreflect.Message {
// Deprecated: Use RequestPruningPointProofMessage.ProtoReflect.Descriptor instead.
func (*RequestPruningPointProofMessage) Descriptor() ([]byte, []int) {
return file_p2p_proto_rawDescGZIP(), []int{49}
return file_p2p_proto_rawDescGZIP(), []int{50}
}
type PruningPointProofMessage struct {
@ -2720,7 +2775,7 @@ type PruningPointProofMessage struct {
func (x *PruningPointProofMessage) Reset() {
*x = PruningPointProofMessage{}
if protoimpl.UnsafeEnabled {
mi := &file_p2p_proto_msgTypes[50]
mi := &file_p2p_proto_msgTypes[51]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@ -2733,7 +2788,7 @@ func (x *PruningPointProofMessage) String() string {
func (*PruningPointProofMessage) ProtoMessage() {}
func (x *PruningPointProofMessage) ProtoReflect() protoreflect.Message {
mi := &file_p2p_proto_msgTypes[50]
mi := &file_p2p_proto_msgTypes[51]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@ -2746,7 +2801,7 @@ func (x *PruningPointProofMessage) ProtoReflect() protoreflect.Message {
// Deprecated: Use PruningPointProofMessage.ProtoReflect.Descriptor instead.
func (*PruningPointProofMessage) Descriptor() ([]byte, []int) {
return file_p2p_proto_rawDescGZIP(), []int{50}
return file_p2p_proto_rawDescGZIP(), []int{51}
}
func (x *PruningPointProofMessage) GetHeaders() []*PruningPointProofHeaderArray {
@ -2767,7 +2822,7 @@ type PruningPointProofHeaderArray struct {
func (x *PruningPointProofHeaderArray) Reset() {
*x = PruningPointProofHeaderArray{}
if protoimpl.UnsafeEnabled {
mi := &file_p2p_proto_msgTypes[51]
mi := &file_p2p_proto_msgTypes[52]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@ -2780,7 +2835,7 @@ func (x *PruningPointProofHeaderArray) String() string {
func (*PruningPointProofHeaderArray) ProtoMessage() {}
func (x *PruningPointProofHeaderArray) ProtoReflect() protoreflect.Message {
mi := &file_p2p_proto_msgTypes[51]
mi := &file_p2p_proto_msgTypes[52]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@ -2793,7 +2848,7 @@ func (x *PruningPointProofHeaderArray) ProtoReflect() protoreflect.Message {
// Deprecated: Use PruningPointProofHeaderArray.ProtoReflect.Descriptor instead.
func (*PruningPointProofHeaderArray) Descriptor() ([]byte, []int) {
return file_p2p_proto_rawDescGZIP(), []int{51}
return file_p2p_proto_rawDescGZIP(), []int{52}
}
func (x *PruningPointProofHeaderArray) GetHeaders() []*BlockHeader {
@ -2812,7 +2867,7 @@ type ReadyMessage struct {
func (x *ReadyMessage) Reset() {
*x = ReadyMessage{}
if protoimpl.UnsafeEnabled {
mi := &file_p2p_proto_msgTypes[52]
mi := &file_p2p_proto_msgTypes[53]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@ -2825,7 +2880,7 @@ func (x *ReadyMessage) String() string {
func (*ReadyMessage) ProtoMessage() {}
func (x *ReadyMessage) ProtoReflect() protoreflect.Message {
mi := &file_p2p_proto_msgTypes[52]
mi := &file_p2p_proto_msgTypes[53]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@ -2838,7 +2893,125 @@ func (x *ReadyMessage) ProtoReflect() protoreflect.Message {
// Deprecated: Use ReadyMessage.ProtoReflect.Descriptor instead.
func (*ReadyMessage) Descriptor() ([]byte, []int) {
return file_p2p_proto_rawDescGZIP(), []int{52}
return file_p2p_proto_rawDescGZIP(), []int{53}
}
type BlockWithTrustedDataV4Message struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
Block *BlockMessage `protobuf:"bytes,1,opt,name=block,proto3" json:"block,omitempty"`
DaaWindowIndices []uint64 `protobuf:"varint,2,rep,packed,name=daaWindowIndices,proto3" json:"daaWindowIndices,omitempty"`
GhostdagDataIndices []uint64 `protobuf:"varint,3,rep,packed,name=ghostdagDataIndices,proto3" json:"ghostdagDataIndices,omitempty"`
}
func (x *BlockWithTrustedDataV4Message) Reset() {
*x = BlockWithTrustedDataV4Message{}
if protoimpl.UnsafeEnabled {
mi := &file_p2p_proto_msgTypes[54]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *BlockWithTrustedDataV4Message) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*BlockWithTrustedDataV4Message) ProtoMessage() {}
func (x *BlockWithTrustedDataV4Message) ProtoReflect() protoreflect.Message {
mi := &file_p2p_proto_msgTypes[54]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use BlockWithTrustedDataV4Message.ProtoReflect.Descriptor instead.
func (*BlockWithTrustedDataV4Message) Descriptor() ([]byte, []int) {
return file_p2p_proto_rawDescGZIP(), []int{54}
}
func (x *BlockWithTrustedDataV4Message) GetBlock() *BlockMessage {
if x != nil {
return x.Block
}
return nil
}
func (x *BlockWithTrustedDataV4Message) GetDaaWindowIndices() []uint64 {
if x != nil {
return x.DaaWindowIndices
}
return nil
}
func (x *BlockWithTrustedDataV4Message) GetGhostdagDataIndices() []uint64 {
if x != nil {
return x.GhostdagDataIndices
}
return nil
}
type TrustedDataMessage struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
DaaWindow []*DaaBlockV4 `protobuf:"bytes,1,rep,name=daaWindow,proto3" json:"daaWindow,omitempty"`
GhostdagData []*BlockGhostdagDataHashPair `protobuf:"bytes,2,rep,name=ghostdagData,proto3" json:"ghostdagData,omitempty"`
}
func (x *TrustedDataMessage) Reset() {
*x = TrustedDataMessage{}
if protoimpl.UnsafeEnabled {
mi := &file_p2p_proto_msgTypes[55]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *TrustedDataMessage) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*TrustedDataMessage) ProtoMessage() {}
func (x *TrustedDataMessage) ProtoReflect() protoreflect.Message {
mi := &file_p2p_proto_msgTypes[55]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use TrustedDataMessage.ProtoReflect.Descriptor instead.
func (*TrustedDataMessage) Descriptor() ([]byte, []int) {
return file_p2p_proto_rawDescGZIP(), []int{55}
}
func (x *TrustedDataMessage) GetDaaWindow() []*DaaBlockV4 {
if x != nil {
return x.DaaWindow
}
return nil
}
func (x *TrustedDataMessage) GetGhostdagData() []*BlockGhostdagDataHashPair {
if x != nil {
return x.GhostdagData
}
return nil
}
var File_p2p_proto protoreflect.FileDescriptor
@ -3126,62 +3299,90 @@ var file_p2p_proto_rawDesc = []byte{
0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
0x77, 0x69, 0x72, 0x65, 0x2e, 0x47, 0x68, 0x6f, 0x73, 0x74, 0x64, 0x61, 0x67, 0x44, 0x61, 0x74,
0x61, 0x52, 0x0c, 0x67, 0x68, 0x6f, 0x73, 0x74, 0x64, 0x61, 0x67, 0x44, 0x61, 0x74, 0x61, 0x22,
0x7d, 0x0a, 0x19, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x47, 0x68, 0x6f, 0x73, 0x74, 0x64, 0x61, 0x67,
0x44, 0x61, 0x74, 0x61, 0x48, 0x61, 0x73, 0x68, 0x50, 0x61, 0x69, 0x72, 0x12, 0x23, 0x0a, 0x04,
0x68, 0x61, 0x73, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x70, 0x72, 0x6f,
0x74, 0x6f, 0x77, 0x69, 0x72, 0x65, 0x2e, 0x48, 0x61, 0x73, 0x68, 0x52, 0x04, 0x68, 0x61, 0x73,
0x68, 0x12, 0x3b, 0x0a, 0x0c, 0x67, 0x68, 0x6f, 0x73, 0x74, 0x64, 0x61, 0x67, 0x44, 0x61, 0x74,
0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x77,
0x69, 0x72, 0x65, 0x2e, 0x47, 0x68, 0x6f, 0x73, 0x74, 0x64, 0x61, 0x67, 0x44, 0x61, 0x74, 0x61,
0x52, 0x0c, 0x67, 0x68, 0x6f, 0x73, 0x74, 0x64, 0x61, 0x67, 0x44, 0x61, 0x74, 0x61, 0x22, 0xbc,
0x02, 0x0a, 0x0c, 0x47, 0x68, 0x6f, 0x73, 0x74, 0x64, 0x61, 0x67, 0x44, 0x61, 0x74, 0x61, 0x12,
0x1c, 0x0a, 0x09, 0x62, 0x6c, 0x75, 0x65, 0x53, 0x63, 0x6f, 0x72, 0x65, 0x18, 0x01, 0x20, 0x01,
0x28, 0x04, 0x52, 0x09, 0x62, 0x6c, 0x75, 0x65, 0x53, 0x63, 0x6f, 0x72, 0x65, 0x12, 0x1a, 0x0a,
0x08, 0x62, 0x6c, 0x75, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52,
0x08, 0x62, 0x6c, 0x75, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x12, 0x37, 0x0a, 0x0e, 0x73, 0x65, 0x6c,
0x65, 0x63, 0x74, 0x65, 0x64, 0x50, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28,
0x0b, 0x32, 0x0f, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x77, 0x69, 0x72, 0x65, 0x2e, 0x48, 0x61,
0x73, 0x68, 0x52, 0x0e, 0x73, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x65, 0x64, 0x50, 0x61, 0x72, 0x65,
0x6e, 0x74, 0x12, 0x35, 0x0a, 0x0d, 0x6d, 0x65, 0x72, 0x67, 0x65, 0x53, 0x65, 0x74, 0x42, 0x6c,
0x75, 0x65, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x70, 0x72, 0x6f, 0x74,
0x6f, 0x77, 0x69, 0x72, 0x65, 0x2e, 0x48, 0x61, 0x73, 0x68, 0x52, 0x0d, 0x6d, 0x65, 0x72, 0x67,
0x65, 0x53, 0x65, 0x74, 0x42, 0x6c, 0x75, 0x65, 0x73, 0x12, 0x33, 0x0a, 0x0c, 0x6d, 0x65, 0x72,
0x67, 0x65, 0x53, 0x65, 0x74, 0x52, 0x65, 0x64, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32,
0x0f, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x77, 0x69, 0x72, 0x65, 0x2e, 0x48, 0x61, 0x73, 0x68,
0x52, 0x0c, 0x6d, 0x65, 0x72, 0x67, 0x65, 0x53, 0x65, 0x74, 0x52, 0x65, 0x64, 0x73, 0x12, 0x4d,
0x0a, 0x12, 0x62, 0x6c, 0x75, 0x65, 0x73, 0x41, 0x6e, 0x74, 0x69, 0x63, 0x6f, 0x6e, 0x65, 0x53,
0x69, 0x7a, 0x65, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x70, 0x72, 0x6f,
0x74, 0x6f, 0x77, 0x69, 0x72, 0x65, 0x2e, 0x42, 0x6c, 0x75, 0x65, 0x73, 0x41, 0x6e, 0x74, 0x69,
0x63, 0x6f, 0x6e, 0x65, 0x53, 0x69, 0x7a, 0x65, 0x73, 0x52, 0x12, 0x62, 0x6c, 0x75, 0x65, 0x73,
0x41, 0x6e, 0x74, 0x69, 0x63, 0x6f, 0x6e, 0x65, 0x53, 0x69, 0x7a, 0x65, 0x73, 0x22, 0x65, 0x0a,
0x12, 0x42, 0x6c, 0x75, 0x65, 0x73, 0x41, 0x6e, 0x74, 0x69, 0x63, 0x6f, 0x6e, 0x65, 0x53, 0x69,
0x7a, 0x65, 0x73, 0x12, 0x2b, 0x0a, 0x08, 0x62, 0x6c, 0x75, 0x65, 0x48, 0x61, 0x73, 0x68, 0x18,
0x79, 0x0a, 0x0a, 0x44, 0x61, 0x61, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x56, 0x34, 0x12, 0x2e, 0x0a,
0x06, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e,
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x77, 0x69, 0x72, 0x65, 0x2e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x48,
0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x06, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x3b, 0x0a,
0x0c, 0x67, 0x68, 0x6f, 0x73, 0x74, 0x64, 0x61, 0x67, 0x44, 0x61, 0x74, 0x61, 0x18, 0x02, 0x20,
0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x77, 0x69, 0x72, 0x65, 0x2e,
0x47, 0x68, 0x6f, 0x73, 0x74, 0x64, 0x61, 0x67, 0x44, 0x61, 0x74, 0x61, 0x52, 0x0c, 0x67, 0x68,
0x6f, 0x73, 0x74, 0x64, 0x61, 0x67, 0x44, 0x61, 0x74, 0x61, 0x22, 0x7d, 0x0a, 0x19, 0x42, 0x6c,
0x6f, 0x63, 0x6b, 0x47, 0x68, 0x6f, 0x73, 0x74, 0x64, 0x61, 0x67, 0x44, 0x61, 0x74, 0x61, 0x48,
0x61, 0x73, 0x68, 0x50, 0x61, 0x69, 0x72, 0x12, 0x23, 0x0a, 0x04, 0x68, 0x61, 0x73, 0x68, 0x18,
0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x77, 0x69, 0x72,
0x65, 0x2e, 0x48, 0x61, 0x73, 0x68, 0x52, 0x08, 0x62, 0x6c, 0x75, 0x65, 0x48, 0x61, 0x73, 0x68,
0x12, 0x22, 0x0a, 0x0c, 0x61, 0x6e, 0x74, 0x69, 0x63, 0x6f, 0x6e, 0x65, 0x53, 0x69, 0x7a, 0x65,
0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0c, 0x61, 0x6e, 0x74, 0x69, 0x63, 0x6f, 0x6e, 0x65,
0x53, 0x69, 0x7a, 0x65, 0x22, 0x22, 0x0a, 0x20, 0x44, 0x6f, 0x6e, 0x65, 0x42, 0x6c, 0x6f, 0x63,
0x6b, 0x73, 0x57, 0x69, 0x74, 0x68, 0x54, 0x72, 0x75, 0x73, 0x74, 0x65, 0x64, 0x44, 0x61, 0x74,
0x61, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x48, 0x0a, 0x14, 0x50, 0x72, 0x75, 0x6e,
0x69, 0x6e, 0x67, 0x50, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
0x12, 0x30, 0x0a, 0x07, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28,
0x0b, 0x32, 0x16, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x77, 0x69, 0x72, 0x65, 0x2e, 0x42, 0x6c,
0x6f, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x07, 0x68, 0x65, 0x61, 0x64, 0x65,
0x72, 0x73, 0x22, 0x21, 0x0a, 0x1f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x50, 0x72, 0x75,
0x6e, 0x69, 0x6e, 0x67, 0x50, 0x6f, 0x69, 0x6e, 0x74, 0x50, 0x72, 0x6f, 0x6f, 0x66, 0x4d, 0x65,
0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x5d, 0x0a, 0x18, 0x50, 0x72, 0x75, 0x6e, 0x69, 0x6e, 0x67,
0x50, 0x6f, 0x69, 0x6e, 0x74, 0x50, 0x72, 0x6f, 0x6f, 0x66, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67,
0x65, 0x12, 0x41, 0x0a, 0x07, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x18, 0x01, 0x20, 0x03,
0x28, 0x0b, 0x32, 0x27, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x77, 0x69, 0x72, 0x65, 0x2e, 0x50,
0x72, 0x75, 0x6e, 0x69, 0x6e, 0x67, 0x50, 0x6f, 0x69, 0x6e, 0x74, 0x50, 0x72, 0x6f, 0x6f, 0x66,
0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x41, 0x72, 0x72, 0x61, 0x79, 0x52, 0x07, 0x68, 0x65, 0x61,
0x64, 0x65, 0x72, 0x73, 0x22, 0x50, 0x0a, 0x1c, 0x50, 0x72, 0x75, 0x6e, 0x69, 0x6e, 0x67, 0x50,
0x6f, 0x69, 0x6e, 0x74, 0x50, 0x72, 0x6f, 0x6f, 0x66, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x41,
0x72, 0x72, 0x61, 0x79, 0x12, 0x30, 0x0a, 0x07, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x18,
0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x77, 0x69, 0x72,
0x65, 0x2e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x07, 0x68,
0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x22, 0x0e, 0x0a, 0x0c, 0x52, 0x65, 0x61, 0x64, 0x79, 0x4d,
0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x42, 0x26, 0x5a, 0x24, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62,
0x65, 0x2e, 0x48, 0x61, 0x73, 0x68, 0x52, 0x04, 0x68, 0x61, 0x73, 0x68, 0x12, 0x3b, 0x0a, 0x0c,
0x67, 0x68, 0x6f, 0x73, 0x74, 0x64, 0x61, 0x67, 0x44, 0x61, 0x74, 0x61, 0x18, 0x02, 0x20, 0x01,
0x28, 0x0b, 0x32, 0x17, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x77, 0x69, 0x72, 0x65, 0x2e, 0x47,
0x68, 0x6f, 0x73, 0x74, 0x64, 0x61, 0x67, 0x44, 0x61, 0x74, 0x61, 0x52, 0x0c, 0x67, 0x68, 0x6f,
0x73, 0x74, 0x64, 0x61, 0x67, 0x44, 0x61, 0x74, 0x61, 0x22, 0xbc, 0x02, 0x0a, 0x0c, 0x47, 0x68,
0x6f, 0x73, 0x74, 0x64, 0x61, 0x67, 0x44, 0x61, 0x74, 0x61, 0x12, 0x1c, 0x0a, 0x09, 0x62, 0x6c,
0x75, 0x65, 0x53, 0x63, 0x6f, 0x72, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x09, 0x62,
0x6c, 0x75, 0x65, 0x53, 0x63, 0x6f, 0x72, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x62, 0x6c, 0x75, 0x65,
0x57, 0x6f, 0x72, 0x6b, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x08, 0x62, 0x6c, 0x75, 0x65,
0x57, 0x6f, 0x72, 0x6b, 0x12, 0x37, 0x0a, 0x0e, 0x73, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x65, 0x64,
0x50, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x70,
0x72, 0x6f, 0x74, 0x6f, 0x77, 0x69, 0x72, 0x65, 0x2e, 0x48, 0x61, 0x73, 0x68, 0x52, 0x0e, 0x73,
0x65, 0x6c, 0x65, 0x63, 0x74, 0x65, 0x64, 0x50, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x12, 0x35, 0x0a,
0x0d, 0x6d, 0x65, 0x72, 0x67, 0x65, 0x53, 0x65, 0x74, 0x42, 0x6c, 0x75, 0x65, 0x73, 0x18, 0x04,
0x20, 0x03, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x77, 0x69, 0x72, 0x65,
0x2e, 0x48, 0x61, 0x73, 0x68, 0x52, 0x0d, 0x6d, 0x65, 0x72, 0x67, 0x65, 0x53, 0x65, 0x74, 0x42,
0x6c, 0x75, 0x65, 0x73, 0x12, 0x33, 0x0a, 0x0c, 0x6d, 0x65, 0x72, 0x67, 0x65, 0x53, 0x65, 0x74,
0x52, 0x65, 0x64, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x70, 0x72, 0x6f,
0x74, 0x6f, 0x77, 0x69, 0x72, 0x65, 0x2e, 0x48, 0x61, 0x73, 0x68, 0x52, 0x0c, 0x6d, 0x65, 0x72,
0x67, 0x65, 0x53, 0x65, 0x74, 0x52, 0x65, 0x64, 0x73, 0x12, 0x4d, 0x0a, 0x12, 0x62, 0x6c, 0x75,
0x65, 0x73, 0x41, 0x6e, 0x74, 0x69, 0x63, 0x6f, 0x6e, 0x65, 0x53, 0x69, 0x7a, 0x65, 0x73, 0x18,
0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x77, 0x69, 0x72,
0x65, 0x2e, 0x42, 0x6c, 0x75, 0x65, 0x73, 0x41, 0x6e, 0x74, 0x69, 0x63, 0x6f, 0x6e, 0x65, 0x53,
0x69, 0x7a, 0x65, 0x73, 0x52, 0x12, 0x62, 0x6c, 0x75, 0x65, 0x73, 0x41, 0x6e, 0x74, 0x69, 0x63,
0x6f, 0x6e, 0x65, 0x53, 0x69, 0x7a, 0x65, 0x73, 0x22, 0x65, 0x0a, 0x12, 0x42, 0x6c, 0x75, 0x65,
0x73, 0x41, 0x6e, 0x74, 0x69, 0x63, 0x6f, 0x6e, 0x65, 0x53, 0x69, 0x7a, 0x65, 0x73, 0x12, 0x2b,
0x0a, 0x08, 0x62, 0x6c, 0x75, 0x65, 0x48, 0x61, 0x73, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b,
0x32, 0x0f, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x77, 0x69, 0x72, 0x65, 0x2e, 0x48, 0x61, 0x73,
0x68, 0x52, 0x08, 0x62, 0x6c, 0x75, 0x65, 0x48, 0x61, 0x73, 0x68, 0x12, 0x22, 0x0a, 0x0c, 0x61,
0x6e, 0x74, 0x69, 0x63, 0x6f, 0x6e, 0x65, 0x53, 0x69, 0x7a, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28,
0x0d, 0x52, 0x0c, 0x61, 0x6e, 0x74, 0x69, 0x63, 0x6f, 0x6e, 0x65, 0x53, 0x69, 0x7a, 0x65, 0x22,
0x22, 0x0a, 0x20, 0x44, 0x6f, 0x6e, 0x65, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x73, 0x57, 0x69, 0x74,
0x68, 0x54, 0x72, 0x75, 0x73, 0x74, 0x65, 0x64, 0x44, 0x61, 0x74, 0x61, 0x4d, 0x65, 0x73, 0x73,
0x61, 0x67, 0x65, 0x22, 0x48, 0x0a, 0x14, 0x50, 0x72, 0x75, 0x6e, 0x69, 0x6e, 0x67, 0x50, 0x6f,
0x69, 0x6e, 0x74, 0x73, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x30, 0x0a, 0x07, 0x68,
0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x70,
0x72, 0x6f, 0x74, 0x6f, 0x77, 0x69, 0x72, 0x65, 0x2e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x48, 0x65,
0x61, 0x64, 0x65, 0x72, 0x52, 0x07, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x22, 0x21, 0x0a,
0x1f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x50, 0x72, 0x75, 0x6e, 0x69, 0x6e, 0x67, 0x50,
0x6f, 0x69, 0x6e, 0x74, 0x50, 0x72, 0x6f, 0x6f, 0x66, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
0x22, 0x5d, 0x0a, 0x18, 0x50, 0x72, 0x75, 0x6e, 0x69, 0x6e, 0x67, 0x50, 0x6f, 0x69, 0x6e, 0x74,
0x50, 0x72, 0x6f, 0x6f, 0x66, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x41, 0x0a, 0x07,
0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x27, 0x2e,
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x77, 0x69, 0x72, 0x65, 0x2e, 0x50, 0x72, 0x75, 0x6e, 0x69, 0x6e,
0x67, 0x50, 0x6f, 0x69, 0x6e, 0x74, 0x50, 0x72, 0x6f, 0x6f, 0x66, 0x48, 0x65, 0x61, 0x64, 0x65,
0x72, 0x41, 0x72, 0x72, 0x61, 0x79, 0x52, 0x07, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x22,
0x50, 0x0a, 0x1c, 0x50, 0x72, 0x75, 0x6e, 0x69, 0x6e, 0x67, 0x50, 0x6f, 0x69, 0x6e, 0x74, 0x50,
0x72, 0x6f, 0x6f, 0x66, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x41, 0x72, 0x72, 0x61, 0x79, 0x12,
0x30, 0x0a, 0x07, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b,
0x32, 0x16, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x77, 0x69, 0x72, 0x65, 0x2e, 0x42, 0x6c, 0x6f,
0x63, 0x6b, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x07, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72,
0x73, 0x22, 0x0e, 0x0a, 0x0c, 0x52, 0x65, 0x61, 0x64, 0x79, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67,
0x65, 0x22, 0xac, 0x01, 0x0a, 0x1d, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x57, 0x69, 0x74, 0x68, 0x54,
0x72, 0x75, 0x73, 0x74, 0x65, 0x64, 0x44, 0x61, 0x74, 0x61, 0x56, 0x34, 0x4d, 0x65, 0x73, 0x73,
0x61, 0x67, 0x65, 0x12, 0x2d, 0x0a, 0x05, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x18, 0x01, 0x20, 0x01,
0x28, 0x0b, 0x32, 0x17, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x77, 0x69, 0x72, 0x65, 0x2e, 0x42,
0x6c, 0x6f, 0x63, 0x6b, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x05, 0x62, 0x6c, 0x6f,
0x63, 0x6b, 0x12, 0x2a, 0x0a, 0x10, 0x64, 0x61, 0x61, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x49,
0x6e, 0x64, 0x69, 0x63, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x04, 0x52, 0x10, 0x64, 0x61,
0x61, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x49, 0x6e, 0x64, 0x69, 0x63, 0x65, 0x73, 0x12, 0x30,
0x0a, 0x13, 0x67, 0x68, 0x6f, 0x73, 0x74, 0x64, 0x61, 0x67, 0x44, 0x61, 0x74, 0x61, 0x49, 0x6e,
0x64, 0x69, 0x63, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x04, 0x52, 0x13, 0x67, 0x68, 0x6f,
0x73, 0x74, 0x64, 0x61, 0x67, 0x44, 0x61, 0x74, 0x61, 0x49, 0x6e, 0x64, 0x69, 0x63, 0x65, 0x73,
0x22, 0x93, 0x01, 0x0a, 0x12, 0x54, 0x72, 0x75, 0x73, 0x74, 0x65, 0x64, 0x44, 0x61, 0x74, 0x61,
0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x33, 0x0a, 0x09, 0x64, 0x61, 0x61, 0x57, 0x69,
0x6e, 0x64, 0x6f, 0x77, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x70, 0x72, 0x6f,
0x74, 0x6f, 0x77, 0x69, 0x72, 0x65, 0x2e, 0x44, 0x61, 0x61, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x56,
0x34, 0x52, 0x09, 0x64, 0x61, 0x61, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x12, 0x48, 0x0a, 0x0c,
0x67, 0x68, 0x6f, 0x73, 0x74, 0x64, 0x61, 0x67, 0x44, 0x61, 0x74, 0x61, 0x18, 0x02, 0x20, 0x03,
0x28, 0x0b, 0x32, 0x24, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x77, 0x69, 0x72, 0x65, 0x2e, 0x42,
0x6c, 0x6f, 0x63, 0x6b, 0x47, 0x68, 0x6f, 0x73, 0x74, 0x64, 0x61, 0x67, 0x44, 0x61, 0x74, 0x61,
0x48, 0x61, 0x73, 0x68, 0x50, 0x61, 0x69, 0x72, 0x52, 0x0c, 0x67, 0x68, 0x6f, 0x73, 0x74, 0x64,
0x61, 0x67, 0x44, 0x61, 0x74, 0x61, 0x42, 0x26, 0x5a, 0x24, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62,
0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x6b, 0x61, 0x73, 0x70, 0x61, 0x6e, 0x65, 0x74, 0x2f, 0x6b, 0x61,
0x73, 0x70, 0x61, 0x64, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x77, 0x69, 0x72, 0x65, 0x62, 0x06,
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
@ -3199,7 +3400,7 @@ func file_p2p_proto_rawDescGZIP() []byte {
return file_p2p_proto_rawDescData
}
var file_p2p_proto_msgTypes = make([]protoimpl.MessageInfo, 53)
var file_p2p_proto_msgTypes = make([]protoimpl.MessageInfo, 56)
var file_p2p_proto_goTypes = []interface{}{
(*RequestAddressesMessage)(nil), // 0: protowire.RequestAddressesMessage
(*AddressesMessage)(nil), // 1: protowire.AddressesMessage
@ -3245,15 +3446,18 @@ var file_p2p_proto_goTypes = []interface{}{
(*RequestPruningPointAndItsAnticoneMessage)(nil), // 41: protowire.RequestPruningPointAndItsAnticoneMessage
(*BlockWithTrustedDataMessage)(nil), // 42: protowire.BlockWithTrustedDataMessage
(*DaaBlock)(nil), // 43: protowire.DaaBlock
(*BlockGhostdagDataHashPair)(nil), // 44: protowire.BlockGhostdagDataHashPair
(*GhostdagData)(nil), // 45: protowire.GhostdagData
(*BluesAnticoneSizes)(nil), // 46: protowire.BluesAnticoneSizes
(*DoneBlocksWithTrustedDataMessage)(nil), // 47: protowire.DoneBlocksWithTrustedDataMessage
(*PruningPointsMessage)(nil), // 48: protowire.PruningPointsMessage
(*RequestPruningPointProofMessage)(nil), // 49: protowire.RequestPruningPointProofMessage
(*PruningPointProofMessage)(nil), // 50: protowire.PruningPointProofMessage
(*PruningPointProofHeaderArray)(nil), // 51: protowire.PruningPointProofHeaderArray
(*ReadyMessage)(nil), // 52: protowire.ReadyMessage
(*DaaBlockV4)(nil), // 44: protowire.DaaBlockV4
(*BlockGhostdagDataHashPair)(nil), // 45: protowire.BlockGhostdagDataHashPair
(*GhostdagData)(nil), // 46: protowire.GhostdagData
(*BluesAnticoneSizes)(nil), // 47: protowire.BluesAnticoneSizes
(*DoneBlocksWithTrustedDataMessage)(nil), // 48: protowire.DoneBlocksWithTrustedDataMessage
(*PruningPointsMessage)(nil), // 49: protowire.PruningPointsMessage
(*RequestPruningPointProofMessage)(nil), // 50: protowire.RequestPruningPointProofMessage
(*PruningPointProofMessage)(nil), // 51: protowire.PruningPointProofMessage
(*PruningPointProofHeaderArray)(nil), // 52: protowire.PruningPointProofHeaderArray
(*ReadyMessage)(nil), // 53: protowire.ReadyMessage
(*BlockWithTrustedDataV4Message)(nil), // 54: protowire.BlockWithTrustedDataV4Message
(*TrustedDataMessage)(nil), // 55: protowire.TrustedDataMessage
}
var file_p2p_proto_depIdxs = []int32{
3, // 0: protowire.RequestAddressesMessage.subnetworkId:type_name -> protowire.SubnetworkId
@ -3295,24 +3499,29 @@ var file_p2p_proto_depIdxs = []int32{
11, // 36: protowire.BlockHeadersMessage.blockHeaders:type_name -> protowire.BlockHeader
10, // 37: protowire.BlockWithTrustedDataMessage.block:type_name -> protowire.BlockMessage
43, // 38: protowire.BlockWithTrustedDataMessage.daaWindow:type_name -> protowire.DaaBlock
44, // 39: protowire.BlockWithTrustedDataMessage.ghostdagData:type_name -> protowire.BlockGhostdagDataHashPair
45, // 39: protowire.BlockWithTrustedDataMessage.ghostdagData:type_name -> protowire.BlockGhostdagDataHashPair
10, // 40: protowire.DaaBlock.block:type_name -> protowire.BlockMessage
45, // 41: protowire.DaaBlock.ghostdagData:type_name -> protowire.GhostdagData
13, // 42: protowire.BlockGhostdagDataHashPair.hash:type_name -> protowire.Hash
45, // 43: protowire.BlockGhostdagDataHashPair.ghostdagData:type_name -> protowire.GhostdagData
13, // 44: protowire.GhostdagData.selectedParent:type_name -> protowire.Hash
13, // 45: protowire.GhostdagData.mergeSetBlues:type_name -> protowire.Hash
13, // 46: protowire.GhostdagData.mergeSetReds:type_name -> protowire.Hash
46, // 47: protowire.GhostdagData.bluesAnticoneSizes:type_name -> protowire.BluesAnticoneSizes
13, // 48: protowire.BluesAnticoneSizes.blueHash:type_name -> protowire.Hash
11, // 49: protowire.PruningPointsMessage.headers:type_name -> protowire.BlockHeader
51, // 50: protowire.PruningPointProofMessage.headers:type_name -> protowire.PruningPointProofHeaderArray
11, // 51: protowire.PruningPointProofHeaderArray.headers:type_name -> protowire.BlockHeader
52, // [52:52] is the sub-list for method output_type
52, // [52:52] is the sub-list for method input_type
52, // [52:52] is the sub-list for extension type_name
52, // [52:52] is the sub-list for extension extendee
0, // [0:52] is the sub-list for field type_name
46, // 41: protowire.DaaBlock.ghostdagData:type_name -> protowire.GhostdagData
11, // 42: protowire.DaaBlockV4.header:type_name -> protowire.BlockHeader
46, // 43: protowire.DaaBlockV4.ghostdagData:type_name -> protowire.GhostdagData
13, // 44: protowire.BlockGhostdagDataHashPair.hash:type_name -> protowire.Hash
46, // 45: protowire.BlockGhostdagDataHashPair.ghostdagData:type_name -> protowire.GhostdagData
13, // 46: protowire.GhostdagData.selectedParent:type_name -> protowire.Hash
13, // 47: protowire.GhostdagData.mergeSetBlues:type_name -> protowire.Hash
13, // 48: protowire.GhostdagData.mergeSetReds:type_name -> protowire.Hash
47, // 49: protowire.GhostdagData.bluesAnticoneSizes:type_name -> protowire.BluesAnticoneSizes
13, // 50: protowire.BluesAnticoneSizes.blueHash:type_name -> protowire.Hash
11, // 51: protowire.PruningPointsMessage.headers:type_name -> protowire.BlockHeader
52, // 52: protowire.PruningPointProofMessage.headers:type_name -> protowire.PruningPointProofHeaderArray
11, // 53: protowire.PruningPointProofHeaderArray.headers:type_name -> protowire.BlockHeader
10, // 54: protowire.BlockWithTrustedDataV4Message.block:type_name -> protowire.BlockMessage
44, // 55: protowire.TrustedDataMessage.daaWindow:type_name -> protowire.DaaBlockV4
45, // 56: protowire.TrustedDataMessage.ghostdagData:type_name -> protowire.BlockGhostdagDataHashPair
57, // [57:57] is the sub-list for method output_type
57, // [57:57] is the sub-list for method input_type
57, // [57:57] is the sub-list for extension type_name
57, // [57:57] is the sub-list for extension extendee
0, // [0:57] is the sub-list for field type_name
}
func init() { file_p2p_proto_init() }
@ -3850,7 +4059,7 @@ func file_p2p_proto_init() {
}
}
file_p2p_proto_msgTypes[44].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*BlockGhostdagDataHashPair); i {
switch v := v.(*DaaBlockV4); i {
case 0:
return &v.state
case 1:
@ -3862,7 +4071,7 @@ func file_p2p_proto_init() {
}
}
file_p2p_proto_msgTypes[45].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*GhostdagData); i {
switch v := v.(*BlockGhostdagDataHashPair); i {
case 0:
return &v.state
case 1:
@ -3874,7 +4083,7 @@ func file_p2p_proto_init() {
}
}
file_p2p_proto_msgTypes[46].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*BluesAnticoneSizes); i {
switch v := v.(*GhostdagData); i {
case 0:
return &v.state
case 1:
@ -3886,7 +4095,7 @@ func file_p2p_proto_init() {
}
}
file_p2p_proto_msgTypes[47].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*DoneBlocksWithTrustedDataMessage); i {
switch v := v.(*BluesAnticoneSizes); i {
case 0:
return &v.state
case 1:
@ -3898,7 +4107,7 @@ func file_p2p_proto_init() {
}
}
file_p2p_proto_msgTypes[48].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*PruningPointsMessage); i {
switch v := v.(*DoneBlocksWithTrustedDataMessage); i {
case 0:
return &v.state
case 1:
@ -3910,7 +4119,7 @@ func file_p2p_proto_init() {
}
}
file_p2p_proto_msgTypes[49].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*RequestPruningPointProofMessage); i {
switch v := v.(*PruningPointsMessage); i {
case 0:
return &v.state
case 1:
@ -3922,7 +4131,7 @@ func file_p2p_proto_init() {
}
}
file_p2p_proto_msgTypes[50].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*PruningPointProofMessage); i {
switch v := v.(*RequestPruningPointProofMessage); i {
case 0:
return &v.state
case 1:
@ -3934,7 +4143,7 @@ func file_p2p_proto_init() {
}
}
file_p2p_proto_msgTypes[51].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*PruningPointProofHeaderArray); i {
switch v := v.(*PruningPointProofMessage); i {
case 0:
return &v.state
case 1:
@ -3946,6 +4155,18 @@ func file_p2p_proto_init() {
}
}
file_p2p_proto_msgTypes[52].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*PruningPointProofHeaderArray); i {
case 0:
return &v.state
case 1:
return &v.sizeCache
case 2:
return &v.unknownFields
default:
return nil
}
}
file_p2p_proto_msgTypes[53].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*ReadyMessage); i {
case 0:
return &v.state
@ -3957,6 +4178,30 @@ func file_p2p_proto_init() {
return nil
}
}
file_p2p_proto_msgTypes[54].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*BlockWithTrustedDataV4Message); i {
case 0:
return &v.state
case 1:
return &v.sizeCache
case 2:
return &v.unknownFields
default:
return nil
}
}
file_p2p_proto_msgTypes[55].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*TrustedDataMessage); i {
case 0:
return &v.state
case 1:
return &v.sizeCache
case 2:
return &v.unknownFields
default:
return nil
}
}
}
type x struct{}
out := protoimpl.TypeBuilder{
@ -3964,7 +4209,7 @@ func file_p2p_proto_init() {
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
RawDescriptor: file_p2p_proto_rawDesc,
NumEnums: 0,
NumMessages: 53,
NumMessages: 56,
NumExtensions: 0,
NumServices: 0,
},

View File

@ -216,6 +216,11 @@ message DaaBlock {
GhostdagData ghostdagData = 2;
}
message DaaBlockV4 {
BlockHeader header = 1;
GhostdagData ghostdagData = 2;
}
message BlockGhostdagDataHashPair {
Hash hash = 1;
GhostdagData ghostdagData = 2;
@ -254,4 +259,15 @@ message PruningPointProofHeaderArray {
}
message ReadyMessage {
}
}
message BlockWithTrustedDataV4Message {
BlockMessage block = 1;
repeated uint64 daaWindowIndices = 2;
repeated uint64 ghostdagDataIndices = 3;
}
message TrustedDataMessage {
repeated DaaBlockV4 daaWindow = 1;
repeated BlockGhostdagDataHashPair ghostdagData = 2;
}

View File

@ -0,0 +1,75 @@
package protowire
import (
"github.com/kaspanet/kaspad/app/appmessage"
"github.com/pkg/errors"
)
func (x *KaspadMessage_BlockWithTrustedDataV4) toAppMessage() (appmessage.Message, error) {
if x == nil {
return nil, errors.Wrapf(errorNil, "KaspadMessage_BlockWithTrustedDataV4 is nil")
}
msgBlock, err := x.BlockWithTrustedDataV4.Block.toAppMessage()
if err != nil {
return nil, err
}
return &appmessage.MsgBlockWithTrustedDataV4{
Block: msgBlock,
DAAWindowIndices: x.BlockWithTrustedDataV4.DaaWindowIndices,
GHOSTDAGDataIndices: x.BlockWithTrustedDataV4.GhostdagDataIndices,
}, nil
}
func (x *KaspadMessage_BlockWithTrustedDataV4) fromAppMessage(msgBlockWithTrustedData *appmessage.MsgBlockWithTrustedDataV4) error {
x.BlockWithTrustedDataV4 = &BlockWithTrustedDataV4Message{
Block: &BlockMessage{},
DaaWindowIndices: msgBlockWithTrustedData.DAAWindowIndices,
GhostdagDataIndices: msgBlockWithTrustedData.GHOSTDAGDataIndices,
}
err := x.BlockWithTrustedDataV4.Block.fromAppMessage(msgBlockWithTrustedData.Block)
if err != nil {
return err
}
return nil
}
func (x *DaaBlockV4) toAppMessage() (*appmessage.TrustedDataDAAHeader, error) {
if x == nil {
return nil, errors.Wrapf(errorNil, "DaaBlockV4 is nil")
}
msgBlockHeader, err := x.Header.toAppMessage()
if err != nil {
return nil, err
}
ghostdagData, err := x.GhostdagData.toAppMessage()
if err != nil {
return nil, err
}
return &appmessage.TrustedDataDAAHeader{
Header: msgBlockHeader,
GHOSTDAGData: ghostdagData,
}, nil
}
func (x *DaaBlockV4) fromAppMessage(daaBlock *appmessage.TrustedDataDAAHeader) error {
*x = DaaBlockV4{
Header: &BlockHeader{},
GhostdagData: &GhostdagData{},
}
err := x.Header.fromAppMessage(daaBlock.Header)
if err != nil {
return err
}
x.GhostdagData.fromAppMessage(daaBlock.GHOSTDAGData)
return nil
}

View File

@ -0,0 +1,70 @@
package protowire
import (
"github.com/kaspanet/kaspad/app/appmessage"
"github.com/pkg/errors"
)
func (x *KaspadMessage_TrustedData) toAppMessage() (appmessage.Message, error) {
if x == nil {
return nil, errors.Wrapf(errorNil, "KaspadMessage_TrustedDataMessage is nil")
}
daaWindow := make([]*appmessage.TrustedDataDAAHeader, len(x.TrustedData.DaaWindow))
for i, daaBlock := range x.TrustedData.DaaWindow {
var err error
daaWindow[i], err = daaBlock.toAppMessage()
if err != nil {
return nil, err
}
}
ghostdagData := make([]*appmessage.BlockGHOSTDAGDataHashPair, len(x.TrustedData.GhostdagData))
for i, pair := range x.TrustedData.GhostdagData {
hash, err := pair.Hash.toDomain()
if err != nil {
return nil, err
}
data, err := pair.GhostdagData.toAppMessage()
if err != nil {
return nil, err
}
ghostdagData[i] = &appmessage.BlockGHOSTDAGDataHashPair{
Hash: hash,
GHOSTDAGData: data,
}
}
return &appmessage.MsgTrustedData{
DAAWindow: daaWindow,
GHOSTDAGData: ghostdagData,
}, nil
}
func (x *KaspadMessage_TrustedData) fromAppMessage(msgTrustedData *appmessage.MsgTrustedData) error {
x.TrustedData = &TrustedDataMessage{
DaaWindow: make([]*DaaBlockV4, len(msgTrustedData.DAAWindow)),
GhostdagData: make([]*BlockGhostdagDataHashPair, len(msgTrustedData.GHOSTDAGData)),
}
for i, daaBlock := range msgTrustedData.DAAWindow {
x.TrustedData.DaaWindow[i] = &DaaBlockV4{}
err := x.TrustedData.DaaWindow[i].fromAppMessage(daaBlock)
if err != nil {
return err
}
}
for i, pair := range msgTrustedData.GHOSTDAGData {
x.TrustedData.GhostdagData[i] = &BlockGhostdagDataHashPair{
Hash: domainHashToProto(pair.Hash),
GhostdagData: &GhostdagData{},
}
x.TrustedData.GhostdagData[i].GhostdagData.fromAppMessage(pair.GHOSTDAGData)
}
return nil
}

View File

@ -317,6 +317,20 @@ func toP2PPayload(message appmessage.Message) (isKaspadMessage_Payload, error) {
return nil, err
}
return payload, nil
case *appmessage.MsgTrustedData:
payload := new(KaspadMessage_TrustedData)
err := payload.fromAppMessage(message)
if err != nil {
return nil, err
}
return payload, nil
case *appmessage.MsgBlockWithTrustedDataV4:
payload := new(KaspadMessage_BlockWithTrustedDataV4)
err := payload.fromAppMessage(message)
if err != nil {
return nil, err
}
return payload, nil
default:
return nil, nil
}

View File

@ -15,6 +15,11 @@ func checkSyncRate(syncerClient, syncedClient *rpc.Client) error {
return err
}
syncerGetSelectedTipHashResponse, err := syncerClient.GetSelectedTipHash()
if err != nil {
return err
}
syncerHeadersCount := syncerBlockCountResponse.HeaderCount
syncerBlockCount := syncerBlockCountResponse.BlockCount
log.Infof("SYNCER block count: %d headers and %d blocks", syncerHeadersCount, syncerBlockCount)
@ -32,8 +37,13 @@ func checkSyncRate(syncerClient, syncedClient *rpc.Client) error {
}
log.Infof("SYNCED block count: %d headers and %d blocks", syncedBlockCountResponse.HeaderCount,
syncedBlockCountResponse.BlockCount)
if syncedBlockCountResponse.BlockCount >= syncerBlockCount &&
syncedBlockCountResponse.HeaderCount >= syncerHeadersCount {
syncedGetSelectedTipHashResponse, err := syncedClient.GetSelectedTipHash()
if err != nil {
return err
}
if syncedGetSelectedTipHashResponse.SelectedTipHash == syncerGetSelectedTipHashResponse.SelectedTipHash {
break
}
if time.Now().After(expectedTime) {

View File

@ -13,10 +13,14 @@ const (
p2pAddress1 = "127.0.0.1:54321"
p2pAddress2 = "127.0.0.1:54322"
p2pAddress3 = "127.0.0.1:54323"
p2pAddress4 = "127.0.0.1:54324"
p2pAddress5 = "127.0.0.1:54325"
rpcAddress1 = "127.0.0.1:12345"
rpcAddress2 = "127.0.0.1:12346"
rpcAddress3 = "127.0.0.1:12347"
rpcAddress4 = "127.0.0.1:12348"
rpcAddress5 = "127.0.0.1:12349"
miningAddress1 = "kaspasim:qqqqnc0pxg7qw3qkc7l6sge8kfhsvvyt7mkw8uamtndqup27ftnd6c769gn66"
miningAddress1PrivateKey = "0d81045b0deb2af36a25403c2154c87aa82d89dd337b575bae27ce7f5de53cee"

View File

@ -175,10 +175,27 @@ func TestIBDWithPruning(t *testing.T) {
overrideDAGParams: &overrideDAGParams,
utxoIndex: true,
},
{
p2pAddress: p2pAddress4,
rpcAddress: rpcAddress4,
miningAddress: miningAddress3,
miningAddressPrivateKey: miningAddress3PrivateKey,
overrideDAGParams: &overrideDAGParams,
utxoIndex: true,
protocolVersion: 3,
},
{
p2pAddress: p2pAddress5,
rpcAddress: rpcAddress5,
miningAddress: miningAddress3,
miningAddressPrivateKey: miningAddress3PrivateKey,
overrideDAGParams: &overrideDAGParams,
utxoIndex: true,
},
})
defer teardown()
syncer, syncee1, syncee2 := harnesses[0], harnesses[1], harnesses[2]
syncer, syncee1, syncee2, synceeV3, synceeV4 := harnesses[0], harnesses[1], harnesses[2], harnesses[3], harnesses[4]
// Let syncee1 have two blocks that the syncer
// doesn't have to test a situation where
@ -199,6 +216,12 @@ func TestIBDWithPruning(t *testing.T) {
// Test a situation where a node with pruned headers syncs another fresh node.
testSync(syncee1, syncee2)
// Test syncing from p2p v4 to p2p v3
testSync(syncee2, synceeV3)
// Test syncing from p2p v3 to p2p v4
testSync(synceeV3, synceeV4)
}
var currentMockTimestamp int64 = 0