Add support for multiple staging areas (#1633)

* Add StagingArea struct

* Implemented staging areas in blockStore

* Move blockStagingShard to separate folder

* Apply staging shard to acceptanceDataStore

* Update blockHeaderStore with StagingArea

* Add StagingArea to BlockRelationStore

* Add StagingArea to blockStatusStore

* Add StagingArea to consensusStateStore

* Add StagingArea to daaBlocksStore

* Add StagingArea to finalityStore

* Add StagingArea to ghostdagDataStore

* Add StagingArea to headersSelectedChainStore and headersSelectedTipStore

* Add StagingArea to multisetStore

* Add StagingArea to pruningStore

* Add StagingArea to reachabilityDataStore

* Add StagingArea to utxoDiffStore

* Fix forgotten compilation error

* Update reachability manager and some more things with StagingArea

* Add StagingArea to dagTopologyManager, and some more

* Add StagingArea to GHOSTDAGManager, and some more

* Add StagingArea to difficultyManager, and some more

* Add StagingArea to dagTraversalManager, and some more

* Add StagingArea to headerTipsManager, and some more

* Add StagingArea to constnsusStateManager, pastMedianTimeManager

* Add StagingArea to transactionValidator

* Add StagingArea to finalityManager

* Add StagingArea to mergeDepthManager

* Add StagingArea to pruningManager

* Add StagingArea to rest of ValidateAndInsertBlock

* Add StagingArea to blockValidator

* Add StagingArea to coinbaseManager

* Add StagingArea to syncManager

* Add StagingArea to blockBuilder

* Update consensus with StagingArea

* Add StagingArea to ghostdag2

* Fix remaining compilation errors

* Update names of stagingShards

* Fix forgotten stagingArea passing

* Mark stagingShard.isCommited = true once commited

* Move isStaged to stagingShard, so that it's available without going through store

* Make blockHeaderStore count be avilable from stagingShard

* Fix remaining forgotten stagingArea passing

* commitAllChanges should call dbTx.Commit in the end

* Fix all tests tests in blockValidator

* Fix all tests in consensusStateManager and some more

* Fix all tests in pruningManager

* Add many missing stagingAreas in tests

* Fix many tests

* Fix most of all other tests

* Fix ghostdag_test.go

* Add comment to StagingArea

* Make list of StagingShards an array

* Add comment to StagingShardID

* Make sure all staging shards are pointer-receiver

* Undo bucket rename in block_store

* Typo: isCommited -> isCommitted

* Add comment explaining why stagingArea.shards is an array
This commit is contained in:
Svarog 2021-03-29 10:34:11 +03:00 committed by GitHub
parent c12e180873
commit f1451406f7
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
160 changed files with 3223 additions and 2437 deletions

View File

@ -55,8 +55,7 @@ func HandleGetBlocks(context *rpccontext.Context, _ *router.Router, request appm
if err != nil { if err != nil {
return nil, err return nil, err
} }
blockHashes, highHash, err := context.Domain.Consensus().GetHashesBetween( blockHashes, highHash, err := context.Domain.Consensus().GetHashesBetween(lowHash, virtualSelectedParent, maxBlocksInGetBlocksResponse)
lowHash, virtualSelectedParent, maxBlocksInGetBlocksResponse)
if err != nil { if err != nil {
return nil, err return nil, err
} }

View File

@ -5,6 +5,8 @@ import (
"sort" "sort"
"testing" "testing"
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/app/appmessage" "github.com/kaspanet/kaspad/app/appmessage"
"github.com/kaspanet/kaspad/app/rpc/rpccontext" "github.com/kaspanet/kaspad/app/rpc/rpccontext"
"github.com/kaspanet/kaspad/app/rpc/rpchandlers" "github.com/kaspanet/kaspad/app/rpc/rpchandlers"
@ -27,6 +29,8 @@ func (d fakeDomain) MiningManager() miningmanager.MiningManager { return nil }
func TestHandleGetBlocks(t *testing.T) { func TestHandleGetBlocks(t *testing.T) {
testutils.ForAllNets(t, true, func(t *testing.T, params *dagconfig.Params) { testutils.ForAllNets(t, true, func(t *testing.T, params *dagconfig.Params) {
stagingArea := model.NewStagingArea()
factory := consensus.NewFactory() factory := consensus.NewFactory()
tc, teardown, err := factory.NewTestConsensus(params, false, "TestHandleGetBlocks") tc, teardown, err := factory.NewTestConsensus(params, false, "TestHandleGetBlocks")
if err != nil { if err != nil {
@ -55,7 +59,7 @@ func TestHandleGetBlocks(t *testing.T) {
antipast := make([]*externalapi.DomainHash, 0, len(slice)) antipast := make([]*externalapi.DomainHash, 0, len(slice))
for _, blockHash := range slice { for _, blockHash := range slice {
isInPastOfPovBlock, err := tc.DAGTopologyManager().IsAncestorOf(blockHash, povBlock) isInPastOfPovBlock, err := tc.DAGTopologyManager().IsAncestorOf(stagingArea, blockHash, povBlock)
if err != nil { if err != nil {
t.Fatalf("Failed doing reachability check: '%v'", err) t.Fatalf("Failed doing reachability check: '%v'", err)
} }
@ -87,7 +91,7 @@ func TestHandleGetBlocks(t *testing.T) {
} }
splitBlocks = append(splitBlocks, blockHash) splitBlocks = append(splitBlocks, blockHash)
} }
sort.Sort(sort.Reverse(testutils.NewTestGhostDAGSorter(splitBlocks, tc, t))) sort.Sort(sort.Reverse(testutils.NewTestGhostDAGSorter(stagingArea, splitBlocks, tc, t)))
restOfSplitBlocks, selectedParent := splitBlocks[:len(splitBlocks)-1], splitBlocks[len(splitBlocks)-1] restOfSplitBlocks, selectedParent := splitBlocks[:len(splitBlocks)-1], splitBlocks[len(splitBlocks)-1]
expectedOrder = append(expectedOrder, selectedParent) expectedOrder = append(expectedOrder, selectedParent)
expectedOrder = append(expectedOrder, restOfSplitBlocks...) expectedOrder = append(expectedOrder, restOfSplitBlocks...)

View File

@ -75,30 +75,34 @@ func (s *consensus) ValidateTransactionAndPopulateWithConsensusData(transaction
s.lock.Lock() s.lock.Lock()
defer s.lock.Unlock() defer s.lock.Unlock()
stagingArea := model.NewStagingArea()
err := s.transactionValidator.ValidateTransactionInIsolation(transaction) err := s.transactionValidator.ValidateTransactionInIsolation(transaction)
if err != nil { if err != nil {
return err return err
} }
err = s.consensusStateManager.PopulateTransactionWithUTXOEntries(transaction) err = s.consensusStateManager.PopulateTransactionWithUTXOEntries(stagingArea, transaction)
if err != nil { if err != nil {
return err return err
} }
virtualSelectedParentMedianTime, err := s.pastMedianTimeManager.PastMedianTime(model.VirtualBlockHash) virtualSelectedParentMedianTime, err := s.pastMedianTimeManager.PastMedianTime(stagingArea, model.VirtualBlockHash)
if err != nil { if err != nil {
return err return err
} }
return s.transactionValidator.ValidateTransactionInContextAndPopulateMassAndFee(transaction, return s.transactionValidator.ValidateTransactionInContextAndPopulateMassAndFee(
model.VirtualBlockHash, virtualSelectedParentMedianTime) stagingArea, transaction, model.VirtualBlockHash, virtualSelectedParentMedianTime)
} }
func (s *consensus) GetBlock(blockHash *externalapi.DomainHash) (*externalapi.DomainBlock, error) { func (s *consensus) GetBlock(blockHash *externalapi.DomainHash) (*externalapi.DomainBlock, error) {
s.lock.Lock() s.lock.Lock()
defer s.lock.Unlock() defer s.lock.Unlock()
block, err := s.blockStore.Block(s.databaseContext, blockHash) stagingArea := model.NewStagingArea()
block, err := s.blockStore.Block(s.databaseContext, stagingArea, blockHash)
if err != nil { if err != nil {
if errors.Is(err, database.ErrNotFound) { if errors.Is(err, database.ErrNotFound) {
return nil, errors.Wrapf(err, "block %s does not exist", blockHash) return nil, errors.Wrapf(err, "block %s does not exist", blockHash)
@ -112,7 +116,9 @@ func (s *consensus) GetBlockHeader(blockHash *externalapi.DomainHash) (externala
s.lock.Lock() s.lock.Lock()
defer s.lock.Unlock() defer s.lock.Unlock()
blockHeader, err := s.blockHeaderStore.BlockHeader(s.databaseContext, blockHash) stagingArea := model.NewStagingArea()
blockHeader, err := s.blockHeaderStore.BlockHeader(s.databaseContext, stagingArea, blockHash)
if err != nil { if err != nil {
if errors.Is(err, database.ErrNotFound) { if errors.Is(err, database.ErrNotFound) {
return nil, errors.Wrapf(err, "block header %s does not exist", blockHash) return nil, errors.Wrapf(err, "block header %s does not exist", blockHash)
@ -126,9 +132,11 @@ func (s *consensus) GetBlockInfo(blockHash *externalapi.DomainHash) (*externalap
s.lock.Lock() s.lock.Lock()
defer s.lock.Unlock() defer s.lock.Unlock()
stagingArea := model.NewStagingArea()
blockInfo := &externalapi.BlockInfo{} blockInfo := &externalapi.BlockInfo{}
exists, err := s.blockStatusStore.Exists(s.databaseContext, blockHash) exists, err := s.blockStatusStore.Exists(s.databaseContext, stagingArea, blockHash)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -137,7 +145,7 @@ func (s *consensus) GetBlockInfo(blockHash *externalapi.DomainHash) (*externalap
return blockInfo, nil return blockInfo, nil
} }
blockStatus, err := s.blockStatusStore.Get(s.databaseContext, blockHash) blockStatus, err := s.blockStatusStore.Get(s.databaseContext, stagingArea, blockHash)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -148,7 +156,7 @@ func (s *consensus) GetBlockInfo(blockHash *externalapi.DomainHash) (*externalap
return blockInfo, nil return blockInfo, nil
} }
ghostdagData, err := s.ghostdagDataStore.Get(s.databaseContext, blockHash) ghostdagData, err := s.ghostdagDataStore.Get(s.databaseContext, stagingArea, blockHash)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -161,7 +169,10 @@ func (s *consensus) GetBlockInfo(blockHash *externalapi.DomainHash) (*externalap
func (s *consensus) GetBlockChildren(blockHash *externalapi.DomainHash) ([]*externalapi.DomainHash, error) { func (s *consensus) GetBlockChildren(blockHash *externalapi.DomainHash) ([]*externalapi.DomainHash, error) {
s.lock.Lock() s.lock.Lock()
defer s.lock.Unlock() defer s.lock.Unlock()
blockRelation, err := s.blockRelationStore.BlockRelation(s.databaseContext, blockHash)
stagingArea := model.NewStagingArea()
blockRelation, err := s.blockRelationStore.BlockRelation(s.databaseContext, stagingArea, blockHash)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -173,42 +184,48 @@ func (s *consensus) GetBlockAcceptanceData(blockHash *externalapi.DomainHash) (e
s.lock.Lock() s.lock.Lock()
defer s.lock.Unlock() defer s.lock.Unlock()
err := s.validateBlockHashExists(blockHash) stagingArea := model.NewStagingArea()
err := s.validateBlockHashExists(stagingArea, blockHash)
if err != nil { if err != nil {
return nil, err return nil, err
} }
return s.acceptanceDataStore.Get(s.databaseContext, blockHash) return s.acceptanceDataStore.Get(s.databaseContext, stagingArea, blockHash)
} }
func (s *consensus) GetHashesBetween(lowHash, highHash *externalapi.DomainHash, func (s *consensus) GetHashesBetween(lowHash, highHash *externalapi.DomainHash, maxBlueScoreDifference uint64) (
maxBlueScoreDifference uint64) (hashes []*externalapi.DomainHash, actualHighHash *externalapi.DomainHash, err error) { hashes []*externalapi.DomainHash, actualHighHash *externalapi.DomainHash, err error) {
s.lock.Lock() s.lock.Lock()
defer s.lock.Unlock() defer s.lock.Unlock()
err = s.validateBlockHashExists(lowHash) stagingArea := model.NewStagingArea()
err = s.validateBlockHashExists(stagingArea, lowHash)
if err != nil { if err != nil {
return nil, nil, err return nil, nil, err
} }
err = s.validateBlockHashExists(highHash) err = s.validateBlockHashExists(stagingArea, highHash)
if err != nil { if err != nil {
return nil, nil, err return nil, nil, err
} }
return s.syncManager.GetHashesBetween(lowHash, highHash, maxBlueScoreDifference) return s.syncManager.GetHashesBetween(stagingArea, lowHash, highHash, maxBlueScoreDifference)
} }
func (s *consensus) GetMissingBlockBodyHashes(highHash *externalapi.DomainHash) ([]*externalapi.DomainHash, error) { func (s *consensus) GetMissingBlockBodyHashes(highHash *externalapi.DomainHash) ([]*externalapi.DomainHash, error) {
s.lock.Lock() s.lock.Lock()
defer s.lock.Unlock() defer s.lock.Unlock()
err := s.validateBlockHashExists(highHash) stagingArea := model.NewStagingArea()
err := s.validateBlockHashExists(stagingArea, highHash)
if err != nil { if err != nil {
return nil, err return nil, err
} }
return s.syncManager.GetMissingBlockBodyHashes(highHash) return s.syncManager.GetMissingBlockBodyHashes(stagingArea, highHash)
} }
func (s *consensus) GetPruningPointUTXOs(expectedPruningPointHash *externalapi.DomainHash, func (s *consensus) GetPruningPointUTXOs(expectedPruningPointHash *externalapi.DomainHash,
@ -217,7 +234,9 @@ func (s *consensus) GetPruningPointUTXOs(expectedPruningPointHash *externalapi.D
s.lock.Lock() s.lock.Lock()
defer s.lock.Unlock() defer s.lock.Unlock()
pruningPointHash, err := s.pruningStore.PruningPoint(s.databaseContext) stagingArea := model.NewStagingArea()
pruningPointHash, err := s.pruningStore.PruningPoint(s.databaseContext, stagingArea)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -241,7 +260,9 @@ func (s *consensus) GetVirtualUTXOs(expectedVirtualParents []*externalapi.Domain
s.lock.Lock() s.lock.Lock()
defer s.lock.Unlock() defer s.lock.Unlock()
virtualParents, err := s.dagTopologyManager.Parents(model.VirtualBlockHash) stagingArea := model.NewStagingArea()
virtualParents, err := s.dagTopologyManager.Parents(stagingArea, model.VirtualBlockHash)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -263,7 +284,9 @@ func (s *consensus) PruningPoint() (*externalapi.DomainHash, error) {
s.lock.Lock() s.lock.Lock()
defer s.lock.Unlock() defer s.lock.Unlock()
return s.pruningStore.PruningPoint(s.databaseContext) stagingArea := model.NewStagingArea()
return s.pruningStore.PruningPoint(s.databaseContext, stagingArea)
} }
func (s *consensus) ClearImportedPruningPointData() error { func (s *consensus) ClearImportedPruningPointData() error {
@ -291,7 +314,9 @@ func (s *consensus) GetVirtualSelectedParent() (*externalapi.DomainHash, error)
s.lock.Lock() s.lock.Lock()
defer s.lock.Unlock() defer s.lock.Unlock()
virtualGHOSTDAGData, err := s.ghostdagDataStore.Get(s.databaseContext, model.VirtualBlockHash) stagingArea := model.NewStagingArea()
virtualGHOSTDAGData, err := s.ghostdagDataStore.Get(s.databaseContext, stagingArea, model.VirtualBlockHash)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -302,26 +327,30 @@ func (s *consensus) Tips() ([]*externalapi.DomainHash, error) {
s.lock.Lock() s.lock.Lock()
defer s.lock.Unlock() defer s.lock.Unlock()
return s.consensusStateStore.Tips(s.databaseContext) stagingArea := model.NewStagingArea()
return s.consensusStateStore.Tips(stagingArea, s.databaseContext)
} }
func (s *consensus) GetVirtualInfo() (*externalapi.VirtualInfo, error) { func (s *consensus) GetVirtualInfo() (*externalapi.VirtualInfo, error) {
s.lock.Lock() s.lock.Lock()
defer s.lock.Unlock() defer s.lock.Unlock()
blockRelations, err := s.blockRelationStore.BlockRelation(s.databaseContext, model.VirtualBlockHash) stagingArea := model.NewStagingArea()
blockRelations, err := s.blockRelationStore.BlockRelation(s.databaseContext, stagingArea, model.VirtualBlockHash)
if err != nil { if err != nil {
return nil, err return nil, err
} }
bits, err := s.difficultyManager.RequiredDifficulty(model.VirtualBlockHash) bits, err := s.difficultyManager.RequiredDifficulty(stagingArea, model.VirtualBlockHash)
if err != nil { if err != nil {
return nil, err return nil, err
} }
pastMedianTime, err := s.pastMedianTimeManager.PastMedianTime(model.VirtualBlockHash) pastMedianTime, err := s.pastMedianTimeManager.PastMedianTime(stagingArea, model.VirtualBlockHash)
if err != nil { if err != nil {
return nil, err return nil, err
} }
virtualGHOSTDAGData, err := s.ghostdagDataStore.Get(s.databaseContext, model.VirtualBlockHash) virtualGHOSTDAGData, err := s.ghostdagDataStore.Get(s.databaseContext, stagingArea, model.VirtualBlockHash)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -338,76 +367,87 @@ func (s *consensus) CreateBlockLocator(lowHash, highHash *externalapi.DomainHash
s.lock.Lock() s.lock.Lock()
defer s.lock.Unlock() defer s.lock.Unlock()
err := s.validateBlockHashExists(lowHash) stagingArea := model.NewStagingArea()
err := s.validateBlockHashExists(stagingArea, lowHash)
if err != nil { if err != nil {
return nil, err return nil, err
} }
err = s.validateBlockHashExists(highHash) err = s.validateBlockHashExists(stagingArea, highHash)
if err != nil { if err != nil {
return nil, err return nil, err
} }
return s.syncManager.CreateBlockLocator(lowHash, highHash, limit) return s.syncManager.CreateBlockLocator(stagingArea, lowHash, highHash, limit)
} }
func (s *consensus) CreateFullHeadersSelectedChainBlockLocator() (externalapi.BlockLocator, error) { func (s *consensus) CreateFullHeadersSelectedChainBlockLocator() (externalapi.BlockLocator, error) {
s.lock.Lock() s.lock.Lock()
defer s.lock.Unlock() defer s.lock.Unlock()
lowHash, err := s.pruningStore.PruningPoint(s.databaseContext) stagingArea := model.NewStagingArea()
lowHash, err := s.pruningStore.PruningPoint(s.databaseContext, stagingArea)
if err != nil { if err != nil {
return nil, err return nil, err
} }
highHash, err := s.headersSelectedTipStore.HeadersSelectedTip(s.databaseContext) highHash, err := s.headersSelectedTipStore.HeadersSelectedTip(s.databaseContext, stagingArea)
if err != nil { if err != nil {
return nil, err return nil, err
} }
return s.syncManager.CreateHeadersSelectedChainBlockLocator(lowHash, highHash) return s.syncManager.CreateHeadersSelectedChainBlockLocator(stagingArea, lowHash, highHash)
} }
func (s *consensus) CreateHeadersSelectedChainBlockLocator(lowHash, func (s *consensus) CreateHeadersSelectedChainBlockLocator(lowHash, highHash *externalapi.DomainHash) (externalapi.BlockLocator, error) {
highHash *externalapi.DomainHash) (externalapi.BlockLocator, error) {
s.lock.Lock() s.lock.Lock()
defer s.lock.Unlock() defer s.lock.Unlock()
return s.syncManager.CreateHeadersSelectedChainBlockLocator(lowHash, highHash) stagingArea := model.NewStagingArea()
return s.syncManager.CreateHeadersSelectedChainBlockLocator(stagingArea, lowHash, highHash)
} }
func (s *consensus) GetSyncInfo() (*externalapi.SyncInfo, error) { func (s *consensus) GetSyncInfo() (*externalapi.SyncInfo, error) {
s.lock.Lock() s.lock.Lock()
defer s.lock.Unlock() defer s.lock.Unlock()
return s.syncManager.GetSyncInfo() stagingArea := model.NewStagingArea()
return s.syncManager.GetSyncInfo(stagingArea)
} }
func (s *consensus) IsValidPruningPoint(blockHash *externalapi.DomainHash) (bool, error) { func (s *consensus) IsValidPruningPoint(blockHash *externalapi.DomainHash) (bool, error) {
s.lock.Lock() s.lock.Lock()
defer s.lock.Unlock() defer s.lock.Unlock()
err := s.validateBlockHashExists(blockHash) stagingArea := model.NewStagingArea()
err := s.validateBlockHashExists(stagingArea, blockHash)
if err != nil { if err != nil {
return false, err return false, err
} }
return s.pruningManager.IsValidPruningPoint(blockHash) return s.pruningManager.IsValidPruningPoint(stagingArea, blockHash)
} }
func (s *consensus) GetVirtualSelectedParentChainFromBlock(blockHash *externalapi.DomainHash) (*externalapi.SelectedChainPath, error) { func (s *consensus) GetVirtualSelectedParentChainFromBlock(blockHash *externalapi.DomainHash) (*externalapi.SelectedChainPath, error) {
s.lock.Lock() s.lock.Lock()
defer s.lock.Unlock() defer s.lock.Unlock()
err := s.validateBlockHashExists(blockHash) stagingArea := model.NewStagingArea()
err := s.validateBlockHashExists(stagingArea, blockHash)
if err != nil { if err != nil {
return nil, err return nil, err
} }
return s.consensusStateManager.GetVirtualSelectedParentChainFromBlock(blockHash) return s.consensusStateManager.GetVirtualSelectedParentChainFromBlock(stagingArea, blockHash)
} }
func (s *consensus) validateBlockHashExists(blockHash *externalapi.DomainHash) error { func (s *consensus) validateBlockHashExists(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) error {
exists, err := s.blockStatusStore.Exists(s.databaseContext, blockHash) exists, err := s.blockStatusStore.Exists(s.databaseContext, stagingArea, blockHash)
if err != nil { if err != nil {
return err return err
} }
@ -421,33 +461,39 @@ func (s *consensus) IsInSelectedParentChainOf(blockHashA *externalapi.DomainHash
s.lock.Lock() s.lock.Lock()
defer s.lock.Unlock() defer s.lock.Unlock()
err := s.validateBlockHashExists(blockHashA) stagingArea := model.NewStagingArea()
err := s.validateBlockHashExists(stagingArea, blockHashA)
if err != nil { if err != nil {
return false, err return false, err
} }
err = s.validateBlockHashExists(blockHashB) err = s.validateBlockHashExists(stagingArea, blockHashB)
if err != nil { if err != nil {
return false, err return false, err
} }
return s.dagTopologyManager.IsInSelectedParentChainOf(blockHashA, blockHashB) return s.dagTopologyManager.IsInSelectedParentChainOf(stagingArea, blockHashA, blockHashB)
} }
func (s *consensus) GetHeadersSelectedTip() (*externalapi.DomainHash, error) { func (s *consensus) GetHeadersSelectedTip() (*externalapi.DomainHash, error) {
s.lock.Lock() s.lock.Lock()
defer s.lock.Unlock() defer s.lock.Unlock()
return s.headersSelectedTipStore.HeadersSelectedTip(s.databaseContext) stagingArea := model.NewStagingArea()
return s.headersSelectedTipStore.HeadersSelectedTip(s.databaseContext, stagingArea)
} }
func (s *consensus) Anticone(blockHash *externalapi.DomainHash) ([]*externalapi.DomainHash, error) { func (s *consensus) Anticone(blockHash *externalapi.DomainHash) ([]*externalapi.DomainHash, error) {
s.lock.Lock() s.lock.Lock()
defer s.lock.Unlock() defer s.lock.Unlock()
err := s.validateBlockHashExists(blockHash) stagingArea := model.NewStagingArea()
err := s.validateBlockHashExists(stagingArea, blockHash)
if err != nil { if err != nil {
return nil, err return nil, err
} }
return s.dagTraversalManager.Anticone(blockHash) return s.dagTraversalManager.Anticone(stagingArea, blockHash)
} }

View File

@ -0,0 +1,50 @@
package acceptancedatastore
import (
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
)
type acceptanceDataStagingShard struct {
store *acceptanceDataStore
toAdd map[externalapi.DomainHash]externalapi.AcceptanceData
toDelete map[externalapi.DomainHash]struct{}
}
func (ads *acceptanceDataStore) stagingShard(stagingArea *model.StagingArea) *acceptanceDataStagingShard {
return stagingArea.GetOrCreateShard(model.StagingShardIDAcceptanceData, func() model.StagingShard {
return &acceptanceDataStagingShard{
store: ads,
toAdd: make(map[externalapi.DomainHash]externalapi.AcceptanceData),
toDelete: make(map[externalapi.DomainHash]struct{}),
}
}).(*acceptanceDataStagingShard)
}
func (adss *acceptanceDataStagingShard) Commit(dbTx model.DBTransaction) error {
for hash, acceptanceData := range adss.toAdd {
acceptanceDataBytes, err := adss.store.serializeAcceptanceData(acceptanceData)
if err != nil {
return err
}
err = dbTx.Put(adss.store.hashAsKey(&hash), acceptanceDataBytes)
if err != nil {
return err
}
adss.store.cache.Add(&hash, acceptanceData)
}
for hash := range adss.toDelete {
err := dbTx.Delete(adss.store.hashAsKey(&hash))
if err != nil {
return err
}
adss.store.cache.Remove(&hash)
}
return nil
}
func (adss *acceptanceDataStagingShard) isStaged() bool {
return len(adss.toAdd) != 0 || len(adss.toDelete) != 0
}

View File

@ -13,62 +13,31 @@ var bucket = database.MakeBucket([]byte("acceptance-data"))
// acceptanceDataStore represents a store of AcceptanceData // acceptanceDataStore represents a store of AcceptanceData
type acceptanceDataStore struct { type acceptanceDataStore struct {
staging map[externalapi.DomainHash]externalapi.AcceptanceData
toDelete map[externalapi.DomainHash]struct{}
cache *lrucache.LRUCache cache *lrucache.LRUCache
} }
// New instantiates a new AcceptanceDataStore // New instantiates a new AcceptanceDataStore
func New(cacheSize int, preallocate bool) model.AcceptanceDataStore { func New(cacheSize int, preallocate bool) model.AcceptanceDataStore {
return &acceptanceDataStore{ return &acceptanceDataStore{
staging: make(map[externalapi.DomainHash]externalapi.AcceptanceData),
toDelete: make(map[externalapi.DomainHash]struct{}),
cache: lrucache.New(cacheSize, preallocate), cache: lrucache.New(cacheSize, preallocate),
} }
} }
// Stage stages the given acceptanceData for the given blockHash // Stage stages the given acceptanceData for the given blockHash
func (ads *acceptanceDataStore) Stage(blockHash *externalapi.DomainHash, acceptanceData externalapi.AcceptanceData) { func (ads *acceptanceDataStore) Stage(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash, acceptanceData externalapi.AcceptanceData) {
ads.staging[*blockHash] = acceptanceData.Clone() stagingShard := ads.stagingShard(stagingArea)
stagingShard.toAdd[*blockHash] = acceptanceData.Clone()
} }
func (ads *acceptanceDataStore) IsStaged() bool { func (ads *acceptanceDataStore) IsStaged(stagingArea *model.StagingArea) bool {
return len(ads.staging) != 0 || len(ads.toDelete) != 0 return ads.stagingShard(stagingArea).isStaged()
}
func (ads *acceptanceDataStore) Discard() {
ads.staging = make(map[externalapi.DomainHash]externalapi.AcceptanceData)
ads.toDelete = make(map[externalapi.DomainHash]struct{})
}
func (ads *acceptanceDataStore) Commit(dbTx model.DBTransaction) error {
for hash, acceptanceData := range ads.staging {
acceptanceDataBytes, err := ads.serializeAcceptanceData(acceptanceData)
if err != nil {
return err
}
err = dbTx.Put(ads.hashAsKey(&hash), acceptanceDataBytes)
if err != nil {
return err
}
ads.cache.Add(&hash, acceptanceData)
}
for hash := range ads.toDelete {
err := dbTx.Delete(ads.hashAsKey(&hash))
if err != nil {
return err
}
ads.cache.Remove(&hash)
}
ads.Discard()
return nil
} }
// Get gets the acceptanceData associated with the given blockHash // Get gets the acceptanceData associated with the given blockHash
func (ads *acceptanceDataStore) Get(dbContext model.DBReader, blockHash *externalapi.DomainHash) (externalapi.AcceptanceData, error) { func (ads *acceptanceDataStore) Get(dbContext model.DBReader, stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) (externalapi.AcceptanceData, error) {
if acceptanceData, ok := ads.staging[*blockHash]; ok { stagingShard := ads.stagingShard(stagingArea)
if acceptanceData, ok := stagingShard.toAdd[*blockHash]; ok {
return acceptanceData.Clone(), nil return acceptanceData.Clone(), nil
} }
@ -90,12 +59,14 @@ func (ads *acceptanceDataStore) Get(dbContext model.DBReader, blockHash *externa
} }
// Delete deletes the acceptanceData associated with the given blockHash // Delete deletes the acceptanceData associated with the given blockHash
func (ads *acceptanceDataStore) Delete(blockHash *externalapi.DomainHash) { func (ads *acceptanceDataStore) Delete(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) {
if _, ok := ads.staging[*blockHash]; ok { stagingShard := ads.stagingShard(stagingArea)
delete(ads.staging, *blockHash)
if _, ok := stagingShard.toAdd[*blockHash]; ok {
delete(stagingShard.toAdd, *blockHash)
return return
} }
ads.toDelete[*blockHash] = struct{}{} stagingShard.toDelete[*blockHash] = struct{}{}
} }
func (ads *acceptanceDataStore) serializeAcceptanceData(acceptanceData externalapi.AcceptanceData) ([]byte, error) { func (ads *acceptanceDataStore) serializeAcceptanceData(acceptanceData externalapi.AcceptanceData) ([]byte, error) {

View File

@ -0,0 +1,69 @@
package blockheaderstore
import (
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
)
type blockHeaderStagingShard struct {
store *blockHeaderStore
toAdd map[externalapi.DomainHash]externalapi.BlockHeader
toDelete map[externalapi.DomainHash]struct{}
}
func (bhs *blockHeaderStore) stagingShard(stagingArea *model.StagingArea) *blockHeaderStagingShard {
return stagingArea.GetOrCreateShard(model.StagingShardIDBlockHeader, func() model.StagingShard {
return &blockHeaderStagingShard{
store: bhs,
toAdd: make(map[externalapi.DomainHash]externalapi.BlockHeader),
toDelete: make(map[externalapi.DomainHash]struct{}),
}
}).(*blockHeaderStagingShard)
}
func (bhss *blockHeaderStagingShard) Commit(dbTx model.DBTransaction) error {
for hash, header := range bhss.toAdd {
headerBytes, err := bhss.store.serializeHeader(header)
if err != nil {
return err
}
err = dbTx.Put(bhss.store.hashAsKey(&hash), headerBytes)
if err != nil {
return err
}
bhss.store.cache.Add(&hash, header)
}
for hash := range bhss.toDelete {
err := dbTx.Delete(bhss.store.hashAsKey(&hash))
if err != nil {
return err
}
bhss.store.cache.Remove(&hash)
}
err := bhss.commitCount(dbTx)
if err != nil {
return err
}
return nil
}
func (bhss *blockHeaderStagingShard) commitCount(dbTx model.DBTransaction) error {
count := bhss.store.count(bhss)
countBytes, err := bhss.store.serializeHeaderCount(count)
if err != nil {
return err
}
err = dbTx.Put(countKey, countBytes)
if err != nil {
return err
}
bhss.store.countCached = count
return nil
}
func (bhss *blockHeaderStagingShard) isStaged() bool {
return len(bhss.toAdd) != 0 || len(bhss.toDelete) != 0
}

View File

@ -14,17 +14,13 @@ var countKey = database.MakeBucket(nil).Key([]byte("block-headers-count"))
// blockHeaderStore represents a store of blocks // blockHeaderStore represents a store of blocks
type blockHeaderStore struct { type blockHeaderStore struct {
staging map[externalapi.DomainHash]externalapi.BlockHeader
toDelete map[externalapi.DomainHash]struct{}
cache *lrucache.LRUCache cache *lrucache.LRUCache
count uint64 countCached uint64
} }
// New instantiates a new BlockHeaderStore // New instantiates a new BlockHeaderStore
func New(dbContext model.DBReader, cacheSize int, preallocate bool) (model.BlockHeaderStore, error) { func New(dbContext model.DBReader, cacheSize int, preallocate bool) (model.BlockHeaderStore, error) {
blockHeaderStore := &blockHeaderStore{ blockHeaderStore := &blockHeaderStore{
staging: make(map[externalapi.DomainHash]externalapi.BlockHeader),
toDelete: make(map[externalapi.DomainHash]struct{}),
cache: lrucache.New(cacheSize, preallocate), cache: lrucache.New(cacheSize, preallocate),
} }
@ -52,57 +48,33 @@ func (bhs *blockHeaderStore) initializeCount(dbContext model.DBReader) error {
return err return err
} }
} }
bhs.count = count bhs.countCached = count
return nil return nil
} }
// Stage stages the given block header for the given blockHash // Stage stages the given block header for the given blockHash
func (bhs *blockHeaderStore) Stage(blockHash *externalapi.DomainHash, blockHeader externalapi.BlockHeader) { func (bhs *blockHeaderStore) Stage(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash, blockHeader externalapi.BlockHeader) {
bhs.staging[*blockHash] = blockHeader stagingShard := bhs.stagingShard(stagingArea)
stagingShard.toAdd[*blockHash] = blockHeader
} }
func (bhs *blockHeaderStore) IsStaged() bool { func (bhs *blockHeaderStore) IsStaged(stagingArea *model.StagingArea) bool {
return len(bhs.staging) != 0 || len(bhs.toDelete) != 0 return bhs.stagingShard(stagingArea).isStaged()
}
func (bhs *blockHeaderStore) Discard() {
bhs.staging = make(map[externalapi.DomainHash]externalapi.BlockHeader)
bhs.toDelete = make(map[externalapi.DomainHash]struct{})
}
func (bhs *blockHeaderStore) Commit(dbTx model.DBTransaction) error {
for hash, header := range bhs.staging {
headerBytes, err := bhs.serializeHeader(header)
if err != nil {
return err
}
err = dbTx.Put(bhs.hashAsKey(&hash), headerBytes)
if err != nil {
return err
}
bhs.cache.Add(&hash, header)
}
for hash := range bhs.toDelete {
err := dbTx.Delete(bhs.hashAsKey(&hash))
if err != nil {
return err
}
bhs.cache.Remove(&hash)
}
err := bhs.commitCount(dbTx)
if err != nil {
return err
}
bhs.Discard()
return nil
} }
// BlockHeader gets the block header associated with the given blockHash // BlockHeader gets the block header associated with the given blockHash
func (bhs *blockHeaderStore) BlockHeader(dbContext model.DBReader, blockHash *externalapi.DomainHash) (externalapi.BlockHeader, error) { func (bhs *blockHeaderStore) BlockHeader(dbContext model.DBReader, stagingArea *model.StagingArea,
if header, ok := bhs.staging[*blockHash]; ok { blockHash *externalapi.DomainHash) (externalapi.BlockHeader, error) {
stagingShard := bhs.stagingShard(stagingArea)
return bhs.blockHeader(dbContext, stagingShard, blockHash)
}
func (bhs *blockHeaderStore) blockHeader(dbContext model.DBReader, stagingShard *blockHeaderStagingShard,
blockHash *externalapi.DomainHash) (externalapi.BlockHeader, error) {
if header, ok := stagingShard.toAdd[*blockHash]; ok {
return header, nil return header, nil
} }
@ -124,8 +96,10 @@ func (bhs *blockHeaderStore) BlockHeader(dbContext model.DBReader, blockHash *ex
} }
// HasBlock returns whether a block header with a given hash exists in the store. // HasBlock returns whether a block header with a given hash exists in the store.
func (bhs *blockHeaderStore) HasBlockHeader(dbContext model.DBReader, blockHash *externalapi.DomainHash) (bool, error) { func (bhs *blockHeaderStore) HasBlockHeader(dbContext model.DBReader, stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) (bool, error) {
if _, ok := bhs.staging[*blockHash]; ok { stagingShard := bhs.stagingShard(stagingArea)
if _, ok := stagingShard.toAdd[*blockHash]; ok {
return true, nil return true, nil
} }
@ -142,11 +116,15 @@ func (bhs *blockHeaderStore) HasBlockHeader(dbContext model.DBReader, blockHash
} }
// BlockHeaders gets the block headers associated with the given blockHashes // BlockHeaders gets the block headers associated with the given blockHashes
func (bhs *blockHeaderStore) BlockHeaders(dbContext model.DBReader, blockHashes []*externalapi.DomainHash) ([]externalapi.BlockHeader, error) { func (bhs *blockHeaderStore) BlockHeaders(dbContext model.DBReader, stagingArea *model.StagingArea,
blockHashes []*externalapi.DomainHash) ([]externalapi.BlockHeader, error) {
stagingShard := bhs.stagingShard(stagingArea)
headers := make([]externalapi.BlockHeader, len(blockHashes)) headers := make([]externalapi.BlockHeader, len(blockHashes))
for i, hash := range blockHashes { for i, hash := range blockHashes {
var err error var err error
headers[i], err = bhs.BlockHeader(dbContext, hash) headers[i], err = bhs.blockHeader(dbContext, stagingShard, hash)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -155,12 +133,14 @@ func (bhs *blockHeaderStore) BlockHeaders(dbContext model.DBReader, blockHashes
} }
// Delete deletes the block associated with the given blockHash // Delete deletes the block associated with the given blockHash
func (bhs *blockHeaderStore) Delete(blockHash *externalapi.DomainHash) { func (bhs *blockHeaderStore) Delete(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) {
if _, ok := bhs.staging[*blockHash]; ok { stagingShard := bhs.stagingShard(stagingArea)
delete(bhs.staging, *blockHash)
if _, ok := stagingShard.toAdd[*blockHash]; ok {
delete(stagingShard.toAdd, *blockHash)
return return
} }
bhs.toDelete[*blockHash] = struct{}{} stagingShard.toDelete[*blockHash] = struct{}{}
} }
func (bhs *blockHeaderStore) hashAsKey(hash *externalapi.DomainHash) model.DBKey { func (bhs *blockHeaderStore) hashAsKey(hash *externalapi.DomainHash) model.DBKey {
@ -181,8 +161,14 @@ func (bhs *blockHeaderStore) deserializeHeader(headerBytes []byte) (externalapi.
return serialization.DbBlockHeaderToDomainBlockHeader(dbBlockHeader) return serialization.DbBlockHeaderToDomainBlockHeader(dbBlockHeader)
} }
func (bhs *blockHeaderStore) Count() uint64 { func (bhs *blockHeaderStore) Count(stagingArea *model.StagingArea) uint64 {
return bhs.count + uint64(len(bhs.staging)) - uint64(len(bhs.toDelete)) stagingShard := bhs.stagingShard(stagingArea)
return bhs.count(stagingShard)
}
func (bhs *blockHeaderStore) count(stagingShard *blockHeaderStagingShard) uint64 {
return bhs.countCached + uint64(len(stagingShard.toAdd)) - uint64(len(stagingShard.toDelete))
} }
func (bhs *blockHeaderStore) deserializeHeaderCount(countBytes []byte) (uint64, error) { func (bhs *blockHeaderStore) deserializeHeaderCount(countBytes []byte) (uint64, error) {
@ -194,20 +180,6 @@ func (bhs *blockHeaderStore) deserializeHeaderCount(countBytes []byte) (uint64,
return dbBlockHeaderCount.Count, nil return dbBlockHeaderCount.Count, nil
} }
func (bhs *blockHeaderStore) commitCount(dbTx model.DBTransaction) error {
count := bhs.Count()
countBytes, err := bhs.serializeHeaderCount(count)
if err != nil {
return err
}
err = dbTx.Put(countKey, countBytes)
if err != nil {
return err
}
bhs.count = count
return nil
}
func (bhs *blockHeaderStore) serializeHeaderCount(count uint64) ([]byte, error) { func (bhs *blockHeaderStore) serializeHeaderCount(count uint64) ([]byte, error) {
dbBlockHeaderCount := &serialization.DbBlockHeaderCount{Count: count} dbBlockHeaderCount := &serialization.DbBlockHeaderCount{Count: count}
return proto.Marshal(dbBlockHeaderCount) return proto.Marshal(dbBlockHeaderCount)

View File

@ -0,0 +1,40 @@
package blockrelationstore
import (
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
)
type blockRelationStagingShard struct {
store *blockRelationStore
toAdd map[externalapi.DomainHash]*model.BlockRelations
}
func (brs *blockRelationStore) stagingShard(stagingArea *model.StagingArea) *blockRelationStagingShard {
return stagingArea.GetOrCreateShard(model.StagingShardIDBlockRelation, func() model.StagingShard {
return &blockRelationStagingShard{
store: brs,
toAdd: make(map[externalapi.DomainHash]*model.BlockRelations),
}
}).(*blockRelationStagingShard)
}
func (brss *blockRelationStagingShard) Commit(dbTx model.DBTransaction) error {
for hash, blockRelations := range brss.toAdd {
blockRelationBytes, err := brss.store.serializeBlockRelations(blockRelations)
if err != nil {
return err
}
err = dbTx.Put(brss.store.hashAsKey(&hash), blockRelationBytes)
if err != nil {
return err
}
brss.store.cache.Add(&hash, blockRelations)
}
return nil
}
func (brss *blockRelationStagingShard) isStaged() bool {
return len(brss.toAdd) != 0
}

View File

@ -13,49 +13,30 @@ var bucket = database.MakeBucket([]byte("block-relations"))
// blockRelationStore represents a store of BlockRelations // blockRelationStore represents a store of BlockRelations
type blockRelationStore struct { type blockRelationStore struct {
staging map[externalapi.DomainHash]*model.BlockRelations
cache *lrucache.LRUCache cache *lrucache.LRUCache
} }
// New instantiates a new BlockRelationStore // New instantiates a new BlockRelationStore
func New(cacheSize int, preallocate bool) model.BlockRelationStore { func New(cacheSize int, preallocate bool) model.BlockRelationStore {
return &blockRelationStore{ return &blockRelationStore{
staging: make(map[externalapi.DomainHash]*model.BlockRelations),
cache: lrucache.New(cacheSize, preallocate), cache: lrucache.New(cacheSize, preallocate),
} }
} }
func (brs *blockRelationStore) StageBlockRelation(blockHash *externalapi.DomainHash, blockRelations *model.BlockRelations) { func (brs *blockRelationStore) StageBlockRelation(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash, blockRelations *model.BlockRelations) {
brs.staging[*blockHash] = blockRelations.Clone() stagingShard := brs.stagingShard(stagingArea)
stagingShard.toAdd[*blockHash] = blockRelations.Clone()
} }
func (brs *blockRelationStore) IsStaged() bool { func (brs *blockRelationStore) IsStaged(stagingArea *model.StagingArea) bool {
return len(brs.staging) != 0 return brs.stagingShard(stagingArea).isStaged()
} }
func (brs *blockRelationStore) Discard() { func (brs *blockRelationStore) BlockRelation(dbContext model.DBReader, stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) (*model.BlockRelations, error) {
brs.staging = make(map[externalapi.DomainHash]*model.BlockRelations) stagingShard := brs.stagingShard(stagingArea)
}
func (brs *blockRelationStore) Commit(dbTx model.DBTransaction) error { if blockRelations, ok := stagingShard.toAdd[*blockHash]; ok {
for hash, blockRelations := range brs.staging {
blockRelationBytes, err := brs.serializeBlockRelations(blockRelations)
if err != nil {
return err
}
err = dbTx.Put(brs.hashAsKey(&hash), blockRelationBytes)
if err != nil {
return err
}
brs.cache.Add(&hash, blockRelations)
}
brs.Discard()
return nil
}
func (brs *blockRelationStore) BlockRelation(dbContext model.DBReader, blockHash *externalapi.DomainHash) (*model.BlockRelations, error) {
if blockRelations, ok := brs.staging[*blockHash]; ok {
return blockRelations.Clone(), nil return blockRelations.Clone(), nil
} }
@ -76,8 +57,10 @@ func (brs *blockRelationStore) BlockRelation(dbContext model.DBReader, blockHash
return blockRelations.Clone(), nil return blockRelations.Clone(), nil
} }
func (brs *blockRelationStore) Has(dbContext model.DBReader, blockHash *externalapi.DomainHash) (bool, error) { func (brs *blockRelationStore) Has(dbContext model.DBReader, stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) (bool, error) {
if _, ok := brs.staging[*blockHash]; ok { stagingShard := brs.stagingShard(stagingArea)
if _, ok := stagingShard.toAdd[*blockHash]; ok {
return true, nil return true, nil
} }

View File

@ -0,0 +1,40 @@
package blockstatusstore
import (
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
)
type blockStatusStagingShard struct {
store *blockStatusStore
toAdd map[externalapi.DomainHash]externalapi.BlockStatus
}
func (bss *blockStatusStore) stagingShard(stagingArea *model.StagingArea) *blockStatusStagingShard {
return stagingArea.GetOrCreateShard(model.StagingShardIDBlockStatus, func() model.StagingShard {
return &blockStatusStagingShard{
store: bss,
toAdd: make(map[externalapi.DomainHash]externalapi.BlockStatus),
}
}).(*blockStatusStagingShard)
}
func (bsss *blockStatusStagingShard) Commit(dbTx model.DBTransaction) error {
for hash, status := range bsss.toAdd {
blockStatusBytes, err := bsss.store.serializeBlockStatus(status)
if err != nil {
return err
}
err = dbTx.Put(bsss.store.hashAsKey(&hash), blockStatusBytes)
if err != nil {
return err
}
bsss.store.cache.Add(&hash, status)
}
return nil
}
func (bsss *blockStatusStagingShard) isStaged() bool {
return len(bsss.toAdd) != 0
}

View File

@ -13,51 +13,31 @@ var bucket = database.MakeBucket([]byte("block-statuses"))
// blockStatusStore represents a store of BlockStatuses // blockStatusStore represents a store of BlockStatuses
type blockStatusStore struct { type blockStatusStore struct {
staging map[externalapi.DomainHash]externalapi.BlockStatus
cache *lrucache.LRUCache cache *lrucache.LRUCache
} }
// New instantiates a new BlockStatusStore // New instantiates a new BlockStatusStore
func New(cacheSize int, preallocate bool) model.BlockStatusStore { func New(cacheSize int, preallocate bool) model.BlockStatusStore {
return &blockStatusStore{ return &blockStatusStore{
staging: make(map[externalapi.DomainHash]externalapi.BlockStatus),
cache: lrucache.New(cacheSize, preallocate), cache: lrucache.New(cacheSize, preallocate),
} }
} }
// Stage stages the given blockStatus for the given blockHash // Stage stages the given blockStatus for the given blockHash
func (bss *blockStatusStore) Stage(blockHash *externalapi.DomainHash, blockStatus externalapi.BlockStatus) { func (bss *blockStatusStore) Stage(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash, blockStatus externalapi.BlockStatus) {
bss.staging[*blockHash] = blockStatus.Clone() stagingShard := bss.stagingShard(stagingArea)
stagingShard.toAdd[*blockHash] = blockStatus.Clone()
} }
func (bss *blockStatusStore) IsStaged() bool { func (bss *blockStatusStore) IsStaged(stagingArea *model.StagingArea) bool {
return len(bss.staging) != 0 return bss.stagingShard(stagingArea).isStaged()
}
func (bss *blockStatusStore) Discard() {
bss.staging = make(map[externalapi.DomainHash]externalapi.BlockStatus)
}
func (bss *blockStatusStore) Commit(dbTx model.DBTransaction) error {
for hash, status := range bss.staging {
blockStatusBytes, err := bss.serializeBlockStatus(status)
if err != nil {
return err
}
err = dbTx.Put(bss.hashAsKey(&hash), blockStatusBytes)
if err != nil {
return err
}
bss.cache.Add(&hash, status)
}
bss.Discard()
return nil
} }
// Get gets the blockStatus associated with the given blockHash // Get gets the blockStatus associated with the given blockHash
func (bss *blockStatusStore) Get(dbContext model.DBReader, blockHash *externalapi.DomainHash) (externalapi.BlockStatus, error) { func (bss *blockStatusStore) Get(dbContext model.DBReader, stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) (externalapi.BlockStatus, error) {
if status, ok := bss.staging[*blockHash]; ok { stagingShard := bss.stagingShard(stagingArea)
if status, ok := stagingShard.toAdd[*blockHash]; ok {
return status, nil return status, nil
} }
@ -79,8 +59,10 @@ func (bss *blockStatusStore) Get(dbContext model.DBReader, blockHash *externalap
} }
// Exists returns true if the blockStatus for the given blockHash exists // Exists returns true if the blockStatus for the given blockHash exists
func (bss *blockStatusStore) Exists(dbContext model.DBReader, blockHash *externalapi.DomainHash) (bool, error) { func (bss *blockStatusStore) Exists(dbContext model.DBReader, stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) (bool, error) {
if _, ok := bss.staging[*blockHash]; ok { stagingShard := bss.stagingShard(stagingArea)
if _, ok := stagingShard.toAdd[*blockHash]; ok {
return true, nil return true, nil
} }

View File

@ -0,0 +1,69 @@
package blockstore
import (
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
)
type blockStagingShard struct {
store *blockStore
toAdd map[externalapi.DomainHash]*externalapi.DomainBlock
toDelete map[externalapi.DomainHash]struct{}
}
func (bs *blockStore) stagingShard(stagingArea *model.StagingArea) *blockStagingShard {
return stagingArea.GetOrCreateShard(model.StagingShardIDBlock, func() model.StagingShard {
return &blockStagingShard{
store: bs,
toAdd: make(map[externalapi.DomainHash]*externalapi.DomainBlock),
toDelete: make(map[externalapi.DomainHash]struct{}),
}
}).(*blockStagingShard)
}
func (bss *blockStagingShard) Commit(dbTx model.DBTransaction) error {
for hash, block := range bss.toAdd {
blockBytes, err := bss.store.serializeBlock(block)
if err != nil {
return err
}
err = dbTx.Put(bss.store.hashAsKey(&hash), blockBytes)
if err != nil {
return err
}
bss.store.cache.Add(&hash, block)
}
for hash := range bss.toDelete {
err := dbTx.Delete(bss.store.hashAsKey(&hash))
if err != nil {
return err
}
bss.store.cache.Remove(&hash)
}
err := bss.commitCount(dbTx)
if err != nil {
return err
}
return nil
}
func (bss *blockStagingShard) commitCount(dbTx model.DBTransaction) error {
count := bss.store.count(bss)
countBytes, err := bss.store.serializeBlockCount(count)
if err != nil {
return err
}
err = dbTx.Put(countKey, countBytes)
if err != nil {
return err
}
bss.store.countCached = count
return nil
}
func (bss *blockStagingShard) isStaged() bool {
return len(bss.toAdd) != 0 || len(bss.toDelete) != 0
}

View File

@ -15,17 +15,13 @@ var countKey = database.MakeBucket(nil).Key([]byte("blocks-count"))
// blockStore represents a store of blocks // blockStore represents a store of blocks
type blockStore struct { type blockStore struct {
staging map[externalapi.DomainHash]*externalapi.DomainBlock
toDelete map[externalapi.DomainHash]struct{}
cache *lrucache.LRUCache cache *lrucache.LRUCache
count uint64 countCached uint64
} }
// New instantiates a new BlockStore // New instantiates a new BlockStore
func New(dbContext model.DBReader, cacheSize int, preallocate bool) (model.BlockStore, error) { func New(dbContext model.DBReader, cacheSize int, preallocate bool) (model.BlockStore, error) {
blockStore := &blockStore{ blockStore := &blockStore{
staging: make(map[externalapi.DomainHash]*externalapi.DomainBlock),
toDelete: make(map[externalapi.DomainHash]struct{}),
cache: lrucache.New(cacheSize, preallocate), cache: lrucache.New(cacheSize, preallocate),
} }
@ -53,57 +49,29 @@ func (bs *blockStore) initializeCount(dbContext model.DBReader) error {
return err return err
} }
} }
bs.count = count bs.countCached = count
return nil return nil
} }
// Stage stages the given block for the given blockHash // Stage stages the given block for the given blockHash
func (bs *blockStore) Stage(blockHash *externalapi.DomainHash, block *externalapi.DomainBlock) { func (bs *blockStore) Stage(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash, block *externalapi.DomainBlock) {
bs.staging[*blockHash] = block.Clone() stagingShard := bs.stagingShard(stagingArea)
stagingShard.toAdd[*blockHash] = block.Clone()
} }
func (bs *blockStore) IsStaged() bool { func (bs *blockStore) IsStaged(stagingArea *model.StagingArea) bool {
return len(bs.staging) != 0 || len(bs.toDelete) != 0 return bs.stagingShard(stagingArea).isStaged()
}
func (bs *blockStore) Discard() {
bs.staging = make(map[externalapi.DomainHash]*externalapi.DomainBlock)
bs.toDelete = make(map[externalapi.DomainHash]struct{})
}
func (bs *blockStore) Commit(dbTx model.DBTransaction) error {
for hash, block := range bs.staging {
blockBytes, err := bs.serializeBlock(block)
if err != nil {
return err
}
err = dbTx.Put(bs.hashAsKey(&hash), blockBytes)
if err != nil {
return err
}
bs.cache.Add(&hash, block)
}
for hash := range bs.toDelete {
err := dbTx.Delete(bs.hashAsKey(&hash))
if err != nil {
return err
}
bs.cache.Remove(&hash)
}
err := bs.commitCount(dbTx)
if err != nil {
return err
}
bs.Discard()
return nil
} }
// Block gets the block associated with the given blockHash // Block gets the block associated with the given blockHash
func (bs *blockStore) Block(dbContext model.DBReader, blockHash *externalapi.DomainHash) (*externalapi.DomainBlock, error) { func (bs *blockStore) Block(dbContext model.DBReader, stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) (*externalapi.DomainBlock, error) {
if block, ok := bs.staging[*blockHash]; ok { stagingShard := bs.stagingShard(stagingArea)
return bs.block(dbContext, stagingShard, blockHash)
}
func (bs *blockStore) block(dbContext model.DBReader, stagingShard *blockStagingShard, blockHash *externalapi.DomainHash) (*externalapi.DomainBlock, error) {
if block, ok := stagingShard.toAdd[*blockHash]; ok {
return block.Clone(), nil return block.Clone(), nil
} }
@ -125,8 +93,10 @@ func (bs *blockStore) Block(dbContext model.DBReader, blockHash *externalapi.Dom
} }
// HasBlock returns whether a block with a given hash exists in the store. // HasBlock returns whether a block with a given hash exists in the store.
func (bs *blockStore) HasBlock(dbContext model.DBReader, blockHash *externalapi.DomainHash) (bool, error) { func (bs *blockStore) HasBlock(dbContext model.DBReader, stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) (bool, error) {
if _, ok := bs.staging[*blockHash]; ok { stagingShard := bs.stagingShard(stagingArea)
if _, ok := stagingShard.toAdd[*blockHash]; ok {
return true, nil return true, nil
} }
@ -143,11 +113,13 @@ func (bs *blockStore) HasBlock(dbContext model.DBReader, blockHash *externalapi.
} }
// Blocks gets the blocks associated with the given blockHashes // Blocks gets the blocks associated with the given blockHashes
func (bs *blockStore) Blocks(dbContext model.DBReader, blockHashes []*externalapi.DomainHash) ([]*externalapi.DomainBlock, error) { func (bs *blockStore) Blocks(dbContext model.DBReader, stagingArea *model.StagingArea, blockHashes []*externalapi.DomainHash) ([]*externalapi.DomainBlock, error) {
stagingShard := bs.stagingShard(stagingArea)
blocks := make([]*externalapi.DomainBlock, len(blockHashes)) blocks := make([]*externalapi.DomainBlock, len(blockHashes))
for i, hash := range blockHashes { for i, hash := range blockHashes {
var err error var err error
blocks[i], err = bs.Block(dbContext, hash) blocks[i], err = bs.block(dbContext, stagingShard, hash)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -156,12 +128,14 @@ func (bs *blockStore) Blocks(dbContext model.DBReader, blockHashes []*externalap
} }
// Delete deletes the block associated with the given blockHash // Delete deletes the block associated with the given blockHash
func (bs *blockStore) Delete(blockHash *externalapi.DomainHash) { func (bs *blockStore) Delete(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) {
if _, ok := bs.staging[*blockHash]; ok { stagingShard := bs.stagingShard(stagingArea)
delete(bs.staging, *blockHash)
if _, ok := stagingShard.toAdd[*blockHash]; ok {
delete(stagingShard.toAdd, *blockHash)
return return
} }
bs.toDelete[*blockHash] = struct{}{} stagingShard.toDelete[*blockHash] = struct{}{}
} }
func (bs *blockStore) serializeBlock(block *externalapi.DomainBlock) ([]byte, error) { func (bs *blockStore) serializeBlock(block *externalapi.DomainBlock) ([]byte, error) {
@ -182,8 +156,13 @@ func (bs *blockStore) hashAsKey(hash *externalapi.DomainHash) model.DBKey {
return bucket.Key(hash.ByteSlice()) return bucket.Key(hash.ByteSlice())
} }
func (bs *blockStore) Count() uint64 { func (bs *blockStore) Count(stagingArea *model.StagingArea) uint64 {
return bs.count + uint64(len(bs.staging)) - uint64(len(bs.toDelete)) stagingShard := bs.stagingShard(stagingArea)
return bs.count(stagingShard)
}
func (bs *blockStore) count(stagingShard *blockStagingShard) uint64 {
return bs.countCached + uint64(len(stagingShard.toAdd)) - uint64(len(stagingShard.toDelete))
} }
func (bs *blockStore) deserializeBlockCount(countBytes []byte) (uint64, error) { func (bs *blockStore) deserializeBlockCount(countBytes []byte) (uint64, error) {
@ -195,20 +174,6 @@ func (bs *blockStore) deserializeBlockCount(countBytes []byte) (uint64, error) {
return dbBlockCount.Count, nil return dbBlockCount.Count, nil
} }
func (bs *blockStore) commitCount(dbTx model.DBTransaction) error {
count := bs.Count()
countBytes, err := bs.serializeBlockCount(count)
if err != nil {
return err
}
err = dbTx.Put(countKey, countBytes)
if err != nil {
return err
}
bs.count = count
return nil
}
func (bs *blockStore) serializeBlockCount(count uint64) ([]byte, error) { func (bs *blockStore) serializeBlockCount(count uint64) ([]byte, error) {
dbBlockCount := &serialization.DbBlockCount{Count: count} dbBlockCount := &serialization.DbBlockCount{Count: count}
return proto.Marshal(dbBlockCount) return proto.Marshal(dbBlockCount)

View File

@ -0,0 +1,40 @@
package consensusstatestore
import (
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
)
type consensusStateStagingShard struct {
store *consensusStateStore
tipsStaging []*externalapi.DomainHash
virtualUTXODiffStaging externalapi.UTXODiff
}
func (bs *consensusStateStore) stagingShard(stagingArea *model.StagingArea) *consensusStateStagingShard {
return stagingArea.GetOrCreateShard(model.StagingShardIDConsensusState, func() model.StagingShard {
return &consensusStateStagingShard{
store: bs,
tipsStaging: nil,
virtualUTXODiffStaging: nil,
}
}).(*consensusStateStagingShard)
}
func (csss *consensusStateStagingShard) Commit(dbTx model.DBTransaction) error {
err := csss.commitTips(dbTx)
if err != nil {
return err
}
err = csss.commitVirtualUTXODiff(dbTx)
if err != nil {
return err
}
return nil
}
func (csss *consensusStateStagingShard) isStaged() bool {
return csss.tipsStaging != nil || csss.virtualUTXODiffStaging != nil
}

View File

@ -8,9 +8,6 @@ import (
// consensusStateStore represents a store for the current consensus state // consensusStateStore represents a store for the current consensus state
type consensusStateStore struct { type consensusStateStore struct {
tipsStaging []*externalapi.DomainHash
virtualUTXODiffStaging externalapi.UTXODiff
virtualUTXOSetCache *utxolrucache.LRUCache virtualUTXOSetCache *utxolrucache.LRUCache
tipsCache []*externalapi.DomainHash tipsCache []*externalapi.DomainHash
@ -23,28 +20,6 @@ func New(utxoSetCacheSize int, preallocate bool) model.ConsensusStateStore {
} }
} }
func (css *consensusStateStore) Discard() { func (css *consensusStateStore) IsStaged(stagingArea *model.StagingArea) bool {
css.tipsStaging = nil return css.stagingShard(stagingArea).isStaged()
css.virtualUTXODiffStaging = nil
}
func (css *consensusStateStore) Commit(dbTx model.DBTransaction) error {
err := css.commitTips(dbTx)
if err != nil {
return err
}
err = css.commitVirtualUTXODiff(dbTx)
if err != nil {
return err
}
css.Discard()
return nil
}
func (css *consensusStateStore) IsStaged() bool {
return css.tipsStaging != nil ||
css.virtualUTXODiffStaging != nil
} }

View File

@ -10,9 +10,11 @@ import (
var tipsKey = database.MakeBucket(nil).Key([]byte("tips")) var tipsKey = database.MakeBucket(nil).Key([]byte("tips"))
func (css *consensusStateStore) Tips(dbContext model.DBReader) ([]*externalapi.DomainHash, error) { func (css *consensusStateStore) Tips(stagingArea *model.StagingArea, dbContext model.DBReader) ([]*externalapi.DomainHash, error) {
if css.tipsStaging != nil { stagingShard := css.stagingShard(stagingArea)
return externalapi.CloneHashes(css.tipsStaging), nil
if stagingShard.tipsStaging != nil {
return externalapi.CloneHashes(stagingShard.tipsStaging), nil
} }
if css.tipsCache != nil { if css.tipsCache != nil {
@ -32,28 +34,10 @@ func (css *consensusStateStore) Tips(dbContext model.DBReader) ([]*externalapi.D
return externalapi.CloneHashes(tips), nil return externalapi.CloneHashes(tips), nil
} }
func (css *consensusStateStore) StageTips(tipHashes []*externalapi.DomainHash) { func (css *consensusStateStore) StageTips(stagingArea *model.StagingArea, tipHashes []*externalapi.DomainHash) {
css.tipsStaging = externalapi.CloneHashes(tipHashes) stagingShard := css.stagingShard(stagingArea)
}
func (css *consensusStateStore) commitTips(dbTx model.DBTransaction) error { stagingShard.tipsStaging = externalapi.CloneHashes(tipHashes)
if css.tipsStaging == nil {
return nil
}
tipsBytes, err := css.serializeTips(css.tipsStaging)
if err != nil {
return err
}
err = dbTx.Put(tipsKey, tipsBytes)
if err != nil {
return err
}
css.tipsCache = css.tipsStaging
// Note: we don't discard the staging here since that's
// being done at the end of Commit()
return nil
} }
func (css *consensusStateStore) serializeTips(tips []*externalapi.DomainHash) ([]byte, error) { func (css *consensusStateStore) serializeTips(tips []*externalapi.DomainHash) ([]byte, error) {
@ -72,3 +56,21 @@ func (css *consensusStateStore) deserializeTips(tipsBytes []byte) ([]*externalap
return serialization.DBTipsToTips(dbTips) return serialization.DBTipsToTips(dbTips)
} }
func (csss *consensusStateStagingShard) commitTips(dbTx model.DBTransaction) error {
if csss.tipsStaging == nil {
return nil
}
tipsBytes, err := csss.store.serializeTips(csss.tipsStaging)
if err != nil {
return err
}
err = dbTx.Put(tipsKey, tipsBytes)
if err != nil {
return err
}
csss.store.tipsCache = csss.tipsStaging
return nil
}

View File

@ -19,12 +19,14 @@ func utxoKey(outpoint *externalapi.DomainOutpoint) (model.DBKey, error) {
return utxoSetBucket.Key(serializedOutpoint), nil return utxoSetBucket.Key(serializedOutpoint), nil
} }
func (css *consensusStateStore) StageVirtualUTXODiff(virtualUTXODiff externalapi.UTXODiff) { func (css *consensusStateStore) StageVirtualUTXODiff(stagingArea *model.StagingArea, virtualUTXODiff externalapi.UTXODiff) {
css.virtualUTXODiffStaging = virtualUTXODiff stagingShard := css.stagingShard(stagingArea)
stagingShard.virtualUTXODiffStaging = virtualUTXODiff
} }
func (css *consensusStateStore) commitVirtualUTXODiff(dbTx model.DBTransaction) error { func (csss *consensusStateStagingShard) commitVirtualUTXODiff(dbTx model.DBTransaction) error {
hadStartedImportingPruningPointUTXOSet, err := css.HadStartedImportingPruningPointUTXOSet(dbTx) hadStartedImportingPruningPointUTXOSet, err := csss.store.HadStartedImportingPruningPointUTXOSet(dbTx)
if err != nil { if err != nil {
return err return err
} }
@ -32,11 +34,11 @@ func (css *consensusStateStore) commitVirtualUTXODiff(dbTx model.DBTransaction)
return errors.New("cannot commit virtual UTXO diff after starting to import the pruning point UTXO set") return errors.New("cannot commit virtual UTXO diff after starting to import the pruning point UTXO set")
} }
if css.virtualUTXODiffStaging == nil { if csss.virtualUTXODiffStaging == nil {
return nil return nil
} }
toRemoveIterator := css.virtualUTXODiffStaging.ToRemove().Iterator() toRemoveIterator := csss.virtualUTXODiffStaging.ToRemove().Iterator()
defer toRemoveIterator.Close() defer toRemoveIterator.Close()
for ok := toRemoveIterator.First(); ok; ok = toRemoveIterator.Next() { for ok := toRemoveIterator.First(); ok; ok = toRemoveIterator.Next() {
toRemoveOutpoint, _, err := toRemoveIterator.Get() toRemoveOutpoint, _, err := toRemoveIterator.Get()
@ -44,7 +46,7 @@ func (css *consensusStateStore) commitVirtualUTXODiff(dbTx model.DBTransaction)
return err return err
} }
css.virtualUTXOSetCache.Remove(toRemoveOutpoint) csss.store.virtualUTXOSetCache.Remove(toRemoveOutpoint)
dbKey, err := utxoKey(toRemoveOutpoint) dbKey, err := utxoKey(toRemoveOutpoint)
if err != nil { if err != nil {
@ -56,7 +58,7 @@ func (css *consensusStateStore) commitVirtualUTXODiff(dbTx model.DBTransaction)
} }
} }
toAddIterator := css.virtualUTXODiffStaging.ToAdd().Iterator() toAddIterator := csss.virtualUTXODiffStaging.ToAdd().Iterator()
defer toAddIterator.Close() defer toAddIterator.Close()
for ok := toAddIterator.First(); ok; ok = toAddIterator.Next() { for ok := toAddIterator.First(); ok; ok = toAddIterator.Next() {
toAddOutpoint, toAddEntry, err := toAddIterator.Get() toAddOutpoint, toAddEntry, err := toAddIterator.Get()
@ -64,7 +66,7 @@ func (css *consensusStateStore) commitVirtualUTXODiff(dbTx model.DBTransaction)
return err return err
} }
css.virtualUTXOSetCache.Add(toAddOutpoint, toAddEntry) csss.store.virtualUTXOSetCache.Add(toAddOutpoint, toAddEntry)
dbKey, err := utxoKey(toAddOutpoint) dbKey, err := utxoKey(toAddOutpoint)
if err != nil { if err != nil {
@ -85,21 +87,22 @@ func (css *consensusStateStore) commitVirtualUTXODiff(dbTx model.DBTransaction)
return nil return nil
} }
func (css *consensusStateStore) UTXOByOutpoint(dbContext model.DBReader, outpoint *externalapi.DomainOutpoint) ( func (css *consensusStateStore) UTXOByOutpoint(dbContext model.DBReader, stagingArea *model.StagingArea,
externalapi.UTXOEntry, error) { outpoint *externalapi.DomainOutpoint) (externalapi.UTXOEntry, error) {
return css.utxoByOutpointFromStagedVirtualUTXODiff(dbContext, outpoint) stagingShard := css.stagingShard(stagingArea)
return css.utxoByOutpointFromStagedVirtualUTXODiff(dbContext, stagingShard, outpoint)
} }
func (css *consensusStateStore) utxoByOutpointFromStagedVirtualUTXODiff(dbContext model.DBReader, func (css *consensusStateStore) utxoByOutpointFromStagedVirtualUTXODiff(dbContext model.DBReader,
outpoint *externalapi.DomainOutpoint) ( stagingShard *consensusStateStagingShard, outpoint *externalapi.DomainOutpoint) (externalapi.UTXOEntry, error) {
externalapi.UTXOEntry, error) {
if css.virtualUTXODiffStaging != nil { if stagingShard.virtualUTXODiffStaging != nil {
if css.virtualUTXODiffStaging.ToRemove().Contains(outpoint) { if stagingShard.virtualUTXODiffStaging.ToRemove().Contains(outpoint) {
return nil, errors.Errorf("outpoint was not found") return nil, errors.Errorf("outpoint was not found")
} }
if utxoEntry, ok := css.virtualUTXODiffStaging.ToAdd().Get(outpoint); ok { if utxoEntry, ok := stagingShard.virtualUTXODiffStaging.ToAdd().Get(outpoint); ok {
return utxoEntry, nil return utxoEntry, nil
} }
} }
@ -127,18 +130,22 @@ func (css *consensusStateStore) utxoByOutpointFromStagedVirtualUTXODiff(dbContex
return entry, nil return entry, nil
} }
func (css *consensusStateStore) HasUTXOByOutpoint(dbContext model.DBReader, outpoint *externalapi.DomainOutpoint) (bool, error) { func (css *consensusStateStore) HasUTXOByOutpoint(dbContext model.DBReader, stagingArea *model.StagingArea,
return css.hasUTXOByOutpointFromStagedVirtualUTXODiff(dbContext, outpoint) outpoint *externalapi.DomainOutpoint) (bool, error) {
stagingShard := css.stagingShard(stagingArea)
return css.hasUTXOByOutpointFromStagedVirtualUTXODiff(dbContext, stagingShard, outpoint)
} }
func (css *consensusStateStore) hasUTXOByOutpointFromStagedVirtualUTXODiff(dbContext model.DBReader, func (css *consensusStateStore) hasUTXOByOutpointFromStagedVirtualUTXODiff(dbContext model.DBReader,
outpoint *externalapi.DomainOutpoint) (bool, error) { stagingShard *consensusStateStagingShard, outpoint *externalapi.DomainOutpoint) (bool, error) {
if css.virtualUTXODiffStaging != nil { if stagingShard.virtualUTXODiffStaging != nil {
if css.virtualUTXODiffStaging.ToRemove().Contains(outpoint) { if stagingShard.virtualUTXODiffStaging.ToRemove().Contains(outpoint) {
return false, nil return false, nil
} }
if _, ok := css.virtualUTXODiffStaging.ToAdd().Get(outpoint); ok { if _, ok := stagingShard.virtualUTXODiffStaging.ToAdd().Get(outpoint); ok {
return true, nil return true, nil
} }
} }
@ -151,8 +158,8 @@ func (css *consensusStateStore) hasUTXOByOutpointFromStagedVirtualUTXODiff(dbCon
return dbContext.Has(key) return dbContext.Has(key)
} }
func (css *consensusStateStore) VirtualUTXOs(dbContext model.DBReader, func (css *consensusStateStore) VirtualUTXOs(dbContext model.DBReader, fromOutpoint *externalapi.DomainOutpoint, limit int) (
fromOutpoint *externalapi.DomainOutpoint, limit int) ([]*externalapi.OutpointAndUTXOEntryPair, error) { []*externalapi.OutpointAndUTXOEntryPair, error) {
cursor, err := dbContext.Cursor(utxoSetBucket) cursor, err := dbContext.Cursor(utxoSetBucket)
if err != nil { if err != nil {
@ -189,15 +196,19 @@ func (css *consensusStateStore) VirtualUTXOs(dbContext model.DBReader,
return outpointAndUTXOEntryPairs, nil return outpointAndUTXOEntryPairs, nil
} }
func (css *consensusStateStore) VirtualUTXOSetIterator(dbContext model.DBReader) (externalapi.ReadOnlyUTXOSetIterator, error) { func (css *consensusStateStore) VirtualUTXOSetIterator(dbContext model.DBReader, stagingArea *model.StagingArea) (
externalapi.ReadOnlyUTXOSetIterator, error) {
stagingShard := css.stagingShard(stagingArea)
cursor, err := dbContext.Cursor(utxoSetBucket) cursor, err := dbContext.Cursor(utxoSetBucket)
if err != nil { if err != nil {
return nil, err return nil, err
} }
mainIterator := newCursorUTXOSetIterator(cursor) mainIterator := newCursorUTXOSetIterator(cursor)
if css.virtualUTXODiffStaging != nil { if stagingShard.virtualUTXODiffStaging != nil {
return utxo.IteratorWithDiff(mainIterator, css.virtualUTXODiffStaging) return utxo.IteratorWithDiff(mainIterator, stagingShard.virtualUTXODiffStaging)
} }
return mainIterator, nil return mainIterator, nil

View File

@ -24,10 +24,6 @@ func (css *consensusStateStore) FinishImportingPruningPointUTXOSet(dbContext mod
func (css *consensusStateStore) ImportPruningPointUTXOSetIntoVirtualUTXOSet(dbContext model.DBWriter, func (css *consensusStateStore) ImportPruningPointUTXOSetIntoVirtualUTXOSet(dbContext model.DBWriter,
pruningPointUTXOSetIterator externalapi.ReadOnlyUTXOSetIterator) error { pruningPointUTXOSetIterator externalapi.ReadOnlyUTXOSetIterator) error {
if css.virtualUTXODiffStaging != nil {
return errors.New("cannot import virtual UTXO set while virtual UTXO diff is staged")
}
hadStartedImportingPruningPointUTXOSet, err := css.HadStartedImportingPruningPointUTXOSet(dbContext) hadStartedImportingPruningPointUTXOSet, err := css.HadStartedImportingPruningPointUTXOSet(dbContext)
if err != nil { if err != nil {
return err return err

View File

@ -0,0 +1,72 @@
package daablocksstore
import (
"github.com/kaspanet/kaspad/domain/consensus/database/binaryserialization"
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
)
type daaBlocksStagingShard struct {
store *daaBlocksStore
daaScoreToAdd map[externalapi.DomainHash]uint64
daaAddedBlocksToAdd map[externalapi.DomainHash][]*externalapi.DomainHash
daaScoreToDelete map[externalapi.DomainHash]struct{}
daaAddedBlocksToDelete map[externalapi.DomainHash]struct{}
}
func (daas *daaBlocksStore) stagingShard(stagingArea *model.StagingArea) *daaBlocksStagingShard {
return stagingArea.GetOrCreateShard(model.StagingShardIDDAABlocks, func() model.StagingShard {
return &daaBlocksStagingShard{
store: daas,
daaScoreToAdd: make(map[externalapi.DomainHash]uint64),
daaAddedBlocksToAdd: make(map[externalapi.DomainHash][]*externalapi.DomainHash),
daaScoreToDelete: make(map[externalapi.DomainHash]struct{}),
daaAddedBlocksToDelete: make(map[externalapi.DomainHash]struct{}),
}
}).(*daaBlocksStagingShard)
}
func (daass *daaBlocksStagingShard) Commit(dbTx model.DBTransaction) error {
for hash, daaScore := range daass.daaScoreToAdd {
daaScoreBytes := binaryserialization.SerializeUint64(daaScore)
err := dbTx.Put(daass.store.daaScoreHashAsKey(&hash), daaScoreBytes)
if err != nil {
return err
}
daass.store.daaScoreLRUCache.Add(&hash, daaScore)
}
for hash, addedBlocks := range daass.daaAddedBlocksToAdd {
addedBlocksBytes := binaryserialization.SerializeHashes(addedBlocks)
err := dbTx.Put(daass.store.daaAddedBlocksHashAsKey(&hash), addedBlocksBytes)
if err != nil {
return err
}
daass.store.daaAddedBlocksLRUCache.Add(&hash, addedBlocks)
}
for hash := range daass.daaScoreToDelete {
err := dbTx.Delete(daass.store.daaScoreHashAsKey(&hash))
if err != nil {
return err
}
daass.store.daaScoreLRUCache.Remove(&hash)
}
for hash := range daass.daaAddedBlocksToDelete {
err := dbTx.Delete(daass.store.daaAddedBlocksHashAsKey(&hash))
if err != nil {
return err
}
daass.store.daaAddedBlocksLRUCache.Remove(&hash)
}
return nil
}
func (daass *daaBlocksStagingShard) isStaged() bool {
return len(daass.daaScoreToAdd) != 0 ||
len(daass.daaAddedBlocksToAdd) != 0 ||
len(daass.daaScoreToDelete) != 0 ||
len(daass.daaAddedBlocksToDelete) != 0
}

View File

@ -0,0 +1,114 @@
package daablocksstore
import (
"github.com/kaspanet/kaspad/domain/consensus/database"
"github.com/kaspanet/kaspad/domain/consensus/database/binaryserialization"
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/kaspanet/kaspad/domain/consensus/utils/lrucache"
)
var daaScoreBucket = database.MakeBucket([]byte("daa-score"))
var daaAddedBlocksBucket = database.MakeBucket([]byte("daa-added-blocks"))
// daaBlocksStore represents a store of DAABlocksStore
type daaBlocksStore struct {
daaScoreLRUCache *lrucache.LRUCache
daaAddedBlocksLRUCache *lrucache.LRUCache
}
// New instantiates a new DAABlocksStore
func New(daaScoreCacheSize int, daaAddedBlocksCacheSize int, preallocate bool) model.DAABlocksStore {
return &daaBlocksStore{
daaScoreLRUCache: lrucache.New(daaScoreCacheSize, preallocate),
daaAddedBlocksLRUCache: lrucache.New(daaAddedBlocksCacheSize, preallocate),
}
}
func (daas *daaBlocksStore) StageDAAScore(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash, daaScore uint64) {
stagingShard := daas.stagingShard(stagingArea)
stagingShard.daaScoreToAdd[*blockHash] = daaScore
}
func (daas *daaBlocksStore) StageBlockDAAAddedBlocks(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash, addedBlocks []*externalapi.DomainHash) {
stagingShard := daas.stagingShard(stagingArea)
stagingShard.daaAddedBlocksToAdd[*blockHash] = externalapi.CloneHashes(addedBlocks)
}
func (daas *daaBlocksStore) IsStaged(stagingArea *model.StagingArea) bool {
return daas.stagingShard(stagingArea).isStaged()
}
func (daas *daaBlocksStore) DAAScore(dbContext model.DBReader, stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) (uint64, error) {
stagingShard := daas.stagingShard(stagingArea)
if daaScore, ok := stagingShard.daaScoreToAdd[*blockHash]; ok {
return daaScore, nil
}
if daaScore, ok := daas.daaScoreLRUCache.Get(blockHash); ok {
return daaScore.(uint64), nil
}
daaScoreBytes, err := dbContext.Get(daas.daaScoreHashAsKey(blockHash))
if err != nil {
return 0, err
}
daaScore, err := binaryserialization.DeserializeUint64(daaScoreBytes)
if err != nil {
return 0, err
}
daas.daaScoreLRUCache.Add(blockHash, daaScore)
return daaScore, nil
}
func (daas *daaBlocksStore) DAAAddedBlocks(dbContext model.DBReader, stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) ([]*externalapi.DomainHash, error) {
stagingShard := daas.stagingShard(stagingArea)
if addedBlocks, ok := stagingShard.daaAddedBlocksToAdd[*blockHash]; ok {
return externalapi.CloneHashes(addedBlocks), nil
}
if addedBlocks, ok := daas.daaAddedBlocksLRUCache.Get(blockHash); ok {
return externalapi.CloneHashes(addedBlocks.([]*externalapi.DomainHash)), nil
}
addedBlocksBytes, err := dbContext.Get(daas.daaAddedBlocksHashAsKey(blockHash))
if err != nil {
return nil, err
}
addedBlocks, err := binaryserialization.DeserializeHashes(addedBlocksBytes)
if err != nil {
return nil, err
}
daas.daaAddedBlocksLRUCache.Add(blockHash, addedBlocks)
return externalapi.CloneHashes(addedBlocks), nil
}
func (daas *daaBlocksStore) daaScoreHashAsKey(hash *externalapi.DomainHash) model.DBKey {
return daaScoreBucket.Key(hash.ByteSlice())
}
func (daas *daaBlocksStore) daaAddedBlocksHashAsKey(hash *externalapi.DomainHash) model.DBKey {
return daaAddedBlocksBucket.Key(hash.ByteSlice())
}
func (daas *daaBlocksStore) Delete(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) {
stagingShard := daas.stagingShard(stagingArea)
if _, ok := stagingShard.daaScoreToAdd[*blockHash]; ok {
delete(stagingShard.daaScoreToAdd, *blockHash)
} else {
stagingShard.daaAddedBlocksToDelete[*blockHash] = struct{}{}
}
if _, ok := stagingShard.daaAddedBlocksToAdd[*blockHash]; ok {
delete(stagingShard.daaAddedBlocksToAdd, *blockHash)
} else {
stagingShard.daaAddedBlocksToDelete[*blockHash] = struct{}{}
}
}

View File

@ -1,160 +0,0 @@
package daablocksstore
import (
"github.com/kaspanet/kaspad/domain/consensus/database"
"github.com/kaspanet/kaspad/domain/consensus/database/binaryserialization"
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/kaspanet/kaspad/domain/consensus/utils/lrucache"
)
var daaScoreBucket = database.MakeBucket([]byte("daa-score"))
var daaAddedBlocksBucket = database.MakeBucket([]byte("daa-added-blocks"))
// daaBlocksStore represents a store of DAABlocksStore
type daaBlocksStore struct {
daaScoreStaging map[externalapi.DomainHash]uint64
daaAddedBlocksStaging map[externalapi.DomainHash][]*externalapi.DomainHash
daaScoreToDelete map[externalapi.DomainHash]struct{}
daaAddedBlocksToDelete map[externalapi.DomainHash]struct{}
daaScoreLRUCache *lrucache.LRUCache
daaAddedBlocksLRUCache *lrucache.LRUCache
}
// New instantiates a new DAABlocksStore
func New(daaScoreCacheSize int, daaAddedBlocksCacheSize int, preallocate bool) model.DAABlocksStore {
return &daaBlocksStore{
daaScoreStaging: make(map[externalapi.DomainHash]uint64),
daaAddedBlocksStaging: make(map[externalapi.DomainHash][]*externalapi.DomainHash),
daaScoreLRUCache: lrucache.New(daaScoreCacheSize, preallocate),
daaAddedBlocksLRUCache: lrucache.New(daaAddedBlocksCacheSize, preallocate),
}
}
func (daas *daaBlocksStore) StageDAAScore(blockHash *externalapi.DomainHash, daaScore uint64) {
daas.daaScoreStaging[*blockHash] = daaScore
}
func (daas *daaBlocksStore) StageBlockDAAAddedBlocks(blockHash *externalapi.DomainHash,
addedBlocks []*externalapi.DomainHash) {
daas.daaAddedBlocksStaging[*blockHash] = externalapi.CloneHashes(addedBlocks)
}
func (daas *daaBlocksStore) IsAnythingStaged() bool {
return len(daas.daaScoreStaging) != 0 ||
len(daas.daaAddedBlocksStaging) != 0 ||
len(daas.daaScoreToDelete) != 0 ||
len(daas.daaAddedBlocksToDelete) != 0
}
func (daas *daaBlocksStore) Discard() {
daas.daaScoreStaging = make(map[externalapi.DomainHash]uint64)
daas.daaAddedBlocksStaging = make(map[externalapi.DomainHash][]*externalapi.DomainHash)
daas.daaScoreToDelete = make(map[externalapi.DomainHash]struct{})
daas.daaAddedBlocksToDelete = make(map[externalapi.DomainHash]struct{})
}
func (daas *daaBlocksStore) Commit(dbTx model.DBTransaction) error {
for hash, daaScore := range daas.daaScoreStaging {
daaScoreBytes := binaryserialization.SerializeUint64(daaScore)
err := dbTx.Put(daas.daaScoreHashAsKey(&hash), daaScoreBytes)
if err != nil {
return err
}
daas.daaScoreLRUCache.Add(&hash, daaScore)
}
for hash, addedBlocks := range daas.daaAddedBlocksStaging {
addedBlocksBytes := binaryserialization.SerializeHashes(addedBlocks)
err := dbTx.Put(daas.daaAddedBlocksHashAsKey(&hash), addedBlocksBytes)
if err != nil {
return err
}
daas.daaAddedBlocksLRUCache.Add(&hash, addedBlocks)
}
for hash := range daas.daaScoreToDelete {
err := dbTx.Delete(daas.daaScoreHashAsKey(&hash))
if err != nil {
return err
}
daas.daaScoreLRUCache.Remove(&hash)
}
for hash := range daas.daaAddedBlocksToDelete {
err := dbTx.Delete(daas.daaAddedBlocksHashAsKey(&hash))
if err != nil {
return err
}
daas.daaAddedBlocksLRUCache.Remove(&hash)
}
daas.Discard()
return nil
}
func (daas *daaBlocksStore) DAAScore(dbContext model.DBReader, blockHash *externalapi.DomainHash) (uint64, error) {
if daaScore, ok := daas.daaScoreStaging[*blockHash]; ok {
return daaScore, nil
}
if daaScore, ok := daas.daaScoreLRUCache.Get(blockHash); ok {
return daaScore.(uint64), nil
}
daaScoreBytes, err := dbContext.Get(daas.daaScoreHashAsKey(blockHash))
if err != nil {
return 0, err
}
daaScore, err := binaryserialization.DeserializeUint64(daaScoreBytes)
if err != nil {
return 0, err
}
daas.daaScoreLRUCache.Add(blockHash, daaScore)
return daaScore, nil
}
func (daas *daaBlocksStore) DAAAddedBlocks(dbContext model.DBReader, blockHash *externalapi.DomainHash) ([]*externalapi.DomainHash, error) {
if addedBlocks, ok := daas.daaAddedBlocksStaging[*blockHash]; ok {
return externalapi.CloneHashes(addedBlocks), nil
}
if addedBlocks, ok := daas.daaAddedBlocksLRUCache.Get(blockHash); ok {
return externalapi.CloneHashes(addedBlocks.([]*externalapi.DomainHash)), nil
}
addedBlocksBytes, err := dbContext.Get(daas.daaAddedBlocksHashAsKey(blockHash))
if err != nil {
return nil, err
}
addedBlocks, err := binaryserialization.DeserializeHashes(addedBlocksBytes)
if err != nil {
return nil, err
}
daas.daaAddedBlocksLRUCache.Add(blockHash, addedBlocks)
return externalapi.CloneHashes(addedBlocks), nil
}
func (daas *daaBlocksStore) daaScoreHashAsKey(hash *externalapi.DomainHash) model.DBKey {
return daaScoreBucket.Key(hash.ByteSlice())
}
func (daas *daaBlocksStore) daaAddedBlocksHashAsKey(hash *externalapi.DomainHash) model.DBKey {
return daaAddedBlocksBucket.Key(hash.ByteSlice())
}
func (daas *daaBlocksStore) Delete(blockHash *externalapi.DomainHash) {
if _, ok := daas.daaScoreStaging[*blockHash]; ok {
delete(daas.daaScoreStaging, *blockHash)
} else {
daas.daaAddedBlocksToDelete[*blockHash] = struct{}{}
}
if _, ok := daas.daaAddedBlocksStaging[*blockHash]; ok {
delete(daas.daaAddedBlocksStaging, *blockHash)
} else {
daas.daaAddedBlocksToDelete[*blockHash] = struct{}{}
}
}

View File

@ -0,0 +1,36 @@
package finalitystore
import (
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
)
type finalityStagingShard struct {
store *finalityStore
toAdd map[externalapi.DomainHash]*externalapi.DomainHash
}
func (fs *finalityStore) stagingShard(stagingArea *model.StagingArea) *finalityStagingShard {
return stagingArea.GetOrCreateShard(model.StagingShardIDFinality, func() model.StagingShard {
return &finalityStagingShard{
store: fs,
toAdd: make(map[externalapi.DomainHash]*externalapi.DomainHash),
}
}).(*finalityStagingShard)
}
func (fss *finalityStagingShard) Commit(dbTx model.DBTransaction) error {
for hash, finalityPointHash := range fss.toAdd {
err := dbTx.Put(fss.store.hashAsKey(&hash), finalityPointHash.ByteSlice())
if err != nil {
return err
}
fss.store.cache.Add(&hash, finalityPointHash)
}
return nil
}
func (fss *finalityStagingShard) isStaged() bool {
return len(fss.toAdd) == 0
}

View File

@ -10,27 +10,26 @@ import (
var bucket = database.MakeBucket([]byte("finality-points")) var bucket = database.MakeBucket([]byte("finality-points"))
type finalityStore struct { type finalityStore struct {
staging map[externalapi.DomainHash]*externalapi.DomainHash
toDelete map[externalapi.DomainHash]struct{}
cache *lrucache.LRUCache cache *lrucache.LRUCache
} }
// New instantiates a new FinalityStore // New instantiates a new FinalityStore
func New(cacheSize int, preallocate bool) model.FinalityStore { func New(cacheSize int, preallocate bool) model.FinalityStore {
return &finalityStore{ return &finalityStore{
staging: make(map[externalapi.DomainHash]*externalapi.DomainHash),
toDelete: make(map[externalapi.DomainHash]struct{}),
cache: lrucache.New(cacheSize, preallocate), cache: lrucache.New(cacheSize, preallocate),
} }
} }
func (fs *finalityStore) StageFinalityPoint(blockHash *externalapi.DomainHash, finalityPointHash *externalapi.DomainHash) { func (fs *finalityStore) StageFinalityPoint(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash, finalityPointHash *externalapi.DomainHash) {
fs.staging[*blockHash] = finalityPointHash stagingShard := fs.stagingShard(stagingArea)
stagingShard.toAdd[*blockHash] = finalityPointHash
} }
func (fs *finalityStore) FinalityPoint( func (fs *finalityStore) FinalityPoint(dbContext model.DBReader, stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) (*externalapi.DomainHash, error) {
dbContext model.DBReader, blockHash *externalapi.DomainHash) (*externalapi.DomainHash, error) { stagingShard := fs.stagingShard(stagingArea)
if finalityPointHash, ok := fs.staging[*blockHash]; ok {
if finalityPointHash, ok := stagingShard.toAdd[*blockHash]; ok {
return finalityPointHash, nil return finalityPointHash, nil
} }
@ -51,25 +50,8 @@ func (fs *finalityStore) FinalityPoint(
return finalityPointHash, nil return finalityPointHash, nil
} }
func (fs *finalityStore) Discard() { func (fs *finalityStore) IsStaged(stagingArea *model.StagingArea) bool {
fs.staging = make(map[externalapi.DomainHash]*externalapi.DomainHash) return fs.stagingShard(stagingArea).isStaged()
}
func (fs *finalityStore) Commit(dbTx model.DBTransaction) error {
for hash, finalityPointHash := range fs.staging {
err := dbTx.Put(fs.hashAsKey(&hash), finalityPointHash.ByteSlice())
if err != nil {
return err
}
fs.cache.Add(&hash, finalityPointHash)
}
fs.Discard()
return nil
}
func (fs *finalityStore) IsStaged() bool {
return len(fs.staging) == 0
} }
func (fs *finalityStore) hashAsKey(hash *externalapi.DomainHash) model.DBKey { func (fs *finalityStore) hashAsKey(hash *externalapi.DomainHash) model.DBKey {

View File

@ -0,0 +1,40 @@
package ghostdagdatastore
import (
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
)
type ghostdagDataStagingShard struct {
store *ghostdagDataStore
toAdd map[externalapi.DomainHash]*model.BlockGHOSTDAGData
}
func (gds *ghostdagDataStore) stagingShard(stagingArea *model.StagingArea) *ghostdagDataStagingShard {
return stagingArea.GetOrCreateShard(model.StagingShardIDGHOSTDAG, func() model.StagingShard {
return &ghostdagDataStagingShard{
store: gds,
toAdd: make(map[externalapi.DomainHash]*model.BlockGHOSTDAGData),
}
}).(*ghostdagDataStagingShard)
}
func (gdss *ghostdagDataStagingShard) Commit(dbTx model.DBTransaction) error {
for hash, blockGHOSTDAGData := range gdss.toAdd {
blockGhostdagDataBytes, err := gdss.store.serializeBlockGHOSTDAGData(blockGHOSTDAGData)
if err != nil {
return err
}
err = dbTx.Put(gdss.store.hashAsKey(&hash), blockGhostdagDataBytes)
if err != nil {
return err
}
gdss.store.cache.Add(&hash, blockGHOSTDAGData)
}
return nil
}
func (gdss *ghostdagDataStagingShard) isStaged() bool {
return len(gdss.toAdd) != 0
}

View File

@ -13,51 +13,32 @@ var bucket = database.MakeBucket([]byte("block-ghostdag-data"))
// ghostdagDataStore represents a store of BlockGHOSTDAGData // ghostdagDataStore represents a store of BlockGHOSTDAGData
type ghostdagDataStore struct { type ghostdagDataStore struct {
staging map[externalapi.DomainHash]*model.BlockGHOSTDAGData
cache *lrucache.LRUCache cache *lrucache.LRUCache
} }
// New instantiates a new GHOSTDAGDataStore // New instantiates a new GHOSTDAGDataStore
func New(cacheSize int, preallocate bool) model.GHOSTDAGDataStore { func New(cacheSize int, preallocate bool) model.GHOSTDAGDataStore {
return &ghostdagDataStore{ return &ghostdagDataStore{
staging: make(map[externalapi.DomainHash]*model.BlockGHOSTDAGData),
cache: lrucache.New(cacheSize, preallocate), cache: lrucache.New(cacheSize, preallocate),
} }
} }
// Stage stages the given blockGHOSTDAGData for the given blockHash // Stage stages the given blockGHOSTDAGData for the given blockHash
func (gds *ghostdagDataStore) Stage(blockHash *externalapi.DomainHash, blockGHOSTDAGData *model.BlockGHOSTDAGData) { func (gds *ghostdagDataStore) Stage(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash, blockGHOSTDAGData *model.BlockGHOSTDAGData) {
gds.staging[*blockHash] = blockGHOSTDAGData stagingShard := gds.stagingShard(stagingArea)
stagingShard.toAdd[*blockHash] = blockGHOSTDAGData
} }
func (gds *ghostdagDataStore) IsStaged() bool { func (gds *ghostdagDataStore) IsStaged(stagingArea *model.StagingArea) bool {
return len(gds.staging) != 0 return gds.stagingShard(stagingArea).isStaged()
}
func (gds *ghostdagDataStore) Discard() {
gds.staging = make(map[externalapi.DomainHash]*model.BlockGHOSTDAGData)
}
func (gds *ghostdagDataStore) Commit(dbTx model.DBTransaction) error {
for hash, blockGHOSTDAGData := range gds.staging {
blockGhostdagDataBytes, err := gds.serializeBlockGHOSTDAGData(blockGHOSTDAGData)
if err != nil {
return err
}
err = dbTx.Put(gds.hashAsKey(&hash), blockGhostdagDataBytes)
if err != nil {
return err
}
gds.cache.Add(&hash, blockGHOSTDAGData)
}
gds.Discard()
return nil
} }
// Get gets the blockGHOSTDAGData associated with the given blockHash // Get gets the blockGHOSTDAGData associated with the given blockHash
func (gds *ghostdagDataStore) Get(dbContext model.DBReader, blockHash *externalapi.DomainHash) (*model.BlockGHOSTDAGData, error) { func (gds *ghostdagDataStore) Get(dbContext model.DBReader, stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) (*model.BlockGHOSTDAGData, error) {
if blockGHOSTDAGData, ok := gds.staging[*blockHash]; ok { stagingShard := gds.stagingShard(stagingArea)
if blockGHOSTDAGData, ok := stagingShard.toAdd[*blockHash]; ok {
return blockGHOSTDAGData, nil return blockGHOSTDAGData, nil
} }

View File

@ -0,0 +1,87 @@
package headersselectedchainstore
import (
"github.com/kaspanet/kaspad/domain/consensus/database/binaryserialization"
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
)
type headersSelectedChainStagingShard struct {
store *headersSelectedChainStore
addedByHash map[externalapi.DomainHash]uint64
removedByHash map[externalapi.DomainHash]struct{}
addedByIndex map[uint64]*externalapi.DomainHash
removedByIndex map[uint64]struct{}
}
func (hscs *headersSelectedChainStore) stagingShard(stagingArea *model.StagingArea) *headersSelectedChainStagingShard {
return stagingArea.GetOrCreateShard(model.StagingShardIDHeadersSelectedChain, func() model.StagingShard {
return &headersSelectedChainStagingShard{
store: hscs,
addedByHash: make(map[externalapi.DomainHash]uint64),
removedByHash: make(map[externalapi.DomainHash]struct{}),
addedByIndex: make(map[uint64]*externalapi.DomainHash),
removedByIndex: make(map[uint64]struct{}),
}
}).(*headersSelectedChainStagingShard)
}
func (hscss *headersSelectedChainStagingShard) Commit(dbTx model.DBTransaction) error {
if !hscss.isStaged() {
return nil
}
for hash := range hscss.removedByHash {
hashCopy := hash
err := dbTx.Delete(hscss.store.hashAsKey(&hashCopy))
if err != nil {
return err
}
hscss.store.cacheByHash.Remove(&hashCopy)
}
for index := range hscss.removedByIndex {
err := dbTx.Delete(hscss.store.indexAsKey(index))
if err != nil {
return err
}
hscss.store.cacheByIndex.Remove(index)
}
highestIndex := uint64(0)
for hash, index := range hscss.addedByHash {
hashCopy := hash
err := dbTx.Put(hscss.store.hashAsKey(&hashCopy), hscss.store.serializeIndex(index))
if err != nil {
return err
}
err = dbTx.Put(hscss.store.indexAsKey(index), binaryserialization.SerializeHash(&hashCopy))
if err != nil {
return err
}
hscss.store.cacheByHash.Add(&hashCopy, index)
hscss.store.cacheByIndex.Add(index, &hashCopy)
if index > highestIndex {
highestIndex = index
}
}
err := dbTx.Put(highestChainBlockIndexKey, hscss.store.serializeIndex(highestIndex))
if err != nil {
return err
}
hscss.store.cacheHighestChainBlockIndex = highestIndex
return nil
}
func (hscss *headersSelectedChainStagingShard) isStaged() bool {
return len(hscss.addedByHash) != 0 ||
len(hscss.removedByHash) != 0 ||
len(hscss.addedByIndex) != 0 ||
len(hscss.addedByIndex) != 0
}

View File

@ -2,6 +2,7 @@ package headersselectedchainstore
import ( import (
"encoding/binary" "encoding/binary"
"github.com/kaspanet/kaspad/domain/consensus/database" "github.com/kaspanet/kaspad/domain/consensus/database"
"github.com/kaspanet/kaspad/domain/consensus/database/binaryserialization" "github.com/kaspanet/kaspad/domain/consensus/database/binaryserialization"
"github.com/kaspanet/kaspad/domain/consensus/model" "github.com/kaspanet/kaspad/domain/consensus/model"
@ -16,10 +17,6 @@ var bucketChainBlockIndexByHash = database.MakeBucket([]byte("chain-block-index-
var highestChainBlockIndexKey = database.MakeBucket(nil).Key([]byte("highest-chain-block-index")) var highestChainBlockIndexKey = database.MakeBucket(nil).Key([]byte("highest-chain-block-index"))
type headersSelectedChainStore struct { type headersSelectedChainStore struct {
stagingAddedByHash map[externalapi.DomainHash]uint64
stagingRemovedByHash map[externalapi.DomainHash]struct{}
stagingAddedByIndex map[uint64]*externalapi.DomainHash
stagingRemovedByIndex map[uint64]struct{}
cacheByIndex *lrucacheuint64tohash.LRUCache cacheByIndex *lrucacheuint64tohash.LRUCache
cacheByHash *lrucache.LRUCache cacheByHash *lrucache.LRUCache
cacheHighestChainBlockIndex uint64 cacheHighestChainBlockIndex uint64
@ -28,31 +25,27 @@ type headersSelectedChainStore struct {
// New instantiates a new HeadersSelectedChainStore // New instantiates a new HeadersSelectedChainStore
func New(cacheSize int, preallocate bool) model.HeadersSelectedChainStore { func New(cacheSize int, preallocate bool) model.HeadersSelectedChainStore {
return &headersSelectedChainStore{ return &headersSelectedChainStore{
stagingAddedByHash: make(map[externalapi.DomainHash]uint64),
stagingRemovedByHash: make(map[externalapi.DomainHash]struct{}),
stagingAddedByIndex: make(map[uint64]*externalapi.DomainHash),
stagingRemovedByIndex: make(map[uint64]struct{}),
cacheByIndex: lrucacheuint64tohash.New(cacheSize, preallocate), cacheByIndex: lrucacheuint64tohash.New(cacheSize, preallocate),
cacheByHash: lrucache.New(cacheSize, preallocate), cacheByHash: lrucache.New(cacheSize, preallocate),
} }
} }
// Stage stages the given chain changes // Stage stages the given chain changes
func (hscs *headersSelectedChainStore) Stage(dbContext model.DBReader, func (hscs *headersSelectedChainStore) Stage(dbContext model.DBReader, stagingArea *model.StagingArea, chainChanges *externalapi.SelectedChainPath) error {
chainChanges *externalapi.SelectedChainPath) error { stagingShard := hscs.stagingShard(stagingArea)
if hscs.IsStaged() { if hscs.IsStaged(stagingArea) {
return errors.Errorf("can't stage when there's already staged data") return errors.Errorf("can't stage when there's already staged data")
} }
for _, blockHash := range chainChanges.Removed { for _, blockHash := range chainChanges.Removed {
index, err := hscs.GetIndexByHash(dbContext, blockHash) index, err := hscs.GetIndexByHash(dbContext, stagingArea, blockHash)
if err != nil { if err != nil {
return err return err
} }
hscs.stagingRemovedByIndex[index] = struct{}{} stagingShard.removedByIndex[index] = struct{}{}
hscs.stagingRemovedByHash[*blockHash] = struct{}{} stagingShard.removedByHash[*blockHash] = struct{}{}
} }
currentIndex := uint64(0) currentIndex := uint64(0)
@ -66,89 +59,27 @@ func (hscs *headersSelectedChainStore) Stage(dbContext model.DBReader,
} }
for _, blockHash := range chainChanges.Added { for _, blockHash := range chainChanges.Added {
hscs.stagingAddedByIndex[currentIndex] = blockHash stagingShard.addedByIndex[currentIndex] = blockHash
hscs.stagingAddedByHash[*blockHash] = currentIndex stagingShard.addedByHash[*blockHash] = currentIndex
currentIndex++ currentIndex++
} }
return nil return nil
} }
func (hscs *headersSelectedChainStore) IsStaged() bool { func (hscs *headersSelectedChainStore) IsStaged(stagingArea *model.StagingArea) bool {
return len(hscs.stagingAddedByHash) != 0 || return hscs.stagingShard(stagingArea).isStaged()
len(hscs.stagingRemovedByHash) != 0 ||
len(hscs.stagingAddedByIndex) != 0 ||
len(hscs.stagingAddedByIndex) != 0
}
func (hscs *headersSelectedChainStore) Discard() {
hscs.stagingAddedByHash = make(map[externalapi.DomainHash]uint64)
hscs.stagingRemovedByHash = make(map[externalapi.DomainHash]struct{})
hscs.stagingAddedByIndex = make(map[uint64]*externalapi.DomainHash)
hscs.stagingRemovedByIndex = make(map[uint64]struct{})
}
func (hscs *headersSelectedChainStore) Commit(dbTx model.DBTransaction) error {
if !hscs.IsStaged() {
return nil
}
for hash := range hscs.stagingRemovedByHash {
hashCopy := hash
err := dbTx.Delete(hscs.hashAsKey(&hashCopy))
if err != nil {
return err
}
hscs.cacheByHash.Remove(&hashCopy)
}
for index := range hscs.stagingRemovedByIndex {
err := dbTx.Delete(hscs.indexAsKey(index))
if err != nil {
return err
}
hscs.cacheByIndex.Remove(index)
}
highestIndex := uint64(0)
for hash, index := range hscs.stagingAddedByHash {
hashCopy := hash
err := dbTx.Put(hscs.hashAsKey(&hashCopy), hscs.serializeIndex(index))
if err != nil {
return err
}
err = dbTx.Put(hscs.indexAsKey(index), binaryserialization.SerializeHash(&hashCopy))
if err != nil {
return err
}
hscs.cacheByHash.Add(&hashCopy, index)
hscs.cacheByIndex.Add(index, &hashCopy)
if index > highestIndex {
highestIndex = index
}
}
err := dbTx.Put(highestChainBlockIndexKey, hscs.serializeIndex(highestIndex))
if err != nil {
return err
}
hscs.cacheHighestChainBlockIndex = highestIndex
hscs.Discard()
return nil
} }
// Get gets the chain block index for the given blockHash // Get gets the chain block index for the given blockHash
func (hscs *headersSelectedChainStore) GetIndexByHash(dbContext model.DBReader, blockHash *externalapi.DomainHash) (uint64, error) { func (hscs *headersSelectedChainStore) GetIndexByHash(dbContext model.DBReader, stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) (uint64, error) {
if index, ok := hscs.stagingAddedByHash[*blockHash]; ok { stagingShard := hscs.stagingShard(stagingArea)
if index, ok := stagingShard.addedByHash[*blockHash]; ok {
return index, nil return index, nil
} }
if _, ok := hscs.stagingRemovedByHash[*blockHash]; ok { if _, ok := stagingShard.removedByHash[*blockHash]; ok {
return 0, errors.Wrapf(database.ErrNotFound, "couldn't find block %s", blockHash) return 0, errors.Wrapf(database.ErrNotFound, "couldn't find block %s", blockHash)
} }
@ -170,12 +101,14 @@ func (hscs *headersSelectedChainStore) GetIndexByHash(dbContext model.DBReader,
return index, nil return index, nil
} }
func (hscs *headersSelectedChainStore) GetHashByIndex(dbContext model.DBReader, index uint64) (*externalapi.DomainHash, error) { func (hscs *headersSelectedChainStore) GetHashByIndex(dbContext model.DBReader, stagingArea *model.StagingArea, index uint64) (*externalapi.DomainHash, error) {
if blockHash, ok := hscs.stagingAddedByIndex[index]; ok { stagingShard := hscs.stagingShard(stagingArea)
if blockHash, ok := stagingShard.addedByIndex[index]; ok {
return blockHash, nil return blockHash, nil
} }
if _, ok := hscs.stagingRemovedByIndex[index]; ok { if _, ok := stagingShard.removedByIndex[index]; ok {
return nil, errors.Wrapf(database.ErrNotFound, "couldn't find chain block with index %d", index) return nil, errors.Wrapf(database.ErrNotFound, "couldn't find chain block with index %d", index)
} }

View File

@ -0,0 +1,42 @@
package headersselectedtipstore
import (
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
)
type headersSelectedTipStagingShard struct {
store *headerSelectedTipStore
newSelectedTip *externalapi.DomainHash
}
func (hsts *headerSelectedTipStore) stagingShard(stagingArea *model.StagingArea) *headersSelectedTipStagingShard {
return stagingArea.GetOrCreateShard(model.StagingShardIDHeadersSelectedTip, func() model.StagingShard {
return &headersSelectedTipStagingShard{
store: hsts,
newSelectedTip: nil,
}
}).(*headersSelectedTipStagingShard)
}
func (hstss *headersSelectedTipStagingShard) Commit(dbTx model.DBTransaction) error {
if hstss.newSelectedTip == nil {
return nil
}
selectedTipBytes, err := hstss.store.serializeHeadersSelectedTip(hstss.newSelectedTip)
if err != nil {
return err
}
err = dbTx.Put(headerSelectedTipKey, selectedTipBytes)
if err != nil {
return err
}
hstss.store.cache = hstss.newSelectedTip
return nil
}
func (hstss *headersSelectedTipStagingShard) isStaged() bool {
return hstss.newSelectedTip != nil
}

View File

@ -0,0 +1,83 @@
package headersselectedtipstore
import (
"github.com/golang/protobuf/proto"
"github.com/kaspanet/kaspad/domain/consensus/database"
"github.com/kaspanet/kaspad/domain/consensus/database/serialization"
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
)
var headerSelectedTipKey = database.MakeBucket(nil).Key([]byte("headers-selected-tip"))
type headerSelectedTipStore struct {
cache *externalapi.DomainHash
}
// New instantiates a new HeaderSelectedTipStore
func New() model.HeaderSelectedTipStore {
return &headerSelectedTipStore{}
}
func (hsts *headerSelectedTipStore) Has(dbContext model.DBReader, stagingArea *model.StagingArea) (bool, error) {
stagingShard := hsts.stagingShard(stagingArea)
if stagingShard.newSelectedTip != nil {
return true, nil
}
if hsts.cache != nil {
return true, nil
}
return dbContext.Has(headerSelectedTipKey)
}
func (hsts *headerSelectedTipStore) Stage(stagingArea *model.StagingArea, selectedTip *externalapi.DomainHash) {
stagingShard := hsts.stagingShard(stagingArea)
stagingShard.newSelectedTip = selectedTip
}
func (hsts *headerSelectedTipStore) IsStaged(stagingArea *model.StagingArea) bool {
return hsts.stagingShard(stagingArea).isStaged()
}
func (hsts *headerSelectedTipStore) HeadersSelectedTip(dbContext model.DBReader, stagingArea *model.StagingArea) (
*externalapi.DomainHash, error) {
stagingShard := hsts.stagingShard(stagingArea)
if stagingShard.newSelectedTip != nil {
return stagingShard.newSelectedTip, nil
}
if hsts.cache != nil {
return hsts.cache, nil
}
selectedTipBytes, err := dbContext.Get(headerSelectedTipKey)
if err != nil {
return nil, err
}
selectedTip, err := hsts.deserializeHeadersSelectedTip(selectedTipBytes)
if err != nil {
return nil, err
}
hsts.cache = selectedTip
return hsts.cache, nil
}
func (hsts *headerSelectedTipStore) serializeHeadersSelectedTip(selectedTip *externalapi.DomainHash) ([]byte, error) {
return proto.Marshal(serialization.DomainHashToDbHash(selectedTip))
}
func (hsts *headerSelectedTipStore) deserializeHeadersSelectedTip(selectedTipBytes []byte) (*externalapi.DomainHash, error) {
dbHash := &serialization.DbHash{}
err := proto.Unmarshal(selectedTipBytes, dbHash)
if err != nil {
return nil, err
}
return serialization.DbHashToDomainHash(dbHash)
}

View File

@ -1,100 +0,0 @@
package headersselectedtipstore
import (
"github.com/golang/protobuf/proto"
"github.com/kaspanet/kaspad/domain/consensus/database"
"github.com/kaspanet/kaspad/domain/consensus/database/serialization"
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
)
var headerSelectedTipKey = database.MakeBucket(nil).Key([]byte("headers-selected-tip"))
type headerSelectedTipStore struct {
staging *externalapi.DomainHash
cache *externalapi.DomainHash
}
// New instantiates a new HeaderSelectedTipStore
func New() model.HeaderSelectedTipStore {
return &headerSelectedTipStore{}
}
func (hts *headerSelectedTipStore) Has(dbContext model.DBReader) (bool, error) {
if hts.staging != nil {
return true, nil
}
if hts.cache != nil {
return true, nil
}
return dbContext.Has(headerSelectedTipKey)
}
func (hts *headerSelectedTipStore) Discard() {
hts.staging = nil
}
func (hts *headerSelectedTipStore) Commit(dbTx model.DBTransaction) error {
if hts.staging == nil {
return nil
}
selectedTipBytes, err := hts.serializeHeadersSelectedTip(hts.staging)
if err != nil {
return err
}
err = dbTx.Put(headerSelectedTipKey, selectedTipBytes)
if err != nil {
return err
}
hts.cache = hts.staging
hts.Discard()
return nil
}
func (hts *headerSelectedTipStore) Stage(selectedTip *externalapi.DomainHash) {
hts.staging = selectedTip
}
func (hts *headerSelectedTipStore) IsStaged() bool {
return hts.staging != nil
}
func (hts *headerSelectedTipStore) HeadersSelectedTip(dbContext model.DBReader) (*externalapi.DomainHash, error) {
if hts.staging != nil {
return hts.staging, nil
}
if hts.cache != nil {
return hts.cache, nil
}
selectedTipBytes, err := dbContext.Get(headerSelectedTipKey)
if err != nil {
return nil, err
}
selectedTip, err := hts.deserializeHeadersSelectedTip(selectedTipBytes)
if err != nil {
return nil, err
}
hts.cache = selectedTip
return hts.cache, nil
}
func (hts *headerSelectedTipStore) serializeHeadersSelectedTip(selectedTip *externalapi.DomainHash) ([]byte, error) {
return proto.Marshal(serialization.DomainHashToDbHash(selectedTip))
}
func (hts *headerSelectedTipStore) deserializeHeadersSelectedTip(selectedTipBytes []byte) (*externalapi.DomainHash, error) {
dbHash := &serialization.DbHash{}
err := proto.Unmarshal(selectedTipBytes, dbHash)
if err != nil {
return nil, err
}
return serialization.DbHashToDomainHash(dbHash)
}

View File

@ -0,0 +1,50 @@
package multisetstore
import (
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
)
type multisetStagingShard struct {
store *multisetStore
toAdd map[externalapi.DomainHash]model.Multiset
toDelete map[externalapi.DomainHash]struct{}
}
func (ms *multisetStore) stagingShard(stagingArea *model.StagingArea) *multisetStagingShard {
return stagingArea.GetOrCreateShard(model.StagingShardIDMultiset, func() model.StagingShard {
return &multisetStagingShard{
store: ms,
toAdd: make(map[externalapi.DomainHash]model.Multiset),
toDelete: make(map[externalapi.DomainHash]struct{}),
}
}).(*multisetStagingShard)
}
func (mss *multisetStagingShard) Commit(dbTx model.DBTransaction) error {
for hash, multiset := range mss.toAdd {
multisetBytes, err := mss.store.serializeMultiset(multiset)
if err != nil {
return err
}
err = dbTx.Put(mss.store.hashAsKey(&hash), multisetBytes)
if err != nil {
return err
}
mss.store.cache.Add(&hash, multiset)
}
for hash := range mss.toDelete {
err := dbTx.Delete(mss.store.hashAsKey(&hash))
if err != nil {
return err
}
mss.store.cache.Remove(&hash)
}
return nil
}
func (mss *multisetStagingShard) isStaged() bool {
return len(mss.toAdd) != 0 || len(mss.toDelete) != 0
}

View File

@ -13,62 +13,32 @@ var bucket = database.MakeBucket([]byte("multisets"))
// multisetStore represents a store of Multisets // multisetStore represents a store of Multisets
type multisetStore struct { type multisetStore struct {
staging map[externalapi.DomainHash]model.Multiset
toDelete map[externalapi.DomainHash]struct{}
cache *lrucache.LRUCache cache *lrucache.LRUCache
} }
// New instantiates a new MultisetStore // New instantiates a new MultisetStore
func New(cacheSize int, preallocate bool) model.MultisetStore { func New(cacheSize int, preallocate bool) model.MultisetStore {
return &multisetStore{ return &multisetStore{
staging: make(map[externalapi.DomainHash]model.Multiset),
toDelete: make(map[externalapi.DomainHash]struct{}),
cache: lrucache.New(cacheSize, preallocate), cache: lrucache.New(cacheSize, preallocate),
} }
} }
// Stage stages the given multiset for the given blockHash // Stage stages the given multiset for the given blockHash
func (ms *multisetStore) Stage(blockHash *externalapi.DomainHash, multiset model.Multiset) { func (ms *multisetStore) Stage(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash, multiset model.Multiset) {
ms.staging[*blockHash] = multiset.Clone() stagingShard := ms.stagingShard(stagingArea)
stagingShard.toAdd[*blockHash] = multiset.Clone()
} }
func (ms *multisetStore) IsStaged() bool { func (ms *multisetStore) IsStaged(stagingArea *model.StagingArea) bool {
return len(ms.staging) != 0 || len(ms.toDelete) != 0 return ms.stagingShard(stagingArea).isStaged()
}
func (ms *multisetStore) Discard() {
ms.staging = make(map[externalapi.DomainHash]model.Multiset)
ms.toDelete = make(map[externalapi.DomainHash]struct{})
}
func (ms *multisetStore) Commit(dbTx model.DBTransaction) error {
for hash, multiset := range ms.staging {
multisetBytes, err := ms.serializeMultiset(multiset)
if err != nil {
return err
}
err = dbTx.Put(ms.hashAsKey(&hash), multisetBytes)
if err != nil {
return err
}
ms.cache.Add(&hash, multiset)
}
for hash := range ms.toDelete {
err := dbTx.Delete(ms.hashAsKey(&hash))
if err != nil {
return err
}
ms.cache.Remove(&hash)
}
ms.Discard()
return nil
} }
// Get gets the multiset associated with the given blockHash // Get gets the multiset associated with the given blockHash
func (ms *multisetStore) Get(dbContext model.DBReader, blockHash *externalapi.DomainHash) (model.Multiset, error) { func (ms *multisetStore) Get(dbContext model.DBReader, stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) (model.Multiset, error) {
if multiset, ok := ms.staging[*blockHash]; ok { stagingShard := ms.stagingShard(stagingArea)
if multiset, ok := stagingShard.toAdd[*blockHash]; ok {
return multiset.Clone(), nil return multiset.Clone(), nil
} }
@ -90,12 +60,14 @@ func (ms *multisetStore) Get(dbContext model.DBReader, blockHash *externalapi.Do
} }
// Delete deletes the multiset associated with the given blockHash // Delete deletes the multiset associated with the given blockHash
func (ms *multisetStore) Delete(blockHash *externalapi.DomainHash) { func (ms *multisetStore) Delete(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) {
if _, ok := ms.staging[*blockHash]; ok { stagingShard := ms.stagingShard(stagingArea)
delete(ms.staging, *blockHash)
if _, ok := stagingShard.toAdd[*blockHash]; ok {
delete(stagingShard.toAdd, *blockHash)
return return
} }
ms.toDelete[*blockHash] = struct{}{} stagingShard.toDelete[*blockHash] = struct{}{}
} }
func (ms *multisetStore) hashAsKey(hash *externalapi.DomainHash) model.DBKey { func (ms *multisetStore) hashAsKey(hash *externalapi.DomainHash) model.DBKey {

View File

@ -0,0 +1,64 @@
package pruningstore
import (
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
)
type pruningStagingShard struct {
store *pruningStore
newPruningPoint *externalapi.DomainHash
newPruningPointCandidate *externalapi.DomainHash
startUpdatingPruningPointUTXOSet bool
}
func (ps *pruningStore) stagingShard(stagingArea *model.StagingArea) *pruningStagingShard {
return stagingArea.GetOrCreateShard(model.StagingShardIDPruning, func() model.StagingShard {
return &pruningStagingShard{
store: ps,
newPruningPoint: nil,
newPruningPointCandidate: nil,
startUpdatingPruningPointUTXOSet: false,
}
}).(*pruningStagingShard)
}
func (mss *pruningStagingShard) Commit(dbTx model.DBTransaction) error {
if mss.newPruningPoint != nil {
pruningPointBytes, err := mss.store.serializeHash(mss.newPruningPoint)
if err != nil {
return err
}
err = dbTx.Put(pruningBlockHashKey, pruningPointBytes)
if err != nil {
return err
}
mss.store.pruningPointCache = mss.newPruningPoint
}
if mss.newPruningPointCandidate != nil {
candidateBytes, err := mss.store.serializeHash(mss.newPruningPointCandidate)
if err != nil {
return err
}
err = dbTx.Put(candidatePruningPointHashKey, candidateBytes)
if err != nil {
return err
}
mss.store.pruningPointCandidateCache = mss.newPruningPointCandidate
}
if mss.startUpdatingPruningPointUTXOSet {
err := dbTx.Put(updatingPruningPointUTXOSetKey, []byte{0})
if err != nil {
return err
}
}
return nil
}
func (mss *pruningStagingShard) isStaged() bool {
return mss.newPruningPoint != nil || mss.startUpdatingPruningPointUTXOSet
}

View File

@ -15,12 +15,8 @@ var updatingPruningPointUTXOSetKey = database.MakeBucket(nil).Key([]byte("updati
// pruningStore represents a store for the current pruning state // pruningStore represents a store for the current pruning state
type pruningStore struct { type pruningStore struct {
pruningPointStaging *externalapi.DomainHash
pruningPointCache *externalapi.DomainHash pruningPointCache *externalapi.DomainHash
pruningPointCandidateStaging *externalapi.DomainHash
pruningPointCandidateCache *externalapi.DomainHash pruningPointCandidateCache *externalapi.DomainHash
startUpdatingPruningPointUTXOSetStaging bool
} }
// New instantiates a new PruningStore // New instantiates a new PruningStore
@ -28,13 +24,17 @@ func New() model.PruningStore {
return &pruningStore{} return &pruningStore{}
} }
func (ps *pruningStore) StagePruningPointCandidate(candidate *externalapi.DomainHash) { func (ps *pruningStore) StagePruningPointCandidate(stagingArea *model.StagingArea, candidate *externalapi.DomainHash) {
ps.pruningPointCandidateStaging = candidate stagingShard := ps.stagingShard(stagingArea)
stagingShard.newPruningPointCandidate = candidate
} }
func (ps *pruningStore) PruningPointCandidate(dbContext model.DBReader) (*externalapi.DomainHash, error) { func (ps *pruningStore) PruningPointCandidate(dbContext model.DBReader, stagingArea *model.StagingArea) (*externalapi.DomainHash, error) {
if ps.pruningPointCandidateStaging != nil { stagingShard := ps.stagingShard(stagingArea)
return ps.pruningPointCandidateStaging, nil
if stagingShard.newPruningPointCandidate != nil {
return stagingShard.newPruningPointCandidate, nil
} }
if ps.pruningPointCandidateCache != nil { if ps.pruningPointCandidateCache != nil {
@ -54,8 +54,10 @@ func (ps *pruningStore) PruningPointCandidate(dbContext model.DBReader) (*extern
return candidate, nil return candidate, nil
} }
func (ps *pruningStore) HasPruningPointCandidate(dbContext model.DBReader) (bool, error) { func (ps *pruningStore) HasPruningPointCandidate(dbContext model.DBReader, stagingArea *model.StagingArea) (bool, error) {
if ps.pruningPointCandidateStaging != nil { stagingShard := ps.stagingShard(stagingArea)
if stagingShard.newPruningPointCandidate != nil {
return true, nil return true, nil
} }
@ -67,53 +69,14 @@ func (ps *pruningStore) HasPruningPointCandidate(dbContext model.DBReader) (bool
} }
// Stage stages the pruning state // Stage stages the pruning state
func (ps *pruningStore) StagePruningPoint(pruningPointBlockHash *externalapi.DomainHash) { func (ps *pruningStore) StagePruningPoint(stagingArea *model.StagingArea, pruningPointBlockHash *externalapi.DomainHash) {
ps.pruningPointStaging = pruningPointBlockHash stagingShard := ps.stagingShard(stagingArea)
stagingShard.newPruningPoint = pruningPointBlockHash
} }
func (ps *pruningStore) IsStaged() bool { func (ps *pruningStore) IsStaged(stagingArea *model.StagingArea) bool {
return ps.pruningPointStaging != nil || ps.startUpdatingPruningPointUTXOSetStaging return ps.stagingShard(stagingArea).isStaged()
}
func (ps *pruningStore) Discard() {
ps.pruningPointStaging = nil
ps.startUpdatingPruningPointUTXOSetStaging = false
}
func (ps *pruningStore) Commit(dbTx model.DBTransaction) error {
if ps.pruningPointStaging != nil {
pruningPointBytes, err := ps.serializeHash(ps.pruningPointStaging)
if err != nil {
return err
}
err = dbTx.Put(pruningBlockHashKey, pruningPointBytes)
if err != nil {
return err
}
ps.pruningPointCache = ps.pruningPointStaging
}
if ps.pruningPointCandidateStaging != nil {
candidateBytes, err := ps.serializeHash(ps.pruningPointCandidateStaging)
if err != nil {
return err
}
err = dbTx.Put(candidatePruningPointHashKey, candidateBytes)
if err != nil {
return err
}
ps.pruningPointCandidateCache = ps.pruningPointCandidateStaging
}
if ps.startUpdatingPruningPointUTXOSetStaging {
err := dbTx.Put(updatingPruningPointUTXOSetKey, []byte{0})
if err != nil {
return err
}
}
ps.Discard()
return nil
} }
func (ps *pruningStore) UpdatePruningPointUTXOSet(dbContext model.DBWriter, func (ps *pruningStore) UpdatePruningPointUTXOSet(dbContext model.DBWriter,
@ -161,9 +124,11 @@ func (ps *pruningStore) UpdatePruningPointUTXOSet(dbContext model.DBWriter,
} }
// PruningPoint gets the current pruning point // PruningPoint gets the current pruning point
func (ps *pruningStore) PruningPoint(dbContext model.DBReader) (*externalapi.DomainHash, error) { func (ps *pruningStore) PruningPoint(dbContext model.DBReader, stagingArea *model.StagingArea) (*externalapi.DomainHash, error) {
if ps.pruningPointStaging != nil { stagingShard := ps.stagingShard(stagingArea)
return ps.pruningPointStaging, nil
if stagingShard.newPruningPoint != nil {
return stagingShard.newPruningPoint, nil
} }
if ps.pruningPointCache != nil { if ps.pruningPointCache != nil {
@ -197,8 +162,10 @@ func (ps *pruningStore) deserializePruningPoint(pruningPointBytes []byte) (*exte
return serialization.DbHashToDomainHash(dbHash) return serialization.DbHashToDomainHash(dbHash)
} }
func (ps *pruningStore) HasPruningPoint(dbContext model.DBReader) (bool, error) { func (ps *pruningStore) HasPruningPoint(dbContext model.DBReader, stagingArea *model.StagingArea) (bool, error) {
if ps.pruningPointStaging != nil { stagingShard := ps.stagingShard(stagingArea)
if stagingShard.newPruningPoint != nil {
return true, nil return true, nil
} }
@ -247,8 +214,10 @@ func (ps *pruningStore) PruningPointUTXOs(dbContext model.DBReader,
return outpointAndUTXOEntryPairs, nil return outpointAndUTXOEntryPairs, nil
} }
func (ps *pruningStore) StageStartUpdatingPruningPointUTXOSet() { func (ps *pruningStore) StageStartUpdatingPruningPointUTXOSet(stagingArea *model.StagingArea) {
ps.startUpdatingPruningPointUTXOSetStaging = true stagingShard := ps.stagingShard(stagingArea)
stagingShard.startUpdatingPruningPointUTXOSet = true
} }
func (ps *pruningStore) HadStartedUpdatingPruningPointUTXOSet(dbContext model.DBWriter) (bool, error) { func (ps *pruningStore) HadStartedUpdatingPruningPointUTXOSet(dbContext model.DBWriter) (bool, error) {

View File

@ -0,0 +1,53 @@
package reachabilitydatastore
import (
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
)
type reachabilityDataStagingShard struct {
store *reachabilityDataStore
reachabilityData map[externalapi.DomainHash]model.ReachabilityData
reachabilityReindexRoot *externalapi.DomainHash
}
func (rds *reachabilityDataStore) stagingShard(stagingArea *model.StagingArea) *reachabilityDataStagingShard {
return stagingArea.GetOrCreateShard(model.StagingShardIDReachabilityData, func() model.StagingShard {
return &reachabilityDataStagingShard{
store: rds,
reachabilityData: make(map[externalapi.DomainHash]model.ReachabilityData),
reachabilityReindexRoot: nil,
}
}).(*reachabilityDataStagingShard)
}
func (rdss *reachabilityDataStagingShard) Commit(dbTx model.DBTransaction) error {
if rdss.reachabilityReindexRoot != nil {
reachabilityReindexRootBytes, err := rdss.store.serializeReachabilityReindexRoot(rdss.reachabilityReindexRoot)
if err != nil {
return err
}
err = dbTx.Put(reachabilityReindexRootKey, reachabilityReindexRootBytes)
if err != nil {
return err
}
rdss.store.reachabilityReindexRootCache = rdss.reachabilityReindexRoot
}
for hash, reachabilityData := range rdss.reachabilityData {
reachabilityDataBytes, err := rdss.store.serializeReachabilityData(reachabilityData)
if err != nil {
return err
}
err = dbTx.Put(rdss.store.reachabilityDataBlockHashAsKey(&hash), reachabilityDataBytes)
if err != nil {
return err
}
rdss.store.reachabilityDataCache.Add(&hash, reachabilityData)
}
return nil
}
func (rdss *reachabilityDataStagingShard) isStaged() bool {
return len(rdss.reachabilityData) != 0 || rdss.reachabilityReindexRoot != nil
}

View File

@ -14,8 +14,6 @@ var reachabilityReindexRootKey = database.MakeBucket(nil).Key([]byte("reachabili
// reachabilityDataStore represents a store of ReachabilityData // reachabilityDataStore represents a store of ReachabilityData
type reachabilityDataStore struct { type reachabilityDataStore struct {
reachabilityDataStaging map[externalapi.DomainHash]model.ReachabilityData
reachabilityReindexRootStaging *externalapi.DomainHash
reachabilityDataCache *lrucache.LRUCache reachabilityDataCache *lrucache.LRUCache
reachabilityReindexRootCache *externalapi.DomainHash reachabilityReindexRootCache *externalapi.DomainHash
} }
@ -23,65 +21,33 @@ type reachabilityDataStore struct {
// New instantiates a new ReachabilityDataStore // New instantiates a new ReachabilityDataStore
func New(cacheSize int, preallocate bool) model.ReachabilityDataStore { func New(cacheSize int, preallocate bool) model.ReachabilityDataStore {
return &reachabilityDataStore{ return &reachabilityDataStore{
reachabilityDataStaging: make(map[externalapi.DomainHash]model.ReachabilityData),
reachabilityDataCache: lrucache.New(cacheSize, preallocate), reachabilityDataCache: lrucache.New(cacheSize, preallocate),
} }
} }
// StageReachabilityData stages the given reachabilityData for the given blockHash // StageReachabilityData stages the given reachabilityData for the given blockHash
func (rds *reachabilityDataStore) StageReachabilityData(blockHash *externalapi.DomainHash, func (rds *reachabilityDataStore) StageReachabilityData(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash, reachabilityData model.ReachabilityData) {
reachabilityData model.ReachabilityData) { stagingShard := rds.stagingShard(stagingArea)
rds.reachabilityDataStaging[*blockHash] = reachabilityData stagingShard.reachabilityData[*blockHash] = reachabilityData
} }
// StageReachabilityReindexRoot stages the given reachabilityReindexRoot // StageReachabilityReindexRoot stages the given reachabilityReindexRoot
func (rds *reachabilityDataStore) StageReachabilityReindexRoot(reachabilityReindexRoot *externalapi.DomainHash) { func (rds *reachabilityDataStore) StageReachabilityReindexRoot(stagingArea *model.StagingArea, reachabilityReindexRoot *externalapi.DomainHash) {
rds.reachabilityReindexRootStaging = reachabilityReindexRoot stagingShard := rds.stagingShard(stagingArea)
stagingShard.reachabilityReindexRoot = reachabilityReindexRoot
} }
func (rds *reachabilityDataStore) IsAnythingStaged() bool { func (rds *reachabilityDataStore) IsStaged(stagingArea *model.StagingArea) bool {
return len(rds.reachabilityDataStaging) != 0 || rds.reachabilityReindexRootStaging != nil return rds.stagingShard(stagingArea).isStaged()
}
func (rds *reachabilityDataStore) Discard() {
rds.reachabilityDataStaging = make(map[externalapi.DomainHash]model.ReachabilityData)
rds.reachabilityReindexRootStaging = nil
}
func (rds *reachabilityDataStore) Commit(dbTx model.DBTransaction) error {
if rds.reachabilityReindexRootStaging != nil {
reachabilityReindexRootBytes, err := rds.serializeReachabilityReindexRoot(rds.reachabilityReindexRootStaging)
if err != nil {
return err
}
err = dbTx.Put(reachabilityReindexRootKey, reachabilityReindexRootBytes)
if err != nil {
return err
}
rds.reachabilityReindexRootCache = rds.reachabilityReindexRootStaging
}
for hash, reachabilityData := range rds.reachabilityDataStaging {
reachabilityDataBytes, err := rds.serializeReachabilityData(reachabilityData)
if err != nil {
return err
}
err = dbTx.Put(rds.reachabilityDataBlockHashAsKey(&hash), reachabilityDataBytes)
if err != nil {
return err
}
rds.reachabilityDataCache.Add(&hash, reachabilityData)
}
rds.Discard()
return nil
} }
// ReachabilityData returns the reachabilityData associated with the given blockHash // ReachabilityData returns the reachabilityData associated with the given blockHash
func (rds *reachabilityDataStore) ReachabilityData(dbContext model.DBReader, func (rds *reachabilityDataStore) ReachabilityData(dbContext model.DBReader, stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) (model.ReachabilityData, error) {
blockHash *externalapi.DomainHash) (model.ReachabilityData, error) { stagingShard := rds.stagingShard(stagingArea)
if reachabilityData, ok := rds.reachabilityDataStaging[*blockHash]; ok { if reachabilityData, ok := stagingShard.reachabilityData[*blockHash]; ok {
return reachabilityData, nil return reachabilityData, nil
} }
@ -102,8 +68,10 @@ func (rds *reachabilityDataStore) ReachabilityData(dbContext model.DBReader,
return reachabilityData, nil return reachabilityData, nil
} }
func (rds *reachabilityDataStore) HasReachabilityData(dbContext model.DBReader, blockHash *externalapi.DomainHash) (bool, error) { func (rds *reachabilityDataStore) HasReachabilityData(dbContext model.DBReader, stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) (bool, error) {
if _, ok := rds.reachabilityDataStaging[*blockHash]; ok { stagingShard := rds.stagingShard(stagingArea)
if _, ok := stagingShard.reachabilityData[*blockHash]; ok {
return true, nil return true, nil
} }
@ -115,9 +83,11 @@ func (rds *reachabilityDataStore) HasReachabilityData(dbContext model.DBReader,
} }
// ReachabilityReindexRoot returns the current reachability reindex root // ReachabilityReindexRoot returns the current reachability reindex root
func (rds *reachabilityDataStore) ReachabilityReindexRoot(dbContext model.DBReader) (*externalapi.DomainHash, error) { func (rds *reachabilityDataStore) ReachabilityReindexRoot(dbContext model.DBReader, stagingArea *model.StagingArea) (*externalapi.DomainHash, error) {
if rds.reachabilityReindexRootStaging != nil { stagingShard := rds.stagingShard(stagingArea)
return rds.reachabilityReindexRootStaging, nil
if stagingShard.reachabilityReindexRoot != nil {
return stagingShard.reachabilityReindexRoot, nil
} }
if rds.reachabilityReindexRootCache != nil { if rds.reachabilityReindexRootCache != nil {

View File

@ -0,0 +1,74 @@
package utxodiffstore
import (
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
)
type utxoDiffStagingShard struct {
store *utxoDiffStore
utxoDiffToAdd map[externalapi.DomainHash]externalapi.UTXODiff
utxoDiffChildToAdd map[externalapi.DomainHash]*externalapi.DomainHash
toDelete map[externalapi.DomainHash]struct{}
}
func (uds *utxoDiffStore) stagingShard(stagingArea *model.StagingArea) *utxoDiffStagingShard {
return stagingArea.GetOrCreateShard(model.StagingShardIDUTXODiff, func() model.StagingShard {
return &utxoDiffStagingShard{
store: uds,
utxoDiffToAdd: make(map[externalapi.DomainHash]externalapi.UTXODiff),
utxoDiffChildToAdd: make(map[externalapi.DomainHash]*externalapi.DomainHash),
toDelete: make(map[externalapi.DomainHash]struct{}),
}
}).(*utxoDiffStagingShard)
}
func (udss *utxoDiffStagingShard) Commit(dbTx model.DBTransaction) error {
for hash, utxoDiff := range udss.utxoDiffToAdd {
utxoDiffBytes, err := udss.store.serializeUTXODiff(utxoDiff)
if err != nil {
return err
}
err = dbTx.Put(udss.store.utxoDiffHashAsKey(&hash), utxoDiffBytes)
if err != nil {
return err
}
udss.store.utxoDiffCache.Add(&hash, utxoDiff)
}
for hash, utxoDiffChild := range udss.utxoDiffChildToAdd {
if utxoDiffChild == nil {
continue
}
utxoDiffChildBytes, err := udss.store.serializeUTXODiffChild(utxoDiffChild)
if err != nil {
return err
}
err = dbTx.Put(udss.store.utxoDiffChildHashAsKey(&hash), utxoDiffChildBytes)
if err != nil {
return err
}
udss.store.utxoDiffChildCache.Add(&hash, utxoDiffChild)
}
for hash := range udss.toDelete {
err := dbTx.Delete(udss.store.utxoDiffHashAsKey(&hash))
if err != nil {
return err
}
udss.store.utxoDiffCache.Remove(&hash)
err = dbTx.Delete(udss.store.utxoDiffChildHashAsKey(&hash))
if err != nil {
return err
}
udss.store.utxoDiffChildCache.Remove(&hash)
}
return nil
}
func (udss *utxoDiffStagingShard) isStaged() bool {
return len(udss.utxoDiffToAdd) != 0 || len(udss.utxoDiffChildToAdd) != 0 || len(udss.toDelete) != 0
}

View File

@ -15,9 +15,6 @@ var utxoDiffChildBucket = database.MakeBucket([]byte("utxo-diff-children"))
// utxoDiffStore represents a store of UTXODiffs // utxoDiffStore represents a store of UTXODiffs
type utxoDiffStore struct { type utxoDiffStore struct {
utxoDiffStaging map[externalapi.DomainHash]externalapi.UTXODiff
utxoDiffChildStaging map[externalapi.DomainHash]*externalapi.DomainHash
toDelete map[externalapi.DomainHash]struct{}
utxoDiffCache *lrucache.LRUCache utxoDiffCache *lrucache.LRUCache
utxoDiffChildCache *lrucache.LRUCache utxoDiffChildCache *lrucache.LRUCache
} }
@ -25,90 +22,39 @@ type utxoDiffStore struct {
// New instantiates a new UTXODiffStore // New instantiates a new UTXODiffStore
func New(cacheSize int, preallocate bool) model.UTXODiffStore { func New(cacheSize int, preallocate bool) model.UTXODiffStore {
return &utxoDiffStore{ return &utxoDiffStore{
utxoDiffStaging: make(map[externalapi.DomainHash]externalapi.UTXODiff),
utxoDiffChildStaging: make(map[externalapi.DomainHash]*externalapi.DomainHash),
toDelete: make(map[externalapi.DomainHash]struct{}),
utxoDiffCache: lrucache.New(cacheSize, preallocate), utxoDiffCache: lrucache.New(cacheSize, preallocate),
utxoDiffChildCache: lrucache.New(cacheSize, preallocate), utxoDiffChildCache: lrucache.New(cacheSize, preallocate),
} }
} }
// Stage stages the given utxoDiff for the given blockHash // Stage stages the given utxoDiff for the given blockHash
func (uds *utxoDiffStore) Stage(blockHash *externalapi.DomainHash, utxoDiff externalapi.UTXODiff, utxoDiffChild *externalapi.DomainHash) { func (uds *utxoDiffStore) Stage(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash, utxoDiff externalapi.UTXODiff, utxoDiffChild *externalapi.DomainHash) {
uds.utxoDiffStaging[*blockHash] = utxoDiff stagingShard := uds.stagingShard(stagingArea)
stagingShard.utxoDiffToAdd[*blockHash] = utxoDiff
if utxoDiffChild != nil { if utxoDiffChild != nil {
uds.utxoDiffChildStaging[*blockHash] = utxoDiffChild stagingShard.utxoDiffChildToAdd[*blockHash] = utxoDiffChild
} }
} }
func (uds *utxoDiffStore) IsStaged() bool { func (uds *utxoDiffStore) IsStaged(stagingArea *model.StagingArea) bool {
return len(uds.utxoDiffStaging) != 0 || len(uds.utxoDiffChildStaging) != 0 || len(uds.toDelete) != 0 return uds.stagingShard(stagingArea).isStaged()
} }
func (uds *utxoDiffStore) IsBlockHashStaged(blockHash *externalapi.DomainHash) bool { func (uds *utxoDiffStore) isBlockHashStaged(stagingShard *utxoDiffStagingShard, blockHash *externalapi.DomainHash) bool {
if _, ok := uds.utxoDiffStaging[*blockHash]; ok { if _, ok := stagingShard.utxoDiffToAdd[*blockHash]; ok {
return true return true
} }
_, ok := uds.utxoDiffChildStaging[*blockHash] _, ok := stagingShard.utxoDiffChildToAdd[*blockHash]
return ok return ok
} }
func (uds *utxoDiffStore) Discard() {
uds.utxoDiffStaging = make(map[externalapi.DomainHash]externalapi.UTXODiff)
uds.utxoDiffChildStaging = make(map[externalapi.DomainHash]*externalapi.DomainHash)
uds.toDelete = make(map[externalapi.DomainHash]struct{})
}
func (uds *utxoDiffStore) Commit(dbTx model.DBTransaction) error {
for hash, utxoDiff := range uds.utxoDiffStaging {
utxoDiffBytes, err := uds.serializeUTXODiff(utxoDiff)
if err != nil {
return err
}
err = dbTx.Put(uds.utxoDiffHashAsKey(&hash), utxoDiffBytes)
if err != nil {
return err
}
uds.utxoDiffCache.Add(&hash, utxoDiff)
}
for hash, utxoDiffChild := range uds.utxoDiffChildStaging {
if utxoDiffChild == nil {
continue
}
utxoDiffChildBytes, err := uds.serializeUTXODiffChild(utxoDiffChild)
if err != nil {
return err
}
err = dbTx.Put(uds.utxoDiffChildHashAsKey(&hash), utxoDiffChildBytes)
if err != nil {
return err
}
uds.utxoDiffChildCache.Add(&hash, utxoDiffChild)
}
for hash := range uds.toDelete {
err := dbTx.Delete(uds.utxoDiffHashAsKey(&hash))
if err != nil {
return err
}
uds.utxoDiffCache.Remove(&hash)
err = dbTx.Delete(uds.utxoDiffChildHashAsKey(&hash))
if err != nil {
return err
}
uds.utxoDiffChildCache.Remove(&hash)
}
uds.Discard()
return nil
}
// UTXODiff gets the utxoDiff associated with the given blockHash // UTXODiff gets the utxoDiff associated with the given blockHash
func (uds *utxoDiffStore) UTXODiff(dbContext model.DBReader, blockHash *externalapi.DomainHash) (externalapi.UTXODiff, error) { func (uds *utxoDiffStore) UTXODiff(dbContext model.DBReader, stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) (externalapi.UTXODiff, error) {
if utxoDiff, ok := uds.utxoDiffStaging[*blockHash]; ok { stagingShard := uds.stagingShard(stagingArea)
if utxoDiff, ok := stagingShard.utxoDiffToAdd[*blockHash]; ok {
return utxoDiff, nil return utxoDiff, nil
} }
@ -130,8 +76,10 @@ func (uds *utxoDiffStore) UTXODiff(dbContext model.DBReader, blockHash *external
} }
// UTXODiffChild gets the utxoDiff child associated with the given blockHash // UTXODiffChild gets the utxoDiff child associated with the given blockHash
func (uds *utxoDiffStore) UTXODiffChild(dbContext model.DBReader, blockHash *externalapi.DomainHash) (*externalapi.DomainHash, error) { func (uds *utxoDiffStore) UTXODiffChild(dbContext model.DBReader, stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) (*externalapi.DomainHash, error) {
if utxoDiffChild, ok := uds.utxoDiffChildStaging[*blockHash]; ok { stagingShard := uds.stagingShard(stagingArea)
if utxoDiffChild, ok := stagingShard.utxoDiffChildToAdd[*blockHash]; ok {
return utxoDiffChild, nil return utxoDiffChild, nil
} }
@ -153,8 +101,10 @@ func (uds *utxoDiffStore) UTXODiffChild(dbContext model.DBReader, blockHash *ext
} }
// HasUTXODiffChild returns true if the given blockHash has a UTXODiffChild // HasUTXODiffChild returns true if the given blockHash has a UTXODiffChild
func (uds *utxoDiffStore) HasUTXODiffChild(dbContext model.DBReader, blockHash *externalapi.DomainHash) (bool, error) { func (uds *utxoDiffStore) HasUTXODiffChild(dbContext model.DBReader, stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) (bool, error) {
if _, ok := uds.utxoDiffChildStaging[*blockHash]; ok { stagingShard := uds.stagingShard(stagingArea)
if _, ok := stagingShard.utxoDiffChildToAdd[*blockHash]; ok {
return true, nil return true, nil
} }
@ -166,17 +116,19 @@ func (uds *utxoDiffStore) HasUTXODiffChild(dbContext model.DBReader, blockHash *
} }
// Delete deletes the utxoDiff associated with the given blockHash // Delete deletes the utxoDiff associated with the given blockHash
func (uds *utxoDiffStore) Delete(blockHash *externalapi.DomainHash) { func (uds *utxoDiffStore) Delete(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) {
if uds.IsBlockHashStaged(blockHash) { stagingShard := uds.stagingShard(stagingArea)
if _, ok := uds.utxoDiffStaging[*blockHash]; ok {
delete(uds.utxoDiffStaging, *blockHash) if uds.isBlockHashStaged(stagingShard, blockHash) {
if _, ok := stagingShard.utxoDiffToAdd[*blockHash]; ok {
delete(stagingShard.utxoDiffToAdd, *blockHash)
} }
if _, ok := uds.utxoDiffChildStaging[*blockHash]; ok { if _, ok := stagingShard.utxoDiffChildToAdd[*blockHash]; ok {
delete(uds.utxoDiffChildStaging, *blockHash) delete(stagingShard.utxoDiffChildToAdd, *blockHash)
} }
return return
} }
uds.toDelete[*blockHash] = struct{}{} stagingShard.toDelete[*blockHash] = struct{}{}
} }
func (uds *utxoDiffStore) utxoDiffHashAsKey(hash *externalapi.DomainHash) model.DBKey { func (uds *utxoDiffStore) utxoDiffHashAsKey(hash *externalapi.DomainHash) model.DBKey {

View File

@ -1,11 +1,12 @@
package consensus package consensus
import ( import (
daablocksstore "github.com/kaspanet/kaspad/domain/consensus/datastructures/daablocksstore"
"io/ioutil" "io/ioutil"
"os" "os"
"sync" "sync"
daablocksstore "github.com/kaspanet/kaspad/domain/consensus/datastructures/daablocksstore"
"github.com/kaspanet/kaspad/domain/consensus/datastructures/headersselectedchainstore" "github.com/kaspanet/kaspad/domain/consensus/datastructures/headersselectedchainstore"
"github.com/kaspanet/kaspad/domain/consensus/processes/dagtraversalmanager" "github.com/kaspanet/kaspad/domain/consensus/processes/dagtraversalmanager"

View File

@ -83,6 +83,8 @@ func TestFinality(t *testing.T) {
} }
} }
stagingArea := model.NewStagingArea()
// Add two more blocks in the side-chain until it becomes the selected chain // Add two more blocks in the side-chain until it becomes the selected chain
for i := uint64(0); i < 2; i++ { for i := uint64(0); i < 2; i++ {
sideChainTip, err = buildAndInsertBlock([]*externalapi.DomainHash{sideChainTipHash}) sideChainTip, err = buildAndInsertBlock([]*externalapi.DomainHash{sideChainTipHash})
@ -120,7 +122,7 @@ func TestFinality(t *testing.T) {
mainChainTipHash = consensushashing.BlockHash(mainChainTip) mainChainTipHash = consensushashing.BlockHash(mainChainTip)
} }
virtualFinality, err := consensus.FinalityManager().VirtualFinalityPoint() virtualFinality, err := consensus.FinalityManager().VirtualFinalityPoint(stagingArea)
if err != nil { if err != nil {
t.Fatalf("TestFinality: Failed getting the virtual's finality point: %v", err) t.Fatalf("TestFinality: Failed getting the virtual's finality point: %v", err)
} }
@ -145,12 +147,14 @@ func TestFinality(t *testing.T) {
if err != nil { if err != nil {
t.Fatalf("TestFinality: Failed getting virtual selectedParent: %v", err) t.Fatalf("TestFinality: Failed getting virtual selectedParent: %v", err)
} }
selectedTipGhostDagData, err := consensus.GHOSTDAGDataStore().Get(consensus.DatabaseContext(), selectedTip) selectedTipGhostDagData, err :=
consensus.GHOSTDAGDataStore().Get(consensus.DatabaseContext(), stagingArea, selectedTip)
if err != nil { if err != nil {
t.Fatalf("TestFinality: Failed getting the ghost dag data of the selected tip: %v", err) t.Fatalf("TestFinality: Failed getting the ghost dag data of the selected tip: %v", err)
} }
sideChainTipGhostDagData, err := consensus.GHOSTDAGDataStore().Get(consensus.DatabaseContext(), sideChainTipHash) sideChainTipGhostDagData, err :=
consensus.GHOSTDAGDataStore().Get(consensus.DatabaseContext(), stagingArea, sideChainTipHash)
if err != nil { if err != nil {
t.Fatalf("TestFinality: Failed getting the ghost dag data of the sidechain tip: %v", err) t.Fatalf("TestFinality: Failed getting the ghost dag data of the sidechain tip: %v", err)
} }
@ -302,7 +306,9 @@ func TestBoundedMergeDepth(t *testing.T) {
t.Fatalf("TestBoundedMergeDepth: Expected blueKosherizingBlock to not violate merge depth") t.Fatalf("TestBoundedMergeDepth: Expected blueKosherizingBlock to not violate merge depth")
} }
virtualGhotDagData, err := consensusReal.GHOSTDAGDataStore().Get(consensusReal.DatabaseContext(), model.VirtualBlockHash) stagingArea := model.NewStagingArea()
virtualGhotDagData, err := consensusReal.GHOSTDAGDataStore().Get(consensusReal.DatabaseContext(),
stagingArea, model.VirtualBlockHash)
if err != nil { if err != nil {
t.Fatalf("TestBoundedMergeDepth: Failed getting the ghostdag data of the virtual: %v", err) t.Fatalf("TestBoundedMergeDepth: Failed getting the ghostdag data of the virtual: %v", err)
} }
@ -350,7 +356,8 @@ func TestBoundedMergeDepth(t *testing.T) {
t.Fatalf("TestBoundedMergeDepth: Expected %s to be the selectedTip but found %s instead", tip, virtualSelectedParent) t.Fatalf("TestBoundedMergeDepth: Expected %s to be the selectedTip but found %s instead", tip, virtualSelectedParent)
} }
virtualGhotDagData, err = consensusReal.GHOSTDAGDataStore().Get(consensusReal.DatabaseContext(), model.VirtualBlockHash) virtualGhotDagData, err = consensusReal.GHOSTDAGDataStore().Get(
consensusReal.DatabaseContext(), stagingArea, model.VirtualBlockHash)
if err != nil { if err != nil {
t.Fatalf("TestBoundedMergeDepth: Failed getting the ghostdag data of the virtual: %v", err) t.Fatalf("TestBoundedMergeDepth: Failed getting the ghostdag data of the virtual: %v", err)
} }
@ -372,7 +379,8 @@ func TestBoundedMergeDepth(t *testing.T) {
} }
// Now `pointAtBlueKosherizing` itself is actually still blue, so we can still point at that even though we can't point at kosherizing directly anymore // Now `pointAtBlueKosherizing` itself is actually still blue, so we can still point at that even though we can't point at kosherizing directly anymore
transitiveBlueKosherizing, isViolatingMergeDepth := checkViolatingMergeDepth(consensusReal, []*externalapi.DomainHash{consensushashing.BlockHash(pointAtBlueKosherizing), tip}) transitiveBlueKosherizing, isViolatingMergeDepth :=
checkViolatingMergeDepth(consensusReal, []*externalapi.DomainHash{consensushashing.BlockHash(pointAtBlueKosherizing), tip})
if isViolatingMergeDepth { if isViolatingMergeDepth {
t.Fatalf("TestBoundedMergeDepth: Expected transitiveBlueKosherizing to not violate merge depth") t.Fatalf("TestBoundedMergeDepth: Expected transitiveBlueKosherizing to not violate merge depth")
} }

View File

@ -5,8 +5,8 @@ import "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
// AcceptanceDataStore represents a store of AcceptanceData // AcceptanceDataStore represents a store of AcceptanceData
type AcceptanceDataStore interface { type AcceptanceDataStore interface {
Store Store
Stage(blockHash *externalapi.DomainHash, acceptanceData externalapi.AcceptanceData) Stage(stagingArea *StagingArea, blockHash *externalapi.DomainHash, acceptanceData externalapi.AcceptanceData)
IsStaged() bool IsStaged(stagingArea *StagingArea) bool
Get(dbContext DBReader, blockHash *externalapi.DomainHash) (externalapi.AcceptanceData, error) Get(dbContext DBReader, stagingArea *StagingArea, blockHash *externalapi.DomainHash) (externalapi.AcceptanceData, error)
Delete(blockHash *externalapi.DomainHash) Delete(stagingArea *StagingArea, blockHash *externalapi.DomainHash)
} }

View File

@ -5,11 +5,11 @@ import "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
// BlockHeaderStore represents a store of block headers // BlockHeaderStore represents a store of block headers
type BlockHeaderStore interface { type BlockHeaderStore interface {
Store Store
Stage(blockHash *externalapi.DomainHash, blockHeader externalapi.BlockHeader) Stage(stagingArea *StagingArea, blockHash *externalapi.DomainHash, blockHeader externalapi.BlockHeader)
IsStaged() bool IsStaged(stagingArea *StagingArea) bool
BlockHeader(dbContext DBReader, blockHash *externalapi.DomainHash) (externalapi.BlockHeader, error) BlockHeader(dbContext DBReader, stagingArea *StagingArea, blockHash *externalapi.DomainHash) (externalapi.BlockHeader, error)
HasBlockHeader(dbContext DBReader, blockHash *externalapi.DomainHash) (bool, error) HasBlockHeader(dbContext DBReader, stagingArea *StagingArea, blockHash *externalapi.DomainHash) (bool, error)
BlockHeaders(dbContext DBReader, blockHashes []*externalapi.DomainHash) ([]externalapi.BlockHeader, error) BlockHeaders(dbContext DBReader, stagingArea *StagingArea, blockHashes []*externalapi.DomainHash) ([]externalapi.BlockHeader, error)
Delete(blockHash *externalapi.DomainHash) Delete(stagingArea *StagingArea, blockHash *externalapi.DomainHash)
Count() uint64 Count(stagingArea *StagingArea) uint64
} }

View File

@ -5,8 +5,8 @@ import "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
// BlockRelationStore represents a store of BlockRelations // BlockRelationStore represents a store of BlockRelations
type BlockRelationStore interface { type BlockRelationStore interface {
Store Store
StageBlockRelation(blockHash *externalapi.DomainHash, blockRelations *BlockRelations) StageBlockRelation(stagingArea *StagingArea, blockHash *externalapi.DomainHash, blockRelations *BlockRelations)
IsStaged() bool IsStaged(stagingArea *StagingArea) bool
BlockRelation(dbContext DBReader, blockHash *externalapi.DomainHash) (*BlockRelations, error) BlockRelation(dbContext DBReader, stagingArea *StagingArea, blockHash *externalapi.DomainHash) (*BlockRelations, error)
Has(dbContext DBReader, blockHash *externalapi.DomainHash) (bool, error) Has(dbContext DBReader, stagingArea *StagingArea, blockHash *externalapi.DomainHash) (bool, error)
} }

View File

@ -5,8 +5,8 @@ import "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
// BlockStatusStore represents a store of BlockStatuses // BlockStatusStore represents a store of BlockStatuses
type BlockStatusStore interface { type BlockStatusStore interface {
Store Store
Stage(blockHash *externalapi.DomainHash, blockStatus externalapi.BlockStatus) Stage(stagingArea *StagingArea, blockHash *externalapi.DomainHash, blockStatus externalapi.BlockStatus)
IsStaged() bool IsStaged(stagingArea *StagingArea) bool
Get(dbContext DBReader, blockHash *externalapi.DomainHash) (externalapi.BlockStatus, error) Get(dbContext DBReader, stagingArea *StagingArea, blockHash *externalapi.DomainHash) (externalapi.BlockStatus, error)
Exists(dbContext DBReader, blockHash *externalapi.DomainHash) (bool, error) Exists(dbContext DBReader, stagingArea *StagingArea, blockHash *externalapi.DomainHash) (bool, error)
} }

View File

@ -5,12 +5,12 @@ import "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
// BlockStore represents a store of blocks // BlockStore represents a store of blocks
type BlockStore interface { type BlockStore interface {
Store Store
Stage(blockHash *externalapi.DomainHash, block *externalapi.DomainBlock) Stage(stagingArea *StagingArea, blockHash *externalapi.DomainHash, block *externalapi.DomainBlock)
IsStaged() bool IsStaged(stagingArea *StagingArea) bool
Block(dbContext DBReader, blockHash *externalapi.DomainHash) (*externalapi.DomainBlock, error) Block(dbContext DBReader, stagingArea *StagingArea, blockHash *externalapi.DomainHash) (*externalapi.DomainBlock, error)
HasBlock(dbContext DBReader, blockHash *externalapi.DomainHash) (bool, error) HasBlock(dbContext DBReader, stagingArea *StagingArea, blockHash *externalapi.DomainHash) (bool, error)
Blocks(dbContext DBReader, blockHashes []*externalapi.DomainHash) ([]*externalapi.DomainBlock, error) Blocks(dbContext DBReader, stagingArea *StagingArea, blockHashes []*externalapi.DomainHash) ([]*externalapi.DomainBlock, error)
Delete(blockHash *externalapi.DomainHash) Delete(stagingArea *StagingArea, blockHash *externalapi.DomainHash)
Count() uint64 Count(stagingArea *StagingArea) uint64
AllBlockHashesIterator(dbContext DBReader) (BlockIterator, error) AllBlockHashesIterator(dbContext DBReader) (BlockIterator, error)
} }

View File

@ -5,17 +5,16 @@ import "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
// ConsensusStateStore represents a store for the current consensus state // ConsensusStateStore represents a store for the current consensus state
type ConsensusStateStore interface { type ConsensusStateStore interface {
Store Store
IsStaged() bool IsStaged(stagingArea *StagingArea) bool
StageVirtualUTXODiff(virtualUTXODiff externalapi.UTXODiff) StageVirtualUTXODiff(stagingArea *StagingArea, virtualUTXODiff externalapi.UTXODiff)
UTXOByOutpoint(dbContext DBReader, outpoint *externalapi.DomainOutpoint) (externalapi.UTXOEntry, error) UTXOByOutpoint(dbContext DBReader, stagingArea *StagingArea, outpoint *externalapi.DomainOutpoint) (externalapi.UTXOEntry, error)
HasUTXOByOutpoint(dbContext DBReader, outpoint *externalapi.DomainOutpoint) (bool, error) HasUTXOByOutpoint(dbContext DBReader, stagingArea *StagingArea, outpoint *externalapi.DomainOutpoint) (bool, error)
VirtualUTXOSetIterator(dbContext DBReader) (externalapi.ReadOnlyUTXOSetIterator, error) VirtualUTXOSetIterator(dbContext DBReader, stagingArea *StagingArea) (externalapi.ReadOnlyUTXOSetIterator, error)
VirtualUTXOs(dbContext DBReader, VirtualUTXOs(dbContext DBReader, fromOutpoint *externalapi.DomainOutpoint, limit int) ([]*externalapi.OutpointAndUTXOEntryPair, error)
fromOutpoint *externalapi.DomainOutpoint, limit int) ([]*externalapi.OutpointAndUTXOEntryPair, error)
StageTips(tipHashes []*externalapi.DomainHash) StageTips(stagingArea *StagingArea, tipHashes []*externalapi.DomainHash)
Tips(dbContext DBReader) ([]*externalapi.DomainHash, error) Tips(stagingArea *StagingArea, dbContext DBReader) ([]*externalapi.DomainHash, error)
StartImportingPruningPointUTXOSet(dbContext DBWriter) error StartImportingPruningPointUTXOSet(dbContext DBWriter) error
HadStartedImportingPruningPointUTXOSet(dbContext DBWriter) (bool, error) HadStartedImportingPruningPointUTXOSet(dbContext DBWriter) (bool, error)

View File

@ -5,10 +5,10 @@ import "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
// DAABlocksStore represents a store of ??? // DAABlocksStore represents a store of ???
type DAABlocksStore interface { type DAABlocksStore interface {
Store Store
StageDAAScore(blockHash *externalapi.DomainHash, daaScore uint64) StageDAAScore(stagingArea *StagingArea, blockHash *externalapi.DomainHash, daaScore uint64)
StageBlockDAAAddedBlocks(blockHash *externalapi.DomainHash, addedBlocks []*externalapi.DomainHash) StageBlockDAAAddedBlocks(stagingArea *StagingArea, blockHash *externalapi.DomainHash, addedBlocks []*externalapi.DomainHash)
IsAnythingStaged() bool IsStaged(stagingArea *StagingArea) bool
DAAAddedBlocks(dbContext DBReader, blockHash *externalapi.DomainHash) ([]*externalapi.DomainHash, error) DAAAddedBlocks(dbContext DBReader, stagingArea *StagingArea, blockHash *externalapi.DomainHash) ([]*externalapi.DomainHash, error)
DAAScore(dbContext DBReader, blockHash *externalapi.DomainHash) (uint64, error) DAAScore(dbContext DBReader, stagingArea *StagingArea, blockHash *externalapi.DomainHash) (uint64, error)
Delete(blockHash *externalapi.DomainHash) Delete(stagingArea *StagingArea, blockHash *externalapi.DomainHash)
} }

View File

@ -7,7 +7,7 @@ import (
// FinalityStore represents a store for finality data // FinalityStore represents a store for finality data
type FinalityStore interface { type FinalityStore interface {
Store Store
IsStaged() bool IsStaged(stagingArea *StagingArea) bool
StageFinalityPoint(blockHash *externalapi.DomainHash, finalityPointHash *externalapi.DomainHash) StageFinalityPoint(stagingArea *StagingArea, blockHash *externalapi.DomainHash, finalityPointHash *externalapi.DomainHash)
FinalityPoint(dbContext DBReader, blockHash *externalapi.DomainHash) (*externalapi.DomainHash, error) FinalityPoint(dbContext DBReader, stagingArea *StagingArea, blockHash *externalapi.DomainHash) (*externalapi.DomainHash, error)
} }

View File

@ -5,7 +5,7 @@ import "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
// GHOSTDAGDataStore represents a store of BlockGHOSTDAGData // GHOSTDAGDataStore represents a store of BlockGHOSTDAGData
type GHOSTDAGDataStore interface { type GHOSTDAGDataStore interface {
Store Store
Stage(blockHash *externalapi.DomainHash, blockGHOSTDAGData *BlockGHOSTDAGData) Stage(stagingArea *StagingArea, blockHash *externalapi.DomainHash, blockGHOSTDAGData *BlockGHOSTDAGData)
IsStaged() bool IsStaged(stagingArea *StagingArea) bool
Get(dbContext DBReader, blockHash *externalapi.DomainHash) (*BlockGHOSTDAGData, error) Get(dbContext DBReader, stagingArea *StagingArea, blockHash *externalapi.DomainHash) (*BlockGHOSTDAGData, error)
} }

View File

@ -5,9 +5,8 @@ import "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
// HeadersSelectedChainStore represents a store of the headers selected chain // HeadersSelectedChainStore represents a store of the headers selected chain
type HeadersSelectedChainStore interface { type HeadersSelectedChainStore interface {
Store Store
Stage(dbContext DBReader, Stage(dbContext DBReader, stagingArea *StagingArea, chainChanges *externalapi.SelectedChainPath) error
chainChanges *externalapi.SelectedChainPath) error IsStaged(stagingArea *StagingArea) bool
IsStaged() bool GetIndexByHash(dbContext DBReader, stagingArea *StagingArea, blockHash *externalapi.DomainHash) (uint64, error)
GetIndexByHash(dbContext DBReader, blockHash *externalapi.DomainHash) (uint64, error) GetHashByIndex(dbContext DBReader, stagingArea *StagingArea, index uint64) (*externalapi.DomainHash, error)
GetHashByIndex(dbContext DBReader, index uint64) (*externalapi.DomainHash, error)
} }

View File

@ -5,8 +5,8 @@ import "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
// HeaderSelectedTipStore represents a store of the headers selected tip // HeaderSelectedTipStore represents a store of the headers selected tip
type HeaderSelectedTipStore interface { type HeaderSelectedTipStore interface {
Store Store
Stage(selectedTip *externalapi.DomainHash) Stage(stagingArea *StagingArea, selectedTip *externalapi.DomainHash)
IsStaged() bool IsStaged(stagingArea *StagingArea) bool
HeadersSelectedTip(dbContext DBReader) (*externalapi.DomainHash, error) HeadersSelectedTip(dbContext DBReader, stagingArea *StagingArea) (*externalapi.DomainHash, error)
Has(dbContext DBReader) (bool, error) Has(dbContext DBReader, stagingArea *StagingArea) (bool, error)
} }

View File

@ -5,8 +5,8 @@ import "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
// MultisetStore represents a store of Multisets // MultisetStore represents a store of Multisets
type MultisetStore interface { type MultisetStore interface {
Store Store
Stage(blockHash *externalapi.DomainHash, multiset Multiset) Stage(stagingArea *StagingArea, blockHash *externalapi.DomainHash, multiset Multiset)
IsStaged() bool IsStaged(stagingArea *StagingArea) bool
Get(dbContext DBReader, blockHash *externalapi.DomainHash) (Multiset, error) Get(dbContext DBReader, stagingArea *StagingArea, blockHash *externalapi.DomainHash) (Multiset, error)
Delete(blockHash *externalapi.DomainHash) Delete(stagingArea *StagingArea, blockHash *externalapi.DomainHash)
} }

View File

@ -5,15 +5,15 @@ import "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
// PruningStore represents a store for the current pruning state // PruningStore represents a store for the current pruning state
type PruningStore interface { type PruningStore interface {
Store Store
StagePruningPoint(pruningPointBlockHash *externalapi.DomainHash) StagePruningPoint(stagingArea *StagingArea, pruningPointBlockHash *externalapi.DomainHash)
StagePruningPointCandidate(candidate *externalapi.DomainHash) StagePruningPointCandidate(stagingArea *StagingArea, candidate *externalapi.DomainHash)
IsStaged() bool IsStaged(stagingArea *StagingArea) bool
PruningPointCandidate(dbContext DBReader) (*externalapi.DomainHash, error) PruningPointCandidate(dbContext DBReader, stagingArea *StagingArea) (*externalapi.DomainHash, error)
HasPruningPointCandidate(dbContext DBReader) (bool, error) HasPruningPointCandidate(dbContext DBReader, stagingArea *StagingArea) (bool, error)
PruningPoint(dbContext DBReader) (*externalapi.DomainHash, error) PruningPoint(dbContext DBReader, stagingArea *StagingArea) (*externalapi.DomainHash, error)
HasPruningPoint(dbContext DBReader) (bool, error) HasPruningPoint(dbContext DBReader, stagingArea *StagingArea) (bool, error)
StageStartUpdatingPruningPointUTXOSet() StageStartUpdatingPruningPointUTXOSet(stagingArea *StagingArea)
HadStartedUpdatingPruningPointUTXOSet(dbContext DBWriter) (bool, error) HadStartedUpdatingPruningPointUTXOSet(dbContext DBWriter) (bool, error)
FinishUpdatingPruningPointUTXOSet(dbContext DBWriter) error FinishUpdatingPruningPointUTXOSet(dbContext DBWriter) error
UpdatePruningPointUTXOSet(dbContext DBWriter, utxoSetIterator externalapi.ReadOnlyUTXOSetIterator) error UpdatePruningPointUTXOSet(dbContext DBWriter, utxoSetIterator externalapi.ReadOnlyUTXOSetIterator) error

View File

@ -5,10 +5,10 @@ import "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
// ReachabilityDataStore represents a store of ReachabilityData // ReachabilityDataStore represents a store of ReachabilityData
type ReachabilityDataStore interface { type ReachabilityDataStore interface {
Store Store
StageReachabilityData(blockHash *externalapi.DomainHash, reachabilityData ReachabilityData) StageReachabilityData(stagingArea *StagingArea, blockHash *externalapi.DomainHash, reachabilityData ReachabilityData)
StageReachabilityReindexRoot(reachabilityReindexRoot *externalapi.DomainHash) StageReachabilityReindexRoot(stagingArea *StagingArea, reachabilityReindexRoot *externalapi.DomainHash)
IsAnythingStaged() bool IsStaged(stagingArea *StagingArea) bool
ReachabilityData(dbContext DBReader, blockHash *externalapi.DomainHash) (ReachabilityData, error) ReachabilityData(dbContext DBReader, stagingArea *StagingArea, blockHash *externalapi.DomainHash) (ReachabilityData, error)
HasReachabilityData(dbContext DBReader, blockHash *externalapi.DomainHash) (bool, error) HasReachabilityData(dbContext DBReader, stagingArea *StagingArea, blockHash *externalapi.DomainHash) (bool, error)
ReachabilityReindexRoot(dbContext DBReader) (*externalapi.DomainHash, error) ReachabilityReindexRoot(dbContext DBReader, stagingArea *StagingArea) (*externalapi.DomainHash, error)
} }

View File

@ -5,10 +5,10 @@ import "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
// UTXODiffStore represents a store of UTXODiffs // UTXODiffStore represents a store of UTXODiffs
type UTXODiffStore interface { type UTXODiffStore interface {
Store Store
Stage(blockHash *externalapi.DomainHash, utxoDiff externalapi.UTXODiff, utxoDiffChild *externalapi.DomainHash) Stage(stagingArea *StagingArea, blockHash *externalapi.DomainHash, utxoDiff externalapi.UTXODiff, utxoDiffChild *externalapi.DomainHash)
IsStaged() bool IsStaged(stagingArea *StagingArea) bool
UTXODiff(dbContext DBReader, blockHash *externalapi.DomainHash) (externalapi.UTXODiff, error) UTXODiff(dbContext DBReader, stagingArea *StagingArea, blockHash *externalapi.DomainHash) (externalapi.UTXODiff, error)
UTXODiffChild(dbContext DBReader, blockHash *externalapi.DomainHash) (*externalapi.DomainHash, error) UTXODiffChild(dbContext DBReader, stagingArea *StagingArea, blockHash *externalapi.DomainHash) (*externalapi.DomainHash, error)
HasUTXODiffChild(dbContext DBReader, blockHash *externalapi.DomainHash) (bool, error) HasUTXODiffChild(dbContext DBReader, stagingArea *StagingArea, blockHash *externalapi.DomainHash) (bool, error)
Delete(blockHash *externalapi.DomainHash) Delete(stagingArea *StagingArea, blockHash *externalapi.DomainHash)
} }

View File

@ -7,9 +7,9 @@ import (
// BlockValidator exposes a set of validation classes, after which // BlockValidator exposes a set of validation classes, after which
// it's possible to determine whether a block is valid // it's possible to determine whether a block is valid
type BlockValidator interface { type BlockValidator interface {
ValidateHeaderInIsolation(blockHash *externalapi.DomainHash) error ValidateHeaderInIsolation(stagingArea *StagingArea, blockHash *externalapi.DomainHash) error
ValidateBodyInIsolation(blockHash *externalapi.DomainHash) error ValidateBodyInIsolation(stagingArea *StagingArea, blockHash *externalapi.DomainHash) error
ValidateHeaderInContext(blockHash *externalapi.DomainHash) error ValidateHeaderInContext(stagingArea *StagingArea, blockHash *externalapi.DomainHash) error
ValidateBodyInContext(blockHash *externalapi.DomainHash, isPruningPoint bool) error ValidateBodyInContext(stagingArea *StagingArea, blockHash *externalapi.DomainHash, isPruningPoint bool) error
ValidatePruningPointViolationAndProofOfWorkAndDifficulty(blockHash *externalapi.DomainHash) error ValidatePruningPointViolationAndProofOfWorkAndDifficulty(stagingArea *StagingArea, blockHash *externalapi.DomainHash) error
} }

View File

@ -5,8 +5,7 @@ import "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
// CoinbaseManager exposes methods for handling blocks' // CoinbaseManager exposes methods for handling blocks'
// coinbase transactions // coinbase transactions
type CoinbaseManager interface { type CoinbaseManager interface {
ExpectedCoinbaseTransaction(blockHash *externalapi.DomainHash, ExpectedCoinbaseTransaction(stagingArea *StagingArea, blockHash *externalapi.DomainHash,
coinbaseData *externalapi.DomainCoinbaseData) (*externalapi.DomainTransaction, error) coinbaseData *externalapi.DomainCoinbaseData) (*externalapi.DomainTransaction, error)
ExtractCoinbaseDataAndBlueScore(coinbaseTx *externalapi.DomainTransaction) (blueScore uint64, ExtractCoinbaseDataAndBlueScore(coinbaseTx *externalapi.DomainTransaction) (blueScore uint64, coinbaseData *externalapi.DomainCoinbaseData, err error)
coinbaseData *externalapi.DomainCoinbaseData, err error)
} }

View File

@ -4,11 +4,11 @@ import "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
// ConsensusStateManager manages the node's consensus state // ConsensusStateManager manages the node's consensus state
type ConsensusStateManager interface { type ConsensusStateManager interface {
AddBlock(blockHash *externalapi.DomainHash) (*externalapi.SelectedChainPath, externalapi.UTXODiff, error) AddBlock(stagingArea *StagingArea, blockHash *externalapi.DomainHash) (*externalapi.SelectedChainPath, externalapi.UTXODiff, error)
PopulateTransactionWithUTXOEntries(transaction *externalapi.DomainTransaction) error PopulateTransactionWithUTXOEntries(stagingArea *StagingArea, transaction *externalapi.DomainTransaction) error
ImportPruningPoint(newPruningPoint *externalapi.DomainBlock) error ImportPruningPoint(stagingArea *StagingArea, newPruningPoint *externalapi.DomainBlock) error
RestorePastUTXOSetIterator(blockHash *externalapi.DomainHash) (externalapi.ReadOnlyUTXOSetIterator, error) RestorePastUTXOSetIterator(stagingArea *StagingArea, blockHash *externalapi.DomainHash) (externalapi.ReadOnlyUTXOSetIterator, error)
CalculatePastUTXOAndAcceptanceData(blockHash *externalapi.DomainHash) (externalapi.UTXODiff, externalapi.AcceptanceData, Multiset, error) CalculatePastUTXOAndAcceptanceData(stagingArea *StagingArea, blockHash *externalapi.DomainHash) (externalapi.UTXODiff, externalapi.AcceptanceData, Multiset, error)
GetVirtualSelectedParentChainFromBlock(blockHash *externalapi.DomainHash) (*externalapi.SelectedChainPath, error) GetVirtualSelectedParentChainFromBlock(stagingArea *StagingArea, blockHash *externalapi.DomainHash) (*externalapi.SelectedChainPath, error)
RecoverUTXOIfRequired() error RecoverUTXOIfRequired() error
} }

View File

@ -5,15 +5,15 @@ import "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
// DAGTopologyManager exposes methods for querying relationships // DAGTopologyManager exposes methods for querying relationships
// between blocks in the DAG // between blocks in the DAG
type DAGTopologyManager interface { type DAGTopologyManager interface {
Parents(blockHash *externalapi.DomainHash) ([]*externalapi.DomainHash, error) Parents(stagingArea *StagingArea, blockHash *externalapi.DomainHash) ([]*externalapi.DomainHash, error)
Children(blockHash *externalapi.DomainHash) ([]*externalapi.DomainHash, error) Children(stagingArea *StagingArea, blockHash *externalapi.DomainHash) ([]*externalapi.DomainHash, error)
IsParentOf(blockHashA *externalapi.DomainHash, blockHashB *externalapi.DomainHash) (bool, error) IsParentOf(stagingArea *StagingArea, blockHashA *externalapi.DomainHash, blockHashB *externalapi.DomainHash) (bool, error)
IsChildOf(blockHashA *externalapi.DomainHash, blockHashB *externalapi.DomainHash) (bool, error) IsChildOf(stagingArea *StagingArea, blockHashA *externalapi.DomainHash, blockHashB *externalapi.DomainHash) (bool, error)
IsAncestorOf(blockHashA *externalapi.DomainHash, blockHashB *externalapi.DomainHash) (bool, error) IsAncestorOf(stagingArea *StagingArea, blockHashA *externalapi.DomainHash, blockHashB *externalapi.DomainHash) (bool, error)
IsAncestorOfAny(blockHash *externalapi.DomainHash, potentialDescendants []*externalapi.DomainHash) (bool, error) IsAncestorOfAny(stagingArea *StagingArea, blockHash *externalapi.DomainHash, potentialDescendants []*externalapi.DomainHash) (bool, error)
IsAnyAncestorOf(potentialAncestors []*externalapi.DomainHash, blockHash *externalapi.DomainHash) (bool, error) IsAnyAncestorOf(stagingArea *StagingArea, potentialAncestors []*externalapi.DomainHash, blockHash *externalapi.DomainHash) (bool, error)
IsInSelectedParentChainOf(blockHashA *externalapi.DomainHash, blockHashB *externalapi.DomainHash) (bool, error) IsInSelectedParentChainOf(stagingArea *StagingArea, blockHashA *externalapi.DomainHash, blockHashB *externalapi.DomainHash) (bool, error)
ChildInSelectedParentChainOf(context, highHash *externalapi.DomainHash) (*externalapi.DomainHash, error) ChildInSelectedParentChainOf(stagingArea *StagingArea, context, highHash *externalapi.DomainHash) (*externalapi.DomainHash, error)
SetParents(blockHash *externalapi.DomainHash, parentHashes []*externalapi.DomainHash) error SetParents(stagingArea *StagingArea, blockHash *externalapi.DomainHash, parentHashes []*externalapi.DomainHash) error
} }

View File

@ -5,15 +5,15 @@ import "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
// DAGTraversalManager exposes methods for traversing blocks // DAGTraversalManager exposes methods for traversing blocks
// in the DAG // in the DAG
type DAGTraversalManager interface { type DAGTraversalManager interface {
BlockAtDepth(highHash *externalapi.DomainHash, depth uint64) (*externalapi.DomainHash, error) BlockAtDepth(stagingArea *StagingArea, highHash *externalapi.DomainHash, depth uint64) (*externalapi.DomainHash, error)
LowestChainBlockAboveOrEqualToBlueScore(highHash *externalapi.DomainHash, blueScore uint64) (*externalapi.DomainHash, error) LowestChainBlockAboveOrEqualToBlueScore(stagingArea *StagingArea, highHash *externalapi.DomainHash, blueScore uint64) (*externalapi.DomainHash, error)
// SelectedChildIterator should return a BlockIterator that iterates // SelectedChildIterator should return a BlockIterator that iterates
// from lowHash (exclusive) to highHash (inclusive) over highHash's selected parent chain // from lowHash (exclusive) to highHash (inclusive) over highHash's selected parent chain
SelectedChildIterator(highHash, lowHash *externalapi.DomainHash) (BlockIterator, error) SelectedChildIterator(stagingArea *StagingArea, highHash, lowHash *externalapi.DomainHash) (BlockIterator, error)
Anticone(blockHash *externalapi.DomainHash) ([]*externalapi.DomainHash, error) Anticone(stagingArea *StagingArea, blockHash *externalapi.DomainHash) ([]*externalapi.DomainHash, error)
BlockWindow(highHash *externalapi.DomainHash, windowSize int) ([]*externalapi.DomainHash, error) BlockWindow(stagingArea *StagingArea, highHash *externalapi.DomainHash, windowSize int) ([]*externalapi.DomainHash, error)
NewDownHeap() BlockHeap NewDownHeap(stagingArea *StagingArea) BlockHeap
NewUpHeap() BlockHeap NewUpHeap(stagingArea *StagingArea) BlockHeap
CalculateChainPath( CalculateChainPath(stagingArea *StagingArea, fromBlockHash, toBlockHash *externalapi.DomainHash) (
fromBlockHash, toBlockHash *externalapi.DomainHash) (*externalapi.SelectedChainPath, error) *externalapi.SelectedChainPath, error)
} }

View File

@ -5,6 +5,6 @@ import "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
// DifficultyManager provides a method to resolve the // DifficultyManager provides a method to resolve the
// difficulty value of a block // difficulty value of a block
type DifficultyManager interface { type DifficultyManager interface {
StageDAADataAndReturnRequiredDifficulty(blockHash *externalapi.DomainHash) (uint32, error) StageDAADataAndReturnRequiredDifficulty(stagingArea *StagingArea, blockHash *externalapi.DomainHash) (uint32, error)
RequiredDifficulty(blockHash *externalapi.DomainHash) (uint32, error) RequiredDifficulty(stagingArea *StagingArea, blockHash *externalapi.DomainHash) (uint32, error)
} }

View File

@ -4,6 +4,6 @@ import "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
// FinalityManager provides method to validate that a block does not violate finality // FinalityManager provides method to validate that a block does not violate finality
type FinalityManager interface { type FinalityManager interface {
VirtualFinalityPoint() (*externalapi.DomainHash, error) VirtualFinalityPoint(stagingArea *StagingArea) (*externalapi.DomainHash, error)
FinalityPoint(blockHash *externalapi.DomainHash) (*externalapi.DomainHash, error) FinalityPoint(stagingArea *StagingArea, blockHash *externalapi.DomainHash) (*externalapi.DomainHash, error)
} }

View File

@ -4,8 +4,8 @@ import "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
// GHOSTDAGManager resolves and manages GHOSTDAG block data // GHOSTDAGManager resolves and manages GHOSTDAG block data
type GHOSTDAGManager interface { type GHOSTDAGManager interface {
GHOSTDAG(blockHash *externalapi.DomainHash) error GHOSTDAG(stagingArea *StagingArea, blockHash *externalapi.DomainHash) error
ChooseSelectedParent(blockHashes ...*externalapi.DomainHash) (*externalapi.DomainHash, error) ChooseSelectedParent(stagingArea *StagingArea, blockHashes ...*externalapi.DomainHash) (*externalapi.DomainHash, error)
Less(blockHashA *externalapi.DomainHash, ghostdagDataA *BlockGHOSTDAGData, Less(blockHashA *externalapi.DomainHash, ghostdagDataA *BlockGHOSTDAGData,
blockHashB *externalapi.DomainHash, ghostdagDataB *BlockGHOSTDAGData) bool blockHashB *externalapi.DomainHash, ghostdagDataB *BlockGHOSTDAGData) bool
} }

View File

@ -4,5 +4,5 @@ import "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
// HeadersSelectedTipManager manages the state of the headers selected tip // HeadersSelectedTipManager manages the state of the headers selected tip
type HeadersSelectedTipManager interface { type HeadersSelectedTipManager interface {
AddHeaderTip(hash *externalapi.DomainHash) error AddHeaderTip(stagingArea *StagingArea, hash *externalapi.DomainHash) error
} }

View File

@ -4,6 +4,6 @@ import "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
// MergeDepthManager is used to validate mergeDepth for blocks // MergeDepthManager is used to validate mergeDepth for blocks
type MergeDepthManager interface { type MergeDepthManager interface {
CheckBoundedMergeDepth(blockHash *externalapi.DomainHash) error CheckBoundedMergeDepth(stagingArea *StagingArea, blockHash *externalapi.DomainHash) error
NonBoundedMergeDepthViolatingBlues(blockHash *externalapi.DomainHash) ([]*externalapi.DomainHash, error) NonBoundedMergeDepthViolatingBlues(stagingArea *StagingArea, blockHash *externalapi.DomainHash) ([]*externalapi.DomainHash, error)
} }

View File

@ -5,5 +5,5 @@ import "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
// PastMedianTimeManager provides a method to resolve the // PastMedianTimeManager provides a method to resolve the
// past median time of a block // past median time of a block
type PastMedianTimeManager interface { type PastMedianTimeManager interface {
PastMedianTime(blockHash *externalapi.DomainHash) (int64, error) PastMedianTime(stagingArea *StagingArea, blockHash *externalapi.DomainHash) (int64, error)
} }

View File

@ -4,10 +4,10 @@ import "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
// PruningManager resolves and manages the current pruning point // PruningManager resolves and manages the current pruning point
type PruningManager interface { type PruningManager interface {
UpdatePruningPointByVirtual() error UpdatePruningPointByVirtual(stagingArea *StagingArea) error
IsValidPruningPoint(blockHash *externalapi.DomainHash) (bool, error) IsValidPruningPoint(stagingArea *StagingArea, blockHash *externalapi.DomainHash) (bool, error)
ClearImportedPruningPointData() error ClearImportedPruningPointData() error
AppendImportedPruningPointUTXOs(outpointAndUTXOEntryPairs []*externalapi.OutpointAndUTXOEntryPair) error AppendImportedPruningPointUTXOs(outpointAndUTXOEntryPairs []*externalapi.OutpointAndUTXOEntryPair) error
UpdatePruningPointUTXOSetIfRequired() error UpdatePruningPointUTXOSetIfRequired() error
PruneAllBlocksBelow(pruningPointHash *externalapi.DomainHash) error PruneAllBlocksBelow(stagingArea *StagingArea, pruningPointHash *externalapi.DomainHash) error
} }

View File

@ -5,9 +5,9 @@ import "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
// ReachabilityManager maintains a structure that allows to answer // ReachabilityManager maintains a structure that allows to answer
// reachability queries in sub-linear time // reachability queries in sub-linear time
type ReachabilityManager interface { type ReachabilityManager interface {
AddBlock(blockHash *externalapi.DomainHash) error AddBlock(stagingArea *StagingArea, blockHash *externalapi.DomainHash) error
IsReachabilityTreeAncestorOf(blockHashA *externalapi.DomainHash, blockHashB *externalapi.DomainHash) (bool, error) IsReachabilityTreeAncestorOf(stagingArea *StagingArea, blockHashA *externalapi.DomainHash, blockHashB *externalapi.DomainHash) (bool, error)
IsDAGAncestorOf(blockHashA *externalapi.DomainHash, blockHashB *externalapi.DomainHash) (bool, error) IsDAGAncestorOf(stagingArea *StagingArea, blockHashA *externalapi.DomainHash, blockHashB *externalapi.DomainHash) (bool, error)
UpdateReindexRoot(selectedTip *externalapi.DomainHash) error UpdateReindexRoot(stagingArea *StagingArea, selectedTip *externalapi.DomainHash) error
FindNextAncestor(descendant, ancestor *externalapi.DomainHash) (*externalapi.DomainHash, error) FindNextAncestor(stagingArea *StagingArea, descendant, ancestor *externalapi.DomainHash) (*externalapi.DomainHash, error)
} }

View File

@ -4,10 +4,13 @@ import "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
// SyncManager exposes functions to support sync between kaspad nodes // SyncManager exposes functions to support sync between kaspad nodes
type SyncManager interface { type SyncManager interface {
GetHashesBetween(lowHash, highHash *externalapi.DomainHash, maxBlueScoreDifference uint64) ( GetHashesBetween(stagingArea *StagingArea, lowHash, highHash *externalapi.DomainHash, maxBlueScoreDifference uint64) (
hashes []*externalapi.DomainHash, actualHighHash *externalapi.DomainHash, err error) hashes []*externalapi.DomainHash, actualHighHash *externalapi.DomainHash, err error)
GetMissingBlockBodyHashes(highHash *externalapi.DomainHash) ([]*externalapi.DomainHash, error) GetMissingBlockBodyHashes(stagingArea *StagingArea, highHash *externalapi.DomainHash) (
CreateBlockLocator(lowHash, highHash *externalapi.DomainHash, limit uint32) (externalapi.BlockLocator, error) []*externalapi.DomainHash, error)
CreateHeadersSelectedChainBlockLocator(lowHash, highHash *externalapi.DomainHash) (externalapi.BlockLocator, error) CreateBlockLocator(stagingArea *StagingArea, lowHash, highHash *externalapi.DomainHash, limit uint32) (
GetSyncInfo() (*externalapi.SyncInfo, error) externalapi.BlockLocator, error)
CreateHeadersSelectedChainBlockLocator(stagingArea *StagingArea, lowHash, highHash *externalapi.DomainHash) (
externalapi.BlockLocator, error)
GetSyncInfo(stagingArea *StagingArea) (*externalapi.SyncInfo, error)
} }

View File

@ -8,6 +8,6 @@ import (
// it's possible to determine whether a transaction is valid // it's possible to determine whether a transaction is valid
type TransactionValidator interface { type TransactionValidator interface {
ValidateTransactionInIsolation(transaction *externalapi.DomainTransaction) error ValidateTransactionInIsolation(transaction *externalapi.DomainTransaction) error
ValidateTransactionInContextAndPopulateMassAndFee(tx *externalapi.DomainTransaction, ValidateTransactionInContextAndPopulateMassAndFee(stagingArea *StagingArea,
povBlockHash *externalapi.DomainHash, selectedParentMedianTime int64) error tx *externalapi.DomainTransaction, povBlockHash *externalapi.DomainHash, selectedParentMedianTime int64) error
} }

View File

@ -0,0 +1,87 @@
package model
import "github.com/pkg/errors"
// StagingShard is an interface that enables every store to have it's own Commit logic
// See StagingArea for more details
type StagingShard interface {
Commit(dbTx DBTransaction) error
}
// StagingShardID is used to identify each of the store's staging shards
type StagingShardID byte
// StagingShardID constants
const (
StagingShardIDAcceptanceData StagingShardID = iota
StagingShardIDBlockHeader
StagingShardIDBlockRelation
StagingShardIDBlockStatus
StagingShardIDBlock
StagingShardIDConsensusState
StagingShardIDDAABlocks
StagingShardIDFinality
StagingShardIDGHOSTDAG
StagingShardIDHeadersSelectedChain
StagingShardIDHeadersSelectedTip
StagingShardIDMultiset
StagingShardIDPruning
StagingShardIDReachabilityData
StagingShardIDUTXODiff
// Always leave StagingShardIDLen as the last constant
StagingShardIDLen
)
// StagingArea is single changeset inside the consensus database, similar to a transaction in a classic database.
// Each StagingArea consists of multiple StagingShards, one for each dataStore that has any changes within it.
// To enable maximum flexibility for all stores, each has to define it's own Commit method, and pass it to the
// StagingArea through the relevant StagingShard.
//
// When the StagingArea is being Committed, it goes over all it's shards, and commits those one-by-one.
// Since Commit happens in a DatabaseTransaction, a StagingArea is atomic.
type StagingArea struct {
// shards is deliberately an array and not a map, as an optimization - since it's being read a lot of time, and
// reads from maps are relatively slow.
shards [StagingShardIDLen]StagingShard
isCommitted bool
}
// NewStagingArea creates a new, empty staging area.
func NewStagingArea() *StagingArea {
return &StagingArea{
shards: [StagingShardIDLen]StagingShard{},
isCommitted: false,
}
}
// GetOrCreateShard attempts to retrieve a shard with the given name.
// If it does not exist - a new shard is created using `createFunc`.
func (sa *StagingArea) GetOrCreateShard(shardID StagingShardID, createFunc func() StagingShard) StagingShard {
if sa.shards[shardID] == nil {
sa.shards[shardID] = createFunc()
}
return sa.shards[shardID]
}
// Commit goes over all the Shards in the StagingArea and commits them, inside the provided database transaction.
// Note: the transaction itself is not committed, this is the callers responsibility to commit it.
func (sa *StagingArea) Commit(dbTx DBTransaction) error {
if sa.isCommitted {
return errors.New("Attempt to call Commit on already committed stagingArea")
}
for _, shard := range sa.shards {
if shard == nil { // since sa.shards is an array and not a map, some shard slots might be empty.
continue
}
err := shard.Commit(dbTx)
if err != nil {
return err
}
}
sa.isCommitted = true
return nil
}

View File

@ -2,6 +2,4 @@ package model
// Store is a common interface for data stores // Store is a common interface for data stores
type Store interface { type Store interface {
Discard()
Commit(dbTx DBTransaction) error
} }

View File

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

View File

@ -10,5 +10,5 @@ type TestConsensusStateManager interface {
model.ConsensusStateManager model.ConsensusStateManager
AddUTXOToMultiset(multiset model.Multiset, entry externalapi.UTXOEntry, AddUTXOToMultiset(multiset model.Multiset, entry externalapi.UTXOEntry,
outpoint *externalapi.DomainOutpoint) error outpoint *externalapi.DomainOutpoint) error
ResolveBlockStatus(blockHash *externalapi.DomainHash) (externalapi.BlockStatus, error) ResolveBlockStatus(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) (externalapi.BlockStatus, error)
} }

View File

@ -1,10 +1,11 @@
package blockbuilder package blockbuilder
import ( import (
"sort"
"github.com/kaspanet/kaspad/domain/consensus/ruleerrors" "github.com/kaspanet/kaspad/domain/consensus/ruleerrors"
"github.com/kaspanet/kaspad/domain/consensus/utils/blockheader" "github.com/kaspanet/kaspad/domain/consensus/utils/blockheader"
"github.com/pkg/errors" "github.com/pkg/errors"
"sort"
"github.com/kaspanet/kaspad/domain/consensus/model" "github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi" "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
@ -72,24 +73,26 @@ func (bb *blockBuilder) BuildBlock(coinbaseData *externalapi.DomainCoinbaseData,
onEnd := logger.LogAndMeasureExecutionTime(log, "BuildBlock") onEnd := logger.LogAndMeasureExecutionTime(log, "BuildBlock")
defer onEnd() defer onEnd()
return bb.buildBlock(coinbaseData, transactions) stagingArea := model.NewStagingArea()
return bb.buildBlock(stagingArea, coinbaseData, transactions)
} }
func (bb *blockBuilder) buildBlock(coinbaseData *externalapi.DomainCoinbaseData, func (bb *blockBuilder) buildBlock(stagingArea *model.StagingArea, coinbaseData *externalapi.DomainCoinbaseData,
transactions []*externalapi.DomainTransaction) (*externalapi.DomainBlock, error) { transactions []*externalapi.DomainTransaction) (*externalapi.DomainBlock, error) {
err := bb.validateTransactions(transactions) err := bb.validateTransactions(stagingArea, transactions)
if err != nil { if err != nil {
return nil, err return nil, err
} }
coinbase, err := bb.newBlockCoinbaseTransaction(coinbaseData) coinbase, err := bb.newBlockCoinbaseTransaction(stagingArea, coinbaseData)
if err != nil { if err != nil {
return nil, err return nil, err
} }
transactionsWithCoinbase := append([]*externalapi.DomainTransaction{coinbase}, transactions...) transactionsWithCoinbase := append([]*externalapi.DomainTransaction{coinbase}, transactions...)
header, err := bb.buildHeader(transactionsWithCoinbase) header, err := bb.buildHeader(stagingArea, transactionsWithCoinbase)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -100,10 +103,12 @@ func (bb *blockBuilder) buildBlock(coinbaseData *externalapi.DomainCoinbaseData,
}, nil }, nil
} }
func (bb *blockBuilder) validateTransactions(transactions []*externalapi.DomainTransaction) error { func (bb *blockBuilder) validateTransactions(stagingArea *model.StagingArea,
transactions []*externalapi.DomainTransaction) error {
invalidTransactions := make([]ruleerrors.InvalidTransaction, 0) invalidTransactions := make([]ruleerrors.InvalidTransaction, 0)
for _, transaction := range transactions { for _, transaction := range transactions {
err := bb.validateTransaction(transaction) err := bb.validateTransaction(stagingArea, transaction)
if err != nil { if err != nil {
if !errors.As(err, &ruleerrors.RuleError{}) { if !errors.As(err, &ruleerrors.RuleError{}) {
return err return err
@ -120,7 +125,9 @@ func (bb *blockBuilder) validateTransactions(transactions []*externalapi.DomainT
return nil return nil
} }
func (bb *blockBuilder) validateTransaction(transaction *externalapi.DomainTransaction) error { func (bb *blockBuilder) validateTransaction(
stagingArea *model.StagingArea, transaction *externalapi.DomainTransaction) error {
originalEntries := make([]externalapi.UTXOEntry, len(transaction.Inputs)) originalEntries := make([]externalapi.UTXOEntry, len(transaction.Inputs))
for i, input := range transaction.Inputs { for i, input := range transaction.Inputs {
originalEntries[i] = input.UTXOEntry originalEntries[i] = input.UTXOEntry
@ -133,45 +140,47 @@ func (bb *blockBuilder) validateTransaction(transaction *externalapi.DomainTrans
} }
}() }()
err := bb.consensusStateManager.PopulateTransactionWithUTXOEntries(transaction) err := bb.consensusStateManager.PopulateTransactionWithUTXOEntries(stagingArea, transaction)
if err != nil { if err != nil {
return err return err
} }
virtualSelectedParentMedianTime, err := bb.pastMedianTimeManager.PastMedianTime(model.VirtualBlockHash) virtualSelectedParentMedianTime, err := bb.pastMedianTimeManager.PastMedianTime(stagingArea, model.VirtualBlockHash)
if err != nil { if err != nil {
return err return err
} }
return bb.transactionValidator.ValidateTransactionInContextAndPopulateMassAndFee(transaction, return bb.transactionValidator.ValidateTransactionInContextAndPopulateMassAndFee(
model.VirtualBlockHash, virtualSelectedParentMedianTime) stagingArea, transaction, model.VirtualBlockHash, virtualSelectedParentMedianTime)
} }
func (bb *blockBuilder) newBlockCoinbaseTransaction( func (bb *blockBuilder) newBlockCoinbaseTransaction(stagingArea *model.StagingArea,
coinbaseData *externalapi.DomainCoinbaseData) (*externalapi.DomainTransaction, error) { coinbaseData *externalapi.DomainCoinbaseData) (*externalapi.DomainTransaction, error) {
return bb.coinbaseManager.ExpectedCoinbaseTransaction(model.VirtualBlockHash, coinbaseData) return bb.coinbaseManager.ExpectedCoinbaseTransaction(stagingArea, model.VirtualBlockHash, coinbaseData)
} }
func (bb *blockBuilder) buildHeader(transactions []*externalapi.DomainTransaction) (externalapi.BlockHeader, error) { func (bb *blockBuilder) buildHeader(stagingArea *model.StagingArea, transactions []*externalapi.DomainTransaction) (
parentHashes, err := bb.newBlockParentHashes() externalapi.BlockHeader, error) {
parentHashes, err := bb.newBlockParentHashes(stagingArea)
if err != nil { if err != nil {
return nil, err return nil, err
} }
timeInMilliseconds, err := bb.newBlockTime() timeInMilliseconds, err := bb.newBlockTime(stagingArea)
if err != nil { if err != nil {
return nil, err return nil, err
} }
bits, err := bb.newBlockDifficulty() bits, err := bb.newBlockDifficulty(stagingArea)
if err != nil { if err != nil {
return nil, err return nil, err
} }
hashMerkleRoot := bb.newBlockHashMerkleRoot(transactions) hashMerkleRoot := bb.newBlockHashMerkleRoot(transactions)
acceptedIDMerkleRoot, err := bb.newBlockAcceptedIDMerkleRoot() acceptedIDMerkleRoot, err := bb.newBlockAcceptedIDMerkleRoot(stagingArea)
if err != nil { if err != nil {
return nil, err return nil, err
} }
utxoCommitment, err := bb.newBlockUTXOCommitment() utxoCommitment, err := bb.newBlockUTXOCommitment(stagingArea)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -188,8 +197,8 @@ func (bb *blockBuilder) buildHeader(transactions []*externalapi.DomainTransactio
), nil ), nil
} }
func (bb *blockBuilder) newBlockParentHashes() ([]*externalapi.DomainHash, error) { func (bb *blockBuilder) newBlockParentHashes(stagingArea *model.StagingArea) ([]*externalapi.DomainHash, error) {
virtualBlockRelations, err := bb.blockRelationStore.BlockRelation(bb.databaseContext, model.VirtualBlockHash) virtualBlockRelations, err := bb.blockRelationStore.BlockRelation(bb.databaseContext, stagingArea, model.VirtualBlockHash)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -197,7 +206,7 @@ func (bb *blockBuilder) newBlockParentHashes() ([]*externalapi.DomainHash, error
return virtualBlockRelations.Parents, nil return virtualBlockRelations.Parents, nil
} }
func (bb *blockBuilder) newBlockTime() (int64, error) { func (bb *blockBuilder) newBlockTime(stagingArea *model.StagingArea) (int64, error) {
// The timestamp for the block must not be before the median timestamp // The timestamp for the block must not be before the median timestamp
// of the last several blocks. Thus, choose the maximum between the // of the last several blocks. Thus, choose the maximum between the
// current time and one second after the past median time. The current // current time and one second after the past median time. The current
@ -205,7 +214,7 @@ func (bb *blockBuilder) newBlockTime() (int64, error) {
// block timestamp does not supported a precision greater than one // block timestamp does not supported a precision greater than one
// millisecond. // millisecond.
newTimestamp := mstime.Now().UnixMilliseconds() newTimestamp := mstime.Now().UnixMilliseconds()
minTimestamp, err := bb.minBlockTime(model.VirtualBlockHash) minTimestamp, err := bb.minBlockTime(stagingArea, model.VirtualBlockHash)
if err != nil { if err != nil {
return 0, err return 0, err
} }
@ -215,8 +224,8 @@ func (bb *blockBuilder) newBlockTime() (int64, error) {
return newTimestamp, nil return newTimestamp, nil
} }
func (bb *blockBuilder) minBlockTime(hash *externalapi.DomainHash) (int64, error) { func (bb *blockBuilder) minBlockTime(stagingArea *model.StagingArea, hash *externalapi.DomainHash) (int64, error) {
pastMedianTime, err := bb.pastMedianTimeManager.PastMedianTime(hash) pastMedianTime, err := bb.pastMedianTimeManager.PastMedianTime(stagingArea, hash)
if err != nil { if err != nil {
return 0, err return 0, err
} }
@ -224,16 +233,16 @@ func (bb *blockBuilder) minBlockTime(hash *externalapi.DomainHash) (int64, error
return pastMedianTime + 1, nil return pastMedianTime + 1, nil
} }
func (bb *blockBuilder) newBlockDifficulty() (uint32, error) { func (bb *blockBuilder) newBlockDifficulty(stagingArea *model.StagingArea) (uint32, error) {
return bb.difficultyManager.RequiredDifficulty(model.VirtualBlockHash) return bb.difficultyManager.RequiredDifficulty(stagingArea, model.VirtualBlockHash)
} }
func (bb *blockBuilder) newBlockHashMerkleRoot(transactions []*externalapi.DomainTransaction) *externalapi.DomainHash { func (bb *blockBuilder) newBlockHashMerkleRoot(transactions []*externalapi.DomainTransaction) *externalapi.DomainHash {
return merkle.CalculateHashMerkleRoot(transactions) return merkle.CalculateHashMerkleRoot(transactions)
} }
func (bb *blockBuilder) newBlockAcceptedIDMerkleRoot() (*externalapi.DomainHash, error) { func (bb *blockBuilder) newBlockAcceptedIDMerkleRoot(stagingArea *model.StagingArea) (*externalapi.DomainHash, error) {
newBlockAcceptanceData, err := bb.acceptanceDataStore.Get(bb.databaseContext, model.VirtualBlockHash) newBlockAcceptanceData, err := bb.acceptanceDataStore.Get(bb.databaseContext, stagingArea, model.VirtualBlockHash)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -260,8 +269,8 @@ func (bb *blockBuilder) calculateAcceptedIDMerkleRoot(acceptanceData externalapi
return merkle.CalculateIDMerkleRoot(acceptedTransactions), nil return merkle.CalculateIDMerkleRoot(acceptedTransactions), nil
} }
func (bb *blockBuilder) newBlockUTXOCommitment() (*externalapi.DomainHash, error) { func (bb *blockBuilder) newBlockUTXOCommitment(stagingArea *model.StagingArea) (*externalapi.DomainHash, error) {
newBlockMultiset, err := bb.multisetStore.Get(bb.databaseContext, model.VirtualBlockHash) newBlockMultiset, err := bb.multisetStore.Get(bb.databaseContext, stagingArea, model.VirtualBlockHash)
if err != nil { if err != nil {
return nil, err return nil, err
} }

View File

@ -49,7 +49,9 @@ func (bb *testBlockBuilder) BuildBlockWithParents(parentHashes []*externalapi.Do
onEnd := logger.LogAndMeasureExecutionTime(log, "BuildBlockWithParents") onEnd := logger.LogAndMeasureExecutionTime(log, "BuildBlockWithParents")
defer onEnd() defer onEnd()
block, diff, err := bb.buildBlockWithParents(parentHashes, coinbaseData, transactions) stagingArea := model.NewStagingArea()
block, diff, err := bb.buildBlockWithParents(stagingArea, parentHashes, coinbaseData, transactions)
if err != nil { if err != nil {
return nil, nil, err return nil, nil, err
} }
@ -61,10 +63,10 @@ func (bb *testBlockBuilder) BuildBlockWithParents(parentHashes []*externalapi.Do
return block, diff, nil return block, diff, nil
} }
func (bb *testBlockBuilder) buildUTXOInvalidHeader(parentHashes []*externalapi.DomainHash, func (bb *testBlockBuilder) buildUTXOInvalidHeader(stagingArea *model.StagingArea, parentHashes []*externalapi.DomainHash,
bits uint32, transactions []*externalapi.DomainTransaction) (externalapi.BlockHeader, error) { bits uint32, transactions []*externalapi.DomainTransaction) (externalapi.BlockHeader, error) {
timeInMilliseconds, err := bb.minBlockTime(tempBlockHash) timeInMilliseconds, err := bb.minBlockTime(stagingArea, tempBlockHash)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -84,11 +86,11 @@ func (bb *testBlockBuilder) buildUTXOInvalidHeader(parentHashes []*externalapi.D
), nil ), nil
} }
func (bb *testBlockBuilder) buildHeaderWithParents(parentHashes []*externalapi.DomainHash, bits uint32, func (bb *testBlockBuilder) buildHeaderWithParents(stagingArea *model.StagingArea, parentHashes []*externalapi.DomainHash,
transactions []*externalapi.DomainTransaction, acceptanceData externalapi.AcceptanceData, multiset model.Multiset) ( bits uint32, transactions []*externalapi.DomainTransaction, acceptanceData externalapi.AcceptanceData, multiset model.Multiset) (
externalapi.BlockHeader, error) { externalapi.BlockHeader, error) {
header, err := bb.buildUTXOInvalidHeader(parentHashes, bits, transactions) header, err := bb.buildUTXOInvalidHeader(stagingArea, parentHashes, bits, transactions)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -112,9 +114,9 @@ func (bb *testBlockBuilder) buildHeaderWithParents(parentHashes []*externalapi.D
), nil ), nil
} }
func (bb *testBlockBuilder) buildBlockWithParents(parentHashes []*externalapi.DomainHash, coinbaseData *externalapi.DomainCoinbaseData, transactions []*externalapi.DomainTransaction) (*externalapi.DomainBlock, externalapi.UTXODiff, error) { func (bb *testBlockBuilder) buildBlockWithParents(stagingArea *model.StagingArea, parentHashes []*externalapi.DomainHash,
coinbaseData *externalapi.DomainCoinbaseData, transactions []*externalapi.DomainTransaction) (
defer bb.testConsensus.DiscardAllStores() *externalapi.DomainBlock, externalapi.UTXODiff, error) {
if coinbaseData == nil { if coinbaseData == nil {
scriptPublicKey, _ := testutils.OpTrueScript() scriptPublicKey, _ := testutils.OpTrueScript()
@ -124,24 +126,24 @@ func (bb *testBlockBuilder) buildBlockWithParents(parentHashes []*externalapi.Do
} }
} }
bb.blockRelationStore.StageBlockRelation(tempBlockHash, &model.BlockRelations{Parents: parentHashes}) bb.blockRelationStore.StageBlockRelation(stagingArea, tempBlockHash, &model.BlockRelations{Parents: parentHashes})
err := bb.ghostdagManager.GHOSTDAG(tempBlockHash) err := bb.ghostdagManager.GHOSTDAG(stagingArea, tempBlockHash)
if err != nil { if err != nil {
return nil, nil, err return nil, nil, err
} }
bits, err := bb.difficultyManager.StageDAADataAndReturnRequiredDifficulty(tempBlockHash) bits, err := bb.difficultyManager.StageDAADataAndReturnRequiredDifficulty(stagingArea, tempBlockHash)
if err != nil { if err != nil {
return nil, nil, err return nil, nil, err
} }
ghostdagData, err := bb.ghostdagDataStore.Get(bb.databaseContext, tempBlockHash) ghostdagData, err := bb.ghostdagDataStore.Get(bb.databaseContext, stagingArea, tempBlockHash)
if err != nil { if err != nil {
return nil, nil, err return nil, nil, err
} }
selectedParentStatus, err := bb.testConsensus.ConsensusStateManager().ResolveBlockStatus(ghostdagData.SelectedParent()) selectedParentStatus, err := bb.testConsensus.ConsensusStateManager().ResolveBlockStatus(stagingArea, ghostdagData.SelectedParent())
if err != nil { if err != nil {
return nil, nil, err return nil, nil, err
} }
@ -150,20 +152,22 @@ func (bb *testBlockBuilder) buildBlockWithParents(parentHashes []*externalapi.Do
ghostdagData.SelectedParent()) ghostdagData.SelectedParent())
} }
pastUTXO, acceptanceData, multiset, err := bb.consensusStateManager.CalculatePastUTXOAndAcceptanceData(tempBlockHash) pastUTXO, acceptanceData, multiset, err :=
bb.consensusStateManager.CalculatePastUTXOAndAcceptanceData(stagingArea, tempBlockHash)
if err != nil { if err != nil {
return nil, nil, err return nil, nil, err
} }
bb.acceptanceDataStore.Stage(tempBlockHash, acceptanceData) bb.acceptanceDataStore.Stage(stagingArea, tempBlockHash, acceptanceData)
coinbase, err := bb.coinbaseManager.ExpectedCoinbaseTransaction(tempBlockHash, coinbaseData) coinbase, err := bb.coinbaseManager.ExpectedCoinbaseTransaction(stagingArea, tempBlockHash, coinbaseData)
if err != nil { if err != nil {
return nil, nil, err return nil, nil, err
} }
transactionsWithCoinbase := append([]*externalapi.DomainTransaction{coinbase}, transactions...) transactionsWithCoinbase := append([]*externalapi.DomainTransaction{coinbase}, transactions...)
header, err := bb.buildHeaderWithParents(parentHashes, bits, transactionsWithCoinbase, acceptanceData, multiset) header, err := bb.buildHeaderWithParents(
stagingArea, parentHashes, bits, transactionsWithCoinbase, acceptanceData, multiset)
if err != nil { if err != nil {
return nil, nil, err return nil, nil, err
} }
@ -188,11 +192,11 @@ func (bb *testBlockBuilder) BuildUTXOInvalidHeader(parentHashes []*externalapi.D
func (bb *testBlockBuilder) BuildUTXOInvalidBlock(parentHashes []*externalapi.DomainHash) (*externalapi.DomainBlock, func (bb *testBlockBuilder) BuildUTXOInvalidBlock(parentHashes []*externalapi.DomainHash) (*externalapi.DomainBlock,
error) { error) {
defer bb.testConsensus.DiscardAllStores() stagingArea := model.NewStagingArea()
bb.blockRelationStore.StageBlockRelation(tempBlockHash, &model.BlockRelations{Parents: parentHashes}) bb.blockRelationStore.StageBlockRelation(stagingArea, tempBlockHash, &model.BlockRelations{Parents: parentHashes})
err := bb.ghostdagManager.GHOSTDAG(tempBlockHash) err := bb.ghostdagManager.GHOSTDAG(stagingArea, tempBlockHash)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -200,12 +204,12 @@ func (bb *testBlockBuilder) BuildUTXOInvalidBlock(parentHashes []*externalapi.Do
// We use genesis transactions so we'll have something to build merkle root and coinbase with // We use genesis transactions so we'll have something to build merkle root and coinbase with
genesisTransactions := bb.testConsensus.DAGParams().GenesisBlock.Transactions genesisTransactions := bb.testConsensus.DAGParams().GenesisBlock.Transactions
bits, err := bb.difficultyManager.RequiredDifficulty(tempBlockHash) bits, err := bb.difficultyManager.RequiredDifficulty(stagingArea, tempBlockHash)
if err != nil { if err != nil {
return nil, err return nil, err
} }
header, err := bb.buildUTXOInvalidHeader(parentHashes, bits, genesisTransactions) header, err := bb.buildUTXOInvalidHeader(stagingArea, parentHashes, bits, genesisTransactions)
if err != nil { if err != nil {
return nil, err return nil, err
} }

View File

@ -1,11 +1,12 @@
package blockprocessor package blockprocessor
import ( import (
"time"
"github.com/kaspanet/kaspad/domain/consensus/model" "github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi" "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/kaspanet/kaspad/domain/consensus/processes/blockprocessor/blocklogger" "github.com/kaspanet/kaspad/domain/consensus/processes/blockprocessor/blocklogger"
"github.com/kaspanet/kaspad/infrastructure/logger" "github.com/kaspanet/kaspad/infrastructure/logger"
"time"
) )
// blockProcessor is responsible for processing incoming blocks // blockProcessor is responsible for processing incoming blocks
@ -142,12 +143,14 @@ func (bp *blockProcessor) ValidateAndInsertBlock(block *externalapi.DomainBlock)
onEnd := logger.LogAndMeasureExecutionTime(log, "ValidateAndInsertBlock") onEnd := logger.LogAndMeasureExecutionTime(log, "ValidateAndInsertBlock")
defer onEnd() defer onEnd()
return bp.validateAndInsertBlock(block, false) stagingArea := model.NewStagingArea()
return bp.validateAndInsertBlock(stagingArea, block, false)
} }
func (bp *blockProcessor) ValidateAndInsertImportedPruningPoint(newPruningPoint *externalapi.DomainBlock) error { func (bp *blockProcessor) ValidateAndInsertImportedPruningPoint(newPruningPoint *externalapi.DomainBlock) error {
onEnd := logger.LogAndMeasureExecutionTime(log, "ValidateAndInsertImportedPruningPoint") onEnd := logger.LogAndMeasureExecutionTime(log, "ValidateAndInsertImportedPruningPoint")
defer onEnd() defer onEnd()
return bp.validateAndInsertImportedPruningPoint(newPruningPoint) stagingArea := model.NewStagingArea()
return bp.validateAndInsertImportedPruningPoint(stagingArea, newPruningPoint)
} }

View File

@ -2,6 +2,7 @@ package blockprocessor
import ( import (
"fmt" "fmt"
"github.com/kaspanet/kaspad/util/difficulty" "github.com/kaspanet/kaspad/util/difficulty"
"github.com/kaspanet/kaspad/domain/consensus/model" "github.com/kaspanet/kaspad/domain/consensus/model"
@ -12,15 +13,17 @@ import (
"github.com/pkg/errors" "github.com/pkg/errors"
) )
func (bp *blockProcessor) setBlockStatusAfterBlockValidation(block *externalapi.DomainBlock, isPruningPoint bool) error { func (bp *blockProcessor) setBlockStatusAfterBlockValidation(
stagingArea *model.StagingArea, block *externalapi.DomainBlock, isPruningPoint bool) error {
blockHash := consensushashing.BlockHash(block) blockHash := consensushashing.BlockHash(block)
exists, err := bp.blockStatusStore.Exists(bp.databaseContext, blockHash) exists, err := bp.blockStatusStore.Exists(bp.databaseContext, stagingArea, blockHash)
if err != nil { if err != nil {
return err return err
} }
if exists { if exists {
status, err := bp.blockStatusStore.Get(bp.databaseContext, blockHash) status, err := bp.blockStatusStore.Get(bp.databaseContext, stagingArea, blockHash)
if err != nil { if err != nil {
return err return err
} }
@ -44,25 +47,26 @@ func (bp *blockProcessor) setBlockStatusAfterBlockValidation(block *externalapi.
if isHeaderOnlyBlock { if isHeaderOnlyBlock {
log.Debugf("Block %s is a header-only block so setting its status as %s", log.Debugf("Block %s is a header-only block so setting its status as %s",
blockHash, externalapi.StatusHeaderOnly) blockHash, externalapi.StatusHeaderOnly)
bp.blockStatusStore.Stage(blockHash, externalapi.StatusHeaderOnly) bp.blockStatusStore.Stage(stagingArea, blockHash, externalapi.StatusHeaderOnly)
} else { } else {
log.Debugf("Block %s has body so setting its status as %s", log.Debugf("Block %s has body so setting its status as %s",
blockHash, externalapi.StatusUTXOPendingVerification) blockHash, externalapi.StatusUTXOPendingVerification)
bp.blockStatusStore.Stage(blockHash, externalapi.StatusUTXOPendingVerification) bp.blockStatusStore.Stage(stagingArea, blockHash, externalapi.StatusUTXOPendingVerification)
} }
return nil return nil
} }
func (bp *blockProcessor) validateAndInsertBlock(block *externalapi.DomainBlock, isPruningPoint bool) (*externalapi.BlockInsertionResult, error) { func (bp *blockProcessor) validateAndInsertBlock(stagingArea *model.StagingArea, block *externalapi.DomainBlock,
isPruningPoint bool) (*externalapi.BlockInsertionResult, error) {
blockHash := consensushashing.HeaderHash(block.Header) blockHash := consensushashing.HeaderHash(block.Header)
err := bp.validateBlock(block, isPruningPoint) err := bp.validateBlock(stagingArea, block, isPruningPoint)
if err != nil { if err != nil {
bp.discardAllChanges()
return nil, err return nil, err
} }
err = bp.setBlockStatusAfterBlockValidation(block, isPruningPoint) err = bp.setBlockStatusAfterBlockValidation(stagingArea, block, isPruningPoint)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -71,13 +75,13 @@ func (bp *blockProcessor) validateAndInsertBlock(block *externalapi.DomainBlock,
isGenesis := blockHash.Equal(bp.genesisHash) isGenesis := blockHash.Equal(bp.genesisHash)
if !isGenesis { if !isGenesis {
var err error var err error
oldHeadersSelectedTip, err = bp.headersSelectedTipStore.HeadersSelectedTip(bp.databaseContext) oldHeadersSelectedTip, err = bp.headersSelectedTipStore.HeadersSelectedTip(bp.databaseContext, stagingArea)
if err != nil { if err != nil {
return nil, err return nil, err
} }
} }
err = bp.headerTipsManager.AddHeaderTip(blockHash) err = bp.headerTipsManager.AddHeaderTip(stagingArea, blockHash)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -91,7 +95,7 @@ func (bp *blockProcessor) validateAndInsertBlock(block *externalapi.DomainBlock,
// in consensusStateManager.ImportPruningPoint // in consensusStateManager.ImportPruningPoint
if !isPruningPoint { if !isPruningPoint {
// Attempt to add the block to the virtual // Attempt to add the block to the virtual
selectedParentChainChanges, virtualUTXODiff, err = bp.consensusStateManager.AddBlock(blockHash) selectedParentChainChanges, virtualUTXODiff, err = bp.consensusStateManager.AddBlock(stagingArea, blockHash)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -99,7 +103,7 @@ func (bp *blockProcessor) validateAndInsertBlock(block *externalapi.DomainBlock,
} }
if !isGenesis { if !isGenesis {
err := bp.updateReachabilityReindexRoot(oldHeadersSelectedTip) err := bp.updateReachabilityReindexRoot(stagingArea, oldHeadersSelectedTip)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -107,13 +111,13 @@ func (bp *blockProcessor) validateAndInsertBlock(block *externalapi.DomainBlock,
if !isHeaderOnlyBlock { if !isHeaderOnlyBlock {
// Trigger pruning, which will check if the pruning point changed and delete the data if it did. // Trigger pruning, which will check if the pruning point changed and delete the data if it did.
err = bp.pruningManager.UpdatePruningPointByVirtual() err = bp.pruningManager.UpdatePruningPointByVirtual(stagingArea)
if err != nil { if err != nil {
return nil, err return nil, err
} }
} }
err = bp.commitAllChanges() err = bp.commitAllChanges(stagingArea)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -130,13 +134,13 @@ func (bp *blockProcessor) validateAndInsertBlock(block *externalapi.DomainBlock,
var logClosureErr error var logClosureErr error
log.Debug(logger.NewLogClosure(func() string { log.Debug(logger.NewLogClosure(func() string {
virtualGhostDAGData, err := bp.ghostdagDataStore.Get(bp.databaseContext, model.VirtualBlockHash) virtualGhostDAGData, err := bp.ghostdagDataStore.Get(bp.databaseContext, stagingArea, model.VirtualBlockHash)
if err != nil { if err != nil {
logClosureErr = err logClosureErr = err
return fmt.Sprintf("Failed to get virtual GHOSTDAG data: %s", err) return fmt.Sprintf("Failed to get virtual GHOSTDAG data: %s", err)
} }
headerCount := bp.blockHeaderStore.Count() headerCount := bp.blockHeaderStore.Count(stagingArea)
blockCount := bp.blockStore.Count() blockCount := bp.blockStore.Count(stagingArea)
return fmt.Sprintf("New virtual's blue score: %d. Block count: %d. Header count: %d", return fmt.Sprintf("New virtual's blue score: %d. Block count: %d. Header count: %d",
virtualGhostDAGData.BlueScore(), blockCount, headerCount) virtualGhostDAGData.BlueScore(), blockCount, headerCount)
})) }))
@ -144,7 +148,7 @@ func (bp *blockProcessor) validateAndInsertBlock(block *externalapi.DomainBlock,
return nil, logClosureErr return nil, logClosureErr
} }
virtualParents, err := bp.dagTopologyManager.Parents(model.VirtualBlockHash) virtualParents, err := bp.dagTopologyManager.Parents(stagingArea, model.VirtualBlockHash)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -162,8 +166,10 @@ func isHeaderOnlyBlock(block *externalapi.DomainBlock) bool {
return len(block.Transactions) == 0 return len(block.Transactions) == 0
} }
func (bp *blockProcessor) updateReachabilityReindexRoot(oldHeadersSelectedTip *externalapi.DomainHash) error { func (bp *blockProcessor) updateReachabilityReindexRoot(stagingArea *model.StagingArea,
headersSelectedTip, err := bp.headersSelectedTipStore.HeadersSelectedTip(bp.databaseContext) oldHeadersSelectedTip *externalapi.DomainHash) error {
headersSelectedTip, err := bp.headersSelectedTipStore.HeadersSelectedTip(bp.databaseContext, stagingArea)
if err != nil { if err != nil {
return err return err
} }
@ -172,13 +178,13 @@ func (bp *blockProcessor) updateReachabilityReindexRoot(oldHeadersSelectedTip *e
return nil return nil
} }
return bp.reachabilityManager.UpdateReindexRoot(headersSelectedTip) return bp.reachabilityManager.UpdateReindexRoot(stagingArea, headersSelectedTip)
} }
func (bp *blockProcessor) checkBlockStatus(block *externalapi.DomainBlock) error { func (bp *blockProcessor) checkBlockStatus(stagingArea *model.StagingArea, block *externalapi.DomainBlock) error {
hash := consensushashing.BlockHash(block) hash := consensushashing.BlockHash(block)
isHeaderOnlyBlock := isHeaderOnlyBlock(block) isHeaderOnlyBlock := isHeaderOnlyBlock(block)
exists, err := bp.blockStatusStore.Exists(bp.databaseContext, hash) exists, err := bp.blockStatusStore.Exists(bp.databaseContext, stagingArea, hash)
if err != nil { if err != nil {
return err return err
} }
@ -186,7 +192,7 @@ func (bp *blockProcessor) checkBlockStatus(block *externalapi.DomainBlock) error
return nil return nil
} }
status, err := bp.blockStatusStore.Get(bp.databaseContext, hash) status, err := bp.blockStatusStore.Get(bp.databaseContext, stagingArea, hash)
if err != nil { if err != nil {
return err return err
} }
@ -196,7 +202,7 @@ func (bp *blockProcessor) checkBlockStatus(block *externalapi.DomainBlock) error
} }
if !isHeaderOnlyBlock { if !isHeaderOnlyBlock {
hasBlock, err := bp.blockStore.HasBlock(bp.databaseContext, hash) hasBlock, err := bp.blockStore.HasBlock(bp.databaseContext, stagingArea, hash)
if err != nil { if err != nil {
return err return err
} }
@ -204,7 +210,7 @@ func (bp *blockProcessor) checkBlockStatus(block *externalapi.DomainBlock) error
return errors.Wrapf(ruleerrors.ErrDuplicateBlock, "block %s already exists", hash) return errors.Wrapf(ruleerrors.ErrDuplicateBlock, "block %s already exists", hash)
} }
} else { } else {
hasHeader, err := bp.blockHeaderStore.HasBlockHeader(bp.databaseContext, hash) hasHeader, err := bp.blockHeaderStore.HasBlockHeader(bp.databaseContext, stagingArea, hash)
if err != nil { if err != nil {
return err return err
} }
@ -216,10 +222,10 @@ func (bp *blockProcessor) checkBlockStatus(block *externalapi.DomainBlock) error
return nil return nil
} }
func (bp *blockProcessor) validatePreProofOfWork(block *externalapi.DomainBlock) error { func (bp *blockProcessor) validatePreProofOfWork(stagingArea *model.StagingArea, block *externalapi.DomainBlock) error {
blockHash := consensushashing.BlockHash(block) blockHash := consensushashing.BlockHash(block)
hasValidatedHeader, err := bp.hasValidatedHeader(blockHash) hasValidatedHeader, err := bp.hasValidatedHeader(stagingArea, blockHash)
if err != nil { if err != nil {
return err return err
} }
@ -229,39 +235,39 @@ func (bp *blockProcessor) validatePreProofOfWork(block *externalapi.DomainBlock)
return nil return nil
} }
err = bp.blockValidator.ValidateHeaderInIsolation(blockHash) err = bp.blockValidator.ValidateHeaderInIsolation(stagingArea, blockHash)
if err != nil { if err != nil {
return err return err
} }
return nil return nil
} }
func (bp *blockProcessor) validatePostProofOfWork(block *externalapi.DomainBlock, isPruningPoint bool) error { func (bp *blockProcessor) validatePostProofOfWork(stagingArea *model.StagingArea, block *externalapi.DomainBlock, isPruningPoint bool) error {
blockHash := consensushashing.BlockHash(block) blockHash := consensushashing.BlockHash(block)
isHeaderOnlyBlock := isHeaderOnlyBlock(block) isHeaderOnlyBlock := isHeaderOnlyBlock(block)
if !isHeaderOnlyBlock { if !isHeaderOnlyBlock {
bp.blockStore.Stage(blockHash, block) bp.blockStore.Stage(stagingArea, blockHash, block)
err := bp.blockValidator.ValidateBodyInIsolation(blockHash) err := bp.blockValidator.ValidateBodyInIsolation(stagingArea, blockHash)
if err != nil { if err != nil {
return err return err
} }
} }
hasValidatedHeader, err := bp.hasValidatedHeader(blockHash) hasValidatedHeader, err := bp.hasValidatedHeader(stagingArea, blockHash)
if err != nil { if err != nil {
return err return err
} }
if !hasValidatedHeader { if !hasValidatedHeader {
err = bp.blockValidator.ValidateHeaderInContext(blockHash) err = bp.blockValidator.ValidateHeaderInContext(stagingArea, blockHash)
if err != nil { if err != nil {
return err return err
} }
} }
if !isHeaderOnlyBlock { if !isHeaderOnlyBlock {
err = bp.blockValidator.ValidateBodyInContext(blockHash, isPruningPoint) err = bp.blockValidator.ValidateBodyInContext(stagingArea, blockHash, isPruningPoint)
if err != nil { if err != nil {
return err return err
} }
@ -275,8 +281,8 @@ func (bp *blockProcessor) validatePostProofOfWork(block *externalapi.DomainBlock
// hasValidatedHeader returns whether the block header was validated. It returns // hasValidatedHeader returns whether the block header was validated. It returns
// true in any case the block header was validated, whether it was validated as a // true in any case the block header was validated, whether it was validated as a
// header-only block or as a block with body. // header-only block or as a block with body.
func (bp *blockProcessor) hasValidatedHeader(blockHash *externalapi.DomainHash) (bool, error) { func (bp *blockProcessor) hasValidatedHeader(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) (bool, error) {
exists, err := bp.blockStatusStore.Exists(bp.databaseContext, blockHash) exists, err := bp.blockStatusStore.Exists(bp.databaseContext, stagingArea, blockHash)
if err != nil { if err != nil {
return false, err return false, err
} }
@ -285,7 +291,7 @@ func (bp *blockProcessor) hasValidatedHeader(blockHash *externalapi.DomainHash)
return false, nil return false, nil
} }
status, err := bp.blockStatusStore.Get(bp.databaseContext, blockHash) status, err := bp.blockStatusStore.Get(bp.databaseContext, stagingArea, blockHash)
if err != nil { if err != nil {
return false, err return false, err
} }
@ -293,13 +299,7 @@ func (bp *blockProcessor) hasValidatedHeader(blockHash *externalapi.DomainHash)
return status != externalapi.StatusInvalid, nil return status != externalapi.StatusInvalid, nil
} }
func (bp *blockProcessor) discardAllChanges() { func (bp *blockProcessor) commitAllChanges(stagingArea *model.StagingArea) error {
for _, store := range bp.stores {
store.Discard()
}
}
func (bp *blockProcessor) commitAllChanges() error {
onEnd := logger.LogAndMeasureExecutionTime(log, "commitAllChanges") onEnd := logger.LogAndMeasureExecutionTime(log, "commitAllChanges")
defer onEnd() defer onEnd()
@ -308,12 +308,10 @@ func (bp *blockProcessor) commitAllChanges() error {
return err return err
} }
for _, store := range bp.stores { err = stagingArea.Commit(dbTx)
err = store.Commit(dbTx)
if err != nil { if err != nil {
return err return err
} }
}
return dbTx.Commit() return dbTx.Commit()
} }

View File

@ -1,11 +1,14 @@
package blockprocessor_test package blockprocessor_test
import ( import (
"github.com/kaspanet/kaspad/domain/consensus/utils/constants"
"github.com/kaspanet/kaspad/domain/consensus/utils/merkle"
"strings" "strings"
"testing" "testing"
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/utils/constants"
"github.com/kaspanet/kaspad/domain/consensus/utils/merkle"
"github.com/kaspanet/kaspad/domain/consensus" "github.com/kaspanet/kaspad/domain/consensus"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi" "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/kaspanet/kaspad/domain/consensus/ruleerrors" "github.com/kaspanet/kaspad/domain/consensus/ruleerrors"
@ -26,7 +29,7 @@ func TestBlockStatus(t *testing.T) {
defer teardown(false) defer teardown(false)
checkStatus := func(hash *externalapi.DomainHash, expectedStatus externalapi.BlockStatus) { checkStatus := func(hash *externalapi.DomainHash, expectedStatus externalapi.BlockStatus) {
blockStatus, err := tc.BlockStatusStore().Get(tc.DatabaseContext(), hash) blockStatus, err := tc.BlockStatusStore().Get(tc.DatabaseContext(), model.NewStagingArea(), hash)
if err != nil { if err != nil {
t.Fatalf("BlockStatusStore().Get: %+v", err) t.Fatalf("BlockStatusStore().Get: %+v", err)
} }

View File

@ -1,17 +1,20 @@
package blockprocessor package blockprocessor
import ( import (
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi" "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/kaspanet/kaspad/domain/consensus/ruleerrors" "github.com/kaspanet/kaspad/domain/consensus/ruleerrors"
"github.com/kaspanet/kaspad/domain/consensus/utils/consensushashing" "github.com/kaspanet/kaspad/domain/consensus/utils/consensushashing"
"github.com/pkg/errors" "github.com/pkg/errors"
) )
func (bp *blockProcessor) validateAndInsertImportedPruningPoint(newPruningPoint *externalapi.DomainBlock) error { func (bp *blockProcessor) validateAndInsertImportedPruningPoint(
stagingArea *model.StagingArea, newPruningPoint *externalapi.DomainBlock) error {
log.Info("Checking that the given pruning point is the expected pruning point") log.Info("Checking that the given pruning point is the expected pruning point")
newPruningPointHash := consensushashing.BlockHash(newPruningPoint) newPruningPointHash := consensushashing.BlockHash(newPruningPoint)
isValidPruningPoint, err := bp.pruningManager.IsValidPruningPoint(newPruningPointHash) isValidPruningPoint, err := bp.pruningManager.IsValidPruningPoint(stagingArea, newPruningPointHash)
if err != nil { if err != nil {
return err return err
} }
@ -29,19 +32,22 @@ func (bp *blockProcessor) validateAndInsertImportedPruningPoint(newPruningPoint
} }
log.Info("Deleting block data for all blocks in blockStore") log.Info("Deleting block data for all blocks in blockStore")
err = bp.pruningManager.PruneAllBlocksBelow(newPruningPointHash) err = bp.pruningManager.PruneAllBlocksBelow(stagingArea, newPruningPointHash)
if err != nil { if err != nil {
return err return err
} }
log.Infof("Updating consensus state manager according to the new pruning point %s", newPruningPointHash) log.Infof("Updating consensus state manager according to the new pruning point %s", newPruningPointHash)
err = bp.consensusStateManager.ImportPruningPoint(newPruningPoint) err = bp.consensusStateManager.ImportPruningPoint(stagingArea, newPruningPoint)
if err != nil { if err != nil {
return err return err
} }
// ImportPruningPoint commits the stagingArea, so create a new one
stagingArea = model.NewStagingArea()
log.Infof("Inserting the new pruning point %s", newPruningPointHash) log.Infof("Inserting the new pruning point %s", newPruningPointHash)
_, err = bp.validateAndInsertBlock(newPruningPoint, true) _, err = bp.validateAndInsertBlock(stagingArea, newPruningPoint, true)
if err != nil && errors.As(err, &ruleerrors.RuleError{}) { if err != nil && errors.As(err, &ruleerrors.RuleError{}) {
// This should never happen because we already validated the block with bp.validateBlockAndDiscardChanges. // This should never happen because we already validated the block with bp.validateBlockAndDiscardChanges.
// We use Errorf so it won't be identified later on to be a rule error and will eventually cause // We use Errorf so it won't be identified later on to be a rule error and will eventually cause

View File

@ -1,6 +1,11 @@
package blockprocessor_test package blockprocessor_test
import ( import (
"testing"
"time"
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus" "github.com/kaspanet/kaspad/domain/consensus"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi" "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/kaspanet/kaspad/domain/consensus/model/testapi" "github.com/kaspanet/kaspad/domain/consensus/model/testapi"
@ -12,8 +17,6 @@ import (
"github.com/kaspanet/kaspad/domain/consensus/utils/utxo" "github.com/kaspanet/kaspad/domain/consensus/utils/utxo"
"github.com/kaspanet/kaspad/domain/dagconfig" "github.com/kaspanet/kaspad/domain/dagconfig"
"github.com/pkg/errors" "github.com/pkg/errors"
"testing"
"time"
) )
func addBlock(tcSyncer, tcSyncee testapi.TestConsensus, parentHashes []*externalapi.DomainHash, t *testing.T) *externalapi.DomainHash { func addBlock(tcSyncer, tcSyncee testapi.TestConsensus, parentHashes []*externalapi.DomainHash, t *testing.T) *externalapi.DomainHash {
@ -524,7 +527,8 @@ func TestGetPruningPointUTXOs(t *testing.T) {
t.Fatalf("Error getting the pruning point: %+v", err) t.Fatalf("Error getting the pruning point: %+v", err)
} }
pruningRelations, err := testConsensus.BlockRelationStore().BlockRelation(testConsensus.DatabaseContext(), pruningPoint) pruningRelations, err := testConsensus.BlockRelationStore().BlockRelation(
testConsensus.DatabaseContext(), model.NewStagingArea(), pruningPoint)
if err != nil { if err != nil {
t.Fatalf("BlockRelation(): %+v", err) t.Fatalf("BlockRelation(): %+v", err)
} }

View File

@ -1,6 +1,7 @@
package blockprocessor package blockprocessor
import ( import (
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi" "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/kaspanet/kaspad/domain/consensus/ruleerrors" "github.com/kaspanet/kaspad/domain/consensus/ruleerrors"
"github.com/kaspanet/kaspad/domain/consensus/utils/consensushashing" "github.com/kaspanet/kaspad/domain/consensus/utils/consensushashing"
@ -8,27 +9,26 @@ import (
) )
func (bp *blockProcessor) validateBlockAndDiscardChanges(block *externalapi.DomainBlock, isPruningPoint bool) error { func (bp *blockProcessor) validateBlockAndDiscardChanges(block *externalapi.DomainBlock, isPruningPoint bool) error {
defer bp.discardAllChanges() return bp.validateBlock(model.NewStagingArea(), block, isPruningPoint)
return bp.validateBlock(block, isPruningPoint)
} }
func (bp *blockProcessor) validateBlock(block *externalapi.DomainBlock, isPruningPoint bool) error { func (bp *blockProcessor) validateBlock(stagingArea *model.StagingArea, block *externalapi.DomainBlock, isPruningPoint bool) error {
blockHash := consensushashing.HeaderHash(block.Header) blockHash := consensushashing.HeaderHash(block.Header)
log.Debugf("Validating block %s", blockHash) log.Debugf("Validating block %s", blockHash)
err := bp.checkBlockStatus(block) err := bp.checkBlockStatus(stagingArea, block)
if err != nil { if err != nil {
return err return err
} }
hasValidatedHeader, err := bp.hasValidatedHeader(blockHash) hasValidatedHeader, err := bp.hasValidatedHeader(stagingArea, blockHash)
if err != nil { if err != nil {
return err return err
} }
if !hasValidatedHeader { if !hasValidatedHeader {
log.Debugf("Staging block %s header", blockHash) log.Debugf("Staging block %s header", blockHash)
bp.blockHeaderStore.Stage(blockHash, block.Header) bp.blockHeaderStore.Stage(stagingArea, blockHash, block.Header)
} else { } else {
log.Debugf("Block %s header is already known, so no need to stage it", blockHash) log.Debugf("Block %s header is already known, so no need to stage it", blockHash)
} }
@ -36,13 +36,13 @@ func (bp *blockProcessor) validateBlock(block *externalapi.DomainBlock, isPrunin
// If any validation until (included) proof-of-work fails, simply // If any validation until (included) proof-of-work fails, simply
// return an error without writing anything in the database. // return an error without writing anything in the database.
// This is to prevent spamming attacks. // This is to prevent spamming attacks.
err = bp.validatePreProofOfWork(block) err = bp.validatePreProofOfWork(stagingArea, block)
if err != nil { if err != nil {
return err return err
} }
if !hasValidatedHeader { if !hasValidatedHeader {
err = bp.blockValidator.ValidatePruningPointViolationAndProofOfWorkAndDifficulty(blockHash) err = bp.blockValidator.ValidatePruningPointViolationAndProofOfWorkAndDifficulty(stagingArea, blockHash)
if err != nil { if err != nil {
return err return err
} }
@ -50,7 +50,7 @@ func (bp *blockProcessor) validateBlock(block *externalapi.DomainBlock, isPrunin
// If in-context validations fail, discard all changes and store the // If in-context validations fail, discard all changes and store the
// block with StatusInvalid. // block with StatusInvalid.
err = bp.validatePostProofOfWork(block, isPruningPoint) err = bp.validatePostProofOfWork(stagingArea, block, isPruningPoint)
if err != nil { if err != nil {
if errors.As(err, &ruleerrors.RuleError{}) { if errors.As(err, &ruleerrors.RuleError{}) {
// We mark invalid blocks with status externalapi.StatusInvalid except in the // We mark invalid blocks with status externalapi.StatusInvalid except in the
@ -66,11 +66,11 @@ func (bp *blockProcessor) validateBlock(block *externalapi.DomainBlock, isPrunin
if !errors.As(err, &ruleerrors.ErrMissingParents{}) && if !errors.As(err, &ruleerrors.ErrMissingParents{}) &&
!errors.Is(err, ruleerrors.ErrBadMerkleRoot) && !errors.Is(err, ruleerrors.ErrBadMerkleRoot) &&
!errors.Is(err, ruleerrors.ErrPrunedBlock) { !errors.Is(err, ruleerrors.ErrPrunedBlock) {
// Discard all changes so we save only the block status // Use a new stagingArea so we save only the block status
bp.discardAllChanges() stagingArea := model.NewStagingArea()
hash := consensushashing.BlockHash(block) hash := consensushashing.BlockHash(block)
bp.blockStatusStore.Stage(hash, externalapi.StatusInvalid) bp.blockStatusStore.Stage(stagingArea, hash, externalapi.StatusInvalid)
commitErr := bp.commitAllChanges() commitErr := bp.commitAllChanges(stagingArea)
if commitErr != nil { if commitErr != nil {
return commitErr return commitErr
} }

View File

@ -1,9 +1,11 @@
package blockvalidator package blockvalidator
import ( import (
"github.com/kaspanet/kaspad/infrastructure/logger"
"math" "math"
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/infrastructure/logger"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi" "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/kaspanet/kaspad/domain/consensus/ruleerrors" "github.com/kaspanet/kaspad/domain/consensus/ruleerrors"
"github.com/kaspanet/kaspad/domain/consensus/utils/consensushashing" "github.com/kaspanet/kaspad/domain/consensus/utils/consensushashing"
@ -13,22 +15,22 @@ import (
// ValidateBodyInContext validates block bodies in the context of the current // ValidateBodyInContext validates block bodies in the context of the current
// consensus state // consensus state
func (v *blockValidator) ValidateBodyInContext(blockHash *externalapi.DomainHash, isPruningPoint bool) error { func (v *blockValidator) ValidateBodyInContext(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash, isPruningPoint bool) error {
onEnd := logger.LogAndMeasureExecutionTime(log, "ValidateBodyInContext") onEnd := logger.LogAndMeasureExecutionTime(log, "ValidateBodyInContext")
defer onEnd() defer onEnd()
err := v.checkBlockIsNotPruned(blockHash) err := v.checkBlockIsNotPruned(stagingArea, blockHash)
if err != nil { if err != nil {
return err return err
} }
err = v.checkBlockTransactionsFinalized(blockHash) err = v.checkBlockTransactionsFinalized(stagingArea, blockHash)
if err != nil { if err != nil {
return err return err
} }
if !isPruningPoint { if !isPruningPoint {
err := v.checkParentBlockBodiesExist(blockHash) err := v.checkParentBlockBodiesExist(stagingArea, blockHash)
if err != nil { if err != nil {
return err return err
} }
@ -37,8 +39,8 @@ func (v *blockValidator) ValidateBodyInContext(blockHash *externalapi.DomainHash
} }
// checkBlockIsNotPruned Checks we don't add block bodies to pruned blocks // checkBlockIsNotPruned Checks we don't add block bodies to pruned blocks
func (v *blockValidator) checkBlockIsNotPruned(blockHash *externalapi.DomainHash) error { func (v *blockValidator) checkBlockIsNotPruned(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) error {
hasValidatedHeader, err := v.hasValidatedHeader(blockHash) hasValidatedHeader, err := v.hasValidatedHeader(stagingArea, blockHash)
if err != nil { if err != nil {
return err return err
} }
@ -49,12 +51,12 @@ func (v *blockValidator) checkBlockIsNotPruned(blockHash *externalapi.DomainHash
return nil return nil
} }
tips, err := v.consensusStateStore.Tips(v.databaseContext) tips, err := v.consensusStateStore.Tips(stagingArea, v.databaseContext)
if err != nil { if err != nil {
return err return err
} }
isAncestorOfSomeTips, err := v.dagTopologyManager.IsAncestorOfAny(blockHash, tips) isAncestorOfSomeTips, err := v.dagTopologyManager.IsAncestorOfAny(stagingArea, blockHash, tips)
if err != nil { if err != nil {
return err return err
} }
@ -67,25 +69,27 @@ func (v *blockValidator) checkBlockIsNotPruned(blockHash *externalapi.DomainHash
return nil return nil
} }
func (v *blockValidator) checkParentBlockBodiesExist(blockHash *externalapi.DomainHash) error { func (v *blockValidator) checkParentBlockBodiesExist(
stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) error {
missingParentHashes := []*externalapi.DomainHash{} missingParentHashes := []*externalapi.DomainHash{}
header, err := v.blockHeaderStore.BlockHeader(v.databaseContext, blockHash) header, err := v.blockHeaderStore.BlockHeader(v.databaseContext, stagingArea, blockHash)
if err != nil { if err != nil {
return err return err
} }
for _, parent := range header.ParentHashes() { for _, parent := range header.ParentHashes() {
hasBlock, err := v.blockStore.HasBlock(v.databaseContext, parent) hasBlock, err := v.blockStore.HasBlock(v.databaseContext, stagingArea, parent)
if err != nil { if err != nil {
return err return err
} }
if !hasBlock { if !hasBlock {
pruningPoint, err := v.pruningStore.PruningPoint(v.databaseContext) pruningPoint, err := v.pruningStore.PruningPoint(v.databaseContext, stagingArea)
if err != nil { if err != nil {
return err return err
} }
isInPastOfPruningPoint, err := v.dagTopologyManager.IsAncestorOf(parent, pruningPoint) isInPastOfPruningPoint, err := v.dagTopologyManager.IsAncestorOf(stagingArea, parent, pruningPoint)
if err != nil { if err != nil {
return err return err
} }
@ -114,18 +118,20 @@ func (v *blockValidator) checkParentBlockBodiesExist(blockHash *externalapi.Doma
return nil return nil
} }
func (v *blockValidator) checkBlockTransactionsFinalized(blockHash *externalapi.DomainHash) error { func (v *blockValidator) checkBlockTransactionsFinalized(
block, err := v.blockStore.Block(v.databaseContext, blockHash) stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) error {
block, err := v.blockStore.Block(v.databaseContext, stagingArea, blockHash)
if err != nil { if err != nil {
return err return err
} }
ghostdagData, err := v.ghostdagDataStore.Get(v.databaseContext, blockHash) ghostdagData, err := v.ghostdagDataStore.Get(v.databaseContext, stagingArea, blockHash)
if err != nil { if err != nil {
return err return err
} }
blockTime, err := v.pastMedianTimeManager.PastMedianTime(blockHash) blockTime, err := v.pastMedianTimeManager.PastMedianTime(stagingArea, blockHash)
if err != nil { if err != nil {
return err return err
} }

View File

@ -1,6 +1,10 @@
package blockvalidator_test package blockvalidator_test
import ( import (
"testing"
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus" "github.com/kaspanet/kaspad/domain/consensus"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi" "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/kaspanet/kaspad/domain/consensus/ruleerrors" "github.com/kaspanet/kaspad/domain/consensus/ruleerrors"
@ -9,7 +13,6 @@ import (
"github.com/kaspanet/kaspad/domain/consensus/utils/testutils" "github.com/kaspanet/kaspad/domain/consensus/utils/testutils"
"github.com/kaspanet/kaspad/domain/dagconfig" "github.com/kaspanet/kaspad/domain/dagconfig"
"github.com/pkg/errors" "github.com/pkg/errors"
"testing"
) )
func TestCheckBlockIsNotPruned(t *testing.T) { func TestCheckBlockIsNotPruned(t *testing.T) {
@ -59,7 +62,7 @@ func TestCheckBlockIsNotPruned(t *testing.T) {
t.Fatalf("Unexpected error: %+v", err) t.Fatalf("Unexpected error: %+v", err)
} }
beforePruningBlockBlockStatus, err := tc.BlockStatusStore().Get(tc.DatabaseContext(), beforePruningBlockBlockStatus, err := tc.BlockStatusStore().Get(tc.DatabaseContext(), model.NewStagingArea(),
consensushashing.BlockHash(beforePruningBlock)) consensushashing.BlockHash(beforePruningBlock))
if err != nil { if err != nil {
t.Fatalf("BlockStatusStore().Get: %+v", err) t.Fatalf("BlockStatusStore().Get: %+v", err)
@ -144,6 +147,8 @@ func TestCheckParentBlockBodiesExist(t *testing.T) {
func TestIsFinalizedTransaction(t *testing.T) { func TestIsFinalizedTransaction(t *testing.T) {
testutils.ForAllNets(t, true, func(t *testing.T, params *dagconfig.Params) { testutils.ForAllNets(t, true, func(t *testing.T, params *dagconfig.Params) {
stagingArea := model.NewStagingArea()
params.BlockCoinbaseMaturity = 0 params.BlockCoinbaseMaturity = 0
factory := consensus.NewFactory() factory := consensus.NewFactory()
@ -174,7 +179,8 @@ func TestIsFinalizedTransaction(t *testing.T) {
} }
} }
block, err := tc.BuildBlock(&externalapi.DomainCoinbaseData{&externalapi.ScriptPublicKey{}, nil}, nil) block, err := tc.BuildBlock(
&externalapi.DomainCoinbaseData{&externalapi.ScriptPublicKey{}, nil}, nil)
if err != nil { if err != nil {
t.Fatalf("Error getting block: %+v", err) t.Fatalf("Error getting block: %+v", err)
} }
@ -182,7 +188,7 @@ func TestIsFinalizedTransaction(t *testing.T) {
if err != nil { if err != nil {
t.Fatalf("Error Inserting block: %+v", err) t.Fatalf("Error Inserting block: %+v", err)
} }
blockGhostDAG, err := tc.GHOSTDAGDataStore().Get(tc.DatabaseContext(), consensushashing.BlockHash(block)) blockGhostDAG, err := tc.GHOSTDAGDataStore().Get(tc.DatabaseContext(), stagingArea, consensushashing.BlockHash(block))
if err != nil { if err != nil {
t.Fatalf("Error getting GhostDAG Data: %+v", err) t.Fatalf("Error getting GhostDAG Data: %+v", err)
} }
@ -212,7 +218,7 @@ func TestIsFinalizedTransaction(t *testing.T) {
checkForLockTimeAndSequence(blockGhostDAG.BlueScore(), 0, false) checkForLockTimeAndSequence(blockGhostDAG.BlueScore(), 0, false)
checkForLockTimeAndSequence(blockGhostDAG.BlueScore()-1, 0, true) checkForLockTimeAndSequence(blockGhostDAG.BlueScore()-1, 0, true)
pastMedianTime, err := tc.PastMedianTimeManager().PastMedianTime(consensushashing.BlockHash(block)) pastMedianTime, err := tc.PastMedianTimeManager().PastMedianTime(stagingArea, consensushashing.BlockHash(block))
if err != nil { if err != nil {
t.Fatalf("PastMedianTime: %+v", err) t.Fatalf("PastMedianTime: %+v", err)
} }

View File

@ -1,6 +1,7 @@
package blockvalidator package blockvalidator
import ( import (
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi" "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/kaspanet/kaspad/domain/consensus/ruleerrors" "github.com/kaspanet/kaspad/domain/consensus/ruleerrors"
"github.com/kaspanet/kaspad/domain/consensus/utils/consensushashing" "github.com/kaspanet/kaspad/domain/consensus/utils/consensushashing"
@ -14,11 +15,11 @@ import (
// ValidateBodyInIsolation validates block bodies in isolation from the current // ValidateBodyInIsolation validates block bodies in isolation from the current
// consensus state // consensus state
func (v *blockValidator) ValidateBodyInIsolation(blockHash *externalapi.DomainHash) error { func (v *blockValidator) ValidateBodyInIsolation(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) error {
onEnd := logger.LogAndMeasureExecutionTime(log, "ValidateBodyInContext") onEnd := logger.LogAndMeasureExecutionTime(log, "ValidateBodyInContext")
defer onEnd() defer onEnd()
block, err := v.blockStore.Block(v.databaseContext, blockHash) block, err := v.blockStore.Block(v.databaseContext, stagingArea, blockHash)
if err != nil { if err != nil {
return err return err
} }

View File

@ -2,6 +2,7 @@ package blockvalidator_test
import ( import (
"bytes" "bytes"
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/testapi" "github.com/kaspanet/kaspad/domain/consensus/model/testapi"
"github.com/kaspanet/kaspad/domain/consensus/utils/constants" "github.com/kaspanet/kaspad/domain/consensus/utils/constants"
"github.com/kaspanet/kaspad/domain/consensus/utils/merkle" "github.com/kaspanet/kaspad/domain/consensus/utils/merkle"
@ -104,17 +105,19 @@ func TestCheckBlockSanity(t *testing.T) {
t.Fatalf("Too few transactions in block, expect at least 3, got %v", len(exampleValidBlock.Transactions)) t.Fatalf("Too few transactions in block, expect at least 3, got %v", len(exampleValidBlock.Transactions))
} }
tc.BlockStore().Stage(blockHash, &exampleValidBlock) stagingArea := model.NewStagingArea()
err = tc.BlockValidator().ValidateBodyInIsolation(blockHash) tc.BlockStore().Stage(stagingArea, blockHash, &exampleValidBlock)
err = tc.BlockValidator().ValidateBodyInIsolation(stagingArea, blockHash)
if err != nil { if err != nil {
t.Fatalf("Failed validating block in isolation: %v", err) t.Fatalf("Failed validating block in isolation: %v", err)
} }
// Test with block with wrong transactions sorting order // Test with block with wrong transactions sorting order
blockHash = consensushashing.BlockHash(&blockWithWrongTxOrder) blockHash = consensushashing.BlockHash(&blockWithWrongTxOrder)
tc.BlockStore().Stage(blockHash, &blockWithWrongTxOrder) tc.BlockStore().Stage(stagingArea, blockHash, &blockWithWrongTxOrder)
err = tc.BlockValidator().ValidateBodyInIsolation(blockHash) err = tc.BlockValidator().ValidateBodyInIsolation(stagingArea, blockHash)
if !errors.Is(err, ruleerrors.ErrTransactionsNotSorted) { if !errors.Is(err, ruleerrors.ErrTransactionsNotSorted) {
t.Errorf("CheckBlockSanity: Expected ErrTransactionsNotSorted error, instead got %v", err) t.Errorf("CheckBlockSanity: Expected ErrTransactionsNotSorted error, instead got %v", err)
} }
@ -122,8 +125,8 @@ func TestCheckBlockSanity(t *testing.T) {
// Test a block with invalid parents order // Test a block with invalid parents order
// We no longer require blocks to have ordered parents // We no longer require blocks to have ordered parents
blockHash = consensushashing.BlockHash(&unOrderedParentsBlock) blockHash = consensushashing.BlockHash(&unOrderedParentsBlock)
tc.BlockStore().Stage(blockHash, &unOrderedParentsBlock) tc.BlockStore().Stage(stagingArea, blockHash, &unOrderedParentsBlock)
err = tc.BlockValidator().ValidateBodyInIsolation(blockHash) err = tc.BlockValidator().ValidateBodyInIsolation(stagingArea, blockHash)
if err != nil { if err != nil {
t.Errorf("CheckBlockSanity: Expected block to be be body in isolation valid, got error instead: %v", err) t.Errorf("CheckBlockSanity: Expected block to be be body in isolation valid, got error instead: %v", err)
} }
@ -1041,9 +1044,10 @@ func TestBlockSize(t *testing.T) {
t.Fatalf("Error BuildBlockWithParents : %+v", err) t.Fatalf("Error BuildBlockWithParents : %+v", err)
} }
blockHash := consensushashing.BlockHash(block) blockHash := consensushashing.BlockHash(block)
tc.BlockStore().Stage(blockHash, block) stagingArea := model.NewStagingArea()
tc.BlockStore().Stage(stagingArea, blockHash, block)
err = tc.BlockValidator().ValidateBodyInIsolation(blockHash) err = tc.BlockValidator().ValidateBodyInIsolation(stagingArea, blockHash)
if err == nil || !errors.Is(err, ruleerrors.ErrBlockSizeTooHigh) { if err == nil || !errors.Is(err, ruleerrors.ErrBlockSizeTooHigh) {
t.Fatalf("ValidateBodyInIsolationTest: TestBlockSize:"+ t.Fatalf("ValidateBodyInIsolationTest: TestBlockSize:"+
" Unexpected error: Expected to: %v, but got : %v", ruleerrors.ErrBlockSizeTooHigh, err) " Unexpected error: Expected to: %v, but got : %v", ruleerrors.ErrBlockSizeTooHigh, err)
@ -1098,9 +1102,10 @@ func TestCheckBlockDuplicateTransactions(t *testing.T) {
t.Fatalf("Error BuildBlockWithParents : %+v", err) t.Fatalf("Error BuildBlockWithParents : %+v", err)
} }
blockHash := consensushashing.BlockHash(block) blockHash := consensushashing.BlockHash(block)
tc.BlockStore().Stage(blockHash, block) stagingArea := model.NewStagingArea()
tc.BlockStore().Stage(stagingArea, blockHash, block)
err = tc.BlockValidator().ValidateBodyInIsolation(blockHash) err = tc.BlockValidator().ValidateBodyInIsolation(stagingArea, blockHash)
if err == nil || !errors.Is(err, ruleerrors.ErrDuplicateTx) { if err == nil || !errors.Is(err, ruleerrors.ErrDuplicateTx) {
t.Fatalf("ValidateBodyInIsolationTest: TestCheckBlockDuplicateTransactions:"+ t.Fatalf("ValidateBodyInIsolationTest: TestCheckBlockDuplicateTransactions:"+
" Unexpected error: Expected to: %v, but got : %v", ruleerrors.ErrDuplicateTx, err) " Unexpected error: Expected to: %v, but got : %v", ruleerrors.ErrDuplicateTx, err)
@ -1154,9 +1159,10 @@ func TestCheckBlockContainsOnlyOneCoinbase(t *testing.T) {
t.Fatalf("Error BuildBlockWithParents : %+v", err) t.Fatalf("Error BuildBlockWithParents : %+v", err)
} }
blockHash := consensushashing.BlockHash(block) blockHash := consensushashing.BlockHash(block)
tc.BlockStore().Stage(blockHash, block) stagingArea := model.NewStagingArea()
tc.BlockStore().Stage(stagingArea, blockHash, block)
err = tc.BlockValidator().ValidateBodyInIsolation(blockHash) err = tc.BlockValidator().ValidateBodyInIsolation(stagingArea, blockHash)
if err == nil || !errors.Is(err, ruleerrors.ErrMultipleCoinbases) { if err == nil || !errors.Is(err, ruleerrors.ErrMultipleCoinbases) {
t.Fatalf("ValidateBodyInIsolationTest: TestCheckBlockContainsOnlyOneCoinbase:"+ t.Fatalf("ValidateBodyInIsolationTest: TestCheckBlockContainsOnlyOneCoinbase:"+
" Unexpected error: Expected to: %v, but got : %v", ruleerrors.ErrMultipleCoinbases, err) " Unexpected error: Expected to: %v, but got : %v", ruleerrors.ErrMultipleCoinbases, err)
@ -1210,9 +1216,10 @@ func TestCheckBlockDoubleSpends(t *testing.T) {
t.Fatalf("Error BuildBlockWithParents : %+v", err) t.Fatalf("Error BuildBlockWithParents : %+v", err)
} }
blockHash := consensushashing.BlockHash(block) blockHash := consensushashing.BlockHash(block)
tc.BlockStore().Stage(blockHash, block) stagingArea := model.NewStagingArea()
tc.BlockStore().Stage(stagingArea, blockHash, block)
err = tc.BlockValidator().ValidateBodyInIsolation(blockHash) err = tc.BlockValidator().ValidateBodyInIsolation(stagingArea, blockHash)
if err == nil || !errors.Is(err, ruleerrors.ErrDoubleSpendInSameBlock) { if err == nil || !errors.Is(err, ruleerrors.ErrDoubleSpendInSameBlock) {
t.Fatalf("ValidateBodyInIsolationTest: TestCheckBlockDoubleSpends:"+ t.Fatalf("ValidateBodyInIsolationTest: TestCheckBlockDoubleSpends:"+
" Unexpected error: Expected to: %v, but got : %v", ruleerrors.ErrDoubleSpendInSameBlock, err) " Unexpected error: Expected to: %v, but got : %v", ruleerrors.ErrDoubleSpendInSameBlock, err)
@ -1282,9 +1289,10 @@ func TestCheckFirstBlockTransactionIsCoinbase(t *testing.T) {
block := initBlockWithFirstTransactionDifferentThanCoinbase(params) block := initBlockWithFirstTransactionDifferentThanCoinbase(params)
blockHash := consensushashing.BlockHash(block) blockHash := consensushashing.BlockHash(block)
tc.BlockStore().Stage(blockHash, block) stagingArea := model.NewStagingArea()
tc.BlockStore().Stage(stagingArea, blockHash, block)
err = tc.BlockValidator().ValidateBodyInIsolation(blockHash) err = tc.BlockValidator().ValidateBodyInIsolation(stagingArea, blockHash)
if err == nil || !errors.Is(err, ruleerrors.ErrFirstTxNotCoinbase) { if err == nil || !errors.Is(err, ruleerrors.ErrFirstTxNotCoinbase) {
t.Fatalf("ValidateBodyInIsolationTest: TestCheckFirstBlockTransactionIsCoinbase:"+ t.Fatalf("ValidateBodyInIsolationTest: TestCheckFirstBlockTransactionIsCoinbase:"+
" Unexpected error: Expected to: %v, but got : %v", ruleerrors.ErrFirstTxNotCoinbase, err) " Unexpected error: Expected to: %v, but got : %v", ruleerrors.ErrFirstTxNotCoinbase, err)

View File

@ -2,6 +2,7 @@ package blockvalidator
import ( import (
"fmt" "fmt"
"github.com/kaspanet/kaspad/domain/consensus/model" "github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi" "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/kaspanet/kaspad/domain/consensus/ruleerrors" "github.com/kaspanet/kaspad/domain/consensus/ruleerrors"
@ -12,16 +13,16 @@ import (
// ValidateHeaderInContext validates block headers in the context of the current // ValidateHeaderInContext validates block headers in the context of the current
// consensus state // consensus state
func (v *blockValidator) ValidateHeaderInContext(blockHash *externalapi.DomainHash) error { func (v *blockValidator) ValidateHeaderInContext(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) error {
onEnd := logger.LogAndMeasureExecutionTime(log, "ValidateHeaderInContext") onEnd := logger.LogAndMeasureExecutionTime(log, "ValidateHeaderInContext")
defer onEnd() defer onEnd()
header, err := v.blockHeaderStore.BlockHeader(v.databaseContext, blockHash) header, err := v.blockHeaderStore.BlockHeader(v.databaseContext, stagingArea, blockHash)
if err != nil { if err != nil {
return err return err
} }
hasValidatedHeader, err := v.hasValidatedHeader(blockHash) hasValidatedHeader, err := v.hasValidatedHeader(stagingArea, blockHash)
if err != nil { if err != nil {
return err return err
} }
@ -30,7 +31,7 @@ func (v *blockValidator) ValidateHeaderInContext(blockHash *externalapi.DomainHa
var logErr error var logErr error
log.Debug(logger.NewLogClosure(func() string { log.Debug(logger.NewLogClosure(func() string {
var ghostdagData *model.BlockGHOSTDAGData var ghostdagData *model.BlockGHOSTDAGData
ghostdagData, logErr = v.ghostdagDataStore.Get(v.databaseContext, blockHash) ghostdagData, logErr = v.ghostdagDataStore.Get(v.databaseContext, stagingArea, blockHash)
if err != nil { if err != nil {
return "" return ""
} }
@ -43,12 +44,12 @@ func (v *blockValidator) ValidateHeaderInContext(blockHash *externalapi.DomainHa
} }
} }
err = v.validateMedianTime(header) err = v.validateMedianTime(stagingArea, header)
if err != nil { if err != nil {
return err return err
} }
err = v.checkMergeSizeLimit(blockHash) err = v.checkMergeSizeLimit(stagingArea, blockHash)
if err != nil { if err != nil {
return err return err
} }
@ -57,18 +58,18 @@ func (v *blockValidator) ValidateHeaderInContext(blockHash *externalapi.DomainHa
// since it's used to find a block's finality point. // since it's used to find a block's finality point.
// This might not be required if this block's header has previously been received during // This might not be required if this block's header has previously been received during
// headers-first synchronization. // headers-first synchronization.
hasReachabilityData, err := v.reachabilityStore.HasReachabilityData(v.databaseContext, blockHash) hasReachabilityData, err := v.reachabilityStore.HasReachabilityData(v.databaseContext, stagingArea, blockHash)
if err != nil { if err != nil {
return err return err
} }
if !hasReachabilityData { if !hasReachabilityData {
err = v.reachabilityManager.AddBlock(blockHash) err = v.reachabilityManager.AddBlock(stagingArea, blockHash)
if err != nil { if err != nil {
return err return err
} }
} }
err = v.mergeDepthManager.CheckBoundedMergeDepth(blockHash) err = v.mergeDepthManager.CheckBoundedMergeDepth(stagingArea, blockHash)
if err != nil { if err != nil {
return err return err
} }
@ -76,8 +77,8 @@ func (v *blockValidator) ValidateHeaderInContext(blockHash *externalapi.DomainHa
return nil return nil
} }
func (v *blockValidator) hasValidatedHeader(blockHash *externalapi.DomainHash) (bool, error) { func (v *blockValidator) hasValidatedHeader(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) (bool, error) {
exists, err := v.blockStatusStore.Exists(v.databaseContext, blockHash) exists, err := v.blockStatusStore.Exists(v.databaseContext, stagingArea, blockHash)
if err != nil { if err != nil {
return false, err return false, err
} }
@ -86,7 +87,7 @@ func (v *blockValidator) hasValidatedHeader(blockHash *externalapi.DomainHash) (
return false, nil return false, nil
} }
status, err := v.blockStatusStore.Get(v.databaseContext, blockHash) status, err := v.blockStatusStore.Get(v.databaseContext, stagingArea, blockHash)
if err != nil { if err != nil {
return false, err return false, err
} }
@ -95,14 +96,14 @@ func (v *blockValidator) hasValidatedHeader(blockHash *externalapi.DomainHash) (
} }
// checkParentsIncest validates that no parent is an ancestor of another parent // checkParentsIncest validates that no parent is an ancestor of another parent
func (v *blockValidator) checkParentsIncest(header externalapi.BlockHeader) error { func (v *blockValidator) checkParentsIncest(stagingArea *model.StagingArea, header externalapi.BlockHeader) error {
for _, parentA := range header.ParentHashes() { for _, parentA := range header.ParentHashes() {
for _, parentB := range header.ParentHashes() { for _, parentB := range header.ParentHashes() {
if parentA.Equal(parentB) { if parentA.Equal(parentB) {
continue continue
} }
isAAncestorOfB, err := v.dagTopologyManager.IsAncestorOf(parentA, parentB) isAAncestorOfB, err := v.dagTopologyManager.IsAncestorOf(stagingArea, parentA, parentB)
if err != nil { if err != nil {
return err return err
} }
@ -119,7 +120,7 @@ func (v *blockValidator) checkParentsIncest(header externalapi.BlockHeader) erro
return nil return nil
} }
func (v *blockValidator) validateMedianTime(header externalapi.BlockHeader) error { func (v *blockValidator) validateMedianTime(stagingArea *model.StagingArea, header externalapi.BlockHeader) error {
if len(header.ParentHashes()) == 0 { if len(header.ParentHashes()) == 0 {
return nil return nil
} }
@ -127,7 +128,7 @@ func (v *blockValidator) validateMedianTime(header externalapi.BlockHeader) erro
// Ensure the timestamp for the block header is not before the // Ensure the timestamp for the block header is not before the
// median time of the last several blocks (medianTimeBlocks). // median time of the last several blocks (medianTimeBlocks).
hash := consensushashing.HeaderHash(header) hash := consensushashing.HeaderHash(header)
pastMedianTime, err := v.pastMedianTimeManager.PastMedianTime(hash) pastMedianTime, err := v.pastMedianTimeManager.PastMedianTime(stagingArea, hash)
if err != nil { if err != nil {
return err return err
} }
@ -140,8 +141,8 @@ func (v *blockValidator) validateMedianTime(header externalapi.BlockHeader) erro
return nil return nil
} }
func (v *blockValidator) checkMergeSizeLimit(hash *externalapi.DomainHash) error { func (v *blockValidator) checkMergeSizeLimit(stagingArea *model.StagingArea, hash *externalapi.DomainHash) error {
ghostdagData, err := v.ghostdagDataStore.Get(v.databaseContext, hash) ghostdagData, err := v.ghostdagDataStore.Get(v.databaseContext, stagingArea, hash)
if err != nil { if err != nil {
return err return err
} }

View File

@ -42,20 +42,19 @@ func TestValidateMedianTime(t *testing.T) {
} }
pastMedianTime := func(parents ...*externalapi.DomainHash) int64 { pastMedianTime := func(parents ...*externalapi.DomainHash) int64 {
stagingArea := model.NewStagingArea()
var tempHash externalapi.DomainHash var tempHash externalapi.DomainHash
tc.BlockRelationStore().StageBlockRelation(&tempHash, &model.BlockRelations{ tc.BlockRelationStore().StageBlockRelation(stagingArea, &tempHash, &model.BlockRelations{
Parents: parents, Parents: parents,
Children: nil, Children: nil,
}) })
defer tc.BlockRelationStore().Discard()
err = tc.GHOSTDAGManager().GHOSTDAG(&tempHash) err = tc.GHOSTDAGManager().GHOSTDAG(stagingArea, &tempHash)
if err != nil { if err != nil {
t.Fatalf("GHOSTDAG: %+v", err) t.Fatalf("GHOSTDAG: %+v", err)
} }
defer tc.GHOSTDAGDataStore().Discard()
pastMedianTime, err := tc.PastMedianTimeManager().PastMedianTime(&tempHash) pastMedianTime, err := tc.PastMedianTimeManager().PastMedianTime(stagingArea, &tempHash)
if err != nil { if err != nil {
t.Fatalf("PastMedianTime: %+v", err) t.Fatalf("PastMedianTime: %+v", err)
} }

View File

@ -1,6 +1,7 @@
package blockvalidator package blockvalidator
import ( import (
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi" "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/kaspanet/kaspad/domain/consensus/ruleerrors" "github.com/kaspanet/kaspad/domain/consensus/ruleerrors"
"github.com/kaspanet/kaspad/domain/consensus/utils/consensushashing" "github.com/kaspanet/kaspad/domain/consensus/utils/consensushashing"
@ -12,11 +13,11 @@ import (
// ValidateHeaderInIsolation validates block headers in isolation from the current // ValidateHeaderInIsolation validates block headers in isolation from the current
// consensus state // consensus state
func (v *blockValidator) ValidateHeaderInIsolation(blockHash *externalapi.DomainHash) error { func (v *blockValidator) ValidateHeaderInIsolation(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) error {
onEnd := logger.LogAndMeasureExecutionTime(log, "ValidateHeaderInIsolation") onEnd := logger.LogAndMeasureExecutionTime(log, "ValidateHeaderInIsolation")
defer onEnd() defer onEnd()
header, err := v.blockHeaderStore.BlockHeader(v.databaseContext, blockHash) header, err := v.blockHeaderStore.BlockHeader(v.databaseContext, stagingArea, blockHash)
if err != nil { if err != nil {
return err return err
} }

View File

@ -1,6 +1,7 @@
package blockvalidator package blockvalidator
import ( import (
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi" "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
"github.com/kaspanet/kaspad/domain/consensus/model/pow" "github.com/kaspanet/kaspad/domain/consensus/model/pow"
"github.com/kaspanet/kaspad/domain/consensus/ruleerrors" "github.com/kaspanet/kaspad/domain/consensus/ruleerrors"
@ -11,26 +12,28 @@ import (
"github.com/pkg/errors" "github.com/pkg/errors"
) )
func (v *blockValidator) ValidatePruningPointViolationAndProofOfWorkAndDifficulty(blockHash *externalapi.DomainHash) error { func (v *blockValidator) ValidatePruningPointViolationAndProofOfWorkAndDifficulty(stagingArea *model.StagingArea,
blockHash *externalapi.DomainHash) error {
onEnd := logger.LogAndMeasureExecutionTime(log, "ValidatePruningPointViolationAndProofOfWorkAndDifficulty") onEnd := logger.LogAndMeasureExecutionTime(log, "ValidatePruningPointViolationAndProofOfWorkAndDifficulty")
defer onEnd() defer onEnd()
header, err := v.blockHeaderStore.BlockHeader(v.databaseContext, blockHash) header, err := v.blockHeaderStore.BlockHeader(v.databaseContext, stagingArea, blockHash)
if err != nil { if err != nil {
return err return err
} }
err = v.checkParentHeadersExist(header) err = v.checkParentHeadersExist(stagingArea, header)
if err != nil { if err != nil {
return err return err
} }
err = v.checkParentsIncest(header) err = v.checkParentsIncest(stagingArea, header)
if err != nil { if err != nil {
return err return err
} }
err = v.checkPruningPointViolation(header) err = v.checkPruningPointViolation(stagingArea, header)
if err != nil { if err != nil {
return err return err
} }
@ -40,12 +43,12 @@ func (v *blockValidator) ValidatePruningPointViolationAndProofOfWorkAndDifficult
return err return err
} }
err = v.dagTopologyManager.SetParents(blockHash, header.ParentHashes()) err = v.dagTopologyManager.SetParents(stagingArea, blockHash, header.ParentHashes())
if err != nil { if err != nil {
return err return err
} }
err = v.validateDifficulty(blockHash) err = v.validateDifficulty(stagingArea, blockHash)
if err != nil { if err != nil {
return err return err
} }
@ -53,9 +56,9 @@ func (v *blockValidator) ValidatePruningPointViolationAndProofOfWorkAndDifficult
return nil return nil
} }
func (v *blockValidator) validateDifficulty(blockHash *externalapi.DomainHash) error { func (v *blockValidator) validateDifficulty(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) error {
// We need to calculate GHOSTDAG for the block in order to check its difficulty // We need to calculate GHOSTDAG for the block in order to check its difficulty
err := v.ghostdagManager.GHOSTDAG(blockHash) err := v.ghostdagManager.GHOSTDAG(stagingArea, blockHash)
if err != nil { if err != nil {
return err return err
} }
@ -63,12 +66,12 @@ func (v *blockValidator) validateDifficulty(blockHash *externalapi.DomainHash) e
// Ensure the difficulty specified in the block header matches // Ensure the difficulty specified in the block header matches
// the calculated difficulty based on the previous block and // the calculated difficulty based on the previous block and
// difficulty retarget rules. // difficulty retarget rules.
expectedBits, err := v.difficultyManager.StageDAADataAndReturnRequiredDifficulty(blockHash) expectedBits, err := v.difficultyManager.StageDAADataAndReturnRequiredDifficulty(stagingArea, blockHash)
if err != nil { if err != nil {
return err return err
} }
header, err := v.blockHeaderStore.BlockHeader(v.databaseContext, blockHash) header, err := v.blockHeaderStore.BlockHeader(v.databaseContext, stagingArea, blockHash)
if err != nil { if err != nil {
return err return err
} }
@ -110,15 +113,15 @@ func (v *blockValidator) checkProofOfWork(header externalapi.BlockHeader) error
return nil return nil
} }
func (v *blockValidator) checkParentHeadersExist(header externalapi.BlockHeader) error { func (v *blockValidator) checkParentHeadersExist(stagingArea *model.StagingArea, header externalapi.BlockHeader) error {
missingParentHashes := []*externalapi.DomainHash{} missingParentHashes := []*externalapi.DomainHash{}
for _, parent := range header.ParentHashes() { for _, parent := range header.ParentHashes() {
parentHeaderExists, err := v.blockHeaderStore.HasBlockHeader(v.databaseContext, parent) parentHeaderExists, err := v.blockHeaderStore.HasBlockHeader(v.databaseContext, stagingArea, parent)
if err != nil { if err != nil {
return err return err
} }
if !parentHeaderExists { if !parentHeaderExists {
parentStatus, err := v.blockStatusStore.Get(v.databaseContext, parent) parentStatus, err := v.blockStatusStore.Get(v.databaseContext, stagingArea, parent)
if err != nil { if err != nil {
if !database.IsNotFoundError(err) { if !database.IsNotFoundError(err) {
return err return err
@ -138,10 +141,10 @@ func (v *blockValidator) checkParentHeadersExist(header externalapi.BlockHeader)
return nil return nil
} }
func (v *blockValidator) checkPruningPointViolation(header externalapi.BlockHeader) error { func (v *blockValidator) checkPruningPointViolation(stagingArea *model.StagingArea, header externalapi.BlockHeader) error {
// check if the pruning point is on past of at least one parent of the header's parents. // check if the pruning point is on past of at least one parent of the header's parents.
hasPruningPoint, err := v.pruningStore.HasPruningPoint(v.databaseContext) hasPruningPoint, err := v.pruningStore.HasPruningPoint(v.databaseContext, stagingArea)
if err != nil { if err != nil {
return err return err
} }
@ -151,12 +154,12 @@ func (v *blockValidator) checkPruningPointViolation(header externalapi.BlockHead
return nil return nil
} }
pruningPoint, err := v.pruningStore.PruningPoint(v.databaseContext) pruningPoint, err := v.pruningStore.PruningPoint(v.databaseContext, stagingArea)
if err != nil { if err != nil {
return err return err
} }
isAncestorOfAny, err := v.dagTopologyManager.IsAncestorOfAny(pruningPoint, header.ParentHashes()) isAncestorOfAny, err := v.dagTopologyManager.IsAncestorOfAny(stagingArea, pruningPoint, header.ParentHashes())
if err != nil { if err != nil {
return err return err
} }

View File

@ -1,6 +1,12 @@
package blockvalidator_test package blockvalidator_test
import ( import (
"math"
"math/big"
"math/rand"
"testing"
"time"
"github.com/kaspanet/kaspad/domain/consensus/model" "github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/pow" "github.com/kaspanet/kaspad/domain/consensus/model/pow"
"github.com/kaspanet/kaspad/domain/consensus/ruleerrors" "github.com/kaspanet/kaspad/domain/consensus/ruleerrors"
@ -10,11 +16,6 @@ import (
"github.com/kaspanet/kaspad/domain/consensus/utils/merkle" "github.com/kaspanet/kaspad/domain/consensus/utils/merkle"
"github.com/kaspanet/kaspad/domain/consensus/utils/mining" "github.com/kaspanet/kaspad/domain/consensus/utils/mining"
"github.com/kaspanet/kaspad/util/difficulty" "github.com/kaspanet/kaspad/util/difficulty"
"math"
"math/big"
"math/rand"
"testing"
"time"
"github.com/kaspanet/kaspad/domain/consensus" "github.com/kaspanet/kaspad/domain/consensus"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi" "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
@ -157,7 +158,8 @@ func TestCheckParentHeadersExist(t *testing.T) {
t.Fatalf("unexpected missing parents %s", errMissingParents.MissingParentHashes) t.Fatalf("unexpected missing parents %s", errMissingParents.MissingParentHashes)
} }
invalidBlock, _, err := tc.BuildBlockWithParents([]*externalapi.DomainHash{params.GenesisHash}, nil, nil) invalidBlock, _, err := tc.BuildBlockWithParents(
[]*externalapi.DomainHash{params.GenesisHash}, nil, nil)
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
@ -250,7 +252,6 @@ func TestCheckPruningPointViolation(t *testing.T) {
// of the type DifficultyManager for this test (defined below). // of the type DifficultyManager for this test (defined below).
func TestValidateDifficulty(t *testing.T) { func TestValidateDifficulty(t *testing.T) {
testutils.ForAllNets(t, true, func(t *testing.T, params *dagconfig.Params) { testutils.ForAllNets(t, true, func(t *testing.T, params *dagconfig.Params) {
factory := consensus.NewFactory() factory := consensus.NewFactory()
mocDifficulty := &mocDifficultyManager{} mocDifficulty := &mocDifficultyManager{}
factory.SetTestDifficultyManager(func(_ model.DBReader, _ model.GHOSTDAGManager, _ model.GHOSTDAGDataStore, factory.SetTestDifficultyManager(func(_ model.DBReader, _ model.GHOSTDAGManager, _ model.GHOSTDAGDataStore,
@ -281,12 +282,13 @@ func TestValidateDifficulty(t *testing.T) {
t.Fatalf("TestValidateDifficulty: Failed build block with parents: %v.", err) t.Fatalf("TestValidateDifficulty: Failed build block with parents: %v.", err)
} }
blockHash := consensushashing.BlockHash(block) blockHash := consensushashing.BlockHash(block)
tc.BlockStore().Stage(blockHash, block) stagingArea := model.NewStagingArea()
tc.BlockHeaderStore().Stage(blockHash, block.Header) tc.BlockStore().Stage(stagingArea, blockHash, block)
tc.BlockHeaderStore().Stage(stagingArea, blockHash, block.Header)
wrongTestDifficulty := mocDifficulty.testDifficulty + uint32(5) wrongTestDifficulty := mocDifficulty.testDifficulty + uint32(5)
mocDifficulty.testDifficulty = wrongTestDifficulty mocDifficulty.testDifficulty = wrongTestDifficulty
err = tc.BlockValidator().ValidatePruningPointViolationAndProofOfWorkAndDifficulty(blockHash) err = tc.BlockValidator().ValidatePruningPointViolationAndProofOfWorkAndDifficulty(stagingArea, blockHash)
if err == nil || !errors.Is(err, ruleerrors.ErrUnexpectedDifficulty) { if err == nil || !errors.Is(err, ruleerrors.ErrUnexpectedDifficulty) {
t.Fatalf("Expected block to be invalid with err: %v, instead found: %v", ruleerrors.ErrUnexpectedDifficulty, err) t.Fatalf("Expected block to be invalid with err: %v, instead found: %v", ruleerrors.ErrUnexpectedDifficulty, err)
} }
@ -300,15 +302,15 @@ type mocDifficultyManager struct {
} }
// RequiredDifficulty returns the difficulty required for the test // RequiredDifficulty returns the difficulty required for the test
func (dm *mocDifficultyManager) RequiredDifficulty(*externalapi.DomainHash) (uint32, error) { func (dm *mocDifficultyManager) RequiredDifficulty(*model.StagingArea, *externalapi.DomainHash) (uint32, error) {
return dm.testDifficulty, nil return dm.testDifficulty, nil
} }
// StageDAADataAndReturnRequiredDifficulty returns the difficulty required for the test // StageDAADataAndReturnRequiredDifficulty returns the difficulty required for the test
func (dm *mocDifficultyManager) StageDAADataAndReturnRequiredDifficulty(blockHash *externalapi.DomainHash) (uint32, error) { func (dm *mocDifficultyManager) StageDAADataAndReturnRequiredDifficulty(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) (uint32, error) {
// Populate daaBlocksStore with fake values // Populate daaBlocksStore with fake values
dm.daaBlocksStore.StageDAAScore(blockHash, 0) dm.daaBlocksStore.StageDAAScore(stagingArea, blockHash, 0)
dm.daaBlocksStore.StageBlockDAAAddedBlocks(blockHash, nil) dm.daaBlocksStore.StageBlockDAAAddedBlocks(stagingArea, blockHash, nil)
return dm.testDifficulty, nil return dm.testDifficulty, nil
} }

View File

@ -20,27 +20,27 @@ type coinbaseManager struct {
daaBlocksStore model.DAABlocksStore daaBlocksStore model.DAABlocksStore
} }
func (c *coinbaseManager) ExpectedCoinbaseTransaction(blockHash *externalapi.DomainHash, func (c *coinbaseManager) ExpectedCoinbaseTransaction(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash,
coinbaseData *externalapi.DomainCoinbaseData) (*externalapi.DomainTransaction, error) { coinbaseData *externalapi.DomainCoinbaseData) (*externalapi.DomainTransaction, error) {
ghostdagData, err := c.ghostdagDataStore.Get(c.databaseContext, blockHash) ghostdagData, err := c.ghostdagDataStore.Get(c.databaseContext, stagingArea, blockHash)
if err != nil { if err != nil {
return nil, err return nil, err
} }
acceptanceData, err := c.acceptanceDataStore.Get(c.databaseContext, blockHash) acceptanceData, err := c.acceptanceDataStore.Get(c.databaseContext, stagingArea, blockHash)
if err != nil { if err != nil {
return nil, err return nil, err
} }
daaAddedBlocksSet, err := c.daaAddedBlocksSet(blockHash) daaAddedBlocksSet, err := c.daaAddedBlocksSet(stagingArea, blockHash)
if err != nil { if err != nil {
return nil, err return nil, err
} }
txOuts := make([]*externalapi.DomainTransactionOutput, 0, len(ghostdagData.MergeSetBlues())) txOuts := make([]*externalapi.DomainTransactionOutput, 0, len(ghostdagData.MergeSetBlues()))
for i, blue := range ghostdagData.MergeSetBlues() { for i, blue := range ghostdagData.MergeSetBlues() {
txOut, hasReward, err := c.coinbaseOutputForBlueBlock(blue, acceptanceData[i], daaAddedBlocksSet) txOut, hasReward, err := c.coinbaseOutputForBlueBlock(stagingArea, blue, acceptanceData[i], daaAddedBlocksSet)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -50,7 +50,8 @@ func (c *coinbaseManager) ExpectedCoinbaseTransaction(blockHash *externalapi.Dom
} }
} }
txOut, hasReward, err := c.coinbaseOutputForRewardFromRedBlocks(ghostdagData, acceptanceData, daaAddedBlocksSet, coinbaseData) txOut, hasReward, err := c.coinbaseOutputForRewardFromRedBlocks(
stagingArea, ghostdagData, acceptanceData, daaAddedBlocksSet, coinbaseData)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -75,8 +76,10 @@ func (c *coinbaseManager) ExpectedCoinbaseTransaction(blockHash *externalapi.Dom
}, nil }, nil
} }
func (c *coinbaseManager) daaAddedBlocksSet(blockHash *externalapi.DomainHash) (hashset.HashSet, error) { func (c *coinbaseManager) daaAddedBlocksSet(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) (
daaAddedBlocks, err := c.daaBlocksStore.DAAAddedBlocks(c.databaseContext, blockHash) hashset.HashSet, error) {
daaAddedBlocks, err := c.daaBlocksStore.DAAAddedBlocks(c.databaseContext, stagingArea, blockHash)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -86,11 +89,11 @@ func (c *coinbaseManager) daaAddedBlocksSet(blockHash *externalapi.DomainHash) (
// coinbaseOutputForBlueBlock calculates the output that should go into the coinbase transaction of blueBlock // coinbaseOutputForBlueBlock calculates the output that should go into the coinbase transaction of blueBlock
// If blueBlock gets no fee - returns nil for txOut // If blueBlock gets no fee - returns nil for txOut
func (c *coinbaseManager) coinbaseOutputForBlueBlock(blueBlock *externalapi.DomainHash, func (c *coinbaseManager) coinbaseOutputForBlueBlock(stagingArea *model.StagingArea,
blockAcceptanceData *externalapi.BlockAcceptanceData, blueBlock *externalapi.DomainHash, blockAcceptanceData *externalapi.BlockAcceptanceData,
mergingBlockDAAAddedBlocksSet hashset.HashSet) (*externalapi.DomainTransactionOutput, bool, error) { mergingBlockDAAAddedBlocksSet hashset.HashSet) (*externalapi.DomainTransactionOutput, bool, error) {
totalReward, err := c.calcMergedBlockReward(blueBlock, blockAcceptanceData, mergingBlockDAAAddedBlocksSet) totalReward, err := c.calcMergedBlockReward(stagingArea, blueBlock, blockAcceptanceData, mergingBlockDAAAddedBlocksSet)
if err != nil { if err != nil {
return nil, false, err return nil, false, err
} }
@ -113,14 +116,14 @@ func (c *coinbaseManager) coinbaseOutputForBlueBlock(blueBlock *externalapi.Doma
return txOut, true, nil return txOut, true, nil
} }
func (c *coinbaseManager) coinbaseOutputForRewardFromRedBlocks(ghostdagData *model.BlockGHOSTDAGData, func (c *coinbaseManager) coinbaseOutputForRewardFromRedBlocks(stagingArea *model.StagingArea,
acceptanceData externalapi.AcceptanceData, daaAddedBlocksSet hashset.HashSet, ghostdagData *model.BlockGHOSTDAGData, acceptanceData externalapi.AcceptanceData, daaAddedBlocksSet hashset.HashSet,
coinbaseData *externalapi.DomainCoinbaseData) (*externalapi.DomainTransactionOutput, bool, error) { coinbaseData *externalapi.DomainCoinbaseData) (*externalapi.DomainTransactionOutput, bool, error) {
totalReward := uint64(0) totalReward := uint64(0)
mergeSetBluesCount := len(ghostdagData.MergeSetBlues()) mergeSetBluesCount := len(ghostdagData.MergeSetBlues())
for i, red := range ghostdagData.MergeSetReds() { for i, red := range ghostdagData.MergeSetReds() {
reward, err := c.calcMergedBlockReward(red, acceptanceData[mergeSetBluesCount+i], daaAddedBlocksSet) reward, err := c.calcMergedBlockReward(stagingArea, red, acceptanceData[mergeSetBluesCount+i], daaAddedBlocksSet)
if err != nil { if err != nil {
return nil, false, err return nil, false, err
} }
@ -148,12 +151,12 @@ func (c *coinbaseManager) coinbaseOutputForRewardFromRedBlocks(ghostdagData *mod
// //
// At the target block generation rate for the main network, this is // At the target block generation rate for the main network, this is
// approximately every 4 years. // approximately every 4 years.
func (c *coinbaseManager) calcBlockSubsidy(blockHash *externalapi.DomainHash) (uint64, error) { func (c *coinbaseManager) calcBlockSubsidy(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) (uint64, error) {
if c.subsidyReductionInterval == 0 { if c.subsidyReductionInterval == 0 {
return c.baseSubsidy, nil return c.baseSubsidy, nil
} }
daaScore, err := c.daaBlocksStore.DAAScore(c.databaseContext, blockHash) daaScore, err := c.daaBlocksStore.DAAScore(c.databaseContext, stagingArea, blockHash)
if err != nil { if err != nil {
return 0, err return 0, err
} }
@ -162,7 +165,7 @@ func (c *coinbaseManager) calcBlockSubsidy(blockHash *externalapi.DomainHash) (u
return c.baseSubsidy >> uint(daaScore/c.subsidyReductionInterval), nil return c.baseSubsidy >> uint(daaScore/c.subsidyReductionInterval), nil
} }
func (c *coinbaseManager) calcMergedBlockReward(blockHash *externalapi.DomainHash, func (c *coinbaseManager) calcMergedBlockReward(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash,
blockAcceptanceData *externalapi.BlockAcceptanceData, mergingBlockDAAAddedBlocksSet hashset.HashSet) (uint64, error) { blockAcceptanceData *externalapi.BlockAcceptanceData, mergingBlockDAAAddedBlocksSet hashset.HashSet) (uint64, error) {
if !blockHash.Equal(blockAcceptanceData.BlockHash) { if !blockHash.Equal(blockAcceptanceData.BlockHash) {
@ -181,7 +184,7 @@ func (c *coinbaseManager) calcMergedBlockReward(blockHash *externalapi.DomainHas
} }
} }
subsidy, err := c.calcBlockSubsidy(blockHash) subsidy, err := c.calcBlockSubsidy(stagingArea, blockHash)
if err != nil { if err != nil {
return 0, err return 0, err
} }

View File

@ -32,8 +32,7 @@ func (c *coinbaseManager) serializeCoinbasePayload(blueScore uint64, coinbaseDat
} }
// ExtractCoinbaseDataAndBlueScore deserializes the coinbase payload to its component (scriptPubKey and extra data). // ExtractCoinbaseDataAndBlueScore deserializes the coinbase payload to its component (scriptPubKey and extra data).
func (c *coinbaseManager) ExtractCoinbaseDataAndBlueScore(coinbaseTx *externalapi.DomainTransaction) (blueScore uint64, func (c *coinbaseManager) ExtractCoinbaseDataAndBlueScore(coinbaseTx *externalapi.DomainTransaction) (blueScore uint64, coinbaseData *externalapi.DomainCoinbaseData, err error) {
coinbaseData *externalapi.DomainCoinbaseData, err error) {
minLength := uint64Len + lengthOfVersionScriptPubKey + lengthOfscriptPubKeyLength minLength := uint64Len + lengthOfVersionScriptPubKey + lengthOfscriptPubKeyLength
if len(coinbaseTx.Payload) < minLength { if len(coinbaseTx.Payload) < minLength {

View File

@ -9,12 +9,12 @@ import (
// AddBlock submits the given block to be added to the // AddBlock submits the given block to be added to the
// current virtual. This process may result in a new virtual block // current virtual. This process may result in a new virtual block
// getting created // getting created
func (csm *consensusStateManager) AddBlock(blockHash *externalapi.DomainHash) (*externalapi.SelectedChainPath, externalapi.UTXODiff, error) { func (csm *consensusStateManager) AddBlock(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) (*externalapi.SelectedChainPath, externalapi.UTXODiff, error) {
onEnd := logger.LogAndMeasureExecutionTime(log, "csm.AddBlock") onEnd := logger.LogAndMeasureExecutionTime(log, "csm.AddBlock")
defer onEnd() defer onEnd()
log.Debugf("Resolving whether the block %s is the next virtual selected parent", blockHash) log.Debugf("Resolving whether the block %s is the next virtual selected parent", blockHash)
isCandidateToBeNextVirtualSelectedParent, err := csm.isCandidateToBeNextVirtualSelectedParent(blockHash) isCandidateToBeNextVirtualSelectedParent, err := csm.isCandidateToBeNextVirtualSelectedParent(stagingArea, blockHash)
if err != nil { if err != nil {
return nil, nil, err return nil, nil, err
} }
@ -25,7 +25,7 @@ func (csm *consensusStateManager) AddBlock(blockHash *externalapi.DomainHash) (*
// eventually try to fetch UTXO diffs from the past of the pruning point. // eventually try to fetch UTXO diffs from the past of the pruning point.
log.Debugf("Block %s is candidate to be the next virtual selected parent. Resolving whether it violates "+ log.Debugf("Block %s is candidate to be the next virtual selected parent. Resolving whether it violates "+
"finality", blockHash) "finality", blockHash)
isViolatingFinality, shouldNotify, err := csm.isViolatingFinality(blockHash) isViolatingFinality, shouldNotify, err := csm.isViolatingFinality(stagingArea, blockHash)
if err != nil { if err != nil {
return nil, nil, err return nil, nil, err
} }
@ -37,7 +37,7 @@ func (csm *consensusStateManager) AddBlock(blockHash *externalapi.DomainHash) (*
if !isViolatingFinality { if !isViolatingFinality {
log.Debugf("Block %s doesn't violate finality. Resolving its block status", blockHash) log.Debugf("Block %s doesn't violate finality. Resolving its block status", blockHash)
blockStatus, err := csm.resolveBlockStatus(blockHash) blockStatus, err := csm.resolveBlockStatus(stagingArea, blockHash)
if err != nil { if err != nil {
return nil, nil, err return nil, nil, err
} }
@ -50,14 +50,14 @@ func (csm *consensusStateManager) AddBlock(blockHash *externalapi.DomainHash) (*
} }
log.Debugf("Adding block %s to the DAG tips", blockHash) log.Debugf("Adding block %s to the DAG tips", blockHash)
newTips, err := csm.addTip(blockHash) newTips, err := csm.addTip(stagingArea, blockHash)
if err != nil { if err != nil {
return nil, nil, err return nil, nil, err
} }
log.Debugf("After adding %s, the amount of new tips are %d", blockHash, len(newTips)) log.Debugf("After adding %s, the amount of new tips are %d", blockHash, len(newTips))
log.Debugf("Updating the virtual with the new tips") log.Debugf("Updating the virtual with the new tips")
selectedParentChainChanges, virtualUTXODiff, err := csm.updateVirtual(blockHash, newTips) selectedParentChainChanges, virtualUTXODiff, err := csm.updateVirtual(stagingArea, blockHash, newTips)
if err != nil { if err != nil {
return nil, nil, err return nil, nil, err
} }
@ -65,7 +65,9 @@ func (csm *consensusStateManager) AddBlock(blockHash *externalapi.DomainHash) (*
return selectedParentChainChanges, virtualUTXODiff, nil return selectedParentChainChanges, virtualUTXODiff, nil
} }
func (csm *consensusStateManager) isCandidateToBeNextVirtualSelectedParent(blockHash *externalapi.DomainHash) (bool, error) { func (csm *consensusStateManager) isCandidateToBeNextVirtualSelectedParent(
stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) (bool, error) {
log.Debugf("isCandidateToBeNextVirtualSelectedParent start for block %s", blockHash) log.Debugf("isCandidateToBeNextVirtualSelectedParent start for block %s", blockHash)
defer log.Debugf("isCandidateToBeNextVirtualSelectedParent end for block %s", blockHash) defer log.Debugf("isCandidateToBeNextVirtualSelectedParent end for block %s", blockHash)
@ -75,14 +77,15 @@ func (csm *consensusStateManager) isCandidateToBeNextVirtualSelectedParent(block
return true, nil return true, nil
} }
virtualGhostdagData, err := csm.ghostdagDataStore.Get(csm.databaseContext, model.VirtualBlockHash) virtualGhostdagData, err := csm.ghostdagDataStore.Get(csm.databaseContext, stagingArea, model.VirtualBlockHash)
if err != nil { if err != nil {
return false, err return false, err
} }
log.Debugf("Selecting the next selected parent between "+ log.Debugf("Selecting the next selected parent between "+
"the block %s the current selected parent %s", blockHash, virtualGhostdagData.SelectedParent()) "the block %s the current selected parent %s", blockHash, virtualGhostdagData.SelectedParent())
nextVirtualSelectedParent, err := csm.ghostdagManager.ChooseSelectedParent(virtualGhostdagData.SelectedParent(), blockHash) nextVirtualSelectedParent, err := csm.ghostdagManager.ChooseSelectedParent(
stagingArea, virtualGhostdagData.SelectedParent(), blockHash)
if err != nil { if err != nil {
return false, err return false, err
} }
@ -91,23 +94,25 @@ func (csm *consensusStateManager) isCandidateToBeNextVirtualSelectedParent(block
return blockHash.Equal(nextVirtualSelectedParent), nil return blockHash.Equal(nextVirtualSelectedParent), nil
} }
func (csm *consensusStateManager) addTip(newTipHash *externalapi.DomainHash) (newTips []*externalapi.DomainHash, err error) { func (csm *consensusStateManager) addTip(stagingArea *model.StagingArea, newTipHash *externalapi.DomainHash) (newTips []*externalapi.DomainHash, err error) {
log.Debugf("addTip start for new tip %s", newTipHash) log.Debugf("addTip start for new tip %s", newTipHash)
defer log.Debugf("addTip end for new tip %s", newTipHash) defer log.Debugf("addTip end for new tip %s", newTipHash)
log.Debugf("Calculating the new tips for new tip %s", newTipHash) log.Debugf("Calculating the new tips for new tip %s", newTipHash)
newTips, err = csm.calculateNewTips(newTipHash) newTips, err = csm.calculateNewTips(stagingArea, newTipHash)
if err != nil { if err != nil {
return nil, err return nil, err
} }
csm.consensusStateStore.StageTips(newTips) csm.consensusStateStore.StageTips(stagingArea, newTips)
log.Debugf("Staged the new tips, len: %d", len(newTips)) log.Debugf("Staged the new tips, len: %d", len(newTips))
return newTips, nil return newTips, nil
} }
func (csm *consensusStateManager) calculateNewTips(newTipHash *externalapi.DomainHash) ([]*externalapi.DomainHash, error) { func (csm *consensusStateManager) calculateNewTips(
stagingArea *model.StagingArea, newTipHash *externalapi.DomainHash) ([]*externalapi.DomainHash, error) {
log.Debugf("calculateNewTips start for new tip %s", newTipHash) log.Debugf("calculateNewTips start for new tip %s", newTipHash)
defer log.Debugf("calculateNewTips end for new tip %s", newTipHash) defer log.Debugf("calculateNewTips end for new tip %s", newTipHash)
@ -116,13 +121,13 @@ func (csm *consensusStateManager) calculateNewTips(newTipHash *externalapi.Domai
return []*externalapi.DomainHash{newTipHash}, nil return []*externalapi.DomainHash{newTipHash}, nil
} }
currentTips, err := csm.consensusStateStore.Tips(csm.databaseContext) currentTips, err := csm.consensusStateStore.Tips(stagingArea, csm.databaseContext)
if err != nil { if err != nil {
return nil, err return nil, err
} }
log.Debugf("The current tips are: %s", currentTips) log.Debugf("The current tips are: %s", currentTips)
newTipParents, err := csm.dagTopologyManager.Parents(newTipHash) newTipParents, err := csm.dagTopologyManager.Parents(stagingArea, newTipHash)
if err != nil { if err != nil {
return nil, err return nil, err
} }

View File

@ -1,9 +1,11 @@
package consensusstatemanager_test package consensusstatemanager_test
import ( import (
"testing"
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/utils/consensushashing" "github.com/kaspanet/kaspad/domain/consensus/utils/consensushashing"
"github.com/kaspanet/kaspad/domain/consensus/utils/utxo" "github.com/kaspanet/kaspad/domain/consensus/utils/utxo"
"testing"
"github.com/kaspanet/kaspad/domain/consensus" "github.com/kaspanet/kaspad/domain/consensus"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi" "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
@ -41,7 +43,7 @@ func TestVirtualDiff(t *testing.T) {
t.Fatalf("Error adding block A: %+v", err) t.Fatalf("Error adding block A: %+v", err)
} }
blockB, err := tc.BlockStore().Block(tc.DatabaseContext(), blockBHash) blockB, err := tc.BlockStore().Block(tc.DatabaseContext(), model.NewStagingArea(), blockBHash)
if err != nil { if err != nil {
t.Fatalf("Block: %+v", err) t.Fatalf("Block: %+v", err)
} }

View File

@ -13,8 +13,8 @@ import (
"github.com/kaspanet/kaspad/domain/consensus/utils/transactionhelper" "github.com/kaspanet/kaspad/domain/consensus/utils/transactionhelper"
) )
func (csm *consensusStateManager) CalculatePastUTXOAndAcceptanceData(blockHash *externalapi.DomainHash) ( func (csm *consensusStateManager) CalculatePastUTXOAndAcceptanceData(stagingArea *model.StagingArea,
externalapi.UTXODiff, externalapi.AcceptanceData, model.Multiset, error) { blockHash *externalapi.DomainHash) (externalapi.UTXODiff, externalapi.AcceptanceData, model.Multiset, error) {
onEnd := logger.LogAndMeasureExecutionTime(log, "CalculatePastUTXOAndAcceptanceData") onEnd := logger.LogAndMeasureExecutionTime(log, "CalculatePastUTXOAndAcceptanceData")
defer onEnd() defer onEnd()
@ -27,19 +27,19 @@ func (csm *consensusStateManager) CalculatePastUTXOAndAcceptanceData(blockHash *
return utxo.NewUTXODiff(), externalapi.AcceptanceData{}, multiset.New(), nil return utxo.NewUTXODiff(), externalapi.AcceptanceData{}, multiset.New(), nil
} }
blockGHOSTDAGData, err := csm.ghostdagDataStore.Get(csm.databaseContext, blockHash) blockGHOSTDAGData, err := csm.ghostdagDataStore.Get(csm.databaseContext, stagingArea, blockHash)
if err != nil { if err != nil {
return nil, nil, nil, err return nil, nil, nil, err
} }
log.Debugf("Restoring the past UTXO of block %s with selectedParent %s", log.Debugf("Restoring the past UTXO of block %s with selectedParent %s",
blockHash, blockGHOSTDAGData.SelectedParent()) blockHash, blockGHOSTDAGData.SelectedParent())
selectedParentPastUTXO, err := csm.restorePastUTXO(blockGHOSTDAGData.SelectedParent()) selectedParentPastUTXO, err := csm.restorePastUTXO(stagingArea, blockGHOSTDAGData.SelectedParent())
if err != nil { if err != nil {
return nil, nil, nil, err return nil, nil, nil, err
} }
daaScore, err := csm.daaBlocksStore.DAAScore(csm.databaseContext, blockHash) daaScore, err := csm.daaBlocksStore.DAAScore(csm.databaseContext, stagingArea, blockHash)
if err != nil { if err != nil {
return nil, nil, nil, err return nil, nil, nil, err
} }
@ -49,13 +49,14 @@ func (csm *consensusStateManager) CalculatePastUTXOAndAcceptanceData(blockHash *
selectedParentPastUTXO.ToAdd().Len(), selectedParentPastUTXO.ToRemove().Len()) selectedParentPastUTXO.ToAdd().Len(), selectedParentPastUTXO.ToRemove().Len())
log.Debugf("Applying blue blocks to the selected parent past UTXO of block %s", blockHash) log.Debugf("Applying blue blocks to the selected parent past UTXO of block %s", blockHash)
acceptanceData, utxoDiff, err := csm.applyMergeSetBlocks(blockHash, selectedParentPastUTXO, blockGHOSTDAGData, daaScore) acceptanceData, utxoDiff, err := csm.applyMergeSetBlocks(
stagingArea, blockHash, selectedParentPastUTXO, blockGHOSTDAGData, daaScore)
if err != nil { if err != nil {
return nil, nil, nil, err return nil, nil, nil, err
} }
log.Debugf("Calculating the multiset of %s", blockHash) log.Debugf("Calculating the multiset of %s", blockHash)
multiset, err := csm.calculateMultiset(acceptanceData, blockGHOSTDAGData, daaScore) multiset, err := csm.calculateMultiset(stagingArea, acceptanceData, blockGHOSTDAGData, daaScore)
if err != nil { if err != nil {
return nil, nil, nil, err return nil, nil, nil, err
} }
@ -64,7 +65,9 @@ func (csm *consensusStateManager) CalculatePastUTXOAndAcceptanceData(blockHash *
return utxoDiff.ToImmutable(), acceptanceData, multiset, nil return utxoDiff.ToImmutable(), acceptanceData, multiset, nil
} }
func (csm *consensusStateManager) restorePastUTXO(blockHash *externalapi.DomainHash) (externalapi.MutableUTXODiff, error) { func (csm *consensusStateManager) restorePastUTXO(
stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) (externalapi.MutableUTXODiff, error) {
onEnd := logger.LogAndMeasureExecutionTime(log, "restorePastUTXO") onEnd := logger.LogAndMeasureExecutionTime(log, "restorePastUTXO")
defer onEnd() defer onEnd()
@ -77,7 +80,7 @@ func (csm *consensusStateManager) restorePastUTXO(blockHash *externalapi.DomainH
nextBlockHash := blockHash nextBlockHash := blockHash
for { for {
log.Debugf("Collecting UTXO diff for block %s", nextBlockHash) log.Debugf("Collecting UTXO diff for block %s", nextBlockHash)
utxoDiff, err := csm.utxoDiffStore.UTXODiff(csm.databaseContext, nextBlockHash) utxoDiff, err := csm.utxoDiffStore.UTXODiff(csm.databaseContext, stagingArea, nextBlockHash)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -85,7 +88,7 @@ func (csm *consensusStateManager) restorePastUTXO(blockHash *externalapi.DomainH
log.Debugf("Collected UTXO diff for block %s: toAdd: %d, toRemove: %d", log.Debugf("Collected UTXO diff for block %s: toAdd: %d, toRemove: %d",
nextBlockHash, utxoDiff.ToAdd().Len(), utxoDiff.ToRemove().Len()) nextBlockHash, utxoDiff.ToAdd().Len(), utxoDiff.ToRemove().Len())
exists, err := csm.utxoDiffStore.HasUTXODiffChild(csm.databaseContext, nextBlockHash) exists, err := csm.utxoDiffStore.HasUTXODiffChild(csm.databaseContext, stagingArea, nextBlockHash)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -95,7 +98,7 @@ func (csm *consensusStateManager) restorePastUTXO(blockHash *externalapi.DomainH
break break
} }
nextBlockHash, err = csm.utxoDiffStore.UTXODiffChild(csm.databaseContext, nextBlockHash) nextBlockHash, err = csm.utxoDiffStore.UTXODiffChild(csm.databaseContext, stagingArea, nextBlockHash)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -120,7 +123,7 @@ func (csm *consensusStateManager) restorePastUTXO(blockHash *externalapi.DomainH
return accumulatedDiff, nil return accumulatedDiff, nil
} }
func (csm *consensusStateManager) applyMergeSetBlocks(blockHash *externalapi.DomainHash, func (csm *consensusStateManager) applyMergeSetBlocks(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash,
selectedParentPastUTXODiff externalapi.MutableUTXODiff, ghostdagData *model.BlockGHOSTDAGData, daaScore uint64) ( selectedParentPastUTXODiff externalapi.MutableUTXODiff, ghostdagData *model.BlockGHOSTDAGData, daaScore uint64) (
externalapi.AcceptanceData, externalapi.MutableUTXODiff, error) { externalapi.AcceptanceData, externalapi.MutableUTXODiff, error) {
@ -129,12 +132,12 @@ func (csm *consensusStateManager) applyMergeSetBlocks(blockHash *externalapi.Dom
mergeSetHashes := ghostdagData.MergeSet() mergeSetHashes := ghostdagData.MergeSet()
log.Debugf("Merge set for block %s is %v", blockHash, mergeSetHashes) log.Debugf("Merge set for block %s is %v", blockHash, mergeSetHashes)
mergeSetBlocks, err := csm.blockStore.Blocks(csm.databaseContext, mergeSetHashes) mergeSetBlocks, err := csm.blockStore.Blocks(csm.databaseContext, stagingArea, mergeSetHashes)
if err != nil { if err != nil {
return nil, nil, err return nil, nil, err
} }
selectedParentMedianTime, err := csm.pastMedianTimeManager.PastMedianTime(blockHash) selectedParentMedianTime, err := csm.pastMedianTimeManager.PastMedianTime(stagingArea, blockHash)
if err != nil { if err != nil {
return nil, nil, err return nil, nil, err
} }
@ -161,8 +164,8 @@ func (csm *consensusStateManager) applyMergeSetBlocks(blockHash *externalapi.Dom
log.Tracef("Attempting to accept transaction %s in block %s", log.Tracef("Attempting to accept transaction %s in block %s",
transactionID, mergeSetBlockHash) transactionID, mergeSetBlockHash)
isAccepted, accumulatedMass, err = csm.maybeAcceptTransaction(transaction, blockHash, isSelectedParent, isAccepted, accumulatedMass, err = csm.maybeAcceptTransaction(stagingArea, transaction, blockHash,
accumulatedUTXODiff, accumulatedMass, selectedParentMedianTime, daaScore) isSelectedParent, accumulatedUTXODiff, accumulatedMass, selectedParentMedianTime, daaScore)
if err != nil { if err != nil {
return nil, nil, err return nil, nil, err
} }
@ -190,17 +193,17 @@ func (csm *consensusStateManager) applyMergeSetBlocks(blockHash *externalapi.Dom
return multiblockAcceptanceData, accumulatedUTXODiff, nil return multiblockAcceptanceData, accumulatedUTXODiff, nil
} }
func (csm *consensusStateManager) maybeAcceptTransaction(transaction *externalapi.DomainTransaction, func (csm *consensusStateManager) maybeAcceptTransaction(stagingArea *model.StagingArea,
blockHash *externalapi.DomainHash, isSelectedParent bool, accumulatedUTXODiff externalapi.MutableUTXODiff, transaction *externalapi.DomainTransaction, blockHash *externalapi.DomainHash, isSelectedParent bool,
accumulatedMassBefore uint64, selectedParentPastMedianTime int64, blockDAAScore uint64) ( accumulatedUTXODiff externalapi.MutableUTXODiff, accumulatedMassBefore uint64, selectedParentPastMedianTime int64,
isAccepted bool, accumulatedMassAfter uint64, err error) { blockDAAScore uint64) (isAccepted bool, accumulatedMassAfter uint64, err error) {
transactionID := consensushashing.TransactionID(transaction) transactionID := consensushashing.TransactionID(transaction)
log.Tracef("maybeAcceptTransaction start for transaction %s in block %s", transactionID, blockHash) log.Tracef("maybeAcceptTransaction start for transaction %s in block %s", transactionID, blockHash)
defer log.Tracef("maybeAcceptTransaction end for transaction %s in block %s", transactionID, blockHash) defer log.Tracef("maybeAcceptTransaction end for transaction %s in block %s", transactionID, blockHash)
log.Tracef("Populating transaction %s with UTXO entries", transactionID) log.Tracef("Populating transaction %s with UTXO entries", transactionID)
err = csm.populateTransactionWithUTXOEntriesFromVirtualOrDiff(transaction, accumulatedUTXODiff.ToImmutable()) err = csm.populateTransactionWithUTXOEntriesFromVirtualOrDiff(stagingArea, transaction, accumulatedUTXODiff.ToImmutable())
if err != nil { if err != nil {
if !errors.As(err, &(ruleerrors.RuleError{})) { if !errors.As(err, &(ruleerrors.RuleError{})) {
return false, 0, err return false, 0, err
@ -221,7 +224,7 @@ func (csm *consensusStateManager) maybeAcceptTransaction(transaction *externalap
} else { } else {
log.Tracef("Validating transaction %s in block %s", transactionID, blockHash) log.Tracef("Validating transaction %s in block %s", transactionID, blockHash)
err = csm.transactionValidator.ValidateTransactionInContextAndPopulateMassAndFee( err = csm.transactionValidator.ValidateTransactionInContextAndPopulateMassAndFee(
transaction, blockHash, selectedParentPastMedianTime) stagingArea, transaction, blockHash, selectedParentPastMedianTime)
if err != nil { if err != nil {
if !errors.As(err, &(ruleerrors.RuleError{})) { if !errors.As(err, &(ruleerrors.RuleError{})) {
return false, 0, err return false, 0, err
@ -274,13 +277,12 @@ func (csm *consensusStateManager) checkTransactionMass(
} }
// RestorePastUTXOSetIterator restores the given block's UTXOSet iterator, and returns it as a externalapi.ReadOnlyUTXOSetIterator // RestorePastUTXOSetIterator restores the given block's UTXOSet iterator, and returns it as a externalapi.ReadOnlyUTXOSetIterator
func (csm *consensusStateManager) RestorePastUTXOSetIterator(blockHash *externalapi.DomainHash) ( func (csm *consensusStateManager) RestorePastUTXOSetIterator(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) (externalapi.ReadOnlyUTXOSetIterator, error) {
externalapi.ReadOnlyUTXOSetIterator, error) {
onEnd := logger.LogAndMeasureExecutionTime(log, "RestorePastUTXOSetIterator") onEnd := logger.LogAndMeasureExecutionTime(log, "RestorePastUTXOSetIterator")
defer onEnd() defer onEnd()
blockStatus, err := csm.resolveBlockStatus(blockHash) blockStatus, err := csm.resolveBlockStatus(stagingArea, blockHash)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -294,12 +296,12 @@ func (csm *consensusStateManager) RestorePastUTXOSetIterator(blockHash *external
defer log.Tracef("RestorePastUTXOSetIterator end for block %s", blockHash) defer log.Tracef("RestorePastUTXOSetIterator end for block %s", blockHash)
log.Debugf("Calculating UTXO diff for block %s", blockHash) log.Debugf("Calculating UTXO diff for block %s", blockHash)
blockDiff, err := csm.restorePastUTXO(blockHash) blockDiff, err := csm.restorePastUTXO(stagingArea, blockHash)
if err != nil { if err != nil {
return nil, err return nil, err
} }
virtualUTXOSetIterator, err := csm.consensusStateStore.VirtualUTXOSetIterator(csm.databaseContext) virtualUTXOSetIterator, err := csm.consensusStateStore.VirtualUTXOSetIterator(csm.databaseContext, stagingArea)
if err != nil { if err != nil {
return nil, err return nil, err
} }

View File

@ -3,6 +3,8 @@ package consensusstatemanager_test
import ( import (
"testing" "testing"
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/testapi" "github.com/kaspanet/kaspad/domain/consensus/model/testapi"
"github.com/kaspanet/kaspad/domain/consensus/utils/transactionhelper" "github.com/kaspanet/kaspad/domain/consensus/utils/transactionhelper"
@ -82,7 +84,7 @@ func checkBlockUTXOCommitment(t *testing.T, consensus testapi.TestConsensus, blo
// Get the past UTXO set of block // Get the past UTXO set of block
csm := consensus.ConsensusStateManager() csm := consensus.ConsensusStateManager()
utxoSetIterator, err := csm.RestorePastUTXOSetIterator(blockHash) utxoSetIterator, err := csm.RestorePastUTXOSetIterator(model.NewStagingArea(), blockHash)
if err != nil { if err != nil {
t.Fatalf("Error restoring past UTXO of block %s: %+v", blockName, err) t.Fatalf("Error restoring past UTXO of block %s: %+v", blockName, err)
} }
@ -114,6 +116,8 @@ func checkBlockUTXOCommitment(t *testing.T, consensus testapi.TestConsensus, blo
func TestPastUTXOMultiset(t *testing.T) { func TestPastUTXOMultiset(t *testing.T) {
testutils.ForAllNets(t, true, func(t *testing.T, params *dagconfig.Params) { testutils.ForAllNets(t, true, func(t *testing.T, params *dagconfig.Params) {
stagingArea := model.NewStagingArea()
factory := consensus.NewFactory() factory := consensus.NewFactory()
consensus, teardown, err := factory.NewTestConsensus(params, false, "TestUTXOCommitment") consensus, teardown, err := factory.NewTestConsensus(params, false, "TestUTXOCommitment")
@ -135,7 +139,7 @@ func TestPastUTXOMultiset(t *testing.T) {
testedBlockHash := currentHash testedBlockHash := currentHash
// Take testedBlock's multiset and hash // Take testedBlock's multiset and hash
firstMultiset, err := consensus.MultisetStore().Get(consensus.DatabaseContext(), testedBlockHash) firstMultiset, err := consensus.MultisetStore().Get(consensus.DatabaseContext(), stagingArea, testedBlockHash)
if err != nil { if err != nil {
return return
} }
@ -148,7 +152,7 @@ func TestPastUTXOMultiset(t *testing.T) {
} }
// Take testedBlock's multiset and hash again // Take testedBlock's multiset and hash again
secondMultiset, err := consensus.MultisetStore().Get(consensus.DatabaseContext(), testedBlockHash) secondMultiset, err := consensus.MultisetStore().Get(consensus.DatabaseContext(), stagingArea, testedBlockHash)
if err != nil { if err != nil {
return return
} }

View File

@ -1,9 +1,12 @@
package consensusstatemanager package consensusstatemanager
import "github.com/kaspanet/kaspad/domain/consensus/model/externalapi" import (
"github.com/kaspanet/kaspad/domain/consensus/model"
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
)
func (csm *consensusStateManager) isViolatingFinality(blockHash *externalapi.DomainHash) (isViolatingFinality bool, func (csm *consensusStateManager) isViolatingFinality(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash,
shouldSendNotification bool, err error) { ) (isViolatingFinality bool, shouldSendNotification bool, err error) {
log.Debugf("isViolatingFinality start for block %s", blockHash) log.Debugf("isViolatingFinality start for block %s", blockHash)
defer log.Debugf("isViolatingFinality end for block %s", blockHash) defer log.Debugf("isViolatingFinality end for block %s", blockHash)
@ -15,7 +18,7 @@ func (csm *consensusStateManager) isViolatingFinality(blockHash *externalapi.Dom
} }
var finalityPoint *externalapi.DomainHash var finalityPoint *externalapi.DomainHash
virtualFinalityPoint, err := csm.finalityManager.VirtualFinalityPoint() virtualFinalityPoint, err := csm.finalityManager.VirtualFinalityPoint(stagingArea)
if err != nil { if err != nil {
return false, false, err return false, false, err
} }
@ -26,13 +29,13 @@ func (csm *consensusStateManager) isViolatingFinality(blockHash *externalapi.Dom
// finality point from the virtual point-of-view is in the past of the pruning point. // finality point from the virtual point-of-view is in the past of the pruning point.
// In such situation we override the finality point to be the pruning point to avoid situations where // In such situation we override the finality point to be the pruning point to avoid situations where
// the virtual selected parent chain don't include the pruning point. // the virtual selected parent chain don't include the pruning point.
pruningPoint, err := csm.pruningStore.PruningPoint(csm.databaseContext) pruningPoint, err := csm.pruningStore.PruningPoint(csm.databaseContext, stagingArea)
if err != nil { if err != nil {
return false, false, err return false, false, err
} }
log.Debugf("The pruning point is: %s", pruningPoint) log.Debugf("The pruning point is: %s", pruningPoint)
isFinalityPointInPastOfPruningPoint, err := csm.dagTopologyManager.IsAncestorOf(virtualFinalityPoint, pruningPoint) isFinalityPointInPastOfPruningPoint, err := csm.dagTopologyManager.IsAncestorOf(stagingArea, virtualFinalityPoint, pruningPoint)
if err != nil { if err != nil {
return false, false, err return false, false, err
} }
@ -45,8 +48,8 @@ func (csm *consensusStateManager) isViolatingFinality(blockHash *externalapi.Dom
finalityPoint = pruningPoint finalityPoint = pruningPoint
} }
isInSelectedParentChainOfFinalityPoint, err := csm.dagTopologyManager.IsInSelectedParentChainOf(finalityPoint, isInSelectedParentChainOfFinalityPoint, err :=
blockHash) csm.dagTopologyManager.IsInSelectedParentChainOf(stagingArea, finalityPoint, blockHash)
if err != nil { if err != nil {
return false, false, err return false, false, err
} }

View File

@ -5,18 +5,17 @@ import (
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi" "github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
) )
func (csm *consensusStateManager) GetVirtualSelectedParentChainFromBlock( func (csm *consensusStateManager) GetVirtualSelectedParentChainFromBlock(stagingArea *model.StagingArea, blockHash *externalapi.DomainHash) (*externalapi.SelectedChainPath, error) {
blockHash *externalapi.DomainHash) (*externalapi.SelectedChainPath, error) {
// Calculate chain changes between the given blockHash and the // Calculate chain changes between the given blockHash and the
// virtual's selected parent. Note that we explicitly don't // virtual's selected parent. Note that we explicitly don't
// do the calculation against the virtual itself so that we // do the calculation against the virtual itself so that we
// won't later need to remove it from the result. // won't later need to remove it from the result.
virtualGHOSTDAGData, err := csm.ghostdagDataStore.Get(csm.databaseContext, model.VirtualBlockHash) virtualGHOSTDAGData, err := csm.ghostdagDataStore.Get(csm.databaseContext, stagingArea, model.VirtualBlockHash)
if err != nil { if err != nil {
return nil, err return nil, err
} }
virtualSelectedParent := virtualGHOSTDAGData.SelectedParent() virtualSelectedParent := virtualGHOSTDAGData.SelectedParent()
return csm.dagTraversalManager.CalculateChainPath(blockHash, virtualSelectedParent) return csm.dagTraversalManager.CalculateChainPath(stagingArea, blockHash, virtualSelectedParent)
} }

View File

@ -48,7 +48,7 @@ func TestCalculateChainPath(t *testing.T) {
} }
// Figure out which among blocks A and B is NOT the virtual selected parent // Figure out which among blocks A and B is NOT the virtual selected parent
virtualGHOSTDAGData, err := consensus.GHOSTDAGDataStore().Get(consensus.DatabaseContext(), model.VirtualBlockHash) virtualGHOSTDAGData, err := consensus.GHOSTDAGDataStore().Get(consensus.DatabaseContext(), model.NewStagingArea(), model.VirtualBlockHash)
if err != nil { if err != nil {
t.Fatalf("Error getting virtual GHOSTDAG data: %+v", err) t.Fatalf("Error getting virtual GHOSTDAG data: %+v", err)
} }

Some files were not shown because too many files have changed in this diff Show More