mirror of
https://github.com/kaspanet/kaspad.git
synced 2025-06-06 14:16:43 +00:00
Fix UTXO diff child error (#2084)
* Avoid creating the chain iterator if high hash is actually low hash * Always use iterator in nextPruningPointAndCandidateByBlockHash * Initial failing test * Minimal failing test + some comments * go lint * Add simpler tests with two different errors * Missed some error checks * Minor * A workaround patch for preventing the missing utxo child diff bug * Make sure we fully resolve virtual * Move ResolveVirtualWithMaxParam to test consensus * Mark virtual not updated and loop in batches * Refactor: remove VirtualChangeSet from functions return values * Remove workaround comments * If block has no body, virtual is still considered updated * Remove special error ErrReverseUTXODiffsUTXODiffChildNotFound Co-authored-by: Ori Newman <orinewman1@gmail.com>
This commit is contained in:
parent
b2648aa5bd
commit
d957a6d93a
@ -107,7 +107,7 @@ func (f *FlowContext) AddBlock(block *externalapi.DomainBlock) error {
|
|||||||
return protocolerrors.Errorf(false, "cannot add header only block")
|
return protocolerrors.Errorf(false, "cannot add header only block")
|
||||||
}
|
}
|
||||||
|
|
||||||
_, err := f.Domain().Consensus().ValidateAndInsertBlock(block, true)
|
err := f.Domain().Consensus().ValidateAndInsertBlock(block, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
if errors.As(err, &ruleerrors.RuleError{}) {
|
if errors.As(err, &ruleerrors.RuleError{}) {
|
||||||
log.Warnf("Validation failed for block %s: %s", consensushashing.BlockHash(block), err)
|
log.Warnf("Validation failed for block %s: %s", consensushashing.BlockHash(block), err)
|
||||||
|
@ -141,7 +141,7 @@ func (f *FlowContext) unorphanBlock(orphanHash externalapi.DomainHash) (bool, er
|
|||||||
}
|
}
|
||||||
delete(f.orphans, orphanHash)
|
delete(f.orphans, orphanHash)
|
||||||
|
|
||||||
_, err := f.domain.Consensus().ValidateAndInsertBlock(orphanBlock, true)
|
err := f.domain.Consensus().ValidateAndInsertBlock(orphanBlock, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
if errors.As(err, &ruleerrors.RuleError{}) {
|
if errors.As(err, &ruleerrors.RuleError{}) {
|
||||||
log.Warnf("Validation failed for orphan block %s: %s", orphanHash, err)
|
log.Warnf("Validation failed for orphan block %s: %s", orphanHash, err)
|
||||||
|
@ -321,7 +321,7 @@ func (flow *handleRelayInvsFlow) readMsgBlock() (msgBlock *appmessage.MsgBlock,
|
|||||||
|
|
||||||
func (flow *handleRelayInvsFlow) processBlock(block *externalapi.DomainBlock) ([]*externalapi.DomainHash, error) {
|
func (flow *handleRelayInvsFlow) processBlock(block *externalapi.DomainBlock) ([]*externalapi.DomainHash, error) {
|
||||||
blockHash := consensushashing.BlockHash(block)
|
blockHash := consensushashing.BlockHash(block)
|
||||||
_, err := flow.Domain().Consensus().ValidateAndInsertBlock(block, true)
|
err := flow.Domain().Consensus().ValidateAndInsertBlock(block, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
if !errors.As(err, &ruleerrors.RuleError{}) {
|
if !errors.As(err, &ruleerrors.RuleError{}) {
|
||||||
return nil, errors.Wrapf(err, "failed to process block %s", blockHash)
|
return nil, errors.Wrapf(err, "failed to process block %s", blockHash)
|
||||||
|
@ -488,7 +488,7 @@ func (flow *handleIBDFlow) processHeader(consensus externalapi.Consensus, msgBlo
|
|||||||
log.Debugf("Block header %s is already in the DAG. Skipping...", blockHash)
|
log.Debugf("Block header %s is already in the DAG. Skipping...", blockHash)
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
_, err = consensus.ValidateAndInsertBlock(block, false)
|
err = consensus.ValidateAndInsertBlock(block, false)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
if !errors.As(err, &ruleerrors.RuleError{}) {
|
if !errors.As(err, &ruleerrors.RuleError{}) {
|
||||||
return errors.Wrapf(err, "failed to process header %s during IBD", blockHash)
|
return errors.Wrapf(err, "failed to process header %s during IBD", blockHash)
|
||||||
@ -654,7 +654,7 @@ func (flow *handleIBDFlow) syncMissingBlockBodies(highHash *externalapi.DomainHa
|
|||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
_, err = flow.Domain().Consensus().ValidateAndInsertBlock(block, false)
|
err = flow.Domain().Consensus().ValidateAndInsertBlock(block, false)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
if errors.Is(err, ruleerrors.ErrDuplicateBlock) {
|
if errors.Is(err, ruleerrors.ErrDuplicateBlock) {
|
||||||
log.Debugf("Skipping IBD Block %s as it has already been added to the DAG", blockHash)
|
log.Debugf("Skipping IBD Block %s as it has already been added to the DAG", blockHash)
|
||||||
@ -705,7 +705,7 @@ func (flow *handleIBDFlow) resolveVirtual(estimatedVirtualDAAScoreTarget uint64)
|
|||||||
}
|
}
|
||||||
log.Infof("Resolving virtual. Estimated progress: %d%%", percents)
|
log.Infof("Resolving virtual. Estimated progress: %d%%", percents)
|
||||||
}
|
}
|
||||||
_, isCompletelyResolved, err := flow.Domain().Consensus().ResolveVirtual()
|
isCompletelyResolved, err := flow.Domain().Consensus().ResolveVirtual()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
@ -280,7 +280,7 @@ func (flow *handleIBDFlow) processBlockWithTrustedData(
|
|||||||
blockWithTrustedData.GHOSTDAGData = append(blockWithTrustedData.GHOSTDAGData, appmessage.GHOSTDAGHashPairToDomainGHOSTDAGHashPair(data.GHOSTDAGData[index]))
|
blockWithTrustedData.GHOSTDAGData = append(blockWithTrustedData.GHOSTDAGData, appmessage.GHOSTDAGHashPairToDomainGHOSTDAGHashPair(data.GHOSTDAGData[index]))
|
||||||
}
|
}
|
||||||
|
|
||||||
_, err := consensus.ValidateAndInsertBlockWithTrustedData(blockWithTrustedData, false)
|
err := consensus.ValidateAndInsertBlockWithTrustedData(blockWithTrustedData, false)
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -61,17 +61,18 @@ type consensus struct {
|
|||||||
blocksWithTrustedDataDAAWindowStore model.BlocksWithTrustedDataDAAWindowStore
|
blocksWithTrustedDataDAAWindowStore model.BlocksWithTrustedDataDAAWindowStore
|
||||||
|
|
||||||
consensusEventsChan chan externalapi.ConsensusEvent
|
consensusEventsChan chan externalapi.ConsensusEvent
|
||||||
|
virtualNotUpdated bool
|
||||||
}
|
}
|
||||||
|
|
||||||
func (s *consensus) ValidateAndInsertBlockWithTrustedData(block *externalapi.BlockWithTrustedData, validateUTXO bool) (*externalapi.VirtualChangeSet, error) {
|
func (s *consensus) ValidateAndInsertBlockWithTrustedData(block *externalapi.BlockWithTrustedData, validateUTXO bool) error {
|
||||||
s.lock.Lock()
|
s.lock.Lock()
|
||||||
defer s.lock.Unlock()
|
defer s.lock.Unlock()
|
||||||
|
|
||||||
virtualChangeSet, _, err := s.blockProcessor.ValidateAndInsertBlockWithTrustedData(block, validateUTXO)
|
_, _, err := s.blockProcessor.ValidateAndInsertBlockWithTrustedData(block, validateUTXO)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, err
|
return err
|
||||||
}
|
}
|
||||||
return virtualChangeSet, nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// Init initializes consensus
|
// Init initializes consensus
|
||||||
@ -193,21 +194,47 @@ func (s *consensus) BuildBlockTemplate(coinbaseData *externalapi.DomainCoinbaseD
|
|||||||
|
|
||||||
// ValidateAndInsertBlock validates the given block and, if valid, applies it
|
// ValidateAndInsertBlock validates the given block and, if valid, applies it
|
||||||
// to the current state
|
// to the current state
|
||||||
func (s *consensus) ValidateAndInsertBlock(block *externalapi.DomainBlock, shouldValidateAgainstUTXO bool) (*externalapi.VirtualChangeSet, error) {
|
func (s *consensus) ValidateAndInsertBlock(block *externalapi.DomainBlock, shouldValidateAgainstUTXO bool) error {
|
||||||
s.lock.Lock()
|
s.lock.Lock()
|
||||||
defer s.lock.Unlock()
|
defer s.lock.Unlock()
|
||||||
|
|
||||||
virtualChangeSet, blockStatus, err := s.blockProcessor.ValidateAndInsertBlock(block, shouldValidateAgainstUTXO)
|
_, err := s.validateAndInsertBlockNoLock(block, shouldValidateAgainstUTXO)
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
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 {
|
if err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// If block has a body, and yet virtual was not updated -- signify that virtual is in non-updated state
|
||||||
|
if !updateVirtual && blockStatus != externalapi.StatusHeaderOnly {
|
||||||
|
s.virtualNotUpdated = true
|
||||||
|
}
|
||||||
|
|
||||||
err = s.sendBlockAddedEvent(block, blockStatus)
|
err = s.sendBlockAddedEvent(block, blockStatus)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
err = s.sendVirtualChangedEvent(virtualChangeSet, shouldValidateAgainstUTXO)
|
err = s.sendVirtualChangedEvent(virtualChangeSet, updateVirtual)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
@ -861,7 +888,7 @@ func (s *consensus) PopulateMass(transaction *externalapi.DomainTransaction) {
|
|||||||
s.transactionValidator.PopulateMass(transaction)
|
s.transactionValidator.PopulateMass(transaction)
|
||||||
}
|
}
|
||||||
|
|
||||||
func (s *consensus) ResolveVirtual() (*externalapi.VirtualChangeSet, bool, error) {
|
func (s *consensus) ResolveVirtual() (bool, error) {
|
||||||
s.lock.Lock()
|
s.lock.Lock()
|
||||||
defer s.lock.Unlock()
|
defer s.lock.Unlock()
|
||||||
|
|
||||||
@ -869,28 +896,33 @@ func (s *consensus) ResolveVirtual() (*externalapi.VirtualChangeSet, bool, error
|
|||||||
// release the lock each time resolve 100 blocks.
|
// release the lock each time resolve 100 blocks.
|
||||||
// Note: maxBlocksToResolve should be smaller than finality interval in order to avoid a situation
|
// Note: maxBlocksToResolve should be smaller than finality interval in order to avoid a situation
|
||||||
// where UpdatePruningPointByVirtual skips a pruning point.
|
// where UpdatePruningPointByVirtual skips a pruning point.
|
||||||
virtualChangeSet, isCompletelyResolved, err := s.consensusStateManager.ResolveVirtual(100)
|
return s.resolveVirtualNoLock(100)
|
||||||
|
}
|
||||||
|
|
||||||
|
func (s *consensus) resolveVirtualNoLock(maxBlocksToResolve uint64) (bool, error) {
|
||||||
|
virtualChangeSet, isCompletelyResolved, err := s.consensusStateManager.ResolveVirtual(maxBlocksToResolve)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, false, err
|
return false, err
|
||||||
}
|
}
|
||||||
|
s.virtualNotUpdated = !isCompletelyResolved
|
||||||
|
|
||||||
stagingArea := model.NewStagingArea()
|
stagingArea := model.NewStagingArea()
|
||||||
err = s.pruningManager.UpdatePruningPointByVirtual(stagingArea)
|
err = s.pruningManager.UpdatePruningPointByVirtual(stagingArea)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, false, err
|
return false, err
|
||||||
}
|
}
|
||||||
|
|
||||||
err = staging.CommitAllChanges(s.databaseContext, stagingArea)
|
err = staging.CommitAllChanges(s.databaseContext, stagingArea)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, false, err
|
return false, err
|
||||||
}
|
}
|
||||||
|
|
||||||
err = s.sendVirtualChangedEvent(virtualChangeSet, true)
|
err = s.sendVirtualChangedEvent(virtualChangeSet, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, false, err
|
return false, err
|
||||||
}
|
}
|
||||||
|
|
||||||
return virtualChangeSet, isCompletelyResolved, nil
|
return isCompletelyResolved, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (s *consensus) BuildPruningPointProof() (*externalapi.PruningPointProof, error) {
|
func (s *consensus) BuildPruningPointProof() (*externalapi.PruningPointProof, error) {
|
||||||
|
@ -27,7 +27,7 @@ func TestConsensus_GetBlockInfo(t *testing.T) {
|
|||||||
newHeader := invalidBlock.Header.ToMutable()
|
newHeader := invalidBlock.Header.ToMutable()
|
||||||
newHeader.SetTimeInMilliseconds(0)
|
newHeader.SetTimeInMilliseconds(0)
|
||||||
invalidBlock.Header = newHeader.ToImmutable()
|
invalidBlock.Header = newHeader.ToImmutable()
|
||||||
_, err = consensus.ValidateAndInsertBlock(invalidBlock, true)
|
err = consensus.ValidateAndInsertBlock(invalidBlock, true)
|
||||||
if !errors.Is(err, ruleerrors.ErrTimeTooOld) {
|
if !errors.Is(err, ruleerrors.ErrTimeTooOld) {
|
||||||
t.Fatalf("Expected block to be invalid with err: %v, instead found: %v", ruleerrors.ErrTimeTooOld, err)
|
t.Fatalf("Expected block to be invalid with err: %v, instead found: %v", ruleerrors.ErrTimeTooOld, err)
|
||||||
}
|
}
|
||||||
@ -55,7 +55,7 @@ func TestConsensus_GetBlockInfo(t *testing.T) {
|
|||||||
t.Fatalf("consensus.BuildBlock with an empty coinbase shouldn't fail: %v", err)
|
t.Fatalf("consensus.BuildBlock with an empty coinbase shouldn't fail: %v", err)
|
||||||
}
|
}
|
||||||
|
|
||||||
_, err = consensus.ValidateAndInsertBlock(validBlock, true)
|
err = consensus.ValidateAndInsertBlock(validBlock, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("consensus.ValidateAndInsertBlock with a block straight from consensus.BuildBlock should not fail: %v", err)
|
t.Fatalf("consensus.ValidateAndInsertBlock with a block straight from consensus.BuildBlock should not fail: %v", err)
|
||||||
}
|
}
|
||||||
|
@ -33,7 +33,7 @@ func TestFinality(t *testing.T) {
|
|||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
_, err = consensus.ValidateAndInsertBlock(block, true)
|
err = consensus.ValidateAndInsertBlock(block, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
@ -201,7 +201,7 @@ func TestBoundedMergeDepth(t *testing.T) {
|
|||||||
return nil, false // fo some reason go doesn't recognize that t.Fatalf never returns
|
return nil, false // fo some reason go doesn't recognize that t.Fatalf never returns
|
||||||
}
|
}
|
||||||
|
|
||||||
_, err = consensus.ValidateAndInsertBlock(block, true)
|
err = consensus.ValidateAndInsertBlock(block, true)
|
||||||
if err == nil {
|
if err == nil {
|
||||||
return block, false
|
return block, false
|
||||||
} else if errors.Is(err, ruleerrors.ErrViolatingBoundedMergeDepth) {
|
} else if errors.Is(err, ruleerrors.ErrViolatingBoundedMergeDepth) {
|
||||||
@ -213,7 +213,7 @@ func TestBoundedMergeDepth(t *testing.T) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
processBlock := func(consensus testapi.TestConsensus, block *externalapi.DomainBlock, name string) {
|
processBlock := func(consensus testapi.TestConsensus, block *externalapi.DomainBlock, name string) {
|
||||||
_, err := consensus.ValidateAndInsertBlock(block, true)
|
err := consensus.ValidateAndInsertBlock(block, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("TestBoundedMergeDepth: %s got unexpected error from ProcessBlock: %+v", name, err)
|
t.Fatalf("TestBoundedMergeDepth: %s got unexpected error from ProcessBlock: %+v", name, err)
|
||||||
|
|
||||||
@ -225,7 +225,7 @@ func TestBoundedMergeDepth(t *testing.T) {
|
|||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("TestBoundedMergeDepth: Failed building block: %+v", err)
|
t.Fatalf("TestBoundedMergeDepth: Failed building block: %+v", err)
|
||||||
}
|
}
|
||||||
_, err = consensus.ValidateAndInsertBlock(block, true)
|
err = consensus.ValidateAndInsertBlock(block, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("TestBoundedMergeDepth: Failed Inserting block to consensus: %v", err)
|
t.Fatalf("TestBoundedMergeDepth: Failed Inserting block to consensus: %v", err)
|
||||||
}
|
}
|
||||||
@ -268,7 +268,7 @@ func TestBoundedMergeDepth(t *testing.T) {
|
|||||||
t.Fatalf("GetBlockHeader: %+v", err)
|
t.Fatalf("GetBlockHeader: %+v", err)
|
||||||
}
|
}
|
||||||
|
|
||||||
_, err = tcSyncee.ValidateAndInsertBlock(block, true)
|
err = tcSyncee.ValidateAndInsertBlock(block, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("ValidateAndInsertBlock %d: %+v", i, err)
|
t.Fatalf("ValidateAndInsertBlock %d: %+v", i, err)
|
||||||
}
|
}
|
||||||
@ -556,7 +556,7 @@ func TestFinalityResolveVirtual(t *testing.T) {
|
|||||||
block.Header = mutableHeader.ToImmutable()
|
block.Header = mutableHeader.ToImmutable()
|
||||||
}
|
}
|
||||||
|
|
||||||
_, err = tcAttacker.ValidateAndInsertBlock(block, true)
|
err = tcAttacker.ValidateAndInsertBlock(block, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
panic(err)
|
panic(err)
|
||||||
}
|
}
|
||||||
@ -583,14 +583,14 @@ func TestFinalityResolveVirtual(t *testing.T) {
|
|||||||
t.Logf("Side chain tip (%s) blue score %d", sideChainTipHash, sideChainTipGHOSTDAGData.BlueScore())
|
t.Logf("Side chain tip (%s) blue score %d", sideChainTipHash, sideChainTipGHOSTDAGData.BlueScore())
|
||||||
|
|
||||||
for _, block := range sideChain {
|
for _, block := range sideChain {
|
||||||
_, err := tc.ValidateAndInsertBlock(block, false)
|
err := tc.ValidateAndInsertBlock(block, false)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
panic(err)
|
panic(err)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
for i := 0; ; i++ {
|
for i := 0; ; i++ {
|
||||||
_, isCompletelyResolved, err := tc.ResolveVirtual()
|
isCompletelyResolved, err := tc.ResolveVirtual()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
panic(err)
|
panic(err)
|
||||||
}
|
}
|
||||||
|
@ -5,8 +5,8 @@ type Consensus interface {
|
|||||||
Init(skipAddingGenesis bool) error
|
Init(skipAddingGenesis bool) error
|
||||||
BuildBlock(coinbaseData *DomainCoinbaseData, transactions []*DomainTransaction) (*DomainBlock, error)
|
BuildBlock(coinbaseData *DomainCoinbaseData, transactions []*DomainTransaction) (*DomainBlock, error)
|
||||||
BuildBlockTemplate(coinbaseData *DomainCoinbaseData, transactions []*DomainTransaction) (*DomainBlockTemplate, error)
|
BuildBlockTemplate(coinbaseData *DomainCoinbaseData, transactions []*DomainTransaction) (*DomainBlockTemplate, error)
|
||||||
ValidateAndInsertBlock(block *DomainBlock, shouldValidateAgainstUTXO bool) (*VirtualChangeSet, error)
|
ValidateAndInsertBlock(block *DomainBlock, shouldValidateAgainstUTXO bool) error
|
||||||
ValidateAndInsertBlockWithTrustedData(block *BlockWithTrustedData, validateUTXO bool) (*VirtualChangeSet, error)
|
ValidateAndInsertBlockWithTrustedData(block *BlockWithTrustedData, validateUTXO bool) error
|
||||||
ValidateTransactionAndPopulateWithConsensusData(transaction *DomainTransaction) error
|
ValidateTransactionAndPopulateWithConsensusData(transaction *DomainTransaction) error
|
||||||
ImportPruningPoints(pruningPoints []BlockHeader) error
|
ImportPruningPoints(pruningPoints []BlockHeader) error
|
||||||
BuildPruningPointProof() (*PruningPointProof, error)
|
BuildPruningPointProof() (*PruningPointProof, error)
|
||||||
@ -48,7 +48,7 @@ type Consensus interface {
|
|||||||
Anticone(blockHash *DomainHash) ([]*DomainHash, error)
|
Anticone(blockHash *DomainHash) ([]*DomainHash, error)
|
||||||
EstimateNetworkHashesPerSecond(startHash *DomainHash, windowSize int) (uint64, error)
|
EstimateNetworkHashesPerSecond(startHash *DomainHash, windowSize int) (uint64, error)
|
||||||
PopulateMass(transaction *DomainTransaction)
|
PopulateMass(transaction *DomainTransaction)
|
||||||
ResolveVirtual() (*VirtualChangeSet, bool, error)
|
ResolveVirtual() (bool, error)
|
||||||
BlockDAAWindowHashes(blockHash *DomainHash) ([]*DomainHash, error)
|
BlockDAAWindowHashes(blockHash *DomainHash) ([]*DomainHash, error)
|
||||||
TrustedDataDataDAAHeader(trustedBlockHash, daaBlockHash *DomainHash, daaBlockWindowIndex uint64) (*TrustedDataDataDAAHeader, error)
|
TrustedDataDataDAAHeader(trustedBlockHash, daaBlockHash *DomainHash, daaBlockWindowIndex uint64) (*TrustedDataDataDAAHeader, error)
|
||||||
TrustedBlockAssociatedGHOSTDAGDataBlockHashes(blockHash *DomainHash) ([]*DomainHash, error)
|
TrustedBlockAssociatedGHOSTDAGDataBlockHashes(blockHash *DomainHash) ([]*DomainHash, error)
|
||||||
|
@ -49,6 +49,8 @@ type TestConsensus interface {
|
|||||||
*externalapi.VirtualChangeSet, error)
|
*externalapi.VirtualChangeSet, error)
|
||||||
UpdatePruningPointByVirtual() error
|
UpdatePruningPointByVirtual() error
|
||||||
|
|
||||||
|
ResolveVirtualWithMaxParam(maxBlocksToResolve uint64) (bool, error)
|
||||||
|
|
||||||
MineJSON(r io.Reader, blockType MineJSONBlockType) (tips []*externalapi.DomainHash, err error)
|
MineJSON(r io.Reader, blockType MineJSONBlockType) (tips []*externalapi.DomainHash, err error)
|
||||||
ToJSON(w io.Writer) error
|
ToJSON(w io.Writer) error
|
||||||
|
|
||||||
|
@ -6,7 +6,6 @@ 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/processes/consensusstatemanager"
|
|
||||||
"github.com/kaspanet/kaspad/domain/consensus/ruleerrors"
|
"github.com/kaspanet/kaspad/domain/consensus/ruleerrors"
|
||||||
"github.com/kaspanet/kaspad/domain/consensus/utils/consensushashing"
|
"github.com/kaspanet/kaspad/domain/consensus/utils/consensushashing"
|
||||||
"github.com/kaspanet/kaspad/domain/consensus/utils/multiset"
|
"github.com/kaspanet/kaspad/domain/consensus/utils/multiset"
|
||||||
@ -165,12 +164,7 @@ func (bp *blockProcessor) validateAndInsertBlock(stagingArea *model.StagingArea,
|
|||||||
|
|
||||||
if reversalData != nil {
|
if reversalData != nil {
|
||||||
err = bp.consensusStateManager.ReverseUTXODiffs(blockHash, reversalData)
|
err = bp.consensusStateManager.ReverseUTXODiffs(blockHash, reversalData)
|
||||||
// It's still not known what causes this error, but we can ignore it and not reverse the UTXO diffs
|
if err != nil {
|
||||||
// and harm performance in some cases.
|
|
||||||
// TODO: Investigate why this error happens in the first place, and remove the workaround.
|
|
||||||
if errors.Is(err, consensusstatemanager.ErrReverseUTXODiffsUTXODiffChildNotFound) {
|
|
||||||
log.Errorf("Could not reverse UTXO diffs while resolving virtual: %s", err)
|
|
||||||
} else if err != nil {
|
|
||||||
return nil, externalapi.StatusInvalid, err
|
return nil, externalapi.StatusInvalid, err
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -79,7 +79,7 @@ func TestBlockStatus(t *testing.T) {
|
|||||||
disqualifiedBlock.Header.PruningPoint(),
|
disqualifiedBlock.Header.PruningPoint(),
|
||||||
)
|
)
|
||||||
|
|
||||||
_, err = tc.ValidateAndInsertBlock(disqualifiedBlock, true)
|
err = tc.ValidateAndInsertBlock(disqualifiedBlock, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("ValidateAndInsertBlock: %+v", err)
|
t.Fatalf("ValidateAndInsertBlock: %+v", err)
|
||||||
}
|
}
|
||||||
@ -106,7 +106,7 @@ func TestBlockStatus(t *testing.T) {
|
|||||||
disqualifiedBlock.Header.PruningPoint(),
|
disqualifiedBlock.Header.PruningPoint(),
|
||||||
)
|
)
|
||||||
|
|
||||||
_, err = tc.ValidateAndInsertBlock(invalidBlock, true)
|
err = tc.ValidateAndInsertBlock(invalidBlock, true)
|
||||||
if err == nil {
|
if err == nil {
|
||||||
t.Fatalf("block is expected to be invalid")
|
t.Fatalf("block is expected to be invalid")
|
||||||
}
|
}
|
||||||
@ -139,11 +139,11 @@ func TestValidateAndInsertErrors(t *testing.T) {
|
|||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("AddBlock: %+v", err)
|
t.Fatalf("AddBlock: %+v", err)
|
||||||
}
|
}
|
||||||
_, err = tc.ValidateAndInsertBlock(blockWithStatusInvalid, true)
|
err = tc.ValidateAndInsertBlock(blockWithStatusInvalid, true)
|
||||||
if err == nil {
|
if err == nil {
|
||||||
t.Fatalf("Test ValidateAndInsertBlock: Expected an error, because the block is invalid.")
|
t.Fatalf("Test ValidateAndInsertBlock: Expected an error, because the block is invalid.")
|
||||||
}
|
}
|
||||||
_, err = tc.ValidateAndInsertBlock(blockWithStatusInvalid, true)
|
err = tc.ValidateAndInsertBlock(blockWithStatusInvalid, true)
|
||||||
if err == nil || !errors.Is(err, ruleerrors.ErrKnownInvalid) {
|
if err == nil || !errors.Is(err, ruleerrors.ErrKnownInvalid) {
|
||||||
t.Fatalf("Expected block to be invalid with err: %v, instead found: %v", ruleerrors.ErrKnownInvalid, err)
|
t.Fatalf("Expected block to be invalid with err: %v, instead found: %v", ruleerrors.ErrKnownInvalid, err)
|
||||||
}
|
}
|
||||||
@ -155,12 +155,12 @@ func TestValidateAndInsertErrors(t *testing.T) {
|
|||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("AddBlock: %+v", err)
|
t.Fatalf("AddBlock: %+v", err)
|
||||||
}
|
}
|
||||||
_, err = tc.ValidateAndInsertBlock(block, true)
|
err = tc.ValidateAndInsertBlock(block, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("ValidateAndInsertBlock: %+v", err)
|
t.Fatalf("ValidateAndInsertBlock: %+v", err)
|
||||||
}
|
}
|
||||||
// resend the same block.
|
// resend the same block.
|
||||||
_, err = tc.ValidateAndInsertBlock(block, true)
|
err = tc.ValidateAndInsertBlock(block, true)
|
||||||
if err == nil || !errors.Is(err, ruleerrors.ErrDuplicateBlock) {
|
if err == nil || !errors.Is(err, ruleerrors.ErrDuplicateBlock) {
|
||||||
t.Fatalf("Expected block to be invalid with err: %v, instead found: %v", ruleerrors.ErrDuplicateBlock, err)
|
t.Fatalf("Expected block to be invalid with err: %v, instead found: %v", ruleerrors.ErrDuplicateBlock, err)
|
||||||
}
|
}
|
||||||
@ -173,12 +173,12 @@ func TestValidateAndInsertErrors(t *testing.T) {
|
|||||||
t.Fatalf("AddBlock: %+v", err)
|
t.Fatalf("AddBlock: %+v", err)
|
||||||
}
|
}
|
||||||
onlyHeader.Transactions = []*externalapi.DomainTransaction{}
|
onlyHeader.Transactions = []*externalapi.DomainTransaction{}
|
||||||
_, err = tc.ValidateAndInsertBlock(onlyHeader, true)
|
err = tc.ValidateAndInsertBlock(onlyHeader, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("AddBlock: %+v", err)
|
t.Fatalf("AddBlock: %+v", err)
|
||||||
}
|
}
|
||||||
// resend the same header.
|
// resend the same header.
|
||||||
_, err = tc.ValidateAndInsertBlock(onlyHeader, true)
|
err = tc.ValidateAndInsertBlock(onlyHeader, true)
|
||||||
if err == nil || !errors.Is(err, ruleerrors.ErrDuplicateBlock) {
|
if err == nil || !errors.Is(err, ruleerrors.ErrDuplicateBlock) {
|
||||||
t.Fatalf("Expected block to be invalid with err: %v, instead found: %v", ruleerrors.ErrDuplicateBlock, err)
|
t.Fatalf("Expected block to be invalid with err: %v, instead found: %v", ruleerrors.ErrDuplicateBlock, err)
|
||||||
}
|
}
|
||||||
|
@ -26,7 +26,7 @@ func addBlock(tc testapi.TestConsensus, parentHashes []*externalapi.DomainHash,
|
|||||||
}
|
}
|
||||||
|
|
||||||
blockHash := consensushashing.BlockHash(block)
|
blockHash := consensushashing.BlockHash(block)
|
||||||
_, err = tc.ValidateAndInsertBlock(block, true)
|
err = tc.ValidateAndInsertBlock(block, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("ValidateAndInsertBlock: %+v", err)
|
t.Fatalf("ValidateAndInsertBlock: %+v", err)
|
||||||
}
|
}
|
||||||
@ -133,7 +133,7 @@ func TestValidateAndInsertImportedPruningPoint(t *testing.T) {
|
|||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
_, err = synceeStaging.ValidateAndInsertBlockWithTrustedData(blockWithTrustedData, false)
|
err = synceeStaging.ValidateAndInsertBlockWithTrustedData(blockWithTrustedData, false)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("ValidateAndInsertBlockWithTrustedData: %+v", err)
|
t.Fatalf("ValidateAndInsertBlockWithTrustedData: %+v", err)
|
||||||
}
|
}
|
||||||
@ -169,7 +169,7 @@ func TestValidateAndInsertImportedPruningPoint(t *testing.T) {
|
|||||||
t.Fatalf("GetBlockHeader: %+v", err)
|
t.Fatalf("GetBlockHeader: %+v", err)
|
||||||
}
|
}
|
||||||
|
|
||||||
_, err = synceeStaging.ValidateAndInsertBlock(&externalapi.DomainBlock{Header: header}, false)
|
err = synceeStaging.ValidateAndInsertBlock(&externalapi.DomainBlock{Header: header}, false)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("ValidateAndInsertBlock %d: %+v", i, err)
|
t.Fatalf("ValidateAndInsertBlock %d: %+v", i, err)
|
||||||
}
|
}
|
||||||
@ -273,7 +273,7 @@ func TestValidateAndInsertImportedPruningPoint(t *testing.T) {
|
|||||||
t.Fatalf("GetBlock: %+v", err)
|
t.Fatalf("GetBlock: %+v", err)
|
||||||
}
|
}
|
||||||
|
|
||||||
_, err = synceeStaging.ValidateAndInsertBlock(block, true)
|
err = synceeStaging.ValidateAndInsertBlock(block, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("ValidateAndInsertBlock: %+v", err)
|
t.Fatalf("ValidateAndInsertBlock: %+v", err)
|
||||||
}
|
}
|
||||||
@ -299,7 +299,7 @@ func TestValidateAndInsertImportedPruningPoint(t *testing.T) {
|
|||||||
t.Fatalf("GetBlock: %+v", err)
|
t.Fatalf("GetBlock: %+v", err)
|
||||||
}
|
}
|
||||||
|
|
||||||
_, err = synceeStaging.ValidateAndInsertBlock(tip, true)
|
err = synceeStaging.ValidateAndInsertBlock(tip, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("ValidateAndInsertBlock: %+v", err)
|
t.Fatalf("ValidateAndInsertBlock: %+v", err)
|
||||||
}
|
}
|
||||||
@ -346,7 +346,7 @@ func TestValidateAndInsertImportedPruningPoint(t *testing.T) {
|
|||||||
t.Fatalf("GetBlock: %+v", err)
|
t.Fatalf("GetBlock: %+v", err)
|
||||||
}
|
}
|
||||||
|
|
||||||
_, err = tcSyncee1.ValidateAndInsertBlock(block, true)
|
err = tcSyncee1.ValidateAndInsertBlock(block, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("ValidateAndInsertBlock: %+v", err)
|
t.Fatalf("ValidateAndInsertBlock: %+v", err)
|
||||||
}
|
}
|
||||||
@ -478,7 +478,7 @@ func TestGetPruningPointUTXOs(t *testing.T) {
|
|||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("Error building block above genesis: %+v", err)
|
t.Fatalf("Error building block above genesis: %+v", err)
|
||||||
}
|
}
|
||||||
_, err = testConsensus.ValidateAndInsertBlock(blockAboveGenesis, true)
|
err = testConsensus.ValidateAndInsertBlock(blockAboveGenesis, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("Error validating and inserting block above genesis: %+v", err)
|
t.Fatalf("Error validating and inserting block above genesis: %+v", err)
|
||||||
}
|
}
|
||||||
@ -490,7 +490,7 @@ func TestGetPruningPointUTXOs(t *testing.T) {
|
|||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("Error building block with spendable coinbase: %+v", err)
|
t.Fatalf("Error building block with spendable coinbase: %+v", err)
|
||||||
}
|
}
|
||||||
_, err = testConsensus.ValidateAndInsertBlock(blockWithSpendableCoinbase, true)
|
err = testConsensus.ValidateAndInsertBlock(blockWithSpendableCoinbase, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("Error validating and inserting block with spendable coinbase: %+v", err)
|
t.Fatalf("Error validating and inserting block with spendable coinbase: %+v", err)
|
||||||
}
|
}
|
||||||
@ -529,7 +529,7 @@ func TestGetPruningPointUTXOs(t *testing.T) {
|
|||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("Error building including block: %+v", err)
|
t.Fatalf("Error building including block: %+v", err)
|
||||||
}
|
}
|
||||||
_, err = testConsensus.ValidateAndInsertBlock(includingBlock, true)
|
err = testConsensus.ValidateAndInsertBlock(includingBlock, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("Error validating and inserting including block: %+v", err)
|
t.Fatalf("Error validating and inserting including block: %+v", err)
|
||||||
}
|
}
|
||||||
@ -540,7 +540,7 @@ func TestGetPruningPointUTXOs(t *testing.T) {
|
|||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("Error building block: %+v", err)
|
t.Fatalf("Error building block: %+v", err)
|
||||||
}
|
}
|
||||||
_, err = testConsensus.ValidateAndInsertBlock(block, true)
|
err = testConsensus.ValidateAndInsertBlock(block, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("Error validating and inserting block: %+v", err)
|
t.Fatalf("Error validating and inserting block: %+v", err)
|
||||||
}
|
}
|
||||||
@ -636,7 +636,7 @@ func BenchmarkGetPruningPointUTXOs(b *testing.B) {
|
|||||||
if err != nil {
|
if err != nil {
|
||||||
b.Fatalf("Error building block with spendable coinbase: %+v", err)
|
b.Fatalf("Error building block with spendable coinbase: %+v", err)
|
||||||
}
|
}
|
||||||
_, err = testConsensus.ValidateAndInsertBlock(blockWithSpendableCoinbase, true)
|
err = testConsensus.ValidateAndInsertBlock(blockWithSpendableCoinbase, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
b.Fatalf("Error validating and inserting block with spendable coinbase: %+v", err)
|
b.Fatalf("Error validating and inserting block with spendable coinbase: %+v", err)
|
||||||
}
|
}
|
||||||
@ -674,7 +674,7 @@ func BenchmarkGetPruningPointUTXOs(b *testing.B) {
|
|||||||
if err != nil {
|
if err != nil {
|
||||||
b.Fatalf("Error building block: %+v", err)
|
b.Fatalf("Error building block: %+v", err)
|
||||||
}
|
}
|
||||||
_, err = testConsensus.ValidateAndInsertBlock(block, true)
|
err = testConsensus.ValidateAndInsertBlock(block, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
b.Fatalf("Error validating and inserting block: %+v", err)
|
b.Fatalf("Error validating and inserting block: %+v", err)
|
||||||
}
|
}
|
||||||
@ -694,7 +694,7 @@ func BenchmarkGetPruningPointUTXOs(b *testing.B) {
|
|||||||
if err != nil {
|
if err != nil {
|
||||||
b.Fatalf("Error building block: %+v", err)
|
b.Fatalf("Error building block: %+v", err)
|
||||||
}
|
}
|
||||||
_, err = testConsensus.ValidateAndInsertBlock(block, true)
|
err = testConsensus.ValidateAndInsertBlock(block, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
b.Fatalf("Error validating and inserting block: %+v", err)
|
b.Fatalf("Error validating and inserting block: %+v", err)
|
||||||
}
|
}
|
||||||
|
@ -62,7 +62,7 @@ func TestCheckBlockIsNotPruned(t *testing.T) {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
_, err = tc.ValidateAndInsertBlock(beforePruningBlock, true)
|
err = tc.ValidateAndInsertBlock(beforePruningBlock, true)
|
||||||
if !errors.Is(err, ruleerrors.ErrPrunedBlock) {
|
if !errors.Is(err, ruleerrors.ErrPrunedBlock) {
|
||||||
t.Fatalf("Unexpected error: %+v", err)
|
t.Fatalf("Unexpected error: %+v", err)
|
||||||
}
|
}
|
||||||
@ -117,7 +117,7 @@ func TestCheckParentBlockBodiesExist(t *testing.T) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// Add only the header of anticonePruningBlock
|
// Add only the header of anticonePruningBlock
|
||||||
_, err = tc.ValidateAndInsertBlock(&externalapi.DomainBlock{
|
err = tc.ValidateAndInsertBlock(&externalapi.DomainBlock{
|
||||||
Header: anticonePruningBlock.Header,
|
Header: anticonePruningBlock.Header,
|
||||||
Transactions: nil,
|
Transactions: nil,
|
||||||
}, true)
|
}, true)
|
||||||
@ -143,7 +143,7 @@ func TestCheckParentBlockBodiesExist(t *testing.T) {
|
|||||||
|
|
||||||
// Add anticonePruningBlock's body and check that it's valid to point to
|
// Add anticonePruningBlock's body and check that it's valid to point to
|
||||||
// a header only block in the past of the pruning point.
|
// a header only block in the past of the pruning point.
|
||||||
_, err = tc.ValidateAndInsertBlock(anticonePruningBlock, true)
|
err = tc.ValidateAndInsertBlock(anticonePruningBlock, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("ValidateAndInsertBlock: %+v", err)
|
t.Fatalf("ValidateAndInsertBlock: %+v", err)
|
||||||
}
|
}
|
||||||
@ -189,7 +189,7 @@ func TestIsFinalizedTransaction(t *testing.T) {
|
|||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("Error getting block: %+v", err)
|
t.Fatalf("Error getting block: %+v", err)
|
||||||
}
|
}
|
||||||
_, err = tc.ValidateAndInsertBlock(block, true)
|
err = tc.ValidateAndInsertBlock(block, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("Error Inserting block: %+v", err)
|
t.Fatalf("Error Inserting block: %+v", err)
|
||||||
}
|
}
|
||||||
|
@ -1052,14 +1052,14 @@ func CheckBlockHashMerkleRoot(t *testing.T, tc testapi.TestConsensus, consensusC
|
|||||||
blockWithInvalidMerkleRoot := block.Clone()
|
blockWithInvalidMerkleRoot := block.Clone()
|
||||||
blockWithInvalidMerkleRoot.Transactions[0].Version += 1
|
blockWithInvalidMerkleRoot.Transactions[0].Version += 1
|
||||||
|
|
||||||
_, err = tc.ValidateAndInsertBlock(blockWithInvalidMerkleRoot, true)
|
err = tc.ValidateAndInsertBlock(blockWithInvalidMerkleRoot, true)
|
||||||
if !errors.Is(err, ruleerrors.ErrBadMerkleRoot) {
|
if !errors.Is(err, ruleerrors.ErrBadMerkleRoot) {
|
||||||
t.Fatalf("Unexpected error: %+v", err)
|
t.Fatalf("Unexpected error: %+v", err)
|
||||||
}
|
}
|
||||||
|
|
||||||
// Check that a block with invalid merkle root is not marked as invalid
|
// Check that a block with invalid merkle root is not marked as invalid
|
||||||
// and can be re-added with the right transactions.
|
// and can be re-added with the right transactions.
|
||||||
_, err = tc.ValidateAndInsertBlock(block, true)
|
err = tc.ValidateAndInsertBlock(block, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("ValidateAndInsertBlock: %+v", err)
|
t.Fatalf("ValidateAndInsertBlock: %+v", err)
|
||||||
}
|
}
|
||||||
|
@ -34,7 +34,7 @@ func TestValidateMedianTime(t *testing.T) {
|
|||||||
newHeader := block.Header.ToMutable()
|
newHeader := block.Header.ToMutable()
|
||||||
newHeader.SetTimeInMilliseconds(blockTime)
|
newHeader.SetTimeInMilliseconds(blockTime)
|
||||||
block.Header = newHeader.ToImmutable()
|
block.Header = newHeader.ToImmutable()
|
||||||
_, err = tc.ValidateAndInsertBlock(block, true)
|
err = tc.ValidateAndInsertBlock(block, true)
|
||||||
if !errors.Is(err, expectedErr) {
|
if !errors.Is(err, expectedErr) {
|
||||||
t.Fatalf("expected error %s but got %+v", expectedErr, err)
|
t.Fatalf("expected error %s but got %+v", expectedErr, err)
|
||||||
}
|
}
|
||||||
@ -127,7 +127,7 @@ func TestCheckParentsIncest(t *testing.T) {
|
|||||||
Transactions: nil,
|
Transactions: nil,
|
||||||
}
|
}
|
||||||
|
|
||||||
_, err = tc.ValidateAndInsertBlock(directParentsRelationBlock, true)
|
err = tc.ValidateAndInsertBlock(directParentsRelationBlock, true)
|
||||||
if !errors.Is(err, ruleerrors.ErrInvalidParentsRelation) {
|
if !errors.Is(err, ruleerrors.ErrInvalidParentsRelation) {
|
||||||
t.Fatalf("unexpected error %+v", err)
|
t.Fatalf("unexpected error %+v", err)
|
||||||
}
|
}
|
||||||
@ -150,7 +150,7 @@ func TestCheckParentsIncest(t *testing.T) {
|
|||||||
Transactions: nil,
|
Transactions: nil,
|
||||||
}
|
}
|
||||||
|
|
||||||
_, err = tc.ValidateAndInsertBlock(indirectParentsRelationBlock, true)
|
err = tc.ValidateAndInsertBlock(indirectParentsRelationBlock, true)
|
||||||
if !errors.Is(err, ruleerrors.ErrInvalidParentsRelation) {
|
if !errors.Is(err, ruleerrors.ErrInvalidParentsRelation) {
|
||||||
t.Fatalf("unexpected error %+v", err)
|
t.Fatalf("unexpected error %+v", err)
|
||||||
}
|
}
|
||||||
|
@ -78,7 +78,7 @@ func CheckBlockVersion(t *testing.T, tc testapi.TestConsensus, consensusConfig *
|
|||||||
block.Header.PruningPoint(),
|
block.Header.PruningPoint(),
|
||||||
)
|
)
|
||||||
|
|
||||||
_, err = tc.ValidateAndInsertBlock(block, true)
|
err = tc.ValidateAndInsertBlock(block, true)
|
||||||
if !errors.Is(err, ruleerrors.ErrWrongBlockVersion) {
|
if !errors.Is(err, ruleerrors.ErrWrongBlockVersion) {
|
||||||
t.Fatalf("Unexpected error: %+v", err)
|
t.Fatalf("Unexpected error: %+v", err)
|
||||||
}
|
}
|
||||||
@ -118,7 +118,7 @@ func CheckBlockTimestampInIsolation(t *testing.T, tc testapi.TestConsensus, cfg
|
|||||||
block.Header.PruningPoint(),
|
block.Header.PruningPoint(),
|
||||||
)
|
)
|
||||||
|
|
||||||
_, err = tc.ValidateAndInsertBlock(block, true)
|
err = tc.ValidateAndInsertBlock(block, true)
|
||||||
if !errors.Is(err, ruleerrors.ErrTimeTooMuchInTheFuture) {
|
if !errors.Is(err, ruleerrors.ErrTimeTooMuchInTheFuture) {
|
||||||
t.Fatalf("Unexpected error: %+v", err)
|
t.Fatalf("Unexpected error: %+v", err)
|
||||||
}
|
}
|
||||||
|
@ -39,7 +39,7 @@ func TestPOW(t *testing.T) {
|
|||||||
t.Fatal(err)
|
t.Fatal(err)
|
||||||
}
|
}
|
||||||
invalidBlockWrongPOW = solveBlockWithWrongPOW(invalidBlockWrongPOW)
|
invalidBlockWrongPOW = solveBlockWithWrongPOW(invalidBlockWrongPOW)
|
||||||
_, err = tc.ValidateAndInsertBlock(invalidBlockWrongPOW, true)
|
err = tc.ValidateAndInsertBlock(invalidBlockWrongPOW, true)
|
||||||
if !errors.Is(err, ruleerrors.ErrInvalidPoW) {
|
if !errors.Is(err, ruleerrors.ErrInvalidPoW) {
|
||||||
t.Fatalf("Expected block to be invalid with err: %v, instead found: %v", ruleerrors.ErrInvalidPoW, err)
|
t.Fatalf("Expected block to be invalid with err: %v, instead found: %v", ruleerrors.ErrInvalidPoW, err)
|
||||||
}
|
}
|
||||||
@ -65,7 +65,7 @@ func TestPOW(t *testing.T) {
|
|||||||
abovePowMaxBlock.Header.PruningPoint(),
|
abovePowMaxBlock.Header.PruningPoint(),
|
||||||
)
|
)
|
||||||
|
|
||||||
_, err = tc.ValidateAndInsertBlock(abovePowMaxBlock, true)
|
err = tc.ValidateAndInsertBlock(abovePowMaxBlock, true)
|
||||||
if !errors.Is(err, ruleerrors.ErrTargetTooHigh) {
|
if !errors.Is(err, ruleerrors.ErrTargetTooHigh) {
|
||||||
t.Fatalf("Unexpected error: %+v", err)
|
t.Fatalf("Unexpected error: %+v", err)
|
||||||
}
|
}
|
||||||
@ -90,7 +90,7 @@ func TestPOW(t *testing.T) {
|
|||||||
negativeTargetBlock.Header.PruningPoint(),
|
negativeTargetBlock.Header.PruningPoint(),
|
||||||
)
|
)
|
||||||
|
|
||||||
_, err = tc.ValidateAndInsertBlock(negativeTargetBlock, true)
|
err = tc.ValidateAndInsertBlock(negativeTargetBlock, true)
|
||||||
if !errors.Is(err, ruleerrors.ErrNegativeTarget) {
|
if !errors.Is(err, ruleerrors.ErrNegativeTarget) {
|
||||||
t.Fatalf("Unexpected error: %+v", err)
|
t.Fatalf("Unexpected error: %+v", err)
|
||||||
}
|
}
|
||||||
@ -104,7 +104,7 @@ func TestPOW(t *testing.T) {
|
|||||||
// Difficulty is too high on mainnet to actually mine.
|
// Difficulty is too high on mainnet to actually mine.
|
||||||
if consensusConfig.Name != "kaspa-mainnet" {
|
if consensusConfig.Name != "kaspa-mainnet" {
|
||||||
mining.SolveBlock(validBlock, random)
|
mining.SolveBlock(validBlock, random)
|
||||||
_, err = tc.ValidateAndInsertBlock(validBlock, true)
|
err = tc.ValidateAndInsertBlock(validBlock, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatal(err)
|
t.Fatal(err)
|
||||||
}
|
}
|
||||||
@ -161,7 +161,7 @@ func TestCheckParentHeadersExist(t *testing.T) {
|
|||||||
orphanBlock.Header.PruningPoint(),
|
orphanBlock.Header.PruningPoint(),
|
||||||
)
|
)
|
||||||
|
|
||||||
_, err = tc.ValidateAndInsertBlock(orphanBlock, true)
|
err = tc.ValidateAndInsertBlock(orphanBlock, true)
|
||||||
errMissingParents := &ruleerrors.ErrMissingParents{}
|
errMissingParents := &ruleerrors.ErrMissingParents{}
|
||||||
if !errors.As(err, errMissingParents) {
|
if !errors.As(err, errMissingParents) {
|
||||||
t.Fatalf("Unexpected error: %+v", err)
|
t.Fatalf("Unexpected error: %+v", err)
|
||||||
@ -193,7 +193,7 @@ func TestCheckParentHeadersExist(t *testing.T) {
|
|||||||
orphanBlock.Header.PruningPoint(),
|
orphanBlock.Header.PruningPoint(),
|
||||||
)
|
)
|
||||||
|
|
||||||
_, err = tc.ValidateAndInsertBlock(invalidBlock, true)
|
err = tc.ValidateAndInsertBlock(invalidBlock, true)
|
||||||
if !errors.Is(err, ruleerrors.ErrTransactionVersionIsUnknown) {
|
if !errors.Is(err, ruleerrors.ErrTransactionVersionIsUnknown) {
|
||||||
t.Fatalf("Unexpected error: %+v", err)
|
t.Fatalf("Unexpected error: %+v", err)
|
||||||
}
|
}
|
||||||
@ -220,7 +220,7 @@ func TestCheckParentHeadersExist(t *testing.T) {
|
|||||||
invalidBlockChild.Header.PruningPoint(),
|
invalidBlockChild.Header.PruningPoint(),
|
||||||
)
|
)
|
||||||
|
|
||||||
_, err = tc.ValidateAndInsertBlock(invalidBlockChild, true)
|
err = tc.ValidateAndInsertBlock(invalidBlockChild, true)
|
||||||
if !errors.Is(err, ruleerrors.ErrInvalidAncestorBlock) {
|
if !errors.Is(err, ruleerrors.ErrInvalidAncestorBlock) {
|
||||||
t.Fatalf("Unexpected error: %+v", err)
|
t.Fatalf("Unexpected error: %+v", err)
|
||||||
}
|
}
|
||||||
@ -284,7 +284,7 @@ func TestCheckPruningPointViolation(t *testing.T) {
|
|||||||
blockWithPruningViolation.Header.PruningPoint(),
|
blockWithPruningViolation.Header.PruningPoint(),
|
||||||
)
|
)
|
||||||
|
|
||||||
_, err = tc.ValidateAndInsertBlock(blockWithPruningViolation, true)
|
err = tc.ValidateAndInsertBlock(blockWithPruningViolation, true)
|
||||||
if !errors.Is(err, ruleerrors.ErrPruningPointViolation) {
|
if !errors.Is(err, ruleerrors.ErrPruningPointViolation) {
|
||||||
t.Fatalf("Unexpected error: %+v", err)
|
t.Fatalf("Unexpected error: %+v", err)
|
||||||
}
|
}
|
||||||
|
@ -5,7 +5,6 @@ import (
|
|||||||
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
|
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
|
||||||
"github.com/kaspanet/kaspad/infrastructure/logger"
|
"github.com/kaspanet/kaspad/infrastructure/logger"
|
||||||
"github.com/kaspanet/kaspad/util/staging"
|
"github.com/kaspanet/kaspad/util/staging"
|
||||||
"github.com/pkg/errors"
|
|
||||||
"sort"
|
"sort"
|
||||||
)
|
)
|
||||||
|
|
||||||
@ -79,12 +78,7 @@ func (csm *consensusStateManager) ResolveVirtual(maxBlocksToResolve uint64) (*ex
|
|||||||
|
|
||||||
if reversalData != nil {
|
if reversalData != nil {
|
||||||
err = csm.ReverseUTXODiffs(resolveTip, reversalData)
|
err = csm.ReverseUTXODiffs(resolveTip, reversalData)
|
||||||
// It's still not known what causes this error, but we can ignore it and not reverse the UTXO diffs
|
if err != nil {
|
||||||
// and harm performance in some cases.
|
|
||||||
// TODO: Investigate why this error happens in the first place, and remove the workaround.
|
|
||||||
if errors.Is(err, ErrReverseUTXODiffsUTXODiffChildNotFound) {
|
|
||||||
log.Errorf("Could not reverse UTXO diffs while resolving virtual: %s", err)
|
|
||||||
} else if err != nil {
|
|
||||||
return nil, false, err
|
return nil, false, err
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -0,0 +1,214 @@
|
|||||||
|
package consensusstatemanager_test
|
||||||
|
|
||||||
|
import (
|
||||||
|
"github.com/kaspanet/kaspad/domain/consensus/utils/consensushashing"
|
||||||
|
"testing"
|
||||||
|
|
||||||
|
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
|
||||||
|
|
||||||
|
"github.com/kaspanet/kaspad/domain/consensus"
|
||||||
|
"github.com/kaspanet/kaspad/domain/consensus/utils/testutils"
|
||||||
|
)
|
||||||
|
|
||||||
|
func TestAddBlockBetweenResolveVirtualCalls(t *testing.T) {
|
||||||
|
|
||||||
|
testutils.ForAllNets(t, true, func(t *testing.T, consensusConfig *consensus.Config) {
|
||||||
|
factory := consensus.NewFactory()
|
||||||
|
|
||||||
|
tc, teardown, err := factory.NewTestConsensus(consensusConfig, "TestAddBlockBetweenResolveVirtualCalls")
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("Error setting up consensus: %+v", err)
|
||||||
|
}
|
||||||
|
defer teardown(false)
|
||||||
|
|
||||||
|
// 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)
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("Error mining block no. %d in initial chain: %+v", i, err)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Mine a chain with more blocks, to re-organize the DAG
|
||||||
|
const reorgChainLength = 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)
|
||||||
|
|
||||||
|
// 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)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Resolve one step
|
||||||
|
_, err = tc.ResolveVirtualWithMaxParam(2)
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("Error resolving virtual in re-org chain: %+v", err)
|
||||||
|
}
|
||||||
|
|
||||||
|
emptyCoinbase := &externalapi.DomainCoinbaseData{
|
||||||
|
ScriptPublicKey: &externalapi.ScriptPublicKey{
|
||||||
|
Script: nil,
|
||||||
|
Version: 0,
|
||||||
|
},
|
||||||
|
}
|
||||||
|
|
||||||
|
// Get template based on current resolve state
|
||||||
|
blockTemplate, err := tc.BuildBlockTemplate(emptyCoinbase, nil)
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("Error building block template during virtual resolution of reorg: %+v", err)
|
||||||
|
}
|
||||||
|
|
||||||
|
// Resolve one more step
|
||||||
|
isCompletelyResolved, err := tc.ResolveVirtualWithMaxParam(2)
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("Error resolving virtual in re-org chain: %+v", err)
|
||||||
|
}
|
||||||
|
|
||||||
|
// Add the mined block (now virtual was modified)
|
||||||
|
err = tc.ValidateAndInsertBlock(blockTemplate.Block, true)
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("Error mining block during virtual resolution of reorg: %+v", err)
|
||||||
|
}
|
||||||
|
|
||||||
|
// Complete resolving virtual
|
||||||
|
for !isCompletelyResolved {
|
||||||
|
isCompletelyResolved, err = tc.ResolveVirtualWithMaxParam(2)
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("Error resolving virtual in re-org chain: %+v", err)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
})
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestAddGenesisChildAfterOneResolveVirtualCall(t *testing.T) {
|
||||||
|
|
||||||
|
testutils.ForAllNets(t, true, func(t *testing.T, consensusConfig *consensus.Config) {
|
||||||
|
factory := consensus.NewFactory()
|
||||||
|
|
||||||
|
tc, teardown, err := factory.NewTestConsensus(consensusConfig, "TestAddGenesisChildAfterOneResolveVirtualCall")
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("Error setting up consensus: %+v", err)
|
||||||
|
}
|
||||||
|
defer teardown(false)
|
||||||
|
|
||||||
|
// 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)
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("Error mining block no. %d in initial chain: %+v", i, err)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Mine a chain with more blocks, to re-organize the DAG
|
||||||
|
const reorgChainLength = 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)
|
||||||
|
|
||||||
|
// 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)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Resolve one step
|
||||||
|
isCompletelyResolved, err := tc.ResolveVirtualWithMaxParam(2)
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("Error resolving virtual in re-org chain: %+v", err)
|
||||||
|
}
|
||||||
|
|
||||||
|
_, _, err = tc.AddBlock([]*externalapi.DomainHash{consensusConfig.GenesisHash}, nil, nil)
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("Error adding block during virtual resolution of reorg: %+v", err)
|
||||||
|
}
|
||||||
|
|
||||||
|
// Complete resolving virtual
|
||||||
|
for !isCompletelyResolved {
|
||||||
|
isCompletelyResolved, err = tc.ResolveVirtualWithMaxParam(2)
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("Error resolving virtual in re-org chain: %+v", err)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
})
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestAddGenesisChildAfterTwoResolveVirtualCalls(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)
|
||||||
|
|
||||||
|
// 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)
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("Error mining block no. %d in initial chain: %+v", i, err)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Mine a chain with more blocks, to re-organize the DAG
|
||||||
|
const reorgChainLength = 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)
|
||||||
|
|
||||||
|
// 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)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Resolve one step
|
||||||
|
_, 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)
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("Error resolving virtual in re-org chain: %+v", err)
|
||||||
|
}
|
||||||
|
|
||||||
|
_, _, err = tc.AddBlock([]*externalapi.DomainHash{consensusConfig.GenesisHash}, nil, nil)
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("Error adding block during virtual resolution of reorg: %+v", err)
|
||||||
|
}
|
||||||
|
|
||||||
|
// Complete resolving virtual
|
||||||
|
for !isCompletelyResolved {
|
||||||
|
isCompletelyResolved, err = tc.ResolveVirtualWithMaxParam(2)
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("Error resolving virtual in re-org chain: %+v", err)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
})
|
||||||
|
}
|
@ -3,18 +3,10 @@ package consensusstatemanager
|
|||||||
import (
|
import (
|
||||||
"github.com/kaspanet/kaspad/domain/consensus/model"
|
"github.com/kaspanet/kaspad/domain/consensus/model"
|
||||||
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
|
"github.com/kaspanet/kaspad/domain/consensus/model/externalapi"
|
||||||
"github.com/kaspanet/kaspad/infrastructure/db/database"
|
|
||||||
"github.com/kaspanet/kaspad/infrastructure/logger"
|
"github.com/kaspanet/kaspad/infrastructure/logger"
|
||||||
"github.com/kaspanet/kaspad/util/staging"
|
"github.com/kaspanet/kaspad/util/staging"
|
||||||
"github.com/pkg/errors"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
// ErrReverseUTXODiffsUTXODiffChildNotFound indicates a UTXO diff child was not found while calling ReverseUTXODiffs.
|
|
||||||
// It's still not known what causes this error, but we can ignore it and not reverse the UTXO diffs
|
|
||||||
// and harm performance in some cases.
|
|
||||||
// TODO: Investigate why this error happens in the first place, and remove the workaround.
|
|
||||||
var ErrReverseUTXODiffsUTXODiffChildNotFound = errors.New("ErrReverseUTXODiffsUTXODiffChildNotFound")
|
|
||||||
|
|
||||||
func (csm *consensusStateManager) ReverseUTXODiffs(tipHash *externalapi.DomainHash,
|
func (csm *consensusStateManager) ReverseUTXODiffs(tipHash *externalapi.DomainHash,
|
||||||
reversalData *model.UTXODiffReversalData) error {
|
reversalData *model.UTXODiffReversalData) error {
|
||||||
|
|
||||||
@ -57,9 +49,6 @@ func (csm *consensusStateManager) ReverseUTXODiffs(tipHash *externalapi.DomainHa
|
|||||||
|
|
||||||
currentBlockUTXODiffChild, err := csm.utxoDiffStore.UTXODiffChild(csm.databaseContext, readStagingArea, currentBlock)
|
currentBlockUTXODiffChild, err := csm.utxoDiffStore.UTXODiffChild(csm.databaseContext, readStagingArea, currentBlock)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
if database.IsNotFoundError(err) {
|
|
||||||
return errors.Wrapf(ErrReverseUTXODiffsUTXODiffChildNotFound, "UTXO diff child was not found for block %s", currentBlock)
|
|
||||||
}
|
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
currentBlockGHOSTDAGData, err := csm.ghostdagDataStore.Get(csm.databaseContext, readStagingArea, currentBlock, false)
|
currentBlockGHOSTDAGData, err := csm.ghostdagDataStore.Get(csm.databaseContext, readStagingArea, currentBlock, false)
|
||||||
|
@ -84,7 +84,7 @@ func TestConsensusStateManager_pickVirtualParents(t *testing.T) {
|
|||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("Failed building a block: %v", err)
|
t.Fatalf("Failed building a block: %v", err)
|
||||||
}
|
}
|
||||||
_, err = tc.ValidateAndInsertBlock(block, true)
|
err = tc.ValidateAndInsertBlock(block, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("Failed Inserting block to tc: %v", err)
|
t.Fatalf("Failed Inserting block to tc: %v", err)
|
||||||
}
|
}
|
||||||
|
@ -65,7 +65,7 @@ func TestDifficulty(t *testing.T) {
|
|||||||
newHeader := block.Header.ToMutable()
|
newHeader := block.Header.ToMutable()
|
||||||
newHeader.SetTimeInMilliseconds(blockTime)
|
newHeader.SetTimeInMilliseconds(blockTime)
|
||||||
block.Header = newHeader.ToImmutable()
|
block.Header = newHeader.ToImmutable()
|
||||||
_, err = tc.ValidateAndInsertBlock(block, true)
|
err = tc.ValidateAndInsertBlock(block, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("ValidateAndInsertBlock: %+v", err)
|
t.Fatalf("ValidateAndInsertBlock: %+v", err)
|
||||||
}
|
}
|
||||||
|
@ -34,7 +34,7 @@ func TestPastMedianTime(t *testing.T) {
|
|||||||
newHeader := block.Header.ToMutable()
|
newHeader := block.Header.ToMutable()
|
||||||
newHeader.SetTimeInMilliseconds(blockTime)
|
newHeader.SetTimeInMilliseconds(blockTime)
|
||||||
block.Header = newHeader.ToImmutable()
|
block.Header = newHeader.ToImmutable()
|
||||||
_, err = tc.ValidateAndInsertBlock(block, true)
|
err = tc.ValidateAndInsertBlock(block, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("ValidateAndInsertBlock: %+v", err)
|
t.Fatalf("ValidateAndInsertBlock: %+v", err)
|
||||||
}
|
}
|
||||||
|
@ -61,7 +61,7 @@ func (tc *testConsensus) AddBlock(parentHashes []*externalapi.DomainHash, coinba
|
|||||||
return nil, nil, err
|
return nil, nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
virtualChangeSet, _, err := tc.blockProcessor.ValidateAndInsertBlock(block, true)
|
virtualChangeSet, err := tc.validateAndInsertBlockNoLock(block, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, nil, err
|
return nil, nil, err
|
||||||
}
|
}
|
||||||
@ -81,7 +81,7 @@ func (tc *testConsensus) AddUTXOInvalidHeader(parentHashes []*externalapi.Domain
|
|||||||
return nil, nil, err
|
return nil, nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
virtualChangeSet, _, err := tc.blockProcessor.ValidateAndInsertBlock(&externalapi.DomainBlock{
|
virtualChangeSet, err := tc.validateAndInsertBlockNoLock(&externalapi.DomainBlock{
|
||||||
Header: header,
|
Header: header,
|
||||||
Transactions: nil,
|
Transactions: nil,
|
||||||
}, true)
|
}, true)
|
||||||
@ -104,7 +104,7 @@ func (tc *testConsensus) AddUTXOInvalidBlock(parentHashes []*externalapi.DomainH
|
|||||||
return nil, nil, err
|
return nil, nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
virtualChangeSet, _, err := tc.blockProcessor.ValidateAndInsertBlock(block, true)
|
virtualChangeSet, err := tc.validateAndInsertBlockNoLock(block, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, nil, err
|
return nil, nil, err
|
||||||
}
|
}
|
||||||
@ -112,6 +112,13 @@ func (tc *testConsensus) AddUTXOInvalidBlock(parentHashes []*externalapi.DomainH
|
|||||||
return consensushashing.BlockHash(block), virtualChangeSet, nil
|
return consensushashing.BlockHash(block), virtualChangeSet, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (tc *testConsensus) ResolveVirtualWithMaxParam(maxBlocksToResolve uint64) (bool, error) {
|
||||||
|
tc.lock.Lock()
|
||||||
|
defer tc.lock.Unlock()
|
||||||
|
|
||||||
|
return tc.resolveVirtualNoLock(maxBlocksToResolve)
|
||||||
|
}
|
||||||
|
|
||||||
// jsonBlock is a json representation of a block in mine format
|
// jsonBlock is a json representation of a block in mine format
|
||||||
type jsonBlock struct {
|
type jsonBlock struct {
|
||||||
ID string `json:"id"`
|
ID string `json:"id"`
|
||||||
|
@ -318,7 +318,7 @@ func TestCheckLockTimeVerifyConditionedByAbsoluteTime(t *testing.T) {
|
|||||||
blockHeader := tipBlock.Header.ToMutable()
|
blockHeader := tipBlock.Header.ToMutable()
|
||||||
blockHeader.SetTimeInMilliseconds(timeStampBlockE + i*1000)
|
blockHeader.SetTimeInMilliseconds(timeStampBlockE + i*1000)
|
||||||
tipBlock.Header = blockHeader.ToImmutable()
|
tipBlock.Header = blockHeader.ToImmutable()
|
||||||
_, err = testConsensus.ValidateAndInsertBlock(tipBlock, true)
|
err = testConsensus.ValidateAndInsertBlock(tipBlock, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("Error validating and inserting tip block: %v", err)
|
t.Fatalf("Error validating and inserting tip block: %v", err)
|
||||||
}
|
}
|
||||||
@ -439,7 +439,7 @@ func TestCheckLockTimeVerifyConditionedByAbsoluteTimeWithWrongLockTime(t *testin
|
|||||||
blockHeader := tipBlock.Header.ToMutable()
|
blockHeader := tipBlock.Header.ToMutable()
|
||||||
blockHeader.SetTimeInMilliseconds(timeStampBlockE + i*1000)
|
blockHeader.SetTimeInMilliseconds(timeStampBlockE + i*1000)
|
||||||
tipBlock.Header = blockHeader.ToImmutable()
|
tipBlock.Header = blockHeader.ToImmutable()
|
||||||
_, err = testConsensus.ValidateAndInsertBlock(tipBlock, true)
|
err = testConsensus.ValidateAndInsertBlock(tipBlock, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("Error validating and inserting tip block: %v", err)
|
t.Fatalf("Error validating and inserting tip block: %v", err)
|
||||||
}
|
}
|
||||||
|
@ -57,7 +57,7 @@ func TestCreateStagingConsensus(t *testing.T) {
|
|||||||
},
|
},
|
||||||
},
|
},
|
||||||
}
|
}
|
||||||
_, err = domainInstance.StagingConsensus().ValidateAndInsertBlockWithTrustedData(genesisWithTrustedData, true)
|
err = domainInstance.StagingConsensus().ValidateAndInsertBlockWithTrustedData(genesisWithTrustedData, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("ValidateAndInsertBlockWithTrustedData: %+v", err)
|
t.Fatalf("ValidateAndInsertBlockWithTrustedData: %+v", err)
|
||||||
}
|
}
|
||||||
@ -74,7 +74,7 @@ func TestCreateStagingConsensus(t *testing.T) {
|
|||||||
t.Fatalf("BuildBlock: %+v", err)
|
t.Fatalf("BuildBlock: %+v", err)
|
||||||
}
|
}
|
||||||
|
|
||||||
_, err = domainInstance.StagingConsensus().ValidateAndInsertBlock(block, true)
|
err = domainInstance.StagingConsensus().ValidateAndInsertBlock(block, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("ValidateAndInsertBlock: %+v", err)
|
t.Fatalf("ValidateAndInsertBlock: %+v", err)
|
||||||
}
|
}
|
||||||
@ -120,7 +120,7 @@ func TestCreateStagingConsensus(t *testing.T) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
addGenesisToStagingConsensus()
|
addGenesisToStagingConsensus()
|
||||||
_, err = domainInstance.StagingConsensus().ValidateAndInsertBlock(block, true)
|
err = domainInstance.StagingConsensus().ValidateAndInsertBlock(block, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("ValidateAndInsertBlock: %+v", err)
|
t.Fatalf("ValidateAndInsertBlock: %+v", err)
|
||||||
}
|
}
|
||||||
|
@ -106,7 +106,7 @@ func syncConsensuses(syncer, syncee externalapi.Consensus) error {
|
|||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
_, err = syncee.ValidateAndInsertBlockWithTrustedData(blockWithTrustedData, false)
|
err = syncee.ValidateAndInsertBlockWithTrustedData(blockWithTrustedData, false)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
@ -161,7 +161,7 @@ func syncConsensuses(syncer, syncee externalapi.Consensus) error {
|
|||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
_, err = syncee.ValidateAndInsertBlock(block, false)
|
err = syncee.ValidateAndInsertBlock(block, false)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
@ -232,7 +232,7 @@ func syncConsensuses(syncer, syncee externalapi.Consensus) error {
|
|||||||
log.Infof("Resolving virtual. Estimated progress: %d%%", percents)
|
log.Infof("Resolving virtual. Estimated progress: %d%%", percents)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
_, isCompletelyResolved, err := syncee.ResolveVirtual()
|
isCompletelyResolved, err := syncee.ResolveVirtual()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
@ -122,7 +122,7 @@ func mineOrFetchBlock(blockData JSONBlock, mdb *miningDB, testConsensus testapi.
|
|||||||
SolveBlock(block)
|
SolveBlock(block)
|
||||||
}
|
}
|
||||||
|
|
||||||
_, err = testConsensus.ValidateAndInsertBlock(block, true)
|
err = testConsensus.ValidateAndInsertBlock(block, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, errors.Wrap(err, "error in ValidateAndInsertBlock")
|
return nil, errors.Wrap(err, "error in ValidateAndInsertBlock")
|
||||||
}
|
}
|
||||||
|
@ -57,7 +57,7 @@ func TestGenerateFastPruningIBDTest(t *testing.T) {
|
|||||||
}
|
}
|
||||||
header.SetNonce(tip.Header.Nonce() + i)
|
header.SetNonce(tip.Header.Nonce() + i)
|
||||||
block := &externalapi.DomainBlock{Header: header.ToImmutable(), Transactions: tip.Transactions}
|
block := &externalapi.DomainBlock{Header: header.ToImmutable(), Transactions: tip.Transactions}
|
||||||
_, err = tc.ValidateAndInsertBlock(block, true)
|
err = tc.ValidateAndInsertBlock(block, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("ValidateAndInsertBlock: %+v", err)
|
t.Fatalf("ValidateAndInsertBlock: %+v", err)
|
||||||
}
|
}
|
||||||
@ -103,7 +103,7 @@ func TestGenerateFastPruningIBDTest(t *testing.T) {
|
|||||||
t.Fatal(err)
|
t.Fatal(err)
|
||||||
}
|
}
|
||||||
|
|
||||||
_, err = tc.ValidateAndInsertBlock(block, true)
|
err = tc.ValidateAndInsertBlock(block, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("ValidateAndInsertBlock: %+v", err)
|
t.Fatalf("ValidateAndInsertBlock: %+v", err)
|
||||||
}
|
}
|
||||||
@ -124,7 +124,7 @@ func TestGenerateFastPruningIBDTest(t *testing.T) {
|
|||||||
t.Fatal(err)
|
t.Fatal(err)
|
||||||
}
|
}
|
||||||
|
|
||||||
_, err = tc.ValidateAndInsertBlock(block, true)
|
err = tc.ValidateAndInsertBlock(block, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("ValidateAndInsertBlock: %+v", err)
|
t.Fatalf("ValidateAndInsertBlock: %+v", err)
|
||||||
}
|
}
|
||||||
|
@ -49,7 +49,7 @@ func prepareBlocks() (blocks []*externalapi.DomainBlock, topBlock *externalapi.D
|
|||||||
}
|
}
|
||||||
|
|
||||||
mine.SolveBlock(block)
|
mine.SolveBlock(block)
|
||||||
_, err = testConsensus.ValidateAndInsertBlock(block, true)
|
err = testConsensus.ValidateAndInsertBlock(block, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, nil, errors.Wrap(err, "error in ValidateAndInsertBlock")
|
return nil, nil, errors.Wrap(err, "error in ValidateAndInsertBlock")
|
||||||
}
|
}
|
||||||
|
@ -86,7 +86,7 @@ func testReorg(cfg *configFlags) {
|
|||||||
block.Header = mutableHeader.ToImmutable()
|
block.Header = mutableHeader.ToImmutable()
|
||||||
}
|
}
|
||||||
|
|
||||||
_, err = tcAttacker.ValidateAndInsertBlock(block, true)
|
err = tcAttacker.ValidateAndInsertBlock(block, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
panic(err)
|
panic(err)
|
||||||
}
|
}
|
||||||
@ -122,7 +122,7 @@ func testReorg(cfg *configFlags) {
|
|||||||
if i%100 == 0 {
|
if i%100 == 0 {
|
||||||
log.Infof("Validated %d blocks from the attacker chain", i)
|
log.Infof("Validated %d blocks from the attacker chain", i)
|
||||||
}
|
}
|
||||||
_, err := tc.ValidateAndInsertBlock(block, true)
|
err := tc.ValidateAndInsertBlock(block, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
panic(err)
|
panic(err)
|
||||||
}
|
}
|
||||||
|
Loading…
x
Reference in New Issue
Block a user