[NOD-195] Make fee tx implicit (#315)

* [NOD-195] Made fee tx implicit

* [NOD-195] Removed redundant checks for fee transactions

* [NOD-195] Add fee tx data into acceptence data and fee data

* [NOD-195] Fix some tests

* [NOD-195] Update Block100000 with new data

* [NOD-195] Fixed remaining tests

* [NOD-195] Save and load feeTx to/from database

* [NOD-195] Remove DisconnectBlock methods from indexers, since they are not used anywhere

* [NOD-195] Add fee tx to addrindex

* [NOD-195] Don't populate inputs for fee transactions

* [NOD-195] Delete feeTxBucket in removeDAGState

* [NOD-195] Got rid of util.FeeTRansactionIndex
This commit is contained in:
Svarog 2019-06-03 17:30:57 +03:00 committed by Ori Newman
parent 84970a8378
commit ccca580a4b
25 changed files with 158 additions and 232 deletions

View File

@ -9,6 +9,7 @@ import (
"sort"
"time"
"github.com/daglabs/btcd/util"
"github.com/daglabs/btcd/util/daghash"
"github.com/daglabs/btcd/wire"
)
@ -105,6 +106,8 @@ type blockNode struct {
acceptedIDMerkleRoot *daghash.Hash
utxoCommitment *daghash.Hash
feeTransaction *util.Tx
// status is a bitfield representing the validation state of the block. The
// status field, unlike the other fields, may be written to and so should
// only be accessed using the concurrent-safe NodeStatus method on

View File

@ -579,7 +579,7 @@ func (dag *BlockDAG) connectBlock(node *blockNode, block *util.Block, fastAdd bo
return errors.New(newErrString)
}
err = node.validateFeeTransaction(dag, block, txsAcceptanceData)
node.feeTransaction, err = node.buildFeeTransaction(dag, txsAcceptanceData)
if err != nil {
return err
}
@ -648,14 +648,23 @@ func (dag *BlockDAG) saveChangesFromBlock(node *blockNode, block *util.Block, vi
// optional indexes with the block being connected so they can
// update themselves accordingly.
if dag.indexManager != nil {
err := dag.indexManager.ConnectBlock(dbTx, block, dag, txsAcceptanceData, virtualTxsAcceptanceData)
err := dag.indexManager.ConnectBlock(dbTx, block, dag, node.feeTransaction, txsAcceptanceData, virtualTxsAcceptanceData)
if err != nil {
return err
}
}
// Apply the fee data into the database
return dbStoreFeeData(dbTx, block.Hash(), feeData)
err = dbStoreFeeData(dbTx, block.Hash(), feeData)
if err != nil {
return err
}
if err := dbPutFeeTx(dbTx, node.hash, node.feeTransaction); err != nil {
return err
}
return nil
})
if err != nil {
return err
@ -749,7 +758,7 @@ func (dag *BlockDAG) updateFinalityPoint() {
// NextBlockFeeTransaction prepares the fee transaction for the next mined block
//
// This function CAN'T be called with the DAG lock held.
func (dag *BlockDAG) NextBlockFeeTransaction() (*wire.MsgTx, error) {
func (dag *BlockDAG) NextBlockFeeTransaction() (*util.Tx, error) {
dag.dagLock.RLock()
defer dag.dagLock.RUnlock()
@ -759,7 +768,7 @@ func (dag *BlockDAG) NextBlockFeeTransaction() (*wire.MsgTx, error) {
// NextBlockFeeTransactionNoLock prepares the fee transaction for the next mined block
//
// This function MUST be called with the DAG read-lock held
func (dag *BlockDAG) NextBlockFeeTransactionNoLock() (*wire.MsgTx, error) {
func (dag *BlockDAG) NextBlockFeeTransactionNoLock() (*util.Tx, error) {
txsAcceptanceData, err := dag.TxsAcceptedByVirtual()
if err != nil {
return nil, err
@ -962,13 +971,18 @@ func (node *blockNode) applyBlueBlocks(selectedParentUTXO UTXOSet, blueBlocks []
pastUTXO UTXOSet, txsAcceptanceData MultiBlockTxsAcceptanceData, err error) {
pastUTXO = selectedParentUTXO
txsAcceptanceData = MultiBlockTxsAcceptanceData{}
txsAcceptanceData = make(MultiBlockTxsAcceptanceData, len(blueBlocks))
for _, blueBlock := range blueBlocks {
transactions := blueBlock.Transactions()
blockTxsAcceptanceData := make(BlockTxsAcceptanceData, len(transactions))
numTransactions := len(transactions)
isSelectedParent := blueBlock.Hash().IsEqual(node.selectedParent.hash)
for i, tx := range blueBlock.Transactions() {
if isSelectedParent { // if this is selected parent - we will also add the fee tx to acceptance data
numTransactions++
}
blockTxsAcceptanceData := make(BlockTxsAcceptanceData, 0, numTransactions)
for _, tx := range blueBlock.Transactions() {
var isAccepted bool
if isSelectedParent {
isAccepted = true
@ -978,8 +992,13 @@ func (node *blockNode) applyBlueBlocks(selectedParentUTXO UTXOSet, blueBlocks []
return nil, nil, err
}
}
blockTxsAcceptanceData[i] = TxAcceptanceData{Tx: tx, IsAccepted: isAccepted}
blockTxsAcceptanceData = append(blockTxsAcceptanceData, TxAcceptanceData{Tx: tx, IsAccepted: isAccepted})
}
// Add fee tx acceptance data for fee transaction
blockTxsAcceptanceData = append(blockTxsAcceptanceData,
TxAcceptanceData{Tx: node.selectedParent.feeTransaction, IsAccepted: isSelectedParent})
txsAcceptanceData[*blueBlock.Hash()] = blockTxsAcceptanceData
}
@ -1742,7 +1761,7 @@ type IndexManager interface {
// ConnectBlock is invoked when a new block has been connected to the
// DAG.
ConnectBlock(database.Tx, *util.Block, *BlockDAG, MultiBlockTxsAcceptanceData, MultiBlockTxsAcceptanceData) error
ConnectBlock(database.Tx, *util.Block, *BlockDAG, *util.Tx, MultiBlockTxsAcceptanceData, MultiBlockTxsAcceptanceData) error
}
// Config is a descriptor which specifies the blockchain instance configuration.
@ -1862,7 +1881,6 @@ func New(config *Config) (*BlockDAG, error) {
SubnetworkStore: newSubnetworkStore(config.DB),
subnetworkID: config.SubnetworkID,
}
dag.utxoDiffStore = newUTXODiffStore(&dag)
// Initialize the chain state from the passed database. When the db

View File

@ -189,10 +189,10 @@ func TestHaveBlock(t *testing.T) {
{hash: dagconfig.SimNetParams.GenesisHash.String(), want: true},
// Block 3b should be present (as a second child of Block 2).
{hash: "4bb2e2f55fabd67e217126dbc41e7101d0d6058800368c428cd6e397c111ee47", want: true},
{hash: "7689c1facfa089e4afbdf9c7b75bd4375fd3fbf362d083e0d92abe799bde6d7a", want: true},
// Block 100000 should be present (as an orphan).
{hash: "4e530ee9f967de3b2cd47ac5cd00109bb9ed7b0e30a60485c94badad29ecb4ce", want: true},
{hash: "6ea023a55d9b7c11a6bb22c17553efb5d045e3543a308208da0ef854d8ad2a0f", want: true},
// Random hashes should not be available.
{hash: "123", want: false},

View File

@ -60,6 +60,10 @@ var (
// node's local subnetwork ID.
localSubnetworkKeyName = []byte("localsubnetworkidkey")
// feeTxBucketName is the name of the db bucket used to house the
// fee transactions of blocks
feeTxBucketName = []byte("feetxs")
// byteOrder is the preferred byte order used for serializing numeric
// fields for storage in the database.
byteOrder = binary.LittleEndian
@ -335,6 +339,13 @@ func (dag *BlockDAG) createDAGState() error {
if err := dbPutLocalSubnetworkID(dbTx, dag.subnetworkID); err != nil {
return err
}
// Create the bucket that houses the fee transactions.
_, err = meta.CreateBucket(feeTxBucketName)
if err != nil {
return err
}
return nil
})
@ -363,7 +374,7 @@ func (dag *BlockDAG) removeDAGState() error {
return err
}
err = dbTx.Metadata().Delete(utxoSetVersionKeyName)
err = meta.Delete(utxoSetVersionKeyName)
if err != nil {
return err
}
@ -373,7 +384,12 @@ func (dag *BlockDAG) removeDAGState() error {
return err
}
err = dbTx.Metadata().Delete(localSubnetworkKeyName)
err = meta.Delete(localSubnetworkKeyName)
if err != nil {
return err
}
err = meta.DeleteBucket(feeTxBucketName)
if err != nil {
return err
}
@ -482,6 +498,11 @@ func (dag *BlockDAG) initDAGState() error {
dag.blockCount++
}
node.feeTransaction, err = dbFetchFeeTx(dbTx, node.hash)
if err != nil {
return err
}
lastNode = node
i++
}

View File

@ -193,9 +193,9 @@ const (
// ErrBadFeeTransaction indicates that the block's fee transaction is not build as expected
ErrBadFeeTransaction
// ErrMultipleFeeTransactions indicates a block contains more than one
// fee transaction.
ErrMultipleFeeTransactions
// ErrExplicitFeeTransaction indicates a block contains an explicit
// fee transaction, while it should be omitted.
ErrExplicitFeeTransaction
// ErrScriptMalformed indicates a transaction script is malformed in
// some way. For example, it might be longer than the maximum allowed
@ -285,7 +285,7 @@ var errorCodeStrings = map[ErrorCode]string{
ErrBadCoinbaseHeight: "ErrBadCoinbaseHeight",
ErrSecondTxNotFeeTransaction: "ErrSecondTxNotFeeTransaction",
ErrBadFeeTransaction: "ErrBadFeeTransaction",
ErrMultipleFeeTransactions: "ErrMultipleFeeTransactions",
ErrExplicitFeeTransaction: "ErrExplicitFeeTransaction",
ErrScriptMalformed: "ErrScriptMalformed",
ErrScriptValidation: "ErrScriptValidation",
ErrParentBlockUnknown: "ErrParentBlockUnknown",

View File

@ -53,7 +53,7 @@ func TestErrorCodeStringer(t *testing.T) {
{ErrBadCoinbaseHeight, "ErrBadCoinbaseHeight"},
{ErrSecondTxNotFeeTransaction, "ErrSecondTxNotFeeTransaction"},
{ErrBadFeeTransaction, "ErrBadFeeTransaction"},
{ErrMultipleFeeTransactions, "ErrMultipleFeeTransactions"},
{ErrExplicitFeeTransaction, "ErrExplicitFeeTransaction"},
{ErrScriptMalformed, "ErrScriptMalformed"},
{ErrScriptValidation, "ErrScriptValidation"},
{ErrParentBlockUnknown, "ErrParentBlockUnknown"},

View File

@ -121,24 +121,8 @@ func dbFetchFeeData(dbTx database.Tx, blockHash *daghash.Hash) (compactFeeData,
// The following functions deal with building and validating the fee transaction
func (node *blockNode) validateFeeTransaction(dag *BlockDAG, block *util.Block, txsAcceptanceData MultiBlockTxsAcceptanceData) error {
if node.isGenesis() {
return nil
}
expectedFeeTransaction, err := node.buildFeeTransaction(dag, txsAcceptanceData)
if err != nil {
return err
}
if !expectedFeeTransaction.TxHash().IsEqual(block.FeeTransaction().Hash()) {
return ruleError(ErrBadFeeTransaction, "Fee transaction is not built as expected")
}
return nil
}
// buildFeeTransaction returns the expected fee transaction for the current block
func (node *blockNode) buildFeeTransaction(dag *BlockDAG, txsAcceptanceData MultiBlockTxsAcceptanceData) (*wire.MsgTx, error) {
func (node *blockNode) buildFeeTransaction(dag *BlockDAG, txsAcceptanceData MultiBlockTxsAcceptanceData) (*util.Tx, error) {
bluesFeeData, err := node.getBluesFeeData(dag)
if err != nil {
return nil, err
@ -158,7 +142,7 @@ func (node *blockNode) buildFeeTransaction(dag *BlockDAG, txsAcceptanceData Mult
}
}
feeTx := wire.NewNativeMsgTx(wire.TxVersion, txIns, txOuts)
return txsort.Sort(feeTx), nil
return util.NewTx(txsort.Sort(feeTx)), nil
}
// feeInputAndOutputForBlueBlock calculates the input and output that should go into the fee transaction of blueBlock
@ -217,3 +201,34 @@ func feeInputAndOutputForBlueBlock(blueBlock *blockNode, txsAcceptanceData Multi
return txIn, txOut, nil
}
func dbPutFeeTx(dbTx database.Tx, blockHash *daghash.Hash, feeTx *util.Tx) error {
w := &bytes.Buffer{}
if err := feeTx.MsgTx().Serialize(w); err != nil {
return err
}
serialized := w.Bytes()
feeTxBucket := dbTx.Metadata().Bucket(feeTxBucketName)
return feeTxBucket.Put(blockHash[:], serialized)
}
func dbFetchFeeTx(dbTx database.Tx, blockHash *daghash.Hash) (*util.Tx, error) {
feeTxBucket := dbTx.Metadata().Bucket(feeTxBucketName)
serialized := feeTxBucket.Get(blockHash[:])
if serialized == nil {
return nil, fmt.Errorf("No feeTx found in database for block %s", blockHash)
}
r := bytes.NewReader(serialized)
msgTx := &wire.MsgTx{}
err := msgTx.Deserialize(r)
if err != nil {
return nil, err
}
return util.NewTx(msgTx), nil
}

View File

@ -662,13 +662,17 @@ 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, 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
// already been proven on the first transaction in the block is
// a coinbase, and the second one is a fee transaction.
if txIdx > 1 {
func (idx *AddrIndex) indexBlock(data writeIndexData, block *util.Block, dag *blockdag.BlockDAG, feeTx *util.Tx) {
transactions := append(block.Transactions(), feeTx)
for txIdx, tx := range transactions {
// Coinbase and fee txs do not reference any previous txs,
// so skip scanning their inputs.
//
// Since the block is required to have already gone through full
// validation, it has already been proven that the first tx in
// the block is a coinbase
if txIdx > 1 && tx != feeTx {
for _, txIn := range tx.MsgTx().TxIn {
// The UTXO should always have the input since
// the index contract requires it, however, be
@ -694,7 +698,7 @@ func (idx *AddrIndex) indexBlock(data writeIndexData, block *util.Block, dag *bl
//
// This is part of the Indexer interface.
func (idx *AddrIndex) ConnectBlock(dbTx database.Tx, block *util.Block, dag *blockdag.BlockDAG,
_ blockdag.MultiBlockTxsAcceptanceData, _ blockdag.MultiBlockTxsAcceptanceData) error {
feeTx *util.Tx, _ blockdag.MultiBlockTxsAcceptanceData, _ blockdag.MultiBlockTxsAcceptanceData) error {
// The offset and length of the transactions within the serialized
// block.
@ -711,7 +715,7 @@ func (idx *AddrIndex) ConnectBlock(dbTx database.Tx, block *util.Block, dag *blo
// Build all of the address to transaction mappings in a local map.
addrsToTxns := make(writeIndexData)
idx.indexBlock(addrsToTxns, block, dag)
idx.indexBlock(addrsToTxns, block, dag, feeTx)
// Add all of the index entries for each address.
addrIdxBucket := dbTx.Metadata().Bucket(addrIndexKey)
@ -728,28 +732,6 @@ func (idx *AddrIndex) ConnectBlock(dbTx database.Tx, block *util.Block, dag *blo
return nil
}
// DisconnectBlock is invoked by the index manager when a block has been
// disconnected from the main chain. This indexer removes the address mappings
// each transaction in the block involve.
//
// This is part of the Indexer interface.
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, dag)
// Remove all of the index entries for each address.
bucket := dbTx.Metadata().Bucket(addrIndexKey)
for addrKey, txIdxs := range addrsToTxns {
err := dbRemoveAddrIndexEntries(bucket, addrKey, len(txIdxs))
if err != nil {
return err
}
}
return nil
}
// TxRegionsForAddress returns a slice of block regions which identify each
// transaction that involves the passed address according to the specified
// number to skip, number requested, and whether or not the results should be

View File

@ -204,8 +204,8 @@ func storeFilter(dbTx database.Tx, block *util.Block, f *gcs.Filter,
// ConnectBlock is invoked by the index manager when a new block has been
// 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,
_ *blockdag.BlockDAG, _ blockdag.MultiBlockTxsAcceptanceData, _ blockdag.MultiBlockTxsAcceptanceData) error {
func (idx *CfIndex) ConnectBlock(dbTx database.Tx, block *util.Block, _ *blockdag.BlockDAG, _ *util.Tx,
_ blockdag.MultiBlockTxsAcceptanceData, _ blockdag.MultiBlockTxsAcceptanceData) error {
f, err := builder.BuildBasicFilter(block.MsgBlock())
if err != nil {
@ -225,36 +225,6 @@ func (idx *CfIndex) ConnectBlock(dbTx database.Tx, block *util.Block,
return storeFilter(dbTx, block, f, wire.GCSFilterExtended)
}
// DisconnectBlock is invoked by the index manager when a block has been
// 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,
_ *blockdag.BlockDAG) error {
for _, key := range cfIndexKeys {
err := dbDeleteFilterIdxEntry(dbTx, key, block.Hash())
if err != nil {
return err
}
}
for _, key := range cfHeaderKeys {
err := dbDeleteFilterIdxEntry(dbTx, key, block.Hash())
if err != nil {
return err
}
}
for _, key := range cfHashKeys {
err := dbDeleteFilterIdxEntry(dbTx, key, block.Hash())
if err != nil {
return err
}
}
return nil
}
// entryByBlockHash fetches a filter index entry of a particular type
// (eg. filter, filter header, etc) for a filter type and block hash.
func (idx *CfIndex) entryByBlockHash(filterTypeKeys [][]byte,

View File

@ -52,7 +52,8 @@ type Indexer interface {
// ConnectBlock is invoked when the index manager is notified that a new
// block has been connected to the DAG.
ConnectBlock(dbTx database.Tx, block *util.Block, dag *blockdag.BlockDAG, _ blockdag.MultiBlockTxsAcceptanceData, _ blockdag.MultiBlockTxsAcceptanceData) error
ConnectBlock(dbTx database.Tx, block *util.Block, dag *blockdag.BlockDAG, feeTx *util.Tx,
acceptedTxsData blockdag.MultiBlockTxsAcceptanceData, virtualTxsAcceptanceData blockdag.MultiBlockTxsAcceptanceData) error
}
// AssertError identifies an error that indicates an internal code consistency

View File

@ -168,13 +168,13 @@ func (m *Manager) Init(db database.DB, blockDAG *blockdag.BlockDAG, interrupt <-
// checks, and invokes each indexer.
//
// This is part of the blockchain.IndexManager interface.
func (m *Manager) ConnectBlock(dbTx database.Tx, block *util.Block, dag *blockdag.BlockDAG,
func (m *Manager) ConnectBlock(dbTx database.Tx, block *util.Block, dag *blockdag.BlockDAG, feeTx *util.Tx,
txsAcceptanceData blockdag.MultiBlockTxsAcceptanceData, virtualTxsAcceptanceData blockdag.MultiBlockTxsAcceptanceData) 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, dag, txsAcceptanceData, virtualTxsAcceptanceData); err != nil {
if err := index.ConnectBlock(dbTx, block, dag, feeTx, txsAcceptanceData, virtualTxsAcceptanceData); err != nil {
return err
}
}

View File

@ -6,6 +6,7 @@ package indexers
import (
"fmt"
"github.com/daglabs/btcd/blockdag"
"github.com/daglabs/btcd/database"
"github.com/daglabs/btcd/util"
@ -440,7 +441,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, dag *blockdag.BlockDAG,
func (idx *TxIndex) ConnectBlock(dbTx database.Tx, block *util.Block, dag *blockdag.BlockDAG, _ *util.Tx,
acceptedTxsData blockdag.MultiBlockTxsAcceptanceData, virtualTxsAcceptanceData blockdag.MultiBlockTxsAcceptanceData) 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.

View File

@ -5,9 +5,10 @@
package blockdag
import (
"github.com/daglabs/btcd/util/daghash"
"testing"
"github.com/daglabs/btcd/util/daghash"
"github.com/daglabs/btcd/util"
)
@ -25,7 +26,7 @@ func TestMerkle(t *testing.T) {
idMerkleTree := BuildIDMerkleTreeStore(block.Transactions())
calculatedIDMerkleRoot := idMerkleTree.Root()
wantIDMerkleRoot, err := daghash.NewHashFromStr("65308857c92c4e5dd3c5e61b73d6b78a87456b5f8f16b13c1e02c47768a0b881")
wantIDMerkleRoot, err := daghash.NewHashFromStr("dba6ffd023a545ed2611653df48edfc20fe38526cf7a43c8c22f463bd8226e77")
if err != nil {
t.Errorf("BuildIDMerkleTreeStore: unexpected error: %s", err)
}

Binary file not shown.

Binary file not shown.

Binary file not shown.

Binary file not shown.

Binary file not shown.

View File

@ -153,7 +153,7 @@ func CalcBlockSubsidy(height uint64, dagParams *dagconfig.Params) uint64 {
// CheckTransactionSanity performs some preliminary checks on a transaction to
// ensure it is sane. These checks are context free.
func CheckTransactionSanity(tx *util.Tx, subnetworkID *subnetworkid.SubnetworkID, isFeeTransaction bool) error {
func CheckTransactionSanity(tx *util.Tx, subnetworkID *subnetworkid.SubnetworkID) error {
// A transaction must have at least one input.
msgTx := tx.MsgTx()
if len(msgTx.TxIn) == 0 {
@ -509,29 +509,20 @@ func (dag *BlockDAG) checkBlockSanity(block *util.Block, flags BehaviorFlags) er
"block is not a coinbase")
}
isGenesis := block.MsgBlock().Header.IsGenesis()
if !isGenesis && !IsFeeTransaction(transactions[1]) {
return ruleError(ErrSecondTxNotFeeTransaction, "second transaction in "+
"block is not a fee transaction")
}
txOffset := 2
if isGenesis {
txOffset = 1
}
txOffset := 1
// A block must not have more than one coinbase. And transactions must be
// ordered by subnetwork
for i, tx := range transactions[txOffset:] {
if IsCoinBase(tx) {
str := fmt.Sprintf("block contains second coinbase at "+
"index %d", i+2)
"index %d", i+txOffset)
return ruleError(ErrMultipleCoinbases, str)
}
if IsFeeTransaction(tx) {
str := fmt.Sprintf("block contains second fee transaction at "+
"index %d", i+2)
return ruleError(ErrMultipleFeeTransactions, str)
str := fmt.Sprintf("block contains an explicit fee transaction at "+
"index %d", i+txOffset)
return ruleError(ErrExplicitFeeTransaction, str)
}
if subnetworkid.Less(&tx.MsgTx().SubnetworkID, &transactions[i].MsgTx().SubnetworkID) {
return ruleError(ErrTransactionsNotSorted, "transactions must be sorted by subnetwork")
@ -540,9 +531,8 @@ func (dag *BlockDAG) checkBlockSanity(block *util.Block, flags BehaviorFlags) er
// Do some preliminary checks on each transaction to ensure they are
// sane before continuing.
for i, tx := range transactions {
isFeeTransaction := i == util.FeeTransactionIndex
err := CheckTransactionSanity(tx, dag.subnetworkID, isFeeTransaction)
for _, tx := range transactions {
err := CheckTransactionSanity(tx, dag.subnetworkID)
if err != nil {
return err
}
@ -1045,6 +1035,10 @@ func (dag *BlockDAG) checkConnectToPastUTXO(block *blockNode, pastUTXO UTXOSet,
return nil, fmt.Errorf("Error adding tx %s fee to compactFeeFactory: %s", tx.ID(), err)
}
}
// Add a 0 fee for implicit fee transaction
compactFeeFactory.add(0)
feeData, err := compactFeeFactory.data()
if err != nil {
return nil, fmt.Errorf("Error getting bytes of fee data: %s", err)

View File

@ -730,7 +730,7 @@ func TestCheckTransactionSanity(t *testing.T) {
test.extraModificationsFunc(tx)
}
err := CheckTransactionSanity(util.NewTx(tx), &test.nodeSubnetworkID, false)
err := CheckTransactionSanity(util.NewTx(tx), &test.nodeSubnetworkID)
if e := checkRuleError(err, test.expectedErr); e != nil {
t.Errorf("TestCheckTransactionSanity: '%s': %v", test.name, e)
continue
@ -758,21 +758,21 @@ var Block100000 = wire.MsgBlock{
},
},
HashMerkleRoot: &daghash.Hash{
0x30, 0xed, 0xf5, 0xbd, 0xd1, 0x4f, 0x8f, 0xb2,
0x0b, 0x6c, 0x92, 0xac, 0xd2, 0x47, 0xb7, 0xd6,
0x6f, 0x22, 0xfa, 0x60, 0x36, 0x80, 0x99, 0xc3,
0x6e, 0x39, 0x14, 0x9b, 0xcc, 0x1f, 0x31, 0xa9,
0x07, 0x86, 0x69, 0xa4, 0x2c, 0x70, 0x6f, 0x67,
0xc3, 0xaa, 0xe5, 0x4c, 0x1b, 0x3e, 0x12, 0x92,
0x5e, 0x90, 0x80, 0x48, 0x85, 0x88, 0x4b, 0x35,
0xb4, 0x72, 0x53, 0x46, 0xbb, 0xa7, 0x38, 0xe8,
},
AcceptedIDMerkleRoot: &daghash.Hash{
0x8a, 0xb7, 0xd6, 0x73, 0x1b, 0xe6, 0xc5, 0xd3,
0x5d, 0x4e, 0x2c, 0xc9, 0x57, 0x88, 0x30, 0x65,
0x81, 0xb8, 0xa0, 0x68, 0x77, 0xc4, 0x02, 0x1e,
0x3c, 0xb1, 0x16, 0x8f, 0x5f, 0x6b, 0x45, 0x87,
0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00,
0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00,
0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00,
0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00,
},
UTXOCommitment: &daghash.ZeroHash,
Timestamp: time.Unix(0x5c404bc3, 0),
Timestamp: time.Unix(0x5cee92cc, 0),
Bits: 0x207fffff,
Nonce: 0xdffffffffffffff9,
Nonce: 0x00000000,
},
Transactions: []*wire.MsgTx{
{
@ -784,8 +784,7 @@ var Block100000 = wire.MsgBlock{
Index: 0xffffffff,
},
SignatureScript: []byte{
0x02, 0x10, 0x27, 0x08, 0x8f, 0x22, 0xfb, 0x88,
0x45, 0x7b, 0xee, 0xeb, 0x0b, 0x2f, 0x50, 0x32,
0x02, 0x10, 0x27, 0x51, 0x0b, 0x2f, 0x50, 0x32,
0x53, 0x48, 0x2f, 0x62, 0x74, 0x63, 0x64, 0x2f,
},
Sequence: math.MaxUint64,
@ -795,43 +794,15 @@ var Block100000 = wire.MsgBlock{
{
Value: 0x12a05f200, // 5000000000
PkScript: []byte{
0x51,
0xa9, 0x14, 0xda, 0x17, 0x45, 0xe9, 0xb5, 0x49,
0xbd, 0x0b, 0xfa, 0x1a, 0x56, 0x99, 0x71, 0xc7,
0x7e, 0xba, 0x30, 0xcd, 0x5a, 0x4b, 0x87,
},
},
},
LockTime: 0,
SubnetworkID: *subnetworkid.SubnetworkIDNative,
},
{
Version: 1,
TxIn: []*wire.TxIn{
{
PreviousOutPoint: wire.OutPoint{
TxID: daghash.TxID{
0x16, 0x5e, 0x38, 0xe8, 0xb3, 0x91, 0x45, 0x95,
0xd9, 0xc6, 0x41, 0xf3, 0xb8, 0xee, 0xc2, 0xf3,
0x46, 0x11, 0x89, 0x6b, 0x82, 0x1a, 0x68, 0x3b,
0x7a, 0x4e, 0xde, 0xfe, 0x2c, 0x00, 0x00, 0x00,
},
Index: 0xffffffff,
},
Sequence: math.MaxUint64,
},
{
PreviousOutPoint: wire.OutPoint{
TxID: daghash.TxID{
0x4b, 0xb0, 0x75, 0x35, 0xdf, 0xd5, 0x8e, 0x0b,
0x3c, 0xd6, 0x4f, 0xd7, 0x15, 0x52, 0x80, 0x87,
0x2a, 0x04, 0x71, 0xbc, 0xf8, 0x30, 0x95, 0x52,
0x6a, 0xce, 0x0e, 0x38, 0xc6, 0x00, 0x00, 0x00,
},
Index: 0xffffffff,
},
Sequence: math.MaxUint64,
},
},
SubnetworkID: *subnetworkid.SubnetworkIDNative,
},
{
Version: 1,
TxIn: []*wire.TxIn{
@ -1106,36 +1077,6 @@ var BlockWithWrongTxOrder = wire.MsgBlock{
},
LockTime: 0,
},
{
Version: 1,
TxIn: []*wire.TxIn{
{
PreviousOutPoint: wire.OutPoint{
TxID: daghash.TxID{
0x16, 0x5e, 0x38, 0xe8, 0xb3, 0x91, 0x45, 0x95,
0xd9, 0xc6, 0x41, 0xf3, 0xb8, 0xee, 0xc2, 0xf3,
0x46, 0x11, 0x89, 0x6b, 0x82, 0x1a, 0x68, 0x3b,
0x7a, 0x4e, 0xde, 0xfe, 0x2c, 0x00, 0x00, 0x00,
},
Index: 0xffffffff,
},
Sequence: math.MaxUint64,
},
{
PreviousOutPoint: wire.OutPoint{
TxID: daghash.TxID{
0x4b, 0xb0, 0x75, 0x35, 0xdf, 0xd5, 0x8e, 0x0b,
0x3c, 0xd6, 0x4f, 0xd7, 0x15, 0x52, 0x80, 0x87,
0x2a, 0x04, 0x71, 0xbc, 0xf8, 0x30, 0x95, 0x52,
0x6a, 0xce, 0x0e, 0x38, 0xc6, 0x00, 0x00, 0x00,
},
Index: 0xffffffff,
},
Sequence: math.MaxUint64,
},
},
SubnetworkID: *subnetworkid.SubnetworkIDNative,
},
{
Version: 1,
TxIn: []*wire.TxIn{

View File

@ -847,7 +847,7 @@ func (mp *TxPool) maybeAcceptTransaction(tx *util.Tx, isNew, rateLimit, rejectDu
// Perform preliminary sanity checks on the transaction. This makes
// use of blockDAG which contains the invariant rules for what
// transactions are allowed into blocks.
err := blockdag.CheckTransactionSanity(tx, subnetworkID, false)
err := blockdag.CheckTransactionSanity(tx, subnetworkID)
if err != nil {
if cerr, ok := err.(blockdag.RuleError); ok {
return nil, nil, dagRuleError(cerr)
@ -1436,12 +1436,12 @@ func (mp *TxPool) HandleNewBlock(block *util.Block, txChan chan NewBlockMsg) err
// no longer an orphan. Transactions which depend on a confirmed
// transaction are NOT removed recursively because they are still
// valid.
err := mp.RemoveTransactions(block.Transactions()[util.FeeTransactionIndex:])
err := mp.RemoveTransactions(block.Transactions()[util.CoinbaseTransactionIndex+1:])
if err != nil {
mp.mpUTXOSet = oldUTXOSet
return err
}
for _, tx := range block.Transactions()[util.FeeTransactionIndex:] {
for _, tx := range block.Transactions()[util.CoinbaseTransactionIndex+1:] {
mp.RemoveDoubleSpends(tx)
mp.RemoveOrphan(tx)
acceptedTxs := mp.ProcessOrphans(tx)

View File

@ -414,13 +414,6 @@ func (g *BlkTmplGenerator) NewBlockTemplate(payToAddress util.Address) (*BlockTe
}
numCoinbaseSigOps := int64(blockdag.CountSigOps(coinbaseTx))
msgFeeTransaction, err := g.dag.NextBlockFeeTransactionNoLock()
if err != nil {
return nil, err
}
feeTransaction := util.NewTx(msgFeeTransaction)
feeTxSigOps := int64(blockdag.CountSigOps(feeTransaction))
// Get the current source transactions and create a priority queue to
// hold the transactions which are ready for inclusion into a block
// along with some priority related and fee metadata. Reserve the same
@ -435,14 +428,14 @@ func (g *BlkTmplGenerator) NewBlockTemplate(payToAddress util.Address) (*BlockTe
// generated block with reserved space. Also create a utxo view to
// house all of the input transactions so multiple lookups can be
// avoided.
blockTxns := make([]*util.Tx, 0, len(sourceTxns)+2)
blockTxns = append(blockTxns, coinbaseTx, feeTransaction)
blockTxns := make([]*util.Tx, 0, len(sourceTxns)+1)
blockTxns = append(blockTxns, coinbaseTx)
// The starting block size is the size of the block header plus the max
// possible transaction count size, plus the size of the coinbase
// transaction.
blockSize := blockHeaderOverhead + uint32(coinbaseTx.MsgTx().SerializeSize())
blockSigOps := numCoinbaseSigOps + feeTxSigOps
blockSigOps := numCoinbaseSigOps
totalFees := uint64(0)
// Create slices to hold the fees and number of signature operations
@ -451,10 +444,10 @@ func (g *BlkTmplGenerator) NewBlockTemplate(payToAddress util.Address) (*BlockTe
// a transaction as it is selected for inclusion in the final block.
// However, since the total fees aren't known yet, use a dummy value for
// the coinbase fee which will be updated later.
txFees := make([]uint64, 0, len(sourceTxns)+2)
txSigOpCounts := make([]int64, 0, len(sourceTxns)+2)
txFees := make([]uint64, 0, len(sourceTxns)+1)
txSigOpCounts := make([]int64, 0, len(sourceTxns)+1)
txFees = append(txFees, 0, 0) // For coinbase and fee txs
txSigOpCounts = append(txSigOpCounts, numCoinbaseSigOps, feeTxSigOps)
txSigOpCounts = append(txSigOpCounts, numCoinbaseSigOps)
log.Debugf("Considering %d transactions for inclusion to new block",
len(sourceTxns))

View File

@ -379,7 +379,7 @@ func TestNewBlockTemplate(t *testing.T) {
*subnetworkTx1.TxID(): false,
}
for _, tx := range template2.Block.Transactions[2:] {
for _, tx := range template2.Block.Transactions[1:] {
id := *tx.TxID()
if _, ok := expectedTxs[id]; !ok {
t.Errorf("Unexpected tx %v in template2's candidate block", id)

View File

@ -26,10 +26,6 @@ const (
// CoinbaseTransactionIndex is the index of the coinbase transaction in every block
CoinbaseTransactionIndex = 0
// FeeTransactionIndex is the index of the fee transaction in every block (except genesis,
// which doesn't have a fee transaction)
FeeTransactionIndex = 1
)
// Error satisfies the error interface and prints human-readable errors.
@ -214,16 +210,6 @@ func (b *Block) CoinbaseTransaction() *Tx {
return b.Transactions()[CoinbaseTransactionIndex]
}
// FeeTransaction returns this block's fee transaction
// If this block is a genesis block, it has no fee transaction, and therefore
// nil is returned.
func (b *Block) FeeTransaction() *Tx {
if b.IsGenesis() {
return nil
}
return b.Transactions()[FeeTransactionIndex]
}
// Timestamp returns this block's timestamp
func (b *Block) Timestamp() time.Time {
return b.msgBlock.Header.Timestamp

View File

@ -306,8 +306,8 @@ func (msg *MsgTx) IsCoinBase() bool {
}
// IsFeeTransaction determines whether or not a transaction is a fee transaction. A fee
// transaction is a special transaction created by miners that distributes fees to the
// previous blocks' miners. Each input of the fee transaction should set index to maximum
// transaction is a special transaction implicitly included in blocks that distributes fees to
// the previous blocks' miners. Each input of the fee transaction should set index to maximum
// value and reference the relevant block id, instead of previous transaction id.
func (msg *MsgTx) IsFeeTransaction() bool {
for _, txIn := range msg.TxIn {