Skip to content

Commit

Permalink
Merge pull request #2641 from OffchainLabs/remove-arbutil-l1interface
Browse files Browse the repository at this point in the history
Remove arbutil's L1Interface and use *ethclient.Client instead
  • Loading branch information
ganeshvanahalli authored Oct 1, 2024
2 parents 6b37f8a + 0c9611b commit a7e762d
Show file tree
Hide file tree
Showing 31 changed files with 139 additions and 142 deletions.
4 changes: 2 additions & 2 deletions arbnode/dataposter/data_poster.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
"github.com/ethereum/go-ethereum/consensus/misc/eip4844"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/crypto/kzg4844"
"github.com/ethereum/go-ethereum/ethclient"
"github.com/ethereum/go-ethereum/ethdb"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/metrics"
Expand All @@ -39,7 +40,6 @@ import (
"github.com/offchainlabs/nitro/arbnode/dataposter/noop"
"github.com/offchainlabs/nitro/arbnode/dataposter/slice"
"github.com/offchainlabs/nitro/arbnode/dataposter/storage"
"github.com/offchainlabs/nitro/arbutil"
"github.com/offchainlabs/nitro/util/arbmath"
"github.com/offchainlabs/nitro/util/blobs"
"github.com/offchainlabs/nitro/util/headerreader"
Expand Down Expand Up @@ -69,7 +69,7 @@ var (
type DataPoster struct {
stopwaiter.StopWaiter
headerReader *headerreader.HeaderReader
client arbutil.L1Interface
client *ethclient.Client
auth *bind.TransactOpts
signer signerFn
config ConfigFetcher
Expand Down
63 changes: 27 additions & 36 deletions arbnode/dataposter/dataposter_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2,17 +2,18 @@ package dataposter

import (
"context"
"errors"
"fmt"
"math/big"
"testing"
"time"

"github.com/Knetic/govaluate"
"github.com/ethereum/go-ethereum"
"github.com/ethereum/go-ethereum/accounts/abi/bind"
"github.com/ethereum/go-ethereum/accounts/abi/bind/backends"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/common/hexutil"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/ethclient"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/params"
"github.com/ethereum/go-ethereum/rpc"
Expand Down Expand Up @@ -152,46 +153,36 @@ func TestMaxFeeCapFormulaCalculation(t *testing.T) {
}
}

type stubL1Client struct {
type stubL1ClientInner struct {
senderNonce uint64
suggestedGasTipCap *big.Int

// Define most of the required methods that aren't used by feeAndTipCaps
backends.SimulatedBackend
}

func (c *stubL1Client) NonceAt(ctx context.Context, account common.Address, blockNumber *big.Int) (uint64, error) {
return c.senderNonce, nil
}

func (c *stubL1Client) SuggestGasTipCap(ctx context.Context) (*big.Int, error) {
return c.suggestedGasTipCap, nil
}

// Not used but we need to define
func (c *stubL1Client) BlockNumber(ctx context.Context) (uint64, error) {
return 0, nil
}

func (c *stubL1Client) CallContractAtHash(ctx context.Context, msg ethereum.CallMsg, blockHash common.Hash) ([]byte, error) {
return []byte{}, nil
}

func (c *stubL1Client) CodeAtHash(ctx context.Context, address common.Address, blockHash common.Hash) ([]byte, error) {
return []byte{}, nil
func (c *stubL1ClientInner) CallContext(ctx_in context.Context, result interface{}, method string, args ...interface{}) error {
switch method {
case "eth_getTransactionCount":
ptr, ok := result.(*hexutil.Uint64)
if !ok {
return errors.New("result is not a *hexutil.Uint64")
}
*ptr = hexutil.Uint64(c.senderNonce)
case "eth_maxPriorityFeePerGas":
ptr, ok := result.(*hexutil.Big)
if !ok {
return errors.New("result is not a *hexutil.Big")
}
*ptr = hexutil.Big(*c.suggestedGasTipCap)
}
return nil
}

func (c *stubL1Client) ChainID(ctx context.Context) (*big.Int, error) {
func (c *stubL1ClientInner) EthSubscribe(ctx context.Context, channel interface{}, args ...interface{}) (*rpc.ClientSubscription, error) {
return nil, nil
}

func (c *stubL1Client) Client() rpc.ClientInterface {
func (c *stubL1ClientInner) BatchCallContext(ctx context.Context, b []rpc.BatchElem) error {
return nil
}

func (c *stubL1Client) TransactionSender(ctx context.Context, tx *types.Transaction, block common.Hash, index uint) (common.Address, error) {
return common.Address{}, nil
}
func (c *stubL1ClientInner) Close() {}

func TestFeeAndTipCaps_EnoughBalance_NoBacklog_NoUnconfirmed_BlobTx(t *testing.T) {
conf := func() *DataPosterConfig {
Expand Down Expand Up @@ -223,10 +214,10 @@ func TestFeeAndTipCaps_EnoughBalance_NoBacklog_NoUnconfirmed_BlobTx(t *testing.T
extraBacklog: func() uint64 { return 0 },
balance: big.NewInt(0).Mul(big.NewInt(params.Ether), big.NewInt(10)),
usingNoOpStorage: false,
client: &stubL1Client{
client: ethclient.NewClient(&stubL1ClientInner{
senderNonce: 1,
suggestedGasTipCap: big.NewInt(2 * params.GWei),
},
}),
auth: &bind.TransactOpts{
From: common.Address{},
},
Expand Down Expand Up @@ -354,10 +345,10 @@ func TestFeeAndTipCaps_RBF_RisingBlobFee_FallingBaseFee(t *testing.T) {
extraBacklog: func() uint64 { return 0 },
balance: big.NewInt(0).Mul(big.NewInt(params.Ether), big.NewInt(10)),
usingNoOpStorage: false,
client: &stubL1Client{
client: ethclient.NewClient(&stubL1ClientInner{
senderNonce: 1,
suggestedGasTipCap: big.NewInt(2 * params.GWei),
},
}),
auth: &bind.TransactOpts{
From: common.Address{},
},
Expand Down
5 changes: 3 additions & 2 deletions arbnode/delayed.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"github.com/ethereum/go-ethereum/common/math"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/ethclient"

"github.com/offchainlabs/nitro/arbos/arbostypes"
"github.com/offchainlabs/nitro/arbutil"
Expand Down Expand Up @@ -58,11 +59,11 @@ type DelayedBridge struct {
con *bridgegen.IBridge
address common.Address
fromBlock uint64
client arbutil.L1Interface
client *ethclient.Client
messageProviders map[common.Address]*bridgegen.IDelayedMessageProvider
}

func NewDelayedBridge(client arbutil.L1Interface, addr common.Address, fromBlock uint64) (*DelayedBridge, error) {
func NewDelayedBridge(client *ethclient.Client, addr common.Address, fromBlock uint64) (*DelayedBridge, error) {
con, err := bridgegen.NewIBridge(addr, client)
if err != nil {
return nil, err
Expand Down
5 changes: 3 additions & 2 deletions arbnode/inbox_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ import (
"time"

"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/ethclient"
"github.com/ethereum/go-ethereum/log"
flag "github.com/spf13/pflag"

Expand Down Expand Up @@ -93,15 +94,15 @@ type InboxReader struct {
delayedBridge *DelayedBridge
sequencerInbox *SequencerInbox
caughtUpChan chan struct{}
client arbutil.L1Interface
client *ethclient.Client
l1Reader *headerreader.HeaderReader

// Atomic
lastSeenBatchCount atomic.Uint64
lastReadBatchCount atomic.Uint64
}

func NewInboxReader(tracker *InboxTracker, client arbutil.L1Interface, l1Reader *headerreader.HeaderReader, firstMessageBlock *big.Int, delayedBridge *DelayedBridge, sequencerInbox *SequencerInbox, config InboxReaderConfigFetcher) (*InboxReader, error) {
func NewInboxReader(tracker *InboxTracker, client *ethclient.Client, l1Reader *headerreader.HeaderReader, firstMessageBlock *big.Int, delayedBridge *DelayedBridge, sequencerInbox *SequencerInbox, config InboxReaderConfigFetcher) (*InboxReader, error) {
err := config().Validate()
if err != nil {
return nil, err
Expand Down
5 changes: 3 additions & 2 deletions arbnode/inbox_tracker.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ import (
"time"

"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/ethclient"
"github.com/ethereum/go-ethereum/ethdb"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/metrics"
Expand Down Expand Up @@ -599,7 +600,7 @@ type multiplexerBackend struct {
positionWithinMessage uint64

ctx context.Context
client arbutil.L1Interface
client *ethclient.Client
inbox *InboxTracker
}

Expand Down Expand Up @@ -639,7 +640,7 @@ func (b *multiplexerBackend) ReadDelayedInbox(seqNum uint64) (*arbostypes.L1Inco

var delayedMessagesMismatch = errors.New("sequencer batch delayed messages missing or different")

func (t *InboxTracker) AddSequencerBatches(ctx context.Context, client arbutil.L1Interface, batches []*SequencerInboxBatch) error {
func (t *InboxTracker) AddSequencerBatches(ctx context.Context, client *ethclient.Client, batches []*SequencerInboxBatch) error {
var nextAcc common.Hash
var prevbatchmeta BatchMetadata
sequenceNumberToKeep := uint64(0)
Expand Down
5 changes: 3 additions & 2 deletions arbnode/node.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/rawdb"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/ethclient"
"github.com/ethereum/go-ethereum/ethdb"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/node"
Expand Down Expand Up @@ -407,7 +408,7 @@ func createNodeImpl(
arbDb ethdb.Database,
configFetcher ConfigFetcher,
l2Config *params.ChainConfig,
l1client arbutil.L1Interface,
l1client *ethclient.Client,
deployInfo *chaininfo.RollupAddresses,
txOptsValidator *bind.TransactOpts,
txOptsBatchPoster *bind.TransactOpts,
Expand Down Expand Up @@ -781,7 +782,7 @@ func CreateNode(
arbDb ethdb.Database,
configFetcher ConfigFetcher,
l2Config *params.ChainConfig,
l1client arbutil.L1Interface,
l1client *ethclient.Client,
deployInfo *chaininfo.RollupAddresses,
txOptsValidator *bind.TransactOpts,
txOptsBatchPoster *bind.TransactOpts,
Expand Down
9 changes: 5 additions & 4 deletions arbnode/sequencer_inbox.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ import (
"github.com/ethereum/go-ethereum/accounts/abi/bind"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/ethclient"
"github.com/offchainlabs/nitro/arbstate/daprovider"
"github.com/offchainlabs/nitro/arbutil"

Expand Down Expand Up @@ -52,10 +53,10 @@ type SequencerInbox struct {
con *bridgegen.SequencerInbox
address common.Address
fromBlock int64
client arbutil.L1Interface
client *ethclient.Client
}

func NewSequencerInbox(client arbutil.L1Interface, addr common.Address, fromBlock int64) (*SequencerInbox, error) {
func NewSequencerInbox(client *ethclient.Client, addr common.Address, fromBlock int64) (*SequencerInbox, error) {
con, err := bridgegen.NewSequencerInbox(addr, client)
if err != nil {
return nil, err
Expand Down Expand Up @@ -111,7 +112,7 @@ type SequencerInboxBatch struct {
serialized []byte // nil if serialization isn't cached yet
}

func (m *SequencerInboxBatch) getSequencerData(ctx context.Context, client arbutil.L1Interface) ([]byte, error) {
func (m *SequencerInboxBatch) getSequencerData(ctx context.Context, client *ethclient.Client) ([]byte, error) {
switch m.dataLocation {
case batchDataTxInput:
data, err := arbutil.GetLogEmitterTxData(ctx, client, m.rawLog)
Expand Down Expand Up @@ -169,7 +170,7 @@ func (m *SequencerInboxBatch) getSequencerData(ctx context.Context, client arbut
}
}

func (m *SequencerInboxBatch) Serialize(ctx context.Context, client arbutil.L1Interface) ([]byte, error) {
func (m *SequencerInboxBatch) Serialize(ctx context.Context, client *ethclient.Client) ([]byte, error) {
if m.serialized != nil {
return m.serialized, nil
}
Expand Down
6 changes: 5 additions & 1 deletion arbutil/correspondingl1blocknumber.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,11 @@ func ParentHeaderToL1BlockNumber(header *types.Header) uint64 {
return header.Number.Uint64()
}

func CorrespondingL1BlockNumber(ctx context.Context, client L1Interface, parentBlockNumber uint64) (uint64, error) {
type ParentHeaderFetcher interface {
HeaderByNumber(ctx context.Context, number *big.Int) (*types.Header, error)
}

func CorrespondingL1BlockNumber(ctx context.Context, client ParentHeaderFetcher, parentBlockNumber uint64) (uint64, error) {
// #nosec G115
header, err := client.HeaderByNumber(ctx, big.NewInt(int64(parentBlockNumber)))
if err != nil {
Expand Down
5 changes: 3 additions & 2 deletions arbutil/transaction_data.go
Original file line number Diff line number Diff line change
Expand Up @@ -8,9 +8,10 @@ import (
"fmt"

"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/ethclient"
)

func GetLogTransaction(ctx context.Context, client L1Interface, log types.Log) (*types.Transaction, error) {
func GetLogTransaction(ctx context.Context, client *ethclient.Client, log types.Log) (*types.Transaction, error) {
tx, err := client.TransactionInBlock(ctx, log.BlockHash, log.TxIndex)
if err != nil {
return nil, err
Expand All @@ -22,7 +23,7 @@ func GetLogTransaction(ctx context.Context, client L1Interface, log types.Log) (
}

// GetLogEmitterTxData requires that the tx's data is at least 4 bytes long
func GetLogEmitterTxData(ctx context.Context, client L1Interface, log types.Log) ([]byte, error) {
func GetLogEmitterTxData(ctx context.Context, client *ethclient.Client, log types.Log) ([]byte, error) {
tx, err := GetLogTransaction(ctx, client, log)
if err != nil {
return nil, err
Expand Down
24 changes: 5 additions & 19 deletions arbutil/wait_for_l1.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,27 +10,13 @@ import (
"math/big"

"github.com/ethereum/go-ethereum"
"github.com/ethereum/go-ethereum/accounts/abi/bind"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/core/vm"
"github.com/ethereum/go-ethereum/rpc"
"github.com/ethereum/go-ethereum/ethclient"
)

type L1Interface interface {
bind.ContractBackend
bind.BlockHashContractCaller
ethereum.ChainReader
ethereum.ChainStateReader
ethereum.TransactionReader
TransactionSender(ctx context.Context, tx *types.Transaction, block common.Hash, index uint) (common.Address, error)
BlockNumber(ctx context.Context) (uint64, error)
PendingCallContract(ctx context.Context, msg ethereum.CallMsg) ([]byte, error)
ChainID(ctx context.Context) (*big.Int, error)
Client() rpc.ClientInterface
}

func SendTxAsCall(ctx context.Context, client L1Interface, tx *types.Transaction, from common.Address, blockNum *big.Int, unlimitedGas bool) ([]byte, error) {
func SendTxAsCall(ctx context.Context, client *ethclient.Client, tx *types.Transaction, from common.Address, blockNum *big.Int, unlimitedGas bool) ([]byte, error) {
var gas uint64
if unlimitedGas {
gas = 0
Expand All @@ -50,7 +36,7 @@ func SendTxAsCall(ctx context.Context, client L1Interface, tx *types.Transaction
return client.CallContract(ctx, callMsg, blockNum)
}

func GetPendingCallBlockNumber(ctx context.Context, client L1Interface) (*big.Int, error) {
func GetPendingCallBlockNumber(ctx context.Context, client *ethclient.Client) (*big.Int, error) {
msg := ethereum.CallMsg{
// Pretend to be a contract deployment to execute EVM code without calling a contract.
To: nil,
Expand All @@ -70,7 +56,7 @@ func GetPendingCallBlockNumber(ctx context.Context, client L1Interface) (*big.In
return new(big.Int).SetBytes(callRes), nil
}

func DetailTxError(ctx context.Context, client L1Interface, tx *types.Transaction, txRes *types.Receipt) error {
func DetailTxError(ctx context.Context, client *ethclient.Client, tx *types.Transaction, txRes *types.Receipt) error {
// Re-execute the transaction as a call to get a better error
if ctx.Err() != nil {
return ctx.Err()
Expand All @@ -96,7 +82,7 @@ func DetailTxError(ctx context.Context, client L1Interface, tx *types.Transactio
return fmt.Errorf("SendTxAsCall got: %w for tx hash %v", err, tx.Hash())
}

func DetailTxErrorUsingCallMsg(ctx context.Context, client L1Interface, txHash common.Hash, txRes *types.Receipt, callMsg ethereum.CallMsg) error {
func DetailTxErrorUsingCallMsg(ctx context.Context, client *ethclient.Client, txHash common.Hash, txRes *types.Receipt, callMsg ethereum.CallMsg) error {
// Re-execute the transaction as a call to get a better error
if ctx.Err() != nil {
return ctx.Err()
Expand Down
4 changes: 2 additions & 2 deletions cmd/nitro/init.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ import (
"github.com/ethereum/go-ethereum/core"
"github.com/ethereum/go-ethereum/core/rawdb"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/ethclient"
"github.com/ethereum/go-ethereum/ethdb"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/node"
Expand All @@ -37,7 +38,6 @@ import (
"github.com/offchainlabs/nitro/arbnode"
"github.com/offchainlabs/nitro/arbos/arbosState"
"github.com/offchainlabs/nitro/arbos/arbostypes"
"github.com/offchainlabs/nitro/arbutil"
"github.com/offchainlabs/nitro/cmd/chaininfo"
"github.com/offchainlabs/nitro/cmd/conf"
"github.com/offchainlabs/nitro/cmd/ipfshelper"
Expand Down Expand Up @@ -560,7 +560,7 @@ func rebuildLocalWasm(ctx context.Context, config *gethexec.Config, l2BlockChain
return chainDb, l2BlockChain, nil
}

func openInitializeChainDb(ctx context.Context, stack *node.Node, config *NodeConfig, chainId *big.Int, cacheConfig *core.CacheConfig, targetConfig *gethexec.StylusTargetConfig, persistentConfig *conf.PersistentConfig, l1Client arbutil.L1Interface, rollupAddrs chaininfo.RollupAddresses) (ethdb.Database, *core.BlockChain, error) {
func openInitializeChainDb(ctx context.Context, stack *node.Node, config *NodeConfig, chainId *big.Int, cacheConfig *core.CacheConfig, targetConfig *gethexec.StylusTargetConfig, persistentConfig *conf.PersistentConfig, l1Client *ethclient.Client, rollupAddrs chaininfo.RollupAddresses) (ethdb.Database, *core.BlockChain, error) {
if !config.Init.Force {
if readOnlyDb, err := stack.OpenDatabaseWithFreezerWithExtraOptions("l2chaindata", 0, 0, config.Persistent.Ancient, "l2chaindata/", true, persistentConfig.Pebble.ExtraOptions("l2chaindata")); err == nil {
if chainConfig := gethexec.TryReadStoredChainConfig(readOnlyDb); chainConfig != nil {
Expand Down
Loading

0 comments on commit a7e762d

Please sign in to comment.