[DEV-204] Unexport VirtualBlock (#93)

* [DEV-201] In handleGetBlockDAGInfo calculate difficulty by the tip with the lowest bits

* [DEV-202] Move VirtualBlock.GetUTXOEntry to BlockDAG

* [DEV-203] Move VirtualBlock.SelectedTip() to BlockDAG

* [DEV-203] Move VirtualBlock.SelectedTip() to BlockDAG

* [DEV-204] Unexport VirtualBlock() and add CalcMedianTime method for DAG

* [DEV-204] add explanation about difficulty in CurrentBits() comment

* [DEV-204] unexport VirtualBlock type
This commit is contained in:
Ori Newman 2018-10-16 15:21:18 +03:00 committed by stasatdaglabs
parent c3316715ae
commit 9ea9098fbc
18 changed files with 140 additions and 148 deletions

View File

@ -130,7 +130,7 @@ func (dag *BlockDAG) findPreviousCheckpoint() (*blockNode, error) {
// 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.virtual.SelectedTip().height < dag.nextCheckpoint.Height {
if dag.SelectedTip().height < dag.nextCheckpoint.Height {
return dag.checkpointNode, nil
}
@ -218,7 +218,7 @@ func (dag *BlockDAG) IsCheckpointCandidate(block *util.Block) (bool, error) {
// A checkpoint must be at least CheckpointConfirmations blocks
// before the end of the main chain.
dagHeight := dag.virtual.SelectedTip().height
dagHeight := dag.SelectedTip().height
if node.height > (dagHeight - CheckpointConfirmations) {
return false, nil
}

View File

@ -6,6 +6,7 @@ package blockdag
import (
"fmt"
"math"
"sync"
"time"
@ -87,7 +88,7 @@ type BlockDAG struct {
index *blockIndex
// virtual tracks the current tips.
virtual *VirtualBlock
virtual *virtualBlock
// These fields are related to handling of orphan blocks. They are
// protected by a combination of the chain lock and the orphan lock.
@ -320,7 +321,7 @@ func (dag *BlockDAG) CalcSequenceLock(tx *util.Tx, utxoSet UTXOSet, mempool bool
dag.dagLock.Lock()
defer dag.dagLock.Unlock()
return dag.calcSequenceLock(dag.virtual.SelectedTip(), utxoSet, tx, mempool)
return dag.calcSequenceLock(dag.SelectedTip(), utxoSet, tx, mempool)
}
// calcSequenceLock computes the relative lock-times for the passed
@ -539,7 +540,7 @@ func (dag *BlockDAG) connectBlock(node *blockNode, block *util.Block) error {
// optional indexes with the block being connected so they can
// update themselves accordingly.
if dag.indexManager != nil {
err := dag.indexManager.ConnectBlock(dbTx, block, dag.virtual)
err := dag.indexManager.ConnectBlock(dbTx, block, dag)
if err != nil {
return err
}
@ -703,7 +704,7 @@ func (pns provisionalNodeSet) newProvisionalNode(node *blockNode, withRelatives
}
// verifyAndBuildUTXO verifies all transactions in the given block (in provisionalNode format) and builds its UTXO
func (p *provisionalNode) verifyAndBuildUTXO(virtual *VirtualBlock, db database.DB) (UTXOSet, error) {
func (p *provisionalNode) verifyAndBuildUTXO(virtual *virtualBlock, db database.DB) (UTXOSet, error) {
pastUTXO, err := p.pastUTXO(virtual, db)
if err != nil {
return nil, err
@ -730,7 +731,7 @@ func (p *provisionalNode) verifyAndBuildUTXO(virtual *VirtualBlock, db database.
}
// pastUTXO returns the UTXO of a given block's (in provisionalNode format) past
func (p *provisionalNode) pastUTXO(virtual *VirtualBlock, db database.DB) (UTXOSet, error) {
func (p *provisionalNode) pastUTXO(virtual *virtualBlock, db database.DB) (UTXOSet, error) {
pastUTXO, err := p.selectedParent.restoreUTXO(virtual)
if err != nil {
return nil, err
@ -780,7 +781,7 @@ func (p *provisionalNode) pastUTXO(virtual *VirtualBlock, db database.DB) (UTXOS
}
// restoreUTXO restores the UTXO of a given block (in provisionalNode format) from its diff
func (p *provisionalNode) restoreUTXO(virtual *VirtualBlock) (UTXOSet, error) {
func (p *provisionalNode) restoreUTXO(virtual *virtualBlock) (UTXOSet, error) {
stack := []*provisionalNode{p}
current := p
@ -804,7 +805,7 @@ func (p *provisionalNode) restoreUTXO(virtual *VirtualBlock) (UTXOSet, error) {
// updateParents adds this block (in provisionalNode format) to the children sets of its parents
// and updates the diff of any parent whose DiffChild is this block
func (p *provisionalNode) updateParents(virtual *VirtualBlock, newBlockUTXO UTXOSet) error {
func (p *provisionalNode) updateParents(virtual *virtualBlock, newBlockUTXO UTXOSet) error {
virtualDiffFromNewBlock, err := virtual.utxoSet.diffFrom(newBlockUTXO)
if err != nil {
return err
@ -832,7 +833,7 @@ func (p *provisionalNode) updateParents(virtual *VirtualBlock, newBlockUTXO UTXO
}
// updateTipsUTXO builds and applies new diff UTXOs for all the DAG's tips (in provisionalNode format)
func updateTipsUTXO(tipProvisionals []*provisionalNode, virtual *VirtualBlock, virtualUTXO UTXOSet) error {
func updateTipsUTXO(tipProvisionals []*provisionalNode, virtual *virtualBlock, virtualUTXO UTXOSet) error {
for _, tipProvisional := range tipProvisionals {
tipUTXO, err := tipProvisional.restoreUTXO(virtual)
if err != nil {
@ -884,7 +885,7 @@ func (dag *BlockDAG) isCurrent() bool {
// Not current if the latest main (best) chain height is before the
// latest known good checkpoint (when checkpoints are enabled).
checkpoint := dag.LatestCheckpoint()
if checkpoint != nil && dag.virtual.SelectedTip().height < checkpoint.Height {
if checkpoint != nil && dag.SelectedTip().height < checkpoint.Height {
return false
}
@ -894,7 +895,7 @@ func (dag *BlockDAG) isCurrent() bool {
// The chain appears to be current if none of the checks reported
// otherwise.
minus24Hours := dag.timeSource.AdjustedTime().Add(-24 * time.Hour).Unix()
return dag.virtual.SelectedTip().timestamp >= minus24Hours
return dag.SelectedTip().timestamp >= minus24Hours
}
// IsCurrent returns whether or not the chain believes it is current. Several
@ -911,18 +912,31 @@ func (dag *BlockDAG) IsCurrent() bool {
return dag.isCurrent()
}
// UTXOSet returns the DAG's UTXO set
func (dag *BlockDAG) UTXOSet() *fullUTXOSet {
return dag.VirtualBlock().utxoSet
}
// VirtualBlock returns the DAG's virtual block in the current point in time.
// The returned instance must be treated as immutable since it is shared by all
// callers.
// SelectedTip returns the current selected tip for the DAG.
// It will return nil if there is no tip.
//
// This function is safe for concurrent access.
func (dag *BlockDAG) VirtualBlock() *VirtualBlock {
return dag.virtual
func (dag *BlockDAG) SelectedTip() *blockNode {
return dag.virtual.selectedParent
}
// UTXOSet returns the DAG's UTXO set
func (dag *BlockDAG) UTXOSet() *fullUTXOSet {
return dag.virtual.utxoSet
}
// CalcPastMedianTime returns the past median time of the DAG.
func (dag *BlockDAG) CalcPastMedianTime() time.Time {
return dag.virtual.tips().bluest().CalcPastMedianTime()
}
// GetUTXOEntry returns the requested unspent transaction output. The returned
// instance must be treated as immutable since it is shared by all callers.
//
// This function is safe for concurrent access. However, the returned entry (if
// any) is NOT.
func (dag *BlockDAG) GetUTXOEntry(outPoint wire.OutPoint) (*UTXOEntry, bool) {
return dag.virtual.utxoSet.get(outPoint)
}
// Height returns the height of the highest tip in the DAG
@ -941,6 +955,18 @@ 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()
minBits := uint32(math.MaxUint32)
for _, tip := range tips {
if minBits > tip.Header().Bits {
minBits = tip.Header().Bits
}
}
return minBits
}
// HeaderByHash returns the block header identified by the given hash or an
// error if it doesn't exist.
func (dag *BlockDAG) HeaderByHash(hash *daghash.Hash) (wire.BlockHeader, error) {
@ -1193,7 +1219,7 @@ func (dag *BlockDAG) locateInventory(locator BlockLocator, hashStop *daghash.Has
}
// Calculate how many entries are needed.
total := uint32((dag.virtual.SelectedTip().height - startNode.height) + 1)
total := uint32((dag.SelectedTip().height - startNode.height) + 1)
if stopNode != nil && stopNode.height >= startNode.height {
total = uint32((stopNode.height - startNode.height) + 1)
}
@ -1316,12 +1342,12 @@ type IndexManager interface {
Init(*BlockDAG, <-chan struct{}) error
// ConnectBlock is invoked when a new block has been connected to the
// main chain.
ConnectBlock(database.Tx, *util.Block, *VirtualBlock) error
// DAG.
ConnectBlock(database.Tx, *util.Block, *BlockDAG) error
// DisconnectBlock is invoked when a block has been disconnected from
// the main chain.
DisconnectBlock(database.Tx, *util.Block, *VirtualBlock) error
// the DAG.
DisconnectBlock(database.Tx, *util.Block, *BlockDAG) error
}
// Config is a descriptor which specifies the blockchain instance configuration.
@ -1459,7 +1485,7 @@ func New(config *Config) (*BlockDAG, error) {
return nil, err
}
selectedTip := dag.virtual.SelectedTip()
selectedTip := dag.SelectedTip()
log.Infof("DAG state (height %d, hash %v, work %v)",
selectedTip.height, selectedTip.hash, selectedTip.workSum)

View File

@ -173,7 +173,7 @@ func TestCalcSequenceLock(t *testing.T) {
// Generate enough synthetic blocks for the rest of the test
dag := newTestDAG(netParams)
node := dag.virtual.SelectedTip()
node := dag.SelectedTip()
blockTime := node.Header().Timestamp
numBlocksToGenerate := uint32(5)
for i := uint32(0); i < numBlocksToGenerate; i++ {

View File

@ -290,7 +290,7 @@ func (dag *BlockDAG) calcNextRequiredDifficulty(lastNode *blockNode, newBlockTim
// This function is safe for concurrent access.
func (dag *BlockDAG) CalcNextRequiredDifficulty(timestamp time.Time) (uint32, error) {
dag.dagLock.Lock()
difficulty, err := dag.calcNextRequiredDifficulty(dag.virtual.SelectedTip(), timestamp)
difficulty, err := dag.calcNextRequiredDifficulty(dag.SelectedTip(), timestamp)
dag.dagLock.Unlock()
return difficulty, err
}

View File

@ -662,7 +662,7 @@ func (idx *AddrIndex) indexPkScript(data writeIndexData, pkScript []byte, txIdx
// indexBlock extract all of the standard addresses from all of the transactions
// in the passed block and maps each of them to the associated transaction using
// the passed map.
func (idx *AddrIndex) indexBlock(data writeIndexData, block *util.Block, virtual *blockdag.VirtualBlock) {
func (idx *AddrIndex) indexBlock(data writeIndexData, block *util.Block, dag *blockdag.BlockDAG) {
for txIdx, tx := range block.Transactions() {
// Coinbases do not reference any inputs. Since the block is
// required to have already gone through full validation, it has
@ -673,7 +673,7 @@ func (idx *AddrIndex) indexBlock(data writeIndexData, block *util.Block, virtual
// The UTXO should always have the input since
// the index contract requires it, however, be
// safe and simply ignore any missing entries.
entry, ok := virtual.GetUTXOEntry(txIn.PreviousOutPoint)
entry, ok := dag.GetUTXOEntry(txIn.PreviousOutPoint)
if !ok {
continue
}
@ -693,7 +693,7 @@ func (idx *AddrIndex) indexBlock(data writeIndexData, block *util.Block, virtual
// the transactions in the block involve.
//
// This is part of the Indexer interface.
func (idx *AddrIndex) ConnectBlock(dbTx database.Tx, block *util.Block, virtual *blockdag.VirtualBlock) error {
func (idx *AddrIndex) ConnectBlock(dbTx database.Tx, block *util.Block, dag *blockdag.BlockDAG) error {
// The offset and length of the transactions within the serialized
// block.
txLocs, err := block.TxLoc()
@ -709,7 +709,7 @@ func (idx *AddrIndex) ConnectBlock(dbTx database.Tx, block *util.Block, virtual
// Build all of the address to transaction mappings in a local map.
addrsToTxns := make(writeIndexData)
idx.indexBlock(addrsToTxns, block, virtual)
idx.indexBlock(addrsToTxns, block, dag)
// Add all of the index entries for each address.
addrIdxBucket := dbTx.Metadata().Bucket(addrIndexKey)
@ -731,10 +731,10 @@ func (idx *AddrIndex) ConnectBlock(dbTx database.Tx, block *util.Block, virtual
// each transaction in the block involve.
//
// This is part of the Indexer interface.
func (idx *AddrIndex) DisconnectBlock(dbTx database.Tx, block *util.Block, virtual *blockdag.VirtualBlock) error {
func (idx *AddrIndex) DisconnectBlock(dbTx database.Tx, block *util.Block, dag *blockdag.BlockDAG) error {
// Build all of the address to transaction mappings in a local map.
addrsToTxns := make(writeIndexData)
idx.indexBlock(addrsToTxns, block, virtual)
idx.indexBlock(addrsToTxns, block, dag)
// Remove all of the index entries for each address.
bucket := dbTx.Metadata().Bucket(addrIndexKey)

View File

@ -11,10 +11,10 @@ import (
"github.com/daglabs/btcd/dagconfig"
"github.com/daglabs/btcd/dagconfig/daghash"
"github.com/daglabs/btcd/database"
"github.com/daglabs/btcd/wire"
"github.com/daglabs/btcd/util"
"github.com/daglabs/btcd/util/gcs"
"github.com/daglabs/btcd/util/gcs/builder"
"github.com/daglabs/btcd/wire"
)
const (
@ -203,7 +203,7 @@ func storeFilter(dbTx database.Tx, block *util.Block, f *gcs.Filter,
// connected to the main chain. This indexer adds a hash-to-cf mapping for
// every passed block. This is part of the Indexer interface.
func (idx *CfIndex) ConnectBlock(dbTx database.Tx, block *util.Block,
virtual *blockdag.VirtualBlock) error {
_ *blockdag.BlockDAG) error {
f, err := builder.BuildBasicFilter(block.MsgBlock())
if err != nil {
@ -227,7 +227,7 @@ func (idx *CfIndex) ConnectBlock(dbTx database.Tx, block *util.Block,
// disconnected from the main chain. This indexer removes the hash-to-cf
// mapping for every passed block. This is part of the Indexer interface.
func (idx *CfIndex) DisconnectBlock(dbTx database.Tx, block *util.Block,
virtual *blockdag.VirtualBlock) error {
_ *blockdag.BlockDAG) error {
for _, key := range cfIndexKeys {
err := dbDeleteFilterIdxEntry(dbTx, key, block.Hash())

View File

@ -51,12 +51,12 @@ type Indexer interface {
Init() error
// ConnectBlock is invoked when the index manager is notified that a new
// block has been connected to the main chain.
ConnectBlock(dbTx database.Tx, block *util.Block, virtual *blockdag.VirtualBlock) error
// block has been connected to the DAG.
ConnectBlock(dbTx database.Tx, block *util.Block, dag *blockdag.BlockDAG) error
// DisconnectBlock is invoked when the index manager is notified that a
// block has been disconnected from the main chain.
DisconnectBlock(dbTx database.Tx, block *util.Block, virtual *blockdag.VirtualBlock) error
// block has been disconnected from the DAG.
DisconnectBlock(dbTx database.Tx, block *util.Block, dag *blockdag.BlockDAG) error
}
// AssertError identifies an error that indicates an internal code consistency

View File

@ -210,12 +210,12 @@ func dbFetchTx(dbTx database.Tx, hash *daghash.Hash) (*wire.MsgTx, error) {
// checks, and invokes each indexer.
//
// This is part of the blockchain.IndexManager interface.
func (m *Manager) ConnectBlock(dbTx database.Tx, block *util.Block, virtual *blockdag.VirtualBlock) error {
func (m *Manager) ConnectBlock(dbTx database.Tx, block *util.Block, dag *blockdag.BlockDAG) error {
// Call each of the currently active optional indexes with the block
// being connected so they can update accordingly.
for _, index := range m.enabledIndexes {
// Notify the indexer with the connected block so it can index it.
if err := index.ConnectBlock(dbTx, block, virtual); err != nil {
if err := index.ConnectBlock(dbTx, block, dag); err != nil {
return err
}
}
@ -228,13 +228,13 @@ func (m *Manager) ConnectBlock(dbTx database.Tx, block *util.Block, virtual *blo
// the index entries associated with the block.
//
// This is part of the blockchain.IndexManager interface.
func (m *Manager) DisconnectBlock(dbTx database.Tx, block *util.Block, virtual *blockdag.VirtualBlock) error {
func (m *Manager) DisconnectBlock(dbTx database.Tx, block *util.Block, dag *blockdag.BlockDAG) error {
// Call each of the currently active optional indexes with the block
// being disconnected so they can update accordingly.
for _, index := range m.enabledIndexes {
// Notify the indexer with the disconnected block so it can remove all
// of the appropriate entries.
if err := index.DisconnectBlock(dbTx, block, virtual); err != nil {
if err := index.DisconnectBlock(dbTx, block, dag); err != nil {
return err
}
}

View File

@ -11,8 +11,8 @@ import (
"github.com/daglabs/btcd/blockdag"
"github.com/daglabs/btcd/dagconfig/daghash"
"github.com/daglabs/btcd/database"
"github.com/daglabs/btcd/wire"
"github.com/daglabs/btcd/util"
"github.com/daglabs/btcd/wire"
)
const (
@ -388,7 +388,7 @@ func (idx *TxIndex) Create(dbTx database.Tx) error {
// for every transaction in the passed block.
//
// This is part of the Indexer interface.
func (idx *TxIndex) ConnectBlock(dbTx database.Tx, block *util.Block, virtual *blockdag.VirtualBlock) error {
func (idx *TxIndex) ConnectBlock(dbTx database.Tx, block *util.Block, _ *blockdag.BlockDAG) error {
// Increment the internal block ID to use for the block being connected
// and add all of the transactions in the block to the index.
newBlockID := idx.curBlockID + 1
@ -411,7 +411,7 @@ func (idx *TxIndex) ConnectBlock(dbTx database.Tx, block *util.Block, virtual *b
// hash-to-transaction mapping for every transaction in the block.
//
// This is part of the Indexer interface.
func (idx *TxIndex) DisconnectBlock(dbTx database.Tx, block *util.Block, virtual *blockdag.VirtualBlock) error {
func (idx *TxIndex) DisconnectBlock(dbTx database.Tx, block *util.Block, _ *blockdag.BlockDAG) error {
// Remove all of the transactions in the block from the index.
if err := dbRemoveTxIndexEntries(dbTx, block); err != nil {
return err

View File

@ -265,7 +265,7 @@ func (dag *BlockDAG) thresholdState(prevNode *blockNode, checker thresholdCondit
// This function is safe for concurrent access.
func (dag *BlockDAG) ThresholdState(deploymentID uint32) (ThresholdState, error) {
dag.dagLock.Lock()
state, err := dag.deploymentState(dag.virtual.SelectedTip(), deploymentID)
state, err := dag.deploymentState(dag.SelectedTip(), deploymentID)
dag.dagLock.Unlock()
return state, err
@ -277,7 +277,7 @@ func (dag *BlockDAG) ThresholdState(deploymentID uint32) (ThresholdState, error)
// This function is safe for concurrent access.
func (dag *BlockDAG) IsDeploymentActive(deploymentID uint32) (bool, error) {
dag.dagLock.Lock()
state, err := dag.deploymentState(dag.virtual.SelectedTip(), deploymentID)
state, err := dag.deploymentState(dag.SelectedTip(), deploymentID)
dag.dagLock.Unlock()
if err != nil {
return false, err
@ -316,7 +316,7 @@ func (dag *BlockDAG) initThresholdCaches() error {
// threshold state for each of them. This will ensure the caches are
// populated and any states that needed to be recalculated due to
// definition changes is done now.
prevNode := dag.virtual.SelectedTip().selectedParent
prevNode := dag.SelectedTip().selectedParent
for bit := uint32(0); bit < vbNumBits; bit++ {
checker := bitConditionChecker{bit: bit, chain: dag}
cache := &dag.warningCaches[bit]
@ -340,7 +340,7 @@ func (dag *BlockDAG) initThresholdCaches() error {
if dag.isCurrent() {
// Warn if a high enough percentage of the last blocks have
// unexpected versions.
bestNode := dag.virtual.SelectedTip()
bestNode := dag.SelectedTip()
if err := dag.warnUnknownVersions(bestNode); err != nil {
return err
}

View File

@ -792,7 +792,7 @@ func (dag *BlockDAG) ensureNoDuplicateTx(node *blockNode, block *util.Block) err
// Duplicate transactions are only allowed if the previous transaction
// is fully spent.
for outpoint := range fetchSet {
utxo, ok := dag.virtual.GetUTXOEntry(outpoint)
utxo, ok := dag.GetUTXOEntry(outpoint)
if ok {
str := fmt.Sprintf("tried to overwrite transaction %v "+
"at block height %d that is not fully spent",
@ -965,7 +965,7 @@ func (dag *BlockDAG) checkConnectBlock(node *blockNode, block *util.Block) error
// countP2SHSigOps for whether or not the transaction is
// a coinbase transaction rather than having to do a
// full coinbase check again.
numP2SHSigOps, err := CountP2SHSigOps(tx, i == 0, dag.VirtualBlock().utxoSet)
numP2SHSigOps, err := CountP2SHSigOps(tx, i == 0, dag.virtual.utxoSet)
if err != nil {
return err
}
@ -992,7 +992,7 @@ func (dag *BlockDAG) checkConnectBlock(node *blockNode, block *util.Block) error
// bounds.
var totalFees int64
for _, tx := range transactions {
txFee, err := CheckTransactionInputs(tx, node.height, dag.VirtualBlock().utxoSet,
txFee, err := CheckTransactionInputs(tx, node.height, dag.virtual.utxoSet,
dag.dagParams)
if err != nil {
return err
@ -1051,7 +1051,7 @@ func (dag *BlockDAG) checkConnectBlock(node *blockNode, block *util.Block) error
// A transaction can only be included within a block
// once the sequence locks of *all* its inputs are
// active.
sequenceLock, err := dag.calcSequenceLock(node, dag.VirtualBlock().utxoSet, tx, false)
sequenceLock, err := dag.calcSequenceLock(node, dag.virtual.utxoSet, tx, false)
if err != nil {
return err
}
@ -1069,7 +1069,7 @@ func (dag *BlockDAG) checkConnectBlock(node *blockNode, block *util.Block) error
// expensive ECDSA signature check scripts. Doing this last helps
// prevent CPU exhaustion attacks.
if runScripts {
err := checkBlockScripts(block, dag.VirtualBlock().utxoSet, scriptFlags, dag.sigCache)
err := checkBlockScripts(block, dag.virtual.utxoSet, scriptFlags, dag.sigCache)
if err != nil {
return err
}
@ -1121,7 +1121,7 @@ func (dag *BlockDAG) CheckConnectBlockTemplate(block *util.Block) error {
return err
}
err = dag.checkBlockContext(block, parents, dag.virtual.SelectedTip(), flags)
err = dag.checkBlockContext(block, parents, dag.SelectedTip(), flags)
if err != nil {
return err
}

View File

@ -221,7 +221,7 @@ func (dag *BlockDAG) calcNextBlockVersion(prevNode *blockNode) (int32, error) {
// This function is safe for concurrent access.
func (dag *BlockDAG) CalcNextBlockVersion() (int32, error) {
dag.dagLock.Lock()
version, err := dag.calcNextBlockVersion(dag.virtual.SelectedTip())
version, err := dag.calcNextBlockVersion(dag.SelectedTip())
dag.dagLock.Unlock()
return version, err
}

View File

@ -6,12 +6,10 @@ package blockdag
import (
"sync"
"github.com/daglabs/btcd/wire"
)
// VirtualBlock is a virtual block whose parents are the tips of the DAG.
type VirtualBlock struct {
// virtualBlock is a virtual block whose parents are the tips of the DAG.
type virtualBlock struct {
mtx sync.Mutex
phantomK uint32
utxoSet *fullUTXOSet
@ -21,9 +19,9 @@ type VirtualBlock struct {
}
// newVirtualBlock creates and returns a new VirtualBlock.
func newVirtualBlock(tips blockSet, phantomK uint32) *VirtualBlock {
func newVirtualBlock(tips blockSet, phantomK uint32) *virtualBlock {
// The mutex is intentionally not held since this is a constructor.
var virtual VirtualBlock
var virtual virtualBlock
virtual.phantomK = phantomK
virtual.utxoSet = NewFullUTXOSet()
virtual.selectedPathSet = newSet()
@ -33,8 +31,8 @@ func newVirtualBlock(tips blockSet, phantomK uint32) *VirtualBlock {
}
// clone creates and returns a clone of the virtual block.
func (v *VirtualBlock) clone() *VirtualBlock {
return &VirtualBlock{
func (v *virtualBlock) clone() *virtualBlock {
return &virtualBlock{
phantomK: v.phantomK,
utxoSet: v.utxoSet.clone().(*fullUTXOSet),
blockNode: v.blockNode,
@ -47,7 +45,7 @@ func (v *VirtualBlock) clone() *VirtualBlock {
// is up to the caller to ensure the lock is held.
//
// This function MUST be called with the view mutex locked (for writes).
func (v *VirtualBlock) setTips(tips blockSet) {
func (v *virtualBlock) setTips(tips blockSet) {
oldSelectedParent := v.selectedParent
v.blockNode = *newBlockNode(nil, tips, v.phantomK)
v.updateSelectedPathSet(oldSelectedParent)
@ -61,7 +59,7 @@ func (v *VirtualBlock) setTips(tips blockSet) {
// parent and are not selected ancestors of the new one, and adding
// blocks that are selected ancestors of the new selected parent
// and aren't selected ancestors of the old one.
func (v *VirtualBlock) updateSelectedPathSet(oldSelectedParent *blockNode) {
func (v *virtualBlock) updateSelectedPathSet(oldSelectedParent *blockNode) {
var intersectionNode *blockNode
for node := v.blockNode.selectedParent; intersectionNode == nil && node != nil; node = node.selectedParent {
if v.selectedPathSet.contains(node) {
@ -86,7 +84,7 @@ func (v *VirtualBlock) updateSelectedPathSet(oldSelectedParent *blockNode) {
// given blockSet.
//
// This function is safe for concurrent access.
func (v *VirtualBlock) SetTips(tips blockSet) {
func (v *virtualBlock) SetTips(tips blockSet) {
v.mtx.Lock()
v.setTips(tips)
v.mtx.Unlock()
@ -98,7 +96,7 @@ func (v *VirtualBlock) SetTips(tips blockSet) {
// is up to the caller to ensure the lock is held.
//
// This function MUST be called with the view mutex locked (for writes).
func (v *VirtualBlock) addTip(newTip *blockNode) {
func (v *virtualBlock) addTip(newTip *blockNode) {
updatedTips := v.tips().clone()
for _, parent := range newTip.parents {
updatedTips.remove(parent)
@ -113,7 +111,7 @@ func (v *VirtualBlock) addTip(newTip *blockNode) {
// from the set.
//
// This function is safe for concurrent access.
func (v *VirtualBlock) AddTip(newTip *blockNode) {
func (v *virtualBlock) AddTip(newTip *blockNode) {
v.mtx.Lock()
v.addTip(newTip)
v.mtx.Unlock()
@ -123,23 +121,6 @@ func (v *VirtualBlock) AddTip(newTip *blockNode) {
// an empty blockSet if there is no tip.
//
// This function is safe for concurrent access.
func (v *VirtualBlock) tips() blockSet {
func (v *virtualBlock) tips() blockSet {
return v.parents
}
// SelectedTip returns the current selected tip for the DAG.
// It will return nil if there is no tip.
//
// This function is safe for concurrent access.
func (v *VirtualBlock) SelectedTip() *blockNode {
return v.selectedParent
}
// GetUTXOEntry returns the requested unspent transaction output. The returned
// instance must be treated as immutable since it is shared by all callers.
//
// This function is safe for concurrent access. However, the returned entry (if
// any) is NOT.
func (v *VirtualBlock) GetUTXOEntry(outPoint wire.OutPoint) (*UTXOEntry, bool) {
return v.utxoSet.get(outPoint)
}

View File

@ -47,41 +47,41 @@ func TestVirtualBlock(t *testing.T) {
// Set its tips to tipsToSet
// Add to it all the tips in tipsToAdd, one after the other
// Call .Tips() on it and compare the result to expectedTips
// Call .SelectedTip() on it and compare the result to expectedSelectedTip
// Call .SelectedTip() on it and compare the result to expectedSelectedParent
tests := []struct {
name string
tipsToSet []*blockNode
tipsToAdd []*blockNode
expectedTips blockSet
expectedSelectedTip *blockNode
name string
tipsToSet []*blockNode
tipsToAdd []*blockNode
expectedTips blockSet
expectedSelectedParent *blockNode
}{
{
name: "empty virtual",
tipsToSet: []*blockNode{},
tipsToAdd: []*blockNode{},
expectedTips: newSet(),
expectedSelectedTip: nil,
name: "empty virtual",
tipsToSet: []*blockNode{},
tipsToAdd: []*blockNode{},
expectedTips: newSet(),
expectedSelectedParent: nil,
},
{
name: "virtual with genesis tip",
tipsToSet: []*blockNode{node0},
tipsToAdd: []*blockNode{},
expectedTips: setFromSlice(node0),
expectedSelectedTip: node0,
name: "virtual with genesis tip",
tipsToSet: []*blockNode{node0},
tipsToAdd: []*blockNode{},
expectedTips: setFromSlice(node0),
expectedSelectedParent: node0,
},
{
name: "virtual with genesis tip, add child of genesis",
tipsToSet: []*blockNode{node0},
tipsToAdd: []*blockNode{node1},
expectedTips: setFromSlice(node1),
expectedSelectedTip: node1,
name: "virtual with genesis tip, add child of genesis",
tipsToSet: []*blockNode{node0},
tipsToAdd: []*blockNode{node1},
expectedTips: setFromSlice(node1),
expectedSelectedParent: node1,
},
{
name: "empty virtual, add a full DAG",
tipsToSet: []*blockNode{},
tipsToAdd: []*blockNode{node0, node1, node2, node3, node4, node5, node6},
expectedTips: setFromSlice(node2, node5, node6),
expectedSelectedTip: node5,
name: "empty virtual, add a full DAG",
tipsToSet: []*blockNode{},
tipsToAdd: []*blockNode{node0, node1, node2, node3, node4, node5, node6},
expectedTips: setFromSlice(node2, node5, node6),
expectedSelectedParent: node5,
},
}
@ -104,11 +104,11 @@ func TestVirtualBlock(t *testing.T) {
"Expected: %v, got: %v.", test.name, test.expectedTips, resultTips)
}
// Ensure that the virtual block's selectedTip is now equal to expectedSelectedTip
resultSelectedTip := virtual.SelectedTip()
if !reflect.DeepEqual(resultSelectedTip, test.expectedSelectedTip) {
// Ensure that the virtual block's selectedParent is now equal to expectedSelectedParent
resultSelectedTip := virtual.selectedParent
if !reflect.DeepEqual(resultSelectedTip, test.expectedSelectedParent) {
t.Errorf("unexpected selected tip in test \"%s\". "+
"Expected: %v, got: %v.", test.name, test.expectedSelectedTip, resultSelectedTip)
"Expected: %v, got: %v.", test.name, test.expectedSelectedParent, resultSelectedTip)
}
}
}

View File

@ -402,7 +402,6 @@ func NewBlkTmplGenerator(policy *Policy, params *dagconfig.Params,
// ----------------------------------- --
func (g *BlkTmplGenerator) NewBlockTemplate(payToAddress util.Address) (*BlockTemplate, error) {
// Extend the most recently known best block.
virtualBlock := g.dag.VirtualBlock()
nextBlockHeight := g.dag.Height() + 1
// Create a standard coinbase transaction paying to the provided
@ -699,7 +698,7 @@ mempoolLoop:
// Calculate the required difficulty for the block. The timestamp
// is potentially adjusted to ensure it comes after the median time of
// the last several blocks per the chain consensus rules.
ts := medianAdjustedTime(virtualBlock.SelectedTip().CalcPastMedianTime(), g.timeSource)
ts := medianAdjustedTime(g.dag.SelectedTip().CalcPastMedianTime(), g.timeSource)
reqDifficulty, err := g.dag.CalcNextRequiredDifficulty(ts)
if err != nil {
return nil, err
@ -762,7 +761,7 @@ func (g *BlkTmplGenerator) UpdateBlockTime(msgBlock *wire.MsgBlock) error {
// The new timestamp is potentially adjusted to ensure it comes after
// the median time of the last several blocks per the chain consensus
// rules.
dagMedianTime := g.dag.VirtualBlock().CalcPastMedianTime()
dagMedianTime := g.dag.CalcPastMedianTime()
newTime := medianAdjustedTime(dagMedianTime, g.timeSource)
msgBlock.Header.Timestamp = newTime
@ -806,15 +805,6 @@ func (g *BlkTmplGenerator) UpdateExtraNonce(msgBlock *wire.MsgBlock, blockHeight
return nil
}
// VirtualBlock returns the DAG's virtual block in the current point in time.
// The returned instance must be treated as immutable since it is shared by all
// callers.
//
// This function is safe for concurrent access.
func (g *BlkTmplGenerator) VirtualBlock() *blockdag.VirtualBlock {
return g.dag.VirtualBlock()
}
// DAGHeight returns the DAG's height
func (g *BlkTmplGenerator) DAGHeight() int32 {
return g.dag.Height()

View File

@ -872,7 +872,7 @@ func (sm *SyncManager) haveInventory(invVect *wire.InvVect) (bool, error) {
prevOut := wire.OutPoint{Hash: invVect.Hash}
for i := uint32(0); i < 2; i++ {
prevOut.Index = i
entry, ok := sm.dag.VirtualBlock().GetUTXOEntry(prevOut)
entry, ok := sm.dag.GetUTXOEntry(prevOut)
if !ok {
return false, nil
}

View File

@ -2437,7 +2437,7 @@ func NewServer(listenAddrs []string, db database.DB, dagParams *dagconfig.Params
},
DAGParams: dagParams,
BestHeight: func() int32 { return s.DAG.Height() }, //TODO: (Ori) This is probably wrong. Done only for compilation
MedianTimePast: func() time.Time { return s.DAG.VirtualBlock().SelectedTip().CalcPastMedianTime() },
MedianTimePast: func() time.Time { return s.DAG.SelectedTip().CalcPastMedianTime() },
CalcSequenceLock: func(tx *util.Tx, utxoSet blockdag.UTXOSet) (*blockdag.SequenceLock, error) {
return s.DAG.CalcSequenceLock(tx, utxoSet, true)
},

View File

@ -1199,15 +1199,14 @@ func handleGetBlockDAGInfo(s *Server, cmd interface{}, closeChan <-chan struct{}
// populate the response to this call primarily from this snapshot.
params := s.cfg.DAGParams
dag := s.cfg.DAG
virtualBlock := dag.VirtualBlock()
dagInfo := &btcjson.GetBlockDAGInfoResult{
DAG: params.Name,
Blocks: dag.Height(), //TODO: (Ori) This is wrong. Done only for compilation
Headers: dag.Height(), //TODO: (Ori) This is wrong. Done only for compilation
TipHashes: daghash.Strings(dag.TipHashes()),
Difficulty: getDifficultyRatio(virtualBlock.SelectedTip().Header().Bits, params),
MedianTime: virtualBlock.SelectedTip().CalcPastMedianTime().Unix(),
Difficulty: getDifficultyRatio(dag.CurrentBits(), params),
MedianTime: dag.SelectedTip().CalcPastMedianTime().Unix(),
Pruned: false,
Bip9SoftForks: make(map[string]*btcjson.Bip9SoftForkDescription),
}
@ -1521,7 +1520,6 @@ func (state *gbtWorkState) updateBlockTemplate(s *Server, useCoinbaseValue bool)
// generated.
var msgBlock *wire.MsgBlock
var targetDifficulty string
virtualBlock := s.cfg.DAG.VirtualBlock()
tipHashes := s.cfg.DAG.TipHashes()
template := state.template
if template == nil || state.tipHashes == nil ||
@ -1561,7 +1559,7 @@ func (state *gbtWorkState) updateBlockTemplate(s *Server, useCoinbaseValue bool)
// Get the minimum allowed timestamp for the block based on the
// median timestamp of the last several blocks per the chain
// consensus rules.
minTimestamp := mining.MinimumMedianTime(virtualBlock.SelectedTip().CalcPastMedianTime())
minTimestamp := mining.MinimumMedianTime(s.cfg.DAG.SelectedTip().CalcPastMedianTime())
// Update work state to ensure another block template isn't
// generated until needed.
@ -2204,8 +2202,7 @@ func handleGetCurrentNet(s *Server, cmd interface{}, closeChan <-chan struct{})
// handleGetDifficulty implements the getdifficulty command.
func handleGetDifficulty(s *Server, cmd interface{}, closeChan <-chan struct{}) (interface{}, error) {
virtualBlock := s.cfg.DAG.VirtualBlock()
return getDifficultyRatio(virtualBlock.SelectedTip().Header().Bits, s.cfg.DAGParams), nil
return getDifficultyRatio(s.cfg.DAG.SelectedTip().Header().Bits, s.cfg.DAGParams), nil
}
// handleGetGenerate implements the getgenerate command.
@ -2262,7 +2259,6 @@ func handleGetHeaders(s *Server, cmd interface{}, closeChan <-chan struct{}) (in
// handleGetInfo implements the getinfo command. We only return the fields
// that are not related to wallet functionality.
func handleGetInfo(s *Server, cmd interface{}, closeChan <-chan struct{}) (interface{}, error) {
virtualBlock := s.cfg.DAG.VirtualBlock()
ret := &btcjson.InfoDAGResult{
Version: int32(1000000*version.AppMajor + 10000*version.AppMinor + 100*version.AppPatch),
ProtocolVersion: int32(maxProtocolVersion),
@ -2270,7 +2266,7 @@ func handleGetInfo(s *Server, cmd interface{}, closeChan <-chan struct{}) (inter
TimeOffset: int64(s.cfg.TimeSource.Offset().Seconds()),
Connections: s.cfg.ConnMgr.ConnectedCount(),
Proxy: config.MainConfig().Proxy,
Difficulty: getDifficultyRatio(virtualBlock.SelectedTip().Header().Bits, s.cfg.DAGParams),
Difficulty: getDifficultyRatio(s.cfg.DAG.CurrentBits(), s.cfg.DAGParams),
TestNet: config.MainConfig().TestNet3,
RelayFee: config.MainConfig().MinRelayTxFee.ToBTC(),
}
@ -2314,7 +2310,6 @@ func handleGetMiningInfo(s *Server, cmd interface{}, closeChan <-chan struct{})
}
}
virtualBlock := s.cfg.DAG.VirtualBlock()
highestTipHash := s.cfg.DAG.HighestTipHash()
selectedBlock, err := s.cfg.DAG.BlockByHash(&highestTipHash)
if err != nil {
@ -2328,7 +2323,7 @@ func handleGetMiningInfo(s *Server, cmd interface{}, closeChan <-chan struct{})
Blocks: int64(s.cfg.DAG.Height()), //TODO: (Ori) This is wrong. Done only for compilation
CurrentBlockSize: uint64(selectedBlock.MsgBlock().SerializeSize()),
CurrentBlockTx: uint64(len(selectedBlock.MsgBlock().Transactions)),
Difficulty: getDifficultyRatio(virtualBlock.SelectedTip().Header().Bits, s.cfg.DAGParams),
Difficulty: getDifficultyRatio(s.cfg.DAG.CurrentBits(), s.cfg.DAGParams),
Generate: s.cfg.CPUMiner.IsMining(),
GenProcLimit: s.cfg.CPUMiner.NumWorkers(),
HashesPerSec: int64(s.cfg.CPUMiner.HashesPerSecond()),
@ -2587,7 +2582,7 @@ func handleGetTxOut(s *Server, cmd interface{}, closeChan <-chan struct{}) (inte
isCoinbase = blockdag.IsCoinBaseTx(mtx)
} else {
out := wire.OutPoint{Hash: *txHash, Index: c.Vout}
entry, ok := s.cfg.DAG.VirtualBlock().GetUTXOEntry(out)
entry, ok := s.cfg.DAG.GetUTXOEntry(out)
if !ok {
return nil, rpcNoTxInfoError(txHash)
}