mirror of
https://github.com/kaspanet/kaspad.git
synced 2025-03-30 15:08:33 +00:00
Crucial fix for the UTXO difference mechanism (#2114)
* Illustrate the bug through prints * Change consensus API to a single ResolveVirtual call * nil changeset is not expected when err=nil * Fixes a deep bug in the resolve virtual process * Be more defensive at resolving virtual when adding a block * When finally resolved, set virtual parents properly * Return nil changeset when nothing happened * Make sure the block at the split point is reversed to new chain as well * bump to version 0.12.4 * Avoid locking consensus twice in the common case of adding block with updateVirtual=true * check err * Parents must be picked first before set as virtual parents * Keep the flag for tracking virtual state, since tip sorting perf is high with many tips * Improve and clarify resolve virtual tests * Addressing minor review comments * Fixed a bug in the reported virtual changeset, and modified the test to verify it * Addressing review comments
This commit is contained in:
parent
b9093d59eb
commit
1c9bb54cc2
@ -686,37 +686,28 @@ func (flow *handleIBDFlow) banIfBlockIsHeaderOnly(block *externalapi.DomainBlock
|
||||
}
|
||||
|
||||
func (flow *handleIBDFlow) resolveVirtual(estimatedVirtualDAAScoreTarget uint64) error {
|
||||
virtualDAAScoreStart, err := flow.Domain().Consensus().GetVirtualDAAScore()
|
||||
err := flow.Domain().Consensus().ResolveVirtual(func(virtualDAAScoreStart uint64, virtualDAAScore uint64) {
|
||||
var percents int
|
||||
if estimatedVirtualDAAScoreTarget-virtualDAAScoreStart <= 0 {
|
||||
percents = 100
|
||||
} else {
|
||||
percents = int(float64(virtualDAAScore-virtualDAAScoreStart) / float64(estimatedVirtualDAAScoreTarget-virtualDAAScoreStart) * 100)
|
||||
}
|
||||
if percents < 0 {
|
||||
percents = 0
|
||||
} else if percents > 100 {
|
||||
percents = 100
|
||||
}
|
||||
log.Infof("Resolving virtual. Estimated progress: %d%%", percents)
|
||||
})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
for i := 0; ; i++ {
|
||||
if i%10 == 0 {
|
||||
virtualDAAScore, err := flow.Domain().Consensus().GetVirtualDAAScore()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
var percents int
|
||||
if estimatedVirtualDAAScoreTarget-virtualDAAScoreStart <= 0 {
|
||||
percents = 100
|
||||
} else {
|
||||
percents = int(float64(virtualDAAScore-virtualDAAScoreStart) / float64(estimatedVirtualDAAScoreTarget-virtualDAAScoreStart) * 100)
|
||||
}
|
||||
log.Infof("Resolving virtual. Estimated progress: %d%%", percents)
|
||||
}
|
||||
isCompletelyResolved, err := flow.Domain().Consensus().ResolveVirtual()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if isCompletelyResolved {
|
||||
log.Infof("Resolved virtual")
|
||||
err = flow.OnNewBlockTemplate()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
log.Infof("Resolved virtual")
|
||||
err = flow.OnNewBlockTemplate()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
@ -194,11 +194,33 @@ func (s *consensus) BuildBlockTemplate(coinbaseData *externalapi.DomainCoinbaseD
|
||||
|
||||
// ValidateAndInsertBlock validates the given block and, if valid, applies it
|
||||
// to the current state
|
||||
func (s *consensus) ValidateAndInsertBlock(block *externalapi.DomainBlock, shouldValidateAgainstUTXO bool) error {
|
||||
func (s *consensus) ValidateAndInsertBlock(block *externalapi.DomainBlock, updateVirtual bool) error {
|
||||
if updateVirtual {
|
||||
s.lock.Lock()
|
||||
if s.virtualNotUpdated {
|
||||
s.lock.Unlock()
|
||||
err := s.ResolveVirtual(nil)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return s.validateAndInsertBlockWithLock(block, updateVirtual)
|
||||
}
|
||||
defer s.lock.Unlock()
|
||||
_, err := s.validateAndInsertBlockNoLock(block, updateVirtual)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
return s.validateAndInsertBlockWithLock(block, updateVirtual)
|
||||
}
|
||||
|
||||
func (s *consensus) validateAndInsertBlockWithLock(block *externalapi.DomainBlock, updateVirtual bool) error {
|
||||
s.lock.Lock()
|
||||
defer s.lock.Unlock()
|
||||
|
||||
_, err := s.validateAndInsertBlockNoLock(block, shouldValidateAgainstUTXO)
|
||||
_, err := s.validateAndInsertBlockNoLock(block, updateVirtual)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -206,19 +228,6 @@ func (s *consensus) ValidateAndInsertBlock(block *externalapi.DomainBlock, shoul
|
||||
}
|
||||
|
||||
func (s *consensus) validateAndInsertBlockNoLock(block *externalapi.DomainBlock, updateVirtual bool) (*externalapi.VirtualChangeSet, error) {
|
||||
// If virtual is in non-updated state, and the caller requests updating virtual -- then we must first
|
||||
// resolve virtual so that the new block can be fully processed properly
|
||||
if updateVirtual && s.virtualNotUpdated {
|
||||
for s.virtualNotUpdated {
|
||||
// We use 10000 << finality interval. See comment in `ResolveVirtual`.
|
||||
// We give up responsiveness of consensus in this rare case.
|
||||
_, err := s.resolveVirtualNoLock(10000) // Note `s.virtualNotUpdated` is updated within the call
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
virtualChangeSet, blockStatus, err := s.blockProcessor.ValidateAndInsertBlock(block, updateVirtual)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
@ -257,7 +266,7 @@ func (s *consensus) sendBlockAddedEvent(block *externalapi.DomainBlock, blockSta
|
||||
}
|
||||
|
||||
func (s *consensus) sendVirtualChangedEvent(virtualChangeSet *externalapi.VirtualChangeSet, wasVirtualUpdated bool) error {
|
||||
if !wasVirtualUpdated || s.consensusEventsChan == nil {
|
||||
if !wasVirtualUpdated || s.consensusEventsChan == nil || virtualChangeSet == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
@ -888,41 +897,68 @@ func (s *consensus) PopulateMass(transaction *externalapi.DomainTransaction) {
|
||||
s.transactionValidator.PopulateMass(transaction)
|
||||
}
|
||||
|
||||
func (s *consensus) ResolveVirtual() (bool, error) {
|
||||
func (s *consensus) ResolveVirtual(progressReportCallback func(uint64, uint64)) error {
|
||||
virtualDAAScoreStart, err := s.GetVirtualDAAScore()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
for i := 0; ; i++ {
|
||||
if i%10 == 0 && progressReportCallback != nil {
|
||||
virtualDAAScore, err := s.GetVirtualDAAScore()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
progressReportCallback(virtualDAAScoreStart, virtualDAAScore)
|
||||
}
|
||||
|
||||
// In order to prevent a situation that the consensus lock is held for too much time, we
|
||||
// release the lock each time we resolve 100 blocks.
|
||||
// Note: maxBlocksToResolve should be smaller than `params.FinalityDuration` in order to avoid a situation
|
||||
// where UpdatePruningPointByVirtual skips a pruning point.
|
||||
_, isCompletelyResolved, err := s.resolveVirtualChunkWithLock(100)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if isCompletelyResolved {
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *consensus) resolveVirtualChunkWithLock(maxBlocksToResolve uint64) (*externalapi.VirtualChangeSet, bool, error) {
|
||||
s.lock.Lock()
|
||||
defer s.lock.Unlock()
|
||||
|
||||
// In order to prevent a situation that the consensus lock is held for too much time, we
|
||||
// release the lock each time resolve 100 blocks.
|
||||
// Note: maxBlocksToResolve should be smaller than finality interval in order to avoid a situation
|
||||
// where UpdatePruningPointByVirtual skips a pruning point.
|
||||
return s.resolveVirtualNoLock(100)
|
||||
return s.resolveVirtualChunkNoLock(maxBlocksToResolve)
|
||||
}
|
||||
|
||||
func (s *consensus) resolveVirtualNoLock(maxBlocksToResolve uint64) (bool, error) {
|
||||
func (s *consensus) resolveVirtualChunkNoLock(maxBlocksToResolve uint64) (*externalapi.VirtualChangeSet, bool, error) {
|
||||
virtualChangeSet, isCompletelyResolved, err := s.consensusStateManager.ResolveVirtual(maxBlocksToResolve)
|
||||
if err != nil {
|
||||
return false, err
|
||||
return nil, false, err
|
||||
}
|
||||
s.virtualNotUpdated = !isCompletelyResolved
|
||||
|
||||
stagingArea := model.NewStagingArea()
|
||||
err = s.pruningManager.UpdatePruningPointByVirtual(stagingArea)
|
||||
if err != nil {
|
||||
return false, err
|
||||
return nil, false, err
|
||||
}
|
||||
|
||||
err = staging.CommitAllChanges(s.databaseContext, stagingArea)
|
||||
if err != nil {
|
||||
return false, err
|
||||
return nil, false, err
|
||||
}
|
||||
|
||||
err = s.sendVirtualChangedEvent(virtualChangeSet, true)
|
||||
if err != nil {
|
||||
return false, err
|
||||
return nil, false, err
|
||||
}
|
||||
|
||||
return isCompletelyResolved, nil
|
||||
return virtualChangeSet, isCompletelyResolved, nil
|
||||
}
|
||||
|
||||
func (s *consensus) BuildPruningPointProof() (*externalapi.PruningPointProof, error) {
|
||||
|
@ -515,6 +515,7 @@ func (f *factory) NewConsensus(config *Config, db infrastructuredatabase.Databas
|
||||
blocksWithTrustedDataDAAWindowStore: daaWindowStore,
|
||||
|
||||
consensusEventsChan: consensusEventsChan,
|
||||
virtualNotUpdated: true,
|
||||
}
|
||||
|
||||
if isOldReachabilityInitialized {
|
||||
|
@ -589,18 +589,13 @@ func TestFinalityResolveVirtual(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
for i := 0; ; i++ {
|
||||
isCompletelyResolved, err := tc.ResolveVirtual()
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
if isCompletelyResolved {
|
||||
t.Log("Resolved virtual")
|
||||
break
|
||||
}
|
||||
err = tc.ResolveVirtual(nil)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
t.Log("Resolved virtual")
|
||||
|
||||
sideChainTipGHOSTDAGData, err = tc.GHOSTDAGDataStore().Get(tc.DatabaseContext(), stagingArea, sideChainTipHash, false)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
|
@ -5,7 +5,7 @@ type Consensus interface {
|
||||
Init(skipAddingGenesis bool) error
|
||||
BuildBlock(coinbaseData *DomainCoinbaseData, transactions []*DomainTransaction) (*DomainBlock, error)
|
||||
BuildBlockTemplate(coinbaseData *DomainCoinbaseData, transactions []*DomainTransaction) (*DomainBlockTemplate, error)
|
||||
ValidateAndInsertBlock(block *DomainBlock, shouldValidateAgainstUTXO bool) error
|
||||
ValidateAndInsertBlock(block *DomainBlock, updateVirtual bool) error
|
||||
ValidateAndInsertBlockWithTrustedData(block *BlockWithTrustedData, validateUTXO bool) error
|
||||
ValidateTransactionAndPopulateWithConsensusData(transaction *DomainTransaction) error
|
||||
ImportPruningPoints(pruningPoints []BlockHeader) error
|
||||
@ -48,7 +48,7 @@ type Consensus interface {
|
||||
Anticone(blockHash *DomainHash) ([]*DomainHash, error)
|
||||
EstimateNetworkHashesPerSecond(startHash *DomainHash, windowSize int) (uint64, error)
|
||||
PopulateMass(transaction *DomainTransaction)
|
||||
ResolveVirtual() (bool, error)
|
||||
ResolveVirtual(progressReportCallback func(uint64, uint64)) error
|
||||
BlockDAAWindowHashes(blockHash *DomainHash) ([]*DomainHash, error)
|
||||
TrustedDataDataDAAHeader(trustedBlockHash, daaBlockHash *DomainHash, daaBlockWindowIndex uint64) (*TrustedDataDataDAAHeader, error)
|
||||
TrustedBlockAssociatedGHOSTDAGDataBlockHashes(blockHash *DomainHash) ([]*DomainHash, error)
|
||||
|
@ -49,7 +49,7 @@ type TestConsensus interface {
|
||||
*externalapi.VirtualChangeSet, error)
|
||||
UpdatePruningPointByVirtual() error
|
||||
|
||||
ResolveVirtualWithMaxParam(maxBlocksToResolve uint64) (bool, error)
|
||||
ResolveVirtualWithMaxParam(maxBlocksToResolve uint64) (*externalapi.VirtualChangeSet, bool, error)
|
||||
|
||||
MineJSON(r io.Reader, blockType MineJSONBlockType) (tips []*externalapi.DomainHash, err error)
|
||||
ToJSON(w io.Writer) error
|
||||
|
@ -5,22 +5,22 @@ import (
|
||||
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
|
||||
"github.com/kaspanet/kaspad/infrastructure/logger"
|
||||
"github.com/kaspanet/kaspad/util/staging"
|
||||
"github.com/pkg/errors"
|
||||
"sort"
|
||||
)
|
||||
|
||||
func (csm *consensusStateManager) ResolveVirtual(maxBlocksToResolve uint64) (*externalapi.VirtualChangeSet, bool, error) {
|
||||
onEnd := logger.LogAndMeasureExecutionTime(log, "csm.ResolveVirtual")
|
||||
defer onEnd()
|
||||
|
||||
readStagingArea := model.NewStagingArea()
|
||||
tips, err := csm.consensusStateStore.Tips(readStagingArea, csm.databaseContext)
|
||||
// tipsInDecreasingGHOSTDAGParentSelectionOrder returns the current DAG tips in decreasing parent selection order.
|
||||
// This means that the first tip in the resulting list would be the GHOSTDAG selected parent, and if removed from the list,
|
||||
// the second tip would be the selected parent, and so on.
|
||||
func (csm *consensusStateManager) tipsInDecreasingGHOSTDAGParentSelectionOrder(stagingArea *model.StagingArea) ([]*externalapi.DomainHash, error) {
|
||||
tips, err := csm.consensusStateStore.Tips(stagingArea, csm.databaseContext)
|
||||
if err != nil {
|
||||
return nil, false, err
|
||||
return nil, err
|
||||
}
|
||||
|
||||
var sortErr error
|
||||
sort.Slice(tips, func(i, j int) bool {
|
||||
selectedParent, err := csm.ghostdagManager.ChooseSelectedParent(readStagingArea, tips[i], tips[j])
|
||||
selectedParent, err := csm.ghostdagManager.ChooseSelectedParent(stagingArea, tips[i], tips[j])
|
||||
if err != nil {
|
||||
sortErr = err
|
||||
return false
|
||||
@ -29,16 +29,22 @@ func (csm *consensusStateManager) ResolveVirtual(maxBlocksToResolve uint64) (*ex
|
||||
return selectedParent.Equal(tips[i])
|
||||
})
|
||||
if sortErr != nil {
|
||||
return nil, false, sortErr
|
||||
return nil, sortErr
|
||||
}
|
||||
return tips, nil
|
||||
}
|
||||
|
||||
func (csm *consensusStateManager) findNextPendingTip(stagingArea *model.StagingArea) (*externalapi.DomainHash, externalapi.BlockStatus, error) {
|
||||
orderedTips, err := csm.tipsInDecreasingGHOSTDAGParentSelectionOrder(stagingArea)
|
||||
if err != nil {
|
||||
return nil, externalapi.StatusInvalid, err
|
||||
}
|
||||
|
||||
var selectedTip *externalapi.DomainHash
|
||||
isCompletelyResolved := true
|
||||
for _, tip := range tips {
|
||||
for _, tip := range orderedTips {
|
||||
log.Debugf("Resolving tip %s", tip)
|
||||
isViolatingFinality, shouldNotify, err := csm.isViolatingFinality(readStagingArea, tip)
|
||||
isViolatingFinality, shouldNotify, err := csm.isViolatingFinality(stagingArea, tip)
|
||||
if err != nil {
|
||||
return nil, false, err
|
||||
return nil, externalapi.StatusInvalid, err
|
||||
}
|
||||
|
||||
if isViolatingFinality {
|
||||
@ -49,55 +55,147 @@ func (csm *consensusStateManager) ResolveVirtual(maxBlocksToResolve uint64) (*ex
|
||||
continue
|
||||
}
|
||||
|
||||
resolveStagingArea := model.NewStagingArea()
|
||||
unverifiedBlocks, err := csm.getUnverifiedChainBlocks(resolveStagingArea, tip)
|
||||
status, err := csm.blockStatusStore.Get(csm.databaseContext, stagingArea, tip)
|
||||
if err != nil {
|
||||
return nil, false, err
|
||||
return nil, externalapi.StatusInvalid, err
|
||||
}
|
||||
|
||||
resolveTip := tip
|
||||
hasMoreUnverifiedThanMax := maxBlocksToResolve != 0 && uint64(len(unverifiedBlocks)) > maxBlocksToResolve
|
||||
if hasMoreUnverifiedThanMax {
|
||||
resolveTip = unverifiedBlocks[uint64(len(unverifiedBlocks))-maxBlocksToResolve]
|
||||
log.Debugf("Has more than %d blocks to resolve. Changing the resolve tip to %s", maxBlocksToResolve, resolveTip)
|
||||
}
|
||||
|
||||
blockStatus, reversalData, err := csm.resolveBlockStatus(resolveStagingArea, resolveTip, true)
|
||||
if err != nil {
|
||||
return nil, false, err
|
||||
}
|
||||
|
||||
if blockStatus == externalapi.StatusUTXOValid {
|
||||
selectedTip = resolveTip
|
||||
isCompletelyResolved = !hasMoreUnverifiedThanMax
|
||||
|
||||
err = staging.CommitAllChanges(csm.databaseContext, resolveStagingArea)
|
||||
if err != nil {
|
||||
return nil, false, err
|
||||
}
|
||||
|
||||
if reversalData != nil {
|
||||
err = csm.ReverseUTXODiffs(resolveTip, reversalData)
|
||||
if err != nil {
|
||||
return nil, false, err
|
||||
}
|
||||
}
|
||||
break
|
||||
if status == externalapi.StatusUTXOValid || status == externalapi.StatusUTXOPendingVerification {
|
||||
return tip, status, nil
|
||||
}
|
||||
}
|
||||
|
||||
if selectedTip == nil {
|
||||
log.Warnf("Non of the DAG tips are valid")
|
||||
return nil, true, nil
|
||||
return nil, externalapi.StatusInvalid, nil
|
||||
}
|
||||
|
||||
// getGHOSTDAGLowerTips returns the set of tips which are lower in GHOSTDAG parent selection order than `pendingTip`. i.e.,
|
||||
// they can be added to virtual parents but `pendingTip` will remain the virtual selected parent
|
||||
func (csm *consensusStateManager) getGHOSTDAGLowerTips(stagingArea *model.StagingArea, pendingTip *externalapi.DomainHash) ([]*externalapi.DomainHash, error) {
|
||||
tips, err := csm.consensusStateStore.Tips(stagingArea, csm.databaseContext)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
oldVirtualGHOSTDAGData, err := csm.ghostdagDataStore.Get(csm.databaseContext, readStagingArea, model.VirtualBlockHash, false)
|
||||
lowerTips := []*externalapi.DomainHash{pendingTip}
|
||||
for _, tip := range tips {
|
||||
if tip.Equal(pendingTip) {
|
||||
continue
|
||||
}
|
||||
selectedParent, err := csm.ghostdagManager.ChooseSelectedParent(stagingArea, tip, pendingTip)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if selectedParent.Equal(pendingTip) {
|
||||
lowerTips = append(lowerTips, tip)
|
||||
}
|
||||
}
|
||||
return lowerTips, nil
|
||||
}
|
||||
|
||||
func (csm *consensusStateManager) ResolveVirtual(maxBlocksToResolve uint64) (*externalapi.VirtualChangeSet, bool, error) {
|
||||
onEnd := logger.LogAndMeasureExecutionTime(log, "csm.ResolveVirtual")
|
||||
defer onEnd()
|
||||
|
||||
// We use a read-only staging area for some read-only actions, to avoid
|
||||
// confusion with the resolve/updateVirtual staging areas below
|
||||
readStagingArea := model.NewStagingArea()
|
||||
|
||||
pendingTip, pendingTipStatus, err := csm.findNextPendingTip(readStagingArea)
|
||||
if err != nil {
|
||||
return nil, false, err
|
||||
}
|
||||
|
||||
if pendingTip == nil {
|
||||
log.Warnf("None of the DAG tips are valid")
|
||||
return nil, true, nil
|
||||
}
|
||||
|
||||
previousVirtualSelectedParent, err := csm.virtualSelectedParent(readStagingArea)
|
||||
if err != nil {
|
||||
return nil, false, err
|
||||
}
|
||||
|
||||
if pendingTipStatus == externalapi.StatusUTXOValid && previousVirtualSelectedParent.Equal(pendingTip) {
|
||||
return nil, true, nil
|
||||
}
|
||||
|
||||
// Resolve a chunk from the pending chain
|
||||
resolveStagingArea := model.NewStagingArea()
|
||||
unverifiedBlocks, err := csm.getUnverifiedChainBlocks(resolveStagingArea, pendingTip)
|
||||
if err != nil {
|
||||
return nil, false, err
|
||||
}
|
||||
|
||||
// Initially set the resolve processing point to the pending tip
|
||||
processingPoint := pendingTip
|
||||
|
||||
// Too many blocks to verify, so we only process a chunk and return
|
||||
if maxBlocksToResolve != 0 && uint64(len(unverifiedBlocks)) > maxBlocksToResolve {
|
||||
processingPointIndex := uint64(len(unverifiedBlocks)) - maxBlocksToResolve
|
||||
processingPoint = unverifiedBlocks[processingPointIndex]
|
||||
isNewVirtualSelectedParent, err := csm.isNewSelectedTip(readStagingArea, processingPoint, previousVirtualSelectedParent)
|
||||
if err != nil {
|
||||
return nil, false, err
|
||||
}
|
||||
|
||||
// We must find a processing point which wins previous virtual selected parent
|
||||
// even if we process more than `maxBlocksToResolve` for that.
|
||||
// Otherwise, internal UTXO diff logic gets all messed up
|
||||
for !isNewVirtualSelectedParent {
|
||||
if processingPointIndex == 0 {
|
||||
return nil, false, errors.Errorf(
|
||||
"Expecting the pending tip %s to overcome the previous selected parent %s", pendingTip, previousVirtualSelectedParent)
|
||||
}
|
||||
processingPointIndex--
|
||||
processingPoint = unverifiedBlocks[processingPointIndex]
|
||||
isNewVirtualSelectedParent, err = csm.isNewSelectedTip(readStagingArea, processingPoint, previousVirtualSelectedParent)
|
||||
if err != nil {
|
||||
return nil, false, err
|
||||
}
|
||||
}
|
||||
log.Debugf("Has more than %d blocks to resolve. Setting the resolve processing point to %s", maxBlocksToResolve, processingPoint)
|
||||
}
|
||||
|
||||
processingPointStatus, reversalData, err := csm.resolveBlockStatus(
|
||||
resolveStagingArea, processingPoint, true)
|
||||
if err != nil {
|
||||
return nil, false, err
|
||||
}
|
||||
|
||||
if processingPointStatus == externalapi.StatusUTXOValid {
|
||||
err = staging.CommitAllChanges(csm.databaseContext, resolveStagingArea)
|
||||
if err != nil {
|
||||
return nil, false, err
|
||||
}
|
||||
|
||||
if reversalData != nil {
|
||||
err = csm.ReverseUTXODiffs(processingPoint, reversalData)
|
||||
if err != nil {
|
||||
return nil, false, err
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
isActualTip := processingPoint.Equal(pendingTip)
|
||||
isCompletelyResolved := isActualTip && processingPointStatus == externalapi.StatusUTXOValid
|
||||
|
||||
updateVirtualStagingArea := model.NewStagingArea()
|
||||
virtualUTXODiff, err := csm.updateVirtualWithParents(updateVirtualStagingArea, []*externalapi.DomainHash{selectedTip})
|
||||
|
||||
virtualParents := []*externalapi.DomainHash{processingPoint}
|
||||
// If `isCompletelyResolved`, set virtual correctly with all tips which have less blue work than pending
|
||||
if isCompletelyResolved {
|
||||
lowerTips, err := csm.getGHOSTDAGLowerTips(readStagingArea, pendingTip)
|
||||
if err != nil {
|
||||
return nil, false, err
|
||||
}
|
||||
log.Debugf("Picking virtual parents from relevant tips len: %d", len(lowerTips))
|
||||
|
||||
virtualParents, err = csm.pickVirtualParents(readStagingArea, lowerTips)
|
||||
if err != nil {
|
||||
return nil, false, err
|
||||
}
|
||||
log.Debugf("Picked virtual parents: %s", virtualParents)
|
||||
}
|
||||
virtualUTXODiff, err := csm.updateVirtualWithParents(updateVirtualStagingArea, virtualParents)
|
||||
if err != nil {
|
||||
return nil, false, err
|
||||
}
|
||||
@ -108,12 +206,12 @@ func (csm *consensusStateManager) ResolveVirtual(maxBlocksToResolve uint64) (*ex
|
||||
}
|
||||
|
||||
selectedParentChainChanges, err := csm.dagTraversalManager.
|
||||
CalculateChainPath(readStagingArea, oldVirtualGHOSTDAGData.SelectedParent(), selectedTip)
|
||||
CalculateChainPath(updateVirtualStagingArea, previousVirtualSelectedParent, processingPoint)
|
||||
if err != nil {
|
||||
return nil, false, err
|
||||
}
|
||||
|
||||
virtualParents, err := csm.dagTopologyManager.Parents(readStagingArea, model.VirtualBlockHash)
|
||||
virtualParentsOutcome, err := csm.dagTopologyManager.Parents(updateVirtualStagingArea, model.VirtualBlockHash)
|
||||
if err != nil {
|
||||
return nil, false, err
|
||||
}
|
||||
@ -121,6 +219,6 @@ func (csm *consensusStateManager) ResolveVirtual(maxBlocksToResolve uint64) (*ex
|
||||
return &externalapi.VirtualChangeSet{
|
||||
VirtualSelectedParentChainChanges: selectedParentChainChanges,
|
||||
VirtualUTXODiff: virtualUTXODiff,
|
||||
VirtualParents: virtualParents,
|
||||
VirtualParents: virtualParentsOutcome,
|
||||
}, isCompletelyResolved, nil
|
||||
}
|
||||
|
@ -233,7 +233,7 @@ func (csm *consensusStateManager) resolveSingleBlockStatus(stagingArea *model.St
|
||||
return externalapi.StatusUTXOValid, nil, nil
|
||||
}
|
||||
|
||||
oldSelectedTip, err := csm.selectedTip(stagingArea)
|
||||
oldSelectedTip, err := csm.virtualSelectedParent(stagingArea)
|
||||
if err != nil {
|
||||
return 0, nil, err
|
||||
}
|
||||
@ -298,7 +298,7 @@ func (csm *consensusStateManager) isNewSelectedTip(stagingArea *model.StagingAre
|
||||
return blockHash.Equal(newSelectedTip), nil
|
||||
}
|
||||
|
||||
func (csm *consensusStateManager) selectedTip(stagingArea *model.StagingArea) (*externalapi.DomainHash, error) {
|
||||
func (csm *consensusStateManager) virtualSelectedParent(stagingArea *model.StagingArea) (*externalapi.DomainHash, error) {
|
||||
virtualGHOSTDAGData, err := csm.ghostdagDataStore.Get(csm.databaseContext, stagingArea, model.VirtualBlockHash, false)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
|
@ -1,6 +1,9 @@
|
||||
package consensusstatemanager_test
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"github.com/kaspanet/kaspad/domain/consensus/model"
|
||||
"github.com/kaspanet/kaspad/domain/consensus/model/testapi"
|
||||
"github.com/kaspanet/kaspad/domain/consensus/utils/consensushashing"
|
||||
"testing"
|
||||
|
||||
@ -21,11 +24,14 @@ func TestAddBlockBetweenResolveVirtualCalls(t *testing.T) {
|
||||
}
|
||||
defer teardown(false)
|
||||
|
||||
hashes := []*externalapi.DomainHash{consensusConfig.GenesisHash}
|
||||
|
||||
// Create a chain of blocks
|
||||
const initialChainLength = 10
|
||||
previousBlockHash := consensusConfig.GenesisHash
|
||||
for i := 0; i < initialChainLength; i++ {
|
||||
previousBlockHash, _, err = tc.AddBlock([]*externalapi.DomainHash{previousBlockHash}, nil, nil)
|
||||
hashes = append(hashes, previousBlockHash)
|
||||
if err != nil {
|
||||
t.Fatalf("Error mining block no. %d in initial chain: %+v", i, err)
|
||||
}
|
||||
@ -40,6 +46,7 @@ func TestAddBlockBetweenResolveVirtualCalls(t *testing.T) {
|
||||
t.Fatalf("Error mining block no. %d in re-org chain: %+v", i, err)
|
||||
}
|
||||
previousBlockHash = consensushashing.BlockHash(previousBlock)
|
||||
hashes = append(hashes, previousBlockHash)
|
||||
|
||||
// Do not UTXO validate in order to resolve virtual later
|
||||
err = tc.ValidateAndInsertBlock(previousBlock, false)
|
||||
@ -49,7 +56,7 @@ func TestAddBlockBetweenResolveVirtualCalls(t *testing.T) {
|
||||
}
|
||||
|
||||
// Resolve one step
|
||||
_, err = tc.ResolveVirtualWithMaxParam(2)
|
||||
_, _, err = tc.ResolveVirtualWithMaxParam(2)
|
||||
if err != nil {
|
||||
t.Fatalf("Error resolving virtual in re-org chain: %+v", err)
|
||||
}
|
||||
@ -68,7 +75,7 @@ func TestAddBlockBetweenResolveVirtualCalls(t *testing.T) {
|
||||
}
|
||||
|
||||
// Resolve one more step
|
||||
isCompletelyResolved, err := tc.ResolveVirtualWithMaxParam(2)
|
||||
_, isCompletelyResolved, err := tc.ResolveVirtualWithMaxParam(2)
|
||||
if err != nil {
|
||||
t.Fatalf("Error resolving virtual in re-org chain: %+v", err)
|
||||
}
|
||||
@ -78,14 +85,17 @@ func TestAddBlockBetweenResolveVirtualCalls(t *testing.T) {
|
||||
if err != nil {
|
||||
t.Fatalf("Error mining block during virtual resolution of reorg: %+v", err)
|
||||
}
|
||||
hashes = append(hashes, consensushashing.BlockHash(blockTemplate.Block))
|
||||
|
||||
// Complete resolving virtual
|
||||
for !isCompletelyResolved {
|
||||
isCompletelyResolved, err = tc.ResolveVirtualWithMaxParam(2)
|
||||
_, isCompletelyResolved, err = tc.ResolveVirtualWithMaxParam(2)
|
||||
if err != nil {
|
||||
t.Fatalf("Error resolving virtual in re-org chain: %+v", err)
|
||||
}
|
||||
}
|
||||
|
||||
verifyUtxoDiffPaths(t, tc, hashes)
|
||||
})
|
||||
}
|
||||
|
||||
@ -100,11 +110,14 @@ func TestAddGenesisChildAfterOneResolveVirtualCall(t *testing.T) {
|
||||
}
|
||||
defer teardown(false)
|
||||
|
||||
hashes := []*externalapi.DomainHash{consensusConfig.GenesisHash}
|
||||
|
||||
// Create a chain of blocks
|
||||
const initialChainLength = 6
|
||||
previousBlockHash := consensusConfig.GenesisHash
|
||||
for i := 0; i < initialChainLength; i++ {
|
||||
previousBlockHash, _, err = tc.AddBlock([]*externalapi.DomainHash{previousBlockHash}, nil, nil)
|
||||
hashes = append(hashes, previousBlockHash)
|
||||
if err != nil {
|
||||
t.Fatalf("Error mining block no. %d in initial chain: %+v", i, err)
|
||||
}
|
||||
@ -119,6 +132,7 @@ func TestAddGenesisChildAfterOneResolveVirtualCall(t *testing.T) {
|
||||
t.Fatalf("Error mining block no. %d in re-org chain: %+v", i, err)
|
||||
}
|
||||
previousBlockHash = consensushashing.BlockHash(previousBlock)
|
||||
hashes = append(hashes, previousBlockHash)
|
||||
|
||||
// Do not UTXO validate in order to resolve virtual later
|
||||
err = tc.ValidateAndInsertBlock(previousBlock, false)
|
||||
@ -128,7 +142,7 @@ func TestAddGenesisChildAfterOneResolveVirtualCall(t *testing.T) {
|
||||
}
|
||||
|
||||
// Resolve one step
|
||||
isCompletelyResolved, err := tc.ResolveVirtualWithMaxParam(2)
|
||||
_, isCompletelyResolved, err := tc.ResolveVirtualWithMaxParam(2)
|
||||
if err != nil {
|
||||
t.Fatalf("Error resolving virtual in re-org chain: %+v", err)
|
||||
}
|
||||
@ -140,11 +154,13 @@ func TestAddGenesisChildAfterOneResolveVirtualCall(t *testing.T) {
|
||||
|
||||
// Complete resolving virtual
|
||||
for !isCompletelyResolved {
|
||||
isCompletelyResolved, err = tc.ResolveVirtualWithMaxParam(2)
|
||||
_, isCompletelyResolved, err = tc.ResolveVirtualWithMaxParam(2)
|
||||
if err != nil {
|
||||
t.Fatalf("Error resolving virtual in re-org chain: %+v", err)
|
||||
}
|
||||
}
|
||||
|
||||
verifyUtxoDiffPaths(t, tc, hashes)
|
||||
})
|
||||
}
|
||||
|
||||
@ -159,11 +175,14 @@ func TestAddGenesisChildAfterTwoResolveVirtualCalls(t *testing.T) {
|
||||
}
|
||||
defer teardown(false)
|
||||
|
||||
hashes := []*externalapi.DomainHash{consensusConfig.GenesisHash}
|
||||
|
||||
// Create a chain of blocks
|
||||
const initialChainLength = 6
|
||||
previousBlockHash := consensusConfig.GenesisHash
|
||||
for i := 0; i < initialChainLength; i++ {
|
||||
previousBlockHash, _, err = tc.AddBlock([]*externalapi.DomainHash{previousBlockHash}, nil, nil)
|
||||
hashes = append(hashes, previousBlockHash)
|
||||
if err != nil {
|
||||
t.Fatalf("Error mining block no. %d in initial chain: %+v", i, err)
|
||||
}
|
||||
@ -178,6 +197,7 @@ func TestAddGenesisChildAfterTwoResolveVirtualCalls(t *testing.T) {
|
||||
t.Fatalf("Error mining block no. %d in re-org chain: %+v", i, err)
|
||||
}
|
||||
previousBlockHash = consensushashing.BlockHash(previousBlock)
|
||||
hashes = append(hashes, previousBlockHash)
|
||||
|
||||
// Do not UTXO validate in order to resolve virtual later
|
||||
err = tc.ValidateAndInsertBlock(previousBlock, false)
|
||||
@ -187,13 +207,13 @@ func TestAddGenesisChildAfterTwoResolveVirtualCalls(t *testing.T) {
|
||||
}
|
||||
|
||||
// Resolve one step
|
||||
_, err = tc.ResolveVirtualWithMaxParam(2)
|
||||
_, _, err = tc.ResolveVirtualWithMaxParam(2)
|
||||
if err != nil {
|
||||
t.Fatalf("Error resolving virtual in re-org chain: %+v", err)
|
||||
}
|
||||
|
||||
// Resolve one more step
|
||||
isCompletelyResolved, err := tc.ResolveVirtualWithMaxParam(2)
|
||||
_, isCompletelyResolved, err := tc.ResolveVirtualWithMaxParam(2)
|
||||
if err != nil {
|
||||
t.Fatalf("Error resolving virtual in re-org chain: %+v", err)
|
||||
}
|
||||
@ -205,10 +225,220 @@ func TestAddGenesisChildAfterTwoResolveVirtualCalls(t *testing.T) {
|
||||
|
||||
// Complete resolving virtual
|
||||
for !isCompletelyResolved {
|
||||
isCompletelyResolved, err = tc.ResolveVirtualWithMaxParam(2)
|
||||
_, isCompletelyResolved, err = tc.ResolveVirtualWithMaxParam(2)
|
||||
if err != nil {
|
||||
t.Fatalf("Error resolving virtual in re-org chain: %+v", err)
|
||||
}
|
||||
}
|
||||
|
||||
verifyUtxoDiffPaths(t, tc, hashes)
|
||||
})
|
||||
}
|
||||
|
||||
func TestResolveVirtualBackAndForthReorgs(t *testing.T) {
|
||||
|
||||
testutils.ForAllNets(t, true, func(t *testing.T, consensusConfig *consensus.Config) {
|
||||
factory := consensus.NewFactory()
|
||||
|
||||
tc, teardown, err := factory.NewTestConsensus(consensusConfig, "TestAddGenesisChildAfterTwoResolveVirtualCalls")
|
||||
if err != nil {
|
||||
t.Fatalf("Error setting up consensus: %+v", err)
|
||||
}
|
||||
defer teardown(false)
|
||||
|
||||
hashes := []*externalapi.DomainHash{consensusConfig.GenesisHash}
|
||||
blocks := make(map[externalapi.DomainHash]string)
|
||||
blocks[*consensusConfig.GenesisHash] = "g"
|
||||
blocks[*model.VirtualBlockHash] = "v"
|
||||
printfDebug("%s\n\n", consensusConfig.GenesisHash)
|
||||
|
||||
// Create a chain of blocks
|
||||
const initialChainLength = 6
|
||||
previousBlockHash := consensusConfig.GenesisHash
|
||||
for i := 0; i < initialChainLength; i++ {
|
||||
previousBlockHash, _, err = tc.AddBlock([]*externalapi.DomainHash{previousBlockHash}, nil, nil)
|
||||
blocks[*previousBlockHash] = fmt.Sprintf("A_%d", i)
|
||||
hashes = append(hashes, previousBlockHash)
|
||||
printfDebug("A_%d: %s\n", i, previousBlockHash)
|
||||
|
||||
if err != nil {
|
||||
t.Fatalf("Error mining block no. %d in initial chain: %+v", i, err)
|
||||
}
|
||||
}
|
||||
|
||||
printfDebug("\n")
|
||||
verifyUtxoDiffPaths(t, tc, hashes)
|
||||
|
||||
firstChainTip := previousBlockHash
|
||||
|
||||
// Mine a chain with more blocks, to re-organize the DAG
|
||||
const reorgChainLength = 12 // initialChainLength + 1
|
||||
previousBlockHash = consensusConfig.GenesisHash
|
||||
for i := 0; i < reorgChainLength; i++ {
|
||||
previousBlock, _, err := tc.BuildBlockWithParents([]*externalapi.DomainHash{previousBlockHash}, nil, nil)
|
||||
if err != nil {
|
||||
t.Fatalf("Error mining block no. %d in re-org chain: %+v", i, err)
|
||||
}
|
||||
previousBlockHash = consensushashing.BlockHash(previousBlock)
|
||||
blocks[*previousBlockHash] = fmt.Sprintf("B_%d", i)
|
||||
hashes = append(hashes, previousBlockHash)
|
||||
printfDebug("B_%d: %s\n", i, previousBlockHash)
|
||||
|
||||
// Do not UTXO validate in order to resolve virtual later
|
||||
err = tc.ValidateAndInsertBlock(previousBlock, false)
|
||||
if err != nil {
|
||||
t.Fatalf("Error mining block no. %d in re-org chain: %+v", i, err)
|
||||
}
|
||||
}
|
||||
|
||||
printfDebug("\n")
|
||||
|
||||
printUtxoDiffChildren(t, tc, hashes, blocks)
|
||||
verifyUtxoDiffPaths(t, tc, hashes)
|
||||
|
||||
previousVirtualSelectedParent, err := tc.GetVirtualSelectedParent()
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
// Resolve one step
|
||||
virtualChangeSet, _, err := tc.ResolveVirtualWithMaxParam(3)
|
||||
if err != nil {
|
||||
printUtxoDiffChildren(t, tc, hashes, blocks)
|
||||
t.Fatalf("Error resolving virtual in re-org chain: %+v", err)
|
||||
}
|
||||
|
||||
newVirtualSelectedParent, err := tc.GetVirtualSelectedParent()
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
// Make sure the reported change-set is compatible with actual changes.
|
||||
// Checking this for one call should suffice to avoid possible bugs.
|
||||
reportedPreviousVirtualSelectedParent := virtualChangeSet.VirtualSelectedParentChainChanges.Removed[0]
|
||||
reportedNewVirtualSelectedParent := virtualChangeSet.VirtualSelectedParentChainChanges.
|
||||
Added[len(virtualChangeSet.VirtualSelectedParentChainChanges.Added)-1]
|
||||
|
||||
if !previousVirtualSelectedParent.Equal(reportedPreviousVirtualSelectedParent) {
|
||||
t.Fatalf("The reported changeset is incompatible with actual changes")
|
||||
}
|
||||
if !newVirtualSelectedParent.Equal(reportedNewVirtualSelectedParent) {
|
||||
t.Fatalf("The reported changeset is incompatible with actual changes")
|
||||
}
|
||||
|
||||
// Resolve one more step
|
||||
_, isCompletelyResolved, err := tc.ResolveVirtualWithMaxParam(3)
|
||||
if err != nil {
|
||||
t.Fatalf("Error resolving virtual in re-org chain: %+v", err)
|
||||
}
|
||||
|
||||
// Complete resolving virtual
|
||||
for !isCompletelyResolved {
|
||||
_, isCompletelyResolved, err = tc.ResolveVirtualWithMaxParam(3)
|
||||
if err != nil {
|
||||
t.Fatalf("Error resolving virtual in re-org chain: %+v", err)
|
||||
}
|
||||
}
|
||||
|
||||
printUtxoDiffChildren(t, tc, hashes, blocks)
|
||||
verifyUtxoDiffPaths(t, tc, hashes)
|
||||
|
||||
// Now get the first chain back to the wining position
|
||||
previousBlockHash = firstChainTip
|
||||
for i := 0; i < reorgChainLength; i++ {
|
||||
previousBlockHash, _, err = tc.AddBlock([]*externalapi.DomainHash{previousBlockHash}, nil, nil)
|
||||
blocks[*previousBlockHash] = fmt.Sprintf("A_%d", initialChainLength+i)
|
||||
hashes = append(hashes, previousBlockHash)
|
||||
printfDebug("A_%d: %s\n", initialChainLength+i, previousBlockHash)
|
||||
|
||||
if err != nil {
|
||||
t.Fatalf("Error mining block no. %d in initial chain: %+v", initialChainLength+i, err)
|
||||
}
|
||||
}
|
||||
|
||||
printfDebug("\n")
|
||||
|
||||
printUtxoDiffChildren(t, tc, hashes, blocks)
|
||||
verifyUtxoDiffPaths(t, tc, hashes)
|
||||
})
|
||||
}
|
||||
|
||||
func verifyUtxoDiffPathToRoot(t *testing.T, tc testapi.TestConsensus, stagingArea *model.StagingArea, block, utxoDiffRoot *externalapi.DomainHash) {
|
||||
current := block
|
||||
for !current.Equal(utxoDiffRoot) {
|
||||
hasUTXODiffChild, err := tc.UTXODiffStore().HasUTXODiffChild(tc.DatabaseContext(), stagingArea, current)
|
||||
if err != nil {
|
||||
t.Fatalf("Error while reading utxo diff store: %+v", err)
|
||||
}
|
||||
if !hasUTXODiffChild {
|
||||
t.Fatalf("%s is expected to have a UTXO diff child", current)
|
||||
}
|
||||
current, err = tc.UTXODiffStore().UTXODiffChild(tc.DatabaseContext(), stagingArea, current)
|
||||
if err != nil {
|
||||
t.Fatalf("Error while reading utxo diff store: %+v", err)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func verifyUtxoDiffPaths(t *testing.T, tc testapi.TestConsensus, hashes []*externalapi.DomainHash) {
|
||||
stagingArea := model.NewStagingArea()
|
||||
|
||||
virtualGHOSTDAGData, err := tc.GHOSTDAGDataStore().Get(tc.DatabaseContext(), stagingArea, model.VirtualBlockHash, false)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
utxoDiffRoot := virtualGHOSTDAGData.SelectedParent()
|
||||
hasUTXODiffChild, err := tc.UTXODiffStore().HasUTXODiffChild(tc.DatabaseContext(), stagingArea, utxoDiffRoot)
|
||||
if err != nil {
|
||||
t.Fatalf("Error while reading utxo diff store: %+v", err)
|
||||
}
|
||||
if hasUTXODiffChild {
|
||||
t.Fatalf("Virtual selected parent is not expected to have an explicit diff child")
|
||||
}
|
||||
_, err = tc.UTXODiffStore().UTXODiff(tc.DatabaseContext(), stagingArea, utxoDiffRoot)
|
||||
if err != nil {
|
||||
t.Fatalf("Virtual selected parent is expected to have a utxo diff: %+v", err)
|
||||
}
|
||||
|
||||
for _, block := range hashes {
|
||||
hasUTXODiffChild, err = tc.UTXODiffStore().HasUTXODiffChild(tc.DatabaseContext(), stagingArea, block)
|
||||
if err != nil {
|
||||
t.Fatalf("Error while reading utxo diff store: %+v", err)
|
||||
}
|
||||
isOnVirtualSelectedChain, err := tc.DAGTopologyManager().IsInSelectedParentChainOf(stagingArea, block, utxoDiffRoot)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
// We expect a valid path to root in both cases: (i) block has a diff child, (ii) block is on the virtual selected chain
|
||||
if hasUTXODiffChild || isOnVirtualSelectedChain {
|
||||
verifyUtxoDiffPathToRoot(t, tc, stagingArea, block, utxoDiffRoot)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func printfDebug(format string, a ...any) {
|
||||
// Uncomment below when debugging the test
|
||||
//fmt.Printf(format, a...)
|
||||
}
|
||||
|
||||
func printUtxoDiffChildren(t *testing.T, tc testapi.TestConsensus, hashes []*externalapi.DomainHash, blocks map[externalapi.DomainHash]string) {
|
||||
printfDebug("\n===============================\nBlock\t\tDiff child\n")
|
||||
stagingArea := model.NewStagingArea()
|
||||
for _, block := range hashes {
|
||||
hasUTXODiffChild, err := tc.UTXODiffStore().HasUTXODiffChild(tc.DatabaseContext(), stagingArea, block)
|
||||
if err != nil {
|
||||
t.Fatalf("Error while reading utxo diff store: %+v", err)
|
||||
}
|
||||
if hasUTXODiffChild {
|
||||
utxoDiffChild, err := tc.UTXODiffStore().UTXODiffChild(tc.DatabaseContext(), stagingArea, block)
|
||||
if err != nil {
|
||||
t.Fatalf("Error while reading utxo diff store: %+v", err)
|
||||
}
|
||||
printfDebug("%s\t\t\t%s\n", blocks[*block], blocks[*utxoDiffChild])
|
||||
} else {
|
||||
printfDebug("%s\n", blocks[*block])
|
||||
}
|
||||
}
|
||||
printfDebug("\n===============================\n")
|
||||
}
|
||||
|
@ -56,12 +56,6 @@ func (csm *consensusStateManager) ReverseUTXODiffs(tipHash *externalapi.DomainHa
|
||||
return err
|
||||
}
|
||||
|
||||
// We stop reversing when current's UTXODiffChild is not current's SelectedParent
|
||||
if !currentBlockGHOSTDAGData.SelectedParent().Equal(currentBlockUTXODiffChild) {
|
||||
log.Debugf("Block %s's UTXODiffChild is not it's selected parent - finish reversing", currentBlock)
|
||||
break
|
||||
}
|
||||
|
||||
currentUTXODiff := previousUTXODiff.Reversed()
|
||||
|
||||
// retrieve current utxoDiff for Bi, to be used by next block
|
||||
@ -75,6 +69,12 @@ func (csm *consensusStateManager) ReverseUTXODiffs(tipHash *externalapi.DomainHa
|
||||
return err
|
||||
}
|
||||
|
||||
// We stop reversing when current's UTXODiffChild is not current's SelectedParent
|
||||
if !currentBlockGHOSTDAGData.SelectedParent().Equal(currentBlockUTXODiffChild) {
|
||||
log.Debugf("Block %s's UTXODiffChild is not it's selected parent - finish reversing", currentBlock)
|
||||
break
|
||||
}
|
||||
|
||||
previousBlock = currentBlock
|
||||
previousBlockGHOSTDAGData = currentBlockGHOSTDAGData
|
||||
|
||||
|
@ -110,7 +110,7 @@ func (csm *consensusStateManager) updateSelectedTipUTXODiff(
|
||||
onEnd := logger.LogAndMeasureExecutionTime(log, "updateSelectedTipUTXODiff")
|
||||
defer onEnd()
|
||||
|
||||
selectedTip, err := csm.selectedTip(stagingArea)
|
||||
selectedTip, err := csm.virtualSelectedParent(stagingArea)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -112,11 +112,11 @@ func (tc *testConsensus) AddUTXOInvalidBlock(parentHashes []*externalapi.DomainH
|
||||
return consensushashing.BlockHash(block), virtualChangeSet, nil
|
||||
}
|
||||
|
||||
func (tc *testConsensus) ResolveVirtualWithMaxParam(maxBlocksToResolve uint64) (bool, error) {
|
||||
func (tc *testConsensus) ResolveVirtualWithMaxParam(maxBlocksToResolve uint64) (*externalapi.VirtualChangeSet, bool, error) {
|
||||
tc.lock.Lock()
|
||||
defer tc.lock.Unlock()
|
||||
|
||||
return tc.resolveVirtualNoLock(maxBlocksToResolve)
|
||||
return tc.resolveVirtualChunkNoLock(maxBlocksToResolve)
|
||||
}
|
||||
|
||||
// jsonBlock is a json representation of a block in mine format
|
||||
|
@ -214,34 +214,19 @@ func syncConsensuses(syncer, syncee externalapi.Consensus) error {
|
||||
return err
|
||||
}
|
||||
|
||||
virtualDAAScoreStart, err := syncee.GetVirtualDAAScore()
|
||||
err = syncer.ResolveVirtual(func(virtualDAAScoreStart uint64, virtualDAAScore uint64) {
|
||||
if estimatedVirtualDAAScoreTarget-virtualDAAScoreStart <= 0 {
|
||||
percents = 100
|
||||
} else {
|
||||
percents = int(float64(virtualDAAScore-virtualDAAScoreStart) / float64(estimatedVirtualDAAScoreTarget-virtualDAAScoreStart) * 100)
|
||||
}
|
||||
log.Infof("Resolving virtual. Estimated progress: %d%%", percents)
|
||||
})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
percents = 0
|
||||
for i := 0; ; i++ {
|
||||
if i%10 == 0 {
|
||||
virtualDAAScore, err := syncee.GetVirtualDAAScore()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
newPercents := int(float64(virtualDAAScore-virtualDAAScoreStart) / float64(estimatedVirtualDAAScoreTarget-virtualDAAScoreStart) * 100)
|
||||
if newPercents > percents {
|
||||
percents = newPercents
|
||||
log.Infof("Resolving virtual. Estimated progress: %d%%", percents)
|
||||
}
|
||||
}
|
||||
isCompletelyResolved, err := syncee.ResolveVirtual()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if isCompletelyResolved {
|
||||
log.Infof("Resolved virtual")
|
||||
break
|
||||
}
|
||||
}
|
||||
log.Infof("Resolved virtual")
|
||||
|
||||
return nil
|
||||
}
|
||||
|
Loading…
x
Reference in New Issue
Block a user