From 70737e4e94e90dfdd0e05cf076dfb193805e03a6 Mon Sep 17 00:00:00 2001 From: stasatdaglabs <39559713+stasatdaglabs@users.noreply.github.com> Date: Mon, 19 Aug 2019 12:08:48 +0300 Subject: [PATCH] [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. --- blockdag/validate.go | 60 ----- btcjson/dagsvrresults.go | 2 - config/config.go | 9 + mempool/mempool.go | 25 +- mempool/mempool_test.go | 18 -- mining/mining.go | 304 +-------------------- mining/mining_test.go | 111 +------- mining/test_utils.go | 3 +- mining/txselection.go | 370 +++++++++++++++++++++++++ mining/txselection_test.go | 539 +++++++++++++++++++++++++++++++++++++ server/p2p/p2p.go | 1 - server/rpc/rpcserver.go | 3 - 12 files changed, 949 insertions(+), 496 deletions(-) create mode 100644 mining/txselection.go create mode 100644 mining/txselection_test.go diff --git a/blockdag/validate.go b/blockdag/validate.go index 16d0025cf..734c78396 100644 --- a/blockdag/validate.go +++ b/blockdag/validate.go @@ -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. diff --git a/btcjson/dagsvrresults.go b/btcjson/dagsvrresults.go index 2ce0cc8c2..32b8e3af0 100644 --- a/btcjson/dagsvrresults.go +++ b/btcjson/dagsvrresults.go @@ -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"` diff --git a/config/config.go b/config/config.go index 23272b0ae..a2b5538ef 100644 --- a/config/config.go +++ b/config/config.go @@ -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 { diff --git a/mempool/mempool.go b/mempool/mempool.go index d96412d64..d2702da43 100644 --- a/mempool/mempool.go +++ b/mempool/mempool.go @@ -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. diff --git a/mempool/mempool_test.go b/mempool/mempool_test.go index 207f8a6bb..40c3d1fac 100644 --- a/mempool/mempool_test.go +++ b/mempool/mempool_test.go @@ -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, diff --git a/mining/mining.go b/mining/mining.go index 6533604fd..3c5e22d7e 100644 --- a/mining/mining.go +++ b/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 } diff --git a/mining/mining_test.go b/mining/mining_test.go index 19f432a42..6385ce283 100644 --- a/mining/mining_test.go +++ b/mining/mining_test.go @@ -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, diff --git a/mining/test_utils.go b/mining/test_utils.go index 3ae4ba426..a40037674 100644 --- a/mining/test_utils.go +++ b/mining/test_utils.go @@ -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, } } diff --git a/mining/txselection.go b/mining/txselection.go new file mode 100644 index 000000000..a3482e9bb --- /dev/null +++ b/mining/txselection.go @@ -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 + } +} diff --git a/mining/txselection_test.go b/mining/txselection_test.go new file mode 100644 index 000000000..c3d616f69 --- /dev/null +++ b/mining/txselection_test.go @@ -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) + } + } + }() + } +} diff --git a/server/p2p/p2p.go b/server/p2p/p2p.go index 20809e808..216cb2881 100644 --- a/server/p2p/p2p.go +++ b/server/p2p/p2p.go @@ -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, }, diff --git a/server/rpc/rpcserver.go b/server/rpc/rpcserver.go index ebdc179dd..6bd2ccf84 100644 --- a/server/rpc/rpcserver.go +++ b/server/rpc/rpcserver.go @@ -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