[NOD-855] Save ECMH for block utxo and not diff utxo (#669)

* [NOD-855] Save ECMH for each block UTXO

* [NOD-855] Remove UpdateExtraNonce method

* [NOD-855] Remove multiset data from UTXO diffs

* [NOD-855] Fix to fetch multiset of selected parent

* [NOD-855] Don't remove coinbase inputs from multiset

* [NOD-855] Create multisetBucketName on startup

* [NOD-855] Remove multiset from UTXO diff tests

* [NOD-855] clear new entries from multisetstore on saveChangesFromBlock

* [NOD-855] Fix tests

* [NOD-855] Use UnacceptedBlueScore when adding current block transactions to multiset

* [NOD-855] Hash utxo before adding it to multiset

* [NOD-855] Pass isCoinbase to NewUTXOEntry

* [NOD-855] Do not use hash when adding entries to multiset

* [NOD-855] When calculating multiset, replace the unaccepted blue score of selected parent transaction with the block blue score

* [NOD-855] Manually add a chained transaction to a block in TestChainedTransactions

* [NOD-855] Change name and comments

* [NOD-855] Use FindAcceptanceData to find a specific block acceptance data

* [NOD-855] Remove redundant copy of txIn.PreviousOutpoint

* [NOD-855] Use fmt.Sprintf when creating internalRPCError
This commit is contained in:
Ori Newman 2020-03-26 13:06:12 +02:00 committed by GitHub
parent 1020402b34
commit d83862f36c
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
20 changed files with 535 additions and 551 deletions

View File

@ -6,6 +6,7 @@ package blockdag
import (
"fmt"
"github.com/kaspanet/kaspad/ecc"
"math"
"sort"
"sync"
@ -153,6 +154,7 @@ type BlockDAG struct {
SubnetworkStore *SubnetworkStore
utxoDiffStore *utxoDiffStore
reachabilityStore *reachabilityStore
multisetStore *multisetStore
}
// IsKnownBlock returns whether or not the DAG instance has the block represented
@ -560,14 +562,13 @@ func (dag *BlockDAG) connectBlock(node *blockNode,
return nil, err
}
newBlockUTXO, txsAcceptanceData, newBlockFeeData, err := node.verifyAndBuildUTXO(dag, block.Transactions(), fastAdd)
newBlockUTXO, txsAcceptanceData, newBlockFeeData, newBlockMultiSet, err := node.verifyAndBuildUTXO(dag, block.Transactions(), fastAdd)
if err != nil {
newErrString := fmt.Sprintf("error verifying UTXO for %s: %s", node, err)
var ruleErr RuleError
if ok := errors.As(err, &ruleErr); ok {
return nil, ruleError(ruleErr.ErrorCode, newErrString)
return nil, ruleError(ruleErr.ErrorCode, fmt.Sprintf("error verifying UTXO for %s: %s", node, err))
}
return nil, errors.New(newErrString)
return nil, errors.Wrapf(err, "error verifying UTXO for %s", node)
}
err = node.validateCoinbaseTransaction(dag, block, txsAcceptanceData)
@ -576,7 +577,7 @@ func (dag *BlockDAG) connectBlock(node *blockNode,
}
// Apply all changes to the DAG.
virtualUTXODiff, virtualTxsAcceptanceData, chainUpdates, err := dag.applyDAGChanges(node, newBlockUTXO, selectedParentAnticone)
virtualUTXODiff, virtualTxsAcceptanceData, chainUpdates, err := dag.applyDAGChanges(node, newBlockUTXO, newBlockMultiSet, selectedParentAnticone)
if err != nil {
// Since all validation logic has already ran, if applyDAGChanges errors out,
// this means we have a problem in the internal structure of the DAG - a problem which is
@ -593,6 +594,126 @@ func (dag *BlockDAG) connectBlock(node *blockNode,
return chainUpdates, nil
}
// calcMultiset returns the multiset of the UTXO of the given block with the given transactions.
func (node *blockNode) calcMultiset(dag *BlockDAG, transactions []*util.Tx, acceptanceData MultiBlockTxsAcceptanceData, selectedParentUTXO, pastUTXO UTXOSet) (*ecc.Multiset, error) {
ms, err := node.pastUTXOMultiSet(dag, acceptanceData, selectedParentUTXO)
if err != nil {
return nil, err
}
for _, tx := range transactions {
ms, err = addTxToMultiset(ms, tx.MsgTx(), pastUTXO, UnacceptedBlueScore)
if err != nil {
return nil, err
}
}
return ms, nil
}
// acceptedSelectedParentMultiset takes the multiset of the selected
// parent, replaces all the selected parent outputs' blue score with
// the block blue score and returns the result.
func (node *blockNode) acceptedSelectedParentMultiset(dag *BlockDAG,
acceptanceData MultiBlockTxsAcceptanceData) (*ecc.Multiset, error) {
if node.isGenesis() {
return ecc.NewMultiset(ecc.S256()), nil
}
ms, err := dag.multisetStore.multisetByBlockNode(node.selectedParent)
if err != nil {
return nil, err
}
selectedParentAcceptanceData, exists := acceptanceData.FindAcceptanceData(node.selectedParent.hash)
if !exists {
return nil, errors.Errorf("couldn't find selected parent acceptance data for block %s", node)
}
for _, txAcceptanceData := range selectedParentAcceptanceData.TxAcceptanceData {
tx := txAcceptanceData.Tx
msgTx := tx.MsgTx()
isCoinbase := tx.IsCoinBase()
for i, txOut := range msgTx.TxOut {
outpoint := *wire.NewOutpoint(tx.ID(), uint32(i))
unacceptedEntry := NewUTXOEntry(txOut, isCoinbase, UnacceptedBlueScore)
acceptedEntry := NewUTXOEntry(txOut, isCoinbase, node.blueScore)
var err error
ms, err = removeUTXOFromMultiset(ms, unacceptedEntry, &outpoint)
if err != nil {
return nil, err
}
ms, err = addUTXOToMultiset(ms, acceptedEntry, &outpoint)
if err != nil {
return nil, err
}
}
}
return ms, nil
}
func (node *blockNode) pastUTXOMultiSet(dag *BlockDAG, acceptanceData MultiBlockTxsAcceptanceData, selectedParentUTXO UTXOSet) (*ecc.Multiset, error) {
ms, err := node.acceptedSelectedParentMultiset(dag, acceptanceData)
if err != nil {
return nil, err
}
for _, blockAcceptanceData := range acceptanceData {
if blockAcceptanceData.BlockHash.IsEqual(node.selectedParent.hash) {
continue
}
for _, txAcceptanceData := range blockAcceptanceData.TxAcceptanceData {
if !txAcceptanceData.IsAccepted {
continue
}
tx := txAcceptanceData.Tx.MsgTx()
var err error
ms, err = addTxToMultiset(ms, tx, selectedParentUTXO, node.blueScore)
if err != nil {
return nil, err
}
}
}
return ms, nil
}
func addTxToMultiset(ms *ecc.Multiset, tx *wire.MsgTx, pastUTXO UTXOSet, blockBlueScore uint64) (*ecc.Multiset, error) {
isCoinbase := tx.IsCoinBase()
if !isCoinbase {
for _, txIn := range tx.TxIn {
entry, ok := pastUTXO.Get(txIn.PreviousOutpoint)
if !ok {
return nil, errors.Errorf("Couldn't find entry for outpoint %s", txIn.PreviousOutpoint)
}
var err error
ms, err = removeUTXOFromMultiset(ms, entry, &txIn.PreviousOutpoint)
if err != nil {
return nil, err
}
}
}
for i, txOut := range tx.TxOut {
outpoint := *wire.NewOutpoint(tx.TxID(), uint32(i))
entry := NewUTXOEntry(txOut, isCoinbase, blockBlueScore)
var err error
ms, err = addUTXOToMultiset(ms, entry, &outpoint)
if err != nil {
return nil, err
}
}
return ms, nil
}
func (dag *BlockDAG) saveChangesFromBlock(block *util.Block, virtualUTXODiff *UTXODiff,
txsAcceptanceData MultiBlockTxsAcceptanceData, virtualTxsAcceptanceData MultiBlockTxsAcceptanceData,
feeData compactFeeData) error {
@ -614,6 +735,11 @@ func (dag *BlockDAG) saveChangesFromBlock(block *util.Block, virtualUTXODiff *UT
return err
}
err = dag.multisetStore.flushToDB(dbTx)
if err != nil {
return err
}
// Update best block state.
state := &dagState{
TipHashes: dag.TipHashes(),
@ -663,6 +789,7 @@ func (dag *BlockDAG) saveChangesFromBlock(block *util.Block, virtualUTXODiff *UT
dag.index.clearDirtyEntries()
dag.utxoDiffStore.clearDirtyEntries()
dag.reachabilityStore.clearDirtyEntries()
dag.multisetStore.clearNewEntries()
return nil
}
@ -839,7 +966,7 @@ func (dag *BlockDAG) NextAcceptedIDMerkleRootNoLock() (*daghash.Hash, error) {
//
// This function MUST be called with the DAG read-lock held
func (dag *BlockDAG) TxsAcceptedByVirtual() (MultiBlockTxsAcceptanceData, error) {
_, txsAcceptanceData, err := dag.pastUTXO(&dag.virtual.blockNode)
_, _, txsAcceptanceData, err := dag.pastUTXO(&dag.virtual.blockNode)
return txsAcceptanceData, err
}
@ -851,7 +978,7 @@ func (dag *BlockDAG) TxsAcceptedByBlockHash(blockHash *daghash.Hash) (MultiBlock
if node == nil {
return nil, errors.Errorf("Couldn't find block %s", blockHash)
}
_, txsAcceptanceData, err := dag.pastUTXO(node)
_, _, txsAcceptanceData, err := dag.pastUTXO(node)
return txsAcceptanceData, err
}
@ -862,12 +989,13 @@ func (dag *BlockDAG) TxsAcceptedByBlockHash(blockHash *daghash.Hash) (MultiBlock
// 4. Updates each of the tips' utxoDiff.
// 5. Applies the new virtual's blue score to all the unaccepted UTXOs
// 6. Adds the block to the reachability structures
// 7. Updates the finality point of the DAG (if required).
// 7. Adds the multiset of the block to the multiset store.
// 8. Updates the finality point of the DAG (if required).
//
// It returns the diff in the virtual block's UTXO set.
//
// This function MUST be called with the DAG state lock held (for writes).
func (dag *BlockDAG) applyDAGChanges(node *blockNode, newBlockUTXO UTXOSet, selectedParentAnticone []*blockNode) (
func (dag *BlockDAG) applyDAGChanges(node *blockNode, newBlockUTXO UTXOSet, newBlockMultiset *ecc.Multiset, selectedParentAnticone []*blockNode) (
virtualUTXODiff *UTXODiff, virtualTxsAcceptanceData MultiBlockTxsAcceptanceData,
chainUpdates *chainUpdates, err error) {
@ -877,6 +1005,8 @@ func (dag *BlockDAG) applyDAGChanges(node *blockNode, newBlockUTXO UTXOSet, sele
return nil, nil, nil, errors.Wrap(err, "failed updating reachability")
}
dag.multisetStore.setMultiset(node, newBlockMultiset)
if err = node.updateParents(dag, newBlockUTXO); err != nil {
return nil, nil, nil, errors.Wrapf(err, "failed updating parents of %s", node)
}
@ -885,7 +1015,7 @@ func (dag *BlockDAG) applyDAGChanges(node *blockNode, newBlockUTXO UTXOSet, sele
chainUpdates = dag.virtual.AddTip(node)
// Build a UTXO set for the new virtual block
newVirtualUTXO, virtualTxsAcceptanceData, err := dag.pastUTXO(&dag.virtual.blockNode)
newVirtualUTXO, _, virtualTxsAcceptanceData, err := dag.pastUTXO(&dag.virtual.blockNode)
if err != nil {
return nil, nil, nil, errors.Wrap(err, "could not restore past UTXO for virtual")
}
@ -936,42 +1066,49 @@ func (node *blockNode) diffFromTxs(pastUTXO UTXOSet, transactions []*util.Tx) (*
}
// verifyAndBuildUTXO verifies all transactions in the given block and builds its UTXO
// to save extra traversals it returns the transactions acceptance data and the compactFeeData for the new block
// to save extra traversals it returns the transactions acceptance data, the compactFeeData
// for the new block and its multiset.
func (node *blockNode) verifyAndBuildUTXO(dag *BlockDAG, transactions []*util.Tx, fastAdd bool) (
newBlockUTXO UTXOSet, txsAcceptanceData MultiBlockTxsAcceptanceData, newBlockFeeData compactFeeData, err error) {
newBlockUTXO UTXOSet, txsAcceptanceData MultiBlockTxsAcceptanceData, newBlockFeeData compactFeeData, multiset *ecc.Multiset, err error) {
pastUTXO, txsAcceptanceData, err := dag.pastUTXO(node)
pastUTXO, selectedParentUTXO, txsAcceptanceData, err := dag.pastUTXO(node)
if err != nil {
return nil, nil, nil, err
return nil, nil, nil, nil, err
}
err = node.validateAcceptedIDMerkleRoot(dag, txsAcceptanceData)
if err != nil {
return nil, nil, nil, err
return nil, nil, nil, nil, err
}
feeData, err := dag.checkConnectToPastUTXO(node, pastUTXO, transactions, fastAdd)
if err != nil {
return nil, nil, nil, err
return nil, nil, nil, nil, err
}
diffFromTxs, err := node.diffFromTxs(pastUTXO, transactions)
if err != nil {
return nil, nil, nil, err
return nil, nil, nil, nil, err
}
utxo, err := pastUTXO.WithDiff(diffFromTxs)
if err != nil {
return nil, nil, nil, err
return nil, nil, nil, nil, err
}
calculatedMultisetHash := utxo.Multiset().Hash()
multiset, err = node.calcMultiset(dag, transactions, txsAcceptanceData, selectedParentUTXO, pastUTXO)
if err != nil {
return nil, nil, nil, nil, err
}
calculatedMultisetHash := multiset.Hash()
if !calculatedMultisetHash.IsEqual(node.utxoCommitment) {
str := fmt.Sprintf("block %s UTXO commitment is invalid - block "+
"header indicates %s, but calculated value is %s", node.hash,
node.utxoCommitment, calculatedMultisetHash)
return nil, nil, nil, ruleError(ErrBadUTXOCommitment, str)
return nil, nil, nil, nil, ruleError(ErrBadUTXOCommitment, str)
}
return utxo, txsAcceptanceData, feeData, nil
return utxo, txsAcceptanceData, feeData, multiset, nil
}
// TxAcceptanceData stores a transaction together with an indication
@ -1125,28 +1262,33 @@ func (node *blockNode) updateParentsDiffs(dag *BlockDAG, newBlockUTXO UTXOSet) e
// To save traversals over the blue blocks, it also returns the transaction acceptance data for
// all blue blocks
func (dag *BlockDAG) pastUTXO(node *blockNode) (
pastUTXO UTXOSet, bluesTxsAcceptanceData MultiBlockTxsAcceptanceData, err error) {
pastUTXO, selectedParentUTXO UTXOSet, bluesTxsAcceptanceData MultiBlockTxsAcceptanceData, err error) {
if node.isGenesis() {
return genesisPastUTXO(dag.virtual), MultiBlockTxsAcceptanceData{}, nil
return genesisPastUTXO(dag.virtual), NewFullUTXOSet(), MultiBlockTxsAcceptanceData{}, nil
}
selectedParentUTXO, err := dag.restoreUTXO(node.selectedParent)
selectedParentUTXO, err = dag.restoreUTXO(node.selectedParent)
if err != nil {
return nil, nil, err
return nil, nil, nil, err
}
blueBlocks, err := node.fetchBlueBlocks(dag.db)
if err != nil {
return nil, nil, err
return nil, nil, nil, err
}
selectedParent := blueBlocks[0]
acceptedSelectedParentUTXO, selectedParentAcceptanceData, err := node.acceptSelectedParentTransactions(selectedParent, selectedParentUTXO)
if err != nil {
return nil, nil, err
return nil, nil, nil, err
}
return node.applyBlueBlocks(acceptedSelectedParentUTXO, selectedParentAcceptanceData, blueBlocks)
pastUTXO, bluesTxsAcceptanceData, err = node.applyBlueBlocks(acceptedSelectedParentUTXO, selectedParentAcceptanceData, blueBlocks)
if err != nil {
return nil, nil, nil, err
}
return pastUTXO, selectedParentUTXO, bluesTxsAcceptanceData, nil
}
func (node *blockNode) acceptSelectedParentTransactions(selectedParent *util.Block, selectedParentUTXO UTXOSet) (acceptedSelectedParentUTXO UTXOSet, txAcceptanceData []TxAcceptanceData, err error) {
@ -1381,11 +1523,6 @@ func (dag *BlockDAG) UTXOConfirmations(outpoint *wire.Outpoint) (uint64, bool) {
return confirmations, true
}
// UTXOCommitment returns a commitment to the dag's current UTXOSet
func (dag *BlockDAG) UTXOCommitment() string {
return dag.UTXOSet().UTXOMultiset.Hash().String()
}
// blockConfirmations returns the current confirmations number of the given node
// The confirmations number is defined as follows:
// * If the node is in the selected tip red set -> 0
@ -1952,6 +2089,7 @@ func New(config *Config) (*BlockDAG, error) {
dag.virtual = newVirtualBlock(dag, nil)
dag.utxoDiffStore = newUTXODiffStore(dag)
dag.reachabilityStore = newReachabilityStore(dag)
dag.multisetStore = newMultisetStore(dag)
// Initialize the DAG state from the passed database. When the db
// does not yet contain any DAG state, both it and the DAG state

View File

@ -59,6 +59,10 @@ var (
// reachability tree nodes and future covering sets of blocks.
reachabilityDataBucketName = []byte("reachability")
// multisetBucketName is the name of the database bucket used to house the
// ECMH multisets of blocks.
multisetBucketName = []byte("multiset")
// subnetworksBucketName is the name of the database bucket used to store the
// subnetwork registry.
subnetworksBucketName = []byte("subnetworks")
@ -265,6 +269,11 @@ func (dag *BlockDAG) createDAGState() error {
return err
}
_, err = meta.CreateBucket(multisetBucketName)
if err != nil {
return err
}
err = dbPutVersion(dbTx, utxoSetVersionKeyName,
latestUTXOSetBucketVersion)
if err != nil {
@ -320,6 +329,11 @@ func (dag *BlockDAG) removeDAGState() error {
return err
}
err = meta.DeleteBucket(multisetBucketName)
if err != nil {
return err
}
err = dbTx.Metadata().Delete(utxoSetVersionKeyName)
if err != nil {
return err
@ -485,11 +499,19 @@ func (dag *BlockDAG) initDAGState() error {
}
// Initialize the reachability store
log.Infof("Loading reachability data...")
err = dag.reachabilityStore.init(dbTx)
if err != nil {
return err
}
// Initialize the multiset store
log.Infof("Loading multiset data...")
err = dag.multisetStore.init(dbTx)
if err != nil {
return err
}
// Apply the loaded utxoCollection to the virtual block.
dag.virtual.utxoSet, err = newFullUTXOSetFromUTXOCollection(fullUTXOCollection)
if err != nil {

View File

@ -267,11 +267,19 @@ func TestChainedTransactions(t *testing.T) {
}
chainedTx := wire.NewNativeMsgTx(wire.TxVersion, []*wire.TxIn{chainedTxIn}, []*wire.TxOut{chainedTxOut})
block2, err := mining.PrepareBlockForTest(dag, &params, []*daghash.Hash{block1.BlockHash()}, []*wire.MsgTx{tx, chainedTx}, true)
block2, err := mining.PrepareBlockForTest(dag, &params, []*daghash.Hash{block1.BlockHash()}, []*wire.MsgTx{tx}, false)
if err != nil {
t.Fatalf("PrepareBlockForTest: %v", err)
}
// Manually add a chained transaction to block2
block2.Transactions = append(block2.Transactions, chainedTx)
block2UtilTxs := make([]*util.Tx, len(block2.Transactions))
for i, tx := range block2.Transactions {
block2UtilTxs[i] = util.NewTx(tx)
}
block2.Header.HashMerkleRoot = blockdag.BuildHashMerkleTreeStore(block2UtilTxs).Root()
//Checks that dag.ProcessBlock fails because we don't allow a transaction to spend another transaction from the same block
isOrphan, isDelayed, err = dag.ProcessBlock(util.NewBlock(block2), blockdag.BFNoPoWCheck)
if err == nil {

View File

@ -3,6 +3,7 @@ package blockdag
import (
"bytes"
"encoding/binary"
"github.com/kaspanet/kaspad/ecc"
"github.com/kaspanet/kaspad/txscript"
"github.com/kaspanet/kaspad/util"
"github.com/kaspanet/kaspad/wire"
@ -12,6 +13,8 @@ import (
// BlockForMining returns a block with the given transactions
// that points to the current DAG tips, that is valid from
// all aspects except proof of work.
//
// This function MUST be called with the DAG state lock held (for reads).
func (dag *BlockDAG) BlockForMining(transactions []*util.Tx) (*wire.MsgBlock, error) {
blockTimestamp := dag.NextBlockTime()
requiredDifficulty := dag.NextRequiredDifficulty(blockTimestamp)
@ -34,18 +37,17 @@ func (dag *BlockDAG) BlockForMining(transactions []*util.Tx) (*wire.MsgBlock, er
msgBlock.AddTransaction(tx.MsgTx())
}
utxoWithTransactions, err := dag.UTXOSet().WithTransactions(msgBlock.Transactions, UnacceptedBlueScore, false)
multiset, err := dag.NextBlockMultiset(transactions)
if err != nil {
return nil, err
}
utxoCommitment := utxoWithTransactions.Multiset().Hash()
msgBlock.Header = wire.BlockHeader{
Version: nextBlockVersion,
ParentHashes: dag.TipHashes(),
HashMerkleRoot: hashMerkleTree.Root(),
AcceptedIDMerkleRoot: acceptedIDMerkleRoot,
UTXOCommitment: utxoCommitment,
UTXOCommitment: multiset.Hash(),
Timestamp: blockTimestamp,
Bits: requiredDifficulty,
}
@ -53,6 +55,19 @@ func (dag *BlockDAG) BlockForMining(transactions []*util.Tx) (*wire.MsgBlock, er
return &msgBlock, nil
}
// NextBlockMultiset returns the multiset of an assumed next block
// built on top of the current tips, with the given transactions.
//
// This function MUST be called with the DAG state lock held (for reads).
func (dag *BlockDAG) NextBlockMultiset(transactions []*util.Tx) (*ecc.Multiset, error) {
pastUTXO, selectedParentUTXO, txsAcceptanceData, err := dag.pastUTXO(&dag.virtual.blockNode)
if err != nil {
return nil, err
}
return dag.virtual.blockNode.calcMultiset(dag, transactions, txsAcceptanceData, selectedParentUTXO, pastUTXO)
}
// CoinbasePayloadExtraData returns coinbase payload extra data parameter
// which is built from extra nonce and coinbase flags.
func CoinbasePayloadExtraData(extraNonce uint64, coinbaseFlags string) ([]byte, error) {

49
blockdag/multisetio.go Normal file
View File

@ -0,0 +1,49 @@
package blockdag
import (
"encoding/binary"
"github.com/kaspanet/kaspad/ecc"
"io"
"math/big"
)
const multisetPointSize = 32
// serializeMultiset serializes an ECMH multiset. The serialization
// uses the following format: <x (32 bytes)><y (32 bytes)>.
func serializeMultiset(w io.Writer, ms *ecc.Multiset) error {
x, y := ms.Point()
xBytes := make([]byte, multisetPointSize)
copy(xBytes, x.Bytes())
yBytes := make([]byte, multisetPointSize)
copy(yBytes, y.Bytes())
err := binary.Write(w, byteOrder, xBytes)
if err != nil {
return err
}
err = binary.Write(w, byteOrder, yBytes)
if err != nil {
return err
}
return nil
}
// deserializeMultiset deserializes an EMCH multiset.
// See serializeMultiset for more details.
func deserializeMultiset(r io.Reader) (*ecc.Multiset, error) {
xBytes := make([]byte, multisetPointSize)
yBytes := make([]byte, multisetPointSize)
err := binary.Read(r, byteOrder, xBytes)
if err != nil {
return nil, err
}
err = binary.Read(r, byteOrder, yBytes)
if err != nil {
return nil, err
}
var x, y big.Int
x.SetBytes(xBytes)
y.SetBytes(yBytes)
return ecc.NewMultisetFromPoint(ecc.S256(), &x, &y), nil
}

112
blockdag/multisetstore.go Normal file
View File

@ -0,0 +1,112 @@
package blockdag
import (
"bytes"
"github.com/kaspanet/kaspad/database"
"github.com/kaspanet/kaspad/ecc"
"github.com/kaspanet/kaspad/util/daghash"
"github.com/kaspanet/kaspad/util/locks"
"github.com/pkg/errors"
)
type multisetStore struct {
dag *BlockDAG
new map[daghash.Hash]struct{}
loaded map[daghash.Hash]*ecc.Multiset
mtx *locks.PriorityMutex
}
func newMultisetStore(dag *BlockDAG) *multisetStore {
return &multisetStore{
dag: dag,
new: make(map[daghash.Hash]struct{}),
loaded: make(map[daghash.Hash]*ecc.Multiset),
}
}
func (store *multisetStore) setMultiset(node *blockNode, ms *ecc.Multiset) {
store.loaded[*node.hash] = ms
store.addToNewBlocks(node.hash)
}
func (store *multisetStore) addToNewBlocks(blockHash *daghash.Hash) {
store.new[*blockHash] = struct{}{}
}
func multisetNotFoundError(blockHash *daghash.Hash) error {
return errors.Errorf("Couldn't find multiset data for block %s", blockHash)
}
func (store *multisetStore) multisetByBlockNode(node *blockNode) (*ecc.Multiset, error) {
ms, exists := store.multisetByBlockHash(node.hash)
if !exists {
return nil, multisetNotFoundError(node.hash)
}
return ms, nil
}
func (store *multisetStore) multisetByBlockHash(hash *daghash.Hash) (*ecc.Multiset, bool) {
ms, ok := store.loaded[*hash]
return ms, ok
}
// flushToDB writes all new multiset data to the database.
func (store *multisetStore) flushToDB(dbTx database.Tx) error {
if len(store.new) == 0 {
return nil
}
w := &bytes.Buffer{}
for hash := range store.new {
hash := hash // Copy hash to a new variable to avoid passing the same pointer
w.Reset()
ms, exists := store.loaded[hash]
if !exists {
return multisetNotFoundError(&hash)
}
err := serializeMultiset(w, ms)
if err != nil {
return err
}
err = store.dbStoreMultiset(dbTx, &hash, w.Bytes())
if err != nil {
return err
}
}
return nil
}
func (store *multisetStore) clearNewEntries() {
store.new = make(map[daghash.Hash]struct{})
}
func (store *multisetStore) init(dbTx database.Tx) error {
bucket := dbTx.Metadata().Bucket(multisetBucketName)
cursor := bucket.Cursor()
for ok := cursor.First(); ok; ok = cursor.Next() {
hash, err := daghash.NewHash(cursor.Key())
if err != nil {
return err
}
ms, err := deserializeMultiset(bytes.NewReader(cursor.Value()))
if err != nil {
return err
}
store.loaded[*hash] = ms
}
return nil
}
// dbStoreMultiset stores the multiset data to the database.
func (store *multisetStore) dbStoreMultiset(dbTx database.Tx, blockHash *daghash.Hash, serializedMS []byte) error {
bucket := dbTx.Metadata().Bucket(multisetBucketName)
if bucket.Get(blockHash[:]) != nil {
return errors.Errorf("Can't override an existing multiset database entry for block %s", blockHash)
}
return bucket.Put(blockHash[:], serializedMS)
}

View File

@ -173,7 +173,7 @@ func GetVirtualFromParentsForTest(dag *BlockDAG, parentHashes []*daghash.Hash) (
}
virtual := newVirtualBlock(dag, parents)
pastUTXO, _, err := dag.pastUTXO(&virtual.blockNode)
pastUTXO, _, _, err := dag.pastUTXO(&virtual.blockNode)
if err != nil {
return nil, err
}

View File

@ -2,31 +2,24 @@ package blockdag
import (
"bytes"
"github.com/golang/groupcache/lru"
"github.com/kaspanet/kaspad/ecc"
"github.com/kaspanet/kaspad/util/daghash"
"github.com/kaspanet/kaspad/wire"
)
const ecmhCacheSize = 4_000_000
var (
utxoToECMHCache = lru.New(ecmhCacheSize)
)
func utxoMultiset(entry *UTXOEntry, outpoint *wire.Outpoint) (*ecc.Multiset, error) {
func addUTXOToMultiset(ms *ecc.Multiset, entry *UTXOEntry, outpoint *wire.Outpoint) (*ecc.Multiset, error) {
w := &bytes.Buffer{}
err := serializeUTXO(w, entry, outpoint)
if err != nil {
return nil, err
}
serializedUTXO := w.Bytes()
utxoHash := daghash.DoubleHashH(serializedUTXO)
if cachedMSPoint, ok := utxoToECMHCache.Get(utxoHash); ok {
return cachedMSPoint.(*ecc.Multiset), nil
}
msPoint := ecc.NewMultiset(ecc.S256()).Add(serializedUTXO)
utxoToECMHCache.Add(utxoHash, msPoint)
return msPoint, nil
return ms.Add(w.Bytes()), nil
}
func removeUTXOFromMultiset(ms *ecc.Multiset, entry *UTXOEntry, outpoint *wire.Outpoint) (*ecc.Multiset, error) {
w := &bytes.Buffer{}
err := serializeUTXO(w, entry, outpoint)
if err != nil {
return nil, err
}
return ms.Remove(w.Bytes()), nil
}

View File

@ -8,8 +8,6 @@ import (
"github.com/pkg/errors"
)
var multisetPointSize = 32
type blockUTXODiffData struct {
diff *UTXODiff
diffChild *blockNode

View File

@ -2,15 +2,11 @@ package blockdag
import (
"bytes"
"encoding/binary"
"github.com/kaspanet/kaspad/util/binaryserializer"
"github.com/pkg/errors"
"io"
"math/big"
"github.com/kaspanet/kaspad/ecc"
"github.com/kaspanet/kaspad/util/daghash"
"github.com/kaspanet/kaspad/wire"
"github.com/pkg/errors"
"io"
)
// serializeBlockUTXODiffData serializes diff data in the following format:
@ -82,9 +78,7 @@ func (diffStore *utxoDiffStore) deserializeBlockUTXODiffData(serializedDiffData
}
func deserializeUTXODiff(r io.Reader) (*UTXODiff, error) {
diff := &UTXODiff{
useMultiset: true,
}
diff := &UTXODiff{}
var err error
diff.toAdd, err = deserializeUTXOCollection(r)
@ -97,11 +91,6 @@ func deserializeUTXODiff(r io.Reader) (*UTXODiff, error) {
return nil, err
}
diff.diffMultiset, err = deserializeMultiset(r)
if err != nil {
return nil, err
}
return diff, nil
}
@ -134,31 +123,9 @@ func deserializeUTXO(r io.Reader) (*UTXOEntry, *wire.Outpoint, error) {
return utxoEntry, outpoint, nil
}
// deserializeMultiset deserializes an EMCH multiset.
// See serializeMultiset for more details.
func deserializeMultiset(r io.Reader) (*ecc.Multiset, error) {
xBytes := make([]byte, multisetPointSize)
yBytes := make([]byte, multisetPointSize)
err := binary.Read(r, byteOrder, xBytes)
if err != nil {
return nil, err
}
err = binary.Read(r, byteOrder, yBytes)
if err != nil {
return nil, err
}
var x, y big.Int
x.SetBytes(xBytes)
y.SetBytes(yBytes)
return ecc.NewMultisetFromPoint(ecc.S256(), &x, &y), nil
}
// serializeUTXODiff serializes UTXODiff by serializing
// UTXODiff.toAdd, UTXODiff.toRemove and UTXODiff.Multiset one after the other.
func serializeUTXODiff(w io.Writer, diff *UTXODiff) error {
if !diff.useMultiset {
return errors.New("Cannot serialize a UTXO diff without a multiset")
}
err := serializeUTXOCollection(w, diff.toAdd)
if err != nil {
return err
@ -168,10 +135,7 @@ func serializeUTXODiff(w io.Writer, diff *UTXODiff) error {
if err != nil {
return err
}
err = serializeMultiset(w, diff.diffMultiset)
if err != nil {
return err
}
return nil
}
@ -192,29 +156,6 @@ func serializeUTXOCollection(w io.Writer, collection utxoCollection) error {
return nil
}
// serializeMultiset serializes an ECMH multiset. The serialization
// is done by taking the (x,y) coordinnates of the multiset point and
// padding each one of them with 32 byte (it'll be 32 byte in most
// cases anyway except one of the coordinates is zero) and writing
// them one after the other.
func serializeMultiset(w io.Writer, ms *ecc.Multiset) error {
x, y := ms.Point()
xBytes := make([]byte, multisetPointSize)
copy(xBytes, x.Bytes())
yBytes := make([]byte, multisetPointSize)
copy(yBytes, y.Bytes())
err := binary.Write(w, byteOrder, xBytes)
if err != nil {
return err
}
err = binary.Write(w, byteOrder, yBytes)
if err != nil {
return err
}
return nil
}
// serializeUTXO serializes a utxo entry-outpoint pair
func serializeUTXO(w io.Writer, entry *UTXOEntry, outpoint *wire.Outpoint) error {
err := serializeOutpoint(w, outpoint)

View File

@ -153,29 +153,16 @@ func (uc utxoCollection) clone() utxoCollection {
// UTXODiff represents a diff between two UTXO Sets.
type UTXODiff struct {
toAdd utxoCollection
toRemove utxoCollection
diffMultiset *ecc.Multiset
useMultiset bool
toAdd utxoCollection
toRemove utxoCollection
}
// NewUTXODiffWithoutMultiset creates a new, empty utxoDiff
// NewUTXODiff creates a new, empty utxoDiff
// without a multiset.
func NewUTXODiffWithoutMultiset() *UTXODiff {
return &UTXODiff{
toAdd: utxoCollection{},
toRemove: utxoCollection{},
useMultiset: false,
}
}
// NewUTXODiff creates a new, empty utxoDiff.
func NewUTXODiff() *UTXODiff {
return &UTXODiff{
toAdd: utxoCollection{},
toRemove: utxoCollection{},
useMultiset: true,
diffMultiset: ecc.NewMultiset(ecc.S256()),
toAdd: utxoCollection{},
toRemove: utxoCollection{},
}
}
@ -209,9 +196,8 @@ func NewUTXODiff() *UTXODiff {
// diffFrom results in the UTXO being added to toAdd
func (d *UTXODiff) diffFrom(other *UTXODiff) (*UTXODiff, error) {
result := UTXODiff{
toAdd: make(utxoCollection, len(d.toRemove)+len(other.toAdd)),
toRemove: make(utxoCollection, len(d.toAdd)+len(other.toRemove)),
useMultiset: d.useMultiset,
toAdd: make(utxoCollection, len(d.toRemove)+len(other.toAdd)),
toRemove: make(utxoCollection, len(d.toAdd)+len(other.toRemove)),
}
// Note that the following cases are not accounted for, as they are impossible
@ -293,11 +279,6 @@ func (d *UTXODiff) diffFrom(other *UTXODiff) (*UTXODiff, error) {
}
}
if d.useMultiset {
// Create a new diffMultiset as the subtraction of the two diffs.
result.diffMultiset = other.diffMultiset.Subtract(d.diffMultiset)
}
return &result, nil
}
@ -343,11 +324,6 @@ func (d *UTXODiff) withDiffInPlace(diff *UTXODiff) error {
d.toAdd.add(outpoint, entryToAdd)
}
// Apply diff.diffMultiset to d.diffMultiset
if d.useMultiset {
d.diffMultiset = d.diffMultiset.Union(diff.diffMultiset)
}
return nil
}
@ -367,12 +343,8 @@ func (d *UTXODiff) WithDiff(diff *UTXODiff) (*UTXODiff, error) {
// clone returns a clone of this utxoDiff
func (d *UTXODiff) clone() *UTXODiff {
clone := &UTXODiff{
toAdd: d.toAdd.clone(),
toRemove: d.toRemove.clone(),
useMultiset: d.useMultiset,
}
if d.useMultiset {
clone.diffMultiset = d.diffMultiset.Clone()
toAdd: d.toAdd.clone(),
toRemove: d.toRemove.clone(),
}
return clone
}
@ -389,14 +361,6 @@ func (d *UTXODiff) AddEntry(outpoint wire.Outpoint, entry *UTXOEntry) error {
} else {
d.toAdd.add(outpoint, entry)
}
if d.useMultiset {
newMs, err := addUTXOToMultiset(d.diffMultiset, entry, &outpoint)
if err != nil {
return err
}
d.diffMultiset = newMs
}
return nil
}
@ -412,21 +376,10 @@ func (d *UTXODiff) RemoveEntry(outpoint wire.Outpoint, entry *UTXOEntry) error {
} else {
d.toRemove.add(outpoint, entry)
}
if d.useMultiset {
newMs, err := removeUTXOFromMultiset(d.diffMultiset, entry, &outpoint)
if err != nil {
return err
}
d.diffMultiset = newMs
}
return nil
}
func (d UTXODiff) String() string {
if d.useMultiset {
return fmt.Sprintf("toAdd: %s; toRemove: %s, Multiset-Hash: %s", d.toAdd, d.toRemove, d.diffMultiset.Hash())
}
return fmt.Sprintf("toAdd: %s; toRemove: %s", d.toAdd, d.toRemove)
}
@ -451,8 +404,6 @@ type UTXOSet interface {
AddTx(tx *wire.MsgTx, blockBlueScore uint64) (ok bool, err error)
clone() UTXOSet
Get(outpoint wire.Outpoint) (*UTXOEntry, bool)
Multiset() *ecc.Multiset
WithTransactions(transactions []*wire.MsgTx, blockBlueScore uint64, ignoreDoubleSpends bool) (UTXOSet, error)
}
// diffFromTx is a common implementation for diffFromTx, that works
@ -522,14 +473,12 @@ func diffFromAcceptedTx(u UTXOSet, tx *wire.MsgTx, acceptingBlueScore uint64) (*
// FullUTXOSet represents a full list of transaction outputs and their values
type FullUTXOSet struct {
utxoCollection
UTXOMultiset *ecc.Multiset
}
// NewFullUTXOSet creates a new utxoSet with full list of transaction outputs and their values
func NewFullUTXOSet() *FullUTXOSet {
return &FullUTXOSet{
utxoCollection: utxoCollection{},
UTXOMultiset: ecc.NewMultiset(ecc.S256()),
}
}
@ -545,7 +494,6 @@ func newFullUTXOSetFromUTXOCollection(collection utxoCollection) (*FullUTXOSet,
}
return &FullUTXOSet{
utxoCollection: collection,
UTXOMultiset: multiset,
}, nil
}
@ -582,22 +530,14 @@ func (fus *FullUTXOSet) AddTx(tx *wire.MsgTx, blueScore uint64) (isAccepted bool
}
for _, txIn := range tx.TxIn {
outpoint := *wire.NewOutpoint(&txIn.PreviousOutpoint.TxID, txIn.PreviousOutpoint.Index)
err := fus.removeAndUpdateMultiset(outpoint)
if err != nil {
return false, err
}
fus.remove(txIn.PreviousOutpoint)
}
}
for i, txOut := range tx.TxOut {
outpoint := *wire.NewOutpoint(tx.TxID(), uint32(i))
entry := NewUTXOEntry(txOut, isCoinbase, blueScore)
err := fus.addAndUpdateMultiset(outpoint, entry)
if err != nil {
return false, err
}
fus.add(outpoint, entry)
}
return true, nil
@ -626,7 +566,7 @@ func (fus *FullUTXOSet) diffFromAcceptedTx(tx *wire.MsgTx, acceptingBlueScore ui
// clone returns a clone of this utxoSet
func (fus *FullUTXOSet) clone() UTXOSet {
return &FullUTXOSet{utxoCollection: fus.utxoCollection.clone(), UTXOMultiset: fus.UTXOMultiset.Clone()}
return &FullUTXOSet{utxoCollection: fus.utxoCollection.clone()}
}
// Get returns the UTXOEntry associated with the given Outpoint, and a boolean indicating if such entry was found
@ -635,55 +575,6 @@ func (fus *FullUTXOSet) Get(outpoint wire.Outpoint) (*UTXOEntry, bool) {
return utxoEntry, ok
}
// Multiset returns the ecmh-Multiset of this utxoSet
func (fus *FullUTXOSet) Multiset() *ecc.Multiset {
return fus.UTXOMultiset
}
// addAndUpdateMultiset adds a UTXOEntry to this utxoSet and updates its multiset accordingly
func (fus *FullUTXOSet) addAndUpdateMultiset(outpoint wire.Outpoint, entry *UTXOEntry) error {
fus.add(outpoint, entry)
newMs, err := addUTXOToMultiset(fus.UTXOMultiset, entry, &outpoint)
if err != nil {
return err
}
fus.UTXOMultiset = newMs
return nil
}
// removeAndUpdateMultiset removes a UTXOEntry from this utxoSet and updates its multiset accordingly
func (fus *FullUTXOSet) removeAndUpdateMultiset(outpoint wire.Outpoint) error {
entry, ok := fus.Get(outpoint)
if !ok {
return errors.Errorf("Couldn't find outpoint %s", outpoint)
}
fus.remove(outpoint)
var err error
newMs, err := removeUTXOFromMultiset(fus.UTXOMultiset, entry, &outpoint)
if err != nil {
return err
}
fus.UTXOMultiset = newMs
return nil
}
// WithTransactions returns a new UTXO Set with the added transactions.
//
// This function MUST be called with the DAG lock held.
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, blockBlueScore)
if err != nil {
return nil, err
}
if !ignoreDoubleSpends && !isAccepted {
return nil, errors.Errorf("Transaction %s is not valid with the current UTXO set", tx.TxID())
}
}
return UTXOSet(diffSet), nil
}
// DiffUTXOSet represents a utxoSet with a base fullUTXOSet and a UTXODiff
type DiffUTXOSet struct {
base *FullUTXOSet
@ -744,12 +635,11 @@ func (dus *DiffUTXOSet) AddTx(tx *wire.MsgTx, blockBlueScore uint64) (bool, erro
func (dus *DiffUTXOSet) appendTx(tx *wire.MsgTx, blockBlueScore uint64, isCoinbase bool) error {
if !isCoinbase {
for _, txIn := range tx.TxIn {
outpoint := *wire.NewOutpoint(&txIn.PreviousOutpoint.TxID, txIn.PreviousOutpoint.Index)
entry, ok := dus.Get(outpoint)
entry, ok := dus.Get(txIn.PreviousOutpoint)
if !ok {
return errors.Errorf("Couldn't find entry for outpoint %s", outpoint)
return errors.Errorf("Couldn't find entry for outpoint %s", txIn.PreviousOutpoint)
}
err := dus.UTXODiff.RemoveEntry(outpoint, entry)
err := dus.UTXODiff.RemoveEntry(txIn.PreviousOutpoint, entry)
if err != nil {
return err
}
@ -795,16 +685,7 @@ func (dus *DiffUTXOSet) meldToBase() error {
for outpoint, utxoEntry := range dus.UTXODiff.toAdd {
dus.base.add(outpoint, utxoEntry)
}
if dus.UTXODiff.useMultiset {
dus.base.UTXOMultiset = dus.base.UTXOMultiset.Union(dus.UTXODiff.diffMultiset)
}
if dus.UTXODiff.useMultiset {
dus.UTXODiff = NewUTXODiff()
} else {
dus.UTXODiff = NewUTXODiffWithoutMultiset()
}
dus.UTXODiff = NewUTXODiff()
return nil
}
@ -819,7 +700,7 @@ func (dus *DiffUTXOSet) diffFromAcceptedTx(tx *wire.MsgTx, acceptingBlueScore ui
}
func (dus *DiffUTXOSet) String() string {
return fmt.Sprintf("{Base: %s, To Add: %s, To Remove: %s, Multiset-Hash:%s}", dus.base, dus.UTXODiff.toAdd, dus.UTXODiff.toRemove, dus.Multiset().Hash())
return fmt.Sprintf("{Base: %s, To Add: %s, To Remove: %s}", dus.base, dus.UTXODiff.toAdd, dus.UTXODiff.toRemove)
}
// clone returns a clone of this UTXO Set
@ -844,42 +725,3 @@ func (dus *DiffUTXOSet) Get(outpoint wire.Outpoint) (*UTXOEntry, bool) {
txOut, ok := dus.UTXODiff.toAdd.get(outpoint)
return txOut, ok
}
// Multiset returns the ecmh-Multiset of this utxoSet
func (dus *DiffUTXOSet) Multiset() *ecc.Multiset {
return dus.base.UTXOMultiset.Union(dus.UTXODiff.diffMultiset)
}
// WithTransactions returns a new UTXO Set with the added transactions.
//
// If dus.UTXODiff.useMultiset is true, this function MUST be
// called with the DAG lock held.
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, blockBlueScore)
if err != nil {
return nil, err
}
if !ignoreDoubleSpends && !isAccepted {
return nil, errors.Errorf("Transaction %s is not valid with the current UTXO set", tx.TxID())
}
}
return UTXOSet(diffSet), nil
}
func addUTXOToMultiset(ms *ecc.Multiset, entry *UTXOEntry, outpoint *wire.Outpoint) (*ecc.Multiset, error) {
utxoMS, err := utxoMultiset(entry, outpoint)
if err != nil {
return nil, err
}
return ms.Union(utxoMS), nil
}
func removeUTXOFromMultiset(ms *ecc.Multiset, entry *UTXOEntry, outpoint *wire.Outpoint) (*ecc.Multiset, error) {
utxoMS, err := utxoMultiset(entry, outpoint)
if err != nil {
return nil, err
}
return ms.Subtract(utxoMS), nil
}

View File

@ -7,7 +7,6 @@ import (
"github.com/kaspanet/kaspad/util/subnetworkid"
"github.com/kaspanet/kaspad/ecc"
"github.com/kaspanet/kaspad/util/daghash"
"github.com/kaspanet/kaspad/wire"
)
@ -80,49 +79,40 @@ func TestUTXODiff(t *testing.T) {
utxoEntry0 := NewUTXOEntry(&wire.TxOut{ScriptPubKey: []byte{}, Value: 10}, true, 0)
utxoEntry1 := NewUTXOEntry(&wire.TxOut{ScriptPubKey: []byte{}, Value: 20}, false, 1)
for i := 0; i < 2; i++ {
withMultiset := i == 0
// Test utxoDiff creation
var diff *UTXODiff
if withMultiset {
diff = NewUTXODiff()
} else {
diff = NewUTXODiffWithoutMultiset()
}
if len(diff.toAdd) != 0 || len(diff.toRemove) != 0 {
t.Errorf("new diff is not empty")
}
// Test utxoDiff creation
err := diff.AddEntry(outpoint0, utxoEntry0)
if err != nil {
t.Fatalf("error adding entry to utxo diff: %s", err)
}
diff := NewUTXODiff()
err = diff.RemoveEntry(outpoint1, utxoEntry1)
if err != nil {
t.Fatalf("error adding entry to utxo diff: %s", err)
}
if len(diff.toAdd) != 0 || len(diff.toRemove) != 0 {
t.Errorf("new diff is not empty")
}
// Test utxoDiff cloning
clonedDiff := diff.clone()
if clonedDiff == diff {
t.Errorf("cloned diff is reference-equal to the original")
}
if !reflect.DeepEqual(clonedDiff, diff) {
t.Errorf("cloned diff not equal to the original"+
"Original: \"%v\", cloned: \"%v\".", diff, clonedDiff)
}
err := diff.AddEntry(outpoint0, utxoEntry0)
if err != nil {
t.Fatalf("error adding entry to utxo diff: %s", err)
}
// Test utxoDiff string representation
expectedDiffString := "toAdd: [ (0000000000000000000000000000000000000000000000000000000000000000, 0) => 10, blueScore: 0 ]; toRemove: [ (1111111111111111111111111111111111111111111111111111111111111111, 0) => 20, blueScore: 1 ]"
if withMultiset {
expectedDiffString = "toAdd: [ (0000000000000000000000000000000000000000000000000000000000000000, 0) => 10, blueScore: 0 ]; toRemove: [ (1111111111111111111111111111111111111111111111111111111111111111, 0) => 20, blueScore: 1 ], Multiset-Hash: 75cb5bbca4e52a9e478dd3c5af3c856ed0f61b848088014c8c52e70432233a57"
}
diffString := clonedDiff.String()
if diffString != expectedDiffString {
t.Errorf("unexpected diff string. "+
"Expected: \"%s\", got: \"%s\".", expectedDiffString, diffString)
}
err = diff.RemoveEntry(outpoint1, utxoEntry1)
if err != nil {
t.Fatalf("error adding entry to utxo diff: %s", err)
}
// Test utxoDiff cloning
clonedDiff := diff.clone()
if clonedDiff == diff {
t.Errorf("cloned diff is reference-equal to the original")
}
if !reflect.DeepEqual(clonedDiff, diff) {
t.Errorf("cloned diff not equal to the original"+
"Original: \"%v\", cloned: \"%v\".", diff, clonedDiff)
}
// Test utxoDiff string representation
expectedDiffString := "toAdd: [ (0000000000000000000000000000000000000000000000000000000000000000, 0) => 10, blueScore: 0 ]; toRemove: [ (1111111111111111111111111111111111111111111111111111111111111111, 0) => 20, blueScore: 1 ]"
diffString := clonedDiff.String()
if diffString != expectedDiffString {
t.Errorf("unexpected diff string. "+
"Expected: \"%s\", got: \"%s\".", expectedDiffString, diffString)
}
}
@ -542,157 +532,101 @@ func TestUTXODiffRules(t *testing.T) {
}
for _, test := range tests {
this := addMultisetToDiff(t, test.this)
other := addMultisetToDiff(t, test.other)
expectedDiffFromResult := addMultisetToDiff(t, test.expectedDiffFromResult)
expectedWithDiffResult := addMultisetToDiff(t, test.expectedWithDiffResult)
// diffFrom from this to other
diffResult, err := this.diffFrom(other)
// diffFrom from test.this to test.other
diffResult, err := test.this.diffFrom(test.other)
// Test whether diffFrom returned an error
isDiffFromOk := err == nil
expectedIsDiffFromOk := expectedDiffFromResult != nil
expectedIsDiffFromOk := test.expectedDiffFromResult != nil
if isDiffFromOk != expectedIsDiffFromOk {
t.Errorf("unexpected diffFrom error in test \"%s\". "+
"Expected: \"%t\", got: \"%t\".", test.name, expectedIsDiffFromOk, isDiffFromOk)
}
// If not error, test the diffFrom result
if isDiffFromOk && !expectedDiffFromResult.equal(diffResult) {
if isDiffFromOk && !test.expectedDiffFromResult.equal(diffResult) {
t.Errorf("unexpected diffFrom result in test \"%s\". "+
"Expected: \"%v\", got: \"%v\".", test.name, expectedDiffFromResult, diffResult)
"Expected: \"%v\", got: \"%v\".", test.name, test.expectedDiffFromResult, diffResult)
}
// Make sure that WithDiff after diffFrom results in the original other
// Make sure that WithDiff after diffFrom results in the original test.other
if isDiffFromOk {
otherResult, err := this.WithDiff(diffResult)
otherResult, err := test.this.WithDiff(diffResult)
if err != nil {
t.Errorf("WithDiff unexpectedly failed in test \"%s\": %s", test.name, err)
}
if !other.equal(otherResult) {
if !test.other.equal(otherResult) {
t.Errorf("unexpected WithDiff result in test \"%s\". "+
"Expected: \"%v\", got: \"%v\".", test.name, other, otherResult)
"Expected: \"%v\", got: \"%v\".", test.name, test.other, otherResult)
}
}
// WithDiff from this to other
withDiffResult, err := this.WithDiff(other)
// WithDiff from test.this to test.other
withDiffResult, err := test.this.WithDiff(test.other)
// Test whether WithDiff returned an error
isWithDiffOk := err == nil
expectedIsWithDiffOk := expectedWithDiffResult != nil
expectedIsWithDiffOk := test.expectedWithDiffResult != nil
if isWithDiffOk != expectedIsWithDiffOk {
t.Errorf("unexpected WithDiff error in test \"%s\". "+
"Expected: \"%t\", got: \"%t\".", test.name, expectedIsWithDiffOk, isWithDiffOk)
}
// If not error, test the WithDiff result
if isWithDiffOk && !withDiffResult.equal(expectedWithDiffResult) {
if isWithDiffOk && !withDiffResult.equal(test.expectedWithDiffResult) {
t.Errorf("unexpected WithDiff result in test \"%s\". "+
"Expected: \"%v\", got: \"%v\".", test.name, expectedWithDiffResult, withDiffResult)
"Expected: \"%v\", got: \"%v\".", test.name, test.expectedWithDiffResult, withDiffResult)
}
// Repeat WithDiff check this time using withDiffInPlace
thisClone := this.clone()
err = thisClone.withDiffInPlace(other)
// Repeat WithDiff check test.this time using withDiffInPlace
thisClone := test.this.clone()
err = thisClone.withDiffInPlace(test.other)
// Test whether withDiffInPlace returned an error
isWithDiffInPlaceOk := err == nil
expectedIsWithDiffInPlaceOk := expectedWithDiffResult != nil
expectedIsWithDiffInPlaceOk := test.expectedWithDiffResult != nil
if isWithDiffInPlaceOk != expectedIsWithDiffInPlaceOk {
t.Errorf("unexpected withDiffInPlace error in test \"%s\". "+
"Expected: \"%t\", got: \"%t\".", test.name, expectedIsWithDiffInPlaceOk, isWithDiffInPlaceOk)
}
// If not error, test the withDiffInPlace result
if isWithDiffInPlaceOk && !thisClone.equal(expectedWithDiffResult) {
if isWithDiffInPlaceOk && !thisClone.equal(test.expectedWithDiffResult) {
t.Errorf("unexpected withDiffInPlace result in test \"%s\". "+
"Expected: \"%v\", got: \"%v\".", test.name, expectedWithDiffResult, thisClone)
"Expected: \"%v\", got: \"%v\".", test.name, test.expectedWithDiffResult, thisClone)
}
// Make sure that diffFrom after WithDiff results in the original other
// Make sure that diffFrom after WithDiff results in the original test.other
if isWithDiffOk {
otherResult, err := this.diffFrom(withDiffResult)
otherResult, err := test.this.diffFrom(withDiffResult)
if err != nil {
t.Errorf("diffFrom unexpectedly failed in test \"%s\": %s", test.name, err)
}
if !other.equal(otherResult) {
if !test.other.equal(otherResult) {
t.Errorf("unexpected diffFrom result in test \"%s\". "+
"Expected: \"%v\", got: \"%v\".", test.name, other, otherResult)
"Expected: \"%v\", got: \"%v\".", test.name, test.other, otherResult)
}
}
}
}
func areMultisetsEqual(a *ecc.Multiset, b *ecc.Multiset) bool {
aX, aY := a.Point()
bX, bY := b.Point()
return aX.Cmp(bX) == 0 && aY.Cmp(bY) == 0
}
func (d *UTXODiff) equal(other *UTXODiff) bool {
if d == nil || other == nil {
return d == other
}
return reflect.DeepEqual(d.toAdd, other.toAdd) &&
reflect.DeepEqual(d.toRemove, other.toRemove) &&
areMultisetsEqual(d.diffMultiset, other.diffMultiset)
reflect.DeepEqual(d.toRemove, other.toRemove)
}
func (fus *FullUTXOSet) equal(other *FullUTXOSet) bool {
return reflect.DeepEqual(fus.utxoCollection, other.utxoCollection) &&
areMultisetsEqual(fus.UTXOMultiset, other.UTXOMultiset)
return reflect.DeepEqual(fus.utxoCollection, other.utxoCollection)
}
func (dus *DiffUTXOSet) equal(other *DiffUTXOSet) bool {
return dus.base.equal(other.base) && dus.UTXODiff.equal(other.UTXODiff)
}
func addMultisetToDiff(t *testing.T, diff *UTXODiff) *UTXODiff {
if diff == nil {
return nil
}
diffWithMs := NewUTXODiff()
for outpoint, entry := range diff.toAdd {
err := diffWithMs.AddEntry(outpoint, entry)
if err != nil {
t.Fatalf("Error with diffWithMs.AddEntry: %s", err)
}
}
for outpoint, entry := range diff.toRemove {
err := diffWithMs.RemoveEntry(outpoint, entry)
if err != nil {
t.Fatalf("Error with diffWithMs.removeEntry: %s", err)
}
}
return diffWithMs
}
func addMultisetToFullUTXOSet(t *testing.T, fus *FullUTXOSet) *FullUTXOSet {
if fus == nil {
return nil
}
fusWithMs := NewFullUTXOSet()
for outpoint, entry := range fus.utxoCollection {
err := fusWithMs.addAndUpdateMultiset(outpoint, entry)
if err != nil {
t.Fatalf("Error with diffWithMs.AddEntry: %s", err)
}
}
return fusWithMs
}
func addMultisetToDiffUTXOSet(t *testing.T, diffSet *DiffUTXOSet) *DiffUTXOSet {
if diffSet == nil {
return nil
}
diffWithMs := addMultisetToDiff(t, diffSet.UTXODiff)
baseWithMs := addMultisetToFullUTXOSet(t, diffSet.base)
return NewDiffUTXOSet(baseWithMs, diffWithMs)
}
// TestFullUTXOSet makes sure that fullUTXOSet is working as expected.
func TestFullUTXOSet(t *testing.T) {
txID0, _ := daghash.NewTxIDFromStr("0000000000000000000000000000000000000000000000000000000000000000")
@ -703,10 +637,10 @@ func TestFullUTXOSet(t *testing.T) {
txOut1 := &wire.TxOut{ScriptPubKey: []byte{}, Value: 20}
utxoEntry0 := NewUTXOEntry(txOut0, true, 0)
utxoEntry1 := NewUTXOEntry(txOut1, false, 1)
diff := addMultisetToDiff(t, &UTXODiff{
diff := &UTXODiff{
toAdd: utxoCollection{outpoint0: utxoEntry0},
toRemove: utxoCollection{outpoint1: utxoEntry1},
})
}
// Test fullUTXOSet creation
emptySet := NewFullUTXOSet()
@ -735,7 +669,7 @@ func TestFullUTXOSet(t *testing.T) {
} else if isAccepted {
t.Errorf("addTx unexpectedly succeeded")
}
emptySet = addMultisetToFullUTXOSet(t, &FullUTXOSet{utxoCollection: utxoCollection{outpoint0: utxoEntry0}})
emptySet = &FullUTXOSet{utxoCollection: utxoCollection{outpoint0: utxoEntry0}}
if isAccepted, err := emptySet.AddTx(transaction0, 0); err != nil {
t.Errorf("addTx unexpectedly failed. Error: %s", err)
} else if !isAccepted {
@ -767,10 +701,10 @@ func TestDiffUTXOSet(t *testing.T) {
txOut1 := &wire.TxOut{ScriptPubKey: []byte{}, Value: 20}
utxoEntry0 := NewUTXOEntry(txOut0, true, 0)
utxoEntry1 := NewUTXOEntry(txOut1, false, 1)
diff := addMultisetToDiff(t, &UTXODiff{
diff := &UTXODiff{
toAdd: utxoCollection{outpoint0: utxoEntry0},
toRemove: utxoCollection{outpoint1: utxoEntry1},
})
}
// Test diffUTXOSet creation
emptySet := NewDiffUTXOSet(NewFullUTXOSet(), NewUTXODiff())
@ -828,7 +762,7 @@ func TestDiffUTXOSet(t *testing.T) {
toRemove: utxoCollection{},
},
},
expectedString: "{Base: [ ], To Add: [ ], To Remove: [ ], Multiset-Hash:0000000000000000000000000000000000000000000000000000000000000000}",
expectedString: "{Base: [ ], To Add: [ ], To Remove: [ ]}",
expectedCollection: utxoCollection{},
},
{
@ -847,7 +781,7 @@ func TestDiffUTXOSet(t *testing.T) {
toRemove: utxoCollection{},
},
},
expectedString: "{Base: [ ], To Add: [ (0000000000000000000000000000000000000000000000000000000000000000, 0) => 10, blueScore: 0 ], To Remove: [ ], Multiset-Hash:2103b44fd413f5c28b5ad9afe7c3bebf19afae01cc202ea63b2f29d26252948d}",
expectedString: "{Base: [ ], To Add: [ (0000000000000000000000000000000000000000000000000000000000000000, 0) => 10, blueScore: 0 ], To Remove: [ ]}",
expectedCollection: utxoCollection{outpoint0: utxoEntry0},
},
{
@ -860,7 +794,7 @@ func TestDiffUTXOSet(t *testing.T) {
},
},
expectedMeldSet: nil,
expectedString: "{Base: [ ], To Add: [ ], To Remove: [ (0000000000000000000000000000000000000000000000000000000000000000, 0) => 10, blueScore: 0 ], Multiset-Hash:bae77d5370880c238265d07b80bdadbc6085990b2d2543a5f8beb2e3bd99b25b}",
expectedString: "{Base: [ ], To Add: [ ], To Remove: [ (0000000000000000000000000000000000000000000000000000000000000000, 0) => 10, blueScore: 0 ]}",
expectedCollection: utxoCollection{},
expectedMeldToBaseError: "Couldn't remove outpoint 0000000000000000000000000000000000000000000000000000000000000000:0 because it doesn't exist in the DiffUTXOSet base",
},
@ -885,7 +819,7 @@ func TestDiffUTXOSet(t *testing.T) {
toRemove: utxoCollection{},
},
},
expectedString: "{Base: [ (0000000000000000000000000000000000000000000000000000000000000000, 0) => 10, blueScore: 0 ], To Add: [ (1111111111111111111111111111111111111111111111111111111111111111, 0) => 20, blueScore: 1 ], To Remove: [ ], Multiset-Hash:c9932064f2f4ba1940d63b747e4a18053d0b022d66894a2ec1bbf88f74570e93}",
expectedString: "{Base: [ (0000000000000000000000000000000000000000000000000000000000000000, 0) => 10, blueScore: 0 ], To Add: [ (1111111111111111111111111111111111111111111111111111111111111111, 0) => 20, blueScore: 1 ], To Remove: [ ]}",
expectedCollection: utxoCollection{
outpoint0: utxoEntry0,
outpoint1: utxoEntry1,
@ -909,24 +843,21 @@ func TestDiffUTXOSet(t *testing.T) {
toRemove: utxoCollection{},
},
},
expectedString: "{Base: [ (0000000000000000000000000000000000000000000000000000000000000000, 0) => 10, blueScore: 0 ], To Add: [ ], To Remove: [ (0000000000000000000000000000000000000000000000000000000000000000, 0) => 10, blueScore: 0 ], Multiset-Hash:0000000000000000000000000000000000000000000000000000000000000000}",
expectedString: "{Base: [ (0000000000000000000000000000000000000000000000000000000000000000, 0) => 10, blueScore: 0 ], To Add: [ ], To Remove: [ (0000000000000000000000000000000000000000000000000000000000000000, 0) => 10, blueScore: 0 ]}",
expectedCollection: utxoCollection{},
},
}
for _, test := range tests {
diffSet := addMultisetToDiffUTXOSet(t, test.diffSet)
expectedMeldSet := addMultisetToDiffUTXOSet(t, test.expectedMeldSet)
// Test string representation
setString := diffSet.String()
setString := test.diffSet.String()
if setString != test.expectedString {
t.Errorf("unexpected string in test \"%s\". "+
"Expected: \"%s\", got: \"%s\".", test.name, test.expectedString, setString)
}
// Test meldToBase
meldSet := diffSet.clone().(*DiffUTXOSet)
meldSet := test.diffSet.clone().(*DiffUTXOSet)
err := meldSet.meldToBase()
errString := ""
if err != nil {
@ -938,27 +869,27 @@ func TestDiffUTXOSet(t *testing.T) {
if err != nil {
continue
}
if !meldSet.equal(expectedMeldSet) {
if !meldSet.equal(test.expectedMeldSet) {
t.Errorf("unexpected melded set in test \"%s\". "+
"Expected: \"%v\", got: \"%v\".", test.name, expectedMeldSet, meldSet)
"Expected: \"%v\", got: \"%v\".", test.name, test.expectedMeldSet, meldSet)
}
// Test collection
setCollection, err := diffSet.collection()
setCollection, err := test.diffSet.collection()
if err != nil {
t.Errorf("Error getting diffSet collection: %s", err)
t.Errorf("Error getting test.diffSet collection: %s", err)
} else if !reflect.DeepEqual(setCollection, test.expectedCollection) {
t.Errorf("unexpected set collection in test \"%s\". "+
"Expected: \"%v\", got: \"%v\".", test.name, test.expectedCollection, setCollection)
}
// Test cloning
clonedSet := diffSet.clone().(*DiffUTXOSet)
if !reflect.DeepEqual(clonedSet, diffSet) {
clonedSet := test.diffSet.clone().(*DiffUTXOSet)
if !reflect.DeepEqual(clonedSet, test.diffSet) {
t.Errorf("unexpected set clone in test \"%s\". "+
"Expected: \"%v\", got: \"%v\".", test.name, diffSet, clonedSet)
"Expected: \"%v\", got: \"%v\".", test.name, test.diffSet, clonedSet)
}
if clonedSet == diffSet {
if clonedSet == test.diffSet {
t.Errorf("cloned set is reference-equal to the original")
}
}
@ -1159,10 +1090,7 @@ func TestDiffUTXOSet_addTx(t *testing.T) {
testLoop:
for _, test := range tests {
startSet := addMultisetToDiffUTXOSet(t, test.startSet)
expectedSet := addMultisetToDiffUTXOSet(t, test.expectedSet)
diffSet := startSet.clone()
diffSet := test.startSet.clone()
// Apply all transactions to diffSet, in order, with the initial block height startHeight
for i, transaction := range test.toAdd {
@ -1174,18 +1102,18 @@ testLoop:
}
}
// Make sure that the result diffSet equals to the expectedSet
if !diffSet.(*DiffUTXOSet).equal(expectedSet) {
// Make sure that the result diffSet equals to test.expectedSet
if !diffSet.(*DiffUTXOSet).equal(test.expectedSet) {
t.Errorf("unexpected diffSet in test \"%s\". "+
"Expected: \"%v\", got: \"%v\".", test.name, expectedSet, diffSet)
"Expected: \"%v\", got: \"%v\".", test.name, test.expectedSet, diffSet)
}
}
}
func TestDiffFromTx(t *testing.T) {
fus := addMultisetToFullUTXOSet(t, &FullUTXOSet{
fus := &FullUTXOSet{
utxoCollection: utxoCollection{},
})
}
txID0, _ := daghash.NewTxIDFromStr("0000000000000000000000000000000000000000000000000000000000000000")
txIn0 := &wire.TxIn{SignatureScript: []byte{}, PreviousOutpoint: wire.Outpoint{TxID: *txID0, Index: math.MaxUint32}, Sequence: 0}
@ -1241,10 +1169,10 @@ func TestDiffFromTx(t *testing.T) {
}
//Test that we get an error if the outpoint is inside diffUTXOSet's toRemove
diff2 := addMultisetToDiff(t, &UTXODiff{
diff2 := &UTXODiff{
toAdd: utxoCollection{},
toRemove: utxoCollection{},
})
}
dus := NewDiffUTXOSet(fus, diff2)
if isAccepted, err := dus.AddTx(tx, 2); err != nil {
t.Fatalf("AddTx unexpectedly failed. Error: %s", err)
@ -1321,7 +1249,6 @@ func TestUTXOSetAddEntry(t *testing.T) {
}
for _, test := range tests {
expectedUTXODiff := addMultisetToDiff(t, test.expectedUTXODiff)
err := utxoDiff.AddEntry(*test.outpointToAdd, test.utxoEntryToAdd)
errString := ""
if err != nil {
@ -1330,9 +1257,9 @@ func TestUTXOSetAddEntry(t *testing.T) {
if errString != test.expectedError {
t.Fatalf("utxoDiff.AddEntry: unexpected err in test \"%s\". Expected: %s but got: %s", test.name, test.expectedError, err)
}
if err == nil && !utxoDiff.equal(expectedUTXODiff) {
if err == nil && !utxoDiff.equal(test.expectedUTXODiff) {
t.Fatalf("utxoDiff.AddEntry: unexpected utxoDiff in test \"%s\". "+
"Expected: %v, got: %v", test.name, expectedUTXODiff, utxoDiff)
"Expected: %v, got: %v", test.name, test.expectedUTXODiff, utxoDiff)
}
}
}

View File

@ -454,7 +454,7 @@ func (mp *TxPool) HaveTransaction(hash *daghash.TxID) bool {
//
// This function MUST be called with the mempool lock held (for writes).
func (mp *TxPool) removeTransactions(txs []*util.Tx) error {
diff := blockdag.NewUTXODiffWithoutMultiset()
diff := blockdag.NewUTXODiff()
for _, tx := range txs {
txID := tx.ID()
@ -502,7 +502,7 @@ func (mp *TxPool) removeTransaction(tx *util.Tx, removeDependants bool, restoreI
return nil
}
diff := blockdag.NewUTXODiffWithoutMultiset()
diff := blockdag.NewUTXODiff()
err := mp.removeTransactionWithDiff(tx, diff, restoreInputs)
if err != nil {
return err
@ -1353,7 +1353,7 @@ func (mp *TxPool) HandleNewBlock(block *util.Block, txChan chan NewBlockMsg) err
// transactions until they are mined into a block.
func New(cfg *Config) *TxPool {
virtualUTXO := cfg.DAG.UTXOSet()
mpUTXO := blockdag.NewDiffUTXOSet(virtualUTXO, blockdag.NewUTXODiffWithoutMultiset())
mpUTXO := blockdag.NewDiffUTXOSet(virtualUTXO, blockdag.NewUTXODiff())
return &TxPool{
cfg: *cfg,
pool: make(map[daghash.TxID]*TxDesc),

View File

@ -254,7 +254,7 @@ func (tc *testContext) mineTransactions(transactions []*util.Tx, numberOfBlocks
if i == 0 {
blockTxs = msgTxs
}
block, err := mining.PrepareBlockForTest(tc.harness.txPool.cfg.DAG, tc.harness.txPool.cfg.DAGParams, tc.harness.txPool.cfg.DAG.TipHashes(), blockTxs, true)
block, err := mining.PrepareBlockForTest(tc.harness.txPool.cfg.DAG, tc.harness.txPool.cfg.DAGParams, tc.harness.txPool.cfg.DAG.TipHashes(), blockTxs, false)
if err != nil {
tc.t.Fatalf("PrepareBlockForTest: %s", err)
}

View File

@ -183,11 +183,11 @@ func NewBlkTmplGenerator(policy *Policy, params *dagconfig.Params,
// | transactions (while block size | |
// | <= policy.BlockMinSize) | |
// ----------------------------------- --
func (g *BlkTmplGenerator) NewBlockTemplate(payToAddress util.Address) (*BlockTemplate, error) {
func (g *BlkTmplGenerator) NewBlockTemplate(payToAddress util.Address, extraNonce uint64) (*BlockTemplate, error) {
g.dag.Lock()
defer g.dag.Unlock()
txsForBlockTemplate, err := g.selectTxs(payToAddress)
txsForBlockTemplate, err := g.selectTxs(payToAddress, extraNonce)
if err != nil {
return nil, errors.Errorf("failed to select transactions: %s", err)
}
@ -219,15 +219,6 @@ func (g *BlkTmplGenerator) NewBlockTemplate(payToAddress util.Address) (*BlockTe
}, nil
}
func (g *BlkTmplGenerator) buildUTXOCommitment(transactions []*wire.MsgTx) (*daghash.Hash, error) {
utxoWithTransactions, err := g.dag.UTXOSet().WithTransactions(transactions, blockdag.UnacceptedBlueScore, false)
if err != nil {
return nil, err
}
return utxoWithTransactions.Multiset().Hash(), nil
}
// UpdateBlockTime updates the timestamp in the header of the passed block to
// the current time while taking into account the median time of the last
// several blocks to ensure the new time is after that time per the DAG
@ -244,59 +235,6 @@ func (g *BlkTmplGenerator) UpdateBlockTime(msgBlock *wire.MsgBlock) error {
return nil
}
// UpdateExtraNonce updates the extra nonce in the coinbase script of the passed
// 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, extraNonce uint64) error {
coinbasePayloadScriptPubKey, _, err := blockdag.DeserializeCoinbasePayload(msgBlock.Transactions[util.CoinbaseTransactionIndex])
if err != nil {
return err
}
coinbasePayloadExtraData, err := blockdag.CoinbasePayloadExtraData(extraNonce, CoinbaseFlags)
if err != nil {
return err
}
coinbasePayload, err := blockdag.SerializeCoinbasePayload(coinbasePayloadScriptPubKey, coinbasePayloadExtraData)
if err != nil {
return err
}
if len(coinbasePayload) > blockdag.MaxCoinbasePayloadLen {
return errors.Errorf("coinbase transaction script length "+
"of %d is out of range (max: %d)",
len(coinbasePayload),
blockdag.MaxCoinbasePayloadLen)
}
oldCoinbaseTx := msgBlock.Transactions[util.CoinbaseTransactionIndex]
msgBlock.Transactions[util.CoinbaseTransactionIndex] = wire.NewSubnetworkMsgTx(oldCoinbaseTx.Version, oldCoinbaseTx.TxIn, oldCoinbaseTx.TxOut, &oldCoinbaseTx.SubnetworkID, oldCoinbaseTx.Gas, coinbasePayload)
// TODO(davec): A util.Block should use saved in the state to avoid
// recalculating all of the other transaction hashes.
// block.Transactions[util.CoinbaseTransactionIndex].InvalidateCache()
// Recalculate the merkle roots with the updated extra nonce.
block := util.NewBlock(msgBlock)
hashMerkleTree := blockdag.BuildHashMerkleTreeStore(block.Transactions())
msgBlock.Header.HashMerkleRoot = hashMerkleTree.Root()
// buildUTXOCommitment is the only function in UpdateExtraNonce that
// requires the dagLock, and as such we lock and unlock it locally.
utxoCommitment, err := func() (*daghash.Hash, error) {
g.dag.Lock()
defer g.dag.Unlock()
return g.buildUTXOCommitment(msgBlock.Transactions)
}()
if err != nil {
return err
}
msgBlock.Header.UTXOCommitment = utxoCommitment
return nil
}
// TxSource returns the associated transaction source.
//
// This function is safe for concurrent access.

View File

@ -68,13 +68,11 @@ func PrepareBlockForTest(dag *blockdag.BlockDAG, params *dagconfig.Params, paren
return nil, err
}
template, err := blockTemplateGenerator.NewBlockTemplate(OpTrueAddr)
if err != nil {
return nil, err
}
// We create a deterministic extra nonce in order of
// creating deterministic coinbase tx ids.
extraNonce := GenerateDeterministicExtraNonceForTest()
// In order of creating deterministic coinbase tx ids.
err = blockTemplateGenerator.UpdateExtraNonce(template.Block, GenerateDeterministicExtraNonceForTest())
template, err := blockTemplateGenerator.NewBlockTemplate(OpTrueAddr, extraNonce)
if err != nil {
return nil, err
}
@ -106,10 +104,12 @@ 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)
ms, err := dag.NextBlockMultiset(utilTxs)
if err != nil {
return nil, err
}
template.Block.Header.UTXOCommitment = ms.Hash()
}
return template.Block, nil
}

View File

@ -3,7 +3,6 @@ package mining
import (
"github.com/kaspanet/kaspad/blockdag"
"github.com/kaspanet/kaspad/util"
"github.com/kaspanet/kaspad/util/random"
"github.com/kaspanet/kaspad/util/subnetworkid"
"math"
"math/rand"
@ -65,13 +64,13 @@ type txsForBlockTemplate struct {
// Once the sum of probabilities of marked transactions is greater than
// rebalanceThreshold percent of the sum of probabilities of all transactions,
// rebalance.
func (g *BlkTmplGenerator) selectTxs(payToAddress util.Address) (*txsForBlockTemplate, error) {
func (g *BlkTmplGenerator) selectTxs(payToAddress util.Address, extraNonce uint64) (*txsForBlockTemplate, error) {
// Fetch the source transactions.
sourceTxs := g.txSource.MiningDescs()
// Create a new txsForBlockTemplate struct, onto which all selectedTxs
// will be appended.
txsForBlockTemplate, err := g.newTxsForBlockTemplate(payToAddress, sourceTxs)
txsForBlockTemplate, err := g.newTxsForBlockTemplate(payToAddress, extraNonce)
if err != nil {
return nil, err
}
@ -91,7 +90,7 @@ func (g *BlkTmplGenerator) selectTxs(payToAddress util.Address) (*txsForBlockTem
// newTxsForBlockTemplate creates a txsForBlockTemplate and initializes it
// with a coinbase transaction.
func (g *BlkTmplGenerator) newTxsForBlockTemplate(payToAddress util.Address, sourceTxs []*TxDesc) (*txsForBlockTemplate, error) {
func (g *BlkTmplGenerator) newTxsForBlockTemplate(payToAddress util.Address, extraNonce uint64) (*txsForBlockTemplate, error) {
// Create a new txsForBlockTemplate struct. The struct holds the mass,
// the fees, and number of signature operations for each of the selected
// transactions and adds an entry for the coinbase. This allows the code
@ -103,10 +102,6 @@ func (g *BlkTmplGenerator) newTxsForBlockTemplate(payToAddress util.Address, sou
txFees: make([]uint64, 0),
}
extraNonce, err := random.Uint64()
if err != nil {
return nil, err
}
coinbasePayloadExtraData, err := blockdag.CoinbasePayloadExtraData(extraNonce, CoinbaseFlags)
if err != nil {
return nil, err

View File

@ -108,7 +108,6 @@ type GetBlockDAGInfoResult struct {
TipHashes []string `json:"tipHashes"`
Difficulty float64 `json:"difficulty"`
MedianTime int64 `json:"medianTime"`
UTXOCommitment string `json:"utxoCommitment"`
VerificationProgress float64 `json:"verificationProgress,omitempty"`
Pruned bool `json:"pruned"`
PruneHeight uint64 `json:"pruneHeight,omitempty"`

View File

@ -18,15 +18,14 @@ func handleGetBlockDAGInfo(s *Server, cmd interface{}, closeChan <-chan struct{}
dag := s.cfg.DAG
dagInfo := &rpcmodel.GetBlockDAGInfoResult{
DAG: params.Name,
Blocks: dag.BlockCount(),
Headers: dag.BlockCount(),
TipHashes: daghash.Strings(dag.TipHashes()),
Difficulty: getDifficultyRatio(dag.CurrentBits(), params),
MedianTime: dag.CalcPastMedianTime().Unix(),
UTXOCommitment: dag.UTXOCommitment(),
Pruned: false,
Bip9SoftForks: make(map[string]*rpcmodel.Bip9SoftForkDescription),
DAG: params.Name,
Blocks: dag.BlockCount(),
Headers: dag.BlockCount(),
TipHashes: daghash.Strings(dag.TipHashes()),
Difficulty: getDifficultyRatio(dag.CurrentBits(), params),
MedianTime: dag.CalcPastMedianTime().Unix(),
Pruned: false,
Bip9SoftForks: make(map[string]*rpcmodel.Bip9SoftForkDescription),
}
// Finally, query the BIP0009 version bits state for all currently

View File

@ -11,6 +11,7 @@ import (
"github.com/kaspanet/kaspad/txscript"
"github.com/kaspanet/kaspad/util"
"github.com/kaspanet/kaspad/util/daghash"
"github.com/kaspanet/kaspad/util/random"
"github.com/kaspanet/kaspad/wire"
"github.com/pkg/errors"
"math/rand"
@ -616,10 +617,17 @@ func (state *gbtWorkState) updateBlockTemplate(s *Server, useCoinbaseValue bool)
// block template doesn't include the coinbase, so the caller
// will ultimately create their own coinbase which pays to the
// appropriate address(es).
blkTemplate, err := generator.NewBlockTemplate(payAddr)
extraNonce, err := random.Uint64()
if err != nil {
return internalRPCError("Failed to create new block "+
"template: "+err.Error(), "")
return internalRPCError(fmt.Sprintf("Failed to randomize "+
"extra nonce: %s", err.Error()), "")
}
blkTemplate, err := generator.NewBlockTemplate(payAddr, extraNonce)
if err != nil {
return internalRPCError(fmt.Sprintf("Failed to create new block "+
"template: %s", err.Error()), "")
}
template = blkTemplate
msgBlock = template.Block