Compare commits

...

28 Commits

Author SHA1 Message Date
Ori Newman
38e5e97b2c Merge branch 'unite-reachability-stores' into v0.11.13-rc1 2022-03-11 18:02:25 +02:00
Ori Newman
ad6310de81 Send pruning point and its anticone in batches 2022-03-11 18:00:19 +02:00
msutton
5957c9ecdc go fmt 2022-03-11 17:09:04 +02:00
msutton
3f51f926bb apply Don't relay blocks in virtual anticone #1970 to v5 2022-03-11 15:30:16 +02:00
msutton
91875da221 Merge remote-tracking branch 'origin/dev' into fix-ibd-shared-past-search 2022-03-11 15:27:54 +02:00
msutton
fb95156cd1 rename pastdiff -> anticone 2022-03-11 15:27:03 +02:00
msutton
047fa30183 apply the traversal limit logic and sort headers 2022-03-11 15:05:26 +02:00
msutton
8f68d5c606 addressing simple comments 2022-03-11 14:11:08 +02:00
msutton
63d7dd9f62 missed one err check 2022-03-11 11:26:06 +02:00
msutton
48eb577cac remove unused methods 2022-03-11 11:22:29 +02:00
msutton
a388aefa37 go fmt 2022-03-11 11:18:25 +02:00
msutton
e67b3b65b4 implement ibd sync past diff of relay and selected tip 2022-03-11 11:04:28 +02:00
msutton
6231eaf231 handle and flow for new request past diff message - logic unimplemented yet 2022-03-11 09:19:09 +02:00
msutton
ba20ff3d34 wire for new request past diff message 2022-03-11 08:58:53 +02:00
msutton
7100716aeb protobuf gen for new past diff request message 2022-03-11 08:44:20 +02:00
msutton
c684bf04bc address the case where pruning points disagree, now both IBD tests pass 2022-03-10 17:23:11 +02:00
Ori Newman
05241bdd72 Merge remote-tracking branch 'origin/dev' into unite-reachability-stores 2022-03-10 09:39:03 +02:00
msutton
947d427f40 new ibd shared block algo -- only basic test passing 2022-03-10 03:02:08 +02:00
msutton
398b06d844 wire for new ibd chain locator types 2022-03-10 01:53:37 +02:00
msutton
0b9abf25f4 protobuf gen for new ibd chain locator 2022-03-10 01:53:02 +02:00
msutton
5639e3ab56 set default version to 5 2022-03-10 01:51:37 +02:00
Ori Newman
264334cbfb Add UpdateReindexRoot to RebuildReachability 2022-03-09 23:17:56 +02:00
msutton
840b2a3353 wip 2022-03-08 03:15:20 +02:00
msutton
ea0586045d set all internal imports to v5 2022-03-07 16:37:42 +02:00
msutton
35f86bc98a add p2p v5 which is currently identical to v4 2022-03-07 15:33:52 +02:00
Ori Newman
7a4c4e943b Fix tests 2022-03-03 16:02:22 +02:00
Ori Newman
17c0711d24 Upgrade script 2022-03-03 15:53:30 +02:00
Ori Newman
9887bfa97e Unite all reachability stores 2022-03-03 14:52:17 +02:00
79 changed files with 5926 additions and 1022 deletions

View File

@ -69,6 +69,10 @@ const (
CmdReady
CmdTrustedData
CmdBlockWithTrustedDataV4
CmdRequestNextPruningPointAndItsAnticoneBlocks
CmdRequestIBDChainBlockLocator
CmdIBDChainBlockLocator
CmdRequestAnticone
// rpc
CmdGetCurrentNetworkRequestMessage
@ -195,6 +199,9 @@ var ProtocolMessageCommandToString = map[MessageCommand]string{
CmdReady: "Ready",
CmdTrustedData: "TrustedData",
CmdBlockWithTrustedDataV4: "BlockWithTrustedDataV4",
CmdRequestNextPruningPointAndItsAnticoneBlocks: "RequestNextPruningPointAndItsAnticoneBlocks",
CmdRequestIBDChainBlockLocator: "RequestIBDChainBlockLocator",
CmdIBDChainBlockLocator: "IBDChainBlockLocator",
}
// RPCMessageCommandToString maps all MessageCommands to their string representation

View File

@ -0,0 +1,27 @@
package appmessage
import (
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
)
// MsgIBDChainBlockLocator implements the Message interface and represents a kaspa
// locator message. It is used to find the blockLocator of a peer that is
// syncing with you.
type MsgIBDChainBlockLocator struct {
baseMessage
BlockLocatorHashes []*externalapi.DomainHash
}
// Command returns the protocol command string for the message. This is part
// of the Message interface implementation.
func (msg *MsgIBDChainBlockLocator) Command() MessageCommand {
return CmdIBDChainBlockLocator
}
// NewMsgIBDChainBlockLocator returns a new kaspa locator message that conforms to
// the Message interface. See MsgBlockLocator for details.
func NewMsgIBDChainBlockLocator(locatorHashes []*externalapi.DomainHash) *MsgIBDChainBlockLocator {
return &MsgIBDChainBlockLocator{
BlockLocatorHashes: locatorHashes,
}
}

View File

@ -0,0 +1,33 @@
// Copyright (c) 2013-2016 The btcsuite developers
// Use of this source code is governed by an ISC
// license that can be found in the LICENSE file.
package appmessage
import (
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
)
// MsgRequestAnticone implements the Message interface and represents a kaspa
// RequestHeaders message. It is used to request the set past(ContextHash) \cap anticone(BlockHash)
type MsgRequestAnticone struct {
baseMessage
BlockHash *externalapi.DomainHash
ContextHash *externalapi.DomainHash
}
// Command returns the protocol command string for the message. This is part
// of the Message interface implementation.
func (msg *MsgRequestAnticone) Command() MessageCommand {
return CmdRequestAnticone
}
// NewMsgRequestAnticone returns a new kaspa RequestPastDiff message that conforms to the
// Message interface using the passed parameters and defaults for the remaining
// fields.
func NewMsgRequestAnticone(blockHash, contextHash *externalapi.DomainHash) *MsgRequestAnticone {
return &MsgRequestAnticone{
BlockHash: blockHash,
ContextHash: contextHash,
}
}

View File

@ -0,0 +1,31 @@
package appmessage
import (
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
)
// MsgRequestIBDChainBlockLocator implements the Message interface and represents a kaspa
// IBDRequestChainBlockLocator message. It is used to request a block locator between low
// and high hash.
// The locator is returned via a locator message (MsgIBDChainBlockLocator).
type MsgRequestIBDChainBlockLocator struct {
baseMessage
HighHash *externalapi.DomainHash
LowHash *externalapi.DomainHash
}
// Command returns the protocol command string for the message. This is part
// of the Message interface implementation.
func (msg *MsgRequestIBDChainBlockLocator) Command() MessageCommand {
return CmdRequestIBDChainBlockLocator
}
// NewMsgIBDRequestChainBlockLocator returns a new IBDRequestChainBlockLocator message that conforms to the
// Message interface using the passed parameters and defaults for the remaining
// fields.
func NewMsgIBDRequestChainBlockLocator(highHash, lowHash *externalapi.DomainHash) *MsgRequestIBDChainBlockLocator {
return &MsgRequestIBDChainBlockLocator{
HighHash: highHash,
LowHash: lowHash,
}
}

View File

@ -0,0 +1,22 @@
package appmessage
// MsgRequestNextPruningPointAndItsAnticoneBlocks implements the Message interface and represents a kaspa
// RequestNextPruningPointAndItsAnticoneBlocks message. It is used to notify the IBD syncer peer to send
// more blocks from the pruning anticone.
//
// This message has no payload.
type MsgRequestNextPruningPointAndItsAnticoneBlocks struct {
baseMessage
}
// Command returns the protocol command string for the message. This is part
// of the Message interface implementation.
func (msg *MsgRequestNextPruningPointAndItsAnticoneBlocks) Command() MessageCommand {
return CmdRequestNextPruningPointAndItsAnticoneBlocks
}
// NewMsgRequestNextHeaders returns a new kaspa RequestNextPruningPointAndItsAnticoneBlocks message that conforms to the
// Message interface.
func NewMsgRequestNextPruningPointAndItsAnticoneBlocks() *MsgRequestNextPruningPointAndItsAnticoneBlocks {
return &MsgRequestNextPruningPointAndItsAnticoneBlocks{}
}

View File

@ -22,7 +22,7 @@ func (*FlowContext) HandleError(err error, flowName string, isStopping *uint32,
panic(err)
}
log.Errorf("error from %s: %s", flowName, err)
log.Errorf("error from %s: %+v", flowName, err)
}
if atomic.AddUint32(isStopping, 1) == 1 {

View File

@ -20,7 +20,7 @@ var (
// connected peer may support.
minAcceptableProtocolVersion = uint32(4)
maxAcceptableProtocolVersion = uint32(4)
maxAcceptableProtocolVersion = uint32(5)
)
type receiveVersionFlow struct {

View File

@ -0,0 +1,39 @@
package addressexchange
import (
"github.com/kaspanet/kaspad/app/appmessage"
"github.com/kaspanet/kaspad/app/protocol/common"
peerpkg "github.com/kaspanet/kaspad/app/protocol/peer"
"github.com/kaspanet/kaspad/app/protocol/protocolerrors"
"github.com/kaspanet/kaspad/infrastructure/network/addressmanager"
"github.com/kaspanet/kaspad/infrastructure/network/netadapter/router"
)
// ReceiveAddressesContext is the interface for the context needed for the ReceiveAddresses flow.
type ReceiveAddressesContext interface {
AddressManager() *addressmanager.AddressManager
}
// ReceiveAddresses asks a peer for more addresses if needed.
func ReceiveAddresses(context ReceiveAddressesContext, incomingRoute *router.Route, outgoingRoute *router.Route,
peer *peerpkg.Peer) error {
subnetworkID := peer.SubnetworkID()
msgGetAddresses := appmessage.NewMsgRequestAddresses(false, subnetworkID)
err := outgoingRoute.Enqueue(msgGetAddresses)
if err != nil {
return err
}
message, err := incomingRoute.DequeueWithTimeout(common.DefaultTimeout)
if err != nil {
return err
}
msgAddresses := message.(*appmessage.MsgAddresses)
if len(msgAddresses.AddressList) > addressmanager.GetAddressesMax {
return protocolerrors.Errorf(true, "address count exceeded %d", addressmanager.GetAddressesMax)
}
return context.AddressManager().AddAddresses(msgAddresses.AddressList...)
}

View File

@ -0,0 +1,52 @@
package addressexchange
import (
"math/rand"
"github.com/kaspanet/kaspad/app/appmessage"
"github.com/kaspanet/kaspad/infrastructure/network/addressmanager"
"github.com/kaspanet/kaspad/infrastructure/network/netadapter/router"
)
// SendAddressesContext is the interface for the context needed for the SendAddresses flow.
type SendAddressesContext interface {
AddressManager() *addressmanager.AddressManager
}
// SendAddresses sends addresses to a peer that requests it.
func SendAddresses(context SendAddressesContext, incomingRoute *router.Route, outgoingRoute *router.Route) error {
for {
_, err := incomingRoute.Dequeue()
if err != nil {
return err
}
addresses := context.AddressManager().Addresses()
msgAddresses := appmessage.NewMsgAddresses(shuffleAddresses(addresses))
err = outgoingRoute.Enqueue(msgAddresses)
if err != nil {
return err
}
}
}
// shuffleAddresses randomizes the given addresses sent if there are more than the maximum allowed in one message.
func shuffleAddresses(addresses []*appmessage.NetAddress) []*appmessage.NetAddress {
addressCount := len(addresses)
if addressCount < appmessage.MaxAddressesPerMsg {
return addresses
}
shuffleAddresses := make([]*appmessage.NetAddress, addressCount)
copy(shuffleAddresses, addresses)
rand.Shuffle(addressCount, func(i, j int) {
shuffleAddresses[i], shuffleAddresses[j] = shuffleAddresses[j], shuffleAddresses[i]
})
// Truncate it to the maximum size.
shuffleAddresses = shuffleAddresses[:appmessage.MaxAddressesPerMsg]
return shuffleAddresses
}

View File

@ -0,0 +1,33 @@
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) {
for {
message, err := flow.incomingRoute.DequeueWithTimeout(common.DefaultTimeout)
if err != nil {
return nil, err
}
switch message := message.(type) {
case *appmessage.MsgInvRelayBlock:
flow.invsQueue = append(flow.invsQueue, message)
case *appmessage.MsgBlockLocator:
return message.BlockLocatorHashes, nil
default:
return nil,
protocolerrors.Errorf(true, "received unexpected message type. "+
"expected: %s, got: %s", appmessage.CmdBlockLocator, message.Command())
}
}
}

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,85 @@
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"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/kaspanet/kaspad/infrastructure/network/netadapter/router"
"github.com/pkg/errors"
)
// RequestIBDChainBlockLocatorContext is the interface for the context needed for the HandleRequestBlockLocator flow.
type RequestIBDChainBlockLocatorContext interface {
Domain() domain.Domain
}
type handleRequestIBDChainBlockLocatorFlow struct {
RequestIBDChainBlockLocatorContext
incomingRoute, outgoingRoute *router.Route
}
// HandleRequestIBDChainBlockLocator handles getBlockLocator messages
func HandleRequestIBDChainBlockLocator(context RequestIBDChainBlockLocatorContext, incomingRoute *router.Route,
outgoingRoute *router.Route) error {
flow := &handleRequestIBDChainBlockLocatorFlow{
RequestIBDChainBlockLocatorContext: context,
incomingRoute: incomingRoute,
outgoingRoute: outgoingRoute,
}
return flow.start()
}
func (flow *handleRequestIBDChainBlockLocatorFlow) start() error {
for {
highHash, lowHash, err := flow.receiveRequestIBDChainBlockLocator()
if err != nil {
return err
}
log.Debugf("Received getIBDChainBlockLocator with highHash: %s, lowHash: %s", highHash, lowHash)
var locator externalapi.BlockLocator
if highHash == nil || lowHash == nil {
locator, err = flow.Domain().Consensus().CreateFullHeadersSelectedChainBlockLocator()
} else {
locator, err = flow.Domain().Consensus().CreateHeadersSelectedChainBlockLocator(lowHash, highHash)
if errors.Is(model.ErrBlockNotInSelectedParentChain, err) {
// The chain has been modified, signal it by sending an empty locator
locator, err = externalapi.BlockLocator{}, nil
}
}
if err != nil {
log.Debugf("Received error from CreateHeadersSelectedChainBlockLocator: %s", err)
return protocolerrors.Errorf(true, "couldn't build a block "+
"locator between %s and %s", lowHash, highHash)
}
err = flow.sendIBDChainBlockLocator(locator)
if err != nil {
return err
}
}
}
func (flow *handleRequestIBDChainBlockLocatorFlow) receiveRequestIBDChainBlockLocator() (highHash, lowHash *externalapi.DomainHash, err error) {
message, err := flow.incomingRoute.Dequeue()
if err != nil {
return nil, nil, err
}
msgGetBlockLocator := message.(*appmessage.MsgRequestIBDChainBlockLocator)
return msgGetBlockLocator.HighHash, msgGetBlockLocator.LowHash, nil
}
func (flow *handleRequestIBDChainBlockLocatorFlow) sendIBDChainBlockLocator(locator externalapi.BlockLocator) error {
msgIBDChainBlockLocator := appmessage.NewMsgIBDChainBlockLocator(locator)
err := flow.outgoingRoute.Enqueue(msgIBDChainBlockLocator)
if err != nil {
return err
}
return nil
}

View File

@ -0,0 +1,156 @@
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 i, 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
}
if (i+1)%ibdBatchSize == 0 {
message, err := incomingRoute.Dequeue()
if err != nil {
return err
}
if _, ok := message.(*appmessage.MsgRequestNextPruningPointAndItsAnticoneBlocks); !ok {
return protocolerrors.Errorf(true, "received unexpected message type. "+
"expected: %s, got: %s", appmessage.CmdRequestNextPruningPointAndItsAnticoneBlocks, message.Command())
}
}
}
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,409 @@
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/domain/consensus/utils/hashset"
"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
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),
}
err := flow.start()
// Currently, HandleRelayInvs flow is the only place where IBD is triggered, so the channel can be closed now
close(peer.IBDRequestChannel())
return err
}
func (flow *handleRelayInvsFlow) start() error {
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)
oldVirtualInfo, err := flow.Domain().Consensus().GetVirtualInfo()
if err != nil {
return err
}
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
}
oldVirtualParents := hashset.New()
for _, parent := range oldVirtualInfo.ParentHashes {
oldVirtualParents.Add(parent)
}
newVirtualInfo, err := flow.Domain().Consensus().GetVirtualInfo()
if err != nil {
return err
}
for _, parent := range newVirtualInfo.ParentHashes {
if oldVirtualParents.Contains(parent) {
continue
}
block, err := flow.Domain().Consensus().GetBlock(parent)
if err != nil {
return err
}
blockHash := consensushashing.BlockHash(block)
log.Debugf("Relaying block %s", blockHash)
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)
// Send the block to IBD flow via the IBDRequestChannel.
// Note that this is a non-blocking send, since if IBD is already running, there is no need to trigger it
select {
case flow.peer.IBDRequestChannel() <- block:
default:
}
return nil
}
func (flow *handleRelayInvsFlow) isGenesisVirtualSelectedParent() (bool, error) {
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,98 @@
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/config"
"github.com/kaspanet/kaspad/infrastructure/network/netadapter/router"
"sort"
)
// RequestAnticoneContext is the interface for the context needed for the HandleRequestHeaders flow.
type RequestAnticoneContext interface {
Domain() domain.Domain
Config() *config.Config
}
type handleRequestAnticoneFlow struct {
RequestAnticoneContext
incomingRoute, outgoingRoute *router.Route
peer *peer.Peer
}
// HandleRequestAnticone handles RequestAnticone messages
func HandleRequestAnticone(context RequestAnticoneContext, incomingRoute *router.Route,
outgoingRoute *router.Route, peer *peer.Peer) error {
flow := &handleRequestAnticoneFlow{
RequestAnticoneContext: context,
incomingRoute: incomingRoute,
outgoingRoute: outgoingRoute,
peer: peer,
}
return flow.start()
}
func (flow *handleRequestAnticoneFlow) start() error {
for {
blockHash, contextHash, err := receiveRequestAnticone(flow.incomingRoute)
if err != nil {
return err
}
log.Debugf("Received requestAnticone with blockHash: %s, contextHash: %s", blockHash, contextHash)
log.Debugf("Getting past(%s) setminus past(%s) to %s", contextHash, blockHash, flow.peer)
// GetAnticone is expected to be called by the syncee for getting the anticone of the header selected tip
// intersected by past of relayed block, and is thus expected to be bounded by mergeset limit since
// we relay blocks only if they enter virtual's mergeset. We add 2 for a small margin error.
blockHashes, err := flow.Domain().Consensus().GetAnticone(blockHash, contextHash,
flow.Config().ActiveNetParams.MergeSetSizeLimit+2)
if err != nil {
return protocolerrors.Wrap(true, err, "Failed querying anticone")
}
log.Debugf("Got %d header hashes in past(%s) cap anticone(%s)", len(blockHashes), contextHash, blockHash)
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)
}
// We sort the headers in bottom-up topological order before sending
sort.Slice(blockHeaders, func(i, j int) bool {
return blockHeaders[i].BlueWork.Cmp(blockHeaders[j].BlueWork) < 0
})
if err != nil {
return err
}
blockHeadersMessage := appmessage.NewBlockHeadersMessage(blockHeaders)
err = flow.outgoingRoute.Enqueue(blockHeadersMessage)
if err != nil {
return err
}
err = flow.outgoingRoute.Enqueue(appmessage.NewMsgDoneHeaders())
if err != nil {
return err
}
}
}
func receiveRequestAnticone(incomingRoute *router.Route) (blockHash *externalapi.DomainHash,
contextHash *externalapi.DomainHash, err error) {
message, err := incomingRoute.Dequeue()
if err != nil {
return nil, nil, err
}
msgRequestAnticone := message.(*appmessage.MsgRequestAnticone)
return msgRequestAnticone.BlockHash, msgRequestAnticone.ContextHash, 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 - 1
// 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,633 @@
package blockrelay
import (
"github.com/kaspanet/kaspad/app/appmessage"
"github.com/kaspanet/kaspad/app/protocol/common"
peerpkg "github.com/kaspanet/kaspad/app/protocol/peer"
"github.com/kaspanet/kaspad/app/protocol/protocolerrors"
"github.com/kaspanet/kaspad/domain"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/kaspanet/kaspad/domain/consensus/ruleerrors"
"github.com/kaspanet/kaspad/domain/consensus/utils/consensushashing"
"github.com/kaspanet/kaspad/infrastructure/config"
"github.com/kaspanet/kaspad/infrastructure/logger"
"github.com/kaspanet/kaspad/infrastructure/network/netadapter/router"
"github.com/pkg/errors"
"time"
)
// IBDContext is the interface for the context needed for the HandleIBD flow.
type IBDContext interface {
Domain() domain.Domain
Config() *config.Config
OnNewBlock(block *externalapi.DomainBlock, virtualChangeSet *externalapi.VirtualChangeSet) error
OnVirtualChange(virtualChangeSet *externalapi.VirtualChangeSet) error
OnPruningPointUTXOSetOverride() error
IsIBDRunning() bool
TrySetIBDRunning(ibdPeer *peerpkg.Peer) bool
UnsetIBDRunning()
IsRecoverableError(err error) bool
}
type handleIBDFlow struct {
IBDContext
incomingRoute, outgoingRoute *router.Route
peer *peerpkg.Peer
}
// HandleIBD handles IBD
func HandleIBD(context IBDContext, incomingRoute *router.Route, outgoingRoute *router.Route,
peer *peerpkg.Peer) error {
flow := &handleIBDFlow{
IBDContext: context,
incomingRoute: incomingRoute,
outgoingRoute: outgoingRoute,
peer: peer,
}
return flow.start()
}
func (flow *handleIBDFlow) start() error {
for {
// Wait for IBD requests triggered by other flows
block, ok := <-flow.peer.IBDRequestChannel()
if !ok {
return nil
}
err := flow.runIBDIfNotRunning(block)
if err != nil {
return err
}
}
}
func (flow *handleIBDFlow) runIBDIfNotRunning(block *externalapi.DomainBlock) error {
wasIBDNotRunning := flow.TrySetIBDRunning(flow.peer)
if !wasIBDNotRunning {
log.Debugf("IBD is already running")
return nil
}
isFinishedSuccessfully := false
defer func() {
flow.UnsetIBDRunning()
flow.logIBDFinished(isFinishedSuccessfully)
}()
relayBlockHash := consensushashing.BlockHash(block)
log.Debugf("IBD started with peer %s and relayBlockHash %s", flow.peer, relayBlockHash)
log.Debugf("Syncing blocks up to %s", relayBlockHash)
log.Debugf("Trying to find highest known syncer chain block from peer %s with relay hash %s", flow.peer, relayBlockHash)
/*
Algorithm:
Request full selected chain block locator from syncer
Find the highest block which we know
Repeat the locator step over the new range until finding max(past(syncee) \cap chain(syncer))
*/
// Empty hashes indicate that the full chain is queried
locatorHashes, err := flow.getSyncerChainBlockLocator(nil, nil)
if err != nil {
return err
}
if len(locatorHashes) == 0 {
return protocolerrors.Errorf(true, "Expecting initial syncer chain block locator "+
"to contain at least one element")
}
syncerHeaderSelectedTipHash := locatorHashes[0]
var highestKnownSyncerChainHash *externalapi.DomainHash
for {
var lowestUnknownSyncerChainHash, currentHighestKnownSyncerChainHash *externalapi.DomainHash
for _, syncerChainHash := range locatorHashes {
info, err := flow.Domain().Consensus().GetBlockInfo(syncerChainHash)
if err != nil {
return err
}
if info.Exists {
currentHighestKnownSyncerChainHash = syncerChainHash
break
}
lowestUnknownSyncerChainHash = syncerChainHash
}
// No shared block, break
if currentHighestKnownSyncerChainHash == nil {
break
}
// No point in zooming further
if len(locatorHashes) == 1 {
highestKnownSyncerChainHash = currentHighestKnownSyncerChainHash
break
}
// Zoom in
locatorHashes, err = flow.getSyncerChainBlockLocator(
lowestUnknownSyncerChainHash,
currentHighestKnownSyncerChainHash)
if err != nil {
return err
}
if len(locatorHashes) == 2 {
if !locatorHashes[0].Equal(lowestUnknownSyncerChainHash) ||
!locatorHashes[1].Equal(currentHighestKnownSyncerChainHash) {
return protocolerrors.Errorf(true, "Expecting the high and low "+
"hashes to match the locatorHashes if len(locatorHashes) is 2")
}
// We found our search target
highestKnownSyncerChainHash = currentHighestKnownSyncerChainHash
break
}
if len(locatorHashes) == 0 {
// An empty locator signals that the syncer chain was modified and no longer contains one of
// the queried hashes, so we restart the search
locatorHashes, err = flow.getSyncerChainBlockLocator(nil, nil)
if err != nil {
return err
}
if len(locatorHashes) == 0 {
return protocolerrors.Errorf(true, "Expecting initial syncer chain block locator "+
"to contain at least one element")
}
// Reset syncer's header selected tip
syncerHeaderSelectedTipHash = locatorHashes[0]
}
}
log.Debugf("Found highest known syncer chain block %s from peer %s",
highestKnownSyncerChainHash, flow.peer)
shouldDownloadHeadersProof, shouldSync, err := flow.shouldSyncAndShouldDownloadHeadersProof(
block, highestKnownSyncerChainHash)
if err != nil {
return err
}
if !shouldSync {
return nil
}
if shouldDownloadHeadersProof {
log.Infof("Starting IBD with headers proof")
err := flow.ibdWithHeadersProof(syncerHeaderSelectedTipHash, relayBlockHash, block.Header.DAAScore())
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.", relayBlockHash)
return nil
}
}
// TODO: need DAA score of syncerHeaderSelectedTipHash
err = flow.syncPruningPointFutureHeaders(
flow.Domain().Consensus(),
syncerHeaderSelectedTipHash, highestKnownSyncerChainHash, relayBlockHash, block.Header.DAAScore())
if err != nil {
return err
}
}
err = flow.syncMissingBlockBodies(relayBlockHash)
if err != nil {
return err
}
log.Debugf("Finished syncing blocks up to %s", relayBlockHash)
isFinishedSuccessfully = true
return nil
}
func (flow *handleIBDFlow) isGenesisVirtualSelectedParent() (bool, error) {
virtualSelectedParent, err := flow.Domain().Consensus().GetVirtualSelectedParent()
if err != nil {
return false, err
}
return virtualSelectedParent.Equal(flow.Config().NetParams().GenesisHash), nil
}
func (flow *handleIBDFlow) logIBDFinished(isFinishedSuccessfully bool) {
successString := "successfully"
if !isFinishedSuccessfully {
successString = "(interrupted)"
}
log.Infof("IBD finished %s", successString)
}
func (flow *handleIBDFlow) getSyncerChainBlockLocator(
highHash, lowHash *externalapi.DomainHash) ([]*externalapi.DomainHash, error) {
requestIbdChainBlockLocatorMessage := appmessage.NewMsgIBDRequestChainBlockLocator(highHash, lowHash)
err := flow.outgoingRoute.Enqueue(requestIbdChainBlockLocatorMessage)
if err != nil {
return nil, err
}
message, err := flow.incomingRoute.DequeueWithTimeout(common.DefaultTimeout)
if err != nil {
return nil, err
}
switch message := message.(type) {
case *appmessage.MsgIBDChainBlockLocator:
return message.BlockLocatorHashes, nil
default:
return nil, protocolerrors.Errorf(true, "received unexpected message type. "+
"expected: %s, got: %s", appmessage.CmdIBDChainBlockLocator, message.Command())
}
}
func (flow *handleIBDFlow) syncPruningPointFutureHeaders(consensus externalapi.Consensus,
syncerHeaderSelectedTipHash, highestKnownSyncerChainHash, relayBlockHash *externalapi.DomainHash,
highBlockDAAScore uint64) error {
log.Infof("Downloading headers from %s", flow.peer)
err := flow.sendRequestHeaders(highestKnownSyncerChainHash, syncerHeaderSelectedTipHash)
if err != nil {
return err
}
highestSharedBlockHeader, err := consensus.GetBlockHeader(highestKnownSyncerChainHash)
if err != nil {
return err
}
progressReporter := newIBDProgressReporter(highestSharedBlockHeader.DAAScore(), highBlockDAAScore, "block headers")
// 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 {
// Finished downloading syncer selected tip blocks,
// check if we already have the triggering relayBlockHash
relayBlockInfo, err := consensus.GetBlockInfo(relayBlockHash)
if err != nil {
return err
}
if !relayBlockInfo.Exists {
// Send a special header request for the past diff. This is expected to be a small,
// as it is bounded to the size of virtual's mergeset
err = flow.sendRequestAnticone(syncerHeaderSelectedTipHash, relayBlockHash)
if err != nil {
return err
}
pastDiffHeadersMessage, pastDiffDone, err := flow.receiveHeaders()
if err != nil {
return err
}
if !pastDiffDone {
return protocolerrors.Errorf(true,
"Expected only one past diff header chunk for past(%s) setminus past(%s)",
syncerHeaderSelectedTipHash, relayBlockHash)
}
for _, header := range pastDiffHeadersMessage.BlockHeaders {
err = flow.processHeader(consensus, header)
if err != nil {
return err
}
}
}
// If the relayBlockHash has still not been received, the peer is misbehaving
relayBlockInfo, err = consensus.GetBlockInfo(relayBlockHash)
if err != nil {
return err
}
if !relayBlockInfo.Exists {
return protocolerrors.Errorf(true, "did not receive "+
"highHash block %s from peer %s during block download", relayBlockHash, flow.peer)
}
return nil
}
for _, header := range ibdBlocksMessage.BlockHeaders {
err = flow.processHeader(consensus, header)
if err != nil {
return err
}
}
lastReceivedHeader := ibdBlocksMessage.BlockHeaders[len(ibdBlocksMessage.BlockHeaders)-1]
progressReporter.reportProgress(len(ibdBlocksMessage.BlockHeaders), lastReceivedHeader.DAAScore)
case err := <-errChan:
return err
}
}
}
func (flow *handleIBDFlow) sendRequestAnticone(
syncerHeaderSelectedTipHash, relayBlockHash *externalapi.DomainHash) error {
msgRequestAnticone := appmessage.NewMsgRequestAnticone(syncerHeaderSelectedTipHash, relayBlockHash)
return flow.outgoingRoute.Enqueue(msgRequestAnticone)
}
func (flow *handleIBDFlow) sendRequestHeaders(
highestKnownSyncerChainHash, syncerHeaderSelectedTipHash *externalapi.DomainHash) error {
msgRequestHeaders := appmessage.NewMsgRequstHeaders(highestKnownSyncerChainHash, syncerHeaderSelectedTipHash)
return flow.outgoingRoute.Enqueue(msgRequestHeaders)
}
func (flow *handleIBDFlow) receiveHeaders() (msgIBDBlock *appmessage.BlockHeadersMessage, doneHeaders bool, err error) {
message, err := flow.incomingRoute.DequeueWithTimeout(common.DefaultTimeout)
if err != nil {
return nil, false, err
}
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 *handleIBDFlow) 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 *handleIBDFlow) 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 *handleIBDFlow) receiveAndInsertPruningPointUTXOSet(
consensus externalapi.Consensus, pruningPointHash *externalapi.DomainHash) (bool, error) {
onEnd := logger.LogAndMeasureExecutionTime(log, "receiveAndInsertPruningPointUTXOSet")
defer onEnd()
receivedChunkCount := 0
receivedUTXOCount := 0
for {
message, err := flow.incomingRoute.DequeueWithTimeout(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 *handleIBDFlow) 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
}
lowBlockHeader, err := flow.Domain().Consensus().GetBlockHeader(hashes[0])
if err != nil {
return err
}
highBlockHeader, err := flow.Domain().Consensus().GetBlockHeader(hashes[len(hashes)-1])
if err != nil {
return err
}
progressReporter := newIBDProgressReporter(lowBlockHeader.DAAScore(), highBlockHeader.DAAScore(), "blocks")
highestProcessedDAAScore := lowBlockHeader.DAAScore()
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.incomingRoute.DequeueWithTimeout(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
}
highestProcessedDAAScore = block.Header.DAAScore()
}
progressReporter.reportProgress(len(hashesToRequest), highestProcessedDAAScore)
}
return flow.resolveVirtual(highestProcessedDAAScore)
}
func (flow *handleIBDFlow) banIfBlockIsHeaderOnly(block *externalapi.DomainBlock) error {
if len(block.Transactions) == 0 {
return protocolerrors.Errorf(true, "sent header of %s block where expected block with body",
consensushashing.BlockHash(block))
}
return nil
}
func (flow *handleIBDFlow) resolveVirtual(estimatedVirtualDAAScoreTarget uint64) error {
virtualDAAScoreStart, err := flow.Domain().Consensus().GetVirtualDAAScore()
if err != nil {
return err
}
for i := 0; ; i++ {
if i%10 == 0 {
virtualDAAScore, err := flow.Domain().Consensus().GetVirtualDAAScore()
if err != nil {
return err
}
var percents int
if estimatedVirtualDAAScoreTarget-virtualDAAScoreStart <= 0 {
percents = 100
} else {
percents = int(float64(virtualDAAScore-virtualDAAScoreStart) / float64(estimatedVirtualDAAScoreTarget-virtualDAAScoreStart) * 100)
}
log.Infof("Resolving virtual. Estimated progress: %d%%", percents)
}
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
}
}
}

View File

@ -0,0 +1,32 @@
package blockrelay
type ibdProgressReporter struct {
lowDAAScore uint64
highDAAScore uint64
objectName string
totalDAAScoreDifference uint64
lastReportedProgressPercent int
processed int
}
func newIBDProgressReporter(lowDAAScore uint64, highDAAScore uint64, objectName string) *ibdProgressReporter {
return &ibdProgressReporter{
lowDAAScore: lowDAAScore,
highDAAScore: highDAAScore,
objectName: objectName,
totalDAAScoreDifference: highDAAScore - lowDAAScore,
lastReportedProgressPercent: 0,
processed: 0,
}
}
func (ipr *ibdProgressReporter) reportProgress(processedDelta int, highestProcessedDAAScore uint64) {
ipr.processed += processedDelta
relativeDAAScore := highestProcessedDAAScore - ipr.lowDAAScore
progressPercent := int((float64(relativeDAAScore) / float64(ipr.totalDAAScoreDifference)) * 100)
if progressPercent > ipr.lastReportedProgressPercent {
log.Infof("IBD: Processed %d %s (%d%%)", ipr.processed, ipr.objectName, progressPercent)
ipr.lastReportedProgressPercent = progressPercent
}
}

View File

@ -0,0 +1,424 @@
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"
"time"
)
func (flow *handleIBDFlow) ibdWithHeadersProof(
syncerHeaderSelectedTipHash, relayBlockHash *externalapi.DomainHash, highBlockDAAScore uint64) error {
err := flow.Domain().InitStagingConsensus()
if err != nil {
return err
}
err = flow.downloadHeadersAndPruningUTXOSet(syncerHeaderSelectedTipHash, relayBlockHash, highBlockDAAScore)
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 *handleIBDFlow) shouldSyncAndShouldDownloadHeadersProof(
relayBlock *externalapi.DomainBlock,
highestKnownSyncerChainHash *externalapi.DomainHash) (shouldDownload, shouldSync bool, err error) {
var highestSharedBlockFound, isPruningPointInSharedBlockChain bool
if highestKnownSyncerChainHash != nil {
highestSharedBlockFound = true
pruningPoint, err := flow.Domain().Consensus().PruningPoint()
if err != nil {
return false, false, err
}
isPruningPointInSharedBlockChain, err = flow.Domain().Consensus().IsInSelectedParentChainOf(
pruningPoint, highestKnownSyncerChainHash)
if err != nil {
return false, false, err
}
}
// Note: in the case where `highestSharedBlockFound == true && isPruningPointInSharedBlockChain == false`
// we might have here info which is relevant to finality conflict decisions. This should be taken into
// account when we improve this aspect.
if !highestSharedBlockFound || !isPruningPointInSharedBlockChain {
hasMoreBlueWorkThanSelectedTipAndPruningDepthMoreBlueScore, err := flow.checkIfHighHashHasMoreBlueWorkThanSelectedTipAndPruningDepthMoreBlueScore(relayBlock)
if err != nil {
return false, false, err
}
if hasMoreBlueWorkThanSelectedTipAndPruningDepthMoreBlueScore {
return true, true, nil
}
return false, false, nil
}
return false, true, nil
}
func (flow *handleIBDFlow) checkIfHighHashHasMoreBlueWorkThanSelectedTipAndPruningDepthMoreBlueScore(relayBlock *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 relayBlock.Header.BlueScore() < headersSelectedTipInfo.BlueScore+flow.Config().NetParams().PruningDepth() {
return false, nil
}
return relayBlock.Header.BlueWork().Cmp(headersSelectedTipInfo.BlueWork) > 0, nil
}
func (flow *handleIBDFlow) syncAndValidatePruningPointProof() (*externalapi.DomainHash, error) {
log.Infof("Downloading the pruning point proof from %s", flow.peer)
err := flow.outgoingRoute.Enqueue(appmessage.NewMsgRequestPruningPointProof())
if err != nil {
return nil, err
}
message, err := flow.incomingRoute.DequeueWithTimeout(10 * time.Minute)
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 *handleIBDFlow) downloadHeadersAndPruningUTXOSet(
syncerHeaderSelectedTipHash, relayBlockHash *externalapi.DomainHash,
highBlockDAAScore uint64) 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(),
syncerHeaderSelectedTipHash, proofPruningPoint, relayBlockHash, highBlockDAAScore)
if err != nil {
return err
}
log.Infof("Headers downloaded from peer %s", flow.peer)
relayBlockInfo, err := flow.Domain().StagingConsensus().GetBlockInfo(relayBlockHash)
if err != nil {
return err
}
if !relayBlockInfo.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 *handleIBDFlow) syncPruningPointsAndPruningPointAnticone(proofPruningPoint *externalapi.DomainHash) error {
log.Infof("Downloading the past pruning points and the pruning point anticone from %s", flow.peer)
err := flow.outgoingRoute.Enqueue(appmessage.NewMsgRequestPruningPointAndItsAnticone())
if err != nil {
return err
}
err = flow.validateAndInsertPruningPoints(proofPruningPoint)
if err != nil {
return err
}
message, err := flow.incomingRoute.DequeueWithTimeout(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
}
i := 0
for ; ; i++ {
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
}
// We're using i+2 because we want to check if the next block will belong to the next batch, but we already downloaded
// the pruning point outside the loop so we use i+2 instead of i+1.
if (i+2)%ibdBatchSize == 0 {
log.Infof("Downloaded %d blocks from the pruning point anticone", i+1)
err := flow.outgoingRoute.Enqueue(appmessage.NewMsgRequestNextPruningPointAndItsAnticoneBlocks())
if err != nil {
return err
}
}
}
log.Infof("Finished downloading pruning point and its anticone from %s. Total blocks downloaded: %d", flow.peer, i+1)
return nil
}
func (flow *handleIBDFlow) 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 *handleIBDFlow) receiveBlockWithTrustedData() (*appmessage.MsgBlockWithTrustedDataV4, bool, error) {
message, err := flow.incomingRoute.DequeueWithTimeout(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 *handleIBDFlow) receivePruningPoints() (*appmessage.MsgPruningPoints, error) {
message, err := flow.incomingRoute.DequeueWithTimeout(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 *handleIBDFlow) 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 *handleIBDFlow) syncPruningPointUTXOSet(consensus externalapi.Consensus,
pruningPoint *externalapi.DomainHash) (bool, error) {
log.Infof("Checking if the suggested pruning point %s is compatible to the node DAG", pruningPoint)
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 *handleIBDFlow) 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

@ -0,0 +1,42 @@
package ping
import (
"github.com/kaspanet/kaspad/app/appmessage"
"github.com/kaspanet/kaspad/infrastructure/network/netadapter/router"
)
// ReceivePingsContext is the interface for the context needed for the ReceivePings flow.
type ReceivePingsContext interface {
}
type receivePingsFlow struct {
ReceivePingsContext
incomingRoute, outgoingRoute *router.Route
}
// ReceivePings handles all ping messages coming through incomingRoute.
// This function assumes that incomingRoute will only return MsgPing.
func ReceivePings(context ReceivePingsContext, incomingRoute *router.Route, outgoingRoute *router.Route) error {
flow := &receivePingsFlow{
ReceivePingsContext: context,
incomingRoute: incomingRoute,
outgoingRoute: outgoingRoute,
}
return flow.start()
}
func (flow *receivePingsFlow) start() error {
for {
message, err := flow.incomingRoute.Dequeue()
if err != nil {
return err
}
pingMessage := message.(*appmessage.MsgPing)
pongMessage := appmessage.NewMsgPong(pingMessage.Nonce)
err = flow.outgoingRoute.Enqueue(pongMessage)
if err != nil {
return err
}
}
}

View File

@ -0,0 +1,72 @@
package ping
import (
"github.com/kaspanet/kaspad/app/protocol/common"
"time"
"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/infrastructure/network/netadapter/router"
"github.com/kaspanet/kaspad/util/random"
)
// SendPingsContext is the interface for the context needed for the SendPings flow.
type SendPingsContext interface {
ShutdownChan() <-chan struct{}
}
type sendPingsFlow struct {
SendPingsContext
incomingRoute, outgoingRoute *router.Route
peer *peerpkg.Peer
}
// SendPings starts sending MsgPings every pingInterval seconds to the
// given peer.
// This function assumes that incomingRoute will only return MsgPong.
func SendPings(context SendPingsContext, incomingRoute *router.Route, outgoingRoute *router.Route, peer *peerpkg.Peer) error {
flow := &sendPingsFlow{
SendPingsContext: context,
incomingRoute: incomingRoute,
outgoingRoute: outgoingRoute,
peer: peer,
}
return flow.start()
}
func (flow *sendPingsFlow) start() error {
const pingInterval = 2 * time.Minute
ticker := time.NewTicker(pingInterval)
defer ticker.Stop()
for {
select {
case <-flow.ShutdownChan():
return nil
case <-ticker.C:
}
nonce, err := random.Uint64()
if err != nil {
return err
}
flow.peer.SetPingPending(nonce)
pingMessage := appmessage.NewMsgPing(nonce)
err = flow.outgoingRoute.Enqueue(pingMessage)
if err != nil {
return err
}
message, err := flow.incomingRoute.DequeueWithTimeout(common.DefaultTimeout)
if err != nil {
return err
}
pongMessage := message.(*appmessage.MsgPong)
if pongMessage.Nonce != pingMessage.Nonce {
return protocolerrors.New(true, "nonce mismatch between ping and pong")
}
flow.peer.SetPingIdle()
}
}

View File

@ -0,0 +1,209 @@
package v5
import (
"github.com/kaspanet/kaspad/app/appmessage"
"github.com/kaspanet/kaspad/app/protocol/common"
"github.com/kaspanet/kaspad/app/protocol/flowcontext"
"github.com/kaspanet/kaspad/app/protocol/flows/v5/addressexchange"
"github.com/kaspanet/kaspad/app/protocol/flows/v5/blockrelay"
"github.com/kaspanet/kaspad/app/protocol/flows/v5/ping"
"github.com/kaspanet/kaspad/app/protocol/flows/v5/rejects"
"github.com/kaspanet/kaspad/app/protocol/flows/v5/transactionrelay"
peerpkg "github.com/kaspanet/kaspad/app/protocol/peer"
routerpkg "github.com/kaspanet/kaspad/infrastructure/network/netadapter/router"
)
type protocolManager interface {
RegisterFlow(name string, router *routerpkg.Router, messageTypes []appmessage.MessageCommand, isStopping *uint32,
errChan chan error, initializeFunc common.FlowInitializeFunc) *common.Flow
RegisterOneTimeFlow(name string, router *routerpkg.Router, messageTypes []appmessage.MessageCommand,
isStopping *uint32, stopChan chan error, initializeFunc common.FlowInitializeFunc) *common.Flow
RegisterFlowWithCapacity(name string, capacity int, router *routerpkg.Router,
messageTypes []appmessage.MessageCommand, isStopping *uint32,
errChan chan error, initializeFunc common.FlowInitializeFunc) *common.Flow
Context() *flowcontext.FlowContext
}
// Register is used in order to register all the protocol flows to the given router.
func Register(m protocolManager, router *routerpkg.Router, errChan chan error, isStopping *uint32) (flows []*common.Flow) {
flows = registerAddressFlows(m, router, isStopping, errChan)
flows = append(flows, registerBlockRelayFlows(m, router, isStopping, errChan)...)
flows = append(flows, registerPingFlows(m, router, isStopping, errChan)...)
flows = append(flows, registerTransactionRelayFlow(m, router, isStopping, errChan)...)
flows = append(flows, registerRejectsFlow(m, router, isStopping, errChan)...)
return flows
}
func registerAddressFlows(m protocolManager, router *routerpkg.Router, isStopping *uint32, errChan chan error) []*common.Flow {
outgoingRoute := router.OutgoingRoute()
return []*common.Flow{
m.RegisterFlow("SendAddresses", router, []appmessage.MessageCommand{appmessage.CmdRequestAddresses}, isStopping, errChan,
func(incomingRoute *routerpkg.Route, peer *peerpkg.Peer) error {
return addressexchange.SendAddresses(m.Context(), incomingRoute, outgoingRoute)
},
),
m.RegisterOneTimeFlow("ReceiveAddresses", router, []appmessage.MessageCommand{appmessage.CmdAddresses}, isStopping, errChan,
func(incomingRoute *routerpkg.Route, peer *peerpkg.Peer) error {
return addressexchange.ReceiveAddresses(m.Context(), incomingRoute, outgoingRoute, peer)
},
),
}
}
func registerBlockRelayFlows(m protocolManager, router *routerpkg.Router, isStopping *uint32, errChan chan error) []*common.Flow {
outgoingRoute := router.OutgoingRoute()
return []*common.Flow{
m.RegisterOneTimeFlow("SendVirtualSelectedParentInv", router, []appmessage.MessageCommand{},
isStopping, errChan, func(route *routerpkg.Route, peer *peerpkg.Peer) error {
return blockrelay.SendVirtualSelectedParentInv(m.Context(), outgoingRoute, peer)
}),
m.RegisterFlow("HandleRelayInvs", router, []appmessage.MessageCommand{
appmessage.CmdInvRelayBlock, appmessage.CmdBlock, appmessage.CmdBlockLocator,
},
isStopping, errChan, func(incomingRoute *routerpkg.Route, peer *peerpkg.Peer) error {
return blockrelay.HandleRelayInvs(m.Context(), incomingRoute,
outgoingRoute, peer)
},
),
m.RegisterFlow("HandleIBD", router, []appmessage.MessageCommand{
appmessage.CmdDoneHeaders, appmessage.CmdUnexpectedPruningPoint, appmessage.CmdPruningPointUTXOSetChunk,
appmessage.CmdBlockHeaders, appmessage.CmdIBDBlockLocatorHighestHash, appmessage.CmdBlockWithTrustedDataV4,
appmessage.CmdDoneBlocksWithTrustedData, appmessage.CmdIBDBlockLocatorHighestHashNotFound,
appmessage.CmdDonePruningPointUTXOSetChunks, appmessage.CmdIBDBlock, appmessage.CmdPruningPoints,
appmessage.CmdPruningPointProof,
appmessage.CmdTrustedData,
appmessage.CmdIBDChainBlockLocator,
},
isStopping, errChan, func(incomingRoute *routerpkg.Route, peer *peerpkg.Peer) error {
return blockrelay.HandleIBD(m.Context(), incomingRoute,
outgoingRoute, peer)
},
),
m.RegisterFlow("HandleRelayBlockRequests", router, []appmessage.MessageCommand{appmessage.CmdRequestRelayBlocks}, isStopping, errChan,
func(incomingRoute *routerpkg.Route, peer *peerpkg.Peer) error {
return blockrelay.HandleRelayBlockRequests(m.Context(), incomingRoute, outgoingRoute, peer)
},
),
m.RegisterFlow("HandleRequestBlockLocator", router,
[]appmessage.MessageCommand{appmessage.CmdRequestBlockLocator}, isStopping, errChan,
func(incomingRoute *routerpkg.Route, peer *peerpkg.Peer) error {
return blockrelay.HandleRequestBlockLocator(m.Context(), incomingRoute, outgoingRoute)
},
),
m.RegisterFlow("HandleRequestHeaders", router,
[]appmessage.MessageCommand{appmessage.CmdRequestHeaders, appmessage.CmdRequestNextHeaders}, isStopping, errChan,
func(incomingRoute *routerpkg.Route, peer *peerpkg.Peer) error {
return blockrelay.HandleRequestHeaders(m.Context(), incomingRoute, outgoingRoute, peer)
},
),
m.RegisterFlow("HandleIBDBlockRequests", router,
[]appmessage.MessageCommand{appmessage.CmdRequestIBDBlocks}, isStopping, errChan,
func(incomingRoute *routerpkg.Route, peer *peerpkg.Peer) error {
return blockrelay.HandleIBDBlockRequests(m.Context(), incomingRoute, outgoingRoute)
},
),
m.RegisterFlow("HandleRequestPruningPointUTXOSet", router,
[]appmessage.MessageCommand{appmessage.CmdRequestPruningPointUTXOSet,
appmessage.CmdRequestNextPruningPointUTXOSetChunk}, isStopping, errChan,
func(incomingRoute *routerpkg.Route, peer *peerpkg.Peer) error {
return blockrelay.HandleRequestPruningPointUTXOSet(m.Context(), incomingRoute, outgoingRoute)
},
),
m.RegisterFlow("HandlePruningPointAndItsAnticoneRequests", router,
[]appmessage.MessageCommand{appmessage.CmdRequestPruningPointAndItsAnticone, appmessage.CmdRequestNextPruningPointAndItsAnticoneBlocks}, isStopping, errChan,
func(incomingRoute *routerpkg.Route, peer *peerpkg.Peer) error {
return blockrelay.HandlePruningPointAndItsAnticoneRequests(m.Context(), incomingRoute, outgoingRoute, peer)
},
),
m.RegisterFlow("HandleIBDBlockLocator", router,
[]appmessage.MessageCommand{appmessage.CmdIBDBlockLocator}, isStopping, errChan,
func(incomingRoute *routerpkg.Route, peer *peerpkg.Peer) error {
return blockrelay.HandleIBDBlockLocator(m.Context(), incomingRoute, outgoingRoute, peer)
},
),
m.RegisterFlow("HandleRequestIBDChainBlockLocator", router,
[]appmessage.MessageCommand{appmessage.CmdRequestIBDChainBlockLocator}, isStopping, errChan,
func(incomingRoute *routerpkg.Route, peer *peerpkg.Peer) error {
return blockrelay.HandleRequestIBDChainBlockLocator(m.Context(), incomingRoute, outgoingRoute)
},
),
m.RegisterFlow("HandleRequestAnticone", router,
[]appmessage.MessageCommand{appmessage.CmdRequestAnticone}, isStopping, errChan,
func(incomingRoute *routerpkg.Route, peer *peerpkg.Peer) error {
return blockrelay.HandleRequestAnticone(m.Context(), incomingRoute, outgoingRoute, peer)
},
),
m.RegisterFlow("HandlePruningPointProofRequests", router,
[]appmessage.MessageCommand{appmessage.CmdRequestPruningPointProof}, isStopping, errChan,
func(incomingRoute *routerpkg.Route, peer *peerpkg.Peer) error {
return blockrelay.HandlePruningPointProofRequests(m.Context(), incomingRoute, outgoingRoute, peer)
},
),
}
}
func registerPingFlows(m protocolManager, router *routerpkg.Router, isStopping *uint32, errChan chan error) []*common.Flow {
outgoingRoute := router.OutgoingRoute()
return []*common.Flow{
m.RegisterFlow("ReceivePings", router, []appmessage.MessageCommand{appmessage.CmdPing}, isStopping, errChan,
func(incomingRoute *routerpkg.Route, peer *peerpkg.Peer) error {
return ping.ReceivePings(m.Context(), incomingRoute, outgoingRoute)
},
),
m.RegisterFlow("SendPings", router, []appmessage.MessageCommand{appmessage.CmdPong}, isStopping, errChan,
func(incomingRoute *routerpkg.Route, peer *peerpkg.Peer) error {
return ping.SendPings(m.Context(), incomingRoute, outgoingRoute, peer)
},
),
}
}
func registerTransactionRelayFlow(m protocolManager, router *routerpkg.Router, isStopping *uint32, errChan chan error) []*common.Flow {
outgoingRoute := router.OutgoingRoute()
return []*common.Flow{
m.RegisterFlowWithCapacity("HandleRelayedTransactions", 10_000, router,
[]appmessage.MessageCommand{appmessage.CmdInvTransaction, appmessage.CmdTx, appmessage.CmdTransactionNotFound}, isStopping, errChan,
func(incomingRoute *routerpkg.Route, peer *peerpkg.Peer) error {
return transactionrelay.HandleRelayedTransactions(m.Context(), incomingRoute, outgoingRoute)
},
),
m.RegisterFlow("HandleRequestTransactions", router,
[]appmessage.MessageCommand{appmessage.CmdRequestTransactions}, isStopping, errChan,
func(incomingRoute *routerpkg.Route, peer *peerpkg.Peer) error {
return transactionrelay.HandleRequestedTransactions(m.Context(), incomingRoute, outgoingRoute)
},
),
}
}
func registerRejectsFlow(m protocolManager, router *routerpkg.Router, isStopping *uint32, errChan chan error) []*common.Flow {
outgoingRoute := router.OutgoingRoute()
return []*common.Flow{
m.RegisterFlow("HandleRejects", router,
[]appmessage.MessageCommand{appmessage.CmdReject}, isStopping, errChan,
func(incomingRoute *routerpkg.Route, peer *peerpkg.Peer) error {
return rejects.HandleRejects(m.Context(), incomingRoute, outgoingRoute)
},
),
}
}

View File

@ -0,0 +1,37 @@
package rejects
import (
"github.com/kaspanet/kaspad/app/appmessage"
"github.com/kaspanet/kaspad/app/protocol/protocolerrors"
"github.com/kaspanet/kaspad/infrastructure/network/netadapter/router"
)
// HandleRejectsContext is the interface for the context needed for the HandleRejects flow.
type HandleRejectsContext interface {
}
type handleRejectsFlow struct {
HandleRejectsContext
incomingRoute, outgoingRoute *router.Route
}
// HandleRejects handles all reject messages coming through incomingRoute.
// This function assumes that incomingRoute will only return MsgReject.
func HandleRejects(context HandleRejectsContext, incomingRoute *router.Route, outgoingRoute *router.Route) error {
flow := &handleRejectsFlow{
HandleRejectsContext: context,
incomingRoute: incomingRoute,
outgoingRoute: outgoingRoute,
}
return flow.start()
}
func (flow *handleRejectsFlow) start() error {
message, err := flow.incomingRoute.Dequeue()
if err != nil {
return err
}
rejectMessage := message.(*appmessage.MsgReject)
return protocolerrors.Errorf(false, "got reject message: `%s`", rejectMessage.Reason)
}

View File

@ -0,0 +1,24 @@
package testing
import (
"strings"
"testing"
"github.com/kaspanet/kaspad/app/protocol/protocolerrors"
"github.com/pkg/errors"
)
func checkFlowError(t *testing.T, err error, isProtocolError bool, shouldBan bool, contains string) {
pErr := protocolerrors.ProtocolError{}
if errors.As(err, &pErr) != isProtocolError {
t.Fatalf("Unexepcted error %+v", err)
}
if pErr.ShouldBan != shouldBan {
t.Fatalf("Exepcted shouldBan %t but got %t", shouldBan, pErr.ShouldBan)
}
if !strings.Contains(err.Error(), contains) {
t.Fatalf("Unexpected error. Expected error to contain '%s' but got: %+v", contains, err)
}
}

View File

@ -0,0 +1,51 @@
package testing
import (
"github.com/kaspanet/kaspad/app/protocol/flows/v5/addressexchange"
"testing"
"time"
"github.com/kaspanet/kaspad/app/appmessage"
peerpkg "github.com/kaspanet/kaspad/app/protocol/peer"
"github.com/kaspanet/kaspad/domain/consensus"
"github.com/kaspanet/kaspad/domain/consensus/utils/testutils"
"github.com/kaspanet/kaspad/infrastructure/network/addressmanager"
"github.com/kaspanet/kaspad/infrastructure/network/netadapter/router"
)
type fakeReceiveAddressesContext struct{}
func (f fakeReceiveAddressesContext) AddressManager() *addressmanager.AddressManager {
return nil
}
func TestReceiveAddressesErrors(t *testing.T) {
testutils.ForAllNets(t, true, func(t *testing.T, consensusConfig *consensus.Config) {
incomingRoute := router.NewRoute("incoming")
outgoingRoute := router.NewRoute("outgoing")
peer := peerpkg.New(nil)
errChan := make(chan error)
go func() {
errChan <- addressexchange.ReceiveAddresses(fakeReceiveAddressesContext{}, incomingRoute, outgoingRoute, peer)
}()
_, err := outgoingRoute.DequeueWithTimeout(time.Second)
if err != nil {
t.Fatalf("DequeueWithTimeout: %+v", err)
}
// Sending addressmanager.GetAddressesMax+1 addresses should trigger a ban
err = incomingRoute.Enqueue(appmessage.NewMsgAddresses(make([]*appmessage.NetAddress,
addressmanager.GetAddressesMax+1)))
if err != nil {
t.Fatalf("Enqueue: %+v", err)
}
select {
case err := <-errChan:
checkFlowError(t, err, true, true, "address count exceeded")
case <-time.After(time.Second):
t.Fatalf("timed out after %s", time.Second)
}
})
}

View File

@ -0,0 +1,4 @@
package testing
// Because of a bug in Go coverage fails if you have packages with test files only. See https://github.com/golang/go/issues/27333
// So this is a dummy non-test go file in the package.

View File

@ -0,0 +1,209 @@
package transactionrelay
import (
"github.com/kaspanet/kaspad/app/appmessage"
"github.com/kaspanet/kaspad/app/protocol/common"
"github.com/kaspanet/kaspad/app/protocol/flowcontext"
"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/utils/consensushashing"
"github.com/kaspanet/kaspad/domain/miningmanager/mempool"
"github.com/kaspanet/kaspad/infrastructure/network/netadapter"
"github.com/kaspanet/kaspad/infrastructure/network/netadapter/router"
"github.com/pkg/errors"
)
// TransactionsRelayContext is the interface for the context needed for the
// HandleRelayedTransactions and HandleRequestedTransactions flows.
type TransactionsRelayContext interface {
NetAdapter() *netadapter.NetAdapter
Domain() domain.Domain
SharedRequestedTransactions() *flowcontext.SharedRequestedTransactions
OnTransactionAddedToMempool()
EnqueueTransactionIDsForPropagation(transactionIDs []*externalapi.DomainTransactionID) error
IsIBDRunning() bool
}
type handleRelayedTransactionsFlow struct {
TransactionsRelayContext
incomingRoute, outgoingRoute *router.Route
invsQueue []*appmessage.MsgInvTransaction
}
// HandleRelayedTransactions listens to appmessage.MsgInvTransaction messages, requests their corresponding transactions if they
// are missing, adds them to the mempool and propagates them to the rest of the network.
func HandleRelayedTransactions(context TransactionsRelayContext, incomingRoute *router.Route, outgoingRoute *router.Route) error {
flow := &handleRelayedTransactionsFlow{
TransactionsRelayContext: context,
incomingRoute: incomingRoute,
outgoingRoute: outgoingRoute,
invsQueue: make([]*appmessage.MsgInvTransaction, 0),
}
return flow.start()
}
func (flow *handleRelayedTransactionsFlow) start() error {
for {
inv, err := flow.readInv()
if err != nil {
return err
}
if flow.IsIBDRunning() {
continue
}
requestedIDs, err := flow.requestInvTransactions(inv)
if err != nil {
return err
}
err = flow.receiveTransactions(requestedIDs)
if err != nil {
return err
}
}
}
func (flow *handleRelayedTransactionsFlow) requestInvTransactions(
inv *appmessage.MsgInvTransaction) (requestedIDs []*externalapi.DomainTransactionID, err error) {
idsToRequest := make([]*externalapi.DomainTransactionID, 0, len(inv.TxIDs))
for _, txID := range inv.TxIDs {
if flow.isKnownTransaction(txID) {
continue
}
exists := flow.SharedRequestedTransactions().AddIfNotExists(txID)
if exists {
continue
}
idsToRequest = append(idsToRequest, txID)
}
if len(idsToRequest) == 0 {
return idsToRequest, nil
}
msgGetTransactions := appmessage.NewMsgRequestTransactions(idsToRequest)
err = flow.outgoingRoute.Enqueue(msgGetTransactions)
if err != nil {
flow.SharedRequestedTransactions().RemoveMany(idsToRequest)
return nil, err
}
return idsToRequest, nil
}
func (flow *handleRelayedTransactionsFlow) isKnownTransaction(txID *externalapi.DomainTransactionID) bool {
// Ask the transaction memory pool if the transaction is known
// to it in any form (main pool or orphan).
if _, ok := flow.Domain().MiningManager().GetTransaction(txID); ok {
return true
}
return false
}
func (flow *handleRelayedTransactionsFlow) readInv() (*appmessage.MsgInvTransaction, error) {
if len(flow.invsQueue) > 0 {
var inv *appmessage.MsgInvTransaction
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.MsgInvTransaction)
if !ok {
return nil, protocolerrors.Errorf(true, "unexpected %s message in the block relay flow while "+
"expecting an inv message", msg.Command())
}
return inv, nil
}
func (flow *handleRelayedTransactionsFlow) broadcastAcceptedTransactions(acceptedTxIDs []*externalapi.DomainTransactionID) error {
return flow.EnqueueTransactionIDsForPropagation(acceptedTxIDs)
}
// readMsgTxOrNotFound returns the next msgTx or msgTransactionNotFound in incomingRoute,
// returning only one of the message types at a time.
//
// and populates invsQueue with any inv messages that meanwhile arrive.
func (flow *handleRelayedTransactionsFlow) readMsgTxOrNotFound() (
msgTx *appmessage.MsgTx, msgNotFound *appmessage.MsgTransactionNotFound, err error) {
for {
message, err := flow.incomingRoute.DequeueWithTimeout(common.DefaultTimeout)
if err != nil {
return nil, nil, err
}
switch message := message.(type) {
case *appmessage.MsgInvTransaction:
flow.invsQueue = append(flow.invsQueue, message)
case *appmessage.MsgTx:
return message, nil, nil
case *appmessage.MsgTransactionNotFound:
return nil, message, nil
default:
return nil, nil, errors.Errorf("unexpected message %s", message.Command())
}
}
}
func (flow *handleRelayedTransactionsFlow) receiveTransactions(requestedTransactions []*externalapi.DomainTransactionID) error {
// In case the function returns earlier than expected, we want to make sure sharedRequestedTransactions is
// clean from any pending transactions.
defer flow.SharedRequestedTransactions().RemoveMany(requestedTransactions)
for _, expectedID := range requestedTransactions {
msgTx, msgTxNotFound, err := flow.readMsgTxOrNotFound()
if err != nil {
return err
}
if msgTxNotFound != nil {
if !msgTxNotFound.ID.Equal(expectedID) {
return protocolerrors.Errorf(true, "expected transaction %s, but got %s",
expectedID, msgTxNotFound.ID)
}
continue
}
tx := appmessage.MsgTxToDomainTransaction(msgTx)
txID := consensushashing.TransactionID(tx)
if !txID.Equal(expectedID) {
return protocolerrors.Errorf(true, "expected transaction %s, but got %s",
expectedID, txID)
}
acceptedTransactions, err :=
flow.Domain().MiningManager().ValidateAndInsertTransaction(tx, false, true)
if err != nil {
ruleErr := &mempool.RuleError{}
if !errors.As(err, ruleErr) {
return errors.Wrapf(err, "failed to process transaction %s", txID)
}
shouldBan := false
if txRuleErr := (&mempool.TxRuleError{}); errors.As(ruleErr.Err, txRuleErr) {
if txRuleErr.RejectCode == mempool.RejectInvalid {
shouldBan = true
}
}
if !shouldBan {
continue
}
return protocolerrors.Errorf(true, "rejected transaction %s: %s", txID, ruleErr)
}
err = flow.broadcastAcceptedTransactions(consensushashing.TransactionIDs(acceptedTransactions))
if err != nil {
return err
}
flow.OnTransactionAddedToMempool()
}
return nil
}

View File

@ -0,0 +1,196 @@
package transactionrelay_test
import (
"errors"
"github.com/kaspanet/kaspad/app/protocol/flowcontext"
"github.com/kaspanet/kaspad/app/protocol/flows/v5/transactionrelay"
"strings"
"testing"
"github.com/kaspanet/kaspad/app/protocol/protocolerrors"
"github.com/kaspanet/kaspad/domain"
"github.com/kaspanet/kaspad/domain/consensus"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/kaspanet/kaspad/domain/consensus/utils/testutils"
"github.com/kaspanet/kaspad/domain/miningmanager/mempool"
"github.com/kaspanet/kaspad/infrastructure/logger"
"github.com/kaspanet/kaspad/util/panics"
"github.com/kaspanet/kaspad/app/appmessage"
"github.com/kaspanet/kaspad/infrastructure/config"
"github.com/kaspanet/kaspad/infrastructure/network/netadapter"
"github.com/kaspanet/kaspad/infrastructure/network/netadapter/router"
)
type mocTransactionsRelayContext struct {
netAdapter *netadapter.NetAdapter
domain domain.Domain
sharedRequestedTransactions *flowcontext.SharedRequestedTransactions
}
func (m *mocTransactionsRelayContext) NetAdapter() *netadapter.NetAdapter {
return m.netAdapter
}
func (m *mocTransactionsRelayContext) Domain() domain.Domain {
return m.domain
}
func (m *mocTransactionsRelayContext) SharedRequestedTransactions() *flowcontext.SharedRequestedTransactions {
return m.sharedRequestedTransactions
}
func (m *mocTransactionsRelayContext) EnqueueTransactionIDsForPropagation(transactionIDs []*externalapi.DomainTransactionID) error {
return nil
}
func (m *mocTransactionsRelayContext) OnTransactionAddedToMempool() {
}
func (m *mocTransactionsRelayContext) IsIBDRunning() bool {
return false
}
// TestHandleRelayedTransactionsNotFound tests the flow of HandleRelayedTransactions when the peer doesn't
// have the requested transactions in the mempool.
func TestHandleRelayedTransactionsNotFound(t *testing.T) {
testutils.ForAllNets(t, true, func(t *testing.T, consensusConfig *consensus.Config) {
var log = logger.RegisterSubSystem("PROT")
var spawn = panics.GoroutineWrapperFunc(log)
factory := consensus.NewFactory()
tc, teardown, err := factory.NewTestConsensus(consensusConfig, "TestHandleRelayedTransactionsNotFound")
if err != nil {
t.Fatalf("Error setting up test consensus: %+v", err)
}
defer teardown(false)
sharedRequestedTransactions := flowcontext.NewSharedRequestedTransactions()
adapter, err := netadapter.NewNetAdapter(config.DefaultConfig())
if err != nil {
t.Fatalf("Failed to create a NetAdapter: %v", err)
}
domainInstance, err := domain.New(consensusConfig, mempool.DefaultConfig(&consensusConfig.Params), tc.Database())
if err != nil {
t.Fatalf("Failed to set up a domain instance: %v", err)
}
context := &mocTransactionsRelayContext{
netAdapter: adapter,
domain: domainInstance,
sharedRequestedTransactions: sharedRequestedTransactions,
}
incomingRoute := router.NewRoute("incoming")
defer incomingRoute.Close()
peerIncomingRoute := router.NewRoute("outgoing")
defer peerIncomingRoute.Close()
txID1 := externalapi.NewDomainTransactionIDFromByteArray(&[externalapi.DomainHashSize]byte{
0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00,
0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00,
0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00,
0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x01})
txID2 := externalapi.NewDomainTransactionIDFromByteArray(&[externalapi.DomainHashSize]byte{
0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00,
0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00,
0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00,
0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02})
txIDs := []*externalapi.DomainTransactionID{txID1, txID2}
invMessage := appmessage.NewMsgInvTransaction(txIDs)
err = incomingRoute.Enqueue(invMessage)
if err != nil {
t.Fatalf("Unexpected error from incomingRoute.Enqueue: %v", err)
}
// The goroutine is representing the peer's actions.
spawn("peerResponseToTheTransactionsRequest", func() {
msg, err := peerIncomingRoute.Dequeue()
if err != nil {
t.Fatalf("Dequeue: %v", err)
}
inv := msg.(*appmessage.MsgRequestTransactions)
if len(txIDs) != len(inv.IDs) {
t.Fatalf("TestHandleRelayedTransactions: expected %d transactions ID, but got %d", len(txIDs), len(inv.IDs))
}
for i, id := range inv.IDs {
if txIDs[i].String() != id.String() {
t.Fatalf("TestHandleRelayedTransactions: expected equal txID: expected %s, but got %s", txIDs[i].String(), id.String())
}
err = incomingRoute.Enqueue(appmessage.NewMsgTransactionNotFound(txIDs[i]))
if err != nil {
t.Fatalf("Unexpected error from incomingRoute.Enqueue: %v", err)
}
}
// Insert an unexpected message type to stop the infinity loop.
err = incomingRoute.Enqueue(&appmessage.MsgAddresses{})
if err != nil {
t.Fatalf("Unexpected error from incomingRoute.Enqueue: %v", err)
}
})
err = transactionrelay.HandleRelayedTransactions(context, incomingRoute, peerIncomingRoute)
// Since we inserted an unexpected message type to stop the infinity loop,
// we expect the error will be infected from this specific message and also the
// error will count as a protocol message.
if protocolErr := (protocolerrors.ProtocolError{}); err == nil || !errors.As(err, &protocolErr) {
t.Fatalf("Expected to protocol error")
} else {
if !protocolErr.ShouldBan {
t.Fatalf("Exepcted shouldBan true, but got false.")
}
if !strings.Contains(err.Error(), "unexpected Addresses [code 3] message in the block relay flow while expecting an inv message") {
t.Fatalf("Unexpected error: expected: an error due to existence of an Addresses message "+
"in the block relay flow, but got: %v", protocolErr.Cause)
}
}
})
}
// TestOnClosedIncomingRoute verifies that an appropriate error message will be returned when
// trying to dequeue a message from a closed route.
func TestOnClosedIncomingRoute(t *testing.T) {
testutils.ForAllNets(t, true, func(t *testing.T, consensusConfig *consensus.Config) {
factory := consensus.NewFactory()
tc, teardown, err := factory.NewTestConsensus(consensusConfig, "TestOnClosedIncomingRoute")
if err != nil {
t.Fatalf("Error setting up test consensus: %+v", err)
}
defer teardown(false)
sharedRequestedTransactions := flowcontext.NewSharedRequestedTransactions()
adapter, err := netadapter.NewNetAdapter(config.DefaultConfig())
if err != nil {
t.Fatalf("Failed to creat a NetAdapter : %v", err)
}
domainInstance, err := domain.New(consensusConfig, mempool.DefaultConfig(&consensusConfig.Params), tc.Database())
if err != nil {
t.Fatalf("Failed to set up a domain instance: %v", err)
}
context := &mocTransactionsRelayContext{
netAdapter: adapter,
domain: domainInstance,
sharedRequestedTransactions: sharedRequestedTransactions,
}
incomingRoute := router.NewRoute("incoming")
outgoingRoute := router.NewRoute("outgoing")
defer outgoingRoute.Close()
txID := externalapi.NewDomainTransactionIDFromByteArray(&[externalapi.DomainHashSize]byte{
0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00,
0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00,
0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00,
0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x01})
txIDs := []*externalapi.DomainTransactionID{txID}
err = incomingRoute.Enqueue(&appmessage.MsgInvTransaction{TxIDs: txIDs})
if err != nil {
t.Fatalf("Unexpected error from incomingRoute.Enqueue: %v", err)
}
incomingRoute.Close()
err = transactionrelay.HandleRelayedTransactions(context, incomingRoute, outgoingRoute)
if err == nil || !errors.Is(err, router.ErrRouteClosed) {
t.Fatalf("Unexpected error: expected: %v, got : %v", router.ErrRouteClosed, err)
}
})
}

View File

@ -0,0 +1,59 @@
package transactionrelay
import (
"github.com/kaspanet/kaspad/app/appmessage"
"github.com/kaspanet/kaspad/infrastructure/network/netadapter/router"
)
type handleRequestedTransactionsFlow struct {
TransactionsRelayContext
incomingRoute, outgoingRoute *router.Route
}
// HandleRequestedTransactions listens to appmessage.MsgRequestTransactions messages, responding with the requested
// transactions if those are in the mempool.
// Missing transactions would be ignored
func HandleRequestedTransactions(context TransactionsRelayContext, incomingRoute *router.Route, outgoingRoute *router.Route) error {
flow := &handleRequestedTransactionsFlow{
TransactionsRelayContext: context,
incomingRoute: incomingRoute,
outgoingRoute: outgoingRoute,
}
return flow.start()
}
func (flow *handleRequestedTransactionsFlow) start() error {
for {
msgRequestTransactions, err := flow.readRequestTransactions()
if err != nil {
return err
}
for _, transactionID := range msgRequestTransactions.IDs {
tx, ok := flow.Domain().MiningManager().GetTransaction(transactionID)
if !ok {
msgTransactionNotFound := appmessage.NewMsgTransactionNotFound(transactionID)
err := flow.outgoingRoute.Enqueue(msgTransactionNotFound)
if err != nil {
return err
}
continue
}
err := flow.outgoingRoute.Enqueue(appmessage.DomainTransactionToMsgTx(tx))
if err != nil {
return err
}
}
}
}
func (flow *handleRequestedTransactionsFlow) readRequestTransactions() (*appmessage.MsgRequestTransactions, error) {
msg, err := flow.incomingRoute.Dequeue()
if err != nil {
return nil, err
}
return msg.(*appmessage.MsgRequestTransactions), nil
}

View File

@ -0,0 +1,91 @@
package transactionrelay_test
import (
"github.com/kaspanet/kaspad/app/protocol/flowcontext"
"github.com/kaspanet/kaspad/app/protocol/flows/v5/transactionrelay"
"testing"
"github.com/kaspanet/kaspad/app/appmessage"
"github.com/kaspanet/kaspad/domain"
"github.com/kaspanet/kaspad/domain/consensus"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/kaspanet/kaspad/domain/consensus/utils/testutils"
"github.com/kaspanet/kaspad/domain/miningmanager/mempool"
"github.com/kaspanet/kaspad/infrastructure/config"
"github.com/kaspanet/kaspad/infrastructure/logger"
"github.com/kaspanet/kaspad/infrastructure/network/netadapter"
"github.com/kaspanet/kaspad/infrastructure/network/netadapter/router"
"github.com/kaspanet/kaspad/util/panics"
"github.com/pkg/errors"
)
// TestHandleRequestedTransactionsNotFound tests the flow of HandleRequestedTransactions
// when the requested transactions don't found in the mempool.
func TestHandleRequestedTransactionsNotFound(t *testing.T) {
testutils.ForAllNets(t, true, func(t *testing.T, consensusConfig *consensus.Config) {
var log = logger.RegisterSubSystem("PROT")
var spawn = panics.GoroutineWrapperFunc(log)
factory := consensus.NewFactory()
tc, teardown, err := factory.NewTestConsensus(consensusConfig, "TestHandleRequestedTransactionsNotFound")
if err != nil {
t.Fatalf("Error setting up test Consensus: %+v", err)
}
defer teardown(false)
sharedRequestedTransactions := flowcontext.NewSharedRequestedTransactions()
adapter, err := netadapter.NewNetAdapter(config.DefaultConfig())
if err != nil {
t.Fatalf("Failed to create a NetAdapter: %v", err)
}
domainInstance, err := domain.New(consensusConfig, mempool.DefaultConfig(&consensusConfig.Params), tc.Database())
if err != nil {
t.Fatalf("Failed to set up a domain Instance: %v", err)
}
context := &mocTransactionsRelayContext{
netAdapter: adapter,
domain: domainInstance,
sharedRequestedTransactions: sharedRequestedTransactions,
}
incomingRoute := router.NewRoute("incoming")
outgoingRoute := router.NewRoute("outgoing")
defer outgoingRoute.Close()
txID1 := externalapi.NewDomainTransactionIDFromByteArray(&[externalapi.DomainHashSize]byte{
0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00,
0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00,
0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00,
0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x01})
txID2 := externalapi.NewDomainTransactionIDFromByteArray(&[externalapi.DomainHashSize]byte{
0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00,
0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00,
0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00,
0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02})
txIDs := []*externalapi.DomainTransactionID{txID1, txID2}
msg := appmessage.NewMsgRequestTransactions(txIDs)
err = incomingRoute.Enqueue(msg)
if err != nil {
t.Fatalf("Unexpected error from incomingRoute.Enqueue: %v", err)
}
// The goroutine is representing the peer's actions.
spawn("peerResponseToTheTransactionsMessages", func() {
for i, id := range txIDs {
msg, err := outgoingRoute.Dequeue()
if err != nil {
t.Fatalf("Dequeue: %s", err)
}
outMsg := msg.(*appmessage.MsgTransactionNotFound)
if txIDs[i].String() != outMsg.ID.String() {
t.Fatalf("TestHandleRelayedTransactions: expected equal txID: expected %s, but got %s", txIDs[i].String(), id.String())
}
}
// Closed the incomingRoute for stop the infinity loop.
incomingRoute.Close()
})
err = transactionrelay.HandleRequestedTransactions(context, incomingRoute, outgoingRoute)
// Make sure the error is due to the closed route.
if err == nil || !errors.Is(err, router.ErrRouteClosed) {
t.Fatalf("Unexpected error: expected: %v, got : %v", router.ErrRouteClosed, err)
}
})
}

View File

@ -4,6 +4,7 @@ import (
"github.com/kaspanet/kaspad/app/protocol/common"
"github.com/kaspanet/kaspad/app/protocol/flows/ready"
v4 "github.com/kaspanet/kaspad/app/protocol/flows/v4"
v5 "github.com/kaspanet/kaspad/app/protocol/flows/v5"
"sync"
"sync/atomic"
@ -78,6 +79,8 @@ func (m *Manager) routerInitializer(router *routerpkg.Router, netConnection *net
switch peer.ProtocolVersion() {
case 4:
flows = v4.Register(m, router, errChan, &isStopping)
case 5:
flows = v5.Register(m, router, errChan, &isStopping)
default:
panic(errors.Errorf("no way to handle protocol version %d", peer.ProtocolVersion()))
}

View File

@ -35,7 +35,7 @@ type consensus struct {
headerTipsManager model.HeadersSelectedTipManager
mergeDepthManager model.MergeDepthManager
pruningManager model.PruningManager
reachabilityManagers []model.ReachabilityManager
reachabilityManager model.ReachabilityManager
finalityManager model.FinalityManager
pruningProofManager model.PruningProofManager
@ -49,7 +49,7 @@ type consensus struct {
consensusStateStore model.ConsensusStateStore
headersSelectedTipStore model.HeaderSelectedTipStore
multisetStore model.MultisetStore
reachabilityDataStores []model.ReachabilityDataStore
reachabilityDataStore model.ReachabilityDataStore
utxoDiffStore model.UTXODiffStore
finalityStore model.FinalityStore
headersSelectedChainStore model.HeadersSelectedChainStore
@ -83,11 +83,9 @@ func (s *consensus) Init(skipAddingGenesis bool) error {
// on a node with pruned header all blocks without known parents points to it.
if !exists {
s.blockStatusStore.Stage(stagingArea, model.VirtualGenesisBlockHash, externalapi.StatusUTXOValid)
for _, reachabilityManager := range s.reachabilityManagers {
err = reachabilityManager.Init(stagingArea)
if err != nil {
return err
}
err = s.reachabilityManager.Init(stagingArea)
if err != nil {
return err
}
for _, dagTopologyManager := range s.dagTopologyManagers {
@ -343,6 +341,25 @@ func (s *consensus) GetHashesBetween(lowHash, highHash *externalapi.DomainHash,
return s.syncManager.GetHashesBetween(stagingArea, lowHash, highHash, maxBlocks)
}
func (s *consensus) GetAnticone(blockHash, contextHash *externalapi.DomainHash,
maxBlocks uint64) (hashes []*externalapi.DomainHash, err error) {
s.lock.Lock()
defer s.lock.Unlock()
stagingArea := model.NewStagingArea()
err = s.validateBlockHashExists(stagingArea, blockHash)
if err != nil {
return nil, err
}
err = s.validateBlockHashExists(stagingArea, contextHash)
if err != nil {
return nil, err
}
return s.syncManager.GetAnticone(stagingArea, blockHash, contextHash, maxBlocks)
}
func (s *consensus) GetMissingBlockBodyHashes(highHash *externalapi.DomainHash) ([]*externalapi.DomainHash, error) {
s.lock.Lock()
defer s.lock.Unlock()
@ -701,7 +718,7 @@ func (s *consensus) Anticone(blockHash *externalapi.DomainHash) ([]*externalapi.
return nil, err
}
return s.dagTraversalManager.AnticoneFromBlocks(stagingArea, tips, blockHash)
return s.dagTraversalManager.AnticoneFromBlocks(stagingArea, tips, blockHash, 0)
}
func (s *consensus) EstimateNetworkHashesPerSecond(startHash *externalapi.DomainHash, windowSize int) (uint64, error) {

View File

@ -75,6 +75,11 @@ func (brs *blockRelationStore) Has(dbContext model.DBReader, stagingArea *model.
return dbContext.Has(brs.hashAsKey(blockHash))
}
func (brs *blockRelationStore) UnstageAll(stagingArea *model.StagingArea) {
stagingShard := brs.stagingShard(stagingArea)
stagingShard.toAdd = make(map[externalapi.DomainHash]*model.BlockRelations)
}
func (brs *blockRelationStore) hashAsKey(hash *externalapi.DomainHash) model.DBKey {
return brs.bucket.Key(hash.ByteSlice())
}

View File

@ -69,6 +69,12 @@ func (gds *ghostdagDataStore) Get(dbContext model.DBReader, stagingArea *model.S
return blockGHOSTDAGData, nil
}
func (gds *ghostdagDataStore) UnstageAll(stagingArea *model.StagingArea) {
stagingShard := gds.stagingShard(stagingArea)
stagingShard.toAdd = make(map[key]*externalapi.BlockGHOSTDAGData)
}
func (gds *ghostdagDataStore) serializeKey(k key) model.DBKey {
if k.isTrustedData {
return gds.trustedDataBucket.Key(k.hash.ByteSlice())

View File

@ -41,6 +41,27 @@ func (rds *reachabilityDataStore) StageReachabilityData(stagingArea *model.Stagi
stagingShard.reachabilityData[*blockHash] = reachabilityData
}
func (rds *reachabilityDataStore) Delete(dbContext model.DBWriter) error {
cursor, err := dbContext.Cursor(rds.reachabilityDataBucket)
if err != nil {
return err
}
for ok := cursor.First(); ok; ok = cursor.Next() {
key, err := cursor.Key()
if err != nil {
return err
}
err = dbContext.Delete(key)
if err != nil {
return err
}
}
return dbContext.Delete(rds.reachabilityReindexRootKey)
}
// StageReachabilityReindexRoot stages the given reachabilityReindexRoot
func (rds *reachabilityDataStore) StageReachabilityReindexRoot(stagingArea *model.StagingArea, reachabilityReindexRoot *externalapi.DomainHash) {
stagingShard := rds.stagingShard(stagingArea)

View File

@ -7,6 +7,7 @@ import (
"github.com/kaspanet/kaspad/domain/consensus/processes/blockparentbuilder"
parentssanager "github.com/kaspanet/kaspad/domain/consensus/processes/parentsmanager"
"github.com/kaspanet/kaspad/domain/consensus/processes/pruningproofmanager"
"github.com/kaspanet/kaspad/util/staging"
"io/ioutil"
"os"
"sync"
@ -147,11 +148,35 @@ func (f *factory) NewConsensus(config *Config, db infrastructuredatabase.Databas
daaBlocksStore := daablocksstore.New(prefixBucket, pruningWindowSizeForCaches, int(config.FinalityDepth()), preallocateCaches)
windowHeapSliceStore := blockwindowheapslicestore.New(2000, preallocateCaches)
newReachabilityDataStore := reachabilitydatastore.New(prefixBucket, pruningWindowSizePlusFinalityDepthForCache*2, preallocateCaches)
blockRelationStores, reachabilityDataStores, ghostdagDataStores := dagStores(config, prefixBucket, pruningWindowSizePlusFinalityDepthForCache, pruningWindowSizeForCaches, preallocateCaches)
reachabilityManagers, dagTopologyManagers, ghostdagManagers, dagTraversalManagers := f.dagProcesses(config, dbManager, blockHeaderStore, daaWindowStore, windowHeapSliceStore, blockRelationStores, reachabilityDataStores, ghostdagDataStores)
oldReachabilityManager := reachabilitymanager.New(
dbManager,
ghostdagDataStores[0],
reachabilityDataStores[0])
isOldReachabilityInitialized, err := reachabilityDataStores[0].HasReachabilityData(dbManager, model.NewStagingArea(), model.VirtualGenesisBlockHash)
if err != nil {
return nil, err
}
newReachabilityManager := reachabilitymanager.New(
dbManager,
ghostdagDataStores[0],
newReachabilityDataStore)
reachabilityManager := newReachabilityManager
if isOldReachabilityInitialized {
reachabilityManager = oldReachabilityManager
} else {
for i := range reachabilityDataStores {
reachabilityDataStores[i] = newReachabilityDataStore
}
}
reachabilityDataStore := reachabilityDataStores[0]
dagTopologyManagers, ghostdagManagers, dagTraversalManagers := f.dagProcesses(config, dbManager, blockHeaderStore, daaWindowStore, windowHeapSliceStore, blockRelationStores, reachabilityDataStores, ghostdagDataStores, isOldReachabilityInitialized)
blockRelationStore := blockRelationStores[0]
reachabilityDataStore := reachabilityDataStores[0]
ghostdagDataStore := ghostdagDataStores[0]
dagTopologyManager := dagTopologyManagers[0]
@ -317,7 +342,7 @@ func (f *factory) NewConsensus(config *Config, db infrastructuredatabase.Databas
dagTraversalManager,
coinbaseManager,
mergeDepthManager,
reachabilityManagers,
reachabilityManager,
finalityManager,
blockParentBuilder,
pruningManager,
@ -379,7 +404,7 @@ func (f *factory) NewConsensus(config *Config, db infrastructuredatabase.Databas
pruningManager,
blockValidator,
dagTopologyManager,
reachabilityManagers,
reachabilityManager,
difficultyManager,
pastMedianTimeManager,
coinbaseManager,
@ -407,9 +432,10 @@ func (f *factory) NewConsensus(config *Config, db infrastructuredatabase.Databas
dbManager,
dagTopologyManagers,
ghostdagManagers,
reachabilityManagers,
reachabilityManager,
dagTraversalManagers,
parentsManager,
pruningManager,
ghostdagDataStores,
pruningStore,
@ -417,6 +443,8 @@ func (f *factory) NewConsensus(config *Config, db infrastructuredatabase.Databas
blockStatusStore,
finalityStore,
consensusStateStore,
blockRelationStore,
reachabilityDataStore,
genesisHash,
config.K,
@ -446,7 +474,7 @@ func (f *factory) NewConsensus(config *Config, db infrastructuredatabase.Databas
headerTipsManager: headerTipsManager,
mergeDepthManager: mergeDepthManager,
pruningManager: pruningManager,
reachabilityManagers: reachabilityManagers,
reachabilityManager: reachabilityManager,
finalityManager: finalityManager,
pruningProofManager: pruningProofManager,
@ -460,7 +488,7 @@ func (f *factory) NewConsensus(config *Config, db infrastructuredatabase.Databas
consensusStateStore: consensusStateStore,
headersSelectedTipStore: headersSelectedTipStore,
multisetStore: multisetStore,
reachabilityDataStores: reachabilityDataStores,
reachabilityDataStore: reachabilityDataStore,
utxoDiffStore: utxoDiffStore,
finalityStore: finalityStore,
headersSelectedChainStore: headersSelectedChainStore,
@ -468,6 +496,58 @@ func (f *factory) NewConsensus(config *Config, db infrastructuredatabase.Databas
blocksWithTrustedDataDAAWindowStore: daaWindowStore,
}
if isOldReachabilityInitialized {
stagingArea := model.NewStagingArea()
dbTx, err := dbManager.Begin()
if err != nil {
return nil, err
}
err = newReachabilityDataStore.Delete(dbManager)
if err != nil {
return nil, err
}
err = dbTx.Commit()
if err != nil {
return nil, err
}
err = newReachabilityManager.Init(stagingArea)
if err != nil {
return nil, err
}
err = staging.CommitAllChanges(dbManager, stagingArea)
if err != nil {
return nil, err
}
err = c.pruningProofManager.RebuildReachability(newReachabilityDataStore)
if err != nil {
return nil, err
}
dbTx, err = dbManager.Begin()
if err != nil {
return nil, err
}
for _, store := range reachabilityDataStores {
err = store.Delete(dbManager)
if err != nil {
return nil, err
}
}
err = dbTx.Commit()
if err != nil {
return nil, err
}
return f.NewConsensus(config, db, dbPrefix)
}
err = c.Init(config.SkipAddingGenesis)
if err != nil {
return nil, err
@ -528,7 +608,7 @@ func (f *factory) NewTestConsensus(config *Config, testName string) (
database: db,
testConsensusStateManager: testConsensusStateManager,
testReachabilityManager: reachabilitymanager.NewTestReachabilityManager(consensusAsImplementation.
reachabilityManagers[0]),
reachabilityManager),
testTransactionValidator: testTransactionValidator,
}
tstConsensus.testBlockBuilder = blockbuilder.NewTestBlockBuilder(consensusAsImplementation.blockBuilder, tstConsensus)
@ -605,8 +685,8 @@ func (f *factory) dagProcesses(config *Config,
windowHeapSliceStore model.WindowHeapSliceStore,
blockRelationStores []model.BlockRelationStore,
reachabilityDataStores []model.ReachabilityDataStore,
ghostdagDataStores []model.GHOSTDAGDataStore) (
[]model.ReachabilityManager,
ghostdagDataStores []model.GHOSTDAGDataStore,
isOldReachabilityInitialized bool) (
[]model.DAGTopologyManager,
[]model.GHOSTDAGManager,
[]model.DAGTraversalManager,
@ -617,11 +697,20 @@ func (f *factory) dagProcesses(config *Config,
ghostdagManagers := make([]model.GHOSTDAGManager, config.MaxBlockLevel+1)
dagTraversalManagers := make([]model.DAGTraversalManager, config.MaxBlockLevel+1)
newReachabilityManager := reachabilitymanager.New(
dbManager,
ghostdagDataStores[0],
reachabilityDataStores[0])
for i := 0; i <= config.MaxBlockLevel; i++ {
reachabilityManagers[i] = reachabilitymanager.New(
dbManager,
ghostdagDataStores[i],
reachabilityDataStores[i])
if isOldReachabilityInitialized {
reachabilityManagers[i] = reachabilitymanager.New(
dbManager,
ghostdagDataStores[i],
reachabilityDataStores[i])
} else {
reachabilityManagers[i] = newReachabilityManager
}
dagTopologyManagers[i] = dagtopologymanager.New(
dbManager,
@ -649,5 +738,5 @@ func (f *factory) dagProcesses(config *Config,
config.DifficultyAdjustmentWindowSize)
}
return reachabilityManagers, dagTopologyManagers, ghostdagManagers, dagTraversalManagers
return dagTopologyManagers, ghostdagManagers, dagTraversalManagers
}

View File

@ -5,3 +5,7 @@ import "github.com/pkg/errors"
// ErrBlockNotInSelectedParentChain is returned from CreateHeadersSelectedChainBlockLocator if one of the parameters
// passed to it are not in the headers selected parent chain
var ErrBlockNotInSelectedParentChain = errors.New("Block is not in selected parent chain")
// ErrReachedMaxTraversalAllowed is returned from AnticoneFromBlocks if `maxTraversalAllowed` was specified
// and the traversal passed it
var ErrReachedMaxTraversalAllowed = errors.New("Traversal searching for anticone passed the maxTraversalAllowed limit")

View File

@ -20,6 +20,7 @@ type Consensus interface {
GetBlockAcceptanceData(blockHash *DomainHash) (AcceptanceData, error)
GetHashesBetween(lowHash, highHash *DomainHash, maxBlocks uint64) (hashes []*DomainHash, actualHighHash *DomainHash, err error)
GetAnticone(blockHash, contextHash *DomainHash, maxBlocks uint64) (hashes []*DomainHash, err error)
GetMissingBlockBodyHashes(highHash *DomainHash) ([]*DomainHash, error)
GetPruningPointUTXOs(expectedPruningPointHash *DomainHash, fromOutpoint *DomainOutpoint, limit int) ([]*OutpointAndUTXOEntryPair, error)
GetVirtualUTXOs(expectedVirtualParents []*DomainHash, fromOutpoint *DomainOutpoint, limit int) ([]*OutpointAndUTXOEntryPair, error)

View File

@ -9,4 +9,5 @@ type BlockRelationStore interface {
IsStaged(stagingArea *StagingArea) bool
BlockRelation(dbContext DBReader, stagingArea *StagingArea, blockHash *externalapi.DomainHash) (*BlockRelations, error)
Has(dbContext DBReader, stagingArea *StagingArea, blockHash *externalapi.DomainHash) (bool, error)
UnstageAll(stagingArea *StagingArea)
}

View File

@ -8,4 +8,5 @@ type GHOSTDAGDataStore interface {
Stage(stagingArea *StagingArea, blockHash *externalapi.DomainHash, blockGHOSTDAGData *externalapi.BlockGHOSTDAGData, isTrustedData bool)
IsStaged(stagingArea *StagingArea) bool
Get(dbContext DBReader, stagingArea *StagingArea, blockHash *externalapi.DomainHash, isTrustedData bool) (*externalapi.BlockGHOSTDAGData, error)
UnstageAll(stagingArea *StagingArea)
}

View File

@ -11,4 +11,5 @@ type ReachabilityDataStore interface {
ReachabilityData(dbContext DBReader, stagingArea *StagingArea, blockHash *externalapi.DomainHash) (ReachabilityData, error)
HasReachabilityData(dbContext DBReader, stagingArea *StagingArea, blockHash *externalapi.DomainHash) (bool, error)
ReachabilityReindexRoot(dbContext DBReader, stagingArea *StagingArea) (*externalapi.DomainHash, error)
Delete(dbContext DBWriter) error
}

View File

@ -10,7 +10,7 @@ type DAGTraversalManager interface {
// from lowHash (exclusive) to highHash (inclusive) over highHash's selected parent chain
SelectedChildIterator(stagingArea *StagingArea, highHash, lowHash *externalapi.DomainHash, includeLowHash bool) (BlockIterator, error)
SelectedChild(stagingArea *StagingArea, highHash, lowHash *externalapi.DomainHash) (*externalapi.DomainHash, error)
AnticoneFromBlocks(stagingArea *StagingArea, tips []*externalapi.DomainHash, blockHash *externalapi.DomainHash) ([]*externalapi.DomainHash, error)
AnticoneFromBlocks(stagingArea *StagingArea, tips []*externalapi.DomainHash, blockHash *externalapi.DomainHash, maxTraversalAllowed uint64) ([]*externalapi.DomainHash, error)
AnticoneFromVirtualPOV(stagingArea *StagingArea, blockHash *externalapi.DomainHash) ([]*externalapi.DomainHash, error)
BlockWindow(stagingArea *StagingArea, highHash *externalapi.DomainHash, windowSize int) ([]*externalapi.DomainHash, error)
DAABlockWindow(stagingArea *StagingArea, highHash *externalapi.DomainHash) ([]*externalapi.DomainHash, error)

View File

@ -7,4 +7,5 @@ type PruningProofManager interface {
BuildPruningPointProof(stagingArea *StagingArea) (*externalapi.PruningPointProof, error)
ValidatePruningPointProof(pruningPointProof *externalapi.PruningPointProof) error
ApplyPruningPointProof(pruningPointProof *externalapi.PruningPointProof) error
RebuildReachability(targetReachabilityManager ReachabilityDataStore) error
}

View File

@ -6,6 +6,7 @@ import "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
type SyncManager interface {
GetHashesBetween(stagingArea *StagingArea, lowHash, highHash *externalapi.DomainHash, maxBlocks uint64) (
hashes []*externalapi.DomainHash, actualHighHash *externalapi.DomainHash, err error)
GetAnticone(stagingArea *StagingArea, blockHash, contextHash *externalapi.DomainHash, maxBlocks uint64) (hashes []*externalapi.DomainHash, err error)
GetMissingBlockBodyHashes(stagingArea *StagingArea, highHash *externalapi.DomainHash) ([]*externalapi.DomainHash, error)
CreateBlockLocator(stagingArea *StagingArea, lowHash, highHash *externalapi.DomainHash, limit uint32) (
externalapi.BlockLocator, error)

View File

@ -104,6 +104,7 @@ type FutureCoveringTreeNodeSet []*externalapi.DomainHash
// Clone returns a clone of FutureCoveringTreeNodeSet
func (fctns FutureCoveringTreeNodeSet) Clone() FutureCoveringTreeNodeSet {
//return fctns
return externalapi.CloneHashes(fctns)
}

View File

@ -49,6 +49,7 @@ type TestConsensus interface {
*externalapi.VirtualChangeSet, error)
MineJSON(r io.Reader, blockType MineJSONBlockType) (tips []*externalapi.DomainHash, err error)
ToJSON(w io.Writer) error
RenderDAGToDot(filename string) error

View File

@ -22,7 +22,7 @@ type blockProcessor struct {
pruningManager model.PruningManager
blockValidator model.BlockValidator
dagTopologyManager model.DAGTopologyManager
reachabilityManagers []model.ReachabilityManager
reachabilityManager model.ReachabilityManager
difficultyManager model.DifficultyManager
pastMedianTimeManager model.PastMedianTimeManager
coinbaseManager model.CoinbaseManager
@ -60,7 +60,7 @@ func New(
pruningManager model.PruningManager,
blockValidator model.BlockValidator,
dagTopologyManager model.DAGTopologyManager,
reachabilityManagers []model.ReachabilityManager,
reachabilityManager model.ReachabilityManager,
difficultyManager model.DifficultyManager,
pastMedianTimeManager model.PastMedianTimeManager,
coinbaseManager model.CoinbaseManager,
@ -94,7 +94,7 @@ func New(
pruningManager: pruningManager,
blockValidator: blockValidator,
dagTopologyManager: dagTopologyManager,
reachabilityManagers: reachabilityManagers,
reachabilityManager: reachabilityManager,
difficultyManager: difficultyManager,
pastMedianTimeManager: pastMedianTimeManager,
coinbaseManager: coinbaseManager,

View File

@ -254,19 +254,7 @@ func (bp *blockProcessor) updateReachabilityReindexRoot(stagingArea *model.Stagi
return nil
}
headersSelectedTipHeader, err := bp.blockHeaderStore.BlockHeader(bp.databaseContext, stagingArea, headersSelectedTip)
if err != nil {
return err
}
headersSelectedTipHeaderBlockLevel := headersSelectedTipHeader.BlockLevel(bp.maxBlockLevel)
for blockLevel := 0; blockLevel <= headersSelectedTipHeaderBlockLevel; blockLevel++ {
err := bp.reachabilityManagers[blockLevel].UpdateReindexRoot(stagingArea, headersSelectedTip)
if err != nil {
return err
}
}
return nil
return bp.reachabilityManager.UpdateReindexRoot(stagingArea, headersSelectedTip)
}
func (bp *blockProcessor) checkBlockStatus(stagingArea *model.StagingArea, block *externalapi.DomainBlock) error {

View File

@ -62,12 +62,9 @@ func (v *blockValidator) ValidateHeaderInContext(stagingArea *model.StagingArea,
return err
}
if !hasReachabilityData {
blockLevel := header.BlockLevel(v.maxBlockLevel)
for i := 0; i <= blockLevel; i++ {
err = v.reachabilityManagers[i].AddBlock(stagingArea, blockHash)
if err != nil {
return err
}
err = v.reachabilityManager.AddBlock(stagingArea, blockHash)
if err != nil {
return err
}
}

View File

@ -35,7 +35,7 @@ type blockValidator struct {
coinbaseManager model.CoinbaseManager
mergeDepthManager model.MergeDepthManager
pruningStore model.PruningStore
reachabilityManagers []model.ReachabilityManager
reachabilityManager model.ReachabilityManager
finalityManager model.FinalityManager
blockParentBuilder model.BlockParentBuilder
pruningManager model.PruningManager
@ -73,7 +73,7 @@ func New(powMax *big.Int,
dagTraversalManager model.DAGTraversalManager,
coinbaseManager model.CoinbaseManager,
mergeDepthManager model.MergeDepthManager,
reachabilityManagers []model.ReachabilityManager,
reachabilityManager model.ReachabilityManager,
finalityManager model.FinalityManager,
blockParentBuilder model.BlockParentBuilder,
pruningManager model.PruningManager,
@ -112,7 +112,7 @@ func New(powMax *big.Int,
dagTraversalManager: dagTraversalManager,
coinbaseManager: coinbaseManager,
mergeDepthManager: mergeDepthManager,
reachabilityManagers: reachabilityManagers,
reachabilityManager: reachabilityManager,
finalityManager: finalityManager,
blockParentBuilder: blockParentBuilder,
pruningManager: pruningManager,

View File

@ -206,5 +206,25 @@ func (dtm *dagTopologyManager) ChildInSelectedParentChainOf(stagingArea *model.S
context, specifiedHighHash)
}
return dtm.reachabilityManager.FindNextAncestor(stagingArea, highHash, context)
children, err := dtm.Children(stagingArea, context)
if err != nil {
return nil, err
}
for _, child := range children {
if child.Equal(model.VirtualBlockHash) {
continue
}
isInSelectedParentChain, err := dtm.IsInSelectedParentChainOf(stagingArea, child, highHash)
if err != nil {
return nil, err
}
if isInSelectedParentChain {
return child, nil
}
}
return nil, errors.Errorf("Couldn't find child in the selected chain of %s", highHash)
}

View File

@ -14,16 +14,18 @@ func (dtm *dagTraversalManager) AnticoneFromVirtualPOV(stagingArea *model.Stagin
return nil, err
}
return dtm.AnticoneFromBlocks(stagingArea, virtualParents, blockHash)
return dtm.AnticoneFromBlocks(stagingArea, virtualParents, blockHash, 0)
}
func (dtm *dagTraversalManager) AnticoneFromBlocks(stagingArea *model.StagingArea, tips []*externalapi.DomainHash, blockHash *externalapi.DomainHash) (
func (dtm *dagTraversalManager) AnticoneFromBlocks(stagingArea *model.StagingArea, tips []*externalapi.DomainHash,
blockHash *externalapi.DomainHash, maxTraversalAllowed uint64) (
[]*externalapi.DomainHash, error) {
anticone := []*externalapi.DomainHash{}
queue := tips
visited := hashset.New()
traversalCounter := uint64(0)
for len(queue) > 0 {
var current *externalapi.DomainHash
current, queue = queue[0], queue[1:]
@ -48,6 +50,13 @@ func (dtm *dagTraversalManager) AnticoneFromBlocks(stagingArea *model.StagingAre
return nil, err
}
// We count the number of blocks in past(tips) \setminus past(blockHash).
// We don't use `len(visited)` since it includes some maximal blocks in past(blockHash) as well.
traversalCounter++
if maxTraversalAllowed > 0 && traversalCounter > maxTraversalAllowed {
return nil, model.ErrReachedMaxTraversalAllowed
}
if !blockIsAncestorOfCurrent {
anticone = append(anticone, current)
}

View File

@ -320,10 +320,6 @@ func (ds *GHOSTDAGDataStoreImpl) IsStaged(*model.StagingArea) bool {
panic("implement me")
}
func (ds *GHOSTDAGDataStoreImpl) Discard() {
panic("implement me")
}
func (ds *GHOSTDAGDataStoreImpl) Commit(dbTx model.DBTransaction) error {
panic("implement me")
}
@ -336,6 +332,10 @@ func (ds *GHOSTDAGDataStoreImpl) Get(dbContext model.DBReader, stagingArea *mode
return nil, nil
}
func (ds *GHOSTDAGDataStoreImpl) UnstageAll(stagingArea *model.StagingArea) {
panic("implement me")
}
type DAGTopologyManagerImpl struct {
parentsMap map[externalapi.DomainHash][]*externalapi.DomainHash
}

View File

@ -9,6 +9,7 @@ import (
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/kaspanet/kaspad/domain/consensus/processes/dagtopologymanager"
"github.com/kaspanet/kaspad/domain/consensus/processes/dagtraversalmanager"
"github.com/kaspanet/kaspad/domain/consensus/processes/ghostdagmanager"
"github.com/kaspanet/kaspad/domain/consensus/processes/reachabilitymanager"
"github.com/kaspanet/kaspad/domain/consensus/ruleerrors"
@ -26,16 +27,19 @@ type pruningProofManager struct {
dagTopologyManagers []model.DAGTopologyManager
ghostdagManagers []model.GHOSTDAGManager
reachabilityManagers []model.ReachabilityManager
reachabilityManager model.ReachabilityManager
dagTraversalManagers []model.DAGTraversalManager
parentsManager model.ParentsManager
pruningManager model.PruningManager
ghostdagDataStores []model.GHOSTDAGDataStore
pruningStore model.PruningStore
blockHeaderStore model.BlockHeaderStore
blockStatusStore model.BlockStatusStore
finalityStore model.FinalityStore
consensusStateStore model.ConsensusStateStore
ghostdagDataStores []model.GHOSTDAGDataStore
pruningStore model.PruningStore
blockHeaderStore model.BlockHeaderStore
blockStatusStore model.BlockStatusStore
finalityStore model.FinalityStore
consensusStateStore model.ConsensusStateStore
blockRelationStore model.BlockRelationStore
reachabilityDataStore model.ReachabilityDataStore
genesisHash *externalapi.DomainHash
k externalapi.KType
@ -52,9 +56,10 @@ func New(
dagTopologyManagers []model.DAGTopologyManager,
ghostdagManagers []model.GHOSTDAGManager,
reachabilityManagers []model.ReachabilityManager,
reachabilityManager model.ReachabilityManager,
dagTraversalManagers []model.DAGTraversalManager,
parentsManager model.ParentsManager,
pruningManager model.PruningManager,
ghostdagDataStores []model.GHOSTDAGDataStore,
pruningStore model.PruningStore,
@ -62,6 +67,8 @@ func New(
blockStatusStore model.BlockStatusStore,
finalityStore model.FinalityStore,
consensusStateStore model.ConsensusStateStore,
blockRelationStore model.BlockRelationStore,
reachabilityDataStore model.ReachabilityDataStore,
genesisHash *externalapi.DomainHash,
k externalapi.KType,
@ -73,16 +80,19 @@ func New(
databaseContext: databaseContext,
dagTopologyManagers: dagTopologyManagers,
ghostdagManagers: ghostdagManagers,
reachabilityManagers: reachabilityManagers,
reachabilityManager: reachabilityManager,
dagTraversalManagers: dagTraversalManagers,
parentsManager: parentsManager,
pruningManager: pruningManager,
ghostdagDataStores: ghostdagDataStores,
pruningStore: pruningStore,
blockHeaderStore: blockHeaderStore,
blockStatusStore: blockStatusStore,
finalityStore: finalityStore,
consensusStateStore: consensusStateStore,
ghostdagDataStores: ghostdagDataStores,
pruningStore: pruningStore,
blockHeaderStore: blockHeaderStore,
blockStatusStore: blockStatusStore,
finalityStore: finalityStore,
consensusStateStore: consensusStateStore,
blockRelationStore: blockRelationStore,
reachabilityDataStore: reachabilityDataStore,
genesisHash: genesisHash,
k: k,
@ -611,6 +621,265 @@ func (ppm *pruningProofManager) dagProcesses(
return reachabilityManagers, dagTopologyManagers, ghostdagManagers
}
func (ppm *pruningProofManager) RebuildReachability(targetReachabilityDataStore model.ReachabilityDataStore) error {
pruningPointProof, err := ppm.buildPruningPointProof(model.NewStagingArea())
if err != nil {
return err
}
err = ppm.populateProofReachabilityAndHeaders(pruningPointProof, targetReachabilityDataStore)
if err != nil {
return err
}
pruningPointAndItsAnticone, err := ppm.pruningManager.PruningPointAndItsAnticone()
if err != nil {
return err
}
targetReachabilityManager := reachabilitymanager.New(
ppm.databaseContext,
ppm.ghostdagDataStores[0],
targetReachabilityDataStore)
pruningPoint, err := ppm.pruningStore.PruningPoint(ppm.databaseContext, model.NewStagingArea())
if err != nil {
return err
}
pruningPointGHOSTDAGData, err := ppm.ghostdagDataStores[0].Get(ppm.databaseContext, model.NewStagingArea(), pruningPoint, false)
if err != nil {
return err
}
virtualGHOSTDAGData, err := ppm.ghostdagDataStores[0].Get(ppm.databaseContext, model.NewStagingArea(), model.VirtualBlockHash, false)
if err != nil {
return err
}
estimatedTotal := virtualGHOSTDAGData.BlueScore() - pruningPointGHOSTDAGData.BlueScore()
stagingArea := model.NewStagingArea()
visited := hashset.New()
queue := ppm.dagTraversalManagers[0].NewUpHeap(model.NewStagingArea())
err = queue.PushSlice(pruningPointAndItsAnticone)
if err != nil {
return err
}
i := uint64(0)
selectedTip := pruningPoint
for queue.Len() > 0 {
current := queue.Pop()
if visited.Contains(current) {
continue
}
visited.Add(current)
if current.Equal(model.VirtualBlockHash) {
continue
}
hasReachabilityData, err := targetReachabilityDataStore.HasReachabilityData(ppm.databaseContext, stagingArea, current)
if err != nil {
return err
}
if !hasReachabilityData {
err = targetReachabilityManager.AddBlock(stagingArea, current)
if err != nil {
return err
}
selectedTip, err = ppm.ghostdagManagers[0].ChooseSelectedParent(stagingArea, current, selectedTip)
if err != nil {
return err
}
if current.Equal(selectedTip) {
err := targetReachabilityManager.UpdateReindexRoot(stagingArea, selectedTip)
if err != nil {
return err
}
}
}
i++
if i%1000 == 0 {
percents := 100 * i / estimatedTotal
if percents > 100 {
percents = 100
}
log.Infof("Rebuilding reachability: %d%%", percents)
}
children, err := ppm.dagTopologyManagers[0].Children(model.NewStagingArea(), current)
if err != nil {
return err
}
err = queue.PushSlice(children)
if err != nil {
return err
}
}
err = reachabilitymanager.NewTestReachabilityManager(targetReachabilityManager).ValidateIntervals(model.VirtualGenesisBlockHash)
if err != nil {
return err
}
return staging.CommitAllChanges(ppm.databaseContext, stagingArea)
}
func (ppm *pruningProofManager) populateProofReachabilityAndHeaders(pruningPointProof *externalapi.PruningPointProof,
targetReachabilityDataStore model.ReachabilityDataStore) error {
// We build a DAG of all multi-level relations between blocks in the proof. We make a upHeap of all blocks, so we can iterate
// over them in a topological way, and then build a DAG where we use all multi-level parents of a block to create edges, except
// parents that are already in the past of another parent (This can happen between two levels). We run GHOSTDAG on each block of
// that DAG, because GHOSTDAG is a requirement to calculate reachability. We then dismiss the GHOSTDAG data because it's not related
// to the GHOSTDAG data of the real DAG, and was used only for reachability.
// We need two staging areas: stagingArea which is used to commit the reachability data, and tmpStagingArea for the GHOSTDAG data
// of allProofBlocksUpHeap. The reason we need two areas is that we use the real GHOSTDAG data in order to order the heap in a topological
// way, and fake GHOSTDAG data for calculating reachability.
stagingArea := model.NewStagingArea()
tmpStagingArea := model.NewStagingArea()
bucket := consensusDB.MakeBucket([]byte("TMP"))
ghostdagDataStoreForTargetReachabilityManager := ghostdagdatastore.New(bucket, 0, false)
ghostdagDataStoreForTargetReachabilityManager.Stage(stagingArea, model.VirtualGenesisBlockHash, externalapi.NewBlockGHOSTDAGData(
0,
big.NewInt(0),
nil,
nil,
nil,
nil,
), false)
targetReachabilityManager := reachabilitymanager.New(ppm.databaseContext, ghostdagDataStoreForTargetReachabilityManager, targetReachabilityDataStore)
blockRelationStoreForTargetReachabilityManager := blockrelationstore.New(bucket, 0, false)
dagTopologyManagerForTargetReachabilityManager := dagtopologymanager.New(ppm.databaseContext, targetReachabilityManager, blockRelationStoreForTargetReachabilityManager, nil)
ghostdagManagerForTargetReachabilityManager := ghostdagmanager.New(ppm.databaseContext, dagTopologyManagerForTargetReachabilityManager, ghostdagDataStoreForTargetReachabilityManager, ppm.blockHeaderStore, 0, nil)
err := dagTopologyManagerForTargetReachabilityManager.SetParents(stagingArea, model.VirtualGenesisBlockHash, nil)
if err != nil {
return err
}
dagTopologyManager := dagtopologymanager.New(ppm.databaseContext, targetReachabilityManager, nil, nil)
ghostdagDataStore := ghostdagdatastore.New(bucket, 0, false)
tmpGHOSTDAGManager := ghostdagmanager.New(ppm.databaseContext, nil, ghostdagDataStore, nil, 0, nil)
dagTraversalManager := dagtraversalmanager.New(ppm.databaseContext, nil, ghostdagDataStore, nil, tmpGHOSTDAGManager, nil, nil, nil, 0)
allProofBlocksUpHeap := dagTraversalManager.NewUpHeap(tmpStagingArea)
dag := make(map[externalapi.DomainHash]struct {
parents hashset.HashSet
header externalapi.BlockHeader
})
for _, headers := range pruningPointProof.Headers {
for _, header := range headers {
blockHash := consensushashing.HeaderHash(header)
if _, ok := dag[*blockHash]; ok {
continue
}
dag[*blockHash] = struct {
parents hashset.HashSet
header externalapi.BlockHeader
}{parents: hashset.New(), header: header}
for level := 0; level <= ppm.maxBlockLevel; level++ {
for _, parent := range ppm.parentsManager.ParentsAtLevel(header, level) {
parent := parent
dag[*blockHash].parents.Add(parent)
}
}
// We stage temporary GHOSTDAG data that is needed in order to sort allProofBlocksUpHeap.
ghostdagDataStore.Stage(tmpStagingArea, blockHash, externalapi.NewBlockGHOSTDAGData(header.BlueScore(), header.BlueWork(), nil, nil, nil, nil), false)
err := allProofBlocksUpHeap.Push(blockHash)
if err != nil {
return err
}
}
}
var selectedTip *externalapi.DomainHash
for allProofBlocksUpHeap.Len() > 0 {
blockHash := allProofBlocksUpHeap.Pop()
block := dag[*blockHash]
parentsHeap := dagTraversalManager.NewDownHeap(tmpStagingArea)
for parent := range block.parents {
parent := parent
if _, ok := dag[parent]; !ok {
continue
}
err := parentsHeap.Push(&parent)
if err != nil {
return err
}
}
fakeParents := []*externalapi.DomainHash{}
for parentsHeap.Len() > 0 {
parent := parentsHeap.Pop()
isAncestorOfAny, err := dagTopologyManager.IsAncestorOfAny(stagingArea, parent, fakeParents)
if err != nil {
return err
}
if isAncestorOfAny {
continue
}
fakeParents = append(fakeParents, parent)
}
if len(fakeParents) == 0 {
fakeParents = append(fakeParents, model.VirtualGenesisBlockHash)
}
err := dagTopologyManagerForTargetReachabilityManager.SetParents(stagingArea, blockHash, fakeParents)
if err != nil {
return err
}
err = ghostdagManagerForTargetReachabilityManager.GHOSTDAG(stagingArea, blockHash)
if err != nil {
return err
}
err = targetReachabilityManager.AddBlock(stagingArea, blockHash)
if err != nil {
return err
}
if selectedTip == nil {
selectedTip = blockHash
} else {
selectedTip, err = ghostdagManagerForTargetReachabilityManager.ChooseSelectedParent(stagingArea, selectedTip, blockHash)
if err != nil {
return err
}
}
if selectedTip.Equal(blockHash) {
err := targetReachabilityManager.UpdateReindexRoot(stagingArea, selectedTip)
if err != nil {
return err
}
}
}
ghostdagDataStoreForTargetReachabilityManager.UnstageAll(stagingArea)
blockRelationStoreForTargetReachabilityManager.UnstageAll(stagingArea)
err = staging.CommitAllChanges(ppm.databaseContext, stagingArea)
if err != nil {
return err
}
return nil
}
// ApplyPruningPointProof applies the given pruning proof to the current consensus. Specifically,
// it's meant to be used against the StagingConsensus during headers-proof IBD. Note that for
// performance reasons this operation is NOT atomic. If the process fails for whatever reason
@ -619,9 +888,25 @@ func (ppm *pruningProofManager) ApplyPruningPointProof(pruningPointProof *extern
onEnd := logger.LogAndMeasureExecutionTime(log, "ApplyPruningPointProof")
defer onEnd()
stagingArea := model.NewStagingArea()
for _, headers := range pruningPointProof.Headers {
for _, header := range headers {
blockHash := consensushashing.HeaderHash(header)
ppm.blockHeaderStore.Stage(stagingArea, blockHash, header)
}
}
err := staging.CommitAllChanges(ppm.databaseContext, stagingArea)
if err != nil {
return err
}
err = ppm.populateProofReachabilityAndHeaders(pruningPointProof, ppm.reachabilityDataStore)
if err != nil {
return err
}
for blockLevel, headers := range pruningPointProof.Headers {
log.Infof("Applying level %d from the pruning point proof", blockLevel)
var selectedTip *externalapi.DomainHash
for i, header := range headers {
if i%1000 == 0 {
log.Infof("Applying level %d from the pruning point proof - applied %d headers out of %d", blockLevel, i, len(headers))
@ -687,27 +972,6 @@ func (ppm *pruningProofManager) ApplyPruningPointProof(pruningPointProof *extern
ppm.blockStatusStore.Stage(stagingArea, blockHash, externalapi.StatusHeaderOnly)
}
if selectedTip == nil {
selectedTip = blockHash
} else {
selectedTip, err = ppm.ghostdagManagers[blockLevel].ChooseSelectedParent(stagingArea, selectedTip, blockHash)
if err != nil {
return err
}
}
err = ppm.reachabilityManagers[blockLevel].AddBlock(stagingArea, blockHash)
if err != nil {
return err
}
if selectedTip.Equal(blockHash) {
err := ppm.reachabilityManagers[blockLevel].UpdateReindexRoot(stagingArea, selectedTip)
if err != nil {
return err
}
}
err = staging.CommitAllChanges(ppm.databaseContext, stagingArea)
if err != nil {
return err
@ -718,7 +982,7 @@ func (ppm *pruningProofManager) ApplyPruningPointProof(pruningPointProof *extern
pruningPointHeader := pruningPointProof.Headers[0][len(pruningPointProof.Headers[0])-1]
pruningPoint := consensushashing.HeaderHash(pruningPointHeader)
stagingArea := model.NewStagingArea()
stagingArea = model.NewStagingArea()
ppm.consensusStateStore.StageTips(stagingArea, []*externalapi.DomainHash{pruningPoint})
return staging.CommitAllChanges(ppm.databaseContext, stagingArea)
}

View File

@ -16,7 +16,7 @@ type reachabilityDataStoreMock struct {
reachabilityReindexRootStaging *externalapi.DomainHash
}
func (r *reachabilityDataStoreMock) Discard() {
func (r *reachabilityDataStoreMock) Delete(_ model.DBWriter) error {
panic("implement me")
}

View File

@ -4,6 +4,7 @@ import (
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/kaspanet/kaspad/infrastructure/logger"
"github.com/pkg/errors"
)
type syncManager struct {
@ -69,6 +70,21 @@ func (sm *syncManager) GetHashesBetween(stagingArea *model.StagingArea, lowHash,
return sm.antiPastHashesBetween(stagingArea, lowHash, highHash, maxBlocks)
}
func (sm *syncManager) GetAnticone(stagingArea *model.StagingArea, blockHash, contextHash *externalapi.DomainHash, maxBlocks uint64) (hashes []*externalapi.DomainHash, err error) {
onEnd := logger.LogAndMeasureExecutionTime(log, "GetAnticone")
defer onEnd()
isContextAncestorOfBlock, err := sm.dagTopologyManager.IsAncestorOf(stagingArea, contextHash, blockHash)
if err != nil {
return nil, err
}
if isContextAncestorOfBlock {
return nil, errors.Errorf("expected block %s to not be in future of %s",
blockHash,
contextHash)
}
return sm.dagTraversalManager.AnticoneFromBlocks(stagingArea, []*externalapi.DomainHash{contextHash}, blockHash, maxBlocks)
}
func (sm *syncManager) GetMissingBlockBodyHashes(stagingArea *model.StagingArea, highHash *externalapi.DomainHash) ([]*externalapi.DomainHash, error) {
onEnd := logger.LogAndMeasureExecutionTime(log, "GetMissingBlockBodyHashes")
defer onEnd()

View File

@ -2,6 +2,9 @@ package consensus
import (
"encoding/json"
"fmt"
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/utils/hashset"
"io"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
@ -108,13 +111,13 @@ func (tc *testConsensus) AddUTXOInvalidBlock(parentHashes []*externalapi.DomainH
return consensushashing.BlockHash(block), virtualChangeSet, nil
}
func (tc *testConsensus) MineJSON(r io.Reader, blockType testapi.MineJSONBlockType) (tips []*externalapi.DomainHash, err error) {
// jsonBlock is a json representation of a block in mine format
type jsonBlock struct {
ID string `json:"id"`
Parents []string `json:"parents"`
}
// jsonBlock is a json representation of a block in mine format
type jsonBlock struct {
ID string `json:"id"`
Parents []string `json:"parents"`
}
func (tc *testConsensus) MineJSON(r io.Reader, blockType testapi.MineJSONBlockType) (tips []*externalapi.DomainHash, err error) {
tipSet := map[externalapi.DomainHash]*externalapi.DomainHash{}
tipSet[*tc.dagParams.GenesisHash] = tc.dagParams.GenesisHash
@ -182,6 +185,64 @@ func (tc *testConsensus) MineJSON(r io.Reader, blockType testapi.MineJSONBlockTy
return tips, nil
}
func (tc *testConsensus) ToJSON(w io.Writer) error {
hashToId := make(map[externalapi.DomainHash]string)
lastID := 0
encoder := json.NewEncoder(w)
visited := hashset.New()
queue := tc.dagTraversalManager.NewUpHeap(model.NewStagingArea())
err := queue.Push(tc.dagParams.GenesisHash)
if err != nil {
return err
}
blocksToAdd := make([]jsonBlock, 0)
for queue.Len() > 0 {
current := queue.Pop()
if visited.Contains(current) {
continue
}
visited.Add(current)
if current.Equal(model.VirtualBlockHash) {
continue
}
header, err := tc.blockHeaderStore.BlockHeader(tc.databaseContext, model.NewStagingArea(), current)
if err != nil {
return err
}
directParents := header.DirectParents()
parentIDs := make([]string, len(directParents))
for i, parent := range directParents {
parentIDs[i] = hashToId[*parent]
}
lastIDStr := fmt.Sprintf("%d", lastID)
blocksToAdd = append(blocksToAdd, jsonBlock{
ID: lastIDStr,
Parents: parentIDs,
})
hashToId[*current] = lastIDStr
lastID++
children, err := tc.dagTopologyManagers[0].Children(model.NewStagingArea(), current)
if err != nil {
return err
}
err = queue.PushSlice(children)
if err != nil {
return err
}
}
return encoder.Encode(blocksToAdd)
}
func (tc *testConsensus) BuildUTXOInvalidBlock(parentHashes []*externalapi.DomainHash) (*externalapi.DomainBlock, error) {
// Require write lock because BuildBlockWithParents stages temporary data
tc.lock.Lock()

View File

@ -60,7 +60,7 @@ func (tc *testConsensus) PruningStore() model.PruningStore {
}
func (tc *testConsensus) ReachabilityDataStore() model.ReachabilityDataStore {
return tc.reachabilityDataStores[0]
return tc.reachabilityDataStore
}
func (tc *testConsensus) UTXODiffStore() model.UTXODiffStore {

View File

@ -57,6 +57,7 @@ func (rd *reachabilityData) FutureCoveringSet() model.FutureCoveringTreeNodeSet
}
func (rd *reachabilityData) CloneMutable() model.MutableReachabilityData {
//return rd
return &reachabilityData{
children: externalapi.CloneHashes(rd.children),
parent: rd.parent,

View File

@ -52,7 +52,7 @@ const (
defaultSigCacheMaxSize = 100000
sampleConfigFilename = "sample-kaspad.conf"
defaultMaxUTXOCacheSize = 5000000000
defaultProtocolVersion = 4
defaultProtocolVersion = 5
)
var (

View File

@ -47,6 +47,10 @@ message KaspadMessage {
ReadyMessage ready = 50;
BlockWithTrustedDataV4Message blockWithTrustedDataV4 = 51;
TrustedDataMessage trustedData = 52;
RequestNextPruningPointAndItsAnticoneBlocksMessage requestNextPruningPointAndItsAnticoneBlocks = 53;
RequestIBDChainBlockLocatorMessage requestIBDChainBlockLocator = 54;
IbdChainBlockLocatorMessage ibdChainBlockLocator = 55;
RequestAnticoneMessage requestAnticone = 56;
GetCurrentNetworkRequestMessage getCurrentNetworkRequest = 1001;
GetCurrentNetworkResponseMessage getCurrentNetworkResponse = 1002;

View File

@ -190,6 +190,20 @@ message IbdBlockLocatorMessage {
repeated Hash blockLocatorHashes = 2;
}
message RequestIBDChainBlockLocatorMessage{
Hash lowHash = 1;
Hash highHash = 2;
}
message IbdChainBlockLocatorMessage {
repeated Hash blockLocatorHashes = 1;
}
message RequestAnticoneMessage{
Hash blockHash = 1;
Hash contextHash = 2;
}
message IbdBlockLocatorHighestHashMessage {
Hash highestHash = 1;
}
@ -204,6 +218,9 @@ message BlockHeadersMessage {
message RequestPruningPointAndItsAnticoneMessage {
}
message RequestNextPruningPointAndItsAnticoneBlocksMessage{
}
message BlockWithTrustedDataMessage {
BlockMessage block = 1;
uint64 daaScore = 2;

View File

@ -0,0 +1,33 @@
package protowire
import (
"github.com/kaspanet/kaspad/app/appmessage"
"github.com/pkg/errors"
)
func (x *KaspadMessage_IbdChainBlockLocator) toAppMessage() (appmessage.Message, error) {
if x == nil {
return nil, errors.Wrapf(errorNil, "KaspadMessage_IbdChainBlockLocator is nil")
}
return x.IbdChainBlockLocator.toAppMessage()
}
func (x *IbdChainBlockLocatorMessage) toAppMessage() (appmessage.Message, error) {
if x == nil {
return nil, errors.Wrapf(errorNil, "IbdChainBlockLocatorMessage is nil")
}
blockLocatorHashes, err := protoHashesToDomain(x.BlockLocatorHashes)
if err != nil {
return nil, err
}
return &appmessage.MsgIBDChainBlockLocator{
BlockLocatorHashes: blockLocatorHashes,
}, nil
}
func (x *KaspadMessage_IbdChainBlockLocator) fromAppMessage(message *appmessage.MsgIBDChainBlockLocator) error {
x.IbdChainBlockLocator = &IbdChainBlockLocatorMessage{
BlockLocatorHashes: domainHashesToProto(message.BlockLocatorHashes),
}
return nil
}

View File

@ -0,0 +1,42 @@
package protowire
import (
"github.com/kaspanet/kaspad/app/appmessage"
"github.com/pkg/errors"
)
func (x *KaspadMessage_RequestAnticone) toAppMessage() (appmessage.Message, error) {
if x == nil {
return nil, errors.Wrapf(errorNil, "KaspadMessage_RequestAnticone is nil")
}
return x.RequestAnticone.toAppMessage()
}
func (x *RequestAnticoneMessage) toAppMessage() (appmessage.Message, error) {
if x == nil {
return nil, errors.Wrapf(errorNil, "RequestAnticoneMessage is nil")
}
blockHash, err := x.BlockHash.toDomain()
if err != nil {
return nil, err
}
contextHash, err := x.ContextHash.toDomain()
if err != nil {
return nil, err
}
return &appmessage.MsgRequestAnticone{
BlockHash: blockHash,
ContextHash: contextHash,
}, nil
}
func (x *KaspadMessage_RequestAnticone) fromAppMessage(msgRequestPastDiff *appmessage.MsgRequestAnticone) error {
x.RequestAnticone = &RequestAnticoneMessage{
BlockHash: domainHashToProto(msgRequestPastDiff.BlockHash),
ContextHash: domainHashToProto(msgRequestPastDiff.ContextHash),
}
return nil
}

View File

@ -0,0 +1,55 @@
package protowire
import (
"github.com/kaspanet/kaspad/app/appmessage"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/pkg/errors"
)
func (x *KaspadMessage_RequestIBDChainBlockLocator) toAppMessage() (appmessage.Message, error) {
if x == nil {
return nil, errors.Wrapf(errorNil, "KaspadMessage_RequestIBDChainBlockLocator is nil")
}
return x.RequestIBDChainBlockLocator.toAppMessage()
}
func (x *RequestIBDChainBlockLocatorMessage) toAppMessage() (appmessage.Message, error) {
if x == nil {
return nil, errors.Wrapf(errorNil, "RequestIBDChainBlockLocatorMessage is nil")
}
var err error
var highHash, lowHash *externalapi.DomainHash
if x.HighHash != nil {
highHash, err = x.HighHash.toDomain()
if err != nil {
return nil, err
}
}
if x.LowHash != nil {
lowHash, err = x.LowHash.toDomain()
if err != nil {
return nil, err
}
}
return &appmessage.MsgRequestIBDChainBlockLocator{
HighHash: highHash,
LowHash: lowHash,
}, nil
}
func (x *KaspadMessage_RequestIBDChainBlockLocator) fromAppMessage(msgGetBlockLocator *appmessage.MsgRequestIBDChainBlockLocator) error {
var highHash, lowHash *Hash
if msgGetBlockLocator.HighHash != nil {
highHash = domainHashToProto(msgGetBlockLocator.HighHash)
}
if msgGetBlockLocator.LowHash != nil {
lowHash = domainHashToProto(msgGetBlockLocator.LowHash)
}
x.RequestIBDChainBlockLocator = &RequestIBDChainBlockLocatorMessage{
HighHash: highHash,
LowHash: lowHash,
}
return nil
}

View File

@ -0,0 +1,17 @@
package protowire
import (
"github.com/kaspanet/kaspad/app/appmessage"
"github.com/pkg/errors"
)
func (x *KaspadMessage_RequestNextPruningPointAndItsAnticoneBlocks) toAppMessage() (appmessage.Message, error) {
if x == nil {
return nil, errors.Wrapf(errorNil, "KaspadMessage_DonePruningPointAndItsAnticoneBlocks is nil")
}
return &appmessage.MsgRequestNextPruningPointAndItsAnticoneBlocks{}, nil
}
func (x *KaspadMessage_RequestNextPruningPointAndItsAnticoneBlocks) fromAppMessage(_ *appmessage.MsgRequestNextPruningPointAndItsAnticoneBlocks) error {
return nil
}

View File

@ -331,6 +331,34 @@ func toP2PPayload(message appmessage.Message) (isKaspadMessage_Payload, error) {
return nil, err
}
return payload, nil
case *appmessage.MsgRequestNextPruningPointAndItsAnticoneBlocks:
payload := new(KaspadMessage_RequestNextPruningPointAndItsAnticoneBlocks)
err := payload.fromAppMessage(message)
if err != nil {
return nil, err
}
return payload, nil
case *appmessage.MsgRequestIBDChainBlockLocator:
payload := new(KaspadMessage_RequestIBDChainBlockLocator)
err := payload.fromAppMessage(message)
if err != nil {
return nil, err
}
return payload, nil
case *appmessage.MsgIBDChainBlockLocator:
payload := new(KaspadMessage_IbdChainBlockLocator)
err := payload.fromAppMessage(message)
if err != nil {
return nil, err
}
return payload, nil
case *appmessage.MsgRequestAnticone:
payload := new(KaspadMessage_RequestAnticone)
err := payload.fromAppMessage(message)
if err != nil {
return nil, err
}
return payload, nil
default:
return nil, nil
}

View File

@ -1 +1,7 @@
{"skipProofOfWork":true, "mergeSetSizeLimit": 30, "finalityDuration": 30000, "hardForkOmitGenesisFromParentsDaaScore": 2505}
{
"skipProofOfWork": true,
"mergeSetSizeLimit": 30,
"finalityDuration": 200000,
"hardForkOmitGenesisFromParentsDaaScore": 2505,
"k": 0
}

View File

@ -0,0 +1,148 @@
package fast_pruning_ibd_test
import (
"github.com/kaspanet/kaspad/domain/consensus"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/kaspanet/kaspad/domain/consensus/utils/testutils"
"github.com/kaspanet/kaspad/domain/dagconfig"
"io/ioutil"
"os"
"path"
"testing"
"time"
)
// TestGenerateFastPruningIBDTest generates the json needed for dag-for-fast-pruning-ibd-test.json.gz
func TestGenerateFastPruningIBDTest(t *testing.T) {
t.Skip()
testutils.ForAllNets(t, true, func(t *testing.T, consensusConfig *consensus.Config) {
if consensusConfig.Name != dagconfig.DevnetParams.Name {
return
}
factory := consensus.NewFactory()
// This is done to reduce the pruning depth to 6 blocks
finalityDepth := 200
consensusConfig.FinalityDuration = time.Duration(finalityDepth) * consensusConfig.TargetTimePerBlock
consensusConfig.K = 0
consensusConfig.PruningProofM = 1
consensusConfig.MergeSetSizeLimit = 30
tc, teardownSyncer, err := factory.NewTestConsensus(consensusConfig, "TestValidateAndInsertPruningPointSyncer")
if err != nil {
t.Fatalf("Error setting up tc: %+v", err)
}
defer teardownSyncer(false)
numBlocks := finalityDepth
tipHash := consensusConfig.GenesisHash
for i := 0; i < numBlocks; i++ {
tipHash, _, err = tc.AddBlock([]*externalapi.DomainHash{tipHash}, nil, nil)
if err != nil {
t.Fatal(err)
}
}
tip, err := tc.GetBlock(tipHash)
if err != nil {
t.Fatal(err)
}
header := tip.Header.ToMutable()
for i := uint64(1); i < 1000; i++ {
if i%100 == 0 {
t.Logf("Added %d tips", i)
}
header.SetNonce(tip.Header.Nonce() + i)
block := &externalapi.DomainBlock{Header: header.ToImmutable(), Transactions: tip.Transactions}
_, err = tc.ValidateAndInsertBlock(block, true)
if err != nil {
t.Fatalf("ValidateAndInsertBlock: %+v", err)
}
}
emptyCoinbase := &externalapi.DomainCoinbaseData{
ScriptPublicKey: &externalapi.ScriptPublicKey{
Script: nil,
Version: 0,
},
}
pruningPoint, err := tc.PruningPoint()
if err != nil {
t.Fatal(err)
}
for i := 0; ; i++ {
currentPruningPoint, err := tc.PruningPoint()
if err != nil {
t.Fatal(err)
}
if !pruningPoint.Equal(currentPruningPoint) {
t.Fatalf("Pruning point unexpectedly changed")
}
tips, err := tc.Tips()
if err != nil {
t.Fatal(err)
}
if len(tips) == 1 {
break
}
if i%10 == 0 {
t.Logf("Number of tips: %d", len(tips))
}
block, err := tc.BuildBlock(emptyCoinbase, nil)
if err != nil {
t.Fatal(err)
}
_, err = tc.ValidateAndInsertBlock(block, true)
if err != nil {
t.Fatalf("ValidateAndInsertBlock: %+v", err)
}
}
for {
currentPruningPoint, err := tc.PruningPoint()
if err != nil {
t.Fatal(err)
}
if !pruningPoint.Equal(currentPruningPoint) {
break
}
block, err := tc.BuildBlock(emptyCoinbase, nil)
if err != nil {
t.Fatal(err)
}
_, err = tc.ValidateAndInsertBlock(block, true)
if err != nil {
t.Fatalf("ValidateAndInsertBlock: %+v", err)
}
}
file, err := ioutil.TempFile("", "")
if err != nil {
t.Fatal(err)
}
err = tc.ToJSON(file)
if err != nil {
t.Fatal(err)
}
stat, err := file.Stat()
if err != nil {
t.Fatal(err)
}
t.Logf("DAG saved at %s", path.Join(os.TempDir(), stat.Name()))
})
}