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:
Michael Sutton 2022-06-15 02:52:14 +03:00 committed by GitHub
parent b2648aa5bd
commit d957a6d93a
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
32 changed files with 352 additions and 120 deletions

View File

@ -107,7 +107,7 @@ func (f *FlowContext) AddBlock(block *externalapi.DomainBlock) error {
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 errors.As(err, &ruleerrors.RuleError{}) {
log.Warnf("Validation failed for block %s: %s", consensushashing.BlockHash(block), err)

View File

@ -141,7 +141,7 @@ func (f *FlowContext) unorphanBlock(orphanHash externalapi.DomainHash) (bool, er
}
delete(f.orphans, orphanHash)
_, err := f.domain.Consensus().ValidateAndInsertBlock(orphanBlock, true)
err := f.domain.Consensus().ValidateAndInsertBlock(orphanBlock, true)
if err != nil {
if errors.As(err, &ruleerrors.RuleError{}) {
log.Warnf("Validation failed for orphan block %s: %s", orphanHash, err)

View File

@ -321,7 +321,7 @@ func (flow *handleRelayInvsFlow) readMsgBlock() (msgBlock *appmessage.MsgBlock,
func (flow *handleRelayInvsFlow) processBlock(block *externalapi.DomainBlock) ([]*externalapi.DomainHash, error) {
blockHash := consensushashing.BlockHash(block)
_, err := flow.Domain().Consensus().ValidateAndInsertBlock(block, true)
err := flow.Domain().Consensus().ValidateAndInsertBlock(block, true)
if err != nil {
if !errors.As(err, &ruleerrors.RuleError{}) {
return nil, errors.Wrapf(err, "failed to process block %s", blockHash)

View File

@ -488,7 +488,7 @@ func (flow *handleIBDFlow) processHeader(consensus externalapi.Consensus, msgBlo
log.Debugf("Block header %s is already in the DAG. Skipping...", blockHash)
return nil
}
_, err = consensus.ValidateAndInsertBlock(block, false)
err = consensus.ValidateAndInsertBlock(block, false)
if err != nil {
if !errors.As(err, &ruleerrors.RuleError{}) {
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
}
_, err = flow.Domain().Consensus().ValidateAndInsertBlock(block, false)
err = flow.Domain().Consensus().ValidateAndInsertBlock(block, false)
if err != nil {
if errors.Is(err, ruleerrors.ErrDuplicateBlock) {
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)
}
_, isCompletelyResolved, err := flow.Domain().Consensus().ResolveVirtual()
isCompletelyResolved, err := flow.Domain().Consensus().ResolveVirtual()
if err != nil {
return err
}

View File

@ -280,7 +280,7 @@ func (flow *handleIBDFlow) processBlockWithTrustedData(
blockWithTrustedData.GHOSTDAGData = append(blockWithTrustedData.GHOSTDAGData, appmessage.GHOSTDAGHashPairToDomainGHOSTDAGHashPair(data.GHOSTDAGData[index]))
}
_, err := consensus.ValidateAndInsertBlockWithTrustedData(blockWithTrustedData, false)
err := consensus.ValidateAndInsertBlockWithTrustedData(blockWithTrustedData, false)
return err
}

View File

@ -61,17 +61,18 @@ type consensus struct {
blocksWithTrustedDataDAAWindowStore model.BlocksWithTrustedDataDAAWindowStore
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()
defer s.lock.Unlock()
virtualChangeSet, _, err := s.blockProcessor.ValidateAndInsertBlockWithTrustedData(block, validateUTXO)
_, _, err := s.blockProcessor.ValidateAndInsertBlockWithTrustedData(block, validateUTXO)
if err != nil {
return nil, err
return err
}
return virtualChangeSet, nil
return nil
}
// Init initializes consensus
@ -193,21 +194,47 @@ 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) (*externalapi.VirtualChangeSet, error) {
func (s *consensus) ValidateAndInsertBlock(block *externalapi.DomainBlock, shouldValidateAgainstUTXO bool) error {
s.lock.Lock()
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 {
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)
if err != nil {
return nil, err
}
err = s.sendVirtualChangedEvent(virtualChangeSet, shouldValidateAgainstUTXO)
err = s.sendVirtualChangedEvent(virtualChangeSet, updateVirtual)
if err != nil {
return nil, err
}
@ -861,7 +888,7 @@ func (s *consensus) PopulateMass(transaction *externalapi.DomainTransaction) {
s.transactionValidator.PopulateMass(transaction)
}
func (s *consensus) ResolveVirtual() (*externalapi.VirtualChangeSet, bool, error) {
func (s *consensus) ResolveVirtual() (bool, error) {
s.lock.Lock()
defer s.lock.Unlock()
@ -869,28 +896,33 @@ func (s *consensus) ResolveVirtual() (*externalapi.VirtualChangeSet, bool, error
// 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.
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 {
return nil, false, err
return false, err
}
s.virtualNotUpdated = !isCompletelyResolved
stagingArea := model.NewStagingArea()
err = s.pruningManager.UpdatePruningPointByVirtual(stagingArea)
if err != nil {
return nil, false, err
return false, err
}
err = staging.CommitAllChanges(s.databaseContext, stagingArea)
if err != nil {
return nil, false, err
return false, err
}
err = s.sendVirtualChangedEvent(virtualChangeSet, true)
if err != nil {
return nil, false, err
return false, err
}
return virtualChangeSet, isCompletelyResolved, nil
return isCompletelyResolved, nil
}
func (s *consensus) BuildPruningPointProof() (*externalapi.PruningPointProof, error) {

View File

@ -27,7 +27,7 @@ func TestConsensus_GetBlockInfo(t *testing.T) {
newHeader := invalidBlock.Header.ToMutable()
newHeader.SetTimeInMilliseconds(0)
invalidBlock.Header = newHeader.ToImmutable()
_, err = consensus.ValidateAndInsertBlock(invalidBlock, true)
err = consensus.ValidateAndInsertBlock(invalidBlock, true)
if !errors.Is(err, ruleerrors.ErrTimeTooOld) {
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)
}
_, err = consensus.ValidateAndInsertBlock(validBlock, true)
err = consensus.ValidateAndInsertBlock(validBlock, true)
if err != nil {
t.Fatalf("consensus.ValidateAndInsertBlock with a block straight from consensus.BuildBlock should not fail: %v", err)
}

View File

@ -33,7 +33,7 @@ func TestFinality(t *testing.T) {
return nil, err
}
_, err = consensus.ValidateAndInsertBlock(block, true)
err = consensus.ValidateAndInsertBlock(block, true)
if err != nil {
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
}
_, err = consensus.ValidateAndInsertBlock(block, true)
err = consensus.ValidateAndInsertBlock(block, true)
if err == nil {
return block, false
} 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) {
_, err := consensus.ValidateAndInsertBlock(block, true)
err := consensus.ValidateAndInsertBlock(block, true)
if err != nil {
t.Fatalf("TestBoundedMergeDepth: %s got unexpected error from ProcessBlock: %+v", name, err)
@ -225,7 +225,7 @@ func TestBoundedMergeDepth(t *testing.T) {
if err != nil {
t.Fatalf("TestBoundedMergeDepth: Failed building block: %+v", err)
}
_, err = consensus.ValidateAndInsertBlock(block, true)
err = consensus.ValidateAndInsertBlock(block, true)
if err != nil {
t.Fatalf("TestBoundedMergeDepth: Failed Inserting block to consensus: %v", err)
}
@ -268,7 +268,7 @@ func TestBoundedMergeDepth(t *testing.T) {
t.Fatalf("GetBlockHeader: %+v", err)
}
_, err = tcSyncee.ValidateAndInsertBlock(block, true)
err = tcSyncee.ValidateAndInsertBlock(block, true)
if err != nil {
t.Fatalf("ValidateAndInsertBlock %d: %+v", i, err)
}
@ -556,7 +556,7 @@ func TestFinalityResolveVirtual(t *testing.T) {
block.Header = mutableHeader.ToImmutable()
}
_, err = tcAttacker.ValidateAndInsertBlock(block, true)
err = tcAttacker.ValidateAndInsertBlock(block, true)
if err != nil {
panic(err)
}
@ -583,14 +583,14 @@ func TestFinalityResolveVirtual(t *testing.T) {
t.Logf("Side chain tip (%s) blue score %d", sideChainTipHash, sideChainTipGHOSTDAGData.BlueScore())
for _, block := range sideChain {
_, err := tc.ValidateAndInsertBlock(block, false)
err := tc.ValidateAndInsertBlock(block, false)
if err != nil {
panic(err)
}
}
for i := 0; ; i++ {
_, isCompletelyResolved, err := tc.ResolveVirtual()
isCompletelyResolved, err := tc.ResolveVirtual()
if err != nil {
panic(err)
}

View File

@ -5,8 +5,8 @@ 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) (*VirtualChangeSet, error)
ValidateAndInsertBlockWithTrustedData(block *BlockWithTrustedData, validateUTXO bool) (*VirtualChangeSet, error)
ValidateAndInsertBlock(block *DomainBlock, shouldValidateAgainstUTXO bool) error
ValidateAndInsertBlockWithTrustedData(block *BlockWithTrustedData, validateUTXO bool) error
ValidateTransactionAndPopulateWithConsensusData(transaction *DomainTransaction) error
ImportPruningPoints(pruningPoints []BlockHeader) error
BuildPruningPointProof() (*PruningPointProof, error)
@ -48,7 +48,7 @@ type Consensus interface {
Anticone(blockHash *DomainHash) ([]*DomainHash, error)
EstimateNetworkHashesPerSecond(startHash *DomainHash, windowSize int) (uint64, error)
PopulateMass(transaction *DomainTransaction)
ResolveVirtual() (*VirtualChangeSet, bool, error)
ResolveVirtual() (bool, error)
BlockDAAWindowHashes(blockHash *DomainHash) ([]*DomainHash, error)
TrustedDataDataDAAHeader(trustedBlockHash, daaBlockHash *DomainHash, daaBlockWindowIndex uint64) (*TrustedDataDataDAAHeader, error)
TrustedBlockAssociatedGHOSTDAGDataBlockHashes(blockHash *DomainHash) ([]*DomainHash, error)

View File

@ -49,6 +49,8 @@ type TestConsensus interface {
*externalapi.VirtualChangeSet, error)
UpdatePruningPointByVirtual() error
ResolveVirtualWithMaxParam(maxBlocksToResolve uint64) (bool, error)
MineJSON(r io.Reader, blockType MineJSONBlockType) (tips []*externalapi.DomainHash, err error)
ToJSON(w io.Writer) error

View File

@ -6,7 +6,6 @@ import (
"fmt"
"github.com/kaspanet/kaspad/domain/consensus/model"
"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/utils/consensushashing"
"github.com/kaspanet/kaspad/domain/consensus/utils/multiset"
@ -165,12 +164,7 @@ func (bp *blockProcessor) validateAndInsertBlock(stagingArea *model.StagingArea,
if reversalData != nil {
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
// 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 {
if err != nil {
return nil, externalapi.StatusInvalid, err
}
}

View File

@ -79,7 +79,7 @@ func TestBlockStatus(t *testing.T) {
disqualifiedBlock.Header.PruningPoint(),
)
_, err = tc.ValidateAndInsertBlock(disqualifiedBlock, true)
err = tc.ValidateAndInsertBlock(disqualifiedBlock, true)
if err != nil {
t.Fatalf("ValidateAndInsertBlock: %+v", err)
}
@ -106,7 +106,7 @@ func TestBlockStatus(t *testing.T) {
disqualifiedBlock.Header.PruningPoint(),
)
_, err = tc.ValidateAndInsertBlock(invalidBlock, true)
err = tc.ValidateAndInsertBlock(invalidBlock, true)
if err == nil {
t.Fatalf("block is expected to be invalid")
}
@ -139,11 +139,11 @@ func TestValidateAndInsertErrors(t *testing.T) {
if err != nil {
t.Fatalf("AddBlock: %+v", err)
}
_, err = tc.ValidateAndInsertBlock(blockWithStatusInvalid, true)
err = tc.ValidateAndInsertBlock(blockWithStatusInvalid, true)
if err == nil {
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) {
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 {
t.Fatalf("AddBlock: %+v", err)
}
_, err = tc.ValidateAndInsertBlock(block, true)
err = tc.ValidateAndInsertBlock(block, true)
if err != nil {
t.Fatalf("ValidateAndInsertBlock: %+v", err)
}
// resend the same block.
_, err = tc.ValidateAndInsertBlock(block, true)
err = tc.ValidateAndInsertBlock(block, true)
if err == nil || !errors.Is(err, ruleerrors.ErrDuplicateBlock) {
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)
}
onlyHeader.Transactions = []*externalapi.DomainTransaction{}
_, err = tc.ValidateAndInsertBlock(onlyHeader, true)
err = tc.ValidateAndInsertBlock(onlyHeader, true)
if err != nil {
t.Fatalf("AddBlock: %+v", err)
}
// resend the same header.
_, err = tc.ValidateAndInsertBlock(onlyHeader, true)
err = tc.ValidateAndInsertBlock(onlyHeader, true)
if err == nil || !errors.Is(err, ruleerrors.ErrDuplicateBlock) {
t.Fatalf("Expected block to be invalid with err: %v, instead found: %v", ruleerrors.ErrDuplicateBlock, err)
}

View File

@ -26,7 +26,7 @@ func addBlock(tc testapi.TestConsensus, parentHashes []*externalapi.DomainHash,
}
blockHash := consensushashing.BlockHash(block)
_, err = tc.ValidateAndInsertBlock(block, true)
err = tc.ValidateAndInsertBlock(block, true)
if err != nil {
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 {
t.Fatalf("ValidateAndInsertBlockWithTrustedData: %+v", err)
}
@ -169,7 +169,7 @@ func TestValidateAndInsertImportedPruningPoint(t *testing.T) {
t.Fatalf("GetBlockHeader: %+v", err)
}
_, err = synceeStaging.ValidateAndInsertBlock(&externalapi.DomainBlock{Header: header}, false)
err = synceeStaging.ValidateAndInsertBlock(&externalapi.DomainBlock{Header: header}, false)
if err != nil {
t.Fatalf("ValidateAndInsertBlock %d: %+v", i, err)
}
@ -273,7 +273,7 @@ func TestValidateAndInsertImportedPruningPoint(t *testing.T) {
t.Fatalf("GetBlock: %+v", err)
}
_, err = synceeStaging.ValidateAndInsertBlock(block, true)
err = synceeStaging.ValidateAndInsertBlock(block, true)
if err != nil {
t.Fatalf("ValidateAndInsertBlock: %+v", err)
}
@ -299,7 +299,7 @@ func TestValidateAndInsertImportedPruningPoint(t *testing.T) {
t.Fatalf("GetBlock: %+v", err)
}
_, err = synceeStaging.ValidateAndInsertBlock(tip, true)
err = synceeStaging.ValidateAndInsertBlock(tip, true)
if err != nil {
t.Fatalf("ValidateAndInsertBlock: %+v", err)
}
@ -346,7 +346,7 @@ func TestValidateAndInsertImportedPruningPoint(t *testing.T) {
t.Fatalf("GetBlock: %+v", err)
}
_, err = tcSyncee1.ValidateAndInsertBlock(block, true)
err = tcSyncee1.ValidateAndInsertBlock(block, true)
if err != nil {
t.Fatalf("ValidateAndInsertBlock: %+v", err)
}
@ -478,7 +478,7 @@ func TestGetPruningPointUTXOs(t *testing.T) {
if err != nil {
t.Fatalf("Error building block above genesis: %+v", err)
}
_, err = testConsensus.ValidateAndInsertBlock(blockAboveGenesis, true)
err = testConsensus.ValidateAndInsertBlock(blockAboveGenesis, true)
if err != nil {
t.Fatalf("Error validating and inserting block above genesis: %+v", err)
}
@ -490,7 +490,7 @@ func TestGetPruningPointUTXOs(t *testing.T) {
if err != nil {
t.Fatalf("Error building block with spendable coinbase: %+v", err)
}
_, err = testConsensus.ValidateAndInsertBlock(blockWithSpendableCoinbase, true)
err = testConsensus.ValidateAndInsertBlock(blockWithSpendableCoinbase, true)
if err != nil {
t.Fatalf("Error validating and inserting block with spendable coinbase: %+v", err)
}
@ -529,7 +529,7 @@ func TestGetPruningPointUTXOs(t *testing.T) {
if err != nil {
t.Fatalf("Error building including block: %+v", err)
}
_, err = testConsensus.ValidateAndInsertBlock(includingBlock, true)
err = testConsensus.ValidateAndInsertBlock(includingBlock, true)
if err != nil {
t.Fatalf("Error validating and inserting including block: %+v", err)
}
@ -540,7 +540,7 @@ func TestGetPruningPointUTXOs(t *testing.T) {
if err != nil {
t.Fatalf("Error building block: %+v", err)
}
_, err = testConsensus.ValidateAndInsertBlock(block, true)
err = testConsensus.ValidateAndInsertBlock(block, true)
if err != nil {
t.Fatalf("Error validating and inserting block: %+v", err)
}
@ -636,7 +636,7 @@ func BenchmarkGetPruningPointUTXOs(b *testing.B) {
if err != nil {
b.Fatalf("Error building block with spendable coinbase: %+v", err)
}
_, err = testConsensus.ValidateAndInsertBlock(blockWithSpendableCoinbase, true)
err = testConsensus.ValidateAndInsertBlock(blockWithSpendableCoinbase, true)
if err != nil {
b.Fatalf("Error validating and inserting block with spendable coinbase: %+v", err)
}
@ -674,7 +674,7 @@ func BenchmarkGetPruningPointUTXOs(b *testing.B) {
if err != nil {
b.Fatalf("Error building block: %+v", err)
}
_, err = testConsensus.ValidateAndInsertBlock(block, true)
err = testConsensus.ValidateAndInsertBlock(block, true)
if err != nil {
b.Fatalf("Error validating and inserting block: %+v", err)
}
@ -694,7 +694,7 @@ func BenchmarkGetPruningPointUTXOs(b *testing.B) {
if err != nil {
b.Fatalf("Error building block: %+v", err)
}
_, err = testConsensus.ValidateAndInsertBlock(block, true)
err = testConsensus.ValidateAndInsertBlock(block, true)
if err != nil {
b.Fatalf("Error validating and inserting block: %+v", err)
}

View File

@ -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) {
t.Fatalf("Unexpected error: %+v", err)
}
@ -117,7 +117,7 @@ func TestCheckParentBlockBodiesExist(t *testing.T) {
}
// Add only the header of anticonePruningBlock
_, err = tc.ValidateAndInsertBlock(&externalapi.DomainBlock{
err = tc.ValidateAndInsertBlock(&externalapi.DomainBlock{
Header: anticonePruningBlock.Header,
Transactions: nil,
}, true)
@ -143,7 +143,7 @@ func TestCheckParentBlockBodiesExist(t *testing.T) {
// Add anticonePruningBlock's body and check that it's valid to point to
// a header only block in the past of the pruning point.
_, err = tc.ValidateAndInsertBlock(anticonePruningBlock, true)
err = tc.ValidateAndInsertBlock(anticonePruningBlock, true)
if err != nil {
t.Fatalf("ValidateAndInsertBlock: %+v", err)
}
@ -189,7 +189,7 @@ func TestIsFinalizedTransaction(t *testing.T) {
if err != nil {
t.Fatalf("Error getting block: %+v", err)
}
_, err = tc.ValidateAndInsertBlock(block, true)
err = tc.ValidateAndInsertBlock(block, true)
if err != nil {
t.Fatalf("Error Inserting block: %+v", err)
}

View File

@ -1052,14 +1052,14 @@ func CheckBlockHashMerkleRoot(t *testing.T, tc testapi.TestConsensus, consensusC
blockWithInvalidMerkleRoot := block.Clone()
blockWithInvalidMerkleRoot.Transactions[0].Version += 1
_, err = tc.ValidateAndInsertBlock(blockWithInvalidMerkleRoot, true)
err = tc.ValidateAndInsertBlock(blockWithInvalidMerkleRoot, true)
if !errors.Is(err, ruleerrors.ErrBadMerkleRoot) {
t.Fatalf("Unexpected error: %+v", err)
}
// Check that a block with invalid merkle root is not marked as invalid
// and can be re-added with the right transactions.
_, err = tc.ValidateAndInsertBlock(block, true)
err = tc.ValidateAndInsertBlock(block, true)
if err != nil {
t.Fatalf("ValidateAndInsertBlock: %+v", err)
}

View File

@ -34,7 +34,7 @@ func TestValidateMedianTime(t *testing.T) {
newHeader := block.Header.ToMutable()
newHeader.SetTimeInMilliseconds(blockTime)
block.Header = newHeader.ToImmutable()
_, err = tc.ValidateAndInsertBlock(block, true)
err = tc.ValidateAndInsertBlock(block, true)
if !errors.Is(err, expectedErr) {
t.Fatalf("expected error %s but got %+v", expectedErr, err)
}
@ -127,7 +127,7 @@ func TestCheckParentsIncest(t *testing.T) {
Transactions: nil,
}
_, err = tc.ValidateAndInsertBlock(directParentsRelationBlock, true)
err = tc.ValidateAndInsertBlock(directParentsRelationBlock, true)
if !errors.Is(err, ruleerrors.ErrInvalidParentsRelation) {
t.Fatalf("unexpected error %+v", err)
}
@ -150,7 +150,7 @@ func TestCheckParentsIncest(t *testing.T) {
Transactions: nil,
}
_, err = tc.ValidateAndInsertBlock(indirectParentsRelationBlock, true)
err = tc.ValidateAndInsertBlock(indirectParentsRelationBlock, true)
if !errors.Is(err, ruleerrors.ErrInvalidParentsRelation) {
t.Fatalf("unexpected error %+v", err)
}

View File

@ -78,7 +78,7 @@ func CheckBlockVersion(t *testing.T, tc testapi.TestConsensus, consensusConfig *
block.Header.PruningPoint(),
)
_, err = tc.ValidateAndInsertBlock(block, true)
err = tc.ValidateAndInsertBlock(block, true)
if !errors.Is(err, ruleerrors.ErrWrongBlockVersion) {
t.Fatalf("Unexpected error: %+v", err)
}
@ -118,7 +118,7 @@ func CheckBlockTimestampInIsolation(t *testing.T, tc testapi.TestConsensus, cfg
block.Header.PruningPoint(),
)
_, err = tc.ValidateAndInsertBlock(block, true)
err = tc.ValidateAndInsertBlock(block, true)
if !errors.Is(err, ruleerrors.ErrTimeTooMuchInTheFuture) {
t.Fatalf("Unexpected error: %+v", err)
}

View File

@ -39,7 +39,7 @@ func TestPOW(t *testing.T) {
t.Fatal(err)
}
invalidBlockWrongPOW = solveBlockWithWrongPOW(invalidBlockWrongPOW)
_, err = tc.ValidateAndInsertBlock(invalidBlockWrongPOW, true)
err = tc.ValidateAndInsertBlock(invalidBlockWrongPOW, true)
if !errors.Is(err, ruleerrors.ErrInvalidPoW) {
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(),
)
_, err = tc.ValidateAndInsertBlock(abovePowMaxBlock, true)
err = tc.ValidateAndInsertBlock(abovePowMaxBlock, true)
if !errors.Is(err, ruleerrors.ErrTargetTooHigh) {
t.Fatalf("Unexpected error: %+v", err)
}
@ -90,7 +90,7 @@ func TestPOW(t *testing.T) {
negativeTargetBlock.Header.PruningPoint(),
)
_, err = tc.ValidateAndInsertBlock(negativeTargetBlock, true)
err = tc.ValidateAndInsertBlock(negativeTargetBlock, true)
if !errors.Is(err, ruleerrors.ErrNegativeTarget) {
t.Fatalf("Unexpected error: %+v", err)
}
@ -104,7 +104,7 @@ func TestPOW(t *testing.T) {
// Difficulty is too high on mainnet to actually mine.
if consensusConfig.Name != "kaspa-mainnet" {
mining.SolveBlock(validBlock, random)
_, err = tc.ValidateAndInsertBlock(validBlock, true)
err = tc.ValidateAndInsertBlock(validBlock, true)
if err != nil {
t.Fatal(err)
}
@ -161,7 +161,7 @@ func TestCheckParentHeadersExist(t *testing.T) {
orphanBlock.Header.PruningPoint(),
)
_, err = tc.ValidateAndInsertBlock(orphanBlock, true)
err = tc.ValidateAndInsertBlock(orphanBlock, true)
errMissingParents := &ruleerrors.ErrMissingParents{}
if !errors.As(err, errMissingParents) {
t.Fatalf("Unexpected error: %+v", err)
@ -193,7 +193,7 @@ func TestCheckParentHeadersExist(t *testing.T) {
orphanBlock.Header.PruningPoint(),
)
_, err = tc.ValidateAndInsertBlock(invalidBlock, true)
err = tc.ValidateAndInsertBlock(invalidBlock, true)
if !errors.Is(err, ruleerrors.ErrTransactionVersionIsUnknown) {
t.Fatalf("Unexpected error: %+v", err)
}
@ -220,7 +220,7 @@ func TestCheckParentHeadersExist(t *testing.T) {
invalidBlockChild.Header.PruningPoint(),
)
_, err = tc.ValidateAndInsertBlock(invalidBlockChild, true)
err = tc.ValidateAndInsertBlock(invalidBlockChild, true)
if !errors.Is(err, ruleerrors.ErrInvalidAncestorBlock) {
t.Fatalf("Unexpected error: %+v", err)
}
@ -284,7 +284,7 @@ func TestCheckPruningPointViolation(t *testing.T) {
blockWithPruningViolation.Header.PruningPoint(),
)
_, err = tc.ValidateAndInsertBlock(blockWithPruningViolation, true)
err = tc.ValidateAndInsertBlock(blockWithPruningViolation, true)
if !errors.Is(err, ruleerrors.ErrPruningPointViolation) {
t.Fatalf("Unexpected error: %+v", err)
}

View File

@ -5,7 +5,6 @@ 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"
)
@ -79,12 +78,7 @@ func (csm *consensusStateManager) ResolveVirtual(maxBlocksToResolve uint64) (*ex
if reversalData != nil {
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
// 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 {
if err != nil {
return nil, false, err
}
}

View File

@ -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)
}
}
})
}

View File

@ -3,18 +3,10 @@ package consensusstatemanager
import (
"github.com/kaspanet/kaspad/domain/consensus/model"
"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/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,
reversalData *model.UTXODiffReversalData) error {
@ -57,9 +49,6 @@ func (csm *consensusStateManager) ReverseUTXODiffs(tipHash *externalapi.DomainHa
currentBlockUTXODiffChild, err := csm.utxoDiffStore.UTXODiffChild(csm.databaseContext, readStagingArea, currentBlock)
if err != nil {
if database.IsNotFoundError(err) {
return errors.Wrapf(ErrReverseUTXODiffsUTXODiffChildNotFound, "UTXO diff child was not found for block %s", currentBlock)
}
return err
}
currentBlockGHOSTDAGData, err := csm.ghostdagDataStore.Get(csm.databaseContext, readStagingArea, currentBlock, false)

View File

@ -84,7 +84,7 @@ func TestConsensusStateManager_pickVirtualParents(t *testing.T) {
if err != nil {
t.Fatalf("Failed building a block: %v", err)
}
_, err = tc.ValidateAndInsertBlock(block, true)
err = tc.ValidateAndInsertBlock(block, true)
if err != nil {
t.Fatalf("Failed Inserting block to tc: %v", err)
}

View File

@ -65,7 +65,7 @@ func TestDifficulty(t *testing.T) {
newHeader := block.Header.ToMutable()
newHeader.SetTimeInMilliseconds(blockTime)
block.Header = newHeader.ToImmutable()
_, err = tc.ValidateAndInsertBlock(block, true)
err = tc.ValidateAndInsertBlock(block, true)
if err != nil {
t.Fatalf("ValidateAndInsertBlock: %+v", err)
}

View File

@ -34,7 +34,7 @@ func TestPastMedianTime(t *testing.T) {
newHeader := block.Header.ToMutable()
newHeader.SetTimeInMilliseconds(blockTime)
block.Header = newHeader.ToImmutable()
_, err = tc.ValidateAndInsertBlock(block, true)
err = tc.ValidateAndInsertBlock(block, true)
if err != nil {
t.Fatalf("ValidateAndInsertBlock: %+v", err)
}

View File

@ -61,7 +61,7 @@ func (tc *testConsensus) AddBlock(parentHashes []*externalapi.DomainHash, coinba
return nil, nil, err
}
virtualChangeSet, _, err := tc.blockProcessor.ValidateAndInsertBlock(block, true)
virtualChangeSet, err := tc.validateAndInsertBlockNoLock(block, true)
if err != nil {
return nil, nil, err
}
@ -81,7 +81,7 @@ func (tc *testConsensus) AddUTXOInvalidHeader(parentHashes []*externalapi.Domain
return nil, nil, err
}
virtualChangeSet, _, err := tc.blockProcessor.ValidateAndInsertBlock(&externalapi.DomainBlock{
virtualChangeSet, err := tc.validateAndInsertBlockNoLock(&externalapi.DomainBlock{
Header: header,
Transactions: nil,
}, true)
@ -104,7 +104,7 @@ func (tc *testConsensus) AddUTXOInvalidBlock(parentHashes []*externalapi.DomainH
return nil, nil, err
}
virtualChangeSet, _, err := tc.blockProcessor.ValidateAndInsertBlock(block, true)
virtualChangeSet, err := tc.validateAndInsertBlockNoLock(block, true)
if err != nil {
return nil, nil, err
}
@ -112,6 +112,13 @@ func (tc *testConsensus) AddUTXOInvalidBlock(parentHashes []*externalapi.DomainH
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
type jsonBlock struct {
ID string `json:"id"`

View File

@ -318,7 +318,7 @@ func TestCheckLockTimeVerifyConditionedByAbsoluteTime(t *testing.T) {
blockHeader := tipBlock.Header.ToMutable()
blockHeader.SetTimeInMilliseconds(timeStampBlockE + i*1000)
tipBlock.Header = blockHeader.ToImmutable()
_, err = testConsensus.ValidateAndInsertBlock(tipBlock, true)
err = testConsensus.ValidateAndInsertBlock(tipBlock, true)
if err != nil {
t.Fatalf("Error validating and inserting tip block: %v", err)
}
@ -439,7 +439,7 @@ func TestCheckLockTimeVerifyConditionedByAbsoluteTimeWithWrongLockTime(t *testin
blockHeader := tipBlock.Header.ToMutable()
blockHeader.SetTimeInMilliseconds(timeStampBlockE + i*1000)
tipBlock.Header = blockHeader.ToImmutable()
_, err = testConsensus.ValidateAndInsertBlock(tipBlock, true)
err = testConsensus.ValidateAndInsertBlock(tipBlock, true)
if err != nil {
t.Fatalf("Error validating and inserting tip block: %v", err)
}

View File

@ -57,7 +57,7 @@ func TestCreateStagingConsensus(t *testing.T) {
},
},
}
_, err = domainInstance.StagingConsensus().ValidateAndInsertBlockWithTrustedData(genesisWithTrustedData, true)
err = domainInstance.StagingConsensus().ValidateAndInsertBlockWithTrustedData(genesisWithTrustedData, true)
if err != nil {
t.Fatalf("ValidateAndInsertBlockWithTrustedData: %+v", err)
}
@ -74,7 +74,7 @@ func TestCreateStagingConsensus(t *testing.T) {
t.Fatalf("BuildBlock: %+v", err)
}
_, err = domainInstance.StagingConsensus().ValidateAndInsertBlock(block, true)
err = domainInstance.StagingConsensus().ValidateAndInsertBlock(block, true)
if err != nil {
t.Fatalf("ValidateAndInsertBlock: %+v", err)
}
@ -120,7 +120,7 @@ func TestCreateStagingConsensus(t *testing.T) {
}
addGenesisToStagingConsensus()
_, err = domainInstance.StagingConsensus().ValidateAndInsertBlock(block, true)
err = domainInstance.StagingConsensus().ValidateAndInsertBlock(block, true)
if err != nil {
t.Fatalf("ValidateAndInsertBlock: %+v", err)
}

View File

@ -106,7 +106,7 @@ func syncConsensuses(syncer, syncee externalapi.Consensus) error {
})
}
_, err = syncee.ValidateAndInsertBlockWithTrustedData(blockWithTrustedData, false)
err = syncee.ValidateAndInsertBlockWithTrustedData(blockWithTrustedData, false)
if err != nil {
return err
}
@ -161,7 +161,7 @@ func syncConsensuses(syncer, syncee externalapi.Consensus) error {
return err
}
_, err = syncee.ValidateAndInsertBlock(block, false)
err = syncee.ValidateAndInsertBlock(block, false)
if err != nil {
return err
}
@ -232,7 +232,7 @@ func syncConsensuses(syncer, syncee externalapi.Consensus) error {
log.Infof("Resolving virtual. Estimated progress: %d%%", percents)
}
}
_, isCompletelyResolved, err := syncee.ResolveVirtual()
isCompletelyResolved, err := syncee.ResolveVirtual()
if err != nil {
return err
}

View File

@ -122,7 +122,7 @@ func mineOrFetchBlock(blockData JSONBlock, mdb *miningDB, testConsensus testapi.
SolveBlock(block)
}
_, err = testConsensus.ValidateAndInsertBlock(block, true)
err = testConsensus.ValidateAndInsertBlock(block, true)
if err != nil {
return nil, errors.Wrap(err, "error in ValidateAndInsertBlock")
}

View File

@ -57,7 +57,7 @@ func TestGenerateFastPruningIBDTest(t *testing.T) {
}
header.SetNonce(tip.Header.Nonce() + i)
block := &externalapi.DomainBlock{Header: header.ToImmutable(), Transactions: tip.Transactions}
_, err = tc.ValidateAndInsertBlock(block, true)
err = tc.ValidateAndInsertBlock(block, true)
if err != nil {
t.Fatalf("ValidateAndInsertBlock: %+v", err)
}
@ -103,7 +103,7 @@ func TestGenerateFastPruningIBDTest(t *testing.T) {
t.Fatal(err)
}
_, err = tc.ValidateAndInsertBlock(block, true)
err = tc.ValidateAndInsertBlock(block, true)
if err != nil {
t.Fatalf("ValidateAndInsertBlock: %+v", err)
}
@ -124,7 +124,7 @@ func TestGenerateFastPruningIBDTest(t *testing.T) {
t.Fatal(err)
}
_, err = tc.ValidateAndInsertBlock(block, true)
err = tc.ValidateAndInsertBlock(block, true)
if err != nil {
t.Fatalf("ValidateAndInsertBlock: %+v", err)
}

View File

@ -49,7 +49,7 @@ func prepareBlocks() (blocks []*externalapi.DomainBlock, topBlock *externalapi.D
}
mine.SolveBlock(block)
_, err = testConsensus.ValidateAndInsertBlock(block, true)
err = testConsensus.ValidateAndInsertBlock(block, true)
if err != nil {
return nil, nil, errors.Wrap(err, "error in ValidateAndInsertBlock")
}

View File

@ -86,7 +86,7 @@ func testReorg(cfg *configFlags) {
block.Header = mutableHeader.ToImmutable()
}
_, err = tcAttacker.ValidateAndInsertBlock(block, true)
err = tcAttacker.ValidateAndInsertBlock(block, true)
if err != nil {
panic(err)
}
@ -122,7 +122,7 @@ func testReorg(cfg *configFlags) {
if i%100 == 0 {
log.Infof("Validated %d blocks from the attacker chain", i)
}
_, err := tc.ValidateAndInsertBlock(block, true)
err := tc.ValidateAndInsertBlock(block, true)
if err != nil {
panic(err)
}