[DEV-338] Remove provisional nodes, and panic in case there's an error after verifying the block is valid (#162)

* [DEV-338] Remove provisional nodes, and panic in case there's an error after verifying the block is valid

* [DEV-338] Improved deffered blockNode cleanup and cosmetic changes

* [DEV-338] Fixed dag.index.SetStatusFlags for parents + cosmetic changes

* [DEV-338] Fixed build

* [DEV-338] Fixed comments
This commit is contained in:
Evgeny Khirin 2019-01-14 16:28:05 +02:00 committed by stasatdaglabs
parent e3994cddac
commit 3a5abb6584
8 changed files with 117 additions and 208 deletions

View File

@ -61,6 +61,14 @@ func (dag *BlockDAG) maybeAcceptBlock(block *util.Block, flags BehaviorFlags) er
blockHeader := &block.MsgBlock().Header blockHeader := &block.MsgBlock().Header
newNode := newBlockNode(blockHeader, parents, dag.dagParams.K) newNode := newBlockNode(blockHeader, parents, dag.dagParams.K)
newNode.status = statusDataStored newNode.status = statusDataStored
// newBlockNode adds node into children maps of its parents. So it must be
// removed in case of error.
isOk := false
defer func() {
if !isOk {
newNode.detachFromParents()
}
}()
dag.index.AddNode(newNode) dag.index.AddNode(newNode)
err = dag.index.flushToDB() err = dag.index.flushToDB()
@ -82,6 +90,8 @@ func (dag *BlockDAG) maybeAcceptBlock(block *util.Block, flags BehaviorFlags) er
dag.sendNotification(NTBlockAccepted, block) dag.sendNotification(NTBlockAccepted, block)
dag.dagLock.Lock() dag.dagLock.Lock()
isOk = true
return nil return nil
} }

View File

@ -135,6 +135,11 @@ func initBlockNode(node *blockNode, blockHeader *wire.BlockHeader, parents block
node.nonce = blockHeader.Nonce node.nonce = blockHeader.Nonce
node.timestamp = blockHeader.Timestamp.Unix() node.timestamp = blockHeader.Timestamp.Unix()
node.merkleRoot = blockHeader.MerkleRoot node.merkleRoot = blockHeader.MerkleRoot
// update parents to point to new node
for _, p := range parents {
p.children[node.hash] = node
}
} }
if len(parents) > 0 { if len(parents) > 0 {
@ -165,6 +170,15 @@ func newBlockNode(blockHeader *wire.BlockHeader, parents blockSet, phantomK uint
return &node return &node
} }
// newBlockNode adds node into children maps of its parents. So it must be
// removed in case of error.
func (node *blockNode) detachFromParents() {
// remove node from parents
for _, p := range node.parents {
delete(p.children, node.hash)
}
}
// Header constructs a block header from the node and returns it. // Header constructs a block header from the node and returns it.
// //
// This function is safe for concurrent access. // This function is safe for concurrent access.

View File

@ -631,17 +631,10 @@ func (dag *BlockDAG) checkFinalityRulesAndGetFinalityPointCandidate(node *blockN
// //
// This function MUST be called with the chain state lock held (for writes). // This function MUST be called with the chain state lock held (for writes).
func (dag *BlockDAG) applyUTXOChanges(node *blockNode, block *util.Block, fastAdd bool) (utxoDiff *UTXODiff, acceptedTxData []*TxWithBlockHash, err error) { func (dag *BlockDAG) applyUTXOChanges(node *blockNode, block *util.Block, fastAdd bool) (utxoDiff *UTXODiff, acceptedTxData []*TxWithBlockHash, err error) {
// Prepare provisionalNodes for all the relevant nodes to avoid modifying the original nodes.
// We avoid modifying the original nodes in this function because it could potentially
// fail if the block is not valid, thus bringing all the affected nodes (and the virtual)
// into an undefined state.
provisionalSet := newProvisionalNodeSet()
newNodeProvisional := provisionalSet.newProvisionalNode(node, true, true, block.Transactions())
// Clone the virtual block so that we don't modify the existing one. // Clone the virtual block so that we don't modify the existing one.
virtualClone := dag.virtual.clone() virtualClone := dag.virtual.clone()
newBlockUTXO, acceptedTxData, err := newNodeProvisional.verifyAndBuildUTXO(virtualClone, dag, fastAdd) newBlockUTXO, acceptedTxData, err := node.verifyAndBuildUTXO(virtualClone, dag, block.Transactions(), fastAdd)
if err != nil { if err != nil {
newErrString := fmt.Sprintf("error verifying UTXO for %v: %s", node, err) newErrString := fmt.Sprintf("error verifying UTXO for %v: %s", node, err)
if err, ok := err.(RuleError); ok { if err, ok := err.(RuleError); ok {
@ -650,33 +643,29 @@ func (dag *BlockDAG) applyUTXOChanges(node *blockNode, block *util.Block, fastAd
return nil, nil, errors.New(newErrString) return nil, nil, errors.New(newErrString)
} }
err = newNodeProvisional.updateParents(virtualClone, newBlockUTXO) // since verifyAndBuildUTXO ran, we know for sure that block is valid -
// therefore, if an error occured - this means there's some problem in the
// internal structure of block nodes, and it's irrecoverable - therefore
// panic
err = node.updateParents(virtualClone, newBlockUTXO)
if err != nil { if err != nil {
return nil, nil, fmt.Errorf("failed updating parents of %v: %s", node, err) panic(fmt.Errorf("failed updating parents of %v: %s", node, err))
} }
// Update the virtual block's children (the DAG tips) to include the new block. // Update the virtual block's children (the DAG tips) to include the new block.
virtualClone.AddTip(node) virtualClone.AddTip(node)
// Build a UTXO set for the new virtual block and update the DAG tips' diffs. // Build a UTXO set for the new virtual block and update the DAG tips' diffs.
virtualNodeProvisional := provisionalSet.newProvisionalNode(&virtualClone.blockNode, true, true, nil) newVirtualUTXO, _, err := virtualClone.blockNode.pastUTXO(virtualClone, dag.db)
newVirtualUTXO, _, err := virtualNodeProvisional.pastUTXO(virtualClone, dag.db)
if err != nil { if err != nil {
newErrString := fmt.Sprintf("could not restore past UTXO for virtual %v: %s", virtualClone, err) panic(fmt.Sprintf("could not restore past UTXO for virtual %v: %s", virtualClone, err))
if err, ok := err.(RuleError); ok {
return nil, nil, ruleError(err.ErrorCode, newErrString)
}
return nil, nil, errors.New(newErrString)
} }
// Apply new utxoDiffs to all the tips // Apply new utxoDiffs to all the tips
err = updateTipsUTXO(virtualNodeProvisional.parents, virtualClone, newVirtualUTXO) err = updateTipsUTXO(virtualClone.parents, virtualClone, newVirtualUTXO)
if err != nil { if err != nil {
newErrString := fmt.Sprintf("failed updating the tips' UTXO: %s", err) panic(fmt.Sprintf("failed updating the tips' UTXO: %s", err))
if err, ok := err.(RuleError); ok {
return nil, nil, ruleError(err.ErrorCode, newErrString)
}
return nil, nil, errors.New(newErrString)
} }
// It is now safe to meld the UTXO set to base. // It is now safe to meld the UTXO set to base.
@ -684,16 +673,12 @@ func (dag *BlockDAG) applyUTXOChanges(node *blockNode, block *util.Block, fastAd
utxoDiff = diffSet.UTXODiff utxoDiff = diffSet.UTXODiff
dag.updateVirtualUTXO(diffSet) dag.updateVirtualUTXO(diffSet)
// It is now safe to commit all the provisionalNodes
for _, provisional := range provisionalSet {
provisional.commit()
// Set the status to valid for all index nodes to make sure the changes get // Set the status to valid for all index nodes to make sure the changes get
// written to the database. // written to the database.
if provisional != virtualNodeProvisional { for _, p := range node.parents {
dag.index.SetStatusFlags(provisional.original, statusValid) dag.index.SetStatusFlags(p, statusValid)
}
} }
dag.index.SetStatusFlags(node, statusValid)
// It is now safe to apply the new virtual block // It is now safe to apply the new virtual block
dag.virtual = virtualClone dag.virtual = virtualClone
@ -707,99 +692,29 @@ func (dag *BlockDAG) updateVirtualUTXO(newVirtualUTXODiffSet *DiffUTXOSet) {
newVirtualUTXODiffSet.meldToBase() newVirtualUTXODiffSet.meldToBase()
} }
// provisionalNodeSet is a temporary collection of provisionalNodes. It is used exclusively // verifyAndBuildUTXO verifies all transactions in the given block and builds its UTXO
// inside applyUTXOChanges. The purpose of this set is twofold: func (node *blockNode) verifyAndBuildUTXO(virtual *virtualBlock, dag *BlockDAG,
// 1. Provide easy access to all provisionalNodes created inside this particular call to applyUTXOChanges transactions []*util.Tx, fastAdd bool) (utxoSet UTXOSet, acceptedTxData []*TxWithBlockHash, err error) {
// 2. Avoid needless recreation of provisionalNodes pastUTXO, pastUTXOaccpetedTxData, err := node.pastUTXO(virtual, dag.db)
type provisionalNodeSet map[daghash.Hash]*provisionalNode
// newProvisionalNodeSet creates an empty provisionalNodeSet
func newProvisionalNodeSet() provisionalNodeSet {
return provisionalNodeSet{}
}
// provisionalNode is a temporary and partial copy of a blockNode. It is used exclusively
// inside applyUTXOChanges. We use this struct instead of the original blockNode because
// applyUTXOChanges has a few points of failure which, were we to modify it, would leave the
// blockNode in an undefined state.
//
// Important: provisionalNode.original must be treated as immutable.
type provisionalNode struct {
original *blockNode
selectedParent *provisionalNode
parents []*provisionalNode
children []*provisionalNode
diff *UTXODiff
diffChild *provisionalNode
transactions []*util.Tx
}
// newProvisionalNode takes a node and builds a provisionalNode from it.
// To avoid building the entire DAG in provisionalNode format we pass withParents = true or withChildren = true,
// only when the node's relatives (parents or children) are required.
func (pns provisionalNodeSet) newProvisionalNode(node *blockNode, withParents bool, withChildren bool,
transactions []*util.Tx) *provisionalNode {
if existingProvisional, ok := pns[node.hash]; ok {
return existingProvisional
}
provisional := &provisionalNode{
original: node,
transactions: transactions,
}
if node.hash != zeroHash {
pns[node.hash] = provisional
}
if withParents {
provisional.parents = []*provisionalNode{}
for _, parent := range node.parents {
provisional.parents = append(provisional.parents, pns.newProvisionalNode(parent, false, true, nil))
}
if node.selectedParent != nil {
provisional.selectedParent = pns[node.selectedParent.hash]
}
}
if withChildren {
provisional.children = []*provisionalNode{}
for _, child := range node.children {
provisional.children = append(provisional.children, pns.newProvisionalNode(child, false, false, nil))
}
if node.diffChild != nil {
provisional.diffChild = pns[node.diffChild.hash]
}
}
if node.diff != nil {
provisional.diff = node.diff.clone()
}
return provisional
}
// verifyAndBuildUTXO verifies all transactions in the given block (in provisionalNode format) and builds its UTXO
func (p *provisionalNode) verifyAndBuildUTXO(virtual *virtualBlock, dag *BlockDAG, fastAdd bool) (utxoSet UTXOSet, acceptedTxData []*TxWithBlockHash, err error) {
pastUTXO, pastUTXOaccpetedTxData, err := p.pastUTXO(virtual, dag.db)
if err != nil { if err != nil {
return nil, nil, err return nil, nil, err
} }
if !fastAdd { if !fastAdd {
err = dag.checkConnectToPastUTXO(p, pastUTXO) err = dag.checkConnectToPastUTXO(node, pastUTXO, transactions)
if err != nil { if err != nil {
return nil, nil, err return nil, nil, err
} }
} }
diff := NewUTXODiff() diff := NewUTXODiff()
acceptedTxData = make([]*TxWithBlockHash, 0, len(pastUTXOaccpetedTxData)+len(p.transactions)) acceptedTxData = make([]*TxWithBlockHash, 0, len(pastUTXOaccpetedTxData)+len(transactions))
if len(pastUTXOaccpetedTxData) != 0 { if len(pastUTXOaccpetedTxData) != 0 {
acceptedTxData = append(acceptedTxData, pastUTXOaccpetedTxData...) acceptedTxData = append(acceptedTxData, pastUTXOaccpetedTxData...)
} }
for _, tx := range p.transactions { for _, tx := range transactions {
txDiff, err := pastUTXO.diffFromTx(tx.MsgTx(), p.original) txDiff, err := pastUTXO.diffFromTx(tx.MsgTx(), node)
if err != nil { if err != nil {
return nil, nil, err return nil, nil, err
} }
@ -809,7 +724,7 @@ func (p *provisionalNode) verifyAndBuildUTXO(virtual *virtualBlock, dag *BlockDA
} }
acceptedTxData = append(acceptedTxData, &TxWithBlockHash{ acceptedTxData = append(acceptedTxData, &TxWithBlockHash{
Tx: tx, Tx: tx,
InBlock: &p.original.hash, InBlock: &node.hash,
}) })
} }
@ -826,9 +741,9 @@ type TxWithBlockHash struct {
InBlock *daghash.Hash InBlock *daghash.Hash
} }
// pastUTXO returns the UTXO of a given block's (in provisionalNode format) past // pastUTXO returns the UTXO of a given block's past
func (p *provisionalNode) pastUTXO(virtual *virtualBlock, db database.DB) (pastUTXO UTXOSet, acceptedTxData []*TxWithBlockHash, err error) { func (node *blockNode) pastUTXO(virtual *virtualBlock, db database.DB) (pastUTXO UTXOSet, acceptedTxData []*TxWithBlockHash, err error) {
pastUTXO, err = p.selectedParent.restoreUTXO(virtual) pastUTXO, err = node.selectedParent.restoreUTXO(virtual)
if err != nil { if err != nil {
return nil, nil, err return nil, nil, err
} }
@ -840,10 +755,10 @@ func (p *provisionalNode) pastUTXO(virtual *virtualBlock, db database.DB) (pastU
// Precalculate the amount of transactions in this block's blue set, besides the selected parent. // Precalculate the amount of transactions in this block's blue set, besides the selected parent.
// This is to avoid an attack in which an attacker fabricates a block that will deliberately cause // This is to avoid an attack in which an attacker fabricates a block that will deliberately cause
// a lot of copying, causing a high cost to the whole network. // a lot of copying, causing a high cost to the whole network.
blueBlocks := make([]*util.Block, 0, len(p.original.blues)-1) blueBlocks := make([]*util.Block, 0, len(node.blues)-1)
for i := len(p.original.blues) - 1; i >= 0; i-- { for i := len(node.blues) - 1; i >= 0; i-- {
blueBlockNode := p.original.blues[i] blueBlockNode := node.blues[i]
if blueBlockNode == p.original.selectedParent { if blueBlockNode == node.selectedParent {
continue continue
} }
@ -874,7 +789,7 @@ func (p *provisionalNode) pastUTXO(virtual *virtualBlock, db database.DB) (pastU
// Add all transactions to the pastUTXO // Add all transactions to the pastUTXO
// Purposefully ignore failures - these are just unaccepted transactions // Purposefully ignore failures - these are just unaccepted transactions
for _, tx := range blueBlockTransactions { for _, tx := range blueBlockTransactions {
isAccepted := pastUTXO.AddTx(tx.Tx.MsgTx(), p.original.height) isAccepted := pastUTXO.AddTx(tx.Tx.MsgTx(), node.height)
if isAccepted { if isAccepted {
acceptedTxData = append(acceptedTxData, tx) acceptedTxData = append(acceptedTxData, tx)
} }
@ -883,10 +798,10 @@ func (p *provisionalNode) pastUTXO(virtual *virtualBlock, db database.DB) (pastU
return pastUTXO, acceptedTxData, nil return pastUTXO, acceptedTxData, nil
} }
// restoreUTXO restores the UTXO of a given block (in provisionalNode format) from its diff // restoreUTXO restores the UTXO of a given block from its diff
func (p *provisionalNode) restoreUTXO(virtual *virtualBlock) (UTXOSet, error) { func (node *blockNode) restoreUTXO(virtual *virtualBlock) (UTXOSet, error) {
stack := []*provisionalNode{p} stack := []*blockNode{node}
current := p current := node
for current.diffChild != nil { for current.diffChild != nil {
current = current.diffChild current = current.diffChild
@ -906,43 +821,41 @@ func (p *provisionalNode) restoreUTXO(virtual *virtualBlock) (UTXOSet, error) {
return utxo, nil return utxo, nil
} }
// updateParents adds this block (in provisionalNode format) to the children sets of its parents // updateParents adds this block to the children sets of its parents
// and updates the diff of any parent whose DiffChild is this block // and updates the diff of any parent whose DiffChild is this block
func (p *provisionalNode) updateParents(virtual *virtualBlock, newBlockUTXO UTXOSet) error { func (node *blockNode) updateParents(virtual *virtualBlock, newBlockUTXO UTXOSet) error {
virtualDiffFromNewBlock, err := virtual.utxoSet.diffFrom(newBlockUTXO) virtualDiffFromNewBlock, err := virtual.utxoSet.diffFrom(newBlockUTXO)
if err != nil { if err != nil {
return err return err
} }
p.diff = virtualDiffFromNewBlock node.diff = virtualDiffFromNewBlock
for _, parent := range p.parents { for _, parent := range node.parents {
if parent.diffChild == nil { if parent.diffChild == nil {
parentUTXO, err := parent.restoreUTXO(virtual) parentUTXO, err := parent.restoreUTXO(virtual)
if err != nil { if err != nil {
return err return err
} }
parent.diffChild = p parent.diffChild = node
parent.diff, err = newBlockUTXO.diffFrom(parentUTXO) parent.diff, err = newBlockUTXO.diffFrom(parentUTXO)
if err != nil { if err != nil {
return err return err
} }
} }
parent.children = append(parent.children, p)
} }
return nil return nil
} }
// updateTipsUTXO builds and applies new diff UTXOs for all the DAG's tips (in provisionalNode format) // updateTipsUTXO builds and applies new diff UTXOs for all the DAG's tips
func updateTipsUTXO(tipProvisionals []*provisionalNode, virtual *virtualBlock, virtualUTXO UTXOSet) error { func updateTipsUTXO(tips blockSet, virtual *virtualBlock, virtualUTXO UTXOSet) error {
for _, tipProvisional := range tipProvisionals { for _, tip := range tips {
tipUTXO, err := tipProvisional.restoreUTXO(virtual) tipUTXO, err := tip.restoreUTXO(virtual)
if err != nil { if err != nil {
return err return err
} }
tipProvisional.diff, err = virtualUTXO.diffFrom(tipUTXO) tip.diff, err = virtualUTXO.diffFrom(tipUTXO)
if err != nil { if err != nil {
return err return err
} }
@ -951,36 +864,6 @@ func updateTipsUTXO(tipProvisionals []*provisionalNode, virtual *virtualBlock, v
return nil return nil
} }
// commit updates the original blockNode this provisionalNode was created from with all the changes made to it
func (p *provisionalNode) commit() {
if p.selectedParent != nil {
p.original.selectedParent = p.selectedParent.original
}
if p.parents != nil {
parents := newSet()
for _, parent := range p.parents {
parents.add(parent.original)
}
p.original.parents = parents
}
if p.children != nil {
children := newSet()
for _, child := range p.children {
children.add(child.original)
}
p.original.children = children
}
if p.diff != nil {
p.original.diff = p.diff
}
if p.diffChild != nil {
p.original.diffChild = p.diffChild.original
}
}
// isCurrent returns whether or not the chain believes it is current. Several // isCurrent returns whether or not the chain believes it is current. Several
// factors are used to guess, but the key factors that allow the chain to // factors are used to guess, but the key factors that allow the chain to
// believe it is current are: // believe it is current are:

View File

@ -770,6 +770,11 @@ func TestIntervalBlockHashes(t *testing.T) {
// The non-error-cases are tested in the more general tests. // The non-error-cases are tested in the more general tests.
func TestPastUTXOErrors(t *testing.T) { func TestPastUTXOErrors(t *testing.T) {
targetErrorMessage := "dbFetchBlockByNode error" targetErrorMessage := "dbFetchBlockByNode error"
defer func() {
if recover() == nil {
t.Errorf("Got no panic on past UTXO error, while expected panic")
}
}()
testErrorThroughPatching( testErrorThroughPatching(
t, t,
targetErrorMessage, targetErrorMessage,

View File

@ -821,6 +821,7 @@ func (dag *BlockDAG) createDAGState() error {
// Store the genesis block into the database. // Store the genesis block into the database.
return dbStoreBlock(dbTx, genesisBlock) return dbStoreBlock(dbTx, genesisBlock)
}) })
return err return err
} }

View File

@ -205,15 +205,15 @@ func ValidateTransactionScripts(tx *util.Tx, utxoSet UTXOSet, flags txscript.Scr
// checkBlockScripts executes and validates the scripts for all transactions in // checkBlockScripts executes and validates the scripts for all transactions in
// the passed block using multiple goroutines. // the passed block using multiple goroutines.
func checkBlockScripts(block *provisionalNode, utxoSet UTXOSet, scriptFlags txscript.ScriptFlags, sigCache *txscript.SigCache) error { func checkBlockScripts(block *blockNode, utxoSet UTXOSet, transactions []*util.Tx, scriptFlags txscript.ScriptFlags, sigCache *txscript.SigCache) error {
// Collect all of the transaction inputs and required information for // Collect all of the transaction inputs and required information for
// validation for all transactions in the block into a single slice. // validation for all transactions in the block into a single slice.
numInputs := 0 numInputs := 0
for _, tx := range block.transactions { for _, tx := range transactions {
numInputs += len(tx.MsgTx().TxIn) numInputs += len(tx.MsgTx().TxIn)
} }
txValItems := make([]*txValidateItem, 0, numInputs) txValItems := make([]*txValidateItem, 0, numInputs)
for _, tx := range block.transactions { for _, tx := range transactions {
for txInIdx, txIn := range tx.MsgTx().TxIn { for txInIdx, txIn := range tx.MsgTx().TxIn {
// Skip coinbases. // Skip coinbases.
if txIn.PreviousOutPoint.Index == math.MaxUint32 { if txIn.PreviousOutPoint.Index == math.MaxUint32 {
@ -237,7 +237,7 @@ func checkBlockScripts(block *provisionalNode, utxoSet UTXOSet, scriptFlags txsc
} }
elapsed := time.Since(start) elapsed := time.Since(start)
log.Tracef("block %v took %v to verify", block.original.hash, elapsed) log.Tracef("block %v took %v to verify", block.hash, elapsed)
return nil return nil
} }

View File

@ -40,15 +40,12 @@ func TestCheckBlockScripts(t *testing.T) {
return return
} }
node := &provisionalNode{ node := &blockNode{
original: &blockNode{
hash: *blocks[0].Hash(), hash: *blocks[0].Hash(),
},
transactions: blocks[0].Transactions(),
} }
scriptFlags := txscript.ScriptNoFlags scriptFlags := txscript.ScriptNoFlags
err = checkBlockScripts(node, utxoSet, scriptFlags, nil) err = checkBlockScripts(node, utxoSet, blocks[0].Transactions(), scriptFlags, nil)
if err != nil { if err != nil {
t.Errorf("Transaction script validation failed: %v\n", err) t.Errorf("Transaction script validation failed: %v\n", err)
return return

View File

@ -67,7 +67,7 @@ func isNullOutpoint(outpoint *wire.OutPoint) bool {
// IsCoinBaseTx determines whether or not a transaction is a coinbase. A coinbase // IsCoinBaseTx determines whether or not a transaction is a coinbase. A coinbase
// is a special transaction created by miners that has no inputs. This is // is a special transaction created by miners that has no inputs. This is
// represented in the block chain by a transaction with a single input that has // represented in the block dag by a transaction with a single input that has
// a previous output transaction index set to the maximum value along with a // a previous output transaction index set to the maximum value along with a
// zero hash. // zero hash.
// //
@ -91,7 +91,7 @@ func IsCoinBaseTx(msgTx *wire.MsgTx) bool {
// IsCoinBase determines whether or not a transaction is a coinbase. A coinbase // IsCoinBase determines whether or not a transaction is a coinbase. A coinbase
// is a special transaction created by miners that has no inputs. This is // is a special transaction created by miners that has no inputs. This is
// represented in the block chain by a transaction with a single input that has // represented in the block dag by a transaction with a single input that has
// a previous output transaction index set to the maximum value along with a // a previous output transaction index set to the maximum value along with a
// zero hash. // zero hash.
// //
@ -642,13 +642,13 @@ func checkSerializedHeight(coinbaseTx *util.Tx, wantHeight int32) error {
} }
// checkBlockHeaderContext performs several validation checks on the block header // checkBlockHeaderContext performs several validation checks on the block header
// which depend on its position within the block chain. // which depend on its position within the block dag.
// //
// The flags modify the behavior of this function as follows: // The flags modify the behavior of this function as follows:
// - BFFastAdd: All checks except those involving comparing the header against // - BFFastAdd: All checks except those involving comparing the header against
// the checkpoints are not performed. // the checkpoints are not performed.
// //
// This function MUST be called with the chain state lock held (for writes). // This function MUST be called with the dag state lock held (for writes).
func (dag *BlockDAG) checkBlockHeaderContext(header *wire.BlockHeader, bluestParent *blockNode, blockHeight int32, flags BehaviorFlags) error { func (dag *BlockDAG) checkBlockHeaderContext(header *wire.BlockHeader, bluestParent *blockNode, blockHeight int32, flags BehaviorFlags) error {
fastAdd := flags&BFFastAdd == BFFastAdd fastAdd := flags&BFFastAdd == BFFastAdd
if !fastAdd { if !fastAdd {
@ -677,7 +677,7 @@ func (dag *BlockDAG) checkBlockHeaderContext(header *wire.BlockHeader, bluestPar
} }
} }
// Ensure chain matches up to predetermined checkpoints. // Ensure dag matches up to predetermined checkpoints.
blockHash := header.BlockHash() blockHash := header.BlockHash()
if !dag.verifyCheckpoint(blockHeight, &blockHash) { if !dag.verifyCheckpoint(blockHeight, &blockHash) {
str := fmt.Sprintf("block at height %d does not match "+ str := fmt.Sprintf("block at height %d does not match "+
@ -686,7 +686,7 @@ func (dag *BlockDAG) checkBlockHeaderContext(header *wire.BlockHeader, bluestPar
} }
// Find the previous checkpoint and prevent blocks which fork the main // Find the previous checkpoint and prevent blocks which fork the main
// chain before it. This prevents storage of new, otherwise valid, // dag before it. This prevents storage of new, otherwise valid,
// blocks which build off of old blocks that are likely at a much easier // blocks which build off of old blocks that are likely at a much easier
// difficulty and therefore could be used to waste cache and disk space. // difficulty and therefore could be used to waste cache and disk space.
checkpointNode, err := dag.findPreviousCheckpoint() checkpointNode, err := dag.findPreviousCheckpoint()
@ -694,7 +694,7 @@ func (dag *BlockDAG) checkBlockHeaderContext(header *wire.BlockHeader, bluestPar
return err return err
} }
if checkpointNode != nil && blockHeight < checkpointNode.height { if checkpointNode != nil && blockHeight < checkpointNode.height {
str := fmt.Sprintf("block at height %d forks the main chain "+ str := fmt.Sprintf("block at height %d forks the main dag "+
"before the previous checkpoint at height %d", "before the previous checkpoint at height %d",
blockHeight, checkpointNode.height) blockHeight, checkpointNode.height)
return ruleError(ErrForkTooOld, str) return ruleError(ErrForkTooOld, str)
@ -740,7 +740,7 @@ func validateParents(blockHeader *wire.BlockHeader, parents blockSet) error {
} }
// checkBlockContext peforms several validation checks on the block which depend // checkBlockContext peforms several validation checks on the block which depend
// on its position within the block chain. // on its position within the block dag.
// //
// The flags modify the behavior of this function as follows: // The flags modify the behavior of this function as follows:
// - BFFastAdd: The transaction are not checked to see if they are finalized // - BFFastAdd: The transaction are not checked to see if they are finalized
@ -749,7 +749,7 @@ func validateParents(blockHeader *wire.BlockHeader, parents blockSet) error {
// The flags are also passed to checkBlockHeaderContext. See its documentation // The flags are also passed to checkBlockHeaderContext. See its documentation
// for how the flags modify its behavior. // for how the flags modify its behavior.
// //
// This function MUST be called with the chain state lock held (for writes). // This function MUST be called with the dag state lock held (for writes).
func (dag *BlockDAG) checkBlockContext(block *util.Block, parents blockSet, bluestParent *blockNode, flags BehaviorFlags) error { func (dag *BlockDAG) checkBlockContext(block *util.Block, parents blockSet, bluestParent *blockNode, flags BehaviorFlags) error {
err := validateParents(&block.MsgBlock().Header, parents) err := validateParents(&block.MsgBlock().Header, parents)
if err != nil { if err != nil {
@ -801,12 +801,13 @@ func (dag *BlockDAG) checkBlockContext(block *util.Block, parents blockSet, blue
// https://github.com/bitcoin/bips/blob/master/bip-0030.mediawiki and // https://github.com/bitcoin/bips/blob/master/bip-0030.mediawiki and
// http://r6.ca/blog/20120206T005236Z.html. // http://r6.ca/blog/20120206T005236Z.html.
// //
// This function MUST be called with the chain state lock held (for reads). // This function MUST be called with the dag state lock held (for reads).
func ensureNoDuplicateTx(block *provisionalNode, utxoSet UTXOSet) error { func ensureNoDuplicateTx(block *blockNode, utxoSet UTXOSet,
transactions []*util.Tx) error {
// Fetch utxos for all of the transaction ouputs in this block. // Fetch utxos for all of the transaction ouputs in this block.
// Typically, there will not be any utxos for any of the outputs. // Typically, there will not be any utxos for any of the outputs.
fetchSet := make(map[wire.OutPoint]struct{}) fetchSet := make(map[wire.OutPoint]struct{})
for _, tx := range block.transactions { for _, tx := range transactions {
prevOut := wire.OutPoint{Hash: *tx.Hash()} prevOut := wire.OutPoint{Hash: *tx.Hash()}
for txOutIdx := range tx.MsgTx().TxOut { for txOutIdx := range tx.MsgTx().TxOut {
prevOut.Index = uint32(txOutIdx) prevOut.Index = uint32(txOutIdx)
@ -939,9 +940,10 @@ func CheckTransactionInputs(tx *util.Tx, txHeight int32, utxoSet UTXOSet, dagPar
// block subsidy, or fail transaction script validation. // block subsidy, or fail transaction script validation.
// //
// This function MUST be called with the dag state lock held (for writes). // This function MUST be called with the dag state lock held (for writes).
func (dag *BlockDAG) checkConnectToPastUTXO(block *provisionalNode, pastUTXO UTXOSet) error { func (dag *BlockDAG) checkConnectToPastUTXO(block *blockNode, pastUTXO UTXOSet,
transactions []*util.Tx) error {
err := ensureNoDuplicateTx(block, pastUTXO) err := ensureNoDuplicateTx(block, pastUTXO, transactions)
if err != nil { if err != nil {
return err return err
} }
@ -953,7 +955,7 @@ func (dag *BlockDAG) checkConnectToPastUTXO(block *provisionalNode, pastUTXO UTX
// signature operations in each of the input transaction public key // signature operations in each of the input transaction public key
// scripts. // scripts.
totalSigOps := 0 totalSigOps := 0
for i, tx := range block.transactions { for i, tx := range transactions {
numsigOps := CountSigOps(tx) numsigOps := CountSigOps(tx)
// Since the first (and only the first) transaction has // Since the first (and only the first) transaction has
// already been verified to be a coinbase transaction, // already been verified to be a coinbase transaction,
@ -987,8 +989,8 @@ func (dag *BlockDAG) checkConnectToPastUTXO(block *provisionalNode, pastUTXO UTX
// against all the inputs when the signature operations are out of // against all the inputs when the signature operations are out of
// bounds. // bounds.
var totalFees uint64 var totalFees uint64
for _, tx := range block.transactions { for _, tx := range transactions {
txFee, err := CheckTransactionInputs(tx, block.original.height, pastUTXO, txFee, err := CheckTransactionInputs(tx, block.height, pastUTXO,
dag.dagParams) dag.dagParams)
if err != nil { if err != nil {
return err return err
@ -1010,10 +1012,10 @@ func (dag *BlockDAG) checkConnectToPastUTXO(block *provisionalNode, pastUTXO UTX
// errors here because those error conditions would have already been // errors here because those error conditions would have already been
// caught by checkTransactionSanity. // caught by checkTransactionSanity.
var totalSatoshiOut uint64 var totalSatoshiOut uint64
for _, txOut := range block.transactions[0].MsgTx().TxOut { for _, txOut := range transactions[0].MsgTx().TxOut {
totalSatoshiOut += txOut.Value totalSatoshiOut += txOut.Value
} }
expectedSatoshiOut := CalcBlockSubsidy(block.original.height, dag.dagParams) + expectedSatoshiOut := CalcBlockSubsidy(block.height, dag.dagParams) +
totalFees totalFees
if totalSatoshiOut > expectedSatoshiOut { if totalSatoshiOut > expectedSatoshiOut {
str := fmt.Sprintf("coinbase transaction for block pays %v "+ str := fmt.Sprintf("coinbase transaction for block pays %v "+
@ -1030,7 +1032,7 @@ func (dag *BlockDAG) checkConnectToPastUTXO(block *provisionalNode, pastUTXO UTX
// portion of block handling. // portion of block handling.
checkpoint := dag.LatestCheckpoint() checkpoint := dag.LatestCheckpoint()
runScripts := true runScripts := true
if checkpoint != nil && block.original.height <= checkpoint.Height { if checkpoint != nil && block.height <= checkpoint.Height {
runScripts = false runScripts = false
} }
@ -1038,20 +1040,20 @@ func (dag *BlockDAG) checkConnectToPastUTXO(block *provisionalNode, pastUTXO UTX
// We obtain the MTP of the *previous* block in order to // We obtain the MTP of the *previous* block in order to
// determine if transactions in the current block are final. // determine if transactions in the current block are final.
medianTime := block.original.selectedParent.CalcPastMedianTime() medianTime := block.selectedParent.CalcPastMedianTime()
// We also enforce the relative sequence number based // We also enforce the relative sequence number based
// lock-times within the inputs of all transactions in this // lock-times within the inputs of all transactions in this
// candidate block. // candidate block.
for _, tx := range block.transactions { for _, tx := range transactions {
// A transaction can only be included within a block // A transaction can only be included within a block
// once the sequence locks of *all* its inputs are // once the sequence locks of *all* its inputs are
// active. // active.
sequenceLock, err := dag.calcSequenceLock(block.original, pastUTXO, tx, false) sequenceLock, err := dag.calcSequenceLock(block, pastUTXO, tx, false)
if err != nil { if err != nil {
return err return err
} }
if !SequenceLockActive(sequenceLock, block.original.height, if !SequenceLockActive(sequenceLock, block.height,
medianTime) { medianTime) {
str := fmt.Sprintf("block contains " + str := fmt.Sprintf("block contains " +
"transaction whose input sequence " + "transaction whose input sequence " +
@ -1065,7 +1067,7 @@ func (dag *BlockDAG) checkConnectToPastUTXO(block *provisionalNode, pastUTXO UTX
// expensive ECDSA signature check scripts. Doing this last helps // expensive ECDSA signature check scripts. Doing this last helps
// prevent CPU exhaustion attacks. // prevent CPU exhaustion attacks.
if runScripts { if runScripts {
err := checkBlockScripts(block, pastUTXO, scriptFlags, dag.sigCache) err := checkBlockScripts(block, pastUTXO, transactions, scriptFlags, dag.sigCache)
if err != nil { if err != nil {
return err return err
} }
@ -1085,8 +1087,8 @@ func countSpentOutputs(block *util.Block) int {
} }
// CheckConnectBlockTemplate fully validates that connecting the passed block to // CheckConnectBlockTemplate fully validates that connecting the passed block to
// the main chain does not violate any consensus rules, aside from the proof of // the main dag does not violate any consensus rules, aside from the proof of
// work requirement. The block must connect to the current tip of the main chain. // work requirement. The block must connect to the current tip of the main dag.
// //
// This function is safe for concurrent access. // This function is safe for concurrent access.
func (dag *BlockDAG) CheckConnectBlockTemplate(block *util.Block) error { func (dag *BlockDAG) CheckConnectBlockTemplate(block *util.Block) error {
@ -1097,7 +1099,7 @@ func (dag *BlockDAG) CheckConnectBlockTemplate(block *util.Block) error {
flags := BFNoPoWCheck flags := BFNoPoWCheck
// This only checks whether the block can be connected to the tip of the // This only checks whether the block can be connected to the tip of the
// current chain. // current dag.
tips := dag.virtual.tips() tips := dag.virtual.tips()
header := block.MsgBlock().Header header := block.MsgBlock().Header
parentHashes := header.ParentHashes parentHashes := header.ParentHashes
@ -1122,9 +1124,6 @@ func (dag *BlockDAG) CheckConnectBlockTemplate(block *util.Block) error {
return err return err
} }
newProvisionalNode := &provisionalNode{ return dag.checkConnectToPastUTXO(newBlockNode(&header, dag.virtual.tips(), dag.dagParams.K),
original: newBlockNode(&header, dag.virtual.tips(), dag.dagParams.K), dag.UTXOSet(), block.Transactions())
transactions: block.Transactions(),
}
return dag.checkConnectToPastUTXO(newProvisionalNode, dag.UTXOSet())
} }