Commit ea8e6237 authored by Joshua Gutow's avatar Joshua Gutow

op-batcher: Simplify configs, dedup init, & simplify init code

This uses the same op-services/crypto signer initialization. It also
simplifies the batcher struct and config.
parent a49e1040
...@@ -3,21 +3,18 @@ package batcher ...@@ -3,21 +3,18 @@ package batcher
import ( import (
"context" "context"
"fmt" "fmt"
"math/big"
_ "net/http/pprof" _ "net/http/pprof"
"os" "os"
"os/signal" "os/signal"
"syscall" "syscall"
"time" "time"
"github.com/urfave/cli"
oplog "github.com/ethereum-optimism/optimism/op-service/log" oplog "github.com/ethereum-optimism/optimism/op-service/log"
opmetrics "github.com/ethereum-optimism/optimism/op-service/metrics" opmetrics "github.com/ethereum-optimism/optimism/op-service/metrics"
oppprof "github.com/ethereum-optimism/optimism/op-service/pprof" oppprof "github.com/ethereum-optimism/optimism/op-service/pprof"
oprpc "github.com/ethereum-optimism/optimism/op-service/rpc" oprpc "github.com/ethereum-optimism/optimism/op-service/rpc"
opsigner "github.com/ethereum-optimism/optimism/op-signer/client"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/types"
"github.com/urfave/cli"
) )
const ( const (
...@@ -30,98 +27,75 @@ const ( ...@@ -30,98 +27,75 @@ const (
// closure that executes the service and blocks until the service exits. The use // closure that executes the service and blocks until the service exits. The use
// of a closure allows the parameters bound to the top-level main package, e.g. // of a closure allows the parameters bound to the top-level main package, e.g.
// GitVersion, to be captured and used once the function is executed. // GitVersion, to be captured and used once the function is executed.
func Main(version string) func(cliCtx *cli.Context) error { func Main(version string, cliCtx *cli.Context) error {
return func(cliCtx *cli.Context) error { cfg := NewConfig(cliCtx)
cfg := NewConfig(cliCtx) if err := cfg.Check(); err != nil {
if err := cfg.Check(); err != nil { return fmt.Errorf("invalid CLI flags: %w", err)
return fmt.Errorf("invalid CLI flags: %w", err) }
}
l := oplog.NewLogger(cfg.LogConfig)
l.Info("Initializing Batch Submitter")
var batchSubmitter *BatchSubmitter l := oplog.NewLogger(cfg.LogConfig)
if !cfg.SignerConfig.Enabled() { l.Info("Initializing Batch Submitter")
bs, err := NewBatchSubmitter(cfg, l)
if err != nil {
l.Error("Unable to create Batch Submitter", "error", err)
return err
}
batchSubmitter = bs
} else {
signerClient, err := opsigner.NewSignerClientFromConfig(l, cfg.SignerConfig)
if err != nil {
l.Error("Unable to create Signer Client", "error", err)
return err
}
signer := func(chainID *big.Int) SignerFn {
return func(ctx context.Context, rawTx types.TxData) (*types.Transaction, error) {
tx := types.NewTx(rawTx)
return signerClient.SignTransaction(ctx, chainID, tx)
}
}
bs, err := NewBatchSubmitterWithSigner(cfg, common.HexToAddress(cfg.SignerConfig.Address), signer, l)
if err != nil {
l.Error("Unable to create Batch Submitter with signer", "error", err)
return err
}
batchSubmitter = bs
}
l.Info("Starting Batch Submitter") batchSubmitter, err := NewBatchSubmitterFromCLIConfig(cfg, l)
if err != nil {
l.Error("Unable to create Batch Submitter", "error", err)
return err
}
if err := batchSubmitter.Start(); err != nil { l.Info("Starting Batch Submitter")
l.Error("Unable to start Batch Submitter", "error", err)
return err
}
defer batchSubmitter.Stop()
ctx, cancel := context.WithCancel(context.Background()) if err := batchSubmitter.Start(); err != nil {
l.Error("Unable to start Batch Submitter", "error", err)
return err
}
defer batchSubmitter.Stop()
l.Info("Batch Submitter started") ctx, cancel := context.WithCancel(context.Background())
pprofConfig := cfg.PprofConfig
if pprofConfig.Enabled {
l.Info("starting pprof", "addr", pprofConfig.ListenAddr, "port", pprofConfig.ListenPort)
go func() {
if err := oppprof.ListenAndServe(ctx, pprofConfig.ListenAddr, pprofConfig.ListenPort); err != nil {
l.Error("error starting pprof", "err", err)
}
}()
}
registry := opmetrics.NewRegistry() l.Info("Batch Submitter started")
metricsCfg := cfg.MetricsConfig pprofConfig := cfg.PprofConfig
if metricsCfg.Enabled { if pprofConfig.Enabled {
l.Info("starting metrics server", "addr", metricsCfg.ListenAddr, "port", metricsCfg.ListenPort) l.Info("starting pprof", "addr", pprofConfig.ListenAddr, "port", pprofConfig.ListenPort)
go func() { go func() {
if err := opmetrics.ListenAndServe(ctx, registry, metricsCfg.ListenAddr, metricsCfg.ListenPort); err != nil { if err := oppprof.ListenAndServe(ctx, pprofConfig.ListenAddr, pprofConfig.ListenPort); err != nil {
l.Error("error starting metrics server", err) l.Error("error starting pprof", "err", err)
} }
}() }()
opmetrics.LaunchBalanceMetrics(ctx, l, registry, "", batchSubmitter.cfg.L1Client, batchSubmitter.addr) }
}
rpcCfg := cfg.RPCConfig registry := opmetrics.NewRegistry()
server := oprpc.NewServer( metricsCfg := cfg.MetricsConfig
rpcCfg.ListenAddr, if metricsCfg.Enabled {
rpcCfg.ListenPort, l.Info("starting metrics server", "addr", metricsCfg.ListenAddr, "port", metricsCfg.ListenPort)
version, go func() {
) if err := opmetrics.ListenAndServe(ctx, registry, metricsCfg.ListenAddr, metricsCfg.ListenPort); err != nil {
if err := server.Start(); err != nil { l.Error("error starting metrics server", err)
cancel() }
return fmt.Errorf("error starting RPC server: %w", err) }()
} opmetrics.LaunchBalanceMetrics(ctx, l, registry, "", batchSubmitter.L1Client, batchSubmitter.From)
}
interruptChannel := make(chan os.Signal, 1) rpcCfg := cfg.RPCConfig
signal.Notify(interruptChannel, []os.Signal{ server := oprpc.NewServer(
os.Interrupt, rpcCfg.ListenAddr,
os.Kill, rpcCfg.ListenPort,
syscall.SIGTERM, version,
syscall.SIGQUIT, )
}...) if err := server.Start(); err != nil {
<-interruptChannel
cancel() cancel()
_ = server.Stop() return fmt.Errorf("error starting RPC server: %w", err)
return nil
} }
interruptChannel := make(chan os.Signal, 1)
signal.Notify(interruptChannel, []os.Signal{
os.Interrupt,
os.Kill,
syscall.SIGTERM,
syscall.SIGQUIT,
}...)
<-interruptChannel
cancel()
_ = server.Stop()
return nil
} }
package batcher package batcher
import ( import (
"math/big"
"time" "time"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/ethclient"
"github.com/ethereum/go-ethereum/log"
"github.com/urfave/cli" "github.com/urfave/cli"
"github.com/ethereum-optimism/optimism/op-batcher/flags" "github.com/ethereum-optimism/optimism/op-batcher/flags"
"github.com/ethereum-optimism/optimism/op-node/sources"
opcrypto "github.com/ethereum-optimism/optimism/op-service/crypto"
oplog "github.com/ethereum-optimism/optimism/op-service/log" oplog "github.com/ethereum-optimism/optimism/op-service/log"
opmetrics "github.com/ethereum-optimism/optimism/op-service/metrics" opmetrics "github.com/ethereum-optimism/optimism/op-service/metrics"
oppprof "github.com/ethereum-optimism/optimism/op-service/pprof" oppprof "github.com/ethereum-optimism/optimism/op-service/pprof"
oprpc "github.com/ethereum-optimism/optimism/op-service/rpc" oprpc "github.com/ethereum-optimism/optimism/op-service/rpc"
"github.com/ethereum-optimism/optimism/op-service/txmgr"
opsigner "github.com/ethereum-optimism/optimism/op-signer/client" opsigner "github.com/ethereum-optimism/optimism/op-signer/client"
) )
type Config struct { type Config struct {
log log.Logger
L1Client *ethclient.Client
L2Client *ethclient.Client
RollupNode *sources.RollupClient
PollInterval time.Duration
TxManagerConfig txmgr.Config
From common.Address
SignerFnFactory opcrypto.SignerFactory
ChainID *big.Int
// Where to send the batch txs to.
BatchInboxAddress common.Address
// Channel creation parameters
Channel ChannelConfig
}
type CLIConfig struct {
/* Required Params */ /* Required Params */
// L1EthRpc is the HTTP provider URL for L1. // L1EthRpc is the HTTP provider URL for L1.
...@@ -89,7 +114,7 @@ type Config struct { ...@@ -89,7 +114,7 @@ type Config struct {
SignerConfig opsigner.CLIConfig SignerConfig opsigner.CLIConfig
} }
func (c Config) Check() error { func (c CLIConfig) Check() error {
if err := c.RPCConfig.Check(); err != nil { if err := c.RPCConfig.Check(); err != nil {
return err return err
} }
...@@ -109,8 +134,8 @@ func (c Config) Check() error { ...@@ -109,8 +134,8 @@ func (c Config) Check() error {
} }
// NewConfig parses the Config from the provided flags or environment variables. // NewConfig parses the Config from the provided flags or environment variables.
func NewConfig(ctx *cli.Context) Config { func NewConfig(ctx *cli.Context) CLIConfig {
return Config{ return CLIConfig{
/* Required Flags */ /* Required Flags */
L1EthRpc: ctx.GlobalString(flags.L1EthRpcFlag.Name), L1EthRpc: ctx.GlobalString(flags.L1EthRpcFlag.Name),
L2EthRpc: ctx.GlobalString(flags.L2EthRpcFlag.Name), L2EthRpc: ctx.GlobalString(flags.L2EthRpcFlag.Name),
......
...@@ -2,34 +2,28 @@ package batcher ...@@ -2,34 +2,28 @@ package batcher
import ( import (
"context" "context"
"crypto/ecdsa"
"errors" "errors"
"fmt" "fmt"
"io" "io"
"math/big" "math/big"
"strings" _ "net/http/pprof"
"sync" "sync"
"time" "time"
hdwallet "github.com/ethereum-optimism/go-ethereum-hdwallet"
"github.com/ethereum-optimism/optimism/op-node/eth" "github.com/ethereum-optimism/optimism/op-node/eth"
opcrypto "github.com/ethereum-optimism/optimism/op-service/crypto"
"github.com/ethereum-optimism/optimism/op-service/txmgr" "github.com/ethereum-optimism/optimism/op-service/txmgr"
"github.com/ethereum/go-ethereum/accounts"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/log" "github.com/ethereum/go-ethereum/log"
) )
// BatchSubmitter encapsulates a service responsible for submitting L2 tx // BatchSubmitter encapsulates a service responsible for submitting L2 tx
// batches to L1 for availability. // batches to L1 for availability.
type BatchSubmitter struct { type BatchSubmitter struct {
Config // directly embed the config + sources
txMgr *TransactionManager txMgr *TransactionManager
addr common.Address
cfg DriverConfig
wg sync.WaitGroup wg sync.WaitGroup
done chan struct{} done chan struct{}
log log.Logger
ctx context.Context ctx context.Context
cancel context.CancelFunc cancel context.CancelFunc
...@@ -40,63 +34,16 @@ type BatchSubmitter struct { ...@@ -40,63 +34,16 @@ type BatchSubmitter struct {
state *channelManager state *channelManager
} }
// NewBatchSubmitter initializes the BatchSubmitter, gathering any resources // NewBatchSubmitterFromCLIConfig initializes the BatchSubmitter, gathering any resources
// that will be needed during operation. // that will be needed during operation.
func NewBatchSubmitter(cfg Config, l log.Logger) (*BatchSubmitter, error) { func NewBatchSubmitterFromCLIConfig(cfg CLIConfig, l log.Logger) (*BatchSubmitter, error) {
var err error ctx := context.Background()
var sequencerPrivKey *ecdsa.PrivateKey
var addr common.Address
if cfg.PrivateKey != "" && cfg.Mnemonic != "" {
return nil, errors.New("cannot specify both a private key and a mnemonic")
}
if cfg.PrivateKey == "" {
// Parse wallet private key that will be used to submit L2 txs to the batch
// inbox address.
wallet, err := hdwallet.NewFromMnemonic(cfg.Mnemonic)
if err != nil {
return nil, err
}
acc := accounts.Account{
URL: accounts.URL{
Path: cfg.SequencerHDPath,
},
}
addr, err = wallet.Address(acc)
if err != nil {
return nil, err
}
sequencerPrivKey, err = wallet.PrivateKey(acc)
if err != nil {
return nil, err
}
} else {
sequencerPrivKey, err = crypto.HexToECDSA(strings.TrimPrefix(cfg.PrivateKey, "0x"))
if err != nil {
return nil, err
}
addr = crypto.PubkeyToAddress(sequencerPrivKey.PublicKey)
}
signer := func(chainID *big.Int) SignerFn { signer, fromAddress, err := opcrypto.SignerFactoryFromConfig(l, cfg.PrivateKey, cfg.Mnemonic, cfg.SequencerHDPath, cfg.SignerConfig)
s := types.LatestSignerForChainID(chainID) if err != nil {
return func(_ context.Context, rawTx types.TxData) (*types.Transaction, error) { return nil, err
return types.SignNewTx(sequencerPrivKey, s, rawTx)
}
} }
return NewBatchSubmitterWithSigner(cfg, addr, signer, l)
}
type SignerFactory func(chainID *big.Int) SignerFn
func NewBatchSubmitterWithSigner(cfg Config, addr common.Address, signer SignerFactory, l log.Logger) (*BatchSubmitter, error) {
ctx := context.Background()
batchInboxAddress, err := parseAddress(cfg.SequencerBatchInboxAddress) batchInboxAddress, err := parseAddress(cfg.SequencerBatchInboxAddress)
if err != nil { if err != nil {
return nil, err return nil, err
...@@ -124,13 +71,6 @@ func NewBatchSubmitterWithSigner(cfg Config, addr common.Address, signer SignerF ...@@ -124,13 +71,6 @@ func NewBatchSubmitterWithSigner(cfg Config, addr common.Address, signer SignerF
return nil, err return nil, err
} }
sequencerBalance, err := l1Client.BalanceAt(ctx, addr, nil)
if err != nil {
return nil, err
}
log.Info("starting batch submitter", "submitter_addr", addr, "submitter_bal", sequencerBalance)
txManagerConfig := txmgr.Config{ txManagerConfig := txmgr.Config{
Log: l, Log: l,
Name: "Batch Submitter", Name: "Batch Submitter",
...@@ -140,12 +80,15 @@ func NewBatchSubmitterWithSigner(cfg Config, addr common.Address, signer SignerF ...@@ -140,12 +80,15 @@ func NewBatchSubmitterWithSigner(cfg Config, addr common.Address, signer SignerF
SafeAbortNonceTooLowCount: cfg.SafeAbortNonceTooLowCount, SafeAbortNonceTooLowCount: cfg.SafeAbortNonceTooLowCount,
} }
batcherCfg := DriverConfig{ batcherCfg := Config{
Log: l,
Name: "Batch Submitter",
L1Client: l1Client, L1Client: l1Client,
L2Client: l2Client, L2Client: l2Client,
RollupNode: rollupClient, RollupNode: rollupClient,
ChainID: chainID,
PollInterval: cfg.PollInterval,
TxManagerConfig: txManagerConfig,
From: fromAddress,
SignerFnFactory: signer,
BatchInboxAddress: batchInboxAddress, BatchInboxAddress: batchInboxAddress,
Channel: ChannelConfig{ Channel: ChannelConfig{
ChannelTimeout: cfg.ChannelTimeout, ChannelTimeout: cfg.ChannelTimeout,
...@@ -154,24 +97,36 @@ func NewBatchSubmitterWithSigner(cfg Config, addr common.Address, signer SignerF ...@@ -154,24 +97,36 @@ func NewBatchSubmitterWithSigner(cfg Config, addr common.Address, signer SignerF
TargetNumFrames: cfg.TargetNumFrames, TargetNumFrames: cfg.TargetNumFrames,
ApproxComprRatio: cfg.ApproxComprRatio, ApproxComprRatio: cfg.ApproxComprRatio,
}, },
ChainID: chainID,
PollInterval: cfg.PollInterval,
} }
return NewBatchSubmitter(batcherCfg, l)
}
// NewBatchSubmitter initializes the BatchSubmitter, gathering any resources
// that will be needed during operation.
func NewBatchSubmitter(cfg Config, l log.Logger) (*BatchSubmitter, error) {
ctx, cancel := context.WithCancel(context.Background()) ctx, cancel := context.WithCancel(context.Background())
balance, err := cfg.L1Client.BalanceAt(ctx, cfg.From, nil)
if err != nil {
cancel()
return nil, err
}
cfg.log = l
cfg.log.Info("creating batch submitter", "submitter_addr", cfg.From, "submitter_bal", balance)
return &BatchSubmitter{ return &BatchSubmitter{
cfg: batcherCfg, Config: cfg,
addr: addr, txMgr: NewTransactionManager(l, cfg.TxManagerConfig, cfg.BatchInboxAddress, cfg.ChainID, cfg.From, cfg.L1Client, cfg.SignerFnFactory(cfg.ChainID)),
txMgr: NewTransactionManager(l, txManagerConfig, batchInboxAddress, chainID, addr, l1Client, signer(chainID)), done: make(chan struct{}),
done: make(chan struct{}),
log: l,
state: NewChannelManager(l, batcherCfg.Channel),
// TODO: this context only exists because the event loop doesn't reach done // TODO: this context only exists because the event loop doesn't reach done
// if the tx manager is blocking forever due to e.g. insufficient balance. // if the tx manager is blocking forever due to e.g. insufficient balance.
ctx: ctx, ctx: ctx,
cancel: cancel, cancel: cancel,
state: NewChannelManager(l, cfg.Channel),
}, nil }, nil
} }
func (l *BatchSubmitter) Start() error { func (l *BatchSubmitter) Start() error {
...@@ -218,7 +173,7 @@ func (l *BatchSubmitter) loadBlocksIntoState(ctx context.Context) { ...@@ -218,7 +173,7 @@ func (l *BatchSubmitter) loadBlocksIntoState(ctx context.Context) {
// loadBlockIntoState fetches & stores a single block into `state`. It returns the block it loaded. // loadBlockIntoState fetches & stores a single block into `state`. It returns the block it loaded.
func (l *BatchSubmitter) loadBlockIntoState(ctx context.Context, blockNumber uint64) (eth.BlockID, error) { func (l *BatchSubmitter) loadBlockIntoState(ctx context.Context, blockNumber uint64) (eth.BlockID, error) {
ctx, cancel := context.WithTimeout(ctx, networkTimeout) ctx, cancel := context.WithTimeout(ctx, networkTimeout)
block, err := l.cfg.L2Client.BlockByNumber(ctx, new(big.Int).SetUint64(blockNumber)) block, err := l.L2Client.BlockByNumber(ctx, new(big.Int).SetUint64(blockNumber))
cancel() cancel()
if err != nil { if err != nil {
return eth.BlockID{}, err return eth.BlockID{}, err
...@@ -236,7 +191,7 @@ func (l *BatchSubmitter) loadBlockIntoState(ctx context.Context, blockNumber uin ...@@ -236,7 +191,7 @@ func (l *BatchSubmitter) loadBlockIntoState(ctx context.Context, blockNumber uin
func (l *BatchSubmitter) calculateL2BlockRangeToStore(ctx context.Context) (eth.BlockID, eth.BlockID, error) { func (l *BatchSubmitter) calculateL2BlockRangeToStore(ctx context.Context) (eth.BlockID, eth.BlockID, error) {
childCtx, cancel := context.WithTimeout(ctx, networkTimeout) childCtx, cancel := context.WithTimeout(ctx, networkTimeout)
defer cancel() defer cancel()
syncStatus, err := l.cfg.RollupNode.SyncStatus(childCtx) syncStatus, err := l.RollupNode.SyncStatus(childCtx)
// Ensure that we have the sync status // Ensure that we have the sync status
if err != nil { if err != nil {
return eth.BlockID{}, eth.BlockID{}, fmt.Errorf("failed to get sync status: %w", err) return eth.BlockID{}, eth.BlockID{}, fmt.Errorf("failed to get sync status: %w", err)
...@@ -277,7 +232,7 @@ func (l *BatchSubmitter) calculateL2BlockRangeToStore(ctx context.Context) (eth. ...@@ -277,7 +232,7 @@ func (l *BatchSubmitter) calculateL2BlockRangeToStore(ctx context.Context) (eth.
func (l *BatchSubmitter) loop() { func (l *BatchSubmitter) loop() {
defer l.wg.Done() defer l.wg.Done()
ticker := time.NewTicker(l.cfg.PollInterval) ticker := time.NewTicker(l.PollInterval)
defer ticker.Stop() defer ticker.Stop()
for { for {
select { select {
......
package batcher
import (
"math/big"
"time"
"github.com/ethereum-optimism/optimism/op-node/sources"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/ethclient"
"github.com/ethereum/go-ethereum/log"
)
type DriverConfig struct {
Log log.Logger
Name string
// API to submit txs to
L1Client *ethclient.Client
// API to hit for batch data
L2Client *ethclient.Client
RollupNode *sources.RollupClient
// Where to send the batch txs to.
BatchInboxAddress common.Address
// Channel creation parameters
Channel ChannelConfig
// Chain ID of the L1 chain to submit txs to.
ChainID *big.Int
PollInterval time.Duration
}
...@@ -13,12 +13,12 @@ import ( ...@@ -13,12 +13,12 @@ import (
"github.com/ethereum/go-ethereum/ethclient" "github.com/ethereum/go-ethereum/ethclient"
"github.com/ethereum/go-ethereum/log" "github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/params" "github.com/ethereum/go-ethereum/params"
opcrypto "github.com/ethereum-optimism/optimism/op-service/crypto"
) )
const networkTimeout = 2 * time.Second // How long a single network request can take. TODO: put in a config somewhere const networkTimeout = 2 * time.Second // How long a single network request can take. TODO: put in a config somewhere
type SignerFn func(ctx context.Context, rawTx types.TxData) (*types.Transaction, error)
// TransactionManager wraps the simple txmgr package to make it easy to send & wait for transactions // TransactionManager wraps the simple txmgr package to make it easy to send & wait for transactions
type TransactionManager struct { type TransactionManager struct {
// Config // Config
...@@ -28,11 +28,11 @@ type TransactionManager struct { ...@@ -28,11 +28,11 @@ type TransactionManager struct {
// Outside world // Outside world
txMgr txmgr.TxManager txMgr txmgr.TxManager
l1Client *ethclient.Client l1Client *ethclient.Client
signerFn SignerFn signerFn opcrypto.SignerFn
log log.Logger log log.Logger
} }
func NewTransactionManager(log log.Logger, txMgrConfg txmgr.Config, batchInboxAddress common.Address, chainID *big.Int, senderAddress common.Address, l1Client *ethclient.Client, signerFn SignerFn) *TransactionManager { func NewTransactionManager(log log.Logger, txMgrConfg txmgr.Config, batchInboxAddress common.Address, chainID *big.Int, senderAddress common.Address, l1Client *ethclient.Client, signerFn opcrypto.SignerFn) *TransactionManager {
t := &TransactionManager{ t := &TransactionManager{
batchInboxAddress: batchInboxAddress, batchInboxAddress: batchInboxAddress,
senderAddress: senderAddress, senderAddress: senderAddress,
...@@ -132,7 +132,8 @@ func (t *TransactionManager) CraftTx(ctx context.Context, data []byte) (*types.T ...@@ -132,7 +132,8 @@ func (t *TransactionManager) CraftTx(ctx context.Context, data []byte) (*types.T
ctx, cancel = context.WithTimeout(ctx, networkTimeout) ctx, cancel = context.WithTimeout(ctx, networkTimeout)
defer cancel() defer cancel()
return t.signerFn(ctx, rawTx) tx := types.NewTx(rawTx)
return t.signerFn(ctx, t.senderAddress, tx)
} }
// UpdateGasPrice signs an otherwise identical txn to the one provided but with // UpdateGasPrice signs an otherwise identical txn to the one provided but with
...@@ -157,5 +158,6 @@ func (t *TransactionManager) UpdateGasPrice(ctx context.Context, tx *types.Trans ...@@ -157,5 +158,6 @@ func (t *TransactionManager) UpdateGasPrice(ctx context.Context, tx *types.Trans
// Only log the new tip/fee cap because the updateGasPrice closure reuses the same initial transaction // Only log the new tip/fee cap because the updateGasPrice closure reuses the same initial transaction
t.log.Trace("updating gas price", "tip_cap", gasTipCap, "fee_cap", gasFeeCap) t.log.Trace("updating gas price", "tip_cap", gasTipCap, "fee_cap", gasFeeCap)
return t.signerFn(ctx, rawTx) finalTx := types.NewTx(rawTx)
return t.signerFn(ctx, t.senderAddress, finalTx)
} }
...@@ -29,9 +29,17 @@ func main() { ...@@ -29,9 +29,17 @@ func main() {
app.Description = "Service for generating and submitting L2 tx batches " + app.Description = "Service for generating and submitting L2 tx batches " +
"to L1" "to L1"
app.Action = batcher.Main(Version) app.Action = curryMain(Version)
err := app.Run(os.Args) err := app.Run(os.Args)
if err != nil { if err != nil {
log.Crit("Application failed", "message", err) log.Crit("Application failed", "message", err)
} }
} }
// curryMain transforms the batcher.Main function into an app.Action
// This is done to capture the Version of the batcher.
func curryMain(version string) func(ctx *cli.Context) error {
return func(ctx *cli.Context) error {
return batcher.Main(version, ctx)
}
}
...@@ -3,7 +3,6 @@ module github.com/ethereum-optimism/optimism/op-batcher ...@@ -3,7 +3,6 @@ module github.com/ethereum-optimism/optimism/op-batcher
go 1.18 go 1.18
require ( require (
github.com/ethereum-optimism/go-ethereum-hdwallet v0.1.3
github.com/ethereum-optimism/optimism/op-node v0.10.11 github.com/ethereum-optimism/optimism/op-node v0.10.11
github.com/ethereum-optimism/optimism/op-service v0.10.11 github.com/ethereum-optimism/optimism/op-service v0.10.11
github.com/ethereum-optimism/optimism/op-signer v0.1.0 github.com/ethereum-optimism/optimism/op-signer v0.1.0
...@@ -15,10 +14,7 @@ require ( ...@@ -15,10 +14,7 @@ require (
github.com/VictoriaMetrics/fastcache v1.10.0 // indirect github.com/VictoriaMetrics/fastcache v1.10.0 // indirect
github.com/benbjohnson/clock v1.3.0 // indirect github.com/benbjohnson/clock v1.3.0 // indirect
github.com/beorn7/perks v1.0.1 // indirect github.com/beorn7/perks v1.0.1 // indirect
github.com/btcsuite/btcd v0.23.3 // indirect
github.com/btcsuite/btcd/btcec/v2 v2.2.0 // indirect github.com/btcsuite/btcd/btcec/v2 v2.2.0 // indirect
github.com/btcsuite/btcd/btcutil v1.1.0 // indirect
github.com/btcsuite/btcd/chaincfg/chainhash v1.0.1 // indirect
github.com/cespare/xxhash/v2 v2.1.2 // indirect github.com/cespare/xxhash/v2 v2.1.2 // indirect
github.com/cpuguy83/go-md2man/v2 v2.0.2 // indirect github.com/cpuguy83/go-md2man/v2 v2.0.2 // indirect
github.com/deckarep/golang-set v1.8.0 // indirect github.com/deckarep/golang-set v1.8.0 // indirect
...@@ -84,7 +80,6 @@ require ( ...@@ -84,7 +80,6 @@ require (
github.com/syndtr/goleveldb v1.0.1-0.20220614013038-64ee5596c38a // indirect github.com/syndtr/goleveldb v1.0.1-0.20220614013038-64ee5596c38a // indirect
github.com/tklauser/go-sysconf v0.3.10 // indirect github.com/tklauser/go-sysconf v0.3.10 // indirect
github.com/tklauser/numcpus v0.5.0 // indirect github.com/tklauser/numcpus v0.5.0 // indirect
github.com/tyler-smith/go-bip39 v1.1.0 // indirect
github.com/urfave/cli/v2 v2.17.2-0.20221006022127-8f469abc00aa // indirect github.com/urfave/cli/v2 v2.17.2-0.20221006022127-8f469abc00aa // indirect
github.com/xrash/smetrics v0.0.0-20201216005158-039620a65673 // indirect github.com/xrash/smetrics v0.0.0-20201216005158-039620a65673 // indirect
github.com/yusufpapurcu/wmi v1.2.2 // indirect github.com/yusufpapurcu/wmi v1.2.2 // indirect
......
This diff is collapsed.
...@@ -319,7 +319,7 @@ func TestMigration(t *testing.T) { ...@@ -319,7 +319,7 @@ func TestMigration(t *testing.T) {
require.NoError(t, rollupNode.Close()) require.NoError(t, rollupNode.Close())
}) })
batcher, err := bss.NewBatchSubmitter(bss.Config{ batcher, err := bss.NewBatchSubmitterFromCLIConfig(bss.CLIConfig{
L1EthRpc: forkedL1URL, L1EthRpc: forkedL1URL,
L2EthRpc: gethNode.WSEndpoint(), L2EthRpc: gethNode.WSEndpoint(),
RollupRpc: rollupNode.HTTPEndpoint(), RollupRpc: rollupNode.HTTPEndpoint(),
......
...@@ -522,7 +522,7 @@ func (cfg SystemConfig) Start() (*System, error) { ...@@ -522,7 +522,7 @@ func (cfg SystemConfig) Start() (*System, error) {
} }
// Batch Submitter // Batch Submitter
sys.BatchSubmitter, err = bss.NewBatchSubmitter(bss.Config{ sys.BatchSubmitter, err = bss.NewBatchSubmitterFromCLIConfig(bss.CLIConfig{
L1EthRpc: sys.Nodes["l1"].WSEndpoint(), L1EthRpc: sys.Nodes["l1"].WSEndpoint(),
L2EthRpc: sys.Nodes["sequencer"].WSEndpoint(), L2EthRpc: sys.Nodes["sequencer"].WSEndpoint(),
RollupRpc: sys.RollupNodes["sequencer"].HTTPEndpoint(), RollupRpc: sys.RollupNodes["sequencer"].HTTPEndpoint(),
......
Markdown is supported
0% or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment