mirror of
https://github.com/kaspanet/kaspad.git
synced 2025-06-03 20:56:42 +00:00
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:
parent
afaac28da1
commit
5dbb1da84b
2
.github/workflows/tests.yaml
vendored
2
.github/workflows/tests.yaml
vendored
@ -14,7 +14,7 @@ jobs:
|
||||
strategy:
|
||||
fail-fast: false
|
||||
matrix:
|
||||
os: [ ubuntu-latest, macos-latest, windows-latest ]
|
||||
os: [ ubuntu-latest, macos-latest ]
|
||||
name: Tests, ${{ matrix.os }}
|
||||
steps:
|
||||
|
||||
|
@ -248,8 +248,8 @@ func (flow *handleRelayInvsFlow) syncPruningPointFutureHeaders(consensus externa
|
||||
}
|
||||
return nil
|
||||
}
|
||||
for _, block := range ibdBlocksMessage.BlockHeaders {
|
||||
err = flow.processHeader(consensus, block)
|
||||
for _, header := range ibdBlocksMessage.BlockHeaders {
|
||||
err = flow.processHeader(consensus, header)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -7,6 +7,7 @@ import (
|
||||
"github.com/kaspanet/kaspad/app/protocol/protocolerrors"
|
||||
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
|
||||
"github.com/kaspanet/kaspad/domain/consensus/ruleerrors"
|
||||
"github.com/kaspanet/kaspad/domain/consensus/utils/consensushashing"
|
||||
"github.com/pkg/errors"
|
||||
)
|
||||
|
||||
@ -71,51 +72,73 @@ func (flow *handleRelayInvsFlow) checkIfHighHashHasMoreBlueWorkThanSelectedTip(h
|
||||
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)
|
||||
err := flow.outgoingRoute.Enqueue(appmessage.NewMsgRequestPruningPointProof())
|
||||
if err != nil {
|
||||
return err
|
||||
return nil, err
|
||||
}
|
||||
message, err := flow.dequeueIncomingMessageAndSkipInvs(common.DefaultTimeout)
|
||||
if err != nil {
|
||||
return err
|
||||
return nil, err
|
||||
}
|
||||
pruningPointProofMessage, ok := message.(*appmessage.MsgPruningPointProof)
|
||||
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())
|
||||
}
|
||||
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 {
|
||||
err := flow.syncAndValidatePruningPointProof()
|
||||
proofPruningPoint, err := flow.syncAndValidatePruningPointProof()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
pruningPoint, err := flow.syncPruningPointsAndPruningPointAnticone()
|
||||
err = flow.syncPruningPointsAndPruningPointAnticone(proofPruningPoint)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// TODO: Remove this condition once there's more proper way to check finality violation
|
||||
// in the headers proof.
|
||||
if pruningPoint.Equal(flow.Config().NetParams().GenesisHash) {
|
||||
if proofPruningPoint.Equal(flow.Config().NetParams().GenesisHash) {
|
||||
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 {
|
||||
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")
|
||||
syncedPruningPointUTXOSetSuccessfully, err := flow.syncPruningPointUTXOSet(flow.Domain().StagingConsensus(), pruningPoint)
|
||||
syncedPruningPointUTXOSetSuccessfully, err := flow.syncPruningPointUTXOSet(flow.Domain().StagingConsensus(), proofPruningPoint)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -127,36 +150,40 @@ func (flow *handleRelayInvsFlow) downloadHeadersAndPruningUTXOSet(highHash *exte
|
||||
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)
|
||||
err := flow.outgoingRoute.Enqueue(appmessage.NewMsgRequestPruningPointAndItsAnticone())
|
||||
if err != nil {
|
||||
return nil, err
|
||||
return err
|
||||
}
|
||||
|
||||
err = flow.validateAndInsertPruningPoints()
|
||||
err = flow.validateAndInsertPruningPoints(proofPruningPoint)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
return err
|
||||
}
|
||||
|
||||
pruningPoint, done, err := flow.receiveBlockWithTrustedData()
|
||||
pruningPointWithMetaData, done, err := flow.receiveBlockWithTrustedData()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
return err
|
||||
}
|
||||
|
||||
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 {
|
||||
return nil, err
|
||||
return err
|
||||
}
|
||||
|
||||
for {
|
||||
blockWithTrustedData, done, err := flow.receiveBlockWithTrustedData()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
return err
|
||||
}
|
||||
|
||||
if done {
|
||||
@ -165,12 +192,12 @@ func (flow *handleRelayInvsFlow) syncPruningPointsAndPruningPointAnticone() (*ex
|
||||
|
||||
err = flow.processBlockWithTrustedData(flow.Domain().StagingConsensus(), blockWithTrustedData)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
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(
|
||||
@ -217,7 +244,16 @@ func (flow *handleRelayInvsFlow) receivePruningPoints() (*appmessage.MsgPruningP
|
||||
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()
|
||||
if err != nil {
|
||||
return err
|
||||
@ -238,6 +274,12 @@ func (flow *handleRelayInvsFlow) validateAndInsertPruningPoints() error {
|
||||
return protocolerrors.Errorf(false, "pruning points are violating finality")
|
||||
}
|
||||
|
||||
lastPruningPoint := consensushashing.HeaderHash(headers[len(headers)-1])
|
||||
if !lastPruningPoint.Equal(proofPruningPoint) {
|
||||
return protocolerrors.Errorf(true, "the proof pruning point is not equal to the last pruning "+
|
||||
"point in the list")
|
||||
}
|
||||
|
||||
err = flow.Domain().StagingConsensus().ImportPruningPoints(headers)
|
||||
if err != nil {
|
||||
return err
|
||||
|
@ -28,27 +28,28 @@ type consensus struct {
|
||||
pastMedianTimeManager model.PastMedianTimeManager
|
||||
blockValidator model.BlockValidator
|
||||
coinbaseManager model.CoinbaseManager
|
||||
dagTopologyManager model.DAGTopologyManager
|
||||
dagTopologyManagers []model.DAGTopologyManager
|
||||
dagTraversalManager model.DAGTraversalManager
|
||||
difficultyManager model.DifficultyManager
|
||||
ghostdagManager model.GHOSTDAGManager
|
||||
ghostdagManagers []model.GHOSTDAGManager
|
||||
headerTipsManager model.HeadersSelectedTipManager
|
||||
mergeDepthManager model.MergeDepthManager
|
||||
pruningManager model.PruningManager
|
||||
reachabilityManager model.ReachabilityManager
|
||||
reachabilityManagers []model.ReachabilityManager
|
||||
finalityManager model.FinalityManager
|
||||
pruningProofManager model.PruningProofManager
|
||||
|
||||
acceptanceDataStore model.AcceptanceDataStore
|
||||
blockStore model.BlockStore
|
||||
blockHeaderStore model.BlockHeaderStore
|
||||
pruningStore model.PruningStore
|
||||
ghostdagDataStore model.GHOSTDAGDataStore
|
||||
blockRelationStore model.BlockRelationStore
|
||||
ghostdagDataStores []model.GHOSTDAGDataStore
|
||||
blockRelationStores []model.BlockRelationStore
|
||||
blockStatusStore model.BlockStatusStore
|
||||
consensusStateStore model.ConsensusStateStore
|
||||
headersSelectedTipStore model.HeaderSelectedTipStore
|
||||
multisetStore model.MultisetStore
|
||||
reachabilityDataStore model.ReachabilityDataStore
|
||||
reachabilityDataStores []model.ReachabilityDataStore
|
||||
utxoDiffStore model.UTXODiffStore
|
||||
finalityStore model.FinalityStore
|
||||
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.
|
||||
if !exists {
|
||||
s.blockStatusStore.Stage(stagingArea, model.VirtualGenesisBlockHash, externalapi.StatusUTXOValid)
|
||||
err = s.reachabilityManager.Init(stagingArea)
|
||||
if err != nil {
|
||||
return err
|
||||
for _, reachabilityManager := range s.reachabilityManagers {
|
||||
err = reachabilityManager.Init(stagingArea)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
err = s.dagTopologyManager.SetParents(stagingArea, model.VirtualGenesisBlockHash, nil)
|
||||
if err != nil {
|
||||
return err
|
||||
for _, dagTopologyManager := range s.dagTopologyManagers {
|
||||
err = dagTopologyManager.SetParents(stagingArea, model.VirtualGenesisBlockHash, nil)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
s.consensusStateStore.StageTips(stagingArea, []*externalapi.DomainHash{model.VirtualGenesisBlockHash})
|
||||
s.ghostdagDataStore.Stage(stagingArea, model.VirtualGenesisBlockHash, externalapi.NewBlockGHOSTDAGData(
|
||||
0,
|
||||
big.NewInt(0),
|
||||
nil,
|
||||
nil,
|
||||
nil,
|
||||
nil,
|
||||
), false)
|
||||
for _, ghostdagDataStore := range s.ghostdagDataStores {
|
||||
ghostdagDataStore.Stage(stagingArea, model.VirtualGenesisBlockHash, externalapi.NewBlockGHOSTDAGData(
|
||||
0,
|
||||
big.NewInt(0),
|
||||
nil,
|
||||
nil,
|
||||
nil,
|
||||
nil,
|
||||
), false)
|
||||
}
|
||||
|
||||
err = staging.CommitAllChanges(s.databaseContext, stagingArea)
|
||||
if err != nil {
|
||||
@ -267,7 +274,7 @@ func (s *consensus) GetBlockInfo(blockHash *externalapi.DomainHash) (*externalap
|
||||
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 {
|
||||
return nil, err
|
||||
}
|
||||
@ -287,12 +294,12 @@ func (s *consensus) GetBlockRelations(blockHash *externalapi.DomainHash) (
|
||||
|
||||
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 {
|
||||
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 {
|
||||
return nil, nil, nil, err
|
||||
}
|
||||
@ -382,7 +389,7 @@ func (s *consensus) GetVirtualUTXOs(expectedVirtualParents []*externalapi.Domain
|
||||
|
||||
stagingArea := model.NewStagingArea()
|
||||
|
||||
virtualParents, err := s.dagTopologyManager.Parents(stagingArea, model.VirtualBlockHash)
|
||||
virtualParents, err := s.dagTopologyManagers[0].Parents(stagingArea, model.VirtualBlockHash)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@ -465,7 +472,7 @@ func (s *consensus) GetVirtualSelectedParent() (*externalapi.DomainHash, error)
|
||||
|
||||
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 {
|
||||
return nil, err
|
||||
}
|
||||
@ -487,7 +494,7 @@ func (s *consensus) GetVirtualInfo() (*externalapi.VirtualInfo, error) {
|
||||
|
||||
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 {
|
||||
return nil, err
|
||||
}
|
||||
@ -499,7 +506,7 @@ func (s *consensus) GetVirtualInfo() (*externalapi.VirtualInfo, error) {
|
||||
if err != nil {
|
||||
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 {
|
||||
return nil, err
|
||||
}
|
||||
@ -664,7 +671,7 @@ func (s *consensus) IsInSelectedParentChainOf(blockHashA *externalapi.DomainHash
|
||||
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) {
|
||||
@ -687,7 +694,12 @@ func (s *consensus) Anticone(blockHash *externalapi.DomainHash) ([]*externalapi.
|
||||
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) {
|
||||
@ -727,18 +739,27 @@ func (s *consensus) BuildPruningPointProof() (*externalapi.PruningPointProof, er
|
||||
s.lock.Lock()
|
||||
defer s.lock.Unlock()
|
||||
|
||||
// TODO: Implement this
|
||||
|
||||
return &externalapi.PruningPointProof{
|
||||
Headers: [][]externalapi.BlockHeader{},
|
||||
}, nil
|
||||
return s.pruningProofManager.BuildPruningPointProof(model.NewStagingArea())
|
||||
}
|
||||
|
||||
func (s *consensus) ValidatePruningPointProof(pruningPointProof *externalapi.PruningPointProof) error {
|
||||
s.lock.Lock()
|
||||
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
|
||||
}
|
||||
|
@ -7,7 +7,9 @@ import (
|
||||
"github.com/kaspanet/kaspad/domain/consensus/model"
|
||||
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
|
||||
"github.com/kaspanet/kaspad/domain/consensus/utils/lrucachehashpairtoblockghostdagdatahashpair"
|
||||
"github.com/kaspanet/kaspad/infrastructure/db/database"
|
||||
"github.com/kaspanet/kaspad/util/staging"
|
||||
"github.com/pkg/errors"
|
||||
)
|
||||
|
||||
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) {
|
||||
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 == nil {
|
||||
return nil, errDAAWindowBlockNotFound
|
||||
}
|
||||
|
||||
return pair, nil
|
||||
}
|
||||
|
||||
pairBytes, err := dbContext.Get(daaws.key(dbKey))
|
||||
if database.IsNotFoundError(err) {
|
||||
daaws.cache.Add(blockHash, index, nil)
|
||||
}
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -2,7 +2,10 @@ package consensus
|
||||
|
||||
import (
|
||||
"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/pruningproofmanager"
|
||||
"github.com/kaspanet/kaspad/domain/consensus/utils/constants"
|
||||
"io/ioutil"
|
||||
"os"
|
||||
"sync"
|
||||
@ -16,7 +19,7 @@ import (
|
||||
"github.com/kaspanet/kaspad/domain/consensus/datastructures/blockstatusstore"
|
||||
"github.com/kaspanet/kaspad/domain/consensus/datastructures/blockstore"
|
||||
"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/ghostdagdatastore"
|
||||
"github.com/kaspanet/kaspad/domain/consensus/datastructures/headersselectedchainstore"
|
||||
@ -130,39 +133,31 @@ func (f *factory) NewConsensus(config *Config, db infrastructuredatabase.Databas
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
blockRelationStore := blockrelationstore.New(prefixBucket, pruningWindowSizePlusFinalityDepthForCache, preallocateCaches)
|
||||
|
||||
blockStatusStore := blockstatusstore.New(prefixBucket, pruningWindowSizePlusFinalityDepthForCache, preallocateCaches)
|
||||
multisetStore := multisetstore.New(prefixBucket, 200, preallocateCaches)
|
||||
pruningStore := pruningstore.New(prefixBucket, 2, preallocateCaches)
|
||||
reachabilityDataStore := reachabilitydatastore.New(prefixBucket, pruningWindowSizePlusFinalityDepthForCache, preallocateCaches)
|
||||
utxoDiffStore := utxodiffstore.New(prefixBucket, 200, 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)
|
||||
finalityStore := finalitystore.New(prefixBucket, 200, preallocateCaches)
|
||||
headersSelectedChainStore := headersselectedchainstore.New(prefixBucket, pruningWindowSizeForCaches, 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
|
||||
reachabilityManager := reachabilitymanager.New(
|
||||
dbManager,
|
||||
ghostdagDataStore,
|
||||
reachabilityDataStore)
|
||||
dagTopologyManager := dagtopologymanager.New(
|
||||
dbManager,
|
||||
reachabilityManager,
|
||||
blockRelationStore,
|
||||
ghostdagDataStore)
|
||||
blockParentBuilder := blockparentbuilder.New(
|
||||
dbManager,
|
||||
blockHeaderStore,
|
||||
@ -170,22 +165,6 @@ func (f *factory) NewConsensus(config *Config, db infrastructuredatabase.Databas
|
||||
reachabilityDataStore,
|
||||
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(
|
||||
config.TimestampDeviationTolerance,
|
||||
dbManager,
|
||||
@ -318,19 +297,19 @@ func (f *factory) NewConsensus(config *Config, db infrastructuredatabase.Databas
|
||||
difficultyManager,
|
||||
pastMedianTimeManager,
|
||||
transactionValidator,
|
||||
ghostdagManager,
|
||||
dagTopologyManager,
|
||||
ghostdagManagers,
|
||||
dagTopologyManagers,
|
||||
dagTraversalManager,
|
||||
coinbaseManager,
|
||||
mergeDepthManager,
|
||||
reachabilityManager,
|
||||
reachabilityManagers,
|
||||
finalityManager,
|
||||
blockParentBuilder,
|
||||
pruningManager,
|
||||
|
||||
pruningStore,
|
||||
blockStore,
|
||||
ghostdagDataStore,
|
||||
ghostdagDataStores,
|
||||
blockHeaderStore,
|
||||
blockStatusStore,
|
||||
reachabilityDataStore,
|
||||
@ -408,6 +387,25 @@ func (f *factory) NewConsensus(config *Config, db infrastructuredatabase.Databas
|
||||
daaBlocksStore,
|
||||
daaWindowStore)
|
||||
|
||||
pruningProofManager := pruningproofmanager.New(
|
||||
dbManager,
|
||||
dagTopologyManagers,
|
||||
ghostdagManagers,
|
||||
reachabilityManagers,
|
||||
dagTraversalManagers,
|
||||
|
||||
ghostdagDataStores,
|
||||
pruningStore,
|
||||
blockHeaderStore,
|
||||
blockStatusStore,
|
||||
finalityStore,
|
||||
consensusStateStore,
|
||||
|
||||
genesisHash,
|
||||
config.K,
|
||||
config.PruningProofM,
|
||||
)
|
||||
|
||||
c := &consensus{
|
||||
lock: &sync.Mutex{},
|
||||
databaseContext: dbManager,
|
||||
@ -423,27 +421,28 @@ func (f *factory) NewConsensus(config *Config, db infrastructuredatabase.Databas
|
||||
pastMedianTimeManager: pastMedianTimeManager,
|
||||
blockValidator: blockValidator,
|
||||
coinbaseManager: coinbaseManager,
|
||||
dagTopologyManager: dagTopologyManager,
|
||||
dagTopologyManagers: dagTopologyManagers,
|
||||
dagTraversalManager: dagTraversalManager,
|
||||
difficultyManager: difficultyManager,
|
||||
ghostdagManager: ghostdagManager,
|
||||
ghostdagManagers: ghostdagManagers,
|
||||
headerTipsManager: headerTipsManager,
|
||||
mergeDepthManager: mergeDepthManager,
|
||||
pruningManager: pruningManager,
|
||||
reachabilityManager: reachabilityManager,
|
||||
reachabilityManagers: reachabilityManagers,
|
||||
finalityManager: finalityManager,
|
||||
pruningProofManager: pruningProofManager,
|
||||
|
||||
acceptanceDataStore: acceptanceDataStore,
|
||||
blockStore: blockStore,
|
||||
blockHeaderStore: blockHeaderStore,
|
||||
pruningStore: pruningStore,
|
||||
ghostdagDataStore: ghostdagDataStore,
|
||||
ghostdagDataStores: ghostdagDataStores,
|
||||
blockStatusStore: blockStatusStore,
|
||||
blockRelationStore: blockRelationStore,
|
||||
blockRelationStores: blockRelationStores,
|
||||
consensusStateStore: consensusStateStore,
|
||||
headersSelectedTipStore: headersSelectedTipStore,
|
||||
multisetStore: multisetStore,
|
||||
reachabilityDataStore: reachabilityDataStore,
|
||||
reachabilityDataStores: reachabilityDataStores,
|
||||
utxoDiffStore: utxoDiffStore,
|
||||
finalityStore: finalityStore,
|
||||
headersSelectedChainStore: headersSelectedChainStore,
|
||||
@ -510,7 +509,7 @@ func (f *factory) NewTestConsensus(config *Config, testName string) (
|
||||
database: db,
|
||||
testConsensusStateManager: testConsensusStateManager,
|
||||
testReachabilityManager: reachabilitymanager.NewTestReachabilityManager(consensusAsImplementation.
|
||||
reachabilityManager),
|
||||
reachabilityManagers[0]),
|
||||
testTransactionValidator: testTransactionValidator,
|
||||
}
|
||||
tstConsensus.testBlockBuilder = blockbuilder.NewTestBlockBuilder(consensusAsImplementation.blockBuilder, tstConsensus)
|
||||
@ -549,3 +548,84 @@ func (f *factory) SetTestLevelDBCacheSize(cacheSizeMiB int) {
|
||||
func (f *factory) SetTestPreAllocateCache(preallocateCaches bool) {
|
||||
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
|
||||
}
|
||||
|
@ -10,6 +10,7 @@ type Consensus interface {
|
||||
ImportPruningPoints(pruningPoints []BlockHeader) error
|
||||
BuildPruningPointProof() (*PruningPointProof, error)
|
||||
ValidatePruningPointProof(pruningPointProof *PruningPointProof) error
|
||||
ApplyPruningPointProof(pruningPointProof *PruningPointProof) error
|
||||
|
||||
GetBlock(blockHash *DomainHash) (*DomainBlock, error)
|
||||
GetBlockEvenIfHeaderOnly(blockHash *DomainHash) (*DomainBlock, error)
|
||||
|
@ -5,13 +5,12 @@ import "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
|
||||
// DAGTraversalManager exposes methods for traversing blocks
|
||||
// in the DAG
|
||||
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)
|
||||
// SelectedChildIterator should return a BlockIterator that iterates
|
||||
// from lowHash (exclusive) to highHash (inclusive) over highHash's selected parent chain
|
||||
SelectedChildIterator(stagingArea *StagingArea, highHash, lowHash *externalapi.DomainHash) (BlockIterator, 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)
|
||||
BlockWindow(stagingArea *StagingArea, highHash *externalapi.DomainHash, windowSize int) ([]*externalapi.DomainHash, error)
|
||||
BlockWindowWithGHOSTDAGData(stagingArea *StagingArea, highHash *externalapi.DomainHash, windowSize int) ([]*externalapi.BlockGHOSTDAGDataHashPair, error)
|
||||
|
@ -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
|
||||
}
|
@ -59,6 +59,7 @@ type TestConsensus interface {
|
||||
BlockStore() model.BlockStore
|
||||
ConsensusStateStore() model.ConsensusStateStore
|
||||
GHOSTDAGDataStore() model.GHOSTDAGDataStore
|
||||
GHOSTDAGDataStores() []model.GHOSTDAGDataStore
|
||||
HeaderTipsStore() model.HeaderSelectedTipStore
|
||||
MultisetStore() model.MultisetStore
|
||||
PruningStore() model.PruningStore
|
||||
|
@ -93,15 +93,12 @@ func (bpb *blockParentBuilder) BuildParents(stagingArea *model.StagingArea,
|
||||
// all the block levels they occupy
|
||||
for _, directParentHeader := range directParentHeaders {
|
||||
directParentHash := consensushashing.HeaderHash(directParentHeader)
|
||||
proofOfWorkValue := pow.CalculateProofOfWorkValue(directParentHeader.ToMutable())
|
||||
for blockLevel := 0; ; blockLevel++ {
|
||||
if _, exists := candidatesByLevelToReferenceBlocksMap[blockLevel]; !exists {
|
||||
candidatesByLevelToReferenceBlocksMap[blockLevel] = make(map[externalapi.DomainHash][]*externalapi.DomainHash)
|
||||
}
|
||||
candidatesByLevelToReferenceBlocksMap[blockLevel][*directParentHash] = []*externalapi.DomainHash{directParentHash}
|
||||
if proofOfWorkValue.Bit(blockLevel+1) != 0 {
|
||||
break
|
||||
blockLevel := pow.BlockLevel(directParentHeader)
|
||||
for i := 0; i <= blockLevel; i++ {
|
||||
if _, exists := candidatesByLevelToReferenceBlocksMap[i]; !exists {
|
||||
candidatesByLevelToReferenceBlocksMap[i] = make(map[externalapi.DomainHash][]*externalapi.DomainHash)
|
||||
}
|
||||
candidatesByLevelToReferenceBlocksMap[i][*directParentHash] = []*externalapi.DomainHash{directParentHash}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -36,24 +36,64 @@ func addBlock(tc testapi.TestConsensus, parentHashes []*externalapi.DomainHash,
|
||||
|
||||
func TestValidateAndInsertImportedPruningPoint(t *testing.T) {
|
||||
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()
|
||||
if err != nil {
|
||||
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 {
|
||||
t.Fatalf("PruningPointHeaders: %+v", err)
|
||||
}
|
||||
|
||||
pointAndItsAnticoneWithTrustedData, err := tcSyncer.PruningPointAndItsAnticoneWithTrustedData()
|
||||
pruningPointAndItsAnticoneWithTrustedData, err := tcSyncer.PruningPointAndItsAnticoneWithTrustedData()
|
||||
if err != nil {
|
||||
t.Fatalf("PruningPointAndItsAnticoneWithTrustedData: %+v", err)
|
||||
}
|
||||
|
||||
for _, blockWithTrustedData := range pointAndItsAnticoneWithTrustedData {
|
||||
_, err := tcSyncee.ValidateAndInsertBlockWithTrustedData(blockWithTrustedData, false)
|
||||
for _, blockWithTrustedData := range pruningPointAndItsAnticoneWithTrustedData {
|
||||
_, err := synceeStaging.ValidateAndInsertBlockWithTrustedData(blockWithTrustedData, false)
|
||||
if err != nil {
|
||||
t.Fatalf("ValidateAndInsertBlockWithTrustedData: %+v", err)
|
||||
}
|
||||
@ -75,7 +115,7 @@ func TestValidateAndInsertImportedPruningPoint(t *testing.T) {
|
||||
}
|
||||
|
||||
for i, blocksHash := range missingHeaderHashes {
|
||||
blockInfo, err := tcSyncee.GetBlockInfo(blocksHash)
|
||||
blockInfo, err := synceeStaging.GetBlockInfo(blocksHash)
|
||||
if err != nil {
|
||||
t.Fatalf("GetBlockInfo: %+v", err)
|
||||
}
|
||||
@ -89,7 +129,7 @@ func TestValidateAndInsertImportedPruningPoint(t *testing.T) {
|
||||
t.Fatalf("GetBlockHeader: %+v", err)
|
||||
}
|
||||
|
||||
_, err = tcSyncee.ValidateAndInsertBlock(&externalapi.DomainBlock{Header: header}, false)
|
||||
_, err = synceeStaging.ValidateAndInsertBlock(&externalapi.DomainBlock{Header: header}, false)
|
||||
if err != nil {
|
||||
t.Fatalf("ValidateAndInsertBlock %d: %+v", i, err)
|
||||
}
|
||||
@ -99,7 +139,7 @@ func TestValidateAndInsertImportedPruningPoint(t *testing.T) {
|
||||
if err != nil {
|
||||
t.Fatalf("GetPruningPointUTXOs: %+v", err)
|
||||
}
|
||||
err = tcSyncee.AppendImportedPruningPointUTXOs(pruningPointUTXOs)
|
||||
err = synceeStaging.AppendImportedPruningPointUTXOs(pruningPointUTXOs)
|
||||
if err != nil {
|
||||
t.Fatalf("AppendImportedPruningPointUTXOs: %+v", err)
|
||||
}
|
||||
@ -110,37 +150,37 @@ func TestValidateAndInsertImportedPruningPoint(t *testing.T) {
|
||||
}
|
||||
|
||||
// Check that ValidateAndInsertImportedPruningPoint fails for invalid pruning point
|
||||
err = tcSyncee.ValidateAndInsertImportedPruningPoint(virtualSelectedParent)
|
||||
err = synceeStaging.ValidateAndInsertImportedPruningPoint(virtualSelectedParent)
|
||||
if !errors.Is(err, ruleerrors.ErrUnexpectedPruningPoint) {
|
||||
t.Fatalf("Unexpected error: %+v", err)
|
||||
}
|
||||
|
||||
err = tcSyncee.ClearImportedPruningPointData()
|
||||
err = synceeStaging.ClearImportedPruningPointData()
|
||||
if err != nil {
|
||||
t.Fatalf("ClearImportedPruningPointData: %+v", err)
|
||||
}
|
||||
err = tcSyncee.AppendImportedPruningPointUTXOs(makeFakeUTXOs())
|
||||
err = synceeStaging.AppendImportedPruningPointUTXOs(makeFakeUTXOs())
|
||||
if err != nil {
|
||||
t.Fatalf("AppendImportedPruningPointUTXOs: %+v", err)
|
||||
}
|
||||
|
||||
// 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) {
|
||||
t.Fatalf("Unexpected error: %+v", err)
|
||||
}
|
||||
|
||||
err = tcSyncee.ClearImportedPruningPointData()
|
||||
err = synceeStaging.ClearImportedPruningPointData()
|
||||
if err != nil {
|
||||
t.Fatalf("ClearImportedPruningPointData: %+v", err)
|
||||
}
|
||||
err = tcSyncee.AppendImportedPruningPointUTXOs(pruningPointUTXOs)
|
||||
err = synceeStaging.AppendImportedPruningPointUTXOs(pruningPointUTXOs)
|
||||
if err != nil {
|
||||
t.Fatalf("AppendImportedPruningPointUTXOs: %+v", err)
|
||||
}
|
||||
|
||||
// Check that ValidateAndInsertImportedPruningPoint works given the right arguments.
|
||||
err = tcSyncee.ValidateAndInsertImportedPruningPoint(pruningPoint)
|
||||
err = synceeStaging.ValidateAndInsertImportedPruningPoint(pruningPoint)
|
||||
if err != nil {
|
||||
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.
|
||||
_, err = tcSyncee.BuildBlock(emptyCoinbase, nil)
|
||||
_, err = synceeStaging.BuildBlock(emptyCoinbase, nil)
|
||||
if err != nil {
|
||||
t.Fatalf("BuildBlock: %+v", err)
|
||||
}
|
||||
|
||||
// Sync block bodies
|
||||
headersSelectedTip, err := tcSyncee.GetHeadersSelectedTip()
|
||||
headersSelectedTip, err := synceeStaging.GetHeadersSelectedTip()
|
||||
if err != nil {
|
||||
t.Fatalf("GetHeadersSelectedTip: %+v", err)
|
||||
}
|
||||
|
||||
missingBlockHashes, err := tcSyncee.GetMissingBlockBodyHashes(headersSelectedTip)
|
||||
missingBlockHashes, err := synceeStaging.GetMissingBlockBodyHashes(headersSelectedTip)
|
||||
if err != nil {
|
||||
t.Fatalf("GetMissingBlockBodyHashes: %+v", err)
|
||||
}
|
||||
@ -175,13 +215,13 @@ func TestValidateAndInsertImportedPruningPoint(t *testing.T) {
|
||||
t.Fatalf("GetBlock: %+v", err)
|
||||
}
|
||||
|
||||
_, err = tcSyncee.ValidateAndInsertBlock(block, true)
|
||||
_, err = synceeStaging.ValidateAndInsertBlock(block, true)
|
||||
if err != nil {
|
||||
t.Fatalf("ValidateAndInsertBlock: %+v", err)
|
||||
}
|
||||
}
|
||||
|
||||
synceeTips, err := tcSyncee.Tips()
|
||||
synceeTips, err := synceeStaging.Tips()
|
||||
if err != nil {
|
||||
t.Fatalf("Tips: %+v", err)
|
||||
}
|
||||
@ -201,12 +241,12 @@ func TestValidateAndInsertImportedPruningPoint(t *testing.T) {
|
||||
t.Fatalf("GetBlock: %+v", err)
|
||||
}
|
||||
|
||||
_, err = tcSyncee.ValidateAndInsertBlock(tip, true)
|
||||
_, err = synceeStaging.ValidateAndInsertBlock(tip, true)
|
||||
if err != nil {
|
||||
t.Fatalf("ValidateAndInsertBlock: %+v", err)
|
||||
}
|
||||
|
||||
blockInfo, err := tcSyncee.GetBlockInfo(tipHash)
|
||||
blockInfo, err := synceeStaging.GetBlockInfo(tipHash)
|
||||
if err != nil {
|
||||
t.Fatalf("GetBlockInfo: %+v", err)
|
||||
}
|
||||
@ -215,7 +255,7 @@ func TestValidateAndInsertImportedPruningPoint(t *testing.T) {
|
||||
t.Fatalf("Tip didn't pass UTXO verification")
|
||||
}
|
||||
|
||||
synceePruningPoint, err := tcSyncee.PruningPoint()
|
||||
synceePruningPoint, err := synceeStaging.PruningPoint()
|
||||
if err != nil {
|
||||
t.Fatalf("PruningPoint: %+v", err)
|
||||
}
|
||||
@ -223,284 +263,88 @@ func TestValidateAndInsertImportedPruningPoint(t *testing.T) {
|
||||
if !synceePruningPoint.Equal(pruningPoint) {
|
||||
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")
|
||||
if err != nil {
|
||||
t.Fatalf("Error setting up tcSyncer: %+v", err)
|
||||
}
|
||||
defer teardownSyncer(false)
|
||||
|
||||
tcSyncee1, teardownSyncee1, err := factory.NewTestConsensus(&synceeConfig, "TestValidateAndInsertPruningPointSyncee1")
|
||||
tcSyncee1, teardownSyncee1, err := factory.NewTestConsensus(consensusConfig, "TestValidateAndInsertPruningPointSyncee1")
|
||||
if err != nil {
|
||||
t.Fatalf("Error setting up tcSyncee1: %+v", err)
|
||||
}
|
||||
defer teardownSyncee1(false)
|
||||
|
||||
const numSharedBlocks = 2
|
||||
tipHash := consensusConfig.GenesisHash
|
||||
for i := 0; i < finalityDepth-2; i++ {
|
||||
for i := 0; i < numSharedBlocks; 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)
|
||||
}
|
||||
|
||||
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)
|
||||
block, err := tcSyncer.GetBlock(tipHash)
|
||||
if err != nil {
|
||||
t.Fatalf("GetBlock: %+v", err)
|
||||
}
|
||||
|
||||
_, err = tcSyncee.ValidateAndInsertBlock(block, false)
|
||||
_, err = tcSyncee1.ValidateAndInsertBlock(block, true)
|
||||
if err != nil {
|
||||
t.Fatalf("ValidateAndInsertBlock: %+v", err)
|
||||
}
|
||||
}
|
||||
|
||||
pruningPointUTXOs, err := tcSyncer.GetPruningPointUTXOs(pruningPoint, nil, 1000)
|
||||
if err != nil {
|
||||
t.Fatalf("GetPruningPointUTXOs: %+v", err)
|
||||
}
|
||||
err = tcSyncee.AppendImportedPruningPointUTXOs(pruningPointUTXOs)
|
||||
if err != nil {
|
||||
t.Fatalf("AppendImportedPruningPointUTXOs: %+v", err)
|
||||
// Add two side blocks to syncee
|
||||
tipHashSyncee := tipHash
|
||||
for i := 0; i < 2; i++ {
|
||||
tipHashSyncee = addBlock(tcSyncee1, []*externalapi.DomainHash{tipHashSyncee}, t)
|
||||
}
|
||||
|
||||
// Check that ValidateAndInsertImportedPruningPoint fails for invalid pruning point
|
||||
err = tcSyncee.ValidateAndInsertImportedPruningPoint(tipHash)
|
||||
if !errors.Is(err, ruleerrors.ErrUnexpectedPruningPoint) {
|
||||
t.Fatalf("Unexpected error: %+v", err)
|
||||
for i := 0; i < finalityDepth-numSharedBlocks-2; i++ {
|
||||
tipHash = addBlock(tcSyncer, []*externalapi.DomainHash{tipHash}, t)
|
||||
}
|
||||
|
||||
err = tcSyncee.ClearImportedPruningPointData()
|
||||
if err != nil {
|
||||
t.Fatalf("ClearImportedPruningPointData: %+v", err)
|
||||
}
|
||||
err = tcSyncee.AppendImportedPruningPointUTXOs(makeFakeUTXOs())
|
||||
if err != nil {
|
||||
t.Fatalf("AppendImportedPruningPointUTXOs: %+v", err)
|
||||
// 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
|
||||
}
|
||||
}
|
||||
|
||||
// Check that ValidateAndInsertImportedPruningPoint fails if the UTXO commitment doesn't fit the provided UTXO set.
|
||||
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()
|
||||
pruningPoint, err := tcSyncer.PruningPoint()
|
||||
if err != nil {
|
||||
t.Fatalf("PruningPoint: %+v", err)
|
||||
}
|
||||
|
||||
if !synceePruningPoint.Equal(pruningPoint) {
|
||||
t.Fatalf("The syncee pruning point has not changed as exepcted")
|
||||
if !pruningPoint.Equal(nextPruningPoint) {
|
||||
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)
|
||||
})
|
||||
}
|
||||
|
||||
|
@ -15,12 +15,14 @@ func (v *blockValidator) ValidateBodyInContext(stagingArea *model.StagingArea, b
|
||||
onEnd := logger.LogAndMeasureExecutionTime(log, "ValidateBodyInContext")
|
||||
defer onEnd()
|
||||
|
||||
err := v.checkBlockIsNotPruned(stagingArea, blockHash)
|
||||
if err != nil {
|
||||
return err
|
||||
if !isBlockWithTrustedData {
|
||||
err := v.checkBlockIsNotPruned(stagingArea, blockHash)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
err = v.checkBlockTransactions(stagingArea, blockHash)
|
||||
err := v.checkBlockTransactions(stagingArea, blockHash)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -52,7 +54,7 @@ func (v *blockValidator) checkBlockIsNotPruned(stagingArea *model.StagingArea, b
|
||||
return err
|
||||
}
|
||||
|
||||
isAncestorOfSomeTips, err := v.dagTopologyManager.IsAncestorOfAny(stagingArea, blockHash, tips)
|
||||
isAncestorOfSomeTips, err := v.dagTopologyManagers[0].IsAncestorOfAny(stagingArea, blockHash, tips)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -69,7 +71,7 @@ func (v *blockValidator) checkParentBlockBodiesExist(
|
||||
stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) error {
|
||||
|
||||
missingParentHashes := []*externalapi.DomainHash{}
|
||||
parents, err := v.dagTopologyManager.Parents(stagingArea, blockHash)
|
||||
parents, err := v.dagTopologyManagers[0].Parents(stagingArea, blockHash)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -90,7 +92,7 @@ func (v *blockValidator) checkParentBlockBodiesExist(
|
||||
return err
|
||||
}
|
||||
|
||||
isInPastOfPruningPoint, err := v.dagTopologyManager.IsAncestorOf(stagingArea, parent, pruningPoint)
|
||||
isInPastOfPruningPoint, err := v.dagTopologyManagers[0].IsAncestorOf(stagingArea, parent, pruningPoint)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -6,6 +6,7 @@ import (
|
||||
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
|
||||
"github.com/kaspanet/kaspad/domain/consensus/ruleerrors"
|
||||
"github.com/kaspanet/kaspad/domain/consensus/utils/consensushashing"
|
||||
"github.com/kaspanet/kaspad/domain/consensus/utils/pow"
|
||||
"github.com/kaspanet/kaspad/infrastructure/logger"
|
||||
"github.com/pkg/errors"
|
||||
)
|
||||
@ -30,7 +31,7 @@ func (v *blockValidator) ValidateHeaderInContext(stagingArea *model.StagingArea,
|
||||
var logErr error
|
||||
log.Debug(logger.NewLogClosure(func() string {
|
||||
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 {
|
||||
return ""
|
||||
}
|
||||
@ -62,9 +63,12 @@ func (v *blockValidator) ValidateHeaderInContext(stagingArea *model.StagingArea,
|
||||
return err
|
||||
}
|
||||
if !hasReachabilityData {
|
||||
err = v.reachabilityManager.AddBlock(stagingArea, blockHash)
|
||||
if err != nil {
|
||||
return err
|
||||
blockLevel := pow.BlockLevel(header)
|
||||
for i := 0; i <= blockLevel; i++ {
|
||||
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
|
||||
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 {
|
||||
return err
|
||||
}
|
||||
@ -136,7 +140,7 @@ func (v *blockValidator) checkParentsIncest(stagingArea *model.StagingArea, bloc
|
||||
continue
|
||||
}
|
||||
|
||||
isAAncestorOfB, err := v.dagTopologyManager.IsAncestorOf(stagingArea, parentA, parentB)
|
||||
isAAncestorOfB, err := v.dagTopologyManagers[0].IsAncestorOf(stagingArea, parentA, parentB)
|
||||
if err != nil {
|
||||
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 {
|
||||
ghostdagData, err := v.ghostdagDataStore.Get(v.databaseContext, stagingArea, hash, false)
|
||||
ghostdagData, err := v.ghostdagDataStores[0].Get(v.databaseContext, stagingArea, hash, false)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -219,7 +223,7 @@ func (v *blockValidator) checkDAAScore(stagingArea *model.StagingArea, blockHash
|
||||
func (v *blockValidator) checkBlueWork(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash,
|
||||
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 {
|
||||
return err
|
||||
}
|
||||
@ -233,7 +237,7 @@ func (v *blockValidator) checkBlueWork(stagingArea *model.StagingArea, blockHash
|
||||
func (v *blockValidator) checkHeaderBlueScore(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash,
|
||||
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 {
|
||||
return err
|
||||
}
|
||||
|
@ -27,19 +27,19 @@ type blockValidator struct {
|
||||
difficultyManager model.DifficultyManager
|
||||
pastMedianTimeManager model.PastMedianTimeManager
|
||||
transactionValidator model.TransactionValidator
|
||||
ghostdagManager model.GHOSTDAGManager
|
||||
dagTopologyManager model.DAGTopologyManager
|
||||
ghostdagManagers []model.GHOSTDAGManager
|
||||
dagTopologyManagers []model.DAGTopologyManager
|
||||
dagTraversalManager model.DAGTraversalManager
|
||||
coinbaseManager model.CoinbaseManager
|
||||
mergeDepthManager model.MergeDepthManager
|
||||
pruningStore model.PruningStore
|
||||
reachabilityManager model.ReachabilityManager
|
||||
reachabilityManagers []model.ReachabilityManager
|
||||
finalityManager model.FinalityManager
|
||||
blockParentBuilder model.BlockParentBuilder
|
||||
pruningManager model.PruningManager
|
||||
|
||||
blockStore model.BlockStore
|
||||
ghostdagDataStore model.GHOSTDAGDataStore
|
||||
ghostdagDataStores []model.GHOSTDAGDataStore
|
||||
blockHeaderStore model.BlockHeaderStore
|
||||
blockStatusStore model.BlockStatusStore
|
||||
reachabilityStore model.ReachabilityDataStore
|
||||
@ -63,19 +63,19 @@ func New(powMax *big.Int,
|
||||
difficultyManager model.DifficultyManager,
|
||||
pastMedianTimeManager model.PastMedianTimeManager,
|
||||
transactionValidator model.TransactionValidator,
|
||||
ghostdagManager model.GHOSTDAGManager,
|
||||
dagTopologyManager model.DAGTopologyManager,
|
||||
ghostdagManagers []model.GHOSTDAGManager,
|
||||
dagTopologyManagers []model.DAGTopologyManager,
|
||||
dagTraversalManager model.DAGTraversalManager,
|
||||
coinbaseManager model.CoinbaseManager,
|
||||
mergeDepthManager model.MergeDepthManager,
|
||||
reachabilityManager model.ReachabilityManager,
|
||||
reachabilityManagers []model.ReachabilityManager,
|
||||
finalityManager model.FinalityManager,
|
||||
blockParentBuilder model.BlockParentBuilder,
|
||||
pruningManager model.PruningManager,
|
||||
|
||||
pruningStore model.PruningStore,
|
||||
blockStore model.BlockStore,
|
||||
ghostdagDataStore model.GHOSTDAGDataStore,
|
||||
ghostdagDataStores []model.GHOSTDAGDataStore,
|
||||
blockHeaderStore model.BlockHeaderStore,
|
||||
blockStatusStore model.BlockStatusStore,
|
||||
reachabilityStore model.ReachabilityDataStore,
|
||||
@ -99,19 +99,19 @@ func New(powMax *big.Int,
|
||||
difficultyManager: difficultyManager,
|
||||
pastMedianTimeManager: pastMedianTimeManager,
|
||||
transactionValidator: transactionValidator,
|
||||
ghostdagManager: ghostdagManager,
|
||||
dagTopologyManager: dagTopologyManager,
|
||||
ghostdagManagers: ghostdagManagers,
|
||||
dagTopologyManagers: dagTopologyManagers,
|
||||
dagTraversalManager: dagTraversalManager,
|
||||
coinbaseManager: coinbaseManager,
|
||||
mergeDepthManager: mergeDepthManager,
|
||||
reachabilityManager: reachabilityManager,
|
||||
reachabilityManagers: reachabilityManagers,
|
||||
finalityManager: finalityManager,
|
||||
blockParentBuilder: blockParentBuilder,
|
||||
pruningManager: pruningManager,
|
||||
|
||||
pruningStore: pruningStore,
|
||||
blockStore: blockStore,
|
||||
ghostdagDataStore: ghostdagDataStore,
|
||||
ghostdagDataStores: ghostdagDataStores,
|
||||
blockHeaderStore: blockHeaderStore,
|
||||
blockStatusStore: blockStatusStore,
|
||||
reachabilityStore: reachabilityStore,
|
||||
|
@ -43,9 +43,11 @@ func (v *blockValidator) ValidatePruningPointViolationAndProofOfWorkAndDifficult
|
||||
return err
|
||||
}
|
||||
|
||||
err = v.checkPruningPointViolation(stagingArea, blockHash)
|
||||
if err != nil {
|
||||
return err
|
||||
if !isBlockWithTrustedData {
|
||||
err = v.checkPruningPointViolation(stagingArea, blockHash)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
err = v.checkProofOfWork(header)
|
||||
@ -66,28 +68,36 @@ func (v *blockValidator) setParents(stagingArea *model.StagingArea,
|
||||
header externalapi.BlockHeader,
|
||||
isBlockWithTrustedData bool) error {
|
||||
|
||||
parents := make([]*externalapi.DomainHash, 0, len(header.DirectParents()))
|
||||
for _, currentParent := range header.DirectParents() {
|
||||
exists, err := v.blockStatusStore.Exists(v.databaseContext, stagingArea, currentParent)
|
||||
for level := 0; level <= pow.BlockLevel(header); level++ {
|
||||
var parents []*externalapi.DomainHash
|
||||
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 {
|
||||
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 {
|
||||
parents = append(parents, model.VirtualGenesisBlockHash)
|
||||
}
|
||||
|
||||
return v.dagTopologyManager.SetParents(stagingArea, blockHash, parents)
|
||||
return nil
|
||||
}
|
||||
|
||||
func (v *blockValidator) validateDifficulty(stagingArea *model.StagingArea,
|
||||
@ -96,7 +106,20 @@ func (v *blockValidator) validateDifficulty(stagingArea *model.StagingArea,
|
||||
|
||||
if !isBlockWithTrustedData {
|
||||
// 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 {
|
||||
return err
|
||||
}
|
||||
@ -110,10 +133,6 @@ func (v *blockValidator) validateDifficulty(stagingArea *model.StagingArea,
|
||||
return err
|
||||
}
|
||||
|
||||
header, err := v.blockHeaderStore.BlockHeader(v.databaseContext, stagingArea, blockHash)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if 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
|
||||
}
|
||||
|
||||
parents, err := v.dagTopologyManager.Parents(stagingArea, blockHash)
|
||||
parents, err := v.dagTopologyManagers[0].Parents(stagingArea, blockHash)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -224,7 +243,7 @@ func (v *blockValidator) checkPruningPointViolation(stagingArea *model.StagingAr
|
||||
return nil
|
||||
}
|
||||
|
||||
isAncestorOfAny, err := v.dagTopologyManager.IsAncestorOfAny(stagingArea, pruningPoint, parents)
|
||||
isAncestorOfAny, err := v.dagTopologyManagers[0].IsAncestorOfAny(stagingArea, pruningPoint, parents)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -14,21 +14,10 @@ func (dtm *dagTraversalManager) AnticoneFromVirtualPOV(stagingArea *model.Stagin
|
||||
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) (
|
||||
[]*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) (
|
||||
func (dtm *dagTraversalManager) AnticoneFromBlocks(stagingArea *model.StagingArea, tips []*externalapi.DomainHash, blockHash *externalapi.DomainHash) (
|
||||
[]*externalapi.DomainHash, error) {
|
||||
|
||||
anticone := []*externalapi.DomainHash{}
|
||||
|
@ -6,7 +6,7 @@ import (
|
||||
"github.com/pkg/errors"
|
||||
)
|
||||
|
||||
// dagTraversalManager exposes methods for travering blocks
|
||||
// dagTraversalManager exposes methods for traversing blocks
|
||||
// in the DAG
|
||||
type dagTraversalManager struct {
|
||||
databaseContext model.DBReader
|
||||
@ -15,7 +15,6 @@ type dagTraversalManager struct {
|
||||
ghostdagManager model.GHOSTDAGManager
|
||||
ghostdagDataStore model.GHOSTDAGDataStore
|
||||
reachabilityDataStore model.ReachabilityDataStore
|
||||
consensusStateStore model.ConsensusStateStore
|
||||
daaWindowStore model.BlocksWithTrustedDataDAAWindowStore
|
||||
genesisHash *externalapi.DomainHash
|
||||
}
|
||||
@ -27,7 +26,6 @@ func New(
|
||||
ghostdagDataStore model.GHOSTDAGDataStore,
|
||||
reachabilityDataStore model.ReachabilityDataStore,
|
||||
ghostdagManager model.GHOSTDAGManager,
|
||||
conssensusStateStore model.ConsensusStateStore,
|
||||
daaWindowStore model.BlocksWithTrustedDataDAAWindowStore,
|
||||
genesisHash *externalapi.DomainHash) model.DAGTraversalManager {
|
||||
return &dagTraversalManager{
|
||||
@ -36,43 +34,12 @@ func New(
|
||||
ghostdagDataStore: ghostdagDataStore,
|
||||
reachabilityDataStore: reachabilityDataStore,
|
||||
ghostdagManager: ghostdagManager,
|
||||
consensusStateStore: conssensusStateStore,
|
||||
daaWindowStore: daaWindowStore,
|
||||
|
||||
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) {
|
||||
highBlockGHOSTDAGData, err := dtm.ghostdagDataStore.Get(dtm.databaseContext, stagingArea, highHash, false)
|
||||
if err != nil {
|
||||
|
@ -7,204 +7,9 @@ import (
|
||||
|
||||
"github.com/kaspanet/kaspad/domain/consensus"
|
||||
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
|
||||
"github.com/kaspanet/kaspad/domain/consensus/model/testapi"
|
||||
"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) {
|
||||
testutils.ForAllNets(t, true, func(t *testing.T, consensusConfig *consensus.Config) {
|
||||
consensusConfig.FinalityDuration = 10 * consensusConfig.TargetTimePerBlock
|
||||
|
@ -53,7 +53,13 @@ func (dtm *dagTraversalManager) calculateBlockWindowHeap(stagingArea *model.Stag
|
||||
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++ {
|
||||
daaBlock, err := dtm.daaWindowStore.DAAWindowBlock(dtm.databaseContext, stagingArea, current, i)
|
||||
if database.IsNotFoundError(err) {
|
||||
|
@ -100,6 +100,11 @@ func (gm *ghostdagManager) GHOSTDAG(stagingArea *model.StagingArea, blockHash *e
|
||||
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
|
||||
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)
|
||||
if err != nil {
|
||||
return err
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -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
|
||||
}
|
@ -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)
|
||||
}
|
||||
}
|
||||
}
|
@ -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,
|
||||
}
|
||||
}
|
@ -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)
|
||||
}
|
@ -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
|
||||
}
|
@ -228,6 +228,15 @@ var (
|
||||
// ErrUnexpectedFinalityPoint indicates a block header pruning point does not align with
|
||||
// the expected value.
|
||||
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
|
||||
|
@ -24,7 +24,7 @@ func (tc *testConsensus) BlockHeaderStore() model.BlockHeaderStore {
|
||||
}
|
||||
|
||||
func (tc *testConsensus) BlockRelationStore() model.BlockRelationStore {
|
||||
return tc.blockRelationStore
|
||||
return tc.blockRelationStores[0]
|
||||
}
|
||||
|
||||
func (tc *testConsensus) BlockStatusStore() model.BlockStatusStore {
|
||||
@ -40,7 +40,11 @@ func (tc *testConsensus) ConsensusStateStore() model.ConsensusStateStore {
|
||||
}
|
||||
|
||||
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 {
|
||||
@ -56,7 +60,7 @@ func (tc *testConsensus) PruningStore() model.PruningStore {
|
||||
}
|
||||
|
||||
func (tc *testConsensus) ReachabilityDataStore() model.ReachabilityDataStore {
|
||||
return tc.reachabilityDataStore
|
||||
return tc.reachabilityDataStores[0]
|
||||
}
|
||||
|
||||
func (tc *testConsensus) UTXODiffStore() model.UTXODiffStore {
|
||||
@ -84,7 +88,7 @@ func (tc *testConsensus) ConsensusStateManager() testapi.TestConsensusStateManag
|
||||
}
|
||||
|
||||
func (tc *testConsensus) DAGTopologyManager() model.DAGTopologyManager {
|
||||
return tc.dagTopologyManager
|
||||
return tc.dagTopologyManagers[0]
|
||||
}
|
||||
|
||||
func (tc *testConsensus) DAGTraversalManager() model.DAGTraversalManager {
|
||||
@ -96,7 +100,7 @@ func (tc *testConsensus) DifficultyManager() model.DifficultyManager {
|
||||
}
|
||||
|
||||
func (tc *testConsensus) GHOSTDAGManager() model.GHOSTDAGManager {
|
||||
return tc.ghostdagManager
|
||||
return tc.ghostdagManagers[0]
|
||||
}
|
||||
|
||||
func (tc *testConsensus) HeaderTipsManager() model.HeadersSelectedTipManager {
|
||||
|
@ -38,7 +38,7 @@ func (tc *testConsensus) convertToDot() (string, error) {
|
||||
}
|
||||
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 {
|
||||
return "", err
|
||||
}
|
||||
|
@ -35,4 +35,7 @@ const (
|
||||
// LockTimeThreshold is the number below which a lock time is
|
||||
// interpreted to be a DAA score.
|
||||
LockTimeThreshold = 5e11 // Tue Nov 5 00:53:20 1985 UTC
|
||||
|
||||
// MaxBlockLevel is the maximum possible block level.
|
||||
MaxBlockLevel = 255
|
||||
)
|
||||
|
@ -3,6 +3,7 @@ package pow
|
||||
import (
|
||||
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
|
||||
"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/serialization"
|
||||
"github.com/kaspanet/kaspad/util/difficulty"
|
||||
@ -65,3 +66,13 @@ func toBig(hash *externalapi.DomainHash) *big.Int {
|
||||
|
||||
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
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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
|
||||
// time the width of the DAG is at most defaultGHOSTDAGK.
|
||||
defaultTargetTimePerBlock = 1 * time.Second
|
||||
|
||||
defaultPruningProofM = 1000
|
||||
)
|
||||
|
@ -174,6 +174,9 @@ type Params struct {
|
||||
|
||||
// BaseSubsidy is the starting subsidy amount for mined blocks.
|
||||
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
|
||||
@ -245,6 +248,7 @@ var MainnetParams = Params{
|
||||
MergeSetSizeLimit: defaultMergeSetSizeLimit,
|
||||
BaseSubsidy: defaultBaseSubsidy,
|
||||
CoinbasePayloadScriptPublicKeyMaxLength: defaultCoinbasePayloadScriptPublicKeyMaxLength,
|
||||
PruningProofM: defaultPruningProofM,
|
||||
}
|
||||
|
||||
// TestnetParams defines the network parameters for the test Kaspa network.
|
||||
@ -301,6 +305,7 @@ var TestnetParams = Params{
|
||||
MergeSetSizeLimit: defaultMergeSetSizeLimit,
|
||||
BaseSubsidy: defaultBaseSubsidy,
|
||||
CoinbasePayloadScriptPublicKeyMaxLength: defaultCoinbasePayloadScriptPublicKeyMaxLength,
|
||||
PruningProofM: defaultPruningProofM,
|
||||
}
|
||||
|
||||
// SimnetParams defines the network parameters for the simulation test Kaspa
|
||||
@ -361,6 +366,7 @@ var SimnetParams = Params{
|
||||
MergeSetSizeLimit: defaultMergeSetSizeLimit,
|
||||
BaseSubsidy: defaultBaseSubsidy,
|
||||
CoinbasePayloadScriptPublicKeyMaxLength: defaultCoinbasePayloadScriptPublicKeyMaxLength,
|
||||
PruningProofM: defaultPruningProofM,
|
||||
}
|
||||
|
||||
// DevnetParams defines the network parameters for the development Kaspa network.
|
||||
@ -417,6 +423,7 @@ var DevnetParams = Params{
|
||||
MergeSetSizeLimit: defaultMergeSetSizeLimit,
|
||||
BaseSubsidy: defaultBaseSubsidy,
|
||||
CoinbasePayloadScriptPublicKeyMaxLength: defaultCoinbasePayloadScriptPublicKeyMaxLength,
|
||||
PruningProofM: defaultPruningProofM,
|
||||
}
|
||||
|
||||
var (
|
||||
@ -461,4 +468,5 @@ func init() {
|
||||
mustRegister(&MainnetParams)
|
||||
mustRegister(&TestnetParams)
|
||||
mustRegister(&SimnetParams)
|
||||
mustRegister(&DevnetParams)
|
||||
}
|
||||
|
@ -91,6 +91,8 @@ func (c *ConnectionManager) 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() {
|
||||
|
@ -18,7 +18,7 @@ type p2pServer struct {
|
||||
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.
|
||||
// Note that inbound connections are not limited by the gRPC server. (A value of 0 means
|
||||
|
@ -9,9 +9,9 @@ import (
|
||||
"github.com/kaspanet/kaspad/app/appmessage"
|
||||
)
|
||||
|
||||
func Test64IncomingConnections(t *testing.T) {
|
||||
// Much more than 64 hosts creates a risk of running out of available file descriptors for leveldb
|
||||
const numBullies = 64
|
||||
func Test16IncomingConnections(t *testing.T) {
|
||||
// Much more than 16 hosts creates a risk of running out of available file descriptors for leveldb
|
||||
const numBullies = 16
|
||||
harnessesParams := make([]*harnessParams, numBullies+1)
|
||||
for i := 0; i < numBullies+1; i++ {
|
||||
harnessesParams[i] = &harnessParams{
|
@ -135,6 +135,7 @@ func TestIBDWithPruning(t *testing.T) {
|
||||
// This is done to make a pruning depth of 6 blocks
|
||||
overrideDAGParams.FinalityDuration = 2 * overrideDAGParams.TargetTimePerBlock
|
||||
overrideDAGParams.K = 0
|
||||
overrideDAGParams.PruningProofM = 20
|
||||
|
||||
expectedPruningDepth := uint64(6)
|
||||
if overrideDAGParams.PruningDepth() != expectedPruningDepth {
|
||||
|
@ -1,10 +1,9 @@
|
||||
package integration
|
||||
|
||||
import (
|
||||
"github.com/kaspanet/kaspad/infrastructure/logger"
|
||||
"os"
|
||||
"testing"
|
||||
|
||||
"github.com/kaspanet/kaspad/infrastructure/logger"
|
||||
)
|
||||
|
||||
func TestMain(m *testing.M) {
|
||||
|
Loading…
x
Reference in New Issue
Block a user