core/txpool: add support for setcode tx

lightclient 3 months ago
parent d3cc618951
commit b2ece7bb98
No known key found for this signature in database
GPG Key ID: 657913021EF45A6A
  1. 5
      core/txpool/errors.go
  2. 87
      core/txpool/legacypool/legacypool.go
  3. 202
      core/txpool/legacypool/legacypool_test.go
  4. 32
      core/txpool/validation.go
  5. 15
      core/types/transaction.go

@ -60,4 +60,9 @@ var (
// input transaction of non-blob type when a blob transaction from this sender
// remains pending (and vice-versa).
ErrAlreadyReserved = errors.New("address already reserved")
// ErrAuthorityReserved is returned if a transaction has an authorization
// signed by an address which already has in-flight transactions known to the
// pool.
ErrAuthorityReserved = errors.New("authority already reserved")
)

@ -217,12 +217,13 @@ type LegacyPool struct {
locals *accountSet // Set of local transaction to exempt from eviction rules
journal *journal // Journal of local transaction to back up to disk
reserve txpool.AddressReserver // Address reserver to ensure exclusivity across subpools
pending map[common.Address]*list // All currently processable transactions
queue map[common.Address]*list // Queued but non-processable transactions
beats map[common.Address]time.Time // Last heartbeat from each known account
all *lookup // All transactions to allow lookups
priced *pricedList // All transactions sorted by price
reserve txpool.AddressReserver // Address reserver to ensure exclusivity across subpools
pending map[common.Address]*list // All currently processable transactions
queue map[common.Address]*list // Queued but non-processable transactions
beats map[common.Address]time.Time // Last heartbeat from each known account
all *lookup // All transactions to allow lookups
priced *pricedList // All transactions sorted by price
auths map[common.Address]*types.Transaction // All accounts with a pooled authorization
reqResetCh chan *txpoolResetRequest
reqPromoteCh chan *accountSet
@ -254,6 +255,7 @@ func New(config Config, chain BlockChain) *LegacyPool {
pending: make(map[common.Address]*list),
queue: make(map[common.Address]*list),
beats: make(map[common.Address]time.Time),
auths: make(map[common.Address]*types.Transaction),
all: newLookup(),
reqResetCh: make(chan *txpoolResetRequest),
reqPromoteCh: make(chan *accountSet),
@ -611,7 +613,8 @@ func (pool *LegacyPool) validateTxBasics(tx *types.Transaction, local bool) erro
Accept: 0 |
1<<types.LegacyTxType |
1<<types.AccessListTxType |
1<<types.DynamicFeeTxType,
1<<types.DynamicFeeTxType |
1<<types.SetCodeTxType,
MaxSize: txMaxSize,
MinTip: pool.gasTip.Load().ToBig(),
}
@ -639,6 +642,14 @@ func (pool *LegacyPool) validateTx(tx *types.Transaction) error {
if list := pool.queue[addr]; list != nil {
have += list.Len()
}
// Limit the number of setcode tranasactions per account
if pool.currentState.GetCode(addr) != nil {
if have >= 1 {
return have, 0
} else {
return have, 1 - have
}
}
return have, math.MaxInt
},
ExistingExpenditure: func(addr common.Address) *big.Int {
@ -655,6 +666,28 @@ func (pool *LegacyPool) validateTx(tx *types.Transaction) error {
}
return nil
},
KnownConflicts: func(sender common.Address, addrs []common.Address) []common.Address {
var conflicts []common.Address
if _, ok := pool.auths[sender]; ok {
conflicts = append(conflicts, sender)
}
for _, addr := range addrs {
var known bool
if list := pool.pending[addr]; list != nil {
known = true
}
if list := pool.queue[addr]; list != nil {
known = true
}
if _, ok := pool.auths[addr]; ok {
known = true
}
if known {
conflicts = append(conflicts, addr)
}
}
return conflicts
},
}
if err := txpool.ValidateTransactionWithState(tx, pool.signer, opts); err != nil {
return err
@ -692,6 +725,7 @@ func (pool *LegacyPool) add(tx *types.Transaction, local bool) (replaced bool, e
// If the address is not yet known, request exclusivity to track the account
// only by this subpool until all transactions are evicted
// TODO: need to track every authority from setcode txs
var (
_, hasPending = pool.pending[from]
_, hasQueued = pool.queue[from]
@ -786,12 +820,16 @@ func (pool *LegacyPool) add(tx *types.Transaction, local bool) (replaced bool, e
if old != nil {
pool.all.Remove(old.Hash())
pool.priced.Removed(1)
pool.removeAuthorities(old)
pendingReplaceMeter.Mark(1)
}
pool.all.Add(tx, isLocal)
pool.priced.Put(tx, isLocal)
pool.journalTx(from, tx)
pool.queueTxEvent(tx)
for _, addr := range tx.Authorities() {
pool.auths[addr] = tx
}
log.Trace("Pooled new executable transaction", "hash", hash, "from", from, "to", tx.To())
// Successful promotion, bump the heartbeat
@ -813,6 +851,9 @@ func (pool *LegacyPool) add(tx *types.Transaction, local bool) (replaced bool, e
localGauge.Inc(1)
}
pool.journalTx(from, tx)
for _, addr := range tx.Authorities() {
pool.auths[addr] = tx
}
log.Trace("Pooled new future transaction", "hash", hash, "from", from, "to", tx.To())
return replaced, nil
@ -860,6 +901,7 @@ func (pool *LegacyPool) enqueueTx(hash common.Hash, tx *types.Transaction, local
// Discard any previous transaction and mark this
if old != nil {
pool.all.Remove(old.Hash())
pool.removeAuthorities(old)
pool.priced.Removed(1)
queuedReplaceMeter.Mark(1)
} else {
@ -879,6 +921,10 @@ func (pool *LegacyPool) enqueueTx(hash common.Hash, tx *types.Transaction, local
if _, exist := pool.beats[from]; !exist {
pool.beats[from] = time.Now()
}
for _, auth := range tx.SetCodeAuthorizations() {
addr, _ := auth.Authority()
pool.auths[addr] = tx
}
return old != nil, nil
}
@ -909,6 +955,7 @@ func (pool *LegacyPool) promoteTx(addr common.Address, hash common.Hash, tx *typ
if !inserted {
// An older transaction was better, discard this
pool.all.Remove(hash)
pool.removeAuthorities(tx)
pool.priced.Removed(1)
pendingDiscardMeter.Mark(1)
return false
@ -916,6 +963,7 @@ func (pool *LegacyPool) promoteTx(addr common.Address, hash common.Hash, tx *typ
// Otherwise discard any previous transaction and mark this
if old != nil {
pool.all.Remove(old.Hash())
pool.removeAuthorities(old)
pool.priced.Removed(1)
pendingReplaceMeter.Mark(1)
} else {
@ -1129,6 +1177,9 @@ func (pool *LegacyPool) removeTx(hash common.Hash, outofbound bool, unreserve bo
if pool.locals.contains(addr) {
localGauge.Dec(1)
}
// Remove any authorities the pool was tracking.
pool.removeAuthorities(tx)
// Remove the transaction from the pending lists and reset the account nonce
if pending := pool.pending[addr]; pending != nil {
if removed, invalids := pending.Remove(tx); removed {
@ -1162,6 +1213,12 @@ func (pool *LegacyPool) removeTx(hash common.Hash, outofbound bool, unreserve bo
return 0
}
func (pool *LegacyPool) removeAuthorities(tx *types.Transaction) {
for _, addr := range tx.Authorities() {
delete(pool.auths, addr)
}
}
// requestReset requests a pool reset to the new head block.
// The returned channel is closed when the reset has occurred.
func (pool *LegacyPool) requestReset(oldHead *types.Header, newHead *types.Header) chan struct{} {
@ -1461,15 +1518,15 @@ func (pool *LegacyPool) promoteExecutables(accounts []common.Address) []*types.T
// Drop all transactions that are deemed too old (low nonce)
forwards := list.Forward(pool.currentState.GetNonce(addr))
for _, tx := range forwards {
hash := tx.Hash()
pool.all.Remove(hash)
pool.all.Remove(tx.Hash())
pool.removeAuthorities(tx)
}
log.Trace("Removed old queued transactions", "count", len(forwards))
// Drop all transactions that are too costly (low balance or out of gas)
drops, _ := list.Filter(pool.currentState.GetBalance(addr), gasLimit)
for _, tx := range drops {
hash := tx.Hash()
pool.all.Remove(hash)
pool.all.Remove(tx.Hash())
pool.removeAuthorities(tx)
}
log.Trace("Removed unpayable queued transactions", "count", len(drops))
queuedNofundsMeter.Mark(int64(len(drops)))
@ -1492,6 +1549,7 @@ func (pool *LegacyPool) promoteExecutables(accounts []common.Address) []*types.T
for _, tx := range caps {
hash := tx.Hash()
pool.all.Remove(hash)
pool.removeAuthorities(tx)
log.Trace("Removed cap-exceeding queued transaction", "hash", hash)
}
queuedRateLimitMeter.Mark(int64(len(caps)))
@ -1557,6 +1615,7 @@ func (pool *LegacyPool) truncatePending() {
// Drop the transaction from the global pools too
hash := tx.Hash()
pool.all.Remove(hash)
pool.removeAuthorities(tx)
// Update the account nonce to the dropped transaction
pool.pendingNonces.setIfLower(offenders[i], tx.Nonce())
@ -1584,6 +1643,7 @@ func (pool *LegacyPool) truncatePending() {
// Drop the transaction from the global pools too
hash := tx.Hash()
pool.all.Remove(hash)
pool.removeAuthorities(tx)
// Update the account nonce to the dropped transaction
pool.pendingNonces.setIfLower(addr, tx.Nonce())
@ -1664,14 +1724,16 @@ func (pool *LegacyPool) demoteUnexecutables() {
for _, tx := range olds {
hash := tx.Hash()
pool.all.Remove(hash)
pool.removeAuthorities(tx)
log.Trace("Removed old pending transaction", "hash", hash)
}
// Drop all transactions that are too costly (low balance or out of gas), and queue any invalids back for later
drops, invalids := list.Filter(pool.currentState.GetBalance(addr), gasLimit)
for _, tx := range drops {
hash := tx.Hash()
log.Trace("Removed unpayable pending transaction", "hash", hash)
pool.all.Remove(hash)
pool.removeAuthorities(tx)
log.Trace("Removed unpayable pending transaction", "hash", hash)
}
pendingNofundsMeter.Mark(int64(len(drops)))
@ -1995,6 +2057,7 @@ func (pool *LegacyPool) Clear() {
pool.pending = make(map[common.Address]*list)
pool.queue = make(map[common.Address]*list)
pool.pendingNonces = newNoncer(pool.currentState)
pool.auths = make(map[common.Address]*types.Transaction)
if !pool.config.NoLocals && pool.config.Journal != "" {
pool.journal = newTxJournal(pool.config.Journal)

@ -35,6 +35,7 @@ import (
"github.com/ethereum/go-ethereum/core/tracing"
"github.com/ethereum/go-ethereum/core/txpool"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/core/vm"
"github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/event"
"github.com/ethereum/go-ethereum/params"
@ -80,8 +81,9 @@ func (bc *testBlockChain) Config() *params.ChainConfig {
func (bc *testBlockChain) CurrentBlock() *types.Header {
return &types.Header{
Number: new(big.Int),
GasLimit: bc.gasLimit.Load(),
Number: new(big.Int),
Difficulty: common.Big0,
GasLimit: bc.gasLimit.Load(),
}
}
@ -129,6 +131,39 @@ func dynamicFeeTx(nonce uint64, gaslimit uint64, gasFee *big.Int, tip *big.Int,
return tx
}
type unsignedAuth struct {
nonce uint64
key *ecdsa.PrivateKey
}
func setCodeTx(nonce uint64, key *ecdsa.PrivateKey, unsigned []unsignedAuth) *types.Transaction {
return pricedSetCodeTx(nonce, 250000, uint256.NewInt(1000), uint256.NewInt(1), key, unsigned)
}
func pricedSetCodeTx(nonce uint64, gaslimit uint64, gasFee, tip *uint256.Int, key *ecdsa.PrivateKey, unsigned []unsignedAuth) *types.Transaction {
var authList []types.SetCodeAuthorization
for _, u := range unsigned {
auth, _ := types.SignSetCode(u.key, types.SetCodeAuthorization{
ChainID: *uint256.MustFromBig(params.TestChainConfig.ChainID),
Address: common.Address{0x42},
Nonce: u.nonce,
})
authList = append(authList, auth)
}
return types.MustSignNewTx(key, types.LatestSignerForChainID(params.TestChainConfig.ChainID), &types.SetCodeTx{
ChainID: uint256.MustFromBig(params.TestChainConfig.ChainID),
Nonce: nonce,
GasTipCap: tip,
GasFeeCap: gasFee,
Gas: gaslimit,
To: common.Address{},
Value: uint256.NewInt(100),
Data: nil,
AccessList: nil,
AuthList: authList,
})
}
func makeAddressReserver() txpool.AddressReserver {
var (
reserved = make(map[common.Address]struct{})
@ -2525,6 +2560,169 @@ func TestSlotCount(t *testing.T) {
}
}
// TestSetCodeTransactions tests a few scenarios regarding the EIP-7702
// SetCodeTx.
func TestSetCodeTransactions(t *testing.T) {
t.Parallel()
// Create the pool to test the status retrievals with
statedb, _ := state.New(types.EmptyRootHash, state.NewDatabaseForTesting())
blockchain := newTestBlockChain(params.MergedTestChainConfig, 1000000, statedb, new(event.Feed))
pool := New(testTxPoolConfig, blockchain)
pool.Init(testTxPoolConfig.PriceLimit, blockchain.CurrentBlock(), makeAddressReserver())
defer pool.Close()
// Create the test accounts
keys := make([]*ecdsa.PrivateKey, 4)
addrs := make([]common.Address, len(keys))
for i := 0; i < len(keys); i++ {
keys[i], _ = crypto.GenerateKey()
addrs[i] = crypto.PubkeyToAddress(keys[i].PublicKey)
testAddBalance(pool, crypto.PubkeyToAddress(keys[i].PublicKey), big.NewInt(params.Ether))
}
// A few situations to test:
// 1. Accounts with delegation set can only have one in-flight transaction.
// 2. Setcode tx should be rejected if any authority has a known pooled tx.
// 3. New txs from senders with pooled delegations should not be accepted.
// 4. Ensure setcode tx can replace itself provided the fee bump is enough.
// 5. Make sure that if a setcode tx is replaced, the auths associated with
// the tx are removed.
// 5.1. This should also work when a self-sponsored setcode tx attempts
// to replace itself.
// make sure auth list recreated correctly after full reorg?
// (in different test?) verify that a setcode tx cannot invalidate a blob tx.
for _, tt := range []struct {
name string
pending int
queued int
run func() error
}{
{
name: "only-one-in-flight",
pending: 1,
run: func() error {
// Check that only one in-flight transaction is allowed for accounts
// with delegation set. Also verify the accepted transaction can be
// replaced by fee.
aa := common.Address{0xaa, 0xaa}
statedb.SetCode(addrs[0], append(types.DelegationPrefix, aa.Bytes()...))
statedb.SetCode(aa, []byte{byte(vm.ADDRESS), byte(vm.PUSH0), byte(vm.SSTORE)})
// Send transactions. First is accepted, second is rejected.
if err := pool.addRemoteSync(pricedTransaction(0, 100000, big.NewInt(1), keys[0])); err != nil {
return fmt.Errorf("failed to add remote transaction: %v", err)
}
if err := pool.addRemoteSync(pricedTransaction(1, 100000, big.NewInt(1), keys[0])); !errors.Is(err, txpool.ErrAccountLimitExceeded) {
return fmt.Errorf("error mismatch: want %v, have %v", txpool.ErrAccountLimitExceeded, err)
}
// Also check gapped transaction.
if err := pool.addRemoteSync(pricedTransaction(2, 100000, big.NewInt(1), keys[0])); !errors.Is(err, txpool.ErrAccountLimitExceeded) {
return fmt.Errorf("error mismatch: want %v, have %v", txpool.ErrAccountLimitExceeded, err)
}
// Replace by fee.
if err := pool.addRemoteSync(pricedTransaction(0, 100000, big.NewInt(10), keys[0])); err != nil {
return fmt.Errorf("failed to replace with remote transaction: %v", err)
}
return nil
},
},
{
name: "reject-setcode-tx-with-pending-authority-tx",
pending: 1,
run: func() error {
// Send two transactions where the first has no conflicting delegations and
// the second should be rejected due to a conflict with the tx sent in 1).
if err := pool.addRemoteSync(setCodeTx(0, keys[1], []unsignedAuth{{1, keys[2]}})); err != nil {
return fmt.Errorf("failed to add with remote setcode transaction: %v", err)
}
if err := pool.addRemoteSync(setCodeTx(1, keys[1], []unsignedAuth{{1, keys[2]}})); !errors.Is(err, txpool.ErrAuthorityReserved) {
return fmt.Errorf("error mismatch: want %v, have %v", txpool.ErrAuthorityReserved, err)
}
return nil
},
},
{
name: "reject-tx-from-pooled-delegation",
pending: 1,
run: func() error {
// Verify key[2] cannot originate another transaction when it has a pooled delegation.
if err := pool.addRemoteSync(setCodeTx(0, keys[0], []unsignedAuth{{0, keys[2]}})); err != nil {
return fmt.Errorf("failed to add with remote setcode transaction: %v", err)
}
if err := pool.addRemoteSync(pricedTransaction(0, 100000, big.NewInt(1), keys[2])); !errors.Is(err, txpool.ErrAuthorityReserved) {
return fmt.Errorf("error mismatch: want %v, have %v", txpool.ErrAuthorityReserved, err)
}
// Also check gapped transaction is rejected.
if err := pool.addRemoteSync(pricedTransaction(1, 100000, big.NewInt(1), keys[2])); !errors.Is(err, txpool.ErrAuthorityReserved) {
return fmt.Errorf("error mismatch: want %v, have %v", txpool.ErrAuthorityReserved, err)
}
return nil
},
},
{
name: "replace-by-fee-setcode-tx",
pending: 1,
run: func() error {
// 4. Fee bump the setcode tx send.
if err := pool.addRemoteSync(setCodeTx(0, keys[1], []unsignedAuth{{1, keys[2]}})); err != nil {
return fmt.Errorf("failed to add with remote setcode transaction: %v", err)
}
if err := pool.addRemoteSync(pricedSetCodeTx(0, 250000, uint256.NewInt(2000), uint256.NewInt(2), keys[1], []unsignedAuth{{0, keys[2]}})); err != nil {
t.Fatalf("failed to add with remote setcode transaction: %v", err)
}
return nil
},
},
{
name: "allow-tx-from-replaced-authority",
pending: 2,
run: func() error {
// Fee bump with a different auth list. Make sure that unlocks the authorities.
if err := pool.addRemoteSync(pricedSetCodeTx(0, 250000, uint256.NewInt(10), uint256.NewInt(3), keys[0], []unsignedAuth{{0, keys[1]}})); err != nil {
t.Fatalf("failed to add with remote setcode transaction: %v", err)
}
if err := pool.addRemoteSync(pricedSetCodeTx(0, 250000, uint256.NewInt(3000), uint256.NewInt(300), keys[0], []unsignedAuth{{0, keys[2]}})); err != nil {
t.Fatalf("failed to add with remote setcode transaction: %v", err)
}
fmt.Println(pool.auths)
// Now send a regular tx from keys[1].
if err := pool.addRemoteSync(pricedTransaction(0, 100000, big.NewInt(10), keys[1])); err != nil {
t.Fatalf("failed to replace with remote transaction: %v", err)
}
return nil
},
},
} {
if err := tt.run(); err != nil {
t.Fatalf("%s: %v", tt.name, err)
}
pending, queued := pool.Stats()
if pending != tt.pending {
t.Fatalf("%s: pending transactions mismatched: have %d, want %d", tt.name, pending, tt.pending)
}
if queued != tt.queued {
t.Fatalf("%s: queued transactions mismatched: have %d, want %d", tt.name, queued, tt.queued)
}
if err := validatePoolInternals(pool); err != nil {
t.Fatalf("%s: pool internal state corrupted: %v", tt.name, err)
}
pool.Clear()
}
/*
pool.Clear()
// if err := pool.addRemoteSync(setCodeTx(1, keys[3], []unsignedAuth{{1, keys[2]}})); !errors.Is(err, txpool.ErrAuthorityReserved) {
// t.Fatalf("expected to reject tx from in-flight authority: want %v, have %v", txpool.ErrAuthorityReserved, err)
// }
*/
}
// Benchmarks the speed of validating the contents of the pending queue of the
// transaction pool.
func BenchmarkPendingDemotion100(b *testing.B) { benchmarkPendingDemotion(b, 100) }

@ -69,17 +69,21 @@ func ValidateTransaction(tx *types.Transaction, head *types.Header, signer types
return fmt.Errorf("%w: transaction size %v, limit %v", ErrOversizedData, tx.Size(), opts.MaxSize)
}
// Ensure only transactions that have been enabled are accepted
if !opts.Config.IsBerlin(head.Number) && tx.Type() != types.LegacyTxType {
rules := opts.Config.Rules(head.Number, head.Difficulty.Sign() == 0, head.Time)
if !rules.IsBerlin && tx.Type() != types.LegacyTxType {
return fmt.Errorf("%w: type %d rejected, pool not yet in Berlin", core.ErrTxTypeNotSupported, tx.Type())
}
if !opts.Config.IsLondon(head.Number) && tx.Type() == types.DynamicFeeTxType {
if !rules.IsLondon && tx.Type() == types.DynamicFeeTxType {
return fmt.Errorf("%w: type %d rejected, pool not yet in London", core.ErrTxTypeNotSupported, tx.Type())
}
if !opts.Config.IsCancun(head.Number, head.Time) && tx.Type() == types.BlobTxType {
if !rules.IsCancun && tx.Type() == types.BlobTxType {
return fmt.Errorf("%w: type %d rejected, pool not yet in Cancun", core.ErrTxTypeNotSupported, tx.Type())
}
if !rules.IsPrague && tx.Type() == types.SetCodeTxType {
return fmt.Errorf("%w: type %d rejected, pool not yet in Prague", core.ErrTxTypeNotSupported, tx.Type())
}
// Check whether the init code size has been exceeded
if opts.Config.IsShanghai(head.Number, head.Time) && tx.To() == nil && len(tx.Data()) > params.MaxInitCodeSize {
if rules.IsShanghai && tx.To() == nil && len(tx.Data()) > params.MaxInitCodeSize {
return fmt.Errorf("%w: code size %v, limit %v", core.ErrMaxInitCodeSizeExceeded, len(tx.Data()), params.MaxInitCodeSize)
}
// Transactions can't be negative. This may never happen using RLP decoded
@ -108,7 +112,7 @@ func ValidateTransaction(tx *types.Transaction, head *types.Header, signer types
}
// Ensure the transaction has more gas than the bare minimum needed to cover
// the transaction metadata
intrGas, err := core.IntrinsicGas(tx.Data(), tx.AccessList(), tx.SetCodeAuthorizations(), tx.To() == nil, true, opts.Config.IsIstanbul(head.Number), opts.Config.IsShanghai(head.Number, head.Time))
intrGas, err := core.IntrinsicGas(tx.Data(), tx.AccessList(), tx.SetCodeAuthorizations(), tx.To() == nil, true, rules.IsIstanbul, rules.IsShanghai)
if err != nil {
return err
}
@ -142,6 +146,11 @@ func ValidateTransaction(tx *types.Transaction, head *types.Header, signer types
return err
}
}
if tx.Type() == types.SetCodeTxType {
if len(tx.SetCodeAuthorizations()) == 0 {
return fmt.Errorf("set code tx must have at least one authorization tuple")
}
}
return nil
}
@ -197,6 +206,11 @@ type ValidationOptionsWithState struct {
// ExistingCost is a mandatory callback to retrieve an already pooled
// transaction's cost with the given nonce to check for overdrafts.
ExistingCost func(addr common.Address, nonce uint64) *big.Int
// KnownConflicts is an optional callback which iterates over the list of
// addresses and returns all addresses known to the pool with in-flight
// transactions.
KnownConflicts func(sender common.Address, authorizers []common.Address) []common.Address
}
// ValidateTransactionWithState is a helper method to check whether a transaction
@ -250,6 +264,14 @@ func ValidateTransactionWithState(tx *types.Transaction, signer types.Signer, op
if used, left := opts.UsedAndLeftSlots(from); left <= 0 {
return fmt.Errorf("%w: pooled %d txs", ErrAccountLimitExceeded, used)
}
// Verify no authorizations will invalidate existing transactions known to
// the pool.
if opts.KnownConflicts != nil {
if conflicts := opts.KnownConflicts(from, tx.Authorities()); len(conflicts) > 0 {
return fmt.Errorf("%w: authorization conflicts with other known tx", ErrAuthorityReserved)
}
}
}
return nil
}

@ -483,6 +483,21 @@ func (tx *Transaction) SetCodeAuthorizations() []SetCodeAuthorization {
return setcodetx.AuthList
}
// Authorities returns a list of each authorization's corresponding authority.
func (tx *Transaction) Authorities() []common.Address {
setcodetx, ok := tx.inner.(*SetCodeTx)
if !ok {
return nil
}
auths := make([]common.Address, len(setcodetx.AuthList))
for _, auth := range setcodetx.AuthList {
if addr, err := auth.Authority(); err == nil {
auths = append(auths, addr)
}
}
return auths
}
// SetTime sets the decoding time of a transaction. This is used by tests to set
// arbitrary times and by persistent transaction pools when loading old txs from
// disk.

Loading…
Cancel
Save