mirror of
https://github.com/kaspanet/kaspad.git
synced 2025-06-06 14:16:43 +00:00
[NOD-264] Implement tx-selection algorithm (#358)
* [NOD-264] Implemented calcTxSelectionValue. * [NOD-264] Fixed bad subnetworkID in calcTxSelectionValue. * [NOD-264] Implemented sorting the txDescs by value. * [NOD-264] Got rid of txPrioItem. * [NOD-264] Moved transaction selection to a separate file. * [NOD-264] Renamed the result object to txsForBlockTemplate. * [NOD-264] Implemented tx selection. * [NOD-264] Fixed trying to get the gas limit for built-in subnetworks. * [NOD-264] Wrote comments where appropriate. * [NOD-264] Moved calcTxSelectionValue to the mining package. (Non-mining nodes shouldn't be forced to calc selection value for every transaction) * [NOD-264] Wrote a test for selectTxs. * [NOD-264] Fixed a comment. * [NOD-264] Fixed misunderstood test. * [NOD-264] Added zero fee check. Added a couple more tests. * [NOD-264] Added probabilistic tests. Fixed a couple of bugs in tx selection. * [NOD-264] Fixed tests with missing fees. * [NOD-264] Added a test over a range of txs with different gas/mass. * [NOD-264] Added expected probability to the rest of the test cases. * [NOD-264] Tightened bounds in probability test. * [NOD-264] Fixed values in probabily test. * [NOD-264] Added a comments for alpha and rebalanceThreshold. * [NOD-264] Fixed a couple of comments, renamed result to txsForBlockTemplate. * [NOD-264] Removed an irrelevant comment. Changed Tracef to Warnf in some logs. * [NOD-264] Renamed selectionValue -> txValue. * [NOD-264] Moved rebalancing to the start of the tx selection loop. * [NOD-264] Added overflow check for gasUsage. * [NOD-264] Renamed blockSigOps and blockMass to totalSigOps and totalMass. * [NOD-264] Removed the need to pass usedCount to reblanaceCandidates. Also relaxed bounds in a test. * [NOD-264] Split selectTxs into smaller functions. Also relaxed bounds in a test some more. * [NOD-264] Added a comment for findTx. * [NOD-264] Ordered candidateTxs by subnetwork instead of txValue. * [NOD-264] Disallowed zero tx fees in mempool and config. Renamed iterateCandidateTxs to populateTemplateFromCandidates. * [NOD-264] Changed isFinalizedTransaction log level from Warn to Debug. * [NOD-264] Removed references to SigOps in txSelection. * [NOD-264] Removed SigOps validation. Validating mass should suffice. * [NOD-264] Renamed wasUsed to isMarkedForDeletion. * [NOD-264] Renamed markCandidateTxUsed to markCandidateTxForDeletion. * [NOD-264] Made some probabilistic tests less likely to fail when they shouldn't. * [NOD-264] Added a message warning people about probabilistic tests. * [NOD-264] Rephrased a comment about rebalanceThreshold. * [NOD-264] Removed IsCoinBase, CheckTransactionInputsAndCalulateFee, and ValidateTransactionScripts from txSelection. * [NOD-264] Removed a condition that is no longer relevant. * [NOD-264] "which's" -> "whose" * [NOD-264] Removed wasteful preallocations. * [NOD-264] Fixed a comment referring to "used" transactions.
This commit is contained in:
parent
5f49115cac
commit
70737e4e94
@ -19,10 +19,6 @@ import (
|
||||
)
|
||||
|
||||
const (
|
||||
// MaxSigOpsPerBlock is the maximum number of signature operations
|
||||
// allowed for a block. It is a fraction of the max block transaction mass.
|
||||
MaxSigOpsPerBlock = wire.MaxMassPerBlock / 50
|
||||
|
||||
// MaxCoinbasePayloadLen is the maximum length a coinbase payload can be.
|
||||
MaxCoinbasePayloadLen = 150
|
||||
|
||||
@ -597,22 +593,6 @@ func (dag *BlockDAG) checkBlockSanity(block *util.Block, flags BehaviorFlags) (t
|
||||
existingTxIDs[*id] = struct{}{}
|
||||
}
|
||||
|
||||
// The number of signature operations must be less than the maximum
|
||||
// allowed per block.
|
||||
totalSigOps := 0
|
||||
for _, tx := range transactions {
|
||||
// We could potentially overflow the accumulator so check for
|
||||
// overflow.
|
||||
lastSigOps := totalSigOps
|
||||
totalSigOps += CountSigOps(tx)
|
||||
if totalSigOps < lastSigOps || totalSigOps > MaxSigOpsPerBlock {
|
||||
str := fmt.Sprintf("block contains too many signature "+
|
||||
"operations - got %d, max %d", totalSigOps,
|
||||
MaxSigOpsPerBlock)
|
||||
return 0, ruleError(ErrTooManySigOps, str)
|
||||
}
|
||||
}
|
||||
|
||||
return delay, nil
|
||||
}
|
||||
|
||||
@ -934,10 +914,6 @@ func (dag *BlockDAG) checkConnectToPastUTXO(block *blockNode, pastUTXO UTXOSet,
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if err := validateSigopsCount(pastUTXO, transactions); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if err := validateBlockMass(pastUTXO, transactions); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@ -1039,42 +1015,6 @@ func (dag *BlockDAG) checkConnectToPastUTXO(block *blockNode, pastUTXO UTXOSet,
|
||||
return feeData, nil
|
||||
}
|
||||
|
||||
func validateSigopsCount(pastUTXO UTXOSet, transactions []*util.Tx) error {
|
||||
// The number of signature operations must be less than the maximum
|
||||
// allowed per block. Note that the preliminary sanity checks on a
|
||||
// block also include a check similar to this one, but this check
|
||||
// expands the count to include a precise count of pay-to-script-hash
|
||||
// signature operations in each of the input transaction public key
|
||||
// scripts.
|
||||
totalSigOps := 0
|
||||
for i, tx := range transactions {
|
||||
numsigOps := CountSigOps(tx)
|
||||
// Since the first transaction has already been verified to be a
|
||||
// coinbase transaction, use i != util.CoinbaseTransactionIndex
|
||||
// as an optimization for the flag to countP2SHSigOps for whether
|
||||
// or not the transaction is a coinbase transaction rather than
|
||||
// having to do a full coinbase check again.
|
||||
numP2SHSigOps, err := CountP2SHSigOps(tx, i == util.CoinbaseTransactionIndex, pastUTXO)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
numsigOps += numP2SHSigOps
|
||||
|
||||
// Check for overflow or going over the limits. We have to do
|
||||
// this on every loop iteration to avoid overflow.
|
||||
lastSigops := totalSigOps
|
||||
totalSigOps += numsigOps
|
||||
if totalSigOps < lastSigops || totalSigOps > MaxSigOpsPerBlock {
|
||||
str := fmt.Sprintf("block contains too many "+
|
||||
"signature operations - got %d, max %d",
|
||||
totalSigOps, MaxSigOpsPerBlock)
|
||||
return ruleError(ErrTooManySigOps, str)
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// CheckConnectBlockTemplate fully validates that connecting the passed block to
|
||||
// the DAG does not violate any consensus rules, aside from the proof of
|
||||
// work requirement.
|
||||
|
@ -123,7 +123,6 @@ type GetBlockTemplateResultTx struct {
|
||||
Depends []int64 `json:"depends"`
|
||||
Mass uint64 `json:"mass"`
|
||||
Fee uint64 `json:"fee"`
|
||||
SigOps int64 `json:"sigOps"`
|
||||
}
|
||||
|
||||
// GetBlockTemplateResultAux models the coinbaseaux field of the
|
||||
@ -141,7 +140,6 @@ type GetBlockTemplateResult struct {
|
||||
CurTime int64 `json:"curTime"`
|
||||
Height uint64 `json:"height"`
|
||||
ParentHashes []string `json:"parentHashes"`
|
||||
SigOpLimit int64 `json:"sigOpLimit,omitempty"`
|
||||
MassLimit int64 `json:"massLimit,omitempty"`
|
||||
Transactions []GetBlockTemplateResultTx `json:"transactions"`
|
||||
AcceptedIDMerkleRoot string `json:"acceptedIdMerkleRoot"`
|
||||
|
@ -655,6 +655,15 @@ func loadConfig() (*Config, []string, error) {
|
||||
return nil, nil, err
|
||||
}
|
||||
|
||||
// Disallow 0 and negative min tx fees.
|
||||
if cfg.MinRelayTxFee <= 0 {
|
||||
str := "%s: The minrelaytxfee option must greater than 0 -- parsed [%d]"
|
||||
err := fmt.Errorf(str, funcName, cfg.MinRelayTxFee)
|
||||
fmt.Fprintln(os.Stderr, err)
|
||||
fmt.Fprintln(os.Stderr, usageMessage)
|
||||
return nil, nil, err
|
||||
}
|
||||
|
||||
// Limit the max block mass to a sane value.
|
||||
if cfg.BlockMaxMass < blockMaxMassMin || cfg.BlockMaxMass >
|
||||
blockMaxMassMax {
|
||||
|
@ -111,11 +111,6 @@ type Policy struct {
|
||||
// of big orphans.
|
||||
MaxOrphanTxSize int
|
||||
|
||||
// MaxSigOpsPerTx is the maximum number of signature operations
|
||||
// in a single transaction we will relay or mine. It is a fraction
|
||||
// of the max signature operations for a block.
|
||||
MaxSigOpsPerTx int
|
||||
|
||||
// MinRelayTxFee defines the minimum transaction fee in BTC/kB to be
|
||||
// considered a non-zero fee.
|
||||
MinRelayTxFee util.Amount
|
||||
@ -983,22 +978,10 @@ func (mp *TxPool) maybeAcceptTransaction(tx *util.Tx, isNew, rejectDupOrphans bo
|
||||
// you should add code here to check that the transaction does a
|
||||
// reasonable number of ECDSA signature verifications.
|
||||
|
||||
// Don't allow transactions with an excessive number of signature
|
||||
// operations which would result in making it impossible to mine. Since
|
||||
// the coinbase address itself can contain signature operations, the
|
||||
// maximum allowed signature operations per transaction is less than
|
||||
// the maximum allowed signature operations per block.
|
||||
sigOpCount, err := blockdag.CountP2SHSigOps(tx, false, mp.mpUTXOSet)
|
||||
if err != nil {
|
||||
if cerr, ok := err.(blockdag.RuleError); ok {
|
||||
return nil, nil, dagRuleError(cerr)
|
||||
}
|
||||
return nil, nil, err
|
||||
}
|
||||
if sigOpCount > mp.cfg.Policy.MaxSigOpsPerTx {
|
||||
str := fmt.Sprintf("transaction %s sigop count is too high: %d > %d",
|
||||
txID, sigOpCount, mp.cfg.Policy.MaxSigOpsPerTx)
|
||||
return nil, nil, txRuleError(wire.RejectNonstandard, str)
|
||||
// Don't allow transactions with 0 fees.
|
||||
if txFee == 0 {
|
||||
str := fmt.Sprintf("transaction %s has 0 fees", txID)
|
||||
return nil, nil, txRuleError(wire.RejectInsufficientFee, str)
|
||||
}
|
||||
|
||||
// Don't allow transactions with fees too low to get into a mined block.
|
||||
|
@ -346,7 +346,6 @@ func newPoolHarness(t *testing.T, dagParams *dagconfig.Params, numOutputs uint32
|
||||
Policy: Policy{
|
||||
MaxOrphanTxs: 5,
|
||||
MaxOrphanTxSize: 1000,
|
||||
MaxSigOpsPerTx: blockdag.MaxSigOpsPerBlock / 5,
|
||||
MinRelayTxFee: 1000, // 1 Satoshi per byte
|
||||
MaxTxVersion: 1,
|
||||
},
|
||||
@ -672,23 +671,6 @@ func TestProcessTransaction(t *testing.T) {
|
||||
t.Errorf("Unexpected error message. Expected \"%s\" but got \"%s\"", expectedErrStr, err.Error())
|
||||
}
|
||||
|
||||
//Checks that even if we accept non standard transactions, we reject by the MaxSigOpsPerTx consensus rule
|
||||
harness.txPool.cfg.Policy.AcceptNonStd = true
|
||||
harness.txPool.cfg.Policy.MaxSigOpsPerTx = 15
|
||||
_, err = harness.txPool.ProcessTransaction(nonStdSigScriptTx, true, 0)
|
||||
if err == nil {
|
||||
t.Errorf("ProcessTransaction: expected an error, not nil")
|
||||
}
|
||||
if code, _ := extractRejectCode(err); code != wire.RejectNonstandard {
|
||||
t.Errorf("Unexpected error code. Expected %v but got %v", wire.RejectNonstandard, code)
|
||||
}
|
||||
expectedErrStr = fmt.Sprintf("transaction %v sigop count is too high: %v > %v",
|
||||
nonStdSigScriptTx.ID(), 16, 15)
|
||||
if expectedErrStr != err.Error() {
|
||||
t.Errorf("Unexpected error message. Expected \"%s\" but got \"%s\"", expectedErrStr, err.Error())
|
||||
}
|
||||
harness.txPool.cfg.Policy.AcceptNonStd = false
|
||||
|
||||
//Checks that a transaction with no outputs will not get rejected
|
||||
noOutsTx := util.NewTx(wire.NewNativeMsgTx(1, []*wire.TxIn{{
|
||||
PreviousOutpoint: dummyPrevOut,
|
||||
|
304
mining/mining.go
304
mining/mining.go
@ -6,7 +6,6 @@ package mining
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"container/heap"
|
||||
"encoding/binary"
|
||||
"fmt"
|
||||
"sort"
|
||||
@ -17,16 +16,11 @@ import (
|
||||
"github.com/daglabs/btcd/txscript"
|
||||
"github.com/daglabs/btcd/util"
|
||||
"github.com/daglabs/btcd/util/daghash"
|
||||
"github.com/daglabs/btcd/util/random"
|
||||
"github.com/daglabs/btcd/util/subnetworkid"
|
||||
"github.com/daglabs/btcd/wire"
|
||||
)
|
||||
|
||||
const (
|
||||
// blockHeaderOverhead is the max number of bytes it takes to serialize
|
||||
// a block header and max possible transaction count.
|
||||
blockHeaderOverhead = wire.MaxBlockHeaderPayload + wire.MaxVarIntPayload
|
||||
|
||||
// CoinbaseFlags is added to the coinbase script of a generated block
|
||||
// and is used to monitor BIP16 support as well as blocks that are
|
||||
// generated via btcd.
|
||||
@ -72,87 +66,6 @@ type TxSource interface {
|
||||
HaveTransaction(txID *daghash.TxID) bool
|
||||
}
|
||||
|
||||
// txPrioItem houses a transaction along with extra information that allows the
|
||||
// transaction to be prioritized and track dependencies on other transactions
|
||||
// which have not been mined into a block yet.
|
||||
type txPrioItem struct {
|
||||
tx *util.Tx
|
||||
fee uint64
|
||||
feePerKB uint64
|
||||
}
|
||||
|
||||
// txPriorityQueueLessFunc describes a function that can be used as a compare
|
||||
// function for a transaction priority queue (txPriorityQueue).
|
||||
type txPriorityQueueLessFunc func(*txPriorityQueue, int, int) bool
|
||||
|
||||
// txPriorityQueue implements a priority queue of txPrioItem elements that
|
||||
// supports an arbitrary compare function as defined by txPriorityQueueLessFunc.
|
||||
type txPriorityQueue struct {
|
||||
lessFunc txPriorityQueueLessFunc
|
||||
items []*txPrioItem
|
||||
}
|
||||
|
||||
// Len returns the number of items in the priority queue. It is part of the
|
||||
// heap.Interface implementation.
|
||||
func (pq *txPriorityQueue) Len() int {
|
||||
return len(pq.items)
|
||||
}
|
||||
|
||||
// Less returns whether the item in the priority queue with index i should sort
|
||||
// before the item with index j by deferring to the assigned less function. It
|
||||
// is part of the heap.Interface implementation.
|
||||
func (pq *txPriorityQueue) Less(i, j int) bool {
|
||||
return pq.lessFunc(pq, i, j)
|
||||
}
|
||||
|
||||
// Swap swaps the items at the passed indices in the priority queue. It is
|
||||
// part of the heap.Interface implementation.
|
||||
func (pq *txPriorityQueue) Swap(i, j int) {
|
||||
pq.items[i], pq.items[j] = pq.items[j], pq.items[i]
|
||||
}
|
||||
|
||||
// Push pushes the passed item onto the priority queue. It is part of the
|
||||
// heap.Interface implementation.
|
||||
func (pq *txPriorityQueue) Push(x interface{}) {
|
||||
pq.items = append(pq.items, x.(*txPrioItem))
|
||||
}
|
||||
|
||||
// Pop removes the highest priority item (according to Less) from the priority
|
||||
// queue and returns it. It is part of the heap.Interface implementation.
|
||||
func (pq *txPriorityQueue) Pop() interface{} {
|
||||
n := len(pq.items)
|
||||
item := pq.items[n-1]
|
||||
pq.items[n-1] = nil
|
||||
pq.items = pq.items[0 : n-1]
|
||||
return item
|
||||
}
|
||||
|
||||
// SetLessFunc sets the compare function for the priority queue to the provided
|
||||
// function. It also invokes heap.Init on the priority queue using the new
|
||||
// function so it can immediately be used with heap.Push/Pop.
|
||||
func (pq *txPriorityQueue) SetLessFunc(lessFunc txPriorityQueueLessFunc) {
|
||||
pq.lessFunc = lessFunc
|
||||
heap.Init(pq)
|
||||
}
|
||||
|
||||
// txPQByFee sorts a txPriorityQueue by fees per kilobyte
|
||||
func txPQByFee(pq *txPriorityQueue, i, j int) bool {
|
||||
return pq.items[i].feePerKB > pq.items[j].feePerKB
|
||||
}
|
||||
|
||||
// newTxPriorityQueue returns a new transaction priority queue that reserves the
|
||||
// passed amount of space for the elements. The new priority queue uses the
|
||||
// txPQByFee compare function and is already initialized for use with heap.Push/Pop.
|
||||
// The priority queue can grow larger than the reserved space, but extra copies
|
||||
// of the underlying array can be avoided by reserving a sane value.
|
||||
func newTxPriorityQueue(reserve int) *txPriorityQueue {
|
||||
pq := &txPriorityQueue{
|
||||
items: make([]*txPrioItem, 0, reserve),
|
||||
}
|
||||
pq.SetLessFunc(txPQByFee)
|
||||
return pq
|
||||
}
|
||||
|
||||
// BlockTemplate houses a block that has yet to be solved along with additional
|
||||
// details about the fees and the number of signature operations for each
|
||||
// transaction in the block.
|
||||
@ -172,10 +85,6 @@ type BlockTemplate struct {
|
||||
// sum of the fees of all other transactions.
|
||||
Fees []uint64
|
||||
|
||||
// SigOpCounts contains the number of signature operations each
|
||||
// transaction in the generated template performs.
|
||||
SigOpCounts []int64
|
||||
|
||||
// Height is the height at which the block template connects to the DAG
|
||||
Height uint64
|
||||
|
||||
@ -313,194 +222,9 @@ func (g *BlkTmplGenerator) NewBlockTemplate(payToAddress util.Address) (*BlockTe
|
||||
g.dag.RLock()
|
||||
defer g.dag.RUnlock()
|
||||
|
||||
nextBlockBlueScore := g.dag.VirtualBlueScore()
|
||||
nextBlockUTXO := g.dag.UTXOSet()
|
||||
|
||||
coinbasePayloadPkScript, err := txscript.PayToAddrScript(payToAddress)
|
||||
txsForBlockTemplate, err := g.selectTxs(payToAddress)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
extraNonce, err := random.Uint64()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
coinbasePayloadExtraData, err := CoinbasePayloadExtraData(extraNonce)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
coinbaseTx, err := g.dag.NextBlockCoinbaseTransactionNoLock(coinbasePayloadPkScript, coinbasePayloadExtraData)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
coinbaseTxMass, err := blockdag.CalcTxMass(coinbaseTx, nextBlockUTXO)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
numCoinbaseSigOps := int64(blockdag.CountSigOps(coinbaseTx))
|
||||
|
||||
// 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
|
||||
// number of items that are available for the priority queue. Also,
|
||||
// choose the initial sort order for the priority queue based on whether
|
||||
// or not there is an area allocated for high-priority transactions.
|
||||
sourceTxns := g.txSource.MiningDescs()
|
||||
priorityQueue := newTxPriorityQueue(len(sourceTxns))
|
||||
|
||||
// Create a slice to hold the transactions to be included in the
|
||||
// generated block with reserved space. Also create a utxo view to
|
||||
// house all of the input transactions so multiple lookups can be
|
||||
// avoided.
|
||||
blockTxns := make([]*util.Tx, 0, len(sourceTxns)+1)
|
||||
blockTxns = append(blockTxns, coinbaseTx)
|
||||
|
||||
blockMass := coinbaseTxMass
|
||||
blockSigOps := numCoinbaseSigOps
|
||||
totalFees := uint64(0)
|
||||
|
||||
// Create slices to hold the mass, the fees, and number of signature
|
||||
// operations for each of the selected transactions and add an entry for
|
||||
// the coinbase. This allows the code below to simply append details
|
||||
// about 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.
|
||||
txMasses := make([]uint64, 0, len(sourceTxns)+1)
|
||||
txFees := make([]uint64, 0, len(sourceTxns)+1)
|
||||
txSigOpCounts := make([]int64, 0, len(sourceTxns)+1)
|
||||
txMasses = append(txMasses, coinbaseTxMass)
|
||||
txFees = append(txFees, 0) // For coinbase tx
|
||||
txSigOpCounts = append(txSigOpCounts, numCoinbaseSigOps)
|
||||
|
||||
log.Debugf("Considering %d transactions for inclusion to new block",
|
||||
len(sourceTxns))
|
||||
for _, txDesc := range sourceTxns {
|
||||
// A block can't have more than one coinbase or contain
|
||||
// non-finalized transactions.
|
||||
tx := txDesc.Tx
|
||||
if tx.IsCoinBase() {
|
||||
log.Tracef("Skipping coinbase tx %s", tx.ID())
|
||||
continue
|
||||
}
|
||||
if !blockdag.IsFinalizedTransaction(tx, nextBlockBlueScore,
|
||||
g.timeSource.AdjustedTime()) {
|
||||
|
||||
log.Tracef("Skipping non-finalized tx %s", tx.ID())
|
||||
continue
|
||||
}
|
||||
|
||||
// Calculate the final transaction priority using the input
|
||||
// value age sum as well as the adjusted transaction size. The
|
||||
// formula is: sum(inputValue * inputAge) / adjustedTxSize
|
||||
prioItem := &txPrioItem{tx: tx}
|
||||
|
||||
// Calculate the fee in Satoshi/kB.
|
||||
prioItem.feePerKB = txDesc.FeePerKB
|
||||
prioItem.fee = txDesc.Fee
|
||||
|
||||
heap.Push(priorityQueue, prioItem)
|
||||
}
|
||||
|
||||
// Create map of GAS usage per subnetwork
|
||||
gasUsageMap := make(map[subnetworkid.SubnetworkID]uint64)
|
||||
|
||||
// Choose which transactions make it into the block.
|
||||
for priorityQueue.Len() > 0 {
|
||||
// Grab the highest priority (or highest fee per kilobyte
|
||||
// depending on the sort order) transaction.
|
||||
prioItem := heap.Pop(priorityQueue).(*txPrioItem)
|
||||
tx := prioItem.tx
|
||||
|
||||
if !tx.MsgTx().SubnetworkID.IsEqual(subnetworkid.SubnetworkIDNative) && !tx.MsgTx().SubnetworkID.IsBuiltIn() {
|
||||
subnetworkID := tx.MsgTx().SubnetworkID
|
||||
gasUsage, ok := gasUsageMap[subnetworkID]
|
||||
if !ok {
|
||||
gasUsage = 0
|
||||
}
|
||||
gasLimit, err := g.dag.SubnetworkStore.GasLimit(&subnetworkID)
|
||||
if err != nil {
|
||||
log.Errorf("Cannot get GAS limit for subnetwork %s", subnetworkID)
|
||||
continue
|
||||
}
|
||||
txGas := tx.MsgTx().Gas
|
||||
if gasLimit-gasUsage < txGas {
|
||||
log.Tracef("Transaction %s (GAS=%d) ignored because gas overusage (GASUsage=%d) in subnetwork %s (GASLimit=%d)",
|
||||
tx.MsgTx().TxID(), txGas, gasUsage, subnetworkID, gasLimit)
|
||||
continue
|
||||
}
|
||||
gasUsageMap[subnetworkID] = gasUsage + txGas
|
||||
}
|
||||
|
||||
// Enforce maximum transaction mass per block. Also check
|
||||
// for overflow.
|
||||
txMass, err := blockdag.CalcTxMass(tx, g.dag.UTXOSet())
|
||||
if err != nil {
|
||||
log.Tracef("Skipping tx %s due to error in "+
|
||||
"CalcTxMass: %s", tx.ID(), err)
|
||||
continue
|
||||
}
|
||||
if blockMass+txMass < blockMass ||
|
||||
blockMass >= g.policy.BlockMaxMass {
|
||||
log.Tracef("Skipping tx %s because it would exceed "+
|
||||
"the max block mass", tx.ID())
|
||||
continue
|
||||
}
|
||||
|
||||
// Enforce maximum signature operations per block. Also check
|
||||
// for overflow.
|
||||
numSigOps := int64(blockdag.CountSigOps(tx))
|
||||
if blockSigOps+numSigOps < blockSigOps ||
|
||||
blockSigOps+numSigOps > blockdag.MaxSigOpsPerBlock {
|
||||
log.Tracef("Skipping tx %s because it would exceed "+
|
||||
"the maximum sigops per block", tx.ID())
|
||||
continue
|
||||
}
|
||||
numP2SHSigOps, err := blockdag.CountP2SHSigOps(tx, false,
|
||||
g.dag.UTXOSet())
|
||||
if err != nil {
|
||||
log.Tracef("Skipping tx %s due to error in "+
|
||||
"GetSigOpCost: %s", tx.ID(), err)
|
||||
continue
|
||||
}
|
||||
numSigOps += int64(numP2SHSigOps)
|
||||
if blockSigOps+numSigOps < blockSigOps ||
|
||||
blockSigOps+numSigOps > blockdag.MaxSigOpsPerBlock {
|
||||
log.Tracef("Skipping tx %s because it would "+
|
||||
"exceed the maximum sigops per block", tx.ID())
|
||||
continue
|
||||
}
|
||||
|
||||
// Ensure the transaction inputs pass all of the necessary
|
||||
// preconditions before allowing it to be added to the block.
|
||||
_, err = blockdag.CheckTransactionInputsAndCalulateFee(tx, nextBlockBlueScore,
|
||||
g.dag.UTXOSet(), g.dagParams, false)
|
||||
if err != nil {
|
||||
log.Tracef("Skipping tx %s due to error in "+
|
||||
"CheckTransactionInputs: %s", tx.ID(), err)
|
||||
continue
|
||||
}
|
||||
err = blockdag.ValidateTransactionScripts(tx, g.dag.UTXOSet(),
|
||||
txscript.StandardVerifyFlags, g.sigCache)
|
||||
if err != nil {
|
||||
log.Tracef("Skipping tx %s due to error in "+
|
||||
"ValidateTransactionScripts: %s", tx.ID(), err)
|
||||
continue
|
||||
}
|
||||
|
||||
// Add the transaction to the block, increment counters, and
|
||||
// save the masses, fees, and signature operation counts to the block
|
||||
// template.
|
||||
blockTxns = append(blockTxns, tx)
|
||||
blockMass += txMass
|
||||
blockSigOps += numSigOps
|
||||
totalFees += prioItem.fee
|
||||
txMasses = append(txMasses, txMass)
|
||||
txFees = append(txFees, prioItem.fee)
|
||||
txSigOpCounts = append(txSigOpCounts, numSigOps)
|
||||
|
||||
log.Tracef("Adding tx %s (feePerKB %.2f)",
|
||||
prioItem.tx.ID(), prioItem.feePerKB)
|
||||
return nil, fmt.Errorf("failed to select txs: %s", err)
|
||||
}
|
||||
|
||||
// Calculate the required difficulty for the block. The timestamp
|
||||
@ -517,24 +241,25 @@ func (g *BlkTmplGenerator) NewBlockTemplate(payToAddress util.Address) (*BlockTe
|
||||
}
|
||||
|
||||
// Sort transactions by subnetwork ID before building Merkle tree
|
||||
sort.Slice(blockTxns, func(i, j int) bool {
|
||||
if blockTxns[i].MsgTx().SubnetworkID.IsEqual(subnetworkid.SubnetworkIDCoinbase) {
|
||||
selectedTxs := txsForBlockTemplate.selectedTxs
|
||||
sort.Slice(selectedTxs, func(i, j int) bool {
|
||||
if selectedTxs[i].MsgTx().SubnetworkID.IsEqual(subnetworkid.SubnetworkIDCoinbase) {
|
||||
return true
|
||||
}
|
||||
if blockTxns[j].MsgTx().SubnetworkID.IsEqual(subnetworkid.SubnetworkIDCoinbase) {
|
||||
if selectedTxs[j].MsgTx().SubnetworkID.IsEqual(subnetworkid.SubnetworkIDCoinbase) {
|
||||
return false
|
||||
}
|
||||
return subnetworkid.Less(&blockTxns[i].MsgTx().SubnetworkID, &blockTxns[j].MsgTx().SubnetworkID)
|
||||
return subnetworkid.Less(&selectedTxs[i].MsgTx().SubnetworkID, &selectedTxs[j].MsgTx().SubnetworkID)
|
||||
})
|
||||
|
||||
// Create a new block ready to be solved.
|
||||
hashMerkleTree := blockdag.BuildHashMerkleTreeStore(blockTxns)
|
||||
hashMerkleTree := blockdag.BuildHashMerkleTreeStore(selectedTxs)
|
||||
acceptedIDMerkleRoot, err := g.dag.NextAcceptedIDMerkleRootNoLock()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
var msgBlock wire.MsgBlock
|
||||
for _, tx := range blockTxns {
|
||||
for _, tx := range selectedTxs {
|
||||
msgBlock.AddTransaction(tx.MsgTx())
|
||||
}
|
||||
utxoCommitment, err := g.buildUTXOCommitment(msgBlock.Transactions)
|
||||
@ -561,15 +286,14 @@ func (g *BlkTmplGenerator) NewBlockTemplate(payToAddress util.Address) (*BlockTe
|
||||
}
|
||||
|
||||
log.Debugf("Created new block template (%d transactions, %d in fees, "+
|
||||
"%d signature operations, %d mass, target difficulty %064x)",
|
||||
len(msgBlock.Transactions), totalFees, blockSigOps, blockMass,
|
||||
util.CompactToBig(msgBlock.Header.Bits))
|
||||
"%d mass, target difficulty %064x)",
|
||||
len(msgBlock.Transactions), txsForBlockTemplate.totalFees,
|
||||
txsForBlockTemplate.totalMass, util.CompactToBig(msgBlock.Header.Bits))
|
||||
|
||||
return &BlockTemplate{
|
||||
Block: &msgBlock,
|
||||
TxMasses: txMasses,
|
||||
Fees: txFees,
|
||||
SigOpCounts: txSigOpCounts,
|
||||
TxMasses: txsForBlockTemplate.txMasses,
|
||||
Fees: txsForBlockTemplate.txFees,
|
||||
ValidPayAddress: payToAddress != nil,
|
||||
}, nil
|
||||
}
|
||||
|
@ -5,9 +5,7 @@
|
||||
package mining
|
||||
|
||||
import (
|
||||
"container/heap"
|
||||
"errors"
|
||||
"math/rand"
|
||||
"testing"
|
||||
|
||||
"github.com/daglabs/btcd/util/subnetworkid"
|
||||
@ -22,55 +20,6 @@ import (
|
||||
"github.com/daglabs/btcd/util"
|
||||
)
|
||||
|
||||
// TestTxFeePrioHeap ensures the priority queue for transaction fees and
|
||||
// priorities works as expected.
|
||||
func TestTxFeePrioHeap(t *testing.T) {
|
||||
// Create some fake priority items that exercise the expected sort
|
||||
// edge conditions.
|
||||
testItems := []*txPrioItem{
|
||||
{feePerKB: 5678},
|
||||
{feePerKB: 5678}, // Duplicate fee
|
||||
{feePerKB: 1234},
|
||||
{feePerKB: 10000}, // High fee
|
||||
{feePerKB: 0}, // Zero fee
|
||||
}
|
||||
|
||||
// Add random data in addition to the edge conditions already manually
|
||||
// specified.
|
||||
randSeed := rand.Int63()
|
||||
defer func() {
|
||||
if t.Failed() {
|
||||
t.Logf("Random numbers using seed: %v", randSeed)
|
||||
}
|
||||
}()
|
||||
prng := rand.New(rand.NewSource(randSeed))
|
||||
for i := 0; i < 1000; i++ {
|
||||
testItems = append(testItems, &txPrioItem{
|
||||
feePerKB: uint64(prng.Float64() * util.SatoshiPerBitcoin),
|
||||
})
|
||||
}
|
||||
|
||||
// Test sorting by fee per KB
|
||||
var highest *txPrioItem
|
||||
priorityQueue := newTxPriorityQueue(len(testItems))
|
||||
for _, prioItem := range testItems {
|
||||
if highest == nil || prioItem.feePerKB >= highest.feePerKB {
|
||||
highest = prioItem
|
||||
}
|
||||
heap.Push(priorityQueue, prioItem)
|
||||
}
|
||||
|
||||
for i := 0; i < len(testItems); i++ {
|
||||
prioItem := heap.Pop(priorityQueue).(*txPrioItem)
|
||||
if prioItem.feePerKB > highest.feePerKB {
|
||||
t.Fatalf("fee sort: item (fee per KB: %v) "+
|
||||
"higher than than prev (fee per KB: %v)",
|
||||
prioItem.feePerKB, highest.feePerKB)
|
||||
}
|
||||
highest = prioItem
|
||||
}
|
||||
}
|
||||
|
||||
func TestNewBlockTemplate(t *testing.T) {
|
||||
params := dagconfig.SimNetParams
|
||||
params.BlockCoinbaseMaturity = 0
|
||||
@ -144,12 +93,6 @@ func TestNewBlockTemplate(t *testing.T) {
|
||||
cbTxs = append(cbTxs, template.Block.Transactions[util.CoinbaseTransactionIndex])
|
||||
}
|
||||
|
||||
// We want to check that the miner filters coinbase transaction
|
||||
cbTx, err := dag.NextBlockCoinbaseTransaction(nil, nil)
|
||||
if err != nil {
|
||||
t.Fatalf("createCoinbaseTx: %v", err)
|
||||
}
|
||||
|
||||
signatureScript, err := txscript.PayToScriptHashSignatureScript(blockdag.OpTrueScript, nil)
|
||||
if err != nil {
|
||||
t.Fatalf("Error creating signature script: %s", err)
|
||||
@ -184,7 +127,7 @@ func TestNewBlockTemplate(t *testing.T) {
|
||||
Value: 1,
|
||||
}
|
||||
nonFinalizedTx := wire.NewNativeMsgTx(wire.TxVersion, []*wire.TxIn{txIn}, []*wire.TxOut{txOut})
|
||||
nonFinalizedTx.LockTime = uint64(dag.ChainHeight() + 2)
|
||||
nonFinalizedTx.LockTime = dag.ChainHeight() + 2
|
||||
|
||||
existingSubnetwork := &subnetworkid.SubnetworkID{0xff}
|
||||
nonExistingSubnetwork := &subnetworkid.SubnetworkID{0xfe}
|
||||
@ -238,48 +181,27 @@ func TestNewBlockTemplate(t *testing.T) {
|
||||
|
||||
txSource.txDescs = []*TxDesc{
|
||||
{
|
||||
Tx: cbTx,
|
||||
Tx: util.NewTx(tx),
|
||||
Fee: 1,
|
||||
},
|
||||
{
|
||||
Tx: util.NewTx(tx),
|
||||
Tx: util.NewTx(nonFinalizedTx),
|
||||
Fee: 1,
|
||||
},
|
||||
{
|
||||
Tx: util.NewTx(nonFinalizedTx),
|
||||
Tx: util.NewTx(subnetworkTx1),
|
||||
Fee: 1,
|
||||
},
|
||||
{
|
||||
Tx: util.NewTx(subnetworkTx1),
|
||||
Tx: util.NewTx(subnetworkTx2),
|
||||
Fee: 1,
|
||||
},
|
||||
{
|
||||
Tx: util.NewTx(subnetworkTx2),
|
||||
},
|
||||
{
|
||||
Tx: util.NewTx(nonExistingSubnetworkTx),
|
||||
Tx: util.NewTx(nonExistingSubnetworkTx),
|
||||
Fee: 1,
|
||||
},
|
||||
}
|
||||
|
||||
// Here we check that the miner's priorty queue has the expected transactions after filtering.
|
||||
popReturnedUnexpectedValue := false
|
||||
expectedPops := map[daghash.TxID]bool{
|
||||
*tx.TxID(): false,
|
||||
*subnetworkTx1.TxID(): false,
|
||||
*subnetworkTx2.TxID(): false,
|
||||
*nonExistingSubnetworkTx.TxID(): false,
|
||||
}
|
||||
var popPatch *monkey.PatchGuard
|
||||
popPatch = monkey.Patch((*txPriorityQueue).Pop, func(pq *txPriorityQueue) interface{} {
|
||||
popPatch.Unpatch()
|
||||
defer popPatch.Restore()
|
||||
|
||||
item, ok := pq.Pop().(*txPrioItem)
|
||||
if _, expected := expectedPops[*item.tx.ID()]; expected && ok {
|
||||
expectedPops[*item.tx.ID()] = true
|
||||
} else {
|
||||
popReturnedUnexpectedValue = true
|
||||
}
|
||||
return item
|
||||
})
|
||||
defer popPatch.Unpatch()
|
||||
|
||||
// Here we define nonExistingSubnetwork to be non-exist, and existingSubnetwork to have a gas limit of 90
|
||||
gasLimitPatch := monkey.Patch((*blockdag.SubnetworkStore).GasLimit, func(_ *blockdag.SubnetworkStore, subnetworkID *subnetworkid.SubnetworkID) (uint64, error) {
|
||||
if subnetworkID.IsEqual(nonExistingSubnetwork) {
|
||||
@ -290,23 +212,12 @@ func TestNewBlockTemplate(t *testing.T) {
|
||||
defer gasLimitPatch.Unpatch()
|
||||
|
||||
template3, err := blockTemplateGenerator.NewBlockTemplate(OpTrueAddr)
|
||||
popPatch.Unpatch()
|
||||
gasLimitPatch.Unpatch()
|
||||
|
||||
if err != nil {
|
||||
t.Errorf("NewBlockTemplate: unexpected error: %v", err)
|
||||
}
|
||||
|
||||
if popReturnedUnexpectedValue {
|
||||
t.Errorf("(*txPriorityQueue).Pop returned unexpected value")
|
||||
}
|
||||
|
||||
for id, popped := range expectedPops {
|
||||
if !popped {
|
||||
t.Errorf("tx %v was expected to pop, but wasn't", id)
|
||||
}
|
||||
}
|
||||
|
||||
expectedTxs := map[daghash.TxID]bool{
|
||||
*tx.TxID(): false,
|
||||
*subnetworkTx1.TxID(): false,
|
||||
|
@ -55,7 +55,8 @@ func PrepareBlockForTest(dag *blockdag.BlockDAG, params *dagconfig.Params, paren
|
||||
|
||||
for i, tx := range transactions {
|
||||
txSource.txDescs[i] = &TxDesc{
|
||||
Tx: util.NewTx(tx),
|
||||
Tx: util.NewTx(tx),
|
||||
Fee: 1,
|
||||
}
|
||||
}
|
||||
|
||||
|
370
mining/txselection.go
Normal file
370
mining/txselection.go
Normal file
@ -0,0 +1,370 @@
|
||||
package mining
|
||||
|
||||
import (
|
||||
"github.com/daglabs/btcd/blockdag"
|
||||
"github.com/daglabs/btcd/txscript"
|
||||
"github.com/daglabs/btcd/util"
|
||||
"github.com/daglabs/btcd/util/random"
|
||||
"github.com/daglabs/btcd/util/subnetworkid"
|
||||
"math"
|
||||
"math/rand"
|
||||
"sort"
|
||||
)
|
||||
|
||||
const (
|
||||
// alpha is a coefficient that defines how uniform the distribution of
|
||||
// candidate transactions should be. A smaller alpha makes the distribution
|
||||
// more uniform. Alpha is used when determining a candidate transaction's
|
||||
// initial p value.
|
||||
alpha = 3
|
||||
|
||||
// rebalanceThreshold is the percentage of candidate transactions under which
|
||||
// we don't rebalance. Rebalancing is a heavy operation so we prefer to avoid
|
||||
// rebalancing very often. On the other hand, if we don't rebalance often enough
|
||||
// we risk having too many collisions.
|
||||
// The value is derived from the max probability of collision. That is to say,
|
||||
// if rebalanceThreshold is 0.95, there's a 1-in-20 chance of collision.
|
||||
// See selectTxs for further details.
|
||||
rebalanceThreshold = 0.95
|
||||
)
|
||||
|
||||
type candidateTx struct {
|
||||
txDesc *TxDesc
|
||||
txValue float64
|
||||
|
||||
txMass uint64
|
||||
gasLimit uint64
|
||||
|
||||
p float64
|
||||
start float64
|
||||
end float64
|
||||
|
||||
isMarkedForDeletion bool
|
||||
}
|
||||
|
||||
type txsForBlockTemplate struct {
|
||||
selectedTxs []*util.Tx
|
||||
txMasses []uint64
|
||||
txFees []uint64
|
||||
totalMass uint64
|
||||
totalFees uint64
|
||||
}
|
||||
|
||||
// selectTxs implements a probabilistic transaction selection algorithm.
|
||||
// The algorithm, roughly, is as follows:
|
||||
// 1. We assign a probability to each transaction equal to:
|
||||
// (candidateTx.Value^alpha) / Σ(tx.Value^alpha)
|
||||
// Where the sum of the probabilities of all txs is 1.
|
||||
// 2. We draw a random number in [0,1) and select a transaction accordingly.
|
||||
// 3. If it's valid, add it to the selectedTxs and remove it from the candidates.
|
||||
// 4. Continue iterating the above until we have either selected all
|
||||
// available transactions or ran out of gas/block space.
|
||||
//
|
||||
// Note that we make two optimizations here:
|
||||
// * Draw a number in [0,Σ(tx.Value^alpha)) to avoid normalization
|
||||
// * Instead of removing a candidate after each iteration, mark it for deletion.
|
||||
// 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) {
|
||||
// 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)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// Collect candidateTxs while excluding txs that will certainly not
|
||||
// be selected.
|
||||
candidateTxs := g.collectCandidatesTxs(sourceTxs)
|
||||
|
||||
log.Debugf("Considering %d transactions for inclusion to new block",
|
||||
len(candidateTxs))
|
||||
|
||||
// Choose which transactions make it into the block.
|
||||
g.populateTemplateFromCandidates(candidateTxs, txsForBlockTemplate)
|
||||
|
||||
return txsForBlockTemplate, nil
|
||||
}
|
||||
|
||||
// newTxsForBlockTemplate creates a txsForBlockTemplate and initializes it
|
||||
// with a coinbase transaction.
|
||||
func (g *BlkTmplGenerator) newTxsForBlockTemplate(payToAddress util.Address, sourceTxs []*TxDesc) (*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
|
||||
// below to simply append details about a transaction as it is selected
|
||||
// for inclusion in the final block.
|
||||
txsForBlockTemplate := &txsForBlockTemplate{
|
||||
selectedTxs: make([]*util.Tx, 0),
|
||||
txMasses: make([]uint64, 0),
|
||||
txFees: make([]uint64, 0),
|
||||
}
|
||||
|
||||
coinbasePayloadPkScript, err := txscript.PayToAddrScript(payToAddress)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
extraNonce, err := random.Uint64()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
coinbasePayloadExtraData, err := CoinbasePayloadExtraData(extraNonce)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
coinbaseTx, err := g.dag.NextBlockCoinbaseTransactionNoLock(coinbasePayloadPkScript, coinbasePayloadExtraData)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
coinbaseTxMass, err := blockdag.CalcTxMass(coinbaseTx, g.dag.UTXOSet())
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// Add the coinbase.
|
||||
txsForBlockTemplate.selectedTxs = append(txsForBlockTemplate.selectedTxs, coinbaseTx)
|
||||
txsForBlockTemplate.totalMass = coinbaseTxMass
|
||||
txsForBlockTemplate.totalFees = uint64(0)
|
||||
txsForBlockTemplate.txMasses = append(txsForBlockTemplate.txMasses, coinbaseTxMass)
|
||||
txsForBlockTemplate.txFees = append(txsForBlockTemplate.txFees, 0) // For coinbase tx
|
||||
|
||||
return txsForBlockTemplate, nil
|
||||
}
|
||||
|
||||
// collectCandidateTxs goes over the sourceTxs and collects only the ones that
|
||||
// may be included in the next block.
|
||||
func (g *BlkTmplGenerator) collectCandidatesTxs(sourceTxs []*TxDesc) []*candidateTx {
|
||||
nextBlockBlueScore := g.dag.VirtualBlueScore()
|
||||
|
||||
candidateTxs := make([]*candidateTx, 0, len(sourceTxs))
|
||||
for _, txDesc := range sourceTxs {
|
||||
tx := txDesc.Tx
|
||||
|
||||
// A block can't contain non-finalized transactions.
|
||||
if !blockdag.IsFinalizedTransaction(tx, nextBlockBlueScore,
|
||||
g.timeSource.AdjustedTime()) {
|
||||
log.Debugf("Skipping non-finalized tx %s", tx.ID())
|
||||
continue
|
||||
}
|
||||
|
||||
// A block can't contain zero-fee transactions.
|
||||
if txDesc.Fee == 0 {
|
||||
log.Warnf("Skipped zero-fee tx %s", tx.ID())
|
||||
continue
|
||||
}
|
||||
|
||||
txMass, err := blockdag.CalcTxMass(tx, g.dag.UTXOSet())
|
||||
if err != nil {
|
||||
log.Warnf("Skipping tx %s due to error in "+
|
||||
"CalcTxMass: %s", tx.ID(), err)
|
||||
continue
|
||||
}
|
||||
|
||||
gasLimit := uint64(0)
|
||||
if !tx.MsgTx().SubnetworkID.IsEqual(subnetworkid.SubnetworkIDNative) && !tx.MsgTx().SubnetworkID.IsBuiltIn() {
|
||||
subnetworkID := tx.MsgTx().SubnetworkID
|
||||
gasLimit, err = g.dag.SubnetworkStore.GasLimit(&subnetworkID)
|
||||
if err != nil {
|
||||
log.Warnf("Skipping tx %s due to error in "+
|
||||
"GasLimit: %s", tx.ID(), err)
|
||||
continue
|
||||
}
|
||||
}
|
||||
|
||||
// Calculate the tx value
|
||||
txValue, err := g.calcTxValue(tx, txDesc.Fee)
|
||||
if err != nil {
|
||||
log.Warnf("Skipping tx %s due to error in "+
|
||||
"calcTxValue: %s", tx.ID(), err)
|
||||
continue
|
||||
}
|
||||
|
||||
candidateTxs = append(candidateTxs, &candidateTx{
|
||||
txDesc: txDesc,
|
||||
txValue: txValue,
|
||||
txMass: txMass,
|
||||
gasLimit: gasLimit,
|
||||
})
|
||||
}
|
||||
|
||||
// Sort the candidate txs by subnetworkID.
|
||||
sort.Slice(candidateTxs, func(i, j int) bool {
|
||||
return subnetworkid.Less(&candidateTxs[i].txDesc.Tx.MsgTx().SubnetworkID,
|
||||
&candidateTxs[j].txDesc.Tx.MsgTx().SubnetworkID)
|
||||
})
|
||||
|
||||
return candidateTxs
|
||||
}
|
||||
|
||||
// calcTxValue calculates a value to be used in transaction selection.
|
||||
// The higher the number the more likely it is that the transaction will be
|
||||
// included in the block.
|
||||
func (g *BlkTmplGenerator) calcTxValue(tx *util.Tx, fee uint64) (float64, error) {
|
||||
mass, err := blockdag.CalcTxMass(tx, g.dag.UTXOSet())
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
massLimit := g.policy.BlockMaxMass
|
||||
|
||||
msgTx := tx.MsgTx()
|
||||
if msgTx.SubnetworkID.IsEqual(subnetworkid.SubnetworkIDNative) ||
|
||||
msgTx.SubnetworkID.IsBuiltIn() {
|
||||
return float64(fee) / (float64(mass) / float64(massLimit)), nil
|
||||
}
|
||||
|
||||
gas := msgTx.Gas
|
||||
gasLimit, err := g.dag.SubnetworkStore.GasLimit(&msgTx.SubnetworkID)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
return float64(fee) / (float64(mass)/float64(massLimit) + float64(gas)/float64(gasLimit)), nil
|
||||
}
|
||||
|
||||
// populateTemplateFromCandidates loops over the candidate transactions
|
||||
// and appends the ones that will be included in the next block into
|
||||
// txsForBlockTemplates.
|
||||
// See selectTxs for further details.
|
||||
func (g *BlkTmplGenerator) populateTemplateFromCandidates(candidateTxs []*candidateTx, txsForBlockTemplate *txsForBlockTemplate) {
|
||||
usedCount, usedP := 0, 0.0
|
||||
candidateTxs, totalP := rebalanceCandidates(candidateTxs, true)
|
||||
gasUsageMap := make(map[subnetworkid.SubnetworkID]uint64)
|
||||
|
||||
markCandidateTxForDeletion := func(candidateTx *candidateTx) {
|
||||
candidateTx.isMarkedForDeletion = true
|
||||
usedCount++
|
||||
usedP += candidateTx.p
|
||||
}
|
||||
|
||||
for len(candidateTxs)-usedCount > 0 {
|
||||
// Rebalance the candidates if it's required
|
||||
if usedP >= rebalanceThreshold*totalP {
|
||||
candidateTxs, totalP = rebalanceCandidates(candidateTxs, false)
|
||||
usedCount, usedP = 0, 0.0
|
||||
|
||||
// Break if we now ran out of transactions
|
||||
if len(candidateTxs) == 0 {
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
// Select a candidate tx at random
|
||||
r := rand.Float64()
|
||||
r *= totalP
|
||||
selectedTx := findTx(candidateTxs, r)
|
||||
|
||||
// If isMarkedForDeletion is set, it means we got a collision.
|
||||
// Ignore and select another Tx.
|
||||
if selectedTx.isMarkedForDeletion == true {
|
||||
continue
|
||||
}
|
||||
tx := selectedTx.txDesc.Tx
|
||||
|
||||
// Enforce maximum transaction mass per block. Also check
|
||||
// for overflow.
|
||||
if txsForBlockTemplate.totalMass+selectedTx.txMass < txsForBlockTemplate.totalMass ||
|
||||
txsForBlockTemplate.totalMass+selectedTx.txMass > g.policy.BlockMaxMass {
|
||||
log.Tracef("Tx %s would exceed the max block mass. "+
|
||||
"As such, stopping.", tx.ID())
|
||||
break
|
||||
}
|
||||
|
||||
// Enforce maximum gas per subnetwork per block. Also check
|
||||
// for overflow.
|
||||
if !tx.MsgTx().SubnetworkID.IsEqual(subnetworkid.SubnetworkIDNative) && !tx.MsgTx().SubnetworkID.IsBuiltIn() {
|
||||
subnetworkID := tx.MsgTx().SubnetworkID
|
||||
gasUsage, ok := gasUsageMap[subnetworkID]
|
||||
if !ok {
|
||||
gasUsage = 0
|
||||
}
|
||||
txGas := tx.MsgTx().Gas
|
||||
if gasUsage+txGas < gasUsage ||
|
||||
gasUsage+txGas > selectedTx.gasLimit {
|
||||
log.Tracef("Tx %s would exceed the gas limit in "+
|
||||
"subnetwork %s. Removing all remaining txs from this "+
|
||||
"subnetwork.",
|
||||
tx.MsgTx().TxID(), subnetworkID)
|
||||
for _, candidateTx := range candidateTxs {
|
||||
// candidateTxs are ordered by subnetwork, so we can safely assume
|
||||
// that transactions after subnetworkID will not be relevant.
|
||||
if subnetworkid.Less(&subnetworkID, &candidateTx.txDesc.Tx.MsgTx().SubnetworkID) {
|
||||
break
|
||||
}
|
||||
|
||||
if candidateTx.txDesc.Tx.MsgTx().SubnetworkID.IsEqual(&subnetworkID) {
|
||||
markCandidateTxForDeletion(candidateTx)
|
||||
}
|
||||
}
|
||||
continue
|
||||
}
|
||||
gasUsageMap[subnetworkID] = gasUsage + txGas
|
||||
}
|
||||
|
||||
// Add the transaction to the result, increment counters, and
|
||||
// save the masses, fees, and signature operation counts to the
|
||||
// result.
|
||||
txsForBlockTemplate.selectedTxs = append(txsForBlockTemplate.selectedTxs, tx)
|
||||
txsForBlockTemplate.totalMass += selectedTx.txMass
|
||||
txsForBlockTemplate.totalFees += selectedTx.txDesc.Fee
|
||||
txsForBlockTemplate.txMasses = append(txsForBlockTemplate.txMasses, selectedTx.txMass)
|
||||
txsForBlockTemplate.txFees = append(txsForBlockTemplate.txFees, selectedTx.txDesc.Fee)
|
||||
|
||||
log.Tracef("Adding tx %s (feePerKB %.2f)",
|
||||
tx.ID(), selectedTx.txDesc.FeePerKB)
|
||||
|
||||
markCandidateTxForDeletion(selectedTx)
|
||||
}
|
||||
}
|
||||
|
||||
func rebalanceCandidates(oldCandidateTxs []*candidateTx, isFirstRun bool) (
|
||||
candidateTxs []*candidateTx, totalP float64) {
|
||||
|
||||
totalP = 0.0
|
||||
|
||||
candidateTxs = make([]*candidateTx, 0, len(oldCandidateTxs))
|
||||
for _, candidateTx := range oldCandidateTxs {
|
||||
if candidateTx.isMarkedForDeletion {
|
||||
continue
|
||||
}
|
||||
|
||||
candidateTxs = append(candidateTxs, candidateTx)
|
||||
}
|
||||
|
||||
for _, candidateTx := range candidateTxs {
|
||||
if isFirstRun {
|
||||
candidateTx.p = math.Pow(candidateTx.txValue, alpha)
|
||||
}
|
||||
candidateTx.start = totalP
|
||||
candidateTx.end = totalP + candidateTx.p
|
||||
|
||||
totalP += candidateTx.p
|
||||
}
|
||||
|
||||
return
|
||||
}
|
||||
|
||||
// findTx finds the candidateTx in whose range r falls.
|
||||
// For example, if we have candidateTxs with starts and ends:
|
||||
// * tx1: start 0, end 100
|
||||
// * tx2: start 100, end 105
|
||||
// * tx3: start 105, end 2000
|
||||
// And r=102, then findTx will return tx2.
|
||||
func findTx(candidateTxs []*candidateTx, r float64) *candidateTx {
|
||||
min := 0
|
||||
max := len(candidateTxs) - 1
|
||||
for {
|
||||
i := (min + max) / 2
|
||||
candidateTx := candidateTxs[i]
|
||||
if candidateTx.end < r {
|
||||
min = i + 1
|
||||
continue
|
||||
} else if candidateTx.start > r {
|
||||
max = i - 1
|
||||
continue
|
||||
}
|
||||
return candidateTx
|
||||
}
|
||||
}
|
539
mining/txselection_test.go
Normal file
539
mining/txselection_test.go
Normal file
@ -0,0 +1,539 @@
|
||||
package mining
|
||||
|
||||
import (
|
||||
"bou.ke/monkey"
|
||||
"fmt"
|
||||
"github.com/daglabs/btcd/blockdag"
|
||||
"github.com/daglabs/btcd/dagconfig"
|
||||
"github.com/daglabs/btcd/txscript"
|
||||
"github.com/daglabs/btcd/util"
|
||||
"github.com/daglabs/btcd/util/subnetworkid"
|
||||
"github.com/daglabs/btcd/wire"
|
||||
"math"
|
||||
"testing"
|
||||
)
|
||||
|
||||
type testTxDescDefinition struct {
|
||||
fee uint64
|
||||
mass uint64
|
||||
gas uint64
|
||||
|
||||
expectedMinSelectedTimes uint64
|
||||
expectedMaxSelectedTimes uint64
|
||||
|
||||
tx *util.Tx
|
||||
}
|
||||
|
||||
func (dd testTxDescDefinition) String() string {
|
||||
return fmt.Sprintf("[fee: %d, gas: %d, mass: %d]", dd.fee, dd.gas, dd.mass)
|
||||
}
|
||||
|
||||
func TestSelectTxs(t *testing.T) {
|
||||
params := dagconfig.SimNetParams
|
||||
params.BlockCoinbaseMaturity = 0
|
||||
|
||||
dag, teardownFunc, err := blockdag.DAGSetup("TestSelectTxs", blockdag.Config{
|
||||
DAGParams: ¶ms,
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatalf("Failed to setup DAG instance: %v", err)
|
||||
}
|
||||
defer teardownFunc()
|
||||
|
||||
txSource := &fakeTxSource{
|
||||
txDescs: []*TxDesc{},
|
||||
}
|
||||
|
||||
blockTemplateGenerator := NewBlkTmplGenerator(&Policy{BlockMaxMass: 50000},
|
||||
¶ms, txSource, dag, blockdag.NewMedianTime(), txscript.NewSigCache(100000))
|
||||
|
||||
OpTrueAddr, err := OpTrueAddress(params.Prefix)
|
||||
if err != nil {
|
||||
t.Fatalf("OpTrueAddress: %s", err)
|
||||
}
|
||||
template, err := blockTemplateGenerator.NewBlockTemplate(OpTrueAddr)
|
||||
if err != nil {
|
||||
t.Fatalf("NewBlockTemplate: %v", err)
|
||||
}
|
||||
isOrphan, delay, err := dag.ProcessBlock(util.NewBlock(template.Block), blockdag.BFNoPoWCheck)
|
||||
if err != nil {
|
||||
t.Fatalf("ProcessBlock: %v", err)
|
||||
}
|
||||
if delay != 0 {
|
||||
t.Fatalf("ProcessBlock: template " +
|
||||
"is too far in the future")
|
||||
}
|
||||
if isOrphan {
|
||||
t.Fatalf("ProcessBlock: template got unexpectedly orphan")
|
||||
}
|
||||
|
||||
fakeSubnetworkID := subnetworkid.SubnetworkID{250}
|
||||
signatureScript, err := txscript.PayToScriptHashSignatureScript(blockdag.OpTrueScript, nil)
|
||||
if err != nil {
|
||||
t.Fatalf("Error creating signature script: %s", err)
|
||||
}
|
||||
pkScript, err := txscript.NewScriptBuilder().AddOp(txscript.OpTrue).Script()
|
||||
if err != nil {
|
||||
t.Fatalf("Failed to create pkScript: %v", err)
|
||||
}
|
||||
|
||||
tests := []struct {
|
||||
name string
|
||||
runTimes int
|
||||
massLimit uint64
|
||||
gasLimit uint64
|
||||
txDefinitions []*testTxDescDefinition
|
||||
}{
|
||||
{
|
||||
name: "no source txs",
|
||||
runTimes: 1,
|
||||
massLimit: 10,
|
||||
gasLimit: 10,
|
||||
txDefinitions: []*testTxDescDefinition{},
|
||||
},
|
||||
{
|
||||
name: "zero fee",
|
||||
runTimes: 1,
|
||||
massLimit: 10,
|
||||
gasLimit: 10,
|
||||
txDefinitions: []*testTxDescDefinition{
|
||||
{
|
||||
mass: 0,
|
||||
gas: 0,
|
||||
fee: 0,
|
||||
|
||||
// Expected probability: 0
|
||||
expectedMinSelectedTimes: 0,
|
||||
expectedMaxSelectedTimes: 0,
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "single transaction",
|
||||
runTimes: 1,
|
||||
massLimit: 100,
|
||||
gasLimit: 100,
|
||||
txDefinitions: []*testTxDescDefinition{
|
||||
{
|
||||
mass: 10,
|
||||
gas: 10,
|
||||
fee: 10,
|
||||
|
||||
// Expected probability: 1
|
||||
expectedMinSelectedTimes: 1,
|
||||
expectedMaxSelectedTimes: 1,
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "none fit, limited gas and mass",
|
||||
runTimes: 1,
|
||||
massLimit: 2,
|
||||
gasLimit: 2,
|
||||
txDefinitions: []*testTxDescDefinition{
|
||||
{
|
||||
mass: 10,
|
||||
gas: 10,
|
||||
fee: 100,
|
||||
|
||||
// Expected probability: 0
|
||||
expectedMinSelectedTimes: 0,
|
||||
expectedMaxSelectedTimes: 0,
|
||||
},
|
||||
{
|
||||
mass: 5,
|
||||
gas: 5,
|
||||
fee: 50,
|
||||
|
||||
// Expected probability: 0
|
||||
expectedMinSelectedTimes: 0,
|
||||
expectedMaxSelectedTimes: 0,
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "only one fits, limited gas and mass",
|
||||
runTimes: 1,
|
||||
massLimit: 2,
|
||||
gasLimit: 2,
|
||||
txDefinitions: []*testTxDescDefinition{
|
||||
{
|
||||
mass: 1,
|
||||
gas: 1,
|
||||
fee: 100,
|
||||
|
||||
// Expected probability: 1
|
||||
expectedMinSelectedTimes: 1,
|
||||
expectedMaxSelectedTimes: 1,
|
||||
},
|
||||
{
|
||||
mass: 10,
|
||||
gas: 10,
|
||||
fee: 100,
|
||||
|
||||
// Expected probability: 0
|
||||
expectedMinSelectedTimes: 0,
|
||||
expectedMaxSelectedTimes: 0,
|
||||
},
|
||||
{
|
||||
mass: 10,
|
||||
gas: 10,
|
||||
fee: 100,
|
||||
|
||||
// Expected probability: 0
|
||||
expectedMinSelectedTimes: 0,
|
||||
expectedMaxSelectedTimes: 0,
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "all fit, limited gas",
|
||||
runTimes: 1,
|
||||
massLimit: wire.MaxMassPerBlock,
|
||||
gasLimit: 10,
|
||||
txDefinitions: []*testTxDescDefinition{
|
||||
{
|
||||
mass: 100,
|
||||
gas: 1,
|
||||
fee: 100,
|
||||
|
||||
// Expected probability: 1
|
||||
expectedMinSelectedTimes: 1,
|
||||
expectedMaxSelectedTimes: 1,
|
||||
},
|
||||
{
|
||||
mass: 0,
|
||||
gas: 1,
|
||||
fee: 1,
|
||||
|
||||
// Expected probability: 1
|
||||
expectedMinSelectedTimes: 1,
|
||||
expectedMaxSelectedTimes: 1,
|
||||
},
|
||||
{
|
||||
mass: 2,
|
||||
gas: 1,
|
||||
fee: 100,
|
||||
|
||||
// Expected probability: 1
|
||||
expectedMinSelectedTimes: 1,
|
||||
expectedMaxSelectedTimes: 1,
|
||||
},
|
||||
{
|
||||
mass: 3,
|
||||
gas: 1,
|
||||
fee: 100,
|
||||
|
||||
// Expected probability: 1
|
||||
expectedMinSelectedTimes: 1,
|
||||
expectedMaxSelectedTimes: 1,
|
||||
},
|
||||
{
|
||||
mass: 4,
|
||||
gas: 1,
|
||||
fee: 100,
|
||||
|
||||
// Expected probability: 1
|
||||
expectedMinSelectedTimes: 1,
|
||||
expectedMaxSelectedTimes: 1,
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "all fit, limited mass",
|
||||
runTimes: 1,
|
||||
massLimit: 10,
|
||||
gasLimit: math.MaxUint64,
|
||||
txDefinitions: []*testTxDescDefinition{
|
||||
{
|
||||
mass: 1,
|
||||
gas: 100,
|
||||
fee: 100,
|
||||
|
||||
// Expected probability: 1
|
||||
expectedMinSelectedTimes: 1,
|
||||
expectedMaxSelectedTimes: 1,
|
||||
},
|
||||
{
|
||||
mass: 1,
|
||||
gas: 0,
|
||||
fee: 1,
|
||||
|
||||
// Expected probability: 1
|
||||
expectedMinSelectedTimes: 1,
|
||||
expectedMaxSelectedTimes: 1,
|
||||
},
|
||||
{
|
||||
mass: 1,
|
||||
gas: 2,
|
||||
fee: 100,
|
||||
|
||||
// Expected probability: 1
|
||||
expectedMinSelectedTimes: 1,
|
||||
expectedMaxSelectedTimes: 1,
|
||||
},
|
||||
{
|
||||
mass: 1,
|
||||
gas: 3,
|
||||
fee: 100,
|
||||
|
||||
// Expected probability: 1
|
||||
expectedMinSelectedTimes: 1,
|
||||
expectedMaxSelectedTimes: 1,
|
||||
},
|
||||
{
|
||||
mass: 1,
|
||||
gas: 4,
|
||||
fee: 100,
|
||||
|
||||
// Expected probability: 1
|
||||
expectedMinSelectedTimes: 1,
|
||||
expectedMaxSelectedTimes: 1,
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "equal selection probability",
|
||||
runTimes: 1000,
|
||||
massLimit: 100,
|
||||
gasLimit: 100,
|
||||
txDefinitions: []*testTxDescDefinition{
|
||||
{
|
||||
mass: 75,
|
||||
gas: 75,
|
||||
fee: 100,
|
||||
|
||||
// Expected probability: 0.25
|
||||
expectedMinSelectedTimes: 200,
|
||||
expectedMaxSelectedTimes: 300,
|
||||
},
|
||||
{
|
||||
mass: 75,
|
||||
gas: 75,
|
||||
fee: 100,
|
||||
|
||||
// Expected probability: 0.25
|
||||
expectedMinSelectedTimes: 200,
|
||||
expectedMaxSelectedTimes: 300,
|
||||
},
|
||||
{
|
||||
mass: 75,
|
||||
gas: 75,
|
||||
fee: 100,
|
||||
|
||||
// Expected probability: 0.25
|
||||
expectedMinSelectedTimes: 200,
|
||||
expectedMaxSelectedTimes: 300,
|
||||
},
|
||||
{
|
||||
mass: 75,
|
||||
gas: 75,
|
||||
fee: 100,
|
||||
|
||||
// Expected probability: 0.25
|
||||
expectedMinSelectedTimes: 200,
|
||||
expectedMaxSelectedTimes: 300,
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "unequal selection probability",
|
||||
runTimes: 1000,
|
||||
massLimit: 100,
|
||||
gasLimit: 100,
|
||||
txDefinitions: []*testTxDescDefinition{
|
||||
{
|
||||
mass: 50,
|
||||
gas: 50,
|
||||
fee: 100,
|
||||
|
||||
// Expected probability: 0.33
|
||||
expectedMinSelectedTimes: 230,
|
||||
expectedMaxSelectedTimes: 430,
|
||||
},
|
||||
{
|
||||
mass: 100,
|
||||
gas: 0,
|
||||
fee: 100,
|
||||
|
||||
// Expected probability: 0.50
|
||||
expectedMinSelectedTimes: 400,
|
||||
expectedMaxSelectedTimes: 600,
|
||||
},
|
||||
{
|
||||
mass: 0,
|
||||
gas: 100,
|
||||
fee: 100,
|
||||
|
||||
// Expected probability: 0.50
|
||||
expectedMinSelectedTimes: 400,
|
||||
expectedMaxSelectedTimes: 600,
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "distributed selection probability",
|
||||
runTimes: 100,
|
||||
massLimit: 32,
|
||||
gasLimit: 32,
|
||||
txDefinitions: []*testTxDescDefinition{
|
||||
{
|
||||
mass: 1,
|
||||
gas: 1,
|
||||
fee: 100,
|
||||
|
||||
// Expected probability: 1
|
||||
expectedMinSelectedTimes: 95,
|
||||
expectedMaxSelectedTimes: 100,
|
||||
},
|
||||
{
|
||||
mass: 2,
|
||||
gas: 2,
|
||||
fee: 100,
|
||||
|
||||
// Expected probability: 1
|
||||
expectedMinSelectedTimes: 95,
|
||||
expectedMaxSelectedTimes: 100,
|
||||
},
|
||||
{
|
||||
mass: 4,
|
||||
gas: 4,
|
||||
fee: 100,
|
||||
|
||||
// Expected probability: 1
|
||||
expectedMinSelectedTimes: 95,
|
||||
expectedMaxSelectedTimes: 100,
|
||||
},
|
||||
{
|
||||
mass: 8,
|
||||
gas: 8,
|
||||
fee: 100,
|
||||
|
||||
// Expected probability: 0.98
|
||||
expectedMinSelectedTimes: 90,
|
||||
expectedMaxSelectedTimes: 100,
|
||||
},
|
||||
{
|
||||
mass: 16,
|
||||
gas: 16,
|
||||
fee: 100,
|
||||
|
||||
// Expected probability: 0.90
|
||||
expectedMinSelectedTimes: 75,
|
||||
expectedMaxSelectedTimes: 100,
|
||||
},
|
||||
{
|
||||
mass: 32,
|
||||
gas: 32,
|
||||
fee: 100,
|
||||
|
||||
// Expected probability: 0
|
||||
expectedMinSelectedTimes: 0,
|
||||
expectedMaxSelectedTimes: 5,
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
for _, test := range tests {
|
||||
func() {
|
||||
// Force the mass limit to always be test.massLimit
|
||||
blockTemplateGenerator.policy.BlockMaxMass = test.massLimit
|
||||
|
||||
// Force the mass to be as defined in the definition.
|
||||
// We use the first payload byte to resolve which definition to use.
|
||||
massPatch := monkey.Patch(blockdag.CalcTxMass, func(tx *util.Tx, _ blockdag.UTXOSet) (uint64, error) {
|
||||
if tx.IsCoinBase() {
|
||||
return 0, nil
|
||||
}
|
||||
index := tx.MsgTx().Payload[0]
|
||||
definition := test.txDefinitions[index]
|
||||
return definition.mass, nil
|
||||
})
|
||||
defer massPatch.Unpatch()
|
||||
|
||||
// Force the gas limit to always be test.gasLimit
|
||||
gasLimitPatch := monkey.Patch((*blockdag.SubnetworkStore).GasLimit, func(_ *blockdag.SubnetworkStore, subnetworkID *subnetworkid.SubnetworkID) (uint64, error) {
|
||||
return test.gasLimit, nil
|
||||
})
|
||||
defer gasLimitPatch.Unpatch()
|
||||
|
||||
// Force the fee to be as defined in the definition.
|
||||
// We use the first payload byte to resolve which definition to use.
|
||||
feePatch := monkey.Patch(blockdag.CheckTransactionInputsAndCalulateFee, func(tx *util.Tx, _ uint64, _ blockdag.UTXOSet, _ *dagconfig.Params, _ bool) (txFeeInSatoshi uint64, err error) {
|
||||
if tx.IsCoinBase() {
|
||||
return 0, nil
|
||||
}
|
||||
index := tx.MsgTx().Payload[0]
|
||||
definition := test.txDefinitions[index]
|
||||
return definition.fee, nil
|
||||
})
|
||||
defer feePatch.Unpatch()
|
||||
|
||||
// Load the txSource with transactions as defined in test.txDefinitions.
|
||||
// Note that we're saving the definition index in the msgTx payload
|
||||
// so that we may use it in massPatch and feePatch.
|
||||
// We also initialize a map that keeps track of how many times a tx
|
||||
// has been selected.
|
||||
txSource.txDescs = make([]*TxDesc, len(test.txDefinitions))
|
||||
selectedTxCountMap := make(map[*util.Tx]uint64, len(test.txDefinitions))
|
||||
for i, definition := range test.txDefinitions {
|
||||
txIn := &wire.TxIn{
|
||||
PreviousOutpoint: wire.Outpoint{
|
||||
TxID: *template.Block.Transactions[util.CoinbaseTransactionIndex].TxID(),
|
||||
Index: 0,
|
||||
},
|
||||
Sequence: wire.MaxTxInSequenceNum,
|
||||
SignatureScript: signatureScript,
|
||||
}
|
||||
txOut := &wire.TxOut{
|
||||
PkScript: pkScript,
|
||||
Value: 1,
|
||||
}
|
||||
msgTx := wire.NewSubnetworkMsgTx(
|
||||
wire.TxVersion, []*wire.TxIn{txIn}, []*wire.TxOut{txOut},
|
||||
&fakeSubnetworkID, definition.gas, []byte{byte(i)})
|
||||
tx := util.NewTx(msgTx)
|
||||
txDesc := TxDesc{
|
||||
Fee: definition.fee,
|
||||
Tx: tx,
|
||||
}
|
||||
txSource.txDescs[i] = &txDesc
|
||||
|
||||
definition.tx = tx
|
||||
selectedTxCountMap[tx] = 0
|
||||
}
|
||||
|
||||
// Run selectTxs test.runTimes times
|
||||
for i := 0; i < test.runTimes; i++ {
|
||||
result, err := blockTemplateGenerator.selectTxs(OpTrueAddr)
|
||||
if err != nil {
|
||||
t.Errorf("selectTxs unexpectedly failed in test '%s': %s",
|
||||
test.name, err)
|
||||
return
|
||||
}
|
||||
|
||||
// Increment the counts of all the selected transactions.
|
||||
// Ignore the first transactions because it's the coinbase.
|
||||
for _, selectedTx := range result.selectedTxs[1:] {
|
||||
selectedTxCountMap[selectedTx]++
|
||||
}
|
||||
}
|
||||
|
||||
// Make sure that each transaction has not been selected either
|
||||
// too little or too much.
|
||||
for i, definition := range test.txDefinitions {
|
||||
tx := definition.tx
|
||||
count := selectedTxCountMap[tx]
|
||||
min := definition.expectedMinSelectedTimes
|
||||
max := definition.expectedMaxSelectedTimes
|
||||
if count < min || count > max {
|
||||
t.Errorf("unexpected selected tx count "+
|
||||
"in test '%s' for tx %d:%s. Want: %d <= count <= %d, got: %d. "+
|
||||
"Note that this test is probabilistic and has a low chance to erroneously fail",
|
||||
test.name, i, definition, min, max, count)
|
||||
}
|
||||
}
|
||||
}()
|
||||
}
|
||||
}
|
@ -2397,7 +2397,6 @@ func NewServer(listenAddrs []string, db database.DB, dagParams *dagconfig.Params
|
||||
AcceptNonStd: config.MainConfig().RelayNonStd,
|
||||
MaxOrphanTxs: config.MainConfig().MaxOrphanTxs,
|
||||
MaxOrphanTxSize: config.DefaultMaxOrphanTxSize,
|
||||
MaxSigOpsPerTx: blockdag.MaxSigOpsPerBlock / 5,
|
||||
MinRelayTxFee: config.MainConfig().MinRelayTxFee,
|
||||
MaxTxVersion: 1,
|
||||
},
|
||||
|
@ -1713,7 +1713,6 @@ func (state *gbtWorkState) blockTemplateResult(dag *blockdag.BlockDAG, useCoinba
|
||||
Depends: depends,
|
||||
Mass: template.TxMasses[i],
|
||||
Fee: template.Fees[i],
|
||||
SigOps: template.SigOpCounts[i],
|
||||
}
|
||||
transactions = append(transactions, resultTx)
|
||||
}
|
||||
@ -1729,7 +1728,6 @@ func (state *gbtWorkState) blockTemplateResult(dag *blockdag.BlockDAG, useCoinba
|
||||
CurTime: header.Timestamp.Unix(),
|
||||
Height: template.Height,
|
||||
ParentHashes: daghash.Strings(header.ParentHashes),
|
||||
SigOpLimit: blockdag.MaxSigOpsPerBlock,
|
||||
MassLimit: wire.MaxMassPerBlock,
|
||||
Transactions: transactions,
|
||||
AcceptedIDMerkleRoot: header.AcceptedIDMerkleRoot.String(),
|
||||
@ -1775,7 +1773,6 @@ func (state *gbtWorkState) blockTemplateResult(dag *blockdag.BlockDAG, useCoinba
|
||||
Depends: []int64{},
|
||||
Mass: template.TxMasses[0],
|
||||
Fee: template.Fees[0],
|
||||
SigOps: template.SigOpCounts[0],
|
||||
}
|
||||
|
||||
reply.CoinbaseTxn = &resultTx
|
||||
|
Loading…
x
Reference in New Issue
Block a user