Merge remote-tracking branch 'upstream/master' into hkhosh_ENG191-Geth-Healthcheck

pull/29056/head
crypto-services 9 months ago
commit 2ee7728d7d
  1. 2
      core/state/sync.go
  2. 2
      core/state_transition.go
  3. 33
      core/txpool/blobpool/blobpool.go
  4. 10
      core/txpool/blobpool/blobpool_test.go
  5. 2
      core/txpool/blobpool/evictheap.go
  6. 2
      core/txpool/blobpool/priority_test.go
  7. 18
      core/txpool/legacypool/legacypool.go
  8. 2
      core/txpool/legacypool/list.go
  9. 17
      core/txpool/subpool.go
  10. 5
      core/txpool/txpool.go
  11. 2
      eth/api_backend.go
  12. 2
      eth/api_miner.go
  13. 5
      eth/catalyst/simulated_beacon.go
  14. 2
      eth/downloader/api.go
  15. 3
      eth/handler.go
  16. 2
      eth/handler_test.go
  17. 2
      eth/protocols/eth/peer.go
  18. 4
      eth/protocols/snap/peer.go
  19. 3
      eth/sync.go
  20. 2
      internal/ethapi/api.go
  21. 36
      internal/ethapi/transaction_args.go
  22. 38
      internal/ethapi/transaction_args_test.go
  23. 11
      miner/ordering.go
  24. 86
      miner/worker.go
  25. 2
      params/config.go
  26. 2
      params/version.go

@ -24,7 +24,7 @@ import (
"github.com/ethereum/go-ethereum/trie"
)
// NewStateSync create a new state trie download scheduler.
// NewStateSync creates a new state trie download scheduler.
func NewStateSync(root common.Hash, database ethdb.KeyValueReader, onLeaf func(keys [][]byte, leaf []byte) error, scheme string) *trie.Sync {
// Register the storage slot callback if the external callback is specified.
var onSlot func(keys [][]byte, path []byte, leaf []byte, parent common.Hash, parentPath []byte) error

@ -67,7 +67,7 @@ func (result *ExecutionResult) Revert() []byte {
}
// IntrinsicGas computes the 'intrinsic gas' for a message with the given data.
func IntrinsicGas(data []byte, accessList types.AccessList, isContractCreation bool, isHomestead, isEIP2028 bool, isEIP3860 bool) (uint64, error) {
func IntrinsicGas(data []byte, accessList types.AccessList, isContractCreation bool, isHomestead, isEIP2028, isEIP3860 bool) (uint64, error) {
// Set the starting gas for the raw transaction
var gas uint64
if isContractCreation && isHomestead {

@ -360,7 +360,7 @@ func (p *BlobPool) Init(gasTip uint64, head *types.Header, reserve txpool.Addres
}
}
// Initialize the state with head block, or fallback to empty one in
// case the head state is not available(might occur when node is not
// case the head state is not available (might occur when node is not
// fully synced).
state, err := p.chain.StateAt(head.Root)
if err != nil {
@ -371,7 +371,7 @@ func (p *BlobPool) Init(gasTip uint64, head *types.Header, reserve txpool.Addres
}
p.head, p.state = head, state
// Index all transactions on disk and delete anything inprocessable
// Index all transactions on disk and delete anything unprocessable
var fails []uint64
index := func(id uint64, size uint32, blob []byte) {
if p.parseTransaction(id, size, blob) != nil {
@ -540,7 +540,7 @@ func (p *BlobPool) recheck(addr common.Address, inclusions map[common.Hash]uint6
}
delete(p.index, addr)
delete(p.spent, addr)
if inclusions != nil { // only during reorgs will the heap will be initialized
if inclusions != nil { // only during reorgs will the heap be initialized
heap.Remove(p.evict, p.evict.index[addr])
}
p.reserve(addr, false)
@ -693,7 +693,7 @@ func (p *BlobPool) recheck(addr common.Address, inclusions map[common.Hash]uint6
if len(txs) == 0 {
delete(p.index, addr)
delete(p.spent, addr)
if inclusions != nil { // only during reorgs will the heap will be initialized
if inclusions != nil { // only during reorgs will the heap be initialized
heap.Remove(p.evict, p.evict.index[addr])
}
p.reserve(addr, false)
@ -809,7 +809,7 @@ func (p *BlobPool) Reset(oldHead, newHead *types.Header) {
}
}
// Recheck the account's pooled transactions to drop included and
// invalidated one
// invalidated ones
p.recheck(addr, inclusions)
}
if len(adds) > 0 {
@ -1226,7 +1226,7 @@ func (p *BlobPool) Add(txs []*types.Transaction, local bool, sync bool) []error
// consensus validity and pool restrictions).
func (p *BlobPool) add(tx *types.Transaction) (err error) {
// The blob pool blocks on adding a transaction. This is because blob txs are
// only even pulled form the network, so this method will act as the overload
// only even pulled from the network, so this method will act as the overload
// protection for fetches.
waitStart := time.Now()
p.lock.Lock()
@ -1446,7 +1446,12 @@ func (p *BlobPool) drop() {
//
// The transactions can also be pre-filtered by the dynamic fee components to
// reduce allocations and load on downstream subsystems.
func (p *BlobPool) Pending(minTip *uint256.Int, baseFee *uint256.Int, blobFee *uint256.Int) map[common.Address][]*txpool.LazyTransaction {
func (p *BlobPool) Pending(filter txpool.PendingFilter) map[common.Address][]*txpool.LazyTransaction {
// If only plain transactions are requested, this pool is unsuitable as it
// contains none, don't even bother.
if filter.OnlyPlainTxs {
return nil
}
// Track the amount of time waiting to retrieve the list of pending blob txs
// from the pool and the amount of time actually spent on assembling the data.
// The latter will be pretty much moot, but we've kept it to have symmetric
@ -1466,20 +1471,20 @@ func (p *BlobPool) Pending(minTip *uint256.Int, baseFee *uint256.Int, blobFee *u
lazies := make([]*txpool.LazyTransaction, 0, len(txs))
for _, tx := range txs {
// If transaction filtering was requested, discard badly priced ones
if minTip != nil && baseFee != nil {
if tx.execFeeCap.Lt(baseFee) {
if filter.MinTip != nil && filter.BaseFee != nil {
if tx.execFeeCap.Lt(filter.BaseFee) {
break // basefee too low, cannot be included, discard rest of txs from the account
}
tip := new(uint256.Int).Sub(tx.execFeeCap, baseFee)
tip := new(uint256.Int).Sub(tx.execFeeCap, filter.BaseFee)
if tip.Gt(tx.execTipCap) {
tip = tx.execTipCap
}
if tip.Lt(minTip) {
if tip.Lt(filter.MinTip) {
break // allowed or remaining tip too low, cannot be included, discard rest of txs from the account
}
}
if blobFee != nil {
if tx.blobFeeCap.Lt(blobFee) {
if filter.BlobFee != nil {
if tx.blobFeeCap.Lt(filter.BlobFee) {
break // blobfee too low, cannot be included, discard rest of txs from the account
}
}
@ -1549,7 +1554,7 @@ func (p *BlobPool) updateStorageMetrics() {
}
// updateLimboMetrics retrieves a bunch of stats from the limbo store and pushes
// // them out as metrics.
// them out as metrics.
func (p *BlobPool) updateLimboMetrics() {
stats := p.limbo.store.Infos()

@ -185,7 +185,7 @@ func makeTx(nonce uint64, gasTipCap uint64, gasFeeCap uint64, blobFeeCap uint64,
return types.MustSignNewTx(key, types.LatestSigner(testChainConfig), blobtx)
}
// makeUnsignedTx is a utility method to construct a random blob tranasaction
// makeUnsignedTx is a utility method to construct a random blob transaction
// without signing it.
func makeUnsignedTx(nonce uint64, gasTipCap uint64, gasFeeCap uint64, blobFeeCap uint64) *types.BlobTx {
return &types.BlobTx{
@ -391,7 +391,7 @@ func TestOpenDrops(t *testing.T) {
id, _ := store.Put(blob)
filled[id] = struct{}{}
}
// Insert a sequence of transactions with partially passed nonces to veirfy
// Insert a sequence of transactions with partially passed nonces to verify
// that the included part of the set will get dropped (case 4).
var (
overlapper, _ = crypto.GenerateKey()
@ -1340,7 +1340,11 @@ func benchmarkPoolPending(b *testing.B, datacap uint64) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
p := pool.Pending(uint256.NewInt(1), chain.basefee, chain.blobfee)
p := pool.Pending(txpool.PendingFilter{
MinTip: uint256.NewInt(1),
BaseFee: chain.basefee,
BlobFee: chain.blobfee,
})
if len(p) != int(capacity) {
b.Fatalf("have %d want %d", len(p), capacity)
}

@ -30,7 +30,7 @@ import (
// transaction from each account to determine which account to evict from.
//
// The heap internally tracks a slice of cheapest transactions from each account
// and a mapping from addresses to indices for direct removals/udates.
// and a mapping from addresses to indices for direct removals/updates.
//
// The goal of the heap is to decide which account has the worst bottleneck to
// evict transactions from.

@ -64,7 +64,7 @@ func BenchmarkDynamicFeeJumpCalculation(b *testing.B) {
// Benchmarks how many priority recalculations can be done.
func BenchmarkPriorityCalculation(b *testing.B) {
// The basefee and blob fee is constant for all transactions across a block,
// so we can assume theit absolute jump counts can be pre-computed.
// so we can assume their absolute jump counts can be pre-computed.
basefee := uint256.NewInt(17_200_000_000) // 17.2 Gwei is the 22.03.2023 zero-emission basefee, random number
blobfee := uint256.NewInt(123_456_789_000) // Completely random, no idea what this will be

@ -296,7 +296,7 @@ func (pool *LegacyPool) Init(gasTip uint64, head *types.Header, reserve txpool.A
pool.gasTip.Store(uint256.NewInt(gasTip))
// Initialize the state with head block, or fallback to empty one in
// case the head state is not available(might occur when node is not
// case the head state is not available (might occur when node is not
// fully synced).
statedb, err := pool.chain.StateAt(head.Root)
if err != nil {
@ -522,7 +522,12 @@ func (pool *LegacyPool) ContentFrom(addr common.Address) ([]*types.Transaction,
//
// The transactions can also be pre-filtered by the dynamic fee components to
// reduce allocations and load on downstream subsystems.
func (pool *LegacyPool) Pending(minTip *uint256.Int, baseFee *uint256.Int, blobFee *uint256.Int) map[common.Address][]*txpool.LazyTransaction {
func (pool *LegacyPool) Pending(filter txpool.PendingFilter) map[common.Address][]*txpool.LazyTransaction {
// If only blob transactions are requested, this pool is unsuitable as it
// contains none, don't even bother.
if filter.OnlyBlobTxs {
return nil
}
pool.mu.Lock()
defer pool.mu.Unlock()
@ -531,13 +536,12 @@ func (pool *LegacyPool) Pending(minTip *uint256.Int, baseFee *uint256.Int, blobF
minTipBig *big.Int
baseFeeBig *big.Int
)
if minTip != nil {
minTipBig = minTip.ToBig()
if filter.MinTip != nil {
minTipBig = filter.MinTip.ToBig()
}
if baseFee != nil {
baseFeeBig = baseFee.ToBig()
if filter.BaseFee != nil {
baseFeeBig = filter.BaseFee.ToBig()
}
pending := make(map[common.Address][]*txpool.LazyTransaction, len(pool.pending))
for addr, list := range pool.pending {
txs := list.Flatten()

@ -278,7 +278,7 @@ type list struct {
totalcost *uint256.Int // Total cost of all transactions in the list
}
// newList create a new transaction list for maintaining nonce-indexable fast,
// newList creates a new transaction list for maintaining nonce-indexable fast,
// gapped, sortable transaction lists.
func newList(strict bool) *list {
return &list{

@ -70,6 +70,21 @@ type LazyResolver interface {
// may request (and relinquish) exclusive access to certain addresses.
type AddressReserver func(addr common.Address, reserve bool) error
// PendingFilter is a collection of filter rules to allow retrieving a subset
// of transactions for announcement or mining.
//
// Note, the entries here are not arbitrary useful filters, rather each one has
// a very specific call site in mind and each one can be evaluated very cheaply
// by the pool implementations. Only add new ones that satisfy those constraints.
type PendingFilter struct {
MinTip *uint256.Int // Minimum miner tip required to include a transaction
BaseFee *uint256.Int // Minimum 1559 basefee needed to include a transaction
BlobFee *uint256.Int // Minimum 4844 blobfee needed to include a blob transaction
OnlyPlainTxs bool // Return only plain EVM transactions (peer-join announces, block space filling)
OnlyBlobTxs bool // Return only blob transactions (block blob-space filling)
}
// SubPool represents a specialized transaction pool that lives on its own (e.g.
// blob pool). Since independent of how many specialized pools we have, they do
// need to be updated in lockstep and assemble into one coherent view for block
@ -118,7 +133,7 @@ type SubPool interface {
//
// The transactions can also be pre-filtered by the dynamic fee components to
// reduce allocations and load on downstream subsystems.
Pending(minTip *uint256.Int, baseFee *uint256.Int, blobFee *uint256.Int) map[common.Address][]*LazyTransaction
Pending(filter PendingFilter) map[common.Address][]*LazyTransaction
// SubscribeTransactions subscribes to new transaction events. The subscriber
// can decide whether to receive notifications only for newly seen transactions

@ -28,7 +28,6 @@ import (
"github.com/ethereum/go-ethereum/event"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/metrics"
"github.com/holiman/uint256"
)
// TxStatus is the current status of a transaction as seen by the pool.
@ -357,10 +356,10 @@ func (p *TxPool) Add(txs []*types.Transaction, local bool, sync bool) []error {
//
// The transactions can also be pre-filtered by the dynamic fee components to
// reduce allocations and load on downstream subsystems.
func (p *TxPool) Pending(minTip *uint256.Int, baseFee *uint256.Int, blobFee *uint256.Int) map[common.Address][]*LazyTransaction {
func (p *TxPool) Pending(filter PendingFilter) map[common.Address][]*LazyTransaction {
txs := make(map[common.Address][]*LazyTransaction)
for _, subpool := range p.subpools {
for addr, set := range subpool.Pending(minTip, baseFee, blobFee) {
for addr, set := range subpool.Pending(filter) {
txs[addr] = set
}
}

@ -292,7 +292,7 @@ func (b *EthAPIBackend) SendTx(ctx context.Context, signedTx *types.Transaction)
}
func (b *EthAPIBackend) GetPoolTransactions() (types.Transactions, error) {
pending := b.eth.txPool.Pending(nil, nil, nil)
pending := b.eth.txPool.Pending(txpool.PendingFilter{})
var txs types.Transactions
for _, batch := range pending {
for _, lazy := range batch {

@ -29,7 +29,7 @@ type MinerAPI struct {
e *Ethereum
}
// NewMinerAPI create a new MinerAPI instance.
// NewMinerAPI creates a new MinerAPI instance.
func NewMinerAPI(e *Ethereum) *MinerAPI {
return &MinerAPI{e}
}

@ -25,6 +25,7 @@ import (
"github.com/ethereum/go-ethereum/beacon/engine"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/txpool"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/eth"
"github.com/ethereum/go-ethereum/log"
@ -263,7 +264,7 @@ func (c *SimulatedBeacon) Rollback() {
// Fork sets the head to the provided hash.
func (c *SimulatedBeacon) Fork(parentHash common.Hash) error {
if len(c.eth.TxPool().Pending(nil, nil, nil)) != 0 {
if len(c.eth.TxPool().Pending(txpool.PendingFilter{})) != 0 {
return errors.New("pending block dirty")
}
parent := c.eth.BlockChain().GetBlockByHash(parentHash)
@ -275,7 +276,7 @@ func (c *SimulatedBeacon) Fork(parentHash common.Hash) error {
// AdjustTime creates a new block with an adjusted timestamp.
func (c *SimulatedBeacon) AdjustTime(adjustment time.Duration) error {
if len(c.eth.TxPool().Pending(nil, nil, nil)) != 0 {
if len(c.eth.TxPool().Pending(txpool.PendingFilter{})) != 0 {
return errors.New("could not adjust time on non-empty block")
}
parent := c.eth.BlockChain().CurrentBlock()

@ -38,7 +38,7 @@ type DownloaderAPI struct {
uninstallSyncSubscription chan *uninstallSyncSubscriptionRequest
}
// NewDownloaderAPI create a new DownloaderAPI. The API has an internal event loop that
// NewDownloaderAPI creates a new DownloaderAPI. The API has an internal event loop that
// listens for events from the downloader through the global event mux. In case it receives one of
// these events it broadcasts it to all syncing subscriptions that are installed through the
// installSyncSubscription channel.

@ -42,7 +42,6 @@ import (
"github.com/ethereum/go-ethereum/metrics"
"github.com/ethereum/go-ethereum/p2p"
"github.com/ethereum/go-ethereum/triedb/pathdb"
"github.com/holiman/uint256"
)
const (
@ -74,7 +73,7 @@ type txPool interface {
// Pending should return pending transactions.
// The slice should be modifiable by the caller.
Pending(minTip *uint256.Int, baseFee *uint256.Int, blobFee *uint256.Int) map[common.Address][]*txpool.LazyTransaction
Pending(filter txpool.PendingFilter) map[common.Address][]*txpool.LazyTransaction
// SubscribeTransactions subscribes to new transaction events. The subscriber
// can decide whether to receive notifications only for newly seen transactions

@ -93,7 +93,7 @@ func (p *testTxPool) Add(txs []*types.Transaction, local bool, sync bool) []erro
}
// Pending returns all the transactions known to the pool
func (p *testTxPool) Pending(minTip *uint256.Int, baseFee *uint256.Int, blobFee *uint256.Int) map[common.Address][]*txpool.LazyTransaction {
func (p *testTxPool) Pending(filter txpool.PendingFilter) map[common.Address][]*txpool.LazyTransaction {
p.lock.RLock()
defer p.lock.RUnlock()

@ -92,7 +92,7 @@ type Peer struct {
lock sync.RWMutex // Mutex protecting the internal fields
}
// NewPeer create a wrapper for a network connection and negotiated protocol
// NewPeer creates a wrapper for a network connection and negotiated protocol
// version.
func NewPeer(version uint, p *p2p.Peer, rw p2p.MsgReadWriter, txpool TxPool) *Peer {
peer := &Peer{

@ -33,7 +33,7 @@ type Peer struct {
logger log.Logger // Contextual logger with the peer id injected
}
// NewPeer create a wrapper for a network connection and negotiated protocol
// NewPeer creates a wrapper for a network connection and negotiated protocol
// version.
func NewPeer(version uint, p *p2p.Peer, rw p2p.MsgReadWriter) *Peer {
id := p.ID().String()
@ -46,7 +46,7 @@ func NewPeer(version uint, p *p2p.Peer, rw p2p.MsgReadWriter) *Peer {
}
}
// NewFakePeer create a fake snap peer without a backing p2p peer, for testing purposes.
// NewFakePeer creates a fake snap peer without a backing p2p peer, for testing purposes.
func NewFakePeer(version uint, id string, rw p2p.MsgReadWriter) *Peer {
return &Peer{
id: id,

@ -23,6 +23,7 @@ import (
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/rawdb"
"github.com/ethereum/go-ethereum/core/txpool"
"github.com/ethereum/go-ethereum/eth/downloader"
"github.com/ethereum/go-ethereum/eth/protocols/eth"
"github.com/ethereum/go-ethereum/log"
@ -36,7 +37,7 @@ const (
// syncTransactions starts sending all currently pending transactions to the given peer.
func (h *handler) syncTransactions(p *eth.Peer) {
var hashes []common.Hash
for _, batch := range h.txpool.Pending(nil, nil, nil) {
for _, batch := range h.txpool.Pending(txpool.PendingFilter{OnlyPlainTxs: true}) {
for _, tx := range batch {
hashes = append(hashes, tx.Hash)
}

@ -288,7 +288,7 @@ type PersonalAccountAPI struct {
b Backend
}
// NewPersonalAccountAPI create a new PersonalAccountAPI.
// NewPersonalAccountAPI creates a new PersonalAccountAPI.
func NewPersonalAccountAPI(b Backend, nonceLock *AddrLocker) *PersonalAccountAPI {
return &PersonalAccountAPI{
am: b.AccountManager(),

@ -177,6 +177,14 @@ func (args *TransactionArgs) setDefaults(ctx context.Context, b Backend) error {
// setFeeDefaults fills in default fee values for unspecified tx fields.
func (args *TransactionArgs) setFeeDefaults(ctx context.Context, b Backend) error {
head := b.CurrentHeader()
// Sanity check the EIP-4844 fee parameters.
if args.BlobFeeCap != nil && args.BlobFeeCap.ToInt().Sign() == 0 {
return errors.New("maxFeePerBlobGas, if specified, must be non-zero")
}
if err := args.setCancunFeeDefaults(ctx, head, b); err != nil {
return err
}
// If both gasPrice and at least one of the EIP-1559 fee parameters are specified, error.
if args.GasPrice != nil && (args.MaxFeePerGas != nil || args.MaxPriorityFeePerGas != nil) {
return errors.New("both gasPrice and (maxFeePerGas or maxPriorityFeePerGas) specified")
@ -186,7 +194,6 @@ func (args *TransactionArgs) setFeeDefaults(ctx context.Context, b Backend) erro
// other tx values. See https://github.com/ethereum/go-ethereum/pull/23274
// for more information.
eip1559ParamsSet := args.MaxFeePerGas != nil && args.MaxPriorityFeePerGas != nil
// Sanity check the EIP-1559 fee parameters if present.
if args.GasPrice == nil && eip1559ParamsSet {
if args.MaxFeePerGas.ToInt().Sign() == 0 {
@ -198,13 +205,7 @@ func (args *TransactionArgs) setFeeDefaults(ctx context.Context, b Backend) erro
return nil // No need to set anything, user already set MaxFeePerGas and MaxPriorityFeePerGas
}
// Sanity check the EIP-4844 fee parameters.
if args.BlobFeeCap != nil && args.BlobFeeCap.ToInt().Sign() == 0 {
return errors.New("maxFeePerBlobGas must be non-zero")
}
// Sanity check the non-EIP-1559 fee parameters.
head := b.CurrentHeader()
isLondon := b.ChainConfig().IsLondon(head.Number)
if args.GasPrice != nil && !eip1559ParamsSet {
// Zero gas-price is not allowed after London fork
@ -215,21 +216,14 @@ func (args *TransactionArgs) setFeeDefaults(ctx context.Context, b Backend) erro
}
// Now attempt to fill in default value depending on whether London is active or not.
if b.ChainConfig().IsCancun(head.Number, head.Time) {
if err := args.setCancunFeeDefaults(ctx, head, b); err != nil {
return err
}
} else if isLondon {
if args.BlobFeeCap != nil {
return errors.New("maxFeePerBlobGas is not valid before Cancun is active")
}
if isLondon {
// London is active, set maxPriorityFeePerGas and maxFeePerGas.
if err := args.setLondonFeeDefaults(ctx, head, b); err != nil {
return err
}
} else {
if args.MaxFeePerGas != nil || args.MaxPriorityFeePerGas != nil || args.BlobFeeCap != nil {
return errors.New("maxFeePerGas and maxPriorityFeePerGas and maxFeePerBlobGas are not valid before London is active")
if args.MaxFeePerGas != nil || args.MaxPriorityFeePerGas != nil {
return errors.New("maxFeePerGas and maxPriorityFeePerGas are not valid before London is active")
}
// London not active, set gas price.
price, err := b.SuggestGasTipCap(ctx)
@ -245,15 +239,19 @@ func (args *TransactionArgs) setFeeDefaults(ctx context.Context, b Backend) erro
func (args *TransactionArgs) setCancunFeeDefaults(ctx context.Context, head *types.Header, b Backend) error {
// Set maxFeePerBlobGas if it is missing.
if args.BlobHashes != nil && args.BlobFeeCap == nil {
var excessBlobGas uint64
if head.ExcessBlobGas != nil {
excessBlobGas = *head.ExcessBlobGas
}
// ExcessBlobGas must be set for a Cancun block.
blobBaseFee := eip4844.CalcBlobFee(*head.ExcessBlobGas)
blobBaseFee := eip4844.CalcBlobFee(excessBlobGas)
// Set the max fee to be 2 times larger than the previous block's blob base fee.
// The additional slack allows the tx to not become invalidated if the base
// fee is rising.
val := new(big.Int).Mul(blobBaseFee, big.NewInt(2))
args.BlobFeeCap = (*hexutil.Big)(val)
}
return args.setLondonFeeDefaults(ctx, head, b)
return nil
}
// setLondonFeeDefaults fills in reasonable default fee values for unspecified fields.

@ -153,14 +153,14 @@ func TestSetFeeDefaults(t *testing.T) {
"legacy",
&TransactionArgs{MaxFeePerGas: maxFee},
nil,
errors.New("maxFeePerGas and maxPriorityFeePerGas and maxFeePerBlobGas are not valid before London is active"),
errors.New("maxFeePerGas and maxPriorityFeePerGas are not valid before London is active"),
},
{
"dynamic fee tx pre-London, priorityFee set",
"legacy",
&TransactionArgs{MaxPriorityFeePerGas: fortytwo},
nil,
errors.New("maxFeePerGas and maxPriorityFeePerGas and maxFeePerBlobGas are not valid before London is active"),
errors.New("maxFeePerGas and maxPriorityFeePerGas are not valid before London is active"),
},
{
"dynamic fee tx, maxFee < priorityFee",
@ -207,20 +207,6 @@ func TestSetFeeDefaults(t *testing.T) {
errors.New("both gasPrice and (maxFeePerGas or maxPriorityFeePerGas) specified"),
},
// EIP-4844
{
"set maxFeePerBlobGas pre cancun",
"london",
&TransactionArgs{BlobFeeCap: fortytwo},
nil,
errors.New("maxFeePerBlobGas is not valid before Cancun is active"),
},
{
"set maxFeePerBlobGas pre london",
"legacy",
&TransactionArgs{BlobFeeCap: fortytwo},
nil,
errors.New("maxFeePerGas and maxPriorityFeePerGas and maxFeePerBlobGas are not valid before London is active"),
},
{
"set gas price and maxFee for blob transaction",
"cancun",
@ -235,6 +221,13 @@ func TestSetFeeDefaults(t *testing.T) {
&TransactionArgs{BlobHashes: []common.Hash{}, BlobFeeCap: (*hexutil.Big)(big.NewInt(4)), MaxFeePerGas: maxFee, MaxPriorityFeePerGas: fortytwo},
nil,
},
{
"fill maxFeePerBlobGas when dynamic fees are set",
"cancun",
&TransactionArgs{BlobHashes: []common.Hash{}, MaxFeePerGas: maxFee, MaxPriorityFeePerGas: fortytwo},
&TransactionArgs{BlobHashes: []common.Hash{}, BlobFeeCap: (*hexutil.Big)(big.NewInt(4)), MaxFeePerGas: maxFee, MaxPriorityFeePerGas: fortytwo},
nil,
},
}
ctx := context.Background()
@ -244,11 +237,16 @@ func TestSetFeeDefaults(t *testing.T) {
}
got := test.in
err := got.setFeeDefaults(ctx, b)
if err != nil && err.Error() == test.err.Error() {
// Test threw expected error.
if err != nil {
if test.err == nil {
t.Fatalf("test %d (%s): unexpected error: %s", i, test.name, err)
} else if err.Error() != test.err.Error() {
t.Fatalf("test %d (%s): unexpected error: (got: %s, want: %s)", i, test.name, err, test.err)
}
// Matching error.
continue
} else if err != nil {
t.Fatalf("test %d (%s): unexpected error: %s", i, test.name, err)
} else if test.err != nil {
t.Fatalf("test %d (%s): expected error: %s", i, test.name, test.err)
}
if !reflect.DeepEqual(got, test.want) {
t.Fatalf("test %d (%s): did not fill defaults as expected: (got: %v, want: %v)", i, test.name, got, test.want)

@ -153,3 +153,14 @@ func (t *transactionsByPriceAndNonce) Shift() {
func (t *transactionsByPriceAndNonce) Pop() {
heap.Pop(&t.heads)
}
// Empty returns if the price heap is empty. It can be used to check it simpler
// than calling peek and checking for nil return.
func (t *transactionsByPriceAndNonce) Empty() bool {
return len(t.heads) == 0
}
// Clear removes the entire content of the heap.
func (t *transactionsByPriceAndNonce) Clear() {
t.heads, t.txs = nil, nil
}

@ -562,9 +562,11 @@ func (w *worker) mainLoop() {
BlobGas: tx.BlobGas(),
})
}
txset := newTransactionsByPriceAndNonce(w.current.signer, txs, w.current.header.BaseFee)
plainTxs := newTransactionsByPriceAndNonce(w.current.signer, txs, w.current.header.BaseFee) // Mixed bag of everrything, yolo
blobTxs := newTransactionsByPriceAndNonce(w.current.signer, nil, w.current.header.BaseFee) // Empty bag, don't bother optimising
tcount := w.current.tcount
w.commitTransactions(w.current, txset, nil, new(uint256.Int))
w.commitTransactions(w.current, plainTxs, blobTxs, nil)
// Only update the snapshot if any new transactions were added
// to the pending block
@ -802,7 +804,7 @@ func (w *worker) applyTransaction(env *environment, tx *types.Transaction) (*typ
return receipt, err
}
func (w *worker) commitTransactions(env *environment, txs *transactionsByPriceAndNonce, interrupt *atomic.Int32, minTip *uint256.Int) error {
func (w *worker) commitTransactions(env *environment, plainTxs, blobTxs *transactionsByPriceAndNonce, interrupt *atomic.Int32) error {
gasLimit := env.header.GasLimit
if env.gasPool == nil {
env.gasPool = new(core.GasPool).AddGas(gasLimit)
@ -821,8 +823,33 @@ func (w *worker) commitTransactions(env *environment, txs *transactionsByPriceAn
log.Trace("Not enough gas for further transactions", "have", env.gasPool, "want", params.TxGas)
break
}
// If we don't have enough blob space for any further blob transactions,
// skip that list altogether
if !blobTxs.Empty() && env.blobs*params.BlobTxBlobGasPerBlob >= params.MaxBlobGasPerBlock {
log.Trace("Not enough blob space for further blob transactions")
blobTxs.Clear()
// Fall though to pick up any plain txs
}
// Retrieve the next transaction and abort if all done.
ltx, tip := txs.Peek()
var (
ltx *txpool.LazyTransaction
txs *transactionsByPriceAndNonce
)
pltx, ptip := plainTxs.Peek()
bltx, btip := blobTxs.Peek()
switch {
case pltx == nil:
txs, ltx = blobTxs, bltx
case bltx == nil:
txs, ltx = plainTxs, pltx
default:
if ptip.Lt(btip) {
txs, ltx = blobTxs, bltx
} else {
txs, ltx = plainTxs, pltx
}
}
if ltx == nil {
break
}
@ -837,11 +864,6 @@ func (w *worker) commitTransactions(env *environment, txs *transactionsByPriceAn
txs.Pop()
continue
}
// If we don't receive enough tip for the next transaction, skip the account
if tip.Cmp(minTip) < 0 {
log.Trace("Not enough tip for transaction", "hash", ltx.Hash, "tip", tip, "needed", minTip)
break // If the next-best is too low, surely no better will be available
}
// Transaction seems to fit, pull it up from the pool
tx := ltx.Resolve()
if tx == nil {
@ -1005,35 +1027,49 @@ func (w *worker) fillTransactions(interrupt *atomic.Int32, env *environment) err
w.mu.RUnlock()
// Retrieve the pending transactions pre-filtered by the 1559/4844 dynamic fees
var baseFee *uint256.Int
filter := txpool.PendingFilter{
MinTip: tip,
}
if env.header.BaseFee != nil {
baseFee = uint256.MustFromBig(env.header.BaseFee)
filter.BaseFee = uint256.MustFromBig(env.header.BaseFee)
}
var blobFee *uint256.Int
if env.header.ExcessBlobGas != nil {
blobFee = uint256.MustFromBig(eip4844.CalcBlobFee(*env.header.ExcessBlobGas))
filter.BlobFee = uint256.MustFromBig(eip4844.CalcBlobFee(*env.header.ExcessBlobGas))
}
pending := w.eth.TxPool().Pending(tip, baseFee, blobFee)
filter.OnlyPlainTxs, filter.OnlyBlobTxs = true, false
pendingPlainTxs := w.eth.TxPool().Pending(filter)
filter.OnlyPlainTxs, filter.OnlyBlobTxs = false, true
pendingBlobTxs := w.eth.TxPool().Pending(filter)
// Split the pending transactions into locals and remotes.
localTxs, remoteTxs := make(map[common.Address][]*txpool.LazyTransaction), pending
localPlainTxs, remotePlainTxs := make(map[common.Address][]*txpool.LazyTransaction), pendingPlainTxs
localBlobTxs, remoteBlobTxs := make(map[common.Address][]*txpool.LazyTransaction), pendingBlobTxs
for _, account := range w.eth.TxPool().Locals() {
if txs := remoteTxs[account]; len(txs) > 0 {
delete(remoteTxs, account)
localTxs[account] = txs
if txs := remotePlainTxs[account]; len(txs) > 0 {
delete(remotePlainTxs, account)
localPlainTxs[account] = txs
}
if txs := remoteBlobTxs[account]; len(txs) > 0 {
delete(remoteBlobTxs, account)
localBlobTxs[account] = txs
}
}
// Fill the block with all available pending transactions.
if len(localTxs) > 0 {
txs := newTransactionsByPriceAndNonce(env.signer, localTxs, env.header.BaseFee)
if err := w.commitTransactions(env, txs, interrupt, new(uint256.Int)); err != nil {
if len(localPlainTxs) > 0 || len(localBlobTxs) > 0 {
plainTxs := newTransactionsByPriceAndNonce(env.signer, localPlainTxs, env.header.BaseFee)
blobTxs := newTransactionsByPriceAndNonce(env.signer, localBlobTxs, env.header.BaseFee)
if err := w.commitTransactions(env, plainTxs, blobTxs, interrupt); err != nil {
return err
}
}
if len(remoteTxs) > 0 {
txs := newTransactionsByPriceAndNonce(env.signer, remoteTxs, env.header.BaseFee)
if err := w.commitTransactions(env, txs, interrupt, tip); err != nil {
if len(remotePlainTxs) > 0 || len(remoteBlobTxs) > 0 {
plainTxs := newTransactionsByPriceAndNonce(env.signer, remotePlainTxs, env.header.BaseFee)
blobTxs := newTransactionsByPriceAndNonce(env.signer, remoteBlobTxs, env.header.BaseFee)
if err := w.commitTransactions(env, plainTxs, blobTxs, interrupt); err != nil {
return err
}
}

@ -467,7 +467,7 @@ func (c *ChainConfig) Description() string {
banner += fmt.Sprintf(" - Shanghai: @%-10v (https://github.com/ethereum/execution-specs/blob/master/network-upgrades/mainnet-upgrades/shanghai.md)\n", *c.ShanghaiTime)
}
if c.CancunTime != nil {
banner += fmt.Sprintf(" - Cancun: @%-10v\n", *c.CancunTime)
banner += fmt.Sprintf(" - Cancun: @%-10v (https://github.com/ethereum/execution-specs/blob/master/network-upgrades/mainnet-upgrades/cancun.md)\n", *c.CancunTime)
}
if c.PragueTime != nil {
banner += fmt.Sprintf(" - Prague: @%-10v\n", *c.PragueTime)

@ -23,7 +23,7 @@ import (
const (
VersionMajor = 1 // Major version component of the current release
VersionMinor = 13 // Minor version component of the current release
VersionPatch = 13 // Patch version component of the current release
VersionPatch = 14 // Patch version component of the current release
VersionMeta = "unstable" // Version metadata to append to the version string
)

Loading…
Cancel
Save