Implement pruning point proof (#1832)

* Calculate GHOSTDAG, reachability etc for each level

* Don't preallocate cache for dag stores except level 0 and reduce the number of connections in the integration test to 32

* Reduce the number of connections in the integration test to 16

* Increase page file

* BuildPruningPointProof

* BuildPruningPointProof

* Add PruningProofManager

* Implement ApplyPruningPointProof

* Add prefix and fix blockAtDepth and fill headersByLevel

* Some bug fixes

* Include all relevant blocks for each level in the proof

* Fix syncAndValidatePruningPointProof to return the right block hash

* Fix block window

* Fix isAncestorOfPruningPoint

* Ban for rule errors on pruning proof

* Find common ancestor for blockAtDepthMAtNextLevel

* Use pruning proof in TestValidateAndInsertImportedPruningPoint

* stage status and finality point for proof blocks

* Uncomment golint

* Change test timeouts

* Calculate merge set for ApplyPruningPointProof

* Increase test timeout

* Add better caching for daa window store

* Return to default timeout

* Add ErrPruningProofMissesBlocksBelowPruningPoint

* Add errDAAWindowBlockNotFound

* Force connection loop next iteration on connection manager stop

* Revert to Test64IncomingConnections

* Remove BlockAtDepth from DAGTraversalManager

* numBullies->16

* Set page file size to 8gb

* Increase p2p max message size

* Test64IncomingConnections->Test16IncomingConnections

* Add comment for PruningProofM

* Add comment in `func (c *ConnectionManager) Stop()`

* Rename isAncestorOfPruningPoint->isAncestorOfSelectedTip

* Revert page file to 16gb

* Improve ExpectedHeaderPruningPoint perf

* Fix comment

* Revert "Improve ExpectedHeaderPruningPoint perf"

This reverts commit bca1080e7140c78d510f51bbea858ae280c2f38e.

* Don't test windows
This commit is contained in:
Ori Newman 2021-10-26 09:48:27 +03:00 committed by GitHub
parent afaac28da1
commit 5dbb1da84b
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
39 changed files with 1175 additions and 623923 deletions

View File

@ -14,7 +14,7 @@ jobs:
strategy: strategy:
fail-fast: false fail-fast: false
matrix: matrix:
os: [ ubuntu-latest, macos-latest, windows-latest ] os: [ ubuntu-latest, macos-latest ]
name: Tests, ${{ matrix.os }} name: Tests, ${{ matrix.os }}
steps: steps:

View File

@ -248,8 +248,8 @@ func (flow *handleRelayInvsFlow) syncPruningPointFutureHeaders(consensus externa
} }
return nil return nil
} }
for _, block := range ibdBlocksMessage.BlockHeaders { for _, header := range ibdBlocksMessage.BlockHeaders {
err = flow.processHeader(consensus, block) err = flow.processHeader(consensus, header)
if err != nil { if err != nil {
return err return err
} }

View File

@ -7,6 +7,7 @@ import (
"github.com/kaspanet/kaspad/app/protocol/protocolerrors" "github.com/kaspanet/kaspad/app/protocol/protocolerrors"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi" "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/kaspanet/kaspad/domain/consensus/ruleerrors" "github.com/kaspanet/kaspad/domain/consensus/ruleerrors"
"github.com/kaspanet/kaspad/domain/consensus/utils/consensushashing"
"github.com/pkg/errors" "github.com/pkg/errors"
) )
@ -71,51 +72,73 @@ func (flow *handleRelayInvsFlow) checkIfHighHashHasMoreBlueWorkThanSelectedTip(h
return highBlock.Header.BlueWork().Cmp(headersSelectedTipInfo.BlueWork) > 0, nil return highBlock.Header.BlueWork().Cmp(headersSelectedTipInfo.BlueWork) > 0, nil
} }
func (flow *handleRelayInvsFlow) syncAndValidatePruningPointProof() error { func (flow *handleRelayInvsFlow) syncAndValidatePruningPointProof() (*externalapi.DomainHash, error) {
log.Infof("Downloading the pruning point proof from %s", flow.peer) log.Infof("Downloading the pruning point proof from %s", flow.peer)
err := flow.outgoingRoute.Enqueue(appmessage.NewMsgRequestPruningPointProof()) err := flow.outgoingRoute.Enqueue(appmessage.NewMsgRequestPruningPointProof())
if err != nil { if err != nil {
return err return nil, err
} }
message, err := flow.dequeueIncomingMessageAndSkipInvs(common.DefaultTimeout) message, err := flow.dequeueIncomingMessageAndSkipInvs(common.DefaultTimeout)
if err != nil { if err != nil {
return err return nil, err
} }
pruningPointProofMessage, ok := message.(*appmessage.MsgPruningPointProof) pruningPointProofMessage, ok := message.(*appmessage.MsgPruningPointProof)
if !ok { if !ok {
return protocolerrors.Errorf(true, "received unexpected message type. "+ return nil, protocolerrors.Errorf(true, "received unexpected message type. "+
"expected: %s, got: %s", appmessage.CmdPruningPointProof, message.Command()) "expected: %s, got: %s", appmessage.CmdPruningPointProof, message.Command())
} }
pruningPointProof := appmessage.MsgPruningPointProofToDomainPruningPointProof(pruningPointProofMessage) pruningPointProof := appmessage.MsgPruningPointProofToDomainPruningPointProof(pruningPointProofMessage)
return flow.Domain().Consensus().ValidatePruningPointProof(pruningPointProof) err = flow.Domain().Consensus().ValidatePruningPointProof(pruningPointProof)
if err != nil {
if errors.As(err, &ruleerrors.RuleError{}) {
return nil, protocolerrors.Wrapf(true, err, "pruning point proof validation failed")
}
return nil, err
}
err = flow.Domain().StagingConsensus().ApplyPruningPointProof(pruningPointProof)
if err != nil {
return nil, err
}
return consensushashing.HeaderHash(pruningPointProof.Headers[0][len(pruningPointProof.Headers[0])-1]), nil
} }
func (flow *handleRelayInvsFlow) downloadHeadersAndPruningUTXOSet(highHash *externalapi.DomainHash) error { func (flow *handleRelayInvsFlow) downloadHeadersAndPruningUTXOSet(highHash *externalapi.DomainHash) error {
err := flow.syncAndValidatePruningPointProof() proofPruningPoint, err := flow.syncAndValidatePruningPointProof()
if err != nil { if err != nil {
return err return err
} }
pruningPoint, err := flow.syncPruningPointsAndPruningPointAnticone() err = flow.syncPruningPointsAndPruningPointAnticone(proofPruningPoint)
if err != nil { if err != nil {
return err return err
} }
// TODO: Remove this condition once there's more proper way to check finality violation // TODO: Remove this condition once there's more proper way to check finality violation
// in the headers proof. // in the headers proof.
if pruningPoint.Equal(flow.Config().NetParams().GenesisHash) { if proofPruningPoint.Equal(flow.Config().NetParams().GenesisHash) {
return protocolerrors.Errorf(true, "the genesis pruning point violates finality") return protocolerrors.Errorf(true, "the genesis pruning point violates finality")
} }
err = flow.syncPruningPointFutureHeaders(flow.Domain().StagingConsensus(), pruningPoint, highHash) err = flow.syncPruningPointFutureHeaders(flow.Domain().StagingConsensus(), proofPruningPoint, highHash)
if err != nil { if err != nil {
return err return err
} }
log.Debugf("Blocks downloaded from peer %s", flow.peer) log.Debugf("Headers downloaded from peer %s", flow.peer)
highHashInfo, err := flow.Domain().StagingConsensus().GetBlockInfo(highHash)
if err != nil {
return err
}
if !highHashInfo.Exists {
return protocolerrors.Errorf(true, "the triggering IBD block was not sent")
}
log.Debugf("Syncing the current pruning point UTXO set") log.Debugf("Syncing the current pruning point UTXO set")
syncedPruningPointUTXOSetSuccessfully, err := flow.syncPruningPointUTXOSet(flow.Domain().StagingConsensus(), pruningPoint) syncedPruningPointUTXOSetSuccessfully, err := flow.syncPruningPointUTXOSet(flow.Domain().StagingConsensus(), proofPruningPoint)
if err != nil { if err != nil {
return err return err
} }
@ -127,36 +150,40 @@ func (flow *handleRelayInvsFlow) downloadHeadersAndPruningUTXOSet(highHash *exte
return nil return nil
} }
func (flow *handleRelayInvsFlow) syncPruningPointsAndPruningPointAnticone() (*externalapi.DomainHash, error) { func (flow *handleRelayInvsFlow) syncPruningPointsAndPruningPointAnticone(proofPruningPoint *externalapi.DomainHash) error {
log.Infof("Downloading the past pruning points and the pruning point anticone from %s", flow.peer) log.Infof("Downloading the past pruning points and the pruning point anticone from %s", flow.peer)
err := flow.outgoingRoute.Enqueue(appmessage.NewMsgRequestPruningPointAndItsAnticone()) err := flow.outgoingRoute.Enqueue(appmessage.NewMsgRequestPruningPointAndItsAnticone())
if err != nil { if err != nil {
return nil, err return err
} }
err = flow.validateAndInsertPruningPoints() err = flow.validateAndInsertPruningPoints(proofPruningPoint)
if err != nil { if err != nil {
return nil, err return err
} }
pruningPoint, done, err := flow.receiveBlockWithTrustedData() pruningPointWithMetaData, done, err := flow.receiveBlockWithTrustedData()
if err != nil { if err != nil {
return nil, err return err
} }
if done { if done {
return nil, protocolerrors.Errorf(true, "got `done` message before receiving the pruning point") return protocolerrors.Errorf(true, "got `done` message before receiving the pruning point")
} }
err = flow.processBlockWithTrustedData(flow.Domain().StagingConsensus(), pruningPoint) 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)
if err != nil { if err != nil {
return nil, err return err
} }
for { for {
blockWithTrustedData, done, err := flow.receiveBlockWithTrustedData() blockWithTrustedData, done, err := flow.receiveBlockWithTrustedData()
if err != nil { if err != nil {
return nil, err return err
} }
if done { if done {
@ -165,12 +192,12 @@ func (flow *handleRelayInvsFlow) syncPruningPointsAndPruningPointAnticone() (*ex
err = flow.processBlockWithTrustedData(flow.Domain().StagingConsensus(), blockWithTrustedData) err = flow.processBlockWithTrustedData(flow.Domain().StagingConsensus(), blockWithTrustedData)
if err != nil { if err != nil {
return nil, err return err
} }
} }
log.Infof("Finished downloading pruning point and its anticone from %s", flow.peer) log.Infof("Finished downloading pruning point and its anticone from %s", flow.peer)
return pruningPoint.Block.Header.BlockHash(), nil return nil
} }
func (flow *handleRelayInvsFlow) processBlockWithTrustedData( func (flow *handleRelayInvsFlow) processBlockWithTrustedData(
@ -217,7 +244,16 @@ func (flow *handleRelayInvsFlow) receivePruningPoints() (*appmessage.MsgPruningP
return msgPruningPoints, nil return msgPruningPoints, nil
} }
func (flow *handleRelayInvsFlow) validateAndInsertPruningPoints() error { func (flow *handleRelayInvsFlow) validateAndInsertPruningPoints(proofPruningPoint *externalapi.DomainHash) error {
currentPruningPoint, err := flow.Domain().Consensus().PruningPoint()
if err != nil {
return err
}
if currentPruningPoint.Equal(proofPruningPoint) {
return protocolerrors.Errorf(true, "the proposed pruning point is the same as the current pruning point")
}
pruningPoints, err := flow.receivePruningPoints() pruningPoints, err := flow.receivePruningPoints()
if err != nil { if err != nil {
return err return err
@ -238,6 +274,12 @@ func (flow *handleRelayInvsFlow) validateAndInsertPruningPoints() error {
return protocolerrors.Errorf(false, "pruning points are violating finality") 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) err = flow.Domain().StagingConsensus().ImportPruningPoints(headers)
if err != nil { if err != nil {
return err return err

View File

@ -28,27 +28,28 @@ type consensus struct {
pastMedianTimeManager model.PastMedianTimeManager pastMedianTimeManager model.PastMedianTimeManager
blockValidator model.BlockValidator blockValidator model.BlockValidator
coinbaseManager model.CoinbaseManager coinbaseManager model.CoinbaseManager
dagTopologyManager model.DAGTopologyManager dagTopologyManagers []model.DAGTopologyManager
dagTraversalManager model.DAGTraversalManager dagTraversalManager model.DAGTraversalManager
difficultyManager model.DifficultyManager difficultyManager model.DifficultyManager
ghostdagManager model.GHOSTDAGManager ghostdagManagers []model.GHOSTDAGManager
headerTipsManager model.HeadersSelectedTipManager headerTipsManager model.HeadersSelectedTipManager
mergeDepthManager model.MergeDepthManager mergeDepthManager model.MergeDepthManager
pruningManager model.PruningManager pruningManager model.PruningManager
reachabilityManager model.ReachabilityManager reachabilityManagers []model.ReachabilityManager
finalityManager model.FinalityManager finalityManager model.FinalityManager
pruningProofManager model.PruningProofManager
acceptanceDataStore model.AcceptanceDataStore acceptanceDataStore model.AcceptanceDataStore
blockStore model.BlockStore blockStore model.BlockStore
blockHeaderStore model.BlockHeaderStore blockHeaderStore model.BlockHeaderStore
pruningStore model.PruningStore pruningStore model.PruningStore
ghostdagDataStore model.GHOSTDAGDataStore ghostdagDataStores []model.GHOSTDAGDataStore
blockRelationStore model.BlockRelationStore blockRelationStores []model.BlockRelationStore
blockStatusStore model.BlockStatusStore blockStatusStore model.BlockStatusStore
consensusStateStore model.ConsensusStateStore consensusStateStore model.ConsensusStateStore
headersSelectedTipStore model.HeaderSelectedTipStore headersSelectedTipStore model.HeaderSelectedTipStore
multisetStore model.MultisetStore multisetStore model.MultisetStore
reachabilityDataStore model.ReachabilityDataStore reachabilityDataStores []model.ReachabilityDataStore
utxoDiffStore model.UTXODiffStore utxoDiffStore model.UTXODiffStore
finalityStore model.FinalityStore finalityStore model.FinalityStore
headersSelectedChainStore model.HeadersSelectedChainStore headersSelectedChainStore model.HeadersSelectedChainStore
@ -81,25 +82,31 @@ func (s *consensus) Init(skipAddingGenesis bool) error {
// on a node with pruned header all blocks without known parents points to it. // on a node with pruned header all blocks without known parents points to it.
if !exists { if !exists {
s.blockStatusStore.Stage(stagingArea, model.VirtualGenesisBlockHash, externalapi.StatusUTXOValid) s.blockStatusStore.Stage(stagingArea, model.VirtualGenesisBlockHash, externalapi.StatusUTXOValid)
err = s.reachabilityManager.Init(stagingArea) for _, reachabilityManager := range s.reachabilityManagers {
if err != nil { err = reachabilityManager.Init(stagingArea)
return err if err != nil {
return err
}
} }
err = s.dagTopologyManager.SetParents(stagingArea, model.VirtualGenesisBlockHash, nil) for _, dagTopologyManager := range s.dagTopologyManagers {
if err != nil { err = dagTopologyManager.SetParents(stagingArea, model.VirtualGenesisBlockHash, nil)
return err if err != nil {
return err
}
} }
s.consensusStateStore.StageTips(stagingArea, []*externalapi.DomainHash{model.VirtualGenesisBlockHash}) s.consensusStateStore.StageTips(stagingArea, []*externalapi.DomainHash{model.VirtualGenesisBlockHash})
s.ghostdagDataStore.Stage(stagingArea, model.VirtualGenesisBlockHash, externalapi.NewBlockGHOSTDAGData( for _, ghostdagDataStore := range s.ghostdagDataStores {
0, ghostdagDataStore.Stage(stagingArea, model.VirtualGenesisBlockHash, externalapi.NewBlockGHOSTDAGData(
big.NewInt(0), 0,
nil, big.NewInt(0),
nil, nil,
nil, nil,
nil, nil,
), false) nil,
), false)
}
err = staging.CommitAllChanges(s.databaseContext, stagingArea) err = staging.CommitAllChanges(s.databaseContext, stagingArea)
if err != nil { if err != nil {
@ -267,7 +274,7 @@ func (s *consensus) GetBlockInfo(blockHash *externalapi.DomainHash) (*externalap
return blockInfo, nil return blockInfo, nil
} }
ghostdagData, err := s.ghostdagDataStore.Get(s.databaseContext, stagingArea, blockHash, false) ghostdagData, err := s.ghostdagDataStores[0].Get(s.databaseContext, stagingArea, blockHash, false)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -287,12 +294,12 @@ func (s *consensus) GetBlockRelations(blockHash *externalapi.DomainHash) (
stagingArea := model.NewStagingArea() stagingArea := model.NewStagingArea()
blockRelation, err := s.blockRelationStore.BlockRelation(s.databaseContext, stagingArea, blockHash) blockRelation, err := s.blockRelationStores[0].BlockRelation(s.databaseContext, stagingArea, blockHash)
if err != nil { if err != nil {
return nil, nil, nil, err return nil, nil, nil, err
} }
blockGHOSTDAGData, err := s.ghostdagDataStore.Get(s.databaseContext, stagingArea, blockHash, false) blockGHOSTDAGData, err := s.ghostdagDataStores[0].Get(s.databaseContext, stagingArea, blockHash, false)
if err != nil { if err != nil {
return nil, nil, nil, err return nil, nil, nil, err
} }
@ -382,7 +389,7 @@ func (s *consensus) GetVirtualUTXOs(expectedVirtualParents []*externalapi.Domain
stagingArea := model.NewStagingArea() stagingArea := model.NewStagingArea()
virtualParents, err := s.dagTopologyManager.Parents(stagingArea, model.VirtualBlockHash) virtualParents, err := s.dagTopologyManagers[0].Parents(stagingArea, model.VirtualBlockHash)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -465,7 +472,7 @@ func (s *consensus) GetVirtualSelectedParent() (*externalapi.DomainHash, error)
stagingArea := model.NewStagingArea() stagingArea := model.NewStagingArea()
virtualGHOSTDAGData, err := s.ghostdagDataStore.Get(s.databaseContext, stagingArea, model.VirtualBlockHash, false) virtualGHOSTDAGData, err := s.ghostdagDataStores[0].Get(s.databaseContext, stagingArea, model.VirtualBlockHash, false)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -487,7 +494,7 @@ func (s *consensus) GetVirtualInfo() (*externalapi.VirtualInfo, error) {
stagingArea := model.NewStagingArea() stagingArea := model.NewStagingArea()
blockRelations, err := s.blockRelationStore.BlockRelation(s.databaseContext, stagingArea, model.VirtualBlockHash) blockRelations, err := s.blockRelationStores[0].BlockRelation(s.databaseContext, stagingArea, model.VirtualBlockHash)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -499,7 +506,7 @@ func (s *consensus) GetVirtualInfo() (*externalapi.VirtualInfo, error) {
if err != nil { if err != nil {
return nil, err return nil, err
} }
virtualGHOSTDAGData, err := s.ghostdagDataStore.Get(s.databaseContext, stagingArea, model.VirtualBlockHash, false) virtualGHOSTDAGData, err := s.ghostdagDataStores[0].Get(s.databaseContext, stagingArea, model.VirtualBlockHash, false)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -664,7 +671,7 @@ func (s *consensus) IsInSelectedParentChainOf(blockHashA *externalapi.DomainHash
return false, err return false, err
} }
return s.dagTopologyManager.IsInSelectedParentChainOf(stagingArea, blockHashA, blockHashB) return s.dagTopologyManagers[0].IsInSelectedParentChainOf(stagingArea, blockHashA, blockHashB)
} }
func (s *consensus) GetHeadersSelectedTip() (*externalapi.DomainHash, error) { func (s *consensus) GetHeadersSelectedTip() (*externalapi.DomainHash, error) {
@ -687,7 +694,12 @@ func (s *consensus) Anticone(blockHash *externalapi.DomainHash) ([]*externalapi.
return nil, err return nil, err
} }
return s.dagTraversalManager.Anticone(stagingArea, blockHash) tips, err := s.consensusStateStore.Tips(stagingArea, s.databaseContext)
if err != nil {
return nil, err
}
return s.dagTraversalManager.AnticoneFromBlocks(stagingArea, tips, blockHash)
} }
func (s *consensus) EstimateNetworkHashesPerSecond(startHash *externalapi.DomainHash, windowSize int) (uint64, error) { func (s *consensus) EstimateNetworkHashesPerSecond(startHash *externalapi.DomainHash, windowSize int) (uint64, error) {
@ -727,18 +739,27 @@ func (s *consensus) BuildPruningPointProof() (*externalapi.PruningPointProof, er
s.lock.Lock() s.lock.Lock()
defer s.lock.Unlock() defer s.lock.Unlock()
// TODO: Implement this return s.pruningProofManager.BuildPruningPointProof(model.NewStagingArea())
return &externalapi.PruningPointProof{
Headers: [][]externalapi.BlockHeader{},
}, nil
} }
func (s *consensus) ValidatePruningPointProof(pruningPointProof *externalapi.PruningPointProof) error { func (s *consensus) ValidatePruningPointProof(pruningPointProof *externalapi.PruningPointProof) error {
s.lock.Lock() s.lock.Lock()
defer s.lock.Unlock() defer s.lock.Unlock()
// TODO: Implement this return s.pruningProofManager.ValidatePruningPointProof(pruningPointProof)
}
func (s *consensus) ApplyPruningPointProof(pruningPointProof *externalapi.PruningPointProof) error {
stagingArea := model.NewStagingArea()
err := s.pruningProofManager.ApplyPruningPointProof(stagingArea, pruningPointProof)
if err != nil {
return err
}
err = staging.CommitAllChanges(s.databaseContext, stagingArea)
if err != nil {
return err
}
return nil return nil
} }

View File

@ -7,7 +7,9 @@ import (
"github.com/kaspanet/kaspad/domain/consensus/model" "github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi" "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/kaspanet/kaspad/domain/consensus/utils/lrucachehashpairtoblockghostdagdatahashpair" "github.com/kaspanet/kaspad/domain/consensus/utils/lrucachehashpairtoblockghostdagdatahashpair"
"github.com/kaspanet/kaspad/infrastructure/db/database"
"github.com/kaspanet/kaspad/util/staging" "github.com/kaspanet/kaspad/util/staging"
"github.com/pkg/errors"
) )
var bucketName = []byte("daa-window") var bucketName = []byte("daa-window")
@ -37,6 +39,8 @@ func (daaws *daaWindowStore) Stage(stagingArea *model.StagingArea, blockHash *ex
} }
var errDAAWindowBlockNotFound = errors.Wrap(database.ErrNotFound, "DAA window block not found")
func (daaws *daaWindowStore) DAAWindowBlock(dbContext model.DBReader, stagingArea *model.StagingArea, blockHash *externalapi.DomainHash, index uint64) (*externalapi.BlockGHOSTDAGDataHashPair, error) { func (daaws *daaWindowStore) DAAWindowBlock(dbContext model.DBReader, stagingArea *model.StagingArea, blockHash *externalapi.DomainHash, index uint64) (*externalapi.BlockGHOSTDAGDataHashPair, error) {
stagingShard := daaws.stagingShard(stagingArea) stagingShard := daaws.stagingShard(stagingArea)
@ -46,10 +50,17 @@ func (daaws *daaWindowStore) DAAWindowBlock(dbContext model.DBReader, stagingAre
} }
if pair, ok := daaws.cache.Get(blockHash, index); ok { if pair, ok := daaws.cache.Get(blockHash, index); ok {
if pair == nil {
return nil, errDAAWindowBlockNotFound
}
return pair, nil return pair, nil
} }
pairBytes, err := dbContext.Get(daaws.key(dbKey)) pairBytes, err := dbContext.Get(daaws.key(dbKey))
if database.IsNotFoundError(err) {
daaws.cache.Add(blockHash, index, nil)
}
if err != nil { if err != nil {
return nil, err return nil, err
} }

View File

@ -2,7 +2,10 @@ package consensus
import ( import (
"github.com/kaspanet/kaspad/domain/consensus/datastructures/daawindowstore" "github.com/kaspanet/kaspad/domain/consensus/datastructures/daawindowstore"
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/processes/blockparentbuilder" "github.com/kaspanet/kaspad/domain/consensus/processes/blockparentbuilder"
"github.com/kaspanet/kaspad/domain/consensus/processes/pruningproofmanager"
"github.com/kaspanet/kaspad/domain/consensus/utils/constants"
"io/ioutil" "io/ioutil"
"os" "os"
"sync" "sync"
@ -16,7 +19,7 @@ import (
"github.com/kaspanet/kaspad/domain/consensus/datastructures/blockstatusstore" "github.com/kaspanet/kaspad/domain/consensus/datastructures/blockstatusstore"
"github.com/kaspanet/kaspad/domain/consensus/datastructures/blockstore" "github.com/kaspanet/kaspad/domain/consensus/datastructures/blockstore"
"github.com/kaspanet/kaspad/domain/consensus/datastructures/consensusstatestore" "github.com/kaspanet/kaspad/domain/consensus/datastructures/consensusstatestore"
daablocksstore "github.com/kaspanet/kaspad/domain/consensus/datastructures/daablocksstore" "github.com/kaspanet/kaspad/domain/consensus/datastructures/daablocksstore"
"github.com/kaspanet/kaspad/domain/consensus/datastructures/finalitystore" "github.com/kaspanet/kaspad/domain/consensus/datastructures/finalitystore"
"github.com/kaspanet/kaspad/domain/consensus/datastructures/ghostdagdatastore" "github.com/kaspanet/kaspad/domain/consensus/datastructures/ghostdagdatastore"
"github.com/kaspanet/kaspad/domain/consensus/datastructures/headersselectedchainstore" "github.com/kaspanet/kaspad/domain/consensus/datastructures/headersselectedchainstore"
@ -130,39 +133,31 @@ func (f *factory) NewConsensus(config *Config, db infrastructuredatabase.Databas
if err != nil { if err != nil {
return nil, err return nil, err
} }
blockRelationStore := blockrelationstore.New(prefixBucket, pruningWindowSizePlusFinalityDepthForCache, preallocateCaches)
blockStatusStore := blockstatusstore.New(prefixBucket, pruningWindowSizePlusFinalityDepthForCache, preallocateCaches) blockStatusStore := blockstatusstore.New(prefixBucket, pruningWindowSizePlusFinalityDepthForCache, preallocateCaches)
multisetStore := multisetstore.New(prefixBucket, 200, preallocateCaches) multisetStore := multisetstore.New(prefixBucket, 200, preallocateCaches)
pruningStore := pruningstore.New(prefixBucket, 2, preallocateCaches) pruningStore := pruningstore.New(prefixBucket, 2, preallocateCaches)
reachabilityDataStore := reachabilitydatastore.New(prefixBucket, pruningWindowSizePlusFinalityDepthForCache, preallocateCaches)
utxoDiffStore := utxodiffstore.New(prefixBucket, 200, preallocateCaches) utxoDiffStore := utxodiffstore.New(prefixBucket, 200, preallocateCaches)
consensusStateStore := consensusstatestore.New(prefixBucket, 10_000, preallocateCaches) consensusStateStore := consensusstatestore.New(prefixBucket, 10_000, preallocateCaches)
// Some tests artificially decrease the pruningWindowSize, thus making the GhostDagStore cache too small for a
// a single DifficultyAdjustmentWindow. To alleviate this problem we make sure that the cache size is at least
// dagParams.DifficultyAdjustmentWindowSize
ghostdagDataCacheSize := pruningWindowSizeForCaches
if ghostdagDataCacheSize < config.DifficultyAdjustmentWindowSize {
ghostdagDataCacheSize = config.DifficultyAdjustmentWindowSize
}
ghostdagDataStore := ghostdagdatastore.New(prefixBucket, ghostdagDataCacheSize, preallocateCaches)
headersSelectedTipStore := headersselectedtipstore.New(prefixBucket) headersSelectedTipStore := headersselectedtipstore.New(prefixBucket)
finalityStore := finalitystore.New(prefixBucket, 200, preallocateCaches) finalityStore := finalitystore.New(prefixBucket, 200, preallocateCaches)
headersSelectedChainStore := headersselectedchainstore.New(prefixBucket, pruningWindowSizeForCaches, preallocateCaches) headersSelectedChainStore := headersselectedchainstore.New(prefixBucket, pruningWindowSizeForCaches, preallocateCaches)
daaBlocksStore := daablocksstore.New(prefixBucket, pruningWindowSizeForCaches, int(config.FinalityDepth()), preallocateCaches) daaBlocksStore := daablocksstore.New(prefixBucket, pruningWindowSizeForCaches, int(config.FinalityDepth()), preallocateCaches)
blockRelationStores, reachabilityDataStores, ghostdagDataStores := dagStores(config, prefixBucket, pruningWindowSizePlusFinalityDepthForCache, pruningWindowSizeForCaches, preallocateCaches)
reachabilityManagers, dagTopologyManagers, ghostdagManagers, dagTraversalManagers := f.dagProcesses(config, dbManager, blockHeaderStore, daaWindowStore, blockRelationStores, reachabilityDataStores, ghostdagDataStores)
blockRelationStore := blockRelationStores[0]
reachabilityDataStore := reachabilityDataStores[0]
ghostdagDataStore := ghostdagDataStores[0]
reachabilityManager := reachabilityManagers[0]
dagTopologyManager := dagTopologyManagers[0]
ghostdagManager := ghostdagManagers[0]
dagTraversalManager := dagTraversalManagers[0]
// Processes // Processes
reachabilityManager := reachabilitymanager.New(
dbManager,
ghostdagDataStore,
reachabilityDataStore)
dagTopologyManager := dagtopologymanager.New(
dbManager,
reachabilityManager,
blockRelationStore,
ghostdagDataStore)
blockParentBuilder := blockparentbuilder.New( blockParentBuilder := blockparentbuilder.New(
dbManager, dbManager,
blockHeaderStore, blockHeaderStore,
@ -170,22 +165,6 @@ func (f *factory) NewConsensus(config *Config, db infrastructuredatabase.Databas
reachabilityDataStore, reachabilityDataStore,
pruningStore, pruningStore,
) )
ghostdagManager := f.ghostdagConstructor(
dbManager,
dagTopologyManager,
ghostdagDataStore,
blockHeaderStore,
config.K,
config.GenesisHash)
dagTraversalManager := dagtraversalmanager.New(
dbManager,
dagTopologyManager,
ghostdagDataStore,
reachabilityDataStore,
ghostdagManager,
consensusStateStore,
daaWindowStore,
config.GenesisHash)
pastMedianTimeManager := f.pastMedianTimeConsructor( pastMedianTimeManager := f.pastMedianTimeConsructor(
config.TimestampDeviationTolerance, config.TimestampDeviationTolerance,
dbManager, dbManager,
@ -318,19 +297,19 @@ func (f *factory) NewConsensus(config *Config, db infrastructuredatabase.Databas
difficultyManager, difficultyManager,
pastMedianTimeManager, pastMedianTimeManager,
transactionValidator, transactionValidator,
ghostdagManager, ghostdagManagers,
dagTopologyManager, dagTopologyManagers,
dagTraversalManager, dagTraversalManager,
coinbaseManager, coinbaseManager,
mergeDepthManager, mergeDepthManager,
reachabilityManager, reachabilityManagers,
finalityManager, finalityManager,
blockParentBuilder, blockParentBuilder,
pruningManager, pruningManager,
pruningStore, pruningStore,
blockStore, blockStore,
ghostdagDataStore, ghostdagDataStores,
blockHeaderStore, blockHeaderStore,
blockStatusStore, blockStatusStore,
reachabilityDataStore, reachabilityDataStore,
@ -408,6 +387,25 @@ func (f *factory) NewConsensus(config *Config, db infrastructuredatabase.Databas
daaBlocksStore, daaBlocksStore,
daaWindowStore) daaWindowStore)
pruningProofManager := pruningproofmanager.New(
dbManager,
dagTopologyManagers,
ghostdagManagers,
reachabilityManagers,
dagTraversalManagers,
ghostdagDataStores,
pruningStore,
blockHeaderStore,
blockStatusStore,
finalityStore,
consensusStateStore,
genesisHash,
config.K,
config.PruningProofM,
)
c := &consensus{ c := &consensus{
lock: &sync.Mutex{}, lock: &sync.Mutex{},
databaseContext: dbManager, databaseContext: dbManager,
@ -423,27 +421,28 @@ func (f *factory) NewConsensus(config *Config, db infrastructuredatabase.Databas
pastMedianTimeManager: pastMedianTimeManager, pastMedianTimeManager: pastMedianTimeManager,
blockValidator: blockValidator, blockValidator: blockValidator,
coinbaseManager: coinbaseManager, coinbaseManager: coinbaseManager,
dagTopologyManager: dagTopologyManager, dagTopologyManagers: dagTopologyManagers,
dagTraversalManager: dagTraversalManager, dagTraversalManager: dagTraversalManager,
difficultyManager: difficultyManager, difficultyManager: difficultyManager,
ghostdagManager: ghostdagManager, ghostdagManagers: ghostdagManagers,
headerTipsManager: headerTipsManager, headerTipsManager: headerTipsManager,
mergeDepthManager: mergeDepthManager, mergeDepthManager: mergeDepthManager,
pruningManager: pruningManager, pruningManager: pruningManager,
reachabilityManager: reachabilityManager, reachabilityManagers: reachabilityManagers,
finalityManager: finalityManager, finalityManager: finalityManager,
pruningProofManager: pruningProofManager,
acceptanceDataStore: acceptanceDataStore, acceptanceDataStore: acceptanceDataStore,
blockStore: blockStore, blockStore: blockStore,
blockHeaderStore: blockHeaderStore, blockHeaderStore: blockHeaderStore,
pruningStore: pruningStore, pruningStore: pruningStore,
ghostdagDataStore: ghostdagDataStore, ghostdagDataStores: ghostdagDataStores,
blockStatusStore: blockStatusStore, blockStatusStore: blockStatusStore,
blockRelationStore: blockRelationStore, blockRelationStores: blockRelationStores,
consensusStateStore: consensusStateStore, consensusStateStore: consensusStateStore,
headersSelectedTipStore: headersSelectedTipStore, headersSelectedTipStore: headersSelectedTipStore,
multisetStore: multisetStore, multisetStore: multisetStore,
reachabilityDataStore: reachabilityDataStore, reachabilityDataStores: reachabilityDataStores,
utxoDiffStore: utxoDiffStore, utxoDiffStore: utxoDiffStore,
finalityStore: finalityStore, finalityStore: finalityStore,
headersSelectedChainStore: headersSelectedChainStore, headersSelectedChainStore: headersSelectedChainStore,
@ -510,7 +509,7 @@ func (f *factory) NewTestConsensus(config *Config, testName string) (
database: db, database: db,
testConsensusStateManager: testConsensusStateManager, testConsensusStateManager: testConsensusStateManager,
testReachabilityManager: reachabilitymanager.NewTestReachabilityManager(consensusAsImplementation. testReachabilityManager: reachabilitymanager.NewTestReachabilityManager(consensusAsImplementation.
reachabilityManager), reachabilityManagers[0]),
testTransactionValidator: testTransactionValidator, testTransactionValidator: testTransactionValidator,
} }
tstConsensus.testBlockBuilder = blockbuilder.NewTestBlockBuilder(consensusAsImplementation.blockBuilder, tstConsensus) tstConsensus.testBlockBuilder = blockbuilder.NewTestBlockBuilder(consensusAsImplementation.blockBuilder, tstConsensus)
@ -549,3 +548,84 @@ func (f *factory) SetTestLevelDBCacheSize(cacheSizeMiB int) {
func (f *factory) SetTestPreAllocateCache(preallocateCaches bool) { func (f *factory) SetTestPreAllocateCache(preallocateCaches bool) {
f.preallocateCaches = &preallocateCaches f.preallocateCaches = &preallocateCaches
} }
func dagStores(config *Config,
prefixBucket model.DBBucket,
pruningWindowSizePlusFinalityDepthForCache, pruningWindowSizeForCaches int,
preallocateCaches bool) ([]model.BlockRelationStore, []model.ReachabilityDataStore, []model.GHOSTDAGDataStore) {
blockRelationStores := make([]model.BlockRelationStore, constants.MaxBlockLevel+1)
reachabilityDataStores := make([]model.ReachabilityDataStore, constants.MaxBlockLevel+1)
ghostdagDataStores := make([]model.GHOSTDAGDataStore, constants.MaxBlockLevel+1)
ghostdagDataCacheSize := pruningWindowSizeForCaches
if ghostdagDataCacheSize < config.DifficultyAdjustmentWindowSize {
ghostdagDataCacheSize = config.DifficultyAdjustmentWindowSize
}
for i := 0; i <= constants.MaxBlockLevel; i++ {
prefixBucket := prefixBucket.Bucket([]byte{byte(i)})
if i == 0 {
blockRelationStores[i] = blockrelationstore.New(prefixBucket, pruningWindowSizePlusFinalityDepthForCache, preallocateCaches)
reachabilityDataStores[i] = reachabilitydatastore.New(prefixBucket, pruningWindowSizePlusFinalityDepthForCache, preallocateCaches)
ghostdagDataStores[i] = ghostdagdatastore.New(prefixBucket, ghostdagDataCacheSize, preallocateCaches)
} else {
blockRelationStores[i] = blockrelationstore.New(prefixBucket, 200, false)
reachabilityDataStores[i] = reachabilitydatastore.New(prefixBucket, 200, false)
ghostdagDataStores[i] = ghostdagdatastore.New(prefixBucket, 200, false)
}
}
return blockRelationStores, reachabilityDataStores, ghostdagDataStores
}
func (f *factory) dagProcesses(config *Config,
dbManager model.DBManager,
blockHeaderStore model.BlockHeaderStore,
daaWindowStore model.BlocksWithTrustedDataDAAWindowStore,
blockRelationStores []model.BlockRelationStore,
reachabilityDataStores []model.ReachabilityDataStore,
ghostdagDataStores []model.GHOSTDAGDataStore) (
[]model.ReachabilityManager,
[]model.DAGTopologyManager,
[]model.GHOSTDAGManager,
[]model.DAGTraversalManager,
) {
reachabilityManagers := make([]model.ReachabilityManager, constants.MaxBlockLevel+1)
dagTopologyManagers := make([]model.DAGTopologyManager, constants.MaxBlockLevel+1)
ghostdagManagers := make([]model.GHOSTDAGManager, constants.MaxBlockLevel+1)
dagTraversalManagers := make([]model.DAGTraversalManager, constants.MaxBlockLevel+1)
for i := 0; i <= constants.MaxBlockLevel; i++ {
reachabilityManagers[i] = reachabilitymanager.New(
dbManager,
ghostdagDataStores[i],
reachabilityDataStores[i])
dagTopologyManagers[i] = dagtopologymanager.New(
dbManager,
reachabilityManagers[i],
blockRelationStores[i],
ghostdagDataStores[i])
ghostdagManagers[i] = f.ghostdagConstructor(
dbManager,
dagTopologyManagers[i],
ghostdagDataStores[i],
blockHeaderStore,
config.K,
config.GenesisHash)
dagTraversalManagers[i] = dagtraversalmanager.New(
dbManager,
dagTopologyManagers[i],
ghostdagDataStores[i],
reachabilityDataStores[i],
ghostdagManagers[i],
daaWindowStore,
config.GenesisHash)
}
return reachabilityManagers, dagTopologyManagers, ghostdagManagers, dagTraversalManagers
}

View File

@ -10,6 +10,7 @@ type Consensus interface {
ImportPruningPoints(pruningPoints []BlockHeader) error ImportPruningPoints(pruningPoints []BlockHeader) error
BuildPruningPointProof() (*PruningPointProof, error) BuildPruningPointProof() (*PruningPointProof, error)
ValidatePruningPointProof(pruningPointProof *PruningPointProof) error ValidatePruningPointProof(pruningPointProof *PruningPointProof) error
ApplyPruningPointProof(pruningPointProof *PruningPointProof) error
GetBlock(blockHash *DomainHash) (*DomainBlock, error) GetBlock(blockHash *DomainHash) (*DomainBlock, error)
GetBlockEvenIfHeaderOnly(blockHash *DomainHash) (*DomainBlock, error) GetBlockEvenIfHeaderOnly(blockHash *DomainHash) (*DomainBlock, error)

View File

@ -5,13 +5,12 @@ import "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
// DAGTraversalManager exposes methods for traversing blocks // DAGTraversalManager exposes methods for traversing blocks
// in the DAG // in the DAG
type DAGTraversalManager interface { type DAGTraversalManager interface {
BlockAtDepth(stagingArea *StagingArea, highHash *externalapi.DomainHash, depth uint64) (*externalapi.DomainHash, error)
LowestChainBlockAboveOrEqualToBlueScore(stagingArea *StagingArea, highHash *externalapi.DomainHash, blueScore uint64) (*externalapi.DomainHash, error) LowestChainBlockAboveOrEqualToBlueScore(stagingArea *StagingArea, highHash *externalapi.DomainHash, blueScore uint64) (*externalapi.DomainHash, error)
// SelectedChildIterator should return a BlockIterator that iterates // SelectedChildIterator should return a BlockIterator that iterates
// from lowHash (exclusive) to highHash (inclusive) over highHash's selected parent chain // from lowHash (exclusive) to highHash (inclusive) over highHash's selected parent chain
SelectedChildIterator(stagingArea *StagingArea, highHash, lowHash *externalapi.DomainHash) (BlockIterator, error) SelectedChildIterator(stagingArea *StagingArea, highHash, lowHash *externalapi.DomainHash) (BlockIterator, error)
SelectedChild(stagingArea *StagingArea, highHash, lowHash *externalapi.DomainHash) (*externalapi.DomainHash, error) SelectedChild(stagingArea *StagingArea, highHash, lowHash *externalapi.DomainHash) (*externalapi.DomainHash, error)
Anticone(stagingArea *StagingArea, blockHash *externalapi.DomainHash) ([]*externalapi.DomainHash, error) AnticoneFromBlocks(stagingArea *StagingArea, tips []*externalapi.DomainHash, blockHash *externalapi.DomainHash) ([]*externalapi.DomainHash, error)
AnticoneFromVirtualPOV(stagingArea *StagingArea, blockHash *externalapi.DomainHash) ([]*externalapi.DomainHash, error) AnticoneFromVirtualPOV(stagingArea *StagingArea, blockHash *externalapi.DomainHash) ([]*externalapi.DomainHash, error)
BlockWindow(stagingArea *StagingArea, highHash *externalapi.DomainHash, windowSize int) ([]*externalapi.DomainHash, error) BlockWindow(stagingArea *StagingArea, highHash *externalapi.DomainHash, windowSize int) ([]*externalapi.DomainHash, error)
BlockWindowWithGHOSTDAGData(stagingArea *StagingArea, highHash *externalapi.DomainHash, windowSize int) ([]*externalapi.BlockGHOSTDAGDataHashPair, error) BlockWindowWithGHOSTDAGData(stagingArea *StagingArea, highHash *externalapi.DomainHash, windowSize int) ([]*externalapi.BlockGHOSTDAGDataHashPair, error)

View File

@ -0,0 +1,10 @@
package model
import "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
// PruningProofManager builds, validates and applies pruning proofs.
type PruningProofManager interface {
BuildPruningPointProof(stagingArea *StagingArea) (*externalapi.PruningPointProof, error)
ValidatePruningPointProof(pruningPointProof *externalapi.PruningPointProof) error
ApplyPruningPointProof(stagingArea *StagingArea, pruningPointProof *externalapi.PruningPointProof) error
}

View File

@ -59,6 +59,7 @@ type TestConsensus interface {
BlockStore() model.BlockStore BlockStore() model.BlockStore
ConsensusStateStore() model.ConsensusStateStore ConsensusStateStore() model.ConsensusStateStore
GHOSTDAGDataStore() model.GHOSTDAGDataStore GHOSTDAGDataStore() model.GHOSTDAGDataStore
GHOSTDAGDataStores() []model.GHOSTDAGDataStore
HeaderTipsStore() model.HeaderSelectedTipStore HeaderTipsStore() model.HeaderSelectedTipStore
MultisetStore() model.MultisetStore MultisetStore() model.MultisetStore
PruningStore() model.PruningStore PruningStore() model.PruningStore

View File

@ -93,15 +93,12 @@ func (bpb *blockParentBuilder) BuildParents(stagingArea *model.StagingArea,
// all the block levels they occupy // all the block levels they occupy
for _, directParentHeader := range directParentHeaders { for _, directParentHeader := range directParentHeaders {
directParentHash := consensushashing.HeaderHash(directParentHeader) directParentHash := consensushashing.HeaderHash(directParentHeader)
proofOfWorkValue := pow.CalculateProofOfWorkValue(directParentHeader.ToMutable()) blockLevel := pow.BlockLevel(directParentHeader)
for blockLevel := 0; ; blockLevel++ { for i := 0; i <= blockLevel; i++ {
if _, exists := candidatesByLevelToReferenceBlocksMap[blockLevel]; !exists { if _, exists := candidatesByLevelToReferenceBlocksMap[i]; !exists {
candidatesByLevelToReferenceBlocksMap[blockLevel] = make(map[externalapi.DomainHash][]*externalapi.DomainHash) candidatesByLevelToReferenceBlocksMap[i] = make(map[externalapi.DomainHash][]*externalapi.DomainHash)
}
candidatesByLevelToReferenceBlocksMap[blockLevel][*directParentHash] = []*externalapi.DomainHash{directParentHash}
if proofOfWorkValue.Bit(blockLevel+1) != 0 {
break
} }
candidatesByLevelToReferenceBlocksMap[i][*directParentHash] = []*externalapi.DomainHash{directParentHash}
} }
} }

View File

@ -36,24 +36,64 @@ func addBlock(tc testapi.TestConsensus, parentHashes []*externalapi.DomainHash,
func TestValidateAndInsertImportedPruningPoint(t *testing.T) { func TestValidateAndInsertImportedPruningPoint(t *testing.T) {
testutils.ForAllNets(t, true, func(t *testing.T, consensusConfig *consensus.Config) { testutils.ForAllNets(t, true, func(t *testing.T, consensusConfig *consensus.Config) {
syncConsensuses := func(tcSyncer, tcSyncee testapi.TestConsensus) { factory := consensus.NewFactory()
// This is done to reduce the pruning depth to 6 blocks
finalityDepth := 5
consensusConfig.FinalityDuration = time.Duration(finalityDepth) * consensusConfig.TargetTimePerBlock
consensusConfig.K = 0
consensusConfig.PruningProofM = 1
syncConsensuses := func(tcSyncerRef, tcSynceeRef *testapi.TestConsensus) {
tcSyncer, tcSyncee := *tcSyncerRef, *tcSynceeRef
pruningPointProof, err := tcSyncer.BuildPruningPointProof()
if err != nil {
t.Fatalf("BuildPruningPointProof: %+v", err)
}
err = tcSyncee.ValidatePruningPointProof(pruningPointProof)
if err != nil {
t.Fatalf("ValidatePruningPointProof: %+v", err)
}
stagingConfig := *consensusConfig
stagingConfig.SkipAddingGenesis = true
synceeStaging, _, err := factory.NewTestConsensus(&stagingConfig, "TestValidateAndInsertPruningPointSyncerStaging")
if err != nil {
t.Fatalf("Error setting up synceeStaging: %+v", err)
}
err = synceeStaging.ApplyPruningPointProof(pruningPointProof)
if err != nil {
t.Fatalf("ApplyPruningPointProof: %+v", err)
}
pruningPointHeaders, err := tcSyncer.PruningPointHeaders() pruningPointHeaders, err := tcSyncer.PruningPointHeaders()
if err != nil { if err != nil {
t.Fatalf("PruningPointHeaders: %+v", err) t.Fatalf("PruningPointHeaders: %+v", err)
} }
err = tcSyncee.ImportPruningPoints(pruningPointHeaders) arePruningPointsViolatingFinality, err := tcSyncee.ArePruningPointsViolatingFinality(pruningPointHeaders)
if err != nil {
t.Fatalf("ArePruningPointsViolatingFinality: %+v", err)
}
if arePruningPointsViolatingFinality {
t.Fatalf("unexpected finality violation")
}
err = synceeStaging.ImportPruningPoints(pruningPointHeaders)
if err != nil { if err != nil {
t.Fatalf("PruningPointHeaders: %+v", err) t.Fatalf("PruningPointHeaders: %+v", err)
} }
pointAndItsAnticoneWithTrustedData, err := tcSyncer.PruningPointAndItsAnticoneWithTrustedData() pruningPointAndItsAnticoneWithTrustedData, err := tcSyncer.PruningPointAndItsAnticoneWithTrustedData()
if err != nil { if err != nil {
t.Fatalf("PruningPointAndItsAnticoneWithTrustedData: %+v", err) t.Fatalf("PruningPointAndItsAnticoneWithTrustedData: %+v", err)
} }
for _, blockWithTrustedData := range pointAndItsAnticoneWithTrustedData { for _, blockWithTrustedData := range pruningPointAndItsAnticoneWithTrustedData {
_, err := tcSyncee.ValidateAndInsertBlockWithTrustedData(blockWithTrustedData, false) _, err := synceeStaging.ValidateAndInsertBlockWithTrustedData(blockWithTrustedData, false)
if err != nil { if err != nil {
t.Fatalf("ValidateAndInsertBlockWithTrustedData: %+v", err) t.Fatalf("ValidateAndInsertBlockWithTrustedData: %+v", err)
} }
@ -75,7 +115,7 @@ func TestValidateAndInsertImportedPruningPoint(t *testing.T) {
} }
for i, blocksHash := range missingHeaderHashes { for i, blocksHash := range missingHeaderHashes {
blockInfo, err := tcSyncee.GetBlockInfo(blocksHash) blockInfo, err := synceeStaging.GetBlockInfo(blocksHash)
if err != nil { if err != nil {
t.Fatalf("GetBlockInfo: %+v", err) t.Fatalf("GetBlockInfo: %+v", err)
} }
@ -89,7 +129,7 @@ func TestValidateAndInsertImportedPruningPoint(t *testing.T) {
t.Fatalf("GetBlockHeader: %+v", err) t.Fatalf("GetBlockHeader: %+v", err)
} }
_, err = tcSyncee.ValidateAndInsertBlock(&externalapi.DomainBlock{Header: header}, false) _, err = synceeStaging.ValidateAndInsertBlock(&externalapi.DomainBlock{Header: header}, false)
if err != nil { if err != nil {
t.Fatalf("ValidateAndInsertBlock %d: %+v", i, err) t.Fatalf("ValidateAndInsertBlock %d: %+v", i, err)
} }
@ -99,7 +139,7 @@ func TestValidateAndInsertImportedPruningPoint(t *testing.T) {
if err != nil { if err != nil {
t.Fatalf("GetPruningPointUTXOs: %+v", err) t.Fatalf("GetPruningPointUTXOs: %+v", err)
} }
err = tcSyncee.AppendImportedPruningPointUTXOs(pruningPointUTXOs) err = synceeStaging.AppendImportedPruningPointUTXOs(pruningPointUTXOs)
if err != nil { if err != nil {
t.Fatalf("AppendImportedPruningPointUTXOs: %+v", err) t.Fatalf("AppendImportedPruningPointUTXOs: %+v", err)
} }
@ -110,37 +150,37 @@ func TestValidateAndInsertImportedPruningPoint(t *testing.T) {
} }
// Check that ValidateAndInsertImportedPruningPoint fails for invalid pruning point // Check that ValidateAndInsertImportedPruningPoint fails for invalid pruning point
err = tcSyncee.ValidateAndInsertImportedPruningPoint(virtualSelectedParent) err = synceeStaging.ValidateAndInsertImportedPruningPoint(virtualSelectedParent)
if !errors.Is(err, ruleerrors.ErrUnexpectedPruningPoint) { if !errors.Is(err, ruleerrors.ErrUnexpectedPruningPoint) {
t.Fatalf("Unexpected error: %+v", err) t.Fatalf("Unexpected error: %+v", err)
} }
err = tcSyncee.ClearImportedPruningPointData() err = synceeStaging.ClearImportedPruningPointData()
if err != nil { if err != nil {
t.Fatalf("ClearImportedPruningPointData: %+v", err) t.Fatalf("ClearImportedPruningPointData: %+v", err)
} }
err = tcSyncee.AppendImportedPruningPointUTXOs(makeFakeUTXOs()) err = synceeStaging.AppendImportedPruningPointUTXOs(makeFakeUTXOs())
if err != nil { if err != nil {
t.Fatalf("AppendImportedPruningPointUTXOs: %+v", err) t.Fatalf("AppendImportedPruningPointUTXOs: %+v", err)
} }
// Check that ValidateAndInsertImportedPruningPoint fails if the UTXO commitment doesn't fit the provided UTXO set. // Check that ValidateAndInsertImportedPruningPoint fails if the UTXO commitment doesn't fit the provided UTXO set.
err = tcSyncee.ValidateAndInsertImportedPruningPoint(pruningPoint) err = synceeStaging.ValidateAndInsertImportedPruningPoint(pruningPoint)
if !errors.Is(err, ruleerrors.ErrBadPruningPointUTXOSet) { if !errors.Is(err, ruleerrors.ErrBadPruningPointUTXOSet) {
t.Fatalf("Unexpected error: %+v", err) t.Fatalf("Unexpected error: %+v", err)
} }
err = tcSyncee.ClearImportedPruningPointData() err = synceeStaging.ClearImportedPruningPointData()
if err != nil { if err != nil {
t.Fatalf("ClearImportedPruningPointData: %+v", err) t.Fatalf("ClearImportedPruningPointData: %+v", err)
} }
err = tcSyncee.AppendImportedPruningPointUTXOs(pruningPointUTXOs) err = synceeStaging.AppendImportedPruningPointUTXOs(pruningPointUTXOs)
if err != nil { if err != nil {
t.Fatalf("AppendImportedPruningPointUTXOs: %+v", err) t.Fatalf("AppendImportedPruningPointUTXOs: %+v", err)
} }
// Check that ValidateAndInsertImportedPruningPoint works given the right arguments. // Check that ValidateAndInsertImportedPruningPoint works given the right arguments.
err = tcSyncee.ValidateAndInsertImportedPruningPoint(pruningPoint) err = synceeStaging.ValidateAndInsertImportedPruningPoint(pruningPoint)
if err != nil { if err != nil {
t.Fatalf("ValidateAndInsertImportedPruningPoint: %+v", err) t.Fatalf("ValidateAndInsertImportedPruningPoint: %+v", err)
} }
@ -153,18 +193,18 @@ func TestValidateAndInsertImportedPruningPoint(t *testing.T) {
} }
// Check that we can build a block just after importing the pruning point. // Check that we can build a block just after importing the pruning point.
_, err = tcSyncee.BuildBlock(emptyCoinbase, nil) _, err = synceeStaging.BuildBlock(emptyCoinbase, nil)
if err != nil { if err != nil {
t.Fatalf("BuildBlock: %+v", err) t.Fatalf("BuildBlock: %+v", err)
} }
// Sync block bodies // Sync block bodies
headersSelectedTip, err := tcSyncee.GetHeadersSelectedTip() headersSelectedTip, err := synceeStaging.GetHeadersSelectedTip()
if err != nil { if err != nil {
t.Fatalf("GetHeadersSelectedTip: %+v", err) t.Fatalf("GetHeadersSelectedTip: %+v", err)
} }
missingBlockHashes, err := tcSyncee.GetMissingBlockBodyHashes(headersSelectedTip) missingBlockHashes, err := synceeStaging.GetMissingBlockBodyHashes(headersSelectedTip)
if err != nil { if err != nil {
t.Fatalf("GetMissingBlockBodyHashes: %+v", err) t.Fatalf("GetMissingBlockBodyHashes: %+v", err)
} }
@ -175,13 +215,13 @@ func TestValidateAndInsertImportedPruningPoint(t *testing.T) {
t.Fatalf("GetBlock: %+v", err) t.Fatalf("GetBlock: %+v", err)
} }
_, err = tcSyncee.ValidateAndInsertBlock(block, true) _, err = synceeStaging.ValidateAndInsertBlock(block, true)
if err != nil { if err != nil {
t.Fatalf("ValidateAndInsertBlock: %+v", err) t.Fatalf("ValidateAndInsertBlock: %+v", err)
} }
} }
synceeTips, err := tcSyncee.Tips() synceeTips, err := synceeStaging.Tips()
if err != nil { if err != nil {
t.Fatalf("Tips: %+v", err) t.Fatalf("Tips: %+v", err)
} }
@ -201,12 +241,12 @@ func TestValidateAndInsertImportedPruningPoint(t *testing.T) {
t.Fatalf("GetBlock: %+v", err) t.Fatalf("GetBlock: %+v", err)
} }
_, err = tcSyncee.ValidateAndInsertBlock(tip, true) _, err = synceeStaging.ValidateAndInsertBlock(tip, true)
if err != nil { if err != nil {
t.Fatalf("ValidateAndInsertBlock: %+v", err) t.Fatalf("ValidateAndInsertBlock: %+v", err)
} }
blockInfo, err := tcSyncee.GetBlockInfo(tipHash) blockInfo, err := synceeStaging.GetBlockInfo(tipHash)
if err != nil { if err != nil {
t.Fatalf("GetBlockInfo: %+v", err) t.Fatalf("GetBlockInfo: %+v", err)
} }
@ -215,7 +255,7 @@ func TestValidateAndInsertImportedPruningPoint(t *testing.T) {
t.Fatalf("Tip didn't pass UTXO verification") t.Fatalf("Tip didn't pass UTXO verification")
} }
synceePruningPoint, err := tcSyncee.PruningPoint() synceePruningPoint, err := synceeStaging.PruningPoint()
if err != nil { if err != nil {
t.Fatalf("PruningPoint: %+v", err) t.Fatalf("PruningPoint: %+v", err)
} }
@ -223,284 +263,88 @@ func TestValidateAndInsertImportedPruningPoint(t *testing.T) {
if !synceePruningPoint.Equal(pruningPoint) { if !synceePruningPoint.Equal(pruningPoint) {
t.Fatalf("The syncee pruning point has not changed as exepcted") t.Fatalf("The syncee pruning point has not changed as exepcted")
} }
*tcSynceeRef = synceeStaging
} }
// This is done to reduce the pruning depth to 6 blocks
finalityDepth := 3
consensusConfig.FinalityDuration = time.Duration(finalityDepth) * consensusConfig.TargetTimePerBlock
consensusConfig.K = 0
synceeConfig := *consensusConfig
synceeConfig.SkipAddingGenesis = true
factory := consensus.NewFactory()
tcSyncer, teardownSyncer, err := factory.NewTestConsensus(consensusConfig, "TestValidateAndInsertPruningPointSyncer") tcSyncer, teardownSyncer, err := factory.NewTestConsensus(consensusConfig, "TestValidateAndInsertPruningPointSyncer")
if err != nil { if err != nil {
t.Fatalf("Error setting up tcSyncer: %+v", err) t.Fatalf("Error setting up tcSyncer: %+v", err)
} }
defer teardownSyncer(false) defer teardownSyncer(false)
tcSyncee1, teardownSyncee1, err := factory.NewTestConsensus(&synceeConfig, "TestValidateAndInsertPruningPointSyncee1") tcSyncee1, teardownSyncee1, err := factory.NewTestConsensus(consensusConfig, "TestValidateAndInsertPruningPointSyncee1")
if err != nil { if err != nil {
t.Fatalf("Error setting up tcSyncee1: %+v", err) t.Fatalf("Error setting up tcSyncee1: %+v", err)
} }
defer teardownSyncee1(false) defer teardownSyncee1(false)
const numSharedBlocks = 2
tipHash := consensusConfig.GenesisHash tipHash := consensusConfig.GenesisHash
for i := 0; i < finalityDepth-2; i++ { for i := 0; i < numSharedBlocks; i++ {
tipHash = addBlock(tcSyncer, []*externalapi.DomainHash{tipHash}, t) tipHash = addBlock(tcSyncer, []*externalapi.DomainHash{tipHash}, t)
} block, err := tcSyncer.GetBlock(tipHash)
// Add block in the anticone of the pruning point to test such situation
pruningPointAnticoneBlock := addBlock(tcSyncer, []*externalapi.DomainHash{tipHash}, t)
tipHash = addBlock(tcSyncer, []*externalapi.DomainHash{tipHash}, t)
nextPruningPoint := addBlock(tcSyncer, []*externalapi.DomainHash{tipHash}, t)
tipHash = addBlock(tcSyncer, []*externalapi.DomainHash{pruningPointAnticoneBlock, nextPruningPoint}, t)
// Add blocks until the pruning point changes
for {
tipHash = addBlock(tcSyncer, []*externalapi.DomainHash{tipHash}, t)
pruningPoint, err := tcSyncer.PruningPoint()
if err != nil {
t.Fatalf("PruningPoint: %+v", err)
}
if !pruningPoint.Equal(consensusConfig.GenesisHash) {
break
}
}
pruningPoint, err := tcSyncer.PruningPoint()
if err != nil {
t.Fatalf("PruningPoint: %+v", err)
}
if !pruningPoint.Equal(nextPruningPoint) {
t.Fatalf("Unexpected pruning point %s", pruningPoint)
}
syncConsensuses(tcSyncer, tcSyncee1)
// Test a situation where a consensus with pruned headers syncs another fresh consensus.
tcSyncee2, teardownSyncee2, err := factory.NewTestConsensus(&synceeConfig, "TestValidateAndInsertPruningPointSyncee2")
if err != nil {
t.Fatalf("Error setting up tcSyncee2: %+v", err)
}
defer teardownSyncee2(false)
syncConsensuses(tcSyncee1, tcSyncee2)
})
}
// TestValidateAndInsertPruningPointWithSideBlocks makes sure that when a node applies a UTXO-Set downloaded during
// IBD, while it already has a non-empty UTXO-Set originating from blocks mined on top of genesis - the resulting
// UTXO set is correct
func TestValidateAndInsertPruningPointWithSideBlocks(t *testing.T) {
testutils.ForAllNets(t, true, func(t *testing.T, consensusConfig *consensus.Config) {
// This is done to reduce the pruning depth to 6 blocks
finalityDepth := 3
consensusConfig.FinalityDuration = time.Duration(finalityDepth) * consensusConfig.TargetTimePerBlock
consensusConfig.K = 0
synceeConfig := *consensusConfig
synceeConfig.SkipAddingGenesis = true
factory := consensus.NewFactory()
tcSyncer, teardownSyncer, err := factory.NewTestConsensus(consensusConfig, "TestValidateAndInsertPruningPointSyncer")
if err != nil {
t.Fatalf("Error setting up tcSyncer: %+v", err)
}
defer teardownSyncer(false)
tcSyncee, teardownSyncee, err := factory.NewTestConsensus(&synceeConfig, "TestValidateAndInsertPruningPointSyncee")
if err != nil {
t.Fatalf("Error setting up tcSyncee: %+v", err)
}
defer teardownSyncee(false)
// Mine two blocks on syncee on top of genesis
synceeOnlyBlock := addBlock(tcSyncer, []*externalapi.DomainHash{consensusConfig.GenesisHash}, t)
addBlock(tcSyncer, []*externalapi.DomainHash{synceeOnlyBlock}, t)
tipHash := consensusConfig.GenesisHash
for i := 0; i < finalityDepth-2; i++ {
tipHash = addBlock(tcSyncer, []*externalapi.DomainHash{tipHash}, t)
}
// Add block in the anticone of the pruning point to test such situation
pruningPointAnticoneBlock := addBlock(tcSyncer, []*externalapi.DomainHash{tipHash}, t)
tipHash = addBlock(tcSyncer, []*externalapi.DomainHash{tipHash}, t)
nextPruningPoint := addBlock(tcSyncer, []*externalapi.DomainHash{tipHash}, t)
tipHash = addBlock(tcSyncer, []*externalapi.DomainHash{pruningPointAnticoneBlock, nextPruningPoint}, t)
// Add blocks until the pruning point changes
for {
tipHash = addBlock(tcSyncer, []*externalapi.DomainHash{tipHash}, t)
pruningPoint, err := tcSyncer.PruningPoint()
if err != nil {
t.Fatalf("PruningPoint: %+v", err)
}
if !pruningPoint.Equal(consensusConfig.GenesisHash) {
break
}
}
pruningPoint, err := tcSyncer.PruningPoint()
if err != nil {
t.Fatalf("PruningPoint: %+v", err)
}
if !pruningPoint.Equal(nextPruningPoint) {
t.Fatalf("Unexpected pruning point %s", pruningPoint)
}
pruningPointHeaders, err := tcSyncer.PruningPointHeaders()
if err != nil {
t.Fatalf("PruningPointHeaders: %+v", err)
}
err = tcSyncee.ImportPruningPoints(pruningPointHeaders)
if err != nil {
t.Fatalf("PruningPointHeaders: %+v", err)
}
pointAndItsAnticoneWithTrustedData, err := tcSyncer.PruningPointAndItsAnticoneWithTrustedData()
if err != nil {
t.Fatalf("PruningPointAndItsAnticoneWithTrustedData: %+v", err)
}
for _, blockWithTrustedData := range pointAndItsAnticoneWithTrustedData {
_, err := tcSyncee.ValidateAndInsertBlockWithTrustedData(blockWithTrustedData, false)
if err != nil {
t.Fatalf("ValidateAndInsertBlockWithTrustedData: %+v", err)
}
}
syncerVirtualSelectedParent, err := tcSyncer.GetVirtualSelectedParent()
if err != nil {
t.Fatalf("GetVirtualSelectedParent: %+v", err)
}
missingBlocksHashes, _, err := tcSyncer.GetHashesBetween(pruningPoint, syncerVirtualSelectedParent, math.MaxUint64)
if err != nil {
t.Fatalf("GetHashesBetween: %+v", err)
}
for _, blocksHash := range missingBlocksHashes {
blockInfo, err := tcSyncee.GetBlockInfo(blocksHash)
if err != nil {
t.Fatalf("GetBlockInfo: %+v", err)
}
if blockInfo.Exists {
continue
}
block, err := tcSyncer.GetBlock(blocksHash)
if err != nil { if err != nil {
t.Fatalf("GetBlock: %+v", err) t.Fatalf("GetBlock: %+v", err)
} }
_, err = tcSyncee.ValidateAndInsertBlock(block, false) _, err = tcSyncee1.ValidateAndInsertBlock(block, true)
if err != nil { if err != nil {
t.Fatalf("ValidateAndInsertBlock: %+v", err) t.Fatalf("ValidateAndInsertBlock: %+v", err)
} }
} }
pruningPointUTXOs, err := tcSyncer.GetPruningPointUTXOs(pruningPoint, nil, 1000) // Add two side blocks to syncee
if err != nil { tipHashSyncee := tipHash
t.Fatalf("GetPruningPointUTXOs: %+v", err) for i := 0; i < 2; i++ {
} tipHashSyncee = addBlock(tcSyncee1, []*externalapi.DomainHash{tipHashSyncee}, t)
err = tcSyncee.AppendImportedPruningPointUTXOs(pruningPointUTXOs)
if err != nil {
t.Fatalf("AppendImportedPruningPointUTXOs: %+v", err)
} }
// Check that ValidateAndInsertImportedPruningPoint fails for invalid pruning point for i := 0; i < finalityDepth-numSharedBlocks-2; i++ {
err = tcSyncee.ValidateAndInsertImportedPruningPoint(tipHash) tipHash = addBlock(tcSyncer, []*externalapi.DomainHash{tipHash}, t)
if !errors.Is(err, ruleerrors.ErrUnexpectedPruningPoint) {
t.Fatalf("Unexpected error: %+v", err)
} }
err = tcSyncee.ClearImportedPruningPointData() // Add block in the anticone of the pruning point to test such situation
if err != nil { pruningPointAnticoneBlock := addBlock(tcSyncer, []*externalapi.DomainHash{tipHash}, t)
t.Fatalf("ClearImportedPruningPointData: %+v", err) tipHash = addBlock(tcSyncer, []*externalapi.DomainHash{tipHash}, t)
} nextPruningPoint := addBlock(tcSyncer, []*externalapi.DomainHash{tipHash}, t)
err = tcSyncee.AppendImportedPruningPointUTXOs(makeFakeUTXOs())
if err != nil { tipHash = addBlock(tcSyncer, []*externalapi.DomainHash{pruningPointAnticoneBlock, nextPruningPoint}, t)
t.Fatalf("AppendImportedPruningPointUTXOs: %+v", err)
// Add blocks until the pruning point changes
for {
tipHash = addBlock(tcSyncer, []*externalapi.DomainHash{tipHash}, t)
pruningPoint, err := tcSyncer.PruningPoint()
if err != nil {
t.Fatalf("PruningPoint: %+v", err)
}
if !pruningPoint.Equal(consensusConfig.GenesisHash) {
break
}
} }
// Check that ValidateAndInsertImportedPruningPoint fails if the UTXO commitment doesn't fit the provided UTXO set. pruningPoint, err := tcSyncer.PruningPoint()
err = tcSyncee.ValidateAndInsertImportedPruningPoint(pruningPoint)
if !errors.Is(err, ruleerrors.ErrBadPruningPointUTXOSet) {
t.Fatalf("Unexpected error: %+v", err)
}
err = tcSyncee.ClearImportedPruningPointData()
if err != nil {
t.Fatalf("ClearImportedPruningPointData: %+v", err)
}
err = tcSyncee.AppendImportedPruningPointUTXOs(pruningPointUTXOs)
if err != nil {
t.Fatalf("AppendImportedPruningPointUTXOs: %+v", err)
}
// Check that ValidateAndInsertImportedPruningPoint works given the right arguments.
err = tcSyncee.ValidateAndInsertImportedPruningPoint(pruningPoint)
if err != nil {
t.Fatalf("ValidateAndInsertImportedPruningPoint: %+v", err)
}
synceeTips, err := tcSyncee.Tips()
if err != nil {
t.Fatalf("Tips: %+v", err)
}
syncerTips, err := tcSyncer.Tips()
if err != nil {
t.Fatalf("Tips: %+v", err)
}
if !externalapi.HashesEqual(synceeTips, syncerTips) {
t.Fatalf("Syncee's tips are %s while syncer's are %s", synceeTips, syncerTips)
}
tipHash = addBlock(tcSyncer, syncerTips, t)
tip, err := tcSyncer.GetBlock(tipHash)
if err != nil {
t.Fatalf("GetBlock: %+v", err)
}
_, err = tcSyncee.ValidateAndInsertBlock(tip, true)
if err != nil {
t.Fatalf("ValidateAndInsertBlock: %+v", err)
}
blockInfo, err := tcSyncee.GetBlockInfo(tipHash)
if err != nil {
t.Fatalf("GetBlockInfo: %+v", err)
}
if blockInfo.BlockStatus != externalapi.StatusUTXOValid {
t.Fatalf("Tip didn't pass UTXO verification")
}
synceePruningPoint, err := tcSyncee.PruningPoint()
if err != nil { if err != nil {
t.Fatalf("PruningPoint: %+v", err) t.Fatalf("PruningPoint: %+v", err)
} }
if !synceePruningPoint.Equal(pruningPoint) { if !pruningPoint.Equal(nextPruningPoint) {
t.Fatalf("The syncee pruning point has not changed as exepcted") t.Fatalf("Unexpected pruning point %s", pruningPoint)
} }
tcSyncee1Ref := &tcSyncee1
syncConsensuses(&tcSyncer, tcSyncee1Ref)
// Test a situation where a consensus with pruned headers syncs another fresh consensus.
tcSyncee2, teardownSyncee2, err := factory.NewTestConsensus(consensusConfig, "TestValidateAndInsertPruningPointSyncee2")
if err != nil {
t.Fatalf("Error setting up tcSyncee2: %+v", err)
}
defer teardownSyncee2(false)
syncConsensuses(tcSyncee1Ref, &tcSyncee2)
}) })
} }

View File

@ -15,12 +15,14 @@ func (v *blockValidator) ValidateBodyInContext(stagingArea *model.StagingArea, b
onEnd := logger.LogAndMeasureExecutionTime(log, "ValidateBodyInContext") onEnd := logger.LogAndMeasureExecutionTime(log, "ValidateBodyInContext")
defer onEnd() defer onEnd()
err := v.checkBlockIsNotPruned(stagingArea, blockHash) if !isBlockWithTrustedData {
if err != nil { err := v.checkBlockIsNotPruned(stagingArea, blockHash)
return err if err != nil {
return err
}
} }
err = v.checkBlockTransactions(stagingArea, blockHash) err := v.checkBlockTransactions(stagingArea, blockHash)
if err != nil { if err != nil {
return err return err
} }
@ -52,7 +54,7 @@ func (v *blockValidator) checkBlockIsNotPruned(stagingArea *model.StagingArea, b
return err return err
} }
isAncestorOfSomeTips, err := v.dagTopologyManager.IsAncestorOfAny(stagingArea, blockHash, tips) isAncestorOfSomeTips, err := v.dagTopologyManagers[0].IsAncestorOfAny(stagingArea, blockHash, tips)
if err != nil { if err != nil {
return err return err
} }
@ -69,7 +71,7 @@ func (v *blockValidator) checkParentBlockBodiesExist(
stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) error { stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) error {
missingParentHashes := []*externalapi.DomainHash{} missingParentHashes := []*externalapi.DomainHash{}
parents, err := v.dagTopologyManager.Parents(stagingArea, blockHash) parents, err := v.dagTopologyManagers[0].Parents(stagingArea, blockHash)
if err != nil { if err != nil {
return err return err
} }
@ -90,7 +92,7 @@ func (v *blockValidator) checkParentBlockBodiesExist(
return err return err
} }
isInPastOfPruningPoint, err := v.dagTopologyManager.IsAncestorOf(stagingArea, parent, pruningPoint) isInPastOfPruningPoint, err := v.dagTopologyManagers[0].IsAncestorOf(stagingArea, parent, pruningPoint)
if err != nil { if err != nil {
return err return err
} }

View File

@ -6,6 +6,7 @@ import (
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi" "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/kaspanet/kaspad/domain/consensus/ruleerrors" "github.com/kaspanet/kaspad/domain/consensus/ruleerrors"
"github.com/kaspanet/kaspad/domain/consensus/utils/consensushashing" "github.com/kaspanet/kaspad/domain/consensus/utils/consensushashing"
"github.com/kaspanet/kaspad/domain/consensus/utils/pow"
"github.com/kaspanet/kaspad/infrastructure/logger" "github.com/kaspanet/kaspad/infrastructure/logger"
"github.com/pkg/errors" "github.com/pkg/errors"
) )
@ -30,7 +31,7 @@ func (v *blockValidator) ValidateHeaderInContext(stagingArea *model.StagingArea,
var logErr error var logErr error
log.Debug(logger.NewLogClosure(func() string { log.Debug(logger.NewLogClosure(func() string {
var ghostdagData *externalapi.BlockGHOSTDAGData var ghostdagData *externalapi.BlockGHOSTDAGData
ghostdagData, logErr = v.ghostdagDataStore.Get(v.databaseContext, stagingArea, blockHash, false) ghostdagData, logErr = v.ghostdagDataStores[0].Get(v.databaseContext, stagingArea, blockHash, false)
if err != nil { if err != nil {
return "" return ""
} }
@ -62,9 +63,12 @@ func (v *blockValidator) ValidateHeaderInContext(stagingArea *model.StagingArea,
return err return err
} }
if !hasReachabilityData { if !hasReachabilityData {
err = v.reachabilityManager.AddBlock(stagingArea, blockHash) blockLevel := pow.BlockLevel(header)
if err != nil { for i := 0; i <= blockLevel; i++ {
return err err = v.reachabilityManagers[i].AddBlock(stagingArea, blockHash)
if err != nil {
return err
}
} }
} }
@ -125,7 +129,7 @@ func (v *blockValidator) hasValidatedHeader(stagingArea *model.StagingArea, bloc
// checkParentsIncest validates that no parent is an ancestor of another parent // checkParentsIncest validates that no parent is an ancestor of another parent
func (v *blockValidator) checkParentsIncest(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) error { func (v *blockValidator) checkParentsIncest(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) error {
parents, err := v.dagTopologyManager.Parents(stagingArea, blockHash) parents, err := v.dagTopologyManagers[0].Parents(stagingArea, blockHash)
if err != nil { if err != nil {
return err return err
} }
@ -136,7 +140,7 @@ func (v *blockValidator) checkParentsIncest(stagingArea *model.StagingArea, bloc
continue continue
} }
isAAncestorOfB, err := v.dagTopologyManager.IsAncestorOf(stagingArea, parentA, parentB) isAAncestorOfB, err := v.dagTopologyManagers[0].IsAncestorOf(stagingArea, parentA, parentB)
if err != nil { if err != nil {
return err return err
} }
@ -175,7 +179,7 @@ func (v *blockValidator) validateMedianTime(stagingArea *model.StagingArea, head
} }
func (v *blockValidator) checkMergeSizeLimit(stagingArea *model.StagingArea, hash *externalapi.DomainHash) error { func (v *blockValidator) checkMergeSizeLimit(stagingArea *model.StagingArea, hash *externalapi.DomainHash) error {
ghostdagData, err := v.ghostdagDataStore.Get(v.databaseContext, stagingArea, hash, false) ghostdagData, err := v.ghostdagDataStores[0].Get(v.databaseContext, stagingArea, hash, false)
if err != nil { if err != nil {
return err return err
} }
@ -219,7 +223,7 @@ func (v *blockValidator) checkDAAScore(stagingArea *model.StagingArea, blockHash
func (v *blockValidator) checkBlueWork(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash, func (v *blockValidator) checkBlueWork(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash,
header externalapi.BlockHeader) error { header externalapi.BlockHeader) error {
ghostdagData, err := v.ghostdagDataStore.Get(v.databaseContext, stagingArea, blockHash, false) ghostdagData, err := v.ghostdagDataStores[0].Get(v.databaseContext, stagingArea, blockHash, false)
if err != nil { if err != nil {
return err return err
} }
@ -233,7 +237,7 @@ func (v *blockValidator) checkBlueWork(stagingArea *model.StagingArea, blockHash
func (v *blockValidator) checkHeaderBlueScore(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash, func (v *blockValidator) checkHeaderBlueScore(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash,
header externalapi.BlockHeader) error { header externalapi.BlockHeader) error {
ghostdagData, err := v.ghostdagDataStore.Get(v.databaseContext, stagingArea, blockHash, false) ghostdagData, err := v.ghostdagDataStores[0].Get(v.databaseContext, stagingArea, blockHash, false)
if err != nil { if err != nil {
return err return err
} }

View File

@ -27,19 +27,19 @@ type blockValidator struct {
difficultyManager model.DifficultyManager difficultyManager model.DifficultyManager
pastMedianTimeManager model.PastMedianTimeManager pastMedianTimeManager model.PastMedianTimeManager
transactionValidator model.TransactionValidator transactionValidator model.TransactionValidator
ghostdagManager model.GHOSTDAGManager ghostdagManagers []model.GHOSTDAGManager
dagTopologyManager model.DAGTopologyManager dagTopologyManagers []model.DAGTopologyManager
dagTraversalManager model.DAGTraversalManager dagTraversalManager model.DAGTraversalManager
coinbaseManager model.CoinbaseManager coinbaseManager model.CoinbaseManager
mergeDepthManager model.MergeDepthManager mergeDepthManager model.MergeDepthManager
pruningStore model.PruningStore pruningStore model.PruningStore
reachabilityManager model.ReachabilityManager reachabilityManagers []model.ReachabilityManager
finalityManager model.FinalityManager finalityManager model.FinalityManager
blockParentBuilder model.BlockParentBuilder blockParentBuilder model.BlockParentBuilder
pruningManager model.PruningManager pruningManager model.PruningManager
blockStore model.BlockStore blockStore model.BlockStore
ghostdagDataStore model.GHOSTDAGDataStore ghostdagDataStores []model.GHOSTDAGDataStore
blockHeaderStore model.BlockHeaderStore blockHeaderStore model.BlockHeaderStore
blockStatusStore model.BlockStatusStore blockStatusStore model.BlockStatusStore
reachabilityStore model.ReachabilityDataStore reachabilityStore model.ReachabilityDataStore
@ -63,19 +63,19 @@ func New(powMax *big.Int,
difficultyManager model.DifficultyManager, difficultyManager model.DifficultyManager,
pastMedianTimeManager model.PastMedianTimeManager, pastMedianTimeManager model.PastMedianTimeManager,
transactionValidator model.TransactionValidator, transactionValidator model.TransactionValidator,
ghostdagManager model.GHOSTDAGManager, ghostdagManagers []model.GHOSTDAGManager,
dagTopologyManager model.DAGTopologyManager, dagTopologyManagers []model.DAGTopologyManager,
dagTraversalManager model.DAGTraversalManager, dagTraversalManager model.DAGTraversalManager,
coinbaseManager model.CoinbaseManager, coinbaseManager model.CoinbaseManager,
mergeDepthManager model.MergeDepthManager, mergeDepthManager model.MergeDepthManager,
reachabilityManager model.ReachabilityManager, reachabilityManagers []model.ReachabilityManager,
finalityManager model.FinalityManager, finalityManager model.FinalityManager,
blockParentBuilder model.BlockParentBuilder, blockParentBuilder model.BlockParentBuilder,
pruningManager model.PruningManager, pruningManager model.PruningManager,
pruningStore model.PruningStore, pruningStore model.PruningStore,
blockStore model.BlockStore, blockStore model.BlockStore,
ghostdagDataStore model.GHOSTDAGDataStore, ghostdagDataStores []model.GHOSTDAGDataStore,
blockHeaderStore model.BlockHeaderStore, blockHeaderStore model.BlockHeaderStore,
blockStatusStore model.BlockStatusStore, blockStatusStore model.BlockStatusStore,
reachabilityStore model.ReachabilityDataStore, reachabilityStore model.ReachabilityDataStore,
@ -99,19 +99,19 @@ func New(powMax *big.Int,
difficultyManager: difficultyManager, difficultyManager: difficultyManager,
pastMedianTimeManager: pastMedianTimeManager, pastMedianTimeManager: pastMedianTimeManager,
transactionValidator: transactionValidator, transactionValidator: transactionValidator,
ghostdagManager: ghostdagManager, ghostdagManagers: ghostdagManagers,
dagTopologyManager: dagTopologyManager, dagTopologyManagers: dagTopologyManagers,
dagTraversalManager: dagTraversalManager, dagTraversalManager: dagTraversalManager,
coinbaseManager: coinbaseManager, coinbaseManager: coinbaseManager,
mergeDepthManager: mergeDepthManager, mergeDepthManager: mergeDepthManager,
reachabilityManager: reachabilityManager, reachabilityManagers: reachabilityManagers,
finalityManager: finalityManager, finalityManager: finalityManager,
blockParentBuilder: blockParentBuilder, blockParentBuilder: blockParentBuilder,
pruningManager: pruningManager, pruningManager: pruningManager,
pruningStore: pruningStore, pruningStore: pruningStore,
blockStore: blockStore, blockStore: blockStore,
ghostdagDataStore: ghostdagDataStore, ghostdagDataStores: ghostdagDataStores,
blockHeaderStore: blockHeaderStore, blockHeaderStore: blockHeaderStore,
blockStatusStore: blockStatusStore, blockStatusStore: blockStatusStore,
reachabilityStore: reachabilityStore, reachabilityStore: reachabilityStore,

View File

@ -43,9 +43,11 @@ func (v *blockValidator) ValidatePruningPointViolationAndProofOfWorkAndDifficult
return err return err
} }
err = v.checkPruningPointViolation(stagingArea, blockHash) if !isBlockWithTrustedData {
if err != nil { err = v.checkPruningPointViolation(stagingArea, blockHash)
return err if err != nil {
return err
}
} }
err = v.checkProofOfWork(header) err = v.checkProofOfWork(header)
@ -66,28 +68,36 @@ func (v *blockValidator) setParents(stagingArea *model.StagingArea,
header externalapi.BlockHeader, header externalapi.BlockHeader,
isBlockWithTrustedData bool) error { isBlockWithTrustedData bool) error {
parents := make([]*externalapi.DomainHash, 0, len(header.DirectParents())) for level := 0; level <= pow.BlockLevel(header); level++ {
for _, currentParent := range header.DirectParents() { var parents []*externalapi.DomainHash
exists, err := v.blockStatusStore.Exists(v.databaseContext, stagingArea, currentParent) for _, parent := range header.ParentsAtLevel(level) {
_, err := v.ghostdagDataStores[level].Get(v.databaseContext, stagingArea, parent, false)
isNotFoundError := database.IsNotFoundError(err)
if !isNotFoundError && err != nil {
return err
}
if isNotFoundError {
if level == 0 && !isBlockWithTrustedData {
return errors.Errorf("direct parent %s is missing: only block with prefilled information can have some missing parents", parent)
}
continue
}
parents = append(parents, parent)
}
if len(parents) == 0 {
parents = append(parents, model.VirtualGenesisBlockHash)
}
err := v.dagTopologyManagers[level].SetParents(stagingArea, blockHash, parents)
if err != nil { if err != nil {
return err return err
} }
if !exists {
if !isBlockWithTrustedData {
return errors.Errorf("direct parent %s is missing: only block with prefilled information can have some missing parents", currentParent)
}
continue
}
parents = append(parents, currentParent)
} }
if len(parents) == 0 { return nil
parents = append(parents, model.VirtualGenesisBlockHash)
}
return v.dagTopologyManager.SetParents(stagingArea, blockHash, parents)
} }
func (v *blockValidator) validateDifficulty(stagingArea *model.StagingArea, func (v *blockValidator) validateDifficulty(stagingArea *model.StagingArea,
@ -96,7 +106,20 @@ func (v *blockValidator) validateDifficulty(stagingArea *model.StagingArea,
if !isBlockWithTrustedData { if !isBlockWithTrustedData {
// We need to calculate GHOSTDAG for the block in order to check its difficulty and blue work // We need to calculate GHOSTDAG for the block in order to check its difficulty and blue work
err := v.ghostdagManager.GHOSTDAG(stagingArea, blockHash) err := v.ghostdagManagers[0].GHOSTDAG(stagingArea, blockHash)
if err != nil {
return err
}
}
header, err := v.blockHeaderStore.BlockHeader(v.databaseContext, stagingArea, blockHash)
if err != nil {
return err
}
blockLevel := pow.BlockLevel(header)
for i := 1; i <= blockLevel; i++ {
err = v.ghostdagManagers[i].GHOSTDAG(stagingArea, blockHash)
if err != nil { if err != nil {
return err return err
} }
@ -110,10 +133,6 @@ func (v *blockValidator) validateDifficulty(stagingArea *model.StagingArea,
return err return err
} }
header, err := v.blockHeaderStore.BlockHeader(v.databaseContext, stagingArea, blockHash)
if err != nil {
return err
}
if header.Bits() != expectedBits { if header.Bits() != expectedBits {
return errors.Wrapf(ruleerrors.ErrUnexpectedDifficulty, "block difficulty of %d is not the expected value of %d", header.Bits(), expectedBits) return errors.Wrapf(ruleerrors.ErrUnexpectedDifficulty, "block difficulty of %d is not the expected value of %d", header.Bits(), expectedBits)
} }
@ -215,7 +234,7 @@ func (v *blockValidator) checkPruningPointViolation(stagingArea *model.StagingAr
return err return err
} }
parents, err := v.dagTopologyManager.Parents(stagingArea, blockHash) parents, err := v.dagTopologyManagers[0].Parents(stagingArea, blockHash)
if err != nil { if err != nil {
return err return err
} }
@ -224,7 +243,7 @@ func (v *blockValidator) checkPruningPointViolation(stagingArea *model.StagingAr
return nil return nil
} }
isAncestorOfAny, err := v.dagTopologyManager.IsAncestorOfAny(stagingArea, pruningPoint, parents) isAncestorOfAny, err := v.dagTopologyManagers[0].IsAncestorOfAny(stagingArea, pruningPoint, parents)
if err != nil { if err != nil {
return err return err
} }

View File

@ -14,21 +14,10 @@ func (dtm *dagTraversalManager) AnticoneFromVirtualPOV(stagingArea *model.Stagin
return nil, err return nil, err
} }
return dtm.anticoneFromBlocks(stagingArea, virtualParents, blockHash) return dtm.AnticoneFromBlocks(stagingArea, virtualParents, blockHash)
} }
func (dtm *dagTraversalManager) Anticone(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) ( func (dtm *dagTraversalManager) AnticoneFromBlocks(stagingArea *model.StagingArea, tips []*externalapi.DomainHash, blockHash *externalapi.DomainHash) (
[]*externalapi.DomainHash, error) {
tips, err := dtm.consensusStateStore.Tips(stagingArea, dtm.databaseContext)
if err != nil {
return nil, err
}
return dtm.anticoneFromBlocks(stagingArea, tips, blockHash)
}
func (dtm *dagTraversalManager) anticoneFromBlocks(stagingArea *model.StagingArea, tips []*externalapi.DomainHash, blockHash *externalapi.DomainHash) (
[]*externalapi.DomainHash, error) { []*externalapi.DomainHash, error) {
anticone := []*externalapi.DomainHash{} anticone := []*externalapi.DomainHash{}

View File

@ -6,7 +6,7 @@ import (
"github.com/pkg/errors" "github.com/pkg/errors"
) )
// dagTraversalManager exposes methods for travering blocks // dagTraversalManager exposes methods for traversing blocks
// in the DAG // in the DAG
type dagTraversalManager struct { type dagTraversalManager struct {
databaseContext model.DBReader databaseContext model.DBReader
@ -15,7 +15,6 @@ type dagTraversalManager struct {
ghostdagManager model.GHOSTDAGManager ghostdagManager model.GHOSTDAGManager
ghostdagDataStore model.GHOSTDAGDataStore ghostdagDataStore model.GHOSTDAGDataStore
reachabilityDataStore model.ReachabilityDataStore reachabilityDataStore model.ReachabilityDataStore
consensusStateStore model.ConsensusStateStore
daaWindowStore model.BlocksWithTrustedDataDAAWindowStore daaWindowStore model.BlocksWithTrustedDataDAAWindowStore
genesisHash *externalapi.DomainHash genesisHash *externalapi.DomainHash
} }
@ -27,7 +26,6 @@ func New(
ghostdagDataStore model.GHOSTDAGDataStore, ghostdagDataStore model.GHOSTDAGDataStore,
reachabilityDataStore model.ReachabilityDataStore, reachabilityDataStore model.ReachabilityDataStore,
ghostdagManager model.GHOSTDAGManager, ghostdagManager model.GHOSTDAGManager,
conssensusStateStore model.ConsensusStateStore,
daaWindowStore model.BlocksWithTrustedDataDAAWindowStore, daaWindowStore model.BlocksWithTrustedDataDAAWindowStore,
genesisHash *externalapi.DomainHash) model.DAGTraversalManager { genesisHash *externalapi.DomainHash) model.DAGTraversalManager {
return &dagTraversalManager{ return &dagTraversalManager{
@ -36,43 +34,12 @@ func New(
ghostdagDataStore: ghostdagDataStore, ghostdagDataStore: ghostdagDataStore,
reachabilityDataStore: reachabilityDataStore, reachabilityDataStore: reachabilityDataStore,
ghostdagManager: ghostdagManager, ghostdagManager: ghostdagManager,
consensusStateStore: conssensusStateStore,
daaWindowStore: daaWindowStore, daaWindowStore: daaWindowStore,
genesisHash: genesisHash, genesisHash: genesisHash,
} }
} }
// BlockAtDepth returns the hash of the highest block with a blue score
// lower than (highHash.blueSore - depth) in the selected-parent-chain
// of the block with the given highHash's selected parent chain.
func (dtm *dagTraversalManager) BlockAtDepth(stagingArea *model.StagingArea, highHash *externalapi.DomainHash, depth uint64) (*externalapi.DomainHash, error) {
currentBlockHash := highHash
highBlockGHOSTDAGData, err := dtm.ghostdagDataStore.Get(dtm.databaseContext, stagingArea, highHash, false)
if err != nil {
return nil, err
}
requiredBlueScore := uint64(0)
if highBlockGHOSTDAGData.BlueScore() > depth {
requiredBlueScore = highBlockGHOSTDAGData.BlueScore() - depth
}
currentBlockGHOSTDAGData := highBlockGHOSTDAGData
// If we used `BlockIterator` we'd need to do more calls to `ghostdagDataStore` so we can get the blueScore
for currentBlockGHOSTDAGData.BlueScore() >= requiredBlueScore {
if currentBlockGHOSTDAGData.SelectedParent() == nil { // genesis
return currentBlockHash, nil
}
currentBlockHash = currentBlockGHOSTDAGData.SelectedParent()
currentBlockGHOSTDAGData, err = dtm.ghostdagDataStore.Get(dtm.databaseContext, stagingArea, currentBlockHash, false)
if err != nil {
return nil, err
}
}
return currentBlockHash, nil
}
func (dtm *dagTraversalManager) LowestChainBlockAboveOrEqualToBlueScore(stagingArea *model.StagingArea, highHash *externalapi.DomainHash, blueScore uint64) (*externalapi.DomainHash, error) { func (dtm *dagTraversalManager) LowestChainBlockAboveOrEqualToBlueScore(stagingArea *model.StagingArea, highHash *externalapi.DomainHash, blueScore uint64) (*externalapi.DomainHash, error) {
highBlockGHOSTDAGData, err := dtm.ghostdagDataStore.Get(dtm.databaseContext, stagingArea, highHash, false) highBlockGHOSTDAGData, err := dtm.ghostdagDataStore.Get(dtm.databaseContext, stagingArea, highHash, false)
if err != nil { if err != nil {

View File

@ -7,204 +7,9 @@ import (
"github.com/kaspanet/kaspad/domain/consensus" "github.com/kaspanet/kaspad/domain/consensus"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi" "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/kaspanet/kaspad/domain/consensus/model/testapi"
"github.com/kaspanet/kaspad/domain/consensus/utils/testutils" "github.com/kaspanet/kaspad/domain/consensus/utils/testutils"
) )
const commonChainSize = 5
const depth uint64 = 2
//TestBlockAtDepthOnChainDag compares the result of BlockAtDepth to the result of looping over the SelectedChain on a single chain DAG.
func TestBlockAtDepthOnChainDag(t *testing.T) {
testutils.ForAllNets(t, true, func(t *testing.T, consensusConfig *consensus.Config) {
stagingArea := model.NewStagingArea()
factory := consensus.NewFactory()
tc, tearDown, err := factory.NewTestConsensus(consensusConfig,
"TestBlockAtDepthOnChainDag")
if err != nil {
t.Fatalf("Failed creating a NewTestConsensus: %s", err)
}
defer tearDown(false)
highHash, err := createAChainDAG(consensusConfig.GenesisHash, tc)
if err != nil {
t.Fatalf("Failed creating a Chain DAG In BlockAtDepthTEST: %+v", err)
}
currentBlockHash := highHash
currentBlockData, err := tc.GHOSTDAGDataStore().Get(tc.DatabaseContext(), stagingArea, currentBlockHash, false)
if err != nil {
t.Fatalf("Failed getting GHOSTDAGData for block with hash %s: %+v", currentBlockHash.String(), err)
}
for i := uint64(0); i <= depth; i++ {
if currentBlockData.SelectedParent() == nil {
break
}
currentBlockHash = currentBlockData.SelectedParent()
currentBlockData, err = tc.GHOSTDAGDataStore().Get(tc.DatabaseContext(), stagingArea, currentBlockHash, false)
if err != nil {
t.Fatalf("Failed getting GHOSTDAGData for block with hash %s: %+v", currentBlockHash.String(), err)
}
}
expectedBlockHash := currentBlockHash
actualBlockHash, err := tc.DAGTraversalManager().BlockAtDepth(stagingArea, highHash, depth)
if err != nil {
t.Fatalf("Failed on BlockAtDepth: %+v", err)
}
if !actualBlockHash.Equal(expectedBlockHash) {
t.Fatalf("Expected block %s but got %s", expectedBlockHash, actualBlockHash)
}
})
}
func createAChainDAG(genesisHash *externalapi.DomainHash, tc testapi.TestConsensus) (*externalapi.DomainHash, error) {
block := genesisHash
var err error
for i := 0; i < commonChainSize; i++ {
block, _, err = tc.AddBlock([]*externalapi.DomainHash{block}, nil, nil)
if err != nil {
return nil, err
}
}
return block, nil
}
// TestBlockAtDepthOnDAGWhereTwoBlocksHaveSameSelectedParent compares the results of BlockAtDepth
// of 2 children that have the same selectedParent.
func TestBlockAtDepthOnDAGWhereTwoBlocksHaveSameSelectedParent(t *testing.T) {
testutils.ForAllNets(t, true, func(t *testing.T, consensusConfig *consensus.Config) {
factory := consensus.NewFactory()
tc, tearDown, err := factory.NewTestConsensus(consensusConfig,
"TestBlockAtDepthOnDAGWhereTwoBlocksHaveSameSelectedParent")
if err != nil {
t.Fatalf("Failed creating a NewTestConsensus: %s", err)
}
defer tearDown(false)
stagingArea := model.NewStagingArea()
firstChild, secondChild, err := createADAGTwoChildrenWithSameSelectedParent(consensusConfig.GenesisHash, tc)
if err != nil {
t.Fatalf("Failed creating a DAG where two blocks have same selected parent: %+v", err)
}
actualBlockHash, err := tc.DAGTraversalManager().BlockAtDepth(stagingArea, firstChild, depth)
if err != nil {
t.Fatalf("Failed at BlockAtDepth: %+v", err)
}
expectedSameHash, err := tc.DAGTraversalManager().BlockAtDepth(stagingArea, secondChild, depth)
if err != nil {
t.Fatalf("Failed in BlockAtDepth: %+v", err)
}
if !actualBlockHash.Equal(expectedSameHash) {
t.Fatalf("Expected block %s but got %s", expectedSameHash, actualBlockHash)
}
})
}
func createADAGTwoChildrenWithSameSelectedParent(genesisHash *externalapi.DomainHash,
tc testapi.TestConsensus) (*externalapi.DomainHash, *externalapi.DomainHash, error) {
block := genesisHash
var err error
for i := 0; i < commonChainSize; i++ {
block, _, err = tc.AddBlock([]*externalapi.DomainHash{block}, nil, nil)
if err != nil {
return nil, nil, err
}
}
firstChild, _, err := tc.AddBlock([]*externalapi.DomainHash{block}, nil, nil)
if err != nil {
return nil, nil, err
}
secondChild, _, err := tc.AddBlock([]*externalapi.DomainHash{block}, nil, nil)
if err != nil {
return nil, nil, err
}
return firstChild, secondChild, nil
}
// TestBlockAtDepthOnDAGWithTwoDifferentChains compares results of BlockAtDepth on two different chains,
// on the same DAG, and validates they merge at the correct point.
func TestBlockAtDepthOnDAGWithTwoDifferentChains(t *testing.T) {
testutils.ForAllNets(t, true, func(t *testing.T, consensusConfig *consensus.Config) {
factory := consensus.NewFactory()
tc, tearDown, err := factory.NewTestConsensus(consensusConfig,
"TestBlockAtDepthOnDAGWithTwoDifferentChains")
if err != nil {
t.Fatalf("Failed creating a NewTestConsensus: %s", err)
}
defer tearDown(false)
const sizeOfTheFirstChildSubChainDAG = 3
const sizeOfTheSecondChildSubChainDAG = 2
firstChild, secondChild, err := createADAGWithTwoDifferentChains(consensusConfig.GenesisHash, tc, sizeOfTheFirstChildSubChainDAG,
sizeOfTheSecondChildSubChainDAG)
if err != nil {
t.Fatalf("Failed creating a DAG with two different chains in BlockAtDepthTEST: %+v", err)
}
stagingArea := model.NewStagingArea()
actualBlockHash, err := tc.DAGTraversalManager().BlockAtDepth(stagingArea, firstChild, sizeOfTheFirstChildSubChainDAG)
if err != nil {
t.Fatalf("Failed in BlockAtDepth: %+v", err)
}
expectedSameHash, err := tc.DAGTraversalManager().BlockAtDepth(stagingArea, secondChild, sizeOfTheSecondChildSubChainDAG)
if err != nil {
t.Fatalf("Failed in BlockAtDepth: %+v", err)
}
if !actualBlockHash.Equal(expectedSameHash) {
t.Fatalf("Expected block %s but got %s", expectedSameHash, actualBlockHash)
}
expectedDiffHash, err := tc.DAGTraversalManager().BlockAtDepth(stagingArea, secondChild, sizeOfTheSecondChildSubChainDAG-1)
if err != nil {
t.Fatalf("Failed in BlockAtDepth: %+v", err)
}
if actualBlockHash.Equal(expectedDiffHash) {
t.Fatalf("Expected to a differente block")
}
})
}
func createADAGWithTwoDifferentChains(genesisHash *externalapi.DomainHash, tc testapi.TestConsensus,
sizeOfTheFirstChildSubChainDAG int, sizeOfTheSecondChildSubChainDAG int) (*externalapi.DomainHash, *externalapi.DomainHash, error) {
block := genesisHash
var err error
for i := 0; i < commonChainSize; i++ {
block, _, err = tc.AddBlock([]*externalapi.DomainHash{block}, nil, nil)
if err != nil {
return nil, nil, err
}
}
firstChainTipHash, _, err := tc.AddBlock([]*externalapi.DomainHash{block}, nil, nil)
if err != nil {
return nil, nil, err
}
secondChainTipHash, _, err := tc.AddBlock([]*externalapi.DomainHash{block}, nil, nil)
if err != nil {
return nil, nil, err
}
for i := 0; i < sizeOfTheFirstChildSubChainDAG; i++ {
firstChainTipHash, _, err = tc.AddBlock([]*externalapi.DomainHash{firstChainTipHash}, nil, nil)
if err != nil {
return nil, nil, err
}
}
for i := 0; i < sizeOfTheSecondChildSubChainDAG; i++ {
secondChainTipHash, _, err = tc.AddBlock([]*externalapi.DomainHash{secondChainTipHash}, nil, nil)
if err != nil {
return nil, nil, err
}
}
return firstChainTipHash, secondChainTipHash, nil
}
func TestLowestChainBlockAboveOrEqualToBlueScore(t *testing.T) { func TestLowestChainBlockAboveOrEqualToBlueScore(t *testing.T) {
testutils.ForAllNets(t, true, func(t *testing.T, consensusConfig *consensus.Config) { testutils.ForAllNets(t, true, func(t *testing.T, consensusConfig *consensus.Config) {
consensusConfig.FinalityDuration = 10 * consensusConfig.TargetTimePerBlock consensusConfig.FinalityDuration = 10 * consensusConfig.TargetTimePerBlock

View File

@ -53,7 +53,13 @@ func (dtm *dagTraversalManager) calculateBlockWindowHeap(stagingArea *model.Stag
break break
} }
if currentGHOSTDAGData.SelectedParent().Equal(model.VirtualGenesisBlockHash) { _, err := dtm.daaWindowStore.DAAWindowBlock(dtm.databaseContext, stagingArea, current, 0)
isNotFoundError := database.IsNotFoundError(err)
if !isNotFoundError && err != nil {
return nil, err
}
if !isNotFoundError {
for i := uint64(0); ; i++ { for i := uint64(0); ; i++ {
daaBlock, err := dtm.daaWindowStore.DAAWindowBlock(dtm.databaseContext, stagingArea, current, i) daaBlock, err := dtm.daaWindowStore.DAAWindowBlock(dtm.databaseContext, stagingArea, current, i)
if database.IsNotFoundError(err) { if database.IsNotFoundError(err) {

View File

@ -100,6 +100,11 @@ func (gm *ghostdagManager) GHOSTDAG(stagingArea *model.StagingArea, blockHash *e
newBlockData.blueWork.Set(selectedParentGHOSTDAGData.BlueWork()) newBlockData.blueWork.Set(selectedParentGHOSTDAGData.BlueWork())
// Then we add up all the *work*(not blueWork) that all of newBlock merge set blues and selected parent did // Then we add up all the *work*(not blueWork) that all of newBlock merge set blues and selected parent did
for _, blue := range newBlockData.mergeSetBlues { for _, blue := range newBlockData.mergeSetBlues {
// We don't count the work of the virtual genesis
if blue.Equal(model.VirtualGenesisBlockHash) {
continue
}
header, err := gm.headerStore.BlockHeader(gm.databaseContext, stagingArea, blue) header, err := gm.headerStore.BlockHeader(gm.databaseContext, stagingArea, blue)
if err != nil { if err != nil {
return err return err

View File

@ -1,132 +0,0 @@
package ghost
import (
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/kaspanet/kaspad/domain/consensus/utils/hashset"
)
// GHOST calculates the GHOST chain for the given `subDAG`
func GHOST(subDAG *model.SubDAG) ([]*externalapi.DomainHash, error) {
futureSizes, err := futureSizes(subDAG)
if err != nil {
return nil, err
}
ghostChain := []*externalapi.DomainHash{}
dagRootHashWithLargestFutureSize := blockHashWithLargestFutureSize(futureSizes, subDAG.RootHashes)
currentHash := dagRootHashWithLargestFutureSize
for {
ghostChain = append(ghostChain, currentHash)
currentBlock := subDAG.Blocks[*currentHash]
childHashes := currentBlock.ChildHashes
if len(childHashes) == 0 {
break
}
childHashWithLargestFutureSize := blockHashWithLargestFutureSize(futureSizes, childHashes)
currentHash = childHashWithLargestFutureSize
}
return ghostChain, nil
}
func blockHashWithLargestFutureSize(futureSizes map[externalapi.DomainHash]uint64,
blockHashes []*externalapi.DomainHash) *externalapi.DomainHash {
var blockHashWithLargestFutureSize *externalapi.DomainHash
largestFutureSize := uint64(0)
for _, blockHash := range blockHashes {
blockFutureSize := futureSizes[*blockHash]
if blockHashWithLargestFutureSize == nil || blockFutureSize > largestFutureSize ||
(blockFutureSize == largestFutureSize && blockHash.Less(blockHashWithLargestFutureSize)) {
largestFutureSize = blockFutureSize
blockHashWithLargestFutureSize = blockHash
}
}
return blockHashWithLargestFutureSize
}
func futureSizes(subDAG *model.SubDAG) (map[externalapi.DomainHash]uint64, error) {
heightMaps := buildHeightMaps(subDAG)
ghostReachabilityManager, err := newGHOSTReachabilityManager(subDAG, heightMaps)
if err != nil {
return nil, err
}
futureSizes := make(map[externalapi.DomainHash]uint64, len(subDAG.Blocks))
height := heightMaps.maxHeight
for {
for _, blockHash := range heightMaps.heightToBlockHashesMap[height] {
block := subDAG.Blocks[*blockHash]
currentBlockReverseMergeSetSize, err := calculateReverseMergeSetSize(subDAG, ghostReachabilityManager, block)
if err != nil {
return nil, err
}
futureSize := currentBlockReverseMergeSetSize
if currentBlockReverseMergeSetSize > 0 {
selectedChild := block.ChildHashes[0]
selectedChildFutureSize := futureSizes[*selectedChild]
futureSize += selectedChildFutureSize
}
futureSizes[*blockHash] = futureSize
}
if height == 0 {
break
}
height--
}
return futureSizes, nil
}
func calculateReverseMergeSetSize(subDAG *model.SubDAG,
ghostReachabilityManager *ghostReachabilityManager, block *model.SubDAGBlock) (uint64, error) {
if len(block.ChildHashes) == 0 {
return 0, nil
}
selectedChild := block.ChildHashes[0]
reverseMergeSetSize := uint64(1)
knownSelectedChildDescendants := hashset.NewFromSlice(selectedChild)
queue := append([]*externalapi.DomainHash{}, block.ChildHashes...)
addedToQueue := hashset.NewFromSlice(block.ChildHashes...)
for len(queue) > 0 {
var currentBlockHash *externalapi.DomainHash
currentBlockHash, queue = queue[0], queue[1:]
currentBlock := subDAG.Blocks[*currentBlockHash]
if knownSelectedChildDescendants.Contains(currentBlockHash) {
for _, childHash := range currentBlock.ChildHashes {
knownSelectedChildDescendants.Add(childHash)
}
continue
}
isCurrentBlockDescendantOfSelectedChild, err := ghostReachabilityManager.isDescendantOf(currentBlockHash, selectedChild)
if err != nil {
return 0, err
}
if isCurrentBlockDescendantOfSelectedChild {
knownSelectedChildDescendants.Add(currentBlockHash)
for _, childHash := range currentBlock.ChildHashes {
knownSelectedChildDescendants.Add(childHash)
}
continue
}
reverseMergeSetSize++
for _, childHash := range currentBlock.ChildHashes {
if addedToQueue.Contains(childHash) {
continue
}
queue = append(queue, childHash)
addedToQueue.Add(childHash)
}
}
return reverseMergeSetSize, nil
}

View File

@ -1,296 +0,0 @@
package ghost
import (
"encoding/json"
"fmt"
"github.com/kaspanet/kaspad/domain/consensus"
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/kaspanet/kaspad/domain/consensus/model/testapi"
"github.com/kaspanet/kaspad/domain/consensus/utils/consensushashing"
"github.com/kaspanet/kaspad/domain/consensus/utils/hashset"
"github.com/kaspanet/kaspad/domain/consensus/utils/testutils"
"os"
"reflect"
"testing"
)
func TestGHOST(t *testing.T) {
testChain := []struct {
parents []string
id string
expectedGHOSTChain []string
}{
{
parents: []string{"A"},
id: "B",
expectedGHOSTChain: []string{"A", "B"},
},
{
parents: []string{"B"},
id: "C",
expectedGHOSTChain: []string{"A", "B", "C"},
},
{
parents: []string{"B"},
id: "D",
expectedGHOSTChain: []string{"A", "B", "D"},
},
{
parents: []string{"C", "D"},
id: "E",
expectedGHOSTChain: []string{"A", "B", "D", "E"},
},
{
parents: []string{"C", "D"},
id: "F",
expectedGHOSTChain: []string{"A", "B", "D", "F"},
},
{
parents: []string{"A"},
id: "G",
expectedGHOSTChain: []string{"A", "B", "D", "F"},
},
{
parents: []string{"G"},
id: "H",
expectedGHOSTChain: []string{"A", "B", "D", "F"},
},
{
parents: []string{"H", "F"},
id: "I",
expectedGHOSTChain: []string{"A", "B", "D", "F", "I"},
},
{
parents: []string{"I"},
id: "J",
expectedGHOSTChain: []string{"A", "B", "D", "F", "I", "J"},
},
}
testutils.ForAllNets(t, true, func(t *testing.T, consensusConfig *consensus.Config) {
factory := consensus.NewFactory()
tc, tearDown, err := factory.NewTestConsensus(consensusConfig, "TestBlockWindow")
if err != nil {
t.Fatalf("NewTestConsensus: %s", err)
}
defer tearDown(false)
blockByIDMap := make(map[string]*externalapi.DomainHash)
idByBlockMap := make(map[externalapi.DomainHash]string)
blockByIDMap["A"] = consensusConfig.GenesisHash
idByBlockMap[*consensusConfig.GenesisHash] = "A"
mostRecentHash := consensusConfig.GenesisHash
for _, blockData := range testChain {
parents := hashset.New()
for _, parentID := range blockData.parents {
parent := blockByIDMap[parentID]
parents.Add(parent)
}
blockHash := addBlockWithHashSmallerThan(t, tc, parents.ToSlice(), mostRecentHash)
if err != nil {
t.Fatalf("AddBlock: %+v", err)
}
blockByIDMap[blockData.id] = blockHash
idByBlockMap[*blockHash] = blockData.id
mostRecentHash = blockHash
subDAG := convertDAGtoSubDAG(t, consensusConfig, tc)
ghostChainHashes, err := GHOST(subDAG)
if err != nil {
t.Fatalf("GHOST: %+v", err)
}
ghostChainIDs := make([]string, len(ghostChainHashes))
for i, ghostChainHash := range ghostChainHashes {
ghostChainIDs[i] = idByBlockMap[*ghostChainHash]
}
if !reflect.DeepEqual(ghostChainIDs, blockData.expectedGHOSTChain) {
t.Errorf("After adding block ID %s, GHOST chain expected to have IDs %s but got IDs %s",
blockData.id, blockData.expectedGHOSTChain, ghostChainIDs)
}
}
})
}
// addBlockWithHashSmallerThan adds a block to the DAG with the given parents such that its
// hash is smaller than `maxHash`. This ensures that the GHOST chain calculated from the
// DAG is deterministic
func addBlockWithHashSmallerThan(t *testing.T, tc testapi.TestConsensus,
parentHashes []*externalapi.DomainHash, maxHash *externalapi.DomainHash) *externalapi.DomainHash {
var block *externalapi.DomainBlock
blockHash := maxHash
for maxHash.LessOrEqual(blockHash) {
var err error
block, _, err = tc.BuildBlockWithParents(parentHashes, nil, nil)
if err != nil {
t.Fatalf("BuildBlockWithParents: %+v", err)
}
blockHash = consensushashing.BlockHash(block)
}
_, err := tc.ValidateAndInsertBlock(block, true)
if err != nil {
t.Fatalf("ValidateAndInsertBlock: %+v", err)
}
return blockHash
}
func convertDAGtoSubDAG(t *testing.T, consensusConfig *consensus.Config, tc testapi.TestConsensus) *model.SubDAG {
genesisHash := consensusConfig.GenesisHash
stagingArea := model.NewStagingArea()
tipHashes, err := tc.ConsensusStateStore().Tips(stagingArea, tc.DatabaseContext())
if err != nil {
t.Fatalf("Tips: %+v", err)
}
subDAG := &model.SubDAG{
RootHashes: []*externalapi.DomainHash{genesisHash},
TipHashes: tipHashes,
Blocks: map[externalapi.DomainHash]*model.SubDAGBlock{},
}
visited := hashset.New()
queue := tc.DAGTraversalManager().NewDownHeap(stagingArea)
err = queue.PushSlice(tipHashes)
if err != nil {
t.Fatalf("PushSlice: %+v", err)
}
for queue.Len() > 0 {
blockHash := queue.Pop()
visited.Add(blockHash)
dagChildHashes, err := tc.DAGTopologyManager().Children(stagingArea, blockHash)
if err != nil {
t.Fatalf("Children: %+v", err)
}
childHashes := []*externalapi.DomainHash{}
for _, dagChildHash := range dagChildHashes {
if dagChildHash.Equal(model.VirtualBlockHash) {
continue
}
childHashes = append(childHashes, dagChildHash)
}
dagParentHashes, err := tc.DAGTopologyManager().Parents(stagingArea, blockHash)
if err != nil {
t.Fatalf("Parents: %+v", err)
}
parentHashes := []*externalapi.DomainHash{}
for _, dagParentHash := range dagParentHashes {
if dagParentHash.Equal(model.VirtualGenesisBlockHash) {
continue
}
parentHashes = append(parentHashes, dagParentHash)
if !visited.Contains(dagParentHash) {
err := queue.Push(dagParentHash)
if err != nil {
t.Fatalf("Push: %+v", err)
}
}
}
subDAG.Blocks[*blockHash] = &model.SubDAGBlock{
BlockHash: blockHash,
ParentHashes: parentHashes,
ChildHashes: childHashes,
}
}
return subDAG
}
type jsonBlock struct {
ID string `json:"ID"`
Parents []string `json:"Parents"`
}
type testJSON struct {
Blocks []*jsonBlock `json:"blocks"`
}
func BenchmarkGHOST(b *testing.B) {
b.StopTimer()
// Load JSON
b.Logf("Loading JSON data")
jsonFile, err := os.Open("benchmark_data.json")
if err != nil {
b.Fatalf("Open: %+v", err)
}
defer jsonFile.Close()
test := &testJSON{}
decoder := json.NewDecoder(jsonFile)
decoder.DisallowUnknownFields()
err = decoder.Decode(&test)
if err != nil {
b.Fatalf("Decode: %+v", err)
}
// Convert JSON data to a SubDAG
b.Logf("Converting JSON data to SubDAG")
subDAG := &model.SubDAG{
RootHashes: []*externalapi.DomainHash{},
TipHashes: []*externalapi.DomainHash{},
Blocks: make(map[externalapi.DomainHash]*model.SubDAGBlock, len(test.Blocks)),
}
blockIDToHash := func(blockID string) *externalapi.DomainHash {
blockHashHex := fmt.Sprintf("%064s", blockID)
blockHash, err := externalapi.NewDomainHashFromString(blockHashHex)
if err != nil {
b.Fatalf("NewDomainHashFromString: %+v", err)
}
return blockHash
}
for _, block := range test.Blocks {
blockHash := blockIDToHash(block.ID)
parentHashes := []*externalapi.DomainHash{}
for _, parentID := range block.Parents {
parentHash := blockIDToHash(parentID)
parentHashes = append(parentHashes, parentHash)
}
subDAG.Blocks[*blockHash] = &model.SubDAGBlock{
BlockHash: blockHash,
ParentHashes: parentHashes,
ChildHashes: []*externalapi.DomainHash{},
}
}
for _, block := range subDAG.Blocks {
for _, parentHash := range block.ParentHashes {
parentBlock := subDAG.Blocks[*parentHash]
parentAlreadyHasBlockAsChild := false
for _, childHash := range parentBlock.ChildHashes {
if block.BlockHash.Equal(childHash) {
parentAlreadyHasBlockAsChild = true
break
}
}
if !parentAlreadyHasBlockAsChild {
parentBlock.ChildHashes = append(parentBlock.ChildHashes, block.BlockHash)
}
}
}
for _, block := range subDAG.Blocks {
if len(block.ParentHashes) == 0 {
subDAG.RootHashes = append(subDAG.RootHashes, block.BlockHash)
}
if len(block.ChildHashes) == 0 {
subDAG.TipHashes = append(subDAG.TipHashes, block.BlockHash)
}
}
b.Logf("Running benchmark")
b.ResetTimer()
b.StartTimer()
for i := 0; i < b.N; i++ {
_, err := GHOST(subDAG)
if err != nil {
b.Fatalf("GHOST: %+v", err)
}
}
}

View File

@ -1,75 +0,0 @@
package ghost
import (
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/kaspanet/kaspad/domain/consensus/utils/hashset"
)
type heightMaps struct {
blockHashToHeightMap map[externalapi.DomainHash]uint64
heightToBlockHashesMap map[uint64][]*externalapi.DomainHash
maxHeight uint64
}
func buildHeightMaps(subDAG *model.SubDAG) *heightMaps {
blockHashToHeightMap := make(map[externalapi.DomainHash]uint64, len(subDAG.Blocks))
heightToBlockHashesMap := make(map[uint64][]*externalapi.DomainHash)
maxHeight := uint64(0)
queue := append([]*externalapi.DomainHash{}, subDAG.RootHashes...)
addedToQueue := hashset.NewFromSlice(subDAG.RootHashes...)
for len(queue) > 0 {
var currentBlockHash *externalapi.DomainHash
currentBlockHash, queue = queue[0], queue[1:]
// Send the block to the back of the queue if one or more of its parents had not been processed yet
currentBlock := subDAG.Blocks[*currentBlockHash]
hasMissingParentData := false
for _, parentHash := range currentBlock.ParentHashes {
if _, ok := blockHashToHeightMap[*parentHash]; !ok {
hasMissingParentData = true
continue
}
}
if hasMissingParentData {
queue = append(queue, currentBlockHash)
continue
}
for _, childHash := range currentBlock.ChildHashes {
if addedToQueue.Contains(childHash) {
continue
}
queue = append(queue, childHash)
addedToQueue.Add(childHash)
}
currentBlockHeight := uint64(0)
if len(currentBlock.ParentHashes) > 0 {
highestParentHeight := uint64(0)
for _, parentHash := range currentBlock.ParentHashes {
parentHeight := blockHashToHeightMap[*parentHash]
if parentHeight > highestParentHeight {
highestParentHeight = parentHeight
}
}
currentBlockHeight = highestParentHeight + 1
}
blockHashToHeightMap[*currentBlockHash] = currentBlockHeight
if _, ok := heightToBlockHashesMap[currentBlockHeight]; !ok {
heightToBlockHashesMap[currentBlockHeight] = []*externalapi.DomainHash{}
}
heightToBlockHashesMap[currentBlockHeight] = append(heightToBlockHashesMap[currentBlockHeight], currentBlockHash)
if currentBlockHeight > maxHeight {
maxHeight = currentBlockHeight
}
}
return &heightMaps{
blockHashToHeightMap: blockHashToHeightMap,
heightToBlockHashesMap: heightToBlockHashesMap,
maxHeight: maxHeight,
}
}

View File

@ -1,144 +0,0 @@
package ghost
import (
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/kaspanet/kaspad/domain/consensus/processes/reachabilitymanager"
"github.com/pkg/errors"
)
type ghostReachabilityManager struct {
ghostdagDataStore *ghostdagDataStore
reachabilityDataStore *reachabilityDataStore
reachabilityManager model.ReachabilityManager
}
type ghostdagDataStore struct {
blockGHOSTDAGData map[externalapi.DomainHash]*externalapi.BlockGHOSTDAGData
}
func newGHOSTDAGDataStore() *ghostdagDataStore {
return &ghostdagDataStore{
blockGHOSTDAGData: map[externalapi.DomainHash]*externalapi.BlockGHOSTDAGData{},
}
}
func (gds *ghostdagDataStore) Stage(_ *model.StagingArea, blockHash *externalapi.DomainHash,
blockGHOSTDAGData *externalapi.BlockGHOSTDAGData, _ bool) {
gds.blockGHOSTDAGData[*blockHash] = blockGHOSTDAGData
}
func (gds *ghostdagDataStore) IsStaged(_ *model.StagingArea) bool {
return true
}
func (gds *ghostdagDataStore) Get(_ model.DBReader, _ *model.StagingArea,
blockHash *externalapi.DomainHash, _ bool) (*externalapi.BlockGHOSTDAGData, error) {
blockGHOSTDAGData, ok := gds.blockGHOSTDAGData[*blockHash]
if !ok {
return nil, errors.Errorf("ghostdag data not found for block hash %s", blockHash)
}
return blockGHOSTDAGData, nil
}
type reachabilityDataStore struct {
reachabilityData map[externalapi.DomainHash]model.ReachabilityData
reachabilityReindexRoot *externalapi.DomainHash
}
func newReachabilityDataStore() *reachabilityDataStore {
return &reachabilityDataStore{
reachabilityData: map[externalapi.DomainHash]model.ReachabilityData{},
reachabilityReindexRoot: nil,
}
}
func (rds *reachabilityDataStore) StageReachabilityData(_ *model.StagingArea,
blockHash *externalapi.DomainHash, reachabilityData model.ReachabilityData) {
rds.reachabilityData[*blockHash] = reachabilityData
}
func (rds *reachabilityDataStore) StageReachabilityReindexRoot(_ *model.StagingArea,
reachabilityReindexRoot *externalapi.DomainHash) {
rds.reachabilityReindexRoot = reachabilityReindexRoot
}
func (rds *reachabilityDataStore) IsStaged(_ *model.StagingArea) bool {
return true
}
func (rds *reachabilityDataStore) ReachabilityData(_ model.DBReader, _ *model.StagingArea,
blockHash *externalapi.DomainHash) (model.ReachabilityData, error) {
reachabilityData, ok := rds.reachabilityData[*blockHash]
if !ok {
return nil, errors.Errorf("reachability data not found for block hash %s", blockHash)
}
return reachabilityData, nil
}
func (rds *reachabilityDataStore) HasReachabilityData(_ model.DBReader, _ *model.StagingArea,
blockHash *externalapi.DomainHash) (bool, error) {
_, ok := rds.reachabilityData[*blockHash]
return ok, nil
}
func (rds *reachabilityDataStore) ReachabilityReindexRoot(_ model.DBReader,
_ *model.StagingArea) (*externalapi.DomainHash, error) {
return rds.reachabilityReindexRoot, nil
}
func newGHOSTReachabilityManager(subDAG *model.SubDAG, heightMaps *heightMaps) (*ghostReachabilityManager, error) {
ghostdagDataStore := newGHOSTDAGDataStore()
reachabilityDataStore := newReachabilityDataStore()
reachabilityManager := reachabilitymanager.New(nil, ghostdagDataStore, reachabilityDataStore)
ghostReachabilityManager := &ghostReachabilityManager{
ghostdagDataStore: ghostdagDataStore,
reachabilityDataStore: reachabilityDataStore,
reachabilityManager: reachabilityManager,
}
err := ghostReachabilityManager.initialize(subDAG, heightMaps)
if err != nil {
return nil, err
}
return ghostReachabilityManager, nil
}
func (grm *ghostReachabilityManager) initialize(subDAG *model.SubDAG, heightMaps *heightMaps) error {
for blockHash, block := range subDAG.Blocks {
blockHeight := heightMaps.blockHashToHeightMap[blockHash]
selectedParent := model.VirtualGenesisBlockHash
if len(block.ParentHashes) > 0 {
selectedParent = block.ParentHashes[0]
}
blockGHOSTDAGData := externalapi.NewBlockGHOSTDAGData(blockHeight, nil, selectedParent, nil, nil, nil)
grm.ghostdagDataStore.Stage(nil, &blockHash, blockGHOSTDAGData, false)
}
err := grm.reachabilityManager.Init(nil)
if err != nil {
return err
}
for height := uint64(0); height <= heightMaps.maxHeight; height++ {
for _, blockHash := range heightMaps.heightToBlockHashesMap[height] {
err := grm.reachabilityManager.AddBlock(nil, blockHash)
if err != nil {
return err
}
}
}
return nil
}
func (grm *ghostReachabilityManager) isDescendantOf(blockAHash *externalapi.DomainHash, blockBHash *externalapi.DomainHash) (bool, error) {
return grm.reachabilityManager.IsDAGAncestorOf(nil, blockBHash, blockAHash)
}

View File

@ -0,0 +1,637 @@
package pruningproofmanager
import (
consensusDB "github.com/kaspanet/kaspad/domain/consensus/database"
"github.com/kaspanet/kaspad/domain/consensus/datastructures/blockheaderstore"
"github.com/kaspanet/kaspad/domain/consensus/datastructures/blockrelationstore"
"github.com/kaspanet/kaspad/domain/consensus/datastructures/ghostdagdatastore"
"github.com/kaspanet/kaspad/domain/consensus/datastructures/reachabilitydatastore"
"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/ghostdagmanager"
"github.com/kaspanet/kaspad/domain/consensus/processes/reachabilitymanager"
"github.com/kaspanet/kaspad/domain/consensus/ruleerrors"
"github.com/kaspanet/kaspad/domain/consensus/utils/consensushashing"
"github.com/kaspanet/kaspad/domain/consensus/utils/constants"
"github.com/kaspanet/kaspad/domain/consensus/utils/hashset"
"github.com/kaspanet/kaspad/domain/consensus/utils/pow"
"github.com/kaspanet/kaspad/infrastructure/db/database"
"github.com/pkg/errors"
"math/big"
)
type pruningProofManager struct {
databaseContext model.DBManager
dagTopologyManagers []model.DAGTopologyManager
ghostdagManagers []model.GHOSTDAGManager
reachabilityManagers []model.ReachabilityManager
dagTraversalManagers []model.DAGTraversalManager
ghostdagDataStores []model.GHOSTDAGDataStore
pruningStore model.PruningStore
blockHeaderStore model.BlockHeaderStore
blockStatusStore model.BlockStatusStore
finalityStore model.FinalityStore
consensusStateStore model.ConsensusStateStore
genesisHash *externalapi.DomainHash
k externalapi.KType
pruningProofM uint64
}
// New instantiates a new PruningManager
func New(
databaseContext model.DBManager,
dagTopologyManagers []model.DAGTopologyManager,
ghostdagManagers []model.GHOSTDAGManager,
reachabilityManagers []model.ReachabilityManager,
dagTraversalManagers []model.DAGTraversalManager,
ghostdagDataStores []model.GHOSTDAGDataStore,
pruningStore model.PruningStore,
blockHeaderStore model.BlockHeaderStore,
blockStatusStore model.BlockStatusStore,
finalityStore model.FinalityStore,
consensusStateStore model.ConsensusStateStore,
genesisHash *externalapi.DomainHash,
k externalapi.KType,
pruningProofM uint64,
) model.PruningProofManager {
return &pruningProofManager{
databaseContext: databaseContext,
dagTopologyManagers: dagTopologyManagers,
ghostdagManagers: ghostdagManagers,
reachabilityManagers: reachabilityManagers,
dagTraversalManagers: dagTraversalManagers,
ghostdagDataStores: ghostdagDataStores,
pruningStore: pruningStore,
blockHeaderStore: blockHeaderStore,
blockStatusStore: blockStatusStore,
finalityStore: finalityStore,
consensusStateStore: consensusStateStore,
genesisHash: genesisHash,
k: k,
pruningProofM: pruningProofM,
}
}
func (ppm *pruningProofManager) BuildPruningPointProof(stagingArea *model.StagingArea) (*externalapi.PruningPointProof, error) {
pruningPoint, err := ppm.pruningStore.PruningPoint(ppm.databaseContext, stagingArea)
if err != nil {
return nil, err
}
pruningPointHeader, err := ppm.blockHeaderStore.BlockHeader(ppm.databaseContext, stagingArea, pruningPoint)
if err != nil {
return nil, err
}
maxLevel := len(pruningPointHeader.Parents()) - 1
headersByLevel := make(map[int][]externalapi.BlockHeader)
selectedTipByLevel := make([]*externalapi.DomainHash, maxLevel+1)
pruningPointLevel := pow.BlockLevel(pruningPointHeader)
for blockLevel := maxLevel; blockLevel >= 0; blockLevel-- {
var selectedTip *externalapi.DomainHash
if blockLevel <= pruningPointLevel {
selectedTip = pruningPoint
} else {
blockLevelParents := pruningPointHeader.ParentsAtLevel(blockLevel)
selectedTip, err = ppm.ghostdagManagers[blockLevel].ChooseSelectedParent(stagingArea, []*externalapi.DomainHash(blockLevelParents)...)
if err != nil {
return nil, err
}
}
selectedTipByLevel[blockLevel] = selectedTip
blockAtDepth2M, err := ppm.blockAtDepth(stagingArea, ppm.ghostdagDataStores[blockLevel], selectedTip, 2*ppm.pruningProofM)
if err != nil {
return nil, err
}
root := blockAtDepth2M
if blockLevel != maxLevel {
blockAtDepthMAtNextLevel, err := ppm.blockAtDepth(stagingArea, ppm.ghostdagDataStores[blockLevel+1], selectedTipByLevel[blockLevel+1], ppm.pruningProofM)
if err != nil {
return nil, err
}
isBlockAtDepthMAtNextLevelAncestorOfBlockAtDepth2M, err := ppm.dagTopologyManagers[blockLevel].IsAncestorOf(stagingArea, blockAtDepthMAtNextLevel, blockAtDepth2M)
if err != nil {
return nil, err
}
if isBlockAtDepthMAtNextLevelAncestorOfBlockAtDepth2M {
root = blockAtDepthMAtNextLevel
} else {
isBlockAtDepth2MAncestorOfBlockAtDepthMAtNextLevel, err := ppm.dagTopologyManagers[blockLevel].IsAncestorOf(stagingArea, blockAtDepth2M, blockAtDepthMAtNextLevel)
if err != nil {
return nil, err
}
if !isBlockAtDepth2MAncestorOfBlockAtDepthMAtNextLevel {
// find common ancestor
current := blockAtDepthMAtNextLevel
for {
ghostdagData, err := ppm.ghostdagDataStores[blockLevel].Get(ppm.databaseContext, stagingArea, current, false)
if err != nil {
return nil, err
}
current = ghostdagData.SelectedParent()
if current.Equal(model.VirtualGenesisBlockHash) {
return nil, errors.Errorf("No common ancestor between %s and %s at level %d", blockAtDepth2M, blockAtDepthMAtNextLevel, blockLevel)
}
isCurrentAncestorOfBlockAtDepth2M, err := ppm.dagTopologyManagers[blockLevel].IsAncestorOf(stagingArea, current, blockAtDepth2M)
if err != nil {
return nil, err
}
if isCurrentAncestorOfBlockAtDepth2M {
root = current
break
}
}
}
}
}
headers := make([]externalapi.BlockHeader, 0, 2*ppm.pruningProofM)
visited := hashset.New()
queue := ppm.dagTraversalManagers[blockLevel].NewUpHeap(stagingArea)
err = queue.Push(root)
if err != nil {
return nil, err
}
for queue.Len() > 0 {
current := queue.Pop()
if visited.Contains(current) {
continue
}
visited.Add(current)
isAncestorOfSelectedTip, err := ppm.dagTopologyManagers[blockLevel].IsAncestorOf(stagingArea, current, selectedTip)
if err != nil {
return nil, err
}
if !isAncestorOfSelectedTip {
continue
}
currentHeader, err := ppm.blockHeaderStore.BlockHeader(ppm.databaseContext, stagingArea, current)
if err != nil {
return nil, err
}
headers = append(headers, currentHeader)
children, err := ppm.dagTopologyManagers[blockLevel].Children(stagingArea, current)
if err != nil {
return nil, err
}
err = queue.PushSlice(children)
if err != nil {
return nil, err
}
}
headersByLevel[blockLevel] = headers
}
proof := &externalapi.PruningPointProof{Headers: make([][]externalapi.BlockHeader, len(headersByLevel))}
for i := 0; i < len(headersByLevel); i++ {
proof.Headers[i] = headersByLevel[i]
}
return proof, nil
}
func (ppm *pruningProofManager) blockAtDepth(stagingArea *model.StagingArea, ghostdagDataStore model.GHOSTDAGDataStore, highHash *externalapi.DomainHash, depth uint64) (*externalapi.DomainHash, error) {
currentBlockHash := highHash
highBlockGHOSTDAGData, err := ghostdagDataStore.Get(ppm.databaseContext, stagingArea, highHash, false)
if err != nil {
return nil, err
}
requiredBlueScore := uint64(0)
if highBlockGHOSTDAGData.BlueScore() > depth {
requiredBlueScore = highBlockGHOSTDAGData.BlueScore() - depth
}
currentBlockGHOSTDAGData := highBlockGHOSTDAGData
// If we used `BlockIterator` we'd need to do more calls to `ghostdagDataStore` so we can get the blueScore
for currentBlockGHOSTDAGData.BlueScore() >= requiredBlueScore {
if currentBlockGHOSTDAGData.SelectedParent().Equal(model.VirtualGenesisBlockHash) {
break
}
currentBlockHash = currentBlockGHOSTDAGData.SelectedParent()
currentBlockGHOSTDAGData, err = ghostdagDataStore.Get(ppm.databaseContext, stagingArea, currentBlockHash, false)
if err != nil {
return nil, err
}
}
return currentBlockHash, nil
}
func (ppm *pruningProofManager) ValidatePruningPointProof(pruningPointProof *externalapi.PruningPointProof) error {
stagingArea := model.NewStagingArea()
level0Headers := pruningPointProof.Headers[0]
pruningPointHeader := level0Headers[len(level0Headers)-1]
pruningPoint := consensushashing.HeaderHash(pruningPointHeader)
pruningPointBlockLevel := pow.BlockLevel(pruningPointHeader)
maxLevel := len(pruningPointHeader.Parents()) - 1
if maxLevel >= len(pruningPointProof.Headers) {
return errors.Wrapf(ruleerrors.ErrPruningProofMissingBlockLevels, "proof has only %d levels while pruning point "+
"has parents from %d levels", len(pruningPointProof.Headers), maxLevel+1)
}
blockHeaderStore, blockRelationStores, reachabilityDataStores, ghostdagDataStores, err := ppm.dagStores(maxLevel)
if err != nil {
return err
}
reachabilityManagers, dagTopologyManagers, ghostdagManagers := ppm.dagProcesses(maxLevel, blockHeaderStore, blockRelationStores, reachabilityDataStores, ghostdagDataStores)
for blockLevel := 0; blockLevel <= maxLevel; blockLevel++ {
err := reachabilityManagers[blockLevel].Init(stagingArea)
if err != nil {
return err
}
err = dagTopologyManagers[blockLevel].SetParents(stagingArea, model.VirtualGenesisBlockHash, nil)
if err != nil {
return err
}
ghostdagDataStores[blockLevel].Stage(stagingArea, model.VirtualGenesisBlockHash, externalapi.NewBlockGHOSTDAGData(
0,
big.NewInt(0),
nil,
nil,
nil,
nil,
), false)
}
selectedTipByLevel := make([]*externalapi.DomainHash, maxLevel+1)
for blockLevel := maxLevel; blockLevel >= 0; blockLevel-- {
headers := make([]externalapi.BlockHeader, len(pruningPointProof.Headers[blockLevel]))
copy(headers, pruningPointProof.Headers[blockLevel])
var selectedTip *externalapi.DomainHash
for i, header := range headers {
blockHash := consensushashing.HeaderHash(header)
if pow.BlockLevel(header) < blockLevel {
return errors.Wrapf(ruleerrors.ErrPruningProofWrongBlockLevel, "block %s level is %d when it's "+
"expected to be at least %d", blockHash, pow.BlockLevel(header), blockLevel)
}
blockHeaderStore.Stage(stagingArea, blockHash, header)
var parents []*externalapi.DomainHash
for _, parent := range header.ParentsAtLevel(blockLevel) {
_, err := ghostdagDataStores[blockLevel].Get(ppm.databaseContext, stagingArea, parent, false)
if database.IsNotFoundError(err) {
continue
}
if err != nil {
return err
}
parents = append(parents, parent)
}
if len(parents) == 0 {
if i != 0 {
return errors.Wrapf(ruleerrors.ErrPruningProofHeaderWithNoKnownParents, "the proof header "+
"%s is missing known parents", blockHash)
}
parents = append(parents, model.VirtualGenesisBlockHash)
}
err := dagTopologyManagers[blockLevel].SetParents(stagingArea, blockHash, parents)
if err != nil {
return err
}
err = ghostdagManagers[blockLevel].GHOSTDAG(stagingArea, blockHash)
if err != nil {
return err
}
if selectedTip == nil {
selectedTip = blockHash
} else {
selectedTip, err = ghostdagManagers[blockLevel].ChooseSelectedParent(stagingArea, selectedTip, blockHash)
if err != nil {
return err
}
}
err = reachabilityManagers[blockLevel].AddBlock(stagingArea, blockHash)
if err != nil {
return err
}
if selectedTip.Equal(blockHash) {
err := reachabilityManagers[blockLevel].UpdateReindexRoot(stagingArea, selectedTip)
if err != nil {
return err
}
}
}
if blockLevel < maxLevel {
blockAtDepthMAtNextLevel, err := ppm.blockAtDepth(stagingArea, ghostdagDataStores[blockLevel+1], selectedTipByLevel[blockLevel+1], ppm.pruningProofM)
if err != nil {
return err
}
hasBlockAtDepthMAtNextLevel, err := blockRelationStores[blockLevel].Has(ppm.databaseContext, stagingArea, blockAtDepthMAtNextLevel)
if err != nil {
return err
}
if !hasBlockAtDepthMAtNextLevel {
return errors.Wrapf(ruleerrors.ErrPruningProofMissingBlockAtDepthMFromNextLevel, "proof level %d "+
"is missing the block at depth m in level %d", blockLevel, blockLevel+1)
}
}
if !selectedTip.Equal(pruningPoint) && !pruningPointHeader.ParentsAtLevel(blockLevel).Contains(selectedTip) {
return errors.Wrapf(ruleerrors.ErrPruningProofMissesBlocksBelowPruningPoint, "the selected tip %s at "+
"level %d is not a parent of the pruning point", selectedTip, blockLevel)
}
selectedTipByLevel[blockLevel] = selectedTip
}
currentDAGPruningPoint, err := ppm.pruningStore.PruningPoint(ppm.databaseContext, model.NewStagingArea())
if err != nil {
return err
}
for blockLevel, selectedTip := range selectedTipByLevel {
if blockLevel <= pruningPointBlockLevel {
if !selectedTip.Equal(consensushashing.HeaderHash(pruningPointHeader)) {
return errors.Wrapf(ruleerrors.ErrPruningProofSelectedTipIsNotThePruningPoint, "the pruning "+
"proof selected tip %s at level %d is not the pruning point", selectedTip, blockLevel)
}
} else if !pruningPointHeader.ParentsAtLevel(blockLevel).Contains(selectedTip) {
return errors.Wrapf(ruleerrors.ErrPruningProofSelectedTipNotParentOfPruningPoint, "the pruning "+
"proof selected tip %s at level %d is not a parent of the of the pruning point on the same "+
"level", selectedTip, blockLevel)
}
selectedTipGHOSTDAGData, err := ghostdagDataStores[blockLevel].Get(ppm.databaseContext, stagingArea, selectedTip, false)
if err != nil {
return err
}
if selectedTipGHOSTDAGData.BlueScore() < 2*ppm.pruningProofM {
continue
}
current := selectedTip
currentGHOSTDAGData := selectedTipGHOSTDAGData
var commonAncestor *externalapi.DomainHash
var commonAncestorGHOSTDAGData *externalapi.BlockGHOSTDAGData
var currentDAGCommonAncestorGHOSTDAGData *externalapi.BlockGHOSTDAGData
for {
currentDAGHOSTDAGData, err := ppm.ghostdagDataStores[blockLevel].Get(ppm.databaseContext, model.NewStagingArea(), current, false)
if err == nil {
commonAncestor = current
commonAncestorGHOSTDAGData = currentGHOSTDAGData
currentDAGCommonAncestorGHOSTDAGData = currentDAGHOSTDAGData
break
}
if !database.IsNotFoundError(err) {
return err
}
current = currentGHOSTDAGData.SelectedParent()
if current.Equal(model.VirtualGenesisBlockHash) {
break
}
currentGHOSTDAGData, err = ghostdagDataStores[blockLevel].Get(ppm.databaseContext, stagingArea, current, false)
if err != nil {
return err
}
}
if commonAncestor != nil {
selectedTipBlueWorkDiff := big.NewInt(0).Sub(selectedTipGHOSTDAGData.BlueWork(), commonAncestorGHOSTDAGData.BlueWork())
currentDAGPruningPointParents, err := ppm.dagTopologyManagers[blockLevel].Parents(model.NewStagingArea(), currentDAGPruningPoint)
if err != nil {
return err
}
foundBetterParent := false
for _, parent := range currentDAGPruningPointParents {
parentGHOSTDAGData, err := ppm.ghostdagDataStores[blockLevel].Get(ppm.databaseContext, model.NewStagingArea(), parent, false)
if err != nil {
return err
}
parentBlueWorkDiff := big.NewInt(0).Sub(parentGHOSTDAGData.BlueWork(), currentDAGCommonAncestorGHOSTDAGData.BlueWork())
if parentBlueWorkDiff.Cmp(selectedTipBlueWorkDiff) >= 0 {
foundBetterParent = true
break
}
}
if foundBetterParent {
return errors.Wrapf(ruleerrors.ErrPruningProofInsufficientBlueWork, "the proof doesn't "+
"have sufficient blue work in order to replace the current DAG")
}
return nil
}
}
for blockLevel := maxLevel; blockLevel >= 0; blockLevel-- {
currentDAGPruningPointParents, err := ppm.dagTopologyManagers[blockLevel].Parents(model.NewStagingArea(), currentDAGPruningPoint)
// If the current pruning point doesn't have a parent at this level, we consider the proof state to be better.
if database.IsNotFoundError(err) {
return nil
}
if err != nil {
return err
}
for _, parent := range currentDAGPruningPointParents {
parentGHOSTDAGData, err := ppm.ghostdagDataStores[blockLevel].Get(ppm.databaseContext, model.NewStagingArea(), parent, false)
if err != nil {
return err
}
if parentGHOSTDAGData.BlueScore() < 2*ppm.pruningProofM {
return nil
}
}
}
return errors.Wrapf(ruleerrors.ErrPruningProofInsufficientBlueWork, "the pruning proof doesn't have any "+
"shared blocks with the known DAGs, but doesn't have enough headers from levels higher than the existing block levels.")
}
func (ppm *pruningProofManager) dagStores(maxLevel int) (model.BlockHeaderStore, []model.BlockRelationStore, []model.ReachabilityDataStore, []model.GHOSTDAGDataStore, error) {
blockRelationStores := make([]model.BlockRelationStore, maxLevel+1)
reachabilityDataStores := make([]model.ReachabilityDataStore, maxLevel+1)
ghostdagDataStores := make([]model.GHOSTDAGDataStore, maxLevel+1)
prefix := consensusDB.MakeBucket([]byte("pruningProofManager"))
blockHeaderStore, err := blockheaderstore.New(ppm.databaseContext, prefix, 0, false)
if err != nil {
return nil, nil, nil, nil, err
}
for i := 0; i <= maxLevel; i++ {
blockRelationStores[i] = blockrelationstore.New(prefix, 0, false)
reachabilityDataStores[i] = reachabilitydatastore.New(prefix, 0, false)
ghostdagDataStores[i] = ghostdagdatastore.New(prefix, 0, false)
}
return blockHeaderStore, blockRelationStores, reachabilityDataStores, ghostdagDataStores, nil
}
func (ppm *pruningProofManager) dagProcesses(
maxLevel int,
blockHeaderStore model.BlockHeaderStore,
blockRelationStores []model.BlockRelationStore,
reachabilityDataStores []model.ReachabilityDataStore,
ghostdagDataStores []model.GHOSTDAGDataStore) (
[]model.ReachabilityManager,
[]model.DAGTopologyManager,
[]model.GHOSTDAGManager,
) {
reachabilityManagers := make([]model.ReachabilityManager, constants.MaxBlockLevel+1)
dagTopologyManagers := make([]model.DAGTopologyManager, constants.MaxBlockLevel+1)
ghostdagManagers := make([]model.GHOSTDAGManager, constants.MaxBlockLevel+1)
for i := 0; i <= maxLevel; i++ {
reachabilityManagers[i] = reachabilitymanager.New(
ppm.databaseContext,
ghostdagDataStores[i],
reachabilityDataStores[i])
dagTopologyManagers[i] = dagtopologymanager.New(
ppm.databaseContext,
reachabilityManagers[i],
blockRelationStores[i],
ghostdagDataStores[i])
ghostdagManagers[i] = ghostdagmanager.New(
ppm.databaseContext,
dagTopologyManagers[i],
ghostdagDataStores[i],
blockHeaderStore,
ppm.k,
ppm.genesisHash)
}
return reachabilityManagers, dagTopologyManagers, ghostdagManagers
}
func (ppm *pruningProofManager) ApplyPruningPointProof(stagingArea *model.StagingArea, pruningPointProof *externalapi.PruningPointProof) error {
for blockLevel, headers := range pruningPointProof.Headers {
var selectedTip *externalapi.DomainHash
for i, header := range headers {
blockHash := consensushashing.HeaderHash(header)
if pow.BlockLevel(header) < blockLevel {
return errors.Wrapf(ruleerrors.ErrPruningProofWrongBlockLevel, "block %s level is %d when it's "+
"expected to be at least %d", blockHash, pow.BlockLevel(header), blockLevel)
}
ppm.blockHeaderStore.Stage(stagingArea, blockHash, header)
var parents []*externalapi.DomainHash
for _, parent := range header.ParentsAtLevel(blockLevel) {
_, err := ppm.ghostdagDataStores[blockLevel].Get(ppm.databaseContext, stagingArea, parent, false)
if database.IsNotFoundError(err) {
continue
}
if err != nil {
return err
}
parents = append(parents, parent)
}
if len(parents) == 0 {
if i != 0 {
return errors.Wrapf(ruleerrors.ErrPruningProofHeaderWithNoKnownParents, "the proof header "+
"%s is missing known parents", blockHash)
}
parents = append(parents, model.VirtualGenesisBlockHash)
}
err := ppm.dagTopologyManagers[blockLevel].SetParents(stagingArea, blockHash, parents)
if err != nil {
return err
}
err = ppm.ghostdagManagers[blockLevel].GHOSTDAG(stagingArea, blockHash)
if err != nil {
return err
}
if blockLevel == 0 {
// Override the ghostdag data with the real blue score and blue work
ghostdagData, err := ppm.ghostdagDataStores[0].Get(ppm.databaseContext, stagingArea, blockHash, false)
if err != nil {
return err
}
ppm.ghostdagDataStores[0].Stage(stagingArea, blockHash, externalapi.NewBlockGHOSTDAGData(
header.BlueScore(),
header.BlueWork(),
ghostdagData.SelectedParent(),
ghostdagData.MergeSetBlues(),
ghostdagData.MergeSetReds(),
ghostdagData.BluesAnticoneSizes(),
), false)
ppm.finalityStore.StageFinalityPoint(stagingArea, blockHash, model.VirtualGenesisBlockHash)
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
}
}
}
}
pruningPointHeader := pruningPointProof.Headers[0][len(pruningPointProof.Headers[0])-1]
pruningPoint := consensushashing.HeaderHash(pruningPointHeader)
ppm.consensusStateStore.StageTips(stagingArea, []*externalapi.DomainHash{pruningPoint})
return nil
}

View File

@ -228,6 +228,15 @@ var (
// ErrUnexpectedFinalityPoint indicates a block header pruning point does not align with // ErrUnexpectedFinalityPoint indicates a block header pruning point does not align with
// the expected value. // the expected value.
ErrUnexpectedHeaderPruningPoint = newRuleError("ErrUnexpectedHeaderPruningPoint") ErrUnexpectedHeaderPruningPoint = newRuleError("ErrUnexpectedHeaderPruningPoint")
ErrPruningProofHeaderWithNoKnownParents = newRuleError("ErrPruningProofHeaderWithNoKnownParents")
ErrPruningProofMissingBlockLevels = newRuleError("ErrPruningProofMissingBlockLevels")
ErrPruningProofWrongBlockLevel = newRuleError("ErrPruningProofWrongBlockLevel")
ErrPruningProofSelectedTipNotParentOfPruningPoint = newRuleError("ErrPruningProofSelectedTipNotParentOfPruningPoint")
ErrPruningProofSelectedTipIsNotThePruningPoint = newRuleError("ErrPruningProofSelectedTipIsNotThePruningPoint")
ErrPruningProofInsufficientBlueWork = newRuleError("ErrPruningProofInsufficientBlueWork")
ErrPruningProofMissingBlockAtDepthMFromNextLevel = newRuleError("ErrPruningProofMissingBlockAtDepthMFromNextLevel")
ErrPruningProofMissesBlocksBelowPruningPoint = newRuleError("ErrPruningProofMissesBlocksBelowPruningPoint")
) )
// RuleError identifies a rule violation. It is used to indicate that // RuleError identifies a rule violation. It is used to indicate that

View File

@ -24,7 +24,7 @@ func (tc *testConsensus) BlockHeaderStore() model.BlockHeaderStore {
} }
func (tc *testConsensus) BlockRelationStore() model.BlockRelationStore { func (tc *testConsensus) BlockRelationStore() model.BlockRelationStore {
return tc.blockRelationStore return tc.blockRelationStores[0]
} }
func (tc *testConsensus) BlockStatusStore() model.BlockStatusStore { func (tc *testConsensus) BlockStatusStore() model.BlockStatusStore {
@ -40,7 +40,11 @@ func (tc *testConsensus) ConsensusStateStore() model.ConsensusStateStore {
} }
func (tc *testConsensus) GHOSTDAGDataStore() model.GHOSTDAGDataStore { func (tc *testConsensus) GHOSTDAGDataStore() model.GHOSTDAGDataStore {
return tc.ghostdagDataStore return tc.ghostdagDataStores[0]
}
func (tc *testConsensus) GHOSTDAGDataStores() []model.GHOSTDAGDataStore {
return tc.ghostdagDataStores
} }
func (tc *testConsensus) HeaderTipsStore() model.HeaderSelectedTipStore { func (tc *testConsensus) HeaderTipsStore() model.HeaderSelectedTipStore {
@ -56,7 +60,7 @@ func (tc *testConsensus) PruningStore() model.PruningStore {
} }
func (tc *testConsensus) ReachabilityDataStore() model.ReachabilityDataStore { func (tc *testConsensus) ReachabilityDataStore() model.ReachabilityDataStore {
return tc.reachabilityDataStore return tc.reachabilityDataStores[0]
} }
func (tc *testConsensus) UTXODiffStore() model.UTXODiffStore { func (tc *testConsensus) UTXODiffStore() model.UTXODiffStore {
@ -84,7 +88,7 @@ func (tc *testConsensus) ConsensusStateManager() testapi.TestConsensusStateManag
} }
func (tc *testConsensus) DAGTopologyManager() model.DAGTopologyManager { func (tc *testConsensus) DAGTopologyManager() model.DAGTopologyManager {
return tc.dagTopologyManager return tc.dagTopologyManagers[0]
} }
func (tc *testConsensus) DAGTraversalManager() model.DAGTraversalManager { func (tc *testConsensus) DAGTraversalManager() model.DAGTraversalManager {
@ -96,7 +100,7 @@ func (tc *testConsensus) DifficultyManager() model.DifficultyManager {
} }
func (tc *testConsensus) GHOSTDAGManager() model.GHOSTDAGManager { func (tc *testConsensus) GHOSTDAGManager() model.GHOSTDAGManager {
return tc.ghostdagManager return tc.ghostdagManagers[0]
} }
func (tc *testConsensus) HeaderTipsManager() model.HeadersSelectedTipManager { func (tc *testConsensus) HeaderTipsManager() model.HeadersSelectedTipManager {

View File

@ -38,7 +38,7 @@ func (tc *testConsensus) convertToDot() (string, error) {
} }
dotScriptBuilder.WriteString(fmt.Sprintf("\t\"%s\";\n", hash)) dotScriptBuilder.WriteString(fmt.Sprintf("\t\"%s\";\n", hash))
parents, err := tc.dagTopologyManager.Parents(stagingArea, hash) parents, err := tc.dagTopologyManagers[0].Parents(stagingArea, hash)
if err != nil { if err != nil {
return "", err return "", err
} }

View File

@ -35,4 +35,7 @@ const (
// LockTimeThreshold is the number below which a lock time is // LockTimeThreshold is the number below which a lock time is
// interpreted to be a DAA score. // interpreted to be a DAA score.
LockTimeThreshold = 5e11 // Tue Nov 5 00:53:20 1985 UTC LockTimeThreshold = 5e11 // Tue Nov 5 00:53:20 1985 UTC
// MaxBlockLevel is the maximum possible block level.
MaxBlockLevel = 255
) )

View File

@ -3,6 +3,7 @@ package pow
import ( import (
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi" "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/kaspanet/kaspad/domain/consensus/utils/consensushashing" "github.com/kaspanet/kaspad/domain/consensus/utils/consensushashing"
"github.com/kaspanet/kaspad/domain/consensus/utils/constants"
"github.com/kaspanet/kaspad/domain/consensus/utils/hashes" "github.com/kaspanet/kaspad/domain/consensus/utils/hashes"
"github.com/kaspanet/kaspad/domain/consensus/utils/serialization" "github.com/kaspanet/kaspad/domain/consensus/utils/serialization"
"github.com/kaspanet/kaspad/util/difficulty" "github.com/kaspanet/kaspad/util/difficulty"
@ -65,3 +66,13 @@ func toBig(hash *externalapi.DomainHash) *big.Int {
return new(big.Int).SetBytes(buf) return new(big.Int).SetBytes(buf)
} }
// BlockLevel returns the block level of the given header.
func BlockLevel(header externalapi.BlockHeader) int {
proofOfWorkValue := CalculateProofOfWorkValue(header.ToMutable())
for blockLevel := 0; ; blockLevel++ {
if blockLevel == constants.MaxBlockLevel || proofOfWorkValue.Bit(blockLevel+1) != 0 {
return blockLevel
}
}
}

View File

@ -71,4 +71,6 @@ const (
// Should be parametrized such that the average width of the DAG is about defaultMaxBlockParents and such that most of the // Should be parametrized such that the average width of the DAG is about defaultMaxBlockParents and such that most of the
// time the width of the DAG is at most defaultGHOSTDAGK. // time the width of the DAG is at most defaultGHOSTDAGK.
defaultTargetTimePerBlock = 1 * time.Second defaultTargetTimePerBlock = 1 * time.Second
defaultPruningProofM = 1000
) )

View File

@ -174,6 +174,9 @@ type Params struct {
// BaseSubsidy is the starting subsidy amount for mined blocks. // BaseSubsidy is the starting subsidy amount for mined blocks.
BaseSubsidy uint64 BaseSubsidy uint64
// PruningProofM is the 'm' constant in the pruning proof. For more details see: https://github.com/kaspanet/research/issues/3
PruningProofM uint64
} }
// NormalizeRPCServerAddress returns addr with the current network default // NormalizeRPCServerAddress returns addr with the current network default
@ -245,6 +248,7 @@ var MainnetParams = Params{
MergeSetSizeLimit: defaultMergeSetSizeLimit, MergeSetSizeLimit: defaultMergeSetSizeLimit,
BaseSubsidy: defaultBaseSubsidy, BaseSubsidy: defaultBaseSubsidy,
CoinbasePayloadScriptPublicKeyMaxLength: defaultCoinbasePayloadScriptPublicKeyMaxLength, CoinbasePayloadScriptPublicKeyMaxLength: defaultCoinbasePayloadScriptPublicKeyMaxLength,
PruningProofM: defaultPruningProofM,
} }
// TestnetParams defines the network parameters for the test Kaspa network. // TestnetParams defines the network parameters for the test Kaspa network.
@ -301,6 +305,7 @@ var TestnetParams = Params{
MergeSetSizeLimit: defaultMergeSetSizeLimit, MergeSetSizeLimit: defaultMergeSetSizeLimit,
BaseSubsidy: defaultBaseSubsidy, BaseSubsidy: defaultBaseSubsidy,
CoinbasePayloadScriptPublicKeyMaxLength: defaultCoinbasePayloadScriptPublicKeyMaxLength, CoinbasePayloadScriptPublicKeyMaxLength: defaultCoinbasePayloadScriptPublicKeyMaxLength,
PruningProofM: defaultPruningProofM,
} }
// SimnetParams defines the network parameters for the simulation test Kaspa // SimnetParams defines the network parameters for the simulation test Kaspa
@ -361,6 +366,7 @@ var SimnetParams = Params{
MergeSetSizeLimit: defaultMergeSetSizeLimit, MergeSetSizeLimit: defaultMergeSetSizeLimit,
BaseSubsidy: defaultBaseSubsidy, BaseSubsidy: defaultBaseSubsidy,
CoinbasePayloadScriptPublicKeyMaxLength: defaultCoinbasePayloadScriptPublicKeyMaxLength, CoinbasePayloadScriptPublicKeyMaxLength: defaultCoinbasePayloadScriptPublicKeyMaxLength,
PruningProofM: defaultPruningProofM,
} }
// DevnetParams defines the network parameters for the development Kaspa network. // DevnetParams defines the network parameters for the development Kaspa network.
@ -417,6 +423,7 @@ var DevnetParams = Params{
MergeSetSizeLimit: defaultMergeSetSizeLimit, MergeSetSizeLimit: defaultMergeSetSizeLimit,
BaseSubsidy: defaultBaseSubsidy, BaseSubsidy: defaultBaseSubsidy,
CoinbasePayloadScriptPublicKeyMaxLength: defaultCoinbasePayloadScriptPublicKeyMaxLength, CoinbasePayloadScriptPublicKeyMaxLength: defaultCoinbasePayloadScriptPublicKeyMaxLength,
PruningProofM: defaultPruningProofM,
} }
var ( var (
@ -461,4 +468,5 @@ func init() {
mustRegister(&MainnetParams) mustRegister(&MainnetParams)
mustRegister(&TestnetParams) mustRegister(&TestnetParams)
mustRegister(&SimnetParams) mustRegister(&SimnetParams)
mustRegister(&DevnetParams)
} }

View File

@ -91,6 +91,8 @@ func (c *ConnectionManager) Stop() {
} }
c.loopTicker.Stop() c.loopTicker.Stop()
// Force the next iteration so the connection loop will stop immediately and not after `connectionsLoopInterval`.
c.run()
} }
func (c *ConnectionManager) run() { func (c *ConnectionManager) run() {

View File

@ -18,7 +18,7 @@ type p2pServer struct {
gRPCServer gRPCServer
} }
const p2pMaxMessageSize = 10 * 1024 * 1024 // 10MB const p2pMaxMessageSize = 100 * 1024 * 1024 // 100MB
// p2pMaxInboundConnections is the max amount of inbound connections for the P2P server. // p2pMaxInboundConnections is the max amount of inbound connections for the P2P server.
// Note that inbound connections are not limited by the gRPC server. (A value of 0 means // Note that inbound connections are not limited by the gRPC server. (A value of 0 means

View File

@ -9,9 +9,9 @@ import (
"github.com/kaspanet/kaspad/app/appmessage" "github.com/kaspanet/kaspad/app/appmessage"
) )
func Test64IncomingConnections(t *testing.T) { func Test16IncomingConnections(t *testing.T) {
// Much more than 64 hosts creates a risk of running out of available file descriptors for leveldb // Much more than 16 hosts creates a risk of running out of available file descriptors for leveldb
const numBullies = 64 const numBullies = 16
harnessesParams := make([]*harnessParams, numBullies+1) harnessesParams := make([]*harnessParams, numBullies+1)
for i := 0; i < numBullies+1; i++ { for i := 0; i < numBullies+1; i++ {
harnessesParams[i] = &harnessParams{ harnessesParams[i] = &harnessParams{

View File

@ -135,6 +135,7 @@ func TestIBDWithPruning(t *testing.T) {
// This is done to make a pruning depth of 6 blocks // This is done to make a pruning depth of 6 blocks
overrideDAGParams.FinalityDuration = 2 * overrideDAGParams.TargetTimePerBlock overrideDAGParams.FinalityDuration = 2 * overrideDAGParams.TargetTimePerBlock
overrideDAGParams.K = 0 overrideDAGParams.K = 0
overrideDAGParams.PruningProofM = 20
expectedPruningDepth := uint64(6) expectedPruningDepth := uint64(6)
if overrideDAGParams.PruningDepth() != expectedPruningDepth { if overrideDAGParams.PruningDepth() != expectedPruningDepth {

View File

@ -1,10 +1,9 @@
package integration package integration
import ( import (
"github.com/kaspanet/kaspad/infrastructure/logger"
"os" "os"
"testing" "testing"
"github.com/kaspanet/kaspad/infrastructure/logger"
) )
func TestMain(m *testing.M) { func TestMain(m *testing.M) {