mirror of
https://github.com/kaspanet/kaspad.git
synced 2025-10-14 00:59:33 +00:00
[NOD-1190] Refactor process.go (#858)
* [NOD-1190] Move non-processBlock stuff out of process.go. * [NOD-1190] Move everything out of accept.go. * [NOD-1190] Move all processBlock functions to process.go. * [NOD-1190] Move orphan stuff to orphan.go. * [NOD-1190] Remove thresholdstate stuff. * [NOD-1190] Move isSynced to sync_rate.go. * [NOD-1190] Move delayed block stuff to delayed_blocks.go. * [NOD-1190] Rename orphans.go to orphaned_blocks.go. * [NOD-1190] Move non-BlockDAG structs out of dag.go. * [NOD-1190] Remove unused fields. * [NOD-1190] Fixup BlockDAG.New a bit. * [NOD-1190] Move sequence lock stuff to sequence_lock.go * [NOD-1190] Move some multiset stuff out of dag.go. * [NOD-1190] Move finality stuff out of dag.go. * [NOD-1190] Move blocklocator stuff out of dag.go. * [NOD-1190] Move confirmation stuff out of dag.go. * [NOD-1190] Move utxo and selected parent chain stuff out of dag.go. * [NOD-1190] Move BlockDAG lock functions to the beginning of dag.go. * [NOD-1190] Move verifyAndBuildUTXO out of process.go. * [NOD-1190] Extract handleProcessBlockError to a function. * [NOD-1190] Remove daglock unlock in notifyBlockAccepted. * [NOD-1190] Extract checkDuplicateBlock to a method. * [NOD-1190] Fix merge errors. * [NOD-1190] Remove unused parameter from CalcSequenceLock. * [NOD-1190] Extract processBlock contents into functions. * [NOD-1190] Fix parent delayed blocks not marking their children as delayed * [NOD-1190] Fix TestProcessDelayedBlocks. * [NOD-1190] Extract stuff in maybeAcceptBlock to separate functions. * [NOD-1190] Rename handleProcessBlockError to handleConnectBlockError. * [NOD-1190] Remove some comments. * [NOD-1190] Use lowercase in error messages. * [NOD-1190] Rename createNewBlockNode to createBlockNodeFromBlock. * [NOD-1190] Rename orphaned_blocks.go to orpan_blocks.go. * [NOD-1190] Extract validateUTXOCommitment to a separate function. * [NOD-1190] Fix a bug in validateUTXOCommitment. * [NOD-1190] Rename checkBlockTxsFinalized to checkBlockTransactionsFinalized. * [NOD-1190] Add a comment over createBlockNodeFromBlock. * [NOD-1190] Fold validateAllTxsFinalized into checkBlockTransactionsFinalized. * [NOD-1190] Return parents from checkBlockParents. * [NOD-1190] Remove the processBlock prefix from the functions that had it. * [NOD-1190] Begin extracting functions out of checkTransactionSanity. * [NOD-1190] Finish extracting functions out of checkTransactionSanity. * [NOD-1190] Remove an unused parameter. * [NOD-1190] Fix merge errors. * [NOD-1190] Added an explanation as to why we change the nonce in TestProcessDelayedBlocks. * [NOD-1190] Fix a comment. * [NOD-1190] Fix a comment. * [NOD-1190] Fix a typo. * [NOD-1190] Replace checkBlockParents with handleLookupParentNodesError.
This commit is contained in:
parent
32463ce906
commit
0653e59e16
@ -195,9 +195,8 @@ func setupMempool(cfg *config.Config, dag *blockdag.BlockDAG, sigCache *txscript
|
||||
MaxTxVersion: 1,
|
||||
},
|
||||
CalcSequenceLockNoLock: func(tx *util.Tx, utxoSet blockdag.UTXOSet) (*blockdag.SequenceLock, error) {
|
||||
return dag.CalcSequenceLockNoLock(tx, utxoSet, true)
|
||||
return dag.CalcSequenceLockNoLock(tx, utxoSet)
|
||||
},
|
||||
IsDeploymentActive: dag.IsDeploymentActive,
|
||||
SigCache: sigCache,
|
||||
DAG: dag,
|
||||
}
|
||||
|
@ -1,154 +0,0 @@
|
||||
// Copyright (c) 2013-2017 The btcsuite developers
|
||||
// Use of this source code is governed by an ISC
|
||||
// license that can be found in the LICENSE file.
|
||||
|
||||
package blockdag
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
||||
"github.com/kaspanet/kaspad/dbaccess"
|
||||
"github.com/kaspanet/kaspad/util"
|
||||
"github.com/pkg/errors"
|
||||
)
|
||||
|
||||
func (dag *BlockDAG) addNodeToIndexWithInvalidAncestor(block *util.Block) error {
|
||||
blockHeader := &block.MsgBlock().Header
|
||||
newNode, _ := dag.newBlockNode(blockHeader, newBlockSet())
|
||||
newNode.status = statusInvalidAncestor
|
||||
dag.index.AddNode(newNode)
|
||||
|
||||
dbTx, err := dag.databaseContext.NewTx()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
defer dbTx.RollbackUnlessClosed()
|
||||
err = dag.index.flushToDB(dbTx)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return dbTx.Commit()
|
||||
}
|
||||
|
||||
// maybeAcceptBlock potentially accepts a block into the block DAG. It
|
||||
// performs several validation checks which depend on its position within
|
||||
// the block DAG before adding it. The block is expected to have already
|
||||
// gone through ProcessBlock before calling this function with it.
|
||||
//
|
||||
// The flags are also passed to checkBlockContext and connectToDAG. See
|
||||
// their documentation for how the flags modify their behavior.
|
||||
//
|
||||
// This function MUST be called with the dagLock held (for writes).
|
||||
func (dag *BlockDAG) maybeAcceptBlock(block *util.Block, flags BehaviorFlags) error {
|
||||
parents, err := lookupParentNodes(block, dag)
|
||||
if err != nil {
|
||||
var ruleErr RuleError
|
||||
if ok := errors.As(err, &ruleErr); ok && ruleErr.ErrorCode == ErrInvalidAncestorBlock {
|
||||
err := dag.addNodeToIndexWithInvalidAncestor(block)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
// The block must pass all of the validation rules which depend on the
|
||||
// position of the block within the block DAG.
|
||||
err = dag.checkBlockContext(block, parents, flags)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Create a new block node for the block and add it to the node index.
|
||||
newNode, selectedParentAnticone := dag.newBlockNode(&block.MsgBlock().Header, parents)
|
||||
newNode.status = statusDataStored
|
||||
dag.index.AddNode(newNode)
|
||||
|
||||
// Insert the block into the database if it's not already there. Even
|
||||
// though it is possible the block will ultimately fail to connect, it
|
||||
// has already passed all proof-of-work and validity tests which means
|
||||
// it would be prohibitively expensive for an attacker to fill up the
|
||||
// disk with a bunch of blocks that fail to connect. This is necessary
|
||||
// since it allows block download to be decoupled from the much more
|
||||
// expensive connection logic. It also has some other nice properties
|
||||
// such as making blocks that never become part of the DAG or
|
||||
// blocks that fail to connect available for further analysis.
|
||||
dbTx, err := dag.databaseContext.NewTx()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
defer dbTx.RollbackUnlessClosed()
|
||||
blockExists, err := dbaccess.HasBlock(dbTx, block.Hash())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if !blockExists {
|
||||
err := storeBlock(dbTx, block)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
err = dag.index.flushToDB(dbTx)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
err = dbTx.Commit()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Make sure that all the block's transactions are finalized
|
||||
fastAdd := flags&BFFastAdd == BFFastAdd
|
||||
bluestParent := parents.bluest()
|
||||
if !fastAdd {
|
||||
if err := dag.validateAllTxsFinalized(block, newNode, bluestParent); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
// Connect the passed block to the DAG. This also handles validation of the
|
||||
// transaction scripts.
|
||||
chainUpdates, err := dag.addBlock(newNode, block, selectedParentAnticone, flags)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Notify the caller that the new block was accepted into the block
|
||||
// DAG. The caller would typically want to react by relaying the
|
||||
// inventory to other peers.
|
||||
dag.dagLock.Unlock()
|
||||
dag.sendNotification(NTBlockAdded, &BlockAddedNotificationData{
|
||||
Block: block,
|
||||
WasUnorphaned: flags&BFWasUnorphaned != 0,
|
||||
})
|
||||
if len(chainUpdates.addedChainBlockHashes) > 0 {
|
||||
dag.sendNotification(NTChainChanged, &ChainChangedNotificationData{
|
||||
RemovedChainBlockHashes: chainUpdates.removedChainBlockHashes,
|
||||
AddedChainBlockHashes: chainUpdates.addedChainBlockHashes,
|
||||
})
|
||||
}
|
||||
dag.dagLock.Lock()
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func lookupParentNodes(block *util.Block, dag *BlockDAG) (blockSet, error) {
|
||||
header := block.MsgBlock().Header
|
||||
parentHashes := header.ParentHashes
|
||||
|
||||
nodes := newBlockSet()
|
||||
for _, parentHash := range parentHashes {
|
||||
node, ok := dag.index.LookupNode(parentHash)
|
||||
if !ok {
|
||||
str := fmt.Sprintf("parent block %s is unknown", parentHash)
|
||||
return nil, ruleError(ErrParentBlockUnknown, str)
|
||||
} else if dag.index.NodeStatus(node).KnownInvalid() {
|
||||
str := fmt.Sprintf("parent block %s is known to be invalid", parentHash)
|
||||
return nil, ruleError(ErrInvalidAncestorBlock, str)
|
||||
}
|
||||
|
||||
nodes.add(node)
|
||||
}
|
||||
|
||||
return nodes, nil
|
||||
}
|
@ -1,107 +0,0 @@
|
||||
package blockdag
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"path/filepath"
|
||||
"testing"
|
||||
|
||||
"github.com/kaspanet/kaspad/dagconfig"
|
||||
)
|
||||
|
||||
func TestMaybeAcceptBlockErrors(t *testing.T) {
|
||||
// Create a new database and DAG instance to run tests against.
|
||||
dag, teardownFunc, err := DAGSetup("TestMaybeAcceptBlockErrors", true, Config{
|
||||
DAGParams: &dagconfig.SimnetParams,
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatalf("TestMaybeAcceptBlockErrors: Failed to setup DAG instance: %v", err)
|
||||
}
|
||||
defer teardownFunc()
|
||||
|
||||
dag.TestSetCoinbaseMaturity(0)
|
||||
|
||||
// Test rejecting the block if its parents are missing
|
||||
orphanBlockFile := "blk_3B.dat"
|
||||
loadedBlocks, err := LoadBlocks(filepath.Join("testdata/", orphanBlockFile))
|
||||
if err != nil {
|
||||
t.Fatalf("TestMaybeAcceptBlockErrors: "+
|
||||
"Error loading file '%s': %s\n", orphanBlockFile, err)
|
||||
}
|
||||
block := loadedBlocks[0]
|
||||
|
||||
err = dag.maybeAcceptBlock(block, BFNone)
|
||||
if err == nil {
|
||||
t.Errorf("TestMaybeAcceptBlockErrors: rejecting the block if its parents are missing: "+
|
||||
"Expected: %s, got: <nil>", ErrParentBlockUnknown)
|
||||
}
|
||||
var ruleErr RuleError
|
||||
if ok := errors.As(err, &ruleErr); !ok {
|
||||
t.Errorf("TestMaybeAcceptBlockErrors: rejecting the block if its parents are missing: "+
|
||||
"Expected RuleError but got %s", err)
|
||||
} else if ruleErr.ErrorCode != ErrParentBlockUnknown {
|
||||
t.Errorf("TestMaybeAcceptBlockErrors: rejecting the block if its parents are missing: "+
|
||||
"Unexpected error code. Want: %s, got: %s", ErrParentBlockUnknown, ruleErr.ErrorCode)
|
||||
}
|
||||
|
||||
// Test rejecting the block if its parents are invalid
|
||||
blocksFile := "blk_0_to_4.dat"
|
||||
blocks, err := LoadBlocks(filepath.Join("testdata/", blocksFile))
|
||||
if err != nil {
|
||||
t.Fatalf("TestMaybeAcceptBlockErrors: "+
|
||||
"Error loading file '%s': %s\n", blocksFile, err)
|
||||
}
|
||||
|
||||
// Add a valid block and mark it as invalid
|
||||
block1 := blocks[1]
|
||||
isOrphan, isDelayed, err := dag.ProcessBlock(block1, BFNone)
|
||||
if err != nil {
|
||||
t.Fatalf("TestMaybeAcceptBlockErrors: Valid block unexpectedly returned an error: %s", err)
|
||||
}
|
||||
if isDelayed {
|
||||
t.Fatalf("TestMaybeAcceptBlockErrors: block 1 is too far in the future")
|
||||
}
|
||||
if isOrphan {
|
||||
t.Fatalf("TestMaybeAcceptBlockErrors: incorrectly returned block 1 is an orphan")
|
||||
}
|
||||
blockNode1, ok := dag.index.LookupNode(block1.Hash())
|
||||
if !ok {
|
||||
t.Fatalf("block %s does not exist in the DAG", block1.Hash())
|
||||
}
|
||||
dag.index.SetStatusFlags(blockNode1, statusValidateFailed)
|
||||
|
||||
block2 := blocks[2]
|
||||
err = dag.maybeAcceptBlock(block2, BFNone)
|
||||
if err == nil {
|
||||
t.Errorf("TestMaybeAcceptBlockErrors: rejecting the block if its parents are invalid: "+
|
||||
"Expected: %s, got: <nil>", ErrInvalidAncestorBlock)
|
||||
}
|
||||
if ok := errors.As(err, &ruleErr); !ok {
|
||||
t.Errorf("TestMaybeAcceptBlockErrors: rejecting the block if its parents are invalid: "+
|
||||
"Expected RuleError but got %s", err)
|
||||
} else if ruleErr.ErrorCode != ErrInvalidAncestorBlock {
|
||||
t.Errorf("TestMaybeAcceptBlockErrors: rejecting the block if its parents are invalid: "+
|
||||
"Unexpected error. Want: %s, got: %s", ErrInvalidAncestorBlock, ruleErr.ErrorCode)
|
||||
}
|
||||
|
||||
// Set block1's status back to valid for next tests
|
||||
dag.index.UnsetStatusFlags(blockNode1, statusValidateFailed)
|
||||
|
||||
// Test rejecting the block due to bad context
|
||||
originalBits := block2.MsgBlock().Header.Bits
|
||||
block2.MsgBlock().Header.Bits = 0
|
||||
err = dag.maybeAcceptBlock(block2, BFNone)
|
||||
if err == nil {
|
||||
t.Errorf("TestMaybeAcceptBlockErrors: rejecting the block due to bad context: "+
|
||||
"Expected: %s, got: <nil>", ErrUnexpectedDifficulty)
|
||||
}
|
||||
if ok := errors.As(err, &ruleErr); !ok {
|
||||
t.Errorf("TestMaybeAcceptBlockErrors: rejecting the block due to bad context: "+
|
||||
"Expected RuleError but got %s", err)
|
||||
} else if ruleErr.ErrorCode != ErrUnexpectedDifficulty {
|
||||
t.Errorf("TestMaybeAcceptBlockErrors: rejecting the block due to bad context: "+
|
||||
"Unexpected error. Want: %s, got: %s", ErrUnexpectedDifficulty, ruleErr.ErrorCode)
|
||||
}
|
||||
|
||||
// Set block2's bits back to valid for next tests
|
||||
block2.MsgBlock().Header.Bits = originalBits
|
||||
}
|
44
blockdag/behavior_flags.go
Normal file
44
blockdag/behavior_flags.go
Normal file
@ -0,0 +1,44 @@
|
||||
package blockdag
|
||||
|
||||
// BehaviorFlags is a bitmask defining tweaks to the normal behavior when
|
||||
// performing DAG processing and consensus rules checks.
|
||||
type BehaviorFlags uint32
|
||||
|
||||
const (
|
||||
// BFFastAdd may be set to indicate that several checks can be avoided
|
||||
// for the block since it is already known to fit into the DAG due to
|
||||
// already proving it correct links into the DAG.
|
||||
BFFastAdd BehaviorFlags = 1 << iota
|
||||
|
||||
// BFNoPoWCheck may be set to indicate the proof of work check which
|
||||
// ensures a block hashes to a value less than the required target will
|
||||
// not be performed.
|
||||
BFNoPoWCheck
|
||||
|
||||
// BFWasUnorphaned may be set to indicate that a block was just now
|
||||
// unorphaned
|
||||
BFWasUnorphaned
|
||||
|
||||
// BFAfterDelay may be set to indicate that a block had timestamp too far
|
||||
// in the future, just finished the delay
|
||||
BFAfterDelay
|
||||
|
||||
// BFWasStored is set to indicate that the block was previously stored
|
||||
// in the block index but was never fully processed
|
||||
BFWasStored
|
||||
|
||||
// BFDisallowDelay is set to indicate that a delayed block should be rejected.
|
||||
// This is used for the case where a block is submitted through RPC.
|
||||
BFDisallowDelay
|
||||
|
||||
// BFDisallowOrphans is set to indicate that an orphan block should be rejected.
|
||||
// This is used for the case where a block is submitted through RPC.
|
||||
BFDisallowOrphans
|
||||
|
||||
// BFNone is a convenience value to specifically indicate no flags.
|
||||
BFNone BehaviorFlags = 0
|
||||
)
|
||||
|
||||
func isBehaviorFlagRaised(flags BehaviorFlags, flag BehaviorFlags) bool {
|
||||
return flags&flag == flag
|
||||
}
|
302
blockdag/block_utxo.go
Normal file
302
blockdag/block_utxo.go
Normal file
@ -0,0 +1,302 @@
|
||||
package blockdag
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"github.com/kaspanet/go-secp256k1"
|
||||
"github.com/kaspanet/kaspad/util"
|
||||
"github.com/kaspanet/kaspad/util/daghash"
|
||||
"github.com/pkg/errors"
|
||||
)
|
||||
|
||||
// TxAcceptanceData stores a transaction together with an indication
|
||||
// if it was accepted or not by some block
|
||||
type TxAcceptanceData struct {
|
||||
Tx *util.Tx
|
||||
IsAccepted bool
|
||||
}
|
||||
|
||||
// BlockTxsAcceptanceData stores all transactions in a block with an indication
|
||||
// if they were accepted or not by some other block
|
||||
type BlockTxsAcceptanceData struct {
|
||||
BlockHash daghash.Hash
|
||||
TxAcceptanceData []TxAcceptanceData
|
||||
}
|
||||
|
||||
// MultiBlockTxsAcceptanceData stores data about which transactions were accepted by a block
|
||||
// It's a slice of the block's blues block IDs and their transaction acceptance data
|
||||
type MultiBlockTxsAcceptanceData []BlockTxsAcceptanceData
|
||||
|
||||
// FindAcceptanceData finds the BlockTxsAcceptanceData that matches blockHash
|
||||
func (data MultiBlockTxsAcceptanceData) FindAcceptanceData(blockHash *daghash.Hash) (*BlockTxsAcceptanceData, bool) {
|
||||
for _, acceptanceData := range data {
|
||||
if acceptanceData.BlockHash.IsEqual(blockHash) {
|
||||
return &acceptanceData, true
|
||||
}
|
||||
}
|
||||
return nil, false
|
||||
}
|
||||
|
||||
// TxsAcceptedByVirtual retrieves transactions accepted by the current virtual block
|
||||
//
|
||||
// This function MUST be called with the DAG read-lock held
|
||||
func (dag *BlockDAG) TxsAcceptedByVirtual() (MultiBlockTxsAcceptanceData, error) {
|
||||
_, _, txsAcceptanceData, err := dag.pastUTXO(&dag.virtual.blockNode)
|
||||
return txsAcceptanceData, err
|
||||
}
|
||||
|
||||
// TxsAcceptedByBlockHash retrieves transactions accepted by the given block
|
||||
//
|
||||
// This function MUST be called with the DAG read-lock held
|
||||
func (dag *BlockDAG) TxsAcceptedByBlockHash(blockHash *daghash.Hash) (MultiBlockTxsAcceptanceData, error) {
|
||||
node, ok := dag.index.LookupNode(blockHash)
|
||||
if !ok {
|
||||
return nil, errors.Errorf("Couldn't find block %s", blockHash)
|
||||
}
|
||||
_, _, txsAcceptanceData, err := dag.pastUTXO(node)
|
||||
return txsAcceptanceData, err
|
||||
}
|
||||
|
||||
func (dag *BlockDAG) meldVirtualUTXO(newVirtualUTXODiffSet *DiffUTXOSet) error {
|
||||
return newVirtualUTXODiffSet.meldToBase()
|
||||
}
|
||||
|
||||
// checkDoubleSpendsWithBlockPast checks that each block transaction
|
||||
// has a corresponding UTXO in the block pastUTXO.
|
||||
func checkDoubleSpendsWithBlockPast(pastUTXO UTXOSet, blockTransactions []*util.Tx) error {
|
||||
for _, tx := range blockTransactions {
|
||||
if tx.IsCoinBase() {
|
||||
continue
|
||||
}
|
||||
|
||||
for _, txIn := range tx.MsgTx().TxIn {
|
||||
if _, ok := pastUTXO.Get(txIn.PreviousOutpoint); !ok {
|
||||
return ruleError(ErrMissingTxOut, fmt.Sprintf("missing transaction "+
|
||||
"output %s in the utxo set", txIn.PreviousOutpoint))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// verifyAndBuildUTXO verifies all transactions in the given block and builds its UTXO
|
||||
// to save extra traversals it returns the transactions acceptance data, the compactFeeData
|
||||
// for the new block and its multiset.
|
||||
func (node *blockNode) verifyAndBuildUTXO(dag *BlockDAG, transactions []*util.Tx, fastAdd bool) (
|
||||
newBlockUTXO UTXOSet, txsAcceptanceData MultiBlockTxsAcceptanceData, newBlockFeeData compactFeeData, multiset *secp256k1.MultiSet, err error) {
|
||||
|
||||
pastUTXO, selectedParentPastUTXO, txsAcceptanceData, err := dag.pastUTXO(node)
|
||||
if err != nil {
|
||||
return nil, nil, nil, nil, err
|
||||
}
|
||||
|
||||
err = node.validateAcceptedIDMerkleRoot(dag, txsAcceptanceData)
|
||||
if err != nil {
|
||||
return nil, nil, nil, nil, err
|
||||
}
|
||||
|
||||
feeData, err := dag.checkConnectToPastUTXO(node, pastUTXO, transactions, fastAdd)
|
||||
if err != nil {
|
||||
return nil, nil, nil, nil, err
|
||||
}
|
||||
|
||||
multiset, err = node.calcMultiset(dag, txsAcceptanceData, selectedParentPastUTXO)
|
||||
if err != nil {
|
||||
return nil, nil, nil, nil, err
|
||||
}
|
||||
|
||||
err = node.validateUTXOCommitment(multiset)
|
||||
if err != nil {
|
||||
return nil, nil, nil, nil, err
|
||||
}
|
||||
|
||||
return pastUTXO, txsAcceptanceData, feeData, multiset, nil
|
||||
}
|
||||
|
||||
func genesisPastUTXO(virtual *virtualBlock) UTXOSet {
|
||||
// The genesis has no past UTXO, so we create an empty UTXO
|
||||
// set by creating a diff UTXO set with the virtual UTXO
|
||||
// set, and adding all of its entries in toRemove
|
||||
diff := NewUTXODiff()
|
||||
for outpoint, entry := range virtual.utxoSet.utxoCollection {
|
||||
diff.toRemove[outpoint] = entry
|
||||
}
|
||||
genesisPastUTXO := UTXOSet(NewDiffUTXOSet(virtual.utxoSet, diff))
|
||||
return genesisPastUTXO
|
||||
}
|
||||
|
||||
// applyBlueBlocks adds all transactions in the blue blocks to the selectedParent's past UTXO set
|
||||
// Purposefully ignoring failures - these are just unaccepted transactions
|
||||
// Writing down which transactions were accepted or not in txsAcceptanceData
|
||||
func (node *blockNode) applyBlueBlocks(selectedParentPastUTXO UTXOSet, blueBlocks []*util.Block) (
|
||||
pastUTXO UTXOSet, multiBlockTxsAcceptanceData MultiBlockTxsAcceptanceData, err error) {
|
||||
|
||||
pastUTXO = selectedParentPastUTXO.(*DiffUTXOSet).cloneWithoutBase()
|
||||
multiBlockTxsAcceptanceData = make(MultiBlockTxsAcceptanceData, len(blueBlocks))
|
||||
|
||||
// Add blueBlocks to multiBlockTxsAcceptanceData in topological order. This
|
||||
// is so that anyone who iterates over it would process blocks (and transactions)
|
||||
// in their order of appearance in the DAG.
|
||||
for i := 0; i < len(blueBlocks); i++ {
|
||||
blueBlock := blueBlocks[i]
|
||||
transactions := blueBlock.Transactions()
|
||||
blockTxsAcceptanceData := BlockTxsAcceptanceData{
|
||||
BlockHash: *blueBlock.Hash(),
|
||||
TxAcceptanceData: make([]TxAcceptanceData, len(transactions)),
|
||||
}
|
||||
isSelectedParent := i == 0
|
||||
|
||||
for j, tx := range blueBlock.Transactions() {
|
||||
var isAccepted bool
|
||||
|
||||
// Coinbase transaction outputs are added to the UTXO
|
||||
// only if they are in the selected parent chain.
|
||||
if !isSelectedParent && tx.IsCoinBase() {
|
||||
isAccepted = false
|
||||
} else {
|
||||
isAccepted, err = pastUTXO.AddTx(tx.MsgTx(), node.blueScore)
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
}
|
||||
blockTxsAcceptanceData.TxAcceptanceData[j] = TxAcceptanceData{Tx: tx, IsAccepted: isAccepted}
|
||||
}
|
||||
multiBlockTxsAcceptanceData[i] = blockTxsAcceptanceData
|
||||
}
|
||||
|
||||
return pastUTXO, multiBlockTxsAcceptanceData, nil
|
||||
}
|
||||
|
||||
// pastUTXO returns the UTXO of a given block's past
|
||||
// To save traversals over the blue blocks, it also returns the transaction acceptance data for
|
||||
// all blue blocks
|
||||
func (dag *BlockDAG) pastUTXO(node *blockNode) (
|
||||
pastUTXO, selectedParentPastUTXO UTXOSet, bluesTxsAcceptanceData MultiBlockTxsAcceptanceData, err error) {
|
||||
|
||||
if node.isGenesis() {
|
||||
return genesisPastUTXO(dag.virtual), nil, MultiBlockTxsAcceptanceData{}, nil
|
||||
}
|
||||
|
||||
selectedParentPastUTXO, err = dag.restorePastUTXO(node.selectedParent)
|
||||
if err != nil {
|
||||
return nil, nil, nil, err
|
||||
}
|
||||
|
||||
blueBlocks, err := dag.fetchBlueBlocks(node)
|
||||
if err != nil {
|
||||
return nil, nil, nil, err
|
||||
}
|
||||
|
||||
pastUTXO, bluesTxsAcceptanceData, err = node.applyBlueBlocks(selectedParentPastUTXO, blueBlocks)
|
||||
if err != nil {
|
||||
return nil, nil, nil, err
|
||||
}
|
||||
|
||||
return pastUTXO, selectedParentPastUTXO, bluesTxsAcceptanceData, nil
|
||||
}
|
||||
|
||||
// restorePastUTXO restores the UTXO of a given block from its diff
|
||||
func (dag *BlockDAG) restorePastUTXO(node *blockNode) (UTXOSet, error) {
|
||||
stack := []*blockNode{}
|
||||
|
||||
// Iterate over the chain of diff-childs from node till virtual and add them
|
||||
// all into a stack
|
||||
for current := node; current != nil; {
|
||||
stack = append(stack, current)
|
||||
var err error
|
||||
current, err = dag.utxoDiffStore.diffChildByNode(current)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
|
||||
// Start with the top item in the stack, going over it top-to-bottom,
|
||||
// applying the UTXO-diff one-by-one.
|
||||
topNode, stack := stack[len(stack)-1], stack[:len(stack)-1] // pop the top item in the stack
|
||||
topNodeDiff, err := dag.utxoDiffStore.diffByNode(topNode)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
accumulatedDiff := topNodeDiff.clone()
|
||||
|
||||
for i := len(stack) - 1; i >= 0; i-- {
|
||||
diff, err := dag.utxoDiffStore.diffByNode(stack[i])
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
// Use withDiffInPlace, otherwise copying the diffs again and again create a polynomial overhead
|
||||
err = accumulatedDiff.withDiffInPlace(diff)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
|
||||
return NewDiffUTXOSet(dag.virtual.utxoSet, accumulatedDiff), nil
|
||||
}
|
||||
|
||||
// updateTipsUTXO builds and applies new diff UTXOs for all the DAG's tips
|
||||
func updateTipsUTXO(dag *BlockDAG, virtualUTXO UTXOSet) error {
|
||||
for tip := range dag.virtual.parents {
|
||||
tipPastUTXO, err := dag.restorePastUTXO(tip)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
diff, err := virtualUTXO.diffFrom(tipPastUTXO)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
err = dag.utxoDiffStore.setBlockDiff(tip, diff)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// updateParents adds this block to the children sets of its parents
|
||||
// and updates the diff of any parent whose DiffChild is this block
|
||||
func (node *blockNode) updateParents(dag *BlockDAG, newBlockUTXO UTXOSet) error {
|
||||
node.updateParentsChildren()
|
||||
return node.updateParentsDiffs(dag, newBlockUTXO)
|
||||
}
|
||||
|
||||
// updateParentsDiffs updates the diff of any parent whose DiffChild is this block
|
||||
func (node *blockNode) updateParentsDiffs(dag *BlockDAG, newBlockUTXO UTXOSet) error {
|
||||
virtualDiffFromNewBlock, err := dag.virtual.utxoSet.diffFrom(newBlockUTXO)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
err = dag.utxoDiffStore.setBlockDiff(node, virtualDiffFromNewBlock)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
for parent := range node.parents {
|
||||
diffChild, err := dag.utxoDiffStore.diffChildByNode(parent)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if diffChild == nil {
|
||||
parentPastUTXO, err := dag.restorePastUTXO(parent)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
err = dag.utxoDiffStore.setBlockDiffChild(parent, node)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
diff, err := newBlockUTXO.diffFrom(parentPastUTXO)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
err = dag.utxoDiffStore.setBlockDiff(parent, diff)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
@ -5,7 +5,9 @@
|
||||
package blockdag
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"github.com/kaspanet/kaspad/dbaccess"
|
||||
"github.com/kaspanet/kaspad/util"
|
||||
"sync"
|
||||
|
||||
"github.com/kaspanet/kaspad/dagconfig"
|
||||
@ -134,3 +136,42 @@ func (bi *blockIndex) flushToDB(dbContext *dbaccess.TxContext) error {
|
||||
func (bi *blockIndex) clearDirtyEntries() {
|
||||
bi.dirty = make(map[*blockNode]struct{})
|
||||
}
|
||||
|
||||
func (dag *BlockDAG) addNodeToIndexWithInvalidAncestor(block *util.Block) error {
|
||||
blockHeader := &block.MsgBlock().Header
|
||||
newNode, _ := dag.newBlockNode(blockHeader, newBlockSet())
|
||||
newNode.status = statusInvalidAncestor
|
||||
dag.index.AddNode(newNode)
|
||||
|
||||
dbTx, err := dag.databaseContext.NewTx()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
defer dbTx.RollbackUnlessClosed()
|
||||
err = dag.index.flushToDB(dbTx)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return dbTx.Commit()
|
||||
}
|
||||
|
||||
func lookupParentNodes(block *util.Block, dag *BlockDAG) (blockSet, error) {
|
||||
header := block.MsgBlock().Header
|
||||
parentHashes := header.ParentHashes
|
||||
|
||||
nodes := newBlockSet()
|
||||
for _, parentHash := range parentHashes {
|
||||
node, ok := dag.index.LookupNode(parentHash)
|
||||
if !ok {
|
||||
str := fmt.Sprintf("parent block %s is unknown", parentHash)
|
||||
return nil, ruleError(ErrParentBlockUnknown, str)
|
||||
} else if dag.index.NodeStatus(node).KnownInvalid() {
|
||||
str := fmt.Sprintf("parent block %s is known to be invalid", parentHash)
|
||||
return nil, ruleError(ErrInvalidAncestorBlock, str)
|
||||
}
|
||||
|
||||
nodes.add(node)
|
||||
}
|
||||
|
||||
return nodes, nil
|
||||
}
|
||||
|
@ -1,6 +1,7 @@
|
||||
package blockdag
|
||||
|
||||
import (
|
||||
"github.com/kaspanet/kaspad/domainmessage"
|
||||
"github.com/kaspanet/kaspad/util/daghash"
|
||||
"github.com/pkg/errors"
|
||||
)
|
||||
@ -107,3 +108,136 @@ func (dag *BlockDAG) FindNextLocatorBoundaries(locator BlockLocator) (highHash,
|
||||
}
|
||||
return locator[nextBlockLocatorIndex], lowNode.hash
|
||||
}
|
||||
|
||||
// antiPastHashesBetween returns the hashes of the blocks between the
|
||||
// lowHash's antiPast and highHash's antiPast, or up to the provided
|
||||
// max number of block hashes.
|
||||
//
|
||||
// This function MUST be called with the DAG state lock held (for reads).
|
||||
func (dag *BlockDAG) antiPastHashesBetween(lowHash, highHash *daghash.Hash, maxHashes uint64) ([]*daghash.Hash, error) {
|
||||
nodes, err := dag.antiPastBetween(lowHash, highHash, maxHashes)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
hashes := make([]*daghash.Hash, len(nodes))
|
||||
for i, node := range nodes {
|
||||
hashes[i] = node.hash
|
||||
}
|
||||
return hashes, nil
|
||||
}
|
||||
|
||||
// antiPastBetween returns the blockNodes between the lowHash's antiPast
|
||||
// and highHash's antiPast, or up to the provided max number of blocks.
|
||||
//
|
||||
// This function MUST be called with the DAG state lock held (for reads).
|
||||
func (dag *BlockDAG) antiPastBetween(lowHash, highHash *daghash.Hash, maxEntries uint64) ([]*blockNode, error) {
|
||||
lowNode, ok := dag.index.LookupNode(lowHash)
|
||||
if !ok {
|
||||
return nil, errors.Errorf("Couldn't find low hash %s", lowHash)
|
||||
}
|
||||
highNode, ok := dag.index.LookupNode(highHash)
|
||||
if !ok {
|
||||
return nil, errors.Errorf("Couldn't find high hash %s", highHash)
|
||||
}
|
||||
if lowNode.blueScore >= highNode.blueScore {
|
||||
return nil, errors.Errorf("Low hash blueScore >= high hash blueScore (%d >= %d)",
|
||||
lowNode.blueScore, highNode.blueScore)
|
||||
}
|
||||
|
||||
// In order to get no more then maxEntries blocks from the
|
||||
// future of the lowNode (including itself), we iterate the
|
||||
// selected parent chain of the highNode and stop once we reach
|
||||
// highNode.blueScore-lowNode.blueScore+1 <= maxEntries. That
|
||||
// stop point becomes the new highNode.
|
||||
// Using blueScore as an approximation is considered to be
|
||||
// fairly accurate because we presume that most DAG blocks are
|
||||
// blue.
|
||||
for highNode.blueScore-lowNode.blueScore+1 > maxEntries {
|
||||
highNode = highNode.selectedParent
|
||||
}
|
||||
|
||||
// Collect every node in highNode's past (including itself) but
|
||||
// NOT in the lowNode's past (excluding itself) into an up-heap
|
||||
// (a heap sorted by blueScore from lowest to greatest).
|
||||
visited := newBlockSet()
|
||||
candidateNodes := newUpHeap()
|
||||
queue := newDownHeap()
|
||||
queue.Push(highNode)
|
||||
for queue.Len() > 0 {
|
||||
current := queue.pop()
|
||||
if visited.contains(current) {
|
||||
continue
|
||||
}
|
||||
visited.add(current)
|
||||
isCurrentAncestorOfLowNode, err := dag.isInPast(current, lowNode)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if isCurrentAncestorOfLowNode {
|
||||
continue
|
||||
}
|
||||
candidateNodes.Push(current)
|
||||
for parent := range current.parents {
|
||||
queue.Push(parent)
|
||||
}
|
||||
}
|
||||
|
||||
// Pop candidateNodes into a slice. Since candidateNodes is
|
||||
// an up-heap, it's guaranteed to be ordered from low to high
|
||||
nodesLen := int(maxEntries)
|
||||
if candidateNodes.Len() < nodesLen {
|
||||
nodesLen = candidateNodes.Len()
|
||||
}
|
||||
nodes := make([]*blockNode, nodesLen)
|
||||
for i := 0; i < nodesLen; i++ {
|
||||
nodes[i] = candidateNodes.pop()
|
||||
}
|
||||
return nodes, nil
|
||||
}
|
||||
|
||||
// AntiPastHashesBetween returns the hashes of the blocks between the
|
||||
// lowHash's antiPast and highHash's antiPast, or up to the provided
|
||||
// max number of block hashes.
|
||||
//
|
||||
// This function is safe for concurrent access.
|
||||
func (dag *BlockDAG) AntiPastHashesBetween(lowHash, highHash *daghash.Hash, maxHashes uint64) ([]*daghash.Hash, error) {
|
||||
dag.dagLock.RLock()
|
||||
defer dag.dagLock.RUnlock()
|
||||
hashes, err := dag.antiPastHashesBetween(lowHash, highHash, maxHashes)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return hashes, nil
|
||||
}
|
||||
|
||||
// antiPastHeadersBetween returns the headers of the blocks between the
|
||||
// lowHash's antiPast and highHash's antiPast, or up to the provided
|
||||
// max number of block headers.
|
||||
//
|
||||
// This function MUST be called with the DAG state lock held (for reads).
|
||||
func (dag *BlockDAG) antiPastHeadersBetween(lowHash, highHash *daghash.Hash, maxHeaders uint64) ([]*domainmessage.BlockHeader, error) {
|
||||
nodes, err := dag.antiPastBetween(lowHash, highHash, maxHeaders)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
headers := make([]*domainmessage.BlockHeader, len(nodes))
|
||||
for i, node := range nodes {
|
||||
headers[i] = node.Header()
|
||||
}
|
||||
return headers, nil
|
||||
}
|
||||
|
||||
// AntiPastHeadersBetween returns the headers of the blocks between the
|
||||
// lowHash's antiPast and highHash's antiPast, or up to
|
||||
// domainmessage.MaxBlockHeadersPerMsg block headers.
|
||||
//
|
||||
// This function is safe for concurrent access.
|
||||
func (dag *BlockDAG) AntiPastHeadersBetween(lowHash, highHash *daghash.Hash, maxHeaders uint64) ([]*domainmessage.BlockHeader, error) {
|
||||
dag.dagLock.RLock()
|
||||
defer dag.dagLock.RUnlock()
|
||||
headers, err := dag.antiPastHeadersBetween(lowHash, highHash, maxHeaders)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return headers, nil
|
||||
}
|
||||
|
@ -197,3 +197,24 @@ func coinbaseOutputForBlueBlock(dag *BlockDAG, blueBlock *blockNode,
|
||||
|
||||
return txOut, nil
|
||||
}
|
||||
|
||||
// NextBlockCoinbaseTransaction prepares the coinbase transaction for the next mined block
|
||||
//
|
||||
// This function CAN'T be called with the DAG lock held.
|
||||
func (dag *BlockDAG) NextBlockCoinbaseTransaction(scriptPubKey []byte, extraData []byte) (*util.Tx, error) {
|
||||
dag.dagLock.RLock()
|
||||
defer dag.dagLock.RUnlock()
|
||||
|
||||
return dag.NextBlockCoinbaseTransactionNoLock(scriptPubKey, extraData)
|
||||
}
|
||||
|
||||
// NextBlockCoinbaseTransactionNoLock prepares the coinbase transaction for the next mined block
|
||||
//
|
||||
// This function MUST be called with the DAG read-lock held
|
||||
func (dag *BlockDAG) NextBlockCoinbaseTransactionNoLock(scriptPubKey []byte, extraData []byte) (*util.Tx, error) {
|
||||
txsAcceptanceData, err := dag.TxsAcceptedByVirtual()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return dag.virtual.blockNode.expectedCoinbaseTransaction(dag, txsAcceptanceData, scriptPubKey, extraData)
|
||||
}
|
||||
|
@ -103,8 +103,6 @@ func newTestDAG(params *dagconfig.Params) *BlockDAG {
|
||||
TimestampDeviationTolerance: params.TimestampDeviationTolerance,
|
||||
powMaxBits: util.BigToCompact(params.PowMax),
|
||||
index: index,
|
||||
warningCaches: newThresholdCaches(vbNumBits),
|
||||
deploymentCaches: newThresholdCaches(dagconfig.DefinedDeployments),
|
||||
}
|
||||
|
||||
// Create a genesis block node and block index index populated with it
|
||||
|
54
blockdag/config.go
Normal file
54
blockdag/config.go
Normal file
@ -0,0 +1,54 @@
|
||||
package blockdag
|
||||
|
||||
import (
|
||||
"github.com/kaspanet/kaspad/dagconfig"
|
||||
"github.com/kaspanet/kaspad/dbaccess"
|
||||
"github.com/kaspanet/kaspad/txscript"
|
||||
"github.com/kaspanet/kaspad/util/subnetworkid"
|
||||
)
|
||||
|
||||
// Config is a descriptor which specifies the blockDAG instance configuration.
|
||||
type Config struct {
|
||||
// Interrupt specifies a channel the caller can close to signal that
|
||||
// long running operations, such as catching up indexes or performing
|
||||
// database migrations, should be interrupted.
|
||||
//
|
||||
// This field can be nil if the caller does not desire the behavior.
|
||||
Interrupt <-chan struct{}
|
||||
|
||||
// DAGParams identifies which DAG parameters the DAG is associated
|
||||
// with.
|
||||
//
|
||||
// This field is required.
|
||||
DAGParams *dagconfig.Params
|
||||
|
||||
// TimeSource defines the time source to use for things such as
|
||||
// block processing and determining whether or not the DAG is current.
|
||||
TimeSource TimeSource
|
||||
|
||||
// SigCache defines a signature cache to use when when validating
|
||||
// signatures. This is typically most useful when individual
|
||||
// transactions are already being validated prior to their inclusion in
|
||||
// a block such as what is usually done via a transaction memory pool.
|
||||
//
|
||||
// This field can be nil if the caller is not interested in using a
|
||||
// signature cache.
|
||||
SigCache *txscript.SigCache
|
||||
|
||||
// IndexManager defines an index manager to use when initializing the
|
||||
// DAG and connecting blocks.
|
||||
//
|
||||
// This field can be nil if the caller does not wish to make use of an
|
||||
// index manager.
|
||||
IndexManager IndexManager
|
||||
|
||||
// SubnetworkID identifies which subnetwork the DAG is associated
|
||||
// with.
|
||||
//
|
||||
// This field is required.
|
||||
SubnetworkID *subnetworkid.SubnetworkID
|
||||
|
||||
// DatabaseContext is the context in which all database queries related to
|
||||
// this DAG are going to run.
|
||||
DatabaseContext *dbaccess.DatabaseContext
|
||||
}
|
137
blockdag/confirmations.go
Normal file
137
blockdag/confirmations.go
Normal file
@ -0,0 +1,137 @@
|
||||
package blockdag
|
||||
|
||||
import (
|
||||
"github.com/kaspanet/kaspad/domainmessage"
|
||||
"github.com/kaspanet/kaspad/util"
|
||||
"github.com/kaspanet/kaspad/util/daghash"
|
||||
"github.com/pkg/errors"
|
||||
)
|
||||
|
||||
// BlockConfirmationsByHash returns the confirmations number for a block with the
|
||||
// given hash. See blockConfirmations for further details.
|
||||
//
|
||||
// This function is safe for concurrent access
|
||||
func (dag *BlockDAG) BlockConfirmationsByHash(hash *daghash.Hash) (uint64, error) {
|
||||
dag.dagLock.RLock()
|
||||
defer dag.dagLock.RUnlock()
|
||||
|
||||
return dag.BlockConfirmationsByHashNoLock(hash)
|
||||
}
|
||||
|
||||
// BlockConfirmationsByHashNoLock is lock free version of BlockConfirmationsByHash
|
||||
//
|
||||
// This function is unsafe for concurrent access.
|
||||
func (dag *BlockDAG) BlockConfirmationsByHashNoLock(hash *daghash.Hash) (uint64, error) {
|
||||
if hash.IsEqual(&daghash.ZeroHash) {
|
||||
return 0, nil
|
||||
}
|
||||
|
||||
node, ok := dag.index.LookupNode(hash)
|
||||
if !ok {
|
||||
return 0, errors.Errorf("block %s is unknown", hash)
|
||||
}
|
||||
|
||||
return dag.blockConfirmations(node)
|
||||
}
|
||||
|
||||
// UTXOConfirmations returns the confirmations for the given outpoint, if it exists
|
||||
// in the DAG's UTXO set.
|
||||
//
|
||||
// This function is safe for concurrent access.
|
||||
func (dag *BlockDAG) UTXOConfirmations(outpoint *domainmessage.Outpoint) (uint64, bool) {
|
||||
dag.dagLock.RLock()
|
||||
defer dag.dagLock.RUnlock()
|
||||
|
||||
utxoEntry, ok := dag.GetUTXOEntry(*outpoint)
|
||||
if !ok {
|
||||
return 0, false
|
||||
}
|
||||
confirmations := dag.SelectedTipBlueScore() - utxoEntry.BlockBlueScore() + 1
|
||||
|
||||
return confirmations, true
|
||||
}
|
||||
|
||||
// blockConfirmations returns the current confirmations number of the given node
|
||||
// The confirmations number is defined as follows:
|
||||
// * If the node is in the selected tip red set -> 0
|
||||
// * If the node is the selected tip -> 1
|
||||
// * Otherwise -> selectedTip.blueScore - acceptingBlock.blueScore + 2
|
||||
func (dag *BlockDAG) blockConfirmations(node *blockNode) (uint64, error) {
|
||||
acceptingBlock, err := dag.acceptingBlock(node)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
|
||||
// if acceptingBlock is nil, the node is red
|
||||
if acceptingBlock == nil {
|
||||
return 0, nil
|
||||
}
|
||||
|
||||
return dag.selectedTip().blueScore - acceptingBlock.blueScore + 1, nil
|
||||
}
|
||||
|
||||
// acceptingBlock finds the node in the selected-parent chain that had accepted
|
||||
// the given node
|
||||
func (dag *BlockDAG) acceptingBlock(node *blockNode) (*blockNode, error) {
|
||||
// Return an error if the node is the virtual block
|
||||
if node == &dag.virtual.blockNode {
|
||||
return nil, errors.New("cannot get acceptingBlock for virtual")
|
||||
}
|
||||
|
||||
// If the node is a chain-block itself, the accepting block is its chain-child
|
||||
isNodeInSelectedParentChain, err := dag.IsInSelectedParentChain(node.hash)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if isNodeInSelectedParentChain {
|
||||
if len(node.children) == 0 {
|
||||
// If the node is the selected tip, it doesn't have an accepting block
|
||||
return nil, nil
|
||||
}
|
||||
for child := range node.children {
|
||||
isChildInSelectedParentChain, err := dag.IsInSelectedParentChain(child.hash)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if isChildInSelectedParentChain {
|
||||
return child, nil
|
||||
}
|
||||
}
|
||||
return nil, errors.Errorf("chain block %s does not have a chain child", node.hash)
|
||||
}
|
||||
|
||||
// Find the only chain block that may contain the node in its blues
|
||||
candidateAcceptingBlock := dag.oldestChainBlockWithBlueScoreGreaterThan(node.blueScore)
|
||||
|
||||
// if no candidate is found, it means that the node has same or more
|
||||
// blue score than the selected tip and is found in its anticone, so
|
||||
// it doesn't have an accepting block
|
||||
if candidateAcceptingBlock == nil {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// candidateAcceptingBlock is the accepting block only if it actually contains
|
||||
// the node in its blues
|
||||
for _, blue := range candidateAcceptingBlock.blues {
|
||||
if blue == node {
|
||||
return candidateAcceptingBlock, nil
|
||||
}
|
||||
}
|
||||
|
||||
// Otherwise, the node is red or in the selected tip anticone, and
|
||||
// doesn't have an accepting block
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// oldestChainBlockWithBlueScoreGreaterThan finds the oldest chain block with a blue score
|
||||
// greater than blueScore. If no such block exists, this method returns nil
|
||||
func (dag *BlockDAG) oldestChainBlockWithBlueScoreGreaterThan(blueScore uint64) *blockNode {
|
||||
chainBlockIndex, ok := util.SearchSlice(len(dag.virtual.selectedParentChainSlice), func(i int) bool {
|
||||
selectedPathNode := dag.virtual.selectedParentChainSlice[i]
|
||||
return selectedPathNode.blueScore > blueScore
|
||||
})
|
||||
if !ok {
|
||||
return nil
|
||||
}
|
||||
return dag.virtual.selectedParentChainSlice[chainBlockIndex]
|
||||
}
|
1821
blockdag/dag.go
1821
blockdag/dag.go
File diff suppressed because it is too large
Load Diff
@ -305,7 +305,6 @@ func TestCalcSequenceLock(t *testing.T) {
|
||||
name string
|
||||
tx *domainmessage.MsgTx
|
||||
utxoSet UTXOSet
|
||||
mempool bool
|
||||
want *SequenceLock
|
||||
}{
|
||||
// A transaction with a single input with max sequence number.
|
||||
@ -462,7 +461,6 @@ func TestCalcSequenceLock(t *testing.T) {
|
||||
name: "single input, unconfirmed, lock-time in blocks",
|
||||
tx: domainmessage.NewNativeMsgTx(1, []*domainmessage.TxIn{{PreviousOutpoint: unConfUtxo, Sequence: LockTimeToSequence(false, 2)}}, nil),
|
||||
utxoSet: utxoSet,
|
||||
mempool: true,
|
||||
want: &SequenceLock{
|
||||
Milliseconds: -1,
|
||||
BlockBlueScore: int64(nextBlockBlueScore) + 1,
|
||||
@ -475,7 +473,6 @@ func TestCalcSequenceLock(t *testing.T) {
|
||||
name: "single input, unconfirmed, lock-time in milliseoncds",
|
||||
tx: domainmessage.NewNativeMsgTx(1, []*domainmessage.TxIn{{PreviousOutpoint: unConfUtxo, Sequence: LockTimeToSequence(true, 1048576)}}, nil),
|
||||
utxoSet: utxoSet,
|
||||
mempool: true,
|
||||
want: &SequenceLock{
|
||||
Milliseconds: nextMedianTime + 1048575,
|
||||
BlockBlueScore: -1,
|
||||
@ -486,7 +483,7 @@ func TestCalcSequenceLock(t *testing.T) {
|
||||
t.Logf("Running %v SequenceLock tests", len(tests))
|
||||
for _, test := range tests {
|
||||
utilTx := util.NewTx(test.tx)
|
||||
seqLock, err := dag.CalcSequenceLock(utilTx, utxoSet, test.mempool)
|
||||
seqLock, err := dag.CalcSequenceLock(utilTx, utxoSet)
|
||||
if err != nil {
|
||||
t.Fatalf("test '%s', unable to calc sequence lock: %v", test.name, err)
|
||||
}
|
||||
@ -1124,21 +1121,6 @@ func TestDAGIndexFailedStatus(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
func TestIsDAGCurrentMaxDiff(t *testing.T) {
|
||||
netParams := []*dagconfig.Params{
|
||||
&dagconfig.MainnetParams,
|
||||
&dagconfig.TestnetParams,
|
||||
&dagconfig.DevnetParams,
|
||||
&dagconfig.RegressionNetParams,
|
||||
&dagconfig.SimnetParams,
|
||||
}
|
||||
for _, params := range netParams {
|
||||
if params.FinalityDuration < isDAGCurrentMaxDiff*params.TargetTimePerBlock {
|
||||
t.Errorf("in %s, a DAG can be considered current even if it's below the finality point", params.Name)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func testProcessBlockRuleError(t *testing.T, dag *BlockDAG, block *domainmessage.MsgBlock, expectedRuleErr error) {
|
||||
isOrphan, isDelayed, err := dag.ProcessBlock(util.NewBlock(block), BFNoPoWCheck)
|
||||
|
||||
|
@ -662,3 +662,16 @@ func (dag *BlockDAG) BlockHashesFrom(lowHash *daghash.Hash, limit int) ([]*dagha
|
||||
|
||||
return blockHashes, nil
|
||||
}
|
||||
|
||||
func (dag *BlockDAG) fetchBlueBlocks(node *blockNode) ([]*util.Block, error) {
|
||||
blueBlocks := make([]*util.Block, len(node.blues))
|
||||
for i, blueBlockNode := range node.blues {
|
||||
blueBlock, err := dag.fetchBlockByHash(blueBlockNode.hash)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
blueBlocks[i] = blueBlock
|
||||
}
|
||||
return blueBlocks, nil
|
||||
}
|
||||
|
95
blockdag/delayed_blocks.go
Normal file
95
blockdag/delayed_blocks.go
Normal file
@ -0,0 +1,95 @@
|
||||
package blockdag
|
||||
|
||||
import (
|
||||
"github.com/kaspanet/kaspad/util"
|
||||
"github.com/kaspanet/kaspad/util/daghash"
|
||||
"github.com/kaspanet/kaspad/util/mstime"
|
||||
"github.com/pkg/errors"
|
||||
"time"
|
||||
)
|
||||
|
||||
// delayedBlock represents a block which has a delayed timestamp and will be processed at processTime
|
||||
type delayedBlock struct {
|
||||
block *util.Block
|
||||
processTime mstime.Time
|
||||
}
|
||||
|
||||
func (dag *BlockDAG) isKnownDelayedBlock(hash *daghash.Hash) bool {
|
||||
_, exists := dag.delayedBlocks[*hash]
|
||||
return exists
|
||||
}
|
||||
|
||||
func (dag *BlockDAG) addDelayedBlock(block *util.Block, delay time.Duration) error {
|
||||
processTime := dag.Now().Add(delay)
|
||||
log.Debugf("Adding block to delayed blocks queue (block hash: %s, process time: %s)", block.Hash().String(), processTime)
|
||||
delayedBlock := &delayedBlock{
|
||||
block: block,
|
||||
processTime: processTime,
|
||||
}
|
||||
|
||||
dag.delayedBlocks[*block.Hash()] = delayedBlock
|
||||
dag.delayedBlocksQueue.Push(delayedBlock)
|
||||
return dag.processDelayedBlocks()
|
||||
}
|
||||
|
||||
// processDelayedBlocks loops over all delayed blocks and processes blocks which are due.
|
||||
// This method is invoked after processing a block (ProcessBlock method).
|
||||
func (dag *BlockDAG) processDelayedBlocks() error {
|
||||
// Check if the delayed block with the earliest process time should be processed
|
||||
for dag.delayedBlocksQueue.Len() > 0 {
|
||||
earliestDelayedBlockProcessTime := dag.peekDelayedBlock().processTime
|
||||
if earliestDelayedBlockProcessTime.After(dag.Now()) {
|
||||
break
|
||||
}
|
||||
delayedBlock := dag.popDelayedBlock()
|
||||
_, _, err := dag.processBlockNoLock(delayedBlock.block, BFAfterDelay)
|
||||
if err != nil {
|
||||
log.Errorf("Error while processing delayed block (block %s): %s", delayedBlock.block.Hash().String(), err)
|
||||
// Rule errors should not be propagated as they refer only to the delayed block,
|
||||
// while this function runs in the context of another block
|
||||
if !errors.As(err, &RuleError{}) {
|
||||
return err
|
||||
}
|
||||
}
|
||||
log.Debugf("Processed delayed block (block %s)", delayedBlock.block.Hash().String())
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// popDelayedBlock removes the topmost (delayed block with earliest process time) of the queue and returns it.
|
||||
func (dag *BlockDAG) popDelayedBlock() *delayedBlock {
|
||||
delayedBlock := dag.delayedBlocksQueue.pop()
|
||||
delete(dag.delayedBlocks, *delayedBlock.block.Hash())
|
||||
return delayedBlock
|
||||
}
|
||||
|
||||
func (dag *BlockDAG) peekDelayedBlock() *delayedBlock {
|
||||
return dag.delayedBlocksQueue.peek()
|
||||
}
|
||||
|
||||
// maxDelayOfParents returns the maximum delay of the given block hashes.
|
||||
// Note that delay could be 0, but isDelayed will return true. This is the case where the parent process time is due.
|
||||
func (dag *BlockDAG) maxDelayOfParents(parentHashes []*daghash.Hash) (delay time.Duration, isDelayed bool) {
|
||||
for _, parentHash := range parentHashes {
|
||||
if delayedParent, exists := dag.delayedBlocks[*parentHash]; exists {
|
||||
isDelayed = true
|
||||
parentDelay := delayedParent.processTime.Sub(dag.Now())
|
||||
if parentDelay > delay {
|
||||
delay = parentDelay
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return delay, isDelayed
|
||||
}
|
||||
|
||||
func (dag *BlockDAG) shouldBlockBeDelayed(block *util.Block) (delay time.Duration, isDelayed bool) {
|
||||
header := &block.MsgBlock().Header
|
||||
|
||||
maxTimestamp := dag.Now().Add(time.Duration(dag.TimestampDeviationTolerance) * dag.Params.TargetTimePerBlock)
|
||||
if header.Timestamp.After(maxTimestamp) {
|
||||
return header.Timestamp.Sub(maxTimestamp), true
|
||||
}
|
||||
return 0, false
|
||||
}
|
32
blockdag/delayed_blocks_test.go
Normal file
32
blockdag/delayed_blocks_test.go
Normal file
@ -0,0 +1,32 @@
|
||||
package blockdag
|
||||
|
||||
import (
|
||||
"github.com/kaspanet/kaspad/dagconfig"
|
||||
"github.com/kaspanet/kaspad/util"
|
||||
"testing"
|
||||
"time"
|
||||
)
|
||||
|
||||
func TestShouldBlockBeDelayed(t *testing.T) {
|
||||
// Create a new database and dag instance to run tests against.
|
||||
dag, teardownFunc, err := DAGSetup("TestShouldBlockBeDelayed", true, Config{
|
||||
DAGParams: &dagconfig.SimnetParams,
|
||||
})
|
||||
if err != nil {
|
||||
t.Errorf("Failed to setup dag instance: %v", err)
|
||||
return
|
||||
}
|
||||
defer teardownFunc()
|
||||
|
||||
blockInTheFuture := Block100000
|
||||
expectedDelay := 10 * time.Second
|
||||
deviationTolerance := time.Duration(dag.TimestampDeviationTolerance) * dag.Params.TargetTimePerBlock
|
||||
blockInTheFuture.Header.Timestamp = dag.Now().Add(deviationTolerance + expectedDelay)
|
||||
delay, isDelayed := dag.shouldBlockBeDelayed(util.NewBlock(&blockInTheFuture))
|
||||
if !isDelayed {
|
||||
t.Errorf("TestShouldBlockBeDelayed: block unexpectedly not delayed")
|
||||
}
|
||||
if delay != expectedDelay {
|
||||
t.Errorf("TestShouldBlockBeDelayed: expected %s delay but got %s", expectedDelay, delay)
|
||||
}
|
||||
}
|
113
blockdag/finality.go
Normal file
113
blockdag/finality.go
Normal file
@ -0,0 +1,113 @@
|
||||
package blockdag
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"github.com/kaspanet/kaspad/util/daghash"
|
||||
)
|
||||
|
||||
// LastFinalityPointHash returns the hash of the last finality point
|
||||
func (dag *BlockDAG) LastFinalityPointHash() *daghash.Hash {
|
||||
if dag.lastFinalityPoint == nil {
|
||||
return nil
|
||||
}
|
||||
return dag.lastFinalityPoint.hash
|
||||
}
|
||||
|
||||
// FinalityInterval is the interval that determines the finality window of the DAG.
|
||||
func (dag *BlockDAG) FinalityInterval() uint64 {
|
||||
return uint64(dag.Params.FinalityDuration / dag.Params.TargetTimePerBlock)
|
||||
}
|
||||
|
||||
// checkFinalityViolation checks the new block does not violate the finality rules
|
||||
// specifically - the new block selectedParent chain should contain the old finality point.
|
||||
func (dag *BlockDAG) checkFinalityViolation(newNode *blockNode) error {
|
||||
// the genesis block can not violate finality rules
|
||||
if newNode.isGenesis() {
|
||||
return nil
|
||||
}
|
||||
|
||||
// Because newNode doesn't have reachability data we
|
||||
// need to check if the last finality point is in the
|
||||
// selected parent chain of newNode.selectedParent, so
|
||||
// we explicitly check if newNode.selectedParent is
|
||||
// the finality point.
|
||||
if dag.lastFinalityPoint == newNode.selectedParent {
|
||||
return nil
|
||||
}
|
||||
|
||||
isInSelectedChain, err := dag.isInSelectedParentChainOf(dag.lastFinalityPoint, newNode.selectedParent)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if !isInSelectedChain {
|
||||
return ruleError(ErrFinality, "the last finality point is not in the selected parent chain of this block")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// updateFinalityPoint updates the dag's last finality point if necessary.
|
||||
func (dag *BlockDAG) updateFinalityPoint() {
|
||||
selectedTip := dag.selectedTip()
|
||||
// if the selected tip is the genesis block - it should be the new finality point
|
||||
if selectedTip.isGenesis() {
|
||||
dag.lastFinalityPoint = selectedTip
|
||||
return
|
||||
}
|
||||
// We are looking for a new finality point only if the new block's finality score is higher
|
||||
// by 2 than the existing finality point's
|
||||
if selectedTip.finalityScore(dag) < dag.lastFinalityPoint.finalityScore(dag)+2 {
|
||||
return
|
||||
}
|
||||
|
||||
var currentNode *blockNode
|
||||
for currentNode = selectedTip.selectedParent; ; currentNode = currentNode.selectedParent {
|
||||
// We look for the first node in the selected parent chain that has a higher finality score than the last finality point.
|
||||
if currentNode.selectedParent.finalityScore(dag) == dag.lastFinalityPoint.finalityScore(dag) {
|
||||
break
|
||||
}
|
||||
}
|
||||
dag.lastFinalityPoint = currentNode
|
||||
spawn("dag.finalizeNodesBelowFinalityPoint", func() {
|
||||
dag.finalizeNodesBelowFinalityPoint(true)
|
||||
})
|
||||
}
|
||||
|
||||
func (dag *BlockDAG) finalizeNodesBelowFinalityPoint(deleteDiffData bool) {
|
||||
queue := make([]*blockNode, 0, len(dag.lastFinalityPoint.parents))
|
||||
for parent := range dag.lastFinalityPoint.parents {
|
||||
queue = append(queue, parent)
|
||||
}
|
||||
var nodesToDelete []*blockNode
|
||||
if deleteDiffData {
|
||||
nodesToDelete = make([]*blockNode, 0, dag.FinalityInterval())
|
||||
}
|
||||
for len(queue) > 0 {
|
||||
var current *blockNode
|
||||
current, queue = queue[0], queue[1:]
|
||||
if !current.isFinalized {
|
||||
current.isFinalized = true
|
||||
if deleteDiffData {
|
||||
nodesToDelete = append(nodesToDelete, current)
|
||||
}
|
||||
for parent := range current.parents {
|
||||
queue = append(queue, parent)
|
||||
}
|
||||
}
|
||||
}
|
||||
if deleteDiffData {
|
||||
err := dag.utxoDiffStore.removeBlocksDiffData(dag.databaseContext, nodesToDelete)
|
||||
if err != nil {
|
||||
panic(fmt.Sprintf("Error removing diff data from utxoDiffStore: %s", err))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// IsKnownFinalizedBlock returns whether the block is below the finality point.
|
||||
// IsKnownFinalizedBlock might be false-negative because node finality status is
|
||||
// updated in a separate goroutine. To get a definite answer if a block
|
||||
// is finalized or not, use dag.checkFinalityViolation.
|
||||
func (dag *BlockDAG) IsKnownFinalizedBlock(blockHash *daghash.Hash) bool {
|
||||
node, ok := dag.index.LookupNode(blockHash)
|
||||
return ok && node.isFinalized
|
||||
}
|
18
blockdag/index_manager.go
Normal file
18
blockdag/index_manager.go
Normal file
@ -0,0 +1,18 @@
|
||||
package blockdag
|
||||
|
||||
import (
|
||||
"github.com/kaspanet/kaspad/dbaccess"
|
||||
"github.com/kaspanet/kaspad/util/daghash"
|
||||
)
|
||||
|
||||
// IndexManager provides a generic interface that is called when blocks are
|
||||
// connected to the DAG for the purpose of supporting optional indexes.
|
||||
type IndexManager interface {
|
||||
// Init is invoked during DAG initialize in order to allow the index
|
||||
// manager to initialize itself and any indexes it is managing.
|
||||
Init(*BlockDAG, *dbaccess.DatabaseContext) error
|
||||
|
||||
// ConnectBlock is invoked when a new block has been connected to the
|
||||
// DAG.
|
||||
ConnectBlock(dbContext *dbaccess.TxContext, blockHash *daghash.Hash, acceptedTxsData MultiBlockTxsAcceptanceData) error
|
||||
}
|
@ -5,7 +5,9 @@
|
||||
package blockdag
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"math"
|
||||
"sort"
|
||||
|
||||
"github.com/kaspanet/kaspad/util"
|
||||
"github.com/kaspanet/kaspad/util/daghash"
|
||||
@ -132,3 +134,49 @@ func buildMerkleTreeStore(hashes []*daghash.Hash) MerkleTree {
|
||||
|
||||
return merkles
|
||||
}
|
||||
|
||||
func calculateAcceptedIDMerkleRoot(multiBlockTxsAcceptanceData MultiBlockTxsAcceptanceData) *daghash.Hash {
|
||||
var acceptedTxs []*util.Tx
|
||||
for _, blockTxsAcceptanceData := range multiBlockTxsAcceptanceData {
|
||||
for _, txAcceptance := range blockTxsAcceptanceData.TxAcceptanceData {
|
||||
if !txAcceptance.IsAccepted {
|
||||
continue
|
||||
}
|
||||
acceptedTxs = append(acceptedTxs, txAcceptance.Tx)
|
||||
}
|
||||
}
|
||||
sort.Slice(acceptedTxs, func(i, j int) bool {
|
||||
return daghash.LessTxID(acceptedTxs[i].ID(), acceptedTxs[j].ID())
|
||||
})
|
||||
|
||||
acceptedIDMerkleTree := BuildIDMerkleTreeStore(acceptedTxs)
|
||||
return acceptedIDMerkleTree.Root()
|
||||
}
|
||||
|
||||
func (node *blockNode) validateAcceptedIDMerkleRoot(dag *BlockDAG, txsAcceptanceData MultiBlockTxsAcceptanceData) error {
|
||||
if node.isGenesis() {
|
||||
return nil
|
||||
}
|
||||
|
||||
calculatedAccepetedIDMerkleRoot := calculateAcceptedIDMerkleRoot(txsAcceptanceData)
|
||||
header := node.Header()
|
||||
if !header.AcceptedIDMerkleRoot.IsEqual(calculatedAccepetedIDMerkleRoot) {
|
||||
str := fmt.Sprintf("block accepted ID merkle root is invalid - block "+
|
||||
"header indicates %s, but calculated value is %s",
|
||||
header.AcceptedIDMerkleRoot, calculatedAccepetedIDMerkleRoot)
|
||||
return ruleError(ErrBadMerkleRoot, str)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// NextAcceptedIDMerkleRootNoLock prepares the acceptedIDMerkleRoot for the next mined block
|
||||
//
|
||||
// This function MUST be called with the DAG read-lock held
|
||||
func (dag *BlockDAG) NextAcceptedIDMerkleRootNoLock() (*daghash.Hash, error) {
|
||||
txsAcceptanceData, err := dag.TxsAcceptedByVirtual()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return calculateAcceptedIDMerkleRoot(txsAcceptanceData), nil
|
||||
}
|
||||
|
@ -3,6 +3,7 @@ package blockdag
|
||||
import (
|
||||
"bytes"
|
||||
"encoding/binary"
|
||||
"math"
|
||||
"time"
|
||||
|
||||
"github.com/kaspanet/go-secp256k1"
|
||||
@ -13,6 +14,9 @@ import (
|
||||
"github.com/kaspanet/kaspad/util/mstime"
|
||||
)
|
||||
|
||||
// The current block version
|
||||
const blockVersion = 0x10000000
|
||||
|
||||
// BlockForMining returns a block with the given transactions
|
||||
// that points to the current DAG tips, that is valid from
|
||||
// all aspects except proof of work.
|
||||
@ -22,13 +26,6 @@ func (dag *BlockDAG) BlockForMining(transactions []*util.Tx) (*domainmessage.Msg
|
||||
blockTimestamp := dag.NextBlockTime()
|
||||
requiredDifficulty := dag.NextRequiredDifficulty(blockTimestamp)
|
||||
|
||||
// Calculate the next expected block version based on the state of the
|
||||
// rule change deployments.
|
||||
nextBlockVersion, err := dag.CalcNextBlockVersion()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// Create a new block ready to be solved.
|
||||
hashMerkleTree := BuildHashMerkleTreeStore(transactions)
|
||||
acceptedIDMerkleRoot, err := dag.NextAcceptedIDMerkleRootNoLock()
|
||||
@ -46,7 +43,7 @@ func (dag *BlockDAG) BlockForMining(transactions []*util.Tx) (*domainmessage.Msg
|
||||
}
|
||||
|
||||
msgBlock.Header = domainmessage.BlockHeader{
|
||||
Version: nextBlockVersion,
|
||||
Version: blockVersion,
|
||||
ParentHashes: dag.TipHashes(),
|
||||
HashMerkleRoot: hashMerkleTree.Root(),
|
||||
AcceptedIDMerkleRoot: acceptedIDMerkleRoot,
|
||||
@ -127,3 +124,15 @@ func (dag *BlockDAG) NextBlockTime() mstime.Time {
|
||||
|
||||
return newTimestamp
|
||||
}
|
||||
|
||||
// CurrentBits returns the bits of the tip with the lowest bits, which also means it has highest difficulty.
|
||||
func (dag *BlockDAG) CurrentBits() uint32 {
|
||||
tips := dag.virtual.tips()
|
||||
minBits := uint32(math.MaxUint32)
|
||||
for tip := range tips {
|
||||
if minBits > tip.Header().Bits {
|
||||
minBits = tip.Header().Bits
|
||||
}
|
||||
}
|
||||
return minBits
|
||||
}
|
||||
|
84
blockdag/multiset.go
Normal file
84
blockdag/multiset.go
Normal file
@ -0,0 +1,84 @@
|
||||
package blockdag
|
||||
|
||||
import (
|
||||
"github.com/kaspanet/go-secp256k1"
|
||||
"github.com/kaspanet/kaspad/domainmessage"
|
||||
"github.com/pkg/errors"
|
||||
)
|
||||
|
||||
// calcMultiset returns the multiset of the past UTXO of the given block.
|
||||
func (node *blockNode) calcMultiset(dag *BlockDAG, acceptanceData MultiBlockTxsAcceptanceData,
|
||||
selectedParentPastUTXO UTXOSet) (*secp256k1.MultiSet, error) {
|
||||
|
||||
return node.pastUTXOMultiSet(dag, acceptanceData, selectedParentPastUTXO)
|
||||
}
|
||||
|
||||
func (node *blockNode) pastUTXOMultiSet(dag *BlockDAG, acceptanceData MultiBlockTxsAcceptanceData,
|
||||
selectedParentPastUTXO UTXOSet) (*secp256k1.MultiSet, error) {
|
||||
|
||||
ms, err := node.selectedParentMultiset(dag)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
for _, blockAcceptanceData := range acceptanceData {
|
||||
for _, txAcceptanceData := range blockAcceptanceData.TxAcceptanceData {
|
||||
if !txAcceptanceData.IsAccepted {
|
||||
continue
|
||||
}
|
||||
|
||||
tx := txAcceptanceData.Tx.MsgTx()
|
||||
|
||||
var err error
|
||||
ms, err = addTxToMultiset(ms, tx, selectedParentPastUTXO, node.blueScore)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
}
|
||||
return ms, nil
|
||||
}
|
||||
|
||||
// selectedParentMultiset returns the multiset of the node's selected
|
||||
// parent. If the node is the genesis blockNode then it does not have
|
||||
// a selected parent, in which case return a new, empty multiset.
|
||||
func (node *blockNode) selectedParentMultiset(dag *BlockDAG) (*secp256k1.MultiSet, error) {
|
||||
if node.isGenesis() {
|
||||
return secp256k1.NewMultiset(), nil
|
||||
}
|
||||
|
||||
ms, err := dag.multisetStore.multisetByBlockNode(node.selectedParent)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return ms, nil
|
||||
}
|
||||
|
||||
func addTxToMultiset(ms *secp256k1.MultiSet, tx *domainmessage.MsgTx, pastUTXO UTXOSet, blockBlueScore uint64) (*secp256k1.MultiSet, error) {
|
||||
for _, txIn := range tx.TxIn {
|
||||
entry, ok := pastUTXO.Get(txIn.PreviousOutpoint)
|
||||
if !ok {
|
||||
return nil, errors.Errorf("Couldn't find entry for outpoint %s", txIn.PreviousOutpoint)
|
||||
}
|
||||
|
||||
var err error
|
||||
ms, err = removeUTXOFromMultiset(ms, entry, &txIn.PreviousOutpoint)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
|
||||
isCoinbase := tx.IsCoinBase()
|
||||
for i, txOut := range tx.TxOut {
|
||||
outpoint := *domainmessage.NewOutpoint(tx.TxID(), uint32(i))
|
||||
entry := NewUTXOEntry(txOut, isCoinbase, blockBlueScore)
|
||||
|
||||
var err error
|
||||
ms, err = addUTXOToMultiset(ms, entry, &outpoint)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
return ms, nil
|
||||
}
|
235
blockdag/orphan_blocks.go
Normal file
235
blockdag/orphan_blocks.go
Normal file
@ -0,0 +1,235 @@
|
||||
package blockdag
|
||||
|
||||
import (
|
||||
"github.com/kaspanet/kaspad/util"
|
||||
"github.com/kaspanet/kaspad/util/daghash"
|
||||
"github.com/kaspanet/kaspad/util/mstime"
|
||||
"github.com/pkg/errors"
|
||||
"time"
|
||||
)
|
||||
|
||||
// maxOrphanBlocks is the maximum number of orphan blocks that can be
|
||||
// queued.
|
||||
const maxOrphanBlocks = 100
|
||||
|
||||
// orphanBlock represents a block that we don't yet have the parent for. It
|
||||
// is a normal block plus an expiration time to prevent caching the orphan
|
||||
// forever.
|
||||
type orphanBlock struct {
|
||||
block *util.Block
|
||||
expiration mstime.Time
|
||||
}
|
||||
|
||||
// IsKnownOrphan returns whether the passed hash is currently a known orphan.
|
||||
// Keep in mind that only a limited number of orphans are held onto for a
|
||||
// limited amount of time, so this function must not be used as an absolute
|
||||
// way to test if a block is an orphan block. A full block (as opposed to just
|
||||
// its hash) must be passed to ProcessBlock for that purpose. However, calling
|
||||
// ProcessBlock with an orphan that already exists results in an error, so this
|
||||
// function provides a mechanism for a caller to intelligently detect *recent*
|
||||
// duplicate orphans and react accordingly.
|
||||
//
|
||||
// This function is safe for concurrent access.
|
||||
func (dag *BlockDAG) IsKnownOrphan(hash *daghash.Hash) bool {
|
||||
// Protect concurrent access. Using a read lock only so multiple
|
||||
// readers can query without blocking each other.
|
||||
dag.orphanLock.RLock()
|
||||
defer dag.orphanLock.RUnlock()
|
||||
_, exists := dag.orphans[*hash]
|
||||
|
||||
return exists
|
||||
}
|
||||
|
||||
// GetOrphanMissingAncestorHashes returns all of the missing parents in the orphan's sub-DAG
|
||||
//
|
||||
// This function is safe for concurrent access.
|
||||
func (dag *BlockDAG) GetOrphanMissingAncestorHashes(orphanHash *daghash.Hash) []*daghash.Hash {
|
||||
// Protect concurrent access. Using a read lock only so multiple
|
||||
// readers can query without blocking each other.
|
||||
dag.orphanLock.RLock()
|
||||
defer dag.orphanLock.RUnlock()
|
||||
|
||||
missingAncestorsHashes := make([]*daghash.Hash, 0)
|
||||
|
||||
visited := make(map[daghash.Hash]bool)
|
||||
queue := []*daghash.Hash{orphanHash}
|
||||
for len(queue) > 0 {
|
||||
var current *daghash.Hash
|
||||
current, queue = queue[0], queue[1:]
|
||||
if !visited[*current] {
|
||||
visited[*current] = true
|
||||
orphan, orphanExists := dag.orphans[*current]
|
||||
if orphanExists {
|
||||
queue = append(queue, orphan.block.MsgBlock().Header.ParentHashes...)
|
||||
} else {
|
||||
if !dag.IsInDAG(current) && current != orphanHash {
|
||||
missingAncestorsHashes = append(missingAncestorsHashes, current)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return missingAncestorsHashes
|
||||
}
|
||||
|
||||
// removeOrphanBlock removes the passed orphan block from the orphan pool and
|
||||
// previous orphan index.
|
||||
func (dag *BlockDAG) removeOrphanBlock(orphan *orphanBlock) {
|
||||
// Protect concurrent access.
|
||||
dag.orphanLock.Lock()
|
||||
defer dag.orphanLock.Unlock()
|
||||
|
||||
// Remove the orphan block from the orphan pool.
|
||||
orphanHash := orphan.block.Hash()
|
||||
delete(dag.orphans, *orphanHash)
|
||||
|
||||
// Remove the reference from the previous orphan index too.
|
||||
for _, parentHash := range orphan.block.MsgBlock().Header.ParentHashes {
|
||||
// An indexing for loop is intentionally used over a range here as range
|
||||
// does not reevaluate the slice on each iteration nor does it adjust the
|
||||
// index for the modified slice.
|
||||
orphans := dag.prevOrphans[*parentHash]
|
||||
for i := 0; i < len(orphans); i++ {
|
||||
hash := orphans[i].block.Hash()
|
||||
if hash.IsEqual(orphanHash) {
|
||||
orphans = append(orphans[:i], orphans[i+1:]...)
|
||||
i--
|
||||
}
|
||||
}
|
||||
|
||||
// Remove the map entry altogether if there are no longer any orphans
|
||||
// which depend on the parent hash.
|
||||
if len(orphans) == 0 {
|
||||
delete(dag.prevOrphans, *parentHash)
|
||||
continue
|
||||
}
|
||||
|
||||
dag.prevOrphans[*parentHash] = orphans
|
||||
}
|
||||
}
|
||||
|
||||
// addOrphanBlock adds the passed block (which is already determined to be
|
||||
// an orphan prior calling this function) to the orphan pool. It lazily cleans
|
||||
// up any expired blocks so a separate cleanup poller doesn't need to be run.
|
||||
// It also imposes a maximum limit on the number of outstanding orphan
|
||||
// blocks and will remove the oldest received orphan block if the limit is
|
||||
// exceeded.
|
||||
func (dag *BlockDAG) addOrphanBlock(block *util.Block) {
|
||||
log.Infof("Adding orphan block %s", block.Hash())
|
||||
|
||||
// Remove expired orphan blocks.
|
||||
for _, oBlock := range dag.orphans {
|
||||
if mstime.Now().After(oBlock.expiration) {
|
||||
dag.removeOrphanBlock(oBlock)
|
||||
continue
|
||||
}
|
||||
|
||||
// Update the newest orphan block pointer so it can be discarded
|
||||
// in case the orphan pool fills up.
|
||||
if dag.newestOrphan == nil || oBlock.block.Timestamp().After(dag.newestOrphan.block.Timestamp()) {
|
||||
dag.newestOrphan = oBlock
|
||||
}
|
||||
}
|
||||
|
||||
// Limit orphan blocks to prevent memory exhaustion.
|
||||
if len(dag.orphans)+1 > maxOrphanBlocks {
|
||||
// If the new orphan is newer than the newest orphan on the orphan
|
||||
// pool, don't add it.
|
||||
if block.Timestamp().After(dag.newestOrphan.block.Timestamp()) {
|
||||
return
|
||||
}
|
||||
// Remove the newest orphan to make room for the added one.
|
||||
dag.removeOrphanBlock(dag.newestOrphan)
|
||||
dag.newestOrphan = nil
|
||||
}
|
||||
|
||||
// Protect concurrent access. This is intentionally done here instead
|
||||
// of near the top since removeOrphanBlock does its own locking and
|
||||
// the range iterator is not invalidated by removing map entries.
|
||||
dag.orphanLock.Lock()
|
||||
defer dag.orphanLock.Unlock()
|
||||
|
||||
// Insert the block into the orphan map with an expiration time
|
||||
// 1 hour from now.
|
||||
expiration := mstime.Now().Add(time.Hour)
|
||||
oBlock := &orphanBlock{
|
||||
block: block,
|
||||
expiration: expiration,
|
||||
}
|
||||
dag.orphans[*block.Hash()] = oBlock
|
||||
|
||||
// Add to parent hash lookup index for faster dependency lookups.
|
||||
for _, parentHash := range block.MsgBlock().Header.ParentHashes {
|
||||
dag.prevOrphans[*parentHash] = append(dag.prevOrphans[*parentHash], oBlock)
|
||||
}
|
||||
}
|
||||
|
||||
// processOrphans determines if there are any orphans which depend on the passed
|
||||
// block hash (they are no longer orphans if true) and potentially accepts them.
|
||||
// It repeats the process for the newly accepted blocks (to detect further
|
||||
// orphans which may no longer be orphans) until there are no more.
|
||||
//
|
||||
// The flags do not modify the behavior of this function directly, however they
|
||||
// are needed to pass along to maybeAcceptBlock.
|
||||
//
|
||||
// This function MUST be called with the DAG state lock held (for writes).
|
||||
func (dag *BlockDAG) processOrphans(hash *daghash.Hash, flags BehaviorFlags) error {
|
||||
// Start with processing at least the passed hash. Leave a little room
|
||||
// for additional orphan blocks that need to be processed without
|
||||
// needing to grow the array in the common case.
|
||||
processHashes := make([]*daghash.Hash, 0, 10)
|
||||
processHashes = append(processHashes, hash)
|
||||
for len(processHashes) > 0 {
|
||||
// Pop the first hash to process from the slice.
|
||||
processHash := processHashes[0]
|
||||
processHashes[0] = nil // Prevent GC leak.
|
||||
processHashes = processHashes[1:]
|
||||
|
||||
// Look up all orphans that are parented by the block we just
|
||||
// accepted. An indexing for loop is
|
||||
// intentionally used over a range here as range does not
|
||||
// reevaluate the slice on each iteration nor does it adjust the
|
||||
// index for the modified slice.
|
||||
for i := 0; i < len(dag.prevOrphans[*processHash]); i++ {
|
||||
orphan := dag.prevOrphans[*processHash][i]
|
||||
if orphan == nil {
|
||||
log.Warnf("Found a nil entry at index %d in the "+
|
||||
"orphan dependency list for block %s", i,
|
||||
processHash)
|
||||
continue
|
||||
}
|
||||
|
||||
// Skip this orphan if one or more of its parents are
|
||||
// still missing.
|
||||
_, err := lookupParentNodes(orphan.block, dag)
|
||||
if err != nil {
|
||||
var ruleErr RuleError
|
||||
if ok := errors.As(err, &ruleErr); ok && ruleErr.ErrorCode == ErrParentBlockUnknown {
|
||||
continue
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
// Remove the orphan from the orphan pool.
|
||||
orphanHash := orphan.block.Hash()
|
||||
dag.removeOrphanBlock(orphan)
|
||||
i--
|
||||
|
||||
// Potentially accept the block into the block DAG.
|
||||
err = dag.maybeAcceptBlock(orphan.block, flags|BFWasUnorphaned)
|
||||
if err != nil {
|
||||
// Since we don't want to reject the original block because of
|
||||
// a bad unorphaned child, only return an error if it's not a RuleError.
|
||||
if !errors.As(err, &RuleError{}) {
|
||||
return err
|
||||
}
|
||||
log.Warnf("Verification failed for orphan block %s: %s", orphanHash, err)
|
||||
}
|
||||
|
||||
// Add this block to the list of blocks to process so
|
||||
// any orphan blocks that depend on this block are
|
||||
// handled too.
|
||||
processHashes = append(processHashes, orphanHash)
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
@ -1,140 +1,20 @@
|
||||
// Copyright (c) 2013-2017 The btcsuite developers
|
||||
// Use of this source code is governed by an ISC
|
||||
// license that can be found in the LICENSE file.
|
||||
|
||||
package blockdag
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"time"
|
||||
|
||||
"github.com/kaspanet/kaspad/dagconfig"
|
||||
"github.com/pkg/errors"
|
||||
|
||||
"github.com/kaspanet/go-secp256k1"
|
||||
"github.com/kaspanet/kaspad/dbaccess"
|
||||
"github.com/kaspanet/kaspad/util"
|
||||
"github.com/kaspanet/kaspad/util/daghash"
|
||||
"github.com/pkg/errors"
|
||||
"time"
|
||||
)
|
||||
|
||||
// BehaviorFlags is a bitmask defining tweaks to the normal behavior when
|
||||
// performing DAG processing and consensus rules checks.
|
||||
type BehaviorFlags uint32
|
||||
|
||||
const (
|
||||
// BFFastAdd may be set to indicate that several checks can be avoided
|
||||
// for the block since it is already known to fit into the DAG due to
|
||||
// already proving it correct links into the DAG.
|
||||
BFFastAdd BehaviorFlags = 1 << iota
|
||||
|
||||
// BFNoPoWCheck may be set to indicate the proof of work check which
|
||||
// ensures a block hashes to a value less than the required target will
|
||||
// not be performed.
|
||||
BFNoPoWCheck
|
||||
|
||||
// BFWasUnorphaned may be set to indicate that a block was just now
|
||||
// unorphaned
|
||||
BFWasUnorphaned
|
||||
|
||||
// BFAfterDelay may be set to indicate that a block had timestamp too far
|
||||
// in the future, just finished the delay
|
||||
BFAfterDelay
|
||||
|
||||
// BFIsSync may be set to indicate that the block was sent as part of the
|
||||
// netsync process
|
||||
BFIsSync
|
||||
|
||||
// BFWasStored is set to indicate that the block was previously stored
|
||||
// in the block index but was never fully processed
|
||||
BFWasStored
|
||||
|
||||
// BFDisallowDelay is set to indicate that a delayed block should be rejected.
|
||||
// This is used for the case where a block is submitted through RPC.
|
||||
BFDisallowDelay
|
||||
|
||||
// BFDisallowOrphans is set to indicate that an orphan block should be rejected.
|
||||
// This is used for the case where a block is submitted through RPC.
|
||||
BFDisallowOrphans
|
||||
|
||||
// BFNone is a convenience value to specifically indicate no flags.
|
||||
BFNone BehaviorFlags = 0
|
||||
)
|
||||
|
||||
// IsInDAG determines whether a block with the given hash exists in
|
||||
// the DAG.
|
||||
//
|
||||
// This function is safe for concurrent access.
|
||||
func (dag *BlockDAG) IsInDAG(hash *daghash.Hash) bool {
|
||||
return dag.index.HaveBlock(hash)
|
||||
}
|
||||
|
||||
// processOrphans determines if there are any orphans which depend on the passed
|
||||
// block hash (they are no longer orphans if true) and potentially accepts them.
|
||||
// It repeats the process for the newly accepted blocks (to detect further
|
||||
// orphans which may no longer be orphans) until there are no more.
|
||||
//
|
||||
// The flags do not modify the behavior of this function directly, however they
|
||||
// are needed to pass along to maybeAcceptBlock.
|
||||
//
|
||||
// This function MUST be called with the DAG state lock held (for writes).
|
||||
func (dag *BlockDAG) processOrphans(hash *daghash.Hash, flags BehaviorFlags) error {
|
||||
// Start with processing at least the passed hash. Leave a little room
|
||||
// for additional orphan blocks that need to be processed without
|
||||
// needing to grow the array in the common case.
|
||||
processHashes := make([]*daghash.Hash, 0, 10)
|
||||
processHashes = append(processHashes, hash)
|
||||
for len(processHashes) > 0 {
|
||||
// Pop the first hash to process from the slice.
|
||||
processHash := processHashes[0]
|
||||
processHashes[0] = nil // Prevent GC leak.
|
||||
processHashes = processHashes[1:]
|
||||
|
||||
// Look up all orphans that are parented by the block we just
|
||||
// accepted. An indexing for loop is
|
||||
// intentionally used over a range here as range does not
|
||||
// reevaluate the slice on each iteration nor does it adjust the
|
||||
// index for the modified slice.
|
||||
for i := 0; i < len(dag.prevOrphans[*processHash]); i++ {
|
||||
orphan := dag.prevOrphans[*processHash][i]
|
||||
if orphan == nil {
|
||||
log.Warnf("Found a nil entry at index %d in the "+
|
||||
"orphan dependency list for block %s", i,
|
||||
processHash)
|
||||
continue
|
||||
}
|
||||
|
||||
// Skip this orphan if one or more of its parents are
|
||||
// still missing.
|
||||
_, err := lookupParentNodes(orphan.block, dag)
|
||||
if err != nil {
|
||||
var ruleErr RuleError
|
||||
if ok := errors.As(err, &ruleErr); ok && ruleErr.ErrorCode == ErrParentBlockUnknown {
|
||||
continue
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
// Remove the orphan from the orphan pool.
|
||||
orphanHash := orphan.block.Hash()
|
||||
dag.removeOrphanBlock(orphan)
|
||||
i--
|
||||
|
||||
// Potentially accept the block into the block DAG.
|
||||
err = dag.maybeAcceptBlock(orphan.block, flags|BFWasUnorphaned)
|
||||
if err != nil {
|
||||
// Since we don't want to reject the original block because of
|
||||
// a bad unorphaned child, only return an error if it's not a RuleError.
|
||||
if !errors.As(err, &RuleError{}) {
|
||||
return err
|
||||
}
|
||||
log.Warnf("Verification failed for orphan block %s: %s", orphanHash, err)
|
||||
}
|
||||
|
||||
// Add this block to the list of blocks to process so
|
||||
// any orphan blocks that depend on this block are
|
||||
// handled too.
|
||||
processHashes = append(processHashes, orphanHash)
|
||||
}
|
||||
}
|
||||
return nil
|
||||
// chainUpdates represents the updates made to the selected parent chain after
|
||||
// a block had been added to the DAG.
|
||||
type chainUpdates struct {
|
||||
removedChainBlockHashes []*daghash.Hash
|
||||
addedChainBlockHashes []*daghash.Hash
|
||||
}
|
||||
|
||||
// ProcessBlock is the main workhorse for handling insertion of new blocks into
|
||||
@ -142,9 +22,6 @@ func (dag *BlockDAG) processOrphans(hash *daghash.Hash, flags BehaviorFlags) err
|
||||
// blocks, ensuring blocks follow all rules, orphan handling, and insertion into
|
||||
// the block DAG.
|
||||
//
|
||||
// When no errors occurred during processing, the first return value indicates
|
||||
// whether or not the block is an orphan.
|
||||
//
|
||||
// This function is safe for concurrent access.
|
||||
func (dag *BlockDAG) ProcessBlock(block *util.Block, flags BehaviorFlags) (isOrphan bool, isDelayed bool, err error) {
|
||||
dag.dagLock.Lock()
|
||||
@ -153,60 +30,82 @@ func (dag *BlockDAG) ProcessBlock(block *util.Block, flags BehaviorFlags) (isOrp
|
||||
}
|
||||
|
||||
func (dag *BlockDAG) processBlockNoLock(block *util.Block, flags BehaviorFlags) (isOrphan bool, isDelayed bool, err error) {
|
||||
isAfterDelay := flags&BFAfterDelay == BFAfterDelay
|
||||
wasBlockStored := flags&BFWasStored == BFWasStored
|
||||
disallowDelay := flags&BFDisallowDelay == BFDisallowDelay
|
||||
disallowOrphans := flags&BFDisallowOrphans == BFDisallowOrphans
|
||||
|
||||
blockHash := block.Hash()
|
||||
log.Tracef("Processing block %s", blockHash)
|
||||
|
||||
// The block must not already exist in the DAG.
|
||||
if dag.IsInDAG(blockHash) && !wasBlockStored {
|
||||
str := fmt.Sprintf("already have block %s", blockHash)
|
||||
return false, false, ruleError(ErrDuplicateBlock, str)
|
||||
}
|
||||
|
||||
// The block must not already exist as an orphan.
|
||||
if _, exists := dag.orphans[*blockHash]; exists {
|
||||
str := fmt.Sprintf("already have block (orphan) %s", blockHash)
|
||||
return false, false, ruleError(ErrDuplicateBlock, str)
|
||||
}
|
||||
|
||||
if dag.isKnownDelayedBlock(blockHash) {
|
||||
str := fmt.Sprintf("already have block (delayed) %s", blockHash)
|
||||
return false, false, ruleError(ErrDuplicateBlock, str)
|
||||
}
|
||||
|
||||
if !isAfterDelay {
|
||||
// Perform preliminary sanity checks on the block and its transactions.
|
||||
delay, err := dag.checkBlockSanity(block, flags)
|
||||
err = dag.checkDuplicateBlock(blockHash, flags)
|
||||
if err != nil {
|
||||
return false, false, err
|
||||
}
|
||||
|
||||
if delay != 0 && disallowDelay {
|
||||
str := fmt.Sprintf("Cannot process blocks beyond the allowed time offset while the BFDisallowDelay flag is raised %s", blockHash)
|
||||
return false, true, ruleError(ErrDelayedBlockIsNotAllowed, str)
|
||||
}
|
||||
|
||||
if delay != 0 {
|
||||
err = dag.addDelayedBlock(block, delay)
|
||||
err = dag.checkBlockSanity(block, flags)
|
||||
if err != nil {
|
||||
return false, false, err
|
||||
}
|
||||
|
||||
isOrphan, isDelayed, err = dag.checkDelayedAndOrphanBlocks(block, flags)
|
||||
if isOrphan || isDelayed || err != nil {
|
||||
return isOrphan, isDelayed, err
|
||||
}
|
||||
|
||||
err = dag.maybeAcceptBlock(block, flags)
|
||||
if err != nil {
|
||||
return false, false, err
|
||||
}
|
||||
|
||||
err = dag.processOrphansAndDelayedBlocks(blockHash, flags)
|
||||
if err != nil {
|
||||
return false, false, err
|
||||
}
|
||||
|
||||
log.Debugf("Accepted block %s", blockHash)
|
||||
|
||||
return false, false, nil
|
||||
}
|
||||
|
||||
func (dag *BlockDAG) checkDelayedAndOrphanBlocks(block *util.Block, flags BehaviorFlags) (isOrphan bool, isDelayed bool, err error) {
|
||||
if !isBehaviorFlagRaised(flags, BFAfterDelay) {
|
||||
isDelayed, err := dag.checkBlockDelay(block, flags)
|
||||
if err != nil {
|
||||
return false, false, err
|
||||
}
|
||||
if isDelayed {
|
||||
return false, true, nil
|
||||
}
|
||||
}
|
||||
return dag.checkMissingParents(block, flags)
|
||||
}
|
||||
|
||||
func (dag *BlockDAG) checkBlockDelay(block *util.Block, flags BehaviorFlags) (isDelayed bool, err error) {
|
||||
delay, isDelayed := dag.shouldBlockBeDelayed(block)
|
||||
if isDelayed && isBehaviorFlagRaised(flags, BFDisallowDelay) {
|
||||
str := fmt.Sprintf("cannot process blocks beyond the "+
|
||||
"allowed time offset while the BFDisallowDelay flag is "+
|
||||
"raised %s", block.Hash())
|
||||
return false, ruleError(ErrDelayedBlockIsNotAllowed, str)
|
||||
}
|
||||
|
||||
if isDelayed {
|
||||
err := dag.addDelayedBlock(block, delay)
|
||||
if err != nil {
|
||||
return false, err
|
||||
}
|
||||
return true, nil
|
||||
}
|
||||
return false, nil
|
||||
}
|
||||
|
||||
func (dag *BlockDAG) checkMissingParents(block *util.Block, flags BehaviorFlags) (isOrphan bool, isDelayed bool, err error) {
|
||||
var missingParents []*daghash.Hash
|
||||
for _, parentHash := range block.MsgBlock().Header.ParentHashes {
|
||||
if !dag.IsInDAG(parentHash) {
|
||||
missingParents = append(missingParents, parentHash)
|
||||
}
|
||||
}
|
||||
if len(missingParents) > 0 && disallowOrphans {
|
||||
str := fmt.Sprintf("Cannot process orphan blocks while the BFDisallowOrphans flag is raised %s", blockHash)
|
||||
|
||||
if len(missingParents) > 0 && isBehaviorFlagRaised(flags, BFDisallowOrphans) {
|
||||
str := fmt.Sprintf("cannot process orphan blocks while the "+
|
||||
"BFDisallowOrphans flag is raised %s", block.Hash())
|
||||
return false, false, ruleError(ErrOrphanBlockIsNotAllowed, str)
|
||||
}
|
||||
|
||||
@ -219,69 +118,341 @@ func (dag *BlockDAG) processBlockNoLock(block *util.Block, flags BehaviorFlags)
|
||||
if err != nil {
|
||||
return false, false, err
|
||||
}
|
||||
return false, true, err
|
||||
return false, true, nil
|
||||
}
|
||||
|
||||
// Handle orphan blocks.
|
||||
if len(missingParents) > 0 {
|
||||
// Some orphans during netsync are a normal part of the process, since the anticone
|
||||
// of the chain-split is never explicitly requested.
|
||||
// Therefore, if we are during netsync - don't report orphans to default logs.
|
||||
//
|
||||
// The number K*2 was chosen since in peace times anticone is limited to K blocks,
|
||||
// while some red block can make it a bit bigger, but much more than that indicates
|
||||
// there might be some problem with the netsync process.
|
||||
if flags&BFIsSync == BFIsSync && dagconfig.KType(len(dag.orphans)) < dag.Params.K*2 {
|
||||
log.Debugf("Adding orphan block %s. This is normal part of netsync process", blockHash)
|
||||
} else {
|
||||
log.Infof("Adding orphan block %s", blockHash)
|
||||
}
|
||||
dag.addOrphanBlock(block)
|
||||
|
||||
return true, false, nil
|
||||
}
|
||||
|
||||
// The block has passed all context independent checks and appears sane
|
||||
// enough to potentially accept it into the block DAG.
|
||||
err = dag.maybeAcceptBlock(block, flags)
|
||||
if err != nil {
|
||||
return false, false, err
|
||||
}
|
||||
|
||||
// Accept any orphan blocks that depend on this block (they are
|
||||
// no longer orphans) and repeat for those accepted blocks until
|
||||
// there are no more.
|
||||
err = dag.processOrphans(blockHash, flags)
|
||||
if err != nil {
|
||||
return false, false, err
|
||||
}
|
||||
|
||||
if !isAfterDelay {
|
||||
err = dag.processDelayedBlocks()
|
||||
if err != nil {
|
||||
return false, false, err
|
||||
}
|
||||
}
|
||||
|
||||
dag.addBlockProcessingTimestamp()
|
||||
|
||||
log.Debugf("Accepted block %s", blockHash)
|
||||
|
||||
return false, false, nil
|
||||
}
|
||||
|
||||
// maxDelayOfParents returns the maximum delay of the given block hashes.
|
||||
// Note that delay could be 0, but isDelayed will return true. This is the case where the parent process time is due.
|
||||
func (dag *BlockDAG) maxDelayOfParents(parentHashes []*daghash.Hash) (delay time.Duration, isDelayed bool) {
|
||||
for _, parentHash := range parentHashes {
|
||||
if delayedParent, exists := dag.delayedBlocks[*parentHash]; exists {
|
||||
isDelayed = true
|
||||
parentDelay := delayedParent.processTime.Sub(dag.Now())
|
||||
if parentDelay > delay {
|
||||
delay = parentDelay
|
||||
func (dag *BlockDAG) processOrphansAndDelayedBlocks(blockHash *daghash.Hash, flags BehaviorFlags) error {
|
||||
err := dag.processOrphans(blockHash, flags)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if !isBehaviorFlagRaised(flags, BFAfterDelay) {
|
||||
err = dag.processDelayedBlocks()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// maybeAcceptBlock potentially accepts a block into the block DAG. It
|
||||
// performs several validation checks which depend on its position within
|
||||
// the block DAG before adding it. The block is expected to have already
|
||||
// gone through ProcessBlock before calling this function with it.
|
||||
//
|
||||
// The flags are also passed to checkBlockContext and connectBlock. See
|
||||
// their documentation for how the flags modify their behavior.
|
||||
//
|
||||
// This function MUST be called with the dagLock held (for writes).
|
||||
func (dag *BlockDAG) maybeAcceptBlock(block *util.Block, flags BehaviorFlags) error {
|
||||
err := dag.checkBlockContext(block, flags)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
newNode, selectedParentAnticone, err := dag.createBlockNodeFromBlock(block)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
chainUpdates, err := dag.connectBlock(newNode, block, selectedParentAnticone, flags)
|
||||
if err != nil {
|
||||
return dag.handleConnectBlockError(err, newNode)
|
||||
}
|
||||
|
||||
dag.notifyBlockAccepted(block, chainUpdates, flags)
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// createBlockNodeFromBlock generates a new block node for the given block
|
||||
// and stores it in the block index with statusDataStored.
|
||||
func (dag *BlockDAG) createBlockNodeFromBlock(block *util.Block) (
|
||||
newNode *blockNode, selectedParentAnticone []*blockNode, err error) {
|
||||
|
||||
// Create a new block node for the block and add it to the node index.
|
||||
parents, err := lookupParentNodes(block, dag)
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
newNode, selectedParentAnticone = dag.newBlockNode(&block.MsgBlock().Header, parents)
|
||||
newNode.status = statusDataStored
|
||||
dag.index.AddNode(newNode)
|
||||
|
||||
// Insert the block into the database if it's not already there. Even
|
||||
// though it is possible the block will ultimately fail to connect, it
|
||||
// has already passed all proof-of-work and validity tests which means
|
||||
// it would be prohibitively expensive for an attacker to fill up the
|
||||
// disk with a bunch of blocks that fail to connect. This is necessary
|
||||
// since it allows block download to be decoupled from the much more
|
||||
// expensive connection logic. It also has some other nice properties
|
||||
// such as making blocks that never become part of the DAG or
|
||||
// blocks that fail to connect available for further analysis.
|
||||
dbTx, err := dag.databaseContext.NewTx()
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
defer dbTx.RollbackUnlessClosed()
|
||||
blockExists, err := dbaccess.HasBlock(dbTx, block.Hash())
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
if !blockExists {
|
||||
err := storeBlock(dbTx, block)
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
}
|
||||
err = dag.index.flushToDB(dbTx)
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
err = dbTx.Commit()
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
return newNode, selectedParentAnticone, nil
|
||||
}
|
||||
|
||||
// connectBlock handles connecting the passed node/block to the DAG.
|
||||
//
|
||||
// This function MUST be called with the DAG state lock held (for writes).
|
||||
func (dag *BlockDAG) connectBlock(node *blockNode,
|
||||
block *util.Block, selectedParentAnticone []*blockNode, flags BehaviorFlags) (*chainUpdates, error) {
|
||||
|
||||
err := dag.checkBlockTransactionsFinalized(block, node, flags)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if err = dag.checkFinalityViolation(node); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if err := dag.validateGasLimit(block); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
newBlockPastUTXO, txsAcceptanceData, newBlockFeeData, newBlockMultiSet, err :=
|
||||
node.verifyAndBuildUTXO(dag, block.Transactions(), isBehaviorFlagRaised(flags, BFFastAdd))
|
||||
if err != nil {
|
||||
return nil, errors.Wrapf(err, "error verifying UTXO for %s", node)
|
||||
}
|
||||
|
||||
err = node.validateCoinbaseTransaction(dag, block, txsAcceptanceData)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
virtualUTXODiff, chainUpdates, err :=
|
||||
dag.applyDAGChanges(node, newBlockPastUTXO, newBlockMultiSet, selectedParentAnticone)
|
||||
if err != nil {
|
||||
// Since all validation logic has already ran, if applyDAGChanges errors out,
|
||||
// this means we have a problem in the internal structure of the DAG - a problem which is
|
||||
// irrecoverable, and it would be a bad idea to attempt adding any more blocks to the DAG.
|
||||
// Therefore - in such cases we panic.
|
||||
panic(err)
|
||||
}
|
||||
|
||||
err = dag.saveChangesFromBlock(block, virtualUTXODiff, txsAcceptanceData, newBlockFeeData)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
dag.addBlockProcessingTimestamp()
|
||||
dag.blockCount++
|
||||
|
||||
return chainUpdates, nil
|
||||
}
|
||||
|
||||
// applyDAGChanges does the following:
|
||||
// 1. Connects each of the new block's parents to the block.
|
||||
// 2. Adds the new block to the DAG's tips.
|
||||
// 3. Updates the DAG's full UTXO set.
|
||||
// 4. Updates each of the tips' utxoDiff.
|
||||
// 5. Applies the new virtual's blue score to all the unaccepted UTXOs
|
||||
// 6. Adds the block to the reachability structures
|
||||
// 7. Adds the multiset of the block to the multiset store.
|
||||
// 8. Updates the finality point of the DAG (if required).
|
||||
//
|
||||
// It returns the diff in the virtual block's UTXO set.
|
||||
//
|
||||
// This function MUST be called with the DAG state lock held (for writes).
|
||||
func (dag *BlockDAG) applyDAGChanges(node *blockNode, newBlockPastUTXO UTXOSet,
|
||||
newBlockMultiset *secp256k1.MultiSet, selectedParentAnticone []*blockNode) (
|
||||
virtualUTXODiff *UTXODiff, chainUpdates *chainUpdates, err error) {
|
||||
|
||||
// Add the block to the reachability tree
|
||||
err = dag.reachabilityTree.addBlock(node, selectedParentAnticone)
|
||||
if err != nil {
|
||||
return nil, nil, errors.Wrap(err, "failed adding block to the reachability tree")
|
||||
}
|
||||
|
||||
dag.multisetStore.setMultiset(node, newBlockMultiset)
|
||||
|
||||
if err = node.updateParents(dag, newBlockPastUTXO); err != nil {
|
||||
return nil, nil, errors.Wrapf(err, "failed updating parents of %s", node)
|
||||
}
|
||||
|
||||
// Update the virtual block's parents (the DAG tips) to include the new block.
|
||||
chainUpdates = dag.virtual.AddTip(node)
|
||||
|
||||
// Build a UTXO set for the new virtual block
|
||||
newVirtualUTXO, _, _, err := dag.pastUTXO(&dag.virtual.blockNode)
|
||||
if err != nil {
|
||||
return nil, nil, errors.Wrap(err, "could not restore past UTXO for virtual")
|
||||
}
|
||||
|
||||
// Apply new utxoDiffs to all the tips
|
||||
err = updateTipsUTXO(dag, newVirtualUTXO)
|
||||
if err != nil {
|
||||
return nil, nil, errors.Wrap(err, "failed updating the tips' UTXO")
|
||||
}
|
||||
|
||||
// It is now safe to meld the UTXO set to base.
|
||||
diffSet := newVirtualUTXO.(*DiffUTXOSet)
|
||||
virtualUTXODiff = diffSet.UTXODiff
|
||||
err = dag.meldVirtualUTXO(diffSet)
|
||||
if err != nil {
|
||||
return nil, nil, errors.Wrap(err, "failed melding the virtual UTXO")
|
||||
}
|
||||
|
||||
dag.index.SetStatusFlags(node, statusValid)
|
||||
|
||||
// And now we can update the finality point of the DAG (if required)
|
||||
dag.updateFinalityPoint()
|
||||
|
||||
return virtualUTXODiff, chainUpdates, nil
|
||||
}
|
||||
|
||||
func (dag *BlockDAG) saveChangesFromBlock(block *util.Block, virtualUTXODiff *UTXODiff,
|
||||
txsAcceptanceData MultiBlockTxsAcceptanceData, feeData compactFeeData) error {
|
||||
|
||||
dbTx, err := dag.databaseContext.NewTx()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
defer dbTx.RollbackUnlessClosed()
|
||||
|
||||
err = dag.index.flushToDB(dbTx)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
err = dag.utxoDiffStore.flushToDB(dbTx)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
err = dag.reachabilityTree.storeState(dbTx)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
err = dag.multisetStore.flushToDB(dbTx)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Update DAG state.
|
||||
state := &dagState{
|
||||
TipHashes: dag.TipHashes(),
|
||||
LastFinalityPoint: dag.lastFinalityPoint.hash,
|
||||
LocalSubnetworkID: dag.subnetworkID,
|
||||
}
|
||||
err = saveDAGState(dbTx, state)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Update the UTXO set using the diffSet that was melded into the
|
||||
// full UTXO set.
|
||||
err = updateUTXOSet(dbTx, virtualUTXODiff)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Scan all accepted transactions and register any subnetwork registry
|
||||
// transaction. If any subnetwork registry transaction is not well-formed,
|
||||
// fail the entire block.
|
||||
err = registerSubnetworks(dbTx, block.Transactions())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Allow the index manager to call each of the currently active
|
||||
// optional indexes with the block being connected so they can
|
||||
// update themselves accordingly.
|
||||
if dag.indexManager != nil {
|
||||
err := dag.indexManager.ConnectBlock(dbTx, block.Hash(), txsAcceptanceData)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
return delay, isDelayed
|
||||
// Apply the fee data into the database
|
||||
err = dbaccess.StoreFeeData(dbTx, block.Hash(), feeData)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
err = dbTx.Commit()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
dag.index.clearDirtyEntries()
|
||||
dag.utxoDiffStore.clearDirtyEntries()
|
||||
dag.utxoDiffStore.clearOldEntries()
|
||||
dag.reachabilityTree.store.clearDirtyEntries()
|
||||
dag.multisetStore.clearNewEntries()
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (dag *BlockDAG) handleConnectBlockError(err error, newNode *blockNode) error {
|
||||
if errors.As(err, &RuleError{}) {
|
||||
dag.index.SetStatusFlags(newNode, statusValidateFailed)
|
||||
|
||||
dbTx, err := dag.databaseContext.NewTx()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
defer dbTx.RollbackUnlessClosed()
|
||||
|
||||
err = dag.index.flushToDB(dbTx)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
err = dbTx.Commit()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
// notifyBlockAccepted notifies the caller that the new block was
|
||||
// accepted into the block DAG. The caller would typically want to
|
||||
// react by relaying the inventory to other peers.
|
||||
func (dag *BlockDAG) notifyBlockAccepted(block *util.Block, chainUpdates *chainUpdates, flags BehaviorFlags) {
|
||||
dag.sendNotification(NTBlockAdded, &BlockAddedNotificationData{
|
||||
Block: block,
|
||||
WasUnorphaned: flags&BFWasUnorphaned != 0,
|
||||
})
|
||||
if len(chainUpdates.addedChainBlockHashes) > 0 {
|
||||
dag.sendNotification(NTChainChanged, &ChainChangedNotificationData{
|
||||
RemovedChainBlockHashes: chainUpdates.removedChainBlockHashes,
|
||||
AddedChainBlockHashes: chainUpdates.addedChainBlockHashes,
|
||||
})
|
||||
}
|
||||
}
|
||||
|
@ -1,6 +1,7 @@
|
||||
package blockdag
|
||||
|
||||
import (
|
||||
"github.com/pkg/errors"
|
||||
"path/filepath"
|
||||
"testing"
|
||||
"time"
|
||||
@ -104,6 +105,9 @@ func TestProcessDelayedBlocks(t *testing.T) {
|
||||
blockDelay := time.Duration(dag1.Params.TimestampDeviationTolerance)*dag1.Params.TargetTimePerBlock + 5*time.Second
|
||||
delayedBlock.Header.Timestamp = initialTime.Add(blockDelay)
|
||||
|
||||
// We change the nonce here because processDelayedBlocks always runs without BFNoPoWCheck.
|
||||
delayedBlock.Header.Nonce = 2
|
||||
|
||||
isOrphan, isDelayed, err := dag1.ProcessBlock(util.NewBlock(delayedBlock), BFNoPoWCheck)
|
||||
if err != nil {
|
||||
t.Fatalf("ProcessBlock returned unexpected error: %s\n", err)
|
||||
@ -236,3 +240,101 @@ func TestProcessDelayedBlocks(t *testing.T) {
|
||||
t.Errorf("delayedBlockChild shouldn't be added to the DAG because its parent has been added to the DAG")
|
||||
}
|
||||
}
|
||||
|
||||
func TestMaybeAcceptBlockErrors(t *testing.T) {
|
||||
// Create a new database and DAG instance to run tests against.
|
||||
dag, teardownFunc, err := DAGSetup("TestMaybeAcceptBlockErrors", true, Config{
|
||||
DAGParams: &dagconfig.SimnetParams,
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatalf("TestMaybeAcceptBlockErrors: Failed to setup DAG instance: %v", err)
|
||||
}
|
||||
defer teardownFunc()
|
||||
|
||||
dag.TestSetCoinbaseMaturity(0)
|
||||
|
||||
// Test rejecting the block if its parents are missing
|
||||
orphanBlockFile := "blk_3B.dat"
|
||||
loadedBlocks, err := LoadBlocks(filepath.Join("testdata/", orphanBlockFile))
|
||||
if err != nil {
|
||||
t.Fatalf("TestMaybeAcceptBlockErrors: "+
|
||||
"Error loading file '%s': %s\n", orphanBlockFile, err)
|
||||
}
|
||||
block := loadedBlocks[0]
|
||||
|
||||
err = dag.maybeAcceptBlock(block, BFNone)
|
||||
if err == nil {
|
||||
t.Errorf("TestMaybeAcceptBlockErrors: rejecting the block if its parents are missing: "+
|
||||
"Expected: %s, got: <nil>", ErrParentBlockUnknown)
|
||||
}
|
||||
var ruleErr RuleError
|
||||
if ok := errors.As(err, &ruleErr); !ok {
|
||||
t.Errorf("TestMaybeAcceptBlockErrors: rejecting the block if its parents are missing: "+
|
||||
"Expected RuleError but got %s", err)
|
||||
} else if ruleErr.ErrorCode != ErrParentBlockUnknown {
|
||||
t.Errorf("TestMaybeAcceptBlockErrors: rejecting the block if its parents are missing: "+
|
||||
"Unexpected error code. Want: %s, got: %s", ErrParentBlockUnknown, ruleErr.ErrorCode)
|
||||
}
|
||||
|
||||
// Test rejecting the block if its parents are invalid
|
||||
blocksFile := "blk_0_to_4.dat"
|
||||
blocks, err := LoadBlocks(filepath.Join("testdata/", blocksFile))
|
||||
if err != nil {
|
||||
t.Fatalf("TestMaybeAcceptBlockErrors: "+
|
||||
"Error loading file '%s': %s\n", blocksFile, err)
|
||||
}
|
||||
|
||||
// Add a valid block and mark it as invalid
|
||||
block1 := blocks[1]
|
||||
isOrphan, isDelayed, err := dag.ProcessBlock(block1, BFNone)
|
||||
if err != nil {
|
||||
t.Fatalf("TestMaybeAcceptBlockErrors: Valid block unexpectedly returned an error: %s", err)
|
||||
}
|
||||
if isDelayed {
|
||||
t.Fatalf("TestMaybeAcceptBlockErrors: block 1 is too far in the future")
|
||||
}
|
||||
if isOrphan {
|
||||
t.Fatalf("TestMaybeAcceptBlockErrors: incorrectly returned block 1 is an orphan")
|
||||
}
|
||||
blockNode1, ok := dag.index.LookupNode(block1.Hash())
|
||||
if !ok {
|
||||
t.Fatalf("block %s does not exist in the DAG", block1.Hash())
|
||||
}
|
||||
dag.index.SetStatusFlags(blockNode1, statusValidateFailed)
|
||||
|
||||
block2 := blocks[2]
|
||||
err = dag.maybeAcceptBlock(block2, BFNone)
|
||||
if err == nil {
|
||||
t.Errorf("TestMaybeAcceptBlockErrors: rejecting the block if its parents are invalid: "+
|
||||
"Expected: %s, got: <nil>", ErrInvalidAncestorBlock)
|
||||
}
|
||||
if ok := errors.As(err, &ruleErr); !ok {
|
||||
t.Errorf("TestMaybeAcceptBlockErrors: rejecting the block if its parents are invalid: "+
|
||||
"Expected RuleError but got %s", err)
|
||||
} else if ruleErr.ErrorCode != ErrInvalidAncestorBlock {
|
||||
t.Errorf("TestMaybeAcceptBlockErrors: rejecting the block if its parents are invalid: "+
|
||||
"Unexpected error. Want: %s, got: %s", ErrInvalidAncestorBlock, ruleErr.ErrorCode)
|
||||
}
|
||||
|
||||
// Set block1's status back to valid for next tests
|
||||
dag.index.UnsetStatusFlags(blockNode1, statusValidateFailed)
|
||||
|
||||
// Test rejecting the block due to bad context
|
||||
originalBits := block2.MsgBlock().Header.Bits
|
||||
block2.MsgBlock().Header.Bits = 0
|
||||
err = dag.maybeAcceptBlock(block2, BFNone)
|
||||
if err == nil {
|
||||
t.Errorf("TestMaybeAcceptBlockErrors: rejecting the block due to bad context: "+
|
||||
"Expected: %s, got: <nil>", ErrUnexpectedDifficulty)
|
||||
}
|
||||
if ok := errors.As(err, &ruleErr); !ok {
|
||||
t.Errorf("TestMaybeAcceptBlockErrors: rejecting the block due to bad context: "+
|
||||
"Expected RuleError but got %s", err)
|
||||
} else if ruleErr.ErrorCode != ErrUnexpectedDifficulty {
|
||||
t.Errorf("TestMaybeAcceptBlockErrors: rejecting the block due to bad context: "+
|
||||
"Unexpected error. Want: %s, got: %s", ErrUnexpectedDifficulty, ruleErr.ErrorCode)
|
||||
}
|
||||
|
||||
// Set block2's bits back to valid for next tests
|
||||
block2.MsgBlock().Header.Bits = originalBits
|
||||
}
|
||||
|
87
blockdag/selected_parent_chain.go
Normal file
87
blockdag/selected_parent_chain.go
Normal file
@ -0,0 +1,87 @@
|
||||
package blockdag
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"github.com/kaspanet/kaspad/util/daghash"
|
||||
"github.com/pkg/errors"
|
||||
)
|
||||
|
||||
// SelectedParentChain returns the selected parent chain starting from blockHash (exclusive)
|
||||
// up to the virtual (exclusive). If blockHash is nil then the genesis block is used. If
|
||||
// blockHash is not within the select parent chain, go down its own selected parent chain,
|
||||
// while collecting each block hash in removedChainHashes, until reaching a block within
|
||||
// the main selected parent chain.
|
||||
//
|
||||
// This method MUST be called with the DAG lock held
|
||||
func (dag *BlockDAG) SelectedParentChain(blockHash *daghash.Hash) ([]*daghash.Hash, []*daghash.Hash, error) {
|
||||
if blockHash == nil {
|
||||
blockHash = dag.genesis.hash
|
||||
}
|
||||
if !dag.IsInDAG(blockHash) {
|
||||
return nil, nil, errors.Errorf("blockHash %s does not exist in the DAG", blockHash)
|
||||
}
|
||||
|
||||
// If blockHash is not in the selected parent chain, go down its selected parent chain
|
||||
// until we find a block that is in the main selected parent chain.
|
||||
var removedChainHashes []*daghash.Hash
|
||||
isBlockInSelectedParentChain, err := dag.IsInSelectedParentChain(blockHash)
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
for !isBlockInSelectedParentChain {
|
||||
removedChainHashes = append(removedChainHashes, blockHash)
|
||||
|
||||
node, ok := dag.index.LookupNode(blockHash)
|
||||
if !ok {
|
||||
return nil, nil, errors.Errorf("block %s does not exist in the DAG", blockHash)
|
||||
}
|
||||
blockHash = node.selectedParent.hash
|
||||
|
||||
isBlockInSelectedParentChain, err = dag.IsInSelectedParentChain(blockHash)
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
}
|
||||
|
||||
// Find the index of the blockHash in the selectedParentChainSlice
|
||||
blockHashIndex := len(dag.virtual.selectedParentChainSlice) - 1
|
||||
for blockHashIndex >= 0 {
|
||||
node := dag.virtual.selectedParentChainSlice[blockHashIndex]
|
||||
if node.hash.IsEqual(blockHash) {
|
||||
break
|
||||
}
|
||||
blockHashIndex--
|
||||
}
|
||||
|
||||
// Copy all the addedChainHashes starting from blockHashIndex (exclusive)
|
||||
addedChainHashes := make([]*daghash.Hash, len(dag.virtual.selectedParentChainSlice)-blockHashIndex-1)
|
||||
for i, node := range dag.virtual.selectedParentChainSlice[blockHashIndex+1:] {
|
||||
addedChainHashes[i] = node.hash
|
||||
}
|
||||
|
||||
return removedChainHashes, addedChainHashes, nil
|
||||
}
|
||||
|
||||
// IsInSelectedParentChain returns whether or not a block hash is found in the selected
|
||||
// parent chain. Note that this method returns an error if the given blockHash does not
|
||||
// exist within the block index.
|
||||
//
|
||||
// This method MUST be called with the DAG lock held
|
||||
func (dag *BlockDAG) IsInSelectedParentChain(blockHash *daghash.Hash) (bool, error) {
|
||||
blockNode, ok := dag.index.LookupNode(blockHash)
|
||||
if !ok {
|
||||
str := fmt.Sprintf("block %s is not in the DAG", blockHash)
|
||||
return false, ErrNotInDAG(str)
|
||||
}
|
||||
return dag.virtual.selectedParentChainSet.contains(blockNode), nil
|
||||
}
|
||||
|
||||
// isInSelectedParentChainOf returns whether `node` is in the selected parent chain of `other`.
|
||||
func (dag *BlockDAG) isInSelectedParentChainOf(node *blockNode, other *blockNode) (bool, error) {
|
||||
// By definition, a node is not in the selected parent chain of itself.
|
||||
if node == other {
|
||||
return false, nil
|
||||
}
|
||||
|
||||
return dag.reachabilityTree.isReachabilityTreeAncestorOf(node, other)
|
||||
}
|
143
blockdag/sequence_lock.go
Normal file
143
blockdag/sequence_lock.go
Normal file
@ -0,0 +1,143 @@
|
||||
package blockdag
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"github.com/kaspanet/kaspad/domainmessage"
|
||||
"github.com/kaspanet/kaspad/util"
|
||||
)
|
||||
|
||||
// SequenceLock represents the converted relative lock-time in seconds, and
|
||||
// absolute block-blue-score for a transaction input's relative lock-times.
|
||||
// According to SequenceLock, after the referenced input has been confirmed
|
||||
// within a block, a transaction spending that input can be included into a
|
||||
// block either after 'seconds' (according to past median time), or once the
|
||||
// 'BlockBlueScore' has been reached.
|
||||
type SequenceLock struct {
|
||||
Milliseconds int64
|
||||
BlockBlueScore int64
|
||||
}
|
||||
|
||||
// CalcSequenceLock computes a relative lock-time SequenceLock for the passed
|
||||
// transaction using the passed UTXOSet to obtain the past median time
|
||||
// for blocks in which the referenced inputs of the transactions were included
|
||||
// within. The generated SequenceLock lock can be used in conjunction with a
|
||||
// block height, and adjusted median block time to determine if all the inputs
|
||||
// referenced within a transaction have reached sufficient maturity allowing
|
||||
// the candidate transaction to be included in a block.
|
||||
//
|
||||
// This function is safe for concurrent access.
|
||||
func (dag *BlockDAG) CalcSequenceLock(tx *util.Tx, utxoSet UTXOSet) (*SequenceLock, error) {
|
||||
dag.dagLock.RLock()
|
||||
defer dag.dagLock.RUnlock()
|
||||
|
||||
return dag.calcSequenceLock(dag.selectedTip(), utxoSet, tx)
|
||||
}
|
||||
|
||||
// CalcSequenceLockNoLock is lock free version of CalcSequenceLockWithLock
|
||||
// This function is unsafe for concurrent access.
|
||||
func (dag *BlockDAG) CalcSequenceLockNoLock(tx *util.Tx, utxoSet UTXOSet) (*SequenceLock, error) {
|
||||
return dag.calcSequenceLock(dag.selectedTip(), utxoSet, tx)
|
||||
}
|
||||
|
||||
// calcSequenceLock computes the relative lock-times for the passed
|
||||
// transaction. See the exported version, CalcSequenceLock for further details.
|
||||
//
|
||||
// This function MUST be called with the DAG state lock held (for writes).
|
||||
func (dag *BlockDAG) calcSequenceLock(node *blockNode, utxoSet UTXOSet, tx *util.Tx) (*SequenceLock, error) {
|
||||
// A value of -1 for each relative lock type represents a relative time
|
||||
// lock value that will allow a transaction to be included in a block
|
||||
// at any given height or time.
|
||||
sequenceLock := &SequenceLock{Milliseconds: -1, BlockBlueScore: -1}
|
||||
|
||||
// Sequence locks don't apply to coinbase transactions Therefore, we
|
||||
// return sequence lock values of -1 indicating that this transaction
|
||||
// can be included within a block at any given height or time.
|
||||
if tx.IsCoinBase() {
|
||||
return sequenceLock, nil
|
||||
}
|
||||
|
||||
mTx := tx.MsgTx()
|
||||
for txInIndex, txIn := range mTx.TxIn {
|
||||
entry, ok := utxoSet.Get(txIn.PreviousOutpoint)
|
||||
if !ok {
|
||||
str := fmt.Sprintf("output %s referenced from "+
|
||||
"transaction %s input %d either does not exist or "+
|
||||
"has already been spent", txIn.PreviousOutpoint,
|
||||
tx.ID(), txInIndex)
|
||||
return sequenceLock, ruleError(ErrMissingTxOut, str)
|
||||
}
|
||||
|
||||
// If the input blue score is set to the mempool blue score, then we
|
||||
// assume the transaction makes it into the next block when
|
||||
// evaluating its sequence blocks.
|
||||
inputBlueScore := entry.BlockBlueScore()
|
||||
if entry.IsUnaccepted() {
|
||||
inputBlueScore = dag.virtual.blueScore
|
||||
}
|
||||
|
||||
// Given a sequence number, we apply the relative time lock
|
||||
// mask in order to obtain the time lock delta required before
|
||||
// this input can be spent.
|
||||
sequenceNum := txIn.Sequence
|
||||
relativeLock := int64(sequenceNum & domainmessage.SequenceLockTimeMask)
|
||||
|
||||
switch {
|
||||
// Relative time locks are disabled for this input, so we can
|
||||
// skip any further calculation.
|
||||
case sequenceNum&domainmessage.SequenceLockTimeDisabled == domainmessage.SequenceLockTimeDisabled:
|
||||
continue
|
||||
case sequenceNum&domainmessage.SequenceLockTimeIsSeconds == domainmessage.SequenceLockTimeIsSeconds:
|
||||
// This input requires a relative time lock expressed
|
||||
// in seconds before it can be spent. Therefore, we
|
||||
// need to query for the block prior to the one in
|
||||
// which this input was accepted within so we can
|
||||
// compute the past median time for the block prior to
|
||||
// the one which accepted this referenced output.
|
||||
blockNode := node
|
||||
for blockNode.selectedParent.blueScore > inputBlueScore {
|
||||
blockNode = blockNode.selectedParent
|
||||
}
|
||||
medianTime := blockNode.PastMedianTime(dag)
|
||||
|
||||
// Time based relative time-locks have a time granularity of
|
||||
// domainmessage.SequenceLockTimeGranularity, so we shift left by this
|
||||
// amount to convert to the proper relative time-lock. We also
|
||||
// subtract one from the relative lock to maintain the original
|
||||
// lockTime semantics.
|
||||
timeLockMilliseconds := (relativeLock << domainmessage.SequenceLockTimeGranularity) - 1
|
||||
timeLock := medianTime.UnixMilliseconds() + timeLockMilliseconds
|
||||
if timeLock > sequenceLock.Milliseconds {
|
||||
sequenceLock.Milliseconds = timeLock
|
||||
}
|
||||
default:
|
||||
// The relative lock-time for this input is expressed
|
||||
// in blocks so we calculate the relative offset from
|
||||
// the input's blue score as its converted absolute
|
||||
// lock-time. We subtract one from the relative lock in
|
||||
// order to maintain the original lockTime semantics.
|
||||
blockBlueScore := int64(inputBlueScore) + relativeLock - 1
|
||||
if blockBlueScore > sequenceLock.BlockBlueScore {
|
||||
sequenceLock.BlockBlueScore = blockBlueScore
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return sequenceLock, nil
|
||||
}
|
||||
|
||||
// LockTimeToSequence converts the passed relative locktime to a sequence
|
||||
// number.
|
||||
func LockTimeToSequence(isMilliseconds bool, locktime uint64) uint64 {
|
||||
// If we're expressing the relative lock time in blocks, then the
|
||||
// corresponding sequence number is simply the desired input age.
|
||||
if !isMilliseconds {
|
||||
return locktime
|
||||
}
|
||||
|
||||
// Set the 22nd bit which indicates the lock time is in milliseconds, then
|
||||
// shift the locktime over by 19 since the time granularity is in
|
||||
// 524288-millisecond intervals (2^19). This results in a max lock-time of
|
||||
// 34,359,214,080 seconds, or 1.1 years.
|
||||
return domainmessage.SequenceLockTimeIsSeconds |
|
||||
locktime>>domainmessage.SequenceLockTimeGranularity
|
||||
}
|
@ -155,3 +155,50 @@ func deserializeSubnetwork(serializedSNetBytes []byte) (*subnetwork, error) {
|
||||
gasLimit: gasLimit,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (dag *BlockDAG) validateGasLimit(block *util.Block) error {
|
||||
var currentSubnetworkID *subnetworkid.SubnetworkID
|
||||
var currentSubnetworkGasLimit uint64
|
||||
var currentGasUsage uint64
|
||||
var err error
|
||||
|
||||
// We assume here that transactions are ordered by subnetworkID,
|
||||
// since it was already validated in checkTransactionSanity
|
||||
for _, tx := range block.Transactions() {
|
||||
msgTx := tx.MsgTx()
|
||||
|
||||
// In native and Built-In subnetworks all txs must have Gas = 0, and that was already validated in checkTransactionSanity
|
||||
// Therefore - no need to check them here.
|
||||
if msgTx.SubnetworkID.IsEqual(subnetworkid.SubnetworkIDNative) || msgTx.SubnetworkID.IsBuiltIn() {
|
||||
continue
|
||||
}
|
||||
|
||||
if !msgTx.SubnetworkID.IsEqual(currentSubnetworkID) {
|
||||
currentSubnetworkID = &msgTx.SubnetworkID
|
||||
currentGasUsage = 0
|
||||
currentSubnetworkGasLimit, err = dag.GasLimit(currentSubnetworkID)
|
||||
if err != nil {
|
||||
return errors.Errorf("Error getting gas limit for subnetworkID '%s': %s", currentSubnetworkID, err)
|
||||
}
|
||||
}
|
||||
|
||||
newGasUsage := currentGasUsage + msgTx.Gas
|
||||
if newGasUsage < currentGasUsage { // check for overflow
|
||||
str := fmt.Sprintf("Block gas usage in subnetwork with ID %s has overflown", currentSubnetworkID)
|
||||
return ruleError(ErrInvalidGas, str)
|
||||
}
|
||||
if newGasUsage > currentSubnetworkGasLimit {
|
||||
str := fmt.Sprintf("Block wastes too much gas in subnetwork with ID %s", currentSubnetworkID)
|
||||
return ruleError(ErrInvalidGas, str)
|
||||
}
|
||||
|
||||
currentGasUsage = newGasUsage
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// SubnetworkID returns the node's subnetwork ID
|
||||
func (dag *BlockDAG) SubnetworkID() *subnetworkid.SubnetworkID {
|
||||
return dag.subnetworkID
|
||||
}
|
||||
|
@ -5,7 +5,13 @@ import (
|
||||
"time"
|
||||
)
|
||||
|
||||
const syncRateWindowDuration = 15 * time.Minute
|
||||
const (
|
||||
syncRateWindowDuration = 15 * time.Minute
|
||||
|
||||
// isDAGCurrentMaxDiff is the number of blocks from the network tips (estimated by timestamps) for the current
|
||||
// to be considered not synced
|
||||
isDAGCurrentMaxDiff = 40_000
|
||||
)
|
||||
|
||||
// addBlockProcessingTimestamp adds the last block processing timestamp in order to measure the recent sync rate.
|
||||
//
|
||||
@ -58,3 +64,40 @@ func (dag *BlockDAG) IsSyncRateBelowThreshold(maxDeviation float64) bool {
|
||||
func (dag *BlockDAG) uptime() time.Duration {
|
||||
return mstime.Now().Sub(dag.startTime)
|
||||
}
|
||||
|
||||
// isSynced returns whether or not the DAG believes it is synced. Several
|
||||
// factors are used to guess, but the key factors that allow the DAG to
|
||||
// believe it is synced are:
|
||||
// - Latest block has a timestamp newer than 24 hours ago
|
||||
//
|
||||
// This function MUST be called with the DAG state lock held (for reads).
|
||||
func (dag *BlockDAG) isSynced() bool {
|
||||
// Not synced if the virtual's selected parent has a timestamp
|
||||
// before 24 hours ago. If the DAG is empty, we take the genesis
|
||||
// block timestamp.
|
||||
//
|
||||
// The DAG appears to be syncned if none of the checks reported
|
||||
// otherwise.
|
||||
var dagTimestamp int64
|
||||
selectedTip := dag.selectedTip()
|
||||
if selectedTip == nil {
|
||||
dagTimestamp = dag.Params.GenesisBlock.Header.Timestamp.UnixMilliseconds()
|
||||
} else {
|
||||
dagTimestamp = selectedTip.timestamp
|
||||
}
|
||||
dagTime := mstime.UnixMilliseconds(dagTimestamp)
|
||||
return dag.Now().Sub(dagTime) <= isDAGCurrentMaxDiff*dag.Params.TargetTimePerBlock
|
||||
}
|
||||
|
||||
// IsSynced returns whether or not the DAG believes it is synced. Several
|
||||
// factors are used to guess, but the key factors that allow the DAG to
|
||||
// believe it is synced are:
|
||||
// - Latest block has a timestamp newer than 24 hours ago
|
||||
//
|
||||
// This function is safe for concurrent access.
|
||||
func (dag *BlockDAG) IsSynced() bool {
|
||||
dag.dagLock.RLock()
|
||||
defer dag.dagLock.RUnlock()
|
||||
|
||||
return dag.isSynced()
|
||||
}
|
||||
|
21
blockdag/sync_rate_test.go
Normal file
21
blockdag/sync_rate_test.go
Normal file
@ -0,0 +1,21 @@
|
||||
package blockdag
|
||||
|
||||
import (
|
||||
"github.com/kaspanet/kaspad/dagconfig"
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestIsDAGCurrentMaxDiff(t *testing.T) {
|
||||
netParams := []*dagconfig.Params{
|
||||
&dagconfig.MainnetParams,
|
||||
&dagconfig.TestnetParams,
|
||||
&dagconfig.DevnetParams,
|
||||
&dagconfig.RegressionNetParams,
|
||||
&dagconfig.SimnetParams,
|
||||
}
|
||||
for _, params := range netParams {
|
||||
if params.FinalityDuration < isDAGCurrentMaxDiff*params.TargetTimePerBlock {
|
||||
t.Errorf("in %s, a DAG can be considered current even if it's below the finality point", params.Name)
|
||||
}
|
||||
}
|
||||
}
|
@ -315,14 +315,9 @@ func PrepareBlockForTest(dag *BlockDAG, parentHashes []*daghash.Hash, transactio
|
||||
msgBlock.AddTransaction(tx.MsgTx())
|
||||
}
|
||||
|
||||
version, err := dag.calcNextBlockVersion(node.selectedParent)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
timestamp := node.parents.bluest().PastMedianTime(dag)
|
||||
msgBlock.Header = domainmessage.BlockHeader{
|
||||
Version: version,
|
||||
Version: blockVersion,
|
||||
ParentHashes: parentHashes,
|
||||
HashMerkleRoot: hashMerkleTree.Root(),
|
||||
AcceptedIDMerkleRoot: calculatedAccepetedIDMerkleRoot,
|
||||
|
@ -1,356 +0,0 @@
|
||||
// Copyright (c) 2016-2017 The btcsuite developers
|
||||
// Use of this source code is governed by an ISC
|
||||
// license that can be found in the LICENSE file.
|
||||
|
||||
package blockdag
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
||||
"github.com/kaspanet/kaspad/util/daghash"
|
||||
"github.com/pkg/errors"
|
||||
)
|
||||
|
||||
// ThresholdState define the various threshold states used when voting on
|
||||
// consensus changes.
|
||||
type ThresholdState byte
|
||||
|
||||
// These constants are used to identify specific threshold states.
|
||||
const (
|
||||
// ThresholdDefined is the first state for each deployment and is the
|
||||
// state for the genesis block has by definition for all deployments.
|
||||
ThresholdDefined ThresholdState = iota
|
||||
|
||||
// ThresholdStarted is the state for a deployment once its start time
|
||||
// has been reached.
|
||||
ThresholdStarted
|
||||
|
||||
// ThresholdLockedIn is the state for a deployment during the retarget
|
||||
// period which is after the ThresholdStarted state period and the
|
||||
// number of blocks that have voted for the deployment equal or exceed
|
||||
// the required number of votes for the deployment.
|
||||
ThresholdLockedIn
|
||||
|
||||
// ThresholdActive is the state for a deployment for all blocks after a
|
||||
// retarget period in which the deployment was in the ThresholdLockedIn
|
||||
// state.
|
||||
ThresholdActive
|
||||
|
||||
// ThresholdFailed is the state for a deployment once its expiration
|
||||
// time has been reached and it did not reach the ThresholdLockedIn
|
||||
// state.
|
||||
ThresholdFailed
|
||||
|
||||
// numThresholdsStates is the maximum number of threshold states used in
|
||||
// tests.
|
||||
numThresholdsStates
|
||||
)
|
||||
|
||||
// thresholdStateStrings is a map of ThresholdState values back to their
|
||||
// constant names for pretty printing.
|
||||
var thresholdStateStrings = map[ThresholdState]string{
|
||||
ThresholdDefined: "ThresholdDefined",
|
||||
ThresholdStarted: "ThresholdStarted",
|
||||
ThresholdLockedIn: "ThresholdLockedIn",
|
||||
ThresholdActive: "ThresholdActive",
|
||||
ThresholdFailed: "ThresholdFailed",
|
||||
}
|
||||
|
||||
// String returns the ThresholdState as a human-readable name.
|
||||
func (t ThresholdState) String() string {
|
||||
if s := thresholdStateStrings[t]; s != "" {
|
||||
return s
|
||||
}
|
||||
return fmt.Sprintf("Unknown ThresholdState (%d)", int(t))
|
||||
}
|
||||
|
||||
// thresholdConditionChecker provides a generic interface that is invoked to
|
||||
// determine when a consensus rule change threshold should be changed.
|
||||
type thresholdConditionChecker interface {
|
||||
// BeginTime returns the unix timestamp for the median block time after
|
||||
// which voting on a rule change starts (at the next window).
|
||||
BeginTime() uint64
|
||||
|
||||
// EndTime returns the unix timestamp for the median block time after
|
||||
// which an attempted rule change fails if it has not already been
|
||||
// locked in or activated.
|
||||
EndTime() uint64
|
||||
|
||||
// RuleChangeActivationThreshold is the number of blocks for which the
|
||||
// condition must be true in order to lock in a rule change.
|
||||
RuleChangeActivationThreshold() uint64
|
||||
|
||||
// MinerConfirmationWindow is the number of blocks in each threshold
|
||||
// state retarget window.
|
||||
MinerConfirmationWindow() uint64
|
||||
|
||||
// Condition returns whether or not the rule change activation condition
|
||||
// has been met. This typically involves checking whether or not the
|
||||
// bit associated with the condition is set, but can be more complex as
|
||||
// needed.
|
||||
Condition(*blockNode) (bool, error)
|
||||
}
|
||||
|
||||
// thresholdStateCache provides a type to cache the threshold states of each
|
||||
// threshold window for a set of IDs.
|
||||
type thresholdStateCache struct {
|
||||
entries map[daghash.Hash]ThresholdState
|
||||
}
|
||||
|
||||
// Lookup returns the threshold state associated with the given hash along with
|
||||
// a boolean that indicates whether or not it is valid.
|
||||
func (c *thresholdStateCache) Lookup(hash *daghash.Hash) (ThresholdState, bool) {
|
||||
state, ok := c.entries[*hash]
|
||||
return state, ok
|
||||
}
|
||||
|
||||
// Update updates the cache to contain the provided hash to threshold state
|
||||
// mapping.
|
||||
func (c *thresholdStateCache) Update(hash *daghash.Hash, state ThresholdState) {
|
||||
c.entries[*hash] = state
|
||||
}
|
||||
|
||||
// newThresholdCaches returns a new array of caches to be used when calculating
|
||||
// threshold states.
|
||||
func newThresholdCaches(numCaches uint32) []thresholdStateCache {
|
||||
caches := make([]thresholdStateCache, numCaches)
|
||||
for i := 0; i < len(caches); i++ {
|
||||
caches[i] = thresholdStateCache{
|
||||
entries: make(map[daghash.Hash]ThresholdState),
|
||||
}
|
||||
}
|
||||
return caches
|
||||
}
|
||||
|
||||
// thresholdState returns the current rule change threshold state for the block
|
||||
// AFTER the given node and deployment ID. The cache is used to ensure the
|
||||
// threshold states for previous windows are only calculated once.
|
||||
//
|
||||
// This function MUST be called with the DAG state lock held (for writes).
|
||||
func (dag *BlockDAG) thresholdState(prevNode *blockNode, checker thresholdConditionChecker, cache *thresholdStateCache) (ThresholdState, error) {
|
||||
// The threshold state for the window that contains the genesis block is
|
||||
// defined by definition.
|
||||
confirmationWindow := checker.MinerConfirmationWindow()
|
||||
if prevNode == nil || (prevNode.blueScore+1) < confirmationWindow {
|
||||
return ThresholdDefined, nil
|
||||
}
|
||||
|
||||
// Get the ancestor that is the last block of the previous confirmation
|
||||
// window in order to get its threshold state. This can be done because
|
||||
// the state is the same for all blocks within a given window.
|
||||
prevNode = prevNode.SelectedAncestor(prevNode.blueScore -
|
||||
(prevNode.blueScore+1)%confirmationWindow)
|
||||
|
||||
// Iterate backwards through each of the previous confirmation windows
|
||||
// to find the most recently cached threshold state.
|
||||
var neededStates []*blockNode
|
||||
for prevNode != nil {
|
||||
// Nothing more to do if the state of the block is already
|
||||
// cached.
|
||||
if _, ok := cache.Lookup(prevNode.hash); ok {
|
||||
break
|
||||
}
|
||||
|
||||
// The start and expiration times are based on the median block
|
||||
// time, so calculate it now.
|
||||
medianTime := prevNode.PastMedianTime(dag)
|
||||
|
||||
// The state is simply defined if the start time hasn't been
|
||||
// been reached yet.
|
||||
if uint64(medianTime.UnixMilliseconds()) < checker.BeginTime() {
|
||||
cache.Update(prevNode.hash, ThresholdDefined)
|
||||
break
|
||||
}
|
||||
|
||||
// Add this node to the list of nodes that need the state
|
||||
// calculated and cached.
|
||||
neededStates = append(neededStates, prevNode)
|
||||
|
||||
// Get the ancestor that is the last block of the previous
|
||||
// confirmation window.
|
||||
prevNode = prevNode.RelativeAncestor(confirmationWindow)
|
||||
}
|
||||
|
||||
// Start with the threshold state for the most recent confirmation
|
||||
// window that has a cached state.
|
||||
state := ThresholdDefined
|
||||
if prevNode != nil {
|
||||
var ok bool
|
||||
state, ok = cache.Lookup(prevNode.hash)
|
||||
if !ok {
|
||||
return ThresholdFailed, errors.Errorf(
|
||||
"thresholdState: cache lookup failed for %s",
|
||||
prevNode.hash)
|
||||
}
|
||||
}
|
||||
|
||||
// Since each threshold state depends on the state of the previous
|
||||
// window, iterate starting from the oldest unknown window.
|
||||
for neededNum := len(neededStates) - 1; neededNum >= 0; neededNum-- {
|
||||
prevNode := neededStates[neededNum]
|
||||
|
||||
switch state {
|
||||
case ThresholdDefined:
|
||||
// The deployment of the rule change fails if it expires
|
||||
// before it is accepted and locked in.
|
||||
medianTime := prevNode.PastMedianTime(dag)
|
||||
medianTimeUnix := uint64(medianTime.UnixMilliseconds())
|
||||
if medianTimeUnix >= checker.EndTime() {
|
||||
state = ThresholdFailed
|
||||
break
|
||||
}
|
||||
|
||||
// The state for the rule moves to the started state
|
||||
// once its start time has been reached (and it hasn't
|
||||
// already expired per the above).
|
||||
if medianTimeUnix >= checker.BeginTime() {
|
||||
state = ThresholdStarted
|
||||
}
|
||||
|
||||
case ThresholdStarted:
|
||||
// The deployment of the rule change fails if it expires
|
||||
// before it is accepted and locked in.
|
||||
medianTime := prevNode.PastMedianTime(dag)
|
||||
if uint64(medianTime.UnixMilliseconds()) >= checker.EndTime() {
|
||||
state = ThresholdFailed
|
||||
break
|
||||
}
|
||||
|
||||
// At this point, the rule change is still being voted
|
||||
// on by the miners, so iterate backwards through the
|
||||
// confirmation window to count all of the votes in it.
|
||||
var count uint64
|
||||
windowNodes := make([]*blockNode, 0, confirmationWindow)
|
||||
windowNodes = append(windowNodes, prevNode)
|
||||
windowNodes = append(windowNodes, blueBlockWindow(prevNode, confirmationWindow-1)...)
|
||||
for _, current := range windowNodes {
|
||||
condition, err := checker.Condition(current)
|
||||
if err != nil {
|
||||
return ThresholdFailed, err
|
||||
}
|
||||
if condition {
|
||||
count++
|
||||
}
|
||||
}
|
||||
|
||||
// The state is locked in if the number of blocks in the
|
||||
// period that voted for the rule change meets the
|
||||
// activation threshold.
|
||||
if count >= checker.RuleChangeActivationThreshold() {
|
||||
state = ThresholdLockedIn
|
||||
}
|
||||
|
||||
case ThresholdLockedIn:
|
||||
// The new rule becomes active when its previous state
|
||||
// was locked in.
|
||||
state = ThresholdActive
|
||||
|
||||
// Nothing to do if the previous state is active or failed since
|
||||
// they are both terminal states.
|
||||
case ThresholdActive:
|
||||
case ThresholdFailed:
|
||||
}
|
||||
|
||||
// Update the cache to avoid recalculating the state in the
|
||||
// future.
|
||||
cache.Update(prevNode.hash, state)
|
||||
}
|
||||
|
||||
return state, nil
|
||||
}
|
||||
|
||||
// ThresholdState returns the current rule change threshold state of the given
|
||||
// deployment ID for the block AFTER the blueScore of the current DAG.
|
||||
//
|
||||
// This function is safe for concurrent access.
|
||||
func (dag *BlockDAG) ThresholdState(deploymentID uint32) (ThresholdState, error) {
|
||||
dag.dagLock.Lock()
|
||||
defer dag.dagLock.Unlock()
|
||||
state, err := dag.deploymentState(dag.selectedTip(), deploymentID)
|
||||
|
||||
return state, err
|
||||
}
|
||||
|
||||
// IsDeploymentActive returns true if the target deploymentID is active, and
|
||||
// false otherwise.
|
||||
//
|
||||
// This function is safe for concurrent access.
|
||||
func (dag *BlockDAG) IsDeploymentActive(deploymentID uint32) (bool, error) {
|
||||
dag.dagLock.Lock()
|
||||
defer dag.dagLock.Unlock()
|
||||
state, err := dag.deploymentState(dag.selectedTip(), deploymentID)
|
||||
if err != nil {
|
||||
return false, err
|
||||
}
|
||||
|
||||
return state == ThresholdActive, nil
|
||||
}
|
||||
|
||||
// deploymentState returns the current rule change threshold for a given
|
||||
// deploymentID. The threshold is evaluated from the point of view of the block
|
||||
// node passed in as the first argument to this method.
|
||||
//
|
||||
// It is important to note that, as the variable name indicates, this function
|
||||
// expects the block node prior to the block for which the deployment state is
|
||||
// desired. In other words, the returned deployment state is for the block
|
||||
// AFTER the passed node.
|
||||
//
|
||||
// This function MUST be called with the DAG state lock held (for writes).
|
||||
func (dag *BlockDAG) deploymentState(prevNode *blockNode, deploymentID uint32) (ThresholdState, error) {
|
||||
if deploymentID > uint32(len(dag.Params.Deployments)) {
|
||||
return ThresholdFailed, errors.Errorf("deployment ID %d does not exist", deploymentID)
|
||||
}
|
||||
|
||||
deployment := &dag.Params.Deployments[deploymentID]
|
||||
checker := deploymentChecker{deployment: deployment, dag: dag}
|
||||
cache := &dag.deploymentCaches[deploymentID]
|
||||
|
||||
return dag.thresholdState(prevNode, checker, cache)
|
||||
}
|
||||
|
||||
// initThresholdCaches initializes the threshold state caches for each warning
|
||||
// bit and defined deployment and provides warnings if the DAG is current per
|
||||
// the warnUnknownVersions and warnUnknownRuleActivations functions.
|
||||
func (dag *BlockDAG) initThresholdCaches() error {
|
||||
// Initialize the warning and deployment caches by calculating the
|
||||
// threshold state for each of them. This will ensure the caches are
|
||||
// populated and any states that needed to be recalculated due to
|
||||
// definition changes is done now.
|
||||
prevNode := dag.selectedTip().selectedParent
|
||||
for bit := uint32(0); bit < vbNumBits; bit++ {
|
||||
checker := bitConditionChecker{bit: bit, dag: dag}
|
||||
cache := &dag.warningCaches[bit]
|
||||
_, err := dag.thresholdState(prevNode, checker, cache)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
for id := 0; id < len(dag.Params.Deployments); id++ {
|
||||
deployment := &dag.Params.Deployments[id]
|
||||
cache := &dag.deploymentCaches[id]
|
||||
checker := deploymentChecker{deployment: deployment, dag: dag}
|
||||
_, err := dag.thresholdState(prevNode, checker, cache)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
// No warnings about unknown rules or versions until the DAG is
|
||||
// synced.
|
||||
if dag.isSynced() {
|
||||
// Warn if a high enough percentage of the last blocks have
|
||||
// unexpected versions.
|
||||
bestNode := dag.selectedTip()
|
||||
if err := dag.warnUnknownVersions(bestNode); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Warn if any unknown new rules are either about to activate or
|
||||
// have already been activated.
|
||||
if err := dag.warnUnknownRuleActivations(bestNode); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
@ -1,134 +0,0 @@
|
||||
// Copyright (c) 2016 The btcsuite developers
|
||||
// Use of this source code is governed by an ISC
|
||||
// license that can be found in the LICENSE file.
|
||||
|
||||
package blockdag
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/kaspanet/kaspad/util/daghash"
|
||||
)
|
||||
|
||||
// TestThresholdStateStringer tests the stringized output for the
|
||||
// ThresholdState type.
|
||||
func TestThresholdStateStringer(t *testing.T) {
|
||||
t.Parallel()
|
||||
|
||||
tests := []struct {
|
||||
in ThresholdState
|
||||
want string
|
||||
}{
|
||||
{ThresholdDefined, "ThresholdDefined"},
|
||||
{ThresholdStarted, "ThresholdStarted"},
|
||||
{ThresholdLockedIn, "ThresholdLockedIn"},
|
||||
{ThresholdActive, "ThresholdActive"},
|
||||
{ThresholdFailed, "ThresholdFailed"},
|
||||
{0xff, "Unknown ThresholdState (255)"},
|
||||
}
|
||||
|
||||
// Detect additional threshold states that don't have the stringer added.
|
||||
if len(tests)-1 != int(numThresholdsStates) {
|
||||
t.Errorf("It appears a threshold statewas added without " +
|
||||
"adding an associated stringer test")
|
||||
}
|
||||
|
||||
t.Logf("Running %d tests", len(tests))
|
||||
for i, test := range tests {
|
||||
result := test.in.String()
|
||||
if result != test.want {
|
||||
t.Errorf("String #%d\n got: %s want: %s", i, result,
|
||||
test.want)
|
||||
continue
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// TestThresholdStateCache ensure the threshold state cache works as intended
|
||||
// including adding entries, updating existing entries, and flushing.
|
||||
func TestThresholdStateCache(t *testing.T) {
|
||||
t.Parallel()
|
||||
|
||||
tests := []struct {
|
||||
name string
|
||||
numEntries int
|
||||
state ThresholdState
|
||||
}{
|
||||
{name: "2 entries defined", numEntries: 2, state: ThresholdDefined},
|
||||
{name: "7 entries started", numEntries: 7, state: ThresholdStarted},
|
||||
{name: "10 entries active", numEntries: 10, state: ThresholdActive},
|
||||
{name: "5 entries locked in", numEntries: 5, state: ThresholdLockedIn},
|
||||
{name: "3 entries failed", numEntries: 3, state: ThresholdFailed},
|
||||
}
|
||||
|
||||
nextTest:
|
||||
for _, test := range tests {
|
||||
cache := &newThresholdCaches(1)[0]
|
||||
for i := 0; i < test.numEntries; i++ {
|
||||
var hash daghash.Hash
|
||||
hash[0] = uint8(i + 1)
|
||||
|
||||
// Ensure the hash isn't available in the cache already.
|
||||
_, ok := cache.Lookup(&hash)
|
||||
if ok {
|
||||
t.Errorf("Lookup (%s): has entry for hash %v",
|
||||
test.name, hash)
|
||||
continue nextTest
|
||||
}
|
||||
|
||||
// Ensure hash that was added to the cache reports it's
|
||||
// available and the state is the expected value.
|
||||
cache.Update(&hash, test.state)
|
||||
state, ok := cache.Lookup(&hash)
|
||||
if !ok {
|
||||
t.Errorf("Lookup (%s): missing entry for hash "+
|
||||
"%v", test.name, hash)
|
||||
continue nextTest
|
||||
}
|
||||
if state != test.state {
|
||||
t.Errorf("Lookup (%s): state mismatch - got "+
|
||||
"%v, want %v", test.name, state,
|
||||
test.state)
|
||||
continue nextTest
|
||||
}
|
||||
|
||||
// Ensure adding an existing hash with the same state
|
||||
// doesn't break the existing entry.
|
||||
cache.Update(&hash, test.state)
|
||||
state, ok = cache.Lookup(&hash)
|
||||
if !ok {
|
||||
t.Errorf("Lookup (%s): missing entry after "+
|
||||
"second add for hash %v", test.name,
|
||||
hash)
|
||||
continue nextTest
|
||||
}
|
||||
if state != test.state {
|
||||
t.Errorf("Lookup (%s): state mismatch after "+
|
||||
"second add - got %v, want %v",
|
||||
test.name, state, test.state)
|
||||
continue nextTest
|
||||
}
|
||||
|
||||
// Ensure adding an existing hash with a different state
|
||||
// updates the existing entry.
|
||||
newState := ThresholdFailed
|
||||
if newState == test.state {
|
||||
newState = ThresholdStarted
|
||||
}
|
||||
cache.Update(&hash, newState)
|
||||
state, ok = cache.Lookup(&hash)
|
||||
if !ok {
|
||||
t.Errorf("Lookup (%s): missing entry after "+
|
||||
"state change for hash %v", test.name,
|
||||
hash)
|
||||
continue nextTest
|
||||
}
|
||||
if state != newState {
|
||||
t.Errorf("Lookup (%s): state mismatch after "+
|
||||
"state change - got %v, want %v",
|
||||
test.name, state, newState)
|
||||
continue nextTest
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
@ -6,13 +6,11 @@ package blockdag
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"github.com/kaspanet/go-secp256k1"
|
||||
"github.com/kaspanet/kaspad/util/mstime"
|
||||
"github.com/pkg/errors"
|
||||
"math"
|
||||
"sort"
|
||||
"time"
|
||||
|
||||
"github.com/kaspanet/kaspad/util/mstime"
|
||||
|
||||
"github.com/pkg/errors"
|
||||
|
||||
"github.com/kaspanet/kaspad/dagconfig"
|
||||
"github.com/kaspanet/kaspad/domainmessage"
|
||||
@ -128,13 +126,55 @@ func CalcBlockSubsidy(blueScore uint64, dagParams *dagconfig.Params) uint64 {
|
||||
// CheckTransactionSanity performs some preliminary checks on a transaction to
|
||||
// ensure it is sane. These checks are context free.
|
||||
func CheckTransactionSanity(tx *util.Tx, subnetworkID *subnetworkid.SubnetworkID) error {
|
||||
isCoinbase := tx.IsCoinBase()
|
||||
// A transaction must have at least one input.
|
||||
err := checkTransactionInputCount(tx)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
err = checkTransactionAmountRanges(tx)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
err = checkDuplicateTransactionInputs(tx)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
err = checkCoinbaseLength(tx)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
err = checkTransactionPayloadHash(tx)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
err = checkGasInBuiltInOrNativeTransactions(tx)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
err = checkSubnetworkRegistryTransaction(tx)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
err = checkNativeTransactionPayload(tx)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
err = checkTransactionSubnetwork(tx, subnetworkID)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func checkTransactionInputCount(tx *util.Tx) error {
|
||||
// A non-coinbase transaction must have at least one input.
|
||||
msgTx := tx.MsgTx()
|
||||
if !isCoinbase && len(msgTx.TxIn) == 0 {
|
||||
if !tx.IsCoinBase() && len(msgTx.TxIn) == 0 {
|
||||
return ruleError(ErrNoTxInputs, "transaction has no inputs")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func checkTransactionAmountRanges(tx *util.Tx) error {
|
||||
// Ensure the transaction amounts are in range. Each transaction
|
||||
// output must not be negative or more than the max allowed per
|
||||
// transaction. Also, the total of all outputs must abide by the same
|
||||
@ -142,7 +182,7 @@ func CheckTransactionSanity(tx *util.Tx, subnetworkID *subnetworkid.SubnetworkID
|
||||
// as a sompi. One kaspa is a quantity of sompi as defined by the
|
||||
// SompiPerKaspa constant.
|
||||
var totalSompi uint64
|
||||
for _, txOut := range msgTx.TxOut {
|
||||
for _, txOut := range tx.MsgTx().TxOut {
|
||||
sompi := txOut.Value
|
||||
if sompi > util.MaxSompi {
|
||||
str := fmt.Sprintf("transaction output value of %d is "+
|
||||
@ -170,20 +210,25 @@ func CheckTransactionSanity(tx *util.Tx, subnetworkID *subnetworkid.SubnetworkID
|
||||
return ruleError(ErrBadTxOutValue, str)
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// Check for duplicate transaction inputs.
|
||||
func checkDuplicateTransactionInputs(tx *util.Tx) error {
|
||||
existingTxOut := make(map[domainmessage.Outpoint]struct{})
|
||||
for _, txIn := range msgTx.TxIn {
|
||||
for _, txIn := range tx.MsgTx().TxIn {
|
||||
if _, exists := existingTxOut[txIn.PreviousOutpoint]; exists {
|
||||
return ruleError(ErrDuplicateTxInputs, "transaction "+
|
||||
"contains duplicate inputs")
|
||||
}
|
||||
existingTxOut[txIn.PreviousOutpoint] = struct{}{}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func checkCoinbaseLength(tx *util.Tx) error {
|
||||
// Coinbase payload length must not exceed the max length.
|
||||
if isCoinbase {
|
||||
payloadLen := len(msgTx.Payload)
|
||||
if tx.IsCoinBase() {
|
||||
payloadLen := len(tx.MsgTx().Payload)
|
||||
if payloadLen > MaxCoinbasePayloadLen {
|
||||
str := fmt.Sprintf("coinbase transaction payload length "+
|
||||
"of %d is out of range (max: %d)",
|
||||
@ -193,7 +238,7 @@ func CheckTransactionSanity(tx *util.Tx, subnetworkID *subnetworkid.SubnetworkID
|
||||
} else {
|
||||
// Previous transaction outputs referenced by the inputs to this
|
||||
// transaction must not be null.
|
||||
for _, txIn := range msgTx.TxIn {
|
||||
for _, txIn := range tx.MsgTx().TxIn {
|
||||
if isNullOutpoint(&txIn.PreviousOutpoint) {
|
||||
return ruleError(ErrBadTxInput, "transaction "+
|
||||
"input refers to previous output that "+
|
||||
@ -201,8 +246,11 @@ func CheckTransactionSanity(tx *util.Tx, subnetworkID *subnetworkid.SubnetworkID
|
||||
}
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// Check payload's hash
|
||||
func checkTransactionPayloadHash(tx *util.Tx) error {
|
||||
msgTx := tx.MsgTx()
|
||||
if !msgTx.SubnetworkID.IsEqual(subnetworkid.SubnetworkIDNative) {
|
||||
payloadHash := daghash.DoubleHashH(msgTx.Payload)
|
||||
if !msgTx.PayloadHash.IsEqual(&payloadHash) {
|
||||
@ -211,29 +259,44 @@ func CheckTransactionSanity(tx *util.Tx, subnetworkID *subnetworkid.SubnetworkID
|
||||
} else if msgTx.PayloadHash != nil {
|
||||
return ruleError(ErrInvalidPayloadHash, "unexpected non-empty payload hash in native subnetwork")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func checkGasInBuiltInOrNativeTransactions(tx *util.Tx) error {
|
||||
// Transactions in native, registry and coinbase subnetworks must have Gas = 0
|
||||
if msgTx.SubnetworkID.IsBuiltInOrNative() && msgTx.Gas > 0 {
|
||||
msgTx := tx.MsgTx()
|
||||
if (msgTx.SubnetworkID.IsEqual(subnetworkid.SubnetworkIDNative) ||
|
||||
msgTx.SubnetworkID.IsBuiltIn()) &&
|
||||
msgTx.Gas > 0 {
|
||||
|
||||
return ruleError(ErrInvalidGas, "transaction in the native or "+
|
||||
"registry subnetworks has gas > 0 ")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
if msgTx.SubnetworkID.IsEqual(subnetworkid.SubnetworkIDRegistry) {
|
||||
err := validateSubnetworkRegistryTransaction(msgTx)
|
||||
func checkSubnetworkRegistryTransaction(tx *util.Tx) error {
|
||||
if tx.MsgTx().SubnetworkID.IsEqual(subnetworkid.SubnetworkIDRegistry) {
|
||||
err := validateSubnetworkRegistryTransaction(tx.MsgTx())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
if msgTx.SubnetworkID.IsEqual(subnetworkid.SubnetworkIDNative) &&
|
||||
len(msgTx.Payload) > 0 {
|
||||
|
||||
return ruleError(ErrInvalidPayload,
|
||||
"transaction in the native subnetwork includes a payload")
|
||||
func checkNativeTransactionPayload(tx *util.Tx) error {
|
||||
msgTx := tx.MsgTx()
|
||||
if msgTx.SubnetworkID.IsEqual(subnetworkid.SubnetworkIDNative) && len(msgTx.Payload) > 0 {
|
||||
return ruleError(ErrInvalidPayload, "transaction in the native subnetwork includes a payload")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func checkTransactionSubnetwork(tx *util.Tx, subnetworkID *subnetworkid.SubnetworkID) error {
|
||||
// If we are a partial node, only transactions on built in subnetworks
|
||||
// or our own subnetwork may have a payload
|
||||
msgTx := tx.MsgTx()
|
||||
isLocalNodeFull := subnetworkID == nil
|
||||
shouldTxBeFull := msgTx.SubnetworkID.IsBuiltIn() ||
|
||||
msgTx.SubnetworkID.IsEqual(subnetworkID)
|
||||
@ -242,7 +305,6 @@ func CheckTransactionSanity(tx *util.Tx, subnetworkID *subnetworkid.SubnetworkID
|
||||
"transaction that was expected to be partial has a payload "+
|
||||
"with length > 0")
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
@ -390,37 +452,28 @@ func CalcTxMass(tx *util.Tx, previousScriptPubKeys [][]byte) uint64 {
|
||||
//
|
||||
// The flags do not modify the behavior of this function directly, however they
|
||||
// are needed to pass along to checkProofOfWork.
|
||||
func (dag *BlockDAG) checkBlockHeaderSanity(block *util.Block, flags BehaviorFlags) (delay time.Duration, err error) {
|
||||
func (dag *BlockDAG) checkBlockHeaderSanity(block *util.Block, flags BehaviorFlags) error {
|
||||
// Ensure the proof of work bits in the block header is in min/max range
|
||||
// and the block hash is less than the target value described by the
|
||||
// bits.
|
||||
header := &block.MsgBlock().Header
|
||||
err = dag.checkProofOfWork(header, flags)
|
||||
err := dag.checkProofOfWork(header, flags)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
return err
|
||||
}
|
||||
|
||||
if len(header.ParentHashes) == 0 {
|
||||
if !header.BlockHash().IsEqual(dag.Params.GenesisHash) {
|
||||
return 0, ruleError(ErrNoParents, "block has no parents")
|
||||
return ruleError(ErrNoParents, "block has no parents")
|
||||
}
|
||||
} else {
|
||||
err = checkBlockParentsOrder(header)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
// Ensure the block time is not too far in the future. If it's too far, return
|
||||
// the duration of time that should be waited before the block becomes valid.
|
||||
// This check needs to be last as it does not return an error but rather marks the
|
||||
// header as delayed (and valid).
|
||||
maxTimestamp := dag.Now().Add(time.Duration(dag.TimestampDeviationTolerance) * dag.Params.TargetTimePerBlock)
|
||||
if header.Timestamp.After(maxTimestamp) {
|
||||
return header.Timestamp.Sub(maxTimestamp), nil
|
||||
}
|
||||
|
||||
return 0, nil
|
||||
return nil
|
||||
}
|
||||
|
||||
//checkBlockParentsOrder ensures that the block's parents are ordered by hash
|
||||
@ -443,56 +496,56 @@ func checkBlockParentsOrder(header *domainmessage.BlockHeader) error {
|
||||
//
|
||||
// The flags do not modify the behavior of this function directly, however they
|
||||
// are needed to pass along to checkBlockHeaderSanity.
|
||||
func (dag *BlockDAG) checkBlockSanity(block *util.Block, flags BehaviorFlags) (time.Duration, error) {
|
||||
delay, err := dag.checkBlockHeaderSanity(block, flags)
|
||||
func (dag *BlockDAG) checkBlockSanity(block *util.Block, flags BehaviorFlags) error {
|
||||
err := dag.checkBlockHeaderSanity(block, flags)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
return err
|
||||
}
|
||||
err = dag.checkBlockContainsAtLeastOneTransaction(block)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
return err
|
||||
}
|
||||
err = dag.checkBlockContainsLessThanMaxBlockMassTransactions(block)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
return err
|
||||
}
|
||||
err = dag.checkFirstBlockTransactionIsCoinbase(block)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
return err
|
||||
}
|
||||
err = dag.checkBlockContainsOnlyOneCoinbase(block)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
return err
|
||||
}
|
||||
err = dag.checkBlockTransactionOrder(block)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
return err
|
||||
}
|
||||
err = dag.checkNoNonNativeTransactions(block)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
return err
|
||||
}
|
||||
err = dag.checkBlockTransactionSanity(block)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
return err
|
||||
}
|
||||
err = dag.checkBlockHashMerkleRoot(block)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
return err
|
||||
}
|
||||
|
||||
// The following check will be fairly quick since the transaction IDs
|
||||
// are already cached due to building the merkle tree above.
|
||||
err = dag.checkBlockDuplicateTransactions(block)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
return err
|
||||
}
|
||||
|
||||
err = dag.checkBlockDoubleSpends(block)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
return err
|
||||
}
|
||||
return delay, nil
|
||||
return nil
|
||||
}
|
||||
|
||||
func (dag *BlockDAG) checkBlockContainsAtLeastOneTransaction(block *util.Block) error {
|
||||
@ -715,11 +768,16 @@ func (dag *BlockDAG) validateParents(blockHeader *domainmessage.BlockHeader, par
|
||||
// for how the flags modify its behavior.
|
||||
//
|
||||
// This function MUST be called with the dag state lock held (for writes).
|
||||
func (dag *BlockDAG) checkBlockContext(block *util.Block, parents blockSet, flags BehaviorFlags) error {
|
||||
func (dag *BlockDAG) checkBlockContext(block *util.Block, flags BehaviorFlags) error {
|
||||
parents, err := lookupParentNodes(block, dag)
|
||||
if err != nil {
|
||||
return dag.handleLookupParentNodesError(block, err)
|
||||
}
|
||||
|
||||
bluestParent := parents.bluest()
|
||||
fastAdd := flags&BFFastAdd == BFFastAdd
|
||||
|
||||
err := dag.validateParents(&block.MsgBlock().Header, parents)
|
||||
err = dag.validateParents(&block.MsgBlock().Header, parents)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -733,10 +791,26 @@ func (dag *BlockDAG) checkBlockContext(block *util.Block, parents blockSet, flag
|
||||
return nil
|
||||
}
|
||||
|
||||
func (dag *BlockDAG) validateAllTxsFinalized(block *util.Block, node *blockNode, bluestParent *blockNode) error {
|
||||
func (dag *BlockDAG) handleLookupParentNodesError(block *util.Block, err error) error {
|
||||
var ruleErr RuleError
|
||||
if ok := errors.As(err, &ruleErr); ok && ruleErr.ErrorCode == ErrInvalidAncestorBlock {
|
||||
err := dag.addNodeToIndexWithInvalidAncestor(block)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
func (dag *BlockDAG) checkBlockTransactionsFinalized(block *util.Block, node *blockNode, flags BehaviorFlags) error {
|
||||
fastAdd := flags&BFFastAdd == BFFastAdd || dag.index.NodeStatus(node).KnownValid()
|
||||
if fastAdd {
|
||||
return nil
|
||||
}
|
||||
|
||||
blockTime := block.MsgBlock().Header.Timestamp
|
||||
if !block.IsGenesis() {
|
||||
blockTime = bluestParent.PastMedianTime(dag)
|
||||
blockTime = node.selectedParent.PastMedianTime(dag)
|
||||
}
|
||||
|
||||
// Ensure all transactions in the block are finalized.
|
||||
@ -977,7 +1051,7 @@ func (dag *BlockDAG) checkConnectToPastUTXO(block *blockNode, pastUTXO UTXOSet,
|
||||
// A transaction can only be included within a block
|
||||
// once the sequence locks of *all* its inputs are
|
||||
// active.
|
||||
sequenceLock, err := dag.calcSequenceLock(block, pastUTXO, tx, false)
|
||||
sequenceLock, err := dag.calcSequenceLock(block, pastUTXO, tx)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@ -1002,6 +1076,18 @@ func (dag *BlockDAG) checkConnectToPastUTXO(block *blockNode, pastUTXO UTXOSet,
|
||||
return feeData, nil
|
||||
}
|
||||
|
||||
func (node *blockNode) validateUTXOCommitment(multiset *secp256k1.MultiSet) error {
|
||||
calculatedMultisetHash := daghash.Hash(*multiset.Finalize())
|
||||
if !calculatedMultisetHash.IsEqual(node.utxoCommitment) {
|
||||
str := fmt.Sprintf("block %s UTXO commitment is invalid - block "+
|
||||
"header indicates %s, but calculated value is %s", node.hash,
|
||||
node.utxoCommitment, calculatedMultisetHash)
|
||||
return ruleError(ErrBadUTXOCommitment, str)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// CheckConnectBlockTemplate fully validates that connecting the passed block to
|
||||
// the DAG does not violate any consensus rules, aside from the proof of
|
||||
// work requirement.
|
||||
@ -1023,21 +1109,17 @@ func (dag *BlockDAG) CheckConnectBlockTemplateNoLock(block *util.Block) error {
|
||||
|
||||
header := block.MsgBlock().Header
|
||||
|
||||
delay, err := dag.checkBlockSanity(block, flags)
|
||||
err := dag.checkBlockSanity(block, flags)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if delay != 0 {
|
||||
_, isDelayed := dag.shouldBlockBeDelayed(block)
|
||||
if isDelayed {
|
||||
return errors.Errorf("Block timestamp is too far in the future")
|
||||
}
|
||||
|
||||
parents, err := lookupParentNodes(block, dag)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
err = dag.checkBlockContext(block, parents, flags)
|
||||
err = dag.checkBlockContext(block, flags)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -1049,3 +1131,24 @@ func (dag *BlockDAG) CheckConnectBlockTemplateNoLock(block *util.Block) error {
|
||||
|
||||
return err
|
||||
}
|
||||
|
||||
func (dag *BlockDAG) checkDuplicateBlock(blockHash *daghash.Hash, flags BehaviorFlags) error {
|
||||
wasBlockStored := flags&BFWasStored == BFWasStored
|
||||
if dag.IsInDAG(blockHash) && !wasBlockStored {
|
||||
str := fmt.Sprintf("already have block %s", blockHash)
|
||||
return ruleError(ErrDuplicateBlock, str)
|
||||
}
|
||||
|
||||
// The block must not already exist as an orphan.
|
||||
if _, exists := dag.orphans[*blockHash]; exists {
|
||||
str := fmt.Sprintf("already have block (orphan) %s", blockHash)
|
||||
return ruleError(ErrDuplicateBlock, str)
|
||||
}
|
||||
|
||||
if dag.isKnownDelayedBlock(blockHash) {
|
||||
str := fmt.Sprintf("already have block (delayed) %s", blockHash)
|
||||
return ruleError(ErrDuplicateBlock, str)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
@ -172,16 +172,13 @@ func TestCheckBlockSanity(t *testing.T) {
|
||||
if len(block.Transactions()) < 3 {
|
||||
t.Fatalf("Too few transactions in block, expect at least 3, got %v", len(block.Transactions()))
|
||||
}
|
||||
delay, err := dag.checkBlockSanity(block, BFNone)
|
||||
err = dag.checkBlockSanity(block, BFNone)
|
||||
if err != nil {
|
||||
t.Errorf("CheckBlockSanity: %v", err)
|
||||
}
|
||||
if delay != 0 {
|
||||
t.Errorf("CheckBlockSanity: unexpected return %s delay", delay)
|
||||
}
|
||||
// Test with block with wrong transactions sorting order
|
||||
blockWithWrongTxOrder := util.NewBlock(&BlockWithWrongTxOrder)
|
||||
delay, err = dag.checkBlockSanity(blockWithWrongTxOrder, BFNone)
|
||||
err = dag.checkBlockSanity(blockWithWrongTxOrder, BFNone)
|
||||
if err == nil {
|
||||
t.Errorf("CheckBlockSanity: transactions disorder is not detected")
|
||||
}
|
||||
@ -192,9 +189,6 @@ func TestCheckBlockSanity(t *testing.T) {
|
||||
t.Errorf("CheckBlockSanity: wrong error returned, expect ErrTransactionsNotSorted, got"+
|
||||
" %v, err %s", ruleErr.ErrorCode, err)
|
||||
}
|
||||
if delay != 0 {
|
||||
t.Errorf("CheckBlockSanity: unexpected return %s delay", delay)
|
||||
}
|
||||
|
||||
var invalidParentsOrderBlock = domainmessage.MsgBlock{
|
||||
Header: domainmessage.BlockHeader{
|
||||
@ -463,7 +457,7 @@ func TestCheckBlockSanity(t *testing.T) {
|
||||
}
|
||||
|
||||
utilInvalidBlock := util.NewBlock(&invalidParentsOrderBlock)
|
||||
delay, err = dag.checkBlockSanity(utilInvalidBlock, BFNone)
|
||||
err = dag.checkBlockSanity(utilInvalidBlock, BFNone)
|
||||
if err == nil {
|
||||
t.Errorf("CheckBlockSanity: error is nil when it shouldn't be")
|
||||
}
|
||||
@ -473,21 +467,6 @@ func TestCheckBlockSanity(t *testing.T) {
|
||||
} else if rError.ErrorCode != ErrWrongParentsOrder {
|
||||
t.Errorf("CheckBlockSanity: Expected error was ErrWrongParentsOrder but got %v", err)
|
||||
}
|
||||
if delay != 0 {
|
||||
t.Errorf("CheckBlockSanity: unexpected return %s delay", delay)
|
||||
}
|
||||
|
||||
blockInTheFuture := Block100000
|
||||
expectedDelay := 10 * time.Second
|
||||
deviationTolerance := time.Duration(dag.TimestampDeviationTolerance) * dag.Params.TargetTimePerBlock
|
||||
blockInTheFuture.Header.Timestamp = dag.Now().Add(deviationTolerance + expectedDelay)
|
||||
delay, err = dag.checkBlockSanity(util.NewBlock(&blockInTheFuture), BFNoPoWCheck)
|
||||
if err != nil {
|
||||
t.Errorf("CheckBlockSanity: %v", err)
|
||||
}
|
||||
if delay != expectedDelay {
|
||||
t.Errorf("CheckBlockSanity: expected %s delay but got %s", expectedDelay, delay)
|
||||
}
|
||||
}
|
||||
|
||||
func TestPastMedianTime(t *testing.T) {
|
||||
|
@ -1,294 +0,0 @@
|
||||
// Copyright (c) 2016-2017 The btcsuite developers
|
||||
// Use of this source code is governed by an ISC
|
||||
// license that can be found in the LICENSE file.
|
||||
|
||||
package blockdag
|
||||
|
||||
import (
|
||||
"math"
|
||||
|
||||
"github.com/kaspanet/kaspad/dagconfig"
|
||||
)
|
||||
|
||||
const (
|
||||
// vbTopBits defines the bits to set in the version to signal that the
|
||||
// version bits scheme is being used.
|
||||
vbTopBits = 0x10000000
|
||||
|
||||
// vbTopMask is the bitmask to use to determine whether or not the
|
||||
// version bits scheme is in use.
|
||||
vbTopMask = 0xe0000000
|
||||
|
||||
// vbNumBits is the total number of bits available for use with the
|
||||
// version bits scheme.
|
||||
vbNumBits = 29
|
||||
|
||||
// unknownVerNumToCheck is the number of previous blocks to consider
|
||||
// when checking for a threshold of unknown block versions for the
|
||||
// purposes of warning the user.
|
||||
unknownVerNumToCheck = 100
|
||||
|
||||
// unknownVerWarnNum is the threshold of previous blocks that have an
|
||||
// unknown version to use for the purposes of warning the user.
|
||||
unknownVerWarnNum = unknownVerNumToCheck / 2
|
||||
)
|
||||
|
||||
// bitConditionChecker provides a thresholdConditionChecker which can be used to
|
||||
// test whether or not a specific bit is set when it's not supposed to be
|
||||
// according to the expected version based on the known deployments and the
|
||||
// current state of the DAG. This is useful for detecting and warning about
|
||||
// unknown rule activations.
|
||||
type bitConditionChecker struct {
|
||||
bit uint32
|
||||
dag *BlockDAG
|
||||
}
|
||||
|
||||
// Ensure the bitConditionChecker type implements the thresholdConditionChecker
|
||||
// interface.
|
||||
var _ thresholdConditionChecker = bitConditionChecker{}
|
||||
|
||||
// BeginTime returns the unix timestamp for the median block time after which
|
||||
// voting on a rule change starts (at the next window).
|
||||
//
|
||||
// Since this implementation checks for unknown rules, it returns 0 so the rule
|
||||
// is always treated as active.
|
||||
//
|
||||
// This is part of the thresholdConditionChecker interface implementation.
|
||||
func (c bitConditionChecker) BeginTime() uint64 {
|
||||
return 0
|
||||
}
|
||||
|
||||
// EndTime returns the unix timestamp for the median block time after which an
|
||||
// attempted rule change fails if it has not already been locked in or
|
||||
// activated.
|
||||
//
|
||||
// Since this implementation checks for unknown rules, it returns the maximum
|
||||
// possible timestamp so the rule is always treated as active.
|
||||
//
|
||||
// This is part of the thresholdConditionChecker interface implementation.
|
||||
func (c bitConditionChecker) EndTime() uint64 {
|
||||
return math.MaxUint64
|
||||
}
|
||||
|
||||
// RuleChangeActivationThreshold is the number of blocks for which the condition
|
||||
// must be true in order to lock in a rule change.
|
||||
//
|
||||
// This implementation returns the value defined by the DAG params the checker
|
||||
// is associated with.
|
||||
//
|
||||
// This is part of the thresholdConditionChecker interface implementation.
|
||||
func (c bitConditionChecker) RuleChangeActivationThreshold() uint64 {
|
||||
return c.dag.Params.RuleChangeActivationThreshold
|
||||
}
|
||||
|
||||
// MinerConfirmationWindow is the number of blocks in each threshold state
|
||||
// retarget window.
|
||||
//
|
||||
// This implementation returns the value defined by the DAG params the checker
|
||||
// is associated with.
|
||||
//
|
||||
// This is part of the thresholdConditionChecker interface implementation.
|
||||
func (c bitConditionChecker) MinerConfirmationWindow() uint64 {
|
||||
return c.dag.Params.MinerConfirmationWindow
|
||||
}
|
||||
|
||||
// Condition returns true when the specific bit associated with the checker is
|
||||
// set and it's not supposed to be according to the expected version based on
|
||||
// the known deployments and the current state of the DAG.
|
||||
//
|
||||
// This function MUST be called with the DAG state lock held (for writes).
|
||||
//
|
||||
// This is part of the thresholdConditionChecker interface implementation.
|
||||
func (c bitConditionChecker) Condition(node *blockNode) (bool, error) {
|
||||
conditionMask := uint32(1) << c.bit
|
||||
version := uint32(node.version)
|
||||
if version&vbTopMask != vbTopBits {
|
||||
return false, nil
|
||||
}
|
||||
if version&conditionMask == 0 {
|
||||
return false, nil
|
||||
}
|
||||
|
||||
expectedVersion, err := c.dag.calcNextBlockVersion(node.selectedParent)
|
||||
if err != nil {
|
||||
return false, err
|
||||
}
|
||||
return uint32(expectedVersion)&conditionMask == 0, nil
|
||||
}
|
||||
|
||||
// deploymentChecker provides a thresholdConditionChecker which can be used to
|
||||
// test a specific deployment rule. This is required for properly detecting
|
||||
// and activating consensus rule changes.
|
||||
type deploymentChecker struct {
|
||||
deployment *dagconfig.ConsensusDeployment
|
||||
dag *BlockDAG
|
||||
}
|
||||
|
||||
// Ensure the deploymentChecker type implements the thresholdConditionChecker
|
||||
// interface.
|
||||
var _ thresholdConditionChecker = deploymentChecker{}
|
||||
|
||||
// BeginTime returns the unix timestamp for the median block time after which
|
||||
// voting on a rule change starts (at the next window).
|
||||
//
|
||||
// This implementation returns the value defined by the specific deployment the
|
||||
// checker is associated with.
|
||||
//
|
||||
// This is part of the thresholdConditionChecker interface implementation.
|
||||
func (c deploymentChecker) BeginTime() uint64 {
|
||||
return c.deployment.StartTime
|
||||
}
|
||||
|
||||
// EndTime returns the unix timestamp for the median block time after which an
|
||||
// attempted rule change fails if it has not already been locked in or
|
||||
// activated.
|
||||
//
|
||||
// This implementation returns the value defined by the specific deployment the
|
||||
// checker is associated with.
|
||||
//
|
||||
// This is part of the thresholdConditionChecker interface implementation.
|
||||
func (c deploymentChecker) EndTime() uint64 {
|
||||
return c.deployment.ExpireTime
|
||||
}
|
||||
|
||||
// RuleChangeActivationThreshold is the number of blocks for which the condition
|
||||
// must be true in order to lock in a rule change.
|
||||
//
|
||||
// This implementation returns the value defined by the DAG params the checker
|
||||
// is associated with.
|
||||
//
|
||||
// This is part of the thresholdConditionChecker interface implementation.
|
||||
func (c deploymentChecker) RuleChangeActivationThreshold() uint64 {
|
||||
return c.dag.Params.RuleChangeActivationThreshold
|
||||
}
|
||||
|
||||
// MinerConfirmationWindow is the number of blocks in each threshold state
|
||||
// retarget window.
|
||||
//
|
||||
// This implementation returns the value defined by the DAG params the checker
|
||||
// is associated with.
|
||||
//
|
||||
// This is part of the thresholdConditionChecker interface implementation.
|
||||
func (c deploymentChecker) MinerConfirmationWindow() uint64 {
|
||||
return c.dag.Params.MinerConfirmationWindow
|
||||
}
|
||||
|
||||
// Condition returns true when the specific bit defined by the deployment
|
||||
// associated with the checker is set.
|
||||
//
|
||||
// This is part of the thresholdConditionChecker interface implementation.
|
||||
func (c deploymentChecker) Condition(node *blockNode) (bool, error) {
|
||||
conditionMask := uint32(1) << c.deployment.BitNumber
|
||||
version := uint32(node.version)
|
||||
return (version&vbTopMask == vbTopBits) && (version&conditionMask != 0),
|
||||
nil
|
||||
}
|
||||
|
||||
// calcNextBlockVersion calculates the expected version of the block after the
|
||||
// passed previous block node based on the state of started and locked in
|
||||
// rule change deployments.
|
||||
//
|
||||
// This function differs from the exported CalcNextBlockVersion in that the
|
||||
// exported version uses the selected tip as the previous block node
|
||||
// while this function accepts any block node.
|
||||
//
|
||||
// This function MUST be called with the DAG state lock held (for writes).
|
||||
func (dag *BlockDAG) calcNextBlockVersion(prevNode *blockNode) (int32, error) {
|
||||
// Set the appropriate bits for each actively defined rule deployment
|
||||
// that is either in the process of being voted on, or locked in for the
|
||||
// activation at the next threshold window change.
|
||||
expectedVersion := uint32(vbTopBits)
|
||||
for id := 0; id < len(dag.Params.Deployments); id++ {
|
||||
deployment := &dag.Params.Deployments[id]
|
||||
cache := &dag.deploymentCaches[id]
|
||||
checker := deploymentChecker{deployment: deployment, dag: dag}
|
||||
state, err := dag.thresholdState(prevNode, checker, cache)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
if state == ThresholdStarted || state == ThresholdLockedIn {
|
||||
expectedVersion |= uint32(1) << deployment.BitNumber
|
||||
}
|
||||
}
|
||||
return int32(expectedVersion), nil
|
||||
}
|
||||
|
||||
// CalcNextBlockVersion calculates the expected version of the block after the
|
||||
// end of the current selected tip based on the state of started and locked in
|
||||
// rule change deployments.
|
||||
//
|
||||
// This function is safe for concurrent access.
|
||||
func (dag *BlockDAG) CalcNextBlockVersion() (int32, error) {
|
||||
version, err := dag.calcNextBlockVersion(dag.selectedTip())
|
||||
return version, err
|
||||
}
|
||||
|
||||
// warnUnknownRuleActivations displays a warning when any unknown new rules are
|
||||
// either about to activate or have been activated. This will only happen once
|
||||
// when new rules have been activated and every block for those about to be
|
||||
// activated.
|
||||
//
|
||||
// This function MUST be called with the DAG state lock held (for writes)
|
||||
func (dag *BlockDAG) warnUnknownRuleActivations(node *blockNode) error {
|
||||
// Warn if any unknown new rules are either about to activate or have
|
||||
// already been activated.
|
||||
for bit := uint32(0); bit < vbNumBits; bit++ {
|
||||
checker := bitConditionChecker{bit: bit, dag: dag}
|
||||
cache := &dag.warningCaches[bit]
|
||||
state, err := dag.thresholdState(node.selectedParent, checker, cache)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
switch state {
|
||||
case ThresholdActive:
|
||||
if !dag.unknownRulesWarned {
|
||||
log.Warnf("Unknown new rules activated (bit %d)",
|
||||
bit)
|
||||
dag.unknownRulesWarned = true
|
||||
}
|
||||
|
||||
case ThresholdLockedIn:
|
||||
window := checker.MinerConfirmationWindow()
|
||||
activationBlueScore := window - (node.blueScore % window)
|
||||
log.Warnf("Unknown new rules are about to activate in "+
|
||||
"%d blueScore (bit %d)", activationBlueScore, bit)
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// warnUnknownVersions logs a warning if a high enough percentage of the last
|
||||
// blocks have unexpected versions.
|
||||
//
|
||||
// This function MUST be called with the DAG state lock held (for writes)
|
||||
func (dag *BlockDAG) warnUnknownVersions(node *blockNode) error {
|
||||
// Nothing to do if already warned.
|
||||
if dag.unknownVersionsWarned {
|
||||
return nil
|
||||
}
|
||||
|
||||
// Warn if enough previous blocks have unexpected versions.
|
||||
numUpgraded := uint32(0)
|
||||
for i := uint32(0); i < unknownVerNumToCheck && node != nil; i++ {
|
||||
expectedVersion, err := dag.calcNextBlockVersion(node.selectedParent)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if (node.version & ^expectedVersion) != 0 {
|
||||
|
||||
numUpgraded++
|
||||
}
|
||||
|
||||
node = node.selectedParent
|
||||
}
|
||||
if numUpgraded > unknownVerWarnNum {
|
||||
log.Warn("Unknown block versions are being mined, so new " +
|
||||
"rules might be in effect. Are you running the " +
|
||||
"latest version of the software?")
|
||||
dag.unknownVersionsWarned = true
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
@ -6,7 +6,6 @@ package dagconfig
|
||||
|
||||
import (
|
||||
"github.com/kaspanet/kaspad/util/network"
|
||||
"math"
|
||||
"math/big"
|
||||
"time"
|
||||
|
||||
@ -161,7 +160,6 @@ type Params struct {
|
||||
// on.
|
||||
RuleChangeActivationThreshold uint64
|
||||
MinerConfirmationWindow uint64
|
||||
Deployments [DefinedDeployments]ConsensusDeployment
|
||||
|
||||
// Mempool parameters
|
||||
RelayNonStdTxs bool
|
||||
@ -215,13 +213,6 @@ var MainnetParams = Params{
|
||||
// target proof of work timespan / target proof of work spacing
|
||||
RuleChangeActivationThreshold: 1916, // 95% of MinerConfirmationWindow
|
||||
MinerConfirmationWindow: 2016, //
|
||||
Deployments: [DefinedDeployments]ConsensusDeployment{
|
||||
DeploymentTestDummy: {
|
||||
BitNumber: 28,
|
||||
StartTime: 1199145601000, // January 1, 2008 UTC
|
||||
ExpireTime: 1230767999000, // December 31, 2008 UTC
|
||||
},
|
||||
},
|
||||
|
||||
// Mempool parameters
|
||||
RelayNonStdTxs: false,
|
||||
@ -270,13 +261,6 @@ var RegressionNetParams = Params{
|
||||
// target proof of work timespan / target proof of work spacing
|
||||
RuleChangeActivationThreshold: 108, // 75% of MinerConfirmationWindow
|
||||
MinerConfirmationWindow: 144,
|
||||
Deployments: [DefinedDeployments]ConsensusDeployment{
|
||||
DeploymentTestDummy: {
|
||||
BitNumber: 28,
|
||||
StartTime: 0, // Always available for vote
|
||||
ExpireTime: math.MaxInt64, // Never expires
|
||||
},
|
||||
},
|
||||
|
||||
// Mempool parameters
|
||||
RelayNonStdTxs: true,
|
||||
@ -323,13 +307,6 @@ var TestnetParams = Params{
|
||||
// target proof of work timespan / target proof of work spacing
|
||||
RuleChangeActivationThreshold: 1512, // 75% of MinerConfirmationWindow
|
||||
MinerConfirmationWindow: 2016,
|
||||
Deployments: [DefinedDeployments]ConsensusDeployment{
|
||||
DeploymentTestDummy: {
|
||||
BitNumber: 28,
|
||||
StartTime: 1199145601000, // January 1, 2008 UTC
|
||||
ExpireTime: 1230767999000, // December 31, 2008 UTC
|
||||
},
|
||||
},
|
||||
|
||||
// Mempool parameters
|
||||
RelayNonStdTxs: true,
|
||||
@ -382,13 +359,6 @@ var SimnetParams = Params{
|
||||
// target proof of work timespan / target proof of work spacing
|
||||
RuleChangeActivationThreshold: 75, // 75% of MinerConfirmationWindow
|
||||
MinerConfirmationWindow: 100,
|
||||
Deployments: [DefinedDeployments]ConsensusDeployment{
|
||||
DeploymentTestDummy: {
|
||||
BitNumber: 28,
|
||||
StartTime: 0, // Always available for vote
|
||||
ExpireTime: math.MaxInt64, // Never expires
|
||||
},
|
||||
},
|
||||
|
||||
// Mempool parameters
|
||||
RelayNonStdTxs: true,
|
||||
@ -433,13 +403,6 @@ var DevnetParams = Params{
|
||||
// target proof of work timespan / target proof of work spacing
|
||||
RuleChangeActivationThreshold: 1512, // 75% of MinerConfirmationWindow
|
||||
MinerConfirmationWindow: 2016,
|
||||
Deployments: [DefinedDeployments]ConsensusDeployment{
|
||||
DeploymentTestDummy: {
|
||||
BitNumber: 28,
|
||||
StartTime: 1199145601000, // January 1, 2008 UTC
|
||||
ExpireTime: 1230767999000, // December 31, 2008 UTC
|
||||
},
|
||||
},
|
||||
|
||||
// Mempool parameters
|
||||
RelayNonStdTxs: true,
|
||||
|
@ -53,12 +53,6 @@ type Config struct {
|
||||
// utxo set.
|
||||
CalcSequenceLockNoLock func(*util.Tx, blockdag.UTXOSet) (*blockdag.SequenceLock, error)
|
||||
|
||||
// IsDeploymentActive returns true if the target deploymentID is
|
||||
// active, and false otherwise. The mempool uses this function to gauge
|
||||
// if transactions using new to be soft-forked rules should be allowed
|
||||
// into the mempool or not.
|
||||
IsDeploymentActive func(deploymentID uint32) (bool, error)
|
||||
|
||||
// SigCache defines a signature cache to use.
|
||||
SigCache *txscript.SigCache
|
||||
|
||||
|
@ -1,13 +1,8 @@
|
||||
package rpc
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"github.com/kaspanet/kaspad/blockdag"
|
||||
"github.com/kaspanet/kaspad/dagconfig"
|
||||
"github.com/kaspanet/kaspad/rpc/model"
|
||||
"github.com/kaspanet/kaspad/util/daghash"
|
||||
"github.com/pkg/errors"
|
||||
"strings"
|
||||
)
|
||||
|
||||
// handleGetBlockDAGInfo implements the getBlockDagInfo command.
|
||||
@ -28,72 +23,5 @@ func handleGetBlockDAGInfo(s *Server, cmd interface{}, closeChan <-chan struct{}
|
||||
Bip9SoftForks: make(map[string]*model.Bip9SoftForkDescription),
|
||||
}
|
||||
|
||||
// Finally, query the BIP0009 version bits state for all currently
|
||||
// defined BIP0009 soft-fork deployments.
|
||||
for deployment, deploymentDetails := range params.Deployments {
|
||||
// Map the integer deployment ID into a human readable
|
||||
// fork-name.
|
||||
var forkName string
|
||||
switch deployment {
|
||||
case dagconfig.DeploymentTestDummy:
|
||||
forkName = "dummy"
|
||||
|
||||
default:
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCInternal.Code,
|
||||
Message: fmt.Sprintf("Unknown deployment %d "+
|
||||
"detected", deployment),
|
||||
}
|
||||
}
|
||||
|
||||
// Query the dag for the current status of the deployment as
|
||||
// identified by its deployment ID.
|
||||
deploymentStatus, err := dag.ThresholdState(uint32(deployment))
|
||||
if err != nil {
|
||||
context := "Failed to obtain deployment status"
|
||||
return nil, internalRPCError(err.Error(), context)
|
||||
}
|
||||
|
||||
// Attempt to convert the current deployment status into a
|
||||
// human readable string. If the status is unrecognized, then a
|
||||
// non-nil error is returned.
|
||||
statusString, err := softForkStatus(deploymentStatus)
|
||||
if err != nil {
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCInternal.Code,
|
||||
Message: fmt.Sprintf("unknown deployment status: %d",
|
||||
deploymentStatus),
|
||||
}
|
||||
}
|
||||
|
||||
// Finally, populate the soft-fork description with all the
|
||||
// information gathered above.
|
||||
dagInfo.Bip9SoftForks[forkName] = &model.Bip9SoftForkDescription{
|
||||
Status: strings.ToLower(statusString),
|
||||
Bit: deploymentDetails.BitNumber,
|
||||
StartTime: int64(deploymentDetails.StartTime),
|
||||
Timeout: int64(deploymentDetails.ExpireTime),
|
||||
}
|
||||
}
|
||||
|
||||
return dagInfo, nil
|
||||
}
|
||||
|
||||
// softForkStatus converts a ThresholdState state into a human readable string
|
||||
// corresponding to the particular state.
|
||||
func softForkStatus(state blockdag.ThresholdState) (string, error) {
|
||||
switch state {
|
||||
case blockdag.ThresholdDefined:
|
||||
return "defined", nil
|
||||
case blockdag.ThresholdStarted:
|
||||
return "started", nil
|
||||
case blockdag.ThresholdLockedIn:
|
||||
return "lockedin", nil
|
||||
case blockdag.ThresholdActive:
|
||||
return "active", nil
|
||||
case blockdag.ThresholdFailed:
|
||||
return "failed", nil
|
||||
default:
|
||||
return "", errors.Errorf("unknown deployment state: %s", state)
|
||||
}
|
||||
}
|
||||
|
Loading…
x
Reference in New Issue
Block a user