[NOD-208] Make block reward maturity use the same mechanism as confirmations (#327)

* [NOD-208] Added blockBlueScore to UTXOEntry.

* [NOD-208] Added blueBlockScore to NewUTXOEntry.

* [NOD-208] Fixed compilation errors in policy, utxoset, and dag tests.

* [NOD-208] Changed validateBlockRewardMaturity and CheckTransactionInputsAndCalulateFee to use blueScore.

* [NOD-208] Changed CalcBlockSubsidy to use blueScore.

* [NOD-208] Changed SequenceLockActive to use blueScore.

* [NOD-208] Removed ExtractCoinbaseHeight.

* [NOD-208] Removed reference to block height in ensureNoDuplicateTx.

* [NOD-208] Changed IsFinalizedTransaction to use blueScore.

* [NOD-208] Fixed merge errors.

* [NOD-208] Made UTXOEntry serialization use blueScore.

* [NOD-208] Changed CalcPriority and calcInputValueAge to use blueScore.

* [NOD-208] Changed calcSequenceLock to use blueScore.

* [NOD-208] Removed blockChainHeight from UTXOEntry.

* [NOD-208] Fixed compilation errors in feeEstimator. Fixed a bug in the test pool hardness.

* [NOD-208] Fixed oldestChainBlockWithBlueScoreGreaterThan not handling an extreme case.

* [NOD-208] Fixed TestDiffFromTx.

* [NOD-208] Got rid of priority and support of free transactions.

* [NOD-208] Fixed TestProcessTransaction.

* [NOD-208] Fixed TestTxFeePrioHeap.

* [NOD-208] Fixed TestAddrIndex and TestFeeEstimatorCfg.

* [NOD-208] Removed unused rateLimit parameter from ProcessTransaction.

* [NOD-208] Fixed tests that rely on CreateTxChain.

* [NOD-208] Fixed tests that rely on CreateSignedTxForSubnetwork.

* [NOD-208] Fixed TestFetchTransaction.

* [NOD-208] Fixed TestHandleNewBlock. Fixed HandleNewBlock erroneously processing fee transactions.

* [NOD-208] Fixed TestTxIndexConnectBlock.

* [NOD-208] Removed the use of Height() from the fee estimator.

* [NOD-208] Removed unused methods from rpcwebsocket.go.

* [NOD-208] Removed Height from util.Block.

* [NOD-208] Removed ErrForkTooOld. It doesn't make sense in a DAG.

* [NOD-208] Made blockHeap use blueScore instead of height.

* [NOD-208] Removed fee estimator.

* [NOD-208] Removed DAG.Height.

* [NOD-208] Made TestAncestorErrors test chainHeight instead of height.

* [NOD-208] Fixed a couple of comments that were still speaking about block height.

* [NOD-208] Replaced all uses of HighestTipHash with SelectedTipHash.

* [NOD-208] Remove blockNode highest and some remaining erroneous uses of height.

* [NOD-208] Fixed a couple of comments. Fixed outPoint -> outpoint merge error.

* [NOD-208] Fixed a couple more comments.

* [NOD-208] Used calcMinRequiredTxRelayFee instead of DefaultMinRelayTxFee for mempool tests.

* [NOD-208] Renamed mempool Config BestHeight to DAGChainHeight.

* [NOD-208] Fixed a bug in oldestChainBlockWithBlueScoreGreaterThan. Made calcSequenceLock use the node's selected parent chain rather than the virtual block's.

* [NOD-208] Removed chainHeight from blockNode String().
Renamed checkpointsByHeight to checkpointsByChainHeight and prevCheckpointHeight to prevCheckpointChainHeight.
Removed reference to chainHeight in blockIndexKey.
Fixed comments in dagio.go.

* [NOD-208] Removed indexers/blocklogger.go, as no one was using it.

* [NOD-208] Made blocklogger.go log blueScore instead of height.

* [NOD-208] Fixed typo.

* [NOD-208] Fixed comments, did minor renaming.

* [NOD-208] Made a "common sense" wrapper around sort.Search.

* [NOD-208] Fixed comment in SearchSlice.
This commit is contained in:
stasatdaglabs 2019-06-16 14:12:02 +03:00 committed by Svarog
parent 76f5619de7
commit ffd886498a
58 changed files with 648 additions and 2899 deletions

View File

@ -28,15 +28,9 @@ func (dag *BlockDAG) maybeAcceptBlock(block *util.Block, flags BehaviorFlags) er
return err
}
bluestParent := parents.bluest()
blockHeight := uint64(0)
if !block.IsGenesis() {
blockHeight = parents.maxHeight() + 1
}
block.SetHeight(blockHeight)
// The block must pass all of the validation rules which depend on the
// position of the block within the block DAG.
bluestParent := parents.bluest()
err = dag.checkBlockContext(block, parents, bluestParent, flags)
if err != nil {
return err
@ -63,6 +57,14 @@ func (dag *BlockDAG) maybeAcceptBlock(block *util.Block, flags BehaviorFlags) er
newNode := newBlockNode(blockHeader, parents, dag.dagParams.K)
newNode.status = statusDataStored
// Make sure that all the block's transactions are finalized
fastAdd := flags&BFFastAdd == BFFastAdd
if !fastAdd {
if err := dag.validateAllTxsFinalized(block, newNode, bluestParent); err != nil {
return err
}
}
dag.index.AddNode(newNode)
err = dag.index.flushToDB()
if err != nil {

View File

@ -28,22 +28,22 @@ func (h *baseHeap) Pop() interface{} {
type upHeap struct{ baseHeap }
func (h upHeap) Less(i, j int) bool {
if h.baseHeap[i].height == h.baseHeap[j].height {
if h.baseHeap[i].blueScore == h.baseHeap[j].blueScore {
return daghash.HashToBig(h.baseHeap[i].hash).Cmp(daghash.HashToBig(h.baseHeap[j].hash)) < 0
}
return h.baseHeap[i].height < h.baseHeap[j].height
return h.baseHeap[i].blueScore < h.baseHeap[j].blueScore
}
// downHeap extends baseHeap to include Less operation that traverses from top to bottom
type downHeap struct{ baseHeap }
func (h downHeap) Less(i, j int) bool {
if h.baseHeap[i].height == h.baseHeap[j].height {
if h.baseHeap[i].blueScore == h.baseHeap[j].blueScore {
return daghash.HashToBig(h.baseHeap[i].hash).Cmp(daghash.HashToBig(h.baseHeap[j].hash)) > 0
}
return h.baseHeap[i].height > h.baseHeap[j].height
return h.baseHeap[i].blueScore > h.baseHeap[j].blueScore
}
// blockHeap represents a mutable heap of Blocks, sorted by their height

View File

@ -13,7 +13,7 @@ import (
func TestAncestorErrors(t *testing.T) {
node := newTestNode(newSet(), int32(0x10000000), 0, time.Unix(0, 0), dagconfig.MainNetParams.K)
node.height = 2
node.chainHeight = 2
ancestor := node.SelectedAncestor(3)
if ancestor != nil {
t.Errorf("TestAncestorErrors: Ancestor() unexpectedly returned a node. Expected: <nil>")

View File

@ -5,7 +5,6 @@
package blockdag
import (
"fmt"
"sort"
"time"
@ -256,7 +255,7 @@ func (node *blockNode) finalityScore() uint64 {
return node.blueScore / FinalityInterval
}
// String returns a string that contains the block hash and height.
// String returns a string that contains the block hash.
func (node blockNode) String() string {
return fmt.Sprintf("%s (%d)", node.hash, node.height)
return node.hash.String()
}

View File

@ -34,19 +34,6 @@ func (bs blockSet) maxHeight() uint64 {
return maxHeight
}
func (bs blockSet) highest() *blockNode {
var highest *blockNode
for _, node := range bs {
if highest == nil ||
highest.height < node.height ||
(highest.height == node.height && daghash.Less(node.hash, highest.hash)) {
highest = node
}
}
return highest
}
// add adds a block to this BlockSet
func (bs blockSet) add(block *blockNode) {
bs[*block.hash] = block

View File

@ -35,47 +35,6 @@ func TestHashes(t *testing.T) {
t.Errorf("TestHashes: hashes order is %s but expected %s", hashes, expected)
}
}
func TestBlockSetHighest(t *testing.T) {
node1 := &blockNode{hash: &daghash.Hash{10}, height: 1}
node2a := &blockNode{hash: &daghash.Hash{20}, height: 2}
node2b := &blockNode{hash: &daghash.Hash{21}, height: 2}
node3 := &blockNode{hash: &daghash.Hash{30}, height: 3}
tests := []struct {
name string
set blockSet
expectedHighest *blockNode
}{
{
name: "empty set",
set: setFromSlice(),
expectedHighest: nil,
},
{
name: "set with one member",
set: setFromSlice(node1),
expectedHighest: node1,
},
{
name: "same-height highest members in set",
set: setFromSlice(node2b, node1, node2a),
expectedHighest: node2a,
},
{
name: "typical set",
set: setFromSlice(node2b, node3, node1, node2a),
expectedHighest: node3,
},
}
for _, test := range tests {
highest := test.set.highest()
if highest != test.expectedHighest {
t.Errorf("blockSet.highest: unexpected value in test '%s'. "+
"Expected: %v, got: %v", test.name, test.expectedHighest, highest)
}
}
}
func TestBlockSetSubtract(t *testing.T) {
node1 := &blockNode{hash: &daghash.Hash{10}}

View File

@ -64,16 +64,16 @@ func (dag *BlockDAG) LatestCheckpoint() *dagconfig.Checkpoint {
return &dag.checkpoints[len(dag.checkpoints)-1]
}
// verifyCheckpoint returns whether the passed block height and hash combination
// verifyCheckpoint returns whether the passed block chain height and hash combination
// match the checkpoint data. It also returns true if there is no checkpoint
// data for the passed block height.
func (dag *BlockDAG) verifyCheckpoint(height uint64, hash *daghash.Hash) bool {
// data for the passed block chain height.
func (dag *BlockDAG) verifyCheckpoint(chainHeight uint64, hash *daghash.Hash) bool {
if !dag.HasCheckpoints() {
return true
}
// Nothing to check if there is no checkpoint data for the block height.
checkpoint, exists := dag.checkpointsByHeight[height]
// Nothing to check if there is no checkpoint data for the block chainHeight.
checkpoint, exists := dag.checkpointsByChainHeight[chainHeight]
if !exists {
return true
}
@ -82,7 +82,7 @@ func (dag *BlockDAG) verifyCheckpoint(height uint64, hash *daghash.Hash) bool {
return false
}
log.Infof("Verified checkpoint at height %d/block %s", checkpoint.Height,
log.Infof("Verified checkpoint at chainHeight %d/block %s", checkpoint.ChainHeight,
checkpoint.Hash)
return true
}
@ -136,10 +136,10 @@ func (dag *BlockDAG) findPreviousCheckpoint() (*blockNode, error) {
return dag.checkpointNode, nil
}
// When there is a next checkpoint and the height of the current best
// chain does not exceed it, the current checkpoint lockin is still
// the latest known checkpoint.
if dag.selectedTip().height < dag.nextCheckpoint.Height {
// When there is a next checkpoint and the chain height of the current
// selected tip of the DAG does not exceed it, the current checkpoint
// lockin is still the latest known checkpoint.
if dag.selectedTip().chainHeight < dag.nextCheckpoint.ChainHeight {
return dag.checkpointNode, nil
}
@ -216,19 +216,19 @@ func (dag *BlockDAG) IsCheckpointCandidate(block *util.Block) (bool, error) {
return false, nil
}
// Ensure the height of the passed block and the entry for the block in
// the main chain match. This should always be the case unless the
// Ensure the chain height of the passed block and the entry for the block
// in the DAG match. This should always be the case unless the
// caller provided an invalid block.
if node.height != block.Height() {
return false, fmt.Errorf("passed block height of %d does not "+
"match the main chain height of %d", block.Height(),
node.height)
if node.chainHeight != block.ChainHeight() {
return false, fmt.Errorf("passed block chain height of %d does not "+
"match the its height in the DAG: %d", block.ChainHeight(),
node.chainHeight)
}
// A checkpoint must be at least CheckpointConfirmations blocks
// before the end of the main chain.
dagHeight := dag.selectedTip().height
if node.height > (dagHeight - CheckpointConfirmations) {
dagChainHeight := dag.selectedTip().chainHeight
if node.chainHeight > (dagChainHeight - CheckpointConfirmations) {
return false, nil
}

View File

@ -75,7 +75,6 @@ func loadBlocks(filename string) (blocks []*util.Block, err error) {
if err != nil {
return
}
block.SetHeight(height)
blocks = append(blocks, block)
}

View File

@ -63,14 +63,14 @@ type BlockDAG struct {
// The following fields are set when the instance is created and can't
// be changed afterwards, so there is no need to protect them with a
// separate mutex.
checkpoints []dagconfig.Checkpoint
checkpointsByHeight map[uint64]*dagconfig.Checkpoint
db database.DB
dagParams *dagconfig.Params
timeSource MedianTimeSource
sigCache *txscript.SigCache
indexManager IndexManager
genesis *blockNode
checkpoints []dagconfig.Checkpoint
checkpointsByChainHeight map[uint64]*dagconfig.Checkpoint
db database.DB
dagParams *dagconfig.Params
timeSource MedianTimeSource
sigCache *txscript.SigCache
indexManager IndexManager
genesis *blockNode
// The following fields are calculated based upon the provided chain
// parameters. They are also set when the instance is created and
@ -329,14 +329,14 @@ func (dag *BlockDAG) addOrphanBlock(block *util.Block) {
}
// SequenceLock represents the converted relative lock-time in seconds, and
// absolute block-chain-height for a transaction input's relative lock-times.
// 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
// 'BlockChainHeight' has been reached.
// 'BlockBlueScore' has been reached.
type SequenceLock struct {
Seconds int64
BlockChainHeight int64
Seconds int64
BlockBlueScore int64
}
// CalcSequenceLock computes a relative lock-time SequenceLock for the passed
@ -369,7 +369,7 @@ func (dag *BlockDAG) calcSequenceLock(node *blockNode, utxoSet UTXOSet, tx *util
// 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{Seconds: -1, BlockChainHeight: -1}
sequenceLock := &SequenceLock{Seconds: -1, BlockBlueScore: -1}
// Sequence locks don't apply to block reward transactions Therefore, we
// return sequence lock values of -1 indicating that this transaction
@ -378,10 +378,6 @@ func (dag *BlockDAG) calcSequenceLock(node *blockNode, utxoSet UTXOSet, tx *util
return sequenceLock, nil
}
// Grab the next height from the PoV of the passed blockNode to use for
// inputs present in the mempool.
nextChainHeight := node.chainHeight + 1
mTx := tx.MsgTx()
for txInIndex, txIn := range mTx.TxIn {
entry, ok := utxoSet.Get(txIn.PreviousOutpoint)
@ -393,12 +389,12 @@ func (dag *BlockDAG) calcSequenceLock(node *blockNode, utxoSet UTXOSet, tx *util
return sequenceLock, ruleError(ErrMissingTxOut, str)
}
// If the input chain-height is set to the mempool height, then we
// 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.
inputChainHeight := entry.BlockChainHeight()
inputBlueScore := entry.BlockBlueScore()
if entry.IsUnmined() {
inputChainHeight = nextChainHeight
inputBlueScore = dag.virtual.blueScore
}
// Given a sequence number, we apply the relative time lock
@ -416,14 +412,13 @@ func (dag *BlockDAG) calcSequenceLock(node *blockNode, utxoSet UTXOSet, tx *util
// 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 included within so we can
// which this input was accepted within so we can
// compute the past median time for the block prior to
// the one which included this referenced output.
prevInputChainHeight := inputChainHeight - 1
if prevInputChainHeight < 0 {
prevInputChainHeight = 0
// the one which accepted this referenced output.
blockNode := node
for blockNode.selectedParent.blueScore > inputBlueScore {
blockNode = blockNode.selectedParent
}
blockNode := node.SelectedAncestor(prevInputChainHeight)
medianTime := blockNode.PastMedianTime()
// Time based relative time-locks as defined by BIP 68
@ -440,12 +435,12 @@ func (dag *BlockDAG) calcSequenceLock(node *blockNode, utxoSet UTXOSet, tx *util
default:
// The relative lock-time for this input is expressed
// in blocks so we calculate the relative offset from
// the input's height as its converted absolute
// 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.
blockChainHeight := int64(inputChainHeight) + relativeLock - 1
if blockChainHeight > sequenceLock.BlockChainHeight {
sequenceLock.BlockChainHeight = blockChainHeight
blockBlueScore := int64(inputBlueScore) + relativeLock - 1
if blockBlueScore > sequenceLock.BlockBlueScore {
sequenceLock.BlockBlueScore = blockBlueScore
}
}
}
@ -973,7 +968,7 @@ func (node *blockNode) applyBlueBlocks(selectedParentUTXO UTXOSet, blueBlocks []
if isSelectedParent {
isAccepted = true
} else {
isAccepted, err = pastUTXO.AddTx(tx.MsgTx(), node.height)
isAccepted, err = pastUTXO.AddTx(tx.MsgTx(), node.blueScore)
if err != nil {
return nil, nil, err
}
@ -1103,10 +1098,10 @@ func updateTipsUTXO(dag *BlockDAG, virtualUTXO UTXOSet) error {
//
// This function MUST be called with the DAG state lock held (for reads).
func (dag *BlockDAG) isCurrent() bool {
// Not current if the virtual's selected tip height is less than
// Not current if the virtual's selected tip chain height is less than
// the latest known good checkpoint (when checkpoints are enabled).
checkpoint := dag.LatestCheckpoint()
if checkpoint != nil && dag.selectedTip().height < checkpoint.Height {
if checkpoint != nil && dag.selectedTip().chainHeight < checkpoint.ChainHeight {
return false
}
@ -1192,6 +1187,16 @@ func (dag *BlockDAG) GetUTXOEntry(outpoint wire.Outpoint) (*UTXOEntry, bool) {
return dag.virtual.utxoSet.get(outpoint)
}
// BlueScoreByBlockHash returns the blue score of a block with the given hash.
func (dag *BlockDAG) BlueScoreByBlockHash(hash *daghash.Hash) (uint64, error) {
node := dag.index.LookupNode(hash)
if node == nil {
return 0, fmt.Errorf("block %s is unknown", hash)
}
return node.blueScore, nil
}
// BlockConfirmationsByHash returns the confirmations number for a block with the
// given hash. See blockConfirmations for further details.
//
@ -1288,12 +1293,15 @@ func (dag *BlockDAG) acceptingBlock(node *blockNode) (*blockNode, error) {
}
// oldestChainBlockWithBlueScoreGreaterThan finds the oldest chain block with a blue score
// greater than blueScore
// greater than blueScore. If no such block exists, this method returns nil
func (dag *BlockDAG) oldestChainBlockWithBlueScoreGreaterThan(blueScore uint64) *blockNode {
chainBlockIndex := sort.Search(len(dag.virtual.selectedPathChainSlice), func(i int) bool {
chainBlockIndex, ok := util.SearchSlice(len(dag.virtual.selectedPathChainSlice), func(i int) bool {
selectedPathNode := dag.virtual.selectedPathChainSlice[i]
return selectedPathNode.blueScore > blueScore
})
if !ok {
return nil
}
return dag.virtual.selectedPathChainSlice[chainBlockIndex]
}
@ -1304,17 +1312,17 @@ func (dag *BlockDAG) IsInSelectedPathChain(blockHash *daghash.Hash) bool {
return dag.virtual.selectedPathChainSet.containsHash(blockHash)
}
// Height returns the height of the highest tip in the DAG
func (dag *BlockDAG) Height() uint64 {
return dag.virtual.tips().maxHeight()
}
// ChainHeight return the chain-height of the selected tip. In other words - it returns
// the length of the dag's selected-parent chain
func (dag *BlockDAG) ChainHeight() uint64 {
return dag.selectedTip().chainHeight
}
// VirtualBlueScore returns the blue score of the current virtual block
func (dag *BlockDAG) VirtualBlueScore() uint64 {
return dag.virtual.blueScore
}
// BlockCount returns the number of blocks in the DAG
func (dag *BlockDAG) BlockCount() uint64 {
return dag.blockCount
@ -1325,12 +1333,6 @@ func (dag *BlockDAG) TipHashes() []*daghash.Hash {
return dag.virtual.tips().hashes()
}
// HighestTipHash returns the hash of the highest tip.
// This function is a placeholder for places that aren't DAG-compatible, and it's needed to be removed in the future
func (dag *BlockDAG) HighestTipHash() *daghash.Hash {
return dag.virtual.tips().highest().hash
}
// 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()
@ -1440,18 +1442,18 @@ func (dag *BlockDAG) blockLocator(node *blockNode) BlockLocator {
return locator
}
// BlockHeightByHash returns the height of the block with the given hash in the
// DAG.
// BlockChainHeightByHash returns the chain height of the block with the given
// hash in the DAG.
//
// This function is safe for concurrent access.
func (dag *BlockDAG) BlockHeightByHash(hash *daghash.Hash) (uint64, error) {
func (dag *BlockDAG) BlockChainHeightByHash(hash *daghash.Hash) (uint64, error) {
node := dag.index.LookupNode(hash)
if node == nil {
str := fmt.Sprintf("block %s is not in the DAG", hash)
return 0, errNotInDAG(str)
}
return node.height, nil
return node.chainHeight, nil
}
// ChildHashesByHash returns the child hashes of the block with the given hash in the
@ -1832,21 +1834,21 @@ func New(config *Config) (*BlockDAG, error) {
return nil, AssertError("BlockDAG.New timesource is nil")
}
// Generate a checkpoint by height map from the provided checkpoints
// and assert the provided checkpoints are sorted by height as required.
var checkpointsByHeight map[uint64]*dagconfig.Checkpoint
var prevCheckpointHeight uint64
// Generate a checkpoint by chain height map from the provided checkpoints
// and assert the provided checkpoints are sorted by chain height as required.
var checkpointsByChainHeight map[uint64]*dagconfig.Checkpoint
var prevCheckpointChainHeight uint64
if len(config.Checkpoints) > 0 {
checkpointsByHeight = make(map[uint64]*dagconfig.Checkpoint)
checkpointsByChainHeight = make(map[uint64]*dagconfig.Checkpoint)
for i := range config.Checkpoints {
checkpoint := &config.Checkpoints[i]
if checkpoint.Height <= prevCheckpointHeight {
if checkpoint.ChainHeight <= prevCheckpointChainHeight {
return nil, AssertError("blockdag.New " +
"checkpoints are not sorted by height")
"checkpoints are not sorted by chain height")
}
checkpointsByHeight[checkpoint.Height] = checkpoint
prevCheckpointHeight = checkpoint.Height
checkpointsByChainHeight[checkpoint.ChainHeight] = checkpoint
prevCheckpointChainHeight = checkpoint.ChainHeight
}
}
@ -1856,25 +1858,25 @@ func New(config *Config) (*BlockDAG, error) {
adjustmentFactor := params.RetargetAdjustmentFactor
index := newBlockIndex(config.DB, params)
dag := BlockDAG{
checkpoints: config.Checkpoints,
checkpointsByHeight: checkpointsByHeight,
db: config.DB,
dagParams: params,
timeSource: config.TimeSource,
sigCache: config.SigCache,
indexManager: config.IndexManager,
minRetargetTimespan: targetTimespan / adjustmentFactor,
maxRetargetTimespan: targetTimespan * adjustmentFactor,
blocksPerRetarget: uint64(targetTimespan / targetTimePerBlock),
index: index,
virtual: newVirtualBlock(nil, params.K),
orphans: make(map[daghash.Hash]*orphanBlock),
prevOrphans: make(map[daghash.Hash][]*orphanBlock),
warningCaches: newThresholdCaches(vbNumBits),
deploymentCaches: newThresholdCaches(dagconfig.DefinedDeployments),
blockCount: 0,
SubnetworkStore: newSubnetworkStore(config.DB),
subnetworkID: config.SubnetworkID,
checkpoints: config.Checkpoints,
checkpointsByChainHeight: checkpointsByChainHeight,
db: config.DB,
dagParams: params,
timeSource: config.TimeSource,
sigCache: config.SigCache,
indexManager: config.IndexManager,
minRetargetTimespan: targetTimespan / adjustmentFactor,
maxRetargetTimespan: targetTimespan * adjustmentFactor,
blocksPerRetarget: uint64(targetTimespan / targetTimePerBlock),
index: index,
virtual: newVirtualBlock(nil, params.K),
orphans: make(map[daghash.Hash]*orphanBlock),
prevOrphans: make(map[daghash.Hash][]*orphanBlock),
warningCaches: newThresholdCaches(vbNumBits),
deploymentCaches: newThresholdCaches(dagconfig.DefinedDeployments),
blockCount: 0,
SubnetworkStore: newSubnetworkStore(config.DB),
subnetworkID: config.SubnetworkID,
}
dag.utxoDiffStore = newUTXODiffStore(&dag)
@ -1929,8 +1931,8 @@ func New(config *Config) (*BlockDAG, error) {
}
selectedTip := dag.selectedTip()
log.Infof("DAG state (height %d, hash %s)",
selectedTip.height, selectedTip.hash)
log.Infof("DAG state (chain height %d, hash %s)",
selectedTip.chainHeight, selectedTip.hash)
return &dag, nil
}

View File

@ -242,7 +242,8 @@ func TestCalcSequenceLock(t *testing.T) {
msgTx := wire.NewNativeMsgTx(wire.TxVersion, nil, []*wire.TxOut{{PkScript: nil, Value: 10}})
targetTx := util.NewTx(msgTx)
utxoSet := NewFullUTXOSet()
if isAccepted, err := utxoSet.AddTx(targetTx.MsgTx(), uint64(numBlocksToGenerate)-4); err != nil {
blueScore := uint64(numBlocksToGenerate) - 4
if isAccepted, err := utxoSet.AddTx(targetTx.MsgTx(), blueScore); err != nil {
t.Fatalf("AddTx unexpectedly failed. Error: %s", err)
} else if !isAccepted {
t.Fatalf("AddTx unexpectedly didn't add tx %s", targetTx.ID())
@ -278,7 +279,7 @@ func TestCalcSequenceLock(t *testing.T) {
TxID: *unConfTx.TxID(),
Index: 0,
}
if isAccepted, err := utxoSet.AddTx(unConfTx, UnminedChainHeight); err != nil {
if isAccepted, err := utxoSet.AddTx(unConfTx, UnminedBlueScore); err != nil {
t.Fatalf("AddTx unexpectedly failed. Error: %s", err)
} else if !isAccepted {
t.Fatalf("AddTx unexpectedly didn't add tx %s", unConfTx.TxID())
@ -299,8 +300,8 @@ func TestCalcSequenceLock(t *testing.T) {
tx: wire.NewNativeMsgTx(1, []*wire.TxIn{{PreviousOutpoint: utxo, Sequence: wire.MaxTxInSequenceNum}}, nil),
utxoSet: utxoSet,
want: &SequenceLock{
Seconds: -1,
BlockChainHeight: -1,
Seconds: -1,
BlockBlueScore: -1,
},
},
// A transaction with a single input whose lock time is
@ -314,8 +315,8 @@ func TestCalcSequenceLock(t *testing.T) {
tx: wire.NewNativeMsgTx(1, []*wire.TxIn{{PreviousOutpoint: utxo, Sequence: LockTimeToSequence(true, 2)}}, nil),
utxoSet: utxoSet,
want: &SequenceLock{
Seconds: medianTime - 1,
BlockChainHeight: -1,
Seconds: medianTime - 1,
BlockBlueScore: -1,
},
},
// A transaction with a single input whose lock time is
@ -327,8 +328,8 @@ func TestCalcSequenceLock(t *testing.T) {
tx: wire.NewNativeMsgTx(1, []*wire.TxIn{{PreviousOutpoint: utxo, Sequence: LockTimeToSequence(true, 1024)}}, nil),
utxoSet: utxoSet,
want: &SequenceLock{
Seconds: medianTime + 1023,
BlockChainHeight: -1,
Seconds: medianTime + 1023,
BlockBlueScore: -1,
},
},
// A transaction with multiple inputs. The first input has a
@ -354,8 +355,8 @@ func TestCalcSequenceLock(t *testing.T) {
nil),
utxoSet: utxoSet,
want: &SequenceLock{
Seconds: medianTime + (5 << wire.SequenceLockTimeGranularity) - 1,
BlockChainHeight: int64(prevUtxoChainHeight) + 3,
Seconds: medianTime + (5 << wire.SequenceLockTimeGranularity) - 1,
BlockBlueScore: int64(prevUtxoChainHeight) + 3,
},
},
// Transaction with a single input. The input's sequence number
@ -367,8 +368,8 @@ func TestCalcSequenceLock(t *testing.T) {
tx: wire.NewNativeMsgTx(1, []*wire.TxIn{{PreviousOutpoint: utxo, Sequence: LockTimeToSequence(false, 3)}}, nil),
utxoSet: utxoSet,
want: &SequenceLock{
Seconds: -1,
BlockChainHeight: int64(prevUtxoChainHeight) + 2,
Seconds: -1,
BlockBlueScore: int64(prevUtxoChainHeight) + 2,
},
},
// A transaction with two inputs with lock times expressed in
@ -385,8 +386,8 @@ func TestCalcSequenceLock(t *testing.T) {
}}, nil),
utxoSet: utxoSet,
want: &SequenceLock{
Seconds: medianTime + (10 << wire.SequenceLockTimeGranularity) - 1,
BlockChainHeight: -1,
Seconds: medianTime + (10 << wire.SequenceLockTimeGranularity) - 1,
BlockBlueScore: -1,
},
},
// A transaction with two inputs with lock times expressed in
@ -406,8 +407,8 @@ func TestCalcSequenceLock(t *testing.T) {
nil),
utxoSet: utxoSet,
want: &SequenceLock{
Seconds: -1,
BlockChainHeight: int64(prevUtxoChainHeight) + 10,
Seconds: -1,
BlockBlueScore: int64(prevUtxoChainHeight) + 10,
},
},
// A transaction with multiple inputs. Two inputs are time
@ -432,8 +433,8 @@ func TestCalcSequenceLock(t *testing.T) {
nil),
utxoSet: utxoSet,
want: &SequenceLock{
Seconds: medianTime + (13 << wire.SequenceLockTimeGranularity) - 1,
BlockChainHeight: int64(prevUtxoChainHeight) + 8,
Seconds: medianTime + (13 << wire.SequenceLockTimeGranularity) - 1,
BlockBlueScore: int64(prevUtxoChainHeight) + 8,
},
},
// A transaction with a single unconfirmed input. As the input
@ -448,8 +449,8 @@ func TestCalcSequenceLock(t *testing.T) {
utxoSet: utxoSet,
mempool: true,
want: &SequenceLock{
Seconds: -1,
BlockChainHeight: int64(nextBlockChainHeight) + 1,
Seconds: -1,
BlockBlueScore: int64(nextBlockChainHeight) + 1,
},
},
// A transaction with a single unconfirmed input. The input has
@ -461,8 +462,8 @@ func TestCalcSequenceLock(t *testing.T) {
utxoSet: utxoSet,
mempool: true,
want: &SequenceLock{
Seconds: nextMedianTime + 1023,
BlockChainHeight: -1,
Seconds: nextMedianTime + 1023,
BlockBlueScore: -1,
},
},
}
@ -479,9 +480,9 @@ func TestCalcSequenceLock(t *testing.T) {
t.Fatalf("test '%s' got %v seconds want %v seconds",
test.name, seqLock.Seconds, test.want.Seconds)
}
if seqLock.BlockChainHeight != test.want.BlockChainHeight {
t.Fatalf("test '%s' got chain-height of %v want chain-height of %v ",
test.name, seqLock.BlockChainHeight, test.want.BlockChainHeight)
if seqLock.BlockBlueScore != test.want.BlockBlueScore {
t.Fatalf("test '%s' got blue score of %v want blue score of %v ",
test.name, seqLock.BlockBlueScore, test.want.BlockBlueScore)
}
}
}

View File

@ -635,7 +635,7 @@ func (dag *BlockDAG) deserializeBlockNode(blockRow []byte) (*blockNode, error) {
// dbFetchBlockByNode uses an existing database transaction to retrieve the
// raw block for the provided node, deserialize it, and return a util.Block
// with the height set.
// of it.
func dbFetchBlockByNode(dbTx database.Tx, node *blockNode) (*util.Block, error) {
// Load the raw block bytes from the database.
blockBytes, err := dbTx.FetchBlock(node.hash)
@ -643,13 +643,11 @@ func dbFetchBlockByNode(dbTx database.Tx, node *blockNode) (*util.Block, error)
return nil, err
}
// Create the encapsulated block and set the height appropriately.
// Create the encapsulated block.
block, err := util.NewBlockFromBytes(blockBytes)
if err != nil {
return nil, err
}
block.SetHeight(node.height)
return block, nil
}
@ -700,7 +698,7 @@ func dbStoreBlockNode(dbTx database.Tx, node *blockNode) error {
// Write block header data to block index bucket.
blockIndexBucket := dbTx.Metadata().Bucket(blockIndexBucketName)
key := blockIndexKey(node.hash, uint32(node.height))
key := blockIndexKey(node.hash)
return blockIndexBucket.Put(key, value)
}
@ -718,22 +716,18 @@ func dbStoreBlock(dbTx database.Tx, block *util.Block) error {
}
// blockIndexKey generates the binary key for an entry in the block index
// bucket. The key is composed of the block height encoded as a big-endian
// 32-bit unsigned int followed by the 32 byte block hash.
func blockIndexKey(blockHash *daghash.Hash, blockHeight uint32) []byte {
indexKey := make([]byte, daghash.HashSize+4)
binary.BigEndian.PutUint32(indexKey[0:4], blockHeight)
copy(indexKey[4:daghash.HashSize+4], blockHash[:])
// bucket. The key is composed only of the 32 byte block hash.
func blockIndexKey(blockHash *daghash.Hash) []byte {
indexKey := make([]byte, daghash.HashSize)
copy(indexKey[:], blockHash[:])
return indexKey
}
// BlockByHash returns the block from the main chain with the given hash with
// the appropriate chain height set.
// BlockByHash returns the block from the DAG with the given hash.
//
// This function is safe for concurrent access.
func (dag *BlockDAG) BlockByHash(hash *daghash.Hash) (*util.Block, error) {
// Lookup the block hash in block index and ensure it is in the best
// chain.
// Lookup the block hash in block index and ensure it is in the DAG
node := dag.index.LookupNode(hash)
if node == nil {
str := fmt.Sprintf("block %s is not in the main chain", hash)

View File

@ -49,24 +49,24 @@ func TestUtxoSerialization(t *testing.T) {
// From tx in main blockchain:
// b7c3332bc138e2c9429818f5fed500bcc1746544218772389054dc8047d7cd3f:0
{
name: "height 1, coinbase",
name: "blue score 1, coinbase",
entry: &UTXOEntry{
amount: 5000000000,
pkScript: hexToBytes("410496b538e853519c726a2c91e61ec11600ae1390813a627c66fb8be7947be63c52da7589379515d4e0a604f8141781e62294721166bf621e73a82cbf2342c858eeac"),
blockChainHeight: 1,
packedFlags: tfBlockReward,
amount: 5000000000,
pkScript: hexToBytes("410496b538e853519c726a2c91e61ec11600ae1390813a627c66fb8be7947be63c52da7589379515d4e0a604f8141781e62294721166bf621e73a82cbf2342c858eeac"),
blockBlueScore: 1,
packedFlags: tfBlockReward,
},
serialized: hexToBytes("03320496b538e853519c726a2c91e61ec11600ae1390813a627c66fb8be7947be63c52"),
},
// From tx in main blockchain:
// 8131ffb0a2c945ecaf9b9063e59558784f9c3a74741ce6ae2a18d0571dac15bb:1
{
name: "height 100001, not coinbase",
name: "blue score 100001, not coinbase",
entry: &UTXOEntry{
amount: 1000000,
pkScript: hexToBytes("76a914ee8bd501094a7d5ca318da2506de35e1cb025ddc88ac"),
blockChainHeight: 100001,
packedFlags: 0,
amount: 1000000,
pkScript: hexToBytes("76a914ee8bd501094a7d5ca318da2506de35e1cb025ddc88ac"),
blockBlueScore: 100001,
packedFlags: 0,
},
serialized: hexToBytes("8b99420700ee8bd501094a7d5ca318da2506de35e1cb025ddc"),
},
@ -105,10 +105,10 @@ func TestUtxoSerialization(t *testing.T) {
utxoEntry.PkScript(), test.entry.PkScript())
continue
}
if utxoEntry.BlockChainHeight() != test.entry.BlockChainHeight() {
if utxoEntry.BlockBlueScore() != test.entry.BlockBlueScore() {
t.Errorf("deserializeUTXOEntry #%d (%s) mismatched "+
"block height: got %d, want %d", i, test.name,
utxoEntry.BlockChainHeight(), test.entry.BlockChainHeight())
"block blue score: got %d, want %d", i, test.name,
utxoEntry.BlockBlueScore(), test.entry.BlockBlueScore())
continue
}
if utxoEntry.IsBlockReward() != test.entry.IsBlockReward() {

View File

@ -147,7 +147,7 @@ func (dag *BlockDAG) calcNextRequiredDifficulty(bluestParent *blockNode, newBloc
// newTarget since conversion to the compact representation loses
// precision.
newTargetBits := util.BigToCompact(newTarget)
log.Debugf("Difficulty retarget at block height %d", bluestParent.height+1)
log.Debugf("Difficulty retarget at block chain height %d", dag.virtual.chainHeight)
log.Debugf("Old target %08x (%064x)", bluestParent.bits, oldTarget)
log.Debugf("New target %08x (%064x)", newTargetBits, util.CompactToBig(newTargetBits))
log.Debugf("Actual timespan %s, adjusted timespan %s, target timespan %s",

View File

@ -92,10 +92,6 @@ const (
// checkpoint height does not match the expected one.
ErrBadCheckpoint
// ErrForkTooOld indicates a block is attempting to fork the block chain
// before the most recent checkpoint.
ErrForkTooOld
// ErrCheckpointTimeTooOld indicates a block has a timestamp before the
// most recent checkpoint.
ErrCheckpointTimeTooOld
@ -176,15 +172,15 @@ const (
// not match the expected value of the subsidy plus the sum of all fees.
ErrBadCoinbaseValue
// ErrMissingCoinbaseHeight indicates the coinbase transaction for a
// block does not start with the serialized block block height as
// ErrMissingCoinbaseBlueScore indicates the coinbase transaction for a
// block does not start with the serialized block blue score as
// required for version 2 and higher blocks.
ErrMissingCoinbaseHeight
ErrMissingCoinbaseBlueScore
// ErrBadCoinbaseHeight indicates the serialized block height in the
// ErrBadCoinbaseBlueScore indicates the serialized block blue score in the
// coinbase transaction for version 2 and higher blocks does not match
// the expected value.
ErrBadCoinbaseHeight
ErrBadCoinbaseBlueScore
// ErrSecondTxNotFeeTransaction indicates the second transaction in
// a block is not a fee transaction.
@ -261,7 +257,6 @@ var errorCodeStrings = map[ErrorCode]string{
ErrHighHash: "ErrHighHash",
ErrBadMerkleRoot: "ErrBadMerkleRoot",
ErrBadCheckpoint: "ErrBadCheckpoint",
ErrForkTooOld: "ErrForkTooOld",
ErrCheckpointTimeTooOld: "ErrCheckpointTimeTooOld",
ErrNoTransactions: "ErrNoTransactions",
ErrNoTxInputs: "ErrNoTxInputs",
@ -281,8 +276,8 @@ var errorCodeStrings = map[ErrorCode]string{
ErrMultipleCoinbases: "ErrMultipleCoinbases",
ErrBadCoinbaseScriptLen: "ErrBadCoinbaseScriptLen",
ErrBadCoinbaseValue: "ErrBadCoinbaseValue",
ErrMissingCoinbaseHeight: "ErrMissingCoinbaseHeight",
ErrBadCoinbaseHeight: "ErrBadCoinbaseHeight",
ErrMissingCoinbaseBlueScore: "ErrMissingCoinbaseBlueScore",
ErrBadCoinbaseBlueScore: "ErrBadCoinbaseBlueScore",
ErrSecondTxNotFeeTransaction: "ErrSecondTxNotFeeTransaction",
ErrBadFeeTransaction: "ErrBadFeeTransaction",
ErrMultipleFeeTransactions: "ErrMultipleFeeTransactions",

View File

@ -28,7 +28,6 @@ func TestErrorCodeStringer(t *testing.T) {
{ErrHighHash, "ErrHighHash"},
{ErrBadMerkleRoot, "ErrBadMerkleRoot"},
{ErrBadCheckpoint, "ErrBadCheckpoint"},
{ErrForkTooOld, "ErrForkTooOld"},
{ErrCheckpointTimeTooOld, "ErrCheckpointTimeTooOld"},
{ErrNoTransactions, "ErrNoTransactions"},
{ErrNoTxInputs, "ErrNoTxInputs"},
@ -49,8 +48,8 @@ func TestErrorCodeStringer(t *testing.T) {
{ErrMultipleCoinbases, "ErrMultipleCoinbases"},
{ErrBadCoinbaseScriptLen, "ErrBadCoinbaseScriptLen"},
{ErrBadCoinbaseValue, "ErrBadCoinbaseValue"},
{ErrMissingCoinbaseHeight, "ErrMissingCoinbaseHeight"},
{ErrBadCoinbaseHeight, "ErrBadCoinbaseHeight"},
{ErrMissingCoinbaseBlueScore, "ErrMissingCoinbaseBlueScore"},
{ErrBadCoinbaseBlueScore, "ErrBadCoinbaseBlueScore"},
{ErrSecondTxNotFeeTransaction, "ErrSecondTxNotFeeTransaction"},
{ErrBadFeeTransaction, "ErrBadFeeTransaction"},
{ErrMultipleFeeTransactions, "ErrMultipleFeeTransactions"},

View File

@ -156,7 +156,7 @@ func TestFullBlocks(t *testing.T) {
testAcceptedBlock := func(item fullblocktests.AcceptedBlock) {
blockHeight := item.Height
block := util.NewBlock(item.Block)
block.SetHeight(blockHeight)
block.SetChainHeight(blockHeight)
t.Logf("Testing block %s (hash %s, height %d)",
item.Name, block.Hash(), blockHeight)
@ -182,7 +182,7 @@ func TestFullBlocks(t *testing.T) {
testRejectedBlock := func(item fullblocktests.RejectedBlock) {
blockHeight := item.Height
block := util.NewBlock(item.Block)
block.SetHeight(blockHeight)
block.SetChainHeight(blockHeight)
t.Logf("Testing block %s (hash %s, height %d)",
item.Name, block.Hash(), blockHeight)
@ -239,7 +239,7 @@ func TestFullBlocks(t *testing.T) {
testOrphanOrRejectedBlock := func(item fullblocktests.OrphanOrRejectedBlock) {
blockHeight := item.Height
block := util.NewBlock(item.Block)
block.SetHeight(blockHeight)
block.SetChainHeight(blockHeight)
t.Logf("Testing block %s (hash %s, height %d)",
item.Name, block.Hash(), blockHeight)
@ -267,18 +267,18 @@ func TestFullBlocks(t *testing.T) {
testExpectedTip := func(item fullblocktests.ExpectedTip) {
blockHeight := item.Height
block := util.NewBlock(item.Block)
block.SetHeight(blockHeight)
block.SetChainHeight(blockHeight)
t.Logf("Testing tip for block %s (hash %s, height %d)",
item.Name, block.Hash(), blockHeight)
// Ensure hash and height match.
if dag.HighestTipHash() != item.Block.BlockHash() ||
dag.Height() != blockHeight { //TODO: (Ori) the use of dag.Height() and virtualBlock.HighestTipHash() is wrong, and was done only for compilation
if dag.SelectedTipHash() != item.Block.BlockHash() ||
dag.ChainHeight() != blockHeight { //TODO: (Ori) the use of dag.ChainHeight() and virtualBlock.HighestTipHash() is wrong, and was done only for compilation
t.Fatalf("block %q (hash %s, height %d) should be "+
"the current tip -- got (hash %s, height %d)",
item.Name, block.Hash(), blockHeight, dag.HighestTipHash(),
dag.Height()) //TODO: (Ori) the use of dag.Height() and virtualBlock.HighestTipHash() is wrong, and was done only for compilation
item.Name, block.Hash(), blockHeight, dag.SelectedTipHash(),
dag.ChainHeight()) //TODO: (Ori) the use of dag.ChainHeight() and virtualBlock.HighestTipHash() is wrong, and was done only for compilation
}
}

View File

@ -241,10 +241,9 @@ func pushDataScript(items ...[]byte) []byte {
}
// standardCoinbaseScript returns a standard script suitable for use as the
// signature script of the coinbase transaction of a new block. In particular,
// it starts with the block height that is required by version 2 blocks.
func standardCoinbaseScript(blockHeight uint64, extraNonce uint64) ([]byte, error) {
return txscript.NewScriptBuilder().AddInt64(int64(blockHeight)).
// signature script of the coinbase transaction of a new block.
func standardCoinbaseScript(blueScore uint64, extraNonce uint64) ([]byte, error) {
return txscript.NewScriptBuilder().AddInt64(int64(blueScore)).
AddInt64(int64(extraNonce)).Script()
}
@ -275,9 +274,9 @@ func uniqueOpReturnScript() []byte {
// createCoinbaseTx returns a coinbase transaction paying an appropriate
// subsidy based on the passed block height. The coinbase signature script
// conforms to the requirements of version 2 blocks.
func (g *testGenerator) createCoinbaseTx(blockHeight uint64) *wire.MsgTx {
func (g *testGenerator) createCoinbaseTx(blueScore uint64) *wire.MsgTx {
extraNonce := uint64(0)
coinbaseScript, err := standardCoinbaseScript(blockHeight, extraNonce)
coinbaseScript, err := standardCoinbaseScript(blueScore, extraNonce)
if err != nil {
panic(err)
}
@ -291,7 +290,7 @@ func (g *testGenerator) createCoinbaseTx(blockHeight uint64) *wire.MsgTx {
SignatureScript: coinbaseScript,
}
txOut := &wire.TxOut{
Value: blockdag.CalcBlockSubsidy(blockHeight, g.params),
Value: blockdag.CalcBlockSubsidy(blueScore, g.params),
PkScript: opTrueScript,
}
return wire.NewNativeMsgTx(1, []*wire.TxIn{txIn}, []*wire.TxOut{txOut})

View File

@ -1,76 +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 indexers
import (
"sync"
"time"
"github.com/btcsuite/btclog"
"github.com/daglabs/btcd/util"
)
// blockProgressLogger provides periodic logging for other services in order
// to show users progress of certain "actions" involving some or all current
// blocks. Ex: syncing to best chain, indexing all blocks, etc.
type blockProgressLogger struct {
receivedLogBlocks int64
receivedLogTx int64
lastBlockLogTime time.Time
subsystemLogger btclog.Logger
progressAction string
sync.Mutex
}
// newBlockProgressLogger returns a new block progress logger.
// The progress message is templated as follows:
// {progressAction} {numProcessed} {blocks|block} in the last {timePeriod}
// ({numTxs}, height {lastBlockHeight}, {lastBlockTimeStamp})
func newBlockProgressLogger(progressMessage string, logger btclog.Logger) *blockProgressLogger {
return &blockProgressLogger{
lastBlockLogTime: time.Now(),
progressAction: progressMessage,
subsystemLogger: logger,
}
}
// LogBlockHeight logs a new block height as an information message to show
// progress to the user. In order to prevent spam, it limits logging to one
// message every 10 seconds with duration and totals included.
func (b *blockProgressLogger) LogBlockHeight(block *util.Block) {
b.Lock()
defer b.Unlock()
b.receivedLogBlocks++
b.receivedLogTx += int64(len(block.MsgBlock().Transactions))
now := time.Now()
duration := now.Sub(b.lastBlockLogTime)
if duration < time.Second*10 {
return
}
// Truncate the duration to 10s of milliseconds.
durationMillis := int64(duration / time.Millisecond)
tDuration := 10 * time.Millisecond * time.Duration(durationMillis/10)
// Log information about new block height.
blockStr := "blocks"
if b.receivedLogBlocks == 1 {
blockStr = "block"
}
txStr := "transactions"
if b.receivedLogTx == 1 {
txStr = "transaction"
}
b.subsystemLogger.Infof("%s %d %s in the last %s (%d %s, height %d, %s)",
b.progressAction, b.receivedLogBlocks, blockStr, tDuration, b.receivedLogTx,
txStr, block.Height(), block.MsgBlock().Header.Timestamp)
b.receivedLogBlocks = 0
b.receivedLogTx = 0
b.lastBlockLogTime = now
}

View File

@ -20,11 +20,6 @@ type testBlockData struct {
expectedBlues []string
}
type hashIDPair struct {
hash *daghash.Hash
id string
}
//TestPhantom iterate over several dag simulations, and checks
//that the blue score, blue set and selected parent of each
//block calculated as expected
@ -113,7 +108,7 @@ func TestPhantom(t *testing.T) {
id: "K",
expectedScore: 9,
expectedSelectedParent: "H",
expectedBlues: []string{"I", "J", "G", "F", "H"},
expectedBlues: []string{"I", "G", "J", "F", "H"},
},
},
},

View File

@ -137,13 +137,13 @@ func createTxForTest(numInputs uint32, numOutputs uint32, outputValue uint64, su
}
// createCoinbaseTxForTest returns a coinbase transaction with the requested number of
// outputs paying an appropriate subsidy based on the passed block height to the
// outputs paying an appropriate subsidy based on the passed block blueScore to the
// address associated with the harness. It automatically uses a standard
// signature script that starts with the block height
func createCoinbaseTxForTest(blockHeight uint64, numOutputs uint32, extraNonce int64, params *dagconfig.Params) (*wire.MsgTx, error) {
// signature script that starts with the block blue score
func createCoinbaseTxForTest(blueScore uint64, numOutputs uint32, extraNonce int64, params *dagconfig.Params) (*wire.MsgTx, error) {
// Create standard coinbase script.
coinbaseScript, err := txscript.NewScriptBuilder().
AddInt64(int64(blockHeight)).AddInt64(extraNonce).Script()
AddInt64(int64(blueScore)).AddInt64(extraNonce).Script()
if err != nil {
return nil, err
}
@ -159,7 +159,7 @@ func createCoinbaseTxForTest(blockHeight uint64, numOutputs uint32, extraNonce i
txOuts := []*wire.TxOut{}
totalInput := CalcBlockSubsidy(blockHeight, params)
totalInput := CalcBlockSubsidy(blueScore, params)
amountPerOutput := totalInput / uint64(numOutputs)
remainder := totalInput - amountPerOutput*uint64(numOutputs)
for i := uint32(0); i < numOutputs; i++ {

View File

@ -44,9 +44,9 @@ func serializeBlockUTXODiffData(diffData *blockUTXODiffData) ([]byte, error) {
// serializing the provided utxo entry.
func utxoEntryHeaderCode(entry *UTXOEntry) uint64 {
// As described in the serialization format comments, the header code
// encodes the height shifted over one bit and the block reward flag in the
// lowest bit.
headerCode := uint64(entry.BlockChainHeight()) << 1
// encodes the blue score shifted over one bit and the block reward flag
// in the lowest bit.
headerCode := uint64(entry.BlockBlueScore()) << 1
if entry.IsBlockReward() {
headerCode |= 0x01
}
@ -283,9 +283,9 @@ func deserializeUTXOEntry(serialized []byte) (*UTXOEntry, error) {
// Decode the header code.
//
// Bit 0 indicates whether the containing transaction is a block reward.
// Bits 1-x encode height of containing transaction.
// Bits 1-x encode the blue score of the containing transaction.
isBlockReward := code&0x01 != 0
blockChainHeight := code >> 1
blockBlueScore := code >> 1
// Decode the compressed unspent transaction output.
amount, pkScript, _, err := decodeCompressedTxOut(serialized[offset:])
@ -295,10 +295,10 @@ func deserializeUTXOEntry(serialized []byte) (*UTXOEntry, error) {
}
entry := &UTXOEntry{
amount: amount,
pkScript: pkScript,
blockChainHeight: blockChainHeight,
packedFlags: 0,
amount: amount,
pkScript: pkScript,
blockBlueScore: blockBlueScore,
packedFlags: 0,
}
if isBlockReward {
entry.packedFlags |= tfBlockReward

View File

@ -13,16 +13,16 @@ import (
)
const (
// UnminedChainHeight is the chain-height used for the "block" height field of the
// UnminedBlueScore is the blue score used for the "block" blueScore field of the
// contextual transaction information provided in a transaction store
// when it has not yet been mined into a block.
UnminedChainHeight = math.MaxUint64
UnminedBlueScore = math.MaxUint64
)
// UTXOEntry houses details about an individual transaction output in a utxo
// set such as whether or not it was contained in a block reward tx, the height of
// the block that contains the tx, whether or not it is spent, its public key
// script, and how much it pays.
// set such as whether or not it was contained in a block reward tx, the blue
// score of the block that contains the tx, its public key script, and how
// much it pays.
type UTXOEntry struct {
// NOTE: Additions, deletions, or modifications to the order of the
// definitions in this struct should not be changed without considering
@ -30,9 +30,9 @@ type UTXOEntry struct {
// specifically crafted to result in minimal padding. There will be a
// lot of these in memory, so a few extra bytes of padding adds up.
amount uint64
pkScript []byte // The public key script for the output.
blockChainHeight uint64 // Chain-height of block containing tx.
amount uint64
pkScript []byte // The public key script for the output.
blockBlueScore uint64 // Blue score of the block containing the tx.
// packedFlags contains additional info about output such as whether it
// is a block reward, and whether it has been modified
@ -47,9 +47,9 @@ func (entry *UTXOEntry) IsBlockReward() bool {
return entry.packedFlags&tfBlockReward == tfBlockReward
}
// BlockChainHeight returns the chain-height of the block containing the output.
func (entry *UTXOEntry) BlockChainHeight() uint64 {
return entry.blockChainHeight
// BlockBlueScore returns the blue score of the block containing the output.
func (entry *UTXOEntry) BlockBlueScore() uint64 {
return entry.blockBlueScore
}
// Amount returns the amount of the output.
@ -65,7 +65,7 @@ func (entry *UTXOEntry) PkScript() []byte {
// IsUnmined returns true iff this UTXOEntry has still not been mined,
// a.k.a. still in the mempool.
func (entry *UTXOEntry) IsUnmined() bool {
return entry.blockChainHeight == UnminedChainHeight
return entry.blockBlueScore == UnminedBlueScore
}
// txoFlags is a bitmask defining additional information and state for a
@ -77,6 +77,21 @@ const (
tfBlockReward txoFlags = 1 << iota
)
// NewUTXOEntry creates a new utxoEntry representing the given txOut
func NewUTXOEntry(txOut *wire.TxOut, isBlockReward bool, blockBlueScore uint64) *UTXOEntry {
entry := &UTXOEntry{
amount: txOut.Value,
pkScript: txOut.PkScript,
blockBlueScore: blockBlueScore,
}
if isBlockReward {
entry.packedFlags |= tfBlockReward
}
return entry
}
// utxoCollection represents a set of UTXOs indexed by their outpoints
type utxoCollection map[wire.Outpoint]*UTXOEntry
@ -362,21 +377,6 @@ func (d UTXODiff) String() string {
return fmt.Sprintf("toAdd: %s; toRemove: %s, Multiset-Hash: %s", d.toAdd, d.toRemove, d.diffMultiset.Hash())
}
// NewUTXOEntry creates a new utxoEntry representing the given txOut
func NewUTXOEntry(txOut *wire.TxOut, isBlockReward bool, blockChainHeight uint64) *UTXOEntry {
entry := &UTXOEntry{
amount: txOut.Value,
pkScript: txOut.PkScript,
blockChainHeight: blockChainHeight,
}
if isBlockReward {
entry.packedFlags |= tfBlockReward
}
return entry
}
// UTXOSet represents a set of unspent transaction outputs
// Every DAG has exactly one fullUTXOSet.
// When a new block arrives, it is validated and applied to the fullUTXOSet in the following manner:
@ -394,11 +394,11 @@ type UTXOSet interface {
diffFrom(other UTXOSet) (*UTXODiff, error)
WithDiff(utxoDiff *UTXODiff) (UTXOSet, error)
diffFromTx(tx *wire.MsgTx, node *blockNode) (*UTXODiff, error)
AddTx(tx *wire.MsgTx, blockHeight uint64) (ok bool, err error)
AddTx(tx *wire.MsgTx, blockBlueScore uint64) (ok bool, err error)
clone() UTXOSet
Get(outpoint wire.Outpoint) (*UTXOEntry, bool)
Multiset() *btcec.Multiset
WithTransactions(transactions []*wire.MsgTx, blockHeight uint64, ignoreDoubleSpends bool) (UTXOSet, error)
WithTransactions(transactions []*wire.MsgTx, blockBlueScore uint64, ignoreDoubleSpends bool) (UTXOSet, error)
}
// diffFromTx is a common implementation for diffFromTx, that works
@ -423,7 +423,7 @@ func diffFromTx(u UTXOSet, tx *wire.MsgTx, containingNode *blockNode) (*UTXODiff
}
}
for i, txOut := range tx.TxOut {
entry := NewUTXOEntry(txOut, isBlockReward, containingNode.height)
entry := NewUTXOEntry(txOut, isBlockReward, containingNode.blueScore)
outpoint := *wire.NewOutpoint(tx.TxID(), uint32(i))
err := diff.AddEntry(outpoint, entry)
if err != nil {
@ -470,7 +470,7 @@ func (fus *FullUTXOSet) WithDiff(other *UTXODiff) (UTXOSet, error) {
// AddTx adds a transaction to this utxoSet and returns isAccepted=true iff it's valid in this UTXO's context.
// It returns error if something unexpected happens, such as serialization error (isAccepted=false doesn't
// necessarily means there's an error).
func (fus *FullUTXOSet) AddTx(tx *wire.MsgTx, blockHeight uint64) (isAccepted bool, err error) {
func (fus *FullUTXOSet) AddTx(tx *wire.MsgTx, blueScore uint64) (isAccepted bool, err error) {
isBlockReward := tx.IsBlockReward()
if !isBlockReward {
if !fus.containsInputs(tx) {
@ -488,7 +488,7 @@ func (fus *FullUTXOSet) AddTx(tx *wire.MsgTx, blockHeight uint64) (isAccepted bo
for i, txOut := range tx.TxOut {
outpoint := *wire.NewOutpoint(tx.TxID(), uint32(i))
entry := NewUTXOEntry(txOut, isBlockReward, blockHeight)
entry := NewUTXOEntry(txOut, isBlockReward, blueScore)
err := fus.addAndUpdateMultiset(outpoint, entry)
if err != nil {
@ -560,10 +560,10 @@ func (fus *FullUTXOSet) removeAndUpdateMultiset(outpoint wire.Outpoint) error {
}
// WithTransactions returns a new UTXO Set with the added transactions
func (fus *FullUTXOSet) WithTransactions(transactions []*wire.MsgTx, blockHeight uint64, ignoreDoubleSpends bool) (UTXOSet, error) {
func (fus *FullUTXOSet) WithTransactions(transactions []*wire.MsgTx, blockBlueScore uint64, ignoreDoubleSpends bool) (UTXOSet, error) {
diffSet := NewDiffUTXOSet(fus, NewUTXODiff())
for _, tx := range transactions {
isAccepted, err := diffSet.AddTx(tx, blockHeight)
isAccepted, err := diffSet.AddTx(tx, blockBlueScore)
if err != nil {
return nil, err
}
@ -614,13 +614,13 @@ func (dus *DiffUTXOSet) WithDiff(other *UTXODiff) (UTXOSet, error) {
}
// AddTx adds a transaction to this utxoSet and returns true iff it's valid in this UTXO's context
func (dus *DiffUTXOSet) AddTx(tx *wire.MsgTx, blockHeight uint64) (bool, error) {
func (dus *DiffUTXOSet) AddTx(tx *wire.MsgTx, blockBlueScore uint64) (bool, error) {
isBlockReward := tx.IsBlockReward()
if !isBlockReward && !dus.containsInputs(tx) {
return false, nil
}
err := dus.appendTx(tx, blockHeight, isBlockReward)
err := dus.appendTx(tx, blockBlueScore, isBlockReward)
if err != nil {
return false, err
}
@ -628,7 +628,7 @@ func (dus *DiffUTXOSet) AddTx(tx *wire.MsgTx, blockHeight uint64) (bool, error)
return true, nil
}
func (dus *DiffUTXOSet) appendTx(tx *wire.MsgTx, blockHeight uint64, isBlockReward bool) error {
func (dus *DiffUTXOSet) appendTx(tx *wire.MsgTx, blockBlueScore uint64, isBlockReward bool) error {
if !isBlockReward {
for _, txIn := range tx.TxIn {
outpoint := *wire.NewOutpoint(&txIn.PreviousOutpoint.TxID, txIn.PreviousOutpoint.Index)
@ -645,7 +645,7 @@ func (dus *DiffUTXOSet) appendTx(tx *wire.MsgTx, blockHeight uint64, isBlockRewa
for i, txOut := range tx.TxOut {
outpoint := *wire.NewOutpoint(tx.TxID(), uint32(i))
entry := NewUTXOEntry(txOut, isBlockReward, blockHeight)
entry := NewUTXOEntry(txOut, isBlockReward, blockBlueScore)
err := dus.UTXODiff.AddEntry(outpoint, entry)
if err != nil {
@ -723,10 +723,10 @@ func (dus *DiffUTXOSet) Multiset() *btcec.Multiset {
}
// WithTransactions returns a new UTXO Set with the added transactions
func (dus *DiffUTXOSet) WithTransactions(transactions []*wire.MsgTx, blockHeight uint64, ignoreDoubleSpends bool) (UTXOSet, error) {
func (dus *DiffUTXOSet) WithTransactions(transactions []*wire.MsgTx, blockBlueScore uint64, ignoreDoubleSpends bool) (UTXOSet, error) {
diffSet := NewDiffUTXOSet(dus.base, dus.UTXODiff.clone())
for _, tx := range transactions {
isAccepted, err := diffSet.AddTx(tx, blockHeight)
isAccepted, err := diffSet.AddTx(tx, blockBlueScore)
if err != nil {
return nil, err
}

View File

@ -873,7 +873,8 @@ testLoop:
// Apply all transactions to diffSet, in order, with the initial block height startHeight
for i, transaction := range test.toAdd {
_, err := diffSet.AddTx(transaction, test.startHeight+uint64(i))
height := test.startHeight + uint64(i)
_, err := diffSet.AddTx(transaction, height)
if err != nil {
t.Errorf("Error adding tx %s in test \"%s\": %s", transaction.TxID(), test.name, err)
continue testLoop
@ -901,14 +902,14 @@ func TestDiffFromTx(t *testing.T) {
} else if !isAccepted {
t.Fatalf("AddTx unexpectedly didn't add tx %s", cbTx.TxID())
}
node := &blockNode{height: 2} //Fake node
node := &blockNode{blueScore: 2} //Fake node
cbOutpoint := wire.Outpoint{TxID: *cbTx.TxID(), Index: 0}
txIns := []*wire.TxIn{&wire.TxIn{
txIns := []*wire.TxIn{{
PreviousOutpoint: cbOutpoint,
SignatureScript: nil,
Sequence: wire.MaxTxInSequenceNum,
}}
txOuts := []*wire.TxOut{&wire.TxOut{
txOuts := []*wire.TxOut{{
PkScript: OpTrueScript,
Value: uint64(1),
}}
@ -930,12 +931,12 @@ func TestDiffFromTx(t *testing.T) {
}
//Test that we get an error if we don't have the outpoint inside the utxo set
invalidTxIns := []*wire.TxIn{&wire.TxIn{
invalidTxIns := []*wire.TxIn{{
PreviousOutpoint: wire.Outpoint{TxID: daghash.TxID{}, Index: 0},
SignatureScript: nil,
Sequence: wire.MaxTxInSequenceNum,
}}
invalidTxOuts := []*wire.TxOut{&wire.TxOut{
invalidTxOuts := []*wire.TxOut{{
PkScript: OpTrueScript,
Value: uint64(1),
}}

View File

@ -5,7 +5,6 @@
package blockdag
import (
"encoding/binary"
"fmt"
"math"
"math/big"
@ -43,10 +42,6 @@ const (
// baseSubsidy is the starting subsidy amount for mined blocks. This
// value is halved every SubsidyHalvingInterval blocks.
baseSubsidy = 50 * util.SatoshiPerBitcoin
// MaxOutputsPerBlock is the maximum number of transaction outputs there
// can be in a block of max size.
MaxOutputsPerBlock = wire.MaxBlockPayload / wire.MinTxOutPayload
)
// isNullOutpoint determines whether or not a previous transaction outpoint
@ -82,15 +77,15 @@ func IsFeeTransaction(tx *util.Tx) bool {
// SequenceLockActive determines if a transaction's sequence locks have been
// met, meaning that all the inputs of a given transaction have reached a
// chain-height or time sufficient for their relative lock-time maturity.
func SequenceLockActive(sequenceLock *SequenceLock, blockChainHeight uint64,
// blue score or time sufficient for their relative lock-time maturity.
func SequenceLockActive(sequenceLock *SequenceLock, blockBlueScore uint64,
medianTimePast time.Time) bool {
// If either the seconds, or chain-height relative-lock time has not yet
// If either the seconds, or blue score relative-lock time has not yet
// reached, then the transaction is not yet mature according to its
// sequence locks.
if sequenceLock.Seconds >= medianTimePast.Unix() ||
sequenceLock.BlockChainHeight >= int64(blockChainHeight) {
sequenceLock.BlockBlueScore >= int64(blockBlueScore) {
return false
}
@ -98,7 +93,7 @@ func SequenceLockActive(sequenceLock *SequenceLock, blockChainHeight uint64,
}
// IsFinalizedTransaction determines whether or not a transaction is finalized.
func IsFinalizedTransaction(tx *util.Tx, blockHeight uint64, blockTime time.Time) bool {
func IsFinalizedTransaction(tx *util.Tx, blockBlueScore uint64, blockTime time.Time) bool {
msgTx := tx.MsgTx()
// Lock time of zero means the transaction is finalized.
@ -107,17 +102,17 @@ func IsFinalizedTransaction(tx *util.Tx, blockHeight uint64, blockTime time.Time
return true
}
// The lock time field of a transaction is either a block height at
// The lock time field of a transaction is either a block blue score at
// which the transaction is finalized or a timestamp depending on if the
// value is before the txscript.LockTimeThreshold. When it is under the
// threshold it is a block height.
blockTimeOrHeight := int64(0)
// threshold it is a block blue score.
blockTimeOrBlueScore := int64(0)
if lockTime < txscript.LockTimeThreshold {
blockTimeOrHeight = int64(blockHeight)
blockTimeOrBlueScore = int64(blockBlueScore)
} else {
blockTimeOrHeight = blockTime.Unix()
blockTimeOrBlueScore = blockTime.Unix()
}
if int64(lockTime) < blockTimeOrHeight {
if int64(lockTime) < blockTimeOrBlueScore {
return true
}
@ -132,28 +127,28 @@ func IsFinalizedTransaction(tx *util.Tx, blockHeight uint64, blockTime time.Time
return true
}
// CalcBlockSubsidy returns the subsidy amount a block at the provided height
// CalcBlockSubsidy returns the subsidy amount a block at the provided blue score
// should have. This is mainly used for determining how much the coinbase for
// newly generated blocks awards as well as validating the coinbase for blocks
// has the expected value.
//
// The subsidy is halved every SubsidyReductionInterval blocks. Mathematically
// this is: baseSubsidy / 2^(height/SubsidyReductionInterval)
// this is: baseSubsidy / 2^(blueScore/SubsidyReductionInterval)
//
// At the target block generation rate for the main network, this is
// approximately every 4 years.
func CalcBlockSubsidy(height uint64, dagParams *dagconfig.Params) uint64 {
func CalcBlockSubsidy(blueScore uint64, dagParams *dagconfig.Params) uint64 {
if dagParams.SubsidyReductionInterval == 0 {
return baseSubsidy
}
// Equivalent to: baseSubsidy / 2^(height/subsidyHalvingInterval)
return baseSubsidy >> uint(height/dagParams.SubsidyReductionInterval)
// Equivalent to: baseSubsidy / 2^(blueScore/subsidyHalvingInterval)
return baseSubsidy >> uint(blueScore/dagParams.SubsidyReductionInterval)
}
// 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, isFeeTransaction bool) error {
func CheckTransactionSanity(tx *util.Tx, subnetworkID *subnetworkid.SubnetworkID) error {
// A transaction must have at least one input.
msgTx := tx.MsgTx()
if len(msgTx.TxIn) == 0 {
@ -540,9 +535,8 @@ func (dag *BlockDAG) checkBlockSanity(block *util.Block, flags BehaviorFlags) er
// Do some preliminary checks on each transaction to ensure they are
// sane before continuing.
for i, tx := range transactions {
isFeeTransaction := i == util.FeeTransactionIndex
err := CheckTransactionSanity(tx, dag.subnetworkID, isFeeTransaction)
for _, tx := range transactions {
err := CheckTransactionSanity(tx, dag.subnetworkID)
if err != nil {
return err
}
@ -604,63 +598,6 @@ func (dag *BlockDAG) CheckBlockSanity(block *util.Block, powLimit *big.Int,
return dag.checkBlockSanity(block, BFNone)
}
// ExtractCoinbaseHeight attempts to extract the height of the block from the
// scriptSig of a coinbase transaction.
func ExtractCoinbaseHeight(coinbaseTx *util.Tx) (uint64, error) {
sigScript := coinbaseTx.MsgTx().TxIn[0].SignatureScript
if len(sigScript) < 1 {
str := "the coinbase signature script" +
"must start with the " +
"length of the serialized block height"
return 0, ruleError(ErrMissingCoinbaseHeight, str)
}
// Detect the case when the block height is a small integer encoded with
// a single byte.
opcode := int(sigScript[0])
if opcode == txscript.Op0 {
return 0, nil
}
if opcode >= txscript.Op1 && opcode <= txscript.Op16 {
return uint64(opcode - (txscript.Op1 - 1)), nil
}
// Otherwise, the opcode is the length of the following bytes which
// encode in the block height.
serializedLen := int(sigScript[0])
if len(sigScript[1:]) < serializedLen {
str := "the coinbase signature script " +
"must start with the " +
"serialized block height"
return 0, ruleError(ErrMissingCoinbaseHeight, str)
}
serializedHeightBytes := make([]byte, 8)
copy(serializedHeightBytes, sigScript[1:serializedLen+1])
serializedHeight := binary.LittleEndian.Uint64(serializedHeightBytes)
return serializedHeight, nil
}
// checkSerializedHeight checks if the signature script in the passed
// transaction starts with the serialized block height of wantHeight.
func checkSerializedHeight(block *util.Block) error {
coinbaseTx := block.CoinbaseTransaction()
serializedHeight, err := ExtractCoinbaseHeight(coinbaseTx)
if err != nil {
return err
}
wantHeight := block.Height()
if serializedHeight != wantHeight {
str := fmt.Sprintf("the coinbase signature script serialized "+
"block height is %d when %d was expected",
serializedHeight, wantHeight)
return ruleError(ErrBadCoinbaseHeight, str)
}
return nil
}
// checkBlockHeaderContext performs several validation checks on the block header
// which depend on its position within the block dag.
//
@ -669,7 +606,7 @@ func checkSerializedHeight(block *util.Block) error {
// the checkpoints are not performed.
//
// This function MUST be called with the dag state lock held (for writes).
func (dag *BlockDAG) checkBlockHeaderContext(header *wire.BlockHeader, bluestParent *blockNode, blockHeight uint64, fastAdd bool) error {
func (dag *BlockDAG) checkBlockHeaderContext(header *wire.BlockHeader, bluestParent *blockNode, blockChainHeight uint64, fastAdd bool) error {
if !fastAdd {
if err := dag.validateDifficulty(header, bluestParent); err != nil {
return err
@ -680,33 +617,18 @@ func (dag *BlockDAG) checkBlockHeaderContext(header *wire.BlockHeader, bluestPar
}
}
return dag.validateCheckpoints(header, blockHeight)
return dag.validateCheckpoints(header, blockChainHeight)
}
func (dag *BlockDAG) validateCheckpoints(header *wire.BlockHeader, blockHeight uint64) error {
// Ensure dag matches up to predetermined checkpoints.
func (dag *BlockDAG) validateCheckpoints(header *wire.BlockHeader, blockChainHeight uint64) error {
// Ensure DAG matches up to predetermined checkpoints.
blockHash := header.BlockHash()
if !dag.verifyCheckpoint(blockHeight, blockHash) {
str := fmt.Sprintf("block at height %d does not match "+
"checkpoint hash", blockHeight)
if !dag.verifyCheckpoint(blockChainHeight, blockHash) {
str := fmt.Sprintf("block at chain height %d does not match "+
"checkpoint hash", blockChainHeight)
return ruleError(ErrBadCheckpoint, str)
}
// Find the previous checkpoint and prevent blocks which fork the main
// dag before it. This prevents storage of new, otherwise valid,
// blocks which build off of old blocks that are likely at a much easier
// difficulty and therefore could be used to waste cache and disk space.
checkpointNode, err := dag.findPreviousCheckpoint()
if err != nil {
return err
}
if checkpointNode != nil && blockHeight < checkpointNode.height {
str := fmt.Sprintf("block at height %d forks the main dag "+
"before the previous checkpoint at height %d",
blockHeight, checkpointNode.height)
return ruleError(ErrForkTooOld, str)
}
return nil
}
@ -761,7 +683,7 @@ func validateParents(blockHeader *wire.BlockHeader, parents blockSet) error {
for queue.Len() > 0 {
current := queue.pop()
if parents.contains(current) {
return fmt.Errorf("Block %s is both a parent of %s and an"+
return fmt.Errorf("block %s is both a parent of %s and an"+
" ancestor of another parent",
current.hash,
blockHeader.BlockHash())
@ -799,35 +721,22 @@ func (dag *BlockDAG) checkBlockContext(block *util.Block, parents blockSet, blue
// Perform all block header related validation checks.
header := &block.MsgBlock().Header
if err = dag.checkBlockHeaderContext(header, bluestParent, block.Height(), fastAdd); err != nil {
if err = dag.checkBlockHeaderContext(header, bluestParent, block.ChainHeight(), fastAdd); err != nil {
return err
}
if !fastAdd {
if err := dag.validateAllTxsFinalized(block, header, bluestParent); err != nil {
return err
}
// Ensure coinbase starts with serialized block heights
err := checkSerializedHeight(block)
if err != nil {
return err
}
}
return nil
}
func (dag *BlockDAG) validateAllTxsFinalized(block *util.Block, header *wire.BlockHeader, bluestParent *blockNode) error {
blockTime := header.Timestamp
func (dag *BlockDAG) validateAllTxsFinalized(block *util.Block, node *blockNode, bluestParent *blockNode) error {
blockTime := block.MsgBlock().Header.Timestamp
if !block.IsGenesis() {
blockTime = bluestParent.PastMedianTime()
}
// Ensure all transactions in the block are finalized.
for _, tx := range block.Transactions() {
if !IsFinalizedTransaction(tx, block.Height(), blockTime) {
if !IsFinalizedTransaction(tx, node.blueScore, blockTime) {
str := fmt.Sprintf("block contains unfinalized "+
"transaction %s", tx.ID())
return ruleError(ErrUnfinalizedTx, str)
@ -848,8 +757,7 @@ func (dag *BlockDAG) validateAllTxsFinalized(block *util.Block, header *wire.Blo
// http://r6.ca/blog/20120206T005236Z.html.
//
// This function MUST be called with the dag state lock held (for reads).
func ensureNoDuplicateTx(block *blockNode, utxoSet UTXOSet,
transactions []*util.Tx) error {
func ensureNoDuplicateTx(utxoSet UTXOSet, transactions []*util.Tx) error {
// Fetch utxos for all of the transaction ouputs in this block.
// Typically, there will not be any utxos for any of the outputs.
fetchSet := make(map[wire.Outpoint]struct{})
@ -864,11 +772,9 @@ func ensureNoDuplicateTx(block *blockNode, utxoSet UTXOSet,
// Duplicate transactions are only allowed if the previous transaction
// is fully spent.
for outpoint := range fetchSet {
utxo, ok := utxoSet.Get(outpoint)
if ok {
if _, ok := utxoSet.Get(outpoint); ok {
str := fmt.Sprintf("tried to overwrite transaction %s "+
"at block chain-height %d that is not fully spent",
outpoint.TxID, utxo.BlockChainHeight())
"that is not fully spent", outpoint.TxID)
return ruleError(ErrOverwriteTx, str)
}
}
@ -886,7 +792,7 @@ func ensureNoDuplicateTx(block *blockNode, utxoSet UTXOSet,
//
// NOTE: The transaction MUST have already been sanity checked with the
// CheckTransactionSanity function prior to calling this function.
func CheckTransactionInputsAndCalulateFee(tx *util.Tx, txHeight uint64, utxoSet UTXOSet, dagParams *dagconfig.Params, fastAdd bool) (
func CheckTransactionInputsAndCalulateFee(tx *util.Tx, txBlueScore uint64, utxoSet UTXOSet, dagParams *dagconfig.Params, fastAdd bool) (
txFeeInSatoshi uint64, err error) {
// Block reward transactions (a.k.a. coinbase or fee transactions)
@ -909,7 +815,7 @@ func CheckTransactionInputsAndCalulateFee(tx *util.Tx, txHeight uint64, utxoSet
}
if !fastAdd {
if err = validateBlockRewardMaturity(dagParams, entry, txHeight, txIn); err != nil {
if err = validateBlockRewardMaturity(dagParams, entry, txBlueScore, txIn); err != nil {
return 0, err
}
}
@ -967,18 +873,18 @@ func CheckTransactionInputsAndCalulateFee(tx *util.Tx, txHeight uint64, utxoSet
return txFeeInSatoshi, nil
}
func validateBlockRewardMaturity(dagParams *dagconfig.Params, entry *UTXOEntry, txChainHeight uint64, txIn *wire.TxIn) error {
func validateBlockRewardMaturity(dagParams *dagconfig.Params, entry *UTXOEntry, txBlueScore uint64, txIn *wire.TxIn) error {
// Ensure the transaction is not spending coins which have not
// yet reached the required block reward maturity.
if entry.IsBlockReward() {
originChainHeight := entry.BlockChainHeight()
blocksSincePrev := txChainHeight - originChainHeight
if blocksSincePrev < dagParams.BlockRewardMaturity {
originBlueScore := entry.BlockBlueScore()
BlueScoreSincePrev := txBlueScore - originBlueScore
if BlueScoreSincePrev < dagParams.BlockRewardMaturity {
str := fmt.Sprintf("tried to spend block reward "+
"transaction output %s from chain-height %d "+
"at chain-height %d before required maturity "+
"of %d blocks", txIn.PreviousOutpoint,
originChainHeight, txChainHeight,
"transaction output %s from blue score %d "+
"to blue score %d before required maturity "+
"of %d", txIn.PreviousOutpoint,
originBlueScore, txBlueScore,
dagParams.BlockRewardMaturity)
return ruleError(ErrImmatureSpend, str)
}
@ -1002,7 +908,7 @@ func (dag *BlockDAG) checkConnectToPastUTXO(block *blockNode, pastUTXO UTXOSet,
transactions []*util.Tx, fastAdd bool) (compactFeeData, error) {
if !fastAdd {
err := ensureNoDuplicateTx(block, pastUTXO, transactions)
err := ensureNoDuplicateTx(pastUTXO, transactions)
if err != nil {
return nil, err
}
@ -1025,7 +931,7 @@ func (dag *BlockDAG) checkConnectToPastUTXO(block *blockNode, pastUTXO UTXOSet,
compactFeeFactory := newCompactFeeFactory()
for _, tx := range transactions {
txFee, err := CheckTransactionInputsAndCalulateFee(tx, block.height, pastUTXO,
txFee, err := CheckTransactionInputsAndCalulateFee(tx, block.blueScore, pastUTXO,
dag.dagParams, fastAdd)
if err != nil {
return nil, err
@ -1042,12 +948,12 @@ func (dag *BlockDAG) checkConnectToPastUTXO(block *blockNode, pastUTXO UTXOSet,
err = compactFeeFactory.add(txFee)
if err != nil {
return nil, fmt.Errorf("Error adding tx %s fee to compactFeeFactory: %s", tx.ID(), err)
return nil, fmt.Errorf("error adding tx %s fee to compactFeeFactory: %s", tx.ID(), err)
}
}
feeData, err := compactFeeFactory.data()
if err != nil {
return nil, fmt.Errorf("Error getting bytes of fee data: %s", err)
return nil, fmt.Errorf("error getting bytes of fee data: %s", err)
}
if !fastAdd {
@ -1060,7 +966,7 @@ func (dag *BlockDAG) checkConnectToPastUTXO(block *blockNode, pastUTXO UTXOSet,
for _, txOut := range transactions[0].MsgTx().TxOut {
totalSatoshiOut += txOut.Value
}
expectedSatoshiOut := CalcBlockSubsidy(block.height, dag.dagParams)
expectedSatoshiOut := CalcBlockSubsidy(block.blueScore, dag.dagParams)
if totalSatoshiOut > expectedSatoshiOut {
str := fmt.Sprintf("coinbase transaction for block pays %d "+
"which is more than expected value of %d",
@ -1076,7 +982,7 @@ func (dag *BlockDAG) checkConnectToPastUTXO(block *blockNode, pastUTXO UTXOSet,
// portion of block handling.
checkpoint := dag.LatestCheckpoint()
runScripts := true
if checkpoint != nil && block.height <= checkpoint.Height {
if checkpoint != nil && block.chainHeight <= checkpoint.ChainHeight {
runScripts = false
}
@ -1100,7 +1006,7 @@ func (dag *BlockDAG) checkConnectToPastUTXO(block *blockNode, pastUTXO UTXOSet,
if err != nil {
return nil, err
}
if !SequenceLockActive(sequenceLock, block.height,
if !SequenceLockActive(sequenceLock, block.blueScore,
medianTime) {
str := fmt.Sprintf("block contains " +
"transaction whose input sequence " +
@ -1161,18 +1067,6 @@ func validateSigopsCount(pastUTXO UTXOSet, transactions []*util.Tx) error {
return nil
}
// countSpentOutputs returns the number of utxos the passed block spends.
func countSpentOutputs(block *util.Block) int {
// Exclude the block reward transactions since they can't spend anything.
var numSpent int
for _, tx := range block.Transactions()[1:] {
if !IsFeeTransaction(tx) {
numSpent += len(tx.MsgTx().TxIn)
}
}
return numSpent
}
// CheckConnectBlockTemplate fully validates that connecting the passed block to
// the DAG does not violate any consensus rules, aside from the proof of
// work requirement.

View File

@ -6,7 +6,6 @@ package blockdag
import (
"math"
"reflect"
"testing"
"time"
@ -22,8 +21,8 @@ import (
func TestSequenceLocksActive(t *testing.T) {
seqLock := func(h int64, s int64) *SequenceLock {
return &SequenceLock{
Seconds: s,
BlockChainHeight: h,
Seconds: s,
BlockBlueScore: h,
}
}
@ -126,7 +125,6 @@ func TestCheckConnectBlockTemplate(t *testing.T) {
}
// Block 3a should connect even though it does not build on dag tips.
blocks[5].SetHeight(3) // set height manually because it was set to 0 in loadBlocks
err = dag.CheckConnectBlockTemplateNoLock(blocks[5])
if err != nil {
t.Fatal("CheckConnectBlockTemplate: Recieved unexpected error on " +
@ -137,7 +135,6 @@ func TestCheckConnectBlockTemplate(t *testing.T) {
invalidPowMsgBlock := *blocks[4].MsgBlock()
invalidPowMsgBlock.Header.Nonce++
invalidPowBlock := util.NewBlock(&invalidPowMsgBlock)
invalidPowBlock.SetHeight(blocks[4].Height())
err = dag.CheckConnectBlockTemplateNoLock(invalidPowBlock)
if err != nil {
t.Fatalf("CheckConnectBlockTemplate: Received unexpected error on "+
@ -476,74 +473,6 @@ func TestCheckBlockSanity(t *testing.T) {
}
}
// TestCheckSerializedHeight tests the checkSerializedHeight function with
// various serialized heights and also does negative tests to ensure errors
// and handled properly.
func TestCheckSerializedHeight(t *testing.T) {
// Create an empty coinbase template to be used in the tests below.
coinbaseOutpoint := wire.NewOutpoint(&daghash.TxID{}, math.MaxUint32)
coinbaseTx := wire.NewNativeMsgTx(1, []*wire.TxIn{wire.NewTxIn(coinbaseOutpoint, nil)}, nil)
// Expected rule errors.
missingHeightError := RuleError{
ErrorCode: ErrMissingCoinbaseHeight,
}
badHeightError := RuleError{
ErrorCode: ErrBadCoinbaseHeight,
}
tests := []struct {
sigScript []byte // Serialized data
wantHeight uint64 // Expected height
err error // Expected error type
}{
// No serialized height length.
{[]byte{}, 0, missingHeightError},
// Serialized height length with no height bytes.
{[]byte{0x02}, 0, missingHeightError},
// Serialized height length with too few height bytes.
{[]byte{0x02, 0x4a}, 0, missingHeightError},
// Serialized height that needs 2 bytes to encode.
{[]byte{0x02, 0x4a, 0x52}, 21066, nil},
// Serialized height that needs 2 bytes to encode, but backwards
// endianness.
{[]byte{0x02, 0x4a, 0x52}, 19026, badHeightError},
// Serialized height that needs 3 bytes to encode.
{[]byte{0x03, 0x40, 0x0d, 0x03}, 200000, nil},
// Serialized height that needs 3 bytes to encode, but backwards
// endianness.
{[]byte{0x03, 0x40, 0x0d, 0x03}, 1074594560, badHeightError},
}
t.Logf("Running %d tests", len(tests))
for i, test := range tests {
msgTx := coinbaseTx.Copy()
msgTx.TxIn[0].SignatureScript = test.sigScript
msgBlock := wire.NewMsgBlock(wire.NewBlockHeader(1, []*daghash.Hash{}, &daghash.Hash{}, &daghash.Hash{}, &daghash.Hash{}, 0, 0))
msgBlock.AddTransaction(msgTx)
block := util.NewBlock(msgBlock)
block.SetHeight(test.wantHeight)
err := checkSerializedHeight(block)
if reflect.TypeOf(err) != reflect.TypeOf(test.err) {
t.Errorf("checkSerializedHeight #%d wrong error type "+
"got: %v <%T>, want: %T", i, err, err, test.err)
continue
}
if rerr, ok := err.(RuleError); ok {
trerr := test.err.(RuleError)
if rerr.ErrorCode != trerr.ErrorCode {
t.Errorf("checkSerializedHeight #%d wrong "+
"error code got: %v, want: %v", i,
rerr.ErrorCode, trerr.ErrorCode)
continue
}
}
}
}
func TestPastMedianTime(t *testing.T) {
dag := newTestDAG(&dagconfig.MainNetParams)
tip := dag.genesis
@ -561,7 +490,7 @@ func TestPastMedianTime(t *testing.T) {
}
// Checks that a block is valid if it has timestamp equals to past median time
height := tip.height + 1
chainHeight := tip.chainHeight + 1
node := newTestNode(setFromSlice(tip),
blockVersion,
0,
@ -569,14 +498,14 @@ func TestPastMedianTime(t *testing.T) {
dagconfig.MainNetParams.K)
header := node.Header()
err := dag.checkBlockHeaderContext(header, node.parents.bluest(), height, false)
err := dag.checkBlockHeaderContext(header, node.parents.bluest(), chainHeight, false)
if err != nil {
t.Errorf("TestPastMedianTime: unexpected error from checkBlockHeaderContext: %v"+
"(a block with timestamp equals to past median time should be valid)", err)
}
// Checks that a block is valid if its timestamp is after past median time
height = tip.height + 1
chainHeight = tip.chainHeight + 1
node = newTestNode(setFromSlice(tip),
blockVersion,
0,
@ -584,14 +513,14 @@ func TestPastMedianTime(t *testing.T) {
dagconfig.MainNetParams.K)
header = node.Header()
err = dag.checkBlockHeaderContext(header, node.parents.bluest(), height, false)
err = dag.checkBlockHeaderContext(header, node.parents.bluest(), chainHeight, false)
if err != nil {
t.Errorf("TestPastMedianTime: unexpected error from checkBlockHeaderContext: %v"+
"(a block with timestamp bigger than past median time should be valid)", err)
}
// Checks that a block is invalid if its timestamp is before past median time
height = tip.height + 1
chainHeight = tip.chainHeight + 1
node = newTestNode(setFromSlice(tip),
blockVersion,
0,
@ -599,7 +528,7 @@ func TestPastMedianTime(t *testing.T) {
dagconfig.MainNetParams.K)
header = node.Header()
err = dag.checkBlockHeaderContext(header, node.parents.bluest(), height, false)
err = dag.checkBlockHeaderContext(header, node.parents.bluest(), chainHeight, false)
if err == nil {
t.Errorf("TestPastMedianTime: unexpected success: block should be invalid if its timestamp is before past median time")
}
@ -730,7 +659,7 @@ func TestCheckTransactionSanity(t *testing.T) {
test.extraModificationsFunc(tx)
}
err := CheckTransactionSanity(util.NewTx(tx), &test.nodeSubnetworkID, false)
err := CheckTransactionSanity(util.NewTx(tx), &test.nodeSubnetworkID)
if e := checkRuleError(err, test.expectedErr); e != nil {
t.Errorf("TestCheckTransactionSanity: '%s': %v", test.name, e)
continue

View File

@ -251,9 +251,9 @@ func (dag *BlockDAG) warnUnknownRuleActivations(node *blockNode) error {
case ThresholdLockedIn:
window := checker.MinerConfirmationWindow()
activationHeight := window - (node.height % window)
activationChainHeight := window - (node.chainHeight % window)
log.Warnf("Unknown new rules are about to activate in "+
"%d blocks (bit %d)", activationHeight, bit)
"%d blocks (bit %d)", activationChainHeight, bit)
}
}

View File

@ -258,13 +258,11 @@ type GetPeerInfoResult struct {
// command when the verbose flag is set. When the verbose flag is not set,
// getrawmempool returns an array of transaction hashes.
type GetRawMempoolVerboseResult struct {
Size int32 `json:"size"`
Fee float64 `json:"fee"`
Time int64 `json:"time"`
Height uint64 `json:"height"`
StartingPriority float64 `json:"startingPriority"`
CurrentPriority float64 `json:"currentPriority"`
Depends []string `json:"depends"`
Size int32 `json:"size"`
Fee float64 `json:"fee"`
Time int64 `json:"time"`
Height uint64 `json:"height"`
Depends []string `json:"depends"`
}
// ScriptPubKeyResult models the scriptPubKey data of a tx script. It is

View File

@ -68,19 +68,6 @@ func NewEncryptWalletCmd(passphrase string) *EncryptWalletCmd {
}
}
// EstimateFeeCmd defines the estimateFee JSON-RPC command.
type EstimateFeeCmd struct {
NumBlocks int64
}
// NewEstimateFeeCmd returns a new instance which can be used to issue a
// estimateFee JSON-RPC command.
func NewEstimateFeeCmd(numBlocks int64) *EstimateFeeCmd {
return &EstimateFeeCmd{
NumBlocks: numBlocks,
}
}
// EstimatePriorityCmd defines the estimatePriority JSON-RPC command.
type EstimatePriorityCmd struct {
NumBlocks int64
@ -648,7 +635,6 @@ func init() {
MustRegisterCmd("createMultisig", (*CreateMultisigCmd)(nil), flags)
MustRegisterCmd("dumpPrivKey", (*DumpPrivKeyCmd)(nil), flags)
MustRegisterCmd("encryptWallet", (*EncryptWalletCmd)(nil), flags)
MustRegisterCmd("estimateFee", (*EstimateFeeCmd)(nil), flags)
MustRegisterCmd("estimatePriority", (*EstimatePriorityCmd)(nil), flags)
MustRegisterCmd("getAccount", (*GetAccountCmd)(nil), flags)
MustRegisterCmd("getAccountAddress", (*GetAccountAddressCmd)(nil), flags)

View File

@ -102,19 +102,6 @@ func TestWalletSvrCmds(t *testing.T) {
Passphrase: "pass",
},
},
{
name: "estimateFee",
newCmd: func() (interface{}, error) {
return btcjson.NewCmd("estimateFee", 6)
},
staticCmd: func() interface{} {
return btcjson.NewEstimateFeeCmd(6)
},
marshalled: `{"jsonrpc":"1.0","method":"estimateFee","params":[6],"id":1}`,
unmarshalled: &btcjson.EstimateFeeCmd{
NumBlocks: 6,
},
},
{
name: "estimatePriority",
newCmd: func() (interface{}, error) {

View File

@ -51,20 +51,20 @@ func findCandidates(dag *blockdag.BlockDAG, highestTipHash *daghash.Hash) ([]*da
// Set the latest checkpoint to the genesis block if there isn't
// already one.
latestCheckpoint = &dagconfig.Checkpoint{
Hash: activeNetParams.GenesisHash,
Height: 0,
Hash: activeNetParams.GenesisHash,
ChainHeight: 0,
}
}
// The latest known block must be at least the last known checkpoint
// plus required checkpoint confirmations.
checkpointConfirmations := uint64(blockdag.CheckpointConfirmations)
requiredHeight := latestCheckpoint.Height + checkpointConfirmations
if block.Height() < requiredHeight {
return nil, fmt.Errorf("the block database is only at height "+
"%d which is less than the latest checkpoint height "+
requiredChainHeight := latestCheckpoint.ChainHeight + checkpointConfirmations
if block.ChainHeight() < requiredChainHeight {
return nil, fmt.Errorf("the block database is only at chain "+
"height %d which is less than the latest checkpoint chain height "+
"of %d plus required confirmations of %d",
block.Height(), latestCheckpoint.Height,
block.ChainHeight(), latestCheckpoint.ChainHeight,
checkpointConfirmations)
}
@ -72,11 +72,11 @@ func findCandidates(dag *blockdag.BlockDAG, highestTipHash *daghash.Hash) ([]*da
// genesis block, so long as the DAG has at least the required number
// of confirmations (which is enforced above).
if len(activeNetParams.Checkpoints) == 0 {
requiredHeight = 1
requiredChainHeight = 1
}
// Indeterminate progress setup.
numBlocksToTest := block.Height() - requiredHeight
numBlocksToTest := block.ChainHeight() - requiredChainHeight
progressInterval := (numBlocksToTest / 100) + 1 // min 1
fmt.Print("Searching for candidates")
defer fmt.Println()
@ -84,7 +84,7 @@ func findCandidates(dag *blockdag.BlockDAG, highestTipHash *daghash.Hash) ([]*da
// Loop backwards through the DAG to find checkpoint candidates.
candidates := make([]*dagconfig.Checkpoint, 0, cfg.NumCandidates)
numTested := uint64(0)
for len(candidates) < cfg.NumCandidates && block.Height() > requiredHeight {
for len(candidates) < cfg.NumCandidates && block.ChainHeight() > requiredChainHeight {
// Display progress.
if numTested%progressInterval == 0 {
fmt.Print(".")
@ -100,8 +100,8 @@ func findCandidates(dag *blockdag.BlockDAG, highestTipHash *daghash.Hash) ([]*da
// checkpoint candidate.
if isCandidate {
checkpoint := dagconfig.Checkpoint{
Height: block.Height(),
Hash: block.Hash(),
ChainHeight: block.ChainHeight(),
Hash: block.Hash(),
}
candidates = append(candidates, &checkpoint)
}
@ -123,12 +123,12 @@ func findCandidates(dag *blockdag.BlockDAG, highestTipHash *daghash.Hash) ([]*da
func showCandidate(candidateNum int, checkpoint *dagconfig.Checkpoint) {
if cfg.UseGoOutput {
fmt.Printf("Candidate %d -- {%d, newShaHashFromStr(\"%s\")},\n",
candidateNum, checkpoint.Height, checkpoint.Hash)
candidateNum, checkpoint.ChainHeight, checkpoint.Hash)
return
}
fmt.Printf("Candidate %d -- Height: %d, Hash: %s\n", candidateNum,
checkpoint.Height, checkpoint.Hash)
fmt.Printf("Candidate %d -- ChainHeight: %d, Hash: %s\n", candidateNum,
checkpoint.ChainHeight, checkpoint.Hash)
}
@ -162,11 +162,11 @@ func main() {
// Get the latest block hash and height from the database and report
// status.
fmt.Printf("Block database loaded with block height %d\n", dag.Height())
fmt.Printf("Block database loaded with block chain height %d\n", dag.ChainHeight())
// Find checkpoint candidates.
highestTipHash := dag.HighestTipHash()
candidates, err := findCandidates(dag, highestTipHash)
selectedTipHash := dag.SelectedTipHash()
candidates, err := findCandidates(dag, selectedTipHash)
if err != nil {
fmt.Fprintln(os.Stderr, "Unable to identify candidates:", err)
return

View File

@ -23,7 +23,6 @@ import (
"github.com/daglabs/btcd/dagconfig"
"github.com/daglabs/btcd/database"
"github.com/daglabs/btcd/logger"
"github.com/daglabs/btcd/mempool"
"github.com/daglabs/btcd/util"
"github.com/daglabs/btcd/util/daghash"
"github.com/daglabs/btcd/util/network"
@ -48,7 +47,6 @@ const (
defaultMaxRPCWebsockets = 25
defaultMaxRPCConcurrentReqs = 20
defaultDbType = "ffldb"
defaultFreeTxRelayLimit = 15.0
defaultBlockMinSize = 0
defaultBlockMaxSize = 750000
blockMaxSizeMin = 1000
@ -147,14 +145,11 @@ type configFlags struct {
DebugLevel string `short:"d" long:"debuglevel" description:"Logging level for all subsystems {trace, debug, info, warn, error, critical} -- You may also specify <subsystem>=<level>,<subsystem2>=<level>,... to set the log level for individual subsystems -- Use show to list available subsystems"`
Upnp bool `long:"upnp" description:"Use UPnP to map our listening port outside of NAT"`
MinRelayTxFee float64 `long:"minrelaytxfee" description:"The minimum transaction fee in BTC/kB to be considered a non-zero fee."`
FreeTxRelayLimit float64 `long:"limitfreerelay" description:"Limit relay of transactions with no transaction fee to the given amount in thousands of bytes per minute"`
NoRelayPriority bool `long:"norelaypriority" description:"Do not require free or low-fee transactions to have high priority for relaying"`
MaxOrphanTxs int `long:"maxorphantx" description:"Max number of orphan transactions to keep in memory"`
Generate bool `long:"generate" description:"Generate (mine) bitcoins using the CPU"`
MiningAddrs []string `long:"miningaddr" description:"Add the specified payment address to the list of addresses to use for generated blocks -- At least one address is required if the generate option is set"`
BlockMinSize uint32 `long:"blockminsize" description:"Mininum block size in bytes to be used when creating a block"`
BlockMaxSize uint32 `long:"blockmaxsize" description:"Maximum block size in bytes to be used when creating a block"`
BlockPrioritySize uint32 `long:"blockprioritysize" description:"Size in bytes for high-priority/low-fee transactions when creating a block"`
UserAgentComments []string `long:"uacomment" description:"Comment to add to the user agent -- See BIP 14 for more information."`
NoPeerBloomFilters bool `long:"nopeerbloomfilters" description:"Disable bloom filtering support"`
EnableCFilters bool `long:"enablecfilters" description:"Enable committed filtering (CF) support"`
@ -243,8 +238,8 @@ func newCheckpointFromStr(checkpoint string) (dagconfig.Checkpoint, error) {
}
return dagconfig.Checkpoint{
Height: uint64(height),
Hash: hash,
ChainHeight: uint64(height),
Hash: hash,
}, nil
}
@ -317,11 +312,8 @@ func loadConfig() (*Config, []string, error) {
DbType: defaultDbType,
RPCKey: defaultRPCKeyFile,
RPCCert: defaultRPCCertFile,
MinRelayTxFee: mempool.DefaultMinRelayTxFee.ToBTC(),
FreeTxRelayLimit: defaultFreeTxRelayLimit,
BlockMinSize: defaultBlockMinSize,
BlockMaxSize: defaultBlockMaxSize,
BlockPrioritySize: mempool.DefaultBlockPrioritySize,
MaxOrphanTxs: defaultMaxOrphanTransactions,
SigCacheMaxSize: defaultSigCacheMaxSize,
Generate: defaultGenerate,
@ -690,8 +682,7 @@ func loadConfig() (*Config, []string, error) {
return nil, nil, err
}
// Limit the block priority and minimum block sizes to max block size.
cfg.BlockPrioritySize = minUint32(cfg.BlockPrioritySize, cfg.BlockMaxSize)
// Limit minimum block sizes to max block size.
cfg.BlockMinSize = minUint32(cfg.BlockMinSize, cfg.BlockMaxSize)
// Look for illegal characters in the user agent comments.

View File

@ -57,8 +57,8 @@ const phantomK = 10
// documentation for blockchain.IsCheckpointCandidate for details on the
// selection criteria.
type Checkpoint struct {
Height uint64
Hash *daghash.Hash
ChainHeight uint64
Hash *daghash.Hash
}
// ConsensusDeployment defines details related to a specific consensus rule

View File

@ -95,7 +95,7 @@ func standardCoinbaseScript(nextBlockHeight uint64, extraNonce uint64) ([]byte,
// createCoinbaseTx returns a coinbase transaction paying an appropriate
// subsidy based on the passed block height to the provided address.
func createCoinbaseTx(coinbaseScript []byte, nextBlockHeight uint64,
func createCoinbaseTx(coinbaseScript []byte, nextBlueScore uint64,
addr util.Address, mineTo []wire.TxOut,
net *dagconfig.Params) (*util.Tx, error) {
@ -116,7 +116,7 @@ func createCoinbaseTx(coinbaseScript []byte, nextBlockHeight uint64,
txOuts := []*wire.TxOut{}
if len(mineTo) == 0 {
txOuts = append(txOuts, &wire.TxOut{
Value: blockdag.CalcBlockSubsidy(nextBlockHeight, net),
Value: blockdag.CalcBlockSubsidy(nextBlueScore, net),
PkScript: pkScript,
})
} else {
@ -138,20 +138,20 @@ func CreateBlock(parentBlock *util.Block, inclusionTxs []*util.Tx,
mineTo []wire.TxOut, net *dagconfig.Params) (*util.Block, error) {
var (
parentHash *daghash.Hash
blockHeight uint64
parentBlockTime time.Time
parentHash *daghash.Hash
blockChainHeight uint64
parentBlockTime time.Time
)
// If the parent block isn't specified, then we'll construct a block
// that builds off of the genesis block for the chain.
if parentBlock == nil {
parentHash = net.GenesisHash
blockHeight = 1
blockChainHeight = 1
parentBlockTime = net.GenesisBlock.Header.Timestamp.Add(time.Minute)
} else {
parentHash = parentBlock.Hash()
blockHeight = parentBlock.Height() + 1
blockChainHeight = parentBlock.ChainHeight() + 1
parentBlockTime = parentBlock.MsgBlock().Header.Timestamp
}
@ -167,11 +167,11 @@ func CreateBlock(parentBlock *util.Block, inclusionTxs []*util.Tx,
}
extraNonce := uint64(0)
coinbaseScript, err := standardCoinbaseScript(blockHeight, extraNonce)
coinbaseScript, err := standardCoinbaseScript(blockChainHeight, extraNonce)
if err != nil {
return nil, err
}
coinbaseTx, err := createCoinbaseTx(coinbaseScript, blockHeight,
coinbaseTx, err := createCoinbaseTx(coinbaseScript, blockChainHeight,
miningAddr, mineTo, net)
if err != nil {
return nil, err
@ -203,6 +203,5 @@ func CreateBlock(parentBlock *util.Block, inclusionTxs []*util.Tx,
}
utilBlock := util.NewBlock(&block)
utilBlock.SetHeight(blockHeight)
return utilBlock, nil
}

View File

@ -435,7 +435,7 @@ func (h *Harness) GenerateAndSubmitBlockWithCustomCoinbaseOutputs(
return nil, err
}
parentBlock := util.NewBlock(mBlock)
parentBlock.SetHeight(parentBlockHeight)
parentBlock.SetChainHeight(parentBlockHeight)
// Create a new block including the specified transactions
newBlock, err := CreateBlock(parentBlock, txns, blockVersion,

View File

@ -86,8 +86,6 @@ func extractRejectCode(err error) (wire.RejectCode, bool) {
case blockdag.ErrDifficultyTooLow:
fallthrough
case blockdag.ErrBadCheckpoint:
fallthrough
case blockdag.ErrForkTooOld:
code = wire.RejectCheckpoint
// Everything else is due to the block or transaction being invalid.

View File

@ -1,755 +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 mempool
import (
"bytes"
"encoding/binary"
"errors"
"fmt"
"io"
"math"
"math/rand"
"sort"
"strings"
"sync"
"github.com/daglabs/btcd/blockdag"
"github.com/daglabs/btcd/util"
"github.com/daglabs/btcd/util/daghash"
)
// TODO incorporate Alex Morcos' modifications to Gavin's initial model
// https://lists.linuxfoundation.org/pipermail/bitcoin-dev/2014-October/006824.html
const (
// estimateFeeDepth is the maximum number of blocks before a transaction
// is confirmed that we want to track.
estimateFeeDepth = 25
// estimateFeeBinSize is the number of txs stored in each bin.
estimateFeeBinSize = 100
// estimateFeeMaxReplacements is the max number of replacements that
// can be made by the txs found in a given block.
estimateFeeMaxReplacements = 10
// DefaultEstimateFeeMaxRollback is the default number of rollbacks
// allowed by the fee estimator for orphaned blocks.
DefaultEstimateFeeMaxRollback = 2
// DefaultEstimateFeeMinRegisteredBlocks is the default minimum
// number of blocks which must be observed by the fee estimator before
// it will provide fee estimations.
DefaultEstimateFeeMinRegisteredBlocks = 3
bytePerKb = 1000
btcPerSatoshi = 1E-8
)
var (
// EstimateFeeDatabaseKey is the key that we use to
// store the fee estimator in the database.
EstimateFeeDatabaseKey = []byte("estimatefee")
)
// SatoshiPerByte is number with units of satoshis per byte.
type SatoshiPerByte float64
// BtcPerKilobyte is number with units of bitcoins per kilobyte.
type BtcPerKilobyte float64
// ToBtcPerKb returns a float value that represents the given
// SatoshiPerByte converted to satoshis per kb.
func (rate SatoshiPerByte) ToBtcPerKb() BtcPerKilobyte {
// If our rate is the error value, return that.
if rate == SatoshiPerByte(-1.0) {
return -1.0
}
return BtcPerKilobyte(float64(rate) * bytePerKb * btcPerSatoshi)
}
// Fee returns the fee for a transaction of a given size for
// the given fee rate.
func (rate SatoshiPerByte) Fee(size uint32) util.Amount {
// If our rate is the error value, return that.
if rate == SatoshiPerByte(-1) {
return math.MaxUint64
}
return util.Amount(float64(rate) * float64(size))
}
// NewSatoshiPerByte creates a SatoshiPerByte from an Amount and a
// size in bytes.
func NewSatoshiPerByte(fee util.Amount, size uint32) SatoshiPerByte {
return SatoshiPerByte(float64(fee) / float64(size))
}
// observedTransaction represents an observed transaction and some
// additional data required for the fee estimation algorithm.
type observedTransaction struct {
// A transaction ID.
txID *daghash.TxID
// The fee per byte of the transaction in satoshis.
feeRate SatoshiPerByte
// The block height when it was observed.
observed uint64
// The height of the block in which it was mined.
// If the transaction has not yet been mined, it is zero.
mined uint64
}
func (o *observedTransaction) Serialize(w io.Writer) {
binary.Write(w, binary.BigEndian, o.txID)
binary.Write(w, binary.BigEndian, o.feeRate)
binary.Write(w, binary.BigEndian, o.observed)
binary.Write(w, binary.BigEndian, o.mined)
}
func deserializeObservedTransaction(r io.Reader) (*observedTransaction, error) {
ot := observedTransaction{
txID: &daghash.TxID{},
}
// The first 32 bytes should be a hash.
binary.Read(r, binary.BigEndian, ot.txID)
// The next 8 are SatoshiPerByte
binary.Read(r, binary.BigEndian, &ot.feeRate)
// And next there are two uint32's.
binary.Read(r, binary.BigEndian, &ot.observed)
binary.Read(r, binary.BigEndian, &ot.mined)
return &ot, nil
}
// registeredBlock has the hash of a block and the list of transactions
// it mined which had been previously observed by the FeeEstimator. It
// is used if Rollback is called to reverse the effect of registering
// a block.
type registeredBlock struct {
hash *daghash.Hash
transactions []*observedTransaction
}
func (rb *registeredBlock) serialize(w io.Writer, txs map[*observedTransaction]uint32) {
binary.Write(w, binary.BigEndian, rb.hash)
binary.Write(w, binary.BigEndian, uint32(len(rb.transactions)))
for _, o := range rb.transactions {
binary.Write(w, binary.BigEndian, txs[o])
}
}
// FeeEstimator manages the data necessary to create
// fee estimations. It is safe for concurrent access.
type FeeEstimator struct {
maxRollback uint32
binSize int32
// The maximum number of replacements that can be made in a single
// bin per block. Default is estimateFeeMaxReplacements
maxReplacements int32
// The minimum number of blocks that can be registered with the fee
// estimator before it will provide answers.
minRegisteredBlocks uint32
// The last known height.
lastKnownHeight uint64
// The number of blocks that have been registered.
numBlocksRegistered uint32
mtx sync.RWMutex
observed map[daghash.TxID]*observedTransaction
bin [estimateFeeDepth][]*observedTransaction
// The cached estimates.
cached []SatoshiPerByte
// Transactions that have been removed from the bins. This allows us to
// revert in case of an orphaned block.
dropped []*registeredBlock
}
// NewFeeEstimator creates a FeeEstimator for which at most maxRollback blocks
// can be unregistered and which returns an error unless minRegisteredBlocks
// have been registered with it.
func NewFeeEstimator(maxRollback, minRegisteredBlocks uint32) *FeeEstimator {
return &FeeEstimator{
maxRollback: maxRollback,
minRegisteredBlocks: minRegisteredBlocks,
lastKnownHeight: blockdag.UnminedChainHeight,
binSize: estimateFeeBinSize,
maxReplacements: estimateFeeMaxReplacements,
observed: make(map[daghash.TxID]*observedTransaction),
dropped: make([]*registeredBlock, 0, maxRollback),
}
}
// ObserveTransaction is called when a new transaction is observed in the mempool.
func (ef *FeeEstimator) ObserveTransaction(t *TxDesc) {
ef.mtx.Lock()
defer ef.mtx.Unlock()
// If we haven't seen a block yet we don't know when this one arrived,
// so we ignore it.
if ef.lastKnownHeight == blockdag.UnminedChainHeight {
return
}
txID := t.Tx.ID()
if _, ok := ef.observed[*txID]; !ok {
size := uint32(t.Tx.MsgTx().SerializeSize())
ef.observed[*txID] = &observedTransaction{
txID: txID,
feeRate: NewSatoshiPerByte(util.Amount(t.Fee), size),
observed: t.Height,
mined: blockdag.UnminedChainHeight,
}
}
}
// RegisterBlock informs the fee estimator of a new block to take into account.
func (ef *FeeEstimator) RegisterBlock(block *util.Block) error {
ef.mtx.Lock()
defer ef.mtx.Unlock()
// The previous sorted list is invalid, so delete it.
ef.cached = nil
height := block.Height()
if height != ef.lastKnownHeight+1 && ef.lastKnownHeight != blockdag.UnminedChainHeight {
return fmt.Errorf("intermediate block not recorded; current height is %d; new height is %d",
ef.lastKnownHeight, height)
}
// Update the last known height.
ef.lastKnownHeight = height
ef.numBlocksRegistered++
// Randomly order txs in block.
transactions := make(map[*util.Tx]struct{})
for _, t := range block.Transactions() {
transactions[t] = struct{}{}
}
// Count the number of replacements we make per bin so that we don't
// replace too many.
var replacementCounts [estimateFeeDepth]int
// Keep track of which txs were dropped in case of an orphan block.
dropped := &registeredBlock{
hash: block.Hash(),
transactions: make([]*observedTransaction, 0, 100),
}
// Go through the txs in the block.
for t := range transactions {
txID := *t.ID()
// Have we observed this tx in the mempool?
o, ok := ef.observed[txID]
if !ok {
continue
}
// Put the observed tx in the oppropriate bin.
blocksToConfirm := height - o.observed - 1
// This shouldn't happen if the fee estimator works correctly,
// but return an error if it does.
if o.mined != blockdag.UnminedChainHeight {
log.Error("Estimate fee: transaction ", txID.String(), " has already been mined")
return errors.New("Transaction has already been mined")
}
// This shouldn't happen but check just in case to avoid
// an out-of-bounds array index later.
if blocksToConfirm >= estimateFeeDepth {
continue
}
// Make sure we do not replace too many transactions per min.
if replacementCounts[blocksToConfirm] == int(ef.maxReplacements) {
continue
}
o.mined = height
replacementCounts[blocksToConfirm]++
bin := ef.bin[blocksToConfirm]
// Remove a random element and replace it with this new tx.
if len(bin) == int(ef.binSize) {
// Don't drop transactions we have just added from this same block.
l := int(ef.binSize) - replacementCounts[blocksToConfirm]
drop := rand.Intn(l)
dropped.transactions = append(dropped.transactions, bin[drop])
bin[drop] = bin[l-1]
bin[l-1] = o
} else {
bin = append(bin, o)
}
ef.bin[blocksToConfirm] = bin
}
// Go through the mempool for txs that have been in too long.
for hash, o := range ef.observed {
if o.mined == blockdag.UnminedChainHeight && height-o.observed >= estimateFeeDepth {
delete(ef.observed, hash)
}
}
// Add dropped list to history.
if ef.maxRollback == 0 {
return nil
}
if uint32(len(ef.dropped)) == ef.maxRollback {
ef.dropped = append(ef.dropped[1:], dropped)
} else {
ef.dropped = append(ef.dropped, dropped)
}
return nil
}
// LastKnownHeight returns the height of the last block which was registered.
func (ef *FeeEstimator) LastKnownHeight() uint64 {
ef.mtx.Lock()
defer ef.mtx.Unlock()
return ef.lastKnownHeight
}
// Rollback unregisters a recently registered block from the FeeEstimator.
// This can be used to reverse the effect of an orphaned block on the fee
// estimator. The maximum number of rollbacks allowed is given by
// maxRollbacks.
//
// Note: not everything can be rolled back because some transactions are
// deleted if they have been observed too long ago. That means the result
// of Rollback won't always be exactly the same as if the last block had not
// happened, but it should be close enough.
func (ef *FeeEstimator) Rollback(hash *daghash.Hash) error {
ef.mtx.Lock()
defer ef.mtx.Unlock()
// Find this block in the stack of recent registered blocks.
var n int
for n = 1; n <= len(ef.dropped); n++ {
if ef.dropped[len(ef.dropped)-n].hash.IsEqual(hash) {
break
}
}
if n > len(ef.dropped) {
return errors.New("no such block was recently registered")
}
for i := 0; i < n; i++ {
ef.rollback()
}
return nil
}
// rollback rolls back the effect of the last block in the stack
// of registered blocks.
func (ef *FeeEstimator) rollback() {
// The previous sorted list is invalid, so delete it.
ef.cached = nil
// pop the last list of dropped txs from the stack.
last := len(ef.dropped) - 1
if last == -1 {
// Cannot really happen because the exported calling function
// only rolls back a block already known to be in the list
// of dropped transactions.
return
}
dropped := ef.dropped[last]
// where we are in each bin as we replace txs?
var replacementCounters [estimateFeeDepth]int
// Go through the txs in the dropped block.
for _, o := range dropped.transactions {
// Which bin was this tx in?
blocksToConfirm := o.mined - o.observed - 1
bin := ef.bin[blocksToConfirm]
var counter = replacementCounters[blocksToConfirm]
// Continue to go through that bin where we left off.
for {
if counter >= len(bin) {
// Panic, as we have entered an unrecoverable invalid state.
panic(errors.New("illegal state: cannot rollback dropped transaction"))
}
prev := bin[counter]
if prev.mined == ef.lastKnownHeight {
prev.mined = blockdag.UnminedChainHeight
bin[counter] = o
counter++
break
}
counter++
}
replacementCounters[blocksToConfirm] = counter
}
// Continue going through bins to find other txs to remove
// which did not replace any other when they were entered.
for i, j := range replacementCounters {
for {
l := len(ef.bin[i])
if j >= l {
break
}
prev := ef.bin[i][j]
if prev.mined == ef.lastKnownHeight {
prev.mined = blockdag.UnminedChainHeight
newBin := append(ef.bin[i][0:j], ef.bin[i][j+1:l]...)
// TODO This line should prevent an unintentional memory
// leak but it causes a panic when it is uncommented.
// ef.bin[i][j] = nil
ef.bin[i] = newBin
continue
}
j++
}
}
ef.dropped = ef.dropped[0:last]
// The number of blocks the fee estimator has seen is decrimented.
ef.numBlocksRegistered--
ef.lastKnownHeight--
}
// estimateFeeSet is a set of txs that can that is sorted
// by the fee per kb rate.
type estimateFeeSet struct {
feeRate []SatoshiPerByte
bin [estimateFeeDepth]uint32
}
func (b *estimateFeeSet) Len() int { return len(b.feeRate) }
func (b *estimateFeeSet) Less(i, j int) bool {
return b.feeRate[i] > b.feeRate[j]
}
func (b *estimateFeeSet) Swap(i, j int) {
b.feeRate[i], b.feeRate[j] = b.feeRate[j], b.feeRate[i]
}
// estimateFee returns the estimated fee for a transaction
// to confirm in confirmations blocks from now, given
// the data set we have collected.
func (b *estimateFeeSet) estimateFee(confirmations int) SatoshiPerByte {
if confirmations <= 0 {
return SatoshiPerByte(math.Inf(1))
}
if confirmations > estimateFeeDepth {
return 0
}
// We don't have any transactions!
if len(b.feeRate) == 0 {
return 0
}
var min, max int = 0, 0
for i := 0; i < confirmations-1; i++ {
min += int(b.bin[i])
}
max = min + int(b.bin[confirmations-1]) - 1
if max < min {
max = min
}
feeIndex := (min + max) / 2
if feeIndex >= len(b.feeRate) {
feeIndex = len(b.feeRate) - 1
}
return b.feeRate[feeIndex]
}
// newEstimateFeeSet creates a temporary data structure that
// can be used to find all fee estimates.
func (ef *FeeEstimator) newEstimateFeeSet() *estimateFeeSet {
set := &estimateFeeSet{}
capacity := 0
for i, b := range ef.bin {
l := len(b)
set.bin[i] = uint32(l)
capacity += l
}
set.feeRate = make([]SatoshiPerByte, capacity)
i := 0
for _, b := range ef.bin {
for _, o := range b {
set.feeRate[i] = o.feeRate
i++
}
}
sort.Sort(set)
return set
}
// estimates returns the set of all fee estimates from 1 to estimateFeeDepth
// confirmations from now.
func (ef *FeeEstimator) estimates() []SatoshiPerByte {
set := ef.newEstimateFeeSet()
estimates := make([]SatoshiPerByte, estimateFeeDepth)
for i := 0; i < estimateFeeDepth; i++ {
estimates[i] = set.estimateFee(i + 1)
}
return estimates
}
// EstimateFee estimates the fee per byte to have a tx confirmed a given
// number of blocks from now.
func (ef *FeeEstimator) EstimateFee(numBlocks uint32) (BtcPerKilobyte, error) {
ef.mtx.Lock()
defer ef.mtx.Unlock()
// If the number of registered blocks is below the minimum, return
// an error.
if ef.numBlocksRegistered < ef.minRegisteredBlocks {
return -1, errors.New("not enough blocks have been observed")
}
if numBlocks == 0 {
return -1, errors.New("cannot confirm transaction in zero blocks")
}
if numBlocks > estimateFeeDepth {
return -1, fmt.Errorf(
"can only estimate fees for up to %d blocks from now",
estimateFeeBinSize)
}
// If there are no cached results, generate them.
if ef.cached == nil {
ef.cached = ef.estimates()
}
return ef.cached[int(numBlocks)-1].ToBtcPerKb(), nil
}
// In case the format for the serialized version of the FeeEstimator changes,
// we use a version number. If the version number changes, it does not make
// sense to try to upgrade a previous version to a new version. Instead, just
// start fee estimation over.
const estimateFeeSaveVersion = 1
func deserializeRegisteredBlock(r io.Reader, txs map[uint32]*observedTransaction) (*registeredBlock, error) {
var lenTransactions uint32
rb := &registeredBlock{hash: &daghash.Hash{}}
if err := binary.Read(r, binary.BigEndian, rb.hash); err != nil {
return nil, err
}
if err := binary.Read(r, binary.BigEndian, &lenTransactions); err != nil {
return nil, err
}
rb.transactions = make([]*observedTransaction, lenTransactions)
for i := uint32(0); i < lenTransactions; i++ {
var index uint32
binary.Read(r, binary.BigEndian, &index)
rb.transactions[i] = txs[index]
}
return rb, nil
}
// FeeEstimatorState represents a saved FeeEstimator that can be
// restored with data from an earlier session of the program.
type FeeEstimatorState []byte
// observedTxSet is a set of txs that can that is sorted
// by hash. It exists for serialization purposes so that
// a serialized state always comes out the same.
type observedTxSet []*observedTransaction
func (q observedTxSet) Len() int { return len(q) }
func (q observedTxSet) Less(i, j int) bool {
return strings.Compare(q[i].txID.String(), q[j].txID.String()) < 0
}
func (q observedTxSet) Swap(i, j int) {
q[i], q[j] = q[j], q[i]
}
// Save records the current state of the FeeEstimator to a []byte that
// can be restored later.
func (ef *FeeEstimator) Save() FeeEstimatorState {
ef.mtx.Lock()
defer ef.mtx.Unlock()
// TODO figure out what the capacity should be.
w := bytes.NewBuffer(make([]byte, 0))
binary.Write(w, binary.BigEndian, uint32(estimateFeeSaveVersion))
// Insert basic parameters.
binary.Write(w, binary.BigEndian, &ef.maxRollback)
binary.Write(w, binary.BigEndian, &ef.binSize)
binary.Write(w, binary.BigEndian, &ef.maxReplacements)
binary.Write(w, binary.BigEndian, &ef.minRegisteredBlocks)
binary.Write(w, binary.BigEndian, &ef.lastKnownHeight)
binary.Write(w, binary.BigEndian, &ef.numBlocksRegistered)
// Put all the observed transactions in a sorted list.
var txCount uint32
ots := make([]*observedTransaction, len(ef.observed))
for hash := range ef.observed {
ots[txCount] = ef.observed[hash]
txCount++
}
sort.Sort(observedTxSet(ots))
txCount = 0
observed := make(map[*observedTransaction]uint32)
binary.Write(w, binary.BigEndian, uint32(len(ef.observed)))
for _, ot := range ots {
ot.Serialize(w)
observed[ot] = txCount
txCount++
}
// Save all the right bins.
for _, list := range ef.bin {
binary.Write(w, binary.BigEndian, uint32(len(list)))
for _, o := range list {
binary.Write(w, binary.BigEndian, observed[o])
}
}
// Dropped transactions.
binary.Write(w, binary.BigEndian, uint32(len(ef.dropped)))
for _, registered := range ef.dropped {
registered.serialize(w, observed)
}
// Commit the tx and return.
return FeeEstimatorState(w.Bytes())
}
// RestoreFeeEstimator takes a FeeEstimatorState that was previously
// returned by Save and restores it to a FeeEstimator
func RestoreFeeEstimator(data FeeEstimatorState) (*FeeEstimator, error) {
r := bytes.NewReader([]byte(data))
// Check version
var version uint32
err := binary.Read(r, binary.BigEndian, &version)
if err != nil {
return nil, err
}
if version != estimateFeeSaveVersion {
return nil, fmt.Errorf("Incorrect version: expected %d found %d", estimateFeeSaveVersion, version)
}
ef := &FeeEstimator{
observed: make(map[daghash.TxID]*observedTransaction),
}
// Read basic parameters.
binary.Read(r, binary.BigEndian, &ef.maxRollback)
binary.Read(r, binary.BigEndian, &ef.binSize)
binary.Read(r, binary.BigEndian, &ef.maxReplacements)
binary.Read(r, binary.BigEndian, &ef.minRegisteredBlocks)
binary.Read(r, binary.BigEndian, &ef.lastKnownHeight)
binary.Read(r, binary.BigEndian, &ef.numBlocksRegistered)
// Read transactions.
var numObserved uint32
observed := make(map[uint32]*observedTransaction)
binary.Read(r, binary.BigEndian, &numObserved)
for i := uint32(0); i < numObserved; i++ {
ot, err := deserializeObservedTransaction(r)
if err != nil {
return nil, err
}
observed[i] = ot
ef.observed[*ot.txID] = ot
}
// Read bins.
for i := 0; i < estimateFeeDepth; i++ {
var numTransactions uint32
binary.Read(r, binary.BigEndian, &numTransactions)
bin := make([]*observedTransaction, numTransactions)
for j := uint32(0); j < numTransactions; j++ {
var index uint32
binary.Read(r, binary.BigEndian, &index)
var exists bool
bin[j], exists = observed[index]
if !exists {
return nil, fmt.Errorf("Invalid transaction reference %d", index)
}
}
ef.bin[i] = bin
}
// Read dropped transactions.
var numDropped uint32
binary.Read(r, binary.BigEndian, &numDropped)
ef.dropped = make([]*registeredBlock, numDropped)
for i := uint32(0); i < numDropped; i++ {
var err error
ef.dropped[int(i)], err = deserializeRegisteredBlock(r, observed)
if err != nil {
return nil, err
}
}
return ef, nil
}

View File

@ -1,428 +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 mempool
import (
"bytes"
"math/rand"
"testing"
"github.com/daglabs/btcd/blockdag"
"github.com/daglabs/btcd/mining"
"github.com/daglabs/btcd/util"
"github.com/daglabs/btcd/util/daghash"
"github.com/daglabs/btcd/wire"
)
// newTestFeeEstimator creates a feeEstimator with some different parameters
// for testing purposes.
func newTestFeeEstimator(binSize, maxReplacements, maxRollback uint32) *FeeEstimator {
return &FeeEstimator{
maxRollback: maxRollback,
lastKnownHeight: 0,
binSize: int32(binSize),
minRegisteredBlocks: 0,
maxReplacements: int32(maxReplacements),
observed: make(map[daghash.TxID]*observedTransaction),
dropped: make([]*registeredBlock, 0, maxRollback),
}
}
// lastBlock is a linked list of the block hashes which have been
// processed by the test FeeEstimator.
type lastBlock struct {
hash *daghash.Hash
prev *lastBlock
}
// estimateFeeTester interacts with the FeeEstimator to keep track
// of its expected state.
type estimateFeeTester struct {
ef *FeeEstimator
t *testing.T
version int32
height uint64
last *lastBlock
}
func (eft *estimateFeeTester) testTx(fee util.Amount) *TxDesc {
eft.version++
return &TxDesc{
TxDesc: mining.TxDesc{
Tx: util.NewTx(wire.NewNativeMsgTx(eft.version, nil, nil)),
Height: eft.height,
Fee: uint64(fee),
},
StartingPriority: 0,
}
}
func expectedFeePerKilobyte(t *TxDesc) BtcPerKilobyte {
size := float64(t.TxDesc.Tx.MsgTx().SerializeSize())
fee := float64(t.TxDesc.Fee)
return SatoshiPerByte(fee / size).ToBtcPerKb()
}
func (eft *estimateFeeTester) newBlock(txs []*wire.MsgTx) {
eft.height++
block := util.NewBlock(&wire.MsgBlock{
Header: wire.BlockHeader{
HashMerkleRoot: &daghash.ZeroHash,
AcceptedIDMerkleRoot: &daghash.ZeroHash,
UTXOCommitment: &daghash.ZeroHash,
},
Transactions: txs,
})
block.SetHeight(eft.height)
eft.last = &lastBlock{block.Hash(), eft.last}
eft.ef.RegisterBlock(block)
}
func (eft *estimateFeeTester) rollback() {
if eft.last == nil {
return
}
err := eft.ef.Rollback(eft.last.hash)
if err != nil {
eft.t.Errorf("Could not rollback: %v", err)
}
eft.height--
eft.last = eft.last.prev
}
// TestEstimateFee tests basic functionality in the FeeEstimator.
func TestEstimateFee(t *testing.T) {
ef := newTestFeeEstimator(5, 3, 1)
eft := estimateFeeTester{ef: ef, t: t}
// Try with no txs and get zero for all queries.
expected := BtcPerKilobyte(0.0)
for i := uint32(1); i <= estimateFeeDepth; i++ {
estimated, _ := ef.EstimateFee(i)
if estimated != expected {
t.Errorf("Estimate fee error: expected %f when estimator is empty; got %f", expected, estimated)
}
}
// Now insert a tx.
tx := eft.testTx(1000000)
ef.ObserveTransaction(tx)
// Expected should still be zero because this is still in the mempool.
expected = BtcPerKilobyte(0.0)
for i := uint32(1); i <= estimateFeeDepth; i++ {
estimated, _ := ef.EstimateFee(i)
if estimated != expected {
t.Errorf("Estimate fee error: expected %f when estimator has one tx in mempool; got %f", expected, estimated)
}
}
// Change minRegisteredBlocks to make sure that works. Error return
// value expected.
ef.minRegisteredBlocks = 1
expected = BtcPerKilobyte(-1.0)
for i := uint32(1); i <= estimateFeeDepth; i++ {
estimated, _ := ef.EstimateFee(i)
if estimated != expected {
t.Errorf("Estimate fee error: expected %f before any blocks have been registered; got %f", expected, estimated)
}
}
// Record a block with the new tx.
eft.newBlock([]*wire.MsgTx{tx.Tx.MsgTx()})
expected = expectedFeePerKilobyte(tx)
for i := uint32(1); i <= estimateFeeDepth; i++ {
estimated, _ := ef.EstimateFee(i)
if estimated != expected {
t.Errorf("Estimate fee error: expected %f when one tx is binned; got %f", expected, estimated)
}
}
// Roll back the last block; this was an orphan block.
ef.minRegisteredBlocks = 0
eft.rollback()
expected = BtcPerKilobyte(0.0)
for i := uint32(1); i <= estimateFeeDepth; i++ {
estimated, _ := ef.EstimateFee(i)
if estimated != expected {
t.Errorf("Estimate fee error: expected %f after rolling back block; got %f", expected, estimated)
}
}
// Record an empty block and then a block with the new tx.
// This test was made because of a bug that only appeared when there
// were no transactions in the first bin.
eft.newBlock([]*wire.MsgTx{})
eft.newBlock([]*wire.MsgTx{tx.Tx.MsgTx()})
expected = expectedFeePerKilobyte(tx)
for i := uint32(1); i <= estimateFeeDepth; i++ {
estimated, _ := ef.EstimateFee(i)
if estimated != expected {
t.Errorf("Estimate fee error: expected %f when one tx is binned; got %f", expected, estimated)
}
}
// Create some more transactions.
txA := eft.testTx(500000)
txB := eft.testTx(2000000)
txC := eft.testTx(4000000)
ef.ObserveTransaction(txA)
ef.ObserveTransaction(txB)
ef.ObserveTransaction(txC)
// Record 7 empty blocks.
for i := 0; i < 7; i++ {
eft.newBlock([]*wire.MsgTx{})
}
// Mine the first tx.
eft.newBlock([]*wire.MsgTx{txA.Tx.MsgTx()})
// Now the estimated amount should depend on the value
// of the argument to estimate fee.
for i := uint32(1); i <= estimateFeeDepth; i++ {
estimated, _ := ef.EstimateFee(i)
if i > 2 {
expected = expectedFeePerKilobyte(txA)
} else {
expected = expectedFeePerKilobyte(tx)
}
if estimated != expected {
t.Errorf("Estimate fee error: expected %f on round %d; got %f", expected, i, estimated)
}
}
// Record 5 more empty blocks.
for i := 0; i < 5; i++ {
eft.newBlock([]*wire.MsgTx{})
}
// Mine the next tx.
eft.newBlock([]*wire.MsgTx{txB.Tx.MsgTx()})
// Now the estimated amount should depend on the value
// of the argument to estimate fee.
for i := uint32(1); i <= estimateFeeDepth; i++ {
estimated, _ := ef.EstimateFee(i)
if i <= 2 {
expected = expectedFeePerKilobyte(txB)
} else if i <= 8 {
expected = expectedFeePerKilobyte(tx)
} else {
expected = expectedFeePerKilobyte(txA)
}
if estimated != expected {
t.Errorf("Estimate fee error: expected %f on round %d; got %f", expected, i, estimated)
}
}
// Record 9 more empty blocks.
for i := 0; i < 10; i++ {
eft.newBlock([]*wire.MsgTx{})
}
// Mine txC.
eft.newBlock([]*wire.MsgTx{txC.Tx.MsgTx()})
// This should have no effect on the outcome because too
// many blocks have been mined for txC to be recorded.
for i := uint32(1); i <= estimateFeeDepth; i++ {
estimated, _ := ef.EstimateFee(i)
if i <= 2 {
expected = expectedFeePerKilobyte(txC)
} else if i <= 8 {
expected = expectedFeePerKilobyte(txB)
} else if i <= 8+6 {
expected = expectedFeePerKilobyte(tx)
} else {
expected = expectedFeePerKilobyte(txA)
}
if estimated != expected {
t.Errorf("Estimate fee error: expected %f on round %d; got %f", expected, i, estimated)
}
}
}
func (eft *estimateFeeTester) estimates() [estimateFeeDepth]BtcPerKilobyte {
// Generate estimates
var estimates [estimateFeeDepth]BtcPerKilobyte
for i := 0; i < estimateFeeDepth; i++ {
estimates[i], _ = eft.ef.EstimateFee(uint32(i + 1))
}
// Check that all estimated fee results go in descending order.
for i := 1; i < estimateFeeDepth; i++ {
if estimates[i] > estimates[i-1] {
eft.t.Error("Estimates not in descending order; got ",
estimates[i], " for estimate ", i, " and ", estimates[i-1], " for ", (i - 1))
panic("invalid state.")
}
}
return estimates
}
func (eft *estimateFeeTester) round(txHistory [][]*TxDesc,
estimateHistory [][estimateFeeDepth]BtcPerKilobyte,
txPerRound, txPerBlock uint32) ([][]*TxDesc, [][estimateFeeDepth]BtcPerKilobyte) {
// generate new txs.
var newTxs []*TxDesc
for i := uint32(0); i < txPerRound; i++ {
newTx := eft.testTx(util.Amount(rand.Intn(1000000)))
eft.ef.ObserveTransaction(newTx)
newTxs = append(newTxs, newTx)
}
// Generate mempool.
mempool := make(map[*observedTransaction]*TxDesc)
for _, h := range txHistory {
for _, t := range h {
if o, exists := eft.ef.observed[*t.Tx.ID()]; exists && o.mined == blockdag.UnminedChainHeight {
mempool[o] = t
}
}
}
// generate new block, with no duplicates.
i := uint32(0)
newBlockList := make([]*wire.MsgTx, 0, txPerBlock)
for _, t := range mempool {
newBlockList = append(newBlockList, t.TxDesc.Tx.MsgTx())
i++
if i == txPerBlock {
break
}
}
// Register a new block.
eft.newBlock(newBlockList)
// return results.
estimates := eft.estimates()
// Return results
return append(txHistory, newTxs), append(estimateHistory, estimates)
}
// TestEstimateFeeRollback tests the rollback function, which undoes the
// effect of a adding a new block.
func TestEstimateFeeRollback(t *testing.T) {
txPerRound := uint32(7)
txPerBlock := uint32(5)
binSize := uint32(6)
maxReplacements := uint32(4)
stepsBack := 2
rounds := 30
eft := estimateFeeTester{ef: newTestFeeEstimator(binSize, maxReplacements, uint32(stepsBack)), t: t}
var txHistory [][]*TxDesc
estimateHistory := [][estimateFeeDepth]BtcPerKilobyte{eft.estimates()}
for round := 0; round < rounds; round++ {
// Go forward a few rounds.
for step := 0; step <= stepsBack; step++ {
txHistory, estimateHistory =
eft.round(txHistory, estimateHistory, txPerRound, txPerBlock)
}
// Now go back.
for step := 0; step < stepsBack; step++ {
eft.rollback()
// After rolling back, we should have the same estimated
// fees as before.
expected := estimateHistory[len(estimateHistory)-step-2]
estimates := eft.estimates()
// Ensure that these are both the same.
for i := 0; i < estimateFeeDepth; i++ {
if expected[i] != estimates[i] {
t.Errorf("Rollback value mismatch. Expected %f, got %f. ",
expected[i], estimates[i])
return
}
}
}
// Erase history.
txHistory = txHistory[0 : len(txHistory)-stepsBack]
estimateHistory = estimateHistory[0 : len(estimateHistory)-stepsBack]
}
}
func (eft *estimateFeeTester) checkSaveAndRestore(
previousEstimates [estimateFeeDepth]BtcPerKilobyte) {
// Get the save state.
save := eft.ef.Save()
// Save and restore database.
var err error
eft.ef, err = RestoreFeeEstimator(save)
if err != nil {
eft.t.Fatalf("Could not restore database: %s", err)
}
// Save again and check that it matches the previous one.
redo := eft.ef.Save()
if !bytes.Equal(save, redo) {
eft.t.Fatalf("Restored states do not match: %v %v", save, redo)
}
// Check that the results match.
newEstimates := eft.estimates()
for i, prev := range previousEstimates {
if prev != newEstimates[i] {
eft.t.Error("Mismatch in estimate ", i, " after restore; got ", newEstimates[i], " but expected ", prev)
}
}
}
// TestDatabase tests saving and restoring to a []byte.
func TestDatabase(t *testing.T) {
txPerRound := uint32(7)
txPerBlock := uint32(5)
binSize := uint32(6)
maxReplacements := uint32(4)
rounds := 8
eft := estimateFeeTester{ef: newTestFeeEstimator(binSize, maxReplacements, uint32(rounds)+1), t: t}
var txHistory [][]*TxDesc
estimateHistory := [][estimateFeeDepth]BtcPerKilobyte{eft.estimates()}
for round := 0; round < rounds; round++ {
eft.checkSaveAndRestore(estimateHistory[len(estimateHistory)-1])
// Go forward one step.
txHistory, estimateHistory =
eft.round(txHistory, estimateHistory, txPerRound, txPerBlock)
}
// Reverse the process and try again.
for round := 1; round <= rounds; round++ {
eft.rollback()
eft.checkSaveAndRestore(estimateHistory[len(estimateHistory)-round-1])
}
}

View File

@ -7,7 +7,6 @@ package mempool
import (
"container/list"
"fmt"
"math"
"sync"
"sync/atomic"
"time"
@ -26,12 +25,6 @@ import (
)
const (
// DefaultBlockPrioritySize is the default size in bytes for high-
// priority / low-fee transactions. It is used to help determine which
// are allowed into the mempool and consequently affects their relay and
// inclusion when generating block templates.
DefaultBlockPrioritySize = 50000
// orphanTTL is the maximum amount of time an orphan is allowed to
// stay in the orphan pool before it expires and is evicted during the
// next scan.
@ -64,9 +57,9 @@ type Config struct {
// associated with.
DAGParams *dagconfig.Params
// BestHeight defines the function to use to access the block height of
// DAGChainHeight defines the function to use to access the block height of
// the current best chain.
BestHeight func() uint64
DAGChainHeight func() uint64
// MedianTimePast defines the function to use in order to access the
// median time past calculated from the point-of-view of the current
@ -92,10 +85,6 @@ type Config struct {
// This can be nil if the address index is not enabled.
AddrIndex *indexers.AddrIndex
// FeeEstimatator provides a feeEstimator. If it is not nil, the mempool
// records all new transactions it observes into the feeEstimator.
FeeEstimator *FeeEstimator
// DAG is the BlockDAG we want to use (mainly for UTXO checks)
DAG *blockdag.BlockDAG
}
@ -108,19 +97,11 @@ type Policy struct {
// non-standard.
MaxTxVersion int32
// DisableRelayPriority defines whether to relay free or low-fee
// transactions that do not have enough priority to be relayed.
DisableRelayPriority bool
// AcceptNonStd defines whether to accept non-standard transactions. If
// true, non-standard transactions will be accepted into the mempool.
// Otherwise, all non-standard transactions will be rejected.
AcceptNonStd bool
// FreeTxRelayLimit defines the given amount in thousands of bytes
// per minute that transactions with no fee are rate limited to.
FreeTxRelayLimit float64
// MaxOrphanTxs is the maximum number of orphan transactions
// that can be queued.
MaxOrphanTxs int
@ -145,10 +126,6 @@ type Policy struct {
type TxDesc struct {
mining.TxDesc
// StartingPriority is the priority of the transaction when it was added
// to the pool.
StartingPriority float64
// depCount is not 0 for dependent transaction. Dependent transaction is
// one that is accepted to pool, but cannot be mined in next block because it
// depends on outputs of accepted, but still not mined transaction
@ -614,7 +591,7 @@ func (mp *TxPool) markTransactionOutputsUnspent(tx *util.Tx, diff *blockdag.UTXO
if restoreInputs {
if prevTxDesc, exists := mp.pool[txIn.PreviousOutpoint.TxID]; exists {
prevOut := prevTxDesc.Tx.MsgTx().TxOut[txIn.PreviousOutpoint.Index]
entry := blockdag.NewUTXOEntry(prevOut, false, blockdag.UnminedChainHeight)
entry := blockdag.NewUTXOEntry(prevOut, false, blockdag.UnminedBlueScore)
err := diff.AddEntry(txIn.PreviousOutpoint, entry)
if err != nil {
return err
@ -622,7 +599,7 @@ func (mp *TxPool) markTransactionOutputsUnspent(tx *util.Tx, diff *blockdag.UTXO
}
if prevTxDesc, exists := mp.depends[txIn.PreviousOutpoint.TxID]; exists {
prevOut := prevTxDesc.Tx.MsgTx().TxOut[txIn.PreviousOutpoint.Index]
entry := blockdag.NewUTXOEntry(prevOut, false, blockdag.UnminedChainHeight)
entry := blockdag.NewUTXOEntry(prevOut, false, blockdag.UnminedBlueScore)
err := diff.AddEntry(txIn.PreviousOutpoint, entry)
if err != nil {
return err
@ -710,7 +687,7 @@ func (mp *TxPool) RemoveDoubleSpends(tx *util.Tx) {
// helper for maybeAcceptTransaction.
//
// This function MUST be called with the mempool lock held (for writes).
func (mp *TxPool) addTransaction(tx *util.Tx, height uint64, fee uint64, parentsInPool []*wire.Outpoint) (*TxDesc, error) {
func (mp *TxPool) addTransaction(tx *util.Tx, height uint64, blueScore uint64, fee uint64, parentsInPool []*wire.Outpoint) (*TxDesc, error) {
// Add the transaction to the pool and mark the referenced outpoints
// as spent by the pool.
txD := &TxDesc{
@ -721,8 +698,7 @@ func (mp *TxPool) addTransaction(tx *util.Tx, height uint64, fee uint64, parents
Fee: fee,
FeePerKB: fee * 1000 / uint64(tx.MsgTx().SerializeSize()),
},
StartingPriority: mining.CalcPriority(tx.MsgTx(), mp.mpUTXOSet, height),
depCount: len(parentsInPool),
depCount: len(parentsInPool),
}
if len(parentsInPool) == 0 {
@ -740,7 +716,7 @@ func (mp *TxPool) addTransaction(tx *util.Tx, height uint64, fee uint64, parents
for _, txIn := range tx.MsgTx().TxIn {
mp.outpoints[txIn.PreviousOutpoint] = tx
}
if isAccepted, err := mp.mpUTXOSet.AddTx(tx.MsgTx(), blockdag.UnminedChainHeight); err != nil {
if isAccepted, err := mp.mpUTXOSet.AddTx(tx.MsgTx(), blockdag.UnminedBlueScore); err != nil {
return nil, err
} else if !isAccepted {
return nil, fmt.Errorf("unexpectedly failed to add tx %s to the mempool utxo set", tx.ID())
@ -753,11 +729,6 @@ func (mp *TxPool) addTransaction(tx *util.Tx, height uint64, fee uint64, parents
mp.cfg.AddrIndex.AddUnconfirmedTx(tx, mp.mpUTXOSet)
}
// Record this tx for fee estimation if enabled.
if mp.cfg.FeeEstimator != nil {
mp.cfg.FeeEstimator.ObserveTransaction(txD)
}
return txD, nil
}
@ -822,7 +793,7 @@ func (mp *TxPool) FetchTransaction(txID *daghash.TxID) (*util.Tx, error) {
// more details.
//
// This function MUST be called with the mempool lock held (for writes).
func (mp *TxPool) maybeAcceptTransaction(tx *util.Tx, isNew, rateLimit, rejectDupOrphans bool) ([]*daghash.TxID, *TxDesc, error) {
func (mp *TxPool) maybeAcceptTransaction(tx *util.Tx, isNew, rejectDupOrphans bool) ([]*daghash.TxID, *TxDesc, error) {
txID := tx.ID()
// Don't accept the transaction if it already exists in the pool. This
@ -847,7 +818,7 @@ func (mp *TxPool) maybeAcceptTransaction(tx *util.Tx, isNew, rateLimit, rejectDu
// Perform preliminary sanity checks on the transaction. This makes
// use of blockDAG which contains the invariant rules for what
// transactions are allowed into blocks.
err := blockdag.CheckTransactionSanity(tx, subnetworkID, false)
err := blockdag.CheckTransactionSanity(tx, subnetworkID)
if err != nil {
if cerr, ok := err.(blockdag.RuleError); ok {
return nil, nil, dagRuleError(cerr)
@ -878,18 +849,16 @@ func (mp *TxPool) maybeAcceptTransaction(tx *util.Tx, isNew, rateLimit, rejectDu
return nil, nil, txRuleError(wire.RejectInvalid, str)
}
// Get the current height of the main chain. A standalone transaction
// will be mined into the next block at best, so its height is at least
// one more than the current height.
bestHeight := mp.cfg.BestHeight()
nextBlockHeight := bestHeight + 1
// We take the blue score of the current virtual block to validate
// the transaction as though it was mined on top of the current tips
nextBlockBlueScore := mp.cfg.DAG.VirtualBlueScore()
medianTimePast := mp.cfg.MedianTimePast()
// Don't allow non-standard transactions if the network parameters
// forbid their acceptance.
if !mp.cfg.Policy.AcceptNonStd {
err = checkTransactionStandard(tx, nextBlockHeight,
err = checkTransactionStandard(tx, nextBlockBlueScore,
medianTimePast, &mp.cfg.Policy)
if err != nil {
// Attempt to extract a reject code from the error so
@ -963,7 +932,7 @@ func (mp *TxPool) maybeAcceptTransaction(tx *util.Tx, isNew, rateLimit, rejectDu
}
return nil, nil, err
}
if !blockdag.SequenceLockActive(sequenceLock, nextBlockHeight,
if !blockdag.SequenceLockActive(sequenceLock, nextBlockBlueScore,
medianTimePast) {
return nil, nil, txRuleError(wire.RejectNonstandard,
"transaction's sequence locks on inputs not met")
@ -973,7 +942,7 @@ func (mp *TxPool) maybeAcceptTransaction(tx *util.Tx, isNew, rateLimit, rejectDu
// rules in blockchain for what transactions are allowed into blocks.
// Also returns the fees associated with the transaction which will be
// used later.
txFee, err := blockdag.CheckTransactionInputsAndCalulateFee(tx, nextBlockHeight,
txFee, err := blockdag.CheckTransactionInputsAndCalulateFee(tx, nextBlockBlueScore,
mp.mpUTXOSet, mp.cfg.DAGParams, false)
if err != nil {
if cerr, ok := err.(blockdag.RuleError); ok {
@ -1036,52 +1005,13 @@ func (mp *TxPool) maybeAcceptTransaction(tx *util.Tx, isNew, rateLimit, rejectDu
serializedSize := int64(tx.MsgTx().SerializeSize())
minFee := uint64(calcMinRequiredTxRelayFee(serializedSize,
mp.cfg.Policy.MinRelayTxFee))
if serializedSize >= (DefaultBlockPrioritySize-1000) && txFee < minFee {
if txFee < minFee {
str := fmt.Sprintf("transaction %s has %d fees which is under "+
"the required amount of %d", txID, txFee,
minFee)
return nil, nil, txRuleError(wire.RejectInsufficientFee, str)
}
// Require that free transactions have sufficient priority to be mined
// in the next block. Transactions which are being added back to the
// memory pool from blocks that have been disconnected during a reorg
// are exempted.
if isNew && !mp.cfg.Policy.DisableRelayPriority && txFee < minFee {
currentPriority := mining.CalcPriority(tx.MsgTx(), mp.mpUTXOSet,
nextBlockHeight)
if currentPriority <= mining.MinHighPriority {
str := fmt.Sprintf("transaction %s has insufficient "+
"priority (%g <= %g)", txID,
currentPriority, mining.MinHighPriority)
return nil, nil, txRuleError(wire.RejectInsufficientFee, str)
}
}
// Free-to-relay transactions are rate limited here to prevent
// penny-flooding with tiny transactions as a form of attack.
if rateLimit && txFee < minFee {
nowUnix := time.Now().Unix()
// Decay passed data with an exponentially decaying ~10 minute
// window - matches bitcoind handling.
mp.pennyTotal *= math.Pow(1.0-1.0/600.0,
float64(nowUnix-mp.lastPennyUnix))
mp.lastPennyUnix = nowUnix
// Are we still over the limit?
if mp.pennyTotal >= mp.cfg.Policy.FreeTxRelayLimit*10*1000 {
str := fmt.Sprintf("transaction %s has been rejected "+
"by the rate limiter due to low fees", txID)
return nil, nil, txRuleError(wire.RejectInsufficientFee, str)
}
oldTotal := mp.pennyTotal
mp.pennyTotal += float64(serializedSize)
log.Tracef("rate limit: curTotal %d, nextTotal: %d, "+
"limit %d", oldTotal, mp.pennyTotal,
mp.cfg.Policy.FreeTxRelayLimit*10*1000)
}
// Verify crypto signatures for each input and reject the transaction if
// any don't verify.
err = blockdag.ValidateTransactionScripts(tx, mp.mpUTXOSet,
@ -1094,7 +1024,8 @@ func (mp *TxPool) maybeAcceptTransaction(tx *util.Tx, isNew, rateLimit, rejectDu
}
// Add to transaction pool.
txD, err := mp.addTransaction(tx, bestHeight, txFee, parentsInPool)
bestHeight := mp.cfg.DAGChainHeight()
txD, err := mp.addTransaction(tx, bestHeight, nextBlockBlueScore, txFee, parentsInPool)
if err != nil {
return nil, nil, err
}
@ -1116,13 +1047,13 @@ func (mp *TxPool) maybeAcceptTransaction(tx *util.Tx, isNew, rateLimit, rejectDu
// be added to the orphan pool.
//
// This function is safe for concurrent access.
func (mp *TxPool) MaybeAcceptTransaction(tx *util.Tx, isNew, rateLimit bool) ([]*daghash.TxID, *TxDesc, error) {
func (mp *TxPool) MaybeAcceptTransaction(tx *util.Tx, isNew bool) ([]*daghash.TxID, *TxDesc, error) {
// Protect concurrent access.
mp.cfg.DAG.RLock()
defer mp.cfg.DAG.RUnlock()
mp.mtx.Lock()
defer mp.mtx.Unlock()
hashes, txD, err := mp.maybeAcceptTransaction(tx, isNew, rateLimit, true)
hashes, txD, err := mp.maybeAcceptTransaction(tx, isNew, true)
return hashes, txD, err
}
@ -1164,7 +1095,7 @@ func (mp *TxPool) processOrphans(acceptedTx *util.Tx) []*TxDesc {
// Potentially accept an orphan into the tx pool.
for _, tx := range orphans {
missing, txD, err := mp.maybeAcceptTransaction(
tx, true, true, false)
tx, true, false)
if err != nil {
// The orphan is now invalid, so there
// is no way any other orphans which
@ -1241,7 +1172,7 @@ func (mp *TxPool) ProcessOrphans(acceptedTx *util.Tx) []*TxDesc {
// the passed one being accepted.
//
// This function is safe for concurrent access.
func (mp *TxPool) ProcessTransaction(tx *util.Tx, allowOrphan, rateLimit bool, tag Tag) ([]*TxDesc, error) {
func (mp *TxPool) ProcessTransaction(tx *util.Tx, allowOrphan bool, tag Tag) ([]*TxDesc, error) {
log.Tracef("Processing transaction %s", tx.ID())
// Protect concurrent access.
@ -1251,8 +1182,7 @@ func (mp *TxPool) ProcessTransaction(tx *util.Tx, allowOrphan, rateLimit bool, t
defer mp.mtx.Unlock()
// Potentially accept the transaction to the memory pool.
missingParents, txD, err := mp.maybeAcceptTransaction(tx, true, rateLimit,
true)
missingParents, txD, err := mp.maybeAcceptTransaction(tx, true, true)
if err != nil {
return nil, err
}
@ -1379,26 +1309,20 @@ func (mp *TxPool) RawMempoolVerbose() map[string]*btcjson.GetRawMempoolVerboseRe
mp.mtx.RLock()
defer mp.mtx.RUnlock()
result := make(map[string]*btcjson.GetRawMempoolVerboseResult,
len(mp.pool))
bestHeight := mp.cfg.BestHeight()
result := make(map[string]*btcjson.GetRawMempoolVerboseResult, len(mp.pool))
for _, desc := range mp.pool {
// Calculate the current priority based on the inputs to
// the transaction. Use zero if one or more of the
// input transactions can't be found for some reason.
tx := desc.Tx
currentPriority := mining.CalcPriority(tx.MsgTx(), mp.mpUTXOSet,
bestHeight+1)
mpd := &btcjson.GetRawMempoolVerboseResult{
Size: int32(tx.MsgTx().SerializeSize()),
Fee: util.Amount(desc.Fee).ToBTC(),
Time: desc.Added.Unix(),
Height: desc.Height,
StartingPriority: desc.StartingPriority,
CurrentPriority: currentPriority,
Depends: make([]string, 0),
Size: int32(tx.MsgTx().SerializeSize()),
Fee: util.Amount(desc.Fee).ToBTC(),
Time: desc.Added.Unix(),
Height: desc.Height,
Depends: make([]string, 0),
}
for _, txIn := range tx.MsgTx().TxIn {
txID := &txIn.PreviousOutpoint.TxID
@ -1436,12 +1360,12 @@ func (mp *TxPool) HandleNewBlock(block *util.Block, txChan chan NewBlockMsg) err
// no longer an orphan. Transactions which depend on a confirmed
// transaction are NOT removed recursively because they are still
// valid.
err := mp.RemoveTransactions(block.Transactions()[util.FeeTransactionIndex:])
err := mp.RemoveTransactions(block.Transactions()[util.FeeTransactionIndex+1:])
if err != nil {
mp.mpUTXOSet = oldUTXOSet
return err
}
for _, tx := range block.Transactions()[util.FeeTransactionIndex:] {
for _, tx := range block.Transactions()[util.FeeTransactionIndex+1:] {
mp.RemoveDoubleSpends(tx)
mp.RemoveOrphan(tx)
acceptedTxs := mp.ProcessOrphans(tx)

View File

@ -75,8 +75,8 @@ func calcSequenceLock(tx *util.Tx,
utxoSet blockdag.UTXOSet) (*blockdag.SequenceLock, error) {
return &blockdag.SequenceLock{
Seconds: -1,
BlockChainHeight: -1,
Seconds: -1,
BlockBlueScore: -1,
}, nil
}
@ -103,22 +103,25 @@ func txOutToSpendableOutpoint(tx *util.Tx, outputNum uint32) spendableOutpoint {
type poolHarness struct {
signatureScript []byte
payScript []byte
chainParams *dagconfig.Params
dagParams *dagconfig.Params
chain *fakeChain
txPool *TxPool
}
// txRelayFeeForTest defines a convenient relay fee amount to pay for test transactions
var txRelayFeeForTest = util.Amount(calcMinRequiredTxRelayFee(1000, DefaultMinRelayTxFee))
// CreateCoinbaseTx returns a coinbase transaction with the requested number of
// outputs paying an appropriate subsidy based on the passed block height to the
// outputs paying an appropriate subsidy based on the passed block blue score to the
// address associated with the harness. It automatically uses a standard
// signature script that starts with the block height that is required by
// version 2 blocks.
func (p *poolHarness) CreateCoinbaseTx(blockHeight uint64, numOutputs uint32) (*util.Tx, error) {
func (p *poolHarness) CreateCoinbaseTx(blueScore uint64, numOutputs uint32) (*util.Tx, error) {
// Create standard coinbase script.
extraNonce := int64(0)
coinbaseScript, err := txscript.NewScriptBuilder().
AddInt64(int64(blockHeight)).AddInt64(extraNonce).Script()
AddInt64(int64(blueScore)).AddInt64(extraNonce).Script()
if err != nil {
return nil, err
}
@ -133,7 +136,7 @@ func (p *poolHarness) CreateCoinbaseTx(blockHeight uint64, numOutputs uint32) (*
}}
txOuts := []*wire.TxOut{}
totalInput := blockdag.CalcBlockSubsidy(blockHeight, p.chainParams)
totalInput := blockdag.CalcBlockSubsidy(blueScore, p.dagParams)
amountPerOutput := totalInput / uint64(numOutputs)
remainder := totalInput - amountPerOutput*uint64(numOutputs)
for i := uint32(0); i < numOutputs; i++ {
@ -163,10 +166,11 @@ func (p *poolHarness) CreateSignedTxForSubnetwork(inputs []spendableOutpoint, nu
for _, input := range inputs {
totalInput += input.amount
}
totalInput -= txRelayFeeForTest
amountPerOutput := uint64(totalInput) / uint64(numOutputs)
remainder := uint64(totalInput) - amountPerOutput*uint64(numOutputs)
txIns := []*wire.TxIn{}
var txIns []*wire.TxIn
for _, input := range inputs {
txIns = append(txIns, &wire.TxIn{
PreviousOutpoint: input.outpoint,
@ -175,7 +179,7 @@ func (p *poolHarness) CreateSignedTxForSubnetwork(inputs []spendableOutpoint, nu
})
}
txOuts := []*wire.TxOut{}
var txOuts []*wire.TxOut
for i := uint32(0); i < numOutputs; i++ {
// Ensure the final output accounts for any remainder that might
// be left from splitting the input amount.
@ -217,8 +221,10 @@ func (p *poolHarness) CreateTxChain(firstOutput spendableOutpoint, numTxns uint3
spendableAmount := firstOutput.amount
for i := uint32(0); i < numTxns; i++ {
// Create the transaction using the previous transaction output
// and paying the full amount to the payment address associated
// with the harness.
// and paying the full amount (minus fees) to the payment address
// associated with the harness.
spendableAmount = spendableAmount - txRelayFeeForTest
txIn := &wire.TxIn{
PreviousOutpoint: prevOutpoint,
SignatureScript: p.signatureScript,
@ -311,22 +317,20 @@ func newPoolHarness(t *testing.T, dagParams *dagconfig.Params, numOutputs uint32
harness := &poolHarness{
signatureScript: signatureScript,
payScript: pkScript,
chainParams: dagParams,
dagParams: &params,
chain: chain,
txPool: New(&Config{
DAG: dag,
Policy: Policy{
DisableRelayPriority: true,
FreeTxRelayLimit: 15.0,
MaxOrphanTxs: 5,
MaxOrphanTxSize: 1000,
MaxSigOpsPerTx: blockdag.MaxSigOpsPerBlock / 5,
MinRelayTxFee: 1000, // 1 Satoshi per byte
MaxTxVersion: 1,
MaxOrphanTxs: 5,
MaxOrphanTxSize: 1000,
MaxSigOpsPerTx: blockdag.MaxSigOpsPerBlock / 5,
MinRelayTxFee: 1000, // 1 Satoshi per byte
MaxTxVersion: 1,
},
DAGParams: dagParams,
BestHeight: chain.BestHeight,
DAGParams: &params,
DAGChainHeight: chain.BestHeight,
MedianTimePast: chain.MedianTimePast,
CalcSequenceLockNoLock: calcSequenceLock,
SigCache: nil,
@ -445,13 +449,13 @@ func testPoolMembership(tc *testContext, tx *util.Tx, inOrphanPool, inTxPool boo
}
func (p *poolHarness) createTx(outpoint spendableOutpoint, fee uint64, numOutputs int64) (*util.Tx, error) {
txIns := []*wire.TxIn{&wire.TxIn{
txIns := []*wire.TxIn{{
PreviousOutpoint: outpoint.outpoint,
SignatureScript: nil,
Sequence: wire.MaxTxInSequenceNum,
}}
txOuts := []*wire.TxOut{}
var txOuts []*wire.TxOut
amountPerOutput := (uint64(outpoint.amount) - fee) / uint64(numOutputs)
for i := int64(0); i < numOutputs; i++ {
txOuts = append(txOuts, &wire.TxOut{
@ -478,15 +482,15 @@ func TestProcessTransaction(t *testing.T) {
harness := tc.harness
//Checks that a transaction cannot be added to the transaction pool if it's already there
tx, err := harness.createTx(spendableOuts[0], 0, 1)
tx, err := harness.createTx(spendableOuts[0], uint64(txRelayFeeForTest), 1)
if err != nil {
t.Fatalf("unable to create transaction: %v", err)
}
_, err = harness.txPool.ProcessTransaction(tx, true, false, 0)
_, err = harness.txPool.ProcessTransaction(tx, true, 0)
if err != nil {
t.Errorf("ProcessTransaction: unexpected error: %v", err)
}
_, err = harness.txPool.ProcessTransaction(tx, true, false, 0)
_, err = harness.txPool.ProcessTransaction(tx, true, 0)
if err == nil {
t.Errorf("ProcessTransaction: expected an error, not nil")
}
@ -505,21 +509,21 @@ func TestProcessTransaction(t *testing.T) {
//Checks that an orphaned transaction cannot be
//added to the orphan pool if MaxOrphanTxs is 0
harness.txPool.cfg.Policy.MaxOrphanTxs = 0
_, err = harness.txPool.ProcessTransaction(orphanedTx, true, false, 0)
_, err = harness.txPool.ProcessTransaction(orphanedTx, true, 0)
if err != nil {
t.Errorf("ProcessTransaction: unexpected error: %v", err)
}
testPoolMembership(tc, orphanedTx, false, false, false)
harness.txPool.cfg.Policy.MaxOrphanTxs = 5
_, err = harness.txPool.ProcessTransaction(orphanedTx, true, false, 0)
_, err = harness.txPool.ProcessTransaction(orphanedTx, true, 0)
if err != nil {
t.Errorf("ProcessTransaction: unexpected error: %v", err)
}
//Checks that an orphaned transaction cannot be
//added to the orphan pool if it's already there
_, err = harness.txPool.ProcessTransaction(tx, true, false, 0)
_, err = harness.txPool.ProcessTransaction(tx, true, 0)
if err == nil {
t.Errorf("ProcessTransaction: expected an error, not nil")
}
@ -533,7 +537,7 @@ func TestProcessTransaction(t *testing.T) {
if err != nil {
t.Errorf("CreateCoinbaseTx: %v", err)
}
_, err = harness.txPool.ProcessTransaction(coinbase, true, false, 0)
_, err = harness.txPool.ProcessTransaction(coinbase, true, 0)
if err == nil {
t.Errorf("ProcessTransaction: expected an error, not nil")
}
@ -544,7 +548,7 @@ func TestProcessTransaction(t *testing.T) {
//Checks that non standard transactions are rejected from the mempool
nonStdTx, err := harness.createTx(spendableOuts[0], 0, 1)
nonStdTx.MsgTx().Version = wire.TxVersion + 1
_, err = harness.txPool.ProcessTransaction(nonStdTx, true, false, 0)
_, err = harness.txPool.ProcessTransaction(nonStdTx, true, 0)
if err == nil {
t.Errorf("ProcessTransaction: expected an error, not nil")
}
@ -558,7 +562,7 @@ func TestProcessTransaction(t *testing.T) {
if err != nil {
t.Fatalf("unable to create transaction: %v", err)
}
_, err = harness.txPool.ProcessTransaction(bigLowFeeTx, true, false, 0)
_, err = harness.txPool.ProcessTransaction(bigLowFeeTx, true, 0)
if err == nil {
t.Errorf("ProcessTransaction: expected an error, not nil")
}
@ -640,7 +644,7 @@ func TestProcessTransaction(t *testing.T) {
PkScript: dummyPkScript,
}}
nonStdSigScriptTx := util.NewTx(wire.NewNativeMsgTx(1, txIns, txOuts))
_, err = harness.txPool.ProcessTransaction(nonStdSigScriptTx, true, false, 0)
_, err = harness.txPool.ProcessTransaction(nonStdSigScriptTx, true, 0)
if err == nil {
t.Errorf("ProcessTransaction: expected an error, not nil")
}
@ -659,7 +663,7 @@ func TestProcessTransaction(t *testing.T) {
//Checks that even if we accept non standard transactions, we reject by the MaxSigOpsPerTx consensus rule
harness.txPool.cfg.Policy.AcceptNonStd = true
harness.txPool.cfg.Policy.MaxSigOpsPerTx = 15
_, err = harness.txPool.ProcessTransaction(nonStdSigScriptTx, true, false, 0)
_, err = harness.txPool.ProcessTransaction(nonStdSigScriptTx, true, 0)
if err == nil {
t.Errorf("ProcessTransaction: expected an error, not nil")
}
@ -680,7 +684,7 @@ func TestProcessTransaction(t *testing.T) {
Sequence: wire.MaxTxInSequenceNum,
}},
nil))
_, err = harness.txPool.ProcessTransaction(noOutsTx, true, false, 0)
_, err = harness.txPool.ProcessTransaction(noOutsTx, true, 0)
if err != nil {
t.Errorf("ProcessTransaction: %v", err)
}
@ -690,15 +694,15 @@ func TestProcessTransaction(t *testing.T) {
view blockdag.UTXOSet) (*blockdag.SequenceLock, error) {
return &blockdag.SequenceLock{
Seconds: math.MaxInt64,
BlockChainHeight: math.MaxInt64,
Seconds: math.MaxInt64,
BlockBlueScore: math.MaxInt64,
}, nil
}
tx, err = harness.createTx(spendableOuts[2], 0, 1)
if err != nil {
t.Fatalf("unable to create transaction: %v", err)
}
_, err = harness.txPool.ProcessTransaction(tx, true, false, 0)
_, err = harness.txPool.ProcessTransaction(tx, true, 0)
if err == nil {
t.Errorf("ProcessTransaction: expected an error, not nil")
}
@ -711,32 +715,18 @@ func TestProcessTransaction(t *testing.T) {
}
harness.txPool.cfg.CalcSequenceLockNoLock = calcSequenceLock
// This is done in order to increase the input age, so the tx priority will be higher
harness.chain.SetHeight(curHeight + 100)
harness.txPool.cfg.Policy.DisableRelayPriority = false
//Transaction should be accepted to mempool although it has low fee, because its priority is above mining.MinHighPriority
tx, err = harness.createTx(spendableOuts[3], 0, 1)
if err != nil {
t.Fatalf("unable to create transaction: %v", err)
}
_, err = harness.txPool.ProcessTransaction(tx, true, false, 0)
if err != nil {
t.Errorf("ProcessTransaction: unexpected error: %v", err)
}
//Transaction should be rejected from mempool because it has low fee, and its priority is above mining.MinHighPriority
tx, err = harness.createTx(spendableOuts[4], 0, 100)
if err != nil {
t.Fatalf("unable to create transaction: %v", err)
}
_, err = harness.txPool.ProcessTransaction(tx, true, false, 0)
_, err = harness.txPool.ProcessTransaction(tx, true, 0)
if err == nil {
t.Errorf("ProcessTransaction: expected an error, not nil")
}
if code, _ := extractRejectCode(err); code != wire.RejectInsufficientFee {
t.Errorf("Unexpected error code. Expected %v but got %v", wire.RejectInsufficientFee, code)
}
harness.txPool.cfg.Policy.DisableRelayPriority = true
txIns = []*wire.TxIn{{
PreviousOutpoint: spendableOuts[5].outpoint,
@ -748,7 +738,7 @@ func TestProcessTransaction(t *testing.T) {
PkScript: dummyPkScript,
}}
tx = util.NewTx(wire.NewNativeMsgTx(1, txIns, txOuts))
_, err = harness.txPool.ProcessTransaction(tx, true, false, 0)
_, err = harness.txPool.ProcessTransaction(tx, true, 0)
fmt.Println(err)
if err == nil {
t.Errorf("ProcessTransaction: expected an error, not nil")
@ -777,11 +767,11 @@ func TestAddrIndex(t *testing.T) {
})
defer guard.Unpatch()
tx, err := harness.createTx(spendableOuts[0], 0, 1)
tx, err := harness.createTx(spendableOuts[0], uint64(txRelayFeeForTest), 1)
if err != nil {
t.Fatalf("unable to create transaction: %v", err)
}
_, err = harness.txPool.ProcessTransaction(tx, true, false, 0)
_, err = harness.txPool.ProcessTransaction(tx, false, 0)
if err != nil {
t.Errorf("ProcessTransaction: unexpected error: %v", err)
}
@ -800,34 +790,6 @@ func TestAddrIndex(t *testing.T) {
}
}
func TestFeeEstimatorCfg(t *testing.T) {
tc, spendableOuts, teardownFunc, err := newPoolHarness(t, &dagconfig.MainNetParams, 2, "TestFeeEstimatorCfg")
if err != nil {
t.Fatalf("unable to create test pool: %v", err)
}
defer teardownFunc()
harness := tc.harness
harness.txPool.cfg.FeeEstimator = &FeeEstimator{}
enteredObserveTransaction := false
guard := monkey.Patch((*FeeEstimator).ObserveTransaction, func(ef *FeeEstimator, t *TxDesc) {
enteredObserveTransaction = true
})
defer guard.Unpatch()
tx, err := harness.createTx(spendableOuts[0], 0, 1)
if err != nil {
t.Fatalf("unable to create transaction: %v", err)
}
_, err = harness.txPool.ProcessTransaction(tx, true, false, 0)
if err != nil {
t.Errorf("ProcessTransaction: unexpected error: %v", err)
}
if !enteredObserveTransaction {
t.Errorf("TestFeeEstimatorCfg: (*FeeEstimator).ObserveTransaction was not called")
}
}
func TestDoubleSpends(t *testing.T) {
tc, spendableOuts, teardownFunc, err := newPoolHarness(t, &dagconfig.MainNetParams, 2, "TestDoubleSpends")
if err != nil {
@ -837,28 +799,28 @@ func TestDoubleSpends(t *testing.T) {
harness := tc.harness
//Add two transactions to the mempool
tx1, err := harness.createTx(spendableOuts[0], 0, 1)
tx1, err := harness.createTx(spendableOuts[0], uint64(txRelayFeeForTest), 1)
if err != nil {
t.Fatalf("unable to create transaction: %v", err)
}
harness.txPool.ProcessTransaction(tx1, true, false, 0)
harness.txPool.ProcessTransaction(tx1, true, 0)
tx2, err := harness.createTx(spendableOuts[1], 1, 1)
tx2, err := harness.createTx(spendableOuts[1], uint64(txRelayFeeForTest)+1, 1)
if err != nil {
t.Fatalf("unable to create transaction: %v", err)
}
harness.txPool.ProcessTransaction(tx2, true, false, 0)
harness.txPool.ProcessTransaction(tx2, true, 0)
testPoolMembership(tc, tx1, false, true, false)
testPoolMembership(tc, tx2, false, true, false)
//Spends the same outpoint as tx2
tx3, err := harness.createTx(spendableOuts[0], 2, 1) //We put here different fee to create different transaction hash
tx3, err := harness.createTx(spendableOuts[0], uint64(txRelayFeeForTest)+2, 1) // We put here different fee to create different transaction hash
if err != nil {
t.Fatalf("unable to create transaction: %v", err)
}
//First we try to add it to the mempool and see it rejected
_, err = harness.txPool.ProcessTransaction(tx3, true, false, 0)
_, err = harness.txPool.ProcessTransaction(tx3, true, 0)
if err == nil {
t.Errorf("ProcessTransaction expected an error, not nil")
}
@ -893,7 +855,7 @@ func TestFetchTransaction(t *testing.T) {
if err != nil {
t.Fatalf("unable to create signed tx: %v", err)
}
harness.txPool.ProcessTransaction(orphanedTx, true, false, 0)
harness.txPool.ProcessTransaction(orphanedTx, true, 0)
testPoolMembership(tc, orphanedTx, true, false, false)
fetchedorphanedTx, err := harness.txPool.FetchTransaction(orphanedTx.ID())
if fetchedorphanedTx != nil {
@ -903,11 +865,11 @@ func TestFetchTransaction(t *testing.T) {
t.Errorf("FetchTransaction: expected an error, not nil")
}
tx, err := harness.createTx(spendableOuts[0], 0, 1)
tx, err := harness.createTx(spendableOuts[0], uint64(txRelayFeeForTest), 1)
if err != nil {
t.Fatalf("unable to create transaction: %v", err)
}
harness.txPool.ProcessTransaction(tx, true, false, 0)
harness.txPool.ProcessTransaction(tx, true, 0)
testPoolMembership(tc, tx, false, true, false)
fetchedTx, err := harness.txPool.FetchTransaction(tx.ID())
if !reflect.DeepEqual(fetchedTx, tx) {
@ -943,8 +905,7 @@ func TestSimpleOrphanChain(t *testing.T) {
// Ensure the orphans are accepted (only up to the maximum allowed so
// none are evicted).
for _, tx := range chainedTxns[1 : maxOrphans+1] {
acceptedTxns, err := harness.txPool.ProcessTransaction(tx, true,
false, 0)
acceptedTxns, err := harness.txPool.ProcessTransaction(tx, true, 0)
if err != nil {
t.Fatalf("ProcessTransaction: failed to accept valid "+
"orphan %v", err)
@ -966,8 +927,7 @@ func TestSimpleOrphanChain(t *testing.T) {
// all get accepted. Notice the accept orphans flag is also false here
// to ensure it has no bearing on whether or not already existing
// orphans in the pool are linked.
acceptedTxns, err := harness.txPool.ProcessTransaction(chainedTxns[0],
false, false, 0)
acceptedTxns, err := harness.txPool.ProcessTransaction(chainedTxns[0], false, 0)
if err != nil {
t.Fatalf("ProcessTransaction: failed to accept valid "+
"orphan %v", err)
@ -1004,8 +964,7 @@ func TestOrphanReject(t *testing.T) {
// Ensure orphans are rejected when the allow orphans flag is not set.
for _, tx := range chainedTxns[1:] {
acceptedTxns, err := harness.txPool.ProcessTransaction(tx, false,
false, 0)
acceptedTxns, err := harness.txPool.ProcessTransaction(tx, false, 0)
if err == nil {
t.Fatalf("ProcessTransaction: did not fail on orphan "+
"%v when allow orphans flag is false", tx.ID())
@ -1053,16 +1012,14 @@ func TestOrphanExpiration(t *testing.T) {
amount: util.Amount(5000000000),
outpoint: wire.Outpoint{TxID: daghash.TxID{}, Index: 0},
}}, 1)
harness.txPool.ProcessTransaction(expiredTx, true,
false, 0)
harness.txPool.ProcessTransaction(expiredTx, true, 0)
harness.txPool.orphans[*expiredTx.ID()].expiration = time.Unix(0, 0)
tx1, err := harness.CreateSignedTx([]spendableOutpoint{{
amount: util.Amount(5000000000),
outpoint: wire.Outpoint{TxID: daghash.TxID{1}, Index: 0},
}}, 1)
harness.txPool.ProcessTransaction(tx1, true,
false, 0)
harness.txPool.ProcessTransaction(tx1, true, 0)
//First check that expired orphan transactions are not removed before nextExpireScan
testPoolMembership(tc, tx1, true, false, false)
@ -1076,8 +1033,7 @@ func TestOrphanExpiration(t *testing.T) {
amount: util.Amount(5000000000),
outpoint: wire.Outpoint{TxID: daghash.TxID{2}, Index: 0},
}}, 1)
harness.txPool.ProcessTransaction(tx2, true,
false, 0)
harness.txPool.ProcessTransaction(tx2, true, 0)
//Check that only expired orphan transactions are removed
testPoolMembership(tc, tx1, true, false, false)
testPoolMembership(tc, tx2, true, false, false)
@ -1103,14 +1059,12 @@ func TestMaxOrphanTxSize(t *testing.T) {
if err != nil {
t.Fatalf("unable to create signed tx: %v", err)
}
harness.txPool.ProcessTransaction(tx, true,
false, 0)
harness.txPool.ProcessTransaction(tx, true, 0)
testPoolMembership(tc, tx, false, false, false)
harness.txPool.cfg.Policy.MaxOrphanTxSize = math.MaxInt32
harness.txPool.ProcessTransaction(tx, true,
false, 0)
harness.txPool.ProcessTransaction(tx, true, 0)
testPoolMembership(tc, tx, true, false, false)
}
@ -1129,8 +1083,7 @@ func TestRemoveTransaction(t *testing.T) {
}
for i, tx := range chainedTxns {
_, err := harness.txPool.ProcessTransaction(tx, true,
false, 0)
_, err := harness.txPool.ProcessTransaction(tx, true, 0)
if err != nil {
t.Fatalf("ProcessTransaction: %v", err)
}
@ -1163,8 +1116,7 @@ func TestRemoveTransaction(t *testing.T) {
defer guard.Unpatch()
tx, err := harness.CreateSignedTx(outputs[1:], 1)
_, err = harness.txPool.ProcessTransaction(tx, true,
false, 0)
_, err = harness.txPool.ProcessTransaction(tx, true, 0)
if err != nil {
t.Fatalf("ProcessTransaction: %v", err)
}
@ -1196,8 +1148,7 @@ func TestOrphanEviction(t *testing.T) {
// Add enough orphans to exceed the max allowed while ensuring they are
// all accepted. This will cause an eviction.
for _, tx := range chainedTxns[1:] {
acceptedTxns, err := harness.txPool.ProcessTransaction(tx, true,
false, 0)
acceptedTxns, err := harness.txPool.ProcessTransaction(tx, true, 0)
if err != nil {
t.Fatalf("ProcessTransaction: failed to accept valid "+
"orphan %v", err)
@ -1253,8 +1204,7 @@ func TestRemoveOrphansByTag(t *testing.T) {
if err != nil {
t.Fatalf("unable to create signed tx: %v", err)
}
harness.txPool.ProcessTransaction(orphanedTx1, true,
false, 1)
harness.txPool.ProcessTransaction(orphanedTx1, true, 1)
orphanedTx2, err := harness.CreateSignedTx([]spendableOutpoint{{
amount: util.Amount(5000000000),
outpoint: wire.Outpoint{TxID: daghash.TxID{2}, Index: 2},
@ -1262,8 +1212,7 @@ func TestRemoveOrphansByTag(t *testing.T) {
if err != nil {
t.Fatalf("unable to create signed tx: %v", err)
}
harness.txPool.ProcessTransaction(orphanedTx2, true,
false, 1)
harness.txPool.ProcessTransaction(orphanedTx2, true, 1)
orphanedTx3, err := harness.CreateSignedTx([]spendableOutpoint{{
amount: util.Amount(5000000000),
outpoint: wire.Outpoint{TxID: daghash.TxID{3}, Index: 3},
@ -1271,8 +1220,7 @@ func TestRemoveOrphansByTag(t *testing.T) {
if err != nil {
t.Fatalf("unable to create signed tx: %v", err)
}
harness.txPool.ProcessTransaction(orphanedTx3, true,
false, 1)
harness.txPool.ProcessTransaction(orphanedTx3, true, 1)
orphanedTx4, err := harness.CreateSignedTx([]spendableOutpoint{{
amount: util.Amount(5000000000),
@ -1281,8 +1229,7 @@ func TestRemoveOrphansByTag(t *testing.T) {
if err != nil {
t.Fatalf("unable to create signed tx: %v", err)
}
harness.txPool.ProcessTransaction(orphanedTx4, true,
false, 2)
harness.txPool.ProcessTransaction(orphanedTx4, true, 2)
harness.txPool.RemoveOrphansByTag(1)
testPoolMembership(tc, orphanedTx1, false, false, false)
@ -1315,8 +1262,7 @@ func TestBasicOrphanRemoval(t *testing.T) {
// Ensure the orphans are accepted (only up to the maximum allowed so
// none are evicted).
for _, tx := range chainedTxns[1 : maxOrphans+1] {
acceptedTxns, err := harness.txPool.ProcessTransaction(tx, true,
false, 0)
acceptedTxns, err := harness.txPool.ProcessTransaction(tx, true, 0)
if err != nil {
t.Fatalf("ProcessTransaction: failed to accept valid "+
"orphan %v", err)
@ -1389,8 +1335,7 @@ func TestOrphanChainRemoval(t *testing.T) {
// Ensure the orphans are accepted (only up to the maximum allowed so
// none are evicted).
for _, tx := range chainedTxns[1 : maxOrphans+1] {
acceptedTxns, err := harness.txPool.ProcessTransaction(tx, true,
false, 0)
acceptedTxns, err := harness.txPool.ProcessTransaction(tx, true, 0)
if err != nil {
t.Fatalf("ProcessTransaction: failed to accept valid "+
"orphan %v", err)
@ -1451,8 +1396,7 @@ func TestMultiInputOrphanDoubleSpend(t *testing.T) {
// Start by adding the orphan transactions from the generated chain
// except the final one.
for _, tx := range chainedTxns[1:maxOrphans] {
acceptedTxns, err := harness.txPool.ProcessTransaction(tx, true,
false, 0)
acceptedTxns, err := harness.txPool.ProcessTransaction(tx, true, 0)
if err != nil {
t.Fatalf("ProcessTransaction: failed to accept valid "+
"orphan %v", err)
@ -1477,8 +1421,7 @@ func TestMultiInputOrphanDoubleSpend(t *testing.T) {
if err != nil {
t.Fatalf("unable to create signed tx: %v", err)
}
acceptedTxns, err := harness.txPool.ProcessTransaction(doubleSpendTx,
true, false, 0)
acceptedTxns, err := harness.txPool.ProcessTransaction(doubleSpendTx, true, 0)
if err != nil {
t.Fatalf("ProcessTransaction: failed to accept valid orphan %v",
err)
@ -1496,8 +1439,7 @@ func TestMultiInputOrphanDoubleSpend(t *testing.T) {
//
// This will cause the shared output to become a concrete spend which
// will in turn must cause the double spending orphan to be removed.
acceptedTxns, err = harness.txPool.ProcessTransaction(chainedTxns[0],
false, false, 0)
acceptedTxns, err = harness.txPool.ProcessTransaction(chainedTxns[0], false, 0)
if err != nil {
t.Fatalf("ProcessTransaction: failed to accept valid tx %v", err)
}
@ -1544,8 +1486,7 @@ func TestCheckSpend(t *testing.T) {
t.Fatalf("unable to create transaction chain: %v", err)
}
for _, tx := range chainedTxns {
_, err := harness.txPool.ProcessTransaction(tx, true,
false, 0)
_, err := harness.txPool.ProcessTransaction(tx, true, 0)
if err != nil {
t.Fatalf("ProcessTransaction: failed to accept "+
"tx: %v", err)
@ -1603,7 +1544,7 @@ func TestCount(t *testing.T) {
}
for i, tx := range chainedTxns {
_, err = harness.txPool.ProcessTransaction(tx, true, false, 0)
_, err = harness.txPool.ProcessTransaction(tx, true, 0)
if err != nil {
t.Errorf("ProcessTransaction: unexpected error: %v", err)
}
@ -1646,10 +1587,6 @@ func TestExtractRejectCode(t *testing.T) {
blockdagRuleErrorCode: blockdag.ErrBadCheckpoint,
wireRejectCode: wire.RejectCheckpoint,
},
{
blockdagRuleErrorCode: blockdag.ErrForkTooOld,
wireRejectCode: wire.RejectCheckpoint,
},
{
blockdagRuleErrorCode: math.MaxUint32,
wireRejectCode: wire.RejectInvalid,
@ -1713,14 +1650,15 @@ func TestHandleNewBlock(t *testing.T) {
// Create orphan transaction and add it to UTXO set
txID := blockTx1.ID()
orphanTx, err := harness.CreateSignedTx([]spendableOutpoint{{
amount: util.Amount(2500000000),
amount: util.Amount(2500000000 - txRelayFeeForTest),
outpoint: wire.Outpoint{TxID: *txID, Index: 0},
}}, 1)
if err != nil {
t.Fatalf("unable to create signed tx: %v", err)
}
_, err = harness.txPool.ProcessTransaction(orphanTx, true, false, 0)
_, err = harness.txPool.ProcessTransaction(orphanTx, true, 0)
if err != nil {
t.Fatalf("ProcessTransaction: unexpected error: %v", err)
}
@ -1752,9 +1690,9 @@ func TestHandleNewBlock(t *testing.T) {
}()
// process messages pushed by HandleNewBlock
blockTransnactions := make(map[daghash.TxID]int)
blockTransactions := make(map[daghash.TxID]int)
for msg := range ch {
blockTransnactions[*msg.Tx.ID()] = 1
blockTransactions[*msg.Tx.ID()] = 1
if *msg.Tx.ID() != *blockTx1.ID() {
if len(msg.AcceptedTxs) != 0 {
t.Fatalf("Expected amount of accepted transactions 0. Got: %v", len(msg.AcceptedTxs))
@ -1774,15 +1712,15 @@ func TestHandleNewBlock(t *testing.T) {
}
// Validate messages pushed by HandleNewBlock into the channel
if len(blockTransnactions) != 2 {
t.Fatalf("Wrong size of blockTransnactions after new block handling")
if len(blockTransactions) != 2 {
t.Fatalf("Wrong size of blockTransactions after new block handling")
}
if _, ok := blockTransnactions[*blockTx1.ID()]; !ok {
if _, ok := blockTransactions[*blockTx1.ID()]; !ok {
t.Fatalf("Transaction 1 of new block is not handled")
}
if _, ok := blockTransnactions[*blockTx2.ID()]; !ok {
if _, ok := blockTransactions[*blockTx2.ID()]; !ok {
t.Fatalf("Transaction 2 of new block is not handled")
}
@ -1853,6 +1791,24 @@ var dummyBlock = wire.MsgBlock{
LockTime: 0,
SubnetworkID: *subnetworkid.SubnetworkIDNative,
},
{
Version: 1,
TxIn: []*wire.TxIn{
{
PreviousOutpoint: wire.Outpoint{
TxID: daghash.TxID{
0x16, 0x5e, 0x38, 0xe8, 0xb3, 0x91, 0x45, 0x95,
0xd9, 0xc6, 0x41, 0xf3, 0xb8, 0xee, 0xc2, 0xf3,
0x46, 0x11, 0x89, 0x6b, 0x82, 0x1a, 0x68, 0x3b,
0x7a, 0x4e, 0xde, 0xfe, 0x2c, 0x00, 0x00, 0x00,
},
Index: 0xffffffff,
},
Sequence: math.MaxUint64,
},
},
SubnetworkID: *subnetworkid.SubnetworkIDNative,
},
},
}
@ -1877,7 +1833,7 @@ func TestTransactionGas(t *testing.T) {
if err != nil {
t.Fatalf("unable to create transaction: %v", err)
}
_, err = harness.txPool.ProcessTransaction(tx, true, false, 0)
_, err = harness.txPool.ProcessTransaction(tx, true, 0)
if err != nil {
t.Errorf("ProcessTransaction: unexpected error: %v", err)
}
@ -1887,7 +1843,7 @@ func TestTransactionGas(t *testing.T) {
if err != nil {
t.Fatalf("unable to create transaction: %v", err)
}
_, err = harness.txPool.ProcessTransaction(tx, true, false, 0)
_, err = harness.txPool.ProcessTransaction(tx, true, 0)
if err == nil {
t.Error("ProcessTransaction did not return error, expecting ErrInvalidGas")
}

View File

@ -245,7 +245,7 @@ func isDust(txOut *wire.TxOut, minRelayTxFee util.Amount) bool {
// finalized, conforming to more stringent size constraints, having scripts
// of recognized forms, and not containing "dust" outputs (those that are
// so small it costs more to process them than they are worth).
func checkTransactionStandard(tx *util.Tx, height uint64,
func checkTransactionStandard(tx *util.Tx, blueScore uint64,
medianTimePast time.Time, policy *Policy) error {
// The transaction must be a currently supported version.
@ -259,7 +259,7 @@ func checkTransactionStandard(tx *util.Tx, height uint64,
// The transaction must be finalized to be standard and therefore
// considered for inclusion in a block.
if !blockdag.IsFinalizedTransaction(tx, height, medianTimePast) {
if !blockdag.IsFinalizedTransaction(tx, blueScore, medianTimePast) {
return txRuleError(wire.RejectNonstandard,
"transaction is not finalized")
}

View File

@ -206,7 +206,7 @@ func (m *CPUMiner) submitBlock(block *util.Block) bool {
// This function will return early with false when conditions that trigger a
// stale block such as a new block showing up or periodically when there are
// new transactions and enough time has elapsed without finding a solution.
func (m *CPUMiner) solveBlock(msgBlock *wire.MsgBlock, blockHeight uint64,
func (m *CPUMiner) solveBlock(msgBlock *wire.MsgBlock, blueScore uint64,
ticker *time.Ticker, quit chan struct{}) bool {
// Create some convenience variables.
@ -228,7 +228,7 @@ func (m *CPUMiner) solveBlock(msgBlock *wire.MsgBlock, blockHeight uint64,
// Update the extra nonce in the block template with the
// new value by regenerating the coinbase script and
// setting the merkle root to the new value.
m.g.UpdateExtraNonce(msgBlock, blockHeight, extraNonce)
m.g.UpdateExtraNonce(msgBlock, blueScore, extraNonce)
// Search through the entire nonce range for a solution while
// periodically checking for early quit and stale block
@ -320,8 +320,8 @@ out:
// this would otherwise end up building a new block template on
// a block that is in the process of becoming stale.
m.submitBlockLock.Lock()
curHeight := m.g.DAGHeight()
if (curHeight != 0 && !m.cfg.IsCurrent()) || (curHeight == 0 && !m.cfg.ShouldMineOnGenesis()) {
currentBlueScore := m.g.VirtualBlueScore()
if (currentBlueScore != 0 && !m.cfg.IsCurrent()) || (currentBlueScore == 0 && !m.cfg.ShouldMineOnGenesis()) {
m.submitBlockLock.Unlock()
time.Sleep(time.Second)
continue
@ -347,7 +347,7 @@ out:
// with false when conditions that trigger a stale block, so
// a new block template can be generated. When the return is
// true a solution was found, so submit the solved block.
if m.solveBlock(template.Block, curHeight+1, ticker, quit) {
if m.solveBlock(template.Block, currentBlueScore+1, ticker, quit) {
block := util.NewBlock(template.Block)
m.submitBlock(block)
}
@ -581,7 +581,7 @@ func (m *CPUMiner) GenerateNBlocks(n uint32) ([]*daghash.Hash, error) {
// be changing and this would otherwise end up building a new block
// template on a block that is in the process of becoming stale.
m.submitBlockLock.Lock()
curHeight := m.g.DAGHeight()
currentBlueScore := m.g.VirtualBlueScore()
// Choose a payment address at random.
rand.Seed(time.Now().UnixNano())
@ -603,7 +603,7 @@ func (m *CPUMiner) GenerateNBlocks(n uint32) ([]*daghash.Hash, error) {
// with false when conditions that trigger a stale block, so
// a new block template can be generated. When the return is
// true a solution was found, so submit the solved block.
if m.solveBlock(template.Block, curHeight+1, ticker, nil) {
if m.solveBlock(template.Block, currentBlueScore, ticker, nil) {
block := util.NewBlock(template.Block)
m.submitBlock(block)
blockHashes[i] = block.Hash()

View File

@ -21,10 +21,6 @@ import (
)
const (
// MinHighPriority is the minimum priority value that allows a
// transaction to be considered high priority.
MinHighPriority = util.SatoshiPerBitcoin * 144.0 / 250
// blockHeaderOverhead is the max number of bytes it takes to serialize
// a block header and max possible transaction count.
blockHeaderOverhead = wire.MaxBlockHeaderPayload + wire.MaxVarIntPayload
@ -80,7 +76,6 @@ type TxSource interface {
type txPrioItem struct {
tx *util.Tx
fee uint64
priority float64
feePerKB uint64
}
@ -138,44 +133,21 @@ func (pq *txPriorityQueue) SetLessFunc(lessFunc txPriorityQueueLessFunc) {
heap.Init(pq)
}
// txPQByPriority sorts a txPriorityQueue by transaction priority and then fees
// per kilobyte.
func txPQByPriority(pq *txPriorityQueue, i, j int) bool {
// Using > here so that pop gives the highest priority item as opposed
// to the lowest. Sort by priority first, then fee.
if pq.items[i].priority == pq.items[j].priority {
return pq.items[i].feePerKB > pq.items[j].feePerKB
}
return pq.items[i].priority > pq.items[j].priority
}
// txPQByFee sorts a txPriorityQueue by fees per kilobyte and then transaction
// priority.
// txPQByFee sorts a txPriorityQueue by fees per kilobyte
func txPQByFee(pq *txPriorityQueue, i, j int) bool {
// Using > here so that pop gives the highest fee item as opposed
// to the lowest. Sort by fee first, then priority.
if pq.items[i].feePerKB == pq.items[j].feePerKB {
return pq.items[i].priority > pq.items[j].priority
}
return pq.items[i].feePerKB > pq.items[j].feePerKB
}
// newTxPriorityQueue returns a new transaction priority queue that reserves the
// passed amount of space for the elements. The new priority queue uses either
// the txPQByPriority or the txPQByFee compare function depending on the
// sortByFee parameter and is already initialized for use with heap.Push/Pop.
// passed amount of space for the elements. The new priority queue uses the
// txPQByFee compare function and is already initialized for use with heap.Push/Pop.
// The priority queue can grow larger than the reserved space, but extra copies
// of the underlying array can be avoided by reserving a sane value.
func newTxPriorityQueue(reserve int, sortByFee bool) *txPriorityQueue {
func newTxPriorityQueue(reserve int) *txPriorityQueue {
pq := &txPriorityQueue{
items: make([]*txPrioItem, 0, reserve),
}
if sortByFee {
pq.SetLessFunc(txPQByFee)
} else {
pq.SetLessFunc(txPQByPriority)
}
pq.SetLessFunc(txPQByFee)
return pq
}
@ -210,21 +182,21 @@ type BlockTemplate struct {
// StandardCoinbaseScript returns a standard script suitable for use as the
// signature script of the coinbase transaction of a new block. In particular,
// it starts with the block height that is required by version 2 blocks and adds
// it starts with the block blue score that is required by version 2 blocks and adds
// the extra nonce as well as additional coinbase flags.
func StandardCoinbaseScript(nextBlockHeight uint64, extraNonce uint64) ([]byte, error) {
return txscript.NewScriptBuilder().AddInt64(int64(nextBlockHeight)).
func StandardCoinbaseScript(nextBlueScore uint64, extraNonce uint64) ([]byte, error) {
return txscript.NewScriptBuilder().AddInt64(int64(nextBlueScore)).
AddInt64(int64(extraNonce)).AddData([]byte(CoinbaseFlags)).
Script()
}
// CreateCoinbaseTx returns a coinbase transaction paying an appropriate subsidy
// based on the passed block height to the provided address. When the address
// based on the passed block blue score to the provided address. When the address
// is nil, the coinbase transaction will instead be redeemable by anyone.
//
// See the comment for NewBlockTemplate for more information about why the nil
// address handling is useful.
func CreateCoinbaseTx(params *dagconfig.Params, coinbaseScript []byte, nextBlockHeight uint64, addr util.Address) (*util.Tx, error) {
func CreateCoinbaseTx(params *dagconfig.Params, coinbaseScript []byte, nextBlueScore uint64, addr util.Address) (*util.Tx, error) {
// Create the script to pay to the provided payment address if one was
// specified. Otherwise create a script that allows the coinbase to be
// redeemable by anyone.
@ -256,7 +228,7 @@ func CreateCoinbaseTx(params *dagconfig.Params, coinbaseScript []byte, nextBlock
Sequence: wire.MaxTxInSequenceNum,
}
txOut := &wire.TxOut{
Value: blockdag.CalcBlockSubsidy(nextBlockHeight, params),
Value: blockdag.CalcBlockSubsidy(nextBlueScore, params),
PkScript: pkScript,
}
return util.NewTx(wire.NewNativeMsgTx(wire.TxVersion, []*wire.TxIn{txIn}, []*wire.TxOut{txOut})), nil
@ -389,9 +361,6 @@ func (g *BlkTmplGenerator) NewBlockTemplate(payToAddress util.Address) (*BlockTe
g.dag.RLock()
defer g.dag.RUnlock()
// Extend the most recently known best block.
nextBlockHeight := g.dag.Height() + 1
// Create a standard coinbase transaction paying to the provided
// address. NOTE: The coinbase value will be updated to include the
// fees from the selected transactions later after they have actually
@ -400,15 +369,16 @@ func (g *BlkTmplGenerator) NewBlockTemplate(payToAddress util.Address) (*BlockTe
// ensure the transaction is not a duplicate transaction (paying the
// same value to the same public key address would otherwise be an
// identical transaction for block version 1).
nextBlockBlueScore := g.dag.VirtualBlueScore()
extraNonce, err := random.Uint64()
if err != nil {
return nil, err
}
coinbaseScript, err := StandardCoinbaseScript(nextBlockHeight, extraNonce)
coinbaseScript, err := StandardCoinbaseScript(nextBlockBlueScore, extraNonce)
if err != nil {
return nil, err
}
coinbaseTx, err := CreateCoinbaseTx(g.dagParams, coinbaseScript, nextBlockHeight, payToAddress)
coinbaseTx, err := CreateCoinbaseTx(g.dagParams, coinbaseScript, nextBlockBlueScore, payToAddress)
if err != nil {
return nil, err
}
@ -428,8 +398,7 @@ func (g *BlkTmplGenerator) NewBlockTemplate(payToAddress util.Address) (*BlockTe
// choose the initial sort order for the priority queue based on whether
// or not there is an area allocated for high-priority transactions.
sourceTxns := g.txSource.MiningDescs()
sortedByFee := g.policy.BlockPrioritySize == 0
priorityQueue := newTxPriorityQueue(len(sourceTxns), sortedByFee)
priorityQueue := newTxPriorityQueue(len(sourceTxns))
// Create a slice to hold the transactions to be included in the
// generated block with reserved space. Also create a utxo view to
@ -466,7 +435,7 @@ func (g *BlkTmplGenerator) NewBlockTemplate(payToAddress util.Address) (*BlockTe
log.Tracef("Skipping block reward tx %s", tx.ID())
continue
}
if !blockdag.IsFinalizedTransaction(tx, nextBlockHeight,
if !blockdag.IsFinalizedTransaction(tx, nextBlockBlueScore,
g.timeSource.AdjustedTime()) {
log.Tracef("Skipping non-finalized tx %s", tx.ID())
@ -477,8 +446,6 @@ func (g *BlkTmplGenerator) NewBlockTemplate(payToAddress util.Address) (*BlockTe
// value age sum as well as the adjusted transaction size. The
// formula is: sum(inputValue * inputAge) / adjustedTxSize
prioItem := &txPrioItem{tx: tx}
prioItem.priority = CalcPriority(tx.MsgTx(), g.dag.UTXOSet(),
nextBlockHeight)
// Calculate the fee in Satoshi/kB.
prioItem.feePerKB = txDesc.FeePerKB
@ -552,52 +519,9 @@ func (g *BlkTmplGenerator) NewBlockTemplate(payToAddress util.Address) (*BlockTe
continue
}
// Skip free transactions once the block is larger than the
// minimum block size.
if sortedByFee &&
prioItem.feePerKB < uint64(g.policy.TxMinFreeFee) &&
blockPlusTxSize >= g.policy.BlockMinSize {
log.Tracef("Skipping tx %s with feePerKB %.2f "+
"< TxMinFreeFee %d and block size %d >= "+
"minBlockSize %d", tx.ID(), prioItem.feePerKB,
g.policy.TxMinFreeFee, blockPlusTxSize,
g.policy.BlockMinSize)
continue
}
// Prioritize by fee per kilobyte once the block is larger than
// the priority size or there are no more high-priority
// transactions.
if !sortedByFee && (blockPlusTxSize >= g.policy.BlockPrioritySize ||
prioItem.priority <= MinHighPriority) {
log.Tracef("Switching to sort by fees per kilobyte "+
"blockSize %d >= BlockPrioritySize %d || "+
"priority %.2f <= minHighPriority %.2f",
blockPlusTxSize, g.policy.BlockPrioritySize,
prioItem.priority, MinHighPriority)
sortedByFee = true
priorityQueue.SetLessFunc(txPQByFee)
// Put the transaction back into the priority queue and
// skip it so it is re-priortized by fees if it won't
// fit into the high-priority section or the priority
// is too low. Otherwise this transaction will be the
// final one in the high-priority section, so just fall
// though to the code below so it is added now.
if blockPlusTxSize > g.policy.BlockPrioritySize ||
prioItem.priority < MinHighPriority {
heap.Push(priorityQueue, prioItem)
continue
}
}
// Ensure the transaction inputs pass all of the necessary
// preconditions before allowing it to be added to the block.
_, err = blockdag.CheckTransactionInputsAndCalulateFee(tx, nextBlockHeight,
_, err = blockdag.CheckTransactionInputsAndCalulateFee(tx, nextBlockBlueScore,
g.dag.UTXOSet(), g.dagParams, false)
if err != nil {
log.Tracef("Skipping tx %s due to error in "+
@ -622,8 +546,8 @@ func (g *BlkTmplGenerator) NewBlockTemplate(payToAddress util.Address) (*BlockTe
txFees = append(txFees, prioItem.fee)
txSigOpCounts = append(txSigOpCounts, numSigOps)
log.Tracef("Adding tx %s (priority %.2f, feePerKB %.2f)",
prioItem.tx.ID(), prioItem.priority, prioItem.feePerKB)
log.Tracef("Adding tx %s (feePerKB %.2f)",
prioItem.tx.ID(), prioItem.feePerKB)
}
// Now that the actual transactions have been selected, update the
@ -663,7 +587,7 @@ func (g *BlkTmplGenerator) NewBlockTemplate(payToAddress util.Address) (*BlockTe
for _, tx := range blockTxns {
msgBlock.AddTransaction(tx.MsgTx())
}
utxoCommitment, err := g.buildUTXOCommitment(msgBlock.Transactions, nextBlockHeight)
utxoCommitment, err := g.buildUTXOCommitment(msgBlock.Transactions, nextBlockBlueScore)
if err != nil {
return nil, err
}
@ -677,11 +601,10 @@ func (g *BlkTmplGenerator) NewBlockTemplate(payToAddress util.Address) (*BlockTe
Bits: reqDifficulty,
}
// Finally, perform a full check on the created block against the chain
// consensus rules to ensure it properly connects to the current best
// chain with no issues.
// Finally, perform a full check on the created block against the DAG
// consensus rules to ensure it properly connects to the DAG with no
// issues.
block := util.NewBlock(&msgBlock)
block.SetHeight(nextBlockHeight)
if err := g.dag.CheckConnectBlockTemplateNoLock(block); err != nil {
return nil, err
@ -696,13 +619,12 @@ func (g *BlkTmplGenerator) NewBlockTemplate(payToAddress util.Address) (*BlockTe
Block: &msgBlock,
Fees: txFees,
SigOpCounts: txSigOpCounts,
Height: nextBlockHeight,
ValidPayAddress: payToAddress != nil,
}, nil
}
func (g *BlkTmplGenerator) buildUTXOCommitment(transactions []*wire.MsgTx, nextBlockHeight uint64) (*daghash.Hash, error) {
utxoWithTransactions, err := g.dag.UTXOSet().WithTransactions(transactions, nextBlockHeight, false)
func (g *BlkTmplGenerator) buildUTXOCommitment(transactions []*wire.MsgTx, nextBlueScore uint64) (*daghash.Hash, error) {
utxoWithTransactions, err := g.dag.UTXOSet().WithTransactions(transactions, nextBlueScore, false)
if err != nil {
return nil, err
}
@ -739,8 +661,8 @@ func (g *BlkTmplGenerator) UpdateBlockTime(msgBlock *wire.MsgBlock) error {
// block by regenerating the coinbase script with the passed value and block
// height. It also recalculates and updates the new merkle root that results
// from changing the coinbase script.
func (g *BlkTmplGenerator) UpdateExtraNonce(msgBlock *wire.MsgBlock, blockHeight uint64, extraNonce uint64) error {
coinbaseScript, err := StandardCoinbaseScript(blockHeight, extraNonce)
func (g *BlkTmplGenerator) UpdateExtraNonce(msgBlock *wire.MsgBlock, blockBlueScore uint64, extraNonce uint64) error {
coinbaseScript, err := StandardCoinbaseScript(blockBlueScore, extraNonce)
if err != nil {
return err
}
@ -761,7 +683,7 @@ func (g *BlkTmplGenerator) UpdateExtraNonce(msgBlock *wire.MsgBlock, blockHeight
hashMerkleTree := blockdag.BuildHashMerkleTreeStore(block.Transactions())
msgBlock.Header.HashMerkleRoot = hashMerkleTree.Root()
utxoCommitment, err := g.buildUTXOCommitment(msgBlock.Transactions, blockHeight)
utxoCommitment, err := g.buildUTXOCommitment(msgBlock.Transactions, blockBlueScore)
if err != nil {
return err
}
@ -771,9 +693,9 @@ func (g *BlkTmplGenerator) UpdateExtraNonce(msgBlock *wire.MsgBlock, blockHeight
return nil
}
// DAGHeight returns the DAG's height
func (g *BlkTmplGenerator) DAGHeight() uint64 {
return g.dag.Height()
// VirtualBlueScore returns the virtual block's current blue score
func (g *BlkTmplGenerator) VirtualBlueScore() uint64 {
return g.dag.VirtualBlueScore()
}
// TipHashes returns the hashes of the DAG's tips

View File

@ -28,18 +28,11 @@ func TestTxFeePrioHeap(t *testing.T) {
// Create some fake priority items that exercise the expected sort
// edge conditions.
testItems := []*txPrioItem{
{feePerKB: 5678, priority: 3},
{feePerKB: 5678, priority: 1},
{feePerKB: 5678, priority: 1}, // Duplicate fee and prio
{feePerKB: 5678, priority: 5},
{feePerKB: 5678, priority: 2},
{feePerKB: 1234, priority: 3},
{feePerKB: 1234, priority: 1},
{feePerKB: 1234, priority: 5},
{feePerKB: 1234, priority: 5}, // Duplicate fee and prio
{feePerKB: 1234, priority: 2},
{feePerKB: 10000, priority: 0}, // Higher fee, smaller prio
{feePerKB: 0, priority: 10000}, // Higher prio, lower fee
{feePerKB: 5678},
{feePerKB: 5678}, // Duplicate fee
{feePerKB: 1234},
{feePerKB: 10000}, // High fee
{feePerKB: 0}, // Zero fee
}
// Add random data in addition to the edge conditions already manually
@ -54,21 +47,14 @@ func TestTxFeePrioHeap(t *testing.T) {
for i := 0; i < 1000; i++ {
testItems = append(testItems, &txPrioItem{
feePerKB: uint64(prng.Float64() * util.SatoshiPerBitcoin),
priority: prng.Float64() * 100,
})
}
// Test sorting by fee per KB then priority.
// Test sorting by fee per KB
var highest *txPrioItem
priorityQueue := newTxPriorityQueue(len(testItems), true)
for i := 0; i < len(testItems); i++ {
prioItem := testItems[i]
if highest == nil {
highest = prioItem
}
if prioItem.feePerKB >= highest.feePerKB &&
prioItem.priority > highest.priority {
priorityQueue := newTxPriorityQueue(len(testItems))
for _, prioItem := range testItems {
if highest == nil || prioItem.feePerKB >= highest.feePerKB {
highest = prioItem
}
heap.Push(priorityQueue, prioItem)
@ -76,44 +62,10 @@ func TestTxFeePrioHeap(t *testing.T) {
for i := 0; i < len(testItems); i++ {
prioItem := heap.Pop(priorityQueue).(*txPrioItem)
if prioItem.feePerKB >= highest.feePerKB &&
prioItem.priority > highest.priority {
t.Fatalf("fee sort: item (fee per KB: %v, "+
"priority: %v) higher than than prev "+
"(fee per KB: %v, priority %v)",
prioItem.feePerKB, prioItem.priority,
highest.feePerKB, highest.priority)
}
highest = prioItem
}
// Test sorting by priority then fee per KB.
highest = nil
priorityQueue = newTxPriorityQueue(len(testItems), false)
for i := 0; i < len(testItems); i++ {
prioItem := testItems[i]
if highest == nil {
highest = prioItem
}
if prioItem.priority >= highest.priority &&
prioItem.feePerKB > highest.feePerKB {
highest = prioItem
}
heap.Push(priorityQueue, prioItem)
}
for i := 0; i < len(testItems); i++ {
prioItem := heap.Pop(priorityQueue).(*txPrioItem)
if prioItem.priority >= highest.priority &&
prioItem.feePerKB > highest.feePerKB {
t.Fatalf("priority sort: item (fee per KB: %v, "+
"priority: %v) higher than than prev "+
"(fee per KB: %v, priority %v)",
prioItem.feePerKB, prioItem.priority,
highest.feePerKB, highest.priority)
if prioItem.feePerKB > highest.feePerKB {
t.Fatalf("fee sort: item (fee per KB: %v) "+
"higher than than prev (fee per KB: %v)",
prioItem.feePerKB, highest.feePerKB)
}
highest = prioItem
}
@ -137,9 +89,7 @@ func TestNewBlockTemplate(t *testing.T) {
}
policy := Policy{
BlockMaxSize: 50000,
BlockPrioritySize: 750000,
TxMinFreeFee: util.Amount(0),
BlockMaxSize: 50000,
}
// First we create a block to have coinbase funds for the rest of the test.
@ -148,10 +98,10 @@ func TestNewBlockTemplate(t *testing.T) {
}
var createCoinbaseTxPatch *monkey.PatchGuard
createCoinbaseTxPatch = monkey.Patch(CreateCoinbaseTx, func(params *dagconfig.Params, coinbaseScript []byte, nextBlockHeight uint64, addr util.Address) (*util.Tx, error) {
createCoinbaseTxPatch = monkey.Patch(CreateCoinbaseTx, func(params *dagconfig.Params, coinbaseScript []byte, nextBlueScore uint64, addr util.Address) (*util.Tx, error) {
createCoinbaseTxPatch.Unpatch()
defer createCoinbaseTxPatch.Restore()
tx, err := CreateCoinbaseTx(params, coinbaseScript, nextBlockHeight, addr)
tx, err := CreateCoinbaseTx(params, coinbaseScript, nextBlueScore, addr)
if err != nil {
return nil, err
}
@ -184,13 +134,13 @@ func TestNewBlockTemplate(t *testing.T) {
t.Fatalf("ProcessBlock: template1 got unexpectedly orphan")
}
cbScript, err := StandardCoinbaseScript(dag.Height()+1, 0)
cbScript, err := StandardCoinbaseScript(dag.VirtualBlueScore(), 0)
if err != nil {
t.Fatalf("standardCoinbaseScript: %v", err)
}
// We want to check that the miner filters coinbase transaction
cbTx, err := CreateCoinbaseTx(&params, cbScript, dag.Height()+1, nil)
cbTx, err := CreateCoinbaseTx(&params, cbScript, dag.VirtualBlueScore(), nil)
if err != nil {
t.Fatalf("createCoinbaseTx: %v", err)
}
@ -231,7 +181,7 @@ func TestNewBlockTemplate(t *testing.T) {
Value: 1,
}
nonFinalizedTx := wire.NewNativeMsgTx(wire.TxVersion, []*wire.TxIn{txIn}, []*wire.TxOut{txOut})
nonFinalizedTx.LockTime = uint64(dag.Height() + 2)
nonFinalizedTx.LockTime = uint64(dag.ChainHeight() + 2)
existingSubnetwork := &subnetworkid.SubnetworkID{0xff}
nonExistingSubnetwork := &subnetworkid.SubnetworkID{0xfe}

View File

@ -4,12 +4,6 @@
package mining
import (
"github.com/daglabs/btcd/blockdag"
"github.com/daglabs/btcd/util"
"github.com/daglabs/btcd/wire"
)
// Policy houses the policy (configuration parameters) which is used to control
// the generation of block templates. See the documentation for
// NewBlockTemplate for more details on each of these parameters are used.
@ -21,96 +15,4 @@ type Policy struct {
// BlockMaxSize is the maximum block size to be used when generating a
// block template.
BlockMaxSize uint32
// BlockPrioritySize is the size in bytes for high-priority / low-fee
// transactions to be used when generating a block template.
BlockPrioritySize uint32
// TxMinFreeFee is the minimum fee in Satoshi/1000 bytes that is
// required for a transaction to be treated as free for mining purposes
// (block template generation).
TxMinFreeFee util.Amount
}
// minInt is a helper function to return the minimum of two ints. This avoids
// a math import and the need to cast to floats.
func minInt(a, b int) int {
if a < b {
return a
}
return b
}
// calcInputValueAge is a helper function used to calculate the input age of
// a transaction. The input age for a txin is the number of confirmations
// since the referenced txout multiplied by its output value. The total input
// age is the sum of this value for each txin. Any inputs to the transaction
// which are currently in the mempool and hence not mined into a block yet,
// contribute no additional input age to the transaction.
func calcInputValueAge(tx *wire.MsgTx, utxoSet blockdag.UTXOSet, nextBlockHeight uint64) float64 {
var totalInputAge float64
for _, txIn := range tx.TxIn {
// Don't attempt to accumulate the total input age if the
// referenced transaction output doesn't exist.
entry, ok := utxoSet.Get(txIn.PreviousOutpoint)
if ok {
// Inputs with dependencies currently in the mempool
// have their block height set to a special constant.
// Their input age should computed as zero since their
// parent hasn't made it into a block yet.
var inputAge uint64
originChainHeight := entry.BlockChainHeight()
if entry.IsUnmined() {
inputAge = 0
} else {
inputAge = nextBlockHeight - originChainHeight
}
// Sum the input value times age.
inputValue := entry.Amount()
totalInputAge += float64(inputValue * uint64(inputAge))
}
}
return totalInputAge
}
// CalcPriority returns a transaction priority given a transaction and the sum
// of each of its input values multiplied by their age (# of confirmations).
// Thus, the final formula for the priority is:
// sum(inputValue * inputAge) / adjustedTxSize
func CalcPriority(tx *wire.MsgTx, utxoSet blockdag.UTXOSet, nextBlockHeight uint64) float64 {
// In order to encourage spending multiple old unspent transaction
// outputs thereby reducing the total set, don't count the constant
// overhead for each input as well as enough bytes of the signature
// script to cover a pay-to-script-hash redemption with a compressed
// pubkey. This makes additional inputs free by boosting the priority
// of the transaction accordingly. No more incentive is given to avoid
// encouraging gaming future transactions through the use of junk
// outputs. This is the same logic used in the reference
// implementation.
//
// The constant overhead for a txin is 41 bytes since the previous
// outpoint is 36 bytes + 4 bytes for the sequence + 1 byte the
// signature script length.
//
// A compressed pubkey pay-to-script-hash redemption with a maximum len
// signature is of the form:
// [OP_DATA_73 <73-byte sig> + OP_DATA_35 + {OP_DATA_33
// <33 byte compresed pubkey> + OP_CHECKSIG}]
//
// Thus 1 + 73 + 1 + 1 + 33 + 1 = 110
overhead := 0
for _, txIn := range tx.TxIn {
// Max inputs + size can't possibly overflow here.
overhead += 41 + minInt(110, len(txIn.SignatureScript))
}
serializedTxSize := tx.SerializeSize()
if overhead >= serializedTxSize {
return 0.0
}
inputValueAge := calcInputValueAge(tx, utxoSet, nextBlockHeight)
return inputValueAge / float64(serializedTxSize-overhead)
}

View File

@ -1,188 +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 mining
import (
"encoding/hex"
"fmt"
"testing"
"github.com/daglabs/btcd/blockdag"
"github.com/daglabs/btcd/btcec"
"github.com/daglabs/btcd/dagconfig"
"github.com/daglabs/btcd/txscript"
"github.com/daglabs/btcd/util"
"github.com/daglabs/btcd/util/daghash"
"github.com/daglabs/btcd/wire"
)
// newHashFromStr converts the passed big-endian hex string into a
// daghash.Hash. It only differs from the one available in daghash in that
// it panics on an error since it will only (and must only) be called with
// hard-coded, and therefore known good, hashes.
func newHashFromStr(hexStr string) *daghash.Hash {
hash, err := daghash.NewHashFromStr(hexStr)
if err != nil {
panic("invalid hash in source file: " + hexStr)
}
return hash
}
// hexToBytes converts the passed hex string into bytes and will panic if there
// is an error. This is only provided for the hard-coded constants so errors in
// the source code can be detected. It will only (and must only) be called with
// hard-coded values.
func hexToBytes(s string) []byte {
b, err := hex.DecodeString(s)
if err != nil {
panic("invalid hex in source file: " + s)
}
return b
}
// newUTXOSet returns a new utxo view populated with outputs of the
// provided source transactions as if there were available at the respective
// block height specified in the heights slice. The length of the source txns
// and source tx heights must match or it will panic.
func newUTXOSet(sourceTxns []*wire.MsgTx, sourceTxHeights []uint64) blockdag.UTXOSet {
if len(sourceTxns) != len(sourceTxHeights) {
panic("each transaction must have its block height specified")
}
utxoSet := blockdag.NewFullUTXOSet()
for i, tx := range sourceTxns {
if isAccepted, err := utxoSet.AddTx(tx, sourceTxHeights[i]); err != nil {
panic(fmt.Sprintf("AddTx unexpectedly failed. Error: %s", err))
} else if !isAccepted {
panic(fmt.Sprintf("AddTx unexpectedly didn't add tx %s", tx.TxID()))
}
}
return utxoSet
}
func createTxIn(originTx *wire.MsgTx, outputIndex uint32) *wire.TxIn {
var prevOut *wire.Outpoint
if originTx != nil {
prevOut = wire.NewOutpoint(originTx.TxID(), 0)
} else {
prevOut = &wire.Outpoint{
TxID: daghash.TxID{},
Index: 0xFFFFFFFF,
}
}
return wire.NewTxIn(prevOut, nil)
}
func createTransaction(value uint64, originTx *wire.MsgTx, originTxoutputIndex uint32, sigScript []byte) (*wire.MsgTx, error) {
lookupKey := func(a util.Address) (*btcec.PrivateKey, bool, error) {
// Ordinarily this function would involve looking up the private
// key for the provided address, but since the only thing being
// signed in this example uses the address associated with the
// private key from above, simply return it with the compressed
// flag set since the address is using the associated compressed
// public key.
return privKey, true, nil
}
txIn := createTxIn(originTx, originTxoutputIndex)
pkScript, err := txscript.PayToAddrScript(addr)
if err != nil {
fmt.Println(err)
return nil, err
}
txOut := wire.NewTxOut(value, pkScript)
tx := wire.NewNativeMsgTx(wire.TxVersion, []*wire.TxIn{txIn}, []*wire.TxOut{txOut})
if sigScript == nil {
sigScript, err = txscript.SignTxOutput(&dagconfig.MainNetParams,
tx, 0, originTx.TxOut[0].PkScript, txscript.SigHashAll,
txscript.KeyClosure(lookupKey), nil, nil)
}
tx.TxIn[0].SignatureScript = sigScript
return tx, nil
}
// TestCalcPriority ensures the priority calculations work as intended.
func TestCalcPriority(t *testing.T) {
// commonSourceTx1 is a valid transaction used in the tests below as an
// input to transactions that are having their priority calculated.
//
// From block 7 in main blockchain.
// tx 0437cd7f8525ceed2324359c2d0ba26006d92d856a9c20fa0241106ee5a597c9
commonSourceTx1, err := createTransaction(5000000000, nil, 0, hexToBytes("04ffff001d0134"))
if err != nil {
t.Errorf("Error with creating source tx: %v", err)
}
// commonRedeemTx1 is a valid transaction used in the tests below as the
// transaction to calculate the priority for.
//
// It originally came from block 170 in main blockchain.
commonRedeemTx1, err := createTransaction(5000000000, commonSourceTx1, 0, nil)
if err != nil {
t.Errorf("Error with creating redeem tx: %v", err)
}
tests := []struct {
name string // test description
tx *wire.MsgTx // tx to calc priority for
utxoSet blockdag.UTXOSet // inputs to tx
nextHeight uint64 // height for priority calc
want float64 // expected priority
}{
{
name: "one height 7 input, prio tx height 169",
tx: commonRedeemTx1,
utxoSet: newUTXOSet([]*wire.MsgTx{commonSourceTx1},
[]uint64{7}),
nextHeight: 169,
want: 1.125e+10,
},
{
name: "one height 100 input, prio tx height 169",
tx: commonRedeemTx1,
utxoSet: newUTXOSet([]*wire.MsgTx{commonSourceTx1},
[]uint64{100}),
nextHeight: 169,
want: 4.791666666666667e+09,
},
{
name: "one height 7 input, prio tx height 100000",
tx: commonRedeemTx1,
utxoSet: newUTXOSet([]*wire.MsgTx{commonSourceTx1},
[]uint64{7}),
nextHeight: 100000,
want: 6.943958333333333e+12,
},
{
name: "one height 100 input, prio tx height 100000",
tx: commonRedeemTx1,
utxoSet: newUTXOSet([]*wire.MsgTx{commonSourceTx1},
[]uint64{100}),
nextHeight: 100000,
want: 6.9375e+12,
},
}
for i, test := range tests {
got := CalcPriority(test.tx, test.utxoSet, test.nextHeight)
if got != test.want {
t.Errorf("CalcPriority #%d (%q): unexpected priority "+
"got %v want %v", i, test.name, got, test.want)
continue
}
}
}
var privKeyBytes, _ = hex.DecodeString("22a47fa09a223f2aa079edf85a7c2" +
"d4f8720ee63e502ee2869afab7de234b80c")
var privKey, pubKey = btcec.PrivKeyFromBytes(btcec.S256(), privKeyBytes)
var pubKeyHash = util.Hash160(pubKey.SerializeCompressed())
var addr, _ = util.NewAddressPubKeyHash(pubKeyHash, util.Bech32PrefixDAGCoin)

View File

@ -46,9 +46,7 @@ func PrepareBlockForTest(dag *blockdag.BlockDAG, params *dagconfig.Params, paren
oldVirtual := blockdag.SetVirtualForTest(dag, newVirtual)
defer blockdag.SetVirtualForTest(dag, oldVirtual)
policy := Policy{
BlockMaxSize: 50000,
BlockPrioritySize: 750000,
TxMinFreeFee: util.Amount(0),
BlockMaxSize: 50000,
}
txSource := &fakeTxSource{
@ -70,7 +68,7 @@ func PrepareBlockForTest(dag *blockdag.BlockDAG, params *dagconfig.Params, paren
}
// In order of creating deterministic coinbase tx ids.
err = blockTemplateGenerator.UpdateExtraNonce(template.Block, dag.Height()+1, GenerateDeterministicExtraNonceForTest())
err = blockTemplateGenerator.UpdateExtraNonce(template.Block, dag.VirtualBlueScore(), GenerateDeterministicExtraNonceForTest())
if err != nil {
return nil, err
}
@ -119,7 +117,7 @@ func PrepareBlockForTest(dag *blockdag.BlockDAG, params *dagconfig.Params, paren
}
template.Block.Header.HashMerkleRoot = blockdag.BuildHashMerkleTreeStore(utilTxs).Root()
template.Block.Header.UTXOCommitment, err = blockTemplateGenerator.buildUTXOCommitment(template.Block.Transactions, dag.Height()+1)
template.Block.Header.UTXOCommitment, err = blockTemplateGenerator.buildUTXOCommitment(template.Block.Transactions, dag.VirtualBlueScore())
if err != nil {
return nil, err
}

View File

@ -37,10 +37,10 @@ func newBlockProgressLogger(progressMessage string, logger btclog.Logger) *block
}
}
// LogBlockHeight logs a new block height as an information message to show
// progress to the user. In order to prevent spam, it limits logging to one
// message every 10 seconds with duration and totals included.
func (b *blockProgressLogger) LogBlockHeight(block *util.Block) {
// LogBlockBlueScore logs a new block blue score as an information message
// to show progress to the user. In order to prevent spam, it limits logging to
// one message every 10 seconds with duration and totals included.
func (b *blockProgressLogger) LogBlockBlueScore(block *util.Block, blueScore uint64) {
b.Lock()
defer b.Unlock()
@ -66,9 +66,9 @@ func (b *blockProgressLogger) LogBlockHeight(block *util.Block) {
if b.receivedLogTx == 1 {
txStr = "transaction"
}
b.subsystemLogger.Infof("%s %d %s in the last %s (%d %s, height %d, %s)",
b.subsystemLogger.Infof("%s %d %s in the last %s (%d %s, blue score %d, %s)",
b.progressAction, b.receivedLogBlocks, blockStr, tDuration, b.receivedLogTx,
txStr, block.Height(), block.MsgBlock().Header.Timestamp)
txStr, blueScore, block.MsgBlock().Header.Timestamp)
b.receivedLogBlocks = 0
b.receivedLogTx = 0

View File

@ -32,6 +32,4 @@ type Config struct {
DisableCheckpoints bool
MaxPeers int
FeeEstimator *mempool.FeeEstimator
}

View File

@ -173,9 +173,6 @@ type SyncManager struct {
headerList *list.List
startHeader *list.Element
nextCheckpoint *dagconfig.Checkpoint
// An optional fee estimator.
feeEstimator *mempool.FeeEstimator
}
// resetHeaderState sets the headers-first mode state to values appropriate for
@ -207,14 +204,14 @@ func (sm *SyncManager) findNextHeaderCheckpoint(height uint64) *dagconfig.Checkp
// There is no next checkpoint if the height is already after the final
// checkpoint.
finalCheckpoint := &checkpoints[len(checkpoints)-1]
if height >= finalCheckpoint.Height {
if height >= finalCheckpoint.ChainHeight {
return nil
}
// Find the next checkpoint.
nextCheckpoint := finalCheckpoint
for i := len(checkpoints) - 2; i >= 0; i-- {
if height >= checkpoints[i].Height {
if height >= checkpoints[i].ChainHeight {
break
}
nextCheckpoint = &checkpoints[i]
@ -285,14 +282,14 @@ func (sm *SyncManager) startSync() {
// not support the headers-first approach so do normal block
// downloads when in regression test mode.
if sm.nextCheckpoint != nil &&
sm.dag.Height() < sm.nextCheckpoint.Height &&
sm.dag.ChainHeight() < sm.nextCheckpoint.ChainHeight &&
sm.chainParams != &dagconfig.RegressionNetParams { //TODO: (Ori) This is probably wrong. Done only for compilation
bestPeer.PushGetHeadersMsg(locator, sm.nextCheckpoint.Hash)
sm.headersFirstMode = true
log.Infof("Downloading headers for blocks %d to "+
"%d from peer %s", sm.dag.Height()+1,
sm.nextCheckpoint.Height, bestPeer.Addr()) //TODO: (Ori) This is probably wrong. Done only for compilation
"%d from peer %s", sm.dag.ChainHeight()+1,
sm.nextCheckpoint.ChainHeight, bestPeer.Addr()) //TODO: (Ori) This is probably wrong. Done only for compilation
} else {
bestPeer.PushGetBlocksMsg(locator, &daghash.ZeroHash)
}
@ -395,8 +392,8 @@ func (sm *SyncManager) handleDonePeerMsg(peer *peerpkg.Peer) {
if sm.syncPeer == peer {
sm.syncPeer = nil
if sm.headersFirstMode {
highestTipHash := sm.dag.HighestTipHash()
sm.resetHeaderState(highestTipHash, sm.dag.Height()) //TODO: (Ori) This is probably wrong. Done only for compilation
selectedTipHash := sm.dag.SelectedTipHash()
sm.resetHeaderState(selectedTipHash, sm.dag.ChainHeight()) //TODO: (Ori) This is probably wrong. Done only for compilation
}
sm.startSync()
}
@ -432,7 +429,7 @@ func (sm *SyncManager) handleTxMsg(tmsg *txMsg) {
// Process the transaction to include validation, insertion in the
// memory pool, orphan handling, etc.
acceptedTxs, err := sm.txMemPool.ProcessTransaction(tmsg.tx,
true, true, mempool.Tag(peer.ID()))
true, mempool.Tag(peer.ID()))
// Remove transaction from request maps. Either the mempool/chain
// already knows about it and as such we shouldn't have any more
@ -597,7 +594,11 @@ func (sm *SyncManager) handleBlockMsg(bmsg *blockMsg) {
} else {
// When the block is not an orphan, log information about it and
// update the chain state.
sm.progressLogger.LogBlockHeight(bmsg.block)
blockBlueScore, err := sm.dag.BlueScoreByBlockHash(blockHash)
if err != nil {
log.Errorf("Failed to get blue score for block %s: %s", blockHash, err)
}
sm.progressLogger.LogBlockBlueScore(bmsg.block, blockBlueScore)
// Clear the rejected transactions.
sm.rejectedTxns = make(map[daghash.TxID]struct{})
@ -634,7 +635,7 @@ func (sm *SyncManager) handleBlockMsg(bmsg *blockMsg) {
// there is a next checkpoint, get the next round of headers by asking
// for headers starting from the block after this one up to the next
// checkpoint.
prevHeight := sm.nextCheckpoint.Height
prevHeight := sm.nextCheckpoint.ChainHeight
parentHash := sm.nextCheckpoint.Hash
sm.nextCheckpoint = sm.findNextHeaderCheckpoint(prevHeight)
if sm.nextCheckpoint != nil {
@ -646,7 +647,7 @@ func (sm *SyncManager) handleBlockMsg(bmsg *blockMsg) {
return
}
log.Infof("Downloading headers for blocks %d to %d from "+
"peer %s", prevHeight+1, sm.nextCheckpoint.Height,
"peer %s", prevHeight+1, sm.nextCheckpoint.ChainHeight,
sm.syncPeer.Addr())
return
}
@ -805,7 +806,7 @@ func (sm *SyncManager) handleHeadersMsg(hmsg *headersMsg) {
}
// Verify the header at the next checkpoint height matches.
if node.height == sm.nextCheckpoint.Height {
if node.height == sm.nextCheckpoint.ChainHeight {
if node.hash.IsEqual(sm.nextCheckpoint.Hash) {
receivedCheckpoint = true
log.Infof("Verified downloaded block "+
@ -1244,20 +1245,6 @@ func (sm *SyncManager) handleBlockDAGNotification(notification *blockdag.Notific
sm.peerNotifier.TransactionConfirmed(msg.Tx)
sm.peerNotifier.AnnounceNewTransactions(msg.AcceptedTxs)
}
// Register block with the fee estimator, if it exists.
if sm.feeEstimator != nil {
err := sm.feeEstimator.RegisterBlock(block)
// If an error is somehow generated then the fee estimator
// has entered an invalid state. Since it doesn't know how
// to recover, create a new one.
if err != nil {
sm.feeEstimator = mempool.NewFeeEstimator(
mempool.DefaultEstimateFeeMaxRollback,
mempool.DefaultEstimateFeeMinRegisteredBlocks)
}
}
}
}
@ -1405,15 +1392,14 @@ func New(config *Config) (*SyncManager, error) {
msgChan: make(chan interface{}, config.MaxPeers*3),
headerList: list.New(),
quit: make(chan struct{}),
feeEstimator: config.FeeEstimator,
}
highestTipHash := sm.dag.HighestTipHash()
selectedTipHash := sm.dag.SelectedTipHash()
if !config.DisableCheckpoints {
// Initialize the next checkpoint based on the current height.
sm.nextCheckpoint = sm.findNextHeaderCheckpoint(sm.dag.Height()) //TODO: (Ori) This is probably wrong. Done only for compilation
// Initialize the next checkpoint based on the current chain height.
sm.nextCheckpoint = sm.findNextHeaderCheckpoint(sm.dag.ChainHeight()) //TODO: (Ori) This is probably wrong. Done only for compilation
if sm.nextCheckpoint != nil {
sm.resetHeaderState(highestTipHash, sm.dag.Height()) //TODO: (Ori) This is probably wrong. Done only for compilation)
sm.resetHeaderState(selectedTipHash, sm.dag.ChainHeight()) //TODO: (Ori) This is probably wrong. Done only for compilation)
}
} else {
log.Info("Checkpoints are disabled")

View File

@ -557,43 +557,6 @@ func (c *Client) GetRawMempoolVerbose() (map[string]btcjson.GetRawMempoolVerbose
return c.GetRawMempoolVerboseAsync().Receive()
}
// FutureEstimateFeeResult is a future promise to deliver the result of a
// EstimateFeeAsync RPC invocation (or an applicable error).
type FutureEstimateFeeResult chan *response
// Receive waits for the response promised by the future and returns the info
// provided by the server.
func (r FutureEstimateFeeResult) Receive() (float64, error) {
res, err := receiveFuture(r)
if err != nil {
return -1, err
}
// Unmarshal result as a getinfo result object.
var fee float64
err = json.Unmarshal(res, &fee)
if err != nil {
return -1, err
}
return fee, nil
}
// EstimateFeeAsync returns an instance of a type that can be used to get the result
// of the RPC at some future time by invoking the Receive function on the
// returned instance.
//
// See EstimateFee for the blocking version and more details.
func (c *Client) EstimateFeeAsync(numBlocks int64) FutureEstimateFeeResult {
cmd := btcjson.NewEstimateFeeCmd(numBlocks)
return c.sendCmd(cmd)
}
// EstimateFee provides an estimated fee in bitcoins per kilobyte.
func (c *Client) EstimateFee(numBlocks int64) (float64, error) {
return c.EstimateFeeAsync(numBlocks).Receive()
}
// FutureGetSubnetworkResult is a future promise to deliver the result of a
// GetSubnetworkAsync RPC invocation (or an applicable error).
type FutureGetSubnetworkResult chan *response

View File

@ -284,10 +284,6 @@ type Server struct {
AddrIndex *indexers.AddrIndex
CfIndex *indexers.CfIndex
// The fee estimator keeps track of how long transactions are left in
// the mempool before they are mined into blocks.
FeeEstimator *mempool.FeeEstimator
// cfCheckptCaches stores a cached slice of filter headers for cfcheckpt
// messages for each filter type.
cfCheckptCaches map[wire.FilterType][]cfHeaderKV
@ -2134,15 +2130,6 @@ func (s *Server) Start() {
// Stop gracefully shuts down the server by stopping and disconnecting all
// peers and the main listener.
func (s *Server) Stop() error {
// Save fee estimator state in the database.
s.db.Update(func(dbTx database.Tx) error {
metadata := dbTx.Metadata()
metadata.Put(mempool.EstimateFeeDatabaseKey, s.FeeEstimator.Save())
return nil
})
// Signal the remaining goroutines to quit.
close(s.quit)
return nil
@ -2405,49 +2392,17 @@ func NewServer(listenAddrs []string, db database.DB, dagParams *dagconfig.Params
return nil, err
}
// Search for a FeeEstimator state in the database. If none can be found
// or if it cannot be loaded, create a new one.
db.Update(func(dbTx database.Tx) error {
metadata := dbTx.Metadata()
feeEstimationData := metadata.Get(mempool.EstimateFeeDatabaseKey)
if feeEstimationData != nil {
// delete it from the database so that we don't try to restore the
// same thing again somehow.
metadata.Delete(mempool.EstimateFeeDatabaseKey)
// If there is an error, log it and make a new fee estimator.
var err error
s.FeeEstimator, err = mempool.RestoreFeeEstimator(feeEstimationData)
if err != nil {
peerLog.Errorf("Failed to restore fee estimator %s", err)
}
}
return nil
})
// If no feeEstimator has been found, or if the one that has been found
// is behind somehow, create a new one and start over.
if s.FeeEstimator == nil || s.FeeEstimator.LastKnownHeight() != s.DAG.Height() { //TODO: (Ori) This is probably wrong. Done only for compilation
s.FeeEstimator = mempool.NewFeeEstimator(
mempool.DefaultEstimateFeeMaxRollback,
mempool.DefaultEstimateFeeMinRegisteredBlocks)
}
txC := mempool.Config{
Policy: mempool.Policy{
DisableRelayPriority: config.MainConfig().NoRelayPriority,
AcceptNonStd: config.MainConfig().RelayNonStd,
FreeTxRelayLimit: config.MainConfig().FreeTxRelayLimit,
MaxOrphanTxs: config.MainConfig().MaxOrphanTxs,
MaxOrphanTxSize: config.DefaultMaxOrphanTxSize,
MaxSigOpsPerTx: blockdag.MaxSigOpsPerBlock / 5,
MinRelayTxFee: config.MainConfig().MinRelayTxFee,
MaxTxVersion: 1,
AcceptNonStd: config.MainConfig().RelayNonStd,
MaxOrphanTxs: config.MainConfig().MaxOrphanTxs,
MaxOrphanTxSize: config.DefaultMaxOrphanTxSize,
MaxSigOpsPerTx: blockdag.MaxSigOpsPerBlock / 5,
MinRelayTxFee: config.MainConfig().MinRelayTxFee,
MaxTxVersion: 1,
},
DAGParams: dagParams,
BestHeight: func() uint64 { return s.DAG.Height() }, //TODO: (Ori) This is probably wrong. Done only for compilation
DAGChainHeight: func() uint64 { return s.DAG.ChainHeight() },
MedianTimePast: func() time.Time { return s.DAG.CalcPastMedianTime() },
CalcSequenceLockNoLock: func(tx *util.Tx, utxoSet blockdag.UTXOSet) (*blockdag.SequenceLock, error) {
return s.DAG.CalcSequenceLockNoLock(tx, utxoSet, true)
@ -2455,7 +2410,6 @@ func NewServer(listenAddrs []string, db database.DB, dagParams *dagconfig.Params
IsDeploymentActive: s.DAG.IsDeploymentActive,
SigCache: s.SigCache,
AddrIndex: s.AddrIndex,
FeeEstimator: s.FeeEstimator,
DAG: s.DAG,
}
s.TxMemPool = mempool.New(&txC)
@ -2469,7 +2423,6 @@ func NewServer(listenAddrs []string, db database.DB, dagParams *dagconfig.Params
ChainParams: s.DAGParams,
DisableCheckpoints: cfg.DisableCheckpoints,
MaxPeers: cfg.MaxPeers,
FeeEstimator: s.FeeEstimator,
})
if err != nil {
return nil, err
@ -2805,7 +2758,7 @@ func (s checkpointSorter) Swap(i, j int) {
// Less returns whether the checkpoint with index i should sort before the
// checkpoint with index j. It is part of the sort.Interface implementation.
func (s checkpointSorter) Less(i, j int) bool {
return s[i].Height < s[j].Height
return s[i].ChainHeight < s[j].ChainHeight
}
// mergeCheckpoints returns two slices of checkpoints merged into one slice
@ -2818,7 +2771,7 @@ func mergeCheckpoints(defaultCheckpoints, additional []dagconfig.Checkpoint) []d
// leaving the most recently-specified checkpoint.
extra := make(map[uint64]dagconfig.Checkpoint)
for _, checkpoint := range additional {
extra[checkpoint.Height] = checkpoint
extra[checkpoint.ChainHeight] = checkpoint
}
// Add all default checkpoints that do not have an override in the
@ -2826,7 +2779,7 @@ func mergeCheckpoints(defaultCheckpoints, additional []dagconfig.Checkpoint) []d
numDefault := len(defaultCheckpoints)
checkpoints := make([]dagconfig.Checkpoint, 0, numDefault+len(extra))
for _, checkpoint := range defaultCheckpoints {
if _, exists := extra[checkpoint.Height]; !exists {
if _, exists := extra[checkpoint.ChainHeight]; !exists {
checkpoints = append(checkpoints, checkpoint)
}
}

View File

@ -142,7 +142,6 @@ var rpcHandlersBeforeInit = map[string]commandHandler{
"debugLevel": handleDebugLevel,
"decodeRawTransaction": handleDecodeRawTransaction,
"decodeScript": handleDecodeScript,
"estimateFee": handleEstimateFee,
"generate": handleGenerate,
"getAllManualNodesInfo": handleGetAllManualNodesInfo,
"getBestBlock": handleGetBestBlock,
@ -267,7 +266,6 @@ var rpcLimited = map[string]struct{}{
"createRawTransaction": {},
"decodeRawTransaction": {},
"decodeScript": {},
"estimateFee": {},
"getBestBlock": {},
"getBestBlockHash": {},
"getBlock": {},
@ -871,28 +869,6 @@ func handleDecodeScript(s *Server, cmd interface{}, closeChan <-chan struct{}) (
return reply, nil
}
// handleEstimateFee handles estimateFee commands.
func handleEstimateFee(s *Server, cmd interface{}, closeChan <-chan struct{}) (interface{}, error) {
c := cmd.(*btcjson.EstimateFeeCmd)
if s.cfg.FeeEstimator == nil {
return nil, errors.New("Fee estimation disabled")
}
if c.NumBlocks <= 0 {
return -1.0, errors.New("Parameter NumBlocks must be positive")
}
feeRate, err := s.cfg.FeeEstimator.EstimateFee(uint32(c.NumBlocks))
if err != nil {
return -1.0, err
}
// Convert to satoshis per kb.
return float64(feeRate), nil
}
// handleGenerate handles generate commands.
func handleGenerate(s *Server, cmd interface{}, closeChan <-chan struct{}) (interface{}, error) {
// Respond with an error if there are no addresses to pay the
@ -1070,15 +1046,15 @@ func handleGetBestBlock(s *Server, cmd interface{}, closeChan <-chan struct{}) (
// hash, or both but require the block SHA. This gets both for
// the best block.
result := &btcjson.GetBestBlockResult{
Hash: s.cfg.DAG.HighestTipHash().String(),
Height: s.cfg.DAG.Height(), //TODO: (Ori) This is probably wrong. Done only for compilation
Hash: s.cfg.DAG.SelectedTipHash().String(),
Height: s.cfg.DAG.ChainHeight(), //TODO: (Ori) This is probably wrong. Done only for compilation
}
return result, nil
}
// handleGetBestBlockHash implements the getBestBlockHash command.
func handleGetBestBlockHash(s *Server, cmd interface{}, closeChan <-chan struct{}) (interface{}, error) {
return s.cfg.DAG.HighestTipHash().String(), nil
return s.cfg.DAG.SelectedTipHash().String(), nil
}
// getDifficultyRatio returns the proof-of-work difficulty as a multiple of the
@ -1174,17 +1150,16 @@ func handleGetBlock(s *Server, cmd interface{}, closeChan <-chan struct{}) (inte
return nil, internalRPCError(err.Error(), context)
}
// Get the block height from chain.
blockHeight, err := s.cfg.DAG.BlockHeightByHash(hash)
// Get the block chain height.
blockChainHeight, err := s.cfg.DAG.BlockChainHeightByHash(hash)
if err != nil {
context := "Failed to obtain block height"
return nil, internalRPCError(err.Error(), context)
}
blk.SetHeight(blockHeight)
// Get the hashes for the next blocks unless there are none.
var nextHashStrings []string
if blockHeight < s.cfg.DAG.Height() { //TODO: (Ori) This is probably wrong. Done only for compilation
if blockChainHeight < s.cfg.DAG.ChainHeight() { //TODO: (Ori) This is probably wrong. Done only for compilation
childHashes, err := s.cfg.DAG.ChildHashesByHash(hash)
if err != nil {
context := "No next block"
@ -1211,7 +1186,7 @@ func handleGetBlock(s *Server, cmd interface{}, closeChan <-chan struct{}) (inte
Nonce: blockHeader.Nonce,
Time: blockHeader.Timestamp.Unix(),
Confirmations: blockConfirmations,
Height: blockHeight,
Height: blockChainHeight,
Size: int32(len(blkBytes)),
Bits: strconv.FormatInt(int64(blockHeader.Bits), 16),
Difficulty: getDifficultyRatio(blockHeader.Bits, params),
@ -1387,8 +1362,8 @@ func handleGetBlockHeader(s *Server, cmd interface{}, closeChan <-chan struct{})
// The verbose flag is set, so generate the JSON object and return it.
// Get the block height from chain.
blockHeight, err := s.cfg.DAG.BlockHeightByHash(hash)
// Get the block chain height from chain.
blockChainHeight, err := s.cfg.DAG.BlockChainHeightByHash(hash)
if err != nil {
context := "Failed to obtain block height"
return nil, internalRPCError(err.Error(), context)
@ -1396,7 +1371,7 @@ func handleGetBlockHeader(s *Server, cmd interface{}, closeChan <-chan struct{})
// Get the hashes for the next blocks unless there are none.
var nextHashStrings []string
if blockHeight < s.cfg.DAG.Height() { //TODO: (Ori) This is probably wrong. Done only for compilation
if blockChainHeight < s.cfg.DAG.ChainHeight() { //TODO: (Ori) This is probably wrong. Done only for compilation
childHashes, err := s.cfg.DAG.ChildHashesByHash(hash)
if err != nil {
context := "No next block"
@ -1415,7 +1390,7 @@ func handleGetBlockHeader(s *Server, cmd interface{}, closeChan <-chan struct{})
blockHeaderReply := btcjson.GetBlockHeaderVerboseResult{
Hash: c.Hash,
Confirmations: blockConfirmations,
Height: blockHeight,
Height: blockChainHeight,
Version: blockHeader.Version,
VersionHex: fmt.Sprintf("%08x", blockHeader.Version),
HashMerkleRoot: blockHeader.HashMerkleRoot.String(),
@ -2006,8 +1981,8 @@ func handleGetBlockTemplateRequest(s *Server, request *btcjson.TemplateRequest,
}
// No point in generating or accepting work before the chain is synced.
currentHeight := s.cfg.DAG.Height()
if currentHeight != 0 && !s.cfg.SyncMgr.IsCurrent() {
currentChainHeight := s.cfg.DAG.ChainHeight()
if currentChainHeight != 0 && !s.cfg.SyncMgr.IsCurrent() {
return nil, &btcjson.RPCError{
Code: btcjson.ErrRPCClientInInitialDownload,
Message: "Bitcoin is downloading blocks...",
@ -2073,8 +2048,6 @@ func chainErrToGBTErrString(err error) string {
return "bad-txnmrklroot"
case blockdag.ErrBadCheckpoint:
return "bad-checkpoint"
case blockdag.ErrForkTooOld:
return "fork-too-old"
case blockdag.ErrCheckpointTimeTooOld:
return "checkpoint-time-too-old"
case blockdag.ErrNoTransactions:
@ -2113,9 +2086,9 @@ func chainErrToGBTErrString(err error) string {
return "bad-cb-length"
case blockdag.ErrBadCoinbaseValue:
return "bad-cb-value"
case blockdag.ErrMissingCoinbaseHeight:
case blockdag.ErrMissingCoinbaseBlueScore:
return "bad-cb-height"
case blockdag.ErrBadCoinbaseHeight:
case blockdag.ErrBadCoinbaseBlueScore:
return "bad-cb-height"
case blockdag.ErrScriptMalformed:
return "bad-script-malformed"
@ -2447,8 +2420,8 @@ func handleGetMiningInfo(s *Server, cmd interface{}, closeChan <-chan struct{})
}
}
highestTipHash := s.cfg.DAG.HighestTipHash()
selectedBlock, err := s.cfg.DAG.BlockByHash(highestTipHash)
selectedTipHash := s.cfg.DAG.SelectedTipHash()
selectedBlock, err := s.cfg.DAG.BlockByHash(selectedTipHash)
if err != nil {
return nil, &btcjson.RPCError{
Code: btcjson.ErrRPCInternal.Code,
@ -2457,7 +2430,7 @@ func handleGetMiningInfo(s *Server, cmd interface{}, closeChan <-chan struct{})
}
result := btcjson.GetMiningInfoResult{
Blocks: int64(s.cfg.DAG.Height()), //TODO: (Ori) This is wrong. Done only for compilation
Blocks: int64(s.cfg.DAG.BlockCount()),
CurrentBlockSize: uint64(selectedBlock.MsgBlock().SerializeSize()),
CurrentBlockTx: uint64(len(selectedBlock.MsgBlock().Transactions)),
Difficulty: getDifficultyRatio(s.cfg.DAG.CurrentBits(), s.cfg.DAGParams),
@ -2740,7 +2713,7 @@ func handleGetTxOut(s *Server, cmd interface{}, closeChan <-chan struct{}) (inte
return nil, internalRPCError(errStr, "")
}
bestBlockHash = s.cfg.DAG.HighestTipHash().String()
bestBlockHash = s.cfg.DAG.SelectedTipHash().String()
value = txOut.Value
pkScript = txOut.PkScript
isCoinbase = mtx.IsCoinBase()
@ -2770,7 +2743,7 @@ func handleGetTxOut(s *Server, cmd interface{}, closeChan <-chan struct{}) (inte
confirmations = &txConfirmations
}
bestBlockHash = s.cfg.DAG.HighestTipHash().String()
bestBlockHash = s.cfg.DAG.SelectedTipHash().String()
value = entry.Amount()
pkScript = entry.PkScript()
isCoinbase = entry.IsBlockReward()
@ -3391,7 +3364,7 @@ func handleSendRawTransaction(s *Server, cmd interface{}, closeChan <-chan struc
// Use 0 for the tag to represent local node.
tx := util.NewTx(&msgTx)
acceptedTxs, err := s.cfg.TxMemPool.ProcessTransaction(tx, false, false, 0)
acceptedTxs, err := s.cfg.TxMemPool.ProcessTransaction(tx, false, 0)
if err != nil {
// When the error is a rule error, it means the transaction was
// simply rejected as opposed to something actually going wrong,
@ -4281,10 +4254,6 @@ type rpcserverConfig struct {
TxIndex *indexers.TxIndex
AddrIndex *indexers.AddrIndex
CfIndex *indexers.CfIndex
// The fee estimator keeps track of how long transactions are left in
// the mempool before they are mined into blocks.
FeeEstimator *mempool.FeeEstimator
}
// setupRPCListeners returns a slice of listeners that are configured for use
@ -4355,21 +4324,20 @@ func NewRPCServer(
return nil, errors.New("RPCS: No valid listen address")
}
cfg := &rpcserverConfig{
Listeners: rpcListeners,
StartupTime: startupTime,
ConnMgr: &rpcConnManager{p2pServer},
SyncMgr: &rpcSyncMgr{p2pServer, p2pServer.SyncManager},
TimeSource: p2pServer.TimeSource,
DAGParams: p2pServer.DAGParams,
DB: db,
TxMemPool: p2pServer.TxMemPool,
Generator: blockTemplateGenerator,
CPUMiner: cpuminer,
TxIndex: p2pServer.TxIndex,
AddrIndex: p2pServer.AddrIndex,
CfIndex: p2pServer.CfIndex,
FeeEstimator: p2pServer.FeeEstimator,
DAG: p2pServer.DAG,
Listeners: rpcListeners,
StartupTime: startupTime,
ConnMgr: &rpcConnManager{p2pServer},
SyncMgr: &rpcSyncMgr{p2pServer, p2pServer.SyncManager},
TimeSource: p2pServer.TimeSource,
DAGParams: p2pServer.DAGParams,
DB: db,
TxMemPool: p2pServer.TxMemPool,
Generator: blockTemplateGenerator,
CPUMiner: cpuminer,
TxIndex: p2pServer.TxIndex,
AddrIndex: p2pServer.AddrIndex,
CfIndex: p2pServer.CfIndex,
DAG: p2pServer.DAG,
}
rpc := Server{
cfg: *cfg,

View File

@ -113,15 +113,6 @@ var helpDescsEnUS = map[string]string{
"decodeScript--synopsis": "Returns a JSON object with information about the provided hex-encoded script.",
"decodeScript-hexScript": "Hex-encoded script",
// EstimateFeeCmd help.
"estimateFee--synopsis": "Estimate the fee per kilobyte in satoshis " +
"required for a transaction to be mined before a certain number of " +
"blocks have been generated.",
"estimateFee-numBlocks": "The maximum number of blocks which can be " +
"generated before the transaction is mined.",
"estimateFee--result0": "Estimated fee per kilobyte in satoshis for a block to " +
"be mined in the next NumBlocks blocks.",
// GenerateCmd help
"generate--synopsis": "Generates a set number of blocks (simnet or regtest only) and returns a JSON\n" +
" array of their hashes.",
@ -677,7 +668,6 @@ var rpcResultTypes = map[string][]interface{}{
"debugLevel": {(*string)(nil), (*string)(nil)},
"decodeRawTransaction": {(*btcjson.TxRawDecodeResult)(nil)},
"decodeScript": {(*btcjson.DecodeScriptResult)(nil)},
"estimateFee": {(*float64)(nil)},
"generate": {(*[]string)(nil)},
"getAllManualNodesInfo": {(*[]string)(nil), (*[]btcjson.GetManualNodeInfoResult)(nil)},
"getBestBlock": {(*btcjson.GetBestBlockResult)(nil)},

View File

@ -757,20 +757,6 @@ func txHexString(tx *wire.MsgTx) string {
return hex.EncodeToString(buf.Bytes())
}
// blockDetails creates a BlockDetails struct to include in btcws notifications
// from a block and a transaction's block index.
func blockDetails(block *util.Block, txIndex int) *btcjson.BlockDetails {
if block == nil {
return nil
}
return &btcjson.BlockDetails{
Height: block.Height(),
Hash: block.Hash().String(),
Index: txIndex,
Time: block.MsgBlock().Header.Timestamp.Unix(),
}
}
// notifyRelevantTxAccepted examines the inputs and outputs of the passed
// transaction, notifying websocket clients of outputs spending to a watched
// address and inputs spending a watched outpoint. Any outputs paying to a
@ -1519,21 +1505,6 @@ func handleStopNotifyNewTransactions(wsc *wsClient, icmd interface{}) (interface
return nil, nil
}
// deserializeOutpoints deserializes each serialized outpoint.
func deserializeOutpoints(serializedOuts []btcjson.Outpoint) ([]*wire.Outpoint, error) {
outpoints := make([]*wire.Outpoint, 0, len(serializedOuts))
for i := range serializedOuts {
txID, err := daghash.NewTxIDFromStr(serializedOuts[i].TxID)
if err != nil {
return nil, rpcDecodeHexError(serializedOuts[i].TxID)
}
index := serializedOuts[i].Index
outpoints = append(outpoints, wire.NewOutpoint(txID, index))
}
return outpoints, nil
}
// rescanBlockFilter rescans a block for any relevant transactions for the
// passed lookup keys. Any discovered transactions are returned hex encoded as
// a string slice.

View File

@ -101,10 +101,8 @@ func NewServer(listenAddrs []string, db database.DB, dagParams *dagconfig.Params
// NOTE: The CPU miner relies on the mempool, so the mempool has to be
// created before calling the function to create the CPU miner.
policy := mining.Policy{
BlockMinSize: cfg.BlockMinSize,
BlockMaxSize: cfg.BlockMaxSize,
BlockPrioritySize: cfg.BlockPrioritySize,
TxMinFreeFee: cfg.MinRelayTxFee,
BlockMinSize: cfg.BlockMinSize,
BlockMaxSize: cfg.BlockMaxSize,
}
blockTemplateGenerator := mining.NewBlkTmplGenerator(&policy,
s.p2pServer.DAGParams, s.p2pServer.TxMemPool, s.p2pServer.DAG, s.p2pServer.TimeSource, s.p2pServer.SigCache)

View File

@ -45,7 +45,6 @@ type Block struct {
msgBlock *wire.MsgBlock // Underlying MsgBlock
serializedBlock []byte // Serialized bytes for the block
blockHash *daghash.Hash // Cached block hash
blockHeight uint64 // Height in the DAG
chainHeight uint64 // Selected-chain height
transactions []*Tx // Transactions
txnsGenerated bool // ALL wrapped transactions generated
@ -194,21 +193,10 @@ func (b *Block) TxLoc() ([]wire.TxLoc, error) {
return txLocs, err
}
// Height returns the saved height of the block in the block chain. This value
// will be BlockHeightUnknown if it hasn't already explicitly been set.
func (b *Block) Height() uint64 {
return b.blockHeight
}
// SetHeight sets the height of the block in the block chain.
func (b *Block) SetHeight(height uint64) {
b.blockHeight = height
}
// ChainHeight returns the saved chan height of the block . This value
// will be BlockHeightUnknown if it hasn't already explicitly been set.
func (b *Block) ChainHeight() uint64 {
return b.blockHeight
return b.chainHeight
}
// SetChainHeight sets the chain height of the block.
@ -246,7 +234,6 @@ func (b *Block) Timestamp() time.Time {
func NewBlock(msgBlock *wire.MsgBlock) *Block {
return &Block{
msgBlock: msgBlock,
blockHeight: BlockHeightUnknown,
chainHeight: BlockHeightUnknown,
}
}
@ -274,8 +261,7 @@ func NewBlockFromReader(r io.Reader) (*Block, error) {
}
b := Block{
msgBlock: &msgBlock,
blockHeight: BlockHeightUnknown,
msgBlock: &msgBlock,
}
return &b, nil
}
@ -286,6 +272,5 @@ func NewBlockFromBlockAndBytes(msgBlock *wire.MsgBlock, serializedBlock []byte)
return &Block{
msgBlock: msgBlock,
serializedBlock: serializedBlock,
blockHeight: BlockHeightUnknown,
}
}

View File

@ -29,12 +29,12 @@ func TestBlock(t *testing.T) {
spew.Sdump(msgBlock), spew.Sdump(&Block100000))
}
// Ensure block height set and get work properly.
wantHeight := uint64(100000)
b.SetHeight(wantHeight)
if gotHeight := b.Height(); gotHeight != wantHeight {
t.Errorf("Height: mismatched height - got %v, want %v",
gotHeight, wantHeight)
// Ensure block chain height set and get work properly.
wantChainHeight := uint64(100000)
b.SetChainHeight(wantChainHeight)
if gotChainHeight := b.ChainHeight(); gotChainHeight != wantChainHeight {
t.Errorf("ChainHeight: mismatched chain height - got %v, want %v",
gotChainHeight, wantChainHeight)
}
// Hash for block 100,000.

18
util/btcutil.go Normal file
View File

@ -0,0 +1,18 @@
package util
import "sort"
// SearchSlice uses binary search to find and return the smallest index i
// in [0, n) at which f(i) is true, assuming that on the range [0, n),
// f(i) == true implies f(i+1) == true. That is, SearchSlice requires that
// f is false for some (possibly empty) prefix of the input range [0, n)
// and then true for the (possibly empty) remainder; SearchSlice returns
// the first true index.
// SearchSlice calls f(i) only for i in the range [0, n).
func SearchSlice(sliceLength int, searchFunc func(int) bool) (foundIndex int, ok bool) {
result := sort.Search(sliceLength, searchFunc)
if result == sliceLength {
return -1, false
}
return result, true
}