diff --git a/README.md b/README.md index 1e236477fa..86dfd72537 100644 --- a/README.md +++ b/README.md @@ -206,7 +206,7 @@ Miner is responsible for block creation. Request from the `builder` is routed to `proposerTxCommit`. We do it in a way so all fees received by the block builder are sent to the fee recipient. * Transaction insertion is done in `fillTransactionsAlgoWorker` \ `fillTransactions`. Depending on the algorithm selected. Algo worker (greedy) inserts bundles whenever they belong in the block by effective gas price but default method inserts bundles on top of the block. - (see `--miner.algo`) + (see `--miner.algotype`) * Worker is also responsible for simulating bundles. Bundles are simulated in parallel and results are cached for the particular parent block. * `algo_greedy.go` implements logic of the block building. Bundles and transactions are sorted in the order of effective gas price then we try to insert everything into to block until gas limit is reached. Failing bundles are reverted during the insertion but txs are not. diff --git a/core/types/transaction.go b/core/types/transaction.go index 319a34431d..503e8fbe18 100644 --- a/core/types/transaction.go +++ b/core/types/transaction.go @@ -521,6 +521,44 @@ func (t *TxWithMinerFee) SBundle() *SimSBundle { return t.order.AsSBundle() } +func (t *TxWithMinerFee) Price() *big.Int { + return new(big.Int).Set(t.minerFee) +} + +func (t *TxWithMinerFee) Profit(baseFee *big.Int, gasUsed uint64) *big.Int { + if tx := t.Tx(); tx != nil { + profit := new(big.Int).Sub(tx.GasPrice(), baseFee) + if gasUsed != 0 { + profit.Mul(profit, new(big.Int).SetUint64(gasUsed)) + } else { + profit.Mul(profit, new(big.Int).SetUint64(tx.Gas())) + } + return profit + } else if bundle := t.Bundle(); bundle != nil { + return bundle.TotalEth + } else if sbundle := t.SBundle(); sbundle != nil { + return sbundle.Profit + } else { + panic("profit called on unsupported order type") + } +} + +// SetPrice sets the miner fee of the wrapped transaction. +func (t *TxWithMinerFee) SetPrice(price *big.Int) { + t.minerFee.Set(price) +} + +// SetProfit sets the profit of the wrapped transaction. +func (t *TxWithMinerFee) SetProfit(profit *big.Int) { + if bundle := t.Bundle(); bundle != nil { + bundle.TotalEth.Set(profit) + } else if sbundle := t.SBundle(); sbundle != nil { + sbundle.Profit.Set(profit) + } else { + panic("SetProfit called on unsupported order type") + } +} + // NewTxWithMinerFee creates a wrapped transaction, calculating the effective // miner gasTipCap if a base fee is provided. // Returns error in case of a negative effective miner gasTipCap. @@ -536,7 +574,7 @@ func NewTxWithMinerFee(tx *Transaction, baseFee *big.Int) (*TxWithMinerFee, erro } // NewBundleWithMinerFee creates a wrapped bundle. -func NewBundleWithMinerFee(bundle *SimulatedBundle, baseFee *big.Int) (*TxWithMinerFee, error) { +func NewBundleWithMinerFee(bundle *SimulatedBundle, _ *big.Int) (*TxWithMinerFee, error) { minerFee := bundle.MevGasPrice return &TxWithMinerFee{ order: _BundleOrder{bundle}, @@ -545,7 +583,7 @@ func NewBundleWithMinerFee(bundle *SimulatedBundle, baseFee *big.Int) (*TxWithMi } // NewSBundleWithMinerFee creates a wrapped bundle. -func NewSBundleWithMinerFee(sbundle *SimSBundle, baseFee *big.Int) (*TxWithMinerFee, error) { +func NewSBundleWithMinerFee(sbundle *SimSBundle, _ *big.Int) (*TxWithMinerFee, error) { minerFee := sbundle.MevGasPrice return &TxWithMinerFee{ order: _SBundleOrder{sbundle}, @@ -683,6 +721,34 @@ func (t *TransactionsByPriceAndNonce) Shift() { heap.Pop(&t.heads) } +// ShiftAndPushByAccountForTx attempts to update the transaction list associated with a given account address +// based on the input transaction account. If the associated account exists and has additional transactions, +// the top of the transaction list is popped and pushed to the heap. +// Note that this operation should only be performed when the head transaction on the heap is different from the +// input transaction. This operation is useful in scenarios where the current best head transaction for an account +// was already popped from the heap and we want to process the next one from the same account. +func (t *TransactionsByPriceAndNonce) ShiftAndPushByAccountForTx(tx *Transaction) { + if tx == nil { + return + } + + acc, _ := Sender(t.signer, tx) + if txs, exists := t.txs[acc]; exists && len(txs) > 0 { + if wrapped, err := NewTxWithMinerFee(txs[0], t.baseFee); err == nil { + t.txs[acc] = txs[1:] + heap.Push(&t.heads, wrapped) + } + } +} + +func (t *TransactionsByPriceAndNonce) Push(tx *TxWithMinerFee) { + if tx == nil { + return + } + + heap.Push(&t.heads, tx) +} + // Pop removes the best transaction, *not* replacing it with the next one from // the same account. This should be used when a transaction cannot be executed // and hence all subsequent ones should be discarded from the same account. diff --git a/miner/algo_common.go b/miner/algo_common.go index a7145d56a2..ad82c8dd94 100644 --- a/miner/algo_common.go +++ b/miner/algo_common.go @@ -22,10 +22,49 @@ const ( popTx = 2 ) +// defaultProfitPercentMinimum is to ensure committed transactions, bundles, sbundles don't fall below this threshold +// when profit is enforced +const defaultProfitPercentMinimum = 70 + +var ( + defaultProfitThreshold = big.NewInt(defaultProfitPercentMinimum) + defaultAlgorithmConfig = algorithmConfig{ + EnforceProfit: false, + ExpectedProfit: common.Big0, + ProfitThresholdPercent: defaultProfitThreshold, + } +) + var emptyCodeHash = common.HexToHash("c5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470") var errInterrupt = errors.New("miner worker interrupted") +// lowProfitError is returned when an order is not committed due to low profit or low effective gas price +type lowProfitError struct { + ExpectedProfit *big.Int + ActualProfit *big.Int + + ExpectedEffectiveGasPrice *big.Int + ActualEffectiveGasPrice *big.Int +} + +func (e *lowProfitError) Error() string { + return fmt.Sprintf( + "low profit: expected %v, actual %v, expected effective gas price %v, actual effective gas price %v", + e.ExpectedProfit, e.ActualProfit, e.ExpectedEffectiveGasPrice, e.ActualEffectiveGasPrice, + ) +} + +type algorithmConfig struct { + // EnforceProfit is true if we want to enforce a minimum profit threshold + // for committing a transaction based on ProfitThresholdPercent + EnforceProfit bool + // ExpectedProfit should be set on a per transaction basis when profit is enforced + ExpectedProfit *big.Int + // ProfitThresholdPercent is the minimum profit threshold for committing a transaction + ProfitThresholdPercent *big.Int +} + type chainData struct { chainConfig *params.ChainConfig chain *core.BlockChain @@ -156,6 +195,7 @@ func (envDiff *environmentDiff) commitTx(tx *types.Transaction, chData chainData receipt, newState, err := applyTransactionWithBlacklist(signer, chData.chainConfig, chData.chain, coinbase, envDiff.gasPool, envDiff.state, header, tx, &header.GasUsed, *chData.chain.GetVMConfig(), chData.blacklist) + envDiff.state = newState if err != nil { switch { @@ -163,42 +203,43 @@ func (envDiff *environmentDiff) commitTx(tx *types.Transaction, chData chainData // Pop the current out-of-gas transaction without shifting in the next from the account from, _ := types.Sender(signer, tx) log.Trace("Gas limit exceeded for current block", "sender", from) - return nil, popTx, err + return receipt, popTx, err case errors.Is(err, core.ErrNonceTooLow): // New head notification data race between the transaction pool and miner, shift from, _ := types.Sender(signer, tx) log.Trace("Skipping transaction with low nonce", "sender", from, "nonce", tx.Nonce()) - return nil, shiftTx, err + return receipt, shiftTx, err case errors.Is(err, core.ErrNonceTooHigh): // Reorg notification data race between the transaction pool and miner, skip account = from, _ := types.Sender(signer, tx) log.Trace("Skipping account with hight nonce", "sender", from, "nonce", tx.Nonce()) - return nil, popTx, err + return receipt, popTx, err case errors.Is(err, core.ErrTxTypeNotSupported): // Pop the unsupported transaction without shifting in the next from the account from, _ := types.Sender(signer, tx) log.Trace("Skipping unsupported transaction type", "sender", from, "type", tx.Type()) - return nil, popTx, err + return receipt, popTx, err default: // Strange error, discard the transaction and get the next in line (note, the // nonce-too-high clause will prevent us from executing in vain). log.Trace("Transaction failed, account skipped", "hash", tx.Hash(), "err", err) - return nil, shiftTx, err + return receipt, shiftTx, err } } envDiff.newProfit = envDiff.newProfit.Add(envDiff.newProfit, gasPrice.Mul(gasPrice, big.NewInt(int64(receipt.GasUsed)))) envDiff.newTxs = append(envDiff.newTxs, tx) envDiff.newReceipts = append(envDiff.newReceipts, receipt) + return receipt, shiftTx, nil } // Commit Bundle to env diff -func (envDiff *environmentDiff) commitBundle(bundle *types.SimulatedBundle, chData chainData, interrupt *int32) error { +func (envDiff *environmentDiff) commitBundle(bundle *types.SimulatedBundle, chData chainData, interrupt *int32, algoConf algorithmConfig) error { coinbase := envDiff.baseEnvironment.coinbase tmpEnvDiff := envDiff.copy() @@ -208,7 +249,7 @@ func (envDiff *environmentDiff) commitBundle(bundle *types.SimulatedBundle, chDa var gasUsed uint64 for _, tx := range bundle.OriginalBundle.Txs { - if tmpEnvDiff.header.BaseFee != nil && tx.Type() == 2 { + if tmpEnvDiff.header.BaseFee != nil && tx.Type() == types.DynamicFeeTxType { // Sanity check for extremely large numbers if tx.GasFeeCap().BitLen() > 256 { return core.ErrFeeCapVeryHigh @@ -264,12 +305,34 @@ func (envDiff *environmentDiff) commitBundle(bundle *types.SimulatedBundle, chDa bundleSimEffGP := new(big.Int).Set(bundle.MevGasPrice) // allow >-1% divergence - bundleActualEffGP.Mul(bundleActualEffGP, big.NewInt(100)) - bundleSimEffGP.Mul(bundleSimEffGP, big.NewInt(99)) + actualEGP := new(big.Int).Mul(bundleActualEffGP, common.Big100) // bundle actual effective gas price * 100 + simulatedEGP := new(big.Int).Mul(bundleSimEffGP, big.NewInt(99)) // bundle simulated effective gas price * 99 - if bundleSimEffGP.Cmp(bundleActualEffGP) == 1 { + if simulatedEGP.Cmp(actualEGP) > 0 { log.Trace("Bundle underpays after inclusion", "bundle", bundle.OriginalBundle.Hash) - return errors.New("bundle underpays") + return &lowProfitError{ + ExpectedEffectiveGasPrice: bundleSimEffGP, + ActualEffectiveGasPrice: bundleActualEffGP, + } + } + + if algoConf.EnforceProfit { + // if profit is enforced between simulation and actual commit, only allow ProfitThresholdPercent divergence + simulatedBundleProfit := new(big.Int).Set(bundle.TotalEth) + actualBundleProfit := new(big.Int).Mul(bundleActualEffGP, big.NewInt(int64(gasUsed))) + + // We want to make simulated profit smaller to allow for some leeway in cases where the actual profit is + // lower due to transaction ordering + simulatedProfitMultiple := new(big.Int).Mul(simulatedBundleProfit, algoConf.ProfitThresholdPercent) + actualProfitMultiple := new(big.Int).Mul(actualBundleProfit, common.Big100) + + if simulatedProfitMultiple.Cmp(actualProfitMultiple) > 0 { + log.Trace("Lower bundle profit found after inclusion", "bundle", bundle.OriginalBundle.Hash) + return &lowProfitError{ + ExpectedProfit: simulatedBundleProfit, + ActualProfit: actualBundleProfit, + } + } } *envDiff = *tmpEnvDiff @@ -395,7 +458,7 @@ func (envDiff *environmentDiff) commitPayoutTx(amount *big.Int, sender, receiver return receipt, nil } -func (envDiff *environmentDiff) commitSBundle(b *types.SimSBundle, chData chainData, interrupt *int32, key *ecdsa.PrivateKey) error { +func (envDiff *environmentDiff) commitSBundle(b *types.SimSBundle, chData chainData, interrupt *int32, key *ecdsa.PrivateKey, algoConf algorithmConfig) error { if key == nil { return errors.New("no private key provided") } @@ -423,11 +486,33 @@ func (envDiff *environmentDiff) commitSBundle(b *types.SimSBundle, chData chainD simEGP := new(big.Int).Set(b.MevGasPrice) // allow > 1% difference - gotEGP = gotEGP.Mul(gotEGP, big.NewInt(101)) - simEGP = simEGP.Mul(simEGP, common.Big100) + actualEGP := new(big.Int).Mul(gotEGP, big.NewInt(101)) + simulatedEGP := new(big.Int).Mul(simEGP, common.Big100) - if gotEGP.Cmp(simEGP) < 0 { - return fmt.Errorf("incorrect EGP: got %d, expected %d", gotEGP, simEGP) + if simulatedEGP.Cmp(actualEGP) > 0 { + return &lowProfitError{ + ExpectedEffectiveGasPrice: simEGP, + ActualEffectiveGasPrice: gotEGP, + } + } + + if algoConf.EnforceProfit { + // if profit is enforced between simulation and actual commit, only allow >-1% divergence + simulatedSbundleProfit := new(big.Int).Set(b.Profit) + actualSbundleProfit := new(big.Int).Set(coinbaseDelta) + + // We want to make simulated profit smaller to allow for some leeway in cases where the actual profit is + // lower due to transaction ordering + simulatedProfitMultiple := new(big.Int).Mul(simulatedSbundleProfit, algoConf.ProfitThresholdPercent) + actualProfitMultiple := new(big.Int).Mul(actualSbundleProfit, common.Big100) + + if simulatedProfitMultiple.Cmp(actualProfitMultiple) > 0 { + log.Trace("Lower sbundle profit found after inclusion", "sbundle", b.Bundle.Hash()) + return &lowProfitError{ + ExpectedProfit: simulatedSbundleProfit, + ActualProfit: actualSbundleProfit, + } + } } *envDiff = *tmpEnvDiff diff --git a/miner/algo_common_test.go b/miner/algo_common_test.go index ae023ca838..676d934135 100644 --- a/miner/algo_common_test.go +++ b/miner/algo_common_test.go @@ -298,7 +298,7 @@ func TestBundleCommit(t *testing.T) { t.Fatal("Failed to simulate bundle", err) } - err = envDiff.commitBundle(&simBundle, chData, nil) + err = envDiff.commitBundle(&simBundle, chData, nil, defaultAlgorithmConfig) if err != nil { t.Fatal("Failed to commit bundle", err) } @@ -408,7 +408,7 @@ func TestErrorBundleCommit(t *testing.T) { newProfitBefore := new(big.Int).Set(envDiff.newProfit) balanceBefore := envDiff.state.GetBalance(signers.addresses[2]) - err = envDiff.commitBundle(&simBundle, chData, nil) + err = envDiff.commitBundle(&simBundle, chData, nil, defaultAlgorithmConfig) if err == nil { t.Fatal("Committed failed bundle", err) } @@ -523,7 +523,7 @@ func TestGetSealingWorkAlgos(t *testing.T) { testConfig.AlgoType = ALGO_MEV_GETH }) - for _, algoType := range []AlgoType{ALGO_MEV_GETH, ALGO_GREEDY} { + for _, algoType := range []AlgoType{ALGO_MEV_GETH, ALGO_GREEDY, ALGO_GREEDY_BUCKETS} { local := new(params.ChainConfig) *local = *ethashChainConfig local.TerminalTotalDifficulty = big.NewInt(0) @@ -538,7 +538,7 @@ func TestGetSealingWorkAlgosWithProfit(t *testing.T) { testConfig.BuilderTxSigningKey = nil }) - for _, algoType := range []AlgoType{ALGO_GREEDY} { + for _, algoType := range []AlgoType{ALGO_GREEDY, ALGO_GREEDY_BUCKETS} { var err error testConfig.BuilderTxSigningKey, err = crypto.GenerateKey() require.NoError(t, err) diff --git a/miner/algo_greedy.go b/miner/algo_greedy.go index 0b36e89c45..508f4f6d52 100644 --- a/miner/algo_greedy.go +++ b/miner/algo_greedy.go @@ -22,7 +22,10 @@ type greedyBuilder struct { interrupt *int32 } -func newGreedyBuilder(chain *core.BlockChain, chainConfig *params.ChainConfig, blacklist map[common.Address]struct{}, env *environment, key *ecdsa.PrivateKey, interrupt *int32) *greedyBuilder { +func newGreedyBuilder( + chain *core.BlockChain, chainConfig *params.ChainConfig, + blacklist map[common.Address]struct{}, env *environment, key *ecdsa.PrivateKey, interrupt *int32, +) *greedyBuilder { return &greedyBuilder{ inputEnvironment: env, chainData: chainData{chainConfig, chain, blacklist}, @@ -31,10 +34,13 @@ func newGreedyBuilder(chain *core.BlockChain, chainConfig *params.ChainConfig, b } } -func (b *greedyBuilder) mergeOrdersIntoEnvDiff(envDiff *environmentDiff, orders *types.TransactionsByPriceAndNonce) ([]types.SimulatedBundle, []types.UsedSBundle) { - usedBundles := []types.SimulatedBundle{} - usedSbundles := []types.UsedSBundle{} - +func (b *greedyBuilder) mergeOrdersIntoEnvDiff( + envDiff *environmentDiff, orders *types.TransactionsByPriceAndNonce) ([]types.SimulatedBundle, []types.UsedSBundle, +) { + var ( + usedBundles []types.SimulatedBundle + usedSbundles []types.UsedSBundle + ) for { order := orders.Peek() if order == nil { @@ -60,7 +66,7 @@ func (b *greedyBuilder) mergeOrdersIntoEnvDiff(envDiff *environmentDiff, orders } } else if bundle := order.Bundle(); bundle != nil { //log.Debug("buildBlock considering bundle", "egp", bundle.MevGasPrice.String(), "hash", bundle.OriginalBundle.Hash) - err := envDiff.commitBundle(bundle, b.chainData, b.interrupt) + err := envDiff.commitBundle(bundle, b.chainData, b.interrupt, defaultAlgorithmConfig) orders.Pop() if err != nil { log.Trace("Could not apply bundle", "bundle", bundle.OriginalBundle.Hash, "err", err) @@ -73,7 +79,7 @@ func (b *greedyBuilder) mergeOrdersIntoEnvDiff(envDiff *environmentDiff, orders usedEntry := types.UsedSBundle{ Bundle: sbundle.Bundle, } - err := envDiff.commitSBundle(sbundle, b.chainData, b.interrupt, b.builderKey) + err := envDiff.commitSBundle(sbundle, b.chainData, b.interrupt, b.builderKey, defaultAlgorithmConfig) orders.Pop() if err != nil { log.Trace("Could not apply sbundle", "bundle", sbundle.Bundle.Hash(), "err", err) @@ -87,7 +93,6 @@ func (b *greedyBuilder) mergeOrdersIntoEnvDiff(envDiff *environmentDiff, orders usedSbundles = append(usedSbundles, usedEntry) } } - return usedBundles, usedSbundles } diff --git a/miner/algo_greedy_buckets.go b/miner/algo_greedy_buckets.go new file mode 100644 index 0000000000..6cac40db7f --- /dev/null +++ b/miner/algo_greedy_buckets.go @@ -0,0 +1,257 @@ +package miner + +import ( + "crypto/ecdsa" + "errors" + "math/big" + "sort" + + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/core" + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/log" + "github.com/ethereum/go-ethereum/params" +) + +// / To use it: +// / 1. Copy relevant data from the worker +// / 2. Call buildBlock +// / 2. If new bundles, txs arrive, call buildBlock again +// / This struct lifecycle is tied to 1 block-building task +type greedyBucketsBuilder struct { + inputEnvironment *environment + chainData chainData + builderKey *ecdsa.PrivateKey + interrupt *int32 + gasUsedMap map[*types.TxWithMinerFee]uint64 + algoConf algorithmConfig +} + +func newGreedyBucketsBuilder( + chain *core.BlockChain, chainConfig *params.ChainConfig, algoConf *algorithmConfig, + blacklist map[common.Address]struct{}, env *environment, key *ecdsa.PrivateKey, interrupt *int32, +) *greedyBucketsBuilder { + if algoConf == nil { + algoConf = &algorithmConfig{ + EnforceProfit: true, + ExpectedProfit: nil, + ProfitThresholdPercent: defaultProfitThreshold, + } + } + return &greedyBucketsBuilder{ + inputEnvironment: env, + chainData: chainData{chainConfig: chainConfig, chain: chain, blacklist: blacklist}, + builderKey: key, + interrupt: interrupt, + gasUsedMap: make(map[*types.TxWithMinerFee]uint64), + algoConf: *algoConf, + } +} + +// CutoffPriceFromOrder returns the cutoff price for a given order based on the cutoff percent. +// For example, if the cutoff percent is 0.9, the cutoff price will be 90% of the order price, rounded down to the nearest integer. +func CutoffPriceFromOrder(order *types.TxWithMinerFee, cutoffPercent *big.Float) *big.Int { + floorPrice := new(big.Float). + Mul( + new(big.Float).SetInt(order.Price()), + cutoffPercent, + ) + round, _ := floorPrice.Int64() + return big.NewInt(round) +} + +// IsOrderInPriceRange returns true if the order price is greater than or equal to the minPrice. +func IsOrderInPriceRange(order *types.TxWithMinerFee, minPrice *big.Int) bool { + return order.Price().Cmp(minPrice) >= 0 +} + +func (b *greedyBucketsBuilder) commit(envDiff *environmentDiff, + transactions []*types.TxWithMinerFee, + orders *types.TransactionsByPriceAndNonce, + gasUsedMap map[*types.TxWithMinerFee]uint64, retryMap map[*types.TxWithMinerFee]int, retryLimit int, +) ([]types.SimulatedBundle, []types.UsedSBundle) { + var ( + usedBundles []types.SimulatedBundle + usedSbundles []types.UsedSBundle + algoConf = b.algoConf + + CheckRetryOrderAndReinsert = func( + order *types.TxWithMinerFee, orders *types.TransactionsByPriceAndNonce, + retryMap map[*types.TxWithMinerFee]int, retryLimit int, + ) bool { + var isRetryable bool = false + if retryCount, exists := retryMap[order]; exists { + if retryCount != retryLimit { + isRetryable = true + retryMap[order] = retryCount + 1 + } + } else { + retryMap[order] = 0 + isRetryable = true + } + + if isRetryable { + orders.Push(order) + } + + return isRetryable + } + ) + + for _, order := range transactions { + if tx := order.Tx(); tx != nil { + receipt, skip, err := envDiff.commitTx(tx, b.chainData) + if err != nil { + log.Trace("could not apply tx", "hash", tx.Hash(), "err", err) + + // attempt to retry transaction commit up to retryLimit + // the gas used is set for the order to re-calculate profit of the transaction for subsequent retries + if receipt != nil { + // if the receipt is nil we don't attempt to retry the transaction - this is to mitigate abuse since + // without a receipt the default profit calculation for a transaction uses the gas limit which + // can cause the transaction to always be first in any profit-sorted transaction list + gasUsedMap[order] = receipt.GasUsed + CheckRetryOrderAndReinsert(order, orders, retryMap, retryLimit) + } + continue + } + + if skip == shiftTx { + orders.ShiftAndPushByAccountForTx(tx) + } + + effGapPrice, err := tx.EffectiveGasTip(envDiff.baseEnvironment.header.BaseFee) + if err == nil { + log.Trace("Included tx", "EGP", effGapPrice.String(), "gasUsed", receipt.GasUsed) + } + } else if bundle := order.Bundle(); bundle != nil { + err := envDiff.commitBundle(bundle, b.chainData, b.interrupt, algoConf) + if err != nil { + log.Trace("Could not apply bundle", "bundle", bundle.OriginalBundle.Hash, "err", err) + + var e *lowProfitError + if errors.As(err, &e) { + if e.ActualEffectiveGasPrice != nil { + order.SetPrice(e.ActualEffectiveGasPrice) + } + + if e.ActualProfit != nil { + order.SetProfit(e.ActualProfit) + } + // if the bundle was not included due to low profit, we can retry the bundle + CheckRetryOrderAndReinsert(order, orders, retryMap, retryLimit) + } + continue + } + + log.Trace("Included bundle", "bundleEGP", bundle.MevGasPrice.String(), + "gasUsed", bundle.TotalGasUsed, "ethToCoinbase", ethIntToFloat(bundle.TotalEth)) + usedBundles = append(usedBundles, *bundle) + } else if sbundle := order.SBundle(); sbundle != nil { + usedEntry := types.UsedSBundle{ + Bundle: sbundle.Bundle, + } + err := envDiff.commitSBundle(sbundle, b.chainData, b.interrupt, b.builderKey, algoConf) + if err != nil { + log.Trace("Could not apply sbundle", "bundle", sbundle.Bundle.Hash(), "err", err) + + var e *lowProfitError + if errors.As(err, &e) { + if e.ActualEffectiveGasPrice != nil { + order.SetPrice(e.ActualEffectiveGasPrice) + } + + if e.ActualProfit != nil { + order.SetProfit(e.ActualProfit) + } + + // if the sbundle was not included due to low profit, we can retry the bundle + if ok := CheckRetryOrderAndReinsert(order, orders, retryMap, retryLimit); !ok { + usedEntry.Success = false + usedSbundles = append(usedSbundles, usedEntry) + } + } + continue + } + + log.Trace("Included sbundle", "bundleEGP", sbundle.MevGasPrice.String(), "ethToCoinbase", ethIntToFloat(sbundle.Profit)) + usedEntry.Success = true + usedSbundles = append(usedSbundles, usedEntry) + } else { + // note: this should never happen because we should not be inserting invalid transaction types into + // the orders heap + panic("unsupported order type found") + } + } + return usedBundles, usedSbundles +} + +func (b *greedyBucketsBuilder) mergeOrdersIntoEnvDiff( + envDiff *environmentDiff, orders *types.TransactionsByPriceAndNonce) ([]types.SimulatedBundle, []types.UsedSBundle, +) { + if orders.Peek() == nil { + return nil, nil + } + + const retryLimit = 1 + + var ( + baseFee = envDiff.baseEnvironment.header.BaseFee + retryMap = make(map[*types.TxWithMinerFee]int) + usedBundles []types.SimulatedBundle + usedSbundles []types.UsedSBundle + transactions []*types.TxWithMinerFee + percent = new(big.Float).Quo( + new(big.Float).SetInt(b.algoConf.ProfitThresholdPercent), + new(big.Float).SetInt(common.Big100), + ) + + SortInPlaceByProfit = func(baseFee *big.Int, transactions []*types.TxWithMinerFee, gasUsedMap map[*types.TxWithMinerFee]uint64) { + sort.SliceStable(transactions, func(i, j int) bool { + return transactions[i].Profit(baseFee, gasUsedMap[transactions[i]]).Cmp(transactions[j].Profit(baseFee, gasUsedMap[transactions[j]])) > 0 + }) + } + ) + + minPrice := CutoffPriceFromOrder(orders.Peek(), percent) + for { + order := orders.Peek() + if order == nil { + if len(transactions) != 0 { + SortInPlaceByProfit(baseFee, transactions, b.gasUsedMap) + bundles, sbundles := b.commit(envDiff, transactions, orders, b.gasUsedMap, retryMap, retryLimit) + usedBundles = append(usedBundles, bundles...) + usedSbundles = append(usedSbundles, sbundles...) + transactions = nil + // re-run since committing transactions may have pushed higher nonce transactions, or previously + // failed transactions back into orders heap + continue + } + break + } + + if ok := IsOrderInPriceRange(order, minPrice); ok { + orders.Pop() + transactions = append(transactions, order) + } else { + if len(transactions) != 0 { + SortInPlaceByProfit(baseFee, transactions, b.gasUsedMap) + bundles, sbundles := b.commit(envDiff, transactions, orders, b.gasUsedMap, retryMap, retryLimit) + usedBundles = append(usedBundles, bundles...) + usedSbundles = append(usedSbundles, sbundles...) + transactions = nil + } + minPrice = CutoffPriceFromOrder(order, percent) + } + } + + return usedBundles, usedSbundles +} + +func (b *greedyBucketsBuilder) buildBlock(simBundles []types.SimulatedBundle, simSBundles []*types.SimSBundle, transactions map[common.Address]types.Transactions) (*environment, []types.SimulatedBundle, []types.UsedSBundle) { + orders := types.NewTransactionsByPriceAndNonce(b.inputEnvironment.signer, transactions, simBundles, simSBundles, b.inputEnvironment.header.BaseFee) + envDiff := newEnvironmentDiff(b.inputEnvironment.copy()) + usedBundles, usedSbundles := b.mergeOrdersIntoEnvDiff(envDiff, orders) + envDiff.applyToBaseEnv() + return envDiff.baseEnvironment, usedBundles, usedSbundles +} diff --git a/miner/algo_greedy_test.go b/miner/algo_greedy_test.go index fde54c1a4b..8b84cf05ee 100644 --- a/miner/algo_greedy_test.go +++ b/miner/algo_greedy_test.go @@ -2,34 +2,45 @@ package miner import ( "fmt" + "math" "math/big" "testing" "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/core/types" - "github.com/ethereum/go-ethereum/log" ) func TestBuildBlockGasLimit(t *testing.T) { - statedb, chData, signers := genTestSetup() - - env := newEnvironment(chData, statedb, signers.addresses[0], 21000, big.NewInt(1)) - - txs := make(map[common.Address]types.Transactions) - - txs[signers.addresses[1]] = types.Transactions{ - signers.signTx(1, 21000, big.NewInt(0), big.NewInt(1), signers.addresses[2], big.NewInt(0), []byte{}), - } - txs[signers.addresses[2]] = types.Transactions{ - signers.signTx(2, 21000, big.NewInt(0), big.NewInt(1), signers.addresses[2], big.NewInt(0), []byte{}), - } + algos := []AlgoType{ALGO_GREEDY, ALGO_GREEDY_BUCKETS} + for _, algo := range algos { + statedb, chData, signers := genTestSetup() + env := newEnvironment(chData, statedb, signers.addresses[0], 21000, big.NewInt(1)) + txs := make(map[common.Address]types.Transactions) + + txs[signers.addresses[1]] = types.Transactions{ + signers.signTx(1, 21000, big.NewInt(0), big.NewInt(1), signers.addresses[2], big.NewInt(0), []byte{}), + } + txs[signers.addresses[2]] = types.Transactions{ + signers.signTx(2, 21000, big.NewInt(0), big.NewInt(1), signers.addresses[2], big.NewInt(0), []byte{}), + } + txs[signers.addresses[3]] = types.Transactions{ + signers.signTx(3, 21000, big.NewInt(math.MaxInt), big.NewInt(math.MaxInt), signers.addresses[2], big.NewInt(math.MaxInt), []byte{}), + } - builder := newGreedyBuilder(chData.chain, chData.chainConfig, nil, env, nil, nil) + var result *environment + switch algo { + case ALGO_GREEDY_BUCKETS: + builder := newGreedyBuilder(chData.chain, chData.chainConfig, nil, env, nil, nil) + result, _, _ = builder.buildBlock([]types.SimulatedBundle{}, nil, txs) + case ALGO_GREEDY: + builder := newGreedyBucketsBuilder(chData.chain, chData.chainConfig, nil, nil, env, nil, nil) + result, _, _ = builder.buildBlock([]types.SimulatedBundle{}, nil, txs) + } - result, _, _ := builder.buildBlock([]types.SimulatedBundle{}, nil, txs) - log.Info("block built", "txs", len(result.txs), "gasPool", result.gasPool.Gas()) - if result.tcount != 1 { - t.Fatal("Incorrect tx count") + t.Log("block built", "txs", len(result.txs), "gasPool", result.gasPool.Gas(), "algorithm", algo.String()) + if result.tcount != 1 { + t.Fatalf("Incorrect tx count [found: %d]", result.tcount) + } } } diff --git a/miner/algo_test.go b/miner/algo_test.go index cf35ab90a6..09ce0bef5f 100644 --- a/miner/algo_test.go +++ b/miner/algo_test.go @@ -37,7 +37,8 @@ var algoTests = []*algoTest{ }, } }, - WantProfit: big.NewInt(2 * 21_000), + WantProfit: big.NewInt(2 * 21_000), + SupportedAlgorithms: []AlgoType{ALGO_GREEDY, ALGO_GREEDY_BUCKETS}, }, { // Trivial tx pool with 3 txs by two accounts and a block gas limit that only allows two txs @@ -62,7 +63,8 @@ var algoTests = []*algoTest{ }, } }, - WantProfit: big.NewInt(4 * 21_000), + WantProfit: big.NewInt(4 * 21_000), + SupportedAlgorithms: []AlgoType{ALGO_GREEDY, ALGO_GREEDY_BUCKETS}, }, { // Trivial bundle with one tx that reverts but is not allowed to revert. @@ -79,7 +81,8 @@ var algoTests = []*algoTest{ {Txs: types.Transactions{sign(0, &types.LegacyTx{Nonce: 0, Gas: 50_000, To: acc(1), GasPrice: big.NewInt(1)})}}, } }, - WantProfit: big.NewInt(0), + WantProfit: big.NewInt(0), + SupportedAlgorithms: []AlgoType{ALGO_GREEDY, ALGO_GREEDY_BUCKETS}, }, { // Trivial bundle with one tx that reverts and is allowed to revert. @@ -99,7 +102,8 @@ var algoTests = []*algoTest{ }, } }, - WantProfit: big.NewInt(50_000), + WantProfit: big.NewInt(50_000), + SupportedAlgorithms: []AlgoType{ALGO_GREEDY, ALGO_GREEDY_BUCKETS}, }, { // Single failing tx that is included in the tx pool and in a bundle that is not allowed to @@ -124,7 +128,8 @@ var algoTests = []*algoTest{ {Txs: types.Transactions{txs(0, 0)}}, } }, - WantProfit: big.NewInt(50_000), + WantProfit: big.NewInt(50_000), + SupportedAlgorithms: []AlgoType{ALGO_GREEDY, ALGO_GREEDY_BUCKETS}, }, } @@ -135,24 +140,27 @@ func TestAlgo(t *testing.T) { ) for _, test := range algoTests { - t.Run(test.Name, func(t *testing.T) { - alloc, txPool, bundles, err := test.build(signer, 1) - if err != nil { - t.Fatalf("Build: %v", err) - } - simBundles, err := simulateBundles(config, test.Header, alloc, bundles) - if err != nil { - t.Fatalf("Simulate Bundles: %v", err) - } + for _, algo := range test.SupportedAlgorithms { + testName := fmt.Sprintf("%s-%s", test.Name, algo.String()) + t.Run(testName, func(t *testing.T) { + alloc, txPool, bundles, err := test.build(signer, 1) + if err != nil { + t.Fatalf("Build: %v", err) + } + simBundles, err := simulateBundles(config, test.Header, alloc, bundles) + if err != nil { + t.Fatalf("Simulate Bundles: %v", err) + } - gotProfit, err := runAlgoTest(config, alloc, txPool, simBundles, test.Header, 1) - if err != nil { - t.Fatal(err) - } - if test.WantProfit.Cmp(gotProfit) != 0 { - t.Fatalf("Profit: want %v, got %v", test.WantProfit, gotProfit) - } - }) + gotProfit, err := runAlgoTest(algo, config, alloc, txPool, simBundles, test.Header, 1) + if err != nil { + t.Fatal(err) + } + if test.WantProfit.Cmp(gotProfit) != 0 { + t.Fatalf("Profit: want %v, got %v", test.WantProfit, gotProfit) + } + }) + } } } @@ -164,59 +172,69 @@ func BenchmarkAlgo(b *testing.B) { ) for _, test := range algoTests { - for _, scale := range scales { - wantScaledProfit := new(big.Int).Mul( - big.NewInt(int64(scale)), - test.WantProfit, - ) - - b.Run(fmt.Sprintf("%s_%d", test.Name, scale), func(b *testing.B) { - alloc, txPool, bundles, err := test.build(signer, scale) - if err != nil { - b.Fatalf("Build: %v", err) - } - simBundles, err := simulateBundles(config, test.Header, alloc, bundles) - if err != nil { - b.Fatalf("Simulate Bundles: %v", err) - } - - b.ResetTimer() - var txPoolCopy map[common.Address]types.Transactions - for i := 0; i < b.N; i++ { - // Note: copy is needed as the greedyAlgo modifies the txPool. - func() { - b.StopTimer() - defer b.StartTimer() - - txPoolCopy = make(map[common.Address]types.Transactions, len(txPool)) - for addr, txs := range txPool { - txPoolCopy[addr] = txs - } - }() - - gotProfit, err := runAlgoTest(config, alloc, txPoolCopy, simBundles, test.Header, scale) + for _, algo := range test.SupportedAlgorithms { + for _, scale := range scales { + wantScaledProfit := new(big.Int).Mul( + big.NewInt(int64(scale)), + test.WantProfit, + ) + + b.Run(fmt.Sprintf("%s-%s-%d", test.Name, algo.String(), scale), func(b *testing.B) { + alloc, txPool, bundles, err := test.build(signer, scale) if err != nil { - b.Fatal(err) + b.Fatalf("Build: %v", err) } - if wantScaledProfit.Cmp(gotProfit) != 0 { - b.Fatalf("Profit: want %v, got %v", wantScaledProfit, gotProfit) + simBundles, err := simulateBundles(config, test.Header, alloc, bundles) + if err != nil { + b.Fatalf("Simulate Bundles: %v", err) } - } - }) + + b.ResetTimer() + var txPoolCopy map[common.Address]types.Transactions + for i := 0; i < b.N; i++ { + // Note: copy is needed as the greedyAlgo modifies the txPool. + func() { + b.StopTimer() + defer b.StartTimer() + + txPoolCopy = make(map[common.Address]types.Transactions, len(txPool)) + for addr, txs := range txPool { + txPoolCopy[addr] = txs + } + }() + + gotProfit, err := runAlgoTest(algo, config, alloc, txPoolCopy, simBundles, test.Header, scale) + if err != nil { + b.Fatal(err) + } + if wantScaledProfit.Cmp(gotProfit) != 0 { + b.Fatalf("Profit: want %v, got %v", wantScaledProfit, gotProfit) + } + } + }) + } } } } // runAlgo executes a single algoTest case and returns the profit. -func runAlgoTest(config *params.ChainConfig, alloc core.GenesisAlloc, txPool map[common.Address]types.Transactions, bundles []types.SimulatedBundle, header *types.Header, scale int) (gotProfit *big.Int, err error) { +func runAlgoTest(algo AlgoType, config *params.ChainConfig, alloc core.GenesisAlloc, + txPool map[common.Address]types.Transactions, bundles []types.SimulatedBundle, header *types.Header, scale int) (gotProfit *big.Int, err error) { var ( statedb, chData = genTestSetupWithAlloc(config, alloc) env = newEnvironment(chData, statedb, header.Coinbase, header.GasLimit*uint64(scale), header.BaseFee) - builder = newGreedyBuilder(chData.chain, chData.chainConfig, nil, env, nil, nil) + resultEnv *environment ) // build block - resultEnv, _, _ := builder.buildBlock(bundles, nil, txPool) + switch algo { + case ALGO_GREEDY_BUCKETS: + builder := newGreedyBucketsBuilder(chData.chain, chData.chainConfig, nil, nil, env, nil, nil) + resultEnv, _, _ = builder.buildBlock(bundles, nil, txPool) + case ALGO_GREEDY: + builder := newGreedyBuilder(chData.chain, chData.chainConfig, nil, env, nil, nil) + resultEnv, _, _ = builder.buildBlock(bundles, nil, txPool) + } return resultEnv.profit, nil } @@ -260,6 +278,8 @@ type algoTest struct { Bundles func(accByIndex, signByIndex, txByAccIndexAndNonce) []*bundle WantProfit *big.Int // Expected block profit + + SupportedAlgorithms []AlgoType } // setDefaults sets default values for the algoTest. diff --git a/miner/miner.go b/miner/miner.go index 3535673116..4c8e908fd6 100644 --- a/miner/miner.go +++ b/miner/miner.go @@ -53,13 +53,29 @@ type AlgoType int const ( ALGO_MEV_GETH AlgoType = iota ALGO_GREEDY + ALGO_GREEDY_BUCKETS ) +func (a AlgoType) String() string { + switch a { + case ALGO_GREEDY: + return "greedy" + case ALGO_MEV_GETH: + return "mev-geth" + case ALGO_GREEDY_BUCKETS: + return "greedy-buckets" + default: + return "unsupported" + } +} + func AlgoTypeFlagToEnum(algoString string) (AlgoType, error) { - switch algoString { - case "mev-geth": + switch strings.ToLower(algoString) { + case ALGO_MEV_GETH.String(): return ALGO_MEV_GETH, nil - case "greedy": + case ALGO_GREEDY_BUCKETS.String(): + return ALGO_GREEDY_BUCKETS, nil + case ALGO_GREEDY.String(): return ALGO_GREEDY, nil default: return ALGO_MEV_GETH, errors.New("algo not recognized") diff --git a/miner/multi_worker.go b/miner/multi_worker.go index ca942090ef..93cb8aadae 100644 --- a/miner/multi_worker.go +++ b/miner/multi_worker.go @@ -138,10 +138,13 @@ func (w *multiWorker) buildPayload(args *BuildPayloadArgs) (*Payload, error) { } func newMultiWorker(config *Config, chainConfig *params.ChainConfig, engine consensus.Engine, eth Backend, mux *event.TypeMux, isLocalBlock func(header *types.Header) bool, init bool) *multiWorker { - if config.AlgoType != ALGO_MEV_GETH { - return newMultiWorkerGreedy(config, chainConfig, engine, eth, mux, isLocalBlock, init) - } else { + switch config.AlgoType { + case ALGO_MEV_GETH: return newMultiWorkerMevGeth(config, chainConfig, engine, eth, mux, isLocalBlock, init) + case ALGO_GREEDY, ALGO_GREEDY_BUCKETS: + return newMultiWorkerGreedy(config, chainConfig, engine, eth, mux, isLocalBlock, init) + default: + panic("unsupported builder algorithm found") } } diff --git a/miner/sbundle_test.go b/miner/sbundle_test.go index f44b755bb2..4278db2461 100644 --- a/miner/sbundle_test.go +++ b/miner/sbundle_test.go @@ -480,7 +480,7 @@ func TestSBundles(t *testing.T) { MevGasPrice: big.NewInt(1), Profit: big.NewInt(1), } - err = envDiff.commitSBundle(&sim, chData, nil, builderPrivKey) + err = envDiff.commitSBundle(&sim, chData, nil, builderPrivKey, defaultAlgorithmConfig) if tt.ShouldFail { require.Error(t, err) } else { diff --git a/miner/worker.go b/miner/worker.go index 136f3ca717..56021b7b49 100644 --- a/miner/worker.go +++ b/miner/worker.go @@ -1303,7 +1303,7 @@ func (w *worker) prepareWork(genParams *generateParams) (*environment, error) { return env, nil } -func (w *worker) fillTransactionsSelectAlgo(interrupt *int32, env *environment) (error, []types.SimulatedBundle, []types.SimulatedBundle, []types.UsedSBundle) { +func (w *worker) fillTransactionsSelectAlgo(interrupt *int32, env *environment) ([]types.SimulatedBundle, []types.SimulatedBundle, []types.UsedSBundle, error) { var ( blockBundles []types.SimulatedBundle allBundles []types.SimulatedBundle @@ -1311,21 +1311,21 @@ func (w *worker) fillTransactionsSelectAlgo(interrupt *int32, env *environment) err error ) switch w.flashbots.algoType { - case ALGO_GREEDY: - err, blockBundles, allBundles, usedSbundles = w.fillTransactionsAlgoWorker(interrupt, env) + case ALGO_GREEDY, ALGO_GREEDY_BUCKETS: + blockBundles, allBundles, usedSbundles, err = w.fillTransactionsAlgoWorker(interrupt, env) case ALGO_MEV_GETH: - err, blockBundles, allBundles = w.fillTransactions(interrupt, env) + blockBundles, allBundles, err = w.fillTransactions(interrupt, env) default: - err, blockBundles, allBundles = w.fillTransactions(interrupt, env) + blockBundles, allBundles, err = w.fillTransactions(interrupt, env) } - return err, blockBundles, allBundles, usedSbundles + return blockBundles, allBundles, usedSbundles, err } // fillTransactions retrieves the pending transactions from the txpool and fills them // into the given sealing block. The transaction selection and ordering strategy can // be customized with the plugin in the future. // Returns error if any, otherwise the bundles that made it into the block and all bundles that passed simulation -func (w *worker) fillTransactions(interrupt *int32, env *environment) (error, []types.SimulatedBundle, []types.SimulatedBundle) { +func (w *worker) fillTransactions(interrupt *int32, env *environment) ([]types.SimulatedBundle, []types.SimulatedBundle, error) { // Split the pending transactions into locals and remotes // Fill the block with all available pending transactions. pending := w.eth.TxPool().Pending(true) @@ -1343,23 +1343,25 @@ func (w *worker) fillTransactions(interrupt *int32, env *environment) (error, [] bundles, ccBundleCh := w.eth.TxPool().MevBundles(env.header.Number, env.header.Time) bundles = append(bundles, <-ccBundleCh...) - var bundleTxs types.Transactions - var resultingBundle simulatedBundle - var mergedBundles []types.SimulatedBundle - var numBundles int - var err error + var ( + bundleTxs types.Transactions + resultingBundle simulatedBundle + mergedBundles []types.SimulatedBundle + numBundles int + err error + ) // Sets allBundles in outer scope bundleTxs, resultingBundle, mergedBundles, numBundles, allBundles, err = w.generateFlashbotsBundle(env, bundles, pending) if err != nil { log.Error("Failed to generate flashbots bundle", "err", err) - return err, nil, nil + return nil, nil, err } log.Info("Flashbots bundle", "ethToCoinbase", ethIntToFloat(resultingBundle.TotalEth), "gasUsed", resultingBundle.TotalGasUsed, "bundleScore", resultingBundle.MevGasPrice, "bundleLength", len(bundleTxs), "numBundles", numBundles, "worker", w.flashbots.maxMergedBundles) if len(bundleTxs) == 0 { - return errors.New("no bundles to apply"), nil, nil + return nil, nil, errors.New("no bundles to apply") } if err := w.commitBundle(env, bundleTxs, interrupt); err != nil { - return err, nil, nil + return nil, nil, err } blockBundles = mergedBundles env.profit.Add(env.profit, resultingBundle.EthSentToCoinbase) @@ -1368,40 +1370,69 @@ func (w *worker) fillTransactions(interrupt *int32, env *environment) (error, [] if len(localTxs) > 0 { txs := types.NewTransactionsByPriceAndNonce(env.signer, localTxs, nil, nil, env.header.BaseFee) if err := w.commitTransactions(env, txs, interrupt); err != nil { - return err, nil, nil + return nil, nil, err } } if len(remoteTxs) > 0 { txs := types.NewTransactionsByPriceAndNonce(env.signer, remoteTxs, nil, nil, env.header.BaseFee) if err := w.commitTransactions(env, txs, interrupt); err != nil { - return err, nil, nil + return nil, nil, err } } - return nil, blockBundles, allBundles + return blockBundles, allBundles, nil } // fillTransactionsAlgoWorker retrieves the pending transactions and bundles from the txpool and fills them // into the given sealing block. // Returns error if any, otherwise the bundles that made it into the block and all bundles that passed simulation -func (w *worker) fillTransactionsAlgoWorker(interrupt *int32, env *environment) (error, []types.SimulatedBundle, []types.SimulatedBundle, []types.UsedSBundle) { +func (w *worker) fillTransactionsAlgoWorker(interrupt *int32, env *environment) ([]types.SimulatedBundle, []types.SimulatedBundle, []types.UsedSBundle, error) { // Split the pending transactions into locals and remotes // Fill the block with all available pending transactions. pending := w.eth.TxPool().Pending(true) bundlesToConsider, sbundlesToConsider, err := w.getSimulatedBundles(env) if err != nil { - return err, nil, nil, nil + return nil, nil, nil, err } - builder := newGreedyBuilder(w.chain, w.chainConfig, w.blockList, env, w.config.BuilderTxSigningKey, interrupt) + var ( + newEnv *environment + blockBundles []types.SimulatedBundle + usedSbundle []types.UsedSBundle + ) + start := time.Now() - newEnv, blockBundles, usedSbundle := builder.buildBlock(bundlesToConsider, sbundlesToConsider, pending) + + switch w.flashbots.algoType { + case ALGO_GREEDY_BUCKETS: + validationConf := &algorithmConfig{ + EnforceProfit: true, + ExpectedProfit: nil, + ProfitThresholdPercent: defaultProfitThreshold, + } + builder := newGreedyBucketsBuilder( + w.chain, w.chainConfig, validationConf, w.blockList, env, + w.config.BuilderTxSigningKey, interrupt, + ) + + newEnv, blockBundles, usedSbundle = builder.buildBlock(bundlesToConsider, sbundlesToConsider, pending) + case ALGO_GREEDY: + fallthrough + default: + builder := newGreedyBuilder( + w.chain, w.chainConfig, w.blockList, env, + w.config.BuilderTxSigningKey, interrupt, + ) + + newEnv, blockBundles, usedSbundle = builder.buildBlock(bundlesToConsider, sbundlesToConsider, pending) + } + if metrics.EnabledBuilder { mergeAlgoTimer.Update(time.Since(start)) } *env = *newEnv - return nil, blockBundles, bundlesToConsider, usedSbundle + return blockBundles, bundlesToConsider, usedSbundle, err } func (w *worker) getSimulatedBundles(env *environment) ([]types.SimulatedBundle, []*types.SimSBundle, error) { @@ -1491,7 +1522,7 @@ func (w *worker) generateWork(params *generateParams) (*types.Block, *big.Int, e orderCloseTime := time.Now() - err, blockBundles, allBundles, usedSbundles := w.fillTransactionsSelectAlgo(nil, work) + blockBundles, allBundles, usedSbundles, err := w.fillTransactionsSelectAlgo(nil, work) if err != nil { return nil, nil, err @@ -1582,7 +1613,7 @@ func (w *worker) commitWork(interrupt *int32, noempty bool, timestamp int64) { } // Fill pending transactions from the txpool - err, _, _, _ = w.fillTransactionsSelectAlgo(interrupt, work) + _, _, _, err = w.fillTransactionsSelectAlgo(interrupt, work) switch { case err == nil: // The entire block is filled, decrease resubmit interval in case