From 8f7f94d5d6682a4c3211b1fe8cc0693479b44d83 Mon Sep 17 00:00:00 2001 From: protolambda Date: Sat, 14 Oct 2023 15:43:07 +0200 Subject: [PATCH 001/279] op-batcher: service lifecycle cleanup --- op-batcher/batcher/batch_submitter.go | 108 +----- op-batcher/batcher/config.go | 45 +-- op-batcher/batcher/driver.go | 204 ++++------ op-batcher/batcher/service.go | 351 ++++++++++++++++++ op-batcher/cmd/main.go | 10 +- op-batcher/metrics/metrics.go | 22 +- op-batcher/metrics/noop.go | 12 +- op-batcher/rpc/api.go | 14 +- .../game/fault/trace/outputs/provider.go | 2 +- op-challenger/game/service.go | 2 +- op-challenger/metrics/metrics.go | 7 +- op-e2e/setup.go | 31 +- op-e2e/system_fpp_test.go | 6 +- op-e2e/system_test.go | 6 +- op-node/metrics/metrics.go | 15 +- op-proposer/metrics/metrics.go | 7 +- op-proposer/proposer/l2_output_submitter.go | 4 +- op-service/clock/loop.go | 62 ++++ op-service/clock/loop_test.go | 27 ++ op-service/dial/dial.go | 8 +- op-service/metrics/balance.go | 60 ++- op-service/metrics/registry.go | 4 + op-service/metrics/rpc_metrics.go | 14 + op-service/sources/rollupclient.go | 4 + 24 files changed, 665 insertions(+), 360 deletions(-) create mode 100644 op-batcher/batcher/service.go create mode 100644 op-service/clock/loop.go create mode 100644 op-service/clock/loop_test.go diff --git a/op-batcher/batcher/batch_submitter.go b/op-batcher/batcher/batch_submitter.go index c2c4093972c1..8590d84b6a3d 100644 --- a/op-batcher/batcher/batch_submitter.go +++ b/op-batcher/batcher/batch_submitter.go @@ -3,110 +3,32 @@ package batcher import ( "context" "fmt" - _ "net/http/pprof" "github.com/urfave/cli/v2" "github.com/ethereum-optimism/optimism/op-batcher/flags" - "github.com/ethereum-optimism/optimism/op-batcher/metrics" - "github.com/ethereum-optimism/optimism/op-batcher/rpc" opservice "github.com/ethereum-optimism/optimism/op-service" + "github.com/ethereum-optimism/optimism/op-service/cliapp" oplog "github.com/ethereum-optimism/optimism/op-service/log" - "github.com/ethereum-optimism/optimism/op-service/opio" - oppprof "github.com/ethereum-optimism/optimism/op-service/pprof" - oprpc "github.com/ethereum-optimism/optimism/op-service/rpc" ) -// Main is the entrypoint into the Batch Submitter. This method returns a -// 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. -// GitVersion, to be captured and used once the function is executed. -func Main(version string, cliCtx *cli.Context) error { - if err := flags.CheckRequired(cliCtx); err != nil { - return err - } - cfg := NewConfig(cliCtx) - if err := cfg.Check(); err != nil { - return fmt.Errorf("invalid CLI flags: %w", err) - } - - l := oplog.NewLogger(oplog.AppOut(cliCtx), cfg.LogConfig) - oplog.SetGlobalLogHandler(l.GetHandler()) - opservice.ValidateEnvVars(flags.EnvVarPrefix, flags.Flags, l) - procName := "default" - m := metrics.NewMetrics(procName) - l.Info("Initializing Batch Submitter") - - batchSubmitter, err := NewBatchSubmitterFromCLIConfig(cfg, l, m) - if err != nil { - l.Error("Unable to create Batch Submitter", "error", err) - return err - } - - if !cfg.Stopped { - if err := batchSubmitter.Start(); err != nil { - l.Error("Unable to start Batch Submitter", "error", err) - return err - } - } - - defer batchSubmitter.StopIfRunning(context.Background()) - - pprofConfig := cfg.PprofConfig - if pprofConfig.Enabled { - l.Debug("starting pprof", "addr", pprofConfig.ListenAddr, "port", pprofConfig.ListenPort) - pprofSrv, err := oppprof.StartServer(pprofConfig.ListenAddr, pprofConfig.ListenPort) - if err != nil { - l.Error("failed to start pprof server", "err", err) - return err +// Main is the entrypoint into the Batch Submitter. +// This method returns a cliapp.LifecycleAction, to create an op-service CLI-lifecycle-managed batch-submitter with. +func Main(version string) cliapp.LifecycleAction { + return func(cliCtx *cli.Context, closeApp context.CancelCauseFunc) (cliapp.Lifecycle, error) { + if err := flags.CheckRequired(cliCtx); err != nil { + return nil, err } - l.Info("started pprof server", "addr", pprofSrv.Addr()) - defer func() { - if err := pprofSrv.Stop(context.Background()); err != nil { - l.Error("failed to stop pprof server", "err", err) - } - }() - } - - metricsCfg := cfg.MetricsConfig - if metricsCfg.Enabled { - l.Debug("starting metrics server", "addr", metricsCfg.ListenAddr, "port", metricsCfg.ListenPort) - metricsSrv, err := m.Start(metricsCfg.ListenAddr, metricsCfg.ListenPort) - if err != nil { - return fmt.Errorf("failed to start metrics server: %w", err) + cfg := NewConfig(cliCtx) + if err := cfg.Check(); err != nil { + return nil, fmt.Errorf("invalid CLI flags: %w", err) } - l.Info("started metrics server", "addr", metricsSrv.Addr()) - defer func() { - if err := metricsSrv.Stop(context.Background()); err != nil { - l.Error("failed to stop pprof server", "err", err) - } - }() - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - m.StartBalanceMetrics(ctx, l, batchSubmitter.L1Client, batchSubmitter.TxManager.From()) - } - - server := oprpc.NewServer( - cfg.RPCFlag.ListenAddr, - cfg.RPCFlag.ListenPort, - version, - oprpc.WithLogger(l), - ) - if cfg.RPCFlag.EnableAdmin { - adminAPI := rpc.NewAdminAPI(batchSubmitter, &m.RPCMetrics, l) - server.AddAPI(rpc.GetAdminAPI(adminAPI)) - l.Info("Admin RPC enabled") - } - if err := server.Start(); err != nil { - return fmt.Errorf("error starting RPC server: %w", err) - } - m.RecordInfo(version) - m.RecordUp() + l := oplog.NewLogger(oplog.AppOut(cliCtx), cfg.LogConfig) + oplog.SetGlobalLogHandler(l.GetHandler()) + opservice.ValidateEnvVars(flags.EnvVarPrefix, flags.Flags, l) - opio.BlockOnInterrupts() - if err := server.Stop(); err != nil { - l.Error("Error shutting down http server: %w", err) + l.Info("Initializing Batch Submitter") + return BatcherServiceFromCLIConfig(cliCtx.Context, version, cfg, l) } - return nil } diff --git a/op-batcher/batcher/config.go b/op-batcher/batcher/config.go index bad964eb9986..a09b7f832e53 100644 --- a/op-batcher/batcher/config.go +++ b/op-batcher/batcher/config.go @@ -3,52 +3,17 @@ package batcher import ( "time" - "github.com/ethereum/go-ethereum/ethclient" - "github.com/ethereum/go-ethereum/log" "github.com/urfave/cli/v2" "github.com/ethereum-optimism/optimism/op-batcher/compressor" "github.com/ethereum-optimism/optimism/op-batcher/flags" - "github.com/ethereum-optimism/optimism/op-batcher/metrics" - "github.com/ethereum-optimism/optimism/op-node/rollup" oplog "github.com/ethereum-optimism/optimism/op-service/log" opmetrics "github.com/ethereum-optimism/optimism/op-service/metrics" oppprof "github.com/ethereum-optimism/optimism/op-service/pprof" oprpc "github.com/ethereum-optimism/optimism/op-service/rpc" - "github.com/ethereum-optimism/optimism/op-service/sources" "github.com/ethereum-optimism/optimism/op-service/txmgr" ) -type Config struct { - log log.Logger - metr metrics.Metricer - L1Client *ethclient.Client - L2Client *ethclient.Client - RollupNode *sources.RollupClient - TxManager txmgr.TxManager - - NetworkTimeout time.Duration - PollInterval time.Duration - MaxPendingTransactions uint64 - - // RollupConfig is queried at startup - Rollup *rollup.Config - - // Channel builder parameters - Channel ChannelConfig -} - -// Check ensures that the [Config] is valid. -func (c *Config) Check() error { - if err := c.Rollup.Check(); err != nil { - return err - } - if err := c.Channel.Check(); err != nil { - return err - } - return nil -} - type CLIConfig struct { // L1EthRpc is the HTTP provider URL for L1. L1EthRpc string @@ -92,7 +57,7 @@ type CLIConfig struct { MetricsConfig opmetrics.CLIConfig PprofConfig oppprof.CLIConfig CompressorConfig compressor.CLIConfig - RPCFlag oprpc.CLIConfig + RPC oprpc.CLIConfig } func (c CLIConfig) Check() error { @@ -107,15 +72,15 @@ func (c CLIConfig) Check() error { if err := c.TxMgrConfig.Check(); err != nil { return err } - if err := c.RPCFlag.Check(); err != nil { + if err := c.RPC.Check(); err != nil { return err } return nil } // NewConfig parses the Config from the provided flags or environment variables. -func NewConfig(ctx *cli.Context) CLIConfig { - return CLIConfig{ +func NewConfig(ctx *cli.Context) *CLIConfig { + return &CLIConfig{ /* Required Flags */ L1EthRpc: ctx.String(flags.L1EthRpcFlag.Name), L2EthRpc: ctx.String(flags.L2EthRpcFlag.Name), @@ -133,6 +98,6 @@ func NewConfig(ctx *cli.Context) CLIConfig { MetricsConfig: opmetrics.ReadCLIConfig(ctx), PprofConfig: oppprof.ReadCLIConfig(ctx), CompressorConfig: compressor.ReadCLIConfig(ctx), - RPCFlag: oprpc.ReadCLIConfig(ctx), + RPC: oprpc.ReadCLIConfig(ctx), } } diff --git a/op-batcher/batcher/driver.go b/op-batcher/batcher/driver.go index bb53d5adfb9e..bcffef6a9edf 100644 --- a/op-batcher/batcher/driver.go +++ b/op-batcher/batcher/driver.go @@ -10,23 +10,50 @@ import ( "sync" "time" + "github.com/ethereum/go-ethereum/core" + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/log" + "github.com/ethereum-optimism/optimism/op-batcher/metrics" + "github.com/ethereum-optimism/optimism/op-node/rollup" "github.com/ethereum-optimism/optimism/op-node/rollup/derive" - "github.com/ethereum-optimism/optimism/op-service/dial" "github.com/ethereum-optimism/optimism/op-service/eth" "github.com/ethereum-optimism/optimism/op-service/txmgr" - "github.com/ethereum/go-ethereum/core" - "github.com/ethereum/go-ethereum/core/types" - "github.com/ethereum/go-ethereum/log" ) +var ErrBatcherNotRunning = errors.New("batcher is not running") + +type L1Client interface { + HeaderByNumber(ctx context.Context, number *big.Int) (*types.Header, error) +} + +type L2Client interface { + BlockByNumber(ctx context.Context, number *big.Int) (*types.Block, error) +} + +type RollupClient interface { + SyncStatus(ctx context.Context) (*eth.SyncStatus, error) +} + +// DriverSetup is the collection of input/output interfaces and configuration that the driver operates on. +type DriverSetup struct { + Log log.Logger + Metr metrics.Metricer + RollupCfg *rollup.Config + Cfg BatcherConfig + Txmgr txmgr.TxManager + L1Client L1Client + L2Client L2Client + RollupClient RollupClient + Channel ChannelConfig +} + // BatchSubmitter encapsulates a service responsible for submitting L2 tx // batches to L1 for availability. type BatchSubmitter struct { - Config // directly embed the config + sources + DriverSetup - txMgr txmgr.TxManager - wg sync.WaitGroup + wg sync.WaitGroup shutdownCtx context.Context cancelShutdownCtx context.CancelFunc @@ -43,88 +70,16 @@ type BatchSubmitter struct { state *channelManager } -// NewBatchSubmitterFromCLIConfig initializes the BatchSubmitter, gathering any resources -// that will be needed during operation. -func NewBatchSubmitterFromCLIConfig(cfg CLIConfig, l log.Logger, m metrics.Metricer) (*BatchSubmitter, error) { - ctx := context.Background() - - // Connect to L1 and L2 providers. Perform these last since they are the - // most expensive. - l1Client, err := dial.DialEthClientWithTimeout(dial.DefaultDialTimeout, l, cfg.L1EthRpc) - if err != nil { - return nil, err - } - - l2Client, err := dial.DialEthClientWithTimeout(dial.DefaultDialTimeout, l, cfg.L2EthRpc) - if err != nil { - return nil, err - } - - rollupClient, err := dial.DialRollupClientWithTimeout(dial.DefaultDialTimeout, l, cfg.RollupRpc) - if err != nil { - return nil, err - } - - rcfg, err := rollupClient.RollupConfig(ctx) - if err != nil { - return nil, fmt.Errorf("querying rollup config: %w", err) - } - - txManager, err := txmgr.NewSimpleTxManager("batcher", l, m, cfg.TxMgrConfig) - if err != nil { - return nil, err - } - - batcherCfg := Config{ - L1Client: l1Client, - L2Client: l2Client, - RollupNode: rollupClient, - PollInterval: cfg.PollInterval, - MaxPendingTransactions: cfg.MaxPendingTransactions, - NetworkTimeout: cfg.TxMgrConfig.NetworkTimeout, - TxManager: txManager, - Rollup: rcfg, - Channel: ChannelConfig{ - SeqWindowSize: rcfg.SeqWindowSize, - ChannelTimeout: rcfg.ChannelTimeout, - MaxChannelDuration: cfg.MaxChannelDuration, - SubSafetyMargin: cfg.SubSafetyMargin, - MaxFrameSize: cfg.MaxL1TxSize - 1, // subtract 1 byte for version - CompressorConfig: cfg.CompressorConfig.Config(), - }, - } - - // Validate the batcher config - if err := batcherCfg.Check(); err != nil { - return nil, err - } - - return NewBatchSubmitter(ctx, batcherCfg, l, m) -} - -// NewBatchSubmitter initializes the BatchSubmitter, gathering any resources -// that will be needed during operation. -func NewBatchSubmitter(ctx context.Context, cfg Config, l log.Logger, m metrics.Metricer) (*BatchSubmitter, error) { - balance, err := cfg.L1Client.BalanceAt(ctx, cfg.TxManager.From(), nil) - if err != nil { - return nil, err - } - - cfg.log = l - cfg.log.Info("creating batch submitter", "submitter_addr", cfg.TxManager.From(), "submitter_bal", balance) - - cfg.metr = m - +// NewBatchSubmitter initializes the BatchSubmitter driver from a preconfigured DriverSetup +func NewBatchSubmitter(setup DriverSetup) *BatchSubmitter { return &BatchSubmitter{ - Config: cfg, - txMgr: cfg.TxManager, - state: NewChannelManager(l, m, cfg.Channel), - }, nil - + DriverSetup: setup, + state: NewChannelManager(setup.Log, setup.Metr, setup.Channel), + } } -func (l *BatchSubmitter) Start() error { - l.log.Info("Starting Batch Submitter") +func (l *BatchSubmitter) StartBatchSubmitting() error { + l.Log.Info("Starting Batch Submitter") l.mutex.Lock() defer l.mutex.Unlock() @@ -142,23 +97,27 @@ func (l *BatchSubmitter) Start() error { l.wg.Add(1) go l.loop() - l.log.Info("Batch Submitter started") - + l.Log.Info("Batch Submitter started") return nil } -func (l *BatchSubmitter) StopIfRunning(ctx context.Context) { - _ = l.Stop(ctx) +func (l *BatchSubmitter) StopBatchSubmittingIfRunning(ctx context.Context) error { + err := l.StopBatchSubmitting(ctx) + if errors.Is(err, ErrBatcherNotRunning) { + return nil + } + return err } -func (l *BatchSubmitter) Stop(ctx context.Context) error { - l.log.Info("Stopping Batch Submitter") +// StopBatchSubmitting stops the batch-submitter loop, and force-kills if the provided ctx is done. +func (l *BatchSubmitter) StopBatchSubmitting(ctx context.Context) error { + l.Log.Info("Stopping Batch Submitter") l.mutex.Lock() defer l.mutex.Unlock() if !l.running { - return errors.New("batcher is not running") + return ErrBatcherNotRunning } l.running = false @@ -175,8 +134,7 @@ func (l *BatchSubmitter) Stop(ctx context.Context) error { l.wg.Wait() l.cancelKillCtx() - l.log.Info("Batch Submitter stopped") - + l.Log.Info("Batch Submitter stopped") return nil } @@ -191,7 +149,7 @@ func (l *BatchSubmitter) Stop(ctx context.Context) error { func (l *BatchSubmitter) loadBlocksIntoState(ctx context.Context) error { start, end, err := l.calculateL2BlockRangeToStore(ctx) if err != nil { - l.log.Warn("Error calculating L2 block range", "err", err) + l.Log.Warn("Error calculating L2 block range", "err", err) return err } else if start.Number >= end.Number { return errors.New("start number is >= end number") @@ -202,30 +160,30 @@ func (l *BatchSubmitter) loadBlocksIntoState(ctx context.Context) error { for i := start.Number + 1; i < end.Number+1; i++ { block, err := l.loadBlockIntoState(ctx, i) if errors.Is(err, ErrReorg) { - l.log.Warn("Found L2 reorg", "block_number", i) + l.Log.Warn("Found L2 reorg", "block_number", i) l.lastStoredBlock = eth.BlockID{} return err } else if err != nil { - l.log.Warn("failed to load block into state", "err", err) + l.Log.Warn("failed to load block into state", "err", err) return err } l.lastStoredBlock = eth.ToBlockID(block) latestBlock = block } - l2ref, err := derive.L2BlockToBlockRef(latestBlock, &l.Rollup.Genesis) + l2ref, err := derive.L2BlockToBlockRef(latestBlock, &l.RollupCfg.Genesis) if err != nil { - l.log.Warn("Invalid L2 block loaded into state", "err", err) + l.Log.Warn("Invalid L2 block loaded into state", "err", err) return err } - l.metr.RecordL2BlocksLoaded(l2ref) + l.Metr.RecordL2BlocksLoaded(l2ref) return nil } // loadBlockIntoState fetches & stores a single block into `state`. It returns the block it loaded. func (l *BatchSubmitter) loadBlockIntoState(ctx context.Context, blockNumber uint64) (*types.Block, error) { - ctx, cancel := context.WithTimeout(ctx, l.NetworkTimeout) + ctx, cancel := context.WithTimeout(ctx, l.Cfg.NetworkTimeout) defer cancel() block, err := l.L2Client.BlockByNumber(ctx, new(big.Int).SetUint64(blockNumber)) if err != nil { @@ -236,16 +194,16 @@ func (l *BatchSubmitter) loadBlockIntoState(ctx context.Context, blockNumber uin return nil, fmt.Errorf("adding L2 block to state: %w", err) } - l.log.Info("added L2 block to local state", "block", eth.ToBlockID(block), "tx_count", len(block.Transactions()), "time", block.Time()) + l.Log.Info("added L2 block to local state", "block", eth.ToBlockID(block), "tx_count", len(block.Transactions()), "time", block.Time()) return block, nil } // calculateL2BlockRangeToStore determines the range (start,end] that should be loaded into the local state. // It also takes care of initializing some local state (i.e. will modify l.lastStoredBlock in certain conditions) func (l *BatchSubmitter) calculateL2BlockRangeToStore(ctx context.Context) (eth.BlockID, eth.BlockID, error) { - ctx, cancel := context.WithTimeout(ctx, l.NetworkTimeout) + ctx, cancel := context.WithTimeout(ctx, l.Cfg.NetworkTimeout) defer cancel() - syncStatus, err := l.RollupNode.SyncStatus(ctx) + syncStatus, err := l.RollupClient.SyncStatus(ctx) // Ensure that we have the sync status if err != nil { return eth.BlockID{}, eth.BlockID{}, fmt.Errorf("failed to get sync status: %w", err) @@ -257,10 +215,10 @@ func (l *BatchSubmitter) calculateL2BlockRangeToStore(ctx context.Context) (eth. // Check last stored to see if it needs to be set on startup OR set if is lagged behind. // It lagging implies that the op-node processed some batches that were submitted prior to the current instance of the batcher being alive. if l.lastStoredBlock == (eth.BlockID{}) { - l.log.Info("Starting batch-submitter work at safe-head", "safe", syncStatus.SafeL2) + l.Log.Info("Starting batch-submitter work at safe-head", "safe", syncStatus.SafeL2) l.lastStoredBlock = syncStatus.SafeL2.ID() } else if l.lastStoredBlock.Number < syncStatus.SafeL2.Number { - l.log.Warn("last submitted block lagged behind L2 safe head: batch submission will continue from the safe head now", "last", l.lastStoredBlock, "safe", syncStatus.SafeL2) + l.Log.Warn("last submitted block lagged behind L2 safe head: batch submission will continue from the safe head now", "last", l.lastStoredBlock, "safe", syncStatus.SafeL2) l.lastStoredBlock = syncStatus.SafeL2.ID() } @@ -286,11 +244,11 @@ func (l *BatchSubmitter) calculateL2BlockRangeToStore(ctx context.Context) (eth. func (l *BatchSubmitter) loop() { defer l.wg.Done() - ticker := time.NewTicker(l.PollInterval) + ticker := time.NewTicker(l.Cfg.PollInterval) defer ticker.Stop() receiptsCh := make(chan txmgr.TxReceipt[txData]) - queue := txmgr.NewQueue[txData](l.killCtx, l.txMgr, l.MaxPendingTransactions) + queue := txmgr.NewQueue[txData](l.killCtx, l.Txmgr, l.Cfg.MaxPendingTransactions) for { select { @@ -298,7 +256,7 @@ func (l *BatchSubmitter) loop() { if err := l.loadBlocksIntoState(l.shutdownCtx); errors.Is(err, ErrReorg) { err := l.state.Close() if err != nil { - l.log.Error("error closing the channel manager to handle a L2 reorg", "err", err) + l.Log.Error("error closing the channel manager to handle a L2 reorg", "err", err) } l.publishStateToL1(queue, receiptsCh, true) l.state.Clear() @@ -310,7 +268,7 @@ func (l *BatchSubmitter) loop() { case <-l.shutdownCtx.Done(): err := l.state.Close() if err != nil { - l.log.Error("error closing the channel manager", "err", err) + l.Log.Error("error closing the channel manager", "err", err) } l.publishStateToL1(queue, receiptsCh, true) return @@ -335,7 +293,7 @@ func (l *BatchSubmitter) publishStateToL1(queue *txmgr.Queue[txData], receiptsCh err := l.publishTxToL1(l.killCtx, queue, receiptsCh) if err != nil { if drain && err != io.EOF { - l.log.Error("error sending tx while draining state", "err", err) + l.Log.Error("error sending tx while draining state", "err", err) } return } @@ -357,7 +315,7 @@ func (l *BatchSubmitter) publishTxToL1(ctx context.Context, queue *txmgr.Queue[t // send all available transactions l1tip, err := l.l1Tip(ctx) if err != nil { - l.log.Error("Failed to query L1 tip", "error", err) + l.Log.Error("Failed to query L1 tip", "error", err) return err } l.recordL1Tip(l1tip) @@ -365,10 +323,10 @@ func (l *BatchSubmitter) publishTxToL1(ctx context.Context, queue *txmgr.Queue[t // Collect next transaction data txdata, err := l.state.TxData(l1tip.ID()) if err == io.EOF { - l.log.Trace("no transaction data available") + l.Log.Trace("no transaction data available") return err } else if err != nil { - l.log.Error("unable to get tx data", "err", err) + l.Log.Error("unable to get tx data", "err", err) return err } @@ -384,12 +342,12 @@ func (l *BatchSubmitter) sendTransaction(txdata txData, queue *txmgr.Queue[txDat data := txdata.Bytes() intrinsicGas, err := core.IntrinsicGas(data, nil, false, true, true, false) if err != nil { - l.log.Error("Failed to calculate intrinsic gas", "error", err) + l.Log.Error("Failed to calculate intrinsic gas", "error", err) return } candidate := txmgr.TxCandidate{ - To: &l.Rollup.BatchInboxAddress, + To: &l.RollupCfg.BatchInboxAddress, TxData: data, GasLimit: intrinsicGas, } @@ -399,10 +357,10 @@ func (l *BatchSubmitter) sendTransaction(txdata txData, queue *txmgr.Queue[txDat func (l *BatchSubmitter) handleReceipt(r txmgr.TxReceipt[txData]) { // Record TX Status if r.Err != nil { - l.log.Warn("unable to publish tx", "err", r.Err, "data_size", r.ID.Len()) + l.Log.Warn("unable to publish tx", "err", r.Err, "data_size", r.ID.Len()) l.recordFailedTx(r.ID.ID(), r.Err) } else { - l.log.Info("tx successfully published", "tx_hash", r.Receipt.TxHash, "data_size", r.ID.Len()) + l.Log.Info("tx successfully published", "tx_hash", r.Receipt.TxHash, "data_size", r.ID.Len()) l.recordConfirmedTx(r.ID.ID(), r.Receipt) } } @@ -412,16 +370,16 @@ func (l *BatchSubmitter) recordL1Tip(l1tip eth.L1BlockRef) { return } l.lastL1Tip = l1tip - l.metr.RecordLatestL1Block(l1tip) + l.Metr.RecordLatestL1Block(l1tip) } func (l *BatchSubmitter) recordFailedTx(id txID, err error) { - l.log.Warn("Failed to send transaction", "err", err) + l.Log.Warn("Failed to send transaction", "err", err) l.state.TxFailed(id) } func (l *BatchSubmitter) recordConfirmedTx(id txID, receipt *types.Receipt) { - l.log.Info("Transaction confirmed", "tx_hash", receipt.TxHash, "status", receipt.Status, "block_hash", receipt.BlockHash, "block_number", receipt.BlockNumber) + l.Log.Info("Transaction confirmed", "tx_hash", receipt.TxHash, "status", receipt.Status, "block_hash", receipt.BlockHash, "block_number", receipt.BlockNumber) l1block := eth.BlockID{Number: receipt.BlockNumber.Uint64(), Hash: receipt.BlockHash} l.state.TxConfirmed(id, l1block) } @@ -429,7 +387,7 @@ func (l *BatchSubmitter) recordConfirmedTx(id txID, receipt *types.Receipt) { // l1Tip gets the current L1 tip as a L1BlockRef. The passed context is assumed // to be a lifetime context, so it is internally wrapped with a network timeout. func (l *BatchSubmitter) l1Tip(ctx context.Context) (eth.L1BlockRef, error) { - tctx, cancel := context.WithTimeout(ctx, l.NetworkTimeout) + tctx, cancel := context.WithTimeout(ctx, l.Cfg.NetworkTimeout) defer cancel() head, err := l.L1Client.HeaderByNumber(tctx, nil) if err != nil { diff --git a/op-batcher/batcher/service.go b/op-batcher/batcher/service.go new file mode 100644 index 000000000000..2a091119454a --- /dev/null +++ b/op-batcher/batcher/service.go @@ -0,0 +1,351 @@ +package batcher + +import ( + "context" + "errors" + "fmt" + "io" + "net" + _ "net/http/pprof" + "strconv" + "sync/atomic" + "time" + + "github.com/ethereum/go-ethereum/ethclient" + "github.com/ethereum/go-ethereum/log" + + "github.com/ethereum-optimism/optimism/op-batcher/metrics" + "github.com/ethereum-optimism/optimism/op-batcher/rpc" + "github.com/ethereum-optimism/optimism/op-node/rollup" + "github.com/ethereum-optimism/optimism/op-service/cliapp" + "github.com/ethereum-optimism/optimism/op-service/dial" + "github.com/ethereum-optimism/optimism/op-service/httputil" + opmetrics "github.com/ethereum-optimism/optimism/op-service/metrics" + oppprof "github.com/ethereum-optimism/optimism/op-service/pprof" + oprpc "github.com/ethereum-optimism/optimism/op-service/rpc" + "github.com/ethereum-optimism/optimism/op-service/sources" + "github.com/ethereum-optimism/optimism/op-service/txmgr" +) + +type BatcherConfig struct { + NetworkTimeout time.Duration + PollInterval time.Duration + MaxPendingTransactions uint64 +} + +// BatcherService represents a full batch-submitter instance and its resources, +// and conforms to the op-service CLI Lifecycle interface. +type BatcherService struct { + Log log.Logger + Metrics metrics.Metricer + L1Client *ethclient.Client + L2Client *ethclient.Client + RollupNode *sources.RollupClient + TxManager txmgr.TxManager + + BatcherConfig + + RollupConfig *rollup.Config + + // Channel builder parameters + Channel ChannelConfig + + driver *BatchSubmitter + + Version string + + pprofSrv *httputil.HTTPServer + metricsSrv *httputil.HTTPServer + rpcServer *oprpc.Server + + balanceMetricer io.Closer + + stopped atomic.Bool + + NotSubmittingOnStart bool +} + +// BatcherServiceFromCLIConfig creates a new BatcherService from a CLIConfig. +// The service components are fully started, except for the driver, +// which will not be submitting batches (if it was configured to) until the Start part of the lifecycle. +func BatcherServiceFromCLIConfig(ctx context.Context, version string, cfg *CLIConfig, log log.Logger) (*BatcherService, error) { + var bs BatcherService + if err := bs.initFromCLIConfig(ctx, version, cfg, log); err != nil { + return nil, errors.Join(err, bs.Stop(ctx)) // try to clean up our failed initialization attempt + } + return &bs, nil +} + +func (bs *BatcherService) initFromCLIConfig(ctx context.Context, version string, cfg *CLIConfig, log log.Logger) error { + bs.Version = version + bs.Log = log + bs.NotSubmittingOnStart = cfg.Stopped + + bs.initMetrics(cfg) + + bs.PollInterval = cfg.PollInterval + bs.MaxPendingTransactions = cfg.MaxPendingTransactions + bs.NetworkTimeout = cfg.TxMgrConfig.NetworkTimeout + + if err := bs.initRPCClients(ctx, cfg); err != nil { + return err + } + if err := bs.initRollupCfg(ctx); err != nil { + return fmt.Errorf("failed to load rollup config: %w", err) + } + if err := bs.initChannelConfig(cfg); err != nil { + return fmt.Errorf("failed to init channel config: %w", err) + } + if err := bs.initTxManager(cfg); err != nil { + return fmt.Errorf("failed to init Tx manager: %w", err) + } + bs.initBalanceMonitor(cfg) + if err := bs.initMetricsServer(cfg); err != nil { + return fmt.Errorf("failed to start metrics server: %w", err) + } + if err := bs.initPProf(cfg); err != nil { + return fmt.Errorf("failed to start pprof server: %w", err) + } + bs.initDriver() + if err := bs.initRPCServer(cfg); err != nil { + return fmt.Errorf("failed to start RPC server: %w", err) + } + // Validate the setup + if err := bs.Check(); err != nil { + return fmt.Errorf("failed post-initialization check: %w", err) + } + + bs.Metrics.RecordInfo(bs.Version) + bs.Metrics.RecordUp() + return nil +} + +func (bs *BatcherService) initRPCClients(ctx context.Context, cfg *CLIConfig) error { + l1Client, err := dial.DialEthClientWithTimeout(ctx, dial.DefaultDialTimeout, bs.Log, cfg.L1EthRpc) + if err != nil { + return fmt.Errorf("failed to dial L1 RPC: %w", err) + } + bs.L1Client = l1Client + + l2Client, err := dial.DialEthClientWithTimeout(ctx, dial.DefaultDialTimeout, bs.Log, cfg.L2EthRpc) + if err != nil { + return fmt.Errorf("failed to dial L2 engine RPC: %w", err) + } + bs.L2Client = l2Client + + rollupClient, err := dial.DialRollupClientWithTimeout(ctx, dial.DefaultDialTimeout, bs.Log, cfg.RollupRpc) + if err != nil { + return fmt.Errorf("failed to dial L2 rollup-client RPC: %w", err) + } + bs.RollupNode = rollupClient + return nil +} + +func (bs *BatcherService) initMetrics(cfg *CLIConfig) { + if cfg.MetricsConfig.Enabled { + procName := "default" + bs.Metrics = metrics.NewMetrics(procName) + } else { + bs.Metrics = metrics.NoopMetrics + } +} + +// initBalanceMonitor depends on Metrics, L1Client and TxManager to start background-monitoring of the batcher balance. +func (bs *BatcherService) initBalanceMonitor(cfg *CLIConfig) { + if cfg.MetricsConfig.Enabled { + bs.balanceMetricer = bs.Metrics.StartBalanceMetrics(bs.Log, bs.L1Client, bs.TxManager.From()) + } +} + +func (bs *BatcherService) initRollupCfg(ctx context.Context) error { + rollupCfg, err := bs.RollupNode.RollupConfig(ctx) + if err != nil { + return fmt.Errorf("failed to retrieve rollup config: %w", err) + } + bs.RollupConfig = rollupCfg + if err := bs.RollupConfig.Check(); err != nil { + return fmt.Errorf("invalid rollup config: %w", err) + } + return nil +} + +func (bs *BatcherService) initChannelConfig(cfg *CLIConfig) error { + bs.Channel = ChannelConfig{ + SeqWindowSize: bs.RollupConfig.SeqWindowSize, + ChannelTimeout: bs.RollupConfig.ChannelTimeout, + MaxChannelDuration: cfg.MaxChannelDuration, + SubSafetyMargin: cfg.SubSafetyMargin, + MaxFrameSize: cfg.MaxL1TxSize - 1, // subtract 1 byte for version + CompressorConfig: cfg.CompressorConfig.Config(), + } + if err := bs.Channel.Check(); err != nil { + return fmt.Errorf("invalid channel configuration: %w", err) + } + return nil +} + +func (bs *BatcherService) initTxManager(cfg *CLIConfig) error { + txManager, err := txmgr.NewSimpleTxManager("batcher", bs.Log, bs.Metrics, cfg.TxMgrConfig) + if err != nil { + return err + } + bs.TxManager = txManager + return nil +} + +func (bs *BatcherService) initPProf(cfg *CLIConfig) error { + if !cfg.PprofConfig.Enabled { + return nil + } + log.Debug("starting pprof server", "addr", net.JoinHostPort(cfg.PprofConfig.ListenAddr, strconv.Itoa(cfg.PprofConfig.ListenPort))) + srv, err := oppprof.StartServer(cfg.PprofConfig.ListenAddr, cfg.PprofConfig.ListenPort) + if err != nil { + return err + } + bs.pprofSrv = srv + log.Info("started pprof server", "addr", srv.Addr()) + return nil +} + +func (bs *BatcherService) initMetricsServer(cfg *CLIConfig) error { + if !cfg.MetricsConfig.Enabled { + bs.Log.Info("metrics disabled") + return nil + } + m, ok := bs.Metrics.(opmetrics.RegistryMetricer) + if !ok { + return fmt.Errorf("metrics were enabled, but metricer %T does not expose registry for metrics-server", bs.Metrics) + } + bs.Log.Debug("starting metrics server", "addr", cfg.MetricsConfig.ListenAddr, "port", cfg.MetricsConfig.ListenPort) + metricsSrv, err := opmetrics.StartServer(m.Registry(), cfg.MetricsConfig.ListenAddr, cfg.MetricsConfig.ListenPort) + if err != nil { + return fmt.Errorf("failed to start metrics server: %w", err) + } + bs.Log.Info("started metrics server", "addr", metricsSrv.Addr()) + bs.metricsSrv = metricsSrv + return nil +} + +func (bs *BatcherService) initDriver() { + bs.driver = NewBatchSubmitter(DriverSetup{ + Log: bs.Log, + Metr: bs.Metrics, + RollupCfg: bs.RollupConfig, + Cfg: bs.BatcherConfig, + Txmgr: bs.TxManager, + L1Client: bs.L1Client, + L2Client: bs.L2Client, + RollupClient: bs.RollupNode, + Channel: bs.Channel, + }) +} + +func (bs *BatcherService) initRPCServer(cfg *CLIConfig) error { + server := oprpc.NewServer( + cfg.RPC.ListenAddr, + cfg.RPC.ListenPort, + bs.Version, + oprpc.WithLogger(bs.Log), + ) + if cfg.RPC.EnableAdmin { + adminAPI := rpc.NewAdminAPI(bs.driver, bs.Metrics, bs.Log) + server.AddAPI(rpc.GetAdminAPI(adminAPI)) + bs.Log.Info("Admin RPC enabled") + } + bs.Log.Info("Starting JSON-RPC server") + if err := server.Start(); err != nil { + return fmt.Errorf("unable to start RPC server: %w", err) + } + bs.rpcServer = server + return nil +} + +// Check ensures that the [BatcherService] is valid +func (bs *BatcherService) Check() error { + + return nil +} + +// Start runs once upon start of the batcher lifecycle, +// and starts batch-submission work if the batcher is configured to start submit data on startup. +func (bs *BatcherService) Start(_ context.Context) error { + bs.driver.Log.Info("Starting batcher", "notSubmittingOnStart", bs.NotSubmittingOnStart) + + if !bs.NotSubmittingOnStart { + return bs.driver.StartBatchSubmitting() + } + return nil +} + +// Stopped returns if the service as a whole is stopped. +func (bs *BatcherService) Stopped() bool { + return bs.stopped.Load() +} + +// Kill is a convenience method to forcefully, non-gracefully, stop the BatcherService. +func (bs *BatcherService) Kill() error { + ctx, cancel := context.WithCancel(context.Background()) + cancel() + return bs.Stop(ctx) +} + +// Stop fully stops the batch-submitter and all its resources gracefully. After stopping, it cannot be restarted. +// See driver.StopBatchSubmitting to temporarily stop the batch submitter. +// If the provided ctx is cancelled, the stopping is forced, i.e. the batching work is killed non-gracefully. +func (bs *BatcherService) Stop(ctx context.Context) error { + if bs.stopped.Load() { + return errors.New("already stopped") + } + bs.Log.Info("Stopping batcher") + + var result error + if err := bs.driver.StopBatchSubmittingIfRunning(ctx); err != nil { + result = errors.Join(result, fmt.Errorf("failed to stop batch submitting: %w", err)) + } + + if bs.rpcServer != nil { + // TODO(7685): the op-service RPC server is not built on top of op-service httputil Server, and has poor shutdown + if err := bs.rpcServer.Stop(); err != nil { + result = errors.Join(result, fmt.Errorf("failed to stop RPC server: %w", err)) + } + } + if bs.pprofSrv != nil { + if err := bs.pprofSrv.Stop(ctx); err != nil { + result = errors.Join(result, fmt.Errorf("failed to stop PProf server: %w", err)) + } + } + if bs.balanceMetricer != nil { + if err := bs.balanceMetricer.Close(); err != nil { + result = errors.Join(result, fmt.Errorf("failed to close balance metricer: %w", err)) + } + } + if bs.metricsSrv != nil { + if err := bs.metricsSrv.Stop(ctx); err != nil { + result = errors.Join(result, fmt.Errorf("failed to stop metrics server: %w", err)) + } + } + + if bs.L1Client != nil { + bs.L1Client.Close() + } + if bs.L2Client != nil { + bs.L2Client.Close() + } + if bs.RollupNode != nil { + bs.RollupNode.Close() + } + + if result == nil { + bs.stopped.Store(true) + bs.driver.Log.Info("Batch Submitter stopped") + } + return result +} + +var _ cliapp.Lifecycle = (*BatcherService)(nil) + +// Driver returns the handler on the batch-submitter driver element, +// to start/stop/restart the batch-submission work, for use in testing. +func (bs *BatcherService) Driver() rpc.BatcherDriver { + return bs.driver +} diff --git a/op-batcher/cmd/main.go b/op-batcher/cmd/main.go index ff3579a37bf0..9bf930f282ea 100644 --- a/op-batcher/cmd/main.go +++ b/op-batcher/cmd/main.go @@ -30,7 +30,7 @@ func main() { app.Name = "op-batcher" app.Usage = "Batch Submitter Service" app.Description = "Service for generating and submitting L2 tx batches to L1" - app.Action = curryMain(Version) + app.Action = cliapp.LifecycleCmd(batcher.Main(Version)) app.Commands = []*cli.Command{ { Name: "doc", @@ -43,11 +43,3 @@ func main() { 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) - } -} diff --git a/op-batcher/metrics/metrics.go b/op-batcher/metrics/metrics.go index dd0d2bcc17a9..4ce45d1e2c1c 100644 --- a/op-batcher/metrics/metrics.go +++ b/op-batcher/metrics/metrics.go @@ -1,17 +1,17 @@ package metrics import ( - "context" + "io" + + "github.com/prometheus/client_golang/prometheus" "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/ethclient" "github.com/ethereum/go-ethereum/log" - "github.com/prometheus/client_golang/prometheus" "github.com/ethereum-optimism/optimism/op-node/rollup/derive" "github.com/ethereum-optimism/optimism/op-service/eth" - "github.com/ethereum-optimism/optimism/op-service/httputil" opmetrics "github.com/ethereum-optimism/optimism/op-service/metrics" txmetrics "github.com/ethereum-optimism/optimism/op-service/txmgr/metrics" ) @@ -28,6 +28,10 @@ type Metricer interface { // Record Tx metrics txmetrics.TxMetricer + opmetrics.RPCMetricer + + StartBalanceMetrics(l log.Logger, client *ethclient.Client, account common.Address) io.Closer + RecordLatestL1Block(l1ref eth.L1BlockRef) RecordL2BlocksLoaded(l2ref eth.L2BlockRef) RecordChannelOpened(id derive.ChannelID, numPendingBlocks int) @@ -79,6 +83,9 @@ type Metrics struct { var _ Metricer = (*Metrics)(nil) +// implements the Registry getter, for metrics HTTP server to hook into +var _ opmetrics.RegistryMetricer = (*Metrics)(nil) + func NewMetrics(procName string) *Metrics { if procName == "" { procName = "default" @@ -179,17 +186,16 @@ func NewMetrics(procName string) *Metrics { } } -func (m *Metrics) Start(host string, port int) (*httputil.HTTPServer, error) { - return opmetrics.StartServer(m.registry, host, port) +func (m *Metrics) Registry() *prometheus.Registry { + return m.registry } func (m *Metrics) Document() []opmetrics.DocumentedMetric { return m.factory.Document() } -func (m *Metrics) StartBalanceMetrics(ctx context.Context, - l log.Logger, client *ethclient.Client, account common.Address) { - opmetrics.LaunchBalanceMetrics(ctx, l, m.registry, m.ns, client, account) +func (m *Metrics) StartBalanceMetrics(l log.Logger, client *ethclient.Client, account common.Address) io.Closer { + return opmetrics.LaunchBalanceMetrics(l, m.registry, m.ns, client, account) } // RecordInfo sets a pseudo-metric that contains versioning and diff --git a/op-batcher/metrics/noop.go b/op-batcher/metrics/noop.go index 6a63ea066f1d..dc68bebda786 100644 --- a/op-batcher/metrics/noop.go +++ b/op-batcher/metrics/noop.go @@ -1,16 +1,23 @@ package metrics import ( + "io" + + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/ethclient" + "github.com/ethereum/go-ethereum/log" + "github.com/ethereum-optimism/optimism/op-node/rollup/derive" "github.com/ethereum-optimism/optimism/op-service/eth" opmetrics "github.com/ethereum-optimism/optimism/op-service/metrics" txmetrics "github.com/ethereum-optimism/optimism/op-service/txmgr/metrics" - "github.com/ethereum/go-ethereum/core/types" ) type noopMetrics struct { opmetrics.NoopRefMetrics txmetrics.NoopTxMetrics + opmetrics.NoopRPCMetrics } var NoopMetrics Metricer = new(noopMetrics) @@ -35,3 +42,6 @@ func (*noopMetrics) RecordChannelTimedOut(derive.ChannelID) {} func (*noopMetrics) RecordBatchTxSubmitted() {} func (*noopMetrics) RecordBatchTxSuccess() {} func (*noopMetrics) RecordBatchTxFailed() {} +func (*noopMetrics) StartBalanceMetrics(log.Logger, *ethclient.Client, common.Address) io.Closer { + return nil +} diff --git a/op-batcher/rpc/api.go b/op-batcher/rpc/api.go index 309c9d3252c9..3641e64def3f 100644 --- a/op-batcher/rpc/api.go +++ b/op-batcher/rpc/api.go @@ -10,17 +10,17 @@ import ( "github.com/ethereum-optimism/optimism/op-service/rpc" ) -type batcherClient interface { - Start() error - Stop(ctx context.Context) error +type BatcherDriver interface { + StartBatchSubmitting() error + StopBatchSubmitting(ctx context.Context) error } type adminAPI struct { *rpc.CommonAdminAPI - b batcherClient + b BatcherDriver } -func NewAdminAPI(dr batcherClient, m metrics.RPCMetricer, log log.Logger) *adminAPI { +func NewAdminAPI(dr BatcherDriver, m metrics.RPCMetricer, log log.Logger) *adminAPI { return &adminAPI{ CommonAdminAPI: rpc.NewCommonAdminAPI(m, log), b: dr, @@ -35,9 +35,9 @@ func GetAdminAPI(api *adminAPI) gethrpc.API { } func (a *adminAPI) StartBatcher(_ context.Context) error { - return a.b.Start() + return a.b.StartBatchSubmitting() } func (a *adminAPI) StopBatcher(ctx context.Context) error { - return a.b.Stop(ctx) + return a.b.StopBatchSubmitting(ctx) } diff --git a/op-challenger/game/fault/trace/outputs/provider.go b/op-challenger/game/fault/trace/outputs/provider.go index d1f6d4c0089d..4d20c2a1e64a 100644 --- a/op-challenger/game/fault/trace/outputs/provider.go +++ b/op-challenger/game/fault/trace/outputs/provider.go @@ -34,7 +34,7 @@ type OutputTraceProvider struct { } func NewTraceProvider(ctx context.Context, logger log.Logger, rollupRpc string, gameDepth, prestateBlock, poststateBlock uint64) (*OutputTraceProvider, error) { - rollupClient, err := dial.DialRollupClientWithTimeout(dial.DefaultDialTimeout, logger, rollupRpc) + rollupClient, err := dial.DialRollupClientWithTimeout(ctx, dial.DefaultDialTimeout, logger, rollupRpc) if err != nil { return nil, err } diff --git a/op-challenger/game/service.go b/op-challenger/game/service.go index 5b73a928eaa7..a9e5cfd14b67 100644 --- a/op-challenger/game/service.go +++ b/op-challenger/game/service.go @@ -55,7 +55,7 @@ func NewService(ctx context.Context, logger log.Logger, cfg *config.Config) (*Se return nil, fmt.Errorf("failed to create the transaction manager: %w", err) } - l1Client, err := dial.DialEthClientWithTimeout(dial.DefaultDialTimeout, logger, cfg.L1EthRpc) + l1Client, err := dial.DialEthClientWithTimeout(ctx, dial.DefaultDialTimeout, logger, cfg.L1EthRpc) if err != nil { return nil, fmt.Errorf("failed to dial L1: %w", err) } diff --git a/op-challenger/metrics/metrics.go b/op-challenger/metrics/metrics.go index 707894ceb49c..30ca2ab15dfa 100644 --- a/op-challenger/metrics/metrics.go +++ b/op-challenger/metrics/metrics.go @@ -133,7 +133,12 @@ func (m *Metrics) StartBalanceMetrics( client *ethclient.Client, account common.Address, ) { - opmetrics.LaunchBalanceMetrics(ctx, l, m.registry, m.ns, client, account) + // TODO(7684): util was refactored to close, but ctx is still being used by caller for shutdown + balanceMetric := opmetrics.LaunchBalanceMetrics(l, m.registry, m.ns, client, account) + go func() { + <-ctx.Done() + _ = balanceMetric.Close() + }() } // RecordInfo sets a pseudo-metric that contains versioning and diff --git a/op-e2e/setup.go b/op-e2e/setup.go index 0dd123909970..9cfc795f2dcc 100644 --- a/op-e2e/setup.go +++ b/op-e2e/setup.go @@ -4,6 +4,7 @@ import ( "context" "crypto/ecdsa" "crypto/rand" + "errors" "fmt" "math/big" "net" @@ -37,7 +38,6 @@ import ( bss "github.com/ethereum-optimism/optimism/op-batcher/batcher" "github.com/ethereum-optimism/optimism/op-batcher/compressor" - batchermetrics "github.com/ethereum-optimism/optimism/op-batcher/metrics" "github.com/ethereum-optimism/optimism/op-bindings/predeploys" "github.com/ethereum-optimism/optimism/op-chain-ops/genesis" "github.com/ethereum-optimism/optimism/op-e2e/config" @@ -252,7 +252,7 @@ type System struct { RawClients map[string]*rpc.Client RollupNodes map[string]*rollupNode.OpNode L2OutputSubmitter *l2os.L2OutputSubmitter - BatchSubmitter *bss.BatchSubmitter + BatchSubmitter *bss.BatcherService Mocknet mocknet.Mocknet // TimeTravelClock is nil unless SystemConfig.SupportL1TimeTravel was set to true @@ -268,18 +268,16 @@ func (sys *System) NodeEndpoint(name string) string { } func (sys *System) Close() { + postCtx, postCancel := context.WithCancel(context.Background()) + postCancel() // immediate shutdown, no allowance for idling + if sys.L2OutputSubmitter != nil { sys.L2OutputSubmitter.Stop() } if sys.BatchSubmitter != nil { - ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) - defer cancel() - sys.BatchSubmitter.StopIfRunning(ctx) + _ = sys.BatchSubmitter.Kill() } - postCtx, postCancel := context.WithCancel(context.Background()) - postCancel() // immediate shutdown, no allowance for idling - for _, node := range sys.RollupNodes { _ = node.Stop(postCtx) } @@ -678,8 +676,7 @@ func (cfg SystemConfig) Start(t *testing.T, _opts ...SystemConfigOption) (*Syste return nil, fmt.Errorf("unable to start l2 output submitter: %w", err) } - // Batch Submitter - sys.BatchSubmitter, err = bss.NewBatchSubmitterFromCLIConfig(bss.CLIConfig{ + batcherCLIConfig := &bss.CLIConfig{ L1EthRpc: sys.EthInstances["l1"].WSEndpoint(), L2EthRpc: sys.EthInstances["sequencer"].WSEndpoint(), RollupRpc: sys.RollupNodes["sequencer"].HTTPEndpoint(), @@ -698,17 +695,17 @@ func (cfg SystemConfig) Start(t *testing.T, _opts ...SystemConfigOption) (*Syste Level: log.LvlInfo, Format: oplog.FormatText, }, - }, sys.cfg.Loggers["batcher"], batchermetrics.NoopMetrics) + Stopped: sys.cfg.DisableBatcher, // Batch submitter may be enabled later + } + // Batch Submitter + batcher, err := bss.BatcherServiceFromCLIConfig(context.Background(), "0.0.1", batcherCLIConfig, sys.cfg.Loggers["batcher"]) if err != nil { return nil, fmt.Errorf("failed to setup batch submitter: %w", err) } - - // Batcher may be enabled later - if !sys.cfg.DisableBatcher { - if err := sys.BatchSubmitter.Start(); err != nil { - return nil, fmt.Errorf("unable to start batch submitter: %w", err) - } + if err := batcher.Start(context.Background()); err != nil { + return nil, errors.Join(fmt.Errorf("failed to start batch submitter: %w", err), batcher.Stop(context.Background())) } + sys.BatchSubmitter = batcher return sys, nil } diff --git a/op-e2e/system_fpp_test.go b/op-e2e/system_fpp_test.go index 307ede8d0bfb..f97ff0ecf334 100644 --- a/op-e2e/system_fpp_test.go +++ b/op-e2e/system_fpp_test.go @@ -93,7 +93,7 @@ func testVerifyL2OutputRootEmptyBlock(t *testing.T, detached bool) { l2OutputRoot := agreedL2Output.OutputRoot t.Log("=====Stopping batch submitter=====") - err = sys.BatchSubmitter.Stop(ctx) + err = sys.BatchSubmitter.Driver().StopBatchSubmitting(context.Background()) require.NoError(t, err, "could not stop batch submitter") // Wait for the sequencer to catch up with the current L1 head so we know all submitted batches are processed @@ -121,7 +121,7 @@ func testVerifyL2OutputRootEmptyBlock(t *testing.T, detached bool) { l2Claim := l2Output.OutputRoot t.Log("=====Restarting batch submitter=====") - err = sys.BatchSubmitter.Start() + err = sys.BatchSubmitter.Driver().StartBatchSubmitting() require.NoError(t, err, "could not start batch submitter") t.Log("Add a transaction to the next batch after sequence of empty blocks") @@ -258,7 +258,7 @@ func testFaultProofProgramScenario(t *testing.T, ctx context.Context, sys *Syste t.Log("Shutting down network") // Shutdown the nodes from the actual chain. Should now be able to run using only the pre-fetched data. - sys.BatchSubmitter.StopIfRunning(context.Background()) + require.NoError(t, sys.BatchSubmitter.Kill()) sys.L2OutputSubmitter.Stop() sys.L2OutputSubmitter = nil for _, node := range sys.EthInstances { diff --git a/op-e2e/system_test.go b/op-e2e/system_test.go index 5e943102888d..19c0b828d105 100644 --- a/op-e2e/system_test.go +++ b/op-e2e/system_test.go @@ -1262,7 +1262,7 @@ func TestStopStartBatcher(t *testing.T) { require.Greater(t, newSeqStatus.SafeL2.Number, seqStatus.SafeL2.Number, "Safe chain did not advance") // stop the batch submission - err = sys.BatchSubmitter.Stop(context.Background()) + err = sys.BatchSubmitter.Driver().StopBatchSubmitting(context.Background()) require.Nil(t, err) // wait for any old safe blocks being submitted / derived @@ -1282,7 +1282,7 @@ func TestStopStartBatcher(t *testing.T) { require.Equal(t, newSeqStatus.SafeL2.Number, seqStatus.SafeL2.Number, "Safe chain advanced while batcher was stopped") // start the batch submission - err = sys.BatchSubmitter.Start() + err = sys.BatchSubmitter.Driver().StartBatchSubmitting() require.Nil(t, err) time.Sleep(safeBlockInclusionDuration) @@ -1321,7 +1321,7 @@ func TestBatcherMultiTx(t *testing.T) { require.Nil(t, err) // start batch submission - err = sys.BatchSubmitter.Start() + err = sys.BatchSubmitter.Driver().StartBatchSubmitting() require.Nil(t, err) totalTxCount := 0 diff --git a/op-node/metrics/metrics.go b/op-node/metrics/metrics.go index a5faba121d53..b0fc76b1c69d 100644 --- a/op-node/metrics/metrics.go +++ b/op-node/metrics/metrics.go @@ -604,7 +604,9 @@ func (m *Metrics) ReportProtocolVersions(local, engine, recommended, required pa m.ProtocolVersions.WithLabelValues(local.String(), engine.String(), recommended.String(), required.String()).Set(1) } -type noopMetricer struct{} +type noopMetricer struct { + metrics.NoopRPCMetrics +} var NoopMetrics Metricer = new(noopMetricer) @@ -614,17 +616,6 @@ func (n *noopMetricer) RecordInfo(version string) { func (n *noopMetricer) RecordUp() { } -func (n *noopMetricer) RecordRPCServerRequest(method string) func() { - return func() {} -} - -func (n *noopMetricer) RecordRPCClientRequest(method string) func(err error) { - return func(err error) {} -} - -func (n *noopMetricer) RecordRPCClientResponse(method string, err error) { -} - func (n *noopMetricer) SetDerivationIdle(status bool) { } diff --git a/op-proposer/metrics/metrics.go b/op-proposer/metrics/metrics.go index bbb326ca4fe5..110e47ef34da 100644 --- a/op-proposer/metrics/metrics.go +++ b/op-proposer/metrics/metrics.go @@ -84,7 +84,12 @@ func (m *Metrics) Start(host string, port int) (*httputil.HTTPServer, error) { func (m *Metrics) StartBalanceMetrics(ctx context.Context, l log.Logger, client *ethclient.Client, account common.Address) { - opmetrics.LaunchBalanceMetrics(ctx, l, m.registry, m.ns, client, account) + // TODO(7684): util was refactored to close, but ctx is still being used by caller for shutdown + balanceMetric := opmetrics.LaunchBalanceMetrics(l, m.registry, m.ns, client, account) + go func() { + <-ctx.Done() + _ = balanceMetric.Close() + }() } // RecordInfo sets a pseudo-metric that contains versioning and diff --git a/op-proposer/proposer/l2_output_submitter.go b/op-proposer/proposer/l2_output_submitter.go index b704c083d8ed..255d3e9306e1 100644 --- a/op-proposer/proposer/l2_output_submitter.go +++ b/op-proposer/proposer/l2_output_submitter.go @@ -172,12 +172,12 @@ func NewL2OutputSubmitterConfigFromCLIConfig(cfg CLIConfig, l log.Logger, m metr } // Connect to L1 and L2 providers. Perform these last since they are the most expensive. - l1Client, err := dial.DialEthClientWithTimeout(dial.DefaultDialTimeout, l, cfg.L1EthRpc) + l1Client, err := dial.DialEthClientWithTimeout(context.Background(), dial.DefaultDialTimeout, l, cfg.L1EthRpc) if err != nil { return nil, err } - rollupClient, err := dial.DialRollupClientWithTimeout(dial.DefaultDialTimeout, l, cfg.RollupRpc) + rollupClient, err := dial.DialRollupClientWithTimeout(context.Background(), dial.DefaultDialTimeout, l, cfg.RollupRpc) if err != nil { return nil, err } diff --git a/op-service/clock/loop.go b/op-service/clock/loop.go new file mode 100644 index 000000000000..92c610e5043a --- /dev/null +++ b/op-service/clock/loop.go @@ -0,0 +1,62 @@ +package clock + +import ( + "context" + "sync" + "time" +) + +// LoopFn is a simple ticker-loop with io.Closer support. +// Note that ticks adapt; slow function calls may result in lost ticks. +type LoopFn struct { + ctx context.Context + cancel context.CancelFunc + + ticker Ticker + fn func(ctx context.Context) + onClose func() error + + wg sync.WaitGroup +} + +// Close cancels the context of the ongoing function call, waits for the call to complete, and cancels further calls. +// Close is safe to call again or concurrently. The onClose callback will be called for each Close call. +func (lf *LoopFn) Close() error { + lf.cancel() // stop any ongoing function call, and close the main loop + lf.wg.Wait() // wait for completion + if lf.onClose != nil { + return lf.onClose() // optional: user can specify function to close resources with + } + return nil +} + +func (lf *LoopFn) work() { + defer lf.wg.Done() + defer lf.ticker.Stop() // clean up the timer + for { + select { + case <-lf.ctx.Done(): + return + case <-lf.ticker.Ch(): + ctx, cancel := context.WithCancel(lf.ctx) + lf.fn(ctx) + cancel() + } + } +} + +// NewLoopFn creates a periodic function call, which can be closed, +// with an optional onClose callback to clean up resources. +func NewLoopFn(clock Clock, fn func(ctx context.Context), onClose func() error, interval time.Duration) *LoopFn { + ctx, cancel := context.WithCancel(context.Background()) + lf := &LoopFn{ + ctx: ctx, + cancel: cancel, + fn: fn, + ticker: clock.NewTicker(interval), + onClose: onClose, + } + lf.wg.Add(1) + go lf.work() + return lf +} diff --git a/op-service/clock/loop_test.go b/op-service/clock/loop_test.go new file mode 100644 index 000000000000..56c1c8c2a463 --- /dev/null +++ b/op-service/clock/loop_test.go @@ -0,0 +1,27 @@ +package clock + +import ( + "context" + "errors" + "testing" + "time" + + "github.com/stretchr/testify/require" +) + +func TestLoopFn(t *testing.T) { + cl := NewDeterministicClock(time.Now()) + calls := make(chan struct{}, 10) + testErr := errors.New("test close error") + loopFn := NewLoopFn(cl, func(ctx context.Context) { + calls <- struct{}{} + }, func() error { + close(calls) + return testErr + }, time.Second*10) + cl.AdvanceTime(time.Second * 15) + <-calls + cl.AdvanceTime(time.Second * 10) + <-calls + require.ErrorIs(t, loopFn.Close(), testErr) +} diff --git a/op-service/dial/dial.go b/op-service/dial/dial.go index cab8b8466b30..11410197b7d3 100644 --- a/op-service/dial/dial.go +++ b/op-service/dial/dial.go @@ -21,8 +21,8 @@ const defaultRetryTime = 2 * time.Second // DialEthClientWithTimeout attempts to dial the L1 provider using the provided // URL. If the dial doesn't complete within defaultDialTimeout seconds, this // method will return an error. -func DialEthClientWithTimeout(timeout time.Duration, log log.Logger, url string) (*ethclient.Client, error) { - ctx, cancel := context.WithTimeout(context.Background(), timeout) +func DialEthClientWithTimeout(ctx context.Context, timeout time.Duration, log log.Logger, url string) (*ethclient.Client, error) { + ctx, cancel := context.WithTimeout(ctx, timeout) defer cancel() c, err := dialRPCClientWithBackoff(ctx, log, url) @@ -35,8 +35,8 @@ func DialEthClientWithTimeout(timeout time.Duration, log log.Logger, url string) // DialRollupClientWithTimeout attempts to dial the RPC provider using the provided URL. // If the dial doesn't complete within timeout seconds, this method will return an error. -func DialRollupClientWithTimeout(timeout time.Duration, log log.Logger, url string) (*sources.RollupClient, error) { - ctx, cancel := context.WithTimeout(context.Background(), timeout) +func DialRollupClientWithTimeout(ctx context.Context, timeout time.Duration, log log.Logger, url string) (*sources.RollupClient, error) { + ctx, cancel := context.WithTimeout(ctx, timeout) defer cancel() rpcCl, err := dialRPCClientWithBackoff(ctx, log, url) diff --git a/op-service/metrics/balance.go b/op-service/metrics/balance.go index de9066846c5e..8da012813268 100644 --- a/op-service/metrics/balance.go +++ b/op-service/metrics/balance.go @@ -5,12 +5,15 @@ import ( "math/big" "time" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" + "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/ethclient" "github.com/ethereum/go-ethereum/log" "github.com/ethereum/go-ethereum/params" - "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/client_golang/prometheus/promauto" + + "github.com/ethereum-optimism/optimism/op-service/clock" ) // weiToEther divides the wei value by 10^18 to get a number in ether as a float64 @@ -22,38 +25,27 @@ func weiToEther(wei *big.Int) float64 { return f } -// LaunchBalanceMetrics fires off a go rountine that queries the balance of the supplied account & periodically records it -// to the balance metric of the namespace. The balance of the account is recorded in Ether (not Wei). +// LaunchBalanceMetrics starts a periodic query of the balance of the supplied account and records it +// to the "balance" metric of the namespace. The balance of the account is recorded in Ether (not Wei). // Cancel the supplied context to shut down the go routine -func LaunchBalanceMetrics(ctx context.Context, log log.Logger, r *prometheus.Registry, ns string, client *ethclient.Client, account common.Address) { - go func() { - balanceGuage := promauto.With(r).NewGauge(prometheus.GaugeOpts{ - Namespace: ns, - Name: "balance", - Help: "balance (in ether) of account " + account.String(), - }) - - ticker := time.NewTicker(10 * time.Second) - defer ticker.Stop() - - for { - select { - case <-ticker.C: - ctx, cancel := context.WithTimeout(ctx, 1*time.Minute) - bigBal, err := client.BalanceAt(ctx, account, nil) - if err != nil { - log.Warn("failed to get balance of account", "err", err, "address", account) - cancel() - continue - } - bal := weiToEther(bigBal) - balanceGuage.Set(bal) - cancel() - case <-ctx.Done(): - log.Info("balance metrics shutting down") - return - } +func LaunchBalanceMetrics(log log.Logger, r *prometheus.Registry, ns string, client *ethclient.Client, account common.Address) *clock.LoopFn { + balanceGuage := promauto.With(r).NewGauge(prometheus.GaugeOpts{ + Namespace: ns, + Name: "balance", + Help: "balance (in ether) of account " + account.String(), + }) + return clock.NewLoopFn(clock.SystemClock, func(ctx context.Context) { + ctx, cancel := context.WithTimeout(ctx, 1*time.Minute) + defer cancel() + bigBal, err := client.BalanceAt(ctx, account, nil) + if err != nil { + log.Warn("failed to get balance of account", "err", err, "address", account) + return } - - }() + bal := weiToEther(bigBal) + balanceGuage.Set(bal) + }, func() error { + log.Info("balance metrics shutting down") + return nil + }, 10*time.Second) } diff --git a/op-service/metrics/registry.go b/op-service/metrics/registry.go index 882bcbe4ae35..9ba44707c586 100644 --- a/op-service/metrics/registry.go +++ b/op-service/metrics/registry.go @@ -11,3 +11,7 @@ func NewRegistry() *prometheus.Registry { registry.MustRegister(collectors.NewGoCollector()) return registry } + +type RegistryMetricer interface { + Registry() *prometheus.Registry +} diff --git a/op-service/metrics/rpc_metrics.go b/op-service/metrics/rpc_metrics.go index a9f0d105798e..b6bda30c8007 100644 --- a/op-service/metrics/rpc_metrics.go +++ b/op-service/metrics/rpc_metrics.go @@ -125,3 +125,17 @@ func (m *RPCMetrics) RecordRPCClientResponse(method string, err error) { } m.RPCClientResponsesTotal.WithLabelValues(method, errStr).Inc() } + +type NoopRPCMetrics struct{} + +func (n *NoopRPCMetrics) RecordRPCServerRequest(method string) func() { + return func() {} +} + +func (n *NoopRPCMetrics) RecordRPCClientRequest(method string) func(err error) { + return func(err error) {} +} +func (n *NoopRPCMetrics) RecordRPCClientResponse(method string, err error) { +} + +var _ RPCMetricer = (*NoopRPCMetrics)(nil) diff --git a/op-service/sources/rollupclient.go b/op-service/sources/rollupclient.go index 3e8bdfc02feb..863305d6a101 100644 --- a/op-service/sources/rollupclient.go +++ b/op-service/sources/rollupclient.go @@ -63,3 +63,7 @@ func (r *RollupClient) SequencerActive(ctx context.Context) (bool, error) { func (r *RollupClient) SetLogLevel(ctx context.Context, lvl log.Lvl) error { return r.rpc.CallContext(ctx, nil, "admin_setLogLevel", lvl.String()) } + +func (r *RollupClient) Close() { + r.rpc.Close() +} From 6bd72f006de98ae28c49cbf3ae97c6e01787ce72 Mon Sep 17 00:00:00 2001 From: protolambda Date: Mon, 16 Oct 2023 14:08:59 +0200 Subject: [PATCH 002/279] op-batcher: apply code-review suggestions Co-authored-by: Sebastian Stammler --- op-batcher/batcher/service.go | 10 ---------- op-e2e/system_fpp_test.go | 2 +- op-service/clock/loop.go | 6 ++++-- op-service/clock/loop_test.go | 5 +++++ 4 files changed, 10 insertions(+), 13 deletions(-) diff --git a/op-batcher/batcher/service.go b/op-batcher/batcher/service.go index 2a091119454a..22e8dd0e2fb8 100644 --- a/op-batcher/batcher/service.go +++ b/op-batcher/batcher/service.go @@ -110,10 +110,6 @@ func (bs *BatcherService) initFromCLIConfig(ctx context.Context, version string, if err := bs.initRPCServer(cfg); err != nil { return fmt.Errorf("failed to start RPC server: %w", err) } - // Validate the setup - if err := bs.Check(); err != nil { - return fmt.Errorf("failed post-initialization check: %w", err) - } bs.Metrics.RecordInfo(bs.Version) bs.Metrics.RecordUp() @@ -260,12 +256,6 @@ func (bs *BatcherService) initRPCServer(cfg *CLIConfig) error { return nil } -// Check ensures that the [BatcherService] is valid -func (bs *BatcherService) Check() error { - - return nil -} - // Start runs once upon start of the batcher lifecycle, // and starts batch-submission work if the batcher is configured to start submit data on startup. func (bs *BatcherService) Start(_ context.Context) error { diff --git a/op-e2e/system_fpp_test.go b/op-e2e/system_fpp_test.go index f97ff0ecf334..b4a20bc086e1 100644 --- a/op-e2e/system_fpp_test.go +++ b/op-e2e/system_fpp_test.go @@ -93,7 +93,7 @@ func testVerifyL2OutputRootEmptyBlock(t *testing.T, detached bool) { l2OutputRoot := agreedL2Output.OutputRoot t.Log("=====Stopping batch submitter=====") - err = sys.BatchSubmitter.Driver().StopBatchSubmitting(context.Background()) + err = sys.BatchSubmitter.Driver().StopBatchSubmitting(ctx) require.NoError(t, err, "could not stop batch submitter") // Wait for the sequencer to catch up with the current L1 head so we know all submitted batches are processed diff --git a/op-service/clock/loop.go b/op-service/clock/loop.go index 92c610e5043a..d5a9aa8a4948 100644 --- a/op-service/clock/loop.go +++ b/op-service/clock/loop.go @@ -39,8 +39,10 @@ func (lf *LoopFn) work() { return case <-lf.ticker.Ch(): ctx, cancel := context.WithCancel(lf.ctx) - lf.fn(ctx) - cancel() + func() { + defer cancel() + lf.fn(ctx) + }() } } } diff --git a/op-service/clock/loop_test.go b/op-service/clock/loop_test.go index 56c1c8c2a463..6eab6af19cb4 100644 --- a/op-service/clock/loop_test.go +++ b/op-service/clock/loop_test.go @@ -23,5 +23,10 @@ func TestLoopFn(t *testing.T) { <-calls cl.AdvanceTime(time.Second * 10) <-calls + select { + case <-calls: + t.Fatal("more calls than expected") + default: + } require.ErrorIs(t, loopFn.Close(), testErr) } From 2a93a0948eee51466e244b39c11fe1b5116a9a30 Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Wed, 18 Oct 2023 11:24:14 +1000 Subject: [PATCH 003/279] op-geth: Pull in op-geth with upstream v1.13.4 changes. --- go.mod | 14 +++++++------- go.sum | 23 +++++++++++------------ 2 files changed, 18 insertions(+), 19 deletions(-) diff --git a/go.mod b/go.mod index efb2a1261c1d..1e0282bbea14 100644 --- a/go.mod +++ b/go.mod @@ -9,7 +9,7 @@ require ( github.com/decred/dcrd/dcrec/secp256k1/v4 v4.2.0 github.com/ethereum-optimism/go-ethereum-hdwallet v0.1.3 github.com/ethereum-optimism/superchain-registry/superchain v0.0.0-20231001123245-7b48d3818686 - github.com/ethereum/go-ethereum v1.13.3 + github.com/ethereum/go-ethereum v1.13.4 github.com/fsnotify/fsnotify v1.6.0 github.com/go-chi/chi/v5 v5.0.10 github.com/go-chi/docgen v1.2.0 @@ -51,7 +51,7 @@ require ( require ( github.com/DataDog/zstd v1.5.2 // indirect github.com/Microsoft/go-winio v0.6.1 // indirect - github.com/VictoriaMetrics/fastcache v1.10.0 // indirect + github.com/VictoriaMetrics/fastcache v1.12.1 // indirect github.com/allegro/bigcache v1.2.1 // indirect github.com/benbjohnson/clock v1.3.5 // indirect github.com/beorn7/perks v1.0.1 // indirect @@ -66,7 +66,7 @@ require ( github.com/cockroachdb/redact v1.1.3 // indirect github.com/cockroachdb/tokenbucket v0.0.0-20230807174530-cc333fc44b06 // indirect github.com/consensys/bavard v0.1.13 // indirect - github.com/consensys/gnark-crypto v0.12.0 // indirect + github.com/consensys/gnark-crypto v0.12.1 // indirect github.com/containerd/cgroups v1.1.0 // indirect github.com/coreos/go-systemd/v22 v22.5.0 // indirect github.com/cpuguy83/go-md2man/v2 v2.0.2 // indirect @@ -82,7 +82,7 @@ require ( github.com/dop251/goja v0.0.0-20230806174421-c933cf95e127 // indirect github.com/elastic/gosigar v0.14.2 // indirect github.com/ethereum/c-kzg-4844 v0.3.1 // indirect - github.com/fatih/color v1.7.0 // indirect + github.com/fatih/color v1.13.0 // indirect github.com/felixge/fgprof v0.9.3 // indirect github.com/fjl/memsize v0.0.1 // indirect github.com/flynn/noise v1.0.0 // indirect @@ -97,7 +97,7 @@ require ( github.com/godbus/dbus/v5 v5.1.0 // indirect github.com/gofrs/flock v0.8.1 // indirect github.com/gogo/protobuf v1.3.2 // indirect - github.com/golang-jwt/jwt/v4 v4.4.2 // indirect + github.com/golang-jwt/jwt/v4 v4.5.0 // indirect github.com/golang/mock v1.6.0 // indirect github.com/golang/protobuf v1.5.3 // indirect github.com/google/gopacket v1.1.19 // indirect @@ -209,7 +209,7 @@ require ( rsc.io/tmplfunc v0.0.3 // indirect ) -replace github.com/ethereum/go-ethereum v1.13.3 => github.com/ethereum-optimism/op-geth v1.101301.2-0.20231018005749-a1972f1894c0 +replace github.com/ethereum/go-ethereum v1.13.4 => github.com/ethereum-optimism/op-geth v1.101301.2-0.20231018010910-b142ad43dda3 //replace github.com/ethereum-optimism/superchain-registry/superchain => ../superchain-registry/superchain -//replace github.com/ethereum/go-ethereum v1.13.3 => ../go-ethereum +//replace github.com/ethereum/go-ethereum v1.13.4 => ../go-ethereum diff --git a/go.sum b/go.sum index b153e93e2317..db5eb2d40635 100644 --- a/go.sum +++ b/go.sum @@ -22,8 +22,8 @@ github.com/Masterminds/semver/v3 v3.1.1/go.mod h1:VPu/7SZ7ePZ3QOrcuXROw5FAcLl4a0 github.com/Microsoft/go-winio v0.6.1 h1:9/kr64B9VUZrLm5YYwbGtUJnMgqWVOdUAXu6Migciow= github.com/Microsoft/go-winio v0.6.1/go.mod h1:LRdKpFKfdobln8UmuiYcKPot9D2v6svN5+sAH+4kjUM= github.com/Shopify/goreferrer v0.0.0-20181106222321-ec9c9a553398/go.mod h1:a1uqRtAwp2Xwc6WNPJEufxJ7fx3npB4UV/JOLmbu5I0= -github.com/VictoriaMetrics/fastcache v1.10.0 h1:5hDJnLsKLpnUEToub7ETuRu8RCkb40woBZAUiKonXzY= -github.com/VictoriaMetrics/fastcache v1.10.0/go.mod h1:tjiYeEfYXCqacuvYw/7UoDIeJaNxq6132xHICNP77w8= +github.com/VictoriaMetrics/fastcache v1.12.1 h1:i0mICQuojGDL3KblA7wUNlY5lOK6a4bwt3uRKnkZU40= +github.com/VictoriaMetrics/fastcache v1.12.1/go.mod h1:tX04vaqcNoQeGLD+ra5pU5sWkuxnzWhEzLwhP9w653o= github.com/aead/siphash v1.0.1/go.mod h1:Nywa3cDsYNNK3gaciGTWPwHt0wlpNV15vwmswBAUSII= github.com/ajg/form v1.5.1/go.mod h1:uL1WgH+h2mgNtvBq0339dVnzXdBETtL2LeUXaIv25UY= github.com/allegro/bigcache v1.2.1-0.20190218064605-e24eb225f156/go.mod h1:Cb/ax3seSYIx7SuZdm2G2xzfwmv3TPSk2ucNfQESPXM= @@ -70,7 +70,6 @@ github.com/cespare/cp v0.1.0 h1:SE+dxFebS7Iik5LK0tsi1k9ZCxEaFX4AjQmoyA+1dJk= github.com/cespare/cp v0.1.0/go.mod h1:SOGHArjBr4JWaSDEVpWpo/hNg6RoKrls6Oh40hiwW+s= github.com/cespare/xxhash v1.1.0 h1:a6HrQnmkObjyL+Gs60czilIUGqrzKutQD6XZog3p+ko= github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghfAqPWnc= -github.com/cespare/xxhash/v2 v2.1.2/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWRnGsAI= @@ -100,8 +99,8 @@ github.com/cockroachdb/tokenbucket v0.0.0-20230807174530-cc333fc44b06/go.mod h1: github.com/codegangsta/inject v0.0.0-20150114235600-33e0aa1cb7c0/go.mod h1:4Zcjuz89kmFXt9morQgcfYZAYZ5n8WHjt81YYWIwtTM= github.com/consensys/bavard v0.1.13 h1:oLhMLOFGTLdlda/kma4VOJazblc7IM5y5QPd2A/YjhQ= github.com/consensys/bavard v0.1.13/go.mod h1:9ItSMtA/dXMAiL7BG6bqW2m3NdSEObYWoH223nGHukI= -github.com/consensys/gnark-crypto v0.12.0 h1:1OnSpOykNkUIBIBJKdhwy2p0JlW5o+Az02ICzZmvvdg= -github.com/consensys/gnark-crypto v0.12.0/go.mod h1:v2Gy7L/4ZRosZ7Ivs+9SfUDr0f5UlG+EM5t7MPHiLuY= +github.com/consensys/gnark-crypto v0.12.1 h1:lHH39WuuFgVHONRl3J0LRBtuYdQTumFSDtJF7HpyG8M= +github.com/consensys/gnark-crypto v0.12.1/go.mod h1:v2Gy7L/4ZRosZ7Ivs+9SfUDr0f5UlG+EM5t7MPHiLuY= github.com/containerd/cgroups v0.0.0-20201119153540-4cbc285b3327/go.mod h1:ZJeTFisyysqgcCdecO57Dj79RfL0LNeGiFUqLYQRYLE= github.com/containerd/cgroups v1.1.0 h1:v8rEWFl6EoqHB+swVNjVoCJE8o3jX7e8nqBGPLaDFBM= github.com/containerd/cgroups v1.1.0/go.mod h1:6ppBcbh/NOOUU+dMKrykgaBnK9lCIBxHqJDGwsa1mIw= @@ -177,15 +176,15 @@ github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7 github.com/etcd-io/bbolt v1.3.3/go.mod h1:ZF2nL25h33cCyBtcyWeZ2/I3HQOfTP+0PIEvHjkjCrw= github.com/ethereum-optimism/go-ethereum-hdwallet v0.1.3 h1:RWHKLhCrQThMfch+QJ1Z8veEq5ZO3DfIhZ7xgRP9WTc= github.com/ethereum-optimism/go-ethereum-hdwallet v0.1.3/go.mod h1:QziizLAiF0KqyLdNJYD7O5cpDlaFMNZzlxYNcWsJUxs= -github.com/ethereum-optimism/op-geth v1.101301.2-0.20231018005749-a1972f1894c0 h1:hN2yjR0Nst/lVYAouhNH49l1vwIC+U/Ayp0hQKEy8QI= -github.com/ethereum-optimism/op-geth v1.101301.2-0.20231018005749-a1972f1894c0/go.mod h1:0UHkkEfuGgDDsuIAcf4m/P3JCX1eARPz/Ou6H+73e2k= +github.com/ethereum-optimism/op-geth v1.101301.2-0.20231018010910-b142ad43dda3 h1:WdAevEuDn3I1YPChnIlgqCXvDFx2/oRvcvDG7aU86sk= +github.com/ethereum-optimism/op-geth v1.101301.2-0.20231018010910-b142ad43dda3/go.mod h1:V2K+IIUITMRKVGLiPCr29RvbDaA9P7YE5JL7UrFsbD8= github.com/ethereum-optimism/superchain-registry/superchain v0.0.0-20231001123245-7b48d3818686 h1:f57hd8G96c8ORWd4ameFpveSnHcb0hA2D1VatviwoDc= github.com/ethereum-optimism/superchain-registry/superchain v0.0.0-20231001123245-7b48d3818686/go.mod h1:q0u2UbyOr1q/y94AgMOj/V8b1KO05ZwILTR/qKt7Auo= github.com/ethereum/c-kzg-4844 v0.3.1 h1:sR65+68+WdnMKxseNWxSJuAv2tsUrihTpVBTfM/U5Zg= github.com/ethereum/c-kzg-4844 v0.3.1/go.mod h1:VewdlzQmpT5QSrVhbBuGoCdFJkpaJlO1aQputP83wc0= github.com/fasthttp-contrib/websocket v0.0.0-20160511215533-1f3b11f56072/go.mod h1:duJ4Jxv5lDcvg4QuQr0oowTf7dz4/CR8NtyCooz9HL8= -github.com/fatih/color v1.7.0 h1:DkWD4oS2D8LGGgTQ6IvwJJXSL5Vp2ffcQg58nFV38Ys= -github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4= +github.com/fatih/color v1.13.0 h1:8LOYc1KYPPmyKMuN8QV2DNRWNbLo6LZ0iLs8+mlH53w= +github.com/fatih/color v1.13.0/go.mod h1:kLAiJbzzSOZDVNGyDpeOxJ47H46qBXwg5ILebYFFOfk= github.com/fatih/structs v1.1.0/go.mod h1:9NiDSp5zOcgEDl+j00MP/WkGVPOlPRLejGD8Ga6PJ7M= github.com/felixge/fgprof v0.9.3 h1:VvyZxILNuCiUCSXtPtYmmtGvb65nqXh2QFWc0Wpf2/g= github.com/felixge/fgprof v0.9.3/go.mod h1:RdbpDgzqYVh/T9fPELJyV7EYJuHB55UTEULNun8eiPw= @@ -262,8 +261,8 @@ github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= github.com/gogo/status v1.1.0/go.mod h1:BFv9nrluPLmrS0EmGVvLaPNmRosr9KapBYd5/hpY1WM= github.com/golang-jwt/jwt v3.2.2+incompatible/go.mod h1:8pz2t5EyA70fFQQSrl6XZXzqecmYZeUEB8OUGHkxJ+I= -github.com/golang-jwt/jwt/v4 v4.4.2 h1:rcc4lwaZgFMCZ5jxF9ABolDcIHdBytAFgqFPbSJQAYs= -github.com/golang-jwt/jwt/v4 v4.4.2/go.mod h1:m21LjoU+eqJr34lmDMbreY2eSTRJ1cv77w39/MY0Ch0= +github.com/golang-jwt/jwt/v4 v4.5.0 h1:7cYmW1XlMY7h7ii7UhUyChSgS5wUJEnm9uZVTGqOWzg= +github.com/golang-jwt/jwt/v4 v4.5.0/go.mod h1:m21LjoU+eqJr34lmDMbreY2eSTRJ1cv77w39/MY0Ch0= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= github.com/golang/lint v0.0.0-20180702182130-06c8688daad7/go.mod h1:tluoj9z5200jBnyusfRPU2LqT6J+DAorxEvtC7LHB+E= github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= @@ -544,6 +543,7 @@ github.com/mattn/go-colorable v0.1.2/go.mod h1:U0ppj6V5qS13XJ6of8GYAs25YV2eR4EVc github.com/mattn/go-colorable v0.1.6/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope9wVRipJSqc= github.com/mattn/go-colorable v0.1.7/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope9wVRipJSqc= github.com/mattn/go-colorable v0.1.8/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope9wVRipJSqc= +github.com/mattn/go-colorable v0.1.9/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope9wVRipJSqc= github.com/mattn/go-colorable v0.1.11/go.mod h1:u5H1YNBxpqRaxsYJYSkiCWKzEfiAb1Gb520KVy5xxl4= github.com/mattn/go-colorable v0.1.13 h1:fFA4WZxdEF4tXPZVKMLwD8oUnCTTo08duU7wxecdEvA= github.com/mattn/go-colorable v0.1.13/go.mod h1:7S9/ev0klgBDR4GtXTXX8a3vIGJpMovkB8vQcUbaXHg= @@ -998,7 +998,6 @@ golang.org/x/sys v0.0.0-20211020174200-9d6173849985/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20211216021012-1d35b9e2eb4e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220209214540-3681064d5158/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220310020820-b874c991c1a5/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20220405052023-b1e9470b6e64/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220412211240-33da011f77ad/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= From 73c751287e7aa95eafcb1d1ac762b628548e5930 Mon Sep 17 00:00:00 2001 From: protolambda Date: Fri, 20 Oct 2023 16:00:30 +0200 Subject: [PATCH 004/279] op-service: use geth withdrawals type, verify withdrawals in rpc-block verification, add test vectors and rpc-block tests --- op-e2e/actions/l2_engine_test.go | 4 +- op-e2e/op_geth.go | 4 +- op-e2e/op_geth_test.go | 2 +- op-node/rollup/derive/attributes.go | 4 +- op-node/rollup/derive/engine_consolidate.go | 4 +- .../rollup/derive/engine_consolidate_test.go | 23 ++--- .../l2/engineapi/test/l2_engine_api_tests.go | 12 +-- op-service/eth/ssz.go | 18 ++-- op-service/eth/ssz_test.go | 28 ++++--- op-service/eth/types.go | 30 +------ .../post-shanghai-bad-receipts_data.json | 1 + .../post-shanghai-bad-receipts_metadata.json | 1 + .../post-shanghai-bad-transactions_data.json | 1 + ...st-shanghai-bad-transactions_metadata.json | 1 + .../post-shanghai-bad-withdrawals_data.json | 1 + ...ost-shanghai-bad-withdrawals_metadata.json | 1 + .../blocks/post-shanghai-success_data.json | 1 + .../post-shanghai-success_metadata.json | 1 + .../pre-shanghai-bad-receipts_data.json | 1 + .../pre-shanghai-bad-receipts_metadata.json | 1 + .../pre-shanghai-bad-transactions_data.json | 1 + ...re-shanghai-bad-transactions_metadata.json | 1 + .../blocks/pre-shanghai-success_data.json | 1 + .../blocks/pre-shanghai-success_metadata.json | 1 + .../post-shanghai-bad-receipts_data.json | 1 + .../post-shanghai-bad-receipts_metadata.json | 1 + .../post-shanghai-bad-transactions_data.json | 1 + ...st-shanghai-bad-transactions_metadata.json | 1 + .../post-shanghai-bad-withdrawals_data.json | 1 + ...ost-shanghai-bad-withdrawals_metadata.json | 1 + .../headers/post-shanghai-success_data.json | 1 + .../post-shanghai-success_metadata.json | 1 + .../pre-shanghai-bad-receipts_data.json | 1 + .../pre-shanghai-bad-receipts_metadata.json | 1 + .../pre-shanghai-bad-transactions_data.json | 1 + ...re-shanghai-bad-transactions_metadata.json | 1 + .../headers/pre-shanghai-success_data.json | 1 + .../pre-shanghai-success_metadata.json | 1 + op-service/sources/testdata/gen.sh | 81 ++++++++++++++++++ op-service/sources/types.go | 24 +++++- op-service/sources/types_test.go | 84 ++++++++++++++----- 41 files changed, 250 insertions(+), 96 deletions(-) create mode 100644 op-service/sources/testdata/data/blocks/post-shanghai-bad-receipts_data.json create mode 100644 op-service/sources/testdata/data/blocks/post-shanghai-bad-receipts_metadata.json create mode 100644 op-service/sources/testdata/data/blocks/post-shanghai-bad-transactions_data.json create mode 100644 op-service/sources/testdata/data/blocks/post-shanghai-bad-transactions_metadata.json create mode 100644 op-service/sources/testdata/data/blocks/post-shanghai-bad-withdrawals_data.json create mode 100644 op-service/sources/testdata/data/blocks/post-shanghai-bad-withdrawals_metadata.json create mode 100644 op-service/sources/testdata/data/blocks/post-shanghai-success_data.json create mode 100644 op-service/sources/testdata/data/blocks/post-shanghai-success_metadata.json create mode 100644 op-service/sources/testdata/data/blocks/pre-shanghai-bad-receipts_data.json create mode 100644 op-service/sources/testdata/data/blocks/pre-shanghai-bad-receipts_metadata.json create mode 100644 op-service/sources/testdata/data/blocks/pre-shanghai-bad-transactions_data.json create mode 100644 op-service/sources/testdata/data/blocks/pre-shanghai-bad-transactions_metadata.json create mode 100644 op-service/sources/testdata/data/blocks/pre-shanghai-success_data.json create mode 100644 op-service/sources/testdata/data/blocks/pre-shanghai-success_metadata.json create mode 100644 op-service/sources/testdata/data/headers/post-shanghai-bad-receipts_data.json create mode 100644 op-service/sources/testdata/data/headers/post-shanghai-bad-receipts_metadata.json create mode 100644 op-service/sources/testdata/data/headers/post-shanghai-bad-transactions_data.json create mode 100644 op-service/sources/testdata/data/headers/post-shanghai-bad-transactions_metadata.json create mode 100644 op-service/sources/testdata/data/headers/post-shanghai-bad-withdrawals_data.json create mode 100644 op-service/sources/testdata/data/headers/post-shanghai-bad-withdrawals_metadata.json create mode 100644 op-service/sources/testdata/data/headers/post-shanghai-success_data.json create mode 100644 op-service/sources/testdata/data/headers/post-shanghai-success_metadata.json create mode 100644 op-service/sources/testdata/data/headers/pre-shanghai-bad-receipts_data.json create mode 100644 op-service/sources/testdata/data/headers/pre-shanghai-bad-receipts_metadata.json create mode 100644 op-service/sources/testdata/data/headers/pre-shanghai-bad-transactions_data.json create mode 100644 op-service/sources/testdata/data/headers/pre-shanghai-bad-transactions_metadata.json create mode 100644 op-service/sources/testdata/data/headers/pre-shanghai-success_data.json create mode 100644 op-service/sources/testdata/data/headers/pre-shanghai-success_metadata.json create mode 100644 op-service/sources/testdata/gen.sh diff --git a/op-e2e/actions/l2_engine_test.go b/op-e2e/actions/l2_engine_test.go index 47ee33f545a2..fc48a49e5262 100644 --- a/op-e2e/actions/l2_engine_test.go +++ b/op-e2e/actions/l2_engine_test.go @@ -127,9 +127,9 @@ func TestL2EngineAPIBlockBuilding(gt *testing.T) { nextBlockTime := eth.Uint64Quantity(parent.Time) + 2 - var w *eth.Withdrawals + var w *types.Withdrawals if sd.RollupCfg.IsCanyon(uint64(nextBlockTime)) { - w = ð.Withdrawals{} + w = &types.Withdrawals{} } // Now let's ask the engine to build a block diff --git a/op-e2e/op_geth.go b/op-e2e/op_geth.go index cfed6a1ace08..8a1bf8bbbe61 100644 --- a/op-e2e/op_geth.go +++ b/op-e2e/op_geth.go @@ -210,9 +210,9 @@ func (d *OpGeth) CreatePayloadAttributes(txs ...*types.Transaction) (*eth.Payloa txBytes = append(txBytes, bin) } - var withdrawals *eth.Withdrawals + var withdrawals *types.Withdrawals if d.L2ChainConfig.IsCanyon(uint64(timestamp)) { - withdrawals = ð.Withdrawals{} + withdrawals = &types.Withdrawals{} } attrs := eth.PayloadAttributes{ diff --git a/op-e2e/op_geth_test.go b/op-e2e/op_geth_test.go index ad1ecb48109c..230e99914ee0 100644 --- a/op-e2e/op_geth_test.go +++ b/op-e2e/op_geth_test.go @@ -823,7 +823,7 @@ func TestCanyon(t *testing.T) { b, err := opGeth.AddL2Block(ctx) require.NoError(t, err) - assert.Equal(t, *b.Withdrawals, eth.Withdrawals{}) + assert.Equal(t, *b.Withdrawals, types.Withdrawals{}) l1Block, err := opGeth.L2Client.BlockByNumber(ctx, nil) require.Nil(t, err) diff --git a/op-node/rollup/derive/attributes.go b/op-node/rollup/derive/attributes.go index 4d2d5bc5624c..b65f680952f3 100644 --- a/op-node/rollup/derive/attributes.go +++ b/op-node/rollup/derive/attributes.go @@ -109,9 +109,9 @@ func (ba *FetchingAttributesBuilder) PreparePayloadAttributes(ctx context.Contex txs = append(txs, l1InfoTx) txs = append(txs, depositTxs...) - var withdrawals *eth.Withdrawals + var withdrawals *types.Withdrawals if ba.cfg.IsCanyon(nextL2Time) { - withdrawals = ð.Withdrawals{} + withdrawals = &types.Withdrawals{} } return ð.PayloadAttributes{ diff --git a/op-node/rollup/derive/engine_consolidate.go b/op-node/rollup/derive/engine_consolidate.go index 163e725adc83..ecac3684e194 100644 --- a/op-node/rollup/derive/engine_consolidate.go +++ b/op-node/rollup/derive/engine_consolidate.go @@ -47,7 +47,7 @@ func AttributesMatchBlock(attrs *eth.PayloadAttributes, parentHash common.Hash, return nil } -func checkWithdrawalsMatch(attrWithdrawals *eth.Withdrawals, blockWithdrawals *eth.Withdrawals) error { +func checkWithdrawalsMatch(attrWithdrawals *types.Withdrawals, blockWithdrawals *types.Withdrawals) error { if attrWithdrawals == nil && blockWithdrawals == nil { return nil } @@ -67,7 +67,7 @@ func checkWithdrawalsMatch(attrWithdrawals *eth.Withdrawals, blockWithdrawals *e for idx, expected := range *attrWithdrawals { actual := (*blockWithdrawals)[idx] - if expected != actual { + if *expected != *actual { return fmt.Errorf("expected withdrawal %d to be %v, actual %v", idx, expected, actual) } } diff --git a/op-node/rollup/derive/engine_consolidate_test.go b/op-node/rollup/derive/engine_consolidate_test.go index 27993587b636..9b54d89ea604 100644 --- a/op-node/rollup/derive/engine_consolidate_test.go +++ b/op-node/rollup/derive/engine_consolidate_test.go @@ -3,14 +3,15 @@ package derive import ( "testing" - "github.com/ethereum-optimism/optimism/op-service/eth" "github.com/stretchr/testify/require" + + "github.com/ethereum/go-ethereum/core/types" ) func TestWithdrawalsMatch(t *testing.T) { tests := []struct { - attrs *eth.Withdrawals - block *eth.Withdrawals + attrs *types.Withdrawals + block *types.Withdrawals shouldMatch bool }{ { @@ -19,36 +20,36 @@ func TestWithdrawalsMatch(t *testing.T) { shouldMatch: true, }, { - attrs: ð.Withdrawals{}, + attrs: &types.Withdrawals{}, block: nil, shouldMatch: false, }, { attrs: nil, - block: ð.Withdrawals{}, + block: &types.Withdrawals{}, shouldMatch: false, }, { - attrs: ð.Withdrawals{}, - block: ð.Withdrawals{}, + attrs: &types.Withdrawals{}, + block: &types.Withdrawals{}, shouldMatch: true, }, { - attrs: ð.Withdrawals{ + attrs: &types.Withdrawals{ { Index: 1, }, }, - block: ð.Withdrawals{}, + block: &types.Withdrawals{}, shouldMatch: false, }, { - attrs: ð.Withdrawals{ + attrs: &types.Withdrawals{ { Index: 1, }, }, - block: ð.Withdrawals{ + block: &types.Withdrawals{ { Index: 2, }, diff --git a/op-program/client/l2/engineapi/test/l2_engine_api_tests.go b/op-program/client/l2/engineapi/test/l2_engine_api_tests.go index 3ec84e21e274..2bdc784886e4 100644 --- a/op-program/client/l2/engineapi/test/l2_engine_api_tests.go +++ b/op-program/client/l2/engineapi/test/l2_engine_api_tests.go @@ -57,9 +57,9 @@ func RunEngineAPITests(t *testing.T, createBackend func(t *testing.T) engineapi. nextBlockTime := eth.Uint64Quantity(genesis.Time + 1) - var w *eth.Withdrawals + var w *types.Withdrawals if api.backend.Config().IsCanyon(uint64(nextBlockTime)) { - w = ð.Withdrawals{} + w = &types.Withdrawals{} } result, err := api.engine.ForkchoiceUpdatedV2(api.ctx, ð.ForkchoiceState{ @@ -111,9 +111,9 @@ func RunEngineAPITests(t *testing.T, createBackend func(t *testing.T) engineapi. t.Run("RejectInvalidBlockHash", func(t *testing.T) { api := newTestHelper(t, createBackend) - var w *eth.Withdrawals + var w *types.Withdrawals if api.backend.Config().IsCanyon(uint64(0)) { - w = ð.Withdrawals{} + w = &types.Withdrawals{} } // Invalid because BlockHash won't be correct (among many other reasons) @@ -385,9 +385,9 @@ func (h *testHelper) startBlockBuilding(head *types.Header, newBlockTimestamp et } canyonTime := h.backend.Config().CanyonTime - var w *eth.Withdrawals + var w *types.Withdrawals if canyonTime != nil && *canyonTime <= uint64(newBlockTimestamp) { - w = ð.Withdrawals{} + w = &types.Withdrawals{} } result, err := h.engine.ForkchoiceUpdatedV2(h.ctx, ð.ForkchoiceState{ diff --git a/op-service/eth/ssz.go b/op-service/eth/ssz.go index 27d2f6259e19..80d5041f3be7 100644 --- a/op-service/eth/ssz.go +++ b/op-service/eth/ssz.go @@ -7,6 +7,8 @@ import ( "io" "math" "sync" + + "github.com/ethereum/go-ethereum/core/types" ) type BlockVersion int @@ -180,16 +182,16 @@ func (payload *ExecutionPayload) MarshalSSZ(w io.Writer) (n int, err error) { offset += transactionSize // dyanmic value 3: Withdrawals if payload.Withdrawals != nil { - marshalWithdrawals(buf[offset:], payload.Withdrawals) + marshalWithdrawals(buf[offset:], *payload.Withdrawals) } return w.Write(buf) } -func marshalWithdrawals(out []byte, withdrawals *Withdrawals) { +func marshalWithdrawals(out []byte, withdrawals types.Withdrawals) { offset := uint32(0) - for _, withdrawal := range *withdrawals { + for _, withdrawal := range withdrawals { binary.LittleEndian.PutUint64(out[offset:offset+8], withdrawal.Index) offset += 8 binary.LittleEndian.PutUint64(out[offset:offset+8], withdrawal.Validator) @@ -305,14 +307,14 @@ func (payload *ExecutionPayload) UnmarshalSSZ(version BlockVersion, scope uint32 if err != nil { return fmt.Errorf("failed to unmarshal withdrawals list: %w", err) } - payload.Withdrawals = withdrawals + payload.Withdrawals = &withdrawals } return nil } -func unmarshalWithdrawals(in []byte) (*Withdrawals, error) { - result := &Withdrawals{} +func unmarshalWithdrawals(in []byte) (types.Withdrawals, error) { + result := types.Withdrawals{} // empty list by default, intentionally non-nil if len(in)%withdrawalSize != 0 { return nil, errors.New("invalid withdrawals data") @@ -327,7 +329,7 @@ func unmarshalWithdrawals(in []byte) (*Withdrawals, error) { offset := 0 for i := 0; i < withdrawalCount; i++ { - withdrawal := Withdrawal{} + withdrawal := &types.Withdrawal{} withdrawal.Index = binary.LittleEndian.Uint64(in[offset : offset+8]) offset += 8 @@ -341,7 +343,7 @@ func unmarshalWithdrawals(in []byte) (*Withdrawals, error) { withdrawal.Amount = binary.LittleEndian.Uint64(in[offset : offset+8]) offset += 8 - *result = append(*result, withdrawal) + result = append(result, withdrawal) } return result, nil diff --git a/op-service/eth/ssz_test.go b/op-service/eth/ssz_test.go index d1a2e2fafca4..79a4c0288839 100644 --- a/op-service/eth/ssz_test.go +++ b/op-service/eth/ssz_test.go @@ -7,10 +7,12 @@ import ( "math" "testing" - "github.com/ethereum/go-ethereum/common" "github.com/google/go-cmp/cmp" "github.com/holiman/uint256" "github.com/stretchr/testify/require" + + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/core/types" ) // FuzzExecutionPayloadUnmarshal checks that our SSZ decoding never panics @@ -138,9 +140,9 @@ func FuzzExecutionPayloadMarshalUnmarshalV2(f *testing.F) { } wCount = wCount % maxWithdrawalsPerPayload - withdrawals := make(Withdrawals, wCount) + withdrawals := make(types.Withdrawals, wCount) for i := 0; i < int(wCount); i++ { - withdrawals[i] = Withdrawal{ + withdrawals[i] = &types.Withdrawal{ Index: a, Validator: b, Address: common.BytesToAddress(data[:20]), @@ -228,10 +230,10 @@ func TestOPB04(t *testing.T) { tests := []struct { version BlockVersion - withdrawals *Withdrawals + withdrawals *types.Withdrawals }{ {BlockV1, nil}, - {BlockV2, &Withdrawals{}}, + {BlockV2, &types.Withdrawals{}}, } for _, test := range tests { @@ -246,7 +248,7 @@ func TestOPB04(t *testing.T) { } -func createPayloadWithWithdrawals(w *Withdrawals) *ExecutionPayload { +func createPayloadWithWithdrawals(w *types.Withdrawals) *ExecutionPayload { return &ExecutionPayload{ ParentHash: common.HexToHash("0x123"), FeeRecipient: common.HexToAddress("0x456"), @@ -267,8 +269,8 @@ func createPayloadWithWithdrawals(w *Withdrawals) *ExecutionPayload { } func TestMarshalUnmarshalWithdrawals(t *testing.T) { - emptyWithdrawal := &Withdrawals{} - withdrawals := &Withdrawals{ + emptyWithdrawal := &types.Withdrawals{} + withdrawals := &types.Withdrawals{ { Index: 987, Validator: 654, @@ -276,18 +278,18 @@ func TestMarshalUnmarshalWithdrawals(t *testing.T) { Amount: 321, }, } - maxWithdrawals := make(Withdrawals, maxWithdrawalsPerPayload) + maxWithdrawals := make(types.Withdrawals, maxWithdrawalsPerPayload) for i := 0; i < maxWithdrawalsPerPayload; i++ { - maxWithdrawals[i] = Withdrawal{ + maxWithdrawals[i] = &types.Withdrawal{ Index: 987, Validator: 654, Address: common.HexToAddress("0x898"), Amount: 321, } } - tooManyWithdrawals := make(Withdrawals, maxWithdrawalsPerPayload+1) + tooManyWithdrawals := make(types.Withdrawals, maxWithdrawalsPerPayload+1) for i := 0; i < maxWithdrawalsPerPayload+1; i++ { - tooManyWithdrawals[i] = Withdrawal{ + tooManyWithdrawals[i] = &types.Withdrawal{ Index: 987, Validator: 654, Address: common.HexToAddress("0x898"), @@ -299,7 +301,7 @@ func TestMarshalUnmarshalWithdrawals(t *testing.T) { name string version BlockVersion hasError bool - withdrawals *Withdrawals + withdrawals *types.Withdrawals }{ {"ZeroWithdrawalsSucceeds", BlockV2, false, emptyWithdrawal}, {"ZeroWithdrawalsFailsToDeserialize", BlockV1, true, emptyWithdrawal}, diff --git a/op-service/eth/types.go b/op-service/eth/types.go index 41cba3da3309..f3bdf8be9df4 100644 --- a/op-service/eth/types.go +++ b/op-service/eth/types.go @@ -10,7 +10,6 @@ import ( "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/rlp" "github.com/ethereum/go-ethereum/trie" "github.com/holiman/uint256" ) @@ -143,7 +142,8 @@ type ExecutionPayload struct { ExtraData BytesMax32 `json:"extraData"` BaseFeePerGas Uint256Quantity `json:"baseFeePerGas"` BlockHash common.Hash `json:"blockHash"` - Withdrawals *Withdrawals `json:"withdrawals,omitempty"` + // nil if not present, pre-shanghai + Withdrawals *types.Withdrawals `json:"withdrawals,omitempty"` // Array of transaction objects, each object is a byte list (DATA) representing // TransactionType || TransactionPayload or LegacyTransaction as defined in EIP-2718 Transactions []Data `json:"transactions"` @@ -237,7 +237,7 @@ func BlockAsPayload(bl *types.Block, canyonForkTime *uint64) (*ExecutionPayload, } if canyonForkTime != nil && uint64(payload.Timestamp) >= *canyonForkTime { - payload.Withdrawals = &Withdrawals{} + payload.Withdrawals = &types.Withdrawals{} } return payload, nil @@ -251,7 +251,7 @@ type PayloadAttributes struct { // suggested value for the coinbase field of the new payload SuggestedFeeRecipient common.Address `json:"suggestedFeeRecipient"` // Withdrawals to include into the block -- should be nil or empty depending on Shanghai enablement - Withdrawals *Withdrawals `json:"withdrawals,omitempty"` + Withdrawals *types.Withdrawals `json:"withdrawals,omitempty"` // Transactions to force into the block (always at the start of the transactions list). Transactions []Data `json:"transactions,omitempty"` // NoTxPool to disable adding any transactions from the transaction-pool. @@ -317,25 +317,3 @@ type SystemConfig struct { GasLimit uint64 `json:"gasLimit"` // More fields can be added for future SystemConfig versions. } - -// Withdrawal represents a validator withdrawal from the consensus layer. -// https://github.com/ethereum/consensus-specs/blob/dev/specs/capella/beacon-chain.md#withdrawal -type Withdrawal struct { - Index uint64 `json:"index"` // monotonically increasing identifier issued by consensus layer - Validator uint64 `json:"validatorIndex"` // index of validator associated with withdrawal - Address common.Address `json:"address"` // target address for withdrawn ether - Amount uint64 `json:"amount"` // value of withdrawal in Gwei -} - -// Withdrawals implements DerivableList for withdrawals. -type Withdrawals []Withdrawal - -// Len returns the length of s. -func (s Withdrawals) Len() int { return len(s) } - -// EncodeIndex encodes the i'th withdrawal to w. Note that this does not check for errors -// because we assume that *Withdrawal will only ever contain valid withdrawals that were either -// constructed by decoding or via public API in this package. -func (s Withdrawals) EncodeIndex(i int, w *bytes.Buffer) { - _ = rlp.Encode(w, s[i]) -} diff --git a/op-service/sources/testdata/data/blocks/post-shanghai-bad-receipts_data.json b/op-service/sources/testdata/data/blocks/post-shanghai-bad-receipts_data.json new file mode 100644 index 000000000000..11c81ca24590 --- /dev/null +++ b/op-service/sources/testdata/data/blocks/post-shanghai-bad-receipts_data.json @@ -0,0 +1 @@ +{"baseFeePerGas":"0x3fb7c357","difficulty":"0x0","extraData":"0x","gasLimit":"0x1c9c380","gasUsed":"0x18f759","hash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","logsBloom":"0x020010404000001a0000021000000080001100410000100001000010040200980220400000008806200200000100000000000000000000008000000400042000000050000040000112080808800002044000040004042008800480002000000000000002020020000042002400000820000080040000000010200010020010100101212050000008000000008000001010200c80000112010000438040020400000000202400000000002002a0210402000622010000000001700144000040000000002204000000c000410105024010000808000000002004002000000261000000822200200800881000000012500400400000000000000040010000800000","miner":"0x000095e79eac4d76aab57cb2c1f091d553b36ca0","mixHash":"0x5b53dc49cbab268ef9950b1d81b5e36a1b2f1b97aee1b7ff6e4db0e06c29a8b0","nonce":"0x0000000000000000","number":"0x84161e","parentHash":"0x72d92c1498e05952988d4e79a695928a6bcbd37239f8a1734051263b4d3504b8","receiptsRoot":"0xaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa","sha3Uncles":"0x1dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347","size":"0x2a51","stateRoot":"0xc56738518b2c7854a640ae25996d2211c9ef0dd2e4dd9e59e9d9cacef39622da","timestamp":"0x64110a5c","totalDifficulty":"0xa4a470","transactions":[{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x1b7aa44088a0ea95bdc65fef6e5071e946bf7d8f","gas":"0x5208","gasPrice":"0x4159f01735","hash":"0x1e8f148a9aea7d8d16ea6e9446723b8f262e8bcd89c7c961d52046ebd43b4598","input":"0x","nonce":"0x279ad","to":"0xa21765a03dd41e2783696d314f235f4d520f6cac","transactionIndex":"0x0","value":"0x3782dace9d90000","type":"0x0","chainId":"0x5","v":"0x2e","r":"0xf9eca8b74ccaec5f4a83e981baeb8602f38d0271179c592b4a18f3866316a367","s":"0x350a9d26718d1fa9dc4f268f9c0e18fb96dbc037816ca8fc20f33f1a76f9f501"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x4d9a852e6aecd3a6e87fece2ca109780e45e6f2d","gas":"0x160ab","gasPrice":"0xd4babc57","maxFeePerGas":"0x11407d03e","maxPriorityFeePerGas":"0x9502f900","hash":"0xab5c870f4c367012bd763172afbfbe68fbf35336a66ae41aff3f2c9dbf4ea3f8","input":"0x60566037600b82828239805160001a607314602a57634e487b7160e01b600052600060045260246000fd5b30600052607381538281f3fe73000000000000000000000000000000000000000030146080604052600080fdfea26469706673582212200c479c99d58d7fbee5ee3aef5001431073a3d465bdfa46cf747a7c0b9289607864736f6c634300080d0033","nonce":"0xafd","to":null,"transactionIndex":"0x1","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x48f6acf4ac4371eb96ffca8cdef5b7704ea8c68a631d1c02927036d4ce92567e","s":"0x1f12261ddde63fd3da2ed9ea1bcbb4b0f25af898248c805a9d4b0a6e42caf9c8","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0xce297b30e56bf2f6142c4048fbae1cb8a756fefd","gas":"0x557300","gasPrice":"0xd4babc57","maxFeePerGas":"0xfabdd183","maxPriorityFeePerGas":"0x9502f900","hash":"0xa81fd92b2d0f0bbd3cc355f869cca3243c98c5e2641db9ecf3eeabb3b13bff6a","input":"0x7bbaf1ea0000000000000000000000000000000000000000000000000000000000000005000000000000000000000000000000000000000000000000000000000000004000000000000000000000000000000000000000000000000000000000000000200000000000000000000000000000000000000000000000000000000000000000","nonce":"0xad92","to":"0x4c0ce02c1219ce5d2afffba97e484272a4637b49","transactionIndex":"0x2","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0xd11b842a5f7b0e020470e764b443fafc4044327bca8dbb37131425359abace2","s":"0x1009867573946fb0a4a35339a6eaaece4d0a168b2346539076e6319e9a466b1","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x47a916d0b694c95bc0c09953c1a6e22c8867be32","gas":"0x5208","gasPrice":"0x991ff257","maxFeePerGas":"0xb2fea055","maxPriorityFeePerGas":"0x59682f00","hash":"0xa92c7b720c08c83f1a0ed7e4c163200e30a3a8c03fcc5a51e685ea20cd0cb577","input":"0x","nonce":"0x2","to":"0xf955dfe99bbfd06a8be208274cd654ecfb66a45e","transactionIndex":"0x3","value":"0x11c37937e08000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xeecd431225215c3ddc2e13cf16fa29f9a2ab1afbaa02988f3ceac27416edf0f5","s":"0x6494062330a9d509f7c778cb45cbcc9932e5f9b4ec94a1ec315f579f8d973b7b","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x672d214fe4db218defae698085a3ccefc773dfa9","gas":"0xf4f6","gasPrice":"0x991ff257","maxFeePerGas":"0xb2fea055","maxPriorityFeePerGas":"0x59682f00","hash":"0x6921b429ad2ec1e97d3457049ad2e893b5a0349beba47ca1c74a9540af75347a","input":"0xa1ddc546000000000000000000000000000000000000000000000000000000000000118f","nonce":"0x8e","to":"0xdef65f8e2f62bde60cd86ad3b4ad7fa6741cf175","transactionIndex":"0x4","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x4732ad21401b509bebb947a8566127fba1e245eb5a262ddc522d6867ab80567e","s":"0x2153013617fefdb06db0a7316d8883056441d33c2abf5ed6291fa0e3cc5c3a2f","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x0b12f7f996579fe7d9302d2574910121c214ee15","gas":"0x56a8","gasPrice":"0x991ff257","maxFeePerGas":"0xb2fea055","maxPriorityFeePerGas":"0x59682f00","hash":"0xf776b2da0b835dde05d0d8b76fd19385d61e7055036cf637f804b36dc94f2384","input":"0x71ec5c05aa669c4922569c1d33f7a81aaa21813800000000000000000000000013a0c5930c028511dc02665e7285134b6d11a5f40000000000000000000000000b12f7f996579fe7d9302d2574910121c214ee150000000000000000000000000000000000000000000000000000000000000000","nonce":"0xe","to":"0x7c125c1d515b8945841b3d5144a060115c58725f","transactionIndex":"0x5","value":"0x2386f26fc10000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xc4505d4c46ef9a0103178e5ef4fbc857114ae5c4686b68b74d07c3ee3a86e575","s":"0x74db34fd675bea7d2ee74f14fd8709cb88c7d22c2267d12f2c671b5ebc180bd5","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0xd36aac0c9676e984d72823fb662ce94d3ab5e551","gas":"0xff212","gasPrice":"0x991ff257","maxFeePerGas":"0xd86d063e","maxPriorityFeePerGas":"0x59682f00","hash":"0x9a08d899cd14ebb930ed59fa774afdb88a22615b3a931e930931ea54d26dc0bc","input":"0x49160658babdf216946a22b96c966ceeba750f14807726306ebeed500a2621a7481df010000000000000000000000000000000000000000000000000000000000000008000000000000000000000000000000000000000000000000000000000000000c0000000000000000000000000000000000000000000000000000000000000012000000000000000000000000000000000000000000000000000000000000000084d6f6f6e6265616d000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000002a3078434133633746363939643335333137316438434643364236334431433764333841333233373238300000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000c0b616f73c8d00641109f0000054f967b8a135e44e233da070ea88c5e4a53bacac0000000000000000000000000000000000000000000000000000000000000002000000000000000000000000c6a3a17399d88e5dfd2de4fa41dcccade6a2d2170000000000000000000000008676147503f761e44b85df01811edd52be01b56c000000000000000000000000000000000000000000000000d02ab486cedc00000000000000000000000000000000000000000000000000000de0b6b3a7640000","nonce":"0x7121","to":"0x2d276bd64fe8643a1d76b2e9530c753423e972c1","transactionIndex":"0x6","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0xce452361d331baf2ac86947a1b3b454b3e842923e1f39139b33a572845dd5d7c","s":"0x388126881782af5a146ce8409f1e3a2c9a4cadcfe9a2c1d27258af6e4eb5cec1","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0xb83a48ee322989a0a4f838cb207abc28ec021b73","gas":"0x326ff","gasPrice":"0x991ff257","maxFeePerGas":"0x5c3362b9c","maxPriorityFeePerGas":"0x59682f00","hash":"0x0fe0d97e25d5eb11a33a3e8278584c3780941fc2675bdf8fc547cee3d1fd3b17","input":"0xa4110dbe190e65a7fe0ad5acd305c6446a02cb39de30f382000fe67927434063bf4c4b2ca86b3ee6d039dc7a0268f9e1c39266c3f12a752a7eb72f0895dedf62291fd292","nonce":"0x12e","to":"0x96e6192eeaf7bb308f79fb5017a9085754b9e12a","transactionIndex":"0x7","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x46819bb540fa459a5a0cbf041a0286507d597fc8958a6af8b9ce193f00005238","s":"0x31495c58bf4b74925f870c59635912681bee947b1cb4b08cc19f22421b1a10ab","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0xb83a48ee322989a0a4f838cb207abc28ec021b73","gas":"0x3270b","gasPrice":"0x991ff257","maxFeePerGas":"0x5c3362b9c","maxPriorityFeePerGas":"0x59682f00","hash":"0xef47a60f57f177a683c723c658137efab66d311e1c5abbc4d74f653535144d03","input":"0xa4110dbe18eba3e815ec7fcb03a8eea0487209c3253024d1dc4f36f775a43fdd3e409b74c0a5c88e54f39b88db6f185a41431b204120d3e66c9b08b86f5e8382623327e5","nonce":"0x12f","to":"0x96e6192eeaf7bb308f79fb5017a9085754b9e12a","transactionIndex":"0x8","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x4d7e11fe3914763a2735493a62b71ae0028e75a750a8b06407996cf478e273c8","s":"0x60c18fdf948721f96fdc2efbf694910122e82191c7a40a5caab247c06ab7f3cf","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x7712a1f96b3f37c3dd88a22d4f7a7977b937fe62","gas":"0x1ff06","gasPrice":"0x991ff257","maxFeePerGas":"0xcf662006","maxPriorityFeePerGas":"0x59682f00","hash":"0xe23a5b35faae5335adc5aca38c5d633b00438b798c2053104b8df48406c9b141","input":"0x3ff9751b0000000000000000000000000000000000000000000000000000000000000034000000000000000000000000000000000000000000000000000000000000003500000000000000000000000000000000000000000000000000000000000000a000000000000000000000000000000000000000000000000000000000000001000000000000000000000000000000000000000000000000000000000000000160000000000000000000000000000000000000000000000000000000000000002e516d566e75374a51566f445271536748427a72615970374879373848774a744c4669366e5546436f775447647a70000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000002e516d5439716b3343525962464457704446596541763854384831676e6f6e67774b6868354a36384e4c6b4c6972360000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000041c9f31b989aaa745aecc563139f89c8ab3ab492e725d1e0305b7d602fcbf9293c463f0d2e4f5fb0c9efb54eb2cde9af10ebc9c3676f2334007291ffa8efb40adb1c00000000000000000000000000000000000000000000000000000000000000","nonce":"0x16","to":"0xc3b87ea584cb9cb5a4523e2bed16a34bf7746317","transactionIndex":"0x9","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xcc693a9544394653e1f393f02635e2c1f78b5e6c26c4d00a867bcf4adc2bc09e","s":"0x23f4b71b773c19f77cf251b2aeea03f2696bddd847d1f6d75acd2fbea6a543e4","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x12bbafcfa06423d87bed591a64c1a1f34791e90a","gas":"0x1b2e020","gasPrice":"0x8f0d3617","maxFeePerGas":"0x1cb3abd5a","maxPriorityFeePerGas":"0x4f5572c0","hash":"0xd8cea4ba619b317bc05d58534af73beec6c2548b31b24d4dc61c9bbd29cfa17a","input":"0x","nonce":"0x4d","to":"0x12bbafcfa06423d87bed591a64c1a1f34791e90a","transactionIndex":"0xa","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x8dae28608985d6ab47cbe122ee88972e38664cede30978f41944f347bad86e5c","s":"0x17c2898b39b7e3b874dc78e7df3fcff68f4d715385c632aad4bdf6e62505651c","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x6999c36394dd5d3d3295af63a4736ad6bd279448","gas":"0x14d94","gasPrice":"0x7b528d57","maxFeePerGas":"0x7df743e0","maxPriorityFeePerGas":"0x3b9aca00","hash":"0x79a4b9d90b02c768baaad305f266281213cc75062cbe99a13222cc0c4b509498","input":"0x6a7612020000000000000000000000009008d19f58aabd9ed0d60971565aa8510560ab4100000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000140000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000022000000000000000000000000000000000000000000000000000000000000000a4ec6cb13f00000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000000000000000000001000000000000000000000000000000000000000000000000000000000000003826d8d7371becafc0afedc3fe85b699b87f4357d55ec39ce83c510ed7d70ff514dce8236351f4627ccec6a7d9a67bc6fc2ec17ca76411113c00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000410000000000000000000000006999c36394dd5d3d3295af63a4736ad6bd27944800000000000000000000000000000000000000000000000000000000000000000100000000000000000000000000000000000000000000000000000000000000","nonce":"0x13","to":"0xdce8236351f4627ccec6a7d9a67bc6fc2ec17ca7","transactionIndex":"0xb","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x669487011643510f7512ec15b07d74efb853ba4ad9fe9c7e9850ccf24e3d4c53","s":"0x2191f9ee81087e1df833e69d6129ec330ed4c99be2031060834e22c589fcf79b","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x08ebc052e6353b4d8392113f0f2c5decd691ddcd","gas":"0x1b2e020","gasPrice":"0x7b528d57","maxFeePerGas":"0xf6e4d9c4","maxPriorityFeePerGas":"0x3b9aca00","hash":"0x6790a3bbddbeb21fcb736a59b3775755051c3a6344d8390cf8ca27f2e8a814f0","input":"0x6871ee40","nonce":"0x4","to":"0xb21b333c40354393f18de57f2d0a0fc21f270482","transactionIndex":"0xc","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0xa7a58ab858e5dd87b9a74d550e5716ac40ddefd1c21d5baa51e542f4f941d73e","s":"0x1fd0cb1563df0368bd332717d7c8210187cd1ac2ff44f61f5242fc02eaa0c2fe","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x08ebc052e6353b4d8392113f0f2c5decd691ddcd","gas":"0x12c36","gasPrice":"0x7b528d57","maxFeePerGas":"0xe596c292","maxPriorityFeePerGas":"0x3b9aca00","hash":"0x87ec7ace5442db252b5751ffddd38dcb04b088d36b6b0e526ff25607a4293c81","input":"0xcb40cbfd","nonce":"0x5","to":"0xb21b333c40354393f18de57f2d0a0fc21f270482","transactionIndex":"0xd","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x8e4bd40195bdfaa868af97dbf22e0b74295d95564a8381bc9bcfe1af3cde2cdb","s":"0x62057dc87c8b2b197d12c648233aab4134b7c2a2f65db89a49728564f3d0864d","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x08ebc052e6353b4d8392113f0f2c5decd691ddcd","gas":"0x12c36","gasPrice":"0x7b528d57","maxFeePerGas":"0xe596c292","maxPriorityFeePerGas":"0x3b9aca00","hash":"0x40cb487ecffda94f97ce7fc0f7163f2f024235df2c8291169edc80dac063e6d0","input":"0xcb40cbfd","nonce":"0x6","to":"0xb21b333c40354393f18de57f2d0a0fc21f270482","transactionIndex":"0xe","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x993c4386ffc2bd63a48dd31cf5f65aa355cba3c063b08778b328d748937098e5","s":"0x203b1faf595628dd78e760bcc1740065625abb6cf00b1b17224e10fa6b424e43","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0xafc7682751d3654f34958e3fb660971bf5918367","gas":"0x1b2e020","gasPrice":"0x7b528d57","maxFeePerGas":"0xefb40664","maxPriorityFeePerGas":"0x3b9aca00","hash":"0xb76bb3d88c9b30d927c45ccfcf8d5b0054411ac8501ad588822a7d04690cccf6","input":"0x40c10f19000000000000000000000000afc7682751d3654f34958e3fb660971bf59183670000000000000000000000000000000000000000000000008ac7230489e80000","nonce":"0x20","to":"0x84ced17d95f3ec7230baf4a369f1e624ae60090d","transactionIndex":"0xf","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x78e9d29c606f6a61a1428ed7326df05210d802c3061f3b469847cc461eb48025","s":"0x517afe5a21175f186233ae7a6c090cd33e028a77e5854e98eb2fa4c9284430e7","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x12bbafcfa06423d87bed591a64c1a1f34791e90a","gas":"0xcafe","gasPrice":"0x7b528d57","maxFeePerGas":"0x16ba45bce","maxPriorityFeePerGas":"0x3b9aca00","hash":"0x798ebe823209869347c08bd81e04fbf60e9bdfe44b1cc923215182d0cf3d4edb","input":"0x40c10f1900000000000000000000000012bbafcfa06423d87bed591a64c1a1f34791e90a0000000000000000000000000000000000000000000000000000000000000032","nonce":"0x4e","to":"0xd25d682189afa7ed0c2811195b3f3d07616927bc","transactionIndex":"0x10","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0xbe2aa95f4c32188c9ecbae97286ab983f7cc5a7017d46c404f8eb80b55e2dc13","s":"0x55d8968f509f51633f49260178941fd5a3db999e1995206348d7a3f05152b3d5","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x12bbafcfa06423d87bed591a64c1a1f34791e90a","gas":"0xcafe","gasPrice":"0x7b528d57","maxFeePerGas":"0xc6132c0c","maxPriorityFeePerGas":"0x3b9aca00","hash":"0xbe68a7e02725f799a65ebb069ccc83a014ac7c40e4119bf7c220a2f6ddfee295","input":"0x40c10f1900000000000000000000000012bbafcfa06423d87bed591a64c1a1f34791e90a0000000000000000000000000000000000000000000000000000000000000064","nonce":"0x4f","to":"0xd25d682189afa7ed0c2811195b3f3d07616927bc","transactionIndex":"0x11","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x1750ed45d0d2d048a61dcc3ff3e7ad0cbd395b77698c4f2eeb6baaea4c966d96","s":"0xe646e3a87b6dc50439468bcfd24cbf12a6f95b2762f1527c97a7d1b7f03530e","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0xd0c0338baac9ce09c1d1caab88b4360507061e37","gas":"0x1b2e020","gasPrice":"0x7b528d57","maxFeePerGas":"0x1918167aa","maxPriorityFeePerGas":"0x3b9aca00","hash":"0xc90c3a72efe81331727fcce4b5bd4906066da314ca9a0b44023a6b09ea7e8114","input":"0x4e71d92d","nonce":"0xe7","to":"0x0137d796936e199b917b93118eb4e002508a48bd","transactionIndex":"0x12","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x22222ce8fe753082c8e5dc7478fa7b87db9c8d0fb070ebc88ab015db13fd6bb","s":"0x6dceadecf6f6a3ddc433a547dba6d5fcdfb7d006ce09eeae1c06c1828ef9fff9","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x8bd834cc1dd17d1a2f7eb6140c3dd5ab51d0f338","gas":"0x1b2e020","gasPrice":"0x7b528d57","maxFeePerGas":"0x1a4019792","maxPriorityFeePerGas":"0x3b9aca00","hash":"0x619a6cbd43cde074d314c19623bd66d9fb1e13c158d7138775236f798dc1245e","input":"0x1da1db5e00000000000000000000000000000000000000000000000000001b5ec4ff740d","nonce":"0x10","to":"0xbad0009f7a8e796e3f975dc1d6ab3e3c237b857c","transactionIndex":"0x13","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x733c496e8c84f78d113bc8b5ad6d04a133bb10bbcecf0db93d97eeedfa3d984a","s":"0x6627c16aa7dea27db44c0f0641c30ffd544b9b9b1c299dc60611e1e01c714ee8","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x8bd834cc1dd17d1a2f7eb6140c3dd5ab51d0f338","gas":"0x11df0","gasPrice":"0x7b528d57","maxFeePerGas":"0x1d3fbccb8","maxPriorityFeePerGas":"0x3b9aca00","hash":"0xca5a56cd77b9e5b0e79020cc6346edf205bc11e901984d805125f28c2e6686e6","input":"0xd7c01032000000000000000000000000356f24c0f3187a6dcb949e4a60aa0a74a8a0327e0000000000000000000000000f8bacfae87aee49a2eec86df0c6b958ec3fd4fd0000000000000000000000000f8bacfae87aee49a2eec86df0c6b958ec3fd4fd","nonce":"0x11","to":"0xbad0009f7a8e796e3f975dc1d6ab3e3c237b857c","transactionIndex":"0x14","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x4aa872c1a7da70126917604193717575b91407f82449bca956d4089b90cd902","s":"0x4b5f788c68024f8958f66e8c768a8e0f65ef297c4ffd420a715258b0a55cb0a0","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x9bedd66b05712a1add7d1fec899b8641b3ecc863","gas":"0x1b2e020","gasPrice":"0x7b528d57","maxFeePerGas":"0x1e706c8d2","maxPriorityFeePerGas":"0x3b9aca00","hash":"0x999c9ddeed67c6ef6fbf02a6e977a6c1b68e18d24814e51643c7157b87a43e0a","input":"0xa8b78ed0","nonce":"0x3","to":"0xf6c2afb18b85175e5ffb28f20f007c047ce28592","transactionIndex":"0x15","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xf1ca074104ee0b6a67223afd797654b8f21060453f7662dd7be79de938c396da","s":"0x4c9a4e56f0ad21e2dd8d1116c7a3e41ea2c6126853772f7897bd959e81e2e28a","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x9bedd66b05712a1add7d1fec899b8641b3ecc863","gas":"0x2dc6c0","gasPrice":"0xd4babc57","maxFeePerGas":"0x21c13e710","maxPriorityFeePerGas":"0x9502f900","hash":"0x47c8f5d0b3778e4c34eba7fcc356fa04a5afd954ccf484728e72c002764dd3c4","input":"0xf688bcfb0000000000000000000000000000000000000000000000000000000005f5e100","nonce":"0x4","to":"0xf6c2afb18b85175e5ffb28f20f007c047ce28592","transactionIndex":"0x16","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x970982425d18b9949141bbabb5c5230fa168626ce210a97f3856d64700fc0eeb","s":"0x6fa6d0aea6ae48783d80c1e51cda47e11c28b90a5497579b93d7f242d786f67e","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x4b596cebd02c35aa99878eb9ecb24d6f5ca5f865","gas":"0x5208","gasPrice":"0x68c88f79","maxFeePerGas":"0x68c88f79","maxPriorityFeePerGas":"0x68c88f79","hash":"0x396797ae0ebcdb72ff1f96fd08b6128f78acc7417353f142f1a5facd425a33e6","input":"0x","nonce":"0x1","to":"0x375980b8e1a417620ba9b18e5a9e0b17dccbac9d","transactionIndex":"0x17","value":"0x1c9a406c58878","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xcb2a7b106097b2f2a61d3a048054a5175c386b1dcdd5b6fcd7849dcac7c32a66","s":"0x10aef76e98276c47470f04a599b81c260ace9d2f6adc95755c6d22d859e805a2","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x6f4aa3bc9732372daa288beb9b2516374ace5786","gas":"0x1b2e020","gasPrice":"0x61010bda","maxFeePerGas":"0x61010bda","maxPriorityFeePerGas":"0x61010bda","hash":"0x454aa43d6546a6f62246826c16b7a49c6c704238c18802ef0d659922f23a573c","input":"0xe99a3f8000000000000000000000000000000000000000000000000000000000000000800000000000000000000000000000000000000000000000000000000000000480000000000000000000000000000000000000000000000000000000000000050000000000000000000000000000000000000000000000000000000000000008a000000000000000000000000058ce7eb44ae575fb3b3974861c1138dad3c5aee40000000000000000000000000000000000000000000000000000000000000120000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000001e06defd69964a6ce467b87c64627a7a2be4be139b504757c625eb6114ec166557d000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000641a269023d235ef0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000002a000000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000002b05699353b600008ae85d840000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000000000000000000020000000000000000000000000b4fbf271143f4fbf7b91a5ded31805e42b2208d600000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000000000000000000001f63c28250000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000000000000000000020000000000000000000000000def65f8e2f62bde60cd86ad3b4ad7fa6741cf17500000000000000000000000000000000000000000000000000000000000001400000000000000000000000000000000000000000000000000000000000000020000000000000000000000000000000000000000000000000000000000000006000000000000000000000000000000000000000000000000000000000000000800000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000020000000000000000000000006e512435020ac228a15543236b1f46656f98136300000000000000000000000000000000000000000000000000000000000000fa000000000000000000000000d6ffd79b52a587a0a9941a61f4e6cb0d386d5458000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000413e24433da3efc4c3cc305339a7b77a1b31abb4e4d4967e97e42aed1dd0bf98936f43035f40bc80f6f6e3a9cfb43524a1171ee7f21e9ac2fa586d33a31f10619d1c000000000000000000000000000000000000000000000000000000000000000000000000000000000000006f4aa3bc9732372daa288beb9b2516374ace5786000000000000000000000000000000000000000000000000000000000000012000000000000000000000000058ce7eb44ae575fb3b3974861c1138dad3c5aee400000000000000000000000000000000000000000000000000000000000002000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000641a269023d235ef0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000002c00000000000000000000000000000000000000000000000000000000000000040000000000000000000000000000000000000000000000000000000000000000173ad21460000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000000000000000000040000000000000000000000000def65f8e2f62bde60cd86ad3b4ad7fa6741cf1750000000000000000000000000000000000000000000000000000000000000ba200000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000002b05699353b600008ae85d840000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000000000000000000020000000000000000000000000b4fbf271143f4fbf7b91a5ded31805e42b2208d600000000000000000000000000000000000000000000000000000000000000c00000000000000000000000000000000000000000000000000000000000000020000000000000000000000000000000000000000000000000000000000000006000000000000000000000000000000000000000000000000000000000000000800000000000000000000000000000000000000000000000000000000000000001000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000def65f8e2f62bde60cd86ad3b4ad7fa6741cf175ffffffff09616c6c64617461","nonce":"0x82","to":"0x02afbd43cad367fcb71305a2dfb9a3928218f0c1","transactionIndex":"0x18","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0xc7ac308b1ae238cb2aa4b030ca79baffb7b498e1b10df0c3354b3cc19cfabac9","s":"0x2358b1093b9aab5fae0cf751766b0bb2e3d8cd2b18ca627e1e12520990a8ed6d","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x6ece61d29b90642941c41240da924690da145696","gas":"0x4399c","gasPrice":"0x506794e0","hash":"0x317ecb5bd19caa42a69f836d41556ebb0e0e00e1c6cd2dee230e6e6192612527","input":"0x46f83b5000000000000000000000000000000000000000000000000000000000000000654666803decc59e7d0a5d3efa3c3653c05645ef4342738445e20a3b43e0b4aa8b000000000000000000000000000000000000000000000000000000000000000f4666803decc59e7d0a5d3efa3c3653c05645ef4342738445e20a3b43e0b4aa8b","nonce":"0x2cc4","to":"0xa6bf2be6c60175601bf88217c75dd4b14abb5fbb","transactionIndex":"0x19","value":"0x0","type":"0x0","chainId":"0x5","v":"0x2d","r":"0x37aeab0db6d550e6cba2c04c9cb10a2af0fec0b11f6e04b85e1ea0b8b9defeae","s":"0x4f32f4ef74872e1385c25e69bf21ee049545a4d423e939eaff29677450efe9eb"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0xc4a5c059840a146b81c77bf79f1398f2b3d28ca7","gas":"0x432ec","gasPrice":"0x45db0a51","hash":"0xc879285db5ef0a6bce98021584d16f134c1dc0aed8cc988802c4f72ba6877ff6","input":"0x9f8420b300000000000000000000000000000000000000000000000000470de4df8200000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x1a7","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0x1a","value":"0x470efc4514e000","type":"0x0","chainId":"0x5","v":"0x2d","r":"0x237e3a541c5ab490843cdc20d0fcb7416bff8da471f6a11ae1b32d2bab119141","s":"0x3c8417d7a60bd5b0bcae4975dbb4aaee54a4b4d588226481be400843bfe2b64e"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x6f4aa3bc9732372daa288beb9b2516374ace5786","gas":"0xffff","gasPrice":"0x44e9acbc","maxFeePerGas":"0x44e9acbc","maxPriorityFeePerGas":"0x44e9acbc","hash":"0xecaa2d6f597608307e5084854854ba6dc1e69395e2abea14f2c6a2fa1d6faf9a","input":"0xe2864fe300000000000000000000000000000000000000000000000000000000000000200000000000000000000000006f4aa3bc9732372daa288beb9b2516374ace57860000000000000000000000000000000000000000000000000000000000000120000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000002004a8312e37fcecd450e6684009a2b54bc64f66afbe5217c7b2f63b03cd9980a9e00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000064893e8223d235ef0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000002a00000000000000000000000000000000000000000000000000000000000000040000000000000000000000000000000000000000000000000000000000000000173ad21460000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000000000000000000040000000000000000000000000def65f8e2f62bde60cd86ad3b4ad7fa6741cf175000000000000000000000000000000000000000000000000000000000000085000000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000000de0b6b3a7640000aaaebeba000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000040000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000001400000000000000000000000000000000000000000000000000000000000000020000000000000000000000000000000000000000000000000000000000000006000000000000000000000000000000000000000000000000000000000000000800000000000000000000000000000000000000000000000000000000000000001000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000020000000000000000000000006e512435020ac228a15543236b1f46656f98136300000000000000000000000000000000000000000000000000000000000000fa000000000000000000000000d6ffd79b52a587a0a9941a61f4e6cb0d386d54580000000000000000000000000000000000000000000000000000000000000000","nonce":"0x83","to":"0x02afbd43cad367fcb71305a2dfb9a3928218f0c1","transactionIndex":"0x1b","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xa18772e4b9e1493ebfd5ae6219cc9ca060b71e83788fb614ab02b03fb36eaf1b","s":"0x482c7e50fde29bb4130fd6e723176e0e9913430ebb9260667d7fb55f5408d9c5","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0xdb33365a8730de2f7574ff1189fb9d337bf4c36d","gas":"0x61a80","gasPrice":"0x433653c8","maxFeePerGas":"0x828367af","maxPriorityFeePerGas":"0x37e9071","hash":"0x4dd69b69a568ff30ae439e2ded72fbd7f2e7aaa345836703663f155c749c5eed","input":"0x7f34c0ddc6e984f46ab7c13b7a53643096e79c39103dc74550204d5bd1b5206ef5a2f49689071bb6a332212df26aa57cd4d1e3c0894187a9fc5e5d686cbedba5b187470f000000000000000000000000000000000000000000000000000000000000000e000000000000000000000000e8c44c30e8e52c38ac7bd232e7b7f0c7463a535100000000000000000000000000000000000000000000000000000000000000a0000000000000000000000000000000000000000000000000000000000000002e516d5044645856613738437a6b7856325a364e52777a67685a785872736e75397261616a695a7555423569465a62000000000000000000000000000000000000","nonce":"0x2b57","to":"0xb8ad54644e2371cbb7940b9fa0936f399ba7495f","transactionIndex":"0x1c","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0xa1fb0e5662b54ceb3dc1623ebfb200668433cb27252d553669bcd136387394f3","s":"0x27c2f8e19f69a52a0960f05d8b21f11c313dd3e86a7139b5783a18353c407415","yParity":"0x0"}],"transactionsRoot":"0x4a87d0cf5990b1c5bac631583e5965c2ba943858bebb2e07f74d0b697f73821a","uncles":[],"withdrawals":[{"index":"0x1170","validatorIndex":"0x38c2c","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x66edfd65"},{"index":"0x1171","validatorIndex":"0x38c2d","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6cd228e4"},{"index":"0x1172","validatorIndex":"0x38c2e","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x77f3431b"},{"index":"0x1173","validatorIndex":"0x38c2f","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6b61f268"},{"index":"0x1174","validatorIndex":"0x38c30","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6e10bb21"},{"index":"0x1175","validatorIndex":"0x38c31","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6eb115a5"},{"index":"0x1176","validatorIndex":"0x38c32","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x7caead1d"},{"index":"0x1177","validatorIndex":"0x38c33","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x772c0ddf"},{"index":"0x1178","validatorIndex":"0x38c34","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x75930a95"},{"index":"0x1179","validatorIndex":"0x38c35","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x76a4db09"},{"index":"0x117a","validatorIndex":"0x38c36","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x7e692b27"},{"index":"0x117b","validatorIndex":"0x38c37","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x72038ae6"},{"index":"0x117c","validatorIndex":"0x38c38","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6ccce352"},{"index":"0x117d","validatorIndex":"0x38c39","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x79ef6898"},{"index":"0x117e","validatorIndex":"0x38c3a","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6d58977d"},{"index":"0x117f","validatorIndex":"0x38c3b","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x76f7d208"}],"withdrawalsRoot":"0xbe712c930a0665264b025ced87cc7839eef95a3cbc26dadc93e9e185a350ad28"} diff --git a/op-service/sources/testdata/data/blocks/post-shanghai-bad-receipts_metadata.json b/op-service/sources/testdata/data/blocks/post-shanghai-bad-receipts_metadata.json new file mode 100644 index 000000000000..4da07c88cd37 --- /dev/null +++ b/op-service/sources/testdata/data/blocks/post-shanghai-bad-receipts_metadata.json @@ -0,0 +1 @@ +{"name":"post-shanghai-bad-receipts","fail":true} diff --git a/op-service/sources/testdata/data/blocks/post-shanghai-bad-transactions_data.json b/op-service/sources/testdata/data/blocks/post-shanghai-bad-transactions_data.json new file mode 100644 index 000000000000..8986999d9ebe --- /dev/null +++ b/op-service/sources/testdata/data/blocks/post-shanghai-bad-transactions_data.json @@ -0,0 +1 @@ +{"baseFeePerGas":"0x3fb7c357","difficulty":"0x0","extraData":"0x","gasLimit":"0x1c9c380","gasUsed":"0x18f759","hash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","logsBloom":"0x020010404000001a0000021000000080001100410000100001000010040200980220400000008806200200000100000000000000000000008000000400042000000050000040000112080808800002044000040004042008800480002000000000000002020020000042002400000820000080040000000010200010020010100101212050000008000000008000001010200c80000112010000438040020400000000202400000000002002a0210402000622010000000001700144000040000000002204000000c000410105024010000808000000002004002000000261000000822200200800881000000012500400400000000000000040010000800000","miner":"0x000095e79eac4d76aab57cb2c1f091d553b36ca0","mixHash":"0x5b53dc49cbab268ef9950b1d81b5e36a1b2f1b97aee1b7ff6e4db0e06c29a8b0","nonce":"0x0000000000000000","number":"0x84161e","parentHash":"0x72d92c1498e05952988d4e79a695928a6bcbd37239f8a1734051263b4d3504b8","receiptsRoot":"0xaff90ae18dcc35924a4bddb68d403b8b7812c10c3ea2a114f34105c87d75bcdb","sha3Uncles":"0x1dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347","size":"0x2a51","stateRoot":"0xc56738518b2c7854a640ae25996d2211c9ef0dd2e4dd9e59e9d9cacef39622da","timestamp":"0x64110a5c","totalDifficulty":"0xa4a470","transactions":[{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x1b7aa44088a0ea95bdc65fef6e5071e946bf7d8f","gas":"0x5208","gasPrice":"0x4159f01735","hash":"0x1e8f148a9aea7d8d16ea6e9446723b8f262e8bcd89c7c961d52046ebd43b4598","input":"0x","nonce":"0x279ad","to":"0xa21765a03dd41e2783696d314f235f4d520f6cac","transactionIndex":"0x0","value":"0x3782dace9d90000","type":"0x0","chainId":"0x5","v":"0x2e","r":"0xf9eca8b74ccaec5f4a83e981baeb8602f38d0271179c592b4a18f3866316a367","s":"0x350a9d26718d1fa9dc4f268f9c0e18fb96dbc037816ca8fc20f33f1a76f9f501"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x4d9a852e6aecd3a6e87fece2ca109780e45e6f2d","gas":"0x160ab","gasPrice":"0xd4babc57","maxFeePerGas":"0x11407d03e","maxPriorityFeePerGas":"0x9502f900","hash":"0xab5c870f4c367012bd763172afbfbe68fbf35336a66ae41aff3f2c9dbf4ea3f8","input":"0x60566037600b82828239805160001a607314602a57634e487b7160e01b600052600060045260246000fd5b30600052607381538281f3fe73000000000000000000000000000000000000000030146080604052600080fdfea26469706673582212200c479c99d58d7fbee5ee3aef5001431073a3d465bdfa46cf747a7c0b9289607864736f6c634300080d0033","nonce":"0xafd","to":null,"transactionIndex":"0x1","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x48f6acf4ac4371eb96ffca8cdef5b7704ea8c68a631d1c02927036d4ce92567e","s":"0x1f12261ddde63fd3da2ed9ea1bcbb4b0f25af898248c805a9d4b0a6e42caf9c8","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0xce297b30e56bf2f6142c4048fbae1cb8a756fefd","gas":"0x557300","gasPrice":"0xd4babc57","maxFeePerGas":"0xfabdd183","maxPriorityFeePerGas":"0x9502f900","hash":"0xa81fd92b2d0f0bbd3cc355f869cca3243c98c5e2641db9ecf3eeabb3b13bff6a","input":"0x7bbaf1ea0000000000000000000000000000000000000000000000000000000000000005000000000000000000000000000000000000000000000000000000000000004000000000000000000000000000000000000000000000000000000000000000200000000000000000000000000000000000000000000000000000000000000000","nonce":"0xad92","to":"0x4c0ce02c1219ce5d2afffba97e484272a4637b49","transactionIndex":"0x2","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0xd11b842a5f7b0e020470e764b443fafc4044327bca8dbb37131425359abace2","s":"0x1009867573946fb0a4a35339a6eaaece4d0a168b2346539076e6319e9a466b1","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x47a916d0b694c95bc0c09953c1a6e22c8867be32","gas":"0x5208","gasPrice":"0x991ff257","maxFeePerGas":"0xb2fea055","maxPriorityFeePerGas":"0x59682f00","hash":"0xa92c7b720c08c83f1a0ed7e4c163200e30a3a8c03fcc5a51e685ea20cd0cb577","input":"0x","nonce":"0x2","to":"0xf955dfe99bbfd06a8be208274cd654ecfb66a45e","transactionIndex":"0x3","value":"0x11c37937e08000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xeecd431225215c3ddc2e13cf16fa29f9a2ab1afbaa02988f3ceac27416edf0f5","s":"0x6494062330a9d509f7c778cb45cbcc9932e5f9b4ec94a1ec315f579f8d973b7b","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x672d214fe4db218defae698085a3ccefc773dfa9","gas":"0xf4f6","gasPrice":"0x991ff257","maxFeePerGas":"0xb2fea055","maxPriorityFeePerGas":"0x59682f00","hash":"0x6921b429ad2ec1e97d3457049ad2e893b5a0349beba47ca1c74a9540af75347a","input":"0xa1ddc546000000000000000000000000000000000000000000000000000000000000118f","nonce":"0x8e","to":"0xdef65f8e2f62bde60cd86ad3b4ad7fa6741cf175","transactionIndex":"0x4","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x4732ad21401b509bebb947a8566127fba1e245eb5a262ddc522d6867ab80567e","s":"0x2153013617fefdb06db0a7316d8883056441d33c2abf5ed6291fa0e3cc5c3a2f","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x0b12f7f996579fe7d9302d2574910121c214ee15","gas":"0x56a8","gasPrice":"0x991ff257","maxFeePerGas":"0xb2fea055","maxPriorityFeePerGas":"0x59682f00","hash":"0xf776b2da0b835dde05d0d8b76fd19385d61e7055036cf637f804b36dc94f2384","input":"0x71ec5c05aa669c4922569c1d33f7a81aaa21813800000000000000000000000013a0c5930c028511dc02665e7285134b6d11a5f40000000000000000000000000b12f7f996579fe7d9302d2574910121c214ee150000000000000000000000000000000000000000000000000000000000000000","nonce":"0xe","to":"0x7c125c1d515b8945841b3d5144a060115c58725f","transactionIndex":"0x5","value":"0x2386f26fc10000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xc4505d4c46ef9a0103178e5ef4fbc857114ae5c4686b68b74d07c3ee3a86e575","s":"0x74db34fd675bea7d2ee74f14fd8709cb88c7d22c2267d12f2c671b5ebc180bd5","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0xd36aac0c9676e984d72823fb662ce94d3ab5e551","gas":"0xff212","gasPrice":"0x991ff257","maxFeePerGas":"0xd86d063e","maxPriorityFeePerGas":"0x59682f00","hash":"0x9a08d899cd14ebb930ed59fa774afdb88a22615b3a931e930931ea54d26dc0bc","input":"0x49160658babdf216946a22b96c966ceeba750f14807726306ebeed500a2621a7481df010000000000000000000000000000000000000000000000000000000000000008000000000000000000000000000000000000000000000000000000000000000c0000000000000000000000000000000000000000000000000000000000000012000000000000000000000000000000000000000000000000000000000000000084d6f6f6e6265616d000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000002a3078434133633746363939643335333137316438434643364236334431433764333841333233373238300000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000c0b616f73c8d00641109f0000054f967b8a135e44e233da070ea88c5e4a53bacac0000000000000000000000000000000000000000000000000000000000000002000000000000000000000000c6a3a17399d88e5dfd2de4fa41dcccade6a2d2170000000000000000000000008676147503f761e44b85df01811edd52be01b56c000000000000000000000000000000000000000000000000d02ab486cedc00000000000000000000000000000000000000000000000000000de0b6b3a7640000","nonce":"0x7121","to":"0x2d276bd64fe8643a1d76b2e9530c753423e972c1","transactionIndex":"0x6","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0xce452361d331baf2ac86947a1b3b454b3e842923e1f39139b33a572845dd5d7c","s":"0x388126881782af5a146ce8409f1e3a2c9a4cadcfe9a2c1d27258af6e4eb5cec1","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0xb83a48ee322989a0a4f838cb207abc28ec021b73","gas":"0x326ff","gasPrice":"0x991ff257","maxFeePerGas":"0x5c3362b9c","maxPriorityFeePerGas":"0x59682f00","hash":"0x0fe0d97e25d5eb11a33a3e8278584c3780941fc2675bdf8fc547cee3d1fd3b17","input":"0xa4110dbe190e65a7fe0ad5acd305c6446a02cb39de30f382000fe67927434063bf4c4b2ca86b3ee6d039dc7a0268f9e1c39266c3f12a752a7eb72f0895dedf62291fd292","nonce":"0x12e","to":"0x96e6192eeaf7bb308f79fb5017a9085754b9e12a","transactionIndex":"0x7","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x46819bb540fa459a5a0cbf041a0286507d597fc8958a6af8b9ce193f00005238","s":"0x31495c58bf4b74925f870c59635912681bee947b1cb4b08cc19f22421b1a10ab","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0xb83a48ee322989a0a4f838cb207abc28ec021b73","gas":"0x3270b","gasPrice":"0x991ff257","maxFeePerGas":"0x5c3362b9c","maxPriorityFeePerGas":"0x59682f00","hash":"0xef47a60f57f177a683c723c658137efab66d311e1c5abbc4d74f653535144d03","input":"0xa4110dbe18eba3e815ec7fcb03a8eea0487209c3253024d1dc4f36f775a43fdd3e409b74c0a5c88e54f39b88db6f185a41431b204120d3e66c9b08b86f5e8382623327e5","nonce":"0x12f","to":"0x96e6192eeaf7bb308f79fb5017a9085754b9e12a","transactionIndex":"0x8","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x4d7e11fe3914763a2735493a62b71ae0028e75a750a8b06407996cf478e273c8","s":"0x60c18fdf948721f96fdc2efbf694910122e82191c7a40a5caab247c06ab7f3cf","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x7712a1f96b3f37c3dd88a22d4f7a7977b937fe62","gas":"0x1ff06","gasPrice":"0x991ff257","maxFeePerGas":"0xcf662006","maxPriorityFeePerGas":"0x59682f00","hash":"0xe23a5b35faae5335adc5aca38c5d633b00438b798c2053104b8df48406c9b141","input":"0x3ff9751b0000000000000000000000000000000000000000000000000000000000000034000000000000000000000000000000000000000000000000000000000000003500000000000000000000000000000000000000000000000000000000000000a000000000000000000000000000000000000000000000000000000000000001000000000000000000000000000000000000000000000000000000000000000160000000000000000000000000000000000000000000000000000000000000002e516d566e75374a51566f445271536748427a72615970374879373848774a744c4669366e5546436f775447647a70000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000002e516d5439716b3343525962464457704446596541763854384831676e6f6e67774b6868354a36384e4c6b4c6972360000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000041c9f31b989aaa745aecc563139f89c8ab3ab492e725d1e0305b7d602fcbf9293c463f0d2e4f5fb0c9efb54eb2cde9af10ebc9c3676f2334007291ffa8efb40adb1c00000000000000000000000000000000000000000000000000000000000000","nonce":"0x16","to":"0xc3b87ea584cb9cb5a4523e2bed16a34bf7746317","transactionIndex":"0x9","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xcc693a9544394653e1f393f02635e2c1f78b5e6c26c4d00a867bcf4adc2bc09e","s":"0x23f4b71b773c19f77cf251b2aeea03f2696bddd847d1f6d75acd2fbea6a543e4","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x12bbafcfa06423d87bed591a64c1a1f34791e90a","gas":"0x1b2e020","gasPrice":"0x8f0d3617","maxFeePerGas":"0x1cb3abd5a","maxPriorityFeePerGas":"0x4f5572c0","hash":"0xd8cea4ba619b317bc05d58534af73beec6c2548b31b24d4dc61c9bbd29cfa17a","input":"0x","nonce":"0x4d","to":"0x12bbafcfa06423d87bed591a64c1a1f34791e90a","transactionIndex":"0xa","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x8dae28608985d6ab47cbe122ee88972e38664cede30978f41944f347bad86e5c","s":"0x17c2898b39b7e3b874dc78e7df3fcff68f4d715385c632aad4bdf6e62505651c","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x6999c36394dd5d3d3295af63a4736ad6bd279448","gas":"0x14d94","gasPrice":"0x7b528d57","maxFeePerGas":"0x7df743e0","maxPriorityFeePerGas":"0x3b9aca00","hash":"0x79a4b9d90b02c768baaad305f266281213cc75062cbe99a13222cc0c4b509498","input":"0x6a7612020000000000000000000000009008d19f58aabd9ed0d60971565aa8510560ab4100000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000140000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000022000000000000000000000000000000000000000000000000000000000000000a4ec6cb13f00000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000000000000000000001000000000000000000000000000000000000000000000000000000000000003826d8d7371becafc0afedc3fe85b699b87f4357d55ec39ce83c510ed7d70ff514dce8236351f4627ccec6a7d9a67bc6fc2ec17ca76411113c00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000410000000000000000000000006999c36394dd5d3d3295af63a4736ad6bd27944800000000000000000000000000000000000000000000000000000000000000000100000000000000000000000000000000000000000000000000000000000000","nonce":"0x13","to":"0xdce8236351f4627ccec6a7d9a67bc6fc2ec17ca7","transactionIndex":"0xb","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x669487011643510f7512ec15b07d74efb853ba4ad9fe9c7e9850ccf24e3d4c53","s":"0x2191f9ee81087e1df833e69d6129ec330ed4c99be2031060834e22c589fcf79b","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x08ebc052e6353b4d8392113f0f2c5decd691ddcd","gas":"0x1b2e020","gasPrice":"0x7b528d57","maxFeePerGas":"0xf6e4d9c4","maxPriorityFeePerGas":"0x3b9aca00","hash":"0x6790a3bbddbeb21fcb736a59b3775755051c3a6344d8390cf8ca27f2e8a814f0","input":"0x6871ee40","nonce":"0x4","to":"0xb21b333c40354393f18de57f2d0a0fc21f270482","transactionIndex":"0xc","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0xa7a58ab858e5dd87b9a74d550e5716ac40ddefd1c21d5baa51e542f4f941d73e","s":"0x1fd0cb1563df0368bd332717d7c8210187cd1ac2ff44f61f5242fc02eaa0c2fe","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x08ebc052e6353b4d8392113f0f2c5decd691ddcd","gas":"0x12c36","gasPrice":"0x7b528d57","maxFeePerGas":"0xe596c292","maxPriorityFeePerGas":"0x3b9aca00","hash":"0x87ec7ace5442db252b5751ffddd38dcb04b088d36b6b0e526ff25607a4293c81","input":"0xcb40cbfd","nonce":"0x5","to":"0xb21b333c40354393f18de57f2d0a0fc21f270482","transactionIndex":"0xd","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x8e4bd40195bdfaa868af97dbf22e0b74295d95564a8381bc9bcfe1af3cde2cdb","s":"0x62057dc87c8b2b197d12c648233aab4134b7c2a2f65db89a49728564f3d0864d","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x08ebc052e6353b4d8392113f0f2c5decd691ddcd","gas":"0x12c36","gasPrice":"0x7b528d57","maxFeePerGas":"0xe596c292","maxPriorityFeePerGas":"0x3b9aca00","hash":"0x40cb487ecffda94f97ce7fc0f7163f2f024235df2c8291169edc80dac063e6d0","input":"0xcb40cbfd","nonce":"0x6","to":"0xb21b333c40354393f18de57f2d0a0fc21f270482","transactionIndex":"0xe","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x993c4386ffc2bd63a48dd31cf5f65aa355cba3c063b08778b328d748937098e5","s":"0x203b1faf595628dd78e760bcc1740065625abb6cf00b1b17224e10fa6b424e43","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0xafc7682751d3654f34958e3fb660971bf5918367","gas":"0x1b2e020","gasPrice":"0x7b528d57","maxFeePerGas":"0xefb40664","maxPriorityFeePerGas":"0x3b9aca00","hash":"0xb76bb3d88c9b30d927c45ccfcf8d5b0054411ac8501ad588822a7d04690cccf6","input":"0x40c10f19000000000000000000000000afc7682751d3654f34958e3fb660971bf59183670000000000000000000000000000000000000000000000008ac7230489e80000","nonce":"0x20","to":"0x84ced17d95f3ec7230baf4a369f1e624ae60090d","transactionIndex":"0xf","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x78e9d29c606f6a61a1428ed7326df05210d802c3061f3b469847cc461eb48025","s":"0x517afe5a21175f186233ae7a6c090cd33e028a77e5854e98eb2fa4c9284430e7","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x12bbafcfa06423d87bed591a64c1a1f34791e90a","gas":"0xcafe","gasPrice":"0x7b528d57","maxFeePerGas":"0x16ba45bce","maxPriorityFeePerGas":"0x3b9aca00","hash":"0x798ebe823209869347c08bd81e04fbf60e9bdfe44b1cc923215182d0cf3d4edb","input":"0x40c10f1900000000000000000000000012bbafcfa06423d87bed591a64c1a1f34791e90a0000000000000000000000000000000000000000000000000000000000000032","nonce":"0x4e","to":"0xd25d682189afa7ed0c2811195b3f3d07616927bc","transactionIndex":"0x10","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0xbe2aa95f4c32188c9ecbae97286ab983f7cc5a7017d46c404f8eb80b55e2dc13","s":"0x55d8968f509f51633f49260178941fd5a3db999e1995206348d7a3f05152b3d5","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x12bbafcfa06423d87bed591a64c1a1f34791e90a","gas":"0xcafe","gasPrice":"0x7b528d57","maxFeePerGas":"0xc6132c0c","maxPriorityFeePerGas":"0x3b9aca00","hash":"0xbe68a7e02725f799a65ebb069ccc83a014ac7c40e4119bf7c220a2f6ddfee295","input":"0x40c10f1900000000000000000000000012bbafcfa06423d87bed591a64c1a1f34791e90a0000000000000000000000000000000000000000000000000000000000000064","nonce":"0x4f","to":"0xd25d682189afa7ed0c2811195b3f3d07616927bc","transactionIndex":"0x11","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x1750ed45d0d2d048a61dcc3ff3e7ad0cbd395b77698c4f2eeb6baaea4c966d96","s":"0xe646e3a87b6dc50439468bcfd24cbf12a6f95b2762f1527c97a7d1b7f03530e","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0xd0c0338baac9ce09c1d1caab88b4360507061e37","gas":"0x1b2e020","gasPrice":"0x7b528d57","maxFeePerGas":"0x1918167aa","maxPriorityFeePerGas":"0x3b9aca00","hash":"0xc90c3a72efe81331727fcce4b5bd4906066da314ca9a0b44023a6b09ea7e8114","input":"0x4e71d92d","nonce":"0xe7","to":"0x0137d796936e199b917b93118eb4e002508a48bd","transactionIndex":"0x12","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x22222ce8fe753082c8e5dc7478fa7b87db9c8d0fb070ebc88ab015db13fd6bb","s":"0x6dceadecf6f6a3ddc433a547dba6d5fcdfb7d006ce09eeae1c06c1828ef9fff9","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x8bd834cc1dd17d1a2f7eb6140c3dd5ab51d0f338","gas":"0x1b2e020","gasPrice":"0x7b528d57","maxFeePerGas":"0x1a4019792","maxPriorityFeePerGas":"0x3b9aca00","hash":"0x619a6cbd43cde074d314c19623bd66d9fb1e13c158d7138775236f798dc1245e","input":"0x1da1db5e00000000000000000000000000000000000000000000000000001b5ec4ff740d","nonce":"0x10","to":"0xbad0009f7a8e796e3f975dc1d6ab3e3c237b857c","transactionIndex":"0x13","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x733c496e8c84f78d113bc8b5ad6d04a133bb10bbcecf0db93d97eeedfa3d984a","s":"0x6627c16aa7dea27db44c0f0641c30ffd544b9b9b1c299dc60611e1e01c714ee8","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x8bd834cc1dd17d1a2f7eb6140c3dd5ab51d0f338","gas":"0x11df0","gasPrice":"0x7b528d57","maxFeePerGas":"0x1d3fbccb8","maxPriorityFeePerGas":"0x3b9aca00","hash":"0xca5a56cd77b9e5b0e79020cc6346edf205bc11e901984d805125f28c2e6686e6","input":"0xd7c01032000000000000000000000000356f24c0f3187a6dcb949e4a60aa0a74a8a0327e0000000000000000000000000f8bacfae87aee49a2eec86df0c6b958ec3fd4fd0000000000000000000000000f8bacfae87aee49a2eec86df0c6b958ec3fd4fd","nonce":"0x11","to":"0xbad0009f7a8e796e3f975dc1d6ab3e3c237b857c","transactionIndex":"0x14","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x4aa872c1a7da70126917604193717575b91407f82449bca956d4089b90cd902","s":"0x4b5f788c68024f8958f66e8c768a8e0f65ef297c4ffd420a715258b0a55cb0a0","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x9bedd66b05712a1add7d1fec899b8641b3ecc863","gas":"0x1b2e020","gasPrice":"0x7b528d57","maxFeePerGas":"0x1e706c8d2","maxPriorityFeePerGas":"0x3b9aca00","hash":"0x999c9ddeed67c6ef6fbf02a6e977a6c1b68e18d24814e51643c7157b87a43e0a","input":"0xa8b78ed0","nonce":"0x3","to":"0xf6c2afb18b85175e5ffb28f20f007c047ce28592","transactionIndex":"0x15","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xf1ca074104ee0b6a67223afd797654b8f21060453f7662dd7be79de938c396da","s":"0x4c9a4e56f0ad21e2dd8d1116c7a3e41ea2c6126853772f7897bd959e81e2e28a","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x9bedd66b05712a1add7d1fec899b8641b3ecc863","gas":"0x2dc6c0","gasPrice":"0xd4babc57","maxFeePerGas":"0x21c13e710","maxPriorityFeePerGas":"0x9502f900","hash":"0x47c8f5d0b3778e4c34eba7fcc356fa04a5afd954ccf484728e72c002764dd3c4","input":"0xf688bcfb0000000000000000000000000000000000000000000000000000000005f5e100","nonce":"0x4","to":"0xf6c2afb18b85175e5ffb28f20f007c047ce28592","transactionIndex":"0x16","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x970982425d18b9949141bbabb5c5230fa168626ce210a97f3856d64700fc0eeb","s":"0x6fa6d0aea6ae48783d80c1e51cda47e11c28b90a5497579b93d7f242d786f67e","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x4b596cebd02c35aa99878eb9ecb24d6f5ca5f865","gas":"0x5208","gasPrice":"0x68c88f79","maxFeePerGas":"0x68c88f79","maxPriorityFeePerGas":"0x68c88f79","hash":"0x396797ae0ebcdb72ff1f96fd08b6128f78acc7417353f142f1a5facd425a33e6","input":"0x","nonce":"0x1","to":"0x375980b8e1a417620ba9b18e5a9e0b17dccbac9d","transactionIndex":"0x17","value":"0x1c9a406c58878","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xcb2a7b106097b2f2a61d3a048054a5175c386b1dcdd5b6fcd7849dcac7c32a66","s":"0x10aef76e98276c47470f04a599b81c260ace9d2f6adc95755c6d22d859e805a2","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x6f4aa3bc9732372daa288beb9b2516374ace5786","gas":"0x1b2e020","gasPrice":"0x61010bda","maxFeePerGas":"0x61010bda","maxPriorityFeePerGas":"0x61010bda","hash":"0x454aa43d6546a6f62246826c16b7a49c6c704238c18802ef0d659922f23a573c","input":"0xe99a3f8000000000000000000000000000000000000000000000000000000000000000800000000000000000000000000000000000000000000000000000000000000480000000000000000000000000000000000000000000000000000000000000050000000000000000000000000000000000000000000000000000000000000008a000000000000000000000000058ce7eb44ae575fb3b3974861c1138dad3c5aee40000000000000000000000000000000000000000000000000000000000000120000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000001e06defd69964a6ce467b87c64627a7a2be4be139b504757c625eb6114ec166557d000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000641a269023d235ef0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000002a000000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000002b05699353b600008ae85d840000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000000000000000000020000000000000000000000000b4fbf271143f4fbf7b91a5ded31805e42b2208d600000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000000000000000000001f63c28250000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000000000000000000020000000000000000000000000def65f8e2f62bde60cd86ad3b4ad7fa6741cf17500000000000000000000000000000000000000000000000000000000000001400000000000000000000000000000000000000000000000000000000000000020000000000000000000000000000000000000000000000000000000000000006000000000000000000000000000000000000000000000000000000000000000800000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000020000000000000000000000006e512435020ac228a15543236b1f46656f98136300000000000000000000000000000000000000000000000000000000000000fa000000000000000000000000d6ffd79b52a587a0a9941a61f4e6cb0d386d5458000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000413e24433da3efc4c3cc305339a7b77a1b31abb4e4d4967e97e42aed1dd0bf98936f43035f40bc80f6f6e3a9cfb43524a1171ee7f21e9ac2fa586d33a31f10619d1c000000000000000000000000000000000000000000000000000000000000000000000000000000000000006f4aa3bc9732372daa288beb9b2516374ace5786000000000000000000000000000000000000000000000000000000000000012000000000000000000000000058ce7eb44ae575fb3b3974861c1138dad3c5aee400000000000000000000000000000000000000000000000000000000000002000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000641a269023d235ef0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000002c00000000000000000000000000000000000000000000000000000000000000040000000000000000000000000000000000000000000000000000000000000000173ad21460000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000000000000000000040000000000000000000000000def65f8e2f62bde60cd86ad3b4ad7fa6741cf1750000000000000000000000000000000000000000000000000000000000000ba200000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000002b05699353b600008ae85d840000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000000000000000000020000000000000000000000000b4fbf271143f4fbf7b91a5ded31805e42b2208d600000000000000000000000000000000000000000000000000000000000000c00000000000000000000000000000000000000000000000000000000000000020000000000000000000000000000000000000000000000000000000000000006000000000000000000000000000000000000000000000000000000000000000800000000000000000000000000000000000000000000000000000000000000001000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000def65f8e2f62bde60cd86ad3b4ad7fa6741cf175ffffffff09616c6c64617461","nonce":"0x82","to":"0x02afbd43cad367fcb71305a2dfb9a3928218f0c1","transactionIndex":"0x18","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0xc7ac308b1ae238cb2aa4b030ca79baffb7b498e1b10df0c3354b3cc19cfabac9","s":"0x2358b1093b9aab5fae0cf751766b0bb2e3d8cd2b18ca627e1e12520990a8ed6d","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x6ece61d29b90642941c41240da924690da145696","gas":"0x4399c","gasPrice":"0x506794e0","hash":"0x317ecb5bd19caa42a69f836d41556ebb0e0e00e1c6cd2dee230e6e6192612527","input":"0x46f83b5000000000000000000000000000000000000000000000000000000000000000654666803decc59e7d0a5d3efa3c3653c05645ef4342738445e20a3b43e0b4aa8b000000000000000000000000000000000000000000000000000000000000000f4666803decc59e7d0a5d3efa3c3653c05645ef4342738445e20a3b43e0b4aa8b","nonce":"0x2cc4","to":"0xa6bf2be6c60175601bf88217c75dd4b14abb5fbb","transactionIndex":"0x19","value":"0x0","type":"0x0","chainId":"0x5","v":"0x2d","r":"0x37aeab0db6d550e6cba2c04c9cb10a2af0fec0b11f6e04b85e1ea0b8b9defeae","s":"0x4f32f4ef74872e1385c25e69bf21ee049545a4d423e939eaff29677450efe9eb"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0xc4a5c059840a146b81c77bf79f1398f2b3d28ca7","gas":"0x432ec","gasPrice":"0x45db0a51","hash":"0xc879285db5ef0a6bce98021584d16f134c1dc0aed8cc988802c4f72ba6877ff6","input":"0x9f8420b300000000000000000000000000000000000000000000000000470de4df8200000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x1a7","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0x1a","value":"0x470efc4514e000","type":"0x0","chainId":"0x5","v":"0x2d","r":"0x237e3a541c5ab490843cdc20d0fcb7416bff8da471f6a11ae1b32d2bab119141","s":"0x3c8417d7a60bd5b0bcae4975dbb4aaee54a4b4d588226481be400843bfe2b64e"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x6f4aa3bc9732372daa288beb9b2516374ace5786","gas":"0xffff","gasPrice":"0x44e9acbc","maxFeePerGas":"0x44e9acbc","maxPriorityFeePerGas":"0x44e9acbc","hash":"0xecaa2d6f597608307e5084854854ba6dc1e69395e2abea14f2c6a2fa1d6faf9a","input":"0xe2864fe300000000000000000000000000000000000000000000000000000000000000200000000000000000000000006f4aa3bc9732372daa288beb9b2516374ace57860000000000000000000000000000000000000000000000000000000000000120000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000002004a8312e37fcecd450e6684009a2b54bc64f66afbe5217c7b2f63b03cd9980a9e00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000064893e8223d235ef0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000002a00000000000000000000000000000000000000000000000000000000000000040000000000000000000000000000000000000000000000000000000000000000173ad21460000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000000000000000000040000000000000000000000000def65f8e2f62bde60cd86ad3b4ad7fa6741cf175000000000000000000000000000000000000000000000000000000000000085000000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000000de0b6b3a7640000aaaebeba000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000040000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000001400000000000000000000000000000000000000000000000000000000000000020000000000000000000000000000000000000000000000000000000000000006000000000000000000000000000000000000000000000000000000000000000800000000000000000000000000000000000000000000000000000000000000001000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000020000000000000000000000006e512435020ac228a15543236b1f46656f98136300000000000000000000000000000000000000000000000000000000000000fa000000000000000000000000d6ffd79b52a587a0a9941a61f4e6cb0d386d54580000000000000000000000000000000000000000000000000000000000000000","nonce":"0x83","to":"0x02afbd43cad367fcb71305a2dfb9a3928218f0c1","transactionIndex":"0x1b","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xa18772e4b9e1493ebfd5ae6219cc9ca060b71e83788fb614ab02b03fb36eaf1b","s":"0x482c7e50fde29bb4130fd6e723176e0e9913430ebb9260667d7fb55f5408d9c5","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0xdb33365a8730de2f7574ff1189fb9d337bf4c36d","gas":"0x61a80","gasPrice":"0x433653c8","maxFeePerGas":"0x828367af","maxPriorityFeePerGas":"0x37e9071","hash":"0x4dd69b69a568ff30ae439e2ded72fbd7f2e7aaa345836703663f155c749c5eed","input":"0x7f34c0ddc6e984f46ab7c13b7a53643096e79c39103dc74550204d5bd1b5206ef5a2f49689071bb6a332212df26aa57cd4d1e3c0894187a9fc5e5d686cbedba5b187470f000000000000000000000000000000000000000000000000000000000000000e000000000000000000000000e8c44c30e8e52c38ac7bd232e7b7f0c7463a535100000000000000000000000000000000000000000000000000000000000000a0000000000000000000000000000000000000000000000000000000000000002e516d5044645856613738437a6b7856325a364e52777a67685a785872736e75397261616a695a7555423569465a62000000000000000000000000000000000000","nonce":"0x2b57","to":"0xb8ad54644e2371cbb7940b9fa0936f399ba7495f","transactionIndex":"0x1c","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0xa1fb0e5662b54ceb3dc1623ebfb200668433cb27252d553669bcd136387394f3","s":"0x27c2f8e19f69a52a0960f05d8b21f11c313dd3e86a7139b5783a18353c407415","yParity":"0x0"}],"transactionsRoot":"0xaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa","uncles":[],"withdrawals":[{"index":"0x1170","validatorIndex":"0x38c2c","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x66edfd65"},{"index":"0x1171","validatorIndex":"0x38c2d","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6cd228e4"},{"index":"0x1172","validatorIndex":"0x38c2e","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x77f3431b"},{"index":"0x1173","validatorIndex":"0x38c2f","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6b61f268"},{"index":"0x1174","validatorIndex":"0x38c30","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6e10bb21"},{"index":"0x1175","validatorIndex":"0x38c31","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6eb115a5"},{"index":"0x1176","validatorIndex":"0x38c32","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x7caead1d"},{"index":"0x1177","validatorIndex":"0x38c33","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x772c0ddf"},{"index":"0x1178","validatorIndex":"0x38c34","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x75930a95"},{"index":"0x1179","validatorIndex":"0x38c35","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x76a4db09"},{"index":"0x117a","validatorIndex":"0x38c36","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x7e692b27"},{"index":"0x117b","validatorIndex":"0x38c37","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x72038ae6"},{"index":"0x117c","validatorIndex":"0x38c38","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6ccce352"},{"index":"0x117d","validatorIndex":"0x38c39","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x79ef6898"},{"index":"0x117e","validatorIndex":"0x38c3a","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6d58977d"},{"index":"0x117f","validatorIndex":"0x38c3b","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x76f7d208"}],"withdrawalsRoot":"0xbe712c930a0665264b025ced87cc7839eef95a3cbc26dadc93e9e185a350ad28"} diff --git a/op-service/sources/testdata/data/blocks/post-shanghai-bad-transactions_metadata.json b/op-service/sources/testdata/data/blocks/post-shanghai-bad-transactions_metadata.json new file mode 100644 index 000000000000..03c892fc1ea3 --- /dev/null +++ b/op-service/sources/testdata/data/blocks/post-shanghai-bad-transactions_metadata.json @@ -0,0 +1 @@ +{"name":"post-shanghai-bad-transactions","fail":true} diff --git a/op-service/sources/testdata/data/blocks/post-shanghai-bad-withdrawals_data.json b/op-service/sources/testdata/data/blocks/post-shanghai-bad-withdrawals_data.json new file mode 100644 index 000000000000..f66ef388b910 --- /dev/null +++ b/op-service/sources/testdata/data/blocks/post-shanghai-bad-withdrawals_data.json @@ -0,0 +1 @@ +{"baseFeePerGas":"0x3fb7c357","difficulty":"0x0","extraData":"0x","gasLimit":"0x1c9c380","gasUsed":"0x18f759","hash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","logsBloom":"0x020010404000001a0000021000000080001100410000100001000010040200980220400000008806200200000100000000000000000000008000000400042000000050000040000112080808800002044000040004042008800480002000000000000002020020000042002400000820000080040000000010200010020010100101212050000008000000008000001010200c80000112010000438040020400000000202400000000002002a0210402000622010000000001700144000040000000002204000000c000410105024010000808000000002004002000000261000000822200200800881000000012500400400000000000000040010000800000","miner":"0x000095e79eac4d76aab57cb2c1f091d553b36ca0","mixHash":"0x5b53dc49cbab268ef9950b1d81b5e36a1b2f1b97aee1b7ff6e4db0e06c29a8b0","nonce":"0x0000000000000000","number":"0x84161e","parentHash":"0x72d92c1498e05952988d4e79a695928a6bcbd37239f8a1734051263b4d3504b8","receiptsRoot":"0xaff90ae18dcc35924a4bddb68d403b8b7812c10c3ea2a114f34105c87d75bcdb","sha3Uncles":"0x1dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347","size":"0x2a51","stateRoot":"0xc56738518b2c7854a640ae25996d2211c9ef0dd2e4dd9e59e9d9cacef39622da","timestamp":"0x64110a5c","totalDifficulty":"0xa4a470","transactions":[{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x1b7aa44088a0ea95bdc65fef6e5071e946bf7d8f","gas":"0x5208","gasPrice":"0x4159f01735","hash":"0x1e8f148a9aea7d8d16ea6e9446723b8f262e8bcd89c7c961d52046ebd43b4598","input":"0x","nonce":"0x279ad","to":"0xa21765a03dd41e2783696d314f235f4d520f6cac","transactionIndex":"0x0","value":"0x3782dace9d90000","type":"0x0","chainId":"0x5","v":"0x2e","r":"0xf9eca8b74ccaec5f4a83e981baeb8602f38d0271179c592b4a18f3866316a367","s":"0x350a9d26718d1fa9dc4f268f9c0e18fb96dbc037816ca8fc20f33f1a76f9f501"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x4d9a852e6aecd3a6e87fece2ca109780e45e6f2d","gas":"0x160ab","gasPrice":"0xd4babc57","maxFeePerGas":"0x11407d03e","maxPriorityFeePerGas":"0x9502f900","hash":"0xab5c870f4c367012bd763172afbfbe68fbf35336a66ae41aff3f2c9dbf4ea3f8","input":"0x60566037600b82828239805160001a607314602a57634e487b7160e01b600052600060045260246000fd5b30600052607381538281f3fe73000000000000000000000000000000000000000030146080604052600080fdfea26469706673582212200c479c99d58d7fbee5ee3aef5001431073a3d465bdfa46cf747a7c0b9289607864736f6c634300080d0033","nonce":"0xafd","to":null,"transactionIndex":"0x1","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x48f6acf4ac4371eb96ffca8cdef5b7704ea8c68a631d1c02927036d4ce92567e","s":"0x1f12261ddde63fd3da2ed9ea1bcbb4b0f25af898248c805a9d4b0a6e42caf9c8","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0xce297b30e56bf2f6142c4048fbae1cb8a756fefd","gas":"0x557300","gasPrice":"0xd4babc57","maxFeePerGas":"0xfabdd183","maxPriorityFeePerGas":"0x9502f900","hash":"0xa81fd92b2d0f0bbd3cc355f869cca3243c98c5e2641db9ecf3eeabb3b13bff6a","input":"0x7bbaf1ea0000000000000000000000000000000000000000000000000000000000000005000000000000000000000000000000000000000000000000000000000000004000000000000000000000000000000000000000000000000000000000000000200000000000000000000000000000000000000000000000000000000000000000","nonce":"0xad92","to":"0x4c0ce02c1219ce5d2afffba97e484272a4637b49","transactionIndex":"0x2","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0xd11b842a5f7b0e020470e764b443fafc4044327bca8dbb37131425359abace2","s":"0x1009867573946fb0a4a35339a6eaaece4d0a168b2346539076e6319e9a466b1","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x47a916d0b694c95bc0c09953c1a6e22c8867be32","gas":"0x5208","gasPrice":"0x991ff257","maxFeePerGas":"0xb2fea055","maxPriorityFeePerGas":"0x59682f00","hash":"0xa92c7b720c08c83f1a0ed7e4c163200e30a3a8c03fcc5a51e685ea20cd0cb577","input":"0x","nonce":"0x2","to":"0xf955dfe99bbfd06a8be208274cd654ecfb66a45e","transactionIndex":"0x3","value":"0x11c37937e08000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xeecd431225215c3ddc2e13cf16fa29f9a2ab1afbaa02988f3ceac27416edf0f5","s":"0x6494062330a9d509f7c778cb45cbcc9932e5f9b4ec94a1ec315f579f8d973b7b","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x672d214fe4db218defae698085a3ccefc773dfa9","gas":"0xf4f6","gasPrice":"0x991ff257","maxFeePerGas":"0xb2fea055","maxPriorityFeePerGas":"0x59682f00","hash":"0x6921b429ad2ec1e97d3457049ad2e893b5a0349beba47ca1c74a9540af75347a","input":"0xa1ddc546000000000000000000000000000000000000000000000000000000000000118f","nonce":"0x8e","to":"0xdef65f8e2f62bde60cd86ad3b4ad7fa6741cf175","transactionIndex":"0x4","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x4732ad21401b509bebb947a8566127fba1e245eb5a262ddc522d6867ab80567e","s":"0x2153013617fefdb06db0a7316d8883056441d33c2abf5ed6291fa0e3cc5c3a2f","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x0b12f7f996579fe7d9302d2574910121c214ee15","gas":"0x56a8","gasPrice":"0x991ff257","maxFeePerGas":"0xb2fea055","maxPriorityFeePerGas":"0x59682f00","hash":"0xf776b2da0b835dde05d0d8b76fd19385d61e7055036cf637f804b36dc94f2384","input":"0x71ec5c05aa669c4922569c1d33f7a81aaa21813800000000000000000000000013a0c5930c028511dc02665e7285134b6d11a5f40000000000000000000000000b12f7f996579fe7d9302d2574910121c214ee150000000000000000000000000000000000000000000000000000000000000000","nonce":"0xe","to":"0x7c125c1d515b8945841b3d5144a060115c58725f","transactionIndex":"0x5","value":"0x2386f26fc10000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xc4505d4c46ef9a0103178e5ef4fbc857114ae5c4686b68b74d07c3ee3a86e575","s":"0x74db34fd675bea7d2ee74f14fd8709cb88c7d22c2267d12f2c671b5ebc180bd5","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0xd36aac0c9676e984d72823fb662ce94d3ab5e551","gas":"0xff212","gasPrice":"0x991ff257","maxFeePerGas":"0xd86d063e","maxPriorityFeePerGas":"0x59682f00","hash":"0x9a08d899cd14ebb930ed59fa774afdb88a22615b3a931e930931ea54d26dc0bc","input":"0x49160658babdf216946a22b96c966ceeba750f14807726306ebeed500a2621a7481df010000000000000000000000000000000000000000000000000000000000000008000000000000000000000000000000000000000000000000000000000000000c0000000000000000000000000000000000000000000000000000000000000012000000000000000000000000000000000000000000000000000000000000000084d6f6f6e6265616d000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000002a3078434133633746363939643335333137316438434643364236334431433764333841333233373238300000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000c0b616f73c8d00641109f0000054f967b8a135e44e233da070ea88c5e4a53bacac0000000000000000000000000000000000000000000000000000000000000002000000000000000000000000c6a3a17399d88e5dfd2de4fa41dcccade6a2d2170000000000000000000000008676147503f761e44b85df01811edd52be01b56c000000000000000000000000000000000000000000000000d02ab486cedc00000000000000000000000000000000000000000000000000000de0b6b3a7640000","nonce":"0x7121","to":"0x2d276bd64fe8643a1d76b2e9530c753423e972c1","transactionIndex":"0x6","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0xce452361d331baf2ac86947a1b3b454b3e842923e1f39139b33a572845dd5d7c","s":"0x388126881782af5a146ce8409f1e3a2c9a4cadcfe9a2c1d27258af6e4eb5cec1","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0xb83a48ee322989a0a4f838cb207abc28ec021b73","gas":"0x326ff","gasPrice":"0x991ff257","maxFeePerGas":"0x5c3362b9c","maxPriorityFeePerGas":"0x59682f00","hash":"0x0fe0d97e25d5eb11a33a3e8278584c3780941fc2675bdf8fc547cee3d1fd3b17","input":"0xa4110dbe190e65a7fe0ad5acd305c6446a02cb39de30f382000fe67927434063bf4c4b2ca86b3ee6d039dc7a0268f9e1c39266c3f12a752a7eb72f0895dedf62291fd292","nonce":"0x12e","to":"0x96e6192eeaf7bb308f79fb5017a9085754b9e12a","transactionIndex":"0x7","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x46819bb540fa459a5a0cbf041a0286507d597fc8958a6af8b9ce193f00005238","s":"0x31495c58bf4b74925f870c59635912681bee947b1cb4b08cc19f22421b1a10ab","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0xb83a48ee322989a0a4f838cb207abc28ec021b73","gas":"0x3270b","gasPrice":"0x991ff257","maxFeePerGas":"0x5c3362b9c","maxPriorityFeePerGas":"0x59682f00","hash":"0xef47a60f57f177a683c723c658137efab66d311e1c5abbc4d74f653535144d03","input":"0xa4110dbe18eba3e815ec7fcb03a8eea0487209c3253024d1dc4f36f775a43fdd3e409b74c0a5c88e54f39b88db6f185a41431b204120d3e66c9b08b86f5e8382623327e5","nonce":"0x12f","to":"0x96e6192eeaf7bb308f79fb5017a9085754b9e12a","transactionIndex":"0x8","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x4d7e11fe3914763a2735493a62b71ae0028e75a750a8b06407996cf478e273c8","s":"0x60c18fdf948721f96fdc2efbf694910122e82191c7a40a5caab247c06ab7f3cf","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x7712a1f96b3f37c3dd88a22d4f7a7977b937fe62","gas":"0x1ff06","gasPrice":"0x991ff257","maxFeePerGas":"0xcf662006","maxPriorityFeePerGas":"0x59682f00","hash":"0xe23a5b35faae5335adc5aca38c5d633b00438b798c2053104b8df48406c9b141","input":"0x3ff9751b0000000000000000000000000000000000000000000000000000000000000034000000000000000000000000000000000000000000000000000000000000003500000000000000000000000000000000000000000000000000000000000000a000000000000000000000000000000000000000000000000000000000000001000000000000000000000000000000000000000000000000000000000000000160000000000000000000000000000000000000000000000000000000000000002e516d566e75374a51566f445271536748427a72615970374879373848774a744c4669366e5546436f775447647a70000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000002e516d5439716b3343525962464457704446596541763854384831676e6f6e67774b6868354a36384e4c6b4c6972360000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000041c9f31b989aaa745aecc563139f89c8ab3ab492e725d1e0305b7d602fcbf9293c463f0d2e4f5fb0c9efb54eb2cde9af10ebc9c3676f2334007291ffa8efb40adb1c00000000000000000000000000000000000000000000000000000000000000","nonce":"0x16","to":"0xc3b87ea584cb9cb5a4523e2bed16a34bf7746317","transactionIndex":"0x9","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xcc693a9544394653e1f393f02635e2c1f78b5e6c26c4d00a867bcf4adc2bc09e","s":"0x23f4b71b773c19f77cf251b2aeea03f2696bddd847d1f6d75acd2fbea6a543e4","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x12bbafcfa06423d87bed591a64c1a1f34791e90a","gas":"0x1b2e020","gasPrice":"0x8f0d3617","maxFeePerGas":"0x1cb3abd5a","maxPriorityFeePerGas":"0x4f5572c0","hash":"0xd8cea4ba619b317bc05d58534af73beec6c2548b31b24d4dc61c9bbd29cfa17a","input":"0x","nonce":"0x4d","to":"0x12bbafcfa06423d87bed591a64c1a1f34791e90a","transactionIndex":"0xa","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x8dae28608985d6ab47cbe122ee88972e38664cede30978f41944f347bad86e5c","s":"0x17c2898b39b7e3b874dc78e7df3fcff68f4d715385c632aad4bdf6e62505651c","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x6999c36394dd5d3d3295af63a4736ad6bd279448","gas":"0x14d94","gasPrice":"0x7b528d57","maxFeePerGas":"0x7df743e0","maxPriorityFeePerGas":"0x3b9aca00","hash":"0x79a4b9d90b02c768baaad305f266281213cc75062cbe99a13222cc0c4b509498","input":"0x6a7612020000000000000000000000009008d19f58aabd9ed0d60971565aa8510560ab4100000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000140000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000022000000000000000000000000000000000000000000000000000000000000000a4ec6cb13f00000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000000000000000000001000000000000000000000000000000000000000000000000000000000000003826d8d7371becafc0afedc3fe85b699b87f4357d55ec39ce83c510ed7d70ff514dce8236351f4627ccec6a7d9a67bc6fc2ec17ca76411113c00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000410000000000000000000000006999c36394dd5d3d3295af63a4736ad6bd27944800000000000000000000000000000000000000000000000000000000000000000100000000000000000000000000000000000000000000000000000000000000","nonce":"0x13","to":"0xdce8236351f4627ccec6a7d9a67bc6fc2ec17ca7","transactionIndex":"0xb","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x669487011643510f7512ec15b07d74efb853ba4ad9fe9c7e9850ccf24e3d4c53","s":"0x2191f9ee81087e1df833e69d6129ec330ed4c99be2031060834e22c589fcf79b","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x08ebc052e6353b4d8392113f0f2c5decd691ddcd","gas":"0x1b2e020","gasPrice":"0x7b528d57","maxFeePerGas":"0xf6e4d9c4","maxPriorityFeePerGas":"0x3b9aca00","hash":"0x6790a3bbddbeb21fcb736a59b3775755051c3a6344d8390cf8ca27f2e8a814f0","input":"0x6871ee40","nonce":"0x4","to":"0xb21b333c40354393f18de57f2d0a0fc21f270482","transactionIndex":"0xc","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0xa7a58ab858e5dd87b9a74d550e5716ac40ddefd1c21d5baa51e542f4f941d73e","s":"0x1fd0cb1563df0368bd332717d7c8210187cd1ac2ff44f61f5242fc02eaa0c2fe","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x08ebc052e6353b4d8392113f0f2c5decd691ddcd","gas":"0x12c36","gasPrice":"0x7b528d57","maxFeePerGas":"0xe596c292","maxPriorityFeePerGas":"0x3b9aca00","hash":"0x87ec7ace5442db252b5751ffddd38dcb04b088d36b6b0e526ff25607a4293c81","input":"0xcb40cbfd","nonce":"0x5","to":"0xb21b333c40354393f18de57f2d0a0fc21f270482","transactionIndex":"0xd","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x8e4bd40195bdfaa868af97dbf22e0b74295d95564a8381bc9bcfe1af3cde2cdb","s":"0x62057dc87c8b2b197d12c648233aab4134b7c2a2f65db89a49728564f3d0864d","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x08ebc052e6353b4d8392113f0f2c5decd691ddcd","gas":"0x12c36","gasPrice":"0x7b528d57","maxFeePerGas":"0xe596c292","maxPriorityFeePerGas":"0x3b9aca00","hash":"0x40cb487ecffda94f97ce7fc0f7163f2f024235df2c8291169edc80dac063e6d0","input":"0xcb40cbfd","nonce":"0x6","to":"0xb21b333c40354393f18de57f2d0a0fc21f270482","transactionIndex":"0xe","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x993c4386ffc2bd63a48dd31cf5f65aa355cba3c063b08778b328d748937098e5","s":"0x203b1faf595628dd78e760bcc1740065625abb6cf00b1b17224e10fa6b424e43","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0xafc7682751d3654f34958e3fb660971bf5918367","gas":"0x1b2e020","gasPrice":"0x7b528d57","maxFeePerGas":"0xefb40664","maxPriorityFeePerGas":"0x3b9aca00","hash":"0xb76bb3d88c9b30d927c45ccfcf8d5b0054411ac8501ad588822a7d04690cccf6","input":"0x40c10f19000000000000000000000000afc7682751d3654f34958e3fb660971bf59183670000000000000000000000000000000000000000000000008ac7230489e80000","nonce":"0x20","to":"0x84ced17d95f3ec7230baf4a369f1e624ae60090d","transactionIndex":"0xf","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x78e9d29c606f6a61a1428ed7326df05210d802c3061f3b469847cc461eb48025","s":"0x517afe5a21175f186233ae7a6c090cd33e028a77e5854e98eb2fa4c9284430e7","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x12bbafcfa06423d87bed591a64c1a1f34791e90a","gas":"0xcafe","gasPrice":"0x7b528d57","maxFeePerGas":"0x16ba45bce","maxPriorityFeePerGas":"0x3b9aca00","hash":"0x798ebe823209869347c08bd81e04fbf60e9bdfe44b1cc923215182d0cf3d4edb","input":"0x40c10f1900000000000000000000000012bbafcfa06423d87bed591a64c1a1f34791e90a0000000000000000000000000000000000000000000000000000000000000032","nonce":"0x4e","to":"0xd25d682189afa7ed0c2811195b3f3d07616927bc","transactionIndex":"0x10","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0xbe2aa95f4c32188c9ecbae97286ab983f7cc5a7017d46c404f8eb80b55e2dc13","s":"0x55d8968f509f51633f49260178941fd5a3db999e1995206348d7a3f05152b3d5","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x12bbafcfa06423d87bed591a64c1a1f34791e90a","gas":"0xcafe","gasPrice":"0x7b528d57","maxFeePerGas":"0xc6132c0c","maxPriorityFeePerGas":"0x3b9aca00","hash":"0xbe68a7e02725f799a65ebb069ccc83a014ac7c40e4119bf7c220a2f6ddfee295","input":"0x40c10f1900000000000000000000000012bbafcfa06423d87bed591a64c1a1f34791e90a0000000000000000000000000000000000000000000000000000000000000064","nonce":"0x4f","to":"0xd25d682189afa7ed0c2811195b3f3d07616927bc","transactionIndex":"0x11","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x1750ed45d0d2d048a61dcc3ff3e7ad0cbd395b77698c4f2eeb6baaea4c966d96","s":"0xe646e3a87b6dc50439468bcfd24cbf12a6f95b2762f1527c97a7d1b7f03530e","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0xd0c0338baac9ce09c1d1caab88b4360507061e37","gas":"0x1b2e020","gasPrice":"0x7b528d57","maxFeePerGas":"0x1918167aa","maxPriorityFeePerGas":"0x3b9aca00","hash":"0xc90c3a72efe81331727fcce4b5bd4906066da314ca9a0b44023a6b09ea7e8114","input":"0x4e71d92d","nonce":"0xe7","to":"0x0137d796936e199b917b93118eb4e002508a48bd","transactionIndex":"0x12","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x22222ce8fe753082c8e5dc7478fa7b87db9c8d0fb070ebc88ab015db13fd6bb","s":"0x6dceadecf6f6a3ddc433a547dba6d5fcdfb7d006ce09eeae1c06c1828ef9fff9","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x8bd834cc1dd17d1a2f7eb6140c3dd5ab51d0f338","gas":"0x1b2e020","gasPrice":"0x7b528d57","maxFeePerGas":"0x1a4019792","maxPriorityFeePerGas":"0x3b9aca00","hash":"0x619a6cbd43cde074d314c19623bd66d9fb1e13c158d7138775236f798dc1245e","input":"0x1da1db5e00000000000000000000000000000000000000000000000000001b5ec4ff740d","nonce":"0x10","to":"0xbad0009f7a8e796e3f975dc1d6ab3e3c237b857c","transactionIndex":"0x13","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x733c496e8c84f78d113bc8b5ad6d04a133bb10bbcecf0db93d97eeedfa3d984a","s":"0x6627c16aa7dea27db44c0f0641c30ffd544b9b9b1c299dc60611e1e01c714ee8","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x8bd834cc1dd17d1a2f7eb6140c3dd5ab51d0f338","gas":"0x11df0","gasPrice":"0x7b528d57","maxFeePerGas":"0x1d3fbccb8","maxPriorityFeePerGas":"0x3b9aca00","hash":"0xca5a56cd77b9e5b0e79020cc6346edf205bc11e901984d805125f28c2e6686e6","input":"0xd7c01032000000000000000000000000356f24c0f3187a6dcb949e4a60aa0a74a8a0327e0000000000000000000000000f8bacfae87aee49a2eec86df0c6b958ec3fd4fd0000000000000000000000000f8bacfae87aee49a2eec86df0c6b958ec3fd4fd","nonce":"0x11","to":"0xbad0009f7a8e796e3f975dc1d6ab3e3c237b857c","transactionIndex":"0x14","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x4aa872c1a7da70126917604193717575b91407f82449bca956d4089b90cd902","s":"0x4b5f788c68024f8958f66e8c768a8e0f65ef297c4ffd420a715258b0a55cb0a0","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x9bedd66b05712a1add7d1fec899b8641b3ecc863","gas":"0x1b2e020","gasPrice":"0x7b528d57","maxFeePerGas":"0x1e706c8d2","maxPriorityFeePerGas":"0x3b9aca00","hash":"0x999c9ddeed67c6ef6fbf02a6e977a6c1b68e18d24814e51643c7157b87a43e0a","input":"0xa8b78ed0","nonce":"0x3","to":"0xf6c2afb18b85175e5ffb28f20f007c047ce28592","transactionIndex":"0x15","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xf1ca074104ee0b6a67223afd797654b8f21060453f7662dd7be79de938c396da","s":"0x4c9a4e56f0ad21e2dd8d1116c7a3e41ea2c6126853772f7897bd959e81e2e28a","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x9bedd66b05712a1add7d1fec899b8641b3ecc863","gas":"0x2dc6c0","gasPrice":"0xd4babc57","maxFeePerGas":"0x21c13e710","maxPriorityFeePerGas":"0x9502f900","hash":"0x47c8f5d0b3778e4c34eba7fcc356fa04a5afd954ccf484728e72c002764dd3c4","input":"0xf688bcfb0000000000000000000000000000000000000000000000000000000005f5e100","nonce":"0x4","to":"0xf6c2afb18b85175e5ffb28f20f007c047ce28592","transactionIndex":"0x16","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x970982425d18b9949141bbabb5c5230fa168626ce210a97f3856d64700fc0eeb","s":"0x6fa6d0aea6ae48783d80c1e51cda47e11c28b90a5497579b93d7f242d786f67e","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x4b596cebd02c35aa99878eb9ecb24d6f5ca5f865","gas":"0x5208","gasPrice":"0x68c88f79","maxFeePerGas":"0x68c88f79","maxPriorityFeePerGas":"0x68c88f79","hash":"0x396797ae0ebcdb72ff1f96fd08b6128f78acc7417353f142f1a5facd425a33e6","input":"0x","nonce":"0x1","to":"0x375980b8e1a417620ba9b18e5a9e0b17dccbac9d","transactionIndex":"0x17","value":"0x1c9a406c58878","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xcb2a7b106097b2f2a61d3a048054a5175c386b1dcdd5b6fcd7849dcac7c32a66","s":"0x10aef76e98276c47470f04a599b81c260ace9d2f6adc95755c6d22d859e805a2","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x6f4aa3bc9732372daa288beb9b2516374ace5786","gas":"0x1b2e020","gasPrice":"0x61010bda","maxFeePerGas":"0x61010bda","maxPriorityFeePerGas":"0x61010bda","hash":"0x454aa43d6546a6f62246826c16b7a49c6c704238c18802ef0d659922f23a573c","input":"0xe99a3f8000000000000000000000000000000000000000000000000000000000000000800000000000000000000000000000000000000000000000000000000000000480000000000000000000000000000000000000000000000000000000000000050000000000000000000000000000000000000000000000000000000000000008a000000000000000000000000058ce7eb44ae575fb3b3974861c1138dad3c5aee40000000000000000000000000000000000000000000000000000000000000120000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000001e06defd69964a6ce467b87c64627a7a2be4be139b504757c625eb6114ec166557d000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000641a269023d235ef0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000002a000000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000002b05699353b600008ae85d840000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000000000000000000020000000000000000000000000b4fbf271143f4fbf7b91a5ded31805e42b2208d600000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000000000000000000001f63c28250000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000000000000000000020000000000000000000000000def65f8e2f62bde60cd86ad3b4ad7fa6741cf17500000000000000000000000000000000000000000000000000000000000001400000000000000000000000000000000000000000000000000000000000000020000000000000000000000000000000000000000000000000000000000000006000000000000000000000000000000000000000000000000000000000000000800000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000020000000000000000000000006e512435020ac228a15543236b1f46656f98136300000000000000000000000000000000000000000000000000000000000000fa000000000000000000000000d6ffd79b52a587a0a9941a61f4e6cb0d386d5458000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000413e24433da3efc4c3cc305339a7b77a1b31abb4e4d4967e97e42aed1dd0bf98936f43035f40bc80f6f6e3a9cfb43524a1171ee7f21e9ac2fa586d33a31f10619d1c000000000000000000000000000000000000000000000000000000000000000000000000000000000000006f4aa3bc9732372daa288beb9b2516374ace5786000000000000000000000000000000000000000000000000000000000000012000000000000000000000000058ce7eb44ae575fb3b3974861c1138dad3c5aee400000000000000000000000000000000000000000000000000000000000002000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000641a269023d235ef0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000002c00000000000000000000000000000000000000000000000000000000000000040000000000000000000000000000000000000000000000000000000000000000173ad21460000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000000000000000000040000000000000000000000000def65f8e2f62bde60cd86ad3b4ad7fa6741cf1750000000000000000000000000000000000000000000000000000000000000ba200000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000002b05699353b600008ae85d840000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000000000000000000020000000000000000000000000b4fbf271143f4fbf7b91a5ded31805e42b2208d600000000000000000000000000000000000000000000000000000000000000c00000000000000000000000000000000000000000000000000000000000000020000000000000000000000000000000000000000000000000000000000000006000000000000000000000000000000000000000000000000000000000000000800000000000000000000000000000000000000000000000000000000000000001000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000def65f8e2f62bde60cd86ad3b4ad7fa6741cf175ffffffff09616c6c64617461","nonce":"0x82","to":"0x02afbd43cad367fcb71305a2dfb9a3928218f0c1","transactionIndex":"0x18","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0xc7ac308b1ae238cb2aa4b030ca79baffb7b498e1b10df0c3354b3cc19cfabac9","s":"0x2358b1093b9aab5fae0cf751766b0bb2e3d8cd2b18ca627e1e12520990a8ed6d","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x6ece61d29b90642941c41240da924690da145696","gas":"0x4399c","gasPrice":"0x506794e0","hash":"0x317ecb5bd19caa42a69f836d41556ebb0e0e00e1c6cd2dee230e6e6192612527","input":"0x46f83b5000000000000000000000000000000000000000000000000000000000000000654666803decc59e7d0a5d3efa3c3653c05645ef4342738445e20a3b43e0b4aa8b000000000000000000000000000000000000000000000000000000000000000f4666803decc59e7d0a5d3efa3c3653c05645ef4342738445e20a3b43e0b4aa8b","nonce":"0x2cc4","to":"0xa6bf2be6c60175601bf88217c75dd4b14abb5fbb","transactionIndex":"0x19","value":"0x0","type":"0x0","chainId":"0x5","v":"0x2d","r":"0x37aeab0db6d550e6cba2c04c9cb10a2af0fec0b11f6e04b85e1ea0b8b9defeae","s":"0x4f32f4ef74872e1385c25e69bf21ee049545a4d423e939eaff29677450efe9eb"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0xc4a5c059840a146b81c77bf79f1398f2b3d28ca7","gas":"0x432ec","gasPrice":"0x45db0a51","hash":"0xc879285db5ef0a6bce98021584d16f134c1dc0aed8cc988802c4f72ba6877ff6","input":"0x9f8420b300000000000000000000000000000000000000000000000000470de4df8200000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x1a7","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0x1a","value":"0x470efc4514e000","type":"0x0","chainId":"0x5","v":"0x2d","r":"0x237e3a541c5ab490843cdc20d0fcb7416bff8da471f6a11ae1b32d2bab119141","s":"0x3c8417d7a60bd5b0bcae4975dbb4aaee54a4b4d588226481be400843bfe2b64e"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x6f4aa3bc9732372daa288beb9b2516374ace5786","gas":"0xffff","gasPrice":"0x44e9acbc","maxFeePerGas":"0x44e9acbc","maxPriorityFeePerGas":"0x44e9acbc","hash":"0xecaa2d6f597608307e5084854854ba6dc1e69395e2abea14f2c6a2fa1d6faf9a","input":"0xe2864fe300000000000000000000000000000000000000000000000000000000000000200000000000000000000000006f4aa3bc9732372daa288beb9b2516374ace57860000000000000000000000000000000000000000000000000000000000000120000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000002004a8312e37fcecd450e6684009a2b54bc64f66afbe5217c7b2f63b03cd9980a9e00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000064893e8223d235ef0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000002a00000000000000000000000000000000000000000000000000000000000000040000000000000000000000000000000000000000000000000000000000000000173ad21460000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000000000000000000040000000000000000000000000def65f8e2f62bde60cd86ad3b4ad7fa6741cf175000000000000000000000000000000000000000000000000000000000000085000000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000000de0b6b3a7640000aaaebeba000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000040000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000001400000000000000000000000000000000000000000000000000000000000000020000000000000000000000000000000000000000000000000000000000000006000000000000000000000000000000000000000000000000000000000000000800000000000000000000000000000000000000000000000000000000000000001000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000020000000000000000000000006e512435020ac228a15543236b1f46656f98136300000000000000000000000000000000000000000000000000000000000000fa000000000000000000000000d6ffd79b52a587a0a9941a61f4e6cb0d386d54580000000000000000000000000000000000000000000000000000000000000000","nonce":"0x83","to":"0x02afbd43cad367fcb71305a2dfb9a3928218f0c1","transactionIndex":"0x1b","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xa18772e4b9e1493ebfd5ae6219cc9ca060b71e83788fb614ab02b03fb36eaf1b","s":"0x482c7e50fde29bb4130fd6e723176e0e9913430ebb9260667d7fb55f5408d9c5","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0xdb33365a8730de2f7574ff1189fb9d337bf4c36d","gas":"0x61a80","gasPrice":"0x433653c8","maxFeePerGas":"0x828367af","maxPriorityFeePerGas":"0x37e9071","hash":"0x4dd69b69a568ff30ae439e2ded72fbd7f2e7aaa345836703663f155c749c5eed","input":"0x7f34c0ddc6e984f46ab7c13b7a53643096e79c39103dc74550204d5bd1b5206ef5a2f49689071bb6a332212df26aa57cd4d1e3c0894187a9fc5e5d686cbedba5b187470f000000000000000000000000000000000000000000000000000000000000000e000000000000000000000000e8c44c30e8e52c38ac7bd232e7b7f0c7463a535100000000000000000000000000000000000000000000000000000000000000a0000000000000000000000000000000000000000000000000000000000000002e516d5044645856613738437a6b7856325a364e52777a67685a785872736e75397261616a695a7555423569465a62000000000000000000000000000000000000","nonce":"0x2b57","to":"0xb8ad54644e2371cbb7940b9fa0936f399ba7495f","transactionIndex":"0x1c","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0xa1fb0e5662b54ceb3dc1623ebfb200668433cb27252d553669bcd136387394f3","s":"0x27c2f8e19f69a52a0960f05d8b21f11c313dd3e86a7139b5783a18353c407415","yParity":"0x0"}],"transactionsRoot":"0x4a87d0cf5990b1c5bac631583e5965c2ba943858bebb2e07f74d0b697f73821a","uncles":[],"withdrawals":[{"index":"0x1170","validatorIndex":"0x38c2c","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x66edfd65"},{"index":"0x1171","validatorIndex":"0x38c2d","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6cd228e4"},{"index":"0x1172","validatorIndex":"0x38c2e","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x77f3431b"},{"index":"0x1173","validatorIndex":"0x38c2f","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6b61f268"},{"index":"0x1174","validatorIndex":"0x38c30","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6e10bb21"},{"index":"0x1175","validatorIndex":"0x38c31","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6eb115a5"},{"index":"0x1176","validatorIndex":"0x38c32","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x7caead1d"},{"index":"0x1177","validatorIndex":"0x38c33","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x772c0ddf"},{"index":"0x1178","validatorIndex":"0x38c34","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x75930a95"},{"index":"0x1179","validatorIndex":"0x38c35","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x76a4db09"},{"index":"0x117a","validatorIndex":"0x38c36","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x7e692b27"},{"index":"0x117b","validatorIndex":"0x38c37","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x72038ae6"},{"index":"0x117c","validatorIndex":"0x38c38","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6ccce352"},{"index":"0x117d","validatorIndex":"0x38c39","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x79ef6898"},{"index":"0x117e","validatorIndex":"0x38c3a","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6d58977d"},{"index":"0x117f","validatorIndex":"0x38c3b","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x76f7d208"}],"withdrawalsRoot":"0xaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa"} diff --git a/op-service/sources/testdata/data/blocks/post-shanghai-bad-withdrawals_metadata.json b/op-service/sources/testdata/data/blocks/post-shanghai-bad-withdrawals_metadata.json new file mode 100644 index 000000000000..16d4bd5bcc0e --- /dev/null +++ b/op-service/sources/testdata/data/blocks/post-shanghai-bad-withdrawals_metadata.json @@ -0,0 +1 @@ +{"name":"post-shanghai-bad-withdrawals","fail":true} diff --git a/op-service/sources/testdata/data/blocks/post-shanghai-success_data.json b/op-service/sources/testdata/data/blocks/post-shanghai-success_data.json new file mode 100644 index 000000000000..fea85638547f --- /dev/null +++ b/op-service/sources/testdata/data/blocks/post-shanghai-success_data.json @@ -0,0 +1 @@ +{"baseFeePerGas":"0x3fb7c357","difficulty":"0x0","extraData":"0x","gasLimit":"0x1c9c380","gasUsed":"0x18f759","hash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","logsBloom":"0x020010404000001a0000021000000080001100410000100001000010040200980220400000008806200200000100000000000000000000008000000400042000000050000040000112080808800002044000040004042008800480002000000000000002020020000042002400000820000080040000000010200010020010100101212050000008000000008000001010200c80000112010000438040020400000000202400000000002002a0210402000622010000000001700144000040000000002204000000c000410105024010000808000000002004002000000261000000822200200800881000000012500400400000000000000040010000800000","miner":"0x000095e79eac4d76aab57cb2c1f091d553b36ca0","mixHash":"0x5b53dc49cbab268ef9950b1d81b5e36a1b2f1b97aee1b7ff6e4db0e06c29a8b0","nonce":"0x0000000000000000","number":"0x84161e","parentHash":"0x72d92c1498e05952988d4e79a695928a6bcbd37239f8a1734051263b4d3504b8","receiptsRoot":"0xaff90ae18dcc35924a4bddb68d403b8b7812c10c3ea2a114f34105c87d75bcdb","sha3Uncles":"0x1dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347","size":"0x2a51","stateRoot":"0xc56738518b2c7854a640ae25996d2211c9ef0dd2e4dd9e59e9d9cacef39622da","timestamp":"0x64110a5c","totalDifficulty":"0xa4a470","transactions":[{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x1b7aa44088a0ea95bdc65fef6e5071e946bf7d8f","gas":"0x5208","gasPrice":"0x4159f01735","hash":"0x1e8f148a9aea7d8d16ea6e9446723b8f262e8bcd89c7c961d52046ebd43b4598","input":"0x","nonce":"0x279ad","to":"0xa21765a03dd41e2783696d314f235f4d520f6cac","transactionIndex":"0x0","value":"0x3782dace9d90000","type":"0x0","chainId":"0x5","v":"0x2e","r":"0xf9eca8b74ccaec5f4a83e981baeb8602f38d0271179c592b4a18f3866316a367","s":"0x350a9d26718d1fa9dc4f268f9c0e18fb96dbc037816ca8fc20f33f1a76f9f501"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x4d9a852e6aecd3a6e87fece2ca109780e45e6f2d","gas":"0x160ab","gasPrice":"0xd4babc57","maxFeePerGas":"0x11407d03e","maxPriorityFeePerGas":"0x9502f900","hash":"0xab5c870f4c367012bd763172afbfbe68fbf35336a66ae41aff3f2c9dbf4ea3f8","input":"0x60566037600b82828239805160001a607314602a57634e487b7160e01b600052600060045260246000fd5b30600052607381538281f3fe73000000000000000000000000000000000000000030146080604052600080fdfea26469706673582212200c479c99d58d7fbee5ee3aef5001431073a3d465bdfa46cf747a7c0b9289607864736f6c634300080d0033","nonce":"0xafd","to":null,"transactionIndex":"0x1","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x48f6acf4ac4371eb96ffca8cdef5b7704ea8c68a631d1c02927036d4ce92567e","s":"0x1f12261ddde63fd3da2ed9ea1bcbb4b0f25af898248c805a9d4b0a6e42caf9c8","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0xce297b30e56bf2f6142c4048fbae1cb8a756fefd","gas":"0x557300","gasPrice":"0xd4babc57","maxFeePerGas":"0xfabdd183","maxPriorityFeePerGas":"0x9502f900","hash":"0xa81fd92b2d0f0bbd3cc355f869cca3243c98c5e2641db9ecf3eeabb3b13bff6a","input":"0x7bbaf1ea0000000000000000000000000000000000000000000000000000000000000005000000000000000000000000000000000000000000000000000000000000004000000000000000000000000000000000000000000000000000000000000000200000000000000000000000000000000000000000000000000000000000000000","nonce":"0xad92","to":"0x4c0ce02c1219ce5d2afffba97e484272a4637b49","transactionIndex":"0x2","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0xd11b842a5f7b0e020470e764b443fafc4044327bca8dbb37131425359abace2","s":"0x1009867573946fb0a4a35339a6eaaece4d0a168b2346539076e6319e9a466b1","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x47a916d0b694c95bc0c09953c1a6e22c8867be32","gas":"0x5208","gasPrice":"0x991ff257","maxFeePerGas":"0xb2fea055","maxPriorityFeePerGas":"0x59682f00","hash":"0xa92c7b720c08c83f1a0ed7e4c163200e30a3a8c03fcc5a51e685ea20cd0cb577","input":"0x","nonce":"0x2","to":"0xf955dfe99bbfd06a8be208274cd654ecfb66a45e","transactionIndex":"0x3","value":"0x11c37937e08000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xeecd431225215c3ddc2e13cf16fa29f9a2ab1afbaa02988f3ceac27416edf0f5","s":"0x6494062330a9d509f7c778cb45cbcc9932e5f9b4ec94a1ec315f579f8d973b7b","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x672d214fe4db218defae698085a3ccefc773dfa9","gas":"0xf4f6","gasPrice":"0x991ff257","maxFeePerGas":"0xb2fea055","maxPriorityFeePerGas":"0x59682f00","hash":"0x6921b429ad2ec1e97d3457049ad2e893b5a0349beba47ca1c74a9540af75347a","input":"0xa1ddc546000000000000000000000000000000000000000000000000000000000000118f","nonce":"0x8e","to":"0xdef65f8e2f62bde60cd86ad3b4ad7fa6741cf175","transactionIndex":"0x4","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x4732ad21401b509bebb947a8566127fba1e245eb5a262ddc522d6867ab80567e","s":"0x2153013617fefdb06db0a7316d8883056441d33c2abf5ed6291fa0e3cc5c3a2f","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x0b12f7f996579fe7d9302d2574910121c214ee15","gas":"0x56a8","gasPrice":"0x991ff257","maxFeePerGas":"0xb2fea055","maxPriorityFeePerGas":"0x59682f00","hash":"0xf776b2da0b835dde05d0d8b76fd19385d61e7055036cf637f804b36dc94f2384","input":"0x71ec5c05aa669c4922569c1d33f7a81aaa21813800000000000000000000000013a0c5930c028511dc02665e7285134b6d11a5f40000000000000000000000000b12f7f996579fe7d9302d2574910121c214ee150000000000000000000000000000000000000000000000000000000000000000","nonce":"0xe","to":"0x7c125c1d515b8945841b3d5144a060115c58725f","transactionIndex":"0x5","value":"0x2386f26fc10000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xc4505d4c46ef9a0103178e5ef4fbc857114ae5c4686b68b74d07c3ee3a86e575","s":"0x74db34fd675bea7d2ee74f14fd8709cb88c7d22c2267d12f2c671b5ebc180bd5","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0xd36aac0c9676e984d72823fb662ce94d3ab5e551","gas":"0xff212","gasPrice":"0x991ff257","maxFeePerGas":"0xd86d063e","maxPriorityFeePerGas":"0x59682f00","hash":"0x9a08d899cd14ebb930ed59fa774afdb88a22615b3a931e930931ea54d26dc0bc","input":"0x49160658babdf216946a22b96c966ceeba750f14807726306ebeed500a2621a7481df010000000000000000000000000000000000000000000000000000000000000008000000000000000000000000000000000000000000000000000000000000000c0000000000000000000000000000000000000000000000000000000000000012000000000000000000000000000000000000000000000000000000000000000084d6f6f6e6265616d000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000002a3078434133633746363939643335333137316438434643364236334431433764333841333233373238300000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000c0b616f73c8d00641109f0000054f967b8a135e44e233da070ea88c5e4a53bacac0000000000000000000000000000000000000000000000000000000000000002000000000000000000000000c6a3a17399d88e5dfd2de4fa41dcccade6a2d2170000000000000000000000008676147503f761e44b85df01811edd52be01b56c000000000000000000000000000000000000000000000000d02ab486cedc00000000000000000000000000000000000000000000000000000de0b6b3a7640000","nonce":"0x7121","to":"0x2d276bd64fe8643a1d76b2e9530c753423e972c1","transactionIndex":"0x6","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0xce452361d331baf2ac86947a1b3b454b3e842923e1f39139b33a572845dd5d7c","s":"0x388126881782af5a146ce8409f1e3a2c9a4cadcfe9a2c1d27258af6e4eb5cec1","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0xb83a48ee322989a0a4f838cb207abc28ec021b73","gas":"0x326ff","gasPrice":"0x991ff257","maxFeePerGas":"0x5c3362b9c","maxPriorityFeePerGas":"0x59682f00","hash":"0x0fe0d97e25d5eb11a33a3e8278584c3780941fc2675bdf8fc547cee3d1fd3b17","input":"0xa4110dbe190e65a7fe0ad5acd305c6446a02cb39de30f382000fe67927434063bf4c4b2ca86b3ee6d039dc7a0268f9e1c39266c3f12a752a7eb72f0895dedf62291fd292","nonce":"0x12e","to":"0x96e6192eeaf7bb308f79fb5017a9085754b9e12a","transactionIndex":"0x7","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x46819bb540fa459a5a0cbf041a0286507d597fc8958a6af8b9ce193f00005238","s":"0x31495c58bf4b74925f870c59635912681bee947b1cb4b08cc19f22421b1a10ab","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0xb83a48ee322989a0a4f838cb207abc28ec021b73","gas":"0x3270b","gasPrice":"0x991ff257","maxFeePerGas":"0x5c3362b9c","maxPriorityFeePerGas":"0x59682f00","hash":"0xef47a60f57f177a683c723c658137efab66d311e1c5abbc4d74f653535144d03","input":"0xa4110dbe18eba3e815ec7fcb03a8eea0487209c3253024d1dc4f36f775a43fdd3e409b74c0a5c88e54f39b88db6f185a41431b204120d3e66c9b08b86f5e8382623327e5","nonce":"0x12f","to":"0x96e6192eeaf7bb308f79fb5017a9085754b9e12a","transactionIndex":"0x8","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x4d7e11fe3914763a2735493a62b71ae0028e75a750a8b06407996cf478e273c8","s":"0x60c18fdf948721f96fdc2efbf694910122e82191c7a40a5caab247c06ab7f3cf","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x7712a1f96b3f37c3dd88a22d4f7a7977b937fe62","gas":"0x1ff06","gasPrice":"0x991ff257","maxFeePerGas":"0xcf662006","maxPriorityFeePerGas":"0x59682f00","hash":"0xe23a5b35faae5335adc5aca38c5d633b00438b798c2053104b8df48406c9b141","input":"0x3ff9751b0000000000000000000000000000000000000000000000000000000000000034000000000000000000000000000000000000000000000000000000000000003500000000000000000000000000000000000000000000000000000000000000a000000000000000000000000000000000000000000000000000000000000001000000000000000000000000000000000000000000000000000000000000000160000000000000000000000000000000000000000000000000000000000000002e516d566e75374a51566f445271536748427a72615970374879373848774a744c4669366e5546436f775447647a70000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000002e516d5439716b3343525962464457704446596541763854384831676e6f6e67774b6868354a36384e4c6b4c6972360000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000041c9f31b989aaa745aecc563139f89c8ab3ab492e725d1e0305b7d602fcbf9293c463f0d2e4f5fb0c9efb54eb2cde9af10ebc9c3676f2334007291ffa8efb40adb1c00000000000000000000000000000000000000000000000000000000000000","nonce":"0x16","to":"0xc3b87ea584cb9cb5a4523e2bed16a34bf7746317","transactionIndex":"0x9","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xcc693a9544394653e1f393f02635e2c1f78b5e6c26c4d00a867bcf4adc2bc09e","s":"0x23f4b71b773c19f77cf251b2aeea03f2696bddd847d1f6d75acd2fbea6a543e4","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x12bbafcfa06423d87bed591a64c1a1f34791e90a","gas":"0x1b2e020","gasPrice":"0x8f0d3617","maxFeePerGas":"0x1cb3abd5a","maxPriorityFeePerGas":"0x4f5572c0","hash":"0xd8cea4ba619b317bc05d58534af73beec6c2548b31b24d4dc61c9bbd29cfa17a","input":"0x","nonce":"0x4d","to":"0x12bbafcfa06423d87bed591a64c1a1f34791e90a","transactionIndex":"0xa","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x8dae28608985d6ab47cbe122ee88972e38664cede30978f41944f347bad86e5c","s":"0x17c2898b39b7e3b874dc78e7df3fcff68f4d715385c632aad4bdf6e62505651c","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x6999c36394dd5d3d3295af63a4736ad6bd279448","gas":"0x14d94","gasPrice":"0x7b528d57","maxFeePerGas":"0x7df743e0","maxPriorityFeePerGas":"0x3b9aca00","hash":"0x79a4b9d90b02c768baaad305f266281213cc75062cbe99a13222cc0c4b509498","input":"0x6a7612020000000000000000000000009008d19f58aabd9ed0d60971565aa8510560ab4100000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000140000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000022000000000000000000000000000000000000000000000000000000000000000a4ec6cb13f00000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000000000000000000001000000000000000000000000000000000000000000000000000000000000003826d8d7371becafc0afedc3fe85b699b87f4357d55ec39ce83c510ed7d70ff514dce8236351f4627ccec6a7d9a67bc6fc2ec17ca76411113c00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000410000000000000000000000006999c36394dd5d3d3295af63a4736ad6bd27944800000000000000000000000000000000000000000000000000000000000000000100000000000000000000000000000000000000000000000000000000000000","nonce":"0x13","to":"0xdce8236351f4627ccec6a7d9a67bc6fc2ec17ca7","transactionIndex":"0xb","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x669487011643510f7512ec15b07d74efb853ba4ad9fe9c7e9850ccf24e3d4c53","s":"0x2191f9ee81087e1df833e69d6129ec330ed4c99be2031060834e22c589fcf79b","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x08ebc052e6353b4d8392113f0f2c5decd691ddcd","gas":"0x1b2e020","gasPrice":"0x7b528d57","maxFeePerGas":"0xf6e4d9c4","maxPriorityFeePerGas":"0x3b9aca00","hash":"0x6790a3bbddbeb21fcb736a59b3775755051c3a6344d8390cf8ca27f2e8a814f0","input":"0x6871ee40","nonce":"0x4","to":"0xb21b333c40354393f18de57f2d0a0fc21f270482","transactionIndex":"0xc","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0xa7a58ab858e5dd87b9a74d550e5716ac40ddefd1c21d5baa51e542f4f941d73e","s":"0x1fd0cb1563df0368bd332717d7c8210187cd1ac2ff44f61f5242fc02eaa0c2fe","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x08ebc052e6353b4d8392113f0f2c5decd691ddcd","gas":"0x12c36","gasPrice":"0x7b528d57","maxFeePerGas":"0xe596c292","maxPriorityFeePerGas":"0x3b9aca00","hash":"0x87ec7ace5442db252b5751ffddd38dcb04b088d36b6b0e526ff25607a4293c81","input":"0xcb40cbfd","nonce":"0x5","to":"0xb21b333c40354393f18de57f2d0a0fc21f270482","transactionIndex":"0xd","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x8e4bd40195bdfaa868af97dbf22e0b74295d95564a8381bc9bcfe1af3cde2cdb","s":"0x62057dc87c8b2b197d12c648233aab4134b7c2a2f65db89a49728564f3d0864d","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x08ebc052e6353b4d8392113f0f2c5decd691ddcd","gas":"0x12c36","gasPrice":"0x7b528d57","maxFeePerGas":"0xe596c292","maxPriorityFeePerGas":"0x3b9aca00","hash":"0x40cb487ecffda94f97ce7fc0f7163f2f024235df2c8291169edc80dac063e6d0","input":"0xcb40cbfd","nonce":"0x6","to":"0xb21b333c40354393f18de57f2d0a0fc21f270482","transactionIndex":"0xe","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x993c4386ffc2bd63a48dd31cf5f65aa355cba3c063b08778b328d748937098e5","s":"0x203b1faf595628dd78e760bcc1740065625abb6cf00b1b17224e10fa6b424e43","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0xafc7682751d3654f34958e3fb660971bf5918367","gas":"0x1b2e020","gasPrice":"0x7b528d57","maxFeePerGas":"0xefb40664","maxPriorityFeePerGas":"0x3b9aca00","hash":"0xb76bb3d88c9b30d927c45ccfcf8d5b0054411ac8501ad588822a7d04690cccf6","input":"0x40c10f19000000000000000000000000afc7682751d3654f34958e3fb660971bf59183670000000000000000000000000000000000000000000000008ac7230489e80000","nonce":"0x20","to":"0x84ced17d95f3ec7230baf4a369f1e624ae60090d","transactionIndex":"0xf","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x78e9d29c606f6a61a1428ed7326df05210d802c3061f3b469847cc461eb48025","s":"0x517afe5a21175f186233ae7a6c090cd33e028a77e5854e98eb2fa4c9284430e7","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x12bbafcfa06423d87bed591a64c1a1f34791e90a","gas":"0xcafe","gasPrice":"0x7b528d57","maxFeePerGas":"0x16ba45bce","maxPriorityFeePerGas":"0x3b9aca00","hash":"0x798ebe823209869347c08bd81e04fbf60e9bdfe44b1cc923215182d0cf3d4edb","input":"0x40c10f1900000000000000000000000012bbafcfa06423d87bed591a64c1a1f34791e90a0000000000000000000000000000000000000000000000000000000000000032","nonce":"0x4e","to":"0xd25d682189afa7ed0c2811195b3f3d07616927bc","transactionIndex":"0x10","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0xbe2aa95f4c32188c9ecbae97286ab983f7cc5a7017d46c404f8eb80b55e2dc13","s":"0x55d8968f509f51633f49260178941fd5a3db999e1995206348d7a3f05152b3d5","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x12bbafcfa06423d87bed591a64c1a1f34791e90a","gas":"0xcafe","gasPrice":"0x7b528d57","maxFeePerGas":"0xc6132c0c","maxPriorityFeePerGas":"0x3b9aca00","hash":"0xbe68a7e02725f799a65ebb069ccc83a014ac7c40e4119bf7c220a2f6ddfee295","input":"0x40c10f1900000000000000000000000012bbafcfa06423d87bed591a64c1a1f34791e90a0000000000000000000000000000000000000000000000000000000000000064","nonce":"0x4f","to":"0xd25d682189afa7ed0c2811195b3f3d07616927bc","transactionIndex":"0x11","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x1750ed45d0d2d048a61dcc3ff3e7ad0cbd395b77698c4f2eeb6baaea4c966d96","s":"0xe646e3a87b6dc50439468bcfd24cbf12a6f95b2762f1527c97a7d1b7f03530e","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0xd0c0338baac9ce09c1d1caab88b4360507061e37","gas":"0x1b2e020","gasPrice":"0x7b528d57","maxFeePerGas":"0x1918167aa","maxPriorityFeePerGas":"0x3b9aca00","hash":"0xc90c3a72efe81331727fcce4b5bd4906066da314ca9a0b44023a6b09ea7e8114","input":"0x4e71d92d","nonce":"0xe7","to":"0x0137d796936e199b917b93118eb4e002508a48bd","transactionIndex":"0x12","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x22222ce8fe753082c8e5dc7478fa7b87db9c8d0fb070ebc88ab015db13fd6bb","s":"0x6dceadecf6f6a3ddc433a547dba6d5fcdfb7d006ce09eeae1c06c1828ef9fff9","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x8bd834cc1dd17d1a2f7eb6140c3dd5ab51d0f338","gas":"0x1b2e020","gasPrice":"0x7b528d57","maxFeePerGas":"0x1a4019792","maxPriorityFeePerGas":"0x3b9aca00","hash":"0x619a6cbd43cde074d314c19623bd66d9fb1e13c158d7138775236f798dc1245e","input":"0x1da1db5e00000000000000000000000000000000000000000000000000001b5ec4ff740d","nonce":"0x10","to":"0xbad0009f7a8e796e3f975dc1d6ab3e3c237b857c","transactionIndex":"0x13","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x733c496e8c84f78d113bc8b5ad6d04a133bb10bbcecf0db93d97eeedfa3d984a","s":"0x6627c16aa7dea27db44c0f0641c30ffd544b9b9b1c299dc60611e1e01c714ee8","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x8bd834cc1dd17d1a2f7eb6140c3dd5ab51d0f338","gas":"0x11df0","gasPrice":"0x7b528d57","maxFeePerGas":"0x1d3fbccb8","maxPriorityFeePerGas":"0x3b9aca00","hash":"0xca5a56cd77b9e5b0e79020cc6346edf205bc11e901984d805125f28c2e6686e6","input":"0xd7c01032000000000000000000000000356f24c0f3187a6dcb949e4a60aa0a74a8a0327e0000000000000000000000000f8bacfae87aee49a2eec86df0c6b958ec3fd4fd0000000000000000000000000f8bacfae87aee49a2eec86df0c6b958ec3fd4fd","nonce":"0x11","to":"0xbad0009f7a8e796e3f975dc1d6ab3e3c237b857c","transactionIndex":"0x14","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x4aa872c1a7da70126917604193717575b91407f82449bca956d4089b90cd902","s":"0x4b5f788c68024f8958f66e8c768a8e0f65ef297c4ffd420a715258b0a55cb0a0","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x9bedd66b05712a1add7d1fec899b8641b3ecc863","gas":"0x1b2e020","gasPrice":"0x7b528d57","maxFeePerGas":"0x1e706c8d2","maxPriorityFeePerGas":"0x3b9aca00","hash":"0x999c9ddeed67c6ef6fbf02a6e977a6c1b68e18d24814e51643c7157b87a43e0a","input":"0xa8b78ed0","nonce":"0x3","to":"0xf6c2afb18b85175e5ffb28f20f007c047ce28592","transactionIndex":"0x15","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xf1ca074104ee0b6a67223afd797654b8f21060453f7662dd7be79de938c396da","s":"0x4c9a4e56f0ad21e2dd8d1116c7a3e41ea2c6126853772f7897bd959e81e2e28a","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x9bedd66b05712a1add7d1fec899b8641b3ecc863","gas":"0x2dc6c0","gasPrice":"0xd4babc57","maxFeePerGas":"0x21c13e710","maxPriorityFeePerGas":"0x9502f900","hash":"0x47c8f5d0b3778e4c34eba7fcc356fa04a5afd954ccf484728e72c002764dd3c4","input":"0xf688bcfb0000000000000000000000000000000000000000000000000000000005f5e100","nonce":"0x4","to":"0xf6c2afb18b85175e5ffb28f20f007c047ce28592","transactionIndex":"0x16","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x970982425d18b9949141bbabb5c5230fa168626ce210a97f3856d64700fc0eeb","s":"0x6fa6d0aea6ae48783d80c1e51cda47e11c28b90a5497579b93d7f242d786f67e","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x4b596cebd02c35aa99878eb9ecb24d6f5ca5f865","gas":"0x5208","gasPrice":"0x68c88f79","maxFeePerGas":"0x68c88f79","maxPriorityFeePerGas":"0x68c88f79","hash":"0x396797ae0ebcdb72ff1f96fd08b6128f78acc7417353f142f1a5facd425a33e6","input":"0x","nonce":"0x1","to":"0x375980b8e1a417620ba9b18e5a9e0b17dccbac9d","transactionIndex":"0x17","value":"0x1c9a406c58878","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xcb2a7b106097b2f2a61d3a048054a5175c386b1dcdd5b6fcd7849dcac7c32a66","s":"0x10aef76e98276c47470f04a599b81c260ace9d2f6adc95755c6d22d859e805a2","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x6f4aa3bc9732372daa288beb9b2516374ace5786","gas":"0x1b2e020","gasPrice":"0x61010bda","maxFeePerGas":"0x61010bda","maxPriorityFeePerGas":"0x61010bda","hash":"0x454aa43d6546a6f62246826c16b7a49c6c704238c18802ef0d659922f23a573c","input":"0xe99a3f8000000000000000000000000000000000000000000000000000000000000000800000000000000000000000000000000000000000000000000000000000000480000000000000000000000000000000000000000000000000000000000000050000000000000000000000000000000000000000000000000000000000000008a000000000000000000000000058ce7eb44ae575fb3b3974861c1138dad3c5aee40000000000000000000000000000000000000000000000000000000000000120000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000001e06defd69964a6ce467b87c64627a7a2be4be139b504757c625eb6114ec166557d000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000641a269023d235ef0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000002a000000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000002b05699353b600008ae85d840000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000000000000000000020000000000000000000000000b4fbf271143f4fbf7b91a5ded31805e42b2208d600000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000000000000000000001f63c28250000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000000000000000000020000000000000000000000000def65f8e2f62bde60cd86ad3b4ad7fa6741cf17500000000000000000000000000000000000000000000000000000000000001400000000000000000000000000000000000000000000000000000000000000020000000000000000000000000000000000000000000000000000000000000006000000000000000000000000000000000000000000000000000000000000000800000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000020000000000000000000000006e512435020ac228a15543236b1f46656f98136300000000000000000000000000000000000000000000000000000000000000fa000000000000000000000000d6ffd79b52a587a0a9941a61f4e6cb0d386d5458000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000413e24433da3efc4c3cc305339a7b77a1b31abb4e4d4967e97e42aed1dd0bf98936f43035f40bc80f6f6e3a9cfb43524a1171ee7f21e9ac2fa586d33a31f10619d1c000000000000000000000000000000000000000000000000000000000000000000000000000000000000006f4aa3bc9732372daa288beb9b2516374ace5786000000000000000000000000000000000000000000000000000000000000012000000000000000000000000058ce7eb44ae575fb3b3974861c1138dad3c5aee400000000000000000000000000000000000000000000000000000000000002000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000641a269023d235ef0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000002c00000000000000000000000000000000000000000000000000000000000000040000000000000000000000000000000000000000000000000000000000000000173ad21460000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000000000000000000040000000000000000000000000def65f8e2f62bde60cd86ad3b4ad7fa6741cf1750000000000000000000000000000000000000000000000000000000000000ba200000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000002b05699353b600008ae85d840000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000000000000000000020000000000000000000000000b4fbf271143f4fbf7b91a5ded31805e42b2208d600000000000000000000000000000000000000000000000000000000000000c00000000000000000000000000000000000000000000000000000000000000020000000000000000000000000000000000000000000000000000000000000006000000000000000000000000000000000000000000000000000000000000000800000000000000000000000000000000000000000000000000000000000000001000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000def65f8e2f62bde60cd86ad3b4ad7fa6741cf175ffffffff09616c6c64617461","nonce":"0x82","to":"0x02afbd43cad367fcb71305a2dfb9a3928218f0c1","transactionIndex":"0x18","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0xc7ac308b1ae238cb2aa4b030ca79baffb7b498e1b10df0c3354b3cc19cfabac9","s":"0x2358b1093b9aab5fae0cf751766b0bb2e3d8cd2b18ca627e1e12520990a8ed6d","yParity":"0x0"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x6ece61d29b90642941c41240da924690da145696","gas":"0x4399c","gasPrice":"0x506794e0","hash":"0x317ecb5bd19caa42a69f836d41556ebb0e0e00e1c6cd2dee230e6e6192612527","input":"0x46f83b5000000000000000000000000000000000000000000000000000000000000000654666803decc59e7d0a5d3efa3c3653c05645ef4342738445e20a3b43e0b4aa8b000000000000000000000000000000000000000000000000000000000000000f4666803decc59e7d0a5d3efa3c3653c05645ef4342738445e20a3b43e0b4aa8b","nonce":"0x2cc4","to":"0xa6bf2be6c60175601bf88217c75dd4b14abb5fbb","transactionIndex":"0x19","value":"0x0","type":"0x0","chainId":"0x5","v":"0x2d","r":"0x37aeab0db6d550e6cba2c04c9cb10a2af0fec0b11f6e04b85e1ea0b8b9defeae","s":"0x4f32f4ef74872e1385c25e69bf21ee049545a4d423e939eaff29677450efe9eb"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0xc4a5c059840a146b81c77bf79f1398f2b3d28ca7","gas":"0x432ec","gasPrice":"0x45db0a51","hash":"0xc879285db5ef0a6bce98021584d16f134c1dc0aed8cc988802c4f72ba6877ff6","input":"0x9f8420b300000000000000000000000000000000000000000000000000470de4df8200000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x1a7","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0x1a","value":"0x470efc4514e000","type":"0x0","chainId":"0x5","v":"0x2d","r":"0x237e3a541c5ab490843cdc20d0fcb7416bff8da471f6a11ae1b32d2bab119141","s":"0x3c8417d7a60bd5b0bcae4975dbb4aaee54a4b4d588226481be400843bfe2b64e"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0x6f4aa3bc9732372daa288beb9b2516374ace5786","gas":"0xffff","gasPrice":"0x44e9acbc","maxFeePerGas":"0x44e9acbc","maxPriorityFeePerGas":"0x44e9acbc","hash":"0xecaa2d6f597608307e5084854854ba6dc1e69395e2abea14f2c6a2fa1d6faf9a","input":"0xe2864fe300000000000000000000000000000000000000000000000000000000000000200000000000000000000000006f4aa3bc9732372daa288beb9b2516374ace57860000000000000000000000000000000000000000000000000000000000000120000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000002004a8312e37fcecd450e6684009a2b54bc64f66afbe5217c7b2f63b03cd9980a9e00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000064893e8223d235ef0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000002a00000000000000000000000000000000000000000000000000000000000000040000000000000000000000000000000000000000000000000000000000000000173ad21460000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000000000000000000040000000000000000000000000def65f8e2f62bde60cd86ad3b4ad7fa6741cf175000000000000000000000000000000000000000000000000000000000000085000000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000000de0b6b3a7640000aaaebeba000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000040000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000001400000000000000000000000000000000000000000000000000000000000000020000000000000000000000000000000000000000000000000000000000000006000000000000000000000000000000000000000000000000000000000000000800000000000000000000000000000000000000000000000000000000000000001000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000020000000000000000000000006e512435020ac228a15543236b1f46656f98136300000000000000000000000000000000000000000000000000000000000000fa000000000000000000000000d6ffd79b52a587a0a9941a61f4e6cb0d386d54580000000000000000000000000000000000000000000000000000000000000000","nonce":"0x83","to":"0x02afbd43cad367fcb71305a2dfb9a3928218f0c1","transactionIndex":"0x1b","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xa18772e4b9e1493ebfd5ae6219cc9ca060b71e83788fb614ab02b03fb36eaf1b","s":"0x482c7e50fde29bb4130fd6e723176e0e9913430ebb9260667d7fb55f5408d9c5","yParity":"0x1"},{"blockHash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","blockNumber":"0x84161e","from":"0xdb33365a8730de2f7574ff1189fb9d337bf4c36d","gas":"0x61a80","gasPrice":"0x433653c8","maxFeePerGas":"0x828367af","maxPriorityFeePerGas":"0x37e9071","hash":"0x4dd69b69a568ff30ae439e2ded72fbd7f2e7aaa345836703663f155c749c5eed","input":"0x7f34c0ddc6e984f46ab7c13b7a53643096e79c39103dc74550204d5bd1b5206ef5a2f49689071bb6a332212df26aa57cd4d1e3c0894187a9fc5e5d686cbedba5b187470f000000000000000000000000000000000000000000000000000000000000000e000000000000000000000000e8c44c30e8e52c38ac7bd232e7b7f0c7463a535100000000000000000000000000000000000000000000000000000000000000a0000000000000000000000000000000000000000000000000000000000000002e516d5044645856613738437a6b7856325a364e52777a67685a785872736e75397261616a695a7555423569465a62000000000000000000000000000000000000","nonce":"0x2b57","to":"0xb8ad54644e2371cbb7940b9fa0936f399ba7495f","transactionIndex":"0x1c","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0xa1fb0e5662b54ceb3dc1623ebfb200668433cb27252d553669bcd136387394f3","s":"0x27c2f8e19f69a52a0960f05d8b21f11c313dd3e86a7139b5783a18353c407415","yParity":"0x0"}],"transactionsRoot":"0x4a87d0cf5990b1c5bac631583e5965c2ba943858bebb2e07f74d0b697f73821a","uncles":[],"withdrawals":[{"index":"0x1170","validatorIndex":"0x38c2c","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x66edfd65"},{"index":"0x1171","validatorIndex":"0x38c2d","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6cd228e4"},{"index":"0x1172","validatorIndex":"0x38c2e","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x77f3431b"},{"index":"0x1173","validatorIndex":"0x38c2f","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6b61f268"},{"index":"0x1174","validatorIndex":"0x38c30","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6e10bb21"},{"index":"0x1175","validatorIndex":"0x38c31","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6eb115a5"},{"index":"0x1176","validatorIndex":"0x38c32","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x7caead1d"},{"index":"0x1177","validatorIndex":"0x38c33","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x772c0ddf"},{"index":"0x1178","validatorIndex":"0x38c34","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x75930a95"},{"index":"0x1179","validatorIndex":"0x38c35","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x76a4db09"},{"index":"0x117a","validatorIndex":"0x38c36","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x7e692b27"},{"index":"0x117b","validatorIndex":"0x38c37","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x72038ae6"},{"index":"0x117c","validatorIndex":"0x38c38","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6ccce352"},{"index":"0x117d","validatorIndex":"0x38c39","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x79ef6898"},{"index":"0x117e","validatorIndex":"0x38c3a","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6d58977d"},{"index":"0x117f","validatorIndex":"0x38c3b","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x76f7d208"}],"withdrawalsRoot":"0xbe712c930a0665264b025ced87cc7839eef95a3cbc26dadc93e9e185a350ad28"} diff --git a/op-service/sources/testdata/data/blocks/post-shanghai-success_metadata.json b/op-service/sources/testdata/data/blocks/post-shanghai-success_metadata.json new file mode 100644 index 000000000000..d6179c6587c3 --- /dev/null +++ b/op-service/sources/testdata/data/blocks/post-shanghai-success_metadata.json @@ -0,0 +1 @@ +{"name":"post-shanghai-success"} diff --git a/op-service/sources/testdata/data/blocks/pre-shanghai-bad-receipts_data.json b/op-service/sources/testdata/data/blocks/pre-shanghai-bad-receipts_data.json new file mode 100644 index 000000000000..1d11d4b8465c --- /dev/null +++ b/op-service/sources/testdata/data/blocks/pre-shanghai-bad-receipts_data.json @@ -0,0 +1 @@ +{"baseFeePerGas":"0x7ccf990f8","difficulty":"0x0","extraData":"0xd883010b02846765746888676f312e32302e31856c696e7578","gasLimit":"0x1c9c380","gasUsed":"0xa79638","hash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","logsBloom":"0xb034000008010014411408c080a0018440087220211154100005a1388807241142a2504080034a00111212a47f05008520200000280202a12800538cc06488486a0141989c7800c0c848011f02249661800e08449145b040a252d18082c009000641004052c80102000804ac10901c24032000980010438a01e50a90a0d8008c138c21204040000b20425000833041028000148124c2012d0aa8d1d0548301808228002015184090000224021040d68220100210220480420308455c382a40020130dc42502986080600000115034c0401c81828490410308005610048026b822e10b4228071ba00bdd20140621b2000c02012300808084181ac308200000011","miner":"0x0000000000000000000000000000000000000000","mixHash":"0x31f0c0305fc07a93b1a33da339c79aadbe8d9811c78d2b514cd18d64e1328f25","nonce":"0x0000000000000000","number":"0x840249","parentHash":"0x2303b55af4add799b19275a491b150c1a03075395f87a7856a4e3327595ed7df","receiptsRoot":"0xaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa","sha3Uncles":"0x1dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347","size":"0xea6d","stateRoot":"0xd12bf4cf3941cf48be329a939b13d3403d326841c69cdcc9a9c13ab2f227e904","timestamp":"0x640fdeb0","totalDifficulty":"0xa4a470","transactions":[{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x1b7aa44088a0ea95bdc65fef6e5071e946bf7d8f","gas":"0x5208","gasPrice":"0x174876e800","hash":"0x39c666d9b5cec429accad7b0f94f789ca2ebeb5294b8b129c1b76f552daf57d3","input":"0x","nonce":"0x26bd1","to":"0x68643ada5885f14e7c15c13bee06168eff0f7c36","transactionIndex":"0x0","value":"0xb1a2bc2ec50000","type":"0x0","chainId":"0x5","v":"0x2e","r":"0x26e648625520c256d87d8e15769e8c4290d2a7e27147594815da42070870825b","s":"0x7db34c91603e51494b59b020597e7118433d50ed548bc7efad8880cbe7820ca"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x030b1cddf635e9e71ad70b8668e235e8ec3c67c4","gas":"0xf618","gasPrice":"0x1701ca2b9d","hash":"0x2ca7289ab3738d17e0f5093bd96c97c06c9a2ea4c22fc84a6a7fbfda93ce55ee","input":"0x","nonce":"0x456d","to":"0x89416096a0a1acad3436a357478c5c548e4d382b","transactionIndex":"0x1","value":"0x16345785d8a0000","type":"0x0","chainId":"0x5","v":"0x2e","r":"0x5aa6ad410ea42dbff3781de67b4815a376cd459206595fa6655c966a7709e5cb","s":"0x82f620f8dbc356dc38ef346d1e152656a1d318becbcc41884f61e85972355c0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x9c67435ce8a33ad23f377ddf08260ffc56417f21","gas":"0xf618","gasPrice":"0x1701ca2b9d","hash":"0xb0085de1476530de3efc6928c4683e7c40f8fac18875f74cbcc47df159de17d9","input":"0x","nonce":"0x446c","to":"0x471f3a988a77472e971112e7ff2ce65996b6c8cd","transactionIndex":"0x2","value":"0x2c68af0bb140000","type":"0x0","chainId":"0x5","v":"0x2d","r":"0x75425f21a6e66e6feaee79455c897656c4a94b6189c4c033facab1b67812fc3f","s":"0x7014f4b9f760165c7757f9d6dccea349b3d00b0a9c2f845af8010140842f826"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x168f6dec26cbbb3749654e0e3cc4fc29314fdf6c","gas":"0xf618","gasPrice":"0x1701ca2b9d","hash":"0xe01c8631c86ded63af95b8dbc0c8aac5d31254c14d6ecb4cc51d98259d838e52","input":"0x","nonce":"0x403c","to":"0xf2119a49a3ae35453c1c2704da248d08977084f1","transactionIndex":"0x3","value":"0x16345785d8a0000","type":"0x0","chainId":"0x5","v":"0x2e","r":"0x56bba57ee69c09dbbd5167d0fee07d5803130fea7c278f4d8b7bc54b34be8ff9","s":"0x7eea69c0de47b6b808ea1bdd9ecf5d25f4f2743e867b1f838db59c456026d7e6"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xc9af69904b4ba7f0fb69ab1a8719bd8950265e73","gas":"0xf618","gasPrice":"0x1701ca2b9d","hash":"0x69414a126a6f07ab5e31ad2f9069fb986b7c490e096898473873e41ece6af783","input":"0x","nonce":"0x44c20","to":"0x6adc3f5394e00cd067d1fad5a33756791e303c25","transactionIndex":"0x4","value":"0x16345785d8a0000","type":"0x0","chainId":"0x5","v":"0x2e","r":"0xdc9a52b87755cd07143bf032d57258d0d6c36500b38a574a9a5addc1293343da","s":"0x581e6fd051d0c72d64303c1f90edecfef46580f84f24dba930cde910923f7c21"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x92964e63cd52450e6e09a266e0c226e524754563","gas":"0x2625a0","gasPrice":"0xf5686c7be","hash":"0xa2fef1133ee726533c7f190f246fede123e3706a03933c1febc92618f90d2804","input":"0xa9059cbb00000000000000000000000011f7db5a824eaa5737c935ad09a925dd707f092b0000000000000000000000000000000000000000000000008ac7230489e80000","nonce":"0x2636","to":"0xbc071c64ed8f536011c78e847755680740d6b73c","transactionIndex":"0x5","value":"0x0","type":"0x0","chainId":"0x5","v":"0x2d","r":"0x422c5026584cd8fb8a1fe66e6edc7a020bc513cc3114083fb35f9846cd9a9792","s":"0x7fcd4e48ac5b53d881531a5651e587188157ed88d85c51533f7c6eda94c67ff7"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xdec1bc71bf91431d60ef2742f412dcd1c5a204b8","gas":"0x7a120","gasPrice":"0xb80e515ce","hash":"0x6585ec5c4c2bbf1f683f90f58e18f3b38d875e94457fe4cbb7bc5bf6581f83af","input":"0x67a5cd060000000000000000000000006343e96c99e2725e845ba04f90e03d853cc9bb2d","nonce":"0x6a5a4","to":"0xf5de760f2e916647fd766b4ad9e85ff943ce3a2b","transactionIndex":"0x6","value":"0x0","type":"0x0","chainId":"0x5","v":"0x2d","r":"0xcb7b32949ac0dedd3151d06d85ec4e99483c8ea706f12a009d97b055975241eb","s":"0x69f0009bce12e24408f9a3d5ad5a8578e2b23efb09002f7b86b3425cf8748889"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xdb954dd5d224004b26cfbb089312e0542e5d1c31","gas":"0xd5ef","gasPrice":"0x9fc748df1","maxFeePerGas":"0x116482c049","maxPriorityFeePerGas":"0x22f7afcf9","hash":"0x1db276b864fbf01dcf8cededf8d597553ecb0eb9438edfaf2f5bd0cc93297c66","input":"0xa3e35f36000000000000000000000000f18f2320e4779f21d6882d83f9133e582dfaa9b6","nonce":"0x570","to":"0x11672c0bbff498c72bc2200f42461c0414855042","transactionIndex":"0x7","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xf23a1fb0d17cdbc2831a2cb0b779795e12533e34425a3abbc4b7e7bde262b92d","s":"0xd4a2c8189378696065b7fc5d13e9cf6595d96047a31a4fcb812c2e677d22a26","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xe84d601e5d945031129a83e5602be0cc7f182cf3","gas":"0x249f0","gasPrice":"0x9502f9000","hash":"0xcbe7ed31654af4e191ca53445b82de040ae2cd92459a3f951bdcce423d780f08","input":"0xa9059cbb000000000000000000000000355f96e3261cc8237d2d36fd81cc9b9cdcbfc5ea0000000000000000000000000000000000000000000000000de0b6b3a7640000","nonce":"0x4f1f0","to":"0x499d11e0b6eac7c0593d8fb292dcbbf815fb29ae","transactionIndex":"0x8","value":"0x0","type":"0x0","chainId":"0x5","v":"0x2e","r":"0xadf7dce5f34d482bb19ff0c85f43be7008cb838043f9abb5fc08421245ac4228","s":"0x5d2c0f0b9d2e68bb8da6084e4971d57bd76bf0cd9d893c9c985962812d2df026"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xea0193f88a0d9c36e83e6fb6d382212b6373faba","gas":"0x43238","gasPrice":"0x933ea77d8","hash":"0x808ba5211f03cc78a732ff0f9383c6355e63c83ae8c6035ced2ba6f7c331dc63","input":"0x9f8420b300000000000000000000000000000000000000000000000000470de4df8200000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x0","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0x9","value":"0x470e9f23394000","type":"0x0","chainId":"0x5","v":"0x2e","r":"0x9536f87e8c3433ea23ef7211c0c32689551906dfdf31bcc73b89fcf18fe86a92","s":"0x3e94bfe3b4ba32a6ce71ee6e0ff9f26dfc2b0d61b5c0fe049469e3c4b88ac227"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x1f9ab300b380313e6a2208e44c01a0602538385d","gas":"0x32b38","gasPrice":"0x87fc9eef8","maxFeePerGas":"0x2e90edd000","maxPriorityFeePerGas":"0xb2d05e00","hash":"0xdd66f1f26672849ef54c420210f479c9f0c46924d8e9f7b210981ffe8d3fac82","input":"0x32db5470000000000000000000000000000000000000000000000000000000000084021000000000000000000000000000000000000000000000085b0af981975a40fc5a00000000000000000000000000000000000000000000053ae1f470262f3b01c4000000000000000000000000000000000000000000000813bca9f8389ffd193f","nonce":"0x5699","to":"0x28cea7b0f3916c1dba667d3d58ec4836ad843c49","transactionIndex":"0xa","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xbe8eddd463a3518a4c94b5922c0f2fb26a32c8afcd819f7deaf4f0416757ad13","s":"0x3e234182acd5261dfba3cf4f125d1a8377fc707b3b1900b565d1a6b8987b398c","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x1d990a2cf61b46f37582dd31a7b242e989e1a1ee","gas":"0x493e0","gasPrice":"0x86f96eddb","hash":"0x254abb2f8cdcffe9ef62ab924312a1e4142578db87e4f7c199fd35991e92f014","input":"0x9f8420b3000000000000000000000000000000000000000000000000002386f26fc100000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x0","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0xb","value":"0x2387acb3784000","type":"0x0","chainId":"0x5","v":"0x2e","r":"0xbfe9b99fd2e9bbd2d65bb4cc17da88130d8f6b569a26afa642ed1cc8efd2e2ef","s":"0x7003dd3962eec2369bc5abe5e3d5b73a9086a3219643b1741f3a9832369448fa"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x1124bbb196fd7b61947f6f03b7d7cebde8340b6f","gas":"0x493e0","gasPrice":"0x86f96eddb","hash":"0xa7b7c654e7073b8043b680b7ffc95d3f2099abaa0b0578d6f954a2a7c99404e1","input":"0x9f8420b3000000000000000000000000000000000000000000000000002386f26fc100000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x0","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0xc","value":"0x2387acb3784000","type":"0x0","chainId":"0x5","v":"0x2e","r":"0xe8dae9efa8fa73a3aebdaa75ba12acf56712b1f99fc128ab802a7610d014547b","s":"0x2ef347ea90a0db21b035637342bfc916c9962d3ec610a4e99cfaf555a2606fe2"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xdedf52a88fe47f6a11d629badd0096d201aef86a","gas":"0x493e0","gasPrice":"0x86f96eddb","hash":"0x7ccdfa698c8acf47ab9316ed078eb40819ff575bcf612c6f59f29e7726df3f96","input":"0x9f8420b3000000000000000000000000000000000000000000000000000003a3529440000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x1","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0xd","value":"0x45d964b8000","type":"0x0","chainId":"0x5","v":"0x2d","r":"0xfc4e092094abc7fed9cf11eea8ad6ec3a760d13b9c603b5570c2143e48090f2f","s":"0x31a0a6016ab51ed6375891bf532d9031bf91c1d0e01df63cf7e46076aeee83a3"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xe1db1fd154b344ce7dfcb53a65279559b298f0df","gas":"0x493e0","gasPrice":"0x86f96eddb","hash":"0xa0b035ef315824a6f6a6565fa8de27042ade3af9cf0583a36dea83d6e01bf2a8","input":"0x9f8420b3000000000000000000000000000000000000000000000000002386f26fc100000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x0","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0xe","value":"0x2387acb3784000","type":"0x0","chainId":"0x5","v":"0x2d","r":"0x79c316492bfa36fa4de2dca02ffac77f6c1772714e153aad8c947ce848d4106f","s":"0x4fe3e083943ca8f974cb0752a2ffb70e4d6c8b615e16ece7e19b2c464be66929"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xd4c3777e1821f6fb532957104e1317a9d3f881ec","gas":"0x493e0","gasPrice":"0x86f96eddb","hash":"0x1ebad7f3e8cb3543d4963686a94d99f61839f666831eab9c9c1b4711de11d3d9","input":"0x9f8420b3000000000000000000000000000000000000000000000000000003a3529440000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x1","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0xf","value":"0x45d964b8000","type":"0x0","chainId":"0x5","v":"0x2d","r":"0x5c1f544ca92b28a014129a9d04f8e895dc8a6987ddc1717c26b0619bb9bac420","s":"0x47574643bd92a0f129f9c5ce3caebe7943cfd828d67fb0e5bdc1eb2ca202822a"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x127418955b15f5ff2fa232431a798b53fd006c82","gas":"0x493e0","gasPrice":"0x86f96eddb","hash":"0x501750278e91d8b5be1ccf60e793d4bbcd9b3bb3ccc518d3634a71caeac65f48","input":"0x9f8420b3000000000000000000000000000000000000000000000000000003a3529440000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x1","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0x10","value":"0x45d964b8000","type":"0x0","chainId":"0x5","v":"0x2d","r":"0xcfc9894bf716ba707521f0969082afb688b7d47cc21f1423b239c40867128f47","s":"0x5d9b9c66d8437d0a37768fb9c3241ef64b7e795d0d908196e1f707215ac1f56"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xc2ac1bdcb3dd77864b99eb2cafef2ed36c1487c3","gas":"0x493e0","gasPrice":"0x86f96eddb","hash":"0xd80ff8af29ae163d5811ba511e60b3a87a279f677bb3872a0f1aa6d0a226e880","input":"0x9f8420b3000000000000000000000000000000000000000000000000002386f26fc100000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x0","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0x11","value":"0x2387acb3784000","type":"0x0","chainId":"0x5","v":"0x2e","r":"0x8bd7220d176d18cfbc824063448cf864c494fcf0958dec82126f26d3fc6c0b6e","s":"0x2f4de3d2fc3be6d2812ee10aee16a995af5affcfec27f9953ae609886d2b563e"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x11a5ad000f8494233df5bd2f2573fe20bad744ab","gas":"0x493e0","gasPrice":"0x86f96eddb","hash":"0x096acab3b3fe47b149d375782d1eb00b9fef7904076d60c54b3c197b04e6bf82","input":"0x9f8420b3000000000000000000000000000000000000000000000000000003a3529440000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x1","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0x12","value":"0x45d964b8000","type":"0x0","chainId":"0x5","v":"0x2d","r":"0x43726c3165958e94d80cb6992121f55df2904a1de4356e0a6ff3d8bfd03c0928","s":"0x6244361e31daccbb5c7e9f3f7d01004080d2efa314d614fdc2aad4c1b7d71613"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x90f8b290ec2929cce086826e971cad3857c6dbb5","gas":"0x50f46","gasPrice":"0x861fc89f8","maxFeePerGas":"0xe22924cba","maxPriorityFeePerGas":"0x9502f900","hash":"0xbe9d1738af74a22400591a9a808fb01a25ab41e2e56f202dd7251eb113e8ceeb","input":"0x6ab150710000000000000000000000000000000000000000000000001bc16d674ec800000000000000000000000000000000000000000000000000000de0b6b3a7640000","nonce":"0xe4","to":"0x2e84c9894f628fd209f48b06d10b46c8cada4c10","transactionIndex":"0x13","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x412c7bfb0236a4317da40645ee4e60f19185e7d0cd2cc9bc88f7629de505b939","s":"0x4f6bd485d10fa134926ec3e3a83e212f3c5d903ba52da87f07794a2ba0a5c1eb","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x7915ec065b644568155c4772a286addad3864c1b","gas":"0xb71b00","gasPrice":"0x861fc89f8","maxFeePerGas":"0x1176592e000","maxPriorityFeePerGas":"0x9502f900","hash":"0x0834c720e55cccd97aaf4f8fb0cb66afb9881fb6a762c0f70473ec53f98a712e","input":"0xedce8b4d6a1fb55588eadfd3a0c3350435a31d31e7c236125eafcc2a892d50711d9a4939a4282888c956c070cd83400fd47ab83bd5287244053b8228711b07386dccacbd000000000000000000000000000000000000000000000000000000000000006000000000000000000000000000000000000000000000000000000000000001a0000000000000000000000000000000000000000000000000000000000000082000000000000000000000000000000000000000000000000000000000000008a0000000000000000000000000000000000000000000000000000000000000092000888ad4975d4079d80bc6da5ded57746f889644719ec160efc2e4a96ad645df10b1683afcd7ce41ba6c06528761f821cbc6e806ef4b71c189603ec218b12a1f1478883498e67b97ae4babef6996acc582db1377d8755374602e00b8148025ac2d2f52e78d7484919cef14bf96e5b7a3f7f8cf09989551888f2c697052ce2bdb02a9296b20016c78683a9ca137037e57c8ff25198146f9d4cdb5a25ab9c95a03038f22f83c8604ca7c035ef756d2963c3657eea4ffb18ef967752d8cff15a16124a1d92a1cda613624c98e3078742803da1417b78bb264b7bb2197515cb4facc2ea395c6241664dbd953d0034900a86eb94897bf89aae775a1d73e6fd034785400000000000000000000000000000000000000000000000000000000000009a0000000000000000000000000000000000000000000000000000000000000000300000000000000000000000000000000000000000000000000000000000000600000000000000000000000000000000000000000000000000000000000000260000000000000000000000000000000000000000000000000000000000000046000000000000000000000000000000000000000000000000000000000000001d802f901d482e7041b846de27f7b846de27f8383026c28946aa397cab00a2a40025dbf839a83f16d5ec7c1eb841dcd6500b901a45ae401dc00000000000000000000000000000000000000000000000000000000640fdf5900000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000000000000000000001000000000000000000000000000000000000000000000000000000000000002000000000000000000000000000000000000000000000000000000000000000e404e45aaf0000000000000000000000002c1b868d6596a18e32e61b901e4060c872647b6c000000000000000000000000964ff70695da981027c81020b1c58d833d49a64000000000000000000000000000000000000000000000000000000000000027100000000000000000000000009a1a41127726f74560a57990fb928120eed6f55e000000000000000000000000000000000000000000000000000000001dcd6500000000000000000000000000000002d4766adcf72ef2d996edfadc866be12c28000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000c0000000000000000000000000000000000000000000000000000000000000000000000000000001d802f901d482e7041c846a969541846a96954983026c28946aa397cab00a2a40025dbf839a83f16d5ec7c1eb841dcd6500b901a45ae401dc00000000000000000000000000000000000000000000000000000000640fdf5a00000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000000000000000000001000000000000000000000000000000000000000000000000000000000000002000000000000000000000000000000000000000000000000000000000000000e404e45aaf0000000000000000000000002c1b868d6596a18e32e61b901e4060c872647b6c000000000000000000000000964ff70695da981027c81020b1c58d833d49a64000000000000000000000000000000000000000000000000000000000000027100000000000000000000000009a1a41127726f74560a57990fb928120eed6f55e000000000000000000000000000000000000000000000000000000001dcd6500000000000000000000000000000002d4766adcf72ef2d996edfadc866be12c28000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000c0000000000000000000000000000000000000000000000000000000000000000000000000000001db02f901d782e70480846a969541846a9695498302e100946aa397cab00a2a40025dbf839a83f16d5ec7c1eb870aa87bee538000b901a45ae401dc00000000000000000000000000000000000000000000000000000000640fdf8d00000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000000000000000000001000000000000000000000000000000000000000000000000000000000000002000000000000000000000000000000000000000000000000000000000000000e404e45aaf0000000000000000000000002c1b868d6596a18e32e61b901e4060c872647b6c000000000000000000000000964ff70695da981027c81020b1c58d833d49a6400000000000000000000000000000000000000000000000000000000000000bb800000000000000000000000048f2c17f8885e135c8689626d5841d0b94013556000000000000000000000000000000000000000000000000000aa87bee5380000000000000000000000000017a8db2711fe83b10bbfabb6a8b86c3e26ca154ef000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000c000000000000000000000000000000000000000000000000000000000000000000000000003000000000000000000000000000000000000000000000000000000000000001b000000000000000000000000000000000000000000000000000000000000001b000000000000000000000000000000000000000000000000000000000000001c0000000000000000000000000000000000000000000000000000000000000003760f4e19e59be10b323fe1b42e42de2754c5da54ff316cef74ef09dbc0144bbf73e609a8d21a38f393bae9a3de6bebbf362608d907d941c8166f44d8ee2c6686b5dbbd6681f5467219d9bb530db013b7c067db659c03efe935caeb7bab36699d00000000000000000000000000000000000000000000000000000000000000032d2d7436bd45b5e81824945ff3dda846e270c81e72929841f4b14f56e02eef39477f9c162a2759c8d7c7e1feefdd064f6aefb4360d483f70671f36fc11ccc6db6c53f74e3019c12d859545ad75b51998b2238887dc0be3c78522c8356150526a0000000000000000000000000000000000000000000000000000000000000000","nonce":"0x48f08","to":"0xe87d317eb8dcc9afe24d9f63d6c760e52bc18a40","transactionIndex":"0x14","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x54159a63b8ce19097cb8afeaffc84cfb5257f9709dcb0f0a258befa0b924b13","s":"0x17f5ef5aeceb6f082a8a0f8fa28e55d453c52e5f86d48dfb4d4f039aad7fa493","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x52db522edf50fc32b8fc0f77d51140031da11da3","gas":"0x201568","gasPrice":"0x861fc89f8","maxFeePerGas":"0xe22924cba","maxPriorityFeePerGas":"0x9502f900","hash":"0x51a0c33c9b37245b416575bdd2751c0d8a5d8bead49585ac427bfc873d4016af","input":"0x96d3f83c0000000000000000000000008cd194f4ced354a14960f76f49305feb8510f73f00000000000000000000000000000000000000000000000000000000000000e000000000000000000000000000000000000000000000000000000000641024ef0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000001b3fd330d479974ad52e18b8767d9f18cf9ff024fa749b1066a803b237570cb73e2868e821124c8128505d883124711ae7fdca0339f4f5cd5dadd8eb6956eb4af500000000000000000000000000000000000000000000000000000000000000040000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000100000000000000000000000000000000000000000000000000000000000000020000000000000000000000000000000000000000000000000000000000000003","nonce":"0x2d9","to":"0x29c1442d7e3b6ed179ceb55fe996e1f4384880da","transactionIndex":"0x15","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xac84f403fa0347be3246ad695f8f6a780332357d49dfa67388fae3e051e5566c","s":"0xac2e148c1045af0ebdb334f20743af7d5499558af5eea43393159ce26a3fb1b","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x3aea5f857b75a946d5f5b329bdcf4db46aec5d2c","gas":"0x50f46","gasPrice":"0x861fc89f8","maxFeePerGas":"0xe22924cba","maxPriorityFeePerGas":"0x9502f900","hash":"0x531c25d51ccda59aa9ea82e85c99be9dd4e285af9b8973cbab9ac4a38e26e55a","input":"0x6ab150710000000000000000000000000000000000000000000000000de0b6b3a76400000000000000000000000000000000000000000000000000000de0b6b3a7640000","nonce":"0x1d","to":"0x6c23a4586cc791d2be6767532e12264992aef74a","transactionIndex":"0x16","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x81abc4fdf3b57fdf3a951fb293853fdd86aadce8f319576ac78c60caac186c43","s":"0x452742335bd1f63fd623d5e6cf91f6339361f863dba1a57fc3c1da8292e82c77","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x7915ec065b644568155c4772a286addad3864c1b","gas":"0xb71b00","gasPrice":"0x861fc89f8","maxFeePerGas":"0x1176592e000","maxPriorityFeePerGas":"0x9502f900","hash":"0x93ac6c08d21cb1b61ff59e5e2d6fa3f9ad54008b0a66c669199050bef219f6e3","input":"0xedce8b4da4282888c956c070cd83400fd47ab83bd5287244053b8228711b07386dccacbdb54cb66757631f9f22e2faf7da0eecc43474ab1a1af21aec22dcf949e2579241000000000000000000000000000000000000000000000000000000000000006000000000000000000000000000000000000000000000000000000000000001a00000000000000000000000000000000000000000000000000000000000000860000000000000000000000000000000000000000000000000000000000000092000000000000000000000000000000000000000000000000000000000000009e011cb7bbabfab668b42b5582cec8d60f326ae14ad55f8d675060dd111dee6c88d0937435261929ba3a881bea107593a10e13c917a0ef5ae4e8bde40de2d1da4ba25ab3a239ba53eb4e260d23b98a7db944f644f40ab75992a89fad40e8d692ad62f8671604db415b54238c5522f6c4e6b25785bffa696d173659309b95a2fae8700d4b9c14382973ea34a3cd8b8428594908dcad2d5d7a01056b8e49a8c25087f12f9e9fec1f4b5fdc57509dba6839100e28016b2e3fec4928bbe11d3789b6cab280c17bebfa5a0e06aea22764e1a18d2d11316bdf81028e65755582586ebdf571cbd0ef59800ba3405539187dfc03eac1dba0cf590e2131a56ef7f260aa38f940000000000000000000000000000000000000000000000000000000000000aa0000000000000000000000000000000000000000000000000000000000000000500000000000000000000000000000000000000000000000000000000000000a000000000000000000000000000000000000000000000000000000000000001a00000000000000000000000000000000000000000000000000000000000000200000000000000000000000000000000000000000000000000000000000000032000000000000000000000000000000000000000000000000000000000000005c000000000000000000000000000000000000000000000000000000000000000da02f8d782e704820101846a969541846a96954982dc1994a59477f7742ba7d51bb1e487a8540ab339d6801d87470de4df820000b8a4220b5b82000000000000000000000000f6213f16f3db9b50c684b2da37ff080895ae4a02000000000000000000000000000000000000000000000000002386f26fc100000000000000000000000000000000000000000000000000000000000064376b6900000000000000000000000000000000000000000000000000000000000000800000000000000000000000000000000000000000000000000000000000000000c0000000000000000000000000000000000000000000000000000000000000000000000000003202f082e70411846de27f7b846de27f8382520894b6605a717a2447edac46b1cdafe35e3e91130c8d87038d7ea4c6800080c0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000f202f8ef82e7040d846de27f7b846de27f838303335f947191061d5d4c60f598214cc6913502184baddf1880b8c44d49e87d00000000000000000000000000000000000000000000000000000000000000600000000000000000000000000000000000000000000000000160f9854c9f595e000000000000000000000000000000000000000000000000000000006417d4b8000000000000000000000000000000000000000000000000000000000000000200000000000000000000000000000000000000000000000000b1a2bc2ec5000000000000000000000000000000000000000000000000000000b2f4f1df2e372ec00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000027502f9027182e7048197846a969541846a96954983029eb1946aa397cab00a2a40025dbf839a83f16d5ec7c1eb80b902445ae401dc00000000000000000000000000000000000000000000000000000000640fdf0b000000000000000000000000000000000000000000000000000000000000004000000000000000000000000000000000000000000000000000000000000000020000000000000000000000000000000000000000000000000000000000000040000000000000000000000000000000000000000000000000000000000000016000000000000000000000000000000000000000000000000000000000000000e404e45aaf000000000000000000000000964ff70695da981027c81020b1c58d833d49a6400000000000000000000000002c1b868d6596a18e32e61b901e4060c872647b6c00000000000000000000000000000000000000000000000000000000000001f400000000000000000000000000000000000000000000000000000000000000020000000000000000000000000000000000c097ce7bc90715b34b9f10000000000000000000000000000000000000000000000000000000000000000000000614000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000004449404b7c0000000000000000000000000000000000000000000000000000000000000614000000000000000000000000694b3f194e94c80b2d7a0e986e1440f204f0503f00000000000000000000000000000000000000000000000000000000c0000000000000000000000000000000000000000000000000000000000000000000000000000000000000b902f8b682e70427846a969541846a969549830dbba094964ff70695da981027c81020b1c58d833d49a640872aa1efb94e0000b884437471fd0000000000000000000000004e44260732136caeffc7d076e17b2a3554b9ce2a000000000000000000000000000000000000000000000000002aa1efb94e00000000000000000000000000000000000000000000000000000000000064376b6600000000000000000000000000000000000000000000000000000000000dbba0c0000000000000000000000000000000000000000000000000000000000000000000000000000005000000000000000000000000000000000000000000000000000000000000001b000000000000000000000000000000000000000000000000000000000000001b000000000000000000000000000000000000000000000000000000000000001b000000000000000000000000000000000000000000000000000000000000001b000000000000000000000000000000000000000000000000000000000000001b0000000000000000000000000000000000000000000000000000000000000005b0efccaa7c845eddd4fb98211c7a58a37abdf06b264fade6cbff8a73ed22fccc602337b33a81e2cc4127be9d913fb3aec24194e14a61515a26429a00cfb86196aa3c2e6983f1a05dec045f0e63db7bd8619cd2a770ed1f7f470eb7c297e8e19d61277fb276ed8a63fd666b4a5ab02347065a3c16539f4ccccbed8fa39f2a7dc6606fc04c49f3d69463ba1c86c6ddcef8b9f06b9741dc2ebae9f5f1c0523b320b00000000000000000000000000000000000000000000000000000000000000050668b9ca5e5b17e2e79f9f5a7d9ea27e4e7f0302e878584fef3c2304c70c1ef53da311342341f89df488646b97a83994a3d07cc622ffde98598548c5326b7ccc0cf77d5d6fca6cf3f5b5b41a266fa2aaf697fd976dddc710fca834c96d2dfd725eec5980733db3c4e8f779f9fde3a598b2860d819419c961012825f0e160ee7b220989005b6ac5c48734db18158c2d85c8499bea1972c9f875c2d2aeafccb33700000000000000000000000000000000000000000000000000000000000000020000000000000000000000000000000000000000000000000000000000000040000000000000000000000000000000000000000000000000000000000000014000000000000000000000000000000000000000000000000000000000000000e0000000000000000000000000f6213f16f3db9b50c684b2da37ff080895ae4a02000000000000000000000000f6213f16f3db9b50c684b2da37ff080895ae4a02000000000000000000000000000000000000000000000000002386f26fc10000000000000000000000000000000000000000000000000000002386f26fc100000000000000000000000000000000000000000000000000000000000064376b6900000000000000000000000000000000000000000000000000000000000000c000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000140000000000000000000000000964ff70695da981027c81020b1c58d833d49a64000000000000000000000000073fee82ba7f6b98d27bcdc2befc1d3f6597fb02d000000000000000000000000000000000000000000000000002aa1efb94e000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000064376b6600000000000000000000000000000000000000000000000000000000000000c00000000000000000000000000000000000000000000000000000000000000044a1c058fa0000000000000000000000004e44260732136caeffc7d076e17b2a3554b9ce2a00000000000000000000000000000000000000000000000000000000000dbba000000000000000000000000000000000000000000000000000000000","nonce":"0x48f09","to":"0xe87d317eb8dcc9afe24d9f63d6c760e52bc18a40","transactionIndex":"0x17","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0xb5f185933d49064643686594f44a2584b1d57ba6a91f2a45d1a122892488f209","s":"0x7bb4af7ea30290b36fa7c266ee2387eff9e3ee0cd79b48aed27f5bfaaf34114c","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x4281ecf07378ee595c564a59048801330f3084ee","gas":"0xea60","gasPrice":"0x861fc89f8","maxFeePerGas":"0xfd18eeafe","maxPriorityFeePerGas":"0x9502f900","hash":"0x3792db6dd6285f409e4281951e9f78dad16c4a78072ff1c909dfadea5658d857","input":"0xa9059cbb0000000000000000000000000cc351a97c2fafde4cfa8a00c4fd79881cf13b53000000000000000000000000000000000000000000000001158e460913d00000","nonce":"0x77ffb2","to":"0x326c977e6efc84e512bb9c30f76e30c160ed06fb","transactionIndex":"0x18","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0xef4173dd45060791cfb6dfc3cdc8a9b417ff6a4b3ba57759ea91fd875b01a522","s":"0x4de926b5099c7d60cc24add3c2eecd8e9550c1fe7f878430eccb887a2ff932ff","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x53e2c2479833ca17f946aeb16dcc49f5e2194701","gas":"0x431eb","gasPrice":"0x8442f24f8","maxFeePerGas":"0xd6aac8818","maxPriorityFeePerGas":"0x77359400","hash":"0xd2d51764c01e8c0a43fbe362704388df5bacf7e5e620c3864e242530ffb3e828","input":"0x9f8420b300000000000000000000000000000000000000000000000000470de4df8200000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x2","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0x19","value":"0x470e9f23394000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x155352758fb04a6c0816c3dd9ca16bd6c66c37e2e19f2aa5d969225e3cb7f118","s":"0x6a4a1237db75dac4322fe12aee8b94edf7083ea661d6e93334c667d2d5c2c101","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x535619cf2bd2e84e567bda1a6164f195cb2bba21","gas":"0x55730","gasPrice":"0x83152e5b0","hash":"0x516b0227d9e64eb6e0de6862764d40f5376b5f12fec878436fea3479b4c36bb8","input":"0xa9059cbb00000000000000000000000030509945d2b329cc253a8bb2e2c54bdbb08b1070000000000000000000000000000000000000000000000010ce1d3d8cb3180000","nonce":"0x2176","to":"0x1173069b0e472a64ce2b6203fec8fb25a05b75c2","transactionIndex":"0x1a","value":"0x0","type":"0x0","chainId":"0x5","v":"0x2e","r":"0x289ee9948b80fc623b8c5d85c7df6fd25786fcc118163042b77a00560f53c3fb","s":"0x5b0a79e30d05370b27664b2a10cf964e110387bd62098561b6ea6a9dcc4634a"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x339d413ccefd986b1b3647a9cfa9cbbe70a30749","gas":"0x180ac","gasPrice":"0x83152e5af","hash":"0x81b0abc78b82840adb666775b182a9e292f663b64bcd35004c04436ed3c8281c","input":"0x711746e200000000000000000000000000000000000000000000000000000000000000200000000000000000000000000000000000000000000000000000000000000001000000000000000000000000000000000000000000000000000000000000009a00000000000000000000000000000000000000000000000000000002540be40100000000000000000000000000000000000000000000000000000006d3c514bd0000000000000000000000000000000000000000000000000000000000000010","nonce":"0x2689","to":"0x7cbd185f21bef4d87310d0171ad5f740bc240e26","transactionIndex":"0x1b","value":"0x0","type":"0x0","chainId":"0x5","v":"0x2d","r":"0x1f70a3d0933aab069a55fb2b16a259d21ea5fb3255e2032c5ff194a600ed2ab1","s":"0x3ac2142fc9f2009d68b08855eb5bc85e889e15d043c33bfc516f28ea38715485"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x339d413ccefd986b1b3647a9cfa9cbbe70a30749","gas":"0x180ac","gasPrice":"0x933ea77d8","hash":"0xd0287570d431d2baea96ecc81cb890e7f4f06ab5df02f9b4067768abca19acb5","input":"0x711746e200000000000000000000000000000000000000000000000000000000000000200000000000000000000000000000000000000000000000000000000000000001000000000000000000000000000000000000000000000000000000000000006500000000000000000000000000000000000000000000000000003a09994473bc00000000000000000000000000000000000000000000000000000005672af7e00000000000000000000000000000000000000000000000000000000000000010","nonce":"0x268a","to":"0x7cbd185f21bef4d87310d0171ad5f740bc240e26","transactionIndex":"0x1c","value":"0x0","type":"0x0","chainId":"0x5","v":"0x2e","r":"0x73a68e10ef9726d47ff314cd16d1846a0876d2b63a65014909140de7e45aa8b3","s":"0x580de27e4626139fbad4129a249c7ac2b22e1062659293e5fc3a244fd74907c"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xb5b9345e21f34b6cfa538f49e2b14ad4c3db3f7e","gas":"0x431e0","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x76ddab2674369f34946c5fa2f05e2aa8566d86235b83e808e9b27bc106e04ac7","input":"0x9f8420b3000000000000000000000000000000000000000000000000016345785d8a00000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x38","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0x1d","value":"0x1634632a1414000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x4314a4757076c92ca696f14cbe037bcc0178f8b10749418d4535195e90b2cdc9","s":"0x15ee917bc3e27928196ad987b61e3fd35ac92a4200b139d355b30502bff7fceb","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xac2fde165d9f13a223f58b9ac1dde51b63feff2b","gas":"0x5208","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x34a5c74011a2c8a00103bc91bfbfd94aa99cd569be69066e4bf64d188fe8714e","input":"0x","nonce":"0x38","to":"0x7155eba8ef327ac3b36fe59b5e00ab85f735f4f4","transactionIndex":"0x1e","value":"0x1c9f78d2893e40000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x52178c5dce512607a6242f501f78bc894917a482e99a3368caea6bca94c6dd69","s":"0x728be90654231b65f790eba478ccab574e2468fc4e031b712efa88af31fc6592","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x1ddedd674bfacb6587e0c68cee8f2f7d8a71e9d7","gas":"0x1d4c0","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x7b9730ead1b9f59b206d0ddea87be9383ba3fc7b496c7863b0cb847889b86617","input":"0x607019b9000000000000000000000000000000000000000000000000000000000000004000000000000000000000000000000000000000000000000000000000000000010000000000000000000000000000000000000000000000000000000000000041971588d95d4d664ac84a45e59fc81871b7051d679376d86cef6b9e1453c762dc33a84a125774319ee47f2a8b8d588da6c8ff7ad84fd9eed22fa124b1a5c5ed4d1b00000000000000000000000000000000000000000000000000000000000000","nonce":"0x1","to":"0xe1e6aef7642b6609e6c54884d1bda4a3cb152461","transactionIndex":"0x1f","value":"0x4380663abb8000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xc0d3e609b2e9f844d300a7784aa9e7cb956415d025cb573a73848f49e4be69e5","s":"0x6ce01527c00c1a93341a541824aed7a1c895f5a0d998f4415cf6d88171c5ad3e","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xf6df8ead0bdabfba61d9b2457d5a5ccec67fb7f3","gas":"0xc836","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x77166ee0409ba86bd26e7c03ad1a927abaf5af8a8a37149e725cd37512091dd6","input":"0x67a5cd06000000000000000000000000fa8449189744799ad2ace7e0ebac8bb7575eff47","nonce":"0xbd","to":"0xccb0f4cf5d3f97f4a55bb5f5ca321c3ed033f244","transactionIndex":"0x20","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x5649e1ff58fa9b6ee27a7316904df02db1ba1c471a2f3409f370a31c6b3d5ec4","s":"0x386b766277e267f8796ba6e92af71a752335bbe45cb3385b9cdcab9ef6c53e41","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xf37cbbc2dad42e8826898d45933afd3a04937e71","gas":"0x112c1","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x3c2b6c2ae505c5c36d5f316c1fcb5f54f7346ed35ae35c93462991ded7968a68","input":"0x095ea7b300000000000000000000000071bcefe9e9fbad3f12f2e6c14ea0cae7f0c91e7900000000000000000000000000000000000000000000000000000004912e6bef","nonce":"0x9","to":"0xb5d86e1f237d52f1f19793d3265e9340394f75b8","transactionIndex":"0x21","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x3ee4d5bfc575fa4b241b4d23e6f099eba5f1e6a20b8cea663c63db6400d01506","s":"0x21481ea7a3e7ad758f2e149829f70f43d67b3cc0d44112cd3744e290a06dd985","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x52b6255b7810d97de8f751737547d1792dff4fe0","gas":"0xece2","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0xf99a792837e13827b5e0a8915fb59c760babc95d242feca99a5594e64ff6b6e2","input":"0x095ea7b3000000000000000000000000c92470d7ffa21473611ab6c6e2fcfb8637c8f330ffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff","nonce":"0x51","to":"0x07865c6e87b9f70255377e024ace6630c1eaa37f","transactionIndex":"0x22","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x5237f7449c0df409c1a4d6cb09005a9224860aa74737debd961e2a24668d1022","s":"0x17e5e842e3f5ffb5a16eca0ec2a92e5b52d43bf9856f97e2083b2e951499b970","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x3c70503d83c5a681aab503d724f47fe810ae8205","gas":"0x17eb2","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x522313f5d923f048ae5bd0b5595c1f4fc883bc0b3cf3cb0939d3fcf8b08c829c","input":"0x94d008ef0000000000000000000000003c70503d83c5a681aab503d724f47fe810ae82050000000000000000000000000000000000000000000000000000000316195298000000000000000000000000000000000000000000000000000000000000006000000000000000000000000000000000000000000000000000000000000000010000000000000000000000000000000000000000000000000000000000000000","nonce":"0x5","to":"0xb5d86e1f237d52f1f19793d3265e9340394f75b8","transactionIndex":"0x23","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x6741309799fccf0f1359bd710291ec1bb66eed23522d6ed60c5924264d4d283e","s":"0x69c43182ee8a650b0042cdb2352781b23eb1de72f1c3a8903a1a33360e7d9732","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x4da9a48963c158a88bf6d4845a6a1890d20e55ad","gas":"0x431eb","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x471ceb0e85af594aa56deca54cb8198567b2afd8406722ea530077aaa6b641b3","input":"0x9f8420b3000000000000000000000000000000000000000000000000001dd7c1681d00000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x0","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0x24","value":"0x1dd87babd44000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0xf977ce19da80b7e155ee2d8b7656e5c89004851407bd5370785befe2c89d82f6","s":"0x1d76177407e73d8b73b0909158825ee375a25fa7c18b04884f4be77bb60795a8","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xe40a2cd9efde08fb7664bcdbbdd877c52c4bd228","gas":"0xf4f6","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x3e9dca502e9039ae0c6d642f62e9562ff00010c6bfbb8234a6135712ba70dfda","input":"0xa1ddc5460000000000000000000000000000000000000000000000000000000000000738","nonce":"0xa","to":"0xdef65f8e2f62bde60cd86ad3b4ad7fa6741cf175","transactionIndex":"0x25","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x34df85658956c9eb71549db3a73abca70dd01b8e91284afcd372c1212a36605d","s":"0x5036e2984bb2edbb5fd647bba873f1f5c5ee4845e539c9b0bc70d3d12a4bd93f","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x17299065e99b12b0c8cad06638624cff8d507334","gas":"0x5208","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0xc95cac67267f4accb9b5950316ac64772f7d082bed6b712c09cf2da0bdc237b7","input":"0x","nonce":"0x7","to":"0x02647cbb1714fa9a5747ec5be6333e06d806e85a","transactionIndex":"0x26","value":"0x71afd498d0000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x82ee5cafbd010cdf7a4604df728d1b8713a5bd26a708f5c92f6b6a55864c9576","s":"0x2b9a70d812123b2ddce1108c4101ddce182f6c83fc8aa0a139ba597aa178b270","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x9aef7b879787baf0c2dfe888e67a7edb7a233209","gas":"0x431eb","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0xfca28fdbd13fc16daf7aec7d4a2ad2c6b5f0b2a7b0fb1d9167c09b5e115ff26e","input":"0x9f8420b3000000000000000000000000000000000000000000000000002386f26fc100000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x2","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0x27","value":"0x2387acb3784000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x7e5f1a27ea5e652ec8705f80365f1703d2fd50c1de7c22ee0181d556dd2fcc38","s":"0xb22fb8b31dbfff8ad0457ba4b659fb0d7031cd86f08fc3cb78d81c97248034e","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x06006a25e0fe65f0eeaad1f4813940b68db782a7","gas":"0x17ea3","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0xc73124ca798b2f7a5df2ea4d568efab2f41b135130ea5cc41d4bcb4b5c57d5bd","input":"0x94d008ef00000000000000000000000006006a25e0fe65f0eeaad1f4813940b68db782a700000000000000000000000000000000000000000000000000000002726bc7f1000000000000000000000000000000000000000000000000000000000000006000000000000000000000000000000000000000000000000000000000000000010000000000000000000000000000000000000000000000000000000000000000","nonce":"0x6","to":"0xb5d86e1f237d52f1f19793d3265e9340394f75b8","transactionIndex":"0x28","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xabe4fc843d3a1628a464884c13c21bd21ae9a0a5738d9ae2761ec7cb24b059d0","s":"0x68408d8096e542b2c06cbbecb9494a3778c3ab644e69dde1fac078079c4bcad4","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x2501bf5c8ddf04a60fdd47a95d53d99b315604e2","gas":"0x5208","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x29abb76b5e7a5ce137bf9c22474d386eb58d249f43178d2b2e15c16dfdc5ca80","input":"0x","nonce":"0xa","to":"0x901a4e04f1513443c6efc6ce4e913d1b960ecd20","transactionIndex":"0x29","value":"0x1a3385ff37f0000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x1a1789d5b19950f45383cb1909cff9128dc2b332e2ea7ba42cfca5806376e3cb","s":"0x617a5a4339503a9c9d08705d98e7a202ed7ede1d1192fd7d98b648c8c102716c","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x507832c92ade971b9a2880427353c6ce6d0ffe53","gas":"0x61a80","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x03e5ab25a58bd44fb9dd0c698b323eab8b8363479dfcbcbb16d0a0bd983880ae","input":"0x85ff842c00000000000000000000000000000000000000000000000000000000000000c0000000000000000000000000000080383847bd75f91c168269aa74004877592f00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000006100000000000000000000000000000000000000000000000000000000000557300000000000000000000000000000000000000000000000000000000000000014507832c92ade971b9a2880427353c6ce6d0ffe53000000000000000000000000","nonce":"0x1e7","to":"0x805fe47d1fe7d86496753bb4b36206953c1ae660","transactionIndex":"0x2a","value":"0x214e8348c4f0000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xf707322f65ba56c95ced5654efafcf9bfd60a2dd80e74bded3190673cda55d8e","s":"0x46412ec3d2137fcef30f53cc4644769f4da126128979920ed4e209acb1c8ebc9","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xd44c850ce9103d6b2398d4ed9bbce321f3b769c7","gas":"0x1e0cf","gasPrice":"0x82661bff8","maxFeePerGas":"0xde6f782ba","maxPriorityFeePerGas":"0x59682f00","hash":"0x3c8ee80ddea7fa2d2b75e44563c10c10756f598e8ad252a49c5d3e8a5c8e6cbf","input":"0xeb672419000000000000000000000000d44c850ce9103d6b2398d4ed9bbce321f3b769c7000000000000000000000000000000000000000000000000002e2f6e5e14800000000000000000000000000000000000000000000000000000000000000000e0000000000000000000000000000000000000000000000000000000000098968000000000000000000000000000000000000000000000000000000000000003200000000000000000000000000000000000000000000000000000000000000100000000000000000000000000d44c850ce9103d6b2398d4ed9bbce321f3b769c700000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000","nonce":"0x9","to":"0x1908e2bf4a88f91e4ef0dc72f02b8ea36bea2319","transactionIndex":"0x2b","value":"0x2e2f6e5e148000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x327a6fbcaa2ac3cfb0ce64b47f6e61de4d18297f1305f23c6bfd42d88e8d068","s":"0x6b3a9b5252ced14b2e2f2833ad7cc62c46fbd0785d57f2747473af3fff7d46b8","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x78aa9046272fec8d51a402681d7c37481db01c6f","gas":"0x431e0","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0xaffa73b68bc7ab0c3f5e28377f5ca0a5df33c0a485f64dc094b7f6ae23353203","input":"0x9f8420b3000000000000000000000000000000000000000000000000016345785d8a00000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x36","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0x2c","value":"0x1634632a1414000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x925370b51dda3e1d1bc6b86580fda3b40875157ea594a9a14174436c726ad46c","s":"0x516c88bf3cb2f46ca81572140b41e5db2d74c4a736ee7d572b508415c438f779","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x56784e51c2ce28dbe9aa68342638d1e95ea4324b","gas":"0xb72e","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0xc66c9c66fbc8fe97fcc16506cde7a58689af1004a18c6171cfe763bcd94f50b2","input":"0x095ea7b3000000000000000000000000805fe47d1fe7d86496753bb4b36206953c1ae66000000000000000000000000000000000000000000000000029a2241af62c0000","nonce":"0x9","to":"0xcc7bb2d219a0fc08033e130629c2b854b7ba9195","transactionIndex":"0x2d","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x407c855bed0fb088e886c3721ee9abee4761221d110512ac29b8fb143c4b34d4","s":"0x1e47aa957d9ebdb564dfa41b72022d525383ffd825a307fbfa9ca13a59994dcc","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x805e39acf2f170251c48a35df16411cfb2f761e3","gas":"0x56a8","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x80fec96707519172b53790610d5800cd09a4243aca9bacfa956c56337d06f820","input":"0x71ec5c05aa669c4922569c1d33f7a81aaa21813800000000000000000000000013a0c5930c028511dc02665e7285134b6d11a5f4000000000000000000000000805e39acf2f170251c48a35df16411cfb2f761e30000000000000000000000000000000000000000000000000000000000000000","nonce":"0xc","to":"0x7c125c1d515b8945841b3d5144a060115c58725f","transactionIndex":"0x2e","value":"0xb1a2bc2ec50000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x8fcf376106af19aac48ecb840ea51df90a5e3ba557eeaad41cd3c8a90785d416","s":"0x4a9e9021ddb5f839bf7f54b54ad52c57b21b6ed285773eea022e3ccae69d1b99","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x4199614bf74f6b3ef47a812d976dd1f57a8f082c","gas":"0x431eb","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x61b33bfcf11214906dcdce7d7ed83ad82f38184c03ded07f7782059d02eeedea","input":"0x9f8420b3000000000000000000000000000000000000000000000000002386f26fc100000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x6","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0x2f","value":"0x2387acb3784000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x3d295f5fa2dbec04793f07adec57e29c03376c628a0256ee297fb3070a1e3364","s":"0x6f115dd4505bd320bd7460a47bab0c20e6bcb4b3f17c6d0e00e3ff1d936ed2af","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xf6df8ead0bdabfba61d9b2457d5a5ccec67fb7f3","gas":"0xc7c7","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x5d4138d4e28a8327e506cb012346b1b38b65f615a2b991d35cf5d4de244b3e6d","input":"0x67a5cd060000000000000000000000001f1f156e0317167c11aa412e3d1435ea29dc3cce","nonce":"0xbe","to":"0xccb0f4cf5d3f97f4a55bb5f5ca321c3ed033f244","transactionIndex":"0x30","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x47bfb48eb89a8b110749d34d171db36a8ad21c8d7cc78c7489e72f21cd86ba14","s":"0xa525976583b19b3d2c2dd6b34324cf55b1a61595cb53e3c58acaf203f8709e6","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x9eda8333e09828571db26576a566668f79f98946","gas":"0x50df20","gasPrice":"0x82661bff8","maxFeePerGas":"0xc8d5e2427","maxPriorityFeePerGas":"0x59682f00","hash":"0x875a142b6dfcf10ffb71a7afe0ce4672c047fc7e162ba0383390516d6334d45d","input":"0xb1dc65a400011a04d404e571ead64b2f08cfae623a0d96b9beb326c20e322001cbbd34470000000000000000000000000000000000000000000000000000000000bd780a7d2e00ee03a29f787096cae99da52d07803c76c13473654239118e2cf0491bcb00000000000000000000000000000000000000000000000000000000000000e000000000000000000000000000000000000000000000000000000000000004000000000000000000000000000000000000000000000000000000000000000480000001000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000003000000000000000000000000000000000000000000000000000000000077359400000000000000000000000000000000000000000000000000000e50bb6420faa6000000000000000000000000000000000000000000000000000000000000008000000000000000000000000000000000000000000000000000000000000000e0000000000000000000000000000000000000000000000000000000000000000217f0aee859cd78a5cdf79c68c8de4342658d52ccc6420181b5fcb7c657c5fdfa5ca2c8f2dd7e6e5799d44d6818e6dd8fee19b2530bed2045e0c665041f053fe70000000000000000000000000000000000000000000000000000000000000002000000000000000000000000000000000000000000000000000000000000004000000000000000000000000000000000000000000000000000000000000001200000000000000000000000000000000000000000000000000000000000840235df8360bffc902a68675311f5eaa79ea2c25e2cc394bff0a43d5d608e98c97de200000000000000000000000000000000000000000000000000000000000000600000000000000000000000000000000000000000000000000000000000000060000000000000000000000000000000000000000000000000000000000000002a00000000000000000000000000000000000000000000000000000000000004b000000000000000000000000000000000000000000000000000000000000493e00000000000000000000000000000000000000000000000000000000000840235df8360bffc902a68675311f5eaa79ea2c25e2cc394bff0a43d5d608e98c97de200000000000000000000000000000000000000000000000000000000000000600000000000000000000000000000000000000000000000000000000000000060000000000000000000000000000000000000000000000000000000000000003f00000000000000000000000000000000000000000000000000000000000007080000000000000000000000000000000000000000000000000000000000061a800000000000000000000000000000000000000000000000000000000000000003312d88cdb7cfd5fc5458d9a424da374cd818847f87fc4fb6eae2c3b97e541a6b81e108e55daf857b71514fb7213db7ba15bd6adc12253afc3c5e949144c13696faabe22d1b7ace91532134a5cc65bed9afdd897d6f84ad1b0b6da463bda8fde5000000000000000000000000000000000000000000000000000000000000000352c2a75f88112864819de485c392c349f718fc257934181c4d8ac8eb3a9b1fdd3b7bfb96496290f2b41fcd20fc8d29bd8e29be38550775dc97d21b17c2d679ad05f519787eb2c92a9a897c9f02f15316865cfeb4bcc2c0ba5ac2dc4aaab5c0c9","nonce":"0x3705a","to":"0x233a95ccebf3c9f934482c637c08b4015cdd6ddd","transactionIndex":"0x31","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x7e9247209bede2c21548c6e4f26e41d2636a2490e09ad062f9235e978094073d","s":"0x288d820793b48e55777e794f1694b1605d74dc8726fff438b481125f219eaec","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x6df2b54f23ed4e95235c6e21ada2cfff5a2615a6","gas":"0x5208","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x79b6df832bfbd04085d0b005a6e3ad8f00fc8717eed59280aa8107268b71e7e0","input":"0x","nonce":"0x17f","to":"0x6143d939b2d117f4b998d8d88a56eb0e0a1e9c31","transactionIndex":"0x32","value":"0x186cc6acd4b0000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0xcb641be908976bc0a035b0b31a96039da9e63e829e0a24a092ecfbce7724b9c7","s":"0x4034e7dc4001cf9f62a2e5dc4390229655b85e9aac5a5b263278e9d2cb43abb2","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x3f5ceedd435b6143eba2254811a39fc3423f18dd","gas":"0x186a0","gasPrice":"0x82661bff8","maxFeePerGas":"0xaa246a58c","maxPriorityFeePerGas":"0x59682f00","hash":"0xcb2fb25d268f65dc9312e89bd3c328c9847a3c9da282026793c54a745f825ab5","input":"0xe9e05c420000000000000000000000003f5ceedd435b6143eba2254811a39fc3423f18dd000000000000000000000000000000000000000000000000001c6bf52634000000000000000000000000000000000000000000000000000000000000000186a0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000a00000000000000000000000000000000000000000000000000000000000000000","nonce":"0x0","to":"0xe93c8cd0d409341205a592f8c4ac1a5fe5585cfa","transactionIndex":"0x33","value":"0x1c6bf526340000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xa48a4ece6fbe881e342a4eda767c82097c3bd47dc70c09ef6a4e7f90c0f8e843","s":"0x66c92f16907488d115f5088ed8ff4d71c966232458479245bf6dae45358c0429","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xdb4f0eeb7c89b00118cc74de8c9ed55d65c2b933","gas":"0xb72e","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0xe483d4a36ad19fd5eacb7f6d9ad3ce080ad70ac673273e710f6e3d5acbc6559c","input":"0x095ea7b3000000000000000000000000805fe47d1fe7d86496753bb4b36206953c1ae66000000000000000000000000000000000000000000000000029a2241af62c0000","nonce":"0x120","to":"0xcc7bb2d219a0fc08033e130629c2b854b7ba9195","transactionIndex":"0x34","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x6d73169394cb6a9bceb2dcb16aacd261705096f50f30ffe146a09659e25dae48","s":"0x496a1cccf01ad2ae7094b8da2ce50c0a990baba6300625c48ca12c5f20005a64","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x9ac253db14a434cbe7653a99c5039648fa4f7353","gas":"0xb4f8","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x0564242c37d5013b671ef4864394cc0f3924c589f8aad64118223a9af2f164f6","input":"0x095ea7b300000000000000000000000028ac7dbbf9ed6ece39a902e76780fba60f1b2e3b000000000000000000000000000000000000000c9f2c9cd04674edea40000000","nonce":"0x4a","to":"0x0dece1605795461158538302b5f0936afe78c4db","transactionIndex":"0x35","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x15684c9697dcc8d89abe562c4315bfb959938fe7644c00615d9e147a4e496ff6","s":"0x7c58abea24e1fa8caeb73a70219b0d63007785c39694fe5ed2a5a342c99292a6","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x22496eece162f52d185c07524d452b9506e9fe12","gas":"0x186a0","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x48db358e80b278c3a46c2a166339797060a40f33984a5d974992cd9722139d5d","input":"0xe9e05c4200000000000000000000000022496eece162f52d185c07524d452b9506e9fe12000000000000000000000000000000000000000000000000001c6bf52634000000000000000000000000000000000000000000000000000000000000000186a0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000a00000000000000000000000000000000000000000000000000000000000000000","nonce":"0x0","to":"0xe93c8cd0d409341205a592f8c4ac1a5fe5585cfa","transactionIndex":"0x36","value":"0x1c6bf526340000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0xa50e6ea1eff21fd33d69f8c74b90c28303b92ebc4a77c71b65c4c8bf3e92ac8c","s":"0x29af3bce2365578b76dc528917b44e96e4fe2e7a8f948eff3e379721f35664d8","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x9ce3f4210dc06889ad5267ef0ce3ded4bd96dc50","gas":"0x249f0","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x69d7758db91fae31fa35ecbed4d40897c5087f45dc796cd796b8ceead21f972e","input":"0x47e7ef2400000000000000000000000089f01bc9865324583380d8d7ed08b8046bffd7fd0000000000000000000000000000000000000000000000000de0b6b3a7640000","nonce":"0x1c","to":"0x6cf3df5d00e842aab1eb504bcf01ebf81646f7b7","transactionIndex":"0x37","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xeefc19e8dca874308c84712c3e48c032a25971a0f7dff16dc47633ba7a4f5b96","s":"0xc0994cf21d29cb0c5f1e666538b58d6dff3b9ba2aa7545337da732c4b5938cc","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xbd1c4e36842c06e53b378eca310cd2dc79a78cbe","gas":"0x35965","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x2951478916ecd27a8e808d08f85be4bf2c0b0e0546f21f4e309145dd96eb8df1","input":"0xab7d7969102507275d685f3f55d10ee45f7b9fa39c29a8a604a7a971c05eee739e63938500000000000000000000000000000000000000000000000000000000641cfd18","nonce":"0x252","to":"0x96e6192eeaf7bb308f79fb5017a9085754b9e12a","transactionIndex":"0x38","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x87fcffa38e0c84cf80fb37ac1e538faeca21462b0a9ec25af0ca0517af22789a","s":"0x72cffece99e79edb7425a9d660590fbc395cdd0db8fd4848074c85f814b3cf63","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x5dbcb3e489fbef1656dbbd7bc4848ed0e520a089","gas":"0x7a120","gasPrice":"0x82661bff8","maxFeePerGas":"0xc8d5e2427","maxPriorityFeePerGas":"0x59682f00","hash":"0xaca9028cb5d55bbf71b7bff9884a9a3b0b38a575ffc8f8807ce345cf8bd298ef","input":"0xc9807539000000000000000000000000000000000000000000000000000000000000008000000000000000000000000000000000000000000000000000000000000001a00000000000000000000000000000000000000000000000000000000000000200010100000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000001000000000000000000000000a7a9d91e687decaa6b045797c2e002d70001b48f03020003010000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000600000000000000000000000000000000000000000000000000000000000000004000000000000000000000000000000000000000000000000000002361273ef6f000000000000000000000000000000000000000000000000000002368a4faa14000000000000000000000000000000000000000000000000000002368a4faa1400000000000000000000000000000000000000000000000000000237b9042149000000000000000000000000000000000000000000000000000000000000000288e1b54f6e54124deb78bdb67cbfd0bf63612da12868fc281afc34bb26200a1771f9e00fce297da11cf49427becf350d0bd20a2277439fbb2e8db0fbb5e3d34700000000000000000000000000000000000000000000000000000000000000027d16a90ad0e1f8d924255e6380764fa7c7d46427c8e8ea19f5ba6a9d595297a340f141f74e705d0fe851d7564eba89fcde574deb5975d9126867cd927a0d92cc","nonce":"0x11d38","to":"0xbe57bac491de9a260abb6ba2c9ad4c5d2eaea09a","transactionIndex":"0x39","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x1eba7bf95bf63d02b74d20554af050eedd78ecfbe1a42ec70badb36ccfae4bfd","s":"0x4d7f8d0ac2be28e59374453f24ae6506a81707cc57f0d1c6877b49b3949a9a79","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x3079be9d8622173f02618ba2b793f00795d4f320","gas":"0xf848e","gasPrice":"0x7d9f6fbd8","maxFeePerGas":"0xd9a8cbe9a","maxPriorityFeePerGas":"0xcfd6ae0","hash":"0xc7f625a19ee41a1750eac9428b4394a9a2476b8ea2d31b4c2f9f5b4fcb86cae3","input":"0x2169f79f000000000000000000000000000000000000000000000000000000000000006000000000000000000000000000000000000000000000000000000000000fdce50000000000000000000000000000000000000000000000000000000000003f2000000000000000000000000000000000000000000000000000000000000001f579a43ce4f3e38c6a948f1ef3bf7954c2059b087b18fdb22c7b69d61727a5d2aa16090b4d1f3281bce660a31d3e2789eeb360192a54e3ac3513c5c8f67b7346fda957c92c1a2981bc68a47d0282e78df2fb63822f06f410e1ffbee7eb83c425122be9be94d817e2e42099632fb9eb5ce7c414cf6ce48e4ed2e64e46e1558ca809742159e841e3de803a8cdc628f7877bfa6d75af4b4ebb185e6d0670ce83332478137fec243a38c0a80d067c955cbe13103294dadfc8c356d0e7996cb62fb2c683e6dd5ad58be934f57375920142990074f8a161ad2390a1e6bea9fc3a58dc3f7e002e9952a68bff229437d3fb4cc1cd8097032b22c884868253d8f2f69c75ada0fbe1dd57145610ae5f451b47645718aca7432d8cd3193d0b96417df6638fd47062a1cb1f63636236fb72195e5a46d1b37f493b8380315e5bb1a6fe67cc6546d62f2c73aad5699e6ffa0d49ce653bcefa173fe0de7b4328aad75f74d15f4f570a1839514550d576e21b8d16c8355b770442f0740e6e5e77ab450ee1fadff8daeeb25a91585192fcd3cdd64dcb6a35e41ff47e854337ebd0923ce280cdacff2ced48eac96bc9f910391a44cfec243a8660ff80c10ff844ea22411953e0387f87d2c2ae81055fc8e87d59e04cd7a923d6aa5204b3eaaf5b842b5fd656e9fda16c59e90c64ef51124fb32f8e63442b1f79d3ba3c0cd5fb90895103719629857d5999b44aad2d69380f48dd824795a89dce14bb82600701ad8fa9bbb36fe93a97d75da26ffcb1a8828918e0293680ecb12e1b772fd98637a7a83cffbca1a994fc889ed0c0b20d447ff727b9f5dcb9a83e1aae9a3518dfe1d7ddab1da48310eaa7ab293d85dd9ca811bfdfee990459fa044c9a9fa6460e5ff7612e42021f16854d790710684663bb6a10ba475fa602b02519f03eb544fe486d9cdde47ab7d4027bfb8f29d778b3a7cc35a9f78dbb05943062efe3b388f1a8c496682a0215265b99fed0363a1061556a40e0539a6656b6cf8967c8fb3092a7fc9b56af124c8f1bd0abbb9af753884b22b333410d47d80c083c9fee52dda8907be0c87c5496f1fe9ac4bf4dd4f06d6ac0ac0cde22138b67bf782e8e9b5c991d58a2a828b4c68620e6d3d26abf8d41005a00866cb4095518b2711636f2fac3b5e26403ef96f5b1765519787ed74c6802b445de029d7bc696aeef5a2e605376aeda4f495c29af66e357c6e1232d7a2c3f22c0bae0c62372d6b2690e503804e2ad314c38a4ffc080d36213458f606075d32dc0a365087616e94a972bf76d7e5eb4a53292fd0d7570cb6fbb4540b3c5ee6deee0f257fe8703d5d8d0180d09754bd9a05894e77318b8c8fb6b4a56999bfc6c405c09d9ce7fe424812db5bf778d7d8634b2f44ad6d00f18615e4c3695d9c5965ba51991508405187275840a3f27cb3101a9742c49edf42dea79ae5418ada42dfe2042f931af15a205e64196b04ef53492a450423c2b63dc849ef72a166fac5f36b25c72d9cb4a2507d5ad15094a979b4ac4ff8eb4c90f448a81fc6dbfcf6a228675ab3308bca37bda8eda22b61ec7e596f2448ee6b7cb6eef589f43f28fc5444d1da3f82ddbe574b774719a3a7869e897e642735517832959ea9cb9164eb2f7665bca1d2e44e1093acc7088271f2be20459c383f5762303b553af1ccfb3371e4eef1f102ce70d62620c5164972facbfbe03b5018265ed62c4c3cc49522166913517e2f2718edb591047a72e818f07c645a18aed87c0384d180dc0bb7fe63bd335b1c09e58571b92e2267ffa5712a33e4a92231d9da7b005730f6d6559ee685edad9a9031c4708c8642c297287af3a3e637d9b5ac6b23218977cf4d30231a79063987f98313a2bc0ca4a73a97e441a45b58cdd8731fc269255b415a9d2ef2bb8a311587f39f6dc82f537cf86a68af19a68c409910f4dbebdfa66445361877a3bf551480d115d8768e8650e9979472a64802da3625cdbddee8f02678a3cc6400824d91b3c7d4c9274b74fb11a9538f1990fc314d35564a0cb0b384590e15cb93db1de1a837b9319828b3fd6b76f1d2bca4ac765f9ec4366905f2dca69a4c8c194cf0a5d821c58bd291e8dd4ddd829a873d093042c1b8e1bb2bb1a95e150c0a0d5c3369c4a791dfcfbc3e631da3d1e704214c3cf8318901f9c58465be78a870a040f34c567c27eb077db2d1ff825a127a038eb14c2890ff9a8a1dc8d6360a91fd788fec97599a318ab03f69bd50a5ece92d96c49413b517579ececf71371b6b1d902b1dbbf40c728c64fadf368582cc4b3c51712c8d6441c0c61802943624e93f9de9eedde87056c21ecf66b3a6977af0261ef48b4c6191ca7ea70d6acc5af910ebf247e2a4429c506c6a41f865175a9bfc81dbb69e59ec2c47370fcee9169b8749c852e7a1275c1278aa3f88bc16d311e58aa9a579ed7bed62f12b4c4dfe9c493cf33f979f667eda043c2aaaf863a747b208f3aa2ccc34f07b12d89bfd75b696a82c493e7eedcdc41eb01321ecfdcb691d14996c3ed7487b5f831436bb7c25876c4149cb6222dbee9e1a05771a8abfd0fd530266476e1bf69d9a15e99c901243585f56a77ef0e6c684a4d49417133ebb91f896176ec3a8f0b966eaffce96f129d18960fdbf5917f8aa0926d52c57192f99e55959369e57ce85ae84ad944e40652f6a81ab8da1dbe81a827366b8b8d2ffb9c7b93087f7a356a16d0022ee8ae891060e29ff11f9f17b01f39b8024baefc8b928a6816309a5c3fc92ed6007499353b5f1bc5700d5fe67f3e34ef3654712b9c67c231f30dff485213b94c20afaa91540560df80ba9ff1cc0f6ea75bf1dac0566551050873c112478f3fce6af162397161e1b41e1ca295f327a4f0ccf4d15fbe58e20a23698bdc25c751d020df371cd1fc59fa51dbf7ad619d208d7084c5752984b5fd0e3a7978f2c11e6b26531fe03c0cb85519483d668bf1dc4dce24509b00aa6e146adb49f31c2c937a2d3a08f1cc0b73cd66f2cb926f0453cf78618e39cbfccd6e6a3b4909d6874872eefed0e395ac2582d8c1d0483007fa6ed7b735323fe8fa1c4be83c725c6b743f63cbd0d6017f1145634b0b905552a2a237561b37c415146d0555a276b4b45d515ed7dab942f9bf40a26d5366032f23fc5fad8b98a1625a841c8cc97dfe0e2074a363a951946df16d1261ef6b2da7f5789dada153448b076637f9d2e04f4390e9dd32fd5e9ab3f5939807bb451bd19d962403da9dd118f1cfa91731973bc721e5ae4a687bb2270e3e76966dd2156e95796d4bfa6b4c10caaee46e1d8f8495f901cfc241c6f9a690349eb0752adb220996ebb95ed26ef376dbb9eea201906a6a3eb346c64eba3c9e4d1b3088baa0ed868cdaf063a4c2434b9d1bbbd69f44ef659262fad9535988ce729aadafcfc6c3ff741dbf8b784b3465fb05f6d5daec91e915f89117e01c9b0c99d7b7cb78e44f5c28de0d3b782de981be058ffa1294072c4e5a66d2d81641d1c07a9a04012f9f1ad15028709fba761a659becf01963de0cc42b0e16bc5432cffa00722a84628373198dbbe75f209115c4e30e60297c2d737996dd9aac330f4a3e82609dd33460438026ae287c172af8aeed0d6c71b1f9470d60588ebd365f20d596999d97c7a6f265890c719dfe43ae1ee5bba122d9d4cfc96b399f610a90606add4f0d25b903c7ddf6eb7bfdc3b9f926243108f591678b0e92a48d52b1dbcdf30ef8af74cfe34d48d0e02d0f99af2adb090c489d4214f3562d3f6a6e9b83b54bb6cfe6c1961ecd2eb2b7eda63601c835c5dc696b32bc495bda9b6567984f83e79a9d4ee88b140dc78cdb64a2ba8761a827cdc8d2b27e690c22402178ade602f2e9092a1b07d56d142bd24d665d7b10bc7646eb6d6823d5622f1fb08f2e48b8deda6cde324d6508fc428e3bf1a9ed608c196786db4e8d6b7271a07e4f73d9ef76244e3e10aa13fd9b82ce2ce72af2df7faaecaf95d885e64bcae16f9b59ab8e0f21a2f459d00a290b0dc57e10bf21ce06a029224ae15cb86bb1cf060ddccb1f6ae82d767eb077fa4ab0f37d02a4bede5d3f8ee3972a6a3a438a3cd92ffd6c1fb11cf2c68c2c6d1efff62b9f769e302f6c5eec96b0c05a942321ef8b9cf0ff2bb80073a8fe138bac5bd8e93f3b27e7ec7a78512059fa37b0b6939f3b1311cb6ea154909e9d316567e013df315e8f02a80e17feb8f3248cc3f11d1253ef3156d7f12d1f6d7a382d6b6ac6abd38fdb5d954e13ce005ce16bd5625a5df01883409f4ab938a59a75b7a816af6a2447b2c65ea55c0ec4e4f23fde145a26ef0b13b740b352809c10b54bfdf30592a8b0d507cb8210391d66d6f68d06f3a4663756697d472adb6ba78976b423ad9ccb3a3bead9de51e324fb77609729c78664f4d37c7c748ad9fad95af355be4f00a76cd5cc2c0b6170fdf3efaa2571bc8fc2d0d24682fec5e034460de55cd0426b8ad448551032e1f6ade6d96682bc38a92c296e5bdbf42fcf106d14c5e8a4acbfc0cc1f0bd6877c4469be8b204ffd5d7c0ee85e7b3265011ad5e4896f9f8a69997e321a797549cd3c1cbb0301ff30c3a136068ba37b2176e556b574a542db90250eca38f3b0e418e352b5728206d6d531ea253d781d771ad974b476a045ff14718d9a1820fb469e981c7b71ae8fe423b0a04278bcfc967ca602254de37751127fe3716456df3fae50ed508e529c2181d63479b6464bfcde6629e8179f8d5fa5b61003edfc2f129affa58608d6145de98ca8f1c0b3b9b5ae996a7fbc6326a9ac2880206fbeb1c99d0448cd1ac5f275b84e4be03258f9fd0d3f44a692513957fa083a462086787315e38aec8c09b96cca034cc6294359fe00285c607d41fb4e3bf25760ed1ef0802bcc40b2fce2cfe1c64f7c34b21d0bb1f5210360079ca8638ab42943b0ab8277f1cfb1e529073d2068a36a4d8ae8179c32bf66ca335d698f76db1cfe5236e6427e01804464c1ed4facf18ae09148c5a0cc9ebcf6fb6c7d03ec6782b7595816245ac2d1fdb3184ea49800d5d0f1dd880ff99d6016549d63ba2a44ab96f0db0d0cbd5fd4f67af1f1da0b1023883d4b2e7cb23cf85c617c180a771267fe63c70f19eee28fdc50c57d73b2718f8001383135b859f4c2fa3b9a12d095beba8857df13e921d9dcda5b87e1fce6038fc2f92282c717305157b6fca5ca2c23cb7307a093512f985628e75326e0bca2c4dc5ed44bed8e0cbce1238f4023089a81bf62b04a31539de9c8a0c867b791c18a34abcc721c0e90a1604e8d2855e78673367ce463307f6036850b81e0aadc9f81d8c9ad6e3b81d7850d4a46bd6772e651e03581a7a55568c9d63c1b1e8ca947174d68afcae6401b75dc7397be3c0dd76ad1827424f9ea9de8ac99803641535dae3eb57ee1eb1e2c193bf66addd488da55055bcf93172b1966364db4412e44bde64c50f7692a38d1536683af14fa057569cc7ef11f614a58995ade1b12f979005a53bee0769847deceab4f58f3d062767f7ac0511a04fea349e7688ab26425cfc369127325ab7fd24d73c4a8b23f099892b3bf52f35f0d775ba753098611ce35c56873a7455737a42672747bcfa99da8c8d0f06d8754d2babe3eb303cb06ce69e4046541ce63b6c28ded5a6fbf729200060132e9b01d8646b512356c92d8a642de0f62d169b195cc2e7ea2efebe550b1cddb5df7668a0d1f94a2f5cdf7897e46cb03b958ad906393c7d252b58493ce6c7d813c8fdfd5ef577a9e778c64d48c21b130c4f6a57985b8122f3cb69de015081cd3e247de65643f9d863fac6935cffb2ca23083d4fca9b8707679d9d4c3cb34d05d092334b1740ccf4b2aeae724902ad4fb4bceaa642b3ffdd5b564d7bfb725fae91f5c7ab9583abc2d80afd2272b14ec6fafbc51e35bebecce63bd95fa3f39e4e689034c47628345c4d8e30a73c36d87d289b6ec715a2769efacf39c9f19eefdb311b9800377c5c61aadeac78dba6946da4e7914415eced875b66f10b96ec98f64e58320bff09e8ac4f5969608171dbd4ab5883548a289b52d55aa82da62ba3d9fa59734d8f810bc1a40a0e3b2e116873116fdd8bad95471f16a14054fe247637febb6081d00e7ded3019ec03f76ebef5401a27f380fbe8a55bb4a0014f61276d9aa5626a34e7fe5a129a75ff130ccfd78cfbde65416882ce9ae9da57a3a04c67d22d3a12e811b2578fce5f6a64a09a86782bc15e5872e546dcca46bd9ccce5b787d17c70d6b075bf4c789ca184dedad3485a0526a61d280dcecd984fcb7ec663f8f491e91253675877987433e11b9cfcb5940a1357b55fcc5f977fc898f72e8199ed5d8d2554d54a55d6b4b3726e7fd7093996e271e218533f3b31257aa989f5efbdb12dce02d18a5188ec32c3c1c91845a17011ffede3f060f19d59b6565a352e90c5ca30eb252c6fb0cc845833e553f0bbe1822b5d6b26eb817608799b5b77c2cce27beb151060261f34ea73db94bb2fee11864aff76da5b10b17aba6634ae655dd396fd7e7155013fe3c6f840ab1ba9b25489c67f6bd5967a62134b238c325986c87aa005396c54af6357ca88baf5a03e353c8b0175386157d9221833585921d581f1fd3f5d9932b24d41895a5c35be91b35643e7091a7585ede900853aebc260b1fbe8b72d527390ddf46472f7c8d5844355ffc0b731dd5e584e13d65f2347131ba8a5161a6749458c1899b60cf9f9f8810889fc23b5d0fe2c843fb5675cfac6dc8ec8125d1c582186eac82d5e0b3f6df2a83eb2fc23c4c3f873267deb5ea86dcf1e3c5bb39df23f06560609b9fbd89511281fc7d69fb19764c41168a539b44f0603a52577c266fd601c0cc849b067746b4da78d280385ef80a3a553dabe4b137c50152f4e1de456d43e015e26f7d76e612b7be29b9f8e15c6a73653f5f7dc28aa245e64f076130bad58bbbd3b024c97e267a96365f019846d965c037116c74ef18415ed5500f12a1816997895f3bbe7b0fa1eb12a565255a64d5eceb2272027e45ecb0a1a8b5dc99128c002058c569076371703e1f10368c6cfb830ae8bc688eb143da7a1780dbe29cc4c83743b25d570f5d8609df46c807edbbc96358cde9ed0d9b8936271b3ade28001a6bfa6ff7ffeb4e6a6687e803b5e8f67b82e6dff215ee4f2caee571b8febeeff368a89539d145842526fef110174ba3a2d9f1b4a348dd25dd964efcf88847d3eef1c18fde81ce84b80cca8588bc1fbf9c9f34059808d3eef3fc2cef7636f7a76c533933b79a9dd2ae2791d1ab5e1e715837a766d9df1947d5986c21808eac758461596870b744a6a2862f7f5bff0c7a3460875b3b326e45fcec1cdad1cb4b99085748d69a62320f7e6c53c061a55ae20368a4e6cfc126cf2c0b410af545f169fcc741cfcec4947f577a1bfad47b5080ecb41760b5056a9bd10a10ca43677de0e06bf36b0d2bf7ca605b9217240871ed1ce43bbd07a7e77d1901dde15e173ca344ee8078ff115e954a2fa660bd4af61ac32256fa7d62af17a4f95adbdcd09b933fa47828d91a6b710a5c67195797bc789647db90107fb527b8ccbc1200bac302dc47a366d170314f5fd7b60535130d76832ab7a06ebcd83d064c1a1c5a7386b0945d39f8ddfc3503415ea22514a0ebd0ad74e3ece5508d83355cba944b12b669d6355a6143921533190eaeda0337a1b89ff9db0cae62a239025bef078137362db71a9eaa6d0a26fcda7e80a9b8fa4a1799819f0489b899cba4048c999fda79e0bb046d7e765d9abb445279efd13d47fde8b731a944af92da8d8965bc4e90d9da8ba0a9c385357e16788cfb663573d9d34810c754e86e3293a868461b241d7872911e676e73136557db1261573962221589850fa5ce049d2356e6630a142427a2cb2efecfed09445edd112323f0071f63f6a42035a9248efe155aae056db25b8004204d530d27027c34eb6937a5a05bd2096eeb1fa266a0325cbc7ac03270b751013b53429aa2d8da2de8c12e7d4996b5f1dd365e270285d2f3a65bdeea9346d960ecf015215a88f465a66417d12d95be2c0bb3d8e15163e0dc66028a30f1f5506558ef534ab28cf6dc2b7d12f49b0e319995efbde71978ef7121df58bbd16be908298b0abb6197bec22c598413283e7893f0fa7398369b83d30b29560ad09a0a24e029d0c22a7cc6fe53d84a7b2104681df35cf4c3ffd90b14cacb2ebbb673ddbbba291feae96498b30d024398676107d3efdce47353a544c9d885cd5cae19588a4b9e29c700c539bbcc119926e3d1adcbf4e8fa3dfb0d5c2686a3fb4ae7b4b278fd94db5efcad7f280488702d9f7675aafd248960806df3e8f95325298968a8e7bf54cd2797decb5bbea7bd3cbe34081c0b65ca9a4d0d08d86ff39f97751471e09f25b486dddfb2975edd51ec77d949862dd4a658ec974b8a5a381ca5b7314b677c4491e07b43a5a9964d95e1a29fc1d87cbf1a64296f4cbd8128ab13dcb60dfa8593a575140f5a9fa6a3708052b3bbed4b129aea88e66b72e583f1fc285ed6f835df0d4751c5d5f004ee0c65203ba9fbd8ff70fc255e497748f24c476d18c94089bcab8ae3f933563c6682e5452da8f760943c7efa631b2980e6589ad4ee83c88fe0b2ed9900571c3df6bac839df1a6d9d281a5ff627d41ada13b867735aa7377386182b1b113f237c14b6d806079d2f7c234a7cb585e12a5b16f7064fccff99386d58f91d9c235eef730660a02ad37dcfb0352b1da76d739ba7bd4982cbb8be48494acab84451bb39bc59521c9873ec9f53cedb527332c47afd919c94d777e603b405e8563248c9ca72f3b2fb52f08a876778d18e7d179f6437b64f79ca01f6be06d7b115b8d3c3320c2dc0ac59c860aca8c92407ca2239998613f44945bbe2c6a0ef0890ceb27467d1ab805ec960e6e58e49678a0841823632f62c58f540f19e82bf719e12bdc63db5363fc0749877539865a05786106b58dcf50bbdd015893b043b21bc1f24ef9eee5f2487d986db13c143d3366260ade522ab824fe82218daafe5e42c50ef740cb9ac570a13cb55fc30c9017228f2c60680f75ca89affdd376ce70ca5223078a47da0aab9bd18333c14aa021e6c088609d7a2460744006f149b648116152e7c814a5730bedeb6ebb59fe395e6b15d379909f197c4703dff34f8915b26a36e30088cb227fc1df108560fce419f462181e468b5d94251fe1373e0dd004fad02d690f8f28bd972e85ec49485cfcdba717db66e6a343044a2453ceb6403fa0795485f8014225d283da4c3aa6fcbbbdd6b04c30c96fce04579578882a44fe7b781285daa11ee71239c1b69791b8fe45868fe6e85341b7ba4292d6da1ae079875b33d7aa2becb02327a17bf2d09f6a42e9681df41890f17de5cf4752880660a35dab912d10480f20f92589d52ef4780ec28920d11de0de06bf63d61829a3d0894f9a32f911ac79041ef51fbcdf3f5b03f6a0b16f0b9e47444b82efa7c012e3bb9324c8649053433306d0640ae953737336cebe9b854186f9dbfda861fd7eedd76fc24634acb9cb925302d0bfd3751b7b6bab36ebe2ea80cb7c5359a505689ed5642118ce93cd783c80a2cf050f57d5acb62853367ac606fc1b8a01abb7b29e49fdb190e2b586e249639889bcf5ad23161f278086f244d0db0f9ff520e6858b68f5bfa0f5ca700096277ddfbcd14a4cf4358c0f042b1eb34bca6353a1383f5d310176262ba45512ea4f172f321efe971c391fa9e9876a250ab653fd6d5a8b046028018ae043da256b78eb26d86e6e89873476ae24dad26ec1d39da44ffdcd93d8c5c1fe0796bd46dbe9b01be0aeae841662c10ea28cbfbef76f0197cee86dd4c7dae3ba46d5c2884ae977c456681d5d9859d854a3e0708bc35fd8415c22ef4cb889bf0e66da72ebc635b71f5df04db801c76564d2fbd946ac17e27fbf995994ad26c3d577ac86b3b19f855d87bf3f0f7c3fd574b89901732dfacf2e13960975176f7418893b31747d96bd981e9b1463590bd35b51b7a2eacf1a7733f76a7d4b02fa2afc5925f5483f82cb529682f785c21cb951d5678a7d09f99c0e2bd98948b79456e4e0b5d9de80943f1df0c639ab8acd6c99a6297cc5a53a829561a6615c6c44b24a30a75174aac207a35fc429e1a415b0813857ed85ab49db6e523cdf9b122c2442a4fc2b4fc21685c9c4a5de8f36c0b405e34ddee91832cc6567d458ab76b4650020d270a65a562e75e06a3d8f1313b4d1d4d29d597bae9eee653bf8e31b82f9cad2d0848ed5daff73d99472c7e71d2f1be60bca4ee9d315196c096cd8260192597c8ce3cefa9cb6f129dcfbcbd65b5b0e19df5a1e81bdd423a2e26d11ca4f7aee99104530c1c8e3b1150669f16acf415559a38cb27eb6d637c78b1c90ddb5eef14ab9b2ff14ddababd7a349b7f6f5b6caa3a28c45835c9beaf8f836848f579598ae80ff9af739cdd800c5a3d7675c780ad66b2c449824d8ccf55a7669fb720604eeab427a0308122c3039ef9be6df46e34b719aca7a001b884e3f7b1d1579176aa4d365a25b6f3d420190e3cd9babefd8382b6bb12b4c38b981b08b5a139cd46e9382c01f361abdac7fb16b87e2c61f3241191028c770c3e3be939c8825c63bc4e523b389d1ee28189b5a2c11ccbe0aa7d3c2b47443028c89f3d4bc1bcac39eb61d91e75e0d372b9f27c8dc622a86bc0cc0a9c9dafa5e2bc67834a75805f3bf1571fbc5528e4c69f3e64ed27c1f2d93f43215ea0d4213bf2809391b8b9201f36ee2a275dc7c44d0d83de3b82490b65d2b399a7348133c4df7ef40e6dcc114d040f65de00a70cc0c9fe615c6fa90a2fad6dbcb39370af043345ebd9d8af094c831f341fa5f7658c2c1ea062948fef5015696eddd181d87dea49ac8bbcc2c7a1ea01ba1d04239153ca401bdc27aa87093a78b2020109d7cbdd39f279a7d67b14a5822fae28854323a2620033a5fcf9817a0d45eca7e2d120a113fd5ccd9fc0011e89398767f7f6b7d01d5a670b4fcb9c5648991817a62edf43d6964de204541a8b5573d7cc10bacaa1bb835f87d12985ad5fc6ac4834ce76c76e5dc34cc322c4220d9d65ca77051e7c1d4edcfb4595db16c641fa445401c72a24e36567631ea02e4875b7f9ac2ddb4712b79d1468c7ed2384ad5d90ea0825f817e42413de33eb30861ac92e19b1aada118beaddb179399008137f28a8f3df1509ab12d6f6ce7872521f5cfe4738722fdb410b3824f139ccf8ea63817d154b1e497a70e6f55b75e4decb73d10ddb0c57d7e0f74b3f4fb41ee57c2da1e4399c2577f28b7e911db08e877c366f231114cc9553c7f3cda2b75895fb60ce9eef4e0c25d253f689dff5e4f9cdc744c301b6f13e9fa9e30bdba8abc203c05879ff01bf8c3a3aa742ea5fffabbd6d165e50bc412bf5e18c60bbdd4fdb276f3bbfef246cfe36612ad61fcbd4498c18321a1ee12e6dee902c737999f9c6b3919f52125cc77e237a788640f262ab76d5c9db9ceb2d988c261dca63455658293b10842ee0b30cc1f928215e0308cf258ee39ba666b16319d0e21df8b9ed91be2e2feb9e117682cdc9edbc20b0b3b9ddaa93ec7a2d8f752ebaae57b7d74037c5cd64cb04746a38c0ea3ba3241319360e585c257fb593c123a653e6e0420fc540bcc0f072de6ae1a31b57dd5acf0b8607e04d21b727c93623e5227202a7c6432b308f74266dd1e1447a2d637a5259fdcd1d8729f341a7dc383b4c8a6ae0e78c34a55793043d2dddbbca1d161c300a8a7cb760298541c90a1c5a890b5257369b8fea0b4d14dc0848e644953cfd5196494e4c1b422f8e188314f2fa903fe93393193e9b76bb71c0caddba43d8586d72657a2d717e4b2c3a0e8bc33a6f8d89b871244eb04e4ecee26f9b71ae7d5c44f828083b0c1920d37cb5d7db86be3b18ef16653863762939cd1ab5c5e0cd1c495ebd0a3037725470d526bf12d21c2fbf4946ec224619c79906e8c87f0699be8a3621ae718aa232e26e43bf7cace7918ce34577a5e27261a4a6bdbbb8ac21a8a40b0c9682a0e0a37b7fc43306820b0db5fb5bbeed529102187fc3e5019185588d1ca6222a5e2b60faa2a93857bd7224e1e1c3c86128fd16993aaacaa34d8679e194ac0da6bf617144a2b1e844e7f6f8884df05ce1624ef1fe5d0895265bef2de61d637a74f78f6e3402923c9363656b7b91131dfc8f8f7360d14e01a25869f0f15a9355c07bba82e4969f5643519611d99ee9566138e4610712d55f22ad36c7acdc9e32b4e1e9b1ee9bd14ca122db05938840b8f47a5fd25a26072e2055494eb4ba916473e973c1efcef8dd7f0f04711b7e968d59d144fc04b9c6d8f951ab973441a6086ab6ab896f4d282793fb14c74322c7ff1c4c1c6d0544b7457635a6fe2632644344b1fabe12633b2f7d8fcb6e0929a05aeb2749fbc99671283c47c0eabe152fb6215bbc6e03d61b1ae955838a8b7b3bfa883a04f5156982ba433dcb21311fe9027a5b2e133cce16ac15786d2d44b5e4979cf2a8667d2976ea386eef4c85bcebc39b9d2dd216e8b4f7c68b7e0a5d7ff570770a20cc605e66470d995ef96f1e0acf6136716cd8a26b3b725db3ff48364f19cb2c640600d00d0d728d8fdac7b8a205ba8109318eb5519ee03848e79a1c33898555b298aa62d6da66188506638918b46926a99ebfccb649cd6bea1eb87fa6c1999a70937f51c4a2d485c6aa2bb5f98b1c8ae558e82287b9f34ad70954f8dbf3a9736604c43bc3062f8707fe88483b0865bd6df65f7e085593d5046554ee465a10ddb0c0b651e2b1031d56f5130e100e4544809a093026b88277e4e1f1e3aa40997871ced6e6eca44f183ee09ca679e4fb0796af62cec931c553ea788e75fd37ced477f9a58e54410c513fa79a4b5552b049fffd7d261d90a16f253a0782ec75219d1f7492ac2a242fc04c0216e6c804f8fd8f761a70eb747e1545313706136b085fc72dff1714ced89ba14a65a34168dc0e997ea8718ff304a962cbe8c7df8b759d76eb0f18155e49822d21d91dc8913c323a824da271f8c57d44a27be3f04991e2a47e5c8da2f0d71610be89dd4459b9a9f5cd4bb5ebf730a72617613667984d9a0e0ccf81c803e46425899b2fffe9158f9e3d7c36c2c42938d9433f764ed1f6d9c7d69e9ebf854db44c398c00b2612cdf19927de8bf45df70a5d4307e3a628a88a09a79d4e03742f89363d891b241383887b4447d0cf3e0eebe4a9143cd6873abfe2993b1f4d1b3b3a18aae4db4465385500ebef246adf17f1d9196f78749dba2b12deecc5c957e022f5b9dfcc3536bfd306a1a70e50f7bab1db22e98c8705328fedf3686938eccea04e7db4abfbdd939d56c0c357612ee3536356c0f5bbf1d428fb31c7b06a75c5b4fd5b020e6d9894b50eaf1df723131712344888adb5a0b4c07d23f6e853c88b51fcd56beefa721557256bd9978caca155db7f9c325f9564479c39b589431ee7d5da402e79ee6e04061c62c1f369176f6b35174b65a978ae3a4e8a25032d987dbaa5ca048734cc1da5f875f72db399df6e6a300db183e96ade56ecae019c4559c8f7c94899178c98e5071aece48a3e9f5a795159b2dfd0a1aa784b263ae5d352a890b7c2c6887d76f0364e2028d4f9046408e76efd6b5092590d39ec343facb16ccfc0000bfe7bebca2575b1f02c8a0aaaf5fb0bb13ac3246e245ace99f2186160a12792fabc0b390cb4c32f87c46f741c8fc940c4aa7843d650f90d422504bf97dd4825e533f646c6b77c4f004b7036b5480b587a913f05bb8ee470f0ab148727835b1659d8c046fd24276c95d30158424914e75d9b01fe52b34d5f76b2a1c98bc5236945fb46724e02851d46aca237fbf047f3055d6331dccab91c0bc20be2260086552fa7c958840843a3c38acfb24f9dcb506ee3cfc67f3125954d9090c1fcd629d1c3fc557b7d6ab33b9369ebc8b5b43f9e23673f688ffe50e6c3dfb12ac58aac60590991c9ce68502873fbdea04455459f6ee721570a87f87ae03a9f97f6e457ce1447358d8e3453632618701d62885ec4f8dd201e26405499dd7f317a8d2456940348827a08950d548b5c0b24cf494eb76dd0a51e3f5c6fda9f8f6545e3a6d1009775d78720f9b20ddded15c162f226873162a5076bc0cfc151e655e29d100b7a0d32c9c783375d1c6af0f39b51d927442386cc2c26337cbe5f4a34bfbb65b544b50b4947f72d839e6255a95fc859d6f1e34bb63dbfae1469be613472f22cbc703468d8552b9c691c829bdaf0d2e7e4093e940398f9ab8837baad8d9e4c710e4584685df01ea15ed9ed95ba9f6420e5766363e86ab8dfd1626eb1c7d2dba91d360821da99977be4642488972ec61fce7d754dd45a4d7fd9fc3498a6b7e0b7235c7843dfe8980bbd3a52c8868c1832f62d9e25a40986d061d7ca9d8fe29c7f234e3448fa315d749a09cf29556060307b970e5e080e90dcdbec4ecab1a7868cc5bad1037c07e17f961f3cb5641405f3d3c1439596a243f83b4f8576019712320ccc40581bf02a0788e3ad9cf4c44113d47f587e96061c61c06be84f6e35711d105f0c46b31f036356f6abd329c4e1e74f7d97ae6e4e0879be998a85257b6de6277d9c18980208e83d9bf266c9909a26f3e19941e728fe4ab0866d103644fd86b203029271361ea97fa93079ff227861aceb55e68d7454d1dc25aa8a7c780590a04756c840d13dfbde1d52d7df47d4146ed8d530907fcb13f6071e5c18acfd0a694938bca6cb081b7dbef2a646aaf55d4da9f3d42996a90655cb48144a4dd2789d75692530dc02a2635c26d428f2c6b66779bd9facebcd73274ebf333b8de26609bedad5739e73ed68798cf9882ebcb12902bed483bd905b92b3df495e708a0914a96840761da66379d85105f29e14e47a3e5273cc23ef5053077ecc513363379d2e6cb3aed80098589032cab7c20eeaaeb7bdae7e3a1c7a704cda4568138ce1ed94e944eb4ea72120ec91117338d0c86116db36f3807a70bcd98de03cc46ae9dd0c5fd3b26d78fed5a8b53c0c8f9566f921175fd193ec521c9914be68f2864715a572cd6b25948fb8dfbd75f4315967d5a6a707f1ceb81b98972e615d3e81b08ad5ef1838af673af9b6d63dccc226eda6d0b6fc1fd104f62b80662ff157999d7508f625fa2408b6f9ab4deb410ebef73750dca3e63d5045300d9b52845afc42f064bd78ce574ba9b4fce22aa3158bd058772ac2eadbf9fc77bfdf85dc745b529e8baf248a0bc1963f8fad78471116b9ac11679a326e92d87e03e316c90d15d2f9e3921e0bb6ab4b7ce91f12e7cdcd9dd141b8ef160e5db1faa47525a77a038558abd98757ac5f6313afb4f4c84b2c768e456f6de4a78b5b25f90926352fbfe83037f81b31251249fe52860f3cfa7ae1b20c73acb015821840dde2ff2de0470450b11aed7c1f84d4020147f8b9aeed4a37909bbb63508e533f939f23c2f5952ba6734745c51b837b72a398a4eb8dd6b4ee7648ecb55799cee7b6385ecb0b2e36b9518439ba550b257230885b83f72312e53afce7e7670b26342e4eb0a6286c0693c54abceaed1811418a9a289b4b09b65e84b3149cada8114e830c0403884ffb74c515bac074e4a981c61447ac99368dbe9cab8830911ba9cbb64633f444716f401aa17f599174581538918e88cd255765142a49412841deb528f44a64acb372498bd66f979d9189fa0f0e70e305015dcd3c9aea085cb0f1d79a63e92eb39b2dd2fb1b9ee9aec0213f3a4a848f3450e4a6d902f5435e234588ce4a7d2c9dbf60de27227ff02ee4ca19314d72674fb06c00ac7e5d15cd05fb35e973f0974a4718ea9b0d35bb4064a6280b654a15a7188c5d5a5998ba1f031a3baecdf7b608169c0ede84d91c9a5f73b0176e5ab2958169ccc6022e0c43481bb8e6217ddb1bd9ba8e8446aebed6afd163dbb6873d81afcb74b034085afec0d626b02fb10839bdf2279acec3d8f3f7c47fcc3ddc8cf1f08de8c8e120d9c06648acbd1ba6824ff74419776b73b05d797a0bc43d77eb75029112380ff4e87e543863ae29065e40c1b2b1b2d62a409a299ee5868036835627dacf4f9f7c84fd2f6707d823eebca0e8ceb2bd258038b137aaf43e0e937be762be5d3a6545e439a45090d397c8d4a068286836780536ad99cb138cbdc6fe5e9db3bd405ae6eb86084be3a49c8af88682dd52a5fa8f7fdad76cc0b452b3c37b27dc3cf66c57a4ae59a3687950ad3f1fdab3002fb2281365348cfcd9306583eb79a9555e8750dc95260a1b277c566babda92cf2b3ef177854fcc3dbac88da1536eb3c6d6fa3e336ed3a7a6da006901c845619fa00596af8e54bc1980aec4aa97bc98266c248bc038013020fdd128db0b9425e8f7e3165f5b9190fbf2dae5e317485d9d103695cd209d5d247ac7e1122bee4538b5b74790277ecfd2051e43a34b7c5440d1198a1875a0501eb3f519ac2f17553c5e99a9c83412200a519251d4d4ed1c84baecf7d8cfbfe4a7e50d789cc034a21043fc13fc3150314bd7a7801bf9c31aa00b7cbd9d0fa4ef0636504f0dab63316d9b9a1f4703b3be23170549fa3a29ec02f418d687589f77939fe068e41abce460fe118b04fd418f28edba4fe70d8fd24b38078a5c47ea236d73610ea116666a49acec52fbd810bb99af1dc5bfc4eaa634dbf2178151acc453f93567f015f2a25002eebc1f778da195b7e9cf683db1a85da98a99d98614f2c25b2237d4393f5799d06cd36254d3f22a53df5133cb1f43836991d3f7529cd260df88bc47e4172ab045b66e081940dff797b28c632898140425c3f81a60236b82522c0da7f0e0ec7a8ccdc73cee175d3b4543293135c31c0e0c926c7506920cea1d0a1eed64ede7572b8c77ade1d95bce10dfa69fab7fb5ba64055800593465dd4c26717e37a66089f550202a0530157e5274f3d3fdae187444f5ed641c608643eee1b361d78c9ed8070a85150c80368a84b14c9dba5b20a9059f1a05a60de938ab914a093b2e741e891c02f638c73c730e60ef8cb5d9e5031fdaf6918750f192ea7a76e035eb05d7556ad64a77109c5e13bb3ac7627d99165567dfbcfb7fdc7bd847ee7dedd7864353f8fdb7a9939867d9a5ff4d9ec9727fd432d8abb43628519cdb6fa49ce49064a421a89bc924cf6286fdec8bea6b5a7b8eb606f584d44c7194d5424f98576dc4b5ce7dccf65509727ed52f8d7210095252403023473af96eb5892cd3881f2e1f4dedd4c9b0755b70776aedead14e2606bbc1ea3c300984ebf27ca69f36beb03c0fe3c13d9df79791d828d52260103be72b832ba235bf1165250b608e2cbda0c9c8d3c96534802534080b2a4ddd5be4f269f37c4cde9d43dbb42fef8d507cce8b8e2eed799c72a2fb836f4160b5e5729192f088ab8557d51b58940065ceb4b48fec1873cc4b178166d15d441f45625f45af62c1d13863feaa8afc320bb36af0d151165c122a47d4306d075b344974835363818009d86c81e90345cf4b7d81786c40318a371fbedcfc325cf0e853fdfe6e91e99c07dc9430c04e6069b9e2efeed0eccfb161ca85d96298ae6dd67c395b78668d5818f567f2039111a7a0c52a21d84ef72dedbabe83f811184a95976b1a90dba5aca60d0313c546ea97d7b2e40b34ae88ccf4f45e28f05f16732d7a0d62dfd86e40a51de7f2b9b66ed780476cebeca547c8eb71ac20a245b1bf7445e64bce260b8871ead95f708974c3a93dcc4ee979975daa399055108854c1c3fe5a07046d13169c084b2e4e81821ff73d5d20e7556f40fa152226a6da48d5f86cfcf1a5520451c3c1e534db654b1a58adea167ffeea90023cc2b0b01710323177059e1c610154cb234d5ea165dcaebdc2fd9d9b66413b8a6848ea11a673f207ad76bd3ab5399dc0ed58a3ac0ac9d3c53781c36415d54b452a4a78e1eaa7046d062e3105541fec8819439da85ce53aa683aaed543704aa8210b26c1e6c8f1d1814595e3bce2d98a36089bd722110bcefd651937892ffd36dbc34f1284eb32c49a1c134ad460c19e01b3f3f2d40f664d90020ae8c546bba9ccf5500171ad07a55f9fd299d7518f00c0017f6c25d4aa01615fc7eec160ac2cc1ecd38bb0324e54cb466c3df4d56508f5239e1b392a50a8c20946a6db9220088d77c494406296921e3ac9997c34f15fc5c39f2529eb2bf6b41b0415e90cc9d7e8a83256cc59b4607be49eb76a7dc07c1ce3d0512888705c9ea145079085273f2236d5bb151dbe7d1bda2ed607fe51847ba09ea6431dcae727ec1ef0a201b8df9e148885e1308425d5c7fde473e90b4382de8bf5e215c3be357f8a2356348bab00a2b629dee017ecbd76d7ebd45fe4df2884e34ecbb86ea371fca29496fd05327c3b985a069a24a32380ac9e4b131ff48d58b1c528b941dc6290cbb41ab4aad2a55e5e03a63813c2113c0740029529eca008d906dcda1094571f5d2cd31ed79e9bca8e815e206b3ff3a7b4b82b0d30775e0b77260a42b1ccd0a58a91f0378df95cfc7f8dcabcabd2d7ae3b603e9061c1e126d00fde94b4b625e953b6b781da167c71a6a7ce0792e7de070863deaff94d74a7ab9dd2e3d39519a6c5c864036f7bcd4663fecd163d78a72a74d2838b6949eb7ba85dacb2eecbff1af2792047b036f77d488e5256ea16252257bdbdc1ca3d18d01c5de00ae8ce7d1c4cf103b2bd7a31dd52fa92d945da38eb9070d6f11aa9cb8bfaebe133cdd7a72f94f0fa067da0481dcc86e6eabe115cdfd8fe2e2c63c3cb3c0f75a4655eacfd337ddc82f770442d18ef9cf3993a9d1830217f50917153de7576917c533247a6725a0cb7447ebb0cf7516e19781a8ea8d1854349db9d3bc4e4931a1fa4fc075186dfc37ac10db1d3feb6298c1fabc8d897e1c806e38981c3fe7f252371b0e92efc61b2e787279f4a8138e7c6527ffc4092de912378ae692e9941d0b6d14354dd25900912f9a907dd970db649dd965c959e60f4d174ab0aeb7bc016735e8b847e231bdd240042aca16731926f537fe388b6496967b750a94cf514c9296805607b5f44451bc622486ff62e7af5c0d243503ff31d1c753d9fa3adecc8f555445298aa10a877250455c43d412719aade5a202076fdf080589a28cbeace5a63de77d188c1fb486d888528c5142f92e54f005239285cc4f8695c05ed32fddf0b0072bbf7642a1daff7246055cf35e077acb567b78c73094e583b9bac3ccea986305eb4322bcb923b93187bb3061fc00ee0b3d10702d7890c4eaf4e5f3da2143282dea0b12c3f043818914ae3ea98cb1322d73878b5d9920e87e6e00ae831a3b76484e0f69d35eb15b053c545464811f9c381337a80a603ca91ddebab47b72d6c80db4cf0646c579a136326be69bae46cb6ecbe23ae355cf7181981d8f7b8e16b3303aade349f44c005602c34878690d91fb7b4bbbaa75a91528fab18c3c8c1380c0c26a98dae3a2e441527e8c7e7df331a506eeaa2d299757e8b0cbba97a91f64e36e5a24cc83b00fa14b8a41e6fcbafab7eaa905c04c30755c551b2f0f1a24fb422bbd76090e212efa160103344200e83545a119cd7d2f53e8e5e6d89b1c75f12a44ddce7b9e9bb4710e30c853ac8d024be6f5b3d162941e2767c61a7f7ac5265d08a680b87f07bdb4d4e2037b99a6eeabef33b7ca6e7e6f4428f6814e294b7c08783525e1f632fc2a43dc057264bed23898edcc35493222a92dbe28da74c07c49fd725670ded86494fbce9476d3a594f4d96ccc3302a9e861933dcf46d03b7648bbe7bca88d9791b5a62694b34a9ee0d6c9d5600f0758e7e0a2721cadd048bf59d84bcd7ab33a5ffa8c1a237dbdf3481f03e6368b0c1a86ddf01b962520372bbb8e330a70cc684e84e1a438fc571ad9bfd297288135934aa4f1310849bc9c1658006eb1639d4b8ad2f858fdec1d5a4aa89259e591e2cc2948ff126caffa24bf41bf71b408da6b5107e1d65fcf37697d307ef3a3ce742e5d117e7bd4d095dfe8598213e0925bc9c8da414b09fd21451ad8f3531a756f15269614ea1ed728d3d37820c6e90578aa43f8f4cddeb9ce97b345840899a016591690a810c5773572f34b68d1f2f1538dd7c541508c990621d54dc2653bb4a7e9a1c637515538840f836d38333de92ff70997478fa7e4eb0b447ef04a96b2376d3e7319b49f8eb0b9c656bbaeb8a000104f3407e33c23030ee9d5d92a2aa7ff76f419ea3b1c9a8953b30a92759a8d0fd37b50e66c8f664792da1448f8d12d60e169382da435c038f10be62dd1c0430f18eb30af8c14512d3311a28d4252fd4b4dcc04b5da190a22c6eda7ad837e470b9d8ca3254f89a3e613ddc74c1865b5386efc7a8811e95163fc4c2a61b738bab3ce75e9450dfc38a52a59b89746cbb216cafa6faa0236db5b86747e7d9d3b6515bb19aee0a85bd4c2bfd76c1916e04db2e0d0382efd97d2ce79eb9370087b19faf4f269e7fdf64f6c30eabdef01c5a4f347dea6e28c48a55643ca36881ac5b283a9f85a56b203c3076e592779f91d3a00344a29eefcda31e2c66e38c6386367a5b43dfcb3978c891feda33c7907d7e4a501376e1786b9cc2ae1e0a60b9c0c979a0c1c96b9c126f647c3bb21abca0d4498c08a4544fc46ad1a33b17b105db91390689ecdbe0b9e6b5cc1d65a52a51d7d3a714c850c14c9329410c9efdbbd55cbc43e8568b01957d664de2a82c04caaf9e55ae5a34c533bc4da5c2a715409a900c7dbda950463131367613ddb0ae44c73625fc8f293e4002d5b704645c08098ead5a0fde1980d8addf0af0efa1288d569fa62d6b3f6227d1b4d628d1142f5eab4f7f1302bc97809b1b022e5976b32da3537a192938b8716307aac1972cc5b97019f4b7a9e5e650d0569ef4e7d12f8b123b1ff1d95d3d713a18dce009364a07d01b3b62a1f3ffe29130bee4f3b275dd27fd6860423629b181194a10ac44077eb2323e931cfdf64504a94a1e7683110ee8a21e67396d09b85237a51d824966527f9e3681ba9fa6db51e571d7d6a1d04d0f9bbd9d1036316a67ce225705018d1f0a9b9817b0d01dc092beb4e1290a7f12b22cc32697a75e17b9a93ff817a721ed0c696643353b80478a8d41e92b0dde35d7f2918f310e829305711683fc8c2119f0759e4bdca673ec93f4f0837aaab4c925925504fab72f35bc494a4d8bca2b2ee392af02422b2f80f174519fc83a06f99290cc7f1812a32511dc9fce9499947c92cc12c241923e62e98824b85ff3ea4979bd4362f87b7a89204598f51629ba1bc6d42354cb13c093d00c7da85b2523fee61b85573aa7d4a3145d518689ffcae21ff6470a8210cc6302bcce2459244e6f9173e1d9447726942749fb6bb13bd0d3479ac5f1e6d70e104990f2c9311046798f906aa215c002ae0b918af3db1bc591a937d5f485489a132fe9098bb77387906ae54d92d368ded939a069d0866279e0f7cd815062dc25c664c168ae4a11b9f1fedcaff8e3c37b6f7b075e737354074381a135291383a35047b1dcc7cf3b815c1b7ac187a4dcbf64fa5ef83c4fa2c0b4f5b7e024af32ab5d72e4e86a92acf858ff0aaf0fb8fc64fa12d0733c6fc530ad739df51038fceaa88e6e47fd80a357f501dc4ba5de4b6c0c573534a84e5aac913bdfd30e548c8009f06e7281d4b91b968852033af7b9e79b644b1532b9324b9788ab6ed930236c5bb2de248231f3046f490131ac34586b43a3c8e0a73cc756a816c79f3f10f12c23501947b29617fdc2a45c7f361d27966f73b1b3a1f50cb0052dcee33ce739cf2da8fbf9b7e5734c5a7fb8eea9f5a598ec6c207735409ca3401894f634fc787fbae311d4bdc8c99e8986f306883d7185a18e71b5166cb9b84cdaa3cb03231ac3053beb31ff3d87838e53ef698f9921a0ebe69c9b1526ba5940bdf21fd745f6f2653adc2d3c36340a6dd7c9ada045dfee2349e26a62b437e0c8f4fae2c7eb9eb577c20886d8a8fdbd02d245c1a1478f52e339e8ddd71fbc6d2cd1668a44e2c82b311720996b8da7111658a04f4b756d84418ef3e99713cb73d87bb4a14f69d5ef4a8b8cd6d0008164fa0a802c0679b26e155d801b40a2c77453dbdaa5a582d7c9d967e8a5706358c68c6da6c3e6746f0648401f896f1674797b0e14b354c27ea94fb12ad6d8494ecb0eca0eb5795b1920c618b429b4e95979174fd21685ead022e7ca4f87958c8a101b8b3a5e0a9d64393395b12dfdbe7e7503b62db0fa8813c4eaf1ffe5f190de55e83fc3a01b10a647d77cc49b07a258cc4e65d489451f849f6aeea5bde2aedd8e184cb9c778a3676dcf87d2bcbc19647bff28953c5d06fa73b860a16785238f9765c0fa958757b4791272cd8d1fae302b8dc47282b782d5c4c5ae75a9c02a3e911a79547df96d2c2306a2e347e83a8d8a126e6a3a21cc46e476a706af9c42a97b3f8fbcc0de5cdfb59bd6cecba11bbb93ec1bac6422b61277eccecb8b28b3d97f0051b89997c8c22a9f517d9fee5e16606614fda11940cf388c04daa1b302d3b0590a9bd0692d6139b77546d71454054197adf346bb94b4f2e85e502e3cd919bb43e8b424339e2af0e7070b035271783b1d7d48ecb6e63f3597d08a9d0677a44c29948d9cf9570e7fab1ab0d8262bb963d62e895c9bc8ea999f6f8770ed5ebf75a39dc0fe2eff88d75ae19690e0f0d52090f18bbfbe39697042cfb5293692f1e66d76aad38bd919b634ee20000000000000000000000000000000000000000000000000000000000000041187a86f715ea1475c76d354d556242c15f6b8f7c441cd862cc7c89d89a58bca02ac74c422eb01833db93e4b067e507da65080f93cf1815f4cf1dc0d53a2802230100000000000000000000000000000000000000000000000000000000000000","nonce":"0x190ed","to":"0x91a5d806ba73d0aa4bfa9b318126dde60582e92a","transactionIndex":"0x3a","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x1f02737dec59ce5f29b7e11309a33cea7b370eddefa1a8007ed3fadfd1c6b23b","s":"0x106ff6938ec0a46da92e74768b54926633945ab503e2f2dcb69469b544907586","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x2d679b567db6187c0c8323fa982cfb88b74dbcc7","gas":"0x5c90c","gasPrice":"0x7d9f6fbd8","maxFeePerGas":"0xf49895cde","maxPriorityFeePerGas":"0xcfd6ae0","hash":"0x45499074aa521ac4151138f0aad969bcc2dfc1648d22ff8c42e51c74cb77414d","input":"0x003851f8368551cdf133121c7cf32ee5690000000057b378dadc7d07581359f7f74c422f52a4d750946601a4898820282002a14608e24ce8d2412c889246951210c5eea0206041c4025828362c2828a8a860418a28208a224185ef99505d5d85ddfdbfefbb9ffb3cd964ee997b4ff99d73cf2ddc5b063a034cd01e114eddde75618857bf21dae562932ea35bbe70493443b0d3bda8e6161bdf9a2e6e1a1d63845cef77f151ab3e17f6819ec2bc7388893dbbbf49e9f0c76d8f4e3a6d5cb9c46eff41ba97c0b3794c105b1a891908a7812b16b0d1d7780802e8c7621a188764369f6bba1bb0c95c54c4e3ecad237716b875175c0e27d3cb133dee5692910d9eb1ceb4679f65e7247f51ad58b4609f4c41fb91c48acf7b7d2a8532654b741723ce4f70eb38a521e9948c62bb5d73328e2aa5e7174426f5da2a4b1c6ef04eb66b2d4d1a48f0a20cd3d8b6bff8794b3975124f8d8149ff9c0b714da5e23b8ff418940c5dadad787c4298fc8486c94c44d45593bb8dcf6cbea18578904b0c99dd61f2b7653a1562b2cfdc3dc96fb15db01fd11710145c7d6b0957b60eb6fe886a047920fc64a496565e715ce7ac34a51a8fb632f62680c9de88bce189d008e29a23679115012cd789ae2fdf6cf5f4dac0cc4aa2e092b846d3d3b36974cc62244625394645ce73c1eaeec885e6dedb131a1d420a57f945698a9e7c57befcea025554af9a4cf6a432908061828ea86605e8ae7ef301f4c388060625fc5cded2d60f4bcdd4270b2cd05a133fe45e31ff72c0fb8465065d16678ff60a4f2ebf901d2403345bd6c4459c6c6b97bc038b3cbf9f0ffcbd7fe06fca91bff93e504946e2051d05e23e28dcf2d35dd2d1c2b16da8fffcf65ec0b789efb1afec71cf481e5b4476b7df95109b64e2c1950beb3cf3ec9a671ecbebbca59aade6ca79f06328bb06669266f9fe9a6673c50f5aced1aabd49947341ec338b82d9ae31d9fff59a0511ed938e016432f9c5a001a7d5d34b54a7d9a2419456e3bd0eaa860ff52e69806188aec7e559a70415ccb6653c885fa17efcfccdb680026e43c9ad0bd5b379d4da2127d429e74dcb29f3e45b3ead08e1e85c2be6505fac13244c6e528ea261329310d9b80b16bb7b562d9477dd95f6415de5bddec30bbcfab9cb9bbf88e71eb47c16df81087fa93c4b39b5e0e1cbad17cb1bbae73eca0e2c75f73e12e89371e79a4d42babb71e9da013fdf5ff1c20a45ac00a02bf1ee09b0efb1254f4862526e5abc8cafc311bb27abd9af1464ed617acf4c1647dc8765d530e4b40fed95768dcfb24a0782f32fa6cb29e165a5745f2b44a8875a96812b99a095edaf5afb1144335625ebac2a489d9133676bffed7d161671de594b27973fdebe55eef99d65ca83e607ce6ed86d32efc36bcb4bffeb2062a99371c93d2eea8a0f47519e0b40ef21ee8d9de5e8d8da61657072561ab1cf9a4b1d51da9926a2d934cb9be7a548af7a5835758b5164f249ef0f0b53051779089dab29284dc10c24a1ee298ff1faf3aee4072cfd4ca19520b23941a4852bf56dfdf0eb075ff435f87aa51ebedab52cc3ebcb4db9e133e57c2aeb10a836d1f5484807f73a9d411e059283bb45fc028cc8f919abb8ad2b008ff7a95ca576c0c04aa4f243f10d2fcbc425ce014f5fa6793335650b99beb722faab1bae677d9939a02d3de5e0ae5d5906ba024cd019c11f0d51c89e73b8cb3609f7faf3d127f969f3beddda1a756fef93ba1a4d33e4c3992957a9c304394b7760063250a5519ae83b304c00fdf0a21e2acdec12482b724df3ac5ea472c065a9e2b7c8de28454d72ced29d817ebf3126b6928c2cd7cabbf0cee395489744896faea55699fc7ec581ad49b782e1bb83bcf535468b91703bcda0c48fef1e919dbbf30aa45b162c92f0e01496643bab3d9020b8e471695769f04020557e8d0fd5812bf346deaabcc118c92dfa146a1b5fa6aaf5db5e6fd704d073d9c6d549004066b97db487f857716ddbe6cc6332f7ddba3ce2df47f0f25911e833adf8f345677a96c7223a3efc77a3f7a5d24e543525591d1fe4afde76a4a3fa883efb81fb9e324c5727a5d2e7c04013a249366aa8f8242373f9c0aa55a7366d364b0bb89ca1da127046eea1de1d39b917fa53d6acde40da381a71d342e3fdbb19254d7df86704eb979ebcb905762ad78a852bc948b97aba45e99ea4fd7e0340ba70097b4884abc81c20e99276eadce78b6b5f075620424b9a83af6457cf29d9fa52b6df6bb7054711e789cbfdfd1bccf3081ed96225866598328089398b84ac237a46aeb513780c62f5ba256d9de6392fb333debe7b2ebf00571be5b6dfb5298b68c0c4f8a281db7b5a819b91297d29e586e6e78dd685f9810e830dad91e79459de4f2cc484ed9cb3a49eebe9b51975dbedfa1699e3c2fb9cd6a7e9ecde33677fd57e07847bdfb9bef75f07bb9f55f73c695a97ac188df3280a5e92f0a4f1c4adaf8f53bbab4682a9dff482e9cfb43db9dcfcda1a466cbfae58cb09a97d8336c88b124bf1cfff8a603a78299b6e6a9ffec57be9c75d73fa3a86049c56f127d5f3a6645bec138e6f9ee98510eef2dfcaba7f894ce575889ed3c89b3c1caaa37f0901d49f7ef9d46c955095899a76feb44cab15d918d6f9e924e77ef594b9af6e7a2bcbd2190758bcdc73a2cb5f7e25e9afd87f893b653e7d81b294fd6e0bfca1f5565a21f725312a2108c950f275a4c0a698ebabc567bc1d0e7b1d74b027f1dd7187cb584fccf0206d997519a61060628e2255ba437bdffb3cba91b64c87899fb19b5790ebbe92b0d55d2b0eeceed76d6e91aa53c6ae2113e33e299d531c4fe70c696050e654c13387f0f5a9006691089779741473c7c7dacc4503fe93cbf7565bbbc4f715dade96b451a21f1e36920bdd10f42f48946f8ac56ab3afafd84800b2a9fb6d643cb17cde43a619d7ceaa915e6f5d209ce489a8b47bab1eeb9a93792c6dc3b7676dbacfbf3e3cf43e2d68c79522b62530e423f3ac341333908eea556442afd4a7035384916fe5ea257e3af6655cd75ed06f3498afbed542b0ad0491373609b66dc14b70fb96868a8962a2d7ee4d16ceebdbf334a8758b9721bc879a8ff8bf9f2fbcc6f63a9718a35d8b5df1d6a38bf1c6123b43566a079d88b94f3870ea1b9ad8d94d2db14bf12db3ba58dee361949713c756937de91dffcef9fabe2f9714589d51e3a72c7c896ce9b1fde893f5ea895975986ed8766eb7c7598bf7371fdab658e7bad069a9b52f9e9761ae014c4c15c25b1f98e3e0dc1d831cbaddb6f4d3a993f2257d61f345b3cec990f79d3edeb2f5ea94016bc4c4448c8f9815e81b7433017a719d074003d7307e3d627653f5cae3753642527b0a49b76bae643e4f0cb4fe00bc7b71202d6b1ed7329f6ec4ae64ae463ba7cae1375d3ae4b36741ed88e3dbfc48efbabebde8c1eedd487fb8a9f42b66808936bb6692353b37642efd113c62f14a711f42aa00a0947ef974dbf1c9455e3c1ef175af9cbd8bb729f5a7069685775d8ba8fc3552e37e8765680a58eed27db682227eb6f9f12e9378bed42d4292a77ca86c2eb79f2d289b557e5be66535a2f7bef852cfa2dea5d97c52fee7124916123d91d6d6911564a5c2be40110b99de71e9d9a6203de69a6f7b57e54fa4af76b82daad4021a3fdf1ed0cb5f2118646dbd38f2ff5c7a107967eff06986db9e2bb8996f3dd20b7905cb8f091bfa1f9ab52307226009181925c4cf729b64c39add884cc606b5cd0ffd92167e5871ca3b85b0eda19eecd1f3876fb98ea4b0ba1df9cb8b5e5a87dc74399170e34ec1a2b29df609aa1f86f24fac697de3bdd5eec694516c5c59c6160a30d90211319e50ed9384a8c8e316331b2a64f65a2f2f72b3793163870b4fbec75e81af7d4353aed284c926575a821938c34a619bc75358126deef5ca9fe7b0a5e69b6a7c19935505c73df8f2c06ceb3c09fabd79df5e67c5f209579df95d20ad0491ca79df3ae257b8ac3e95aa302f84de6fc847c8cb147bbef136f355a240aebf1d19e1db26bab8ad4c6cc6d6e27c92d0cc8420e0f9e9cd2bab9e9ef2196e07f04997abc6c1054e015c72da67366699fc045c21974f9d6433740a3abc30b8fea46456ddc7c3adf8ff00b85cfd8e8b6ea2ead437ba951d6d5c6c73322d29be2a35e63081d752933b7938c212b1289d2775222261d18633dad1e73e5f323ff2e0ecf36784f52179e6bdc31c5c19616886a139ad0c632e47d0d32d1ba2ac6c0807fae7bd766ec53b94fab2d2ff22a93483fbb6445e72d409a398840f476eb72f1a0ae8143b70fd04676db9411782f83dee540fbe6aa82664c779605bf155030785e13d8f0bbf86951c62ecaa7e25881f491ef1ffec48fc67ecfe2b9247df77673fee6cb0f66f64fb982379d6deb13a79a3f893488596938b1bd7369f9b99832c4c3c23b04cd34cec0dc5236c4105ede9dcad1943c5813cd18525daa70fcbacda879ad6675aa6bdd5174f0b9e9b6c53e67f7771408aabf9fefe79e92c5e8a6638ae8f1e38b621647d0d51aad753ad3b5a5dffd869f18bcb9a8f73c76b452f45c2f8ef6bbc39c9a479bd1c66bb74f6e3f555db06f55a1566364ace5a637cabb4b6b5f422305086e8542d0ebfcdfff1d1e7ac9d2deab374165267cd49df53bcfaecc76a6139337e9fd029079ca503816826e13fb54ca2dd757e147bf8cb7d793ab299cb920dfd306ab4654bb7661cd83814994236d8b50cd984ccb5d210aa54dc73415c96b92d6079d19ad0da60496d2b64f3d687b8cb154565602ec004b311f96d330f172d255bc6f0c550afe9acd9927abfc7bf67c37ca527edf3f61625f6dd9d32ffe64cd0b10cc433411bc558b67cf11bc09f8f4d7f44bcdf6395fde7ea9f70c80cd66aed764d8dd0ce88679b5caed8fc9c2ee8e32d2ef5a8ed9efd1e418feb40aacf7f19f16c9a1b35357f8bf8049ea7fb91c7ba47d670ec211f547ddfac75bc552f487263f859af5dea16f91baf2104b6dac8af88eefba672312ee7fb9f96886ca51ead0fda9213f8aafb5bc4a7840f65a01cc004a510ee5ad1dcfb0fb650b747acbfff26a26f568544c0d24186f8ee4b169615fdec32de53b6d3f2813ba5b7076edaff3224797d524bba359d90f417140cb222acc5ea65d83c0e7bab279f4bd8b9f44e0e7c907865bb83d3ededf3f539ee85c3420f10ebfb0725fd91f3cd2f3cb28d9ea9911fe136e6acee0bb05a7f18bcd4b5caa9784b19f63dc0c47623730f54eb3f52b2610e9cac31bf560d29def6b778e3a6512fec915eff7cb1c18981292bc892893d581a311046dd9c132b6eb9fe354075e0caece17f573454c035ec85e973988dedccbb7041ce2801733da9f782c8d85491ddce93a943b1b647b41d02902ccebdc3c82b7901adb91cbad9d18b7915866a1d9098e1c43de52e70477048d02ab565547ad68c33c7138d30fb165f1e5e76a837bd74a4af58fbcf4e34203d0a1b56be539f2ddeb3706d3f25e663ed919a97ff8abe62a76be6bc6fa6e61aaf2c8ec01768c7eadc9bee7ceae30c31dcba975d6b558ea932e22a71a5c728d2dce05067b1d4a27dcfe0438fefcca5cb4a95efe83e63a7dc247570449dbed353e7efbad6ad8c981a2b47de139ff5bab370c969e154b9e4f07f813a9310a9c021f52b22829622dba52fc47814892fca57ded9158663b293356decac92b391c56d7ab26702767e30b9726aa51bb16b79ead28fb4e00b815b66ed75006e443a9e41bb5ead6975bd3f8326cbb4de758e7907ef62dfdbaddcb7b1fd85b3785ab27cc0128d4a8f0306ce3c92cfe0f708ffdb335bf69c55348e88597f711e5d35efb6e05bfb9d266f107bf8497e8ef403b791e1c3f23d6f83f7c7f2c7bdc63776dd4f3ab24178e30daebb616cec37cecfc9756e2809a6d131c648f88c6eaae019a9c3579c6287d7681f8b5a24bedd607ed629eae12f9e2bdf55ca28a01ebfa2b20cf30460621e22b49241d178f5884481f3cb5e062f2b5a1f90e4a66da24bcd9328d9501e5f7369ef94ab5cc9c42494d60edc51fa6d5ff843a8fd5d5ff89743ad44ca92594df4fd090a213696a51aa7c3845fb4d89b7c8ce12a9e57ea61d3651b8868e7d8af070f78ec48dfba75613df791126e81aefbbeed1911f5ee74eca37da5f0c478e8d9d4c6433984cb8ffef678888c982b7fdbd36c736ccbc9c74f65d3b6659d3f23ea63f6614974a4d62507951089c22d885256b29d96a7f196cdde15515ef5b95bed572ef4bd1b712467c7bd67f96b1674378d8f87a07fe35443630157d419d038c7fbb0486d5ff6dc908db1319271d24b7d8ec8b0cd7fa16d812828368871866b96fbd5fbc63f5db2e85991c937e1e7fe1dd93972d9013b5e56e69661f40026461bc9fde8a9a46f19da2be783f9f07695579e22609fe4b6252ef1ddbc0bbbbf687d9e316570db32c1563487f09b5e0ef117862d5301762222e99860629271f05b8d38d06658ead0e686113b9aa67ba89cb26bdd3e8fa4d64244cb955fef548f69f9c5059c0de748799e245be3b9cac822e2927319db667124ee46639bc3b462dbd6ba6f375e94842e4e5dfde5d409975afdb02d03b52c27f33f98f745c0d9ca2d2d3405f2babb8231fb446e87f1f97b776e3e7a862db3fc7a05512b633f75ab70bd7244aefa306fe9c5975202ef0d360dc8b725b06b961c6257d02b4d1c8867d3a1817257a7c80adf3e15ac7108cfcca555bde972db3b1bcf38bb19b35839f0fad1166f936c23877501f220ff81b93c06ad91f798214b63bbfc9e9cb6af0f41548c3772da35590ca55d22b6decc5d1d7b3149fb5e71e09a40fcd997da77eed62a8d84d9aee853c68fcbdd79b210ce34a9ac25093a57572d557d47cfeeac5beaf9eaa9e0bc29c3065f59c6190a303903918dd88b9b4c7624db2ee1b48de9cd4d0ca875095650d5c8b5d903717d33bc5b1635e52a1d989c72e3d397e29367db456ffe6afa124496a6f331d5243f40dd192b1a2e230badcf97acb3130f5e2af588fda4bac803875788d425867687a67dfbbb7da5fefd9443c7b565b79d3dd9e669e76d59a797b3ed7a1e3a580b98da60adba5c7c71d0acfd5bde9ab5ccb69fefbf6915976653797ef913ed65260a5c6286eb11e962d9358e3bfa3c8fef1526293c72e4bbba4522a7e07a5f6da6012775d5b1d487283eb5ff89e5b024e4b918e98ca203a0eac6bb26a5be64bb478ba936f1b410c298db9260f6969ef4023125bdbc7db8b725cf7e4b69e0c0bb1733f3a4ce5f6b2eb374713fbfd4f488c5d1be324c038689a943154e1568a3dca5823234ce6c5c26ed2da769d0b1a78bdb4fa9a71d59a2af1241315f462ec378dd24472efc8d17937fe3e4bf89a2987f3201db5ebf41726d27cf6c26b167556dfbbd0c00589c8fc1827fab8ddf45f95f76efec6f8d3ffcb2bc9754f9ebf26c935f967f74e5fe0d7f18c35eb6fe9b112b9767b418c56d9ef750f7725ad45de079a5d42b2569d5857cbdba2f426619e57273da74b5cf5af951e76b3dfc6843519c8e6ed6dc7b8290fdb0b1f7efeae73a579a7cb26abf03c7e7e8bdce598e578a5a6f1dd57f77ec99edba37793aef1fee973cb8ae75cee284055e15d62e19178d3707522ae7e3ee9ae5d00e9d4d82d21e79578288e3068737f317a7c865c4ce6f38eedfa4502deb116b7735b897b77f8d41b7cdb7fd07973e715eb9995a40986da462fa51b9e682506f2559526a6bffbe8b6f6ebcbe97568679806162eeb1602d386302d65aadbb374419f9dd1e08dcad0467c5f3e96934cdf94fc11a04ff3158473a8a7e3db77f79c292f017ed42a28ab2ff7d58f7ae65fcba3cb1ee97e5ef3feafe1ad6a4dfcd1061fa5d171d2f21debb4a99b5ec48d57adee17b492faac44efb1bed06b538a30302078e9e17bbb15b25d3ba8a4e9255b1ebb3905ab17ccebefd6591d9feea87e3660fe17e57bf9f60cf0992dab3aacb619b0902162a967eaa4709aa8b0c65aef4bbb2a56fe2695b51f950ea8b5544e6fc4d5d81b96fdccbd55df29cbae1ac667efbf62b51c39c6b2ac9c839cf8f85c7869fe8162a5f6a3925e16cea1e6cba00bffb4863d8c73c355ead66692428b3a88f939c136818237f256159881265b37aadff156ec96f73aaa3b7f8783f2bed00065a914715bb57d12b5f65a5ddda5ea31c67d9b9e6d2aab2c0a425ba857c471fd552097a343a660972b6d4cefe7e3671d5d34d5da54312451d414f00d3944635dad335e7f56b00b30ab4c3751ac81adfd8c33e950db63ff44c85b901983717a35575561570ef166c747d3abfb8a112440aeed6cc8f575a3d285f3a6c1c163bf745b4d79172d13da49263bb2415b00f520d118b3767240e1dd82f7666b352989c7a6fc8e7dde6c5ee6bcfd79e9ea9bf4ba342865886f90c30317d88903d4fc501a706f9b7070c15ef543ed850963cab079ef72e75090761717efdca575316d58589c9fdf9424ac0cf070ef2cc885b83e49f0c1c82e91a1a00b3ac246599bcc4db1ebdbbd70d956efd071652cc5e2d14f79a0f78614f7fb472185af9cabfd138b7705b9f9bf6725e29faa5804f8838a6e5f5ca00385808eb2ab34363ab50b2f2f9f39233eddd4e463d18d6352ada86a61dfeffc4424a22b2986f9791436fd96cc3ed2bdaa399b32dfad71bdd1e36cedf2f6996f6c2b6ebab2bb2ea4bf11b2295f77d0bdd9e91295cdd23ea548e8b8839e7dfec6bb74976a68232ca8be3b478f1bf6ce7868db922b5bad6f87ab1e162b27256ed71560a54aa622cdc91785046ee2dbbb4c8f220a9334b8aa9d6bbf73fe724f69fc8a0bec2236e0d39769befbf3128e88811c39d55dfccdefed9a8fdcdaac78c8df18882b467cff80e2ce569edc0fad9247625883cf6577f16f08d9efc2ce1f8c3976b73e789773f6c32b6d1baeded08ee286e68f046148b66e61ea93da5735c56a5c956b076d067db079b76bce5add735a078b5db7ce3328c04c0c488205ee15f6f3b81b559f9bc83df308fb6b974bce453e894adb0df2061d8b3c3cf62ef9491bf8a095e1fcfaab174aede7b64f4e32c0d84737f9d559b16bb8b3acccf662b81b5a01d5f8da95ea91483152a4c93feb61db3f5d2aa8791009f43820bce2c1076edf0d56a31311c6c40b4bd9bd65764de5cf8e886805e3c6d64d22ef09f9d03fd5df9ffeca41dc983f7d403e190d873607781e2d756177c31b750dea08658fba0dd215e28a501096dfbe83e5b51e068a650ebbca797ea4508fd1a570e3cc5bfa6d46fce6e35580e9581150013bc8004a93d12b222cd5dfffaa8ecc66fa9694daf5e333b4ceb24d584bd67573d34f50c9e3234dc9860d0785084ff75b32920b2539af3f26d5363e70b94bbc71615c564d61d5df3527885b8dae59373333db7ecad439655a79c12b0c39c0f552cbafd6ef742e2e682ea274bcd9a5a7a7de6de1f7a1ec5561a341040e5e75ec0da406ab741f458c3a11bf714f23f730c19ca32d78af3c52b6d79a2600b698e2c0ac8dc7548b658bc4bc3f975bf2ebfdb916e214be74fcdd8c60737bbd2e5bf68de1e444c28562b70515cc9bbce2cbf911b7ac2aa18a20b1dbbe556bb8e6788921c34285766b410601ae9221fe28ffb48169cac5e277cef909925dbec8f8e3b6fa884553a865f085fd7f9992b75ca1674672eea28d37f8f61ea77b3a6f29e4f9aca9bfde138995ca6cf80c89089830e59940400420408207f23e221137b32640242260e64084b718070600caca5ff4b6d531cfee25b1408cba09000202d9d4102b86a5c887832c4c67a923dfe048f2701c05512009492009007c2494338191722090085277d675b400280c724006c1c7b08f19000a003ad85d7cc85e8c9fee92540028001d6834cd63b2f27dee7d7752142c002069e81fe1fe2860086130306011e065bc4f04ffe8931f014940a25a138517e4a83911463d07efe36469ce18cfff95b627ff23866db0bce7d8e62ae2233cdefc8d7e57ca04b5ac564291e7e72f2d5198990002fc20bc8c41eb5413a9671188f6749213c2acecf2ba4fefc319604601c0990897d0cd7954dc8afcd07b1812400b3824102f84c5c88a3ad7f2392008c2f04d06824006b3ba662328481002ad589046036a326e023b91033f0780ac3194f860032fde7ccfc8986507431589f108063550b8ed6f527cafb331dc4347f703821dd7263afffe522bfaffa29477768e4bdf7bc9a2da6fac9cde693d2e97734c8c49e0460e7b1582e1c97f13009c0ae181531604c4454eb2400eb43633d0e8570d224802f4a7cb40c3ffa4102b0092c0a367f1605bff608c59fb148fb5333615b0924007b974a02d88227908c7d4602d8ee4dfc66f3fa2b1060332190003675daf775b319fea16e4fd6a7fd1fc948dff144fc4e7ab6bb103b04d0febabd4900db65b4c14f130db22f1eb701ab893e888304b0759000f69b3f23421542fe5b3c3831f0e90c96939100763b12c05e3f5637034f2601ec3e7408004900fbfbb1c782a22480fd2a810a0120854186408a1348a63a93d3f1780864601910e000e12000ad9b918efe8777a64220489d1ae9886a2100c2388d7a3c0960ef82b8480087f908c6f4c54799639592a96427061e4b0238b80910c0c093008e252e440a04d2d0e72480439605f7db2e448a134590047068114800872609e0e770211227387062e089a36aa0400038a1148afde8d7185bcbcd61bfeec71d205388cd015286806f5323571c274725a6b231f01048152393008e06021443154bc713211c559082caf70665bc93047024bb1089e3dca1b17b54867196211c244f03d9c67f53a9d20c7c46faa862b9681812c0e91433fc9b7f5412c0298a828d138aa38d7cdf81274bd2c5b6e3c9716274f10c3c1ebf3d63d45c14c6248ec6e1c479820470e69100ce6c3a09e0dcc8aa8ccafadc881af00f6f9000ce9b6843c9acaf2f49006723cac31917229dee440238155d885467fc68830c16d10009e0ec44bf71b153e8681be5a3be49a450509872cd2590002e0508402bea71214220054451c1c53b12e22079aa104a168d922da790008e5e566df6685dec2e440a08e1625915c1040a038f166d22015ca1b123c534904a02b878d0bac6704ba5ca4ce89a0470654380241de2c290002e8a0b51924c134bc74f2ebf88864fae5417623a9e0a613054f1b1b74900d7232a09e0707021fa8e9984c262b60f65f60309e058ed4244c986d19e88e302fa0a371f951a4b02b8b44660c0c043f2342108a48890006e730204d0454900f77c02351eb46749c3bd281695a0d08518e78c2701dcc7509548d2c54800b753dc883620804865e9927b2b819e08da539d21900aa6e3215c2c98898770230ae526b810e3682cb2032c3212c0bd07924f14a38e484773c6132110c22460c004d486a8ba581a9b929b084d72130a04ca8c43874c4dc7d31868d5b1a86a784c09e4383696967824092480478204709f7621a2e562e8433502998a49c713c9140cca35cfc25812c0dde242cc444560a2ffcf1c43349504f0b813209674bcf35d881412c013381a6258bf47d4ca134b8030ac2f0cb4c1eb28c6d2f1a334d0309dc54b2e5a9433ca0b0dc2f0a4e3e92ce0f154a025e5a325a36fe1211c4d100617d004c5205bba20c44b171413172501bc00814ae561e06923020ea2af32475f1d799304f00ad0e974082401dcefd09c8481a7d047f4c1ab462001bcaaa3e414547ee2b8f742c354b1496ecb8ba78eb8683acad16a1a09e0d5436b2301bcc9bf0d1ba81bf2ac40dd7b02e5bcd9932b2ca48dfcc8c053a91812c05b4e192ba452d9e82234517112c0db88b2fb684c31635551a90e108e0a8ed51db3ccc14c5bd31017e41fbc0eb72e04b7cecf1b17e51d1e8283bdbcc2bd23220008a0820ea34cf0a942f2a828cf47d8e0d3a592005e14e0ac0ac74327fa8342c142000d7420017ca8ddf92427452b12c01732491abee849d2800c3c996557be2c0209e0db39cafe9843f3e58f4a4a75c04f6a8d04f05d439ba39300befd2ec409b5f1354e6ea86d5243589447b4217e006d68f8bb86a6e452dae32e05ab5b934900ff32085808017d4363832818101c1b43b1e909c2a00d010274a1173264cf5084aa250403570864cf5dd2ca754230103dfa5504c62204b2670ee1f22311185b47207b16b0efbb2002b30104b2e7e3c7c9ad22309b2681ecd94cea361681d92002d9f3c3b757db4460360601066d5c8823958bc0583602d9330c2f714404c6ea10c89e9b6a7c192230d69840f64cc9bd4d1481b16e04b2e78ec6bda62230762381ec992514e6260263778e55b310bfdd5e08064509235f45600c1f81ec695225202902638c0864cfa53b03fd4460cc2a02d9d3b6e5ce561118b38e40f6741fee7c2602637208644f58dbf998088c6918ab91e3db33ac080c3a10c89edc1e79a74560309440f69410e36f168141546aa5d270f46b3981eca965d4e62b02833d04224b8330e806b31d6725dc30e89bce80a59ac6b3ef3e220c26416c643d2c0564c0606c3a0396dee74284d98d47e9b3d0279fbfa3cf87d9e1d1d2b3e90c5846e1bbd2c689daead15223b4b68051fab61f6beb87d9f78c9462b028fdb1c9a598d9e3b56114c6793b3c4abfe087da302e13f4f6e3f44f5c880011c6844f9405a12d5d47cb7a47ca5227ca12c7dee3601b6d67ef0f5262ca27e8cbd219b0ec4e947edd28fded1ff9ea99a0ef4c67c072722e44986b548b982f3fd48f951ba7c74a8ed3378ed063357ea43782b9712c49b0ab26de741c93845b67a48c04733b8dd61182d6eaf65d1d64987beb6869ca0f28c11e807904476a393fd1c2d9313df2288dbe59fd236f6f26e8dbc638e219e3e3f30fba62931aa767131da7bf3d42cfa6f223fd8a09fa65e3f43da3f4841ff8618b9aa08f44b510e94284794347e9e37fa43f3a419f3b660bdecfa82e2008c2929d60b60b0c58ae151d98cb83108e9cce8641873626f60c0a6b0e27239d41618d76a8286dc728ad28ccae432043f282303b3701060132cd890d4373488770603accaee342a482e9e3e303063a7c70604020c8807020992604b38b13a80c2c242f4605d347b299b11109cc6e05f32f43bb0c98dd85121b0bf3db8c6605e339f8f8e0860cb3af1f27a6c55263610131d42b52636363618143a3ef4d74d4c213ef158ebf578b36c25e199bc8d222cc7e2b9d01cb9bbb10ffd830ccde020b4ba0ffefa25261e19d233930cc01c2ec5f589551e1995dac71276b4c0973d8d3d1f16dc25f1f5bc2a26568ef958ea7c01c7e0498c33396e54430c75a172299884a35a61298232696e59074215198a36aea933630472903967f3369ca06adec4e5c1c0b820c3cccf1383e8e256c8c59ba65ef85ebd45b9c07963f6cb9dc5bf5aa14f7fe507a917f63827a45f2fb94c7f13f17c953e915dfb605898971494990897dbce7c82ccc0be991e63917efdc91b93d239d01e308683281b241c3522020deedbd285914e674272c8600b2f162b4af45610299d82f86e42521792a284605d1de168f429b938c56d1e9421481392ba7213e6709035648f85e7cce5bb058214b78ce96044d7a22ccd93c0609145e10401521c35cb204fc9f4da6fcc9d493085114e63222c05cfa09b404984b7bac521435309703fab13c217ef4a77d0295ce82301102514befc8c48f0dde34a73585230273ed274000cc750fad3697d5f9fd360d85b9ead219b0423e3ac81f1f2dfc23f32430b7ea8817716b43002c76da85887e5f4e85b94d584e441b71616e3c75d49df0539e86455d1b82d8c84e3077cad4231a77d97844e3e1188968dc4f7f88683c1cbf8b68dc9d3f8d68308fc488bc3c38587cd3a4f0c0b3800e714cf2701e2b02cc63ce320fcc633de6e063ba8779d68c062cf4d5206a1c2b58a1a319415198a7781a88e72960c08ab693100ff35450e3e2e8d8d1783739ac90619e87637192e7156572501d23d19cd65ca508cceb310d5e799d19b0e2f9c9bcf2fad168b078ede401120442c0df9a34fe7ba81eed1c788759669e6648c0c27cc1d3d0079f3703565a39591f7c1b282c274259e08ba5d16830bff6f4679061be5a02cc77990a8b3f42ebb9054ba8a3fddfdce9cf17c3fc9204989f8f365213bfc8684d73d00fc5b1a71a230d4d9e0dfee3bce1825f2e58fca91cfc17a7a14dfed30c58e9d977b17f9ac2ced0984673331419b072e8a4e6c8287263ff0e72ff06ec7fb144f227da8d4926be6c7bdbb58578cf2e6ce916397a5f3356a4f273b19a1c11c773a3fa4ad92c1a9a810a707f9f811e1e352f2ca048a78f06addf763f141158601f8182c63681d469a8582096012b374df60f81bd743a7d34b611c73ad069824a50681a2c087231e059c17f075482e4e934b78101cf06264b2cb80d8d0212c6a3fd4cecdf0a6e2c13084d477e212e063c3b631243acd92072dcdf442a049063ffa185c13f513cfd2fae0bd251d40b0dff01f5b0b0309d4e87256c5d4651070be3583de9cf5b81858d5963db3feb358533a76102e16d0c78f6d02413fc85303e937d3a0d7e65c02a6b278370a610048ca2706cb9179ea9349af0fe4b167be1990d7f34eaccd72362d98ee5a5d3d4aa08711a5a157160c02a5f276b55c4970a01937c9bfef77d5ba47b3a1cb53260d5d41f7c3bf6effaf60840b0a3b69c667c9e9a838a227fb4a5e8191a048c7be898468560d1e1696844b49f01ab0e4eb69118272ce138320e17b39a464d62260c58cde7fb81a8181c1b0b01202ce1c548678c0c9c59393f048064d48058586c2b0102c819f80c3c1e16cb4353733a0482548a13451016db4980c576c06a71dfafd4b2888919f80cc6d84ae0ff2fbb5c60f158880b96941df99ec6b203031f43d67cbd88ad71f33b47cb19b3676bfb3cbce51367988f5f7b2b6496ebbda4ecdbbe14141fe295dfe3239b9c8e1fc94047f37b0995bfb959019650a48d4cdf482e18c9d7258c200e58427f64c8873e449f59a1bd375f94cb77b9312cb106e282f9f527d6ed59ebf35858228ab5a6064b8ab39ed2d1a70c3c914c15148525eea2c37e89dd341a2cc99acea0c31247e269b064034a0b3a0989c212a504723a1e96b8c15adb8525ae8d4768221e968865cd7508914561897e029e064bf4c641602c2cf176746a60bc2611a2282c294480402a2cc9355a880e23596c51d06f63cb32209ae57f17b947e4935c3c6a2d322cb9928e625eb2df8528280a4b16fc9fef94a0d22560c98a7406acd6ea42a4d2c560c99a51354c8c31c9340870a08ed481aabb9b024b2d4547ee1314233b296029be118b482d9ed847014bc98d6a85e59852f309b0d4bc1f1c73d2168a1fb74ffcb078070144d6d85f2a8e40a682f6143ae830b29a0c4b05a0a5743ae83096efb2e6cb68e0c422252c554885a556ba1061a9524a2c2c4570216ec7c3523728b1102e0e84a5fcd0927b14582ac38548ff6eeaf5bb053e58aa9f4287a50e7fa7a96f631f3f758a492ba5b0f4022a2ccd35be38084b9b516069551722450496b61d5d3d632941978187a5210a2c6d38b1be47197f6b130596f619798bfec35b3b29b074e8c45b64c884168b151685a54f1360e962586ac885c862a52a961a07c2d2f62ce0423858fafec883f811238f454b8aa8e878bbef29b074d31f572361193ed662242cfd7aac5558467a84c33f52ea8d50ca084fe60f80207a1ccaa18c330196711ae750c6338e1a3fc2e176168732ebc61fa02c9bc032b1230fc659c667fc61751a96c9a5c032693f725239cac9be094e201c95ce62a38900cb3c9d60e30d9d1a3bd1c8d8f4b3cc109565bab18664675060993e96516425ff6814590d0a2ccb3e49e86fdf7fd020793a280ccbba106059675856cb8548a4ea32f093f771c0b2110458361c96d543d9c2d1c0743c0d02a9181a8896a51160d9545876910b71a29f238f28971c97804a255b4c80654f8d4b255b99404d0461591917e20e967265efb11ea0cacd642957b665e441fc88faa13850f73b7ee4b004580e034b635d8844584e60a43a13d43419787cc60e7cc60ec6f6eff42aa74380e5168c7320b77444aff693f40acb398d38c41fcc251732622e39eb9f005b2e9900cb6d9ba877cf1f802d776cfcc104b0ffce6e2b58eeebf79bad607901022c3f03963b3bbed36adadb5460793b2acb67d2f1b0bc1b8df575d21601583e9c00cb87c1f2a213cbf6accd0ccb03e1083f1bffe075863838282432781dce7ba3a7b7b757042e88b5992106dec82259190207ff9c11dcd8960758be01d2a0b16223cac4732a2cbf114d0ffe74bb038cc35260f9ea499b1dfe10b2a9149a3e3a7e6005665613383d5a2c8ce361455f9c192d960ee3a45d88306e256d34f432262d5e212ce3e302d1c61259c687719b50ec278cfe88834c68603c2b9ec0b8ed104403e3583f5006c92c068f50e871304e1d65316b271e3fbea2301e4c71d72930ee34cb6f71f726fbad3ddaf42b0a8cbbf46388c00d1360dcd038e414f85950c6354c82f20fd6f10f5e0f07fa7be1c2bdd7458607e3d68f5a2730f2977f363ac93a0ae190f2b87514b6506105b15f5b47219b022b784fb20eac707a1c630ae513181bdd1d022b3c24c00a0f7ec49821cecbdb33dc1b8ef0f6c2c18181211be0604f6f1cc93b3064c38814ac4d34539342d1009a352e85a219155668fab5148abe145851635c8a1ff4ba213c24d817b72e24c03b78d4b126daca81a427da3a498515837fb97d0756aca7c08a59136dd96d08864981de86b860ef0db8900dc1dee138ff8849bb8646a41fdd3b3415e995f4a0d9e31c29995261c527bf965ec99b022ba9fd417a431c1c1a1a1eb2de1be7131e12f4dd2ea6118e7ecdcff71cdd98842aa57a2aac14f81b8e3e5360a50b7fcad10fbbaa4638fa353fdf71a4ec06298e73a4ec4d8595867ecd9172220556b6f98123ffe088481f1f7f4f7fefe07513b0fd1e21cad721d989b6ee516165c6af11a2fc89022b97fdd0d6ba703838c2c73bfc07838c22e437069924fdac5593fc6396271556fefa6be967c553e059d67fced11f0d32c2d16f00f21d476f20a5098e3e51e159a9bfe668b624059ed5fce71cfda19f228d45c251f3fc9ea3d9d193bc68763c159e8dfb0d47c728f0ec909ff8b5271c1838e2d4c121eb587a6279f92484cc7e0de126daeaa3c2b38b7f8d1015710a3cfbe90fd24f61cb21ace204c98fb7a5b2860aabc8fda62d1a0556b1f8a12d12dac1fc101ac2bd23467d1198b22faab4430a131cbda7c22a49bfd6b4aa28055679fc738efe6877d4e2231c797a4f9923d58d93a2832a8d0aabcafc86a33c0aac1af0078e1c83e1d008bf10b45bf642adaf3939319b68ab15129b68eb1d15562dfcb53dd4665260d5477fda567048b0f746ff88756840f2f7fabe2d35fca448a446a4c26a52bf692b8602ab994db435bd4d9c5322179e9c9faabd1bcf1dd406c77307589d8700ab73c36a68ea4b45536456e23074c8cb3fd4276295a2300e42f2d627d33db73d2e215cb5582376872c7f78913f55a7ecc1de0bdb63dae58dbc2242023dcd002e6160412588987173ae8e5b27cf4f7aff45cd789beb623d3257a7d987e01bc744a564495bf45e22b622f60c4299f92c7e0176a0831accecffa49c527fab463d9dbd2e79fd368b79a54903090ba6f507d4f92ffd8e10198bcd14acf8bb6b396a37973eeab665fd31772387554c5dc6d1a3cd9ab752acde3d37601a2d8f84c0a57bf5380469a7af9f7b89c86fc9b58b0c192ccd59a7286c622ab7e6a09ebfcfc0f21b9b3e5992c29f71debfffdd611b7a13876d0ce800cb5f8a85c3ecda17e7473a29cbf408a595fcc70edbf8e7fe1e58eac4ce7beca97babdd2d73308addc04600589c8f0531d8ffd2611ba00cb2cbfe17f73d61b4b51de4257e51ae175e5774fc57e509aa949cdf1d8651a12f5e5f107db9d63d717763b87540f7056e97eaaaa13967a8d8769d756bdf3d9a79c6edee968c37a0cb9d3da27b9f6f5d5935bb5c2e7287cef5eaab83fb08363586bfab3f7cf0525012670ea652ccac4c26c6e5f055bb3b4c25ef973b1c3f53da9e2b0e58b27d7c224917afbe5690beb725cbefe09b6371a79c5e2846dcd8c6df41db79ffb169258898da3baa42ebdf0db917bb9a2cd5d077a0dfbba64e0cc3c844b7d8749d9fbf401859b196193df322d703cd96c48f85ae9d4b86b6bd791a95b369e9c0fb28a28b69ef9cf18301664deb608016bf37d71e0c959c557940e6cbb1a57cec4ade78b41244f25579b61edf6a6a6d76d63ca67d97294dbb708de30701fed8f564eb87678ad6442378d2936376699f3a9c382fec4c63ef70680783b006da4dd7f56e78e75ccc9c1758869d0f30b11ac871f35bd1999e87be191938dfd876f0b0ddd015419fd398c7b6a53efa56f77a4ee54ef96f873d9898e632cc130c13f370e4fca7ebe38703dcfdaf9f96f30ffe41ffcf4ecbc162c1bfd5c8df3b2da7c7eacbaf4fcbd1bff1ebd3706c9ffdb2fcd37a95df3958aae660c1e0aefb222e9d22e1bc5ca5ef177a540cd51faeec332c5ed1a85ea4f676ddba35fa37d9573ad7dc32ed345ac00ff770d739c8e4ea7fb4b8bdcf791934f0bb536731361eaffb234b29abd5df955fac2ab7b19c2556cfec5bbde106ffb621e058506ca45b489b4468bbef57be4367ed45d7dc3e15f56691f5a5f26d5767ae38793c31d3c2b4928c34dfe792edf7bf8bdd30e0ec5c34681054a83127ce17283930c7d007ab0e3db6463c664458e262a5afc7a45775d83bab757a2bef29e5fe68a69751abd84d0aff5c58862d0198d8d3c8e6dc35a56a2e72e27d43ecef9bb21796f6e2b4df735679c5ec55727cf075c0559d46c7982043f47bdb0566a489bd0aeb28a433fcae75303addac5c5f5fa06e5ede8cf3800ea11e0331b1ded33a8d1fa3504c6afed92132afd72203576cf6166c8adb6e9555bc252167be5bee7fe0f449a4f7d529f2d5f52fc436ef375ab75144e4e31ed1f6595107d3a2f2b744b57aec588688cdea71e0b0376ad0d0400849cb63edcaeba859e40b8f3e712be8b31f5f593d84e61d6ba79577545569633092dafa1b485b07773c3b9375b1b6559d9577845ea687b5c52a8add5ffef9eaf28550650455dd97f3a2a293b9ddd6d32f228ea722162dad9f940645b784ec5b70b92aa6bee240f74a64a1dda2f925bc1ffbb615fae4a2bc38fd3307da2cdc838481d78d8fcfbcbe92a9738570d113b83958cdd79122eab161d7994b455d089758583c367ad68b8e60da9b3de96e4a292b6bed77776e5ab0203aea2e9735b56de4a493a07ff6a4939f9d76ffaf38e9e4f5420d2d4d0d3fe7757250c93262c9cc5ebb0e19cf83afcf6a9c2fb6555bc15b82cc8a765d1cc57d4c0c4aea945dcea66c3953f5aa6258fd76a7ba0f518dc9ca6fa44bdb8181570865c52b2b719a4707e7cc674dd6d0d04be81c59dbfd5ecff1add6dafe8d36f724a7ecb7a4819de3b70d89fe95db86847a8af42ec6eddcd8eba9a2fffec54cd307a4b3d44a32d27da963f95a5b5dbfd21d5bd45e7da9d9472d7e5ec2bdc6cf4eb5b7bf490d799b84444bf2ae6e911e6a6a150a65cb977bef5505e1d2e6ca5f176e1754d1bcfbaa5db20c0c0298a03f32074e19ea482ef7d66953ace9d20a95d54830da146fabd5b3b6e1907069c9bc5d5396d48b094aa37ea1332dbf98c7b620a0b741aa7afdfd0a7f7cbbfb7d0fcc8037cb907b4e5e153a6a6cccd979e919d648759e5d9ba79a68c3870637e9baab6edcf96b3d9195cbe1abc78fe6c404d1d6cd33ad2c49a6ea362ff1c25dde7cb630fa9c5ec5ae67e339941cc6671a3994a7b4e422f752874c152e1fb354dd7a4422ffa34b2588cc30d4de56539ab1ccd43f4c90a344a9caa65fb238cfef4a71824c5d453bcf85b588f7cb8556a722dc9a3738489ceff802710eac0fb9be769ffe605a20efcaf5378343c7cedf7fa6e45780177eb2e3b367e58ea36def7431f9e7f6de9f41b821db53798693ff55f994f5ed3372fe7ef0f4cecefd0b47184ded50e8778a09af45ab62be7eb469616fdd71fe1b368590cad96452edfce66c5d9155d02b448291cd6598947f50b2e8f40bd2dba3f2dec75f0ead259e6aef693b659bc85e2d5986f10398182fa4467df656d292f543b948c846ae27ecadbd315629214e86e6af2d16087d583b77c99415e4c7c4888d7799a47fe181cd5e87cb35f15bd6ee7549da2c58a7d3fd7a79a397ff12eca73cb677973fec7a3b7f0db24cc3a2648656a80e589692641f722c090857e9ae52b5caacc5462eeb4dfa701375c78069b96368efe226e1c30098574ec2ee513bdbeb3cb72e9065def5073fa4be316e13c3c9ab0b7743cf5b6eecdc2d6c17b0d7f0915982baf909291bc4daeccab7f3f7522d021ecbb7765db43856a27b8fa47162a3e1a0c2a607e75a375d437971fe278e7d4f42b6fb9a78bec6da1b13b92e1eca6a2b709dcbe3a12f351454b72ed0d96c438af23564eb7d8900dd3df5ab0b6fb53d176e219c5eea6061aebcd0202e99c3cb4af1a4b16b19580d30c12bc88288e604bdeb98ddbe0c53c34b47d722c7ceaeb76d5eddb31c6f9cfea1dfb0e9f094a1b69609468e1f74a638f5dbcf4044e75afdfc17caedf39d06dbefcc34715ff1e933fd5c5535ad375641e3c23db1b52b9079e5865c8992c72ed5e50ce95ec67646a5ddae7d548995bcf45c0eebadfc98c4f393a4d0e84f118e3be9faf6ddcf102e52ced3f5764df2fa067eb5a75ea70d050ec4b477fd070ed12ae7a9bfbc5baa90c3345a79cb97e75a6c0fb27dd964aa579df6d8bb82e2f1a6e332a2a0f526a93455c131c93e40f644fe0937578ef50f4c254b1845a09ce02645ede2324c1dc0c4d4200ae6f194bd6da4ab7b376e5978a2f9ce85c08e6df8a84fabbdf44ccf2f9bcd85659fb225039998ad932e5ec44ce568ca1fd3ab85fb7456e70b06cf5a73f2746a90bfc99502d703772797fb24f069e57f4e00927b56fa155c9e9b207d6ca7fcff7a7a554946ba3ccf168928dce79fed5473a262d623e387af7dc959331e241d7fa57241d4d12d1309ad192c22c9ac320895f556313b223fac82975948d4bb676c6da3aa39cbe64b77e94eccc076d6881f8c9d04d729274306af0ac3bbaf34d6b45bfbbf56583f2f7b11ee8d6b258854b4be6e93c2a8a5ad585825b7babe46a46e5f6a99ca7d99e836c32f9a335e7a1d46562721da1b2e447f34dbd8e5989ccc56646af1f90ec5a64cbc546aa5c7eb3e9d323438e9fe33f9fc7611e2666ffd4b5bd602f319cb3b0f54b938575f1d006f495eeef572b0c699462292eeda2dce6baf9223d66df4b1d551106bc052eb5fea751dbeec7974b6a8d4375c19c80d304176c4eee8c74b995eaf8cd87949d9333d9e35f2eb29343d152268390d2f7971cb392568ca900e1e282d3d871938cd3aa5edc5c4296dd41281ccaa334ea52503fd77025cad78760824efcb55996f4d2e35e7f6c83bfdf7ee5b901faa04912b56add25970f83771c7900a3e678dad2762ec782282535b12cf5ca075de6ddb8f2c7c69fa7c26b8cbcd8862eb47ad3db78729b142988bb24266c7ed93359a9a3be78c5c8dc5f886e4add54c8f28ba5fde7c83cc10ebf03a1b156b5a74fc134fb631b7ed9a8d53d644e8c8d5582153396d3d09a940c0b0dcab5c3565267616774379fc12be84bbc794a40fc52ff4632e75333744a264670708cef5dc1b710d7ce32e8898a87eda8a0935ae6f280998c9c7b3cf6e43198727c0e45883449f6a0ba3056cdbc5bdec80b5ca5efdc282aa24fbfd166b0e3ce8ed64323992be4c99ff702687d0c8d5584ad3bf1a4bbd63bd5de5e75abb1ec9b977973614fb952fb6f4ff5de2fb5f8e4953b91a2b09094e91cc2e7838ff3c69d12b45be450bef6725c5455fdff1a5e2ababa8515f4e6323027a3f2c189c3b3baea14b48dafa74bb732678413c94db499023e2d3303975b7c3a4782ff1d72edafddd70f9d509bb66effe0e1b6d46da716ebdec6d94fb73fbffe7e33d88dc5493dc4f69e278efee965e5ab85b7ed329bbaf72314733535f6dfd6492fb4d6a0962dfa4372be41ac7c90277c255c53d350d408ac05320a7a37cf9ed5514c69e24e7c9eb3302d594bb5450eebfbe3e03fcdfaecfb003580cf8df99dec5e8453ff53cffabf27e05c6c35f94ebcbd4a925fcaa7ce3f69d06bf9b7e6513b85560bdd93a49353fb7494a6b1746e831c702c6a1d876e5b79ebcf147b419d7ec367bdc3e367859be6fb82af7f88e142eb9ec7589273adf9e2899afc53b58dcf1bbfa030d1c33d7c6f5a53b1fef3b3d3f4a78759d74dd9138de77b7cbf69bd9b8ef8879be38f49b6ce2f2b673164a12fc396a9f5235e44b186f6f37a8d56749e429bffb86af242309183933cb8a8b579d95e7151f78d86efe5e9ae78d85c8c5855497bbe5f03ead14c45cafa7e3a094a3fac1baccd5c7fa25edec1b3514cfcd5deee4c8b1fac2d20bcc54345f709956bef0b3cb9858f9c2be8470bb752598cb745a97ca9bed6fc181f4e5fbeeb93fbdd61c124d5d3927a41b5987e195589496714d78bd2397dda645ba552be9a488c88d38f5e379490afead1bcb30960013638ed4a79a75dc8aaa5829b541b9981ebf6c16ce3de4f66ccd859a42ca6e09a6ab3a64a6dccbac6382fde3a30acf298c2afed7c6cdc203d073e52fc9c5d1856d2ee485c7300a96aaafcb94d9b59243ea3b691e852b10310db7974d7bb34befaddd3fd79c229e6b2caccf99da7a2685bd2d64cecc1689443449089c5a9270a66ecfb7c5f854bad8fe1747973ed7fe665ceff9f298a3fab0ed5200c833492222726aaf13173ce1cee60c3e7275c9f6e72909bb0a4ef03759f4dc4c9ef3e6957786260a25bd694169454f0bffaefd5b949ff1efdb9c9d78b007a4b8e6b278c9feb0dd25e735247a63f91ea5f543c6516f37bff5bba5e4aa77f6cbeed715aa05414884220e5ed34bd3166b65cf6b1c50bfba982bb167dfeb3aa3eb360749f74e0af18edcd3736cf62646c62ee5d4cf2b7d9f9d3dd69037df85bfe2f53d05db7a24660dfcd09e7bca405a5f598639013031054834cd27d840ed65fbdce1ed29efe35d1d0d56b7a99b15dcb50c5d56e4561b95866640a688dc96eb7d31a6550a671cbf3dded3e16f5d737ffd16d17bfb9f25ee2bb2ebecd61d42abdcc8c41047ee54cc9db8539154b7fa1c3692ca79d0dba495d3337d0e8d0cdbfde14ec5e06a7e0f4697d190b05ca3f991ba47ad75067dbc2bb3de9e2bee7a2cde607d4216d9087f783317df7621578087cbcaf7a3586cf2d5c44f43c71d92e62a3b50931edd1cbfac80f2723c83f6a21e2afd79069db37467a0df6f80ca5f494616dfdede5c6953f16260752e9c45dfb55d2f89246bafba738ea543f91d2b73ec5b8467f683c7cdedbdd9db3747cd122d19d4513821c9ceddbb555066f3f58c82f9dbe78e4cfe87feb393ff06c5a11c65cd7389d5d66a7b8efbbc78710b8a14f8174cfe27222efc7e74bd1c98f3884467b638e3ce7be9aa1bb3bb8fad70b5a9ef2a983b33e50e22d51dea5bd5faad78de90fdc10e47a719853cf88557f248ee811ee73755bdecd319c3abfc5094a9652370b1b3dcf26c9c86f2de70430f8941051d67cea297596215379029e3350ac56bedc01de5e95fdff7bb7cfd2fced426223c5a417509f4dd15e77862ce6f0bb7a19d3da3607cd46367c8b2afd64bcc86eb3088de270a5620ffcc1981270f36e987e55fc8bbaa3ba810b7dc2a391ff6f2341fe41bef1c38a6d039fce93ae47fe5ca7932b2abd8f4e3f2968bd8bdc2724eab07f92f55fb9bd8149dacb64c322a3bc07ee6f900e26ad07265de1b33eea017b73825ece4dd42031b938e852df93420baa433509bc2bae38df0cfdc0ace5d5630fc499bf0799ec0f5708dfefdfb1f2cf55e57ed946c3fe3b2b269eee134552462561897338effa6dd1dac6b67e497b28fcccd356e4edf743bc55e208df3a51ccb40358009ce426abeacf7f6da35436c2567a30562fb44f8c4968ff3961cfaec9fad7ef6cdaaf076dc94f11a3dd058fa1033508f5a5770c2ba34506470aab81568ad891f72af987f39e07dc232832e8bb3477bfff60a43258844c62f4d6abc7041c820aa7ca8eb90fa4afb55a5060d73bcdfa67d01f1d22e895710f0a29dc40185561ba6733bc375f70c77cf3031d0d19cdedd7970e155c62ac7e1328e4b0093a30c79f2cc033b57c2ed6693ca296fa5a153cd3b8779a343797085bea4797d6050d8d929eb6a2b9323a0740366601daa2cb649aee0c09509983d9c117fad99eb46a096920e277bb673a08564bc28fe7353deacf9e44a32725ddba6fb2ae35dc2e2bcdb7ef657ec8f3a398b8a6fe67f6a305ce0ba93c195aa83ac53e4d7319e19b4ff63b8b017f6a60878bc2d7adecd4d34e6a9259564770c1e9cdeb46f74806ffdcf7cb029d5bbb653c2966d9f907d52407ae867cce13cb5ffc0b4ef5a3ac1cb77bd096753e04b524b4ccc815877aafbd61acf79f35e95cc8d18e0a120562619f47ba927b5aec327331478b81047932fcb2ccc17d9cab8ba8349ea7d8746ba3dfde9757bbf9be0f8d7deee71108b7de8ca57bb7585af469878f02da924cbb59cef3d4eee8cae3f24d430d3b5105996bf5de9442155f6e88b9365874f78e428a5ea55dd52b418548dbd1e71ace034f2dd364afdff91abf8feb93bcb7e76151f0788fd6fde59f69babf6def7d5fcb2fc833ef9d7bbb4dc7ebb8d727d3a533d66cbad63f39958639f2417d0698f032757575fcf5cfe85daa0af87d5708874bf6af5fb85079b7702b33c81dc5df197575665c7a8777677e3ee76b697fdaefe2dd9a9fee147bc1fdb575d7ec031d83ab4dcec6ca67db4fb01f3fd29ef2e8671399a9d5b32e38b2e8f884bcaeb9033253afeebe6846d4ffb76ffc5b194d3fbd3a50edfaa242338eb1505f72fc845db87e6ceadd83428bff2c88b6a695fafba9d05ed73f63c73388e2cad5f926960659fcb5917a0dfdafe285fd3ffc2a60fc9ee9a51f5a47cfd4739acf5f1b0ff89f5f144c4fb66d5dc01e5edade5f107dc43afcef9585ddd1961f36ace66a354f7878577536621a6974e35d816ac12d3098e1172ea911c7cb2d0acbea06cc6bae7372ecd3ea854d930b62e40caefaad6ac99d958f5b44d2023bed3e77c9fdf4deca768e323891fbe896337c64fb9ff224f5e176899b42e70f9c49fad0b6caaf165fcbd750180ad928cc80cf9c6e49c28b1e08ebc40e2eeb06d9ba9e4cca315356397a68e0cd675ab6907429891a8bab7f2c693e39ed677452b39795c783dddb08015724f29bba38efd464019d80430c146a48922e81e6cb6c87c83e860ca37bf4a27d3d6fe84fc790ff82b4fcc3159c6afdb39655d509960d27897eaf52fdc2b90adb9fdd0fcab0642f34fbfb8850815f7f21f0d292c3911f7a2e2ddbbc4c3e6f50a888bf63259cd0daf766f363f6a6d2dbe897670556fbacb894afdc646e2a0fc81072bd0b476d5b4d2da19ab92755615a4cec899b3b5fff63e0b8b38efaca52ca8575eccd6f45ba9480adb36c41f6bf964edaefe07ca225b0a45407089369d90ba03c1cb9eb23750ad5ff385d1631e98356740acf123d476fbc5b1c3590ada816ef53565986a8089b982043d894ba3088b75787ceedb967931f623b69caf77c155f8ead2e2fa6f8331b93e343a6629d2c8f398a1608d8d9f4dcf95a05012771cdd9fae4eaca7debeecb9f1fd92ea6721a879e94c4c64e9da01bfa0a94deab464608ded756266dee0caaef6edd735d0e1bdadae6063717be1e6ce050673027510d7fba73bb35d796c1a1fb1edb86bb0cb6837653342eb2f6fdf2390d6ac7ff97c0b1a780ca61778fec2f2cbd436e6a463d85f3ee7f0ac9634b8eee738ef86eac6971a430ca399da2d8e0e3912d9bb1d1005a3c53378b7b5f4377f7d12ca7f29e78a28a088b49e594c8d7b7420b7e52e44fc8b9bd17fba918a8cdc483ebc45c3ed3e59cba0d372e92a8147a106d5ff8fbb2f0f8772fdff9f194b832c85ecbb644b96905d681864295bb652b66c59b33396c328644bb678b2644d945d458821eb945df63142488591e57739ae3aaeeb774e679cbabed7399f3fe6afe79ae79ee7fdbaeff7fd7abddfaf7b1edb92ea14987071ea8c13ae760180e9cf04d615f5e48c0a3067222b9f4ca90b43032ccea3a43229571218527b78ab0ad7f3834958c44220f1fa091f2bb4b48e940f16848f4bf1f8620fdb67b3741620aa54e842cc98ffbf68b83f4edafd8018b26d41a01fbeca16b49b1d7f0f9ea4a1631fc813a937f452dd50577479dbfd963af4b542ed2903e1e09b6c6cf6db192f7866318e11ee628c2cd242cd55a415259f02081a0cee86df389283f93c5a5f0d7e00c281ef0357aa9a7a79247fb3f4411fbfc62da7abc87e72e87ee1b08b4c71f475610ea9462fbc6772180eacbdc77e5d7f6dd127cd3296529da988786d8cddbe6e96355d3d5a8ee83f51f4813d4d3b19e4dfc12c3daeb04eaec59eb8f462c2502e3d54e6c5a171a608394934a0bb22466f94d63d3c21debdc46370ddd1b53c37a0a1d4c88ac67ae3c5333462ea9b8896b97ddd223c4a9d4885c572b455285347e24a610d04b97089c99b533a10e03a873746c87d229afc9f89e81cba0c354191ae561356034037a1c489f0158ee8a745340200483ce9df7d9e8d70a0975883d1896fb55dafd259179b9e164c8c8c2f51862e03015e386fc679e32a7e7a018ff366ba48e262c226cdb6279467259342656d8873aa0950201c411330d299e5c01c1948155071a66004c910529d9849d95a363f9f365a77cc93f1550bdeb1ba8523b8f9070b99dac7423c607fc542b20d1bfa7f928510d58381cd6d53472f92b02cc97b56fdac19132a67fb8802e8275e87c355830b9b97d20d00e7e177b1728d2c27aac86cdee4918c8a0c9cf88dc67c715bfeb9879931f3e680c477c717fb411c5f66fef90bc2be304b87a3f04469d326d457b832f3631ff7c4582b8af6345de9c3802d9629c9fbf69015268922da35314f48e32dee9a5a68054ea9e5c28ed74ee79bbde5ef74b0e5ff6735a8835cffd72eff73e9758e85627e60a38e47e6120068788061eb6bb68de493538bde6225ae163e8047a05aea293e73519d2c860c2d6627484a8cf02bb25cb12ed5f418dd50d466cf2e2fb884dffbf75db4b95ae0561d9b74e57e18817cf2aefefeb238df9796c9d95a22797728ef14008209cc3223e7b33448df67642642ad385643903e51cdfab7c6a2cb863facca2ced922ca95fd1ec89006ab7ce9a57f4083569b3483c1ab8e63c56a71d04e39924b359ba669de2aae82c049c99469b64caf1e6b0adf2ea4d9aea1a4a170482c71539513d172291aba76dc9bf19922ff149130a278dbfbdcae75458de51c4a6203c9bf3ec8edf45736b939449d68b77f05eca917b8664b77f89e0f2dd843f2be5bfe06c37c7e7ce78b96d16c2888ab62d9b599468adaf465cba7611a0b32d6a7d4333c14b61de3f4bae014f78c0fde471ad8e3553db75f1f6ae681fdf6a702508077e0a7c66ab3c6278ab9e694518ed77e62502ab8a3dff86d324c4aef890476a8013cb32de018ac681adbee70ce87ea7096deb8f7346ec5dbeb77c0fb411911e9b44921462d96172c7da939a26af8a97192bc7982b9002fc5c370280c63e54fdb272c686ba0c99f0b62f5557367965c6fc8add35bb39d5efc3921c64d813204862166c896880e1716ca776bcff25f233a41b30bdcec80c8735c3931e35c071770b2abea98240b9a6a5b3b9d3d44ccfe966faf22548b424c2cb4f18762106bf0f4bba7f1bfcbb618bcc54f886de7d585c199b34fde2fdc4c68432457c0b56db58f1406447a3776a01e099106d21338488beb83c04cdc9b09c8531323fd112de6e17218321aba5da73f6fa981730f10d9a5c823e6313ae62759d5802d94e63b567b95eec225b57278582b57b434221cac0430da8a12fb0aaa8d889b1c716f82087d64eb2357f00053e788a0a826b938077718ba9af26a205e1888e00d051651e8139d4e455adcd12d9707ab3a31fc37cb050aa4eec6d18a7fc64e704deb78cc31136ee39b9380eeee4820f59d322636a7dd80eb378cc45b7a8587d2c8bfb97e7727c9c5c1180958e7bf66cb31c2832e6d92cbfeaea369350f6adcad2ce8bd0abf4592b95c061c0e76e3dc1722d98da4d869da7c7ef5629249566f45509ddeb410d21a7873529b0ef12deea3f28e1e7c3328349b0a7d30cb8238094cab4b35fdb481f893ce41145850919e25eeec40146812ad8d5ca693deda352a6aa794b5845a273b70b02edea4a2451fc3b67b00adf7bd4c4fb17b9701c9e5b8d3285fa8ad767f10ba9348773e41f50a73a8c0586d6230048340a639faec54a7a41354b76e3ed8c95c38c1b8d48a94b058dc290d8a5f348e0ba2c751b91b9c5c504e355c932f123d471500f849d183cd167ca4909d3ae706f8faf181f8caffc5971e17fe2805ad70295920b5a9d425263838bf70b46a9222cf7a61b51086dc6b593f6b3cd9c01c0b9eca50f4e2f9008b113cf6a74387448aa3f4ddea17f55e43a1c48434bdf777b6e9743481f8843008b1c9e9a4bfc3c748b5276ab41819fbb1eb64ffcfe5b8815cd1d578f267de0b5e7bdefeb694a003240bdb8f7e2ec08474d90eb95c4866ec08d596bc9d3a8fa7a029d92896c9d7205ec29347fdd9dd9939f81ede3ac2f895b350403c2412600cda3cfdba0415c57a268f97f0b94c3c42852703a600a8e0b1a093ced92e38f4bc73b2f26e020f17fc881e9fd72a0feafe480944ebcee4fca01e27a04c0c860ddfe7051bc7eb0a83d91c8d1adfb9dbff1e711afb420916c1a448168c857801663ebfd6640cdc2103ea137b53c3569c3d8290f49f778c7ee956ae23e9a77631718f70301036f1614747850728b2ccea69d31f7da90f2e649ffdf136172a4653f307ef37c7698048c667d1ab130a2d038e55cac055e5b966c0dafdc02f469b55cf2c4ce0979255defb21f7fcff3fe4d09ad01a8aae6e3c7cb3e8dbe92647b6beec6afd508d7aea6b5f9b5e80c8b8ebfb6a63a9fe19060533bfd9f58736ede03e59943dae5d4f5d2023b3bc4645f3ed569aa070c905a0f16d6c2f3357b01d618cae20eb22a3d78cfcc391345c93bc5341cd657f559adf99220d8b4e3d43e7b87425fe7cfa62fa0c12ca9156b8d0d98dbba63693e6942433225f4b7e23af4d2d9d3f09ee589fb0e851edbb7537d68feb9544df8825c8d58c2ea5d17eda9b7646f1972b912b4cd593d3daea4aa7fdd24171850060ec9717871e4d731a13fccd527f61535e6de66121077b25639db5434df9998eff74d13e47d72e531876f6bd159b2f913930ce6c1d5cd4b37d846a2ac6db891924a157d783f69d29e2630399826f8bbe4ff8f354109f9715213553d6355818a76497b47c63838ea5e780bf7a6acc0c8734c54f23d002a6496d48b75f832a1b166c472c2c331f30282ea665e66607afb910ee913fd65d5e065100ebc00385e3ff17c6653d5cf74b22d92fb26e1dbbbc76e64a8c06adadd027da92ba3ce7ae31da0141c38bdaa6bbd83f3e006a1bfa381ffb862acafb37151882a3a81d95db79a7fddf8dafb23d0f317ae425978529fd75241474e0214c14484665d32d0fec3332dd45c0587f3a1b26fc3535a9e5a3b5f6c77e10dc3ed42ee7c30c8ff0167c50f72494ff10537e788dcc75f4d89233b121ed0849324f42550e4197af1850a4a43f50038abce58ab3fdb97c36ed30123e501299fe6ebf2da57641bbd9b1dcb03099937aa4640eb830076c52e4344d96b9ef96ca6fb43274ce1cc74e33944eb0aa26068f0ac8b0f78136fc0d3d62377370a995f711432027846dafea4b6d8993d0379a682c943a9dee6f4c6a7fc49ced512f9ee8af75ccd148029733fcb51d39b14e82ee9b595c9de3ef74bf22a8e3708300c15fde8cd858d8fd584c6201ca10110620cb2dca07add2e1f089bf05473c54595d984131f2a80c8b71abffaf8b05b10efe74bc71142abc12238b050f0a189507b385a3a048c284ad097a4a0acbd6c12553ce855c37cc8189e5e7aa80a51d5e442079bfc1b18d97f6e1a9cf0ffe1657be0dd8fee6d91e55afe37e353fd0e04efac02fac9a94f4cad36da0659fcb6342bcbeed3324c371202b87a775843f50601db165de5649f4be414dc2f83bb798ffb7c8589bce20c73c435062f7e94af3b1ff42d6693ff86983dfa51675f8f6cfa87dff7b26608c02b66a237a8a21763730fbff1e4a07edfbdb3c16b9dea0a3fe7ce503094af258745da0090e5de0f61f54a56ada14cc96aaf06ecd59d079f9e66582c6e49ba37a8da1f4ef12d6653ff869845fc44cc88f9d93aedf088d92de0b4d2cc1d27ff88cc42bb64abecb0e8471e8f6a47f3087afafb7b1727ad32cd8d01f2d3495b1990cd378b4c02a26cbe16616acf95b86256a52763c96322ac75dce7aaecd66d39f02b77a22c6d0d3e838dbcb70dd36a1ea7c913a4385c6e34cb4737151de14bbcdb3bce02088ff0760d8b2d7bc35081bf853b16d7570ed2081ec58683bfa87d40bdd09775ac06278270e0388081bbbf3f5ee87295478fb88524da5eacd4e77e0955ef619a90d202c8d5c2bc28bc930b8003abfc210f30fbe4c1a3eebf92078a2f29197e4e1e10c8d58381a3626e23fdf14f1b261402da750767331a1ca2065f691dc73c5f3575ce545d4b02fc4ba7e706f5dc8695dcdedd0e7abff205414f5e1d298c72af3c26d2d1969fa2b29bf54d7f4dd6b7dbda2229b430599e9b12324445e545da2077348bdb569c4607e60760845acf00a7c6f1ecac116edeb5e2c910dfd4fb7e1b8ae9141a3b40d9329ad3cace2dbea01a920ec24152006e228e37d26766c7b91b5649635d7bc8a4d0f3062b264ca03b75cf08bb86efa3f106261307d1acfa0a595fc7f3ef89c2da7a6450cb7f527b29c7b632ddf7a0f71098e4f017687ee03607e585fc1f38125d241db744eeba6ee4396707dae590296a84d2d05b8e7ff22b1c4b22ce0eb39e05a4128e1bfa7e323d9673a572baa660bdfdf69965cbd8da159c981cff9df93922fa3dc1e5f26b05d711544c4b22cdc9e5b98756aa8c6a6b0dbea16941ff09c18584d2c4c66c57d20adc2aa917bcf4785bef50de2663c5cd2b0647e62bec3e138b02a61770236d5dce412e8564e853a3913df3dd102343bb1ad2b759d85287906d996a88300807110444eb4a9315a2e73e37f1075bf0713812703ebd328188437629b950da747e09c59f6767e3c0a37b18711e0c23cfd703fd3e335d12ddf7270e45d06b680a299e92fe1f28444500c69e4ed69878761ca577100512bed36e3630e59ddcab5eb53c245d2ef59cf82a20b3186f7a51aa8035f9f5b2b5a6522f6b475627ef73a9741706e15e969b719a6cdf7d215c07f285fcd9a4af070329c746938d422f67de7022d37f8498459f4f149ef50f71f92097c6547526895b1670999a96e9e36fe315d1379379894db6569ead7eb460763b52fa6d992fc2ef02e65b1bdf656c504bf6b4088f1ccb3c0a162b2bb465cc390cdd316fb821ee21b34c0fa1c27bca3cdcd7c667d85f51c5bf8dff0f26073e5ef870c7e0ba37b9a74ec9bd93e047493b472a050130456e64d4c360e34b77e55d8d000d4a61ddbe8b3af09a00356dea2f26d1e68a3c5fa508c68aeee4a189178edaebffbf000000ffff1ed9a0d101","nonce":"0x128cd","to":"0x8453100000000000000000000000000000000000","transactionIndex":"0x3b","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x41535904d7b20be8d733e03d95ec725d0c6ee4307d899cd00e0ca0de91533b13","s":"0x3f98c81207d078199bf74a600a77cfa389574646d63ff55ccfd194491faf7226","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xd3343d473f624c02045a01f4e7ab604bc56c4f42","gas":"0x156c0","gasPrice":"0x7d9f6fbd8","maxFeePerGas":"0xf49895cde","maxPriorityFeePerGas":"0xcfd6ae0","hash":"0x00b5b05c6d1a2eb8abe2c383da600516515e383fc8a29953bb6e6d167e9705b2","input":"0x9aaab64814636b3e36ce9c4227a5fb53273a04c3fdc94c8a00a4d6d5b6521571b94ec7ac00000000000000000000000000000000000000000000000000000000000028a416eb20a9e75d33ef7eeb4a0bad7413cea6c0b3a7f6ea249a93a428c0c36b52240000000000000000000000000000000000000000000000000000000000840244","nonce":"0x6c7","to":"0x1a8b038f5dff48f541e8cc658b9e186b9828a726","transactionIndex":"0x3c","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xfcd40d6ec6fa94f40f866c18457c22ebf5d8e3aaf2616c4c1cee5245ce8f042c","s":"0x45e5962efdda48e38f706cf8fb7c2f722d80da6e0a1d2589b6627627b44feb82","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x9aba7eeb134fa94dfe735205dda6ac6447d76f9b","gas":"0x3e64f","gasPrice":"0x7d9f6fbd8","maxFeePerGas":"0xf49895cde","maxPriorityFeePerGas":"0xcfd6ae0","hash":"0x6fc411f24c7b4b8d821b45de32b9edc5ac998d1ac748a98abe8e983c6f39fc19","input":"0x2e260ac30000000000000000000000000000000000000000000000000000000000000200a5c46e2dd70854a0fe8167b705eb7df71cc927469c345248c81b21a5ab2f5f19000000000000000000000000000000000000000000000000000000000000051d000000000000000000000000000000000000000000000000000000000000051e000000000000000000000000000000000000000000000000000000000084024700000000000000000000000000000000000000000000000000000001b3fd9f800000000000000000000000007ebc0d2451003f2d3bc5f6eafbb7e6bad81e250d0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000100000000000000000000000000000000000000000000000000000000640fdea40000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000002e43c000000000000000000000000000000000000000000000000000000000002e7d600000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000f6cb686532a3ece4a20f823e59cc386e1ba384d8504e902c932fa1497c4afdfb0b000000000008008402470000051e00000000000000082ffe82e7b18f0778493c989ecae1740325207ea29328999d5ae646723a4bb75a1305c8ca593670972f535ff8afda050351549ed800e9b98c805788acfe20e6ac076ae024efa64ac3ceb8f0a3cc9ce93d122f6faf8376c854da95567fb7576f762299ca2135608c7ac5a824257b50043526903fc8e01ad88006448a69a4afda4c01ad8a6ed1260582538f2311e902c1e7d5853a75b74482117f2cf920897c1b9428eaeb7e74a95d3d32db03931d157a48aaca9748bfebfecffc2d140caddc66b300000000000000000000","nonce":"0x108af","to":"0x7ebc0d2451003f2d3bc5f6eafbb7e6bad81e250d","transactionIndex":"0x3d","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x1b284cf86146fa2094c7c709cf9fcb8a177f4fde1d1c5ca112e41f82a5e973d2","s":"0x5618261c40d23e5beb7ac0bac77b16ead56e547f103f649462387c5acdf18ba5","yParity":"0x0"}],"transactionsRoot":"0x1ad3212eca045505cfc4cacf675b5fa2e7dc7b9f9cee88191464f97d1c9fbca4","uncles":[]} diff --git a/op-service/sources/testdata/data/blocks/pre-shanghai-bad-receipts_metadata.json b/op-service/sources/testdata/data/blocks/pre-shanghai-bad-receipts_metadata.json new file mode 100644 index 000000000000..7deed556e8b2 --- /dev/null +++ b/op-service/sources/testdata/data/blocks/pre-shanghai-bad-receipts_metadata.json @@ -0,0 +1 @@ +{"name":"pre-shanghai-bad-receipts","fail":true} diff --git a/op-service/sources/testdata/data/blocks/pre-shanghai-bad-transactions_data.json b/op-service/sources/testdata/data/blocks/pre-shanghai-bad-transactions_data.json new file mode 100644 index 000000000000..e1371c9118c1 --- /dev/null +++ b/op-service/sources/testdata/data/blocks/pre-shanghai-bad-transactions_data.json @@ -0,0 +1 @@ +{"baseFeePerGas":"0x7ccf990f8","difficulty":"0x0","extraData":"0xd883010b02846765746888676f312e32302e31856c696e7578","gasLimit":"0x1c9c380","gasUsed":"0xa79638","hash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","logsBloom":"0xb034000008010014411408c080a0018440087220211154100005a1388807241142a2504080034a00111212a47f05008520200000280202a12800538cc06488486a0141989c7800c0c848011f02249661800e08449145b040a252d18082c009000641004052c80102000804ac10901c24032000980010438a01e50a90a0d8008c138c21204040000b20425000833041028000148124c2012d0aa8d1d0548301808228002015184090000224021040d68220100210220480420308455c382a40020130dc42502986080600000115034c0401c81828490410308005610048026b822e10b4228071ba00bdd20140621b2000c02012300808084181ac308200000011","miner":"0x0000000000000000000000000000000000000000","mixHash":"0x31f0c0305fc07a93b1a33da339c79aadbe8d9811c78d2b514cd18d64e1328f25","nonce":"0x0000000000000000","number":"0x840249","parentHash":"0x2303b55af4add799b19275a491b150c1a03075395f87a7856a4e3327595ed7df","receiptsRoot":"0x99da71b17ae1929db912c3315ebe349d37f2bb600454616fdde0ee90d6dbc59e","sha3Uncles":"0x1dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347","size":"0xea6d","stateRoot":"0xd12bf4cf3941cf48be329a939b13d3403d326841c69cdcc9a9c13ab2f227e904","timestamp":"0x640fdeb0","totalDifficulty":"0xa4a470","transactions":[{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x1b7aa44088a0ea95bdc65fef6e5071e946bf7d8f","gas":"0x5208","gasPrice":"0x174876e800","hash":"0x39c666d9b5cec429accad7b0f94f789ca2ebeb5294b8b129c1b76f552daf57d3","input":"0x","nonce":"0x26bd1","to":"0x68643ada5885f14e7c15c13bee06168eff0f7c36","transactionIndex":"0x0","value":"0xb1a2bc2ec50000","type":"0x0","chainId":"0x5","v":"0x2e","r":"0x26e648625520c256d87d8e15769e8c4290d2a7e27147594815da42070870825b","s":"0x7db34c91603e51494b59b020597e7118433d50ed548bc7efad8880cbe7820ca"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x030b1cddf635e9e71ad70b8668e235e8ec3c67c4","gas":"0xf618","gasPrice":"0x1701ca2b9d","hash":"0x2ca7289ab3738d17e0f5093bd96c97c06c9a2ea4c22fc84a6a7fbfda93ce55ee","input":"0x","nonce":"0x456d","to":"0x89416096a0a1acad3436a357478c5c548e4d382b","transactionIndex":"0x1","value":"0x16345785d8a0000","type":"0x0","chainId":"0x5","v":"0x2e","r":"0x5aa6ad410ea42dbff3781de67b4815a376cd459206595fa6655c966a7709e5cb","s":"0x82f620f8dbc356dc38ef346d1e152656a1d318becbcc41884f61e85972355c0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x9c67435ce8a33ad23f377ddf08260ffc56417f21","gas":"0xf618","gasPrice":"0x1701ca2b9d","hash":"0xb0085de1476530de3efc6928c4683e7c40f8fac18875f74cbcc47df159de17d9","input":"0x","nonce":"0x446c","to":"0x471f3a988a77472e971112e7ff2ce65996b6c8cd","transactionIndex":"0x2","value":"0x2c68af0bb140000","type":"0x0","chainId":"0x5","v":"0x2d","r":"0x75425f21a6e66e6feaee79455c897656c4a94b6189c4c033facab1b67812fc3f","s":"0x7014f4b9f760165c7757f9d6dccea349b3d00b0a9c2f845af8010140842f826"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x168f6dec26cbbb3749654e0e3cc4fc29314fdf6c","gas":"0xf618","gasPrice":"0x1701ca2b9d","hash":"0xe01c8631c86ded63af95b8dbc0c8aac5d31254c14d6ecb4cc51d98259d838e52","input":"0x","nonce":"0x403c","to":"0xf2119a49a3ae35453c1c2704da248d08977084f1","transactionIndex":"0x3","value":"0x16345785d8a0000","type":"0x0","chainId":"0x5","v":"0x2e","r":"0x56bba57ee69c09dbbd5167d0fee07d5803130fea7c278f4d8b7bc54b34be8ff9","s":"0x7eea69c0de47b6b808ea1bdd9ecf5d25f4f2743e867b1f838db59c456026d7e6"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xc9af69904b4ba7f0fb69ab1a8719bd8950265e73","gas":"0xf618","gasPrice":"0x1701ca2b9d","hash":"0x69414a126a6f07ab5e31ad2f9069fb986b7c490e096898473873e41ece6af783","input":"0x","nonce":"0x44c20","to":"0x6adc3f5394e00cd067d1fad5a33756791e303c25","transactionIndex":"0x4","value":"0x16345785d8a0000","type":"0x0","chainId":"0x5","v":"0x2e","r":"0xdc9a52b87755cd07143bf032d57258d0d6c36500b38a574a9a5addc1293343da","s":"0x581e6fd051d0c72d64303c1f90edecfef46580f84f24dba930cde910923f7c21"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x92964e63cd52450e6e09a266e0c226e524754563","gas":"0x2625a0","gasPrice":"0xf5686c7be","hash":"0xa2fef1133ee726533c7f190f246fede123e3706a03933c1febc92618f90d2804","input":"0xa9059cbb00000000000000000000000011f7db5a824eaa5737c935ad09a925dd707f092b0000000000000000000000000000000000000000000000008ac7230489e80000","nonce":"0x2636","to":"0xbc071c64ed8f536011c78e847755680740d6b73c","transactionIndex":"0x5","value":"0x0","type":"0x0","chainId":"0x5","v":"0x2d","r":"0x422c5026584cd8fb8a1fe66e6edc7a020bc513cc3114083fb35f9846cd9a9792","s":"0x7fcd4e48ac5b53d881531a5651e587188157ed88d85c51533f7c6eda94c67ff7"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xdec1bc71bf91431d60ef2742f412dcd1c5a204b8","gas":"0x7a120","gasPrice":"0xb80e515ce","hash":"0x6585ec5c4c2bbf1f683f90f58e18f3b38d875e94457fe4cbb7bc5bf6581f83af","input":"0x67a5cd060000000000000000000000006343e96c99e2725e845ba04f90e03d853cc9bb2d","nonce":"0x6a5a4","to":"0xf5de760f2e916647fd766b4ad9e85ff943ce3a2b","transactionIndex":"0x6","value":"0x0","type":"0x0","chainId":"0x5","v":"0x2d","r":"0xcb7b32949ac0dedd3151d06d85ec4e99483c8ea706f12a009d97b055975241eb","s":"0x69f0009bce12e24408f9a3d5ad5a8578e2b23efb09002f7b86b3425cf8748889"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xdb954dd5d224004b26cfbb089312e0542e5d1c31","gas":"0xd5ef","gasPrice":"0x9fc748df1","maxFeePerGas":"0x116482c049","maxPriorityFeePerGas":"0x22f7afcf9","hash":"0x1db276b864fbf01dcf8cededf8d597553ecb0eb9438edfaf2f5bd0cc93297c66","input":"0xa3e35f36000000000000000000000000f18f2320e4779f21d6882d83f9133e582dfaa9b6","nonce":"0x570","to":"0x11672c0bbff498c72bc2200f42461c0414855042","transactionIndex":"0x7","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xf23a1fb0d17cdbc2831a2cb0b779795e12533e34425a3abbc4b7e7bde262b92d","s":"0xd4a2c8189378696065b7fc5d13e9cf6595d96047a31a4fcb812c2e677d22a26","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xe84d601e5d945031129a83e5602be0cc7f182cf3","gas":"0x249f0","gasPrice":"0x9502f9000","hash":"0xcbe7ed31654af4e191ca53445b82de040ae2cd92459a3f951bdcce423d780f08","input":"0xa9059cbb000000000000000000000000355f96e3261cc8237d2d36fd81cc9b9cdcbfc5ea0000000000000000000000000000000000000000000000000de0b6b3a7640000","nonce":"0x4f1f0","to":"0x499d11e0b6eac7c0593d8fb292dcbbf815fb29ae","transactionIndex":"0x8","value":"0x0","type":"0x0","chainId":"0x5","v":"0x2e","r":"0xadf7dce5f34d482bb19ff0c85f43be7008cb838043f9abb5fc08421245ac4228","s":"0x5d2c0f0b9d2e68bb8da6084e4971d57bd76bf0cd9d893c9c985962812d2df026"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xea0193f88a0d9c36e83e6fb6d382212b6373faba","gas":"0x43238","gasPrice":"0x933ea77d8","hash":"0x808ba5211f03cc78a732ff0f9383c6355e63c83ae8c6035ced2ba6f7c331dc63","input":"0x9f8420b300000000000000000000000000000000000000000000000000470de4df8200000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x0","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0x9","value":"0x470e9f23394000","type":"0x0","chainId":"0x5","v":"0x2e","r":"0x9536f87e8c3433ea23ef7211c0c32689551906dfdf31bcc73b89fcf18fe86a92","s":"0x3e94bfe3b4ba32a6ce71ee6e0ff9f26dfc2b0d61b5c0fe049469e3c4b88ac227"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x1f9ab300b380313e6a2208e44c01a0602538385d","gas":"0x32b38","gasPrice":"0x87fc9eef8","maxFeePerGas":"0x2e90edd000","maxPriorityFeePerGas":"0xb2d05e00","hash":"0xdd66f1f26672849ef54c420210f479c9f0c46924d8e9f7b210981ffe8d3fac82","input":"0x32db5470000000000000000000000000000000000000000000000000000000000084021000000000000000000000000000000000000000000000085b0af981975a40fc5a00000000000000000000000000000000000000000000053ae1f470262f3b01c4000000000000000000000000000000000000000000000813bca9f8389ffd193f","nonce":"0x5699","to":"0x28cea7b0f3916c1dba667d3d58ec4836ad843c49","transactionIndex":"0xa","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xbe8eddd463a3518a4c94b5922c0f2fb26a32c8afcd819f7deaf4f0416757ad13","s":"0x3e234182acd5261dfba3cf4f125d1a8377fc707b3b1900b565d1a6b8987b398c","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x1d990a2cf61b46f37582dd31a7b242e989e1a1ee","gas":"0x493e0","gasPrice":"0x86f96eddb","hash":"0x254abb2f8cdcffe9ef62ab924312a1e4142578db87e4f7c199fd35991e92f014","input":"0x9f8420b3000000000000000000000000000000000000000000000000002386f26fc100000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x0","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0xb","value":"0x2387acb3784000","type":"0x0","chainId":"0x5","v":"0x2e","r":"0xbfe9b99fd2e9bbd2d65bb4cc17da88130d8f6b569a26afa642ed1cc8efd2e2ef","s":"0x7003dd3962eec2369bc5abe5e3d5b73a9086a3219643b1741f3a9832369448fa"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x1124bbb196fd7b61947f6f03b7d7cebde8340b6f","gas":"0x493e0","gasPrice":"0x86f96eddb","hash":"0xa7b7c654e7073b8043b680b7ffc95d3f2099abaa0b0578d6f954a2a7c99404e1","input":"0x9f8420b3000000000000000000000000000000000000000000000000002386f26fc100000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x0","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0xc","value":"0x2387acb3784000","type":"0x0","chainId":"0x5","v":"0x2e","r":"0xe8dae9efa8fa73a3aebdaa75ba12acf56712b1f99fc128ab802a7610d014547b","s":"0x2ef347ea90a0db21b035637342bfc916c9962d3ec610a4e99cfaf555a2606fe2"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xdedf52a88fe47f6a11d629badd0096d201aef86a","gas":"0x493e0","gasPrice":"0x86f96eddb","hash":"0x7ccdfa698c8acf47ab9316ed078eb40819ff575bcf612c6f59f29e7726df3f96","input":"0x9f8420b3000000000000000000000000000000000000000000000000000003a3529440000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x1","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0xd","value":"0x45d964b8000","type":"0x0","chainId":"0x5","v":"0x2d","r":"0xfc4e092094abc7fed9cf11eea8ad6ec3a760d13b9c603b5570c2143e48090f2f","s":"0x31a0a6016ab51ed6375891bf532d9031bf91c1d0e01df63cf7e46076aeee83a3"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xe1db1fd154b344ce7dfcb53a65279559b298f0df","gas":"0x493e0","gasPrice":"0x86f96eddb","hash":"0xa0b035ef315824a6f6a6565fa8de27042ade3af9cf0583a36dea83d6e01bf2a8","input":"0x9f8420b3000000000000000000000000000000000000000000000000002386f26fc100000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x0","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0xe","value":"0x2387acb3784000","type":"0x0","chainId":"0x5","v":"0x2d","r":"0x79c316492bfa36fa4de2dca02ffac77f6c1772714e153aad8c947ce848d4106f","s":"0x4fe3e083943ca8f974cb0752a2ffb70e4d6c8b615e16ece7e19b2c464be66929"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xd4c3777e1821f6fb532957104e1317a9d3f881ec","gas":"0x493e0","gasPrice":"0x86f96eddb","hash":"0x1ebad7f3e8cb3543d4963686a94d99f61839f666831eab9c9c1b4711de11d3d9","input":"0x9f8420b3000000000000000000000000000000000000000000000000000003a3529440000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x1","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0xf","value":"0x45d964b8000","type":"0x0","chainId":"0x5","v":"0x2d","r":"0x5c1f544ca92b28a014129a9d04f8e895dc8a6987ddc1717c26b0619bb9bac420","s":"0x47574643bd92a0f129f9c5ce3caebe7943cfd828d67fb0e5bdc1eb2ca202822a"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x127418955b15f5ff2fa232431a798b53fd006c82","gas":"0x493e0","gasPrice":"0x86f96eddb","hash":"0x501750278e91d8b5be1ccf60e793d4bbcd9b3bb3ccc518d3634a71caeac65f48","input":"0x9f8420b3000000000000000000000000000000000000000000000000000003a3529440000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x1","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0x10","value":"0x45d964b8000","type":"0x0","chainId":"0x5","v":"0x2d","r":"0xcfc9894bf716ba707521f0969082afb688b7d47cc21f1423b239c40867128f47","s":"0x5d9b9c66d8437d0a37768fb9c3241ef64b7e795d0d908196e1f707215ac1f56"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xc2ac1bdcb3dd77864b99eb2cafef2ed36c1487c3","gas":"0x493e0","gasPrice":"0x86f96eddb","hash":"0xd80ff8af29ae163d5811ba511e60b3a87a279f677bb3872a0f1aa6d0a226e880","input":"0x9f8420b3000000000000000000000000000000000000000000000000002386f26fc100000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x0","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0x11","value":"0x2387acb3784000","type":"0x0","chainId":"0x5","v":"0x2e","r":"0x8bd7220d176d18cfbc824063448cf864c494fcf0958dec82126f26d3fc6c0b6e","s":"0x2f4de3d2fc3be6d2812ee10aee16a995af5affcfec27f9953ae609886d2b563e"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x11a5ad000f8494233df5bd2f2573fe20bad744ab","gas":"0x493e0","gasPrice":"0x86f96eddb","hash":"0x096acab3b3fe47b149d375782d1eb00b9fef7904076d60c54b3c197b04e6bf82","input":"0x9f8420b3000000000000000000000000000000000000000000000000000003a3529440000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x1","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0x12","value":"0x45d964b8000","type":"0x0","chainId":"0x5","v":"0x2d","r":"0x43726c3165958e94d80cb6992121f55df2904a1de4356e0a6ff3d8bfd03c0928","s":"0x6244361e31daccbb5c7e9f3f7d01004080d2efa314d614fdc2aad4c1b7d71613"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x90f8b290ec2929cce086826e971cad3857c6dbb5","gas":"0x50f46","gasPrice":"0x861fc89f8","maxFeePerGas":"0xe22924cba","maxPriorityFeePerGas":"0x9502f900","hash":"0xbe9d1738af74a22400591a9a808fb01a25ab41e2e56f202dd7251eb113e8ceeb","input":"0x6ab150710000000000000000000000000000000000000000000000001bc16d674ec800000000000000000000000000000000000000000000000000000de0b6b3a7640000","nonce":"0xe4","to":"0x2e84c9894f628fd209f48b06d10b46c8cada4c10","transactionIndex":"0x13","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x412c7bfb0236a4317da40645ee4e60f19185e7d0cd2cc9bc88f7629de505b939","s":"0x4f6bd485d10fa134926ec3e3a83e212f3c5d903ba52da87f07794a2ba0a5c1eb","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x7915ec065b644568155c4772a286addad3864c1b","gas":"0xb71b00","gasPrice":"0x861fc89f8","maxFeePerGas":"0x1176592e000","maxPriorityFeePerGas":"0x9502f900","hash":"0x0834c720e55cccd97aaf4f8fb0cb66afb9881fb6a762c0f70473ec53f98a712e","input":"0xedce8b4d6a1fb55588eadfd3a0c3350435a31d31e7c236125eafcc2a892d50711d9a4939a4282888c956c070cd83400fd47ab83bd5287244053b8228711b07386dccacbd000000000000000000000000000000000000000000000000000000000000006000000000000000000000000000000000000000000000000000000000000001a0000000000000000000000000000000000000000000000000000000000000082000000000000000000000000000000000000000000000000000000000000008a0000000000000000000000000000000000000000000000000000000000000092000888ad4975d4079d80bc6da5ded57746f889644719ec160efc2e4a96ad645df10b1683afcd7ce41ba6c06528761f821cbc6e806ef4b71c189603ec218b12a1f1478883498e67b97ae4babef6996acc582db1377d8755374602e00b8148025ac2d2f52e78d7484919cef14bf96e5b7a3f7f8cf09989551888f2c697052ce2bdb02a9296b20016c78683a9ca137037e57c8ff25198146f9d4cdb5a25ab9c95a03038f22f83c8604ca7c035ef756d2963c3657eea4ffb18ef967752d8cff15a16124a1d92a1cda613624c98e3078742803da1417b78bb264b7bb2197515cb4facc2ea395c6241664dbd953d0034900a86eb94897bf89aae775a1d73e6fd034785400000000000000000000000000000000000000000000000000000000000009a0000000000000000000000000000000000000000000000000000000000000000300000000000000000000000000000000000000000000000000000000000000600000000000000000000000000000000000000000000000000000000000000260000000000000000000000000000000000000000000000000000000000000046000000000000000000000000000000000000000000000000000000000000001d802f901d482e7041b846de27f7b846de27f8383026c28946aa397cab00a2a40025dbf839a83f16d5ec7c1eb841dcd6500b901a45ae401dc00000000000000000000000000000000000000000000000000000000640fdf5900000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000000000000000000001000000000000000000000000000000000000000000000000000000000000002000000000000000000000000000000000000000000000000000000000000000e404e45aaf0000000000000000000000002c1b868d6596a18e32e61b901e4060c872647b6c000000000000000000000000964ff70695da981027c81020b1c58d833d49a64000000000000000000000000000000000000000000000000000000000000027100000000000000000000000009a1a41127726f74560a57990fb928120eed6f55e000000000000000000000000000000000000000000000000000000001dcd6500000000000000000000000000000002d4766adcf72ef2d996edfadc866be12c28000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000c0000000000000000000000000000000000000000000000000000000000000000000000000000001d802f901d482e7041c846a969541846a96954983026c28946aa397cab00a2a40025dbf839a83f16d5ec7c1eb841dcd6500b901a45ae401dc00000000000000000000000000000000000000000000000000000000640fdf5a00000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000000000000000000001000000000000000000000000000000000000000000000000000000000000002000000000000000000000000000000000000000000000000000000000000000e404e45aaf0000000000000000000000002c1b868d6596a18e32e61b901e4060c872647b6c000000000000000000000000964ff70695da981027c81020b1c58d833d49a64000000000000000000000000000000000000000000000000000000000000027100000000000000000000000009a1a41127726f74560a57990fb928120eed6f55e000000000000000000000000000000000000000000000000000000001dcd6500000000000000000000000000000002d4766adcf72ef2d996edfadc866be12c28000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000c0000000000000000000000000000000000000000000000000000000000000000000000000000001db02f901d782e70480846a969541846a9695498302e100946aa397cab00a2a40025dbf839a83f16d5ec7c1eb870aa87bee538000b901a45ae401dc00000000000000000000000000000000000000000000000000000000640fdf8d00000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000000000000000000001000000000000000000000000000000000000000000000000000000000000002000000000000000000000000000000000000000000000000000000000000000e404e45aaf0000000000000000000000002c1b868d6596a18e32e61b901e4060c872647b6c000000000000000000000000964ff70695da981027c81020b1c58d833d49a6400000000000000000000000000000000000000000000000000000000000000bb800000000000000000000000048f2c17f8885e135c8689626d5841d0b94013556000000000000000000000000000000000000000000000000000aa87bee5380000000000000000000000000017a8db2711fe83b10bbfabb6a8b86c3e26ca154ef000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000c000000000000000000000000000000000000000000000000000000000000000000000000003000000000000000000000000000000000000000000000000000000000000001b000000000000000000000000000000000000000000000000000000000000001b000000000000000000000000000000000000000000000000000000000000001c0000000000000000000000000000000000000000000000000000000000000003760f4e19e59be10b323fe1b42e42de2754c5da54ff316cef74ef09dbc0144bbf73e609a8d21a38f393bae9a3de6bebbf362608d907d941c8166f44d8ee2c6686b5dbbd6681f5467219d9bb530db013b7c067db659c03efe935caeb7bab36699d00000000000000000000000000000000000000000000000000000000000000032d2d7436bd45b5e81824945ff3dda846e270c81e72929841f4b14f56e02eef39477f9c162a2759c8d7c7e1feefdd064f6aefb4360d483f70671f36fc11ccc6db6c53f74e3019c12d859545ad75b51998b2238887dc0be3c78522c8356150526a0000000000000000000000000000000000000000000000000000000000000000","nonce":"0x48f08","to":"0xe87d317eb8dcc9afe24d9f63d6c760e52bc18a40","transactionIndex":"0x14","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x54159a63b8ce19097cb8afeaffc84cfb5257f9709dcb0f0a258befa0b924b13","s":"0x17f5ef5aeceb6f082a8a0f8fa28e55d453c52e5f86d48dfb4d4f039aad7fa493","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x52db522edf50fc32b8fc0f77d51140031da11da3","gas":"0x201568","gasPrice":"0x861fc89f8","maxFeePerGas":"0xe22924cba","maxPriorityFeePerGas":"0x9502f900","hash":"0x51a0c33c9b37245b416575bdd2751c0d8a5d8bead49585ac427bfc873d4016af","input":"0x96d3f83c0000000000000000000000008cd194f4ced354a14960f76f49305feb8510f73f00000000000000000000000000000000000000000000000000000000000000e000000000000000000000000000000000000000000000000000000000641024ef0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000001b3fd330d479974ad52e18b8767d9f18cf9ff024fa749b1066a803b237570cb73e2868e821124c8128505d883124711ae7fdca0339f4f5cd5dadd8eb6956eb4af500000000000000000000000000000000000000000000000000000000000000040000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000100000000000000000000000000000000000000000000000000000000000000020000000000000000000000000000000000000000000000000000000000000003","nonce":"0x2d9","to":"0x29c1442d7e3b6ed179ceb55fe996e1f4384880da","transactionIndex":"0x15","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xac84f403fa0347be3246ad695f8f6a780332357d49dfa67388fae3e051e5566c","s":"0xac2e148c1045af0ebdb334f20743af7d5499558af5eea43393159ce26a3fb1b","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x3aea5f857b75a946d5f5b329bdcf4db46aec5d2c","gas":"0x50f46","gasPrice":"0x861fc89f8","maxFeePerGas":"0xe22924cba","maxPriorityFeePerGas":"0x9502f900","hash":"0x531c25d51ccda59aa9ea82e85c99be9dd4e285af9b8973cbab9ac4a38e26e55a","input":"0x6ab150710000000000000000000000000000000000000000000000000de0b6b3a76400000000000000000000000000000000000000000000000000000de0b6b3a7640000","nonce":"0x1d","to":"0x6c23a4586cc791d2be6767532e12264992aef74a","transactionIndex":"0x16","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x81abc4fdf3b57fdf3a951fb293853fdd86aadce8f319576ac78c60caac186c43","s":"0x452742335bd1f63fd623d5e6cf91f6339361f863dba1a57fc3c1da8292e82c77","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x7915ec065b644568155c4772a286addad3864c1b","gas":"0xb71b00","gasPrice":"0x861fc89f8","maxFeePerGas":"0x1176592e000","maxPriorityFeePerGas":"0x9502f900","hash":"0x93ac6c08d21cb1b61ff59e5e2d6fa3f9ad54008b0a66c669199050bef219f6e3","input":"0xedce8b4da4282888c956c070cd83400fd47ab83bd5287244053b8228711b07386dccacbdb54cb66757631f9f22e2faf7da0eecc43474ab1a1af21aec22dcf949e2579241000000000000000000000000000000000000000000000000000000000000006000000000000000000000000000000000000000000000000000000000000001a00000000000000000000000000000000000000000000000000000000000000860000000000000000000000000000000000000000000000000000000000000092000000000000000000000000000000000000000000000000000000000000009e011cb7bbabfab668b42b5582cec8d60f326ae14ad55f8d675060dd111dee6c88d0937435261929ba3a881bea107593a10e13c917a0ef5ae4e8bde40de2d1da4ba25ab3a239ba53eb4e260d23b98a7db944f644f40ab75992a89fad40e8d692ad62f8671604db415b54238c5522f6c4e6b25785bffa696d173659309b95a2fae8700d4b9c14382973ea34a3cd8b8428594908dcad2d5d7a01056b8e49a8c25087f12f9e9fec1f4b5fdc57509dba6839100e28016b2e3fec4928bbe11d3789b6cab280c17bebfa5a0e06aea22764e1a18d2d11316bdf81028e65755582586ebdf571cbd0ef59800ba3405539187dfc03eac1dba0cf590e2131a56ef7f260aa38f940000000000000000000000000000000000000000000000000000000000000aa0000000000000000000000000000000000000000000000000000000000000000500000000000000000000000000000000000000000000000000000000000000a000000000000000000000000000000000000000000000000000000000000001a00000000000000000000000000000000000000000000000000000000000000200000000000000000000000000000000000000000000000000000000000000032000000000000000000000000000000000000000000000000000000000000005c000000000000000000000000000000000000000000000000000000000000000da02f8d782e704820101846a969541846a96954982dc1994a59477f7742ba7d51bb1e487a8540ab339d6801d87470de4df820000b8a4220b5b82000000000000000000000000f6213f16f3db9b50c684b2da37ff080895ae4a02000000000000000000000000000000000000000000000000002386f26fc100000000000000000000000000000000000000000000000000000000000064376b6900000000000000000000000000000000000000000000000000000000000000800000000000000000000000000000000000000000000000000000000000000000c0000000000000000000000000000000000000000000000000000000000000000000000000003202f082e70411846de27f7b846de27f8382520894b6605a717a2447edac46b1cdafe35e3e91130c8d87038d7ea4c6800080c0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000f202f8ef82e7040d846de27f7b846de27f838303335f947191061d5d4c60f598214cc6913502184baddf1880b8c44d49e87d00000000000000000000000000000000000000000000000000000000000000600000000000000000000000000000000000000000000000000160f9854c9f595e000000000000000000000000000000000000000000000000000000006417d4b8000000000000000000000000000000000000000000000000000000000000000200000000000000000000000000000000000000000000000000b1a2bc2ec5000000000000000000000000000000000000000000000000000000b2f4f1df2e372ec00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000027502f9027182e7048197846a969541846a96954983029eb1946aa397cab00a2a40025dbf839a83f16d5ec7c1eb80b902445ae401dc00000000000000000000000000000000000000000000000000000000640fdf0b000000000000000000000000000000000000000000000000000000000000004000000000000000000000000000000000000000000000000000000000000000020000000000000000000000000000000000000000000000000000000000000040000000000000000000000000000000000000000000000000000000000000016000000000000000000000000000000000000000000000000000000000000000e404e45aaf000000000000000000000000964ff70695da981027c81020b1c58d833d49a6400000000000000000000000002c1b868d6596a18e32e61b901e4060c872647b6c00000000000000000000000000000000000000000000000000000000000001f400000000000000000000000000000000000000000000000000000000000000020000000000000000000000000000000000c097ce7bc90715b34b9f10000000000000000000000000000000000000000000000000000000000000000000000614000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000004449404b7c0000000000000000000000000000000000000000000000000000000000000614000000000000000000000000694b3f194e94c80b2d7a0e986e1440f204f0503f00000000000000000000000000000000000000000000000000000000c0000000000000000000000000000000000000000000000000000000000000000000000000000000000000b902f8b682e70427846a969541846a969549830dbba094964ff70695da981027c81020b1c58d833d49a640872aa1efb94e0000b884437471fd0000000000000000000000004e44260732136caeffc7d076e17b2a3554b9ce2a000000000000000000000000000000000000000000000000002aa1efb94e00000000000000000000000000000000000000000000000000000000000064376b6600000000000000000000000000000000000000000000000000000000000dbba0c0000000000000000000000000000000000000000000000000000000000000000000000000000005000000000000000000000000000000000000000000000000000000000000001b000000000000000000000000000000000000000000000000000000000000001b000000000000000000000000000000000000000000000000000000000000001b000000000000000000000000000000000000000000000000000000000000001b000000000000000000000000000000000000000000000000000000000000001b0000000000000000000000000000000000000000000000000000000000000005b0efccaa7c845eddd4fb98211c7a58a37abdf06b264fade6cbff8a73ed22fccc602337b33a81e2cc4127be9d913fb3aec24194e14a61515a26429a00cfb86196aa3c2e6983f1a05dec045f0e63db7bd8619cd2a770ed1f7f470eb7c297e8e19d61277fb276ed8a63fd666b4a5ab02347065a3c16539f4ccccbed8fa39f2a7dc6606fc04c49f3d69463ba1c86c6ddcef8b9f06b9741dc2ebae9f5f1c0523b320b00000000000000000000000000000000000000000000000000000000000000050668b9ca5e5b17e2e79f9f5a7d9ea27e4e7f0302e878584fef3c2304c70c1ef53da311342341f89df488646b97a83994a3d07cc622ffde98598548c5326b7ccc0cf77d5d6fca6cf3f5b5b41a266fa2aaf697fd976dddc710fca834c96d2dfd725eec5980733db3c4e8f779f9fde3a598b2860d819419c961012825f0e160ee7b220989005b6ac5c48734db18158c2d85c8499bea1972c9f875c2d2aeafccb33700000000000000000000000000000000000000000000000000000000000000020000000000000000000000000000000000000000000000000000000000000040000000000000000000000000000000000000000000000000000000000000014000000000000000000000000000000000000000000000000000000000000000e0000000000000000000000000f6213f16f3db9b50c684b2da37ff080895ae4a02000000000000000000000000f6213f16f3db9b50c684b2da37ff080895ae4a02000000000000000000000000000000000000000000000000002386f26fc10000000000000000000000000000000000000000000000000000002386f26fc100000000000000000000000000000000000000000000000000000000000064376b6900000000000000000000000000000000000000000000000000000000000000c000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000140000000000000000000000000964ff70695da981027c81020b1c58d833d49a64000000000000000000000000073fee82ba7f6b98d27bcdc2befc1d3f6597fb02d000000000000000000000000000000000000000000000000002aa1efb94e000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000064376b6600000000000000000000000000000000000000000000000000000000000000c00000000000000000000000000000000000000000000000000000000000000044a1c058fa0000000000000000000000004e44260732136caeffc7d076e17b2a3554b9ce2a00000000000000000000000000000000000000000000000000000000000dbba000000000000000000000000000000000000000000000000000000000","nonce":"0x48f09","to":"0xe87d317eb8dcc9afe24d9f63d6c760e52bc18a40","transactionIndex":"0x17","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0xb5f185933d49064643686594f44a2584b1d57ba6a91f2a45d1a122892488f209","s":"0x7bb4af7ea30290b36fa7c266ee2387eff9e3ee0cd79b48aed27f5bfaaf34114c","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x4281ecf07378ee595c564a59048801330f3084ee","gas":"0xea60","gasPrice":"0x861fc89f8","maxFeePerGas":"0xfd18eeafe","maxPriorityFeePerGas":"0x9502f900","hash":"0x3792db6dd6285f409e4281951e9f78dad16c4a78072ff1c909dfadea5658d857","input":"0xa9059cbb0000000000000000000000000cc351a97c2fafde4cfa8a00c4fd79881cf13b53000000000000000000000000000000000000000000000001158e460913d00000","nonce":"0x77ffb2","to":"0x326c977e6efc84e512bb9c30f76e30c160ed06fb","transactionIndex":"0x18","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0xef4173dd45060791cfb6dfc3cdc8a9b417ff6a4b3ba57759ea91fd875b01a522","s":"0x4de926b5099c7d60cc24add3c2eecd8e9550c1fe7f878430eccb887a2ff932ff","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x53e2c2479833ca17f946aeb16dcc49f5e2194701","gas":"0x431eb","gasPrice":"0x8442f24f8","maxFeePerGas":"0xd6aac8818","maxPriorityFeePerGas":"0x77359400","hash":"0xd2d51764c01e8c0a43fbe362704388df5bacf7e5e620c3864e242530ffb3e828","input":"0x9f8420b300000000000000000000000000000000000000000000000000470de4df8200000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x2","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0x19","value":"0x470e9f23394000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x155352758fb04a6c0816c3dd9ca16bd6c66c37e2e19f2aa5d969225e3cb7f118","s":"0x6a4a1237db75dac4322fe12aee8b94edf7083ea661d6e93334c667d2d5c2c101","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x535619cf2bd2e84e567bda1a6164f195cb2bba21","gas":"0x55730","gasPrice":"0x83152e5b0","hash":"0x516b0227d9e64eb6e0de6862764d40f5376b5f12fec878436fea3479b4c36bb8","input":"0xa9059cbb00000000000000000000000030509945d2b329cc253a8bb2e2c54bdbb08b1070000000000000000000000000000000000000000000000010ce1d3d8cb3180000","nonce":"0x2176","to":"0x1173069b0e472a64ce2b6203fec8fb25a05b75c2","transactionIndex":"0x1a","value":"0x0","type":"0x0","chainId":"0x5","v":"0x2e","r":"0x289ee9948b80fc623b8c5d85c7df6fd25786fcc118163042b77a00560f53c3fb","s":"0x5b0a79e30d05370b27664b2a10cf964e110387bd62098561b6ea6a9dcc4634a"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x339d413ccefd986b1b3647a9cfa9cbbe70a30749","gas":"0x180ac","gasPrice":"0x83152e5af","hash":"0x81b0abc78b82840adb666775b182a9e292f663b64bcd35004c04436ed3c8281c","input":"0x711746e200000000000000000000000000000000000000000000000000000000000000200000000000000000000000000000000000000000000000000000000000000001000000000000000000000000000000000000000000000000000000000000009a00000000000000000000000000000000000000000000000000000002540be40100000000000000000000000000000000000000000000000000000006d3c514bd0000000000000000000000000000000000000000000000000000000000000010","nonce":"0x2689","to":"0x7cbd185f21bef4d87310d0171ad5f740bc240e26","transactionIndex":"0x1b","value":"0x0","type":"0x0","chainId":"0x5","v":"0x2d","r":"0x1f70a3d0933aab069a55fb2b16a259d21ea5fb3255e2032c5ff194a600ed2ab1","s":"0x3ac2142fc9f2009d68b08855eb5bc85e889e15d043c33bfc516f28ea38715485"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x339d413ccefd986b1b3647a9cfa9cbbe70a30749","gas":"0x180ac","gasPrice":"0x933ea77d8","hash":"0xd0287570d431d2baea96ecc81cb890e7f4f06ab5df02f9b4067768abca19acb5","input":"0x711746e200000000000000000000000000000000000000000000000000000000000000200000000000000000000000000000000000000000000000000000000000000001000000000000000000000000000000000000000000000000000000000000006500000000000000000000000000000000000000000000000000003a09994473bc00000000000000000000000000000000000000000000000000000005672af7e00000000000000000000000000000000000000000000000000000000000000010","nonce":"0x268a","to":"0x7cbd185f21bef4d87310d0171ad5f740bc240e26","transactionIndex":"0x1c","value":"0x0","type":"0x0","chainId":"0x5","v":"0x2e","r":"0x73a68e10ef9726d47ff314cd16d1846a0876d2b63a65014909140de7e45aa8b3","s":"0x580de27e4626139fbad4129a249c7ac2b22e1062659293e5fc3a244fd74907c"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xb5b9345e21f34b6cfa538f49e2b14ad4c3db3f7e","gas":"0x431e0","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x76ddab2674369f34946c5fa2f05e2aa8566d86235b83e808e9b27bc106e04ac7","input":"0x9f8420b3000000000000000000000000000000000000000000000000016345785d8a00000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x38","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0x1d","value":"0x1634632a1414000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x4314a4757076c92ca696f14cbe037bcc0178f8b10749418d4535195e90b2cdc9","s":"0x15ee917bc3e27928196ad987b61e3fd35ac92a4200b139d355b30502bff7fceb","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xac2fde165d9f13a223f58b9ac1dde51b63feff2b","gas":"0x5208","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x34a5c74011a2c8a00103bc91bfbfd94aa99cd569be69066e4bf64d188fe8714e","input":"0x","nonce":"0x38","to":"0x7155eba8ef327ac3b36fe59b5e00ab85f735f4f4","transactionIndex":"0x1e","value":"0x1c9f78d2893e40000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x52178c5dce512607a6242f501f78bc894917a482e99a3368caea6bca94c6dd69","s":"0x728be90654231b65f790eba478ccab574e2468fc4e031b712efa88af31fc6592","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x1ddedd674bfacb6587e0c68cee8f2f7d8a71e9d7","gas":"0x1d4c0","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x7b9730ead1b9f59b206d0ddea87be9383ba3fc7b496c7863b0cb847889b86617","input":"0x607019b9000000000000000000000000000000000000000000000000000000000000004000000000000000000000000000000000000000000000000000000000000000010000000000000000000000000000000000000000000000000000000000000041971588d95d4d664ac84a45e59fc81871b7051d679376d86cef6b9e1453c762dc33a84a125774319ee47f2a8b8d588da6c8ff7ad84fd9eed22fa124b1a5c5ed4d1b00000000000000000000000000000000000000000000000000000000000000","nonce":"0x1","to":"0xe1e6aef7642b6609e6c54884d1bda4a3cb152461","transactionIndex":"0x1f","value":"0x4380663abb8000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xc0d3e609b2e9f844d300a7784aa9e7cb956415d025cb573a73848f49e4be69e5","s":"0x6ce01527c00c1a93341a541824aed7a1c895f5a0d998f4415cf6d88171c5ad3e","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xf6df8ead0bdabfba61d9b2457d5a5ccec67fb7f3","gas":"0xc836","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x77166ee0409ba86bd26e7c03ad1a927abaf5af8a8a37149e725cd37512091dd6","input":"0x67a5cd06000000000000000000000000fa8449189744799ad2ace7e0ebac8bb7575eff47","nonce":"0xbd","to":"0xccb0f4cf5d3f97f4a55bb5f5ca321c3ed033f244","transactionIndex":"0x20","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x5649e1ff58fa9b6ee27a7316904df02db1ba1c471a2f3409f370a31c6b3d5ec4","s":"0x386b766277e267f8796ba6e92af71a752335bbe45cb3385b9cdcab9ef6c53e41","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xf37cbbc2dad42e8826898d45933afd3a04937e71","gas":"0x112c1","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x3c2b6c2ae505c5c36d5f316c1fcb5f54f7346ed35ae35c93462991ded7968a68","input":"0x095ea7b300000000000000000000000071bcefe9e9fbad3f12f2e6c14ea0cae7f0c91e7900000000000000000000000000000000000000000000000000000004912e6bef","nonce":"0x9","to":"0xb5d86e1f237d52f1f19793d3265e9340394f75b8","transactionIndex":"0x21","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x3ee4d5bfc575fa4b241b4d23e6f099eba5f1e6a20b8cea663c63db6400d01506","s":"0x21481ea7a3e7ad758f2e149829f70f43d67b3cc0d44112cd3744e290a06dd985","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x52b6255b7810d97de8f751737547d1792dff4fe0","gas":"0xece2","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0xf99a792837e13827b5e0a8915fb59c760babc95d242feca99a5594e64ff6b6e2","input":"0x095ea7b3000000000000000000000000c92470d7ffa21473611ab6c6e2fcfb8637c8f330ffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff","nonce":"0x51","to":"0x07865c6e87b9f70255377e024ace6630c1eaa37f","transactionIndex":"0x22","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x5237f7449c0df409c1a4d6cb09005a9224860aa74737debd961e2a24668d1022","s":"0x17e5e842e3f5ffb5a16eca0ec2a92e5b52d43bf9856f97e2083b2e951499b970","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x3c70503d83c5a681aab503d724f47fe810ae8205","gas":"0x17eb2","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x522313f5d923f048ae5bd0b5595c1f4fc883bc0b3cf3cb0939d3fcf8b08c829c","input":"0x94d008ef0000000000000000000000003c70503d83c5a681aab503d724f47fe810ae82050000000000000000000000000000000000000000000000000000000316195298000000000000000000000000000000000000000000000000000000000000006000000000000000000000000000000000000000000000000000000000000000010000000000000000000000000000000000000000000000000000000000000000","nonce":"0x5","to":"0xb5d86e1f237d52f1f19793d3265e9340394f75b8","transactionIndex":"0x23","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x6741309799fccf0f1359bd710291ec1bb66eed23522d6ed60c5924264d4d283e","s":"0x69c43182ee8a650b0042cdb2352781b23eb1de72f1c3a8903a1a33360e7d9732","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x4da9a48963c158a88bf6d4845a6a1890d20e55ad","gas":"0x431eb","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x471ceb0e85af594aa56deca54cb8198567b2afd8406722ea530077aaa6b641b3","input":"0x9f8420b3000000000000000000000000000000000000000000000000001dd7c1681d00000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x0","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0x24","value":"0x1dd87babd44000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0xf977ce19da80b7e155ee2d8b7656e5c89004851407bd5370785befe2c89d82f6","s":"0x1d76177407e73d8b73b0909158825ee375a25fa7c18b04884f4be77bb60795a8","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xe40a2cd9efde08fb7664bcdbbdd877c52c4bd228","gas":"0xf4f6","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x3e9dca502e9039ae0c6d642f62e9562ff00010c6bfbb8234a6135712ba70dfda","input":"0xa1ddc5460000000000000000000000000000000000000000000000000000000000000738","nonce":"0xa","to":"0xdef65f8e2f62bde60cd86ad3b4ad7fa6741cf175","transactionIndex":"0x25","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x34df85658956c9eb71549db3a73abca70dd01b8e91284afcd372c1212a36605d","s":"0x5036e2984bb2edbb5fd647bba873f1f5c5ee4845e539c9b0bc70d3d12a4bd93f","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x17299065e99b12b0c8cad06638624cff8d507334","gas":"0x5208","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0xc95cac67267f4accb9b5950316ac64772f7d082bed6b712c09cf2da0bdc237b7","input":"0x","nonce":"0x7","to":"0x02647cbb1714fa9a5747ec5be6333e06d806e85a","transactionIndex":"0x26","value":"0x71afd498d0000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x82ee5cafbd010cdf7a4604df728d1b8713a5bd26a708f5c92f6b6a55864c9576","s":"0x2b9a70d812123b2ddce1108c4101ddce182f6c83fc8aa0a139ba597aa178b270","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x9aef7b879787baf0c2dfe888e67a7edb7a233209","gas":"0x431eb","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0xfca28fdbd13fc16daf7aec7d4a2ad2c6b5f0b2a7b0fb1d9167c09b5e115ff26e","input":"0x9f8420b3000000000000000000000000000000000000000000000000002386f26fc100000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x2","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0x27","value":"0x2387acb3784000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x7e5f1a27ea5e652ec8705f80365f1703d2fd50c1de7c22ee0181d556dd2fcc38","s":"0xb22fb8b31dbfff8ad0457ba4b659fb0d7031cd86f08fc3cb78d81c97248034e","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x06006a25e0fe65f0eeaad1f4813940b68db782a7","gas":"0x17ea3","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0xc73124ca798b2f7a5df2ea4d568efab2f41b135130ea5cc41d4bcb4b5c57d5bd","input":"0x94d008ef00000000000000000000000006006a25e0fe65f0eeaad1f4813940b68db782a700000000000000000000000000000000000000000000000000000002726bc7f1000000000000000000000000000000000000000000000000000000000000006000000000000000000000000000000000000000000000000000000000000000010000000000000000000000000000000000000000000000000000000000000000","nonce":"0x6","to":"0xb5d86e1f237d52f1f19793d3265e9340394f75b8","transactionIndex":"0x28","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xabe4fc843d3a1628a464884c13c21bd21ae9a0a5738d9ae2761ec7cb24b059d0","s":"0x68408d8096e542b2c06cbbecb9494a3778c3ab644e69dde1fac078079c4bcad4","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x2501bf5c8ddf04a60fdd47a95d53d99b315604e2","gas":"0x5208","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x29abb76b5e7a5ce137bf9c22474d386eb58d249f43178d2b2e15c16dfdc5ca80","input":"0x","nonce":"0xa","to":"0x901a4e04f1513443c6efc6ce4e913d1b960ecd20","transactionIndex":"0x29","value":"0x1a3385ff37f0000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x1a1789d5b19950f45383cb1909cff9128dc2b332e2ea7ba42cfca5806376e3cb","s":"0x617a5a4339503a9c9d08705d98e7a202ed7ede1d1192fd7d98b648c8c102716c","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x507832c92ade971b9a2880427353c6ce6d0ffe53","gas":"0x61a80","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x03e5ab25a58bd44fb9dd0c698b323eab8b8363479dfcbcbb16d0a0bd983880ae","input":"0x85ff842c00000000000000000000000000000000000000000000000000000000000000c0000000000000000000000000000080383847bd75f91c168269aa74004877592f00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000006100000000000000000000000000000000000000000000000000000000000557300000000000000000000000000000000000000000000000000000000000000014507832c92ade971b9a2880427353c6ce6d0ffe53000000000000000000000000","nonce":"0x1e7","to":"0x805fe47d1fe7d86496753bb4b36206953c1ae660","transactionIndex":"0x2a","value":"0x214e8348c4f0000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xf707322f65ba56c95ced5654efafcf9bfd60a2dd80e74bded3190673cda55d8e","s":"0x46412ec3d2137fcef30f53cc4644769f4da126128979920ed4e209acb1c8ebc9","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xd44c850ce9103d6b2398d4ed9bbce321f3b769c7","gas":"0x1e0cf","gasPrice":"0x82661bff8","maxFeePerGas":"0xde6f782ba","maxPriorityFeePerGas":"0x59682f00","hash":"0x3c8ee80ddea7fa2d2b75e44563c10c10756f598e8ad252a49c5d3e8a5c8e6cbf","input":"0xeb672419000000000000000000000000d44c850ce9103d6b2398d4ed9bbce321f3b769c7000000000000000000000000000000000000000000000000002e2f6e5e14800000000000000000000000000000000000000000000000000000000000000000e0000000000000000000000000000000000000000000000000000000000098968000000000000000000000000000000000000000000000000000000000000003200000000000000000000000000000000000000000000000000000000000000100000000000000000000000000d44c850ce9103d6b2398d4ed9bbce321f3b769c700000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000","nonce":"0x9","to":"0x1908e2bf4a88f91e4ef0dc72f02b8ea36bea2319","transactionIndex":"0x2b","value":"0x2e2f6e5e148000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x327a6fbcaa2ac3cfb0ce64b47f6e61de4d18297f1305f23c6bfd42d88e8d068","s":"0x6b3a9b5252ced14b2e2f2833ad7cc62c46fbd0785d57f2747473af3fff7d46b8","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x78aa9046272fec8d51a402681d7c37481db01c6f","gas":"0x431e0","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0xaffa73b68bc7ab0c3f5e28377f5ca0a5df33c0a485f64dc094b7f6ae23353203","input":"0x9f8420b3000000000000000000000000000000000000000000000000016345785d8a00000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x36","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0x2c","value":"0x1634632a1414000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x925370b51dda3e1d1bc6b86580fda3b40875157ea594a9a14174436c726ad46c","s":"0x516c88bf3cb2f46ca81572140b41e5db2d74c4a736ee7d572b508415c438f779","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x56784e51c2ce28dbe9aa68342638d1e95ea4324b","gas":"0xb72e","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0xc66c9c66fbc8fe97fcc16506cde7a58689af1004a18c6171cfe763bcd94f50b2","input":"0x095ea7b3000000000000000000000000805fe47d1fe7d86496753bb4b36206953c1ae66000000000000000000000000000000000000000000000000029a2241af62c0000","nonce":"0x9","to":"0xcc7bb2d219a0fc08033e130629c2b854b7ba9195","transactionIndex":"0x2d","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x407c855bed0fb088e886c3721ee9abee4761221d110512ac29b8fb143c4b34d4","s":"0x1e47aa957d9ebdb564dfa41b72022d525383ffd825a307fbfa9ca13a59994dcc","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x805e39acf2f170251c48a35df16411cfb2f761e3","gas":"0x56a8","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x80fec96707519172b53790610d5800cd09a4243aca9bacfa956c56337d06f820","input":"0x71ec5c05aa669c4922569c1d33f7a81aaa21813800000000000000000000000013a0c5930c028511dc02665e7285134b6d11a5f4000000000000000000000000805e39acf2f170251c48a35df16411cfb2f761e30000000000000000000000000000000000000000000000000000000000000000","nonce":"0xc","to":"0x7c125c1d515b8945841b3d5144a060115c58725f","transactionIndex":"0x2e","value":"0xb1a2bc2ec50000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x8fcf376106af19aac48ecb840ea51df90a5e3ba557eeaad41cd3c8a90785d416","s":"0x4a9e9021ddb5f839bf7f54b54ad52c57b21b6ed285773eea022e3ccae69d1b99","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x4199614bf74f6b3ef47a812d976dd1f57a8f082c","gas":"0x431eb","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x61b33bfcf11214906dcdce7d7ed83ad82f38184c03ded07f7782059d02eeedea","input":"0x9f8420b3000000000000000000000000000000000000000000000000002386f26fc100000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x6","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0x2f","value":"0x2387acb3784000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x3d295f5fa2dbec04793f07adec57e29c03376c628a0256ee297fb3070a1e3364","s":"0x6f115dd4505bd320bd7460a47bab0c20e6bcb4b3f17c6d0e00e3ff1d936ed2af","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xf6df8ead0bdabfba61d9b2457d5a5ccec67fb7f3","gas":"0xc7c7","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x5d4138d4e28a8327e506cb012346b1b38b65f615a2b991d35cf5d4de244b3e6d","input":"0x67a5cd060000000000000000000000001f1f156e0317167c11aa412e3d1435ea29dc3cce","nonce":"0xbe","to":"0xccb0f4cf5d3f97f4a55bb5f5ca321c3ed033f244","transactionIndex":"0x30","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x47bfb48eb89a8b110749d34d171db36a8ad21c8d7cc78c7489e72f21cd86ba14","s":"0xa525976583b19b3d2c2dd6b34324cf55b1a61595cb53e3c58acaf203f8709e6","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x9eda8333e09828571db26576a566668f79f98946","gas":"0x50df20","gasPrice":"0x82661bff8","maxFeePerGas":"0xc8d5e2427","maxPriorityFeePerGas":"0x59682f00","hash":"0x875a142b6dfcf10ffb71a7afe0ce4672c047fc7e162ba0383390516d6334d45d","input":"0xb1dc65a400011a04d404e571ead64b2f08cfae623a0d96b9beb326c20e322001cbbd34470000000000000000000000000000000000000000000000000000000000bd780a7d2e00ee03a29f787096cae99da52d07803c76c13473654239118e2cf0491bcb00000000000000000000000000000000000000000000000000000000000000e000000000000000000000000000000000000000000000000000000000000004000000000000000000000000000000000000000000000000000000000000000480000001000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000003000000000000000000000000000000000000000000000000000000000077359400000000000000000000000000000000000000000000000000000e50bb6420faa6000000000000000000000000000000000000000000000000000000000000008000000000000000000000000000000000000000000000000000000000000000e0000000000000000000000000000000000000000000000000000000000000000217f0aee859cd78a5cdf79c68c8de4342658d52ccc6420181b5fcb7c657c5fdfa5ca2c8f2dd7e6e5799d44d6818e6dd8fee19b2530bed2045e0c665041f053fe70000000000000000000000000000000000000000000000000000000000000002000000000000000000000000000000000000000000000000000000000000004000000000000000000000000000000000000000000000000000000000000001200000000000000000000000000000000000000000000000000000000000840235df8360bffc902a68675311f5eaa79ea2c25e2cc394bff0a43d5d608e98c97de200000000000000000000000000000000000000000000000000000000000000600000000000000000000000000000000000000000000000000000000000000060000000000000000000000000000000000000000000000000000000000000002a00000000000000000000000000000000000000000000000000000000000004b000000000000000000000000000000000000000000000000000000000000493e00000000000000000000000000000000000000000000000000000000000840235df8360bffc902a68675311f5eaa79ea2c25e2cc394bff0a43d5d608e98c97de200000000000000000000000000000000000000000000000000000000000000600000000000000000000000000000000000000000000000000000000000000060000000000000000000000000000000000000000000000000000000000000003f00000000000000000000000000000000000000000000000000000000000007080000000000000000000000000000000000000000000000000000000000061a800000000000000000000000000000000000000000000000000000000000000003312d88cdb7cfd5fc5458d9a424da374cd818847f87fc4fb6eae2c3b97e541a6b81e108e55daf857b71514fb7213db7ba15bd6adc12253afc3c5e949144c13696faabe22d1b7ace91532134a5cc65bed9afdd897d6f84ad1b0b6da463bda8fde5000000000000000000000000000000000000000000000000000000000000000352c2a75f88112864819de485c392c349f718fc257934181c4d8ac8eb3a9b1fdd3b7bfb96496290f2b41fcd20fc8d29bd8e29be38550775dc97d21b17c2d679ad05f519787eb2c92a9a897c9f02f15316865cfeb4bcc2c0ba5ac2dc4aaab5c0c9","nonce":"0x3705a","to":"0x233a95ccebf3c9f934482c637c08b4015cdd6ddd","transactionIndex":"0x31","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x7e9247209bede2c21548c6e4f26e41d2636a2490e09ad062f9235e978094073d","s":"0x288d820793b48e55777e794f1694b1605d74dc8726fff438b481125f219eaec","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x6df2b54f23ed4e95235c6e21ada2cfff5a2615a6","gas":"0x5208","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x79b6df832bfbd04085d0b005a6e3ad8f00fc8717eed59280aa8107268b71e7e0","input":"0x","nonce":"0x17f","to":"0x6143d939b2d117f4b998d8d88a56eb0e0a1e9c31","transactionIndex":"0x32","value":"0x186cc6acd4b0000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0xcb641be908976bc0a035b0b31a96039da9e63e829e0a24a092ecfbce7724b9c7","s":"0x4034e7dc4001cf9f62a2e5dc4390229655b85e9aac5a5b263278e9d2cb43abb2","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x3f5ceedd435b6143eba2254811a39fc3423f18dd","gas":"0x186a0","gasPrice":"0x82661bff8","maxFeePerGas":"0xaa246a58c","maxPriorityFeePerGas":"0x59682f00","hash":"0xcb2fb25d268f65dc9312e89bd3c328c9847a3c9da282026793c54a745f825ab5","input":"0xe9e05c420000000000000000000000003f5ceedd435b6143eba2254811a39fc3423f18dd000000000000000000000000000000000000000000000000001c6bf52634000000000000000000000000000000000000000000000000000000000000000186a0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000a00000000000000000000000000000000000000000000000000000000000000000","nonce":"0x0","to":"0xe93c8cd0d409341205a592f8c4ac1a5fe5585cfa","transactionIndex":"0x33","value":"0x1c6bf526340000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xa48a4ece6fbe881e342a4eda767c82097c3bd47dc70c09ef6a4e7f90c0f8e843","s":"0x66c92f16907488d115f5088ed8ff4d71c966232458479245bf6dae45358c0429","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xdb4f0eeb7c89b00118cc74de8c9ed55d65c2b933","gas":"0xb72e","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0xe483d4a36ad19fd5eacb7f6d9ad3ce080ad70ac673273e710f6e3d5acbc6559c","input":"0x095ea7b3000000000000000000000000805fe47d1fe7d86496753bb4b36206953c1ae66000000000000000000000000000000000000000000000000029a2241af62c0000","nonce":"0x120","to":"0xcc7bb2d219a0fc08033e130629c2b854b7ba9195","transactionIndex":"0x34","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x6d73169394cb6a9bceb2dcb16aacd261705096f50f30ffe146a09659e25dae48","s":"0x496a1cccf01ad2ae7094b8da2ce50c0a990baba6300625c48ca12c5f20005a64","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x9ac253db14a434cbe7653a99c5039648fa4f7353","gas":"0xb4f8","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x0564242c37d5013b671ef4864394cc0f3924c589f8aad64118223a9af2f164f6","input":"0x095ea7b300000000000000000000000028ac7dbbf9ed6ece39a902e76780fba60f1b2e3b000000000000000000000000000000000000000c9f2c9cd04674edea40000000","nonce":"0x4a","to":"0x0dece1605795461158538302b5f0936afe78c4db","transactionIndex":"0x35","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x15684c9697dcc8d89abe562c4315bfb959938fe7644c00615d9e147a4e496ff6","s":"0x7c58abea24e1fa8caeb73a70219b0d63007785c39694fe5ed2a5a342c99292a6","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x22496eece162f52d185c07524d452b9506e9fe12","gas":"0x186a0","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x48db358e80b278c3a46c2a166339797060a40f33984a5d974992cd9722139d5d","input":"0xe9e05c4200000000000000000000000022496eece162f52d185c07524d452b9506e9fe12000000000000000000000000000000000000000000000000001c6bf52634000000000000000000000000000000000000000000000000000000000000000186a0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000a00000000000000000000000000000000000000000000000000000000000000000","nonce":"0x0","to":"0xe93c8cd0d409341205a592f8c4ac1a5fe5585cfa","transactionIndex":"0x36","value":"0x1c6bf526340000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0xa50e6ea1eff21fd33d69f8c74b90c28303b92ebc4a77c71b65c4c8bf3e92ac8c","s":"0x29af3bce2365578b76dc528917b44e96e4fe2e7a8f948eff3e379721f35664d8","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x9ce3f4210dc06889ad5267ef0ce3ded4bd96dc50","gas":"0x249f0","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x69d7758db91fae31fa35ecbed4d40897c5087f45dc796cd796b8ceead21f972e","input":"0x47e7ef2400000000000000000000000089f01bc9865324583380d8d7ed08b8046bffd7fd0000000000000000000000000000000000000000000000000de0b6b3a7640000","nonce":"0x1c","to":"0x6cf3df5d00e842aab1eb504bcf01ebf81646f7b7","transactionIndex":"0x37","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xeefc19e8dca874308c84712c3e48c032a25971a0f7dff16dc47633ba7a4f5b96","s":"0xc0994cf21d29cb0c5f1e666538b58d6dff3b9ba2aa7545337da732c4b5938cc","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xbd1c4e36842c06e53b378eca310cd2dc79a78cbe","gas":"0x35965","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x2951478916ecd27a8e808d08f85be4bf2c0b0e0546f21f4e309145dd96eb8df1","input":"0xab7d7969102507275d685f3f55d10ee45f7b9fa39c29a8a604a7a971c05eee739e63938500000000000000000000000000000000000000000000000000000000641cfd18","nonce":"0x252","to":"0x96e6192eeaf7bb308f79fb5017a9085754b9e12a","transactionIndex":"0x38","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x87fcffa38e0c84cf80fb37ac1e538faeca21462b0a9ec25af0ca0517af22789a","s":"0x72cffece99e79edb7425a9d660590fbc395cdd0db8fd4848074c85f814b3cf63","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x5dbcb3e489fbef1656dbbd7bc4848ed0e520a089","gas":"0x7a120","gasPrice":"0x82661bff8","maxFeePerGas":"0xc8d5e2427","maxPriorityFeePerGas":"0x59682f00","hash":"0xaca9028cb5d55bbf71b7bff9884a9a3b0b38a575ffc8f8807ce345cf8bd298ef","input":"0xc9807539000000000000000000000000000000000000000000000000000000000000008000000000000000000000000000000000000000000000000000000000000001a00000000000000000000000000000000000000000000000000000000000000200010100000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000001000000000000000000000000a7a9d91e687decaa6b045797c2e002d70001b48f03020003010000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000600000000000000000000000000000000000000000000000000000000000000004000000000000000000000000000000000000000000000000000002361273ef6f000000000000000000000000000000000000000000000000000002368a4faa14000000000000000000000000000000000000000000000000000002368a4faa1400000000000000000000000000000000000000000000000000000237b9042149000000000000000000000000000000000000000000000000000000000000000288e1b54f6e54124deb78bdb67cbfd0bf63612da12868fc281afc34bb26200a1771f9e00fce297da11cf49427becf350d0bd20a2277439fbb2e8db0fbb5e3d34700000000000000000000000000000000000000000000000000000000000000027d16a90ad0e1f8d924255e6380764fa7c7d46427c8e8ea19f5ba6a9d595297a340f141f74e705d0fe851d7564eba89fcde574deb5975d9126867cd927a0d92cc","nonce":"0x11d38","to":"0xbe57bac491de9a260abb6ba2c9ad4c5d2eaea09a","transactionIndex":"0x39","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x1eba7bf95bf63d02b74d20554af050eedd78ecfbe1a42ec70badb36ccfae4bfd","s":"0x4d7f8d0ac2be28e59374453f24ae6506a81707cc57f0d1c6877b49b3949a9a79","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x3079be9d8622173f02618ba2b793f00795d4f320","gas":"0xf848e","gasPrice":"0x7d9f6fbd8","maxFeePerGas":"0xd9a8cbe9a","maxPriorityFeePerGas":"0xcfd6ae0","hash":"0xc7f625a19ee41a1750eac9428b4394a9a2476b8ea2d31b4c2f9f5b4fcb86cae3","input":"0x2169f79f000000000000000000000000000000000000000000000000000000000000006000000000000000000000000000000000000000000000000000000000000fdce50000000000000000000000000000000000000000000000000000000000003f2000000000000000000000000000000000000000000000000000000000000001f579a43ce4f3e38c6a948f1ef3bf7954c2059b087b18fdb22c7b69d61727a5d2aa16090b4d1f3281bce660a31d3e2789eeb360192a54e3ac3513c5c8f67b7346fda957c92c1a2981bc68a47d0282e78df2fb63822f06f410e1ffbee7eb83c425122be9be94d817e2e42099632fb9eb5ce7c414cf6ce48e4ed2e64e46e1558ca809742159e841e3de803a8cdc628f7877bfa6d75af4b4ebb185e6d0670ce83332478137fec243a38c0a80d067c955cbe13103294dadfc8c356d0e7996cb62fb2c683e6dd5ad58be934f57375920142990074f8a161ad2390a1e6bea9fc3a58dc3f7e002e9952a68bff229437d3fb4cc1cd8097032b22c884868253d8f2f69c75ada0fbe1dd57145610ae5f451b47645718aca7432d8cd3193d0b96417df6638fd47062a1cb1f63636236fb72195e5a46d1b37f493b8380315e5bb1a6fe67cc6546d62f2c73aad5699e6ffa0d49ce653bcefa173fe0de7b4328aad75f74d15f4f570a1839514550d576e21b8d16c8355b770442f0740e6e5e77ab450ee1fadff8daeeb25a91585192fcd3cdd64dcb6a35e41ff47e854337ebd0923ce280cdacff2ced48eac96bc9f910391a44cfec243a8660ff80c10ff844ea22411953e0387f87d2c2ae81055fc8e87d59e04cd7a923d6aa5204b3eaaf5b842b5fd656e9fda16c59e90c64ef51124fb32f8e63442b1f79d3ba3c0cd5fb90895103719629857d5999b44aad2d69380f48dd824795a89dce14bb82600701ad8fa9bbb36fe93a97d75da26ffcb1a8828918e0293680ecb12e1b772fd98637a7a83cffbca1a994fc889ed0c0b20d447ff727b9f5dcb9a83e1aae9a3518dfe1d7ddab1da48310eaa7ab293d85dd9ca811bfdfee990459fa044c9a9fa6460e5ff7612e42021f16854d790710684663bb6a10ba475fa602b02519f03eb544fe486d9cdde47ab7d4027bfb8f29d778b3a7cc35a9f78dbb05943062efe3b388f1a8c496682a0215265b99fed0363a1061556a40e0539a6656b6cf8967c8fb3092a7fc9b56af124c8f1bd0abbb9af753884b22b333410d47d80c083c9fee52dda8907be0c87c5496f1fe9ac4bf4dd4f06d6ac0ac0cde22138b67bf782e8e9b5c991d58a2a828b4c68620e6d3d26abf8d41005a00866cb4095518b2711636f2fac3b5e26403ef96f5b1765519787ed74c6802b445de029d7bc696aeef5a2e605376aeda4f495c29af66e357c6e1232d7a2c3f22c0bae0c62372d6b2690e503804e2ad314c38a4ffc080d36213458f606075d32dc0a365087616e94a972bf76d7e5eb4a53292fd0d7570cb6fbb4540b3c5ee6deee0f257fe8703d5d8d0180d09754bd9a05894e77318b8c8fb6b4a56999bfc6c405c09d9ce7fe424812db5bf778d7d8634b2f44ad6d00f18615e4c3695d9c5965ba51991508405187275840a3f27cb3101a9742c49edf42dea79ae5418ada42dfe2042f931af15a205e64196b04ef53492a450423c2b63dc849ef72a166fac5f36b25c72d9cb4a2507d5ad15094a979b4ac4ff8eb4c90f448a81fc6dbfcf6a228675ab3308bca37bda8eda22b61ec7e596f2448ee6b7cb6eef589f43f28fc5444d1da3f82ddbe574b774719a3a7869e897e642735517832959ea9cb9164eb2f7665bca1d2e44e1093acc7088271f2be20459c383f5762303b553af1ccfb3371e4eef1f102ce70d62620c5164972facbfbe03b5018265ed62c4c3cc49522166913517e2f2718edb591047a72e818f07c645a18aed87c0384d180dc0bb7fe63bd335b1c09e58571b92e2267ffa5712a33e4a92231d9da7b005730f6d6559ee685edad9a9031c4708c8642c297287af3a3e637d9b5ac6b23218977cf4d30231a79063987f98313a2bc0ca4a73a97e441a45b58cdd8731fc269255b415a9d2ef2bb8a311587f39f6dc82f537cf86a68af19a68c409910f4dbebdfa66445361877a3bf551480d115d8768e8650e9979472a64802da3625cdbddee8f02678a3cc6400824d91b3c7d4c9274b74fb11a9538f1990fc314d35564a0cb0b384590e15cb93db1de1a837b9319828b3fd6b76f1d2bca4ac765f9ec4366905f2dca69a4c8c194cf0a5d821c58bd291e8dd4ddd829a873d093042c1b8e1bb2bb1a95e150c0a0d5c3369c4a791dfcfbc3e631da3d1e704214c3cf8318901f9c58465be78a870a040f34c567c27eb077db2d1ff825a127a038eb14c2890ff9a8a1dc8d6360a91fd788fec97599a318ab03f69bd50a5ece92d96c49413b517579ececf71371b6b1d902b1dbbf40c728c64fadf368582cc4b3c51712c8d6441c0c61802943624e93f9de9eedde87056c21ecf66b3a6977af0261ef48b4c6191ca7ea70d6acc5af910ebf247e2a4429c506c6a41f865175a9bfc81dbb69e59ec2c47370fcee9169b8749c852e7a1275c1278aa3f88bc16d311e58aa9a579ed7bed62f12b4c4dfe9c493cf33f979f667eda043c2aaaf863a747b208f3aa2ccc34f07b12d89bfd75b696a82c493e7eedcdc41eb01321ecfdcb691d14996c3ed7487b5f831436bb7c25876c4149cb6222dbee9e1a05771a8abfd0fd530266476e1bf69d9a15e99c901243585f56a77ef0e6c684a4d49417133ebb91f896176ec3a8f0b966eaffce96f129d18960fdbf5917f8aa0926d52c57192f99e55959369e57ce85ae84ad944e40652f6a81ab8da1dbe81a827366b8b8d2ffb9c7b93087f7a356a16d0022ee8ae891060e29ff11f9f17b01f39b8024baefc8b928a6816309a5c3fc92ed6007499353b5f1bc5700d5fe67f3e34ef3654712b9c67c231f30dff485213b94c20afaa91540560df80ba9ff1cc0f6ea75bf1dac0566551050873c112478f3fce6af162397161e1b41e1ca295f327a4f0ccf4d15fbe58e20a23698bdc25c751d020df371cd1fc59fa51dbf7ad619d208d7084c5752984b5fd0e3a7978f2c11e6b26531fe03c0cb85519483d668bf1dc4dce24509b00aa6e146adb49f31c2c937a2d3a08f1cc0b73cd66f2cb926f0453cf78618e39cbfccd6e6a3b4909d6874872eefed0e395ac2582d8c1d0483007fa6ed7b735323fe8fa1c4be83c725c6b743f63cbd0d6017f1145634b0b905552a2a237561b37c415146d0555a276b4b45d515ed7dab942f9bf40a26d5366032f23fc5fad8b98a1625a841c8cc97dfe0e2074a363a951946df16d1261ef6b2da7f5789dada153448b076637f9d2e04f4390e9dd32fd5e9ab3f5939807bb451bd19d962403da9dd118f1cfa91731973bc721e5ae4a687bb2270e3e76966dd2156e95796d4bfa6b4c10caaee46e1d8f8495f901cfc241c6f9a690349eb0752adb220996ebb95ed26ef376dbb9eea201906a6a3eb346c64eba3c9e4d1b3088baa0ed868cdaf063a4c2434b9d1bbbd69f44ef659262fad9535988ce729aadafcfc6c3ff741dbf8b784b3465fb05f6d5daec91e915f89117e01c9b0c99d7b7cb78e44f5c28de0d3b782de981be058ffa1294072c4e5a66d2d81641d1c07a9a04012f9f1ad15028709fba761a659becf01963de0cc42b0e16bc5432cffa00722a84628373198dbbe75f209115c4e30e60297c2d737996dd9aac330f4a3e82609dd33460438026ae287c172af8aeed0d6c71b1f9470d60588ebd365f20d596999d97c7a6f265890c719dfe43ae1ee5bba122d9d4cfc96b399f610a90606add4f0d25b903c7ddf6eb7bfdc3b9f926243108f591678b0e92a48d52b1dbcdf30ef8af74cfe34d48d0e02d0f99af2adb090c489d4214f3562d3f6a6e9b83b54bb6cfe6c1961ecd2eb2b7eda63601c835c5dc696b32bc495bda9b6567984f83e79a9d4ee88b140dc78cdb64a2ba8761a827cdc8d2b27e690c22402178ade602f2e9092a1b07d56d142bd24d665d7b10bc7646eb6d6823d5622f1fb08f2e48b8deda6cde324d6508fc428e3bf1a9ed608c196786db4e8d6b7271a07e4f73d9ef76244e3e10aa13fd9b82ce2ce72af2df7faaecaf95d885e64bcae16f9b59ab8e0f21a2f459d00a290b0dc57e10bf21ce06a029224ae15cb86bb1cf060ddccb1f6ae82d767eb077fa4ab0f37d02a4bede5d3f8ee3972a6a3a438a3cd92ffd6c1fb11cf2c68c2c6d1efff62b9f769e302f6c5eec96b0c05a942321ef8b9cf0ff2bb80073a8fe138bac5bd8e93f3b27e7ec7a78512059fa37b0b6939f3b1311cb6ea154909e9d316567e013df315e8f02a80e17feb8f3248cc3f11d1253ef3156d7f12d1f6d7a382d6b6ac6abd38fdb5d954e13ce005ce16bd5625a5df01883409f4ab938a59a75b7a816af6a2447b2c65ea55c0ec4e4f23fde145a26ef0b13b740b352809c10b54bfdf30592a8b0d507cb8210391d66d6f68d06f3a4663756697d472adb6ba78976b423ad9ccb3a3bead9de51e324fb77609729c78664f4d37c7c748ad9fad95af355be4f00a76cd5cc2c0b6170fdf3efaa2571bc8fc2d0d24682fec5e034460de55cd0426b8ad448551032e1f6ade6d96682bc38a92c296e5bdbf42fcf106d14c5e8a4acbfc0cc1f0bd6877c4469be8b204ffd5d7c0ee85e7b3265011ad5e4896f9f8a69997e321a797549cd3c1cbb0301ff30c3a136068ba37b2176e556b574a542db90250eca38f3b0e418e352b5728206d6d531ea253d781d771ad974b476a045ff14718d9a1820fb469e981c7b71ae8fe423b0a04278bcfc967ca602254de37751127fe3716456df3fae50ed508e529c2181d63479b6464bfcde6629e8179f8d5fa5b61003edfc2f129affa58608d6145de98ca8f1c0b3b9b5ae996a7fbc6326a9ac2880206fbeb1c99d0448cd1ac5f275b84e4be03258f9fd0d3f44a692513957fa083a462086787315e38aec8c09b96cca034cc6294359fe00285c607d41fb4e3bf25760ed1ef0802bcc40b2fce2cfe1c64f7c34b21d0bb1f5210360079ca8638ab42943b0ab8277f1cfb1e529073d2068a36a4d8ae8179c32bf66ca335d698f76db1cfe5236e6427e01804464c1ed4facf18ae09148c5a0cc9ebcf6fb6c7d03ec6782b7595816245ac2d1fdb3184ea49800d5d0f1dd880ff99d6016549d63ba2a44ab96f0db0d0cbd5fd4f67af1f1da0b1023883d4b2e7cb23cf85c617c180a771267fe63c70f19eee28fdc50c57d73b2718f8001383135b859f4c2fa3b9a12d095beba8857df13e921d9dcda5b87e1fce6038fc2f92282c717305157b6fca5ca2c23cb7307a093512f985628e75326e0bca2c4dc5ed44bed8e0cbce1238f4023089a81bf62b04a31539de9c8a0c867b791c18a34abcc721c0e90a1604e8d2855e78673367ce463307f6036850b81e0aadc9f81d8c9ad6e3b81d7850d4a46bd6772e651e03581a7a55568c9d63c1b1e8ca947174d68afcae6401b75dc7397be3c0dd76ad1827424f9ea9de8ac99803641535dae3eb57ee1eb1e2c193bf66addd488da55055bcf93172b1966364db4412e44bde64c50f7692a38d1536683af14fa057569cc7ef11f614a58995ade1b12f979005a53bee0769847deceab4f58f3d062767f7ac0511a04fea349e7688ab26425cfc369127325ab7fd24d73c4a8b23f099892b3bf52f35f0d775ba753098611ce35c56873a7455737a42672747bcfa99da8c8d0f06d8754d2babe3eb303cb06ce69e4046541ce63b6c28ded5a6fbf729200060132e9b01d8646b512356c92d8a642de0f62d169b195cc2e7ea2efebe550b1cddb5df7668a0d1f94a2f5cdf7897e46cb03b958ad906393c7d252b58493ce6c7d813c8fdfd5ef577a9e778c64d48c21b130c4f6a57985b8122f3cb69de015081cd3e247de65643f9d863fac6935cffb2ca23083d4fca9b8707679d9d4c3cb34d05d092334b1740ccf4b2aeae724902ad4fb4bceaa642b3ffdd5b564d7bfb725fae91f5c7ab9583abc2d80afd2272b14ec6fafbc51e35bebecce63bd95fa3f39e4e689034c47628345c4d8e30a73c36d87d289b6ec715a2769efacf39c9f19eefdb311b9800377c5c61aadeac78dba6946da4e7914415eced875b66f10b96ec98f64e58320bff09e8ac4f5969608171dbd4ab5883548a289b52d55aa82da62ba3d9fa59734d8f810bc1a40a0e3b2e116873116fdd8bad95471f16a14054fe247637febb6081d00e7ded3019ec03f76ebef5401a27f380fbe8a55bb4a0014f61276d9aa5626a34e7fe5a129a75ff130ccfd78cfbde65416882ce9ae9da57a3a04c67d22d3a12e811b2578fce5f6a64a09a86782bc15e5872e546dcca46bd9ccce5b787d17c70d6b075bf4c789ca184dedad3485a0526a61d280dcecd984fcb7ec663f8f491e91253675877987433e11b9cfcb5940a1357b55fcc5f977fc898f72e8199ed5d8d2554d54a55d6b4b3726e7fd7093996e271e218533f3b31257aa989f5efbdb12dce02d18a5188ec32c3c1c91845a17011ffede3f060f19d59b6565a352e90c5ca30eb252c6fb0cc845833e553f0bbe1822b5d6b26eb817608799b5b77c2cce27beb151060261f34ea73db94bb2fee11864aff76da5b10b17aba6634ae655dd396fd7e7155013fe3c6f840ab1ba9b25489c67f6bd5967a62134b238c325986c87aa005396c54af6357ca88baf5a03e353c8b0175386157d9221833585921d581f1fd3f5d9932b24d41895a5c35be91b35643e7091a7585ede900853aebc260b1fbe8b72d527390ddf46472f7c8d5844355ffc0b731dd5e584e13d65f2347131ba8a5161a6749458c1899b60cf9f9f8810889fc23b5d0fe2c843fb5675cfac6dc8ec8125d1c582186eac82d5e0b3f6df2a83eb2fc23c4c3f873267deb5ea86dcf1e3c5bb39df23f06560609b9fbd89511281fc7d69fb19764c41168a539b44f0603a52577c266fd601c0cc849b067746b4da78d280385ef80a3a553dabe4b137c50152f4e1de456d43e015e26f7d76e612b7be29b9f8e15c6a73653f5f7dc28aa245e64f076130bad58bbbd3b024c97e267a96365f019846d965c037116c74ef18415ed5500f12a1816997895f3bbe7b0fa1eb12a565255a64d5eceb2272027e45ecb0a1a8b5dc99128c002058c569076371703e1f10368c6cfb830ae8bc688eb143da7a1780dbe29cc4c83743b25d570f5d8609df46c807edbbc96358cde9ed0d9b8936271b3ade28001a6bfa6ff7ffeb4e6a6687e803b5e8f67b82e6dff215ee4f2caee571b8febeeff368a89539d145842526fef110174ba3a2d9f1b4a348dd25dd964efcf88847d3eef1c18fde81ce84b80cca8588bc1fbf9c9f34059808d3eef3fc2cef7636f7a76c533933b79a9dd2ae2791d1ab5e1e715837a766d9df1947d5986c21808eac758461596870b744a6a2862f7f5bff0c7a3460875b3b326e45fcec1cdad1cb4b99085748d69a62320f7e6c53c061a55ae20368a4e6cfc126cf2c0b410af545f169fcc741cfcec4947f577a1bfad47b5080ecb41760b5056a9bd10a10ca43677de0e06bf36b0d2bf7ca605b9217240871ed1ce43bbd07a7e77d1901dde15e173ca344ee8078ff115e954a2fa660bd4af61ac32256fa7d62af17a4f95adbdcd09b933fa47828d91a6b710a5c67195797bc789647db90107fb527b8ccbc1200bac302dc47a366d170314f5fd7b60535130d76832ab7a06ebcd83d064c1a1c5a7386b0945d39f8ddfc3503415ea22514a0ebd0ad74e3ece5508d83355cba944b12b669d6355a6143921533190eaeda0337a1b89ff9db0cae62a239025bef078137362db71a9eaa6d0a26fcda7e80a9b8fa4a1799819f0489b899cba4048c999fda79e0bb046d7e765d9abb445279efd13d47fde8b731a944af92da8d8965bc4e90d9da8ba0a9c385357e16788cfb663573d9d34810c754e86e3293a868461b241d7872911e676e73136557db1261573962221589850fa5ce049d2356e6630a142427a2cb2efecfed09445edd112323f0071f63f6a42035a9248efe155aae056db25b8004204d530d27027c34eb6937a5a05bd2096eeb1fa266a0325cbc7ac03270b751013b53429aa2d8da2de8c12e7d4996b5f1dd365e270285d2f3a65bdeea9346d960ecf015215a88f465a66417d12d95be2c0bb3d8e15163e0dc66028a30f1f5506558ef534ab28cf6dc2b7d12f49b0e319995efbde71978ef7121df58bbd16be908298b0abb6197bec22c598413283e7893f0fa7398369b83d30b29560ad09a0a24e029d0c22a7cc6fe53d84a7b2104681df35cf4c3ffd90b14cacb2ebbb673ddbbba291feae96498b30d024398676107d3efdce47353a544c9d885cd5cae19588a4b9e29c700c539bbcc119926e3d1adcbf4e8fa3dfb0d5c2686a3fb4ae7b4b278fd94db5efcad7f280488702d9f7675aafd248960806df3e8f95325298968a8e7bf54cd2797decb5bbea7bd3cbe34081c0b65ca9a4d0d08d86ff39f97751471e09f25b486dddfb2975edd51ec77d949862dd4a658ec974b8a5a381ca5b7314b677c4491e07b43a5a9964d95e1a29fc1d87cbf1a64296f4cbd8128ab13dcb60dfa8593a575140f5a9fa6a3708052b3bbed4b129aea88e66b72e583f1fc285ed6f835df0d4751c5d5f004ee0c65203ba9fbd8ff70fc255e497748f24c476d18c94089bcab8ae3f933563c6682e5452da8f760943c7efa631b2980e6589ad4ee83c88fe0b2ed9900571c3df6bac839df1a6d9d281a5ff627d41ada13b867735aa7377386182b1b113f237c14b6d806079d2f7c234a7cb585e12a5b16f7064fccff99386d58f91d9c235eef730660a02ad37dcfb0352b1da76d739ba7bd4982cbb8be48494acab84451bb39bc59521c9873ec9f53cedb527332c47afd919c94d777e603b405e8563248c9ca72f3b2fb52f08a876778d18e7d179f6437b64f79ca01f6be06d7b115b8d3c3320c2dc0ac59c860aca8c92407ca2239998613f44945bbe2c6a0ef0890ceb27467d1ab805ec960e6e58e49678a0841823632f62c58f540f19e82bf719e12bdc63db5363fc0749877539865a05786106b58dcf50bbdd015893b043b21bc1f24ef9eee5f2487d986db13c143d3366260ade522ab824fe82218daafe5e42c50ef740cb9ac570a13cb55fc30c9017228f2c60680f75ca89affdd376ce70ca5223078a47da0aab9bd18333c14aa021e6c088609d7a2460744006f149b648116152e7c814a5730bedeb6ebb59fe395e6b15d379909f197c4703dff34f8915b26a36e30088cb227fc1df108560fce419f462181e468b5d94251fe1373e0dd004fad02d690f8f28bd972e85ec49485cfcdba717db66e6a343044a2453ceb6403fa0795485f8014225d283da4c3aa6fcbbbdd6b04c30c96fce04579578882a44fe7b781285daa11ee71239c1b69791b8fe45868fe6e85341b7ba4292d6da1ae079875b33d7aa2becb02327a17bf2d09f6a42e9681df41890f17de5cf4752880660a35dab912d10480f20f92589d52ef4780ec28920d11de0de06bf63d61829a3d0894f9a32f911ac79041ef51fbcdf3f5b03f6a0b16f0b9e47444b82efa7c012e3bb9324c8649053433306d0640ae953737336cebe9b854186f9dbfda861fd7eedd76fc24634acb9cb925302d0bfd3751b7b6bab36ebe2ea80cb7c5359a505689ed5642118ce93cd783c80a2cf050f57d5acb62853367ac606fc1b8a01abb7b29e49fdb190e2b586e249639889bcf5ad23161f278086f244d0db0f9ff520e6858b68f5bfa0f5ca700096277ddfbcd14a4cf4358c0f042b1eb34bca6353a1383f5d310176262ba45512ea4f172f321efe971c391fa9e9876a250ab653fd6d5a8b046028018ae043da256b78eb26d86e6e89873476ae24dad26ec1d39da44ffdcd93d8c5c1fe0796bd46dbe9b01be0aeae841662c10ea28cbfbef76f0197cee86dd4c7dae3ba46d5c2884ae977c456681d5d9859d854a3e0708bc35fd8415c22ef4cb889bf0e66da72ebc635b71f5df04db801c76564d2fbd946ac17e27fbf995994ad26c3d577ac86b3b19f855d87bf3f0f7c3fd574b89901732dfacf2e13960975176f7418893b31747d96bd981e9b1463590bd35b51b7a2eacf1a7733f76a7d4b02fa2afc5925f5483f82cb529682f785c21cb951d5678a7d09f99c0e2bd98948b79456e4e0b5d9de80943f1df0c639ab8acd6c99a6297cc5a53a829561a6615c6c44b24a30a75174aac207a35fc429e1a415b0813857ed85ab49db6e523cdf9b122c2442a4fc2b4fc21685c9c4a5de8f36c0b405e34ddee91832cc6567d458ab76b4650020d270a65a562e75e06a3d8f1313b4d1d4d29d597bae9eee653bf8e31b82f9cad2d0848ed5daff73d99472c7e71d2f1be60bca4ee9d315196c096cd8260192597c8ce3cefa9cb6f129dcfbcbd65b5b0e19df5a1e81bdd423a2e26d11ca4f7aee99104530c1c8e3b1150669f16acf415559a38cb27eb6d637c78b1c90ddb5eef14ab9b2ff14ddababd7a349b7f6f5b6caa3a28c45835c9beaf8f836848f579598ae80ff9af739cdd800c5a3d7675c780ad66b2c449824d8ccf55a7669fb720604eeab427a0308122c3039ef9be6df46e34b719aca7a001b884e3f7b1d1579176aa4d365a25b6f3d420190e3cd9babefd8382b6bb12b4c38b981b08b5a139cd46e9382c01f361abdac7fb16b87e2c61f3241191028c770c3e3be939c8825c63bc4e523b389d1ee28189b5a2c11ccbe0aa7d3c2b47443028c89f3d4bc1bcac39eb61d91e75e0d372b9f27c8dc622a86bc0cc0a9c9dafa5e2bc67834a75805f3bf1571fbc5528e4c69f3e64ed27c1f2d93f43215ea0d4213bf2809391b8b9201f36ee2a275dc7c44d0d83de3b82490b65d2b399a7348133c4df7ef40e6dcc114d040f65de00a70cc0c9fe615c6fa90a2fad6dbcb39370af043345ebd9d8af094c831f341fa5f7658c2c1ea062948fef5015696eddd181d87dea49ac8bbcc2c7a1ea01ba1d04239153ca401bdc27aa87093a78b2020109d7cbdd39f279a7d67b14a5822fae28854323a2620033a5fcf9817a0d45eca7e2d120a113fd5ccd9fc0011e89398767f7f6b7d01d5a670b4fcb9c5648991817a62edf43d6964de204541a8b5573d7cc10bacaa1bb835f87d12985ad5fc6ac4834ce76c76e5dc34cc322c4220d9d65ca77051e7c1d4edcfb4595db16c641fa445401c72a24e36567631ea02e4875b7f9ac2ddb4712b79d1468c7ed2384ad5d90ea0825f817e42413de33eb30861ac92e19b1aada118beaddb179399008137f28a8f3df1509ab12d6f6ce7872521f5cfe4738722fdb410b3824f139ccf8ea63817d154b1e497a70e6f55b75e4decb73d10ddb0c57d7e0f74b3f4fb41ee57c2da1e4399c2577f28b7e911db08e877c366f231114cc9553c7f3cda2b75895fb60ce9eef4e0c25d253f689dff5e4f9cdc744c301b6f13e9fa9e30bdba8abc203c05879ff01bf8c3a3aa742ea5fffabbd6d165e50bc412bf5e18c60bbdd4fdb276f3bbfef246cfe36612ad61fcbd4498c18321a1ee12e6dee902c737999f9c6b3919f52125cc77e237a788640f262ab76d5c9db9ceb2d988c261dca63455658293b10842ee0b30cc1f928215e0308cf258ee39ba666b16319d0e21df8b9ed91be2e2feb9e117682cdc9edbc20b0b3b9ddaa93ec7a2d8f752ebaae57b7d74037c5cd64cb04746a38c0ea3ba3241319360e585c257fb593c123a653e6e0420fc540bcc0f072de6ae1a31b57dd5acf0b8607e04d21b727c93623e5227202a7c6432b308f74266dd1e1447a2d637a5259fdcd1d8729f341a7dc383b4c8a6ae0e78c34a55793043d2dddbbca1d161c300a8a7cb760298541c90a1c5a890b5257369b8fea0b4d14dc0848e644953cfd5196494e4c1b422f8e188314f2fa903fe93393193e9b76bb71c0caddba43d8586d72657a2d717e4b2c3a0e8bc33a6f8d89b871244eb04e4ecee26f9b71ae7d5c44f828083b0c1920d37cb5d7db86be3b18ef16653863762939cd1ab5c5e0cd1c495ebd0a3037725470d526bf12d21c2fbf4946ec224619c79906e8c87f0699be8a3621ae718aa232e26e43bf7cace7918ce34577a5e27261a4a6bdbbb8ac21a8a40b0c9682a0e0a37b7fc43306820b0db5fb5bbeed529102187fc3e5019185588d1ca6222a5e2b60faa2a93857bd7224e1e1c3c86128fd16993aaacaa34d8679e194ac0da6bf617144a2b1e844e7f6f8884df05ce1624ef1fe5d0895265bef2de61d637a74f78f6e3402923c9363656b7b91131dfc8f8f7360d14e01a25869f0f15a9355c07bba82e4969f5643519611d99ee9566138e4610712d55f22ad36c7acdc9e32b4e1e9b1ee9bd14ca122db05938840b8f47a5fd25a26072e2055494eb4ba916473e973c1efcef8dd7f0f04711b7e968d59d144fc04b9c6d8f951ab973441a6086ab6ab896f4d282793fb14c74322c7ff1c4c1c6d0544b7457635a6fe2632644344b1fabe12633b2f7d8fcb6e0929a05aeb2749fbc99671283c47c0eabe152fb6215bbc6e03d61b1ae955838a8b7b3bfa883a04f5156982ba433dcb21311fe9027a5b2e133cce16ac15786d2d44b5e4979cf2a8667d2976ea386eef4c85bcebc39b9d2dd216e8b4f7c68b7e0a5d7ff570770a20cc605e66470d995ef96f1e0acf6136716cd8a26b3b725db3ff48364f19cb2c640600d00d0d728d8fdac7b8a205ba8109318eb5519ee03848e79a1c33898555b298aa62d6da66188506638918b46926a99ebfccb649cd6bea1eb87fa6c1999a70937f51c4a2d485c6aa2bb5f98b1c8ae558e82287b9f34ad70954f8dbf3a9736604c43bc3062f8707fe88483b0865bd6df65f7e085593d5046554ee465a10ddb0c0b651e2b1031d56f5130e100e4544809a093026b88277e4e1f1e3aa40997871ced6e6eca44f183ee09ca679e4fb0796af62cec931c553ea788e75fd37ced477f9a58e54410c513fa79a4b5552b049fffd7d261d90a16f253a0782ec75219d1f7492ac2a242fc04c0216e6c804f8fd8f761a70eb747e1545313706136b085fc72dff1714ced89ba14a65a34168dc0e997ea8718ff304a962cbe8c7df8b759d76eb0f18155e49822d21d91dc8913c323a824da271f8c57d44a27be3f04991e2a47e5c8da2f0d71610be89dd4459b9a9f5cd4bb5ebf730a72617613667984d9a0e0ccf81c803e46425899b2fffe9158f9e3d7c36c2c42938d9433f764ed1f6d9c7d69e9ebf854db44c398c00b2612cdf19927de8bf45df70a5d4307e3a628a88a09a79d4e03742f89363d891b241383887b4447d0cf3e0eebe4a9143cd6873abfe2993b1f4d1b3b3a18aae4db4465385500ebef246adf17f1d9196f78749dba2b12deecc5c957e022f5b9dfcc3536bfd306a1a70e50f7bab1db22e98c8705328fedf3686938eccea04e7db4abfbdd939d56c0c357612ee3536356c0f5bbf1d428fb31c7b06a75c5b4fd5b020e6d9894b50eaf1df723131712344888adb5a0b4c07d23f6e853c88b51fcd56beefa721557256bd9978caca155db7f9c325f9564479c39b589431ee7d5da402e79ee6e04061c62c1f369176f6b35174b65a978ae3a4e8a25032d987dbaa5ca048734cc1da5f875f72db399df6e6a300db183e96ade56ecae019c4559c8f7c94899178c98e5071aece48a3e9f5a795159b2dfd0a1aa784b263ae5d352a890b7c2c6887d76f0364e2028d4f9046408e76efd6b5092590d39ec343facb16ccfc0000bfe7bebca2575b1f02c8a0aaaf5fb0bb13ac3246e245ace99f2186160a12792fabc0b390cb4c32f87c46f741c8fc940c4aa7843d650f90d422504bf97dd4825e533f646c6b77c4f004b7036b5480b587a913f05bb8ee470f0ab148727835b1659d8c046fd24276c95d30158424914e75d9b01fe52b34d5f76b2a1c98bc5236945fb46724e02851d46aca237fbf047f3055d6331dccab91c0bc20be2260086552fa7c958840843a3c38acfb24f9dcb506ee3cfc67f3125954d9090c1fcd629d1c3fc557b7d6ab33b9369ebc8b5b43f9e23673f688ffe50e6c3dfb12ac58aac60590991c9ce68502873fbdea04455459f6ee721570a87f87ae03a9f97f6e457ce1447358d8e3453632618701d62885ec4f8dd201e26405499dd7f317a8d2456940348827a08950d548b5c0b24cf494eb76dd0a51e3f5c6fda9f8f6545e3a6d1009775d78720f9b20ddded15c162f226873162a5076bc0cfc151e655e29d100b7a0d32c9c783375d1c6af0f39b51d927442386cc2c26337cbe5f4a34bfbb65b544b50b4947f72d839e6255a95fc859d6f1e34bb63dbfae1469be613472f22cbc703468d8552b9c691c829bdaf0d2e7e4093e940398f9ab8837baad8d9e4c710e4584685df01ea15ed9ed95ba9f6420e5766363e86ab8dfd1626eb1c7d2dba91d360821da99977be4642488972ec61fce7d754dd45a4d7fd9fc3498a6b7e0b7235c7843dfe8980bbd3a52c8868c1832f62d9e25a40986d061d7ca9d8fe29c7f234e3448fa315d749a09cf29556060307b970e5e080e90dcdbec4ecab1a7868cc5bad1037c07e17f961f3cb5641405f3d3c1439596a243f83b4f8576019712320ccc40581bf02a0788e3ad9cf4c44113d47f587e96061c61c06be84f6e35711d105f0c46b31f036356f6abd329c4e1e74f7d97ae6e4e0879be998a85257b6de6277d9c18980208e83d9bf266c9909a26f3e19941e728fe4ab0866d103644fd86b203029271361ea97fa93079ff227861aceb55e68d7454d1dc25aa8a7c780590a04756c840d13dfbde1d52d7df47d4146ed8d530907fcb13f6071e5c18acfd0a694938bca6cb081b7dbef2a646aaf55d4da9f3d42996a90655cb48144a4dd2789d75692530dc02a2635c26d428f2c6b66779bd9facebcd73274ebf333b8de26609bedad5739e73ed68798cf9882ebcb12902bed483bd905b92b3df495e708a0914a96840761da66379d85105f29e14e47a3e5273cc23ef5053077ecc513363379d2e6cb3aed80098589032cab7c20eeaaeb7bdae7e3a1c7a704cda4568138ce1ed94e944eb4ea72120ec91117338d0c86116db36f3807a70bcd98de03cc46ae9dd0c5fd3b26d78fed5a8b53c0c8f9566f921175fd193ec521c9914be68f2864715a572cd6b25948fb8dfbd75f4315967d5a6a707f1ceb81b98972e615d3e81b08ad5ef1838af673af9b6d63dccc226eda6d0b6fc1fd104f62b80662ff157999d7508f625fa2408b6f9ab4deb410ebef73750dca3e63d5045300d9b52845afc42f064bd78ce574ba9b4fce22aa3158bd058772ac2eadbf9fc77bfdf85dc745b529e8baf248a0bc1963f8fad78471116b9ac11679a326e92d87e03e316c90d15d2f9e3921e0bb6ab4b7ce91f12e7cdcd9dd141b8ef160e5db1faa47525a77a038558abd98757ac5f6313afb4f4c84b2c768e456f6de4a78b5b25f90926352fbfe83037f81b31251249fe52860f3cfa7ae1b20c73acb015821840dde2ff2de0470450b11aed7c1f84d4020147f8b9aeed4a37909bbb63508e533f939f23c2f5952ba6734745c51b837b72a398a4eb8dd6b4ee7648ecb55799cee7b6385ecb0b2e36b9518439ba550b257230885b83f72312e53afce7e7670b26342e4eb0a6286c0693c54abceaed1811418a9a289b4b09b65e84b3149cada8114e830c0403884ffb74c515bac074e4a981c61447ac99368dbe9cab8830911ba9cbb64633f444716f401aa17f599174581538918e88cd255765142a49412841deb528f44a64acb372498bd66f979d9189fa0f0e70e305015dcd3c9aea085cb0f1d79a63e92eb39b2dd2fb1b9ee9aec0213f3a4a848f3450e4a6d902f5435e234588ce4a7d2c9dbf60de27227ff02ee4ca19314d72674fb06c00ac7e5d15cd05fb35e973f0974a4718ea9b0d35bb4064a6280b654a15a7188c5d5a5998ba1f031a3baecdf7b608169c0ede84d91c9a5f73b0176e5ab2958169ccc6022e0c43481bb8e6217ddb1bd9ba8e8446aebed6afd163dbb6873d81afcb74b034085afec0d626b02fb10839bdf2279acec3d8f3f7c47fcc3ddc8cf1f08de8c8e120d9c06648acbd1ba6824ff74419776b73b05d797a0bc43d77eb75029112380ff4e87e543863ae29065e40c1b2b1b2d62a409a299ee5868036835627dacf4f9f7c84fd2f6707d823eebca0e8ceb2bd258038b137aaf43e0e937be762be5d3a6545e439a45090d397c8d4a068286836780536ad99cb138cbdc6fe5e9db3bd405ae6eb86084be3a49c8af88682dd52a5fa8f7fdad76cc0b452b3c37b27dc3cf66c57a4ae59a3687950ad3f1fdab3002fb2281365348cfcd9306583eb79a9555e8750dc95260a1b277c566babda92cf2b3ef177854fcc3dbac88da1536eb3c6d6fa3e336ed3a7a6da006901c845619fa00596af8e54bc1980aec4aa97bc98266c248bc038013020fdd128db0b9425e8f7e3165f5b9190fbf2dae5e317485d9d103695cd209d5d247ac7e1122bee4538b5b74790277ecfd2051e43a34b7c5440d1198a1875a0501eb3f519ac2f17553c5e99a9c83412200a519251d4d4ed1c84baecf7d8cfbfe4a7e50d789cc034a21043fc13fc3150314bd7a7801bf9c31aa00b7cbd9d0fa4ef0636504f0dab63316d9b9a1f4703b3be23170549fa3a29ec02f418d687589f77939fe068e41abce460fe118b04fd418f28edba4fe70d8fd24b38078a5c47ea236d73610ea116666a49acec52fbd810bb99af1dc5bfc4eaa634dbf2178151acc453f93567f015f2a25002eebc1f778da195b7e9cf683db1a85da98a99d98614f2c25b2237d4393f5799d06cd36254d3f22a53df5133cb1f43836991d3f7529cd260df88bc47e4172ab045b66e081940dff797b28c632898140425c3f81a60236b82522c0da7f0e0ec7a8ccdc73cee175d3b4543293135c31c0e0c926c7506920cea1d0a1eed64ede7572b8c77ade1d95bce10dfa69fab7fb5ba64055800593465dd4c26717e37a66089f550202a0530157e5274f3d3fdae187444f5ed641c608643eee1b361d78c9ed8070a85150c80368a84b14c9dba5b20a9059f1a05a60de938ab914a093b2e741e891c02f638c73c730e60ef8cb5d9e5031fdaf6918750f192ea7a76e035eb05d7556ad64a77109c5e13bb3ac7627d99165567dfbcfb7fdc7bd847ee7dedd7864353f8fdb7a9939867d9a5ff4d9ec9727fd432d8abb43628519cdb6fa49ce49064a421a89bc924cf6286fdec8bea6b5a7b8eb606f584d44c7194d5424f98576dc4b5ce7dccf65509727ed52f8d7210095252403023473af96eb5892cd3881f2e1f4dedd4c9b0755b70776aedead14e2606bbc1ea3c300984ebf27ca69f36beb03c0fe3c13d9df79791d828d52260103be72b832ba235bf1165250b608e2cbda0c9c8d3c96534802534080b2a4ddd5be4f269f37c4cde9d43dbb42fef8d507cce8b8e2eed799c72a2fb836f4160b5e5729192f088ab8557d51b58940065ceb4b48fec1873cc4b178166d15d441f45625f45af62c1d13863feaa8afc320bb36af0d151165c122a47d4306d075b344974835363818009d86c81e90345cf4b7d81786c40318a371fbedcfc325cf0e853fdfe6e91e99c07dc9430c04e6069b9e2efeed0eccfb161ca85d96298ae6dd67c395b78668d5818f567f2039111a7a0c52a21d84ef72dedbabe83f811184a95976b1a90dba5aca60d0313c546ea97d7b2e40b34ae88ccf4f45e28f05f16732d7a0d62dfd86e40a51de7f2b9b66ed780476cebeca547c8eb71ac20a245b1bf7445e64bce260b8871ead95f708974c3a93dcc4ee979975daa399055108854c1c3fe5a07046d13169c084b2e4e81821ff73d5d20e7556f40fa152226a6da48d5f86cfcf1a5520451c3c1e534db654b1a58adea167ffeea90023cc2b0b01710323177059e1c610154cb234d5ea165dcaebdc2fd9d9b66413b8a6848ea11a673f207ad76bd3ab5399dc0ed58a3ac0ac9d3c53781c36415d54b452a4a78e1eaa7046d062e3105541fec8819439da85ce53aa683aaed543704aa8210b26c1e6c8f1d1814595e3bce2d98a36089bd722110bcefd651937892ffd36dbc34f1284eb32c49a1c134ad460c19e01b3f3f2d40f664d90020ae8c546bba9ccf5500171ad07a55f9fd299d7518f00c0017f6c25d4aa01615fc7eec160ac2cc1ecd38bb0324e54cb466c3df4d56508f5239e1b392a50a8c20946a6db9220088d77c494406296921e3ac9997c34f15fc5c39f2529eb2bf6b41b0415e90cc9d7e8a83256cc59b4607be49eb76a7dc07c1ce3d0512888705c9ea145079085273f2236d5bb151dbe7d1bda2ed607fe51847ba09ea6431dcae727ec1ef0a201b8df9e148885e1308425d5c7fde473e90b4382de8bf5e215c3be357f8a2356348bab00a2b629dee017ecbd76d7ebd45fe4df2884e34ecbb86ea371fca29496fd05327c3b985a069a24a32380ac9e4b131ff48d58b1c528b941dc6290cbb41ab4aad2a55e5e03a63813c2113c0740029529eca008d906dcda1094571f5d2cd31ed79e9bca8e815e206b3ff3a7b4b82b0d30775e0b77260a42b1ccd0a58a91f0378df95cfc7f8dcabcabd2d7ae3b603e9061c1e126d00fde94b4b625e953b6b781da167c71a6a7ce0792e7de070863deaff94d74a7ab9dd2e3d39519a6c5c864036f7bcd4663fecd163d78a72a74d2838b6949eb7ba85dacb2eecbff1af2792047b036f77d488e5256ea16252257bdbdc1ca3d18d01c5de00ae8ce7d1c4cf103b2bd7a31dd52fa92d945da38eb9070d6f11aa9cb8bfaebe133cdd7a72f94f0fa067da0481dcc86e6eabe115cdfd8fe2e2c63c3cb3c0f75a4655eacfd337ddc82f770442d18ef9cf3993a9d1830217f50917153de7576917c533247a6725a0cb7447ebb0cf7516e19781a8ea8d1854349db9d3bc4e4931a1fa4fc075186dfc37ac10db1d3feb6298c1fabc8d897e1c806e38981c3fe7f252371b0e92efc61b2e787279f4a8138e7c6527ffc4092de912378ae692e9941d0b6d14354dd25900912f9a907dd970db649dd965c959e60f4d174ab0aeb7bc016735e8b847e231bdd240042aca16731926f537fe388b6496967b750a94cf514c9296805607b5f44451bc622486ff62e7af5c0d243503ff31d1c753d9fa3adecc8f555445298aa10a877250455c43d412719aade5a202076fdf080589a28cbeace5a63de77d188c1fb486d888528c5142f92e54f005239285cc4f8695c05ed32fddf0b0072bbf7642a1daff7246055cf35e077acb567b78c73094e583b9bac3ccea986305eb4322bcb923b93187bb3061fc00ee0b3d10702d7890c4eaf4e5f3da2143282dea0b12c3f043818914ae3ea98cb1322d73878b5d9920e87e6e00ae831a3b76484e0f69d35eb15b053c545464811f9c381337a80a603ca91ddebab47b72d6c80db4cf0646c579a136326be69bae46cb6ecbe23ae355cf7181981d8f7b8e16b3303aade349f44c005602c34878690d91fb7b4bbbaa75a91528fab18c3c8c1380c0c26a98dae3a2e441527e8c7e7df331a506eeaa2d299757e8b0cbba97a91f64e36e5a24cc83b00fa14b8a41e6fcbafab7eaa905c04c30755c551b2f0f1a24fb422bbd76090e212efa160103344200e83545a119cd7d2f53e8e5e6d89b1c75f12a44ddce7b9e9bb4710e30c853ac8d024be6f5b3d162941e2767c61a7f7ac5265d08a680b87f07bdb4d4e2037b99a6eeabef33b7ca6e7e6f4428f6814e294b7c08783525e1f632fc2a43dc057264bed23898edcc35493222a92dbe28da74c07c49fd725670ded86494fbce9476d3a594f4d96ccc3302a9e861933dcf46d03b7648bbe7bca88d9791b5a62694b34a9ee0d6c9d5600f0758e7e0a2721cadd048bf59d84bcd7ab33a5ffa8c1a237dbdf3481f03e6368b0c1a86ddf01b962520372bbb8e330a70cc684e84e1a438fc571ad9bfd297288135934aa4f1310849bc9c1658006eb1639d4b8ad2f858fdec1d5a4aa89259e591e2cc2948ff126caffa24bf41bf71b408da6b5107e1d65fcf37697d307ef3a3ce742e5d117e7bd4d095dfe8598213e0925bc9c8da414b09fd21451ad8f3531a756f15269614ea1ed728d3d37820c6e90578aa43f8f4cddeb9ce97b345840899a016591690a810c5773572f34b68d1f2f1538dd7c541508c990621d54dc2653bb4a7e9a1c637515538840f836d38333de92ff70997478fa7e4eb0b447ef04a96b2376d3e7319b49f8eb0b9c656bbaeb8a000104f3407e33c23030ee9d5d92a2aa7ff76f419ea3b1c9a8953b30a92759a8d0fd37b50e66c8f664792da1448f8d12d60e169382da435c038f10be62dd1c0430f18eb30af8c14512d3311a28d4252fd4b4dcc04b5da190a22c6eda7ad837e470b9d8ca3254f89a3e613ddc74c1865b5386efc7a8811e95163fc4c2a61b738bab3ce75e9450dfc38a52a59b89746cbb216cafa6faa0236db5b86747e7d9d3b6515bb19aee0a85bd4c2bfd76c1916e04db2e0d0382efd97d2ce79eb9370087b19faf4f269e7fdf64f6c30eabdef01c5a4f347dea6e28c48a55643ca36881ac5b283a9f85a56b203c3076e592779f91d3a00344a29eefcda31e2c66e38c6386367a5b43dfcb3978c891feda33c7907d7e4a501376e1786b9cc2ae1e0a60b9c0c979a0c1c96b9c126f647c3bb21abca0d4498c08a4544fc46ad1a33b17b105db91390689ecdbe0b9e6b5cc1d65a52a51d7d3a714c850c14c9329410c9efdbbd55cbc43e8568b01957d664de2a82c04caaf9e55ae5a34c533bc4da5c2a715409a900c7dbda950463131367613ddb0ae44c73625fc8f293e4002d5b704645c08098ead5a0fde1980d8addf0af0efa1288d569fa62d6b3f6227d1b4d628d1142f5eab4f7f1302bc97809b1b022e5976b32da3537a192938b8716307aac1972cc5b97019f4b7a9e5e650d0569ef4e7d12f8b123b1ff1d95d3d713a18dce009364a07d01b3b62a1f3ffe29130bee4f3b275dd27fd6860423629b181194a10ac44077eb2323e931cfdf64504a94a1e7683110ee8a21e67396d09b85237a51d824966527f9e3681ba9fa6db51e571d7d6a1d04d0f9bbd9d1036316a67ce225705018d1f0a9b9817b0d01dc092beb4e1290a7f12b22cc32697a75e17b9a93ff817a721ed0c696643353b80478a8d41e92b0dde35d7f2918f310e829305711683fc8c2119f0759e4bdca673ec93f4f0837aaab4c925925504fab72f35bc494a4d8bca2b2ee392af02422b2f80f174519fc83a06f99290cc7f1812a32511dc9fce9499947c92cc12c241923e62e98824b85ff3ea4979bd4362f87b7a89204598f51629ba1bc6d42354cb13c093d00c7da85b2523fee61b85573aa7d4a3145d518689ffcae21ff6470a8210cc6302bcce2459244e6f9173e1d9447726942749fb6bb13bd0d3479ac5f1e6d70e104990f2c9311046798f906aa215c002ae0b918af3db1bc591a937d5f485489a132fe9098bb77387906ae54d92d368ded939a069d0866279e0f7cd815062dc25c664c168ae4a11b9f1fedcaff8e3c37b6f7b075e737354074381a135291383a35047b1dcc7cf3b815c1b7ac187a4dcbf64fa5ef83c4fa2c0b4f5b7e024af32ab5d72e4e86a92acf858ff0aaf0fb8fc64fa12d0733c6fc530ad739df51038fceaa88e6e47fd80a357f501dc4ba5de4b6c0c573534a84e5aac913bdfd30e548c8009f06e7281d4b91b968852033af7b9e79b644b1532b9324b9788ab6ed930236c5bb2de248231f3046f490131ac34586b43a3c8e0a73cc756a816c79f3f10f12c23501947b29617fdc2a45c7f361d27966f73b1b3a1f50cb0052dcee33ce739cf2da8fbf9b7e5734c5a7fb8eea9f5a598ec6c207735409ca3401894f634fc787fbae311d4bdc8c99e8986f306883d7185a18e71b5166cb9b84cdaa3cb03231ac3053beb31ff3d87838e53ef698f9921a0ebe69c9b1526ba5940bdf21fd745f6f2653adc2d3c36340a6dd7c9ada045dfee2349e26a62b437e0c8f4fae2c7eb9eb577c20886d8a8fdbd02d245c1a1478f52e339e8ddd71fbc6d2cd1668a44e2c82b311720996b8da7111658a04f4b756d84418ef3e99713cb73d87bb4a14f69d5ef4a8b8cd6d0008164fa0a802c0679b26e155d801b40a2c77453dbdaa5a582d7c9d967e8a5706358c68c6da6c3e6746f0648401f896f1674797b0e14b354c27ea94fb12ad6d8494ecb0eca0eb5795b1920c618b429b4e95979174fd21685ead022e7ca4f87958c8a101b8b3a5e0a9d64393395b12dfdbe7e7503b62db0fa8813c4eaf1ffe5f190de55e83fc3a01b10a647d77cc49b07a258cc4e65d489451f849f6aeea5bde2aedd8e184cb9c778a3676dcf87d2bcbc19647bff28953c5d06fa73b860a16785238f9765c0fa958757b4791272cd8d1fae302b8dc47282b782d5c4c5ae75a9c02a3e911a79547df96d2c2306a2e347e83a8d8a126e6a3a21cc46e476a706af9c42a97b3f8fbcc0de5cdfb59bd6cecba11bbb93ec1bac6422b61277eccecb8b28b3d97f0051b89997c8c22a9f517d9fee5e16606614fda11940cf388c04daa1b302d3b0590a9bd0692d6139b77546d71454054197adf346bb94b4f2e85e502e3cd919bb43e8b424339e2af0e7070b035271783b1d7d48ecb6e63f3597d08a9d0677a44c29948d9cf9570e7fab1ab0d8262bb963d62e895c9bc8ea999f6f8770ed5ebf75a39dc0fe2eff88d75ae19690e0f0d52090f18bbfbe39697042cfb5293692f1e66d76aad38bd919b634ee20000000000000000000000000000000000000000000000000000000000000041187a86f715ea1475c76d354d556242c15f6b8f7c441cd862cc7c89d89a58bca02ac74c422eb01833db93e4b067e507da65080f93cf1815f4cf1dc0d53a2802230100000000000000000000000000000000000000000000000000000000000000","nonce":"0x190ed","to":"0x91a5d806ba73d0aa4bfa9b318126dde60582e92a","transactionIndex":"0x3a","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x1f02737dec59ce5f29b7e11309a33cea7b370eddefa1a8007ed3fadfd1c6b23b","s":"0x106ff6938ec0a46da92e74768b54926633945ab503e2f2dcb69469b544907586","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x2d679b567db6187c0c8323fa982cfb88b74dbcc7","gas":"0x5c90c","gasPrice":"0x7d9f6fbd8","maxFeePerGas":"0xf49895cde","maxPriorityFeePerGas":"0xcfd6ae0","hash":"0x45499074aa521ac4151138f0aad969bcc2dfc1648d22ff8c42e51c74cb77414d","input":"0x003851f8368551cdf133121c7cf32ee5690000000057b378dadc7d07581359f7f74c422f52a4d750946601a4898820282002a14608e24ce8d2412c889246951210c5eea0206041c4025828362c2828a8a860418a28208a224185ef99505d5d85ddfdbfefbb9ffb3cd964ee997b4ff99d73cf2ddc5b063a034cd01e114eddde75618857bf21dae562932ea35bbe70493443b0d3bda8e6161bdf9a2e6e1a1d63845cef77f151ab3e17f6819ec2bc7388893dbbbf49e9f0c76d8f4e3a6d5cb9c46eff41ba97c0b3794c105b1a891908a7812b16b0d1d7780802e8c7621a188764369f6bba1bb0c95c54c4e3ecad237716b875175c0e27d3cb133dee5692910d9eb1ceb4679f65e7247f51ad58b4609f4c41fb91c48acf7b7d2a8532654b741723ce4f70eb38a521e9948c62bb5d73328e2aa5e7174426f5da2a4b1c6ef04eb66b2d4d1a48f0a20cd3d8b6bff8794b3975124f8d8149ff9c0b714da5e23b8ff418940c5dadad787c4298fc8486c94c44d45593bb8dcf6cbea18578904b0c99dd61f2b7653a1562b2cfdc3dc96fb15db01fd11710145c7d6b0957b60eb6fe886a047920fc64a496565e715ce7ac34a51a8fb632f62680c9de88bce189d008e29a23679115012cd789ae2fdf6cf5f4dac0cc4aa2e092b846d3d3b36974cc62244625394645ce73c1eaeec885e6dedb131a1d420a57f945698a9e7c57befcea025554af9a4cf6a432908061828ea86605e8ae7ef301f4c388060625fc5cded2d60f4bcdd4270b2cd05a133fe45e31ff72c0fb8465065d16678ff60a4f2ebf901d2403345bd6c4459c6c6b97bc038b3cbf9f0ffcbd7fe06fca91bff93e504946e2051d05e23e28dcf2d35dd2d1c2b16da8fffcf65ec0b789efb1afec71cf481e5b4476b7df95109b64e2c1950beb3cf3ec9a671ecbebbca59aade6ca79f06328bb06669266f9fe9a6673c50f5aced1aabd49947341ec338b82d9ae31d9fff59a0511ed938e016432f9c5a001a7d5d34b54a7d9a2419456e3bd0eaa860ff52e69806188aec7e559a70415ccb6653c885fa17efcfccdb680026e43c9ad0bd5b379d4da2127d429e74dcb29f3e45b3ead08e1e85c2be6505fac13244c6e528ea261329310d9b80b16bb7b562d9477dd95f6415de5bddec30bbcfab9cb9bbf88e71eb47c16df81087fa93c4b39b5e0e1cbad17cb1bbae73eca0e2c75f73e12e89371e79a4d42babb71e9da013fdf5ff1c20a45ac00a02bf1ee09b0efb1254f4862526e5abc8cafc311bb27abd9af1464ed617acf4c1647dc8765d530e4b40fed95768dcfb24a0782f32fa6cb29e165a5745f2b44a8875a96812b99a095edaf5afb1144335625ebac2a489d9133676bffed7d161671de594b27973fdebe55eef99d65ca83e607ce6ed86d32efc36bcb4bffeb2062a99371c93d2eea8a0f47519e0b40ef21ee8d9de5e8d8da61657072561ab1cf9a4b1d51da9926a2d934cb9be7a548af7a5835758b5164f249ef0f0b53051779089dab29284dc10c24a1ee298ff1faf3aee4072cfd4ca19520b23941a4852bf56dfdf0eb075ff435f87aa51ebedab52cc3ebcb4db9e133e57c2aeb10a836d1f5484807f73a9d411e059283bb45fc028cc8f919abb8ad2b008ff7a95ca576c0c04aa4f243f10d2fcbc425ce014f5fa6793335650b99beb722faab1bae677d9939a02d3de5e0ae5d5906ba024cd019c11f0d51c89e73b8cb3609f7faf3d127f969f3beddda1a756fef93ba1a4d33e4c3992957a9c304394b7760063250a5519ae83b304c00fdf0a21e2acdec12482b724df3ac5ea472c065a9e2b7c8de28454d72ced29d817ebf3126b6928c2cd7cabbf0cee395489744896faea55699fc7ec581ad49b782e1bb83bcf535468b91703bcda0c48fef1e919dbbf30aa45b162c92f0e01496643bab3d9020b8e471695769f04020557e8d0fd5812bf346deaabcc118c92dfa146a1b5fa6aaf5db5e6fd704d073d9c6d549004066b97db487f857716ddbe6cc6332f7ddba3ce2df47f0f25911e833adf8f345677a96c7223a3efc77a3f7a5d24e543525591d1fe4afde76a4a3fa883efb81fb9e324c5727a5d2e7c04013a249366aa8f8242373f9c0aa55a7366d364b0bb89ca1da127046eea1de1d39b917fa53d6acde40da381a71d342e3fdbb19254d7df86704eb979ebcb905762ad78a852bc948b97aba45e99ea4fd7e0340ba70097b4884abc81c20e99276eadce78b6b5f075620424b9a83af6457cf29d9fa52b6df6bb7054711e789cbfdfd1bccf3081ed96225866598328089398b84ac237a46aeb513780c62f5ba256d9de6392fb333debe7b2ebf00571be5b6dfb5298b68c0c4f8a281db7b5a819b91297d29e586e6e78dd685f9810e830dad91e79459de4f2cc484ed9cb3a49eebe9b51975dbedfa1699e3c2fb9cd6a7e9ecde33677fd57e07847bdfb9bef75f07bb9f55f73c695a97ac188df3280a5e92f0a4f1c4adaf8f53bbab4682a9dff482e9cfb43db9dcfcda1a466cbfae58cb09a97d8336c88b124bf1cfff8a603a78299b6e6a9ffec57be9c75d73fa3a86049c56f127d5f3a6645bec138e6f9ee98510eef2dfcaba7f894ce575889ed3c89b3c1caaa37f0901d49f7ef9d46c955095899a76feb44cab15d918d6f9e924e77ef594b9af6e7a2bcbd2190758bcdc73a2cb5f7e25e9afd87f893b653e7d81b294fd6e0bfca1f5565a21f725312a2108c950f275a4c0a698ebabc567bc1d0e7b1d74b027f1dd7187cb584fccf0206d997519a61060628e2255ba437bdffb3cba91b64c87899fb19b5790ebbe92b0d55d2b0eeceed76d6e91aa53c6ae2113e33e299d531c4fe70c696050e654c13387f0f5a9006691089779741473c7c7dacc4503fe93cbf7565bbbc4f715dade96b451a21f1e36920bdd10f42f48946f8ac56ab3afafd84800b2a9fb6d643cb17cde43a619d7ceaa915e6f5d209ce489a8b47bab1eeb9a93792c6dc3b7676dbacfbf3e3cf43e2d68c79522b62530e423f3ac341333908eea556442afd4a7035384916fe5ea257e3af6655cd75ed06f3498afbed542b0ad0491373609b66dc14b70fb96868a8962a2d7ee4d16ceebdbf334a8758b9721bc879a8ff8bf9f2fbcc6f63a9718a35d8b5df1d6a38bf1c6123b43566a079d88b94f3870ea1b9ad8d94d2db14bf12db3ba58dee361949713c756937de91dffcef9fabe2f9714589d51e3a72c7c896ce9b1fde893f5ea895975986ed8766eb7c7598bf7371fdab658e7bad069a9b52f9e9761ae014c4c15c25b1f98e3e0dc1d831cbaddb6f4d3a993f2257d61f345b3cec990f79d3edeb2f5ea94016bc4c4448c8f9815e81b7433017a719d074003d7307e3d627653f5cae3753642527b0a49b76bae643e4f0cb4fe00bc7b71202d6b1ed7329f6ec4ae64ae463ba7cae1375d3ae4b36741ed88e3dbfc48efbabebde8c1eedd487fb8a9f42b66808936bb6692353b37642efd113c62f14a711f42aa00a0947ef974dbf1c9455e3c1ef175af9cbd8bb729f5a7069685775d8ba8fc3552e37e8765680a58eed27db682227eb6f9f12e9378bed42d4292a77ca86c2eb79f2d289b557e5be66535a2f7bef852cfa2dea5d97c52fee7124916123d91d6d6911564a5c2be40110b99de71e9d9a6203de69a6f7b57e54fa4af76b82daad4021a3fdf1ed0cb5f2118646dbd38f2ff5c7a107967eff06986db9e2bb8996f3dd20b7905cb8f091bfa1f9ab52307226009181925c4cf729b64c39add884cc606b5cd0ffd92167e5871ca3b85b0eda19eecd1f3876fb98ea4b0ba1df9cb8b5e5a87dc74399170e34ec1a2b29df609aa1f86f24fac697de3bdd5eec694516c5c59c6160a30d90211319e50ed9384a8c8e316331b2a64f65a2f2f72b3793163870b4fbec75e81af7d4353aed284c926575a821938c34a619bc75358126deef5ca9fe7b0a5e69b6a7c19935505c73df8f2c06ceb3c09fabd79df5e67c5f209579df95d20ad0491ca79df3ae257b8ac3e95aa302f84de6fc847c8cb147bbef136f355a240aebf1d19e1db26bab8ad4c6cc6d6e27c92d0cc8420e0f9e9cd2bab9e9ef2196e07f04997abc6c1054e015c72da67366699fc045c21974f9d6433740a3abc30b8fea46456ddc7c3adf8ff00b85cfd8e8b6ea2ead437ba951d6d5c6c73322d29be2a35e63081d752933b7938c212b1289d2775222261d18633dad1e73e5f323ff2e0ecf36784f52179e6bdc31c5c19616886a139ad0c632e47d0d32d1ba2ac6c0807fae7bd766ec53b94fab2d2ff22a93483fbb6445e72d409a398840f476eb72f1a0ae8143b70fd04676db9411782f83dee540fbe6aa82664c779605bf155030785e13d8f0bbf86951c62ecaa7e25881f491ef1ffec48fc67ecfe2b9247df77673fee6cb0f66f64fb982379d6deb13a79a3f893488596938b1bd7369f9b99832c4c3c23b04cd34cec0dc5236c4105ede9dcad1943c5813cd18525daa70fcbacda879ad6675aa6bdd5174f0b9e9b6c53e67f7771408aabf9fefe79e92c5e8a6638ae8f1e38b621647d0d51aad753ad3b5a5dffd869f18bcb9a8f73c76b452f45c2f8ef6bbc39c9a479bd1c66bb74f6e3f555db06f55a1566364ace5a637cabb4b6b5f422305086e8542d0ebfcdfff1d1e7ac9d2deab374165267cd49df53bcfaecc76a6139337e9fd029079ca503816826e13fb54ca2dd757e147bf8cb7d793ab299cb920dfd306ab4654bb7661cd83814994236d8b50cd984ccb5d210aa54dc73415c96b92d6079d19ad0da60496d2b64f3d687b8cb154565602ec004b311f96d330f172d255bc6f0c550afe9acd9927abfc7bf67c37ca527edf3f61625f6dd9d32ffe64cd0b10cc433411bc558b67cf11bc09f8f4d7f44bcdf6395fde7ea9f70c80cd66aed764d8dd0ce88679b5caed8fc9c2ee8e32d2ef5a8ed9efd1e418feb40aacf7f19f16c9a1b35357f8bf8049ea7fb91c7ba47d670ec211f547ddfac75bc552f487263f859af5dea16f91baf2104b6dac8af88eefba672312ee7fb9f96886ca51ead0fda9213f8aafb5bc4a7840f65a01cc004a510ee5ad1dcfb0fb650b747acbfff26a26f568544c0d24186f8ee4b169615fdec32de53b6d3f2813ba5b7076edaff3224797d524bba359d90f417140cb222acc5ea65d83c0e7bab279f4bd8b9f44e0e7c907865bb83d3ededf3f539ee85c3420f10ebfb0725fd91f3cd2f3cb28d9ea9911fe136e6acee0bb05a7f18bcd4b5caa9784b19f63dc0c47623730f54eb3f52b2610e9cac31bf560d29def6b778e3a6512fec915eff7cb1c18981292bc892893d581a311046dd9c132b6eb9fe354075e0caece17f573454c035ec85e973988dedccbb7041ce2801733da9f782c8d85491ddce93a943b1b647b41d02902ccebdc3c82b7901adb91cbad9d18b7915866a1d9098e1c43de52e70477048d02ab565547ad68c33c7138d30fb165f1e5e76a837bd74a4af58fbcf4e34203d0a1b56be539f2ddeb3706d3f25e663ed919a97ff8abe62a76be6bc6fa6e61aaf2c8ec01768c7eadc9bee7ceae30c31dcba975d6b558ea932e22a71a5c728d2dce05067b1d4a27dcfe0438fefcca5cb4a95efe83e63a7dc247570449dbed353e7efbad6ad8c981a2b47de139ff5bab370c969e154b9e4f07f813a9310a9c021f52b22829622dba52fc47814892fca57ded9158663b293356decac92b391c56d7ab26702767e30b9726aa51bb16b79ead28fb4e00b815b66ed75006e443a9e41bb5ead6975bd3f8326cbb4de758e7907ef62dfdbaddcb7b1fd85b3785ab27cc0128d4a8f0306ce3c92cfe0f708ffdb335bf69c55348e88597f711e5d35efb6e05bfb9d266f107bf8497e8ef403b791e1c3f23d6f83f7c7f2c7bdc63776dd4f3ab24178e30daebb616cec37cecfc9756e2809a6d131c648f88c6eaae019a9c3579c6287d7681f8b5a24bedd607ed629eae12f9e2bdf55ca28a01ebfa2b20cf30460621e22b49241d178f5884481f3cb5e062f2b5a1f90e4a66da24bcd9328d9501e5f7369ef94ab5cc9c42494d60edc51fa6d5ff843a8fd5d5ff89743ad44ca92594df4fd090a213696a51aa7c3845fb4d89b7c8ce12a9e57ea61d3651b8868e7d8af070f78ec48dfba75613df791126e81aefbbeed1911f5ee74eca37da5f0c478e8d9d4c6433984cb8ffef678888c982b7fdbd36c736ccbc9c74f65d3b6659d3f23ea63f6614974a4d62507951089c22d885256b29d96a7f196cdde15515ef5b95bed572ef4bd1b712467c7bd67f96b1674378d8f87a07fe35443630157d419d038c7fbb0486d5ff6dc908db1319271d24b7d8ec8b0cd7fa16d812828368871866b96fbd5fbc63f5db2e85991c937e1e7fe1dd93972d9013b5e56e69661f40026461bc9fde8a9a46f19da2be783f9f07695579e22609fe4b6252ef1ddbc0bbbbf687d9e316570db32c1563487f09b5e0ef117862d5301762222e99860629271f05b8d38d06658ead0e686113b9aa67ba89cb26bdd3e8fa4d64244cb955fef548f69f9c5059c0de748799e245be3b9cac822e2927319db667124ee46639bc3b462dbd6ba6f375e94842e4e5dfde5d409975afdb02d03b52c27f33f98f745c0d9ca2d2d3405f2babb8231fb446e87f1f97b776e3e7a862db3fc7a05512b633f75ab70bd7244aefa306fe9c5975202ef0d360dc8b725b06b961c6257d02b4d1c8867d3a1817257a7c80adf3e15ac7108cfcca555bde972db3b1bcf38bb19b35839f0fad1166f936c23877501f220ff81b93c06ad91f798214b63bbfc9e9cb6af0f41548c3772da35590ca55d22b6decc5d1d7b3149fb5e71e09a40fcd997da77eed62a8d84d9aee853c68fcbdd79b210ce34a9ac25093a57572d557d47cfeeac5beaf9eaa9e0bc29c3065f59c6190a303903918dd88b9b4c7624db2ee1b48de9cd4d0ca875095650d5c8b5d903717d33bc5b1635e52a1d989c72e3d397e29367db456ffe6afa124496a6f331d5243f40dd192b1a2e230badcf97acb3130f5e2af588fda4bac803875788d425867687a67dfbbb7da5fefd9443c7b565b79d3dd9e669e76d59a797b3ed7a1e3a580b98da60adba5c7c71d0acfd5bde9ab5ccb69fefbf6915976653797ef913ed65260a5c6286eb11e962d9358e3bfa3c8fef1526293c72e4bbba4522a7e07a5f6da6012775d5b1d487283eb5ff89e5b024e4b918e98ca203a0eac6bb26a5be64bb478ba936f1b410c298db9260f6969ef4023125bdbc7db8b725cf7e4b69e0c0bb1733f3a4ce5f6b2eb374713fbfd4f488c5d1be324c038689a943154e1568a3dca5823234ce6c5c26ed2da769d0b1a78bdb4fa9a71d59a2af1241315f462ec378dd24472efc8d17937fe3e4bf89a2987f3201db5ebf41726d27cf6c26b167556dfbbd0c00589c8fc1827fab8ddf45f95f76efec6f8d3ffcb2bc9754f9ebf26c935f967f74e5fe0d7f18c35eb6fe9b112b9767b418c56d9ef750f7725ad45de079a5d42b2569d5857cbdba2f426619e57273da74b5cf5af951e76b3dfc6843519c8e6ed6dc7b8290fdb0b1f7efeae73a579a7cb26abf03c7e7e8bdce598e578a5a6f1dd57f77ec99edba37793aef1fee973cb8ae75cee284055e15d62e19178d3707522ae7e3ee9ae5d00e9d4d82d21e79578288e3068737f317a7c865c4ce6f38eedfa4502deb116b7735b897b77f8d41b7cdb7fd07973e715eb9995a40986da462fa51b9e682506f2559526a6bffbe8b6f6ebcbe97568679806162eeb1602d386302d65aadbb374419f9dd1e08dcad0467c5f3e96934cdf94fc11a04ff3158473a8a7e3db77f79c292f017ed42a28ab2ff7d58f7ae65fcba3cb1ee97e5ef3feafe1ad6a4dfcd1061fa5d171d2f21debb4a99b5ec48d57adee17b492faac44efb1bed06b538a30302078e9e17bbb15b25d3ba8a4e9255b1ebb3905ab17ccebefd6591d9feea87e3660fe17e57bf9f60cf0992dab3aacb619b0902162a967eaa4709aa8b0c65aef4bbb2a56fe2695b51f950ea8b5544e6fc4d5d81b96fdccbd55df29cbae1ac667efbf62b51c39c6b2ac9c839cf8f85c7869fe8162a5f6a3925e16cea1e6cba00bffb4863d8c73c355ead66692428b3a88f939c136818237f256159881265b37aadff156ec96f73aaa3b7f8783f2bed00065a914715bb57d12b5f65a5ddda5ea31c67d9b9e6d2aab2c0a425ba857c471fd552097a343a660972b6d4cefe7e3671d5d34d5da54312451d414f00d3944635dad335e7f56b00b30ab4c3751ac81adfd8c33e950db63ff44c85b901983717a35575561570ef166c747d3abfb8a112440aeed6cc8f575a3d285f3a6c1c163bf745b4d79172d13da49263bb2415b00f520d118b3767240e1dd82f7666b352989c7a6fc8e7dde6c5ee6bcfd79e9ea9bf4ba342865886f90c30317d88903d4fc501a706f9b7070c15ef543ed850963cab079ef72e75090761717efdca575316d58589c9fdf9424ac0cf070ef2cc885b83e49f0c1c82e91a1a00b3ac246599bcc4db1ebdbbd70d956efd071652cc5e2d14f79a0f78614f7fb472185af9cabfd138b7705b9f9bf6725e29faa5804f8838a6e5f5ca00385808eb2ab34363ab50b2f2f9f39233eddd4e463d18d6352ada86a61dfeffc4424a22b2986f9791436fd96cc3ed2bdaa399b32dfad71bdd1e36cedf2f6996f6c2b6ebab2bb2ea4bf11b2295f77d0bdd9e91295cdd23ea548e8b8839e7dfec6bb74976a68232ca8be3b478f1bf6ce7868db922b5bad6f87ab1e162b27256ed71560a54aa622cdc91785046ee2dbbb4c8f220a9334b8aa9d6bbf73fe724f69fc8a0bec2236e0d39769befbf3128e88811c39d55dfccdefed9a8fdcdaac78c8df18882b467cff80e2ce569edc0fad9247625883cf6577f16f08d9efc2ce1f8c3976b73e789773f6c32b6d1baeded08ee286e68f046148b66e61ea93da5735c56a5c956b076d067db079b76bce5add735a078b5db7ce3328c04c0c488205ee15f6f3b81b559f9bc83df308fb6b974bce453e894adb0df2061d8b3c3cf62ef9491bf8a095e1fcfaab174aede7b64f4e32c0d84737f9d559b16bb8b3acccf662b81b5a01d5f8da95ea91483152a4c93feb61db3f5d2aa8791009f43820bce2c1076edf0d56a31311c6c40b4bd9bd65764de5cf8e886805e3c6d64d22ef09f9d03fd5df9ffeca41dc983f7d403e190d873607781e2d756177c31b750dea08658fba0dd215e28a501096dfbe83e5b51e068a650ebbca797ea4508fd1a570e3cc5bfa6d46fce6e35580e9581150013bc8004a93d12b222cd5dfffaa8ecc66fa9694daf5e333b4ceb24d584bd67573d34f50c9e3234dc9860d0785084ff75b32920b2539af3f26d5363e70b94bbc71615c564d61d5df3527885b8dae59373333db7ecad439655a79c12b0c39c0f552cbafd6ef742e2e682ea274bcd9a5a7a7de6de1f7a1ec5561a341040e5e75ec0da406ab741f458c3a11bf714f23f730c19ca32d78af3c52b6d79a2600b698e2c0ac8dc7548b658bc4bc3f975bf2ebfdb916e214be74fcdd8c60737bbd2e5bf68de1e444c28562b70515cc9bbce2cbf911b7ac2aa18a20b1dbbe556bb8e6788921c34285766b410601ae9221fe28ffb48169cac5e277cef909925dbec8f8e3b6fa884553a865f085fd7f9992b75ca1674672eea28d37f8f61ea77b3a6f29e4f9aca9bfde138995ca6cf80c89089830e59940400420408207f23e221137b32640242260e64084b718070600caca5ff4b6d531cfee25b1408cba09000202d9d4102b86a5c887832c4c67a923dfe048f2701c05512009492009007c2494338191722090085277d675b400280c724006c1c7b08f19000a003ad85d7cc85e8c9fee92540028001d6834cd63b2f27dee7d7752142c002069e81fe1fe2860086130306011e065bc4f04ffe8931f014940a25a138517e4a83911463d07efe36469ce18cfff95b627ff23866db0bce7d8e62ae2233cdefc8d7e57ca04b5ac564291e7e72f2d5198990002fc20bc8c41eb5413a9671188f6749213c2acecf2ba4fefc319604601c0990897d0cd7954dc8afcd07b1812400b3824102f84c5c88a3ad7f2392008c2f04d06824006b3ba662328481002ad589046036a326e023b91033f0780ac3194f860032fde7ccfc8986507431589f108063550b8ed6f527cafb331dc4347f703821dd7263afffe522bfaffa29477768e4bdf7bc9a2da6fac9cde693d2e97734c8c49e0460e7b1582e1c97f13009c0ae181531604c4454eb2400eb43633d0e8570d224802f4a7cb40c3ffa4102b0092c0a367f1605bff608c59fb148fb5333615b0924007b974a02d88227908c7d4602d8ee4dfc66f3fa2b1060332190003675daf775b319fea16e4fd6a7fd1fc948dff144fc4e7ab6bb103b04d0febabd4900db65b4c14f130db22f1eb701ab893e888304b0759000f69b3f23421542fe5b3c3831f0e90c96939100763b12c05e3f5637034f2601ec3e7408004900fbfbb1c782a22480fd2a810a0120854186408a1348a63a93d3f1780864601910e000e12000ad9b918efe8777a64220489d1ae9886a2100c2388d7a3c0960ef82b8480087f908c6f4c54799639592a96427061e4b0238b80910c0c093008e252e440a04d2d0e72480439605f7db2e448a134590047068114800872609e0e770211227387062e089a36aa0400038a1148afde8d7185bcbcd61bfeec71d205388cd015286806f5323571c274725a6b231f01048152393008e06021443154bc713211c559082caf70665bc93047024bb1089e3dca1b17b54867196211c244f03d9c67f53a9d20c7c46faa862b9681812c0e91433fc9b7f5412c0298a828d138aa38d7cdf81274bd2c5b6e3c9716274f10c3c1ebf3d63d45c14c6248ec6e1c479820470e69100ce6c3a09e0dcc8aa8ccafadc881af00f6f9000ce9b6843c9acaf2f49006723cac31917229dee440238155d885467fc68830c16d10009e0ec44bf71b153e8681be5a3be49a450509872cd2590002e0508402bea71214220054451c1c53b12e22079aa104a168d922da790008e5e566df6685dec2e440a08e1625915c1040a038f166d22015ca1b123c534904a02b878d0bac6704ba5ca4ce89a0470654380241de2c290002e8a0b51924c134bc74f2ebf88864fae5417623a9e0a613054f1b1b74900d7232a09e0707021fa8e9984c262b60f65f60309e058ed4244c986d19e88e302fa0a371f951a4b02b8b44660c0c043f2342108a48890006e730204d0454900f77c02351eb46749c3bd281695a0d08518e78c2701dcc7509548d2c54800b753dc883620804865e9927b2b819e08da539d21900aa6e3215c2c98898770230ae526b810e3682cb2032c3212c0bd07924f14a38e484773c6132110c22460c004d486a8ba581a9b929b084d72130a04ca8c43874c4dc7d31868d5b1a86a784c09e4383696967824092480478204709f7621a2e562e8433502998a49c713c9140cca35cfc25812c0dde242cc444560a2ffcf1c43349504f0b813209674bcf35d881412c013381a6258bf47d4ca134b8030ac2f0cb4c1eb28c6d2f1a334d0309dc54b2e5a9433ca0b0dc2f0a4e3e92ce0f154a025e5a325a36fe1211c4d100617d004c5205bba20c44b171413172501bc00814ae561e06923020ea2af32475f1d799304f00ad0e974082401dcefd09c8481a7d047f4c1ab462001bcaaa3e414547ee2b8f742c354b1496ecb8ba78eb8683acad16a1a09e0d5436b2301bcc9bf0d1ba81bf2ac40dd7b02e5bcd9932b2ca48dfcc8c053a91812c05b4e192ba452d9e82234517112c0db88b2fb684c31635551a90e108e0a8ed51db3ccc14c5bd31017e41fbc0eb72e04b7cecf1b17e51d1e8283bdbcc2bd23220008a0820ea34cf0a942f2a828cf47d8e0d3a592005e14e0ac0ac74327fa8342c142000d7420017ca8ddf92427452b12c01732491abee849d2800c3c996557be2c0209e0db39cafe9843f3e58f4a4a75c04f6a8d04f05d439ba39300befd2ec409b5f1354e6ea86d5243589447b4217e006d68f8bb86a6e452dae32e05ab5b934900ff32085808017d4363832818101c1b43b1e909c2a00d010274a1173264cf5084aa250403570864cf5dd2ca754230103dfa5504c62204b2670ee1f22311185b47207b16b0efbb2002b30104b2e7e3c7c9ad22309b2681ecd94cea361681d92002d9f3c3b757db4460360601066d5c8823958bc0583602d9330c2f714404c6ea10c89e9b6a7c192230d69840f64cc9bd4d1481b16e04b2e78ec6bda62230762381ec992514e6260263778e55b310bfdd5e08064509235f45600c1f81ec695225202902638c0864cfa53b03fd4460cc2a02d9d3b6e5ce561118b38e40f6741fee7c2602637208644f58dbf998088c6918ab91e3db33ac080c3a10c89edc1e79a74560309440f69410e36f168141546aa5d270f46b3981eca965d4e62b02833d04224b8330e806b31d6725dc30e89bce80a59ac6b3ef3e220c26416c643d2c0564c0606c3a0396dee74284d98d47e9b3d0279fbfa3cf87d9e1d1d2b3e90c5846e1bbd2c689daead15223b4b68051fab61f6beb87d9f78c9462b028fdb1c9a598d9e3b56114c6793b3c4abfe087da302e13f4f6e3f44f5c880011c6844f9405a12d5d47cb7a47ca5227ca12c7dee3601b6d67ef0f5262ca27e8cbd219b0ec4e947edd28fded1ff9ea99a0ef4c67c072722e44986b548b982f3fd48f951ba7c74a8ed3378ed063357ea43782b9712c49b0ab26de741c93845b67a48c04733b8dd61182d6eaf65d1d64987beb6869ca0f28c11e807904476a393fd1c2d9313df2288dbe59fd236f6f26e8dbc638e219e3e3f30fba62931aa767131da7bf3d42cfa6f223fd8a09fa65e3f43da3f4841ff8618b9aa08f44b510e94284794347e9e37fa43f3a419f3b660bdecfa82e2008c2929d60b60b0c58ae151d98cb83108e9cce8641873626f60c0a6b0e27239d41618d76a8286dc728ad28ccae432043f282303b3701060132cd890d4373488770603accaee342a482e9e3e303063a7c70604020c8807020992604b38b13a80c2c242f4605d347b299b11109cc6e05f32f43bb0c98dd85121b0bf3db8c6605e339f8f8e0860cb3af1f27a6c55263610131d42b52636363618143a3ef4d74d4c213ef158ebf578b36c25e199bc8d222cc7e2b9d01cb9bbb10ffd830ccde020b4ba0ffefa25261e19d233930cc01c2ec5f589551e1995dac71276b4c0973d8d3d1f16dc25f1f5bc2a26568ef958ea7c01c7e0498c33396e54430c75a172299884a35a61298232696e59074215198a36aea933630472903967f3369ca06adec4e5c1c0b820c3cccf1383e8e256c8c59ba65ef85ebd45b9c07963f6cb9dc5bf5aa14f7fe507a917f63827a45f2fb94c7f13f17c953e915dfb605898971494990897dbce7c82ccc0be991e63917efdc91b93d239d01e308683281b241c3522020deedbd285914e674272c8600b2f162b4af45610299d82f86e42521792a284605d1de168f429b938c56d1e9421481392ba7213e6709035648f85e7cce5bb058214b78ce96044d7a22ccd93c0609145e10401521c35cb204fc9f4da6fcc9d493085114e63222c05cfa09b404984b7bac521435309703fab13c217ef4a77d0295ce82301102514befc8c48f0dde34a73585230273ed274000cc750fad3697d5f9fd360d85b9ead219b0423e3ac81f1f2dfc23f32430b7ea8817716b43002c76da85887e5f4e85b94d584e441b71616e3c75d49df0539e86455d1b82d8c84e3077cad4231a77d97844e3e1188968dc4f7f88683c1cbf8b68dc9d3f8d68308fc488bc3c38587cd3a4f0c0b3800e714cf2701e2b02cc63ce320fcc633de6e063ba8779d68c062cf4d5206a1c2b58a1a319415198a7781a88e72960c08ab693100ff35450e3e2e8d8d1783739ac90619e87637192e7156572501d23d19cd65ca508cceb310d5e799d19b0e2f9c9bcf2fad168b078ede401120442c0df9a34fe7ba81eed1c788759669e6648c0c27cc1d3d0079f3703565a39591f7c1b282c274259e08ba5d16830bff6f4679061be5a02cc77990a8b3f42ebb9054ba8a3fddfdce9cf17c3fc9204989f8f365213bfc8684d73d00fc5b1a71a230d4d9e0dfee3bce1825f2e58fca91cfc17a7a14dfed30c58e9d977b17f9ac2ced0984673331419b072e8a4e6c8287263ff0e72ff06ec7fb144f227da8d4926be6c7bdbb58578cf2e6ce916397a5f3356a4f273b19a1c11c773a3fa4ad92c1a9a810a707f9f811e1e352f2ca048a78f06addf763f141158601f8182c63681d469a8582096012b374df60f81bd743a7d34b611c73ad069824a50681a2c087231e059c17f075482e4e934b78101cf06264b2cb80d8d0212c6a3fd4cecdf0a6e2c13084d477e212e063c3b631243acd92072dcdf442a049063ffa185c13f513cfd2fae0bd251d40b0dff01f5b0b0309d4e87256c5d4651070be3583de9cf5b81858d5963db3feb358533a76102e16d0c78f6d02413fc85303e937d3a0d7e65c02a6b278370a610048ca2706cb9179ea9349af0fe4b167be1990d7f34eaccd72362d98ee5a5d3d4aa08711a5a157160c02a5f276b55c4970a01937c9bfef77d5ba47b3a1cb53260d5d41f7c3bf6effaf60840b0a3b69c667c9e9a838a227fb4a5e8191a048c7be898468560d1e1696844b49f01ab0e4eb69118272ce138320e17b39a464d62260c58cde7fb81a8181c1b0b01202ce1c548678c0c9c59393f048064d48058586c2b0102c819f80c3c1e16cb4353733a0482548a13451016db4980c576c06a71dfafd4b2888919f80cc6d84ae0ff2fbb5c60f158880b96941df99ec6b203031f43d67cbd88ad71f33b47cb19b3676bfb3cbce51367988f5f7b2b6496ebbda4ecdbbe14141fe295dfe3239b9c8e1fc94047f37b0995bfb959019650a48d4cdf482e18c9d7258c200e58427f64c8873e449f59a1bd375f94cb77b9312cb106e282f9f527d6ed59ebf35858228ab5a6064b8ab39ed2d1a70c3c914c15148525eea2c37e89dd341a2cc99acea0c31247e269b064034a0b3a0989c212a504723a1e96b8c15adb8525ae8d4768221e968865cd7508914561897e029e064bf4c641602c2cf176746a60bc2611a2282c294480402a2cc9355a880e23596c51d06f63cb32209ae57f17b947e4935c3c6a2d322cb9928e625eb2df8528280a4b16fc9fef94a0d22560c98a7406acd6ea42a4d2c560c99a51354c8c31c9340870a08ed481aabb9b024b2d4547ee1314233b296029be118b482d9ed847014bc98d6a85e59852f309b0d4bc1f1c73d2168a1fb74ffcb078070144d6d85f2a8e40a682f6143ae830b29a0c4b05a0a5743ae83096efb2e6cb68e0c422252c554885a556ba1061a9524a2c2c4570216ec7c3523728b1102e0e84a5fcd0927b14582ac38548ff6eeaf5bb053e58aa9f4287a50e7fa7a96f631f3f758a492ba5b0f4022a2ccd35be38084b9b516069551722450496b61d5d3d632941978187a5210a2c6d38b1be47197f6b130596f619798bfec35b3b29b074e8c45b64c884168b151685a54f1360e962586ac885c862a52a961a07c2d2f62ce0423858fafec883f811238f454b8aa8e878bbef29b074d31f572361193ed662242cfd7aac5558467a84c33f52ea8d50ca084fe60f80207a1ccaa18c330196711ae750c6338e1a3fc2e176168732ebc61fa02c9bc032b1230fc659c667fc61751a96c9a5c032693f725239cac9be094e201c95ce62a38900cb3c9d60e30d9d1a3bd1c8d8f4b3cc109565bab18664675060993e96516425ff6814590d0a2ccb3e49e86fdf7fd020793a280ccbba106059675856cb8548a4ea32f093f771c0b2110458361c96d543d9c2d1c0743c0d02a9181a8896a51160d9545876910b71a29f238f28971c97804a255b4c80654f8d4b255b99404d0461591917e20e967265efb11ea0cacd642957b665e441fc88faa13850f73b7ee4b004580e034b635d8844584e60a43a13d43419787cc60e7cc60ec6f6eff42aa74380e5168c7320b77444aff693f40acb398d38c41fcc251732622e39eb9f005b2e9900cb6d9ba877cf1f802d776cfcc104b0ffce6e2b58eeebf79bad607901022c3f03963b3bbed36adadb5460793b2acb67d2f1b0bc1b8df575d21601583e9c00cb87c1f2a213cbf6accd0ccb03e1083f1bffe075863838282432781dce7ba3a7b7b757042e88b5992106dec82259190207ff9c11dcd8960758be01d2a0b16223cac4732a2cbf114d0ffe74bb038cc35260f9ea499b1dfe10b2a9149a3e3a7e6005665613383d5a2c8ce361455f9c192d960ee3a45d88306e256d34f432262d5e212ce3e302d1c61259c687719b50ec278cfe88834c68603c2b9ec0b8ed104403e3583f5006c92c068f50e871304e1d65316b271e3fbea2301e4c71d72930ee34cb6f71f726fbad3ddaf42b0a8cbbf46388c00d1360dcd038e414f85950c6354c82f20fd6f10f5e0f07fa7be1c2bdd7458607e3d68f5a2730f2977f363ac93a0ae190f2b87514b6506105b15f5b47219b022b784fb20eac707a1c630ae513181bdd1d022b3c24c00a0f7ec49821cecbdb33dc1b8ef0f6c2c18181211be0604f6f1cc93b3064c38814ac4d34539342d1009a352e85a219155668fab5148abe145851635c8a1ff4ba213c24d817b72e24c03b78d4b126daca81a427da3a498515837fb97d0756aca7c08a59136dd96d08864981de86b860ef0db8900dc1dee138ff8849bb8646a41fdd3b3415e995f4a0d9e31c29995261c527bf965ec99b022ba9fd417a431c1c1a1a1eb2de1be7131e12f4dd2ea6118e7ecdcff71cdd98842aa57a2aac14f81b8e3e5360a50b7fcad10fbbaa4638fa353fdf71a4ec06298e73a4ec4d8595867ecd9172220556b6f98123ffe088481f1f7f4f7fefe07513b0fd1e21cad721d989b6ee516165c6af11a2fc89022b97fdd0d6ba703838c2c73bfc07838c22e437069924fdac5593fc6396271556fefa6be967c553e059d67fced11f0d32c2d16f00f21d476f20a5098e3e51e159a9bfe668b624059ed5fce71cfda19f228d45c251f3fc9ea3d9d193bc68763c159e8dfb0d47c728f0ec909ff8b5271c1838e2d4c121eb587a6279f92484cc7e0de126daeaa3c2b38b7f8d1015710a3cfbe90fd24f61cb21ace204c98fb7a5b2860aabc8fda62d1a0556b1f8a12d12dac1fc101ac2bd23467d1198b22faab4430a131cbda7c22a49bfd6b4aa28055679fc738efe6877d4e2231c797a4f9923d58d93a2832a8d0aabcafc86a33c0aac1af0078e1c83e1d008bf10b45bf642adaf3939319b68ab15129b68eb1d15562dfcb53dd4665260d5477fda567048b0f746ff88756840f2f7fabe2d35fca448a446a4c26a52bf692b8602ab994db435bd4d9c5322179e9c9faabd1bcf1dd406c77307589d8700ab73c36a68ea4b45536456e23074c8cb3fd4276295a2300e42f2d627d33db73d2e215cb5582376872c7f78913f55a7ecc1de0bdb63dae58dbc2242023dcd002e6160412588987173ae8e5b27cf4f7aff45cd789beb623d3257a7d987e01bc744a564495bf45e22b622f60c4299f92c7e0176a0831accecffa49c527fab463d9dbd2e79fd368b79a54903090ba6f507d4f92ffd8e10198bcd14acf8bb6b396a37973eeab665fd31772387554c5dc6d1a3cd9ab752acde3d37601a2d8f84c0a57bf5380469a7af9f7b89c86fc9b58b0c192ccd59a7286c622ab7e6a09ebfcfc0f21b9b3e5992c29f71debfffdd611b7a13876d0ce800cb5f8a85c3ecda17e7473a29cbf408a595fcc70edbf8e7fe1e58eac4ce7beca97babdd2d73308addc04600589c8f0531d8ffd2611ba00cb2cbfe17f73d61b4b51de4257e51ae175e5774fc57e509aa949cdf1d8651a12f5e5f107db9d63d717763b87540f7056e97eaaaa13967a8d8769d756bdf3d9a79c6edee968c37a0cb9d3da27b9f6f5d5935bb5c2e7287cef5eaab83fb08363586bfab3f7cf0525012670ea652ccac4c26c6e5f055bb3b4c25ef973b1c3f53da9e2b0e58b27d7c224917afbe5690beb725cbefe09b6371a79c5e2846dcd8c6df41db79ffb169258898da3baa42ebdf0db917bb9a2cd5d077a0dfbba64e0cc3c844b7d8749d9fbf401859b196193df322d703cd96c48f85ae9d4b86b6bd791a95b369e9c0fb28a28b69ef9cf18301664deb608016bf37d71e0c959c557940e6cbb1a57cec4ade78b41244f25579b61edf6a6a6d76d63ca67d97294dbb708de30701fed8f564eb87678ad6442378d2936376699f3a9c382fec4c63ef70680783b006da4dd7f56e78e75ccc9c1758869d0f30b11ac871f35bd1999e87be191938dfd876f0b0ddd015419fd398c7b6a53efa56f77a4ee54ef96f873d9898e632cc130c13f370e4fca7ebe38703dcfdaf9f96f30ffe41ffcf4ecbc162c1bfd5c8df3b2da7c7eacbaf4fcbd1bff1ebd3706c9ffdb2fcd37a95df3958aae660c1e0aefb222e9d22e1bc5ca5ef177a540cd51faeec332c5ed1a85ea4f676ddba35fa37d9573ad7dc32ed345ac00ff770d739c8e4ea7fb4b8bdcf791934f0bb536731361eaffb234b29abd5df955fac2ab7b19c2556cfec5bbde106ffb621e058506ca45b489b4468bbef57be4367ed45d7dc3e15f56691f5a5f26d5767ae38793c31d3c2b4928c34dfe792edf7bf8bdd30e0ec5c34681054a83127ce17283930c7d007ab0e3db6463c664458e262a5afc7a45775d83bab757a2bef29e5fe68a69751abd84d0aff5c58862d0198d8d3c8e6dc35a56a2e72e27d43ecef9bb21796f6e2b4df735679c5ec55727cf075c0559d46c7982043f47bdb0566a489bd0aeb28a433fcae75303addac5c5f5fa06e5ede8cf3800ea11e0331b1ded33a8d1fa3504c6afed92132afd72203576cf6166c8adb6e9555bc252167be5bee7fe0f449a4f7d529f2d5f52fc436ef375ab75144e4e31ed1f6595107d3a2f2b744b57aec588688cdea71e0b0376ad0d0400849cb63edcaeba859e40b8f3e712be8b31f5f593d84e61d6ba79577545569633092dafa1b485b07773c3b9375b1b6559d9577845ea687b5c52a8add5ffef9eaf28550650455dd97f3a2a293b9ddd6d32f228ea722162dad9f940645b784ec5b70b92aa6bee240f74a64a1dda2f925bc1ffbb615fae4a2bc38fd3307da2cdc838481d78d8fcfbcbe92a9738570d113b83958cdd79122eab161d7994b455d089758583c367ad68b8e60da9b3de96e4a292b6bed77776e5ab0203aea2e9735b56de4a493a07ff6a4939f9d76ffaf38e9e4f5420d2d4d0d3fe7757250c93262c9cc5ebb0e19cf83afcf6a9c2fb6555bc15b82cc8a765d1cc57d4c0c4aea945dcea66c3953f5aa6258fd76a7ba0f518dc9ca6fa44bdb8181570865c52b2b719a4707e7cc674dd6d0d04be81c59dbfd5ecff1add6dafe8d36f724a7ecb7a4819de3b70d89fe95db86847a8af42ec6eddcd8eba9a2fffec54cd307a4b3d44a32d27da963f95a5b5dbfd21d5bd45e7da9d9472d7e5ec2bdc6cf4eb5b7bf490d799b84444bf2ae6e911e6a6a150a65cb977bef5505e1d2e6ca5f176e1754d1bcfbaa5db20c0c0298a03f32074e19ea482ef7d66953ace9d20a95d54830da146fabd5b3b6e1907069c9bc5d5396d48b094aa37ea1332dbf98c7b620a0b741aa7afdfd0a7f7cbbfb7d0fcc8037cb907b4e5e153a6a6cccd979e919d648759e5d9ba79a68c3870637e9baab6edcf96b3d9195cbe1abc78fe6c404d1d6cd33ad2c49a6ea362ff1c25dde7cb630fa9c5ec5ae67e339941cc6671a3994a7b4e422f752874c152e1fb354dd7a4422ffa34b2588cc30d4de56539ab1ccd43f4c90a344a9caa65fb238cfef4a71824c5d453bcf85b588f7cb8556a722dc9a3738489ceff802710eac0fb9be769ffe605a20efcaf5378343c7cedf7fa6e45780177eb2e3b367e58ea36def7431f9e7f6de9f41b821db53798693ff55f994f5ed3372fe7ef0f4cecefd0b47184ded50e8778a09af45ab62be7eb469616fdd71fe1b368590cad96452edfce66c5d9155d02b448291cd6598947f50b2e8f40bd2dba3f2dec75f0ead259e6aef693b659bc85e2d5986f10398182fa4467df656d292f543b948c846ae27ecadbd315629214e86e6af2d16087d583b77c99415e4c7c4888d7799a47fe181cd5e87cb35f15bd6ee7549da2c58a7d3fd7a79a397ff12eca73cb677973fec7a3b7f0db24cc3a2648656a80e589692641f722c090857e9ae52b5caacc5462eeb4dfa701375c78069b96368efe226e1c30098574ec2ee513bdbeb3cb72e9065def5073fa4be316e13c3c9ab0b7743cf5b6eecdc2d6c17b0d7f0915982baf909291bc4daeccab7f3f7522d021ecbb7765db43856a27b8fa47162a3e1a0c2a607e75a375d437971fe278e7d4f42b6fb9a78bec6da1b13b92e1eca6a2b709dcbe3a12f351454b72ed0d96c438af23564eb7d8900dd3df5ab0b6fb53d176e219c5eea6061aebcd0202e99c3cb4af1a4b16b19580d30c12bc88288e604bdeb98ddbe0c53c34b47d722c7ceaeb76d5eddb31c6f9cfea1dfb0e9f094a1b69609468e1f74a638f5dbcf4044e75afdfc17caedf39d06dbefcc34715ff1e933fd5c5535ad375641e3c23db1b52b9079e5865c8992c72ed5e50ce95ec67646a5ddae7d548995bcf45c0eebadfc98c4f393a4d0e84f118e3be9faf6ddcf102e52ced3f5764df2fa067eb5a75ea70d050ec4b477fd070ed12ae7a9bfbc5baa90c3345a79cb97e75a6c0fb27dd964aa579df6d8bb82e2f1a6e332a2a0f526a93455c131c93e40f644fe0937578ef50f4c254b1845a09ce02645ede2324c1dc0c4d4200ae6f194bd6da4ab7b376e5978a2f9ce85c08e6df8a84fabbdf44ccf2f9bcd85659fb225039998ad932e5ec44ce568ca1fd3ab85fb7456e70b06cf5a73f2746a90bfc99502d703772797fb24f069e57f4e00927b56fa155c9e9b207d6ca7fcff7a7a554946ba3ccf168928dce79fed5473a262d623e387af7dc959331e241d7fa57241d4d12d1309ad192c22c9ac320895f556313b223fac82975948d4bb676c6da3aa39cbe64b77e94eccc076d6881f8c9d04d729274306af0ac3bbaf34d6b45bfbbf56583f2f7b11ee8d6b258854b4be6e93c2a8a5ad585825b7babe46a46e5f6a99ca7d99e836c32f9a335e7a1d46562721da1b2e447f34dbd8e5989ccc56646af1f90ec5a64cbc546aa5c7eb3e9d323438e9fe33f9fc7611e2666ffd4b5bd602f319cb3b0f54b938575f1d006f495eeef572b0c699462292eeda2dce6baf9223d66df4b1d551106bc052eb5fea751dbeec7974b6a8d4375c19c80d304176c4eee8c74b995eaf8cd87949d9333d9e35f2eb29343d152268390d2f7971cb392568ca900e1e282d3d871938cd3aa5edc5c4296dd41281ccaa334ea52503fd77025cad78760824efcb55996f4d2e35e7f6c83bfdf7ee5b901faa04912b56add25970f83771c7900a3e678dad2762ec782282535b12cf5ca075de6ddb8f2c7c69fa7c26b8cbcd8862eb47ad3db78729b142988bb24266c7ed93359a9a3be78c5c8dc5f886e4add54c8f28ba5fde7c83cc10ebf03a1b156b5a74fc134fb631b7ed9a8d53d644e8c8d5582153396d3d09a940c0b0dcab5c3565267616774379fc12be84bbc794a40fc52ff4632e75333744a264670708cef5dc1b710d7ce32e8898a87eda8a0935ae6f280998c9c7b3cf6e43198727c0e45883449f6a0ba3056cdbc5bdec80b5ca5efdc282aa24fbfd166b0e3ce8ed64323992be4c99ff702687d0c8d5584ad3bf1a4bbd63bd5de5e75abb1ec9b977973614fb952fb6f4ff5de2fb5f8e4953b91a2b09094e91cc2e7838ff3c69d12b45be450bef6725c5455fdff1a5e2ababa8515f4e6323027a3f2c189c3b3baea14b48dafa74bb732678413c94db499023e2d3303975b7c3a4782ff1d72edafddd70f9d509bb66effe0e1b6d46da716ebdec6d94fb73fbffe7e33d88dc5493dc4f69e278efee965e5ab85b7ed329bbaf72314733535f6dfd6492fb4d6a0962dfa4372be41ac7c90277c255c53d350d408ac05320a7a37cf9ed5514c69e24e7c9eb3302d594bb5450eebfbe3e03fcdfaecfb003580cf8df99dec5e8453ff53cffabf27e05c6c35f94ebcbd4a925fcaa7ce3f69d06bf9b7e6513b85560bdd93a49353fb7494a6b1746e831c702c6a1d876e5b79ebcf147b419d7ec367bdc3e367859be6fb82af7f88e142eb9ec7589273adf9e2899afc53b58dcf1bbfa030d1c33d7c6f5a53b1fef3b3d3f4a78759d74dd9138de77b7cbf69bd9b8ef8879be38f49b6ce2f2b673164a12fc396a9f5235e44b186f6f37a8d56749e429bffb86af242309183933cb8a8b579d95e7151f78d86efe5e9ae78d85c8c5855497bbe5f03ead14c45cafa7e3a094a3fac1baccd5c7fa25edec1b3514cfcd5deee4c8b1fac2d20bcc54345f709956bef0b3cb9858f9c2be8470bb752598cb745a97ca9bed6fc181f4e5fbeeb93fbdd61c124d5d3927a41b5987e195589496714d78bd2397dda645ba552be9a488c88d38f5e379490afead1bcb30960013638ed4a79a75dc8aaa5829b541b9981ebf6c16ce3de4f66ccd859a42ca6e09a6ab3a64a6dccbac6382fde3a30acf298c2afed7c6cdc203d073e52fc9c5d1856d2ee485c7300a96aaafcb94d9b59243ea3b691e852b10310db7974d7bb34befaddd3fd79c229e6b2caccf99da7a2685bd2d64cecc1689443449089c5a9270a66ecfb7c5f854bad8fe1747973ed7fe665ceff9f298a3fab0ed5200c833492222726aaf13173ce1cee60c3e7275c9f6e72909bb0a4ef03759f4dc4c9ef3e6957786260a25bd694169454f0bffaefd5b949ff1efdb9c9d78b007a4b8e6b278c9feb0dd25e735247a63f91ea5f543c6516f37bff5bba5e4aa77f6cbeed715aa05414884220e5ed34bd3166b65cf6b1c50bfba982bb167dfeb3aa3eb360749f74e0af18edcd3736cf62646c62ee5d4cf2b7d9f9d3dd69037df85bfe2f53d05db7a24660dfcd09e7bca405a5f598639013031054834cd27d840ed65fbdce1ed29efe35d1d0d56b7a99b15dcb50c5d56e4561b95866640a688dc96eb7d31a6550a671cbf3dded3e16f5d737ffd16d17bfb9f25ee2bb2ebecd61d42abdcc8c41047ee54cc9db8539154b7fa1c3692ca79d0dba495d3337d0e8d0cdbfde14ec5e06a7e0f4697d190b05ca3f991ba47ad75067dbc2bb3de9e2bee7a2cde607d4216d9087f783317df7621578087cbcaf7a3586cf2d5c44f43c71d92e62a3b50931edd1cbfac80f2723c83f6a21e2afd79069db37467a0df6f80ca5f494616dfdede5c6953f16260752e9c45dfb55d2f89246bafba738ea543f91d2b73ec5b8467f683c7cdedbdd9db3747cd122d19d4513821c9ceddbb555066f3f58c82f9dbe78e4cfe87feb393ff06c5a11c65cd7389d5d66a7b8efbbc78710b8a14f8174cfe27222efc7e74bd1c98f3884467b638e3ce7be9aa1bb3bb8fad70b5a9ef2a983b33e50e22d51dea5bd5faad78de90fdc10e47a719853cf88557f248ee811ee73755bdecd319c3abfc5094a9652370b1b3dcf26c9c86f2de70430f8941051d67cea297596215379029e3350ac56bedc01de5e95fdff7bb7cfd2fced426223c5a417509f4dd15e77862ce6f0bb7a19d3da3607cd46367c8b2afd64bcc86eb3088de270a5620ffcc1981270f36e987e55fc8bbaa3ba810b7dc2a391ff6f2341fe41bef1c38a6d039fce93ae47fe5ca7932b2abd8f4e3f2968bd8bdc2724eab07f92f55fb9bd8149dacb64c322a3bc07ee6f900e26ad07265de1b33eea017b73825ece4dd42031b938e852df93420baa433509bc2bae38df0cfdc0ace5d5630fc499bf0799ec0f5708dfefdfb1f2cf55e57ed946c3fe3b2b269eee134552462561897338effa6dd1dac6b67e497b28fcccd356e4edf743bc55e208df3a51ccb40358009ce426abeacf7f6da35436c2567a30562fb44f8c4968ff3961cfaec9fad7ef6cdaaf076dc94f11a3dd058fa1033508f5a5770c2ba34506470aab81568ad891f72af987f39e07dc232832e8bb3477bfff60a43258844c62f4d6abc7041c820aa7ca8eb90fa4afb55a5060d73bcdfa67d01f1d22e895710f0a29dc40185561ba6733bc375f70c77cf3031d0d19cdedd7970e155c62ac7e1328e4b0093a30c79f2cc033b57c2ed6693ca296fa5a153cd3b8779a343797085bea4797d6050d8d929eb6a2b9323a0740366601daa2cb649aee0c09509983d9c117fad99eb46a096920e277bb673a08564bc28fe7353deacf9e44a32725ddba6fb2ae35dc2e2bcdb7ef657ec8f3a398b8a6fe67f6a305ce0ba93c195aa83ac53e4d7319e19b4ff63b8b017f6a60878bc2d7adecd4d34e6a9259564770c1e9cdeb46f74806ffdcf7cb029d5bbb653c2966d9f907d52407ae867cce13cb5ffc0b4ef5a3ac1cb77bd096753e04b524b4ccc815877aafbd61acf79f35e95cc8d18e0a120562619f47ba927b5aec327331478b81047932fcb2ccc17d9cab8ba8349ea7d8746ba3dfde9757bbf9be0f8d7deee71108b7de8ca57bb7585af469878f02da924cbb59cef3d4eee8cae3f24d430d3b5105996bf5de9442155f6e88b9365874f78e428a5ea55dd52b418548dbd1e71ace034f2dd364afdff91abf8feb93bcb7e76151f0788fd6fde59f69babf6def7d5fcb2fc833ef9d7bbb4dc7ebb8d727d3a533d66cbad63f39958639f2417d0698f032757575fcf5cfe85daa0af87d5708874bf6af5fb85079b7702b33c81dc5df197575665c7a8777677e3ee76b697fdaefe2dd9a9fee147bc1fdb575d7ec031d83ab4dcec6ca67db4fb01f3fd29ef2e8671399a9d5b32e38b2e8f884bcaeb9033253afeebe6846d4ffb76ffc5b194d3fbd3a50edfaa242338eb1505f72fc845db87e6ceadd83428bff2c88b6a695fafba9d05ed73f63c73388e2cad5f926960659fcb5917a0dfdafe285fd3ffc2a60fc9ee9a51f5a47cfd4739acf5f1b0ff89f5f144c4fb66d5dc01e5edade5f107dc43afcef9585ddd1961f36ace66a354f7878577536621a6974e35d816ac12d3098e1172ea911c7cb2d0acbea06cc6bae7372ecd3ea854d930b62e40caefaad6ac99d958f5b44d2023bed3e77c9fdf4deca768e323891fbe896337c64fb9ff224f5e176899b42e70f9c49fad0b6caaf165fcbd750180ad928cc80cf9c6e49c28b1e08ebc40e2eeb06d9ba9e4cca315356397a68e0cd675ab6907429891a8bab7f2c693e39ed677452b39795c783dddb08015724f29bba38efd464019d80430c146a48922e81e6cb6c87c83e860ca37bf4a27d3d6fe84fc790ff82b4fcc3159c6afdb39655d509960d27897eaf52fdc2b90adb9fdd0fcab0642f34fbfb8850815f7f21f0d292c3911f7a2e2ddbbc4c3e6f50a888bf63259cd0daf766f363f6a6d2dbe897670556fbacb894afdc646e2a0fc81072bd0b476d5b4d2da19ab92755615a4cec899b3b5fff63e0b8b38efaca52ca8575eccd6f45ba9480adb36c41f6bf964edaefe07ca225b0a45407089369d90ba03c1cb9eb23750ad5ff385d1631e98356740acf123d476fbc5b1c3590ada816ef53565986a8089b982043d894ba3088b75787ceedb967931f623b69caf77c155f8ead2e2fa6f8331b93e343a6629d2c8f398a1608d8d9f4dcf95a05012771cdd9fae4eaca7debeecb9f1fd92ea6721a879e94c4c64e9da01bfa0a94deab464608ded756266dee0caaef6edd735d0e1bdadae6063717be1e6ce050673027510d7fba73bb35d796c1a1fb1edb86bb0cb6837653342eb2f6fdf2390d6ac7ff97c0b1a780ca61778fec2f2cbd436e6a463d85f3ee7f0ac9634b8eee738ef86eac6971a430ca399da2d8e0e3912d9bb1d1005a3c53378b7b5f4377f7d12ca7f29e78a28a088b49e594c8d7b7420b7e52e44fc8b9bd17fba918a8cdc483ebc45c3ed3e59cba0d372e92a8147a106d5ff8fbb2f0f8772fdff9f194b832c85ecbb644b96905d681864295bb652b66c59b33396c328644bb678b2644d945d458821eb945df63142488591e57739ae3aaeeb774e679cbabed7399f3fe6afe79ae79ee7fdbaeff7fd7abddfaf7b1edb92ea14987071ea8c13ae760180e9cf04d615f5e48c0a3067222b9f4ca90b43032ccea3a43229571218527b78ab0ad7f3834958c44220f1fa091f2bb4b48e940f16848f4bf1f8620fdb67b3741620aa54e842cc98ffbf68b83f4edafd8018b26d41a01fbeca16b49b1d7f0f9ea4a1631fc813a937f452dd50577479dbfd963af4b542ed2903e1e09b6c6cf6db192f7866318e11ee628c2cd242cd55a415259f02081a0cee86df389283f93c5a5f0d7e00c281ef0357aa9a7a79247fb3f4411fbfc62da7abc87e72e87ee1b08b4c71f475610ea9462fbc6772180eacbdc77e5d7f6dd127cd3296529da988786d8cddbe6e96355d3d5a8ee83f51f4813d4d3b19e4dfc12c3daeb04eaec59eb8f462c2502e3d54e6c5a171a608394934a0bb22466f94d63d3c21debdc46370ddd1b53c37a0a1d4c88ac67ae3c5333462ea9b8896b97ddd223c4a9d4885c572b455285347e24a610d04b97089c99b533a10e03a873746c87d229afc9f89e81cba0c354191ae561356034037a1c489f0158ee8a745340200483ce9df7d9e8d70a0975883d1896fb55dafd259179b9e164c8c8c2f51862e03015e386fc679e32a7e7a018ff366ba48e262c226cdb6279467259342656d8873aa0950201c411330d299e5c01c1948155071a66004c910529d9849d95a363f9f365a77cc93f1550bdeb1ba8523b8f9070b99dac7423c607fc542b20d1bfa7f928510d58381cd6d53472f92b02cc97b56fdac19132a67fb8802e8275e87c355830b9b97d20d00e7e177b1728d2c27aac86cdee4918c8a0c9cf88dc67c715bfeb9879931f3e680c477c717fb411c5f66fef90bc2be304b87a3f04469d326d457b832f3631ff7c4582b8af6345de9c3802d9629c9fbf69015268922da35314f48e32dee9a5a68054ea9e5c28ed74ee79bbde5ef74b0e5ff6735a8835cffd72eff73e9758e85627e60a38e47e6120068788061eb6bb68de493538bde6225ae163e8047a05aea293e73519d2c860c2d6627484a8cf02bb25cb12ed5f418dd50d466cf2e2fb884dffbf75db4b95ae0561d9b74e57e18817cf2aefefeb238df9796c9d95a22797728ef14008209cc3223e7b33448df67642642ad385643903e51cdfab7c6a2cb863facca2ced922ca95fd1ec89006ab7ce9a57f4083569b3483c1ab8e63c56a71d04e39924b359ba669de2aae82c049c99469b64caf1e6b0adf2ea4d9aea1a4a170482c71539513d172291aba76dc9bf19922ff149130a278dbfbdcae75458de51c4a6203c9bf3ec8edf45736b939449d68b77f05eca917b8664b77f89e0f2dd843f2be5bfe06c37c7e7ce78b96d16c2888ab62d9b599468adaf465cba7611a0b32d6a7d4333c14b61de3f4bae014f78c0fde471ad8e3553db75f1f6ae681fdf6a702508077e0a7c66ab3c6278ab9e694518ed77e62502ab8a3dff86d324c4aef890476a8013cb32de018ac681adbee70ce87ea7096deb8f7346ec5dbeb77c0fb411911e9b44921462d96172c7da939a26af8a97192bc7982b9002fc5c370280c63e54fdb272c686ba0c99f0b62f5557367965c6fc8add35bb39d5efc3921c64d813204862166c896880e1716ca776bcff25f233a41b30bdcec80c8735c3931e35c071770b2abea98240b9a6a5b3b9d3d44ccfe966faf22548b424c2cb4f18762106bf0f4bba7f1bfcbb618bcc54f886de7d585c199b34fde2fdc4c68432457c0b56db58f1406447a3776a01e099106d21338488beb83c04cdc9b09c8531323fd112de6e17218321aba5da73f6fa981730f10d9a5c823e6313ae62759d5802d94e63b567b95eec225b57278582b57b434221cac0430da8a12fb0aaa8d889b1c716f82087d64eb2357f00053e788a0a826b938077718ba9af26a205e1888e00d051651e8139d4e455adcd12d9707ab3a31fc37cb050aa4eec6d18a7fc64e704deb78cc31136ee39b9380eeee4820f59d322636a7dd80eb378cc45b7a8587d2c8bfb97e7727c9c5c1180958e7bf66cb31c2832e6d92cbfeaea369350f6adcad2ce8bd0abf4592b95c061c0e76e3dc1722d98da4d869da7c7ef5629249566f45509ddeb410d21a7873529b0ef12deea3f28e1e7c3328349b0a7d30cb8238094cab4b35fdb481f893ce41145850919e25eeec40146812ad8d5ca693deda352a6aa794b5845a273b70b02edea4a2451fc3b67b00adf7bd4c4fb17b9701c9e5b8d3285fa8ad767f10ba9348773e41f50a73a8c0586d6230048340a639faec54a7a41354b76e3ed8c95c38c1b8d48a94b058dc290d8a5f348e0ba2c751b91b9c5c504e355c932f123d471500f849d183cd167ca4909d3ae706f8faf181f8caffc5971e17fe2805ad70295920b5a9d425263838bf70b46a9222cf7a61b51086dc6b593f6b3cd9c01c0b9eca50f4e2f9008b113cf6a74387448aa3f4ddea17f55e43a1c48434bdf777b6e9743481f8843008b1c9e9a4bfc3c748b5276ab41819fbb1eb64ffcfe5b8815cd1d578f267de0b5e7bdefeb694a003240bdb8f7e2ec08474d90eb95c4866ec08d596bc9d3a8fa7a029d92896c9d7205ec29347fdd9dd9939f81ede3ac2f895b350403c2412600cda3cfdba0415c57a268f97f0b94c3c42852703a600a8e0b1a093ced92e38f4bc73b2f26e020f17fc881e9fd72a0feafe480944ebcee4fca01e27a04c0c860ddfe7051bc7eb0a83d91c8d1adfb9dbff1e711afb420916c1a448168c857801663ebfd6640cdc2103ea137b53c3569c3d8290f49f778c7ee956ae23e9a77631718f70301036f1614747850728b2ccea69d31f7da90f2e649ffdf136172a4653f307ef37c7698048c667d1ab130a2d038e55cac055e5b966c0dafdc02f469b55cf2c4ce0979255defb21f7fcff3fe4d09ad01a8aae6e3c7cb3e8dbe92647b6beec6afd508d7aea6b5f9b5e80c8b8ebfb6a63a9fe19060533bfd9f58736ede03e59943dae5d4f5d2023b3bc4645f3ed569aa070c905a0f16d6c2f3357b01d618cae20eb22a3d78cfcc391345c93bc5341cd657f559adf99220d8b4e3d43e7b87425fe7cfa62fa0c12ca9156b8d0d98dbba63693e6942433225f4b7e23af4d2d9d3f09ee589fb0e851edbb7537d68feb9544df8825c8d58c2ea5d17eda9b7646f1972b912b4cd593d3daea4aa7fdd24171850060ec9717871e4d731a13fccd527f61535e6de66121077b25639db5434df9998eff74d13e47d72e531876f6bd159b2f913930ce6c1d5cd4b37d846a2ac6db891924a157d783f69d29e2630399826f8bbe4ff8f354109f9715213553d6355818a76497b47c63838ea5e780bf7a6acc0c8734c54f23d002a6496d48b75f832a1b166c472c2c331f30282ea665e66607afb910ee913fd65d5e065100ebc00385e3ff17c6653d5cf74b22d92fb26e1dbbbc76e64a8c06adadd027da92ba3ce7ae31da0141c38bdaa6bbd83f3e006a1bfa381ffb862acafb37151882a3a81d95db79a7fddf8dafb23d0f317ae425978529fd75241474e0214c14484665d32d0fec3332dd45c0587f3a1b26fc3535a9e5a3b5f6c77e10dc3ed42ee7c30c8ff0167c50f72494ff10537e788dcc75f4d89233b121ed0849324f42550e4197af1850a4a43f50038abce58ab3fdb97c36ed30123e501299fe6ebf2da57641bbd9b1dcb03099937aa4640eb830076c52e4344d96b9ef96ca6fb43274ce1cc74e33944eb0aa26068f0ac8b0f78136fc0d3d62377370a995f711432027846dafea4b6d8993d0379a682c943a9dee6f4c6a7fc49ced512f9ee8af75ccd148029733fcb51d39b14e82ee9b595c9de3ef74bf22a8e3708300c15fde8cd858d8fd584c6201ca10110620cb2dca07add2e1f089bf05473c54595d984131f2a80c8b71abffaf8b05b10efe74bc71142abc12238b050f0a189507b385a3a048c284ad097a4a0acbd6c12553ce855c37cc8189e5e7aa80a51d5e442079bfc1b18d97f6e1a9cf0ffe1657be0dd8fee6d91e55afe37e353fd0e04efac02fac9a94f4cad36da0659fcb6342bcbeed3324c371202b87a775843f50601db165de5649f4be414dc2f83bb798ffb7c8589bce20c73c435062f7e94af3b1ff42d6693ff86983dfa51675f8f6cfa87dff7b26608c02b66a237a8a21763730fbff1e4a07edfbdb3c16b9dea0a3fe7ce503094af258745da0090e5de0f61f54a56ada14cc96aaf06ecd59d079f9e66582c6e49ba37a8da1f4ef12d6653ff869845fc44cc88f9d93aedf088d92de0b4d2cc1d27ff88cc42bb64abecb0e8471e8f6a47f3087afafb7b1727ad32cd8d01f2d3495b1990cd378b4c02a26cbe16616acf95b86256a52763c96322ac75dce7aaecd66d39f02b77a22c6d0d3e838dbcb70dd36a1ea7c913a4385c6e34cb4737151de14bbcdb3bce02088ff0760d8b2d7bc35081bf853b16d7570ed2081ec58683bfa87d40bdd09775ac06278270e0388081bbbf3f5ee87295478fb88524da5eacd4e77e0955ef619a90d202c8d5c2bc28bc930b8003abfc210f30fbe4c1a3eebf92078a2f29197e4e1e10c8d58381a3626e23fdf14f1b261402da750767331a1ca2065f691dc73c5f3575ce545d4b02fc4ba7e706f5dc8695dcdedd0e7abff205414f5e1d298c72af3c26d2d1969fa2b29bf54d7f4dd6b7dbda2229b430599e9b12324445e545da2077348bdb569c4607e60760845acf00a7c6f1ecac116edeb5e2c910dfd4fb7e1b8ae9141a3b40d9329ad3cace2dbea01a920ec24152006e228e37d26766c7b91b5649635d7bc8a4d0f3062b264ca03b75cf08bb86efa3f106261307d1acfa0a595fc7f3ef89c2da7a6450cb7f527b29c7b632ddf7a0f71098e4f017687ee03607e585fc1f38125d241db744eeba6ee4396707dae590296a84d2d05b8e7ff22b1c4b22ce0eb39e05a4128e1bfa7e323d9673a572baa660bdfdf69965cbd8da159c981cff9df93922fa3dc1e5f26b05d711544c4b22cdc9e5b98756aa8c6a6b0dbea16941ff09c18584d2c4c66c57d20adc2aa917bcf4785bef50de2663c5cd2b0647e62bec3e138b02a61770236d5dce412e8564e853a3913df3dd102343bb1ad2b759d85287906d996a88300807110444eb4a9315a2e73e37f1075bf0713812703ebd328188437629b950da747e09c59f6767e3c0a37b18711e0c23cfd703fd3e335d12ddf7270e45d06b680a299e92fe1f28444500c69e4ed69878761ca577100512bed36e3630e59ddcab5eb53c245d2ef59cf82a20b3186f7a51aa8035f9f5b2b5a6522f6b475627ef73a9741706e15e969b719a6cdf7d215c07f285fcd9a4af070329c746938d422f67de7022d37f8498459f4f149ef50f71f92097c6547526895b1670999a96e9e36fe315d1379379894db6569ead7eb460763b52fa6d992fc2ef02e65b1bdf656c504bf6b4088f1ccb3c0a162b2bb465cc390cdd316fb821ee21b34c0fa1c27bca3cdcd7c667d85f51c5bf8dff0f26073e5ef870c7e0ba37b9a74ec9bd93e047493b472a050130456e64d4c360e34b77e55d8d000d4a61ddbe8b3af09a00356dea2f26d1e68a3c5fa508c68aeee4a189178edaebffbf000000ffff1ed9a0d101","nonce":"0x128cd","to":"0x8453100000000000000000000000000000000000","transactionIndex":"0x3b","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x41535904d7b20be8d733e03d95ec725d0c6ee4307d899cd00e0ca0de91533b13","s":"0x3f98c81207d078199bf74a600a77cfa389574646d63ff55ccfd194491faf7226","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xd3343d473f624c02045a01f4e7ab604bc56c4f42","gas":"0x156c0","gasPrice":"0x7d9f6fbd8","maxFeePerGas":"0xf49895cde","maxPriorityFeePerGas":"0xcfd6ae0","hash":"0x00b5b05c6d1a2eb8abe2c383da600516515e383fc8a29953bb6e6d167e9705b2","input":"0x9aaab64814636b3e36ce9c4227a5fb53273a04c3fdc94c8a00a4d6d5b6521571b94ec7ac00000000000000000000000000000000000000000000000000000000000028a416eb20a9e75d33ef7eeb4a0bad7413cea6c0b3a7f6ea249a93a428c0c36b52240000000000000000000000000000000000000000000000000000000000840244","nonce":"0x6c7","to":"0x1a8b038f5dff48f541e8cc658b9e186b9828a726","transactionIndex":"0x3c","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xfcd40d6ec6fa94f40f866c18457c22ebf5d8e3aaf2616c4c1cee5245ce8f042c","s":"0x45e5962efdda48e38f706cf8fb7c2f722d80da6e0a1d2589b6627627b44feb82","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x9aba7eeb134fa94dfe735205dda6ac6447d76f9b","gas":"0x3e64f","gasPrice":"0x7d9f6fbd8","maxFeePerGas":"0xf49895cde","maxPriorityFeePerGas":"0xcfd6ae0","hash":"0x6fc411f24c7b4b8d821b45de32b9edc5ac998d1ac748a98abe8e983c6f39fc19","input":"0x2e260ac30000000000000000000000000000000000000000000000000000000000000200a5c46e2dd70854a0fe8167b705eb7df71cc927469c345248c81b21a5ab2f5f19000000000000000000000000000000000000000000000000000000000000051d000000000000000000000000000000000000000000000000000000000000051e000000000000000000000000000000000000000000000000000000000084024700000000000000000000000000000000000000000000000000000001b3fd9f800000000000000000000000007ebc0d2451003f2d3bc5f6eafbb7e6bad81e250d0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000100000000000000000000000000000000000000000000000000000000640fdea40000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000002e43c000000000000000000000000000000000000000000000000000000000002e7d600000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000f6cb686532a3ece4a20f823e59cc386e1ba384d8504e902c932fa1497c4afdfb0b000000000008008402470000051e00000000000000082ffe82e7b18f0778493c989ecae1740325207ea29328999d5ae646723a4bb75a1305c8ca593670972f535ff8afda050351549ed800e9b98c805788acfe20e6ac076ae024efa64ac3ceb8f0a3cc9ce93d122f6faf8376c854da95567fb7576f762299ca2135608c7ac5a824257b50043526903fc8e01ad88006448a69a4afda4c01ad8a6ed1260582538f2311e902c1e7d5853a75b74482117f2cf920897c1b9428eaeb7e74a95d3d32db03931d157a48aaca9748bfebfecffc2d140caddc66b300000000000000000000","nonce":"0x108af","to":"0x7ebc0d2451003f2d3bc5f6eafbb7e6bad81e250d","transactionIndex":"0x3d","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x1b284cf86146fa2094c7c709cf9fcb8a177f4fde1d1c5ca112e41f82a5e973d2","s":"0x5618261c40d23e5beb7ac0bac77b16ead56e547f103f649462387c5acdf18ba5","yParity":"0x0"}],"transactionsRoot":"0xaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa","uncles":[]} diff --git a/op-service/sources/testdata/data/blocks/pre-shanghai-bad-transactions_metadata.json b/op-service/sources/testdata/data/blocks/pre-shanghai-bad-transactions_metadata.json new file mode 100644 index 000000000000..6ae97b067cf5 --- /dev/null +++ b/op-service/sources/testdata/data/blocks/pre-shanghai-bad-transactions_metadata.json @@ -0,0 +1 @@ +{"name":"pre-shanghai-bad-transactions","fail":true} diff --git a/op-service/sources/testdata/data/blocks/pre-shanghai-success_data.json b/op-service/sources/testdata/data/blocks/pre-shanghai-success_data.json new file mode 100644 index 000000000000..fefd87d7b071 --- /dev/null +++ b/op-service/sources/testdata/data/blocks/pre-shanghai-success_data.json @@ -0,0 +1 @@ +{"baseFeePerGas":"0x7ccf990f8","difficulty":"0x0","extraData":"0xd883010b02846765746888676f312e32302e31856c696e7578","gasLimit":"0x1c9c380","gasUsed":"0xa79638","hash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","logsBloom":"0xb034000008010014411408c080a0018440087220211154100005a1388807241142a2504080034a00111212a47f05008520200000280202a12800538cc06488486a0141989c7800c0c848011f02249661800e08449145b040a252d18082c009000641004052c80102000804ac10901c24032000980010438a01e50a90a0d8008c138c21204040000b20425000833041028000148124c2012d0aa8d1d0548301808228002015184090000224021040d68220100210220480420308455c382a40020130dc42502986080600000115034c0401c81828490410308005610048026b822e10b4228071ba00bdd20140621b2000c02012300808084181ac308200000011","miner":"0x0000000000000000000000000000000000000000","mixHash":"0x31f0c0305fc07a93b1a33da339c79aadbe8d9811c78d2b514cd18d64e1328f25","nonce":"0x0000000000000000","number":"0x840249","parentHash":"0x2303b55af4add799b19275a491b150c1a03075395f87a7856a4e3327595ed7df","receiptsRoot":"0x99da71b17ae1929db912c3315ebe349d37f2bb600454616fdde0ee90d6dbc59e","sha3Uncles":"0x1dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347","size":"0xea6d","stateRoot":"0xd12bf4cf3941cf48be329a939b13d3403d326841c69cdcc9a9c13ab2f227e904","timestamp":"0x640fdeb0","totalDifficulty":"0xa4a470","transactions":[{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x1b7aa44088a0ea95bdc65fef6e5071e946bf7d8f","gas":"0x5208","gasPrice":"0x174876e800","hash":"0x39c666d9b5cec429accad7b0f94f789ca2ebeb5294b8b129c1b76f552daf57d3","input":"0x","nonce":"0x26bd1","to":"0x68643ada5885f14e7c15c13bee06168eff0f7c36","transactionIndex":"0x0","value":"0xb1a2bc2ec50000","type":"0x0","chainId":"0x5","v":"0x2e","r":"0x26e648625520c256d87d8e15769e8c4290d2a7e27147594815da42070870825b","s":"0x7db34c91603e51494b59b020597e7118433d50ed548bc7efad8880cbe7820ca"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x030b1cddf635e9e71ad70b8668e235e8ec3c67c4","gas":"0xf618","gasPrice":"0x1701ca2b9d","hash":"0x2ca7289ab3738d17e0f5093bd96c97c06c9a2ea4c22fc84a6a7fbfda93ce55ee","input":"0x","nonce":"0x456d","to":"0x89416096a0a1acad3436a357478c5c548e4d382b","transactionIndex":"0x1","value":"0x16345785d8a0000","type":"0x0","chainId":"0x5","v":"0x2e","r":"0x5aa6ad410ea42dbff3781de67b4815a376cd459206595fa6655c966a7709e5cb","s":"0x82f620f8dbc356dc38ef346d1e152656a1d318becbcc41884f61e85972355c0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x9c67435ce8a33ad23f377ddf08260ffc56417f21","gas":"0xf618","gasPrice":"0x1701ca2b9d","hash":"0xb0085de1476530de3efc6928c4683e7c40f8fac18875f74cbcc47df159de17d9","input":"0x","nonce":"0x446c","to":"0x471f3a988a77472e971112e7ff2ce65996b6c8cd","transactionIndex":"0x2","value":"0x2c68af0bb140000","type":"0x0","chainId":"0x5","v":"0x2d","r":"0x75425f21a6e66e6feaee79455c897656c4a94b6189c4c033facab1b67812fc3f","s":"0x7014f4b9f760165c7757f9d6dccea349b3d00b0a9c2f845af8010140842f826"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x168f6dec26cbbb3749654e0e3cc4fc29314fdf6c","gas":"0xf618","gasPrice":"0x1701ca2b9d","hash":"0xe01c8631c86ded63af95b8dbc0c8aac5d31254c14d6ecb4cc51d98259d838e52","input":"0x","nonce":"0x403c","to":"0xf2119a49a3ae35453c1c2704da248d08977084f1","transactionIndex":"0x3","value":"0x16345785d8a0000","type":"0x0","chainId":"0x5","v":"0x2e","r":"0x56bba57ee69c09dbbd5167d0fee07d5803130fea7c278f4d8b7bc54b34be8ff9","s":"0x7eea69c0de47b6b808ea1bdd9ecf5d25f4f2743e867b1f838db59c456026d7e6"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xc9af69904b4ba7f0fb69ab1a8719bd8950265e73","gas":"0xf618","gasPrice":"0x1701ca2b9d","hash":"0x69414a126a6f07ab5e31ad2f9069fb986b7c490e096898473873e41ece6af783","input":"0x","nonce":"0x44c20","to":"0x6adc3f5394e00cd067d1fad5a33756791e303c25","transactionIndex":"0x4","value":"0x16345785d8a0000","type":"0x0","chainId":"0x5","v":"0x2e","r":"0xdc9a52b87755cd07143bf032d57258d0d6c36500b38a574a9a5addc1293343da","s":"0x581e6fd051d0c72d64303c1f90edecfef46580f84f24dba930cde910923f7c21"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x92964e63cd52450e6e09a266e0c226e524754563","gas":"0x2625a0","gasPrice":"0xf5686c7be","hash":"0xa2fef1133ee726533c7f190f246fede123e3706a03933c1febc92618f90d2804","input":"0xa9059cbb00000000000000000000000011f7db5a824eaa5737c935ad09a925dd707f092b0000000000000000000000000000000000000000000000008ac7230489e80000","nonce":"0x2636","to":"0xbc071c64ed8f536011c78e847755680740d6b73c","transactionIndex":"0x5","value":"0x0","type":"0x0","chainId":"0x5","v":"0x2d","r":"0x422c5026584cd8fb8a1fe66e6edc7a020bc513cc3114083fb35f9846cd9a9792","s":"0x7fcd4e48ac5b53d881531a5651e587188157ed88d85c51533f7c6eda94c67ff7"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xdec1bc71bf91431d60ef2742f412dcd1c5a204b8","gas":"0x7a120","gasPrice":"0xb80e515ce","hash":"0x6585ec5c4c2bbf1f683f90f58e18f3b38d875e94457fe4cbb7bc5bf6581f83af","input":"0x67a5cd060000000000000000000000006343e96c99e2725e845ba04f90e03d853cc9bb2d","nonce":"0x6a5a4","to":"0xf5de760f2e916647fd766b4ad9e85ff943ce3a2b","transactionIndex":"0x6","value":"0x0","type":"0x0","chainId":"0x5","v":"0x2d","r":"0xcb7b32949ac0dedd3151d06d85ec4e99483c8ea706f12a009d97b055975241eb","s":"0x69f0009bce12e24408f9a3d5ad5a8578e2b23efb09002f7b86b3425cf8748889"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xdb954dd5d224004b26cfbb089312e0542e5d1c31","gas":"0xd5ef","gasPrice":"0x9fc748df1","maxFeePerGas":"0x116482c049","maxPriorityFeePerGas":"0x22f7afcf9","hash":"0x1db276b864fbf01dcf8cededf8d597553ecb0eb9438edfaf2f5bd0cc93297c66","input":"0xa3e35f36000000000000000000000000f18f2320e4779f21d6882d83f9133e582dfaa9b6","nonce":"0x570","to":"0x11672c0bbff498c72bc2200f42461c0414855042","transactionIndex":"0x7","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xf23a1fb0d17cdbc2831a2cb0b779795e12533e34425a3abbc4b7e7bde262b92d","s":"0xd4a2c8189378696065b7fc5d13e9cf6595d96047a31a4fcb812c2e677d22a26","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xe84d601e5d945031129a83e5602be0cc7f182cf3","gas":"0x249f0","gasPrice":"0x9502f9000","hash":"0xcbe7ed31654af4e191ca53445b82de040ae2cd92459a3f951bdcce423d780f08","input":"0xa9059cbb000000000000000000000000355f96e3261cc8237d2d36fd81cc9b9cdcbfc5ea0000000000000000000000000000000000000000000000000de0b6b3a7640000","nonce":"0x4f1f0","to":"0x499d11e0b6eac7c0593d8fb292dcbbf815fb29ae","transactionIndex":"0x8","value":"0x0","type":"0x0","chainId":"0x5","v":"0x2e","r":"0xadf7dce5f34d482bb19ff0c85f43be7008cb838043f9abb5fc08421245ac4228","s":"0x5d2c0f0b9d2e68bb8da6084e4971d57bd76bf0cd9d893c9c985962812d2df026"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xea0193f88a0d9c36e83e6fb6d382212b6373faba","gas":"0x43238","gasPrice":"0x933ea77d8","hash":"0x808ba5211f03cc78a732ff0f9383c6355e63c83ae8c6035ced2ba6f7c331dc63","input":"0x9f8420b300000000000000000000000000000000000000000000000000470de4df8200000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x0","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0x9","value":"0x470e9f23394000","type":"0x0","chainId":"0x5","v":"0x2e","r":"0x9536f87e8c3433ea23ef7211c0c32689551906dfdf31bcc73b89fcf18fe86a92","s":"0x3e94bfe3b4ba32a6ce71ee6e0ff9f26dfc2b0d61b5c0fe049469e3c4b88ac227"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x1f9ab300b380313e6a2208e44c01a0602538385d","gas":"0x32b38","gasPrice":"0x87fc9eef8","maxFeePerGas":"0x2e90edd000","maxPriorityFeePerGas":"0xb2d05e00","hash":"0xdd66f1f26672849ef54c420210f479c9f0c46924d8e9f7b210981ffe8d3fac82","input":"0x32db5470000000000000000000000000000000000000000000000000000000000084021000000000000000000000000000000000000000000000085b0af981975a40fc5a00000000000000000000000000000000000000000000053ae1f470262f3b01c4000000000000000000000000000000000000000000000813bca9f8389ffd193f","nonce":"0x5699","to":"0x28cea7b0f3916c1dba667d3d58ec4836ad843c49","transactionIndex":"0xa","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xbe8eddd463a3518a4c94b5922c0f2fb26a32c8afcd819f7deaf4f0416757ad13","s":"0x3e234182acd5261dfba3cf4f125d1a8377fc707b3b1900b565d1a6b8987b398c","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x1d990a2cf61b46f37582dd31a7b242e989e1a1ee","gas":"0x493e0","gasPrice":"0x86f96eddb","hash":"0x254abb2f8cdcffe9ef62ab924312a1e4142578db87e4f7c199fd35991e92f014","input":"0x9f8420b3000000000000000000000000000000000000000000000000002386f26fc100000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x0","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0xb","value":"0x2387acb3784000","type":"0x0","chainId":"0x5","v":"0x2e","r":"0xbfe9b99fd2e9bbd2d65bb4cc17da88130d8f6b569a26afa642ed1cc8efd2e2ef","s":"0x7003dd3962eec2369bc5abe5e3d5b73a9086a3219643b1741f3a9832369448fa"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x1124bbb196fd7b61947f6f03b7d7cebde8340b6f","gas":"0x493e0","gasPrice":"0x86f96eddb","hash":"0xa7b7c654e7073b8043b680b7ffc95d3f2099abaa0b0578d6f954a2a7c99404e1","input":"0x9f8420b3000000000000000000000000000000000000000000000000002386f26fc100000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x0","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0xc","value":"0x2387acb3784000","type":"0x0","chainId":"0x5","v":"0x2e","r":"0xe8dae9efa8fa73a3aebdaa75ba12acf56712b1f99fc128ab802a7610d014547b","s":"0x2ef347ea90a0db21b035637342bfc916c9962d3ec610a4e99cfaf555a2606fe2"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xdedf52a88fe47f6a11d629badd0096d201aef86a","gas":"0x493e0","gasPrice":"0x86f96eddb","hash":"0x7ccdfa698c8acf47ab9316ed078eb40819ff575bcf612c6f59f29e7726df3f96","input":"0x9f8420b3000000000000000000000000000000000000000000000000000003a3529440000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x1","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0xd","value":"0x45d964b8000","type":"0x0","chainId":"0x5","v":"0x2d","r":"0xfc4e092094abc7fed9cf11eea8ad6ec3a760d13b9c603b5570c2143e48090f2f","s":"0x31a0a6016ab51ed6375891bf532d9031bf91c1d0e01df63cf7e46076aeee83a3"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xe1db1fd154b344ce7dfcb53a65279559b298f0df","gas":"0x493e0","gasPrice":"0x86f96eddb","hash":"0xa0b035ef315824a6f6a6565fa8de27042ade3af9cf0583a36dea83d6e01bf2a8","input":"0x9f8420b3000000000000000000000000000000000000000000000000002386f26fc100000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x0","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0xe","value":"0x2387acb3784000","type":"0x0","chainId":"0x5","v":"0x2d","r":"0x79c316492bfa36fa4de2dca02ffac77f6c1772714e153aad8c947ce848d4106f","s":"0x4fe3e083943ca8f974cb0752a2ffb70e4d6c8b615e16ece7e19b2c464be66929"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xd4c3777e1821f6fb532957104e1317a9d3f881ec","gas":"0x493e0","gasPrice":"0x86f96eddb","hash":"0x1ebad7f3e8cb3543d4963686a94d99f61839f666831eab9c9c1b4711de11d3d9","input":"0x9f8420b3000000000000000000000000000000000000000000000000000003a3529440000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x1","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0xf","value":"0x45d964b8000","type":"0x0","chainId":"0x5","v":"0x2d","r":"0x5c1f544ca92b28a014129a9d04f8e895dc8a6987ddc1717c26b0619bb9bac420","s":"0x47574643bd92a0f129f9c5ce3caebe7943cfd828d67fb0e5bdc1eb2ca202822a"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x127418955b15f5ff2fa232431a798b53fd006c82","gas":"0x493e0","gasPrice":"0x86f96eddb","hash":"0x501750278e91d8b5be1ccf60e793d4bbcd9b3bb3ccc518d3634a71caeac65f48","input":"0x9f8420b3000000000000000000000000000000000000000000000000000003a3529440000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x1","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0x10","value":"0x45d964b8000","type":"0x0","chainId":"0x5","v":"0x2d","r":"0xcfc9894bf716ba707521f0969082afb688b7d47cc21f1423b239c40867128f47","s":"0x5d9b9c66d8437d0a37768fb9c3241ef64b7e795d0d908196e1f707215ac1f56"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xc2ac1bdcb3dd77864b99eb2cafef2ed36c1487c3","gas":"0x493e0","gasPrice":"0x86f96eddb","hash":"0xd80ff8af29ae163d5811ba511e60b3a87a279f677bb3872a0f1aa6d0a226e880","input":"0x9f8420b3000000000000000000000000000000000000000000000000002386f26fc100000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x0","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0x11","value":"0x2387acb3784000","type":"0x0","chainId":"0x5","v":"0x2e","r":"0x8bd7220d176d18cfbc824063448cf864c494fcf0958dec82126f26d3fc6c0b6e","s":"0x2f4de3d2fc3be6d2812ee10aee16a995af5affcfec27f9953ae609886d2b563e"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x11a5ad000f8494233df5bd2f2573fe20bad744ab","gas":"0x493e0","gasPrice":"0x86f96eddb","hash":"0x096acab3b3fe47b149d375782d1eb00b9fef7904076d60c54b3c197b04e6bf82","input":"0x9f8420b3000000000000000000000000000000000000000000000000000003a3529440000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x1","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0x12","value":"0x45d964b8000","type":"0x0","chainId":"0x5","v":"0x2d","r":"0x43726c3165958e94d80cb6992121f55df2904a1de4356e0a6ff3d8bfd03c0928","s":"0x6244361e31daccbb5c7e9f3f7d01004080d2efa314d614fdc2aad4c1b7d71613"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x90f8b290ec2929cce086826e971cad3857c6dbb5","gas":"0x50f46","gasPrice":"0x861fc89f8","maxFeePerGas":"0xe22924cba","maxPriorityFeePerGas":"0x9502f900","hash":"0xbe9d1738af74a22400591a9a808fb01a25ab41e2e56f202dd7251eb113e8ceeb","input":"0x6ab150710000000000000000000000000000000000000000000000001bc16d674ec800000000000000000000000000000000000000000000000000000de0b6b3a7640000","nonce":"0xe4","to":"0x2e84c9894f628fd209f48b06d10b46c8cada4c10","transactionIndex":"0x13","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x412c7bfb0236a4317da40645ee4e60f19185e7d0cd2cc9bc88f7629de505b939","s":"0x4f6bd485d10fa134926ec3e3a83e212f3c5d903ba52da87f07794a2ba0a5c1eb","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x7915ec065b644568155c4772a286addad3864c1b","gas":"0xb71b00","gasPrice":"0x861fc89f8","maxFeePerGas":"0x1176592e000","maxPriorityFeePerGas":"0x9502f900","hash":"0x0834c720e55cccd97aaf4f8fb0cb66afb9881fb6a762c0f70473ec53f98a712e","input":"0xedce8b4d6a1fb55588eadfd3a0c3350435a31d31e7c236125eafcc2a892d50711d9a4939a4282888c956c070cd83400fd47ab83bd5287244053b8228711b07386dccacbd000000000000000000000000000000000000000000000000000000000000006000000000000000000000000000000000000000000000000000000000000001a0000000000000000000000000000000000000000000000000000000000000082000000000000000000000000000000000000000000000000000000000000008a0000000000000000000000000000000000000000000000000000000000000092000888ad4975d4079d80bc6da5ded57746f889644719ec160efc2e4a96ad645df10b1683afcd7ce41ba6c06528761f821cbc6e806ef4b71c189603ec218b12a1f1478883498e67b97ae4babef6996acc582db1377d8755374602e00b8148025ac2d2f52e78d7484919cef14bf96e5b7a3f7f8cf09989551888f2c697052ce2bdb02a9296b20016c78683a9ca137037e57c8ff25198146f9d4cdb5a25ab9c95a03038f22f83c8604ca7c035ef756d2963c3657eea4ffb18ef967752d8cff15a16124a1d92a1cda613624c98e3078742803da1417b78bb264b7bb2197515cb4facc2ea395c6241664dbd953d0034900a86eb94897bf89aae775a1d73e6fd034785400000000000000000000000000000000000000000000000000000000000009a0000000000000000000000000000000000000000000000000000000000000000300000000000000000000000000000000000000000000000000000000000000600000000000000000000000000000000000000000000000000000000000000260000000000000000000000000000000000000000000000000000000000000046000000000000000000000000000000000000000000000000000000000000001d802f901d482e7041b846de27f7b846de27f8383026c28946aa397cab00a2a40025dbf839a83f16d5ec7c1eb841dcd6500b901a45ae401dc00000000000000000000000000000000000000000000000000000000640fdf5900000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000000000000000000001000000000000000000000000000000000000000000000000000000000000002000000000000000000000000000000000000000000000000000000000000000e404e45aaf0000000000000000000000002c1b868d6596a18e32e61b901e4060c872647b6c000000000000000000000000964ff70695da981027c81020b1c58d833d49a64000000000000000000000000000000000000000000000000000000000000027100000000000000000000000009a1a41127726f74560a57990fb928120eed6f55e000000000000000000000000000000000000000000000000000000001dcd6500000000000000000000000000000002d4766adcf72ef2d996edfadc866be12c28000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000c0000000000000000000000000000000000000000000000000000000000000000000000000000001d802f901d482e7041c846a969541846a96954983026c28946aa397cab00a2a40025dbf839a83f16d5ec7c1eb841dcd6500b901a45ae401dc00000000000000000000000000000000000000000000000000000000640fdf5a00000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000000000000000000001000000000000000000000000000000000000000000000000000000000000002000000000000000000000000000000000000000000000000000000000000000e404e45aaf0000000000000000000000002c1b868d6596a18e32e61b901e4060c872647b6c000000000000000000000000964ff70695da981027c81020b1c58d833d49a64000000000000000000000000000000000000000000000000000000000000027100000000000000000000000009a1a41127726f74560a57990fb928120eed6f55e000000000000000000000000000000000000000000000000000000001dcd6500000000000000000000000000000002d4766adcf72ef2d996edfadc866be12c28000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000c0000000000000000000000000000000000000000000000000000000000000000000000000000001db02f901d782e70480846a969541846a9695498302e100946aa397cab00a2a40025dbf839a83f16d5ec7c1eb870aa87bee538000b901a45ae401dc00000000000000000000000000000000000000000000000000000000640fdf8d00000000000000000000000000000000000000000000000000000000000000400000000000000000000000000000000000000000000000000000000000000001000000000000000000000000000000000000000000000000000000000000002000000000000000000000000000000000000000000000000000000000000000e404e45aaf0000000000000000000000002c1b868d6596a18e32e61b901e4060c872647b6c000000000000000000000000964ff70695da981027c81020b1c58d833d49a6400000000000000000000000000000000000000000000000000000000000000bb800000000000000000000000048f2c17f8885e135c8689626d5841d0b94013556000000000000000000000000000000000000000000000000000aa87bee5380000000000000000000000000017a8db2711fe83b10bbfabb6a8b86c3e26ca154ef000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000c000000000000000000000000000000000000000000000000000000000000000000000000003000000000000000000000000000000000000000000000000000000000000001b000000000000000000000000000000000000000000000000000000000000001b000000000000000000000000000000000000000000000000000000000000001c0000000000000000000000000000000000000000000000000000000000000003760f4e19e59be10b323fe1b42e42de2754c5da54ff316cef74ef09dbc0144bbf73e609a8d21a38f393bae9a3de6bebbf362608d907d941c8166f44d8ee2c6686b5dbbd6681f5467219d9bb530db013b7c067db659c03efe935caeb7bab36699d00000000000000000000000000000000000000000000000000000000000000032d2d7436bd45b5e81824945ff3dda846e270c81e72929841f4b14f56e02eef39477f9c162a2759c8d7c7e1feefdd064f6aefb4360d483f70671f36fc11ccc6db6c53f74e3019c12d859545ad75b51998b2238887dc0be3c78522c8356150526a0000000000000000000000000000000000000000000000000000000000000000","nonce":"0x48f08","to":"0xe87d317eb8dcc9afe24d9f63d6c760e52bc18a40","transactionIndex":"0x14","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x54159a63b8ce19097cb8afeaffc84cfb5257f9709dcb0f0a258befa0b924b13","s":"0x17f5ef5aeceb6f082a8a0f8fa28e55d453c52e5f86d48dfb4d4f039aad7fa493","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x52db522edf50fc32b8fc0f77d51140031da11da3","gas":"0x201568","gasPrice":"0x861fc89f8","maxFeePerGas":"0xe22924cba","maxPriorityFeePerGas":"0x9502f900","hash":"0x51a0c33c9b37245b416575bdd2751c0d8a5d8bead49585ac427bfc873d4016af","input":"0x96d3f83c0000000000000000000000008cd194f4ced354a14960f76f49305feb8510f73f00000000000000000000000000000000000000000000000000000000000000e000000000000000000000000000000000000000000000000000000000641024ef0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000001b3fd330d479974ad52e18b8767d9f18cf9ff024fa749b1066a803b237570cb73e2868e821124c8128505d883124711ae7fdca0339f4f5cd5dadd8eb6956eb4af500000000000000000000000000000000000000000000000000000000000000040000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000100000000000000000000000000000000000000000000000000000000000000020000000000000000000000000000000000000000000000000000000000000003","nonce":"0x2d9","to":"0x29c1442d7e3b6ed179ceb55fe996e1f4384880da","transactionIndex":"0x15","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xac84f403fa0347be3246ad695f8f6a780332357d49dfa67388fae3e051e5566c","s":"0xac2e148c1045af0ebdb334f20743af7d5499558af5eea43393159ce26a3fb1b","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x3aea5f857b75a946d5f5b329bdcf4db46aec5d2c","gas":"0x50f46","gasPrice":"0x861fc89f8","maxFeePerGas":"0xe22924cba","maxPriorityFeePerGas":"0x9502f900","hash":"0x531c25d51ccda59aa9ea82e85c99be9dd4e285af9b8973cbab9ac4a38e26e55a","input":"0x6ab150710000000000000000000000000000000000000000000000000de0b6b3a76400000000000000000000000000000000000000000000000000000de0b6b3a7640000","nonce":"0x1d","to":"0x6c23a4586cc791d2be6767532e12264992aef74a","transactionIndex":"0x16","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x81abc4fdf3b57fdf3a951fb293853fdd86aadce8f319576ac78c60caac186c43","s":"0x452742335bd1f63fd623d5e6cf91f6339361f863dba1a57fc3c1da8292e82c77","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x7915ec065b644568155c4772a286addad3864c1b","gas":"0xb71b00","gasPrice":"0x861fc89f8","maxFeePerGas":"0x1176592e000","maxPriorityFeePerGas":"0x9502f900","hash":"0x93ac6c08d21cb1b61ff59e5e2d6fa3f9ad54008b0a66c669199050bef219f6e3","input":"0xedce8b4da4282888c956c070cd83400fd47ab83bd5287244053b8228711b07386dccacbdb54cb66757631f9f22e2faf7da0eecc43474ab1a1af21aec22dcf949e2579241000000000000000000000000000000000000000000000000000000000000006000000000000000000000000000000000000000000000000000000000000001a00000000000000000000000000000000000000000000000000000000000000860000000000000000000000000000000000000000000000000000000000000092000000000000000000000000000000000000000000000000000000000000009e011cb7bbabfab668b42b5582cec8d60f326ae14ad55f8d675060dd111dee6c88d0937435261929ba3a881bea107593a10e13c917a0ef5ae4e8bde40de2d1da4ba25ab3a239ba53eb4e260d23b98a7db944f644f40ab75992a89fad40e8d692ad62f8671604db415b54238c5522f6c4e6b25785bffa696d173659309b95a2fae8700d4b9c14382973ea34a3cd8b8428594908dcad2d5d7a01056b8e49a8c25087f12f9e9fec1f4b5fdc57509dba6839100e28016b2e3fec4928bbe11d3789b6cab280c17bebfa5a0e06aea22764e1a18d2d11316bdf81028e65755582586ebdf571cbd0ef59800ba3405539187dfc03eac1dba0cf590e2131a56ef7f260aa38f940000000000000000000000000000000000000000000000000000000000000aa0000000000000000000000000000000000000000000000000000000000000000500000000000000000000000000000000000000000000000000000000000000a000000000000000000000000000000000000000000000000000000000000001a00000000000000000000000000000000000000000000000000000000000000200000000000000000000000000000000000000000000000000000000000000032000000000000000000000000000000000000000000000000000000000000005c000000000000000000000000000000000000000000000000000000000000000da02f8d782e704820101846a969541846a96954982dc1994a59477f7742ba7d51bb1e487a8540ab339d6801d87470de4df820000b8a4220b5b82000000000000000000000000f6213f16f3db9b50c684b2da37ff080895ae4a02000000000000000000000000000000000000000000000000002386f26fc100000000000000000000000000000000000000000000000000000000000064376b6900000000000000000000000000000000000000000000000000000000000000800000000000000000000000000000000000000000000000000000000000000000c0000000000000000000000000000000000000000000000000000000000000000000000000003202f082e70411846de27f7b846de27f8382520894b6605a717a2447edac46b1cdafe35e3e91130c8d87038d7ea4c6800080c0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000f202f8ef82e7040d846de27f7b846de27f838303335f947191061d5d4c60f598214cc6913502184baddf1880b8c44d49e87d00000000000000000000000000000000000000000000000000000000000000600000000000000000000000000000000000000000000000000160f9854c9f595e000000000000000000000000000000000000000000000000000000006417d4b8000000000000000000000000000000000000000000000000000000000000000200000000000000000000000000000000000000000000000000b1a2bc2ec5000000000000000000000000000000000000000000000000000000b2f4f1df2e372ec00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000027502f9027182e7048197846a969541846a96954983029eb1946aa397cab00a2a40025dbf839a83f16d5ec7c1eb80b902445ae401dc00000000000000000000000000000000000000000000000000000000640fdf0b000000000000000000000000000000000000000000000000000000000000004000000000000000000000000000000000000000000000000000000000000000020000000000000000000000000000000000000000000000000000000000000040000000000000000000000000000000000000000000000000000000000000016000000000000000000000000000000000000000000000000000000000000000e404e45aaf000000000000000000000000964ff70695da981027c81020b1c58d833d49a6400000000000000000000000002c1b868d6596a18e32e61b901e4060c872647b6c00000000000000000000000000000000000000000000000000000000000001f400000000000000000000000000000000000000000000000000000000000000020000000000000000000000000000000000c097ce7bc90715b34b9f10000000000000000000000000000000000000000000000000000000000000000000000614000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000004449404b7c0000000000000000000000000000000000000000000000000000000000000614000000000000000000000000694b3f194e94c80b2d7a0e986e1440f204f0503f00000000000000000000000000000000000000000000000000000000c0000000000000000000000000000000000000000000000000000000000000000000000000000000000000b902f8b682e70427846a969541846a969549830dbba094964ff70695da981027c81020b1c58d833d49a640872aa1efb94e0000b884437471fd0000000000000000000000004e44260732136caeffc7d076e17b2a3554b9ce2a000000000000000000000000000000000000000000000000002aa1efb94e00000000000000000000000000000000000000000000000000000000000064376b6600000000000000000000000000000000000000000000000000000000000dbba0c0000000000000000000000000000000000000000000000000000000000000000000000000000005000000000000000000000000000000000000000000000000000000000000001b000000000000000000000000000000000000000000000000000000000000001b000000000000000000000000000000000000000000000000000000000000001b000000000000000000000000000000000000000000000000000000000000001b000000000000000000000000000000000000000000000000000000000000001b0000000000000000000000000000000000000000000000000000000000000005b0efccaa7c845eddd4fb98211c7a58a37abdf06b264fade6cbff8a73ed22fccc602337b33a81e2cc4127be9d913fb3aec24194e14a61515a26429a00cfb86196aa3c2e6983f1a05dec045f0e63db7bd8619cd2a770ed1f7f470eb7c297e8e19d61277fb276ed8a63fd666b4a5ab02347065a3c16539f4ccccbed8fa39f2a7dc6606fc04c49f3d69463ba1c86c6ddcef8b9f06b9741dc2ebae9f5f1c0523b320b00000000000000000000000000000000000000000000000000000000000000050668b9ca5e5b17e2e79f9f5a7d9ea27e4e7f0302e878584fef3c2304c70c1ef53da311342341f89df488646b97a83994a3d07cc622ffde98598548c5326b7ccc0cf77d5d6fca6cf3f5b5b41a266fa2aaf697fd976dddc710fca834c96d2dfd725eec5980733db3c4e8f779f9fde3a598b2860d819419c961012825f0e160ee7b220989005b6ac5c48734db18158c2d85c8499bea1972c9f875c2d2aeafccb33700000000000000000000000000000000000000000000000000000000000000020000000000000000000000000000000000000000000000000000000000000040000000000000000000000000000000000000000000000000000000000000014000000000000000000000000000000000000000000000000000000000000000e0000000000000000000000000f6213f16f3db9b50c684b2da37ff080895ae4a02000000000000000000000000f6213f16f3db9b50c684b2da37ff080895ae4a02000000000000000000000000000000000000000000000000002386f26fc10000000000000000000000000000000000000000000000000000002386f26fc100000000000000000000000000000000000000000000000000000000000064376b6900000000000000000000000000000000000000000000000000000000000000c000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000140000000000000000000000000964ff70695da981027c81020b1c58d833d49a64000000000000000000000000073fee82ba7f6b98d27bcdc2befc1d3f6597fb02d000000000000000000000000000000000000000000000000002aa1efb94e000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000064376b6600000000000000000000000000000000000000000000000000000000000000c00000000000000000000000000000000000000000000000000000000000000044a1c058fa0000000000000000000000004e44260732136caeffc7d076e17b2a3554b9ce2a00000000000000000000000000000000000000000000000000000000000dbba000000000000000000000000000000000000000000000000000000000","nonce":"0x48f09","to":"0xe87d317eb8dcc9afe24d9f63d6c760e52bc18a40","transactionIndex":"0x17","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0xb5f185933d49064643686594f44a2584b1d57ba6a91f2a45d1a122892488f209","s":"0x7bb4af7ea30290b36fa7c266ee2387eff9e3ee0cd79b48aed27f5bfaaf34114c","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x4281ecf07378ee595c564a59048801330f3084ee","gas":"0xea60","gasPrice":"0x861fc89f8","maxFeePerGas":"0xfd18eeafe","maxPriorityFeePerGas":"0x9502f900","hash":"0x3792db6dd6285f409e4281951e9f78dad16c4a78072ff1c909dfadea5658d857","input":"0xa9059cbb0000000000000000000000000cc351a97c2fafde4cfa8a00c4fd79881cf13b53000000000000000000000000000000000000000000000001158e460913d00000","nonce":"0x77ffb2","to":"0x326c977e6efc84e512bb9c30f76e30c160ed06fb","transactionIndex":"0x18","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0xef4173dd45060791cfb6dfc3cdc8a9b417ff6a4b3ba57759ea91fd875b01a522","s":"0x4de926b5099c7d60cc24add3c2eecd8e9550c1fe7f878430eccb887a2ff932ff","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x53e2c2479833ca17f946aeb16dcc49f5e2194701","gas":"0x431eb","gasPrice":"0x8442f24f8","maxFeePerGas":"0xd6aac8818","maxPriorityFeePerGas":"0x77359400","hash":"0xd2d51764c01e8c0a43fbe362704388df5bacf7e5e620c3864e242530ffb3e828","input":"0x9f8420b300000000000000000000000000000000000000000000000000470de4df8200000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x2","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0x19","value":"0x470e9f23394000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x155352758fb04a6c0816c3dd9ca16bd6c66c37e2e19f2aa5d969225e3cb7f118","s":"0x6a4a1237db75dac4322fe12aee8b94edf7083ea661d6e93334c667d2d5c2c101","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x535619cf2bd2e84e567bda1a6164f195cb2bba21","gas":"0x55730","gasPrice":"0x83152e5b0","hash":"0x516b0227d9e64eb6e0de6862764d40f5376b5f12fec878436fea3479b4c36bb8","input":"0xa9059cbb00000000000000000000000030509945d2b329cc253a8bb2e2c54bdbb08b1070000000000000000000000000000000000000000000000010ce1d3d8cb3180000","nonce":"0x2176","to":"0x1173069b0e472a64ce2b6203fec8fb25a05b75c2","transactionIndex":"0x1a","value":"0x0","type":"0x0","chainId":"0x5","v":"0x2e","r":"0x289ee9948b80fc623b8c5d85c7df6fd25786fcc118163042b77a00560f53c3fb","s":"0x5b0a79e30d05370b27664b2a10cf964e110387bd62098561b6ea6a9dcc4634a"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x339d413ccefd986b1b3647a9cfa9cbbe70a30749","gas":"0x180ac","gasPrice":"0x83152e5af","hash":"0x81b0abc78b82840adb666775b182a9e292f663b64bcd35004c04436ed3c8281c","input":"0x711746e200000000000000000000000000000000000000000000000000000000000000200000000000000000000000000000000000000000000000000000000000000001000000000000000000000000000000000000000000000000000000000000009a00000000000000000000000000000000000000000000000000000002540be40100000000000000000000000000000000000000000000000000000006d3c514bd0000000000000000000000000000000000000000000000000000000000000010","nonce":"0x2689","to":"0x7cbd185f21bef4d87310d0171ad5f740bc240e26","transactionIndex":"0x1b","value":"0x0","type":"0x0","chainId":"0x5","v":"0x2d","r":"0x1f70a3d0933aab069a55fb2b16a259d21ea5fb3255e2032c5ff194a600ed2ab1","s":"0x3ac2142fc9f2009d68b08855eb5bc85e889e15d043c33bfc516f28ea38715485"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x339d413ccefd986b1b3647a9cfa9cbbe70a30749","gas":"0x180ac","gasPrice":"0x933ea77d8","hash":"0xd0287570d431d2baea96ecc81cb890e7f4f06ab5df02f9b4067768abca19acb5","input":"0x711746e200000000000000000000000000000000000000000000000000000000000000200000000000000000000000000000000000000000000000000000000000000001000000000000000000000000000000000000000000000000000000000000006500000000000000000000000000000000000000000000000000003a09994473bc00000000000000000000000000000000000000000000000000000005672af7e00000000000000000000000000000000000000000000000000000000000000010","nonce":"0x268a","to":"0x7cbd185f21bef4d87310d0171ad5f740bc240e26","transactionIndex":"0x1c","value":"0x0","type":"0x0","chainId":"0x5","v":"0x2e","r":"0x73a68e10ef9726d47ff314cd16d1846a0876d2b63a65014909140de7e45aa8b3","s":"0x580de27e4626139fbad4129a249c7ac2b22e1062659293e5fc3a244fd74907c"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xb5b9345e21f34b6cfa538f49e2b14ad4c3db3f7e","gas":"0x431e0","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x76ddab2674369f34946c5fa2f05e2aa8566d86235b83e808e9b27bc106e04ac7","input":"0x9f8420b3000000000000000000000000000000000000000000000000016345785d8a00000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x38","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0x1d","value":"0x1634632a1414000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x4314a4757076c92ca696f14cbe037bcc0178f8b10749418d4535195e90b2cdc9","s":"0x15ee917bc3e27928196ad987b61e3fd35ac92a4200b139d355b30502bff7fceb","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xac2fde165d9f13a223f58b9ac1dde51b63feff2b","gas":"0x5208","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x34a5c74011a2c8a00103bc91bfbfd94aa99cd569be69066e4bf64d188fe8714e","input":"0x","nonce":"0x38","to":"0x7155eba8ef327ac3b36fe59b5e00ab85f735f4f4","transactionIndex":"0x1e","value":"0x1c9f78d2893e40000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x52178c5dce512607a6242f501f78bc894917a482e99a3368caea6bca94c6dd69","s":"0x728be90654231b65f790eba478ccab574e2468fc4e031b712efa88af31fc6592","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x1ddedd674bfacb6587e0c68cee8f2f7d8a71e9d7","gas":"0x1d4c0","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x7b9730ead1b9f59b206d0ddea87be9383ba3fc7b496c7863b0cb847889b86617","input":"0x607019b9000000000000000000000000000000000000000000000000000000000000004000000000000000000000000000000000000000000000000000000000000000010000000000000000000000000000000000000000000000000000000000000041971588d95d4d664ac84a45e59fc81871b7051d679376d86cef6b9e1453c762dc33a84a125774319ee47f2a8b8d588da6c8ff7ad84fd9eed22fa124b1a5c5ed4d1b00000000000000000000000000000000000000000000000000000000000000","nonce":"0x1","to":"0xe1e6aef7642b6609e6c54884d1bda4a3cb152461","transactionIndex":"0x1f","value":"0x4380663abb8000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xc0d3e609b2e9f844d300a7784aa9e7cb956415d025cb573a73848f49e4be69e5","s":"0x6ce01527c00c1a93341a541824aed7a1c895f5a0d998f4415cf6d88171c5ad3e","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xf6df8ead0bdabfba61d9b2457d5a5ccec67fb7f3","gas":"0xc836","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x77166ee0409ba86bd26e7c03ad1a927abaf5af8a8a37149e725cd37512091dd6","input":"0x67a5cd06000000000000000000000000fa8449189744799ad2ace7e0ebac8bb7575eff47","nonce":"0xbd","to":"0xccb0f4cf5d3f97f4a55bb5f5ca321c3ed033f244","transactionIndex":"0x20","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x5649e1ff58fa9b6ee27a7316904df02db1ba1c471a2f3409f370a31c6b3d5ec4","s":"0x386b766277e267f8796ba6e92af71a752335bbe45cb3385b9cdcab9ef6c53e41","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xf37cbbc2dad42e8826898d45933afd3a04937e71","gas":"0x112c1","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x3c2b6c2ae505c5c36d5f316c1fcb5f54f7346ed35ae35c93462991ded7968a68","input":"0x095ea7b300000000000000000000000071bcefe9e9fbad3f12f2e6c14ea0cae7f0c91e7900000000000000000000000000000000000000000000000000000004912e6bef","nonce":"0x9","to":"0xb5d86e1f237d52f1f19793d3265e9340394f75b8","transactionIndex":"0x21","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x3ee4d5bfc575fa4b241b4d23e6f099eba5f1e6a20b8cea663c63db6400d01506","s":"0x21481ea7a3e7ad758f2e149829f70f43d67b3cc0d44112cd3744e290a06dd985","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x52b6255b7810d97de8f751737547d1792dff4fe0","gas":"0xece2","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0xf99a792837e13827b5e0a8915fb59c760babc95d242feca99a5594e64ff6b6e2","input":"0x095ea7b3000000000000000000000000c92470d7ffa21473611ab6c6e2fcfb8637c8f330ffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff","nonce":"0x51","to":"0x07865c6e87b9f70255377e024ace6630c1eaa37f","transactionIndex":"0x22","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x5237f7449c0df409c1a4d6cb09005a9224860aa74737debd961e2a24668d1022","s":"0x17e5e842e3f5ffb5a16eca0ec2a92e5b52d43bf9856f97e2083b2e951499b970","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x3c70503d83c5a681aab503d724f47fe810ae8205","gas":"0x17eb2","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x522313f5d923f048ae5bd0b5595c1f4fc883bc0b3cf3cb0939d3fcf8b08c829c","input":"0x94d008ef0000000000000000000000003c70503d83c5a681aab503d724f47fe810ae82050000000000000000000000000000000000000000000000000000000316195298000000000000000000000000000000000000000000000000000000000000006000000000000000000000000000000000000000000000000000000000000000010000000000000000000000000000000000000000000000000000000000000000","nonce":"0x5","to":"0xb5d86e1f237d52f1f19793d3265e9340394f75b8","transactionIndex":"0x23","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x6741309799fccf0f1359bd710291ec1bb66eed23522d6ed60c5924264d4d283e","s":"0x69c43182ee8a650b0042cdb2352781b23eb1de72f1c3a8903a1a33360e7d9732","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x4da9a48963c158a88bf6d4845a6a1890d20e55ad","gas":"0x431eb","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x471ceb0e85af594aa56deca54cb8198567b2afd8406722ea530077aaa6b641b3","input":"0x9f8420b3000000000000000000000000000000000000000000000000001dd7c1681d00000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x0","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0x24","value":"0x1dd87babd44000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0xf977ce19da80b7e155ee2d8b7656e5c89004851407bd5370785befe2c89d82f6","s":"0x1d76177407e73d8b73b0909158825ee375a25fa7c18b04884f4be77bb60795a8","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xe40a2cd9efde08fb7664bcdbbdd877c52c4bd228","gas":"0xf4f6","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x3e9dca502e9039ae0c6d642f62e9562ff00010c6bfbb8234a6135712ba70dfda","input":"0xa1ddc5460000000000000000000000000000000000000000000000000000000000000738","nonce":"0xa","to":"0xdef65f8e2f62bde60cd86ad3b4ad7fa6741cf175","transactionIndex":"0x25","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x34df85658956c9eb71549db3a73abca70dd01b8e91284afcd372c1212a36605d","s":"0x5036e2984bb2edbb5fd647bba873f1f5c5ee4845e539c9b0bc70d3d12a4bd93f","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x17299065e99b12b0c8cad06638624cff8d507334","gas":"0x5208","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0xc95cac67267f4accb9b5950316ac64772f7d082bed6b712c09cf2da0bdc237b7","input":"0x","nonce":"0x7","to":"0x02647cbb1714fa9a5747ec5be6333e06d806e85a","transactionIndex":"0x26","value":"0x71afd498d0000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x82ee5cafbd010cdf7a4604df728d1b8713a5bd26a708f5c92f6b6a55864c9576","s":"0x2b9a70d812123b2ddce1108c4101ddce182f6c83fc8aa0a139ba597aa178b270","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x9aef7b879787baf0c2dfe888e67a7edb7a233209","gas":"0x431eb","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0xfca28fdbd13fc16daf7aec7d4a2ad2c6b5f0b2a7b0fb1d9167c09b5e115ff26e","input":"0x9f8420b3000000000000000000000000000000000000000000000000002386f26fc100000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x2","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0x27","value":"0x2387acb3784000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x7e5f1a27ea5e652ec8705f80365f1703d2fd50c1de7c22ee0181d556dd2fcc38","s":"0xb22fb8b31dbfff8ad0457ba4b659fb0d7031cd86f08fc3cb78d81c97248034e","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x06006a25e0fe65f0eeaad1f4813940b68db782a7","gas":"0x17ea3","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0xc73124ca798b2f7a5df2ea4d568efab2f41b135130ea5cc41d4bcb4b5c57d5bd","input":"0x94d008ef00000000000000000000000006006a25e0fe65f0eeaad1f4813940b68db782a700000000000000000000000000000000000000000000000000000002726bc7f1000000000000000000000000000000000000000000000000000000000000006000000000000000000000000000000000000000000000000000000000000000010000000000000000000000000000000000000000000000000000000000000000","nonce":"0x6","to":"0xb5d86e1f237d52f1f19793d3265e9340394f75b8","transactionIndex":"0x28","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xabe4fc843d3a1628a464884c13c21bd21ae9a0a5738d9ae2761ec7cb24b059d0","s":"0x68408d8096e542b2c06cbbecb9494a3778c3ab644e69dde1fac078079c4bcad4","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x2501bf5c8ddf04a60fdd47a95d53d99b315604e2","gas":"0x5208","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x29abb76b5e7a5ce137bf9c22474d386eb58d249f43178d2b2e15c16dfdc5ca80","input":"0x","nonce":"0xa","to":"0x901a4e04f1513443c6efc6ce4e913d1b960ecd20","transactionIndex":"0x29","value":"0x1a3385ff37f0000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x1a1789d5b19950f45383cb1909cff9128dc2b332e2ea7ba42cfca5806376e3cb","s":"0x617a5a4339503a9c9d08705d98e7a202ed7ede1d1192fd7d98b648c8c102716c","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x507832c92ade971b9a2880427353c6ce6d0ffe53","gas":"0x61a80","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x03e5ab25a58bd44fb9dd0c698b323eab8b8363479dfcbcbb16d0a0bd983880ae","input":"0x85ff842c00000000000000000000000000000000000000000000000000000000000000c0000000000000000000000000000080383847bd75f91c168269aa74004877592f00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000006100000000000000000000000000000000000000000000000000000000000557300000000000000000000000000000000000000000000000000000000000000014507832c92ade971b9a2880427353c6ce6d0ffe53000000000000000000000000","nonce":"0x1e7","to":"0x805fe47d1fe7d86496753bb4b36206953c1ae660","transactionIndex":"0x2a","value":"0x214e8348c4f0000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xf707322f65ba56c95ced5654efafcf9bfd60a2dd80e74bded3190673cda55d8e","s":"0x46412ec3d2137fcef30f53cc4644769f4da126128979920ed4e209acb1c8ebc9","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xd44c850ce9103d6b2398d4ed9bbce321f3b769c7","gas":"0x1e0cf","gasPrice":"0x82661bff8","maxFeePerGas":"0xde6f782ba","maxPriorityFeePerGas":"0x59682f00","hash":"0x3c8ee80ddea7fa2d2b75e44563c10c10756f598e8ad252a49c5d3e8a5c8e6cbf","input":"0xeb672419000000000000000000000000d44c850ce9103d6b2398d4ed9bbce321f3b769c7000000000000000000000000000000000000000000000000002e2f6e5e14800000000000000000000000000000000000000000000000000000000000000000e0000000000000000000000000000000000000000000000000000000000098968000000000000000000000000000000000000000000000000000000000000003200000000000000000000000000000000000000000000000000000000000000100000000000000000000000000d44c850ce9103d6b2398d4ed9bbce321f3b769c700000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000","nonce":"0x9","to":"0x1908e2bf4a88f91e4ef0dc72f02b8ea36bea2319","transactionIndex":"0x2b","value":"0x2e2f6e5e148000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x327a6fbcaa2ac3cfb0ce64b47f6e61de4d18297f1305f23c6bfd42d88e8d068","s":"0x6b3a9b5252ced14b2e2f2833ad7cc62c46fbd0785d57f2747473af3fff7d46b8","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x78aa9046272fec8d51a402681d7c37481db01c6f","gas":"0x431e0","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0xaffa73b68bc7ab0c3f5e28377f5ca0a5df33c0a485f64dc094b7f6ae23353203","input":"0x9f8420b3000000000000000000000000000000000000000000000000016345785d8a00000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x36","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0x2c","value":"0x1634632a1414000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x925370b51dda3e1d1bc6b86580fda3b40875157ea594a9a14174436c726ad46c","s":"0x516c88bf3cb2f46ca81572140b41e5db2d74c4a736ee7d572b508415c438f779","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x56784e51c2ce28dbe9aa68342638d1e95ea4324b","gas":"0xb72e","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0xc66c9c66fbc8fe97fcc16506cde7a58689af1004a18c6171cfe763bcd94f50b2","input":"0x095ea7b3000000000000000000000000805fe47d1fe7d86496753bb4b36206953c1ae66000000000000000000000000000000000000000000000000029a2241af62c0000","nonce":"0x9","to":"0xcc7bb2d219a0fc08033e130629c2b854b7ba9195","transactionIndex":"0x2d","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x407c855bed0fb088e886c3721ee9abee4761221d110512ac29b8fb143c4b34d4","s":"0x1e47aa957d9ebdb564dfa41b72022d525383ffd825a307fbfa9ca13a59994dcc","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x805e39acf2f170251c48a35df16411cfb2f761e3","gas":"0x56a8","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x80fec96707519172b53790610d5800cd09a4243aca9bacfa956c56337d06f820","input":"0x71ec5c05aa669c4922569c1d33f7a81aaa21813800000000000000000000000013a0c5930c028511dc02665e7285134b6d11a5f4000000000000000000000000805e39acf2f170251c48a35df16411cfb2f761e30000000000000000000000000000000000000000000000000000000000000000","nonce":"0xc","to":"0x7c125c1d515b8945841b3d5144a060115c58725f","transactionIndex":"0x2e","value":"0xb1a2bc2ec50000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x8fcf376106af19aac48ecb840ea51df90a5e3ba557eeaad41cd3c8a90785d416","s":"0x4a9e9021ddb5f839bf7f54b54ad52c57b21b6ed285773eea022e3ccae69d1b99","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x4199614bf74f6b3ef47a812d976dd1f57a8f082c","gas":"0x431eb","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x61b33bfcf11214906dcdce7d7ed83ad82f38184c03ded07f7782059d02eeedea","input":"0x9f8420b3000000000000000000000000000000000000000000000000002386f26fc100000000000000000000000000000000000000000000000000000000000000009c40","nonce":"0x6","to":"0xe5e30e7c24e4dfcb281a682562e53154c15d3332","transactionIndex":"0x2f","value":"0x2387acb3784000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x3d295f5fa2dbec04793f07adec57e29c03376c628a0256ee297fb3070a1e3364","s":"0x6f115dd4505bd320bd7460a47bab0c20e6bcb4b3f17c6d0e00e3ff1d936ed2af","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xf6df8ead0bdabfba61d9b2457d5a5ccec67fb7f3","gas":"0xc7c7","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x5d4138d4e28a8327e506cb012346b1b38b65f615a2b991d35cf5d4de244b3e6d","input":"0x67a5cd060000000000000000000000001f1f156e0317167c11aa412e3d1435ea29dc3cce","nonce":"0xbe","to":"0xccb0f4cf5d3f97f4a55bb5f5ca321c3ed033f244","transactionIndex":"0x30","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x47bfb48eb89a8b110749d34d171db36a8ad21c8d7cc78c7489e72f21cd86ba14","s":"0xa525976583b19b3d2c2dd6b34324cf55b1a61595cb53e3c58acaf203f8709e6","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x9eda8333e09828571db26576a566668f79f98946","gas":"0x50df20","gasPrice":"0x82661bff8","maxFeePerGas":"0xc8d5e2427","maxPriorityFeePerGas":"0x59682f00","hash":"0x875a142b6dfcf10ffb71a7afe0ce4672c047fc7e162ba0383390516d6334d45d","input":"0xb1dc65a400011a04d404e571ead64b2f08cfae623a0d96b9beb326c20e322001cbbd34470000000000000000000000000000000000000000000000000000000000bd780a7d2e00ee03a29f787096cae99da52d07803c76c13473654239118e2cf0491bcb00000000000000000000000000000000000000000000000000000000000000e000000000000000000000000000000000000000000000000000000000000004000000000000000000000000000000000000000000000000000000000000000480000001000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000003000000000000000000000000000000000000000000000000000000000077359400000000000000000000000000000000000000000000000000000e50bb6420faa6000000000000000000000000000000000000000000000000000000000000008000000000000000000000000000000000000000000000000000000000000000e0000000000000000000000000000000000000000000000000000000000000000217f0aee859cd78a5cdf79c68c8de4342658d52ccc6420181b5fcb7c657c5fdfa5ca2c8f2dd7e6e5799d44d6818e6dd8fee19b2530bed2045e0c665041f053fe70000000000000000000000000000000000000000000000000000000000000002000000000000000000000000000000000000000000000000000000000000004000000000000000000000000000000000000000000000000000000000000001200000000000000000000000000000000000000000000000000000000000840235df8360bffc902a68675311f5eaa79ea2c25e2cc394bff0a43d5d608e98c97de200000000000000000000000000000000000000000000000000000000000000600000000000000000000000000000000000000000000000000000000000000060000000000000000000000000000000000000000000000000000000000000002a00000000000000000000000000000000000000000000000000000000000004b000000000000000000000000000000000000000000000000000000000000493e00000000000000000000000000000000000000000000000000000000000840235df8360bffc902a68675311f5eaa79ea2c25e2cc394bff0a43d5d608e98c97de200000000000000000000000000000000000000000000000000000000000000600000000000000000000000000000000000000000000000000000000000000060000000000000000000000000000000000000000000000000000000000000003f00000000000000000000000000000000000000000000000000000000000007080000000000000000000000000000000000000000000000000000000000061a800000000000000000000000000000000000000000000000000000000000000003312d88cdb7cfd5fc5458d9a424da374cd818847f87fc4fb6eae2c3b97e541a6b81e108e55daf857b71514fb7213db7ba15bd6adc12253afc3c5e949144c13696faabe22d1b7ace91532134a5cc65bed9afdd897d6f84ad1b0b6da463bda8fde5000000000000000000000000000000000000000000000000000000000000000352c2a75f88112864819de485c392c349f718fc257934181c4d8ac8eb3a9b1fdd3b7bfb96496290f2b41fcd20fc8d29bd8e29be38550775dc97d21b17c2d679ad05f519787eb2c92a9a897c9f02f15316865cfeb4bcc2c0ba5ac2dc4aaab5c0c9","nonce":"0x3705a","to":"0x233a95ccebf3c9f934482c637c08b4015cdd6ddd","transactionIndex":"0x31","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x7e9247209bede2c21548c6e4f26e41d2636a2490e09ad062f9235e978094073d","s":"0x288d820793b48e55777e794f1694b1605d74dc8726fff438b481125f219eaec","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x6df2b54f23ed4e95235c6e21ada2cfff5a2615a6","gas":"0x5208","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x79b6df832bfbd04085d0b005a6e3ad8f00fc8717eed59280aa8107268b71e7e0","input":"0x","nonce":"0x17f","to":"0x6143d939b2d117f4b998d8d88a56eb0e0a1e9c31","transactionIndex":"0x32","value":"0x186cc6acd4b0000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0xcb641be908976bc0a035b0b31a96039da9e63e829e0a24a092ecfbce7724b9c7","s":"0x4034e7dc4001cf9f62a2e5dc4390229655b85e9aac5a5b263278e9d2cb43abb2","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x3f5ceedd435b6143eba2254811a39fc3423f18dd","gas":"0x186a0","gasPrice":"0x82661bff8","maxFeePerGas":"0xaa246a58c","maxPriorityFeePerGas":"0x59682f00","hash":"0xcb2fb25d268f65dc9312e89bd3c328c9847a3c9da282026793c54a745f825ab5","input":"0xe9e05c420000000000000000000000003f5ceedd435b6143eba2254811a39fc3423f18dd000000000000000000000000000000000000000000000000001c6bf52634000000000000000000000000000000000000000000000000000000000000000186a0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000a00000000000000000000000000000000000000000000000000000000000000000","nonce":"0x0","to":"0xe93c8cd0d409341205a592f8c4ac1a5fe5585cfa","transactionIndex":"0x33","value":"0x1c6bf526340000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xa48a4ece6fbe881e342a4eda767c82097c3bd47dc70c09ef6a4e7f90c0f8e843","s":"0x66c92f16907488d115f5088ed8ff4d71c966232458479245bf6dae45358c0429","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xdb4f0eeb7c89b00118cc74de8c9ed55d65c2b933","gas":"0xb72e","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0xe483d4a36ad19fd5eacb7f6d9ad3ce080ad70ac673273e710f6e3d5acbc6559c","input":"0x095ea7b3000000000000000000000000805fe47d1fe7d86496753bb4b36206953c1ae66000000000000000000000000000000000000000000000000029a2241af62c0000","nonce":"0x120","to":"0xcc7bb2d219a0fc08033e130629c2b854b7ba9195","transactionIndex":"0x34","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x6d73169394cb6a9bceb2dcb16aacd261705096f50f30ffe146a09659e25dae48","s":"0x496a1cccf01ad2ae7094b8da2ce50c0a990baba6300625c48ca12c5f20005a64","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x9ac253db14a434cbe7653a99c5039648fa4f7353","gas":"0xb4f8","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x0564242c37d5013b671ef4864394cc0f3924c589f8aad64118223a9af2f164f6","input":"0x095ea7b300000000000000000000000028ac7dbbf9ed6ece39a902e76780fba60f1b2e3b000000000000000000000000000000000000000c9f2c9cd04674edea40000000","nonce":"0x4a","to":"0x0dece1605795461158538302b5f0936afe78c4db","transactionIndex":"0x35","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x15684c9697dcc8d89abe562c4315bfb959938fe7644c00615d9e147a4e496ff6","s":"0x7c58abea24e1fa8caeb73a70219b0d63007785c39694fe5ed2a5a342c99292a6","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x22496eece162f52d185c07524d452b9506e9fe12","gas":"0x186a0","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x48db358e80b278c3a46c2a166339797060a40f33984a5d974992cd9722139d5d","input":"0xe9e05c4200000000000000000000000022496eece162f52d185c07524d452b9506e9fe12000000000000000000000000000000000000000000000000001c6bf52634000000000000000000000000000000000000000000000000000000000000000186a0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000a00000000000000000000000000000000000000000000000000000000000000000","nonce":"0x0","to":"0xe93c8cd0d409341205a592f8c4ac1a5fe5585cfa","transactionIndex":"0x36","value":"0x1c6bf526340000","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0xa50e6ea1eff21fd33d69f8c74b90c28303b92ebc4a77c71b65c4c8bf3e92ac8c","s":"0x29af3bce2365578b76dc528917b44e96e4fe2e7a8f948eff3e379721f35664d8","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x9ce3f4210dc06889ad5267ef0ce3ded4bd96dc50","gas":"0x249f0","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x69d7758db91fae31fa35ecbed4d40897c5087f45dc796cd796b8ceead21f972e","input":"0x47e7ef2400000000000000000000000089f01bc9865324583380d8d7ed08b8046bffd7fd0000000000000000000000000000000000000000000000000de0b6b3a7640000","nonce":"0x1c","to":"0x6cf3df5d00e842aab1eb504bcf01ebf81646f7b7","transactionIndex":"0x37","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xeefc19e8dca874308c84712c3e48c032a25971a0f7dff16dc47633ba7a4f5b96","s":"0xc0994cf21d29cb0c5f1e666538b58d6dff3b9ba2aa7545337da732c4b5938cc","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xbd1c4e36842c06e53b378eca310cd2dc79a78cbe","gas":"0x35965","gasPrice":"0x82661bff8","maxFeePerGas":"0x97f5c211e","maxPriorityFeePerGas":"0x59682f00","hash":"0x2951478916ecd27a8e808d08f85be4bf2c0b0e0546f21f4e309145dd96eb8df1","input":"0xab7d7969102507275d685f3f55d10ee45f7b9fa39c29a8a604a7a971c05eee739e63938500000000000000000000000000000000000000000000000000000000641cfd18","nonce":"0x252","to":"0x96e6192eeaf7bb308f79fb5017a9085754b9e12a","transactionIndex":"0x38","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x87fcffa38e0c84cf80fb37ac1e538faeca21462b0a9ec25af0ca0517af22789a","s":"0x72cffece99e79edb7425a9d660590fbc395cdd0db8fd4848074c85f814b3cf63","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x5dbcb3e489fbef1656dbbd7bc4848ed0e520a089","gas":"0x7a120","gasPrice":"0x82661bff8","maxFeePerGas":"0xc8d5e2427","maxPriorityFeePerGas":"0x59682f00","hash":"0xaca9028cb5d55bbf71b7bff9884a9a3b0b38a575ffc8f8807ce345cf8bd298ef","input":"0xc9807539000000000000000000000000000000000000000000000000000000000000008000000000000000000000000000000000000000000000000000000000000001a00000000000000000000000000000000000000000000000000000000000000200010100000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000001000000000000000000000000a7a9d91e687decaa6b045797c2e002d70001b48f03020003010000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000600000000000000000000000000000000000000000000000000000000000000004000000000000000000000000000000000000000000000000000002361273ef6f000000000000000000000000000000000000000000000000000002368a4faa14000000000000000000000000000000000000000000000000000002368a4faa1400000000000000000000000000000000000000000000000000000237b9042149000000000000000000000000000000000000000000000000000000000000000288e1b54f6e54124deb78bdb67cbfd0bf63612da12868fc281afc34bb26200a1771f9e00fce297da11cf49427becf350d0bd20a2277439fbb2e8db0fbb5e3d34700000000000000000000000000000000000000000000000000000000000000027d16a90ad0e1f8d924255e6380764fa7c7d46427c8e8ea19f5ba6a9d595297a340f141f74e705d0fe851d7564eba89fcde574deb5975d9126867cd927a0d92cc","nonce":"0x11d38","to":"0xbe57bac491de9a260abb6ba2c9ad4c5d2eaea09a","transactionIndex":"0x39","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x1eba7bf95bf63d02b74d20554af050eedd78ecfbe1a42ec70badb36ccfae4bfd","s":"0x4d7f8d0ac2be28e59374453f24ae6506a81707cc57f0d1c6877b49b3949a9a79","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x3079be9d8622173f02618ba2b793f00795d4f320","gas":"0xf848e","gasPrice":"0x7d9f6fbd8","maxFeePerGas":"0xd9a8cbe9a","maxPriorityFeePerGas":"0xcfd6ae0","hash":"0xc7f625a19ee41a1750eac9428b4394a9a2476b8ea2d31b4c2f9f5b4fcb86cae3","input":"0x2169f79f000000000000000000000000000000000000000000000000000000000000006000000000000000000000000000000000000000000000000000000000000fdce50000000000000000000000000000000000000000000000000000000000003f2000000000000000000000000000000000000000000000000000000000000001f579a43ce4f3e38c6a948f1ef3bf7954c2059b087b18fdb22c7b69d61727a5d2aa16090b4d1f3281bce660a31d3e2789eeb360192a54e3ac3513c5c8f67b7346fda957c92c1a2981bc68a47d0282e78df2fb63822f06f410e1ffbee7eb83c425122be9be94d817e2e42099632fb9eb5ce7c414cf6ce48e4ed2e64e46e1558ca809742159e841e3de803a8cdc628f7877bfa6d75af4b4ebb185e6d0670ce83332478137fec243a38c0a80d067c955cbe13103294dadfc8c356d0e7996cb62fb2c683e6dd5ad58be934f57375920142990074f8a161ad2390a1e6bea9fc3a58dc3f7e002e9952a68bff229437d3fb4cc1cd8097032b22c884868253d8f2f69c75ada0fbe1dd57145610ae5f451b47645718aca7432d8cd3193d0b96417df6638fd47062a1cb1f63636236fb72195e5a46d1b37f493b8380315e5bb1a6fe67cc6546d62f2c73aad5699e6ffa0d49ce653bcefa173fe0de7b4328aad75f74d15f4f570a1839514550d576e21b8d16c8355b770442f0740e6e5e77ab450ee1fadff8daeeb25a91585192fcd3cdd64dcb6a35e41ff47e854337ebd0923ce280cdacff2ced48eac96bc9f910391a44cfec243a8660ff80c10ff844ea22411953e0387f87d2c2ae81055fc8e87d59e04cd7a923d6aa5204b3eaaf5b842b5fd656e9fda16c59e90c64ef51124fb32f8e63442b1f79d3ba3c0cd5fb90895103719629857d5999b44aad2d69380f48dd824795a89dce14bb82600701ad8fa9bbb36fe93a97d75da26ffcb1a8828918e0293680ecb12e1b772fd98637a7a83cffbca1a994fc889ed0c0b20d447ff727b9f5dcb9a83e1aae9a3518dfe1d7ddab1da48310eaa7ab293d85dd9ca811bfdfee990459fa044c9a9fa6460e5ff7612e42021f16854d790710684663bb6a10ba475fa602b02519f03eb544fe486d9cdde47ab7d4027bfb8f29d778b3a7cc35a9f78dbb05943062efe3b388f1a8c496682a0215265b99fed0363a1061556a40e0539a6656b6cf8967c8fb3092a7fc9b56af124c8f1bd0abbb9af753884b22b333410d47d80c083c9fee52dda8907be0c87c5496f1fe9ac4bf4dd4f06d6ac0ac0cde22138b67bf782e8e9b5c991d58a2a828b4c68620e6d3d26abf8d41005a00866cb4095518b2711636f2fac3b5e26403ef96f5b1765519787ed74c6802b445de029d7bc696aeef5a2e605376aeda4f495c29af66e357c6e1232d7a2c3f22c0bae0c62372d6b2690e503804e2ad314c38a4ffc080d36213458f606075d32dc0a365087616e94a972bf76d7e5eb4a53292fd0d7570cb6fbb4540b3c5ee6deee0f257fe8703d5d8d0180d09754bd9a05894e77318b8c8fb6b4a56999bfc6c405c09d9ce7fe424812db5bf778d7d8634b2f44ad6d00f18615e4c3695d9c5965ba51991508405187275840a3f27cb3101a9742c49edf42dea79ae5418ada42dfe2042f931af15a205e64196b04ef53492a450423c2b63dc849ef72a166fac5f36b25c72d9cb4a2507d5ad15094a979b4ac4ff8eb4c90f448a81fc6dbfcf6a228675ab3308bca37bda8eda22b61ec7e596f2448ee6b7cb6eef589f43f28fc5444d1da3f82ddbe574b774719a3a7869e897e642735517832959ea9cb9164eb2f7665bca1d2e44e1093acc7088271f2be20459c383f5762303b553af1ccfb3371e4eef1f102ce70d62620c5164972facbfbe03b5018265ed62c4c3cc49522166913517e2f2718edb591047a72e818f07c645a18aed87c0384d180dc0bb7fe63bd335b1c09e58571b92e2267ffa5712a33e4a92231d9da7b005730f6d6559ee685edad9a9031c4708c8642c297287af3a3e637d9b5ac6b23218977cf4d30231a79063987f98313a2bc0ca4a73a97e441a45b58cdd8731fc269255b415a9d2ef2bb8a311587f39f6dc82f537cf86a68af19a68c409910f4dbebdfa66445361877a3bf551480d115d8768e8650e9979472a64802da3625cdbddee8f02678a3cc6400824d91b3c7d4c9274b74fb11a9538f1990fc314d35564a0cb0b384590e15cb93db1de1a837b9319828b3fd6b76f1d2bca4ac765f9ec4366905f2dca69a4c8c194cf0a5d821c58bd291e8dd4ddd829a873d093042c1b8e1bb2bb1a95e150c0a0d5c3369c4a791dfcfbc3e631da3d1e704214c3cf8318901f9c58465be78a870a040f34c567c27eb077db2d1ff825a127a038eb14c2890ff9a8a1dc8d6360a91fd788fec97599a318ab03f69bd50a5ece92d96c49413b517579ececf71371b6b1d902b1dbbf40c728c64fadf368582cc4b3c51712c8d6441c0c61802943624e93f9de9eedde87056c21ecf66b3a6977af0261ef48b4c6191ca7ea70d6acc5af910ebf247e2a4429c506c6a41f865175a9bfc81dbb69e59ec2c47370fcee9169b8749c852e7a1275c1278aa3f88bc16d311e58aa9a579ed7bed62f12b4c4dfe9c493cf33f979f667eda043c2aaaf863a747b208f3aa2ccc34f07b12d89bfd75b696a82c493e7eedcdc41eb01321ecfdcb691d14996c3ed7487b5f831436bb7c25876c4149cb6222dbee9e1a05771a8abfd0fd530266476e1bf69d9a15e99c901243585f56a77ef0e6c684a4d49417133ebb91f896176ec3a8f0b966eaffce96f129d18960fdbf5917f8aa0926d52c57192f99e55959369e57ce85ae84ad944e40652f6a81ab8da1dbe81a827366b8b8d2ffb9c7b93087f7a356a16d0022ee8ae891060e29ff11f9f17b01f39b8024baefc8b928a6816309a5c3fc92ed6007499353b5f1bc5700d5fe67f3e34ef3654712b9c67c231f30dff485213b94c20afaa91540560df80ba9ff1cc0f6ea75bf1dac0566551050873c112478f3fce6af162397161e1b41e1ca295f327a4f0ccf4d15fbe58e20a23698bdc25c751d020df371cd1fc59fa51dbf7ad619d208d7084c5752984b5fd0e3a7978f2c11e6b26531fe03c0cb85519483d668bf1dc4dce24509b00aa6e146adb49f31c2c937a2d3a08f1cc0b73cd66f2cb926f0453cf78618e39cbfccd6e6a3b4909d6874872eefed0e395ac2582d8c1d0483007fa6ed7b735323fe8fa1c4be83c725c6b743f63cbd0d6017f1145634b0b905552a2a237561b37c415146d0555a276b4b45d515ed7dab942f9bf40a26d5366032f23fc5fad8b98a1625a841c8cc97dfe0e2074a363a951946df16d1261ef6b2da7f5789dada153448b076637f9d2e04f4390e9dd32fd5e9ab3f5939807bb451bd19d962403da9dd118f1cfa91731973bc721e5ae4a687bb2270e3e76966dd2156e95796d4bfa6b4c10caaee46e1d8f8495f901cfc241c6f9a690349eb0752adb220996ebb95ed26ef376dbb9eea201906a6a3eb346c64eba3c9e4d1b3088baa0ed868cdaf063a4c2434b9d1bbbd69f44ef659262fad9535988ce729aadafcfc6c3ff741dbf8b784b3465fb05f6d5daec91e915f89117e01c9b0c99d7b7cb78e44f5c28de0d3b782de981be058ffa1294072c4e5a66d2d81641d1c07a9a04012f9f1ad15028709fba761a659becf01963de0cc42b0e16bc5432cffa00722a84628373198dbbe75f209115c4e30e60297c2d737996dd9aac330f4a3e82609dd33460438026ae287c172af8aeed0d6c71b1f9470d60588ebd365f20d596999d97c7a6f265890c719dfe43ae1ee5bba122d9d4cfc96b399f610a90606add4f0d25b903c7ddf6eb7bfdc3b9f926243108f591678b0e92a48d52b1dbcdf30ef8af74cfe34d48d0e02d0f99af2adb090c489d4214f3562d3f6a6e9b83b54bb6cfe6c1961ecd2eb2b7eda63601c835c5dc696b32bc495bda9b6567984f83e79a9d4ee88b140dc78cdb64a2ba8761a827cdc8d2b27e690c22402178ade602f2e9092a1b07d56d142bd24d665d7b10bc7646eb6d6823d5622f1fb08f2e48b8deda6cde324d6508fc428e3bf1a9ed608c196786db4e8d6b7271a07e4f73d9ef76244e3e10aa13fd9b82ce2ce72af2df7faaecaf95d885e64bcae16f9b59ab8e0f21a2f459d00a290b0dc57e10bf21ce06a029224ae15cb86bb1cf060ddccb1f6ae82d767eb077fa4ab0f37d02a4bede5d3f8ee3972a6a3a438a3cd92ffd6c1fb11cf2c68c2c6d1efff62b9f769e302f6c5eec96b0c05a942321ef8b9cf0ff2bb80073a8fe138bac5bd8e93f3b27e7ec7a78512059fa37b0b6939f3b1311cb6ea154909e9d316567e013df315e8f02a80e17feb8f3248cc3f11d1253ef3156d7f12d1f6d7a382d6b6ac6abd38fdb5d954e13ce005ce16bd5625a5df01883409f4ab938a59a75b7a816af6a2447b2c65ea55c0ec4e4f23fde145a26ef0b13b740b352809c10b54bfdf30592a8b0d507cb8210391d66d6f68d06f3a4663756697d472adb6ba78976b423ad9ccb3a3bead9de51e324fb77609729c78664f4d37c7c748ad9fad95af355be4f00a76cd5cc2c0b6170fdf3efaa2571bc8fc2d0d24682fec5e034460de55cd0426b8ad448551032e1f6ade6d96682bc38a92c296e5bdbf42fcf106d14c5e8a4acbfc0cc1f0bd6877c4469be8b204ffd5d7c0ee85e7b3265011ad5e4896f9f8a69997e321a797549cd3c1cbb0301ff30c3a136068ba37b2176e556b574a542db90250eca38f3b0e418e352b5728206d6d531ea253d781d771ad974b476a045ff14718d9a1820fb469e981c7b71ae8fe423b0a04278bcfc967ca602254de37751127fe3716456df3fae50ed508e529c2181d63479b6464bfcde6629e8179f8d5fa5b61003edfc2f129affa58608d6145de98ca8f1c0b3b9b5ae996a7fbc6326a9ac2880206fbeb1c99d0448cd1ac5f275b84e4be03258f9fd0d3f44a692513957fa083a462086787315e38aec8c09b96cca034cc6294359fe00285c607d41fb4e3bf25760ed1ef0802bcc40b2fce2cfe1c64f7c34b21d0bb1f5210360079ca8638ab42943b0ab8277f1cfb1e529073d2068a36a4d8ae8179c32bf66ca335d698f76db1cfe5236e6427e01804464c1ed4facf18ae09148c5a0cc9ebcf6fb6c7d03ec6782b7595816245ac2d1fdb3184ea49800d5d0f1dd880ff99d6016549d63ba2a44ab96f0db0d0cbd5fd4f67af1f1da0b1023883d4b2e7cb23cf85c617c180a771267fe63c70f19eee28fdc50c57d73b2718f8001383135b859f4c2fa3b9a12d095beba8857df13e921d9dcda5b87e1fce6038fc2f92282c717305157b6fca5ca2c23cb7307a093512f985628e75326e0bca2c4dc5ed44bed8e0cbce1238f4023089a81bf62b04a31539de9c8a0c867b791c18a34abcc721c0e90a1604e8d2855e78673367ce463307f6036850b81e0aadc9f81d8c9ad6e3b81d7850d4a46bd6772e651e03581a7a55568c9d63c1b1e8ca947174d68afcae6401b75dc7397be3c0dd76ad1827424f9ea9de8ac99803641535dae3eb57ee1eb1e2c193bf66addd488da55055bcf93172b1966364db4412e44bde64c50f7692a38d1536683af14fa057569cc7ef11f614a58995ade1b12f979005a53bee0769847deceab4f58f3d062767f7ac0511a04fea349e7688ab26425cfc369127325ab7fd24d73c4a8b23f099892b3bf52f35f0d775ba753098611ce35c56873a7455737a42672747bcfa99da8c8d0f06d8754d2babe3eb303cb06ce69e4046541ce63b6c28ded5a6fbf729200060132e9b01d8646b512356c92d8a642de0f62d169b195cc2e7ea2efebe550b1cddb5df7668a0d1f94a2f5cdf7897e46cb03b958ad906393c7d252b58493ce6c7d813c8fdfd5ef577a9e778c64d48c21b130c4f6a57985b8122f3cb69de015081cd3e247de65643f9d863fac6935cffb2ca23083d4fca9b8707679d9d4c3cb34d05d092334b1740ccf4b2aeae724902ad4fb4bceaa642b3ffdd5b564d7bfb725fae91f5c7ab9583abc2d80afd2272b14ec6fafbc51e35bebecce63bd95fa3f39e4e689034c47628345c4d8e30a73c36d87d289b6ec715a2769efacf39c9f19eefdb311b9800377c5c61aadeac78dba6946da4e7914415eced875b66f10b96ec98f64e58320bff09e8ac4f5969608171dbd4ab5883548a289b52d55aa82da62ba3d9fa59734d8f810bc1a40a0e3b2e116873116fdd8bad95471f16a14054fe247637febb6081d00e7ded3019ec03f76ebef5401a27f380fbe8a55bb4a0014f61276d9aa5626a34e7fe5a129a75ff130ccfd78cfbde65416882ce9ae9da57a3a04c67d22d3a12e811b2578fce5f6a64a09a86782bc15e5872e546dcca46bd9ccce5b787d17c70d6b075bf4c789ca184dedad3485a0526a61d280dcecd984fcb7ec663f8f491e91253675877987433e11b9cfcb5940a1357b55fcc5f977fc898f72e8199ed5d8d2554d54a55d6b4b3726e7fd7093996e271e218533f3b31257aa989f5efbdb12dce02d18a5188ec32c3c1c91845a17011ffede3f060f19d59b6565a352e90c5ca30eb252c6fb0cc845833e553f0bbe1822b5d6b26eb817608799b5b77c2cce27beb151060261f34ea73db94bb2fee11864aff76da5b10b17aba6634ae655dd396fd7e7155013fe3c6f840ab1ba9b25489c67f6bd5967a62134b238c325986c87aa005396c54af6357ca88baf5a03e353c8b0175386157d9221833585921d581f1fd3f5d9932b24d41895a5c35be91b35643e7091a7585ede900853aebc260b1fbe8b72d527390ddf46472f7c8d5844355ffc0b731dd5e584e13d65f2347131ba8a5161a6749458c1899b60cf9f9f8810889fc23b5d0fe2c843fb5675cfac6dc8ec8125d1c582186eac82d5e0b3f6df2a83eb2fc23c4c3f873267deb5ea86dcf1e3c5bb39df23f06560609b9fbd89511281fc7d69fb19764c41168a539b44f0603a52577c266fd601c0cc849b067746b4da78d280385ef80a3a553dabe4b137c50152f4e1de456d43e015e26f7d76e612b7be29b9f8e15c6a73653f5f7dc28aa245e64f076130bad58bbbd3b024c97e267a96365f019846d965c037116c74ef18415ed5500f12a1816997895f3bbe7b0fa1eb12a565255a64d5eceb2272027e45ecb0a1a8b5dc99128c002058c569076371703e1f10368c6cfb830ae8bc688eb143da7a1780dbe29cc4c83743b25d570f5d8609df46c807edbbc96358cde9ed0d9b8936271b3ade28001a6bfa6ff7ffeb4e6a6687e803b5e8f67b82e6dff215ee4f2caee571b8febeeff368a89539d145842526fef110174ba3a2d9f1b4a348dd25dd964efcf88847d3eef1c18fde81ce84b80cca8588bc1fbf9c9f34059808d3eef3fc2cef7636f7a76c533933b79a9dd2ae2791d1ab5e1e715837a766d9df1947d5986c21808eac758461596870b744a6a2862f7f5bff0c7a3460875b3b326e45fcec1cdad1cb4b99085748d69a62320f7e6c53c061a55ae20368a4e6cfc126cf2c0b410af545f169fcc741cfcec4947f577a1bfad47b5080ecb41760b5056a9bd10a10ca43677de0e06bf36b0d2bf7ca605b9217240871ed1ce43bbd07a7e77d1901dde15e173ca344ee8078ff115e954a2fa660bd4af61ac32256fa7d62af17a4f95adbdcd09b933fa47828d91a6b710a5c67195797bc789647db90107fb527b8ccbc1200bac302dc47a366d170314f5fd7b60535130d76832ab7a06ebcd83d064c1a1c5a7386b0945d39f8ddfc3503415ea22514a0ebd0ad74e3ece5508d83355cba944b12b669d6355a6143921533190eaeda0337a1b89ff9db0cae62a239025bef078137362db71a9eaa6d0a26fcda7e80a9b8fa4a1799819f0489b899cba4048c999fda79e0bb046d7e765d9abb445279efd13d47fde8b731a944af92da8d8965bc4e90d9da8ba0a9c385357e16788cfb663573d9d34810c754e86e3293a868461b241d7872911e676e73136557db1261573962221589850fa5ce049d2356e6630a142427a2cb2efecfed09445edd112323f0071f63f6a42035a9248efe155aae056db25b8004204d530d27027c34eb6937a5a05bd2096eeb1fa266a0325cbc7ac03270b751013b53429aa2d8da2de8c12e7d4996b5f1dd365e270285d2f3a65bdeea9346d960ecf015215a88f465a66417d12d95be2c0bb3d8e15163e0dc66028a30f1f5506558ef534ab28cf6dc2b7d12f49b0e319995efbde71978ef7121df58bbd16be908298b0abb6197bec22c598413283e7893f0fa7398369b83d30b29560ad09a0a24e029d0c22a7cc6fe53d84a7b2104681df35cf4c3ffd90b14cacb2ebbb673ddbbba291feae96498b30d024398676107d3efdce47353a544c9d885cd5cae19588a4b9e29c700c539bbcc119926e3d1adcbf4e8fa3dfb0d5c2686a3fb4ae7b4b278fd94db5efcad7f280488702d9f7675aafd248960806df3e8f95325298968a8e7bf54cd2797decb5bbea7bd3cbe34081c0b65ca9a4d0d08d86ff39f97751471e09f25b486dddfb2975edd51ec77d949862dd4a658ec974b8a5a381ca5b7314b677c4491e07b43a5a9964d95e1a29fc1d87cbf1a64296f4cbd8128ab13dcb60dfa8593a575140f5a9fa6a3708052b3bbed4b129aea88e66b72e583f1fc285ed6f835df0d4751c5d5f004ee0c65203ba9fbd8ff70fc255e497748f24c476d18c94089bcab8ae3f933563c6682e5452da8f760943c7efa631b2980e6589ad4ee83c88fe0b2ed9900571c3df6bac839df1a6d9d281a5ff627d41ada13b867735aa7377386182b1b113f237c14b6d806079d2f7c234a7cb585e12a5b16f7064fccff99386d58f91d9c235eef730660a02ad37dcfb0352b1da76d739ba7bd4982cbb8be48494acab84451bb39bc59521c9873ec9f53cedb527332c47afd919c94d777e603b405e8563248c9ca72f3b2fb52f08a876778d18e7d179f6437b64f79ca01f6be06d7b115b8d3c3320c2dc0ac59c860aca8c92407ca2239998613f44945bbe2c6a0ef0890ceb27467d1ab805ec960e6e58e49678a0841823632f62c58f540f19e82bf719e12bdc63db5363fc0749877539865a05786106b58dcf50bbdd015893b043b21bc1f24ef9eee5f2487d986db13c143d3366260ade522ab824fe82218daafe5e42c50ef740cb9ac570a13cb55fc30c9017228f2c60680f75ca89affdd376ce70ca5223078a47da0aab9bd18333c14aa021e6c088609d7a2460744006f149b648116152e7c814a5730bedeb6ebb59fe395e6b15d379909f197c4703dff34f8915b26a36e30088cb227fc1df108560fce419f462181e468b5d94251fe1373e0dd004fad02d690f8f28bd972e85ec49485cfcdba717db66e6a343044a2453ceb6403fa0795485f8014225d283da4c3aa6fcbbbdd6b04c30c96fce04579578882a44fe7b781285daa11ee71239c1b69791b8fe45868fe6e85341b7ba4292d6da1ae079875b33d7aa2becb02327a17bf2d09f6a42e9681df41890f17de5cf4752880660a35dab912d10480f20f92589d52ef4780ec28920d11de0de06bf63d61829a3d0894f9a32f911ac79041ef51fbcdf3f5b03f6a0b16f0b9e47444b82efa7c012e3bb9324c8649053433306d0640ae953737336cebe9b854186f9dbfda861fd7eedd76fc24634acb9cb925302d0bfd3751b7b6bab36ebe2ea80cb7c5359a505689ed5642118ce93cd783c80a2cf050f57d5acb62853367ac606fc1b8a01abb7b29e49fdb190e2b586e249639889bcf5ad23161f278086f244d0db0f9ff520e6858b68f5bfa0f5ca700096277ddfbcd14a4cf4358c0f042b1eb34bca6353a1383f5d310176262ba45512ea4f172f321efe971c391fa9e9876a250ab653fd6d5a8b046028018ae043da256b78eb26d86e6e89873476ae24dad26ec1d39da44ffdcd93d8c5c1fe0796bd46dbe9b01be0aeae841662c10ea28cbfbef76f0197cee86dd4c7dae3ba46d5c2884ae977c456681d5d9859d854a3e0708bc35fd8415c22ef4cb889bf0e66da72ebc635b71f5df04db801c76564d2fbd946ac17e27fbf995994ad26c3d577ac86b3b19f855d87bf3f0f7c3fd574b89901732dfacf2e13960975176f7418893b31747d96bd981e9b1463590bd35b51b7a2eacf1a7733f76a7d4b02fa2afc5925f5483f82cb529682f785c21cb951d5678a7d09f99c0e2bd98948b79456e4e0b5d9de80943f1df0c639ab8acd6c99a6297cc5a53a829561a6615c6c44b24a30a75174aac207a35fc429e1a415b0813857ed85ab49db6e523cdf9b122c2442a4fc2b4fc21685c9c4a5de8f36c0b405e34ddee91832cc6567d458ab76b4650020d270a65a562e75e06a3d8f1313b4d1d4d29d597bae9eee653bf8e31b82f9cad2d0848ed5daff73d99472c7e71d2f1be60bca4ee9d315196c096cd8260192597c8ce3cefa9cb6f129dcfbcbd65b5b0e19df5a1e81bdd423a2e26d11ca4f7aee99104530c1c8e3b1150669f16acf415559a38cb27eb6d637c78b1c90ddb5eef14ab9b2ff14ddababd7a349b7f6f5b6caa3a28c45835c9beaf8f836848f579598ae80ff9af739cdd800c5a3d7675c780ad66b2c449824d8ccf55a7669fb720604eeab427a0308122c3039ef9be6df46e34b719aca7a001b884e3f7b1d1579176aa4d365a25b6f3d420190e3cd9babefd8382b6bb12b4c38b981b08b5a139cd46e9382c01f361abdac7fb16b87e2c61f3241191028c770c3e3be939c8825c63bc4e523b389d1ee28189b5a2c11ccbe0aa7d3c2b47443028c89f3d4bc1bcac39eb61d91e75e0d372b9f27c8dc622a86bc0cc0a9c9dafa5e2bc67834a75805f3bf1571fbc5528e4c69f3e64ed27c1f2d93f43215ea0d4213bf2809391b8b9201f36ee2a275dc7c44d0d83de3b82490b65d2b399a7348133c4df7ef40e6dcc114d040f65de00a70cc0c9fe615c6fa90a2fad6dbcb39370af043345ebd9d8af094c831f341fa5f7658c2c1ea062948fef5015696eddd181d87dea49ac8bbcc2c7a1ea01ba1d04239153ca401bdc27aa87093a78b2020109d7cbdd39f279a7d67b14a5822fae28854323a2620033a5fcf9817a0d45eca7e2d120a113fd5ccd9fc0011e89398767f7f6b7d01d5a670b4fcb9c5648991817a62edf43d6964de204541a8b5573d7cc10bacaa1bb835f87d12985ad5fc6ac4834ce76c76e5dc34cc322c4220d9d65ca77051e7c1d4edcfb4595db16c641fa445401c72a24e36567631ea02e4875b7f9ac2ddb4712b79d1468c7ed2384ad5d90ea0825f817e42413de33eb30861ac92e19b1aada118beaddb179399008137f28a8f3df1509ab12d6f6ce7872521f5cfe4738722fdb410b3824f139ccf8ea63817d154b1e497a70e6f55b75e4decb73d10ddb0c57d7e0f74b3f4fb41ee57c2da1e4399c2577f28b7e911db08e877c366f231114cc9553c7f3cda2b75895fb60ce9eef4e0c25d253f689dff5e4f9cdc744c301b6f13e9fa9e30bdba8abc203c05879ff01bf8c3a3aa742ea5fffabbd6d165e50bc412bf5e18c60bbdd4fdb276f3bbfef246cfe36612ad61fcbd4498c18321a1ee12e6dee902c737999f9c6b3919f52125cc77e237a788640f262ab76d5c9db9ceb2d988c261dca63455658293b10842ee0b30cc1f928215e0308cf258ee39ba666b16319d0e21df8b9ed91be2e2feb9e117682cdc9edbc20b0b3b9ddaa93ec7a2d8f752ebaae57b7d74037c5cd64cb04746a38c0ea3ba3241319360e585c257fb593c123a653e6e0420fc540bcc0f072de6ae1a31b57dd5acf0b8607e04d21b727c93623e5227202a7c6432b308f74266dd1e1447a2d637a5259fdcd1d8729f341a7dc383b4c8a6ae0e78c34a55793043d2dddbbca1d161c300a8a7cb760298541c90a1c5a890b5257369b8fea0b4d14dc0848e644953cfd5196494e4c1b422f8e188314f2fa903fe93393193e9b76bb71c0caddba43d8586d72657a2d717e4b2c3a0e8bc33a6f8d89b871244eb04e4ecee26f9b71ae7d5c44f828083b0c1920d37cb5d7db86be3b18ef16653863762939cd1ab5c5e0cd1c495ebd0a3037725470d526bf12d21c2fbf4946ec224619c79906e8c87f0699be8a3621ae718aa232e26e43bf7cace7918ce34577a5e27261a4a6bdbbb8ac21a8a40b0c9682a0e0a37b7fc43306820b0db5fb5bbeed529102187fc3e5019185588d1ca6222a5e2b60faa2a93857bd7224e1e1c3c86128fd16993aaacaa34d8679e194ac0da6bf617144a2b1e844e7f6f8884df05ce1624ef1fe5d0895265bef2de61d637a74f78f6e3402923c9363656b7b91131dfc8f8f7360d14e01a25869f0f15a9355c07bba82e4969f5643519611d99ee9566138e4610712d55f22ad36c7acdc9e32b4e1e9b1ee9bd14ca122db05938840b8f47a5fd25a26072e2055494eb4ba916473e973c1efcef8dd7f0f04711b7e968d59d144fc04b9c6d8f951ab973441a6086ab6ab896f4d282793fb14c74322c7ff1c4c1c6d0544b7457635a6fe2632644344b1fabe12633b2f7d8fcb6e0929a05aeb2749fbc99671283c47c0eabe152fb6215bbc6e03d61b1ae955838a8b7b3bfa883a04f5156982ba433dcb21311fe9027a5b2e133cce16ac15786d2d44b5e4979cf2a8667d2976ea386eef4c85bcebc39b9d2dd216e8b4f7c68b7e0a5d7ff570770a20cc605e66470d995ef96f1e0acf6136716cd8a26b3b725db3ff48364f19cb2c640600d00d0d728d8fdac7b8a205ba8109318eb5519ee03848e79a1c33898555b298aa62d6da66188506638918b46926a99ebfccb649cd6bea1eb87fa6c1999a70937f51c4a2d485c6aa2bb5f98b1c8ae558e82287b9f34ad70954f8dbf3a9736604c43bc3062f8707fe88483b0865bd6df65f7e085593d5046554ee465a10ddb0c0b651e2b1031d56f5130e100e4544809a093026b88277e4e1f1e3aa40997871ced6e6eca44f183ee09ca679e4fb0796af62cec931c553ea788e75fd37ced477f9a58e54410c513fa79a4b5552b049fffd7d261d90a16f253a0782ec75219d1f7492ac2a242fc04c0216e6c804f8fd8f761a70eb747e1545313706136b085fc72dff1714ced89ba14a65a34168dc0e997ea8718ff304a962cbe8c7df8b759d76eb0f18155e49822d21d91dc8913c323a824da271f8c57d44a27be3f04991e2a47e5c8da2f0d71610be89dd4459b9a9f5cd4bb5ebf730a72617613667984d9a0e0ccf81c803e46425899b2fffe9158f9e3d7c36c2c42938d9433f764ed1f6d9c7d69e9ebf854db44c398c00b2612cdf19927de8bf45df70a5d4307e3a628a88a09a79d4e03742f89363d891b241383887b4447d0cf3e0eebe4a9143cd6873abfe2993b1f4d1b3b3a18aae4db4465385500ebef246adf17f1d9196f78749dba2b12deecc5c957e022f5b9dfcc3536bfd306a1a70e50f7bab1db22e98c8705328fedf3686938eccea04e7db4abfbdd939d56c0c357612ee3536356c0f5bbf1d428fb31c7b06a75c5b4fd5b020e6d9894b50eaf1df723131712344888adb5a0b4c07d23f6e853c88b51fcd56beefa721557256bd9978caca155db7f9c325f9564479c39b589431ee7d5da402e79ee6e04061c62c1f369176f6b35174b65a978ae3a4e8a25032d987dbaa5ca048734cc1da5f875f72db399df6e6a300db183e96ade56ecae019c4559c8f7c94899178c98e5071aece48a3e9f5a795159b2dfd0a1aa784b263ae5d352a890b7c2c6887d76f0364e2028d4f9046408e76efd6b5092590d39ec343facb16ccfc0000bfe7bebca2575b1f02c8a0aaaf5fb0bb13ac3246e245ace99f2186160a12792fabc0b390cb4c32f87c46f741c8fc940c4aa7843d650f90d422504bf97dd4825e533f646c6b77c4f004b7036b5480b587a913f05bb8ee470f0ab148727835b1659d8c046fd24276c95d30158424914e75d9b01fe52b34d5f76b2a1c98bc5236945fb46724e02851d46aca237fbf047f3055d6331dccab91c0bc20be2260086552fa7c958840843a3c38acfb24f9dcb506ee3cfc67f3125954d9090c1fcd629d1c3fc557b7d6ab33b9369ebc8b5b43f9e23673f688ffe50e6c3dfb12ac58aac60590991c9ce68502873fbdea04455459f6ee721570a87f87ae03a9f97f6e457ce1447358d8e3453632618701d62885ec4f8dd201e26405499dd7f317a8d2456940348827a08950d548b5c0b24cf494eb76dd0a51e3f5c6fda9f8f6545e3a6d1009775d78720f9b20ddded15c162f226873162a5076bc0cfc151e655e29d100b7a0d32c9c783375d1c6af0f39b51d927442386cc2c26337cbe5f4a34bfbb65b544b50b4947f72d839e6255a95fc859d6f1e34bb63dbfae1469be613472f22cbc703468d8552b9c691c829bdaf0d2e7e4093e940398f9ab8837baad8d9e4c710e4584685df01ea15ed9ed95ba9f6420e5766363e86ab8dfd1626eb1c7d2dba91d360821da99977be4642488972ec61fce7d754dd45a4d7fd9fc3498a6b7e0b7235c7843dfe8980bbd3a52c8868c1832f62d9e25a40986d061d7ca9d8fe29c7f234e3448fa315d749a09cf29556060307b970e5e080e90dcdbec4ecab1a7868cc5bad1037c07e17f961f3cb5641405f3d3c1439596a243f83b4f8576019712320ccc40581bf02a0788e3ad9cf4c44113d47f587e96061c61c06be84f6e35711d105f0c46b31f036356f6abd329c4e1e74f7d97ae6e4e0879be998a85257b6de6277d9c18980208e83d9bf266c9909a26f3e19941e728fe4ab0866d103644fd86b203029271361ea97fa93079ff227861aceb55e68d7454d1dc25aa8a7c780590a04756c840d13dfbde1d52d7df47d4146ed8d530907fcb13f6071e5c18acfd0a694938bca6cb081b7dbef2a646aaf55d4da9f3d42996a90655cb48144a4dd2789d75692530dc02a2635c26d428f2c6b66779bd9facebcd73274ebf333b8de26609bedad5739e73ed68798cf9882ebcb12902bed483bd905b92b3df495e708a0914a96840761da66379d85105f29e14e47a3e5273cc23ef5053077ecc513363379d2e6cb3aed80098589032cab7c20eeaaeb7bdae7e3a1c7a704cda4568138ce1ed94e944eb4ea72120ec91117338d0c86116db36f3807a70bcd98de03cc46ae9dd0c5fd3b26d78fed5a8b53c0c8f9566f921175fd193ec521c9914be68f2864715a572cd6b25948fb8dfbd75f4315967d5a6a707f1ceb81b98972e615d3e81b08ad5ef1838af673af9b6d63dccc226eda6d0b6fc1fd104f62b80662ff157999d7508f625fa2408b6f9ab4deb410ebef73750dca3e63d5045300d9b52845afc42f064bd78ce574ba9b4fce22aa3158bd058772ac2eadbf9fc77bfdf85dc745b529e8baf248a0bc1963f8fad78471116b9ac11679a326e92d87e03e316c90d15d2f9e3921e0bb6ab4b7ce91f12e7cdcd9dd141b8ef160e5db1faa47525a77a038558abd98757ac5f6313afb4f4c84b2c768e456f6de4a78b5b25f90926352fbfe83037f81b31251249fe52860f3cfa7ae1b20c73acb015821840dde2ff2de0470450b11aed7c1f84d4020147f8b9aeed4a37909bbb63508e533f939f23c2f5952ba6734745c51b837b72a398a4eb8dd6b4ee7648ecb55799cee7b6385ecb0b2e36b9518439ba550b257230885b83f72312e53afce7e7670b26342e4eb0a6286c0693c54abceaed1811418a9a289b4b09b65e84b3149cada8114e830c0403884ffb74c515bac074e4a981c61447ac99368dbe9cab8830911ba9cbb64633f444716f401aa17f599174581538918e88cd255765142a49412841deb528f44a64acb372498bd66f979d9189fa0f0e70e305015dcd3c9aea085cb0f1d79a63e92eb39b2dd2fb1b9ee9aec0213f3a4a848f3450e4a6d902f5435e234588ce4a7d2c9dbf60de27227ff02ee4ca19314d72674fb06c00ac7e5d15cd05fb35e973f0974a4718ea9b0d35bb4064a6280b654a15a7188c5d5a5998ba1f031a3baecdf7b608169c0ede84d91c9a5f73b0176e5ab2958169ccc6022e0c43481bb8e6217ddb1bd9ba8e8446aebed6afd163dbb6873d81afcb74b034085afec0d626b02fb10839bdf2279acec3d8f3f7c47fcc3ddc8cf1f08de8c8e120d9c06648acbd1ba6824ff74419776b73b05d797a0bc43d77eb75029112380ff4e87e543863ae29065e40c1b2b1b2d62a409a299ee5868036835627dacf4f9f7c84fd2f6707d823eebca0e8ceb2bd258038b137aaf43e0e937be762be5d3a6545e439a45090d397c8d4a068286836780536ad99cb138cbdc6fe5e9db3bd405ae6eb86084be3a49c8af88682dd52a5fa8f7fdad76cc0b452b3c37b27dc3cf66c57a4ae59a3687950ad3f1fdab3002fb2281365348cfcd9306583eb79a9555e8750dc95260a1b277c566babda92cf2b3ef177854fcc3dbac88da1536eb3c6d6fa3e336ed3a7a6da006901c845619fa00596af8e54bc1980aec4aa97bc98266c248bc038013020fdd128db0b9425e8f7e3165f5b9190fbf2dae5e317485d9d103695cd209d5d247ac7e1122bee4538b5b74790277ecfd2051e43a34b7c5440d1198a1875a0501eb3f519ac2f17553c5e99a9c83412200a519251d4d4ed1c84baecf7d8cfbfe4a7e50d789cc034a21043fc13fc3150314bd7a7801bf9c31aa00b7cbd9d0fa4ef0636504f0dab63316d9b9a1f4703b3be23170549fa3a29ec02f418d687589f77939fe068e41abce460fe118b04fd418f28edba4fe70d8fd24b38078a5c47ea236d73610ea116666a49acec52fbd810bb99af1dc5bfc4eaa634dbf2178151acc453f93567f015f2a25002eebc1f778da195b7e9cf683db1a85da98a99d98614f2c25b2237d4393f5799d06cd36254d3f22a53df5133cb1f43836991d3f7529cd260df88bc47e4172ab045b66e081940dff797b28c632898140425c3f81a60236b82522c0da7f0e0ec7a8ccdc73cee175d3b4543293135c31c0e0c926c7506920cea1d0a1eed64ede7572b8c77ade1d95bce10dfa69fab7fb5ba64055800593465dd4c26717e37a66089f550202a0530157e5274f3d3fdae187444f5ed641c608643eee1b361d78c9ed8070a85150c80368a84b14c9dba5b20a9059f1a05a60de938ab914a093b2e741e891c02f638c73c730e60ef8cb5d9e5031fdaf6918750f192ea7a76e035eb05d7556ad64a77109c5e13bb3ac7627d99165567dfbcfb7fdc7bd847ee7dedd7864353f8fdb7a9939867d9a5ff4d9ec9727fd432d8abb43628519cdb6fa49ce49064a421a89bc924cf6286fdec8bea6b5a7b8eb606f584d44c7194d5424f98576dc4b5ce7dccf65509727ed52f8d7210095252403023473af96eb5892cd3881f2e1f4dedd4c9b0755b70776aedead14e2606bbc1ea3c300984ebf27ca69f36beb03c0fe3c13d9df79791d828d52260103be72b832ba235bf1165250b608e2cbda0c9c8d3c96534802534080b2a4ddd5be4f269f37c4cde9d43dbb42fef8d507cce8b8e2eed799c72a2fb836f4160b5e5729192f088ab8557d51b58940065ceb4b48fec1873cc4b178166d15d441f45625f45af62c1d13863feaa8afc320bb36af0d151165c122a47d4306d075b344974835363818009d86c81e90345cf4b7d81786c40318a371fbedcfc325cf0e853fdfe6e91e99c07dc9430c04e6069b9e2efeed0eccfb161ca85d96298ae6dd67c395b78668d5818f567f2039111a7a0c52a21d84ef72dedbabe83f811184a95976b1a90dba5aca60d0313c546ea97d7b2e40b34ae88ccf4f45e28f05f16732d7a0d62dfd86e40a51de7f2b9b66ed780476cebeca547c8eb71ac20a245b1bf7445e64bce260b8871ead95f708974c3a93dcc4ee979975daa399055108854c1c3fe5a07046d13169c084b2e4e81821ff73d5d20e7556f40fa152226a6da48d5f86cfcf1a5520451c3c1e534db654b1a58adea167ffeea90023cc2b0b01710323177059e1c610154cb234d5ea165dcaebdc2fd9d9b66413b8a6848ea11a673f207ad76bd3ab5399dc0ed58a3ac0ac9d3c53781c36415d54b452a4a78e1eaa7046d062e3105541fec8819439da85ce53aa683aaed543704aa8210b26c1e6c8f1d1814595e3bce2d98a36089bd722110bcefd651937892ffd36dbc34f1284eb32c49a1c134ad460c19e01b3f3f2d40f664d90020ae8c546bba9ccf5500171ad07a55f9fd299d7518f00c0017f6c25d4aa01615fc7eec160ac2cc1ecd38bb0324e54cb466c3df4d56508f5239e1b392a50a8c20946a6db9220088d77c494406296921e3ac9997c34f15fc5c39f2529eb2bf6b41b0415e90cc9d7e8a83256cc59b4607be49eb76a7dc07c1ce3d0512888705c9ea145079085273f2236d5bb151dbe7d1bda2ed607fe51847ba09ea6431dcae727ec1ef0a201b8df9e148885e1308425d5c7fde473e90b4382de8bf5e215c3be357f8a2356348bab00a2b629dee017ecbd76d7ebd45fe4df2884e34ecbb86ea371fca29496fd05327c3b985a069a24a32380ac9e4b131ff48d58b1c528b941dc6290cbb41ab4aad2a55e5e03a63813c2113c0740029529eca008d906dcda1094571f5d2cd31ed79e9bca8e815e206b3ff3a7b4b82b0d30775e0b77260a42b1ccd0a58a91f0378df95cfc7f8dcabcabd2d7ae3b603e9061c1e126d00fde94b4b625e953b6b781da167c71a6a7ce0792e7de070863deaff94d74a7ab9dd2e3d39519a6c5c864036f7bcd4663fecd163d78a72a74d2838b6949eb7ba85dacb2eecbff1af2792047b036f77d488e5256ea16252257bdbdc1ca3d18d01c5de00ae8ce7d1c4cf103b2bd7a31dd52fa92d945da38eb9070d6f11aa9cb8bfaebe133cdd7a72f94f0fa067da0481dcc86e6eabe115cdfd8fe2e2c63c3cb3c0f75a4655eacfd337ddc82f770442d18ef9cf3993a9d1830217f50917153de7576917c533247a6725a0cb7447ebb0cf7516e19781a8ea8d1854349db9d3bc4e4931a1fa4fc075186dfc37ac10db1d3feb6298c1fabc8d897e1c806e38981c3fe7f252371b0e92efc61b2e787279f4a8138e7c6527ffc4092de912378ae692e9941d0b6d14354dd25900912f9a907dd970db649dd965c959e60f4d174ab0aeb7bc016735e8b847e231bdd240042aca16731926f537fe388b6496967b750a94cf514c9296805607b5f44451bc622486ff62e7af5c0d243503ff31d1c753d9fa3adecc8f555445298aa10a877250455c43d412719aade5a202076fdf080589a28cbeace5a63de77d188c1fb486d888528c5142f92e54f005239285cc4f8695c05ed32fddf0b0072bbf7642a1daff7246055cf35e077acb567b78c73094e583b9bac3ccea986305eb4322bcb923b93187bb3061fc00ee0b3d10702d7890c4eaf4e5f3da2143282dea0b12c3f043818914ae3ea98cb1322d73878b5d9920e87e6e00ae831a3b76484e0f69d35eb15b053c545464811f9c381337a80a603ca91ddebab47b72d6c80db4cf0646c579a136326be69bae46cb6ecbe23ae355cf7181981d8f7b8e16b3303aade349f44c005602c34878690d91fb7b4bbbaa75a91528fab18c3c8c1380c0c26a98dae3a2e441527e8c7e7df331a506eeaa2d299757e8b0cbba97a91f64e36e5a24cc83b00fa14b8a41e6fcbafab7eaa905c04c30755c551b2f0f1a24fb422bbd76090e212efa160103344200e83545a119cd7d2f53e8e5e6d89b1c75f12a44ddce7b9e9bb4710e30c853ac8d024be6f5b3d162941e2767c61a7f7ac5265d08a680b87f07bdb4d4e2037b99a6eeabef33b7ca6e7e6f4428f6814e294b7c08783525e1f632fc2a43dc057264bed23898edcc35493222a92dbe28da74c07c49fd725670ded86494fbce9476d3a594f4d96ccc3302a9e861933dcf46d03b7648bbe7bca88d9791b5a62694b34a9ee0d6c9d5600f0758e7e0a2721cadd048bf59d84bcd7ab33a5ffa8c1a237dbdf3481f03e6368b0c1a86ddf01b962520372bbb8e330a70cc684e84e1a438fc571ad9bfd297288135934aa4f1310849bc9c1658006eb1639d4b8ad2f858fdec1d5a4aa89259e591e2cc2948ff126caffa24bf41bf71b408da6b5107e1d65fcf37697d307ef3a3ce742e5d117e7bd4d095dfe8598213e0925bc9c8da414b09fd21451ad8f3531a756f15269614ea1ed728d3d37820c6e90578aa43f8f4cddeb9ce97b345840899a016591690a810c5773572f34b68d1f2f1538dd7c541508c990621d54dc2653bb4a7e9a1c637515538840f836d38333de92ff70997478fa7e4eb0b447ef04a96b2376d3e7319b49f8eb0b9c656bbaeb8a000104f3407e33c23030ee9d5d92a2aa7ff76f419ea3b1c9a8953b30a92759a8d0fd37b50e66c8f664792da1448f8d12d60e169382da435c038f10be62dd1c0430f18eb30af8c14512d3311a28d4252fd4b4dcc04b5da190a22c6eda7ad837e470b9d8ca3254f89a3e613ddc74c1865b5386efc7a8811e95163fc4c2a61b738bab3ce75e9450dfc38a52a59b89746cbb216cafa6faa0236db5b86747e7d9d3b6515bb19aee0a85bd4c2bfd76c1916e04db2e0d0382efd97d2ce79eb9370087b19faf4f269e7fdf64f6c30eabdef01c5a4f347dea6e28c48a55643ca36881ac5b283a9f85a56b203c3076e592779f91d3a00344a29eefcda31e2c66e38c6386367a5b43dfcb3978c891feda33c7907d7e4a501376e1786b9cc2ae1e0a60b9c0c979a0c1c96b9c126f647c3bb21abca0d4498c08a4544fc46ad1a33b17b105db91390689ecdbe0b9e6b5cc1d65a52a51d7d3a714c850c14c9329410c9efdbbd55cbc43e8568b01957d664de2a82c04caaf9e55ae5a34c533bc4da5c2a715409a900c7dbda950463131367613ddb0ae44c73625fc8f293e4002d5b704645c08098ead5a0fde1980d8addf0af0efa1288d569fa62d6b3f6227d1b4d628d1142f5eab4f7f1302bc97809b1b022e5976b32da3537a192938b8716307aac1972cc5b97019f4b7a9e5e650d0569ef4e7d12f8b123b1ff1d95d3d713a18dce009364a07d01b3b62a1f3ffe29130bee4f3b275dd27fd6860423629b181194a10ac44077eb2323e931cfdf64504a94a1e7683110ee8a21e67396d09b85237a51d824966527f9e3681ba9fa6db51e571d7d6a1d04d0f9bbd9d1036316a67ce225705018d1f0a9b9817b0d01dc092beb4e1290a7f12b22cc32697a75e17b9a93ff817a721ed0c696643353b80478a8d41e92b0dde35d7f2918f310e829305711683fc8c2119f0759e4bdca673ec93f4f0837aaab4c925925504fab72f35bc494a4d8bca2b2ee392af02422b2f80f174519fc83a06f99290cc7f1812a32511dc9fce9499947c92cc12c241923e62e98824b85ff3ea4979bd4362f87b7a89204598f51629ba1bc6d42354cb13c093d00c7da85b2523fee61b85573aa7d4a3145d518689ffcae21ff6470a8210cc6302bcce2459244e6f9173e1d9447726942749fb6bb13bd0d3479ac5f1e6d70e104990f2c9311046798f906aa215c002ae0b918af3db1bc591a937d5f485489a132fe9098bb77387906ae54d92d368ded939a069d0866279e0f7cd815062dc25c664c168ae4a11b9f1fedcaff8e3c37b6f7b075e737354074381a135291383a35047b1dcc7cf3b815c1b7ac187a4dcbf64fa5ef83c4fa2c0b4f5b7e024af32ab5d72e4e86a92acf858ff0aaf0fb8fc64fa12d0733c6fc530ad739df51038fceaa88e6e47fd80a357f501dc4ba5de4b6c0c573534a84e5aac913bdfd30e548c8009f06e7281d4b91b968852033af7b9e79b644b1532b9324b9788ab6ed930236c5bb2de248231f3046f490131ac34586b43a3c8e0a73cc756a816c79f3f10f12c23501947b29617fdc2a45c7f361d27966f73b1b3a1f50cb0052dcee33ce739cf2da8fbf9b7e5734c5a7fb8eea9f5a598ec6c207735409ca3401894f634fc787fbae311d4bdc8c99e8986f306883d7185a18e71b5166cb9b84cdaa3cb03231ac3053beb31ff3d87838e53ef698f9921a0ebe69c9b1526ba5940bdf21fd745f6f2653adc2d3c36340a6dd7c9ada045dfee2349e26a62b437e0c8f4fae2c7eb9eb577c20886d8a8fdbd02d245c1a1478f52e339e8ddd71fbc6d2cd1668a44e2c82b311720996b8da7111658a04f4b756d84418ef3e99713cb73d87bb4a14f69d5ef4a8b8cd6d0008164fa0a802c0679b26e155d801b40a2c77453dbdaa5a582d7c9d967e8a5706358c68c6da6c3e6746f0648401f896f1674797b0e14b354c27ea94fb12ad6d8494ecb0eca0eb5795b1920c618b429b4e95979174fd21685ead022e7ca4f87958c8a101b8b3a5e0a9d64393395b12dfdbe7e7503b62db0fa8813c4eaf1ffe5f190de55e83fc3a01b10a647d77cc49b07a258cc4e65d489451f849f6aeea5bde2aedd8e184cb9c778a3676dcf87d2bcbc19647bff28953c5d06fa73b860a16785238f9765c0fa958757b4791272cd8d1fae302b8dc47282b782d5c4c5ae75a9c02a3e911a79547df96d2c2306a2e347e83a8d8a126e6a3a21cc46e476a706af9c42a97b3f8fbcc0de5cdfb59bd6cecba11bbb93ec1bac6422b61277eccecb8b28b3d97f0051b89997c8c22a9f517d9fee5e16606614fda11940cf388c04daa1b302d3b0590a9bd0692d6139b77546d71454054197adf346bb94b4f2e85e502e3cd919bb43e8b424339e2af0e7070b035271783b1d7d48ecb6e63f3597d08a9d0677a44c29948d9cf9570e7fab1ab0d8262bb963d62e895c9bc8ea999f6f8770ed5ebf75a39dc0fe2eff88d75ae19690e0f0d52090f18bbfbe39697042cfb5293692f1e66d76aad38bd919b634ee20000000000000000000000000000000000000000000000000000000000000041187a86f715ea1475c76d354d556242c15f6b8f7c441cd862cc7c89d89a58bca02ac74c422eb01833db93e4b067e507da65080f93cf1815f4cf1dc0d53a2802230100000000000000000000000000000000000000000000000000000000000000","nonce":"0x190ed","to":"0x91a5d806ba73d0aa4bfa9b318126dde60582e92a","transactionIndex":"0x3a","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0x1f02737dec59ce5f29b7e11309a33cea7b370eddefa1a8007ed3fadfd1c6b23b","s":"0x106ff6938ec0a46da92e74768b54926633945ab503e2f2dcb69469b544907586","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x2d679b567db6187c0c8323fa982cfb88b74dbcc7","gas":"0x5c90c","gasPrice":"0x7d9f6fbd8","maxFeePerGas":"0xf49895cde","maxPriorityFeePerGas":"0xcfd6ae0","hash":"0x45499074aa521ac4151138f0aad969bcc2dfc1648d22ff8c42e51c74cb77414d","input":"0x003851f8368551cdf133121c7cf32ee5690000000057b378dadc7d07581359f7f74c422f52a4d750946601a4898820282002a14608e24ce8d2412c889246951210c5eea0206041c4025828362c2828a8a860418a28208a224185ef99505d5d85ddfdbfefbb9ffb3cd964ee997b4ff99d73cf2ddc5b063a034cd01e114eddde75618857bf21dae562932ea35bbe70493443b0d3bda8e6161bdf9a2e6e1a1d63845cef77f151ab3e17f6819ec2bc7388893dbbbf49e9f0c76d8f4e3a6d5cb9c46eff41ba97c0b3794c105b1a891908a7812b16b0d1d7780802e8c7621a188764369f6bba1bb0c95c54c4e3ecad237716b875175c0e27d3cb133dee5692910d9eb1ceb4679f65e7247f51ad58b4609f4c41fb91c48acf7b7d2a8532654b741723ce4f70eb38a521e9948c62bb5d73328e2aa5e7174426f5da2a4b1c6ef04eb66b2d4d1a48f0a20cd3d8b6bff8794b3975124f8d8149ff9c0b714da5e23b8ff418940c5dadad787c4298fc8486c94c44d45593bb8dcf6cbea18578904b0c99dd61f2b7653a1562b2cfdc3dc96fb15db01fd11710145c7d6b0957b60eb6fe886a047920fc64a496565e715ce7ac34a51a8fb632f62680c9de88bce189d008e29a23679115012cd789ae2fdf6cf5f4dac0cc4aa2e092b846d3d3b36974cc62244625394645ce73c1eaeec885e6dedb131a1d420a57f945698a9e7c57befcea025554af9a4cf6a432908061828ea86605e8ae7ef301f4c388060625fc5cded2d60f4bcdd4270b2cd05a133fe45e31ff72c0fb8465065d16678ff60a4f2ebf901d2403345bd6c4459c6c6b97bc038b3cbf9f0ffcbd7fe06fca91bff93e504946e2051d05e23e28dcf2d35dd2d1c2b16da8fffcf65ec0b789efb1afec71cf481e5b4476b7df95109b64e2c1950beb3cf3ec9a671ecbebbca59aade6ca79f06328bb06669266f9fe9a6673c50f5aced1aabd49947341ec338b82d9ae31d9fff59a0511ed938e016432f9c5a001a7d5d34b54a7d9a2419456e3bd0eaa860ff52e69806188aec7e559a70415ccb6653c885fa17efcfccdb680026e43c9ad0bd5b379d4da2127d429e74dcb29f3e45b3ead08e1e85c2be6505fac13244c6e528ea261329310d9b80b16bb7b562d9477dd95f6415de5bddec30bbcfab9cb9bbf88e71eb47c16df81087fa93c4b39b5e0e1cbad17cb1bbae73eca0e2c75f73e12e89371e79a4d42babb71e9da013fdf5ff1c20a45ac00a02bf1ee09b0efb1254f4862526e5abc8cafc311bb27abd9af1464ed617acf4c1647dc8765d530e4b40fed95768dcfb24a0782f32fa6cb29e165a5745f2b44a8875a96812b99a095edaf5afb1144335625ebac2a489d9133676bffed7d161671de594b27973fdebe55eef99d65ca83e607ce6ed86d32efc36bcb4bffeb2062a99371c93d2eea8a0f47519e0b40ef21ee8d9de5e8d8da61657072561ab1cf9a4b1d51da9926a2d934cb9be7a548af7a5835758b5164f249ef0f0b53051779089dab29284dc10c24a1ee298ff1faf3aee4072cfd4ca19520b23941a4852bf56dfdf0eb075ff435f87aa51ebedab52cc3ebcb4db9e133e57c2aeb10a836d1f5484807f73a9d411e059283bb45fc028cc8f919abb8ad2b008ff7a95ca576c0c04aa4f243f10d2fcbc425ce014f5fa6793335650b99beb722faab1bae677d9939a02d3de5e0ae5d5906ba024cd019c11f0d51c89e73b8cb3609f7faf3d127f969f3beddda1a756fef93ba1a4d33e4c3992957a9c304394b7760063250a5519ae83b304c00fdf0a21e2acdec12482b724df3ac5ea472c065a9e2b7c8de28454d72ced29d817ebf3126b6928c2cd7cabbf0cee395489744896faea55699fc7ec581ad49b782e1bb83bcf535468b91703bcda0c48fef1e919dbbf30aa45b162c92f0e01496643bab3d9020b8e471695769f04020557e8d0fd5812bf346deaabcc118c92dfa146a1b5fa6aaf5db5e6fd704d073d9c6d549004066b97db487f857716ddbe6cc6332f7ddba3ce2df47f0f25911e833adf8f345677a96c7223a3efc77a3f7a5d24e543525591d1fe4afde76a4a3fa883efb81fb9e324c5727a5d2e7c04013a249366aa8f8242373f9c0aa55a7366d364b0bb89ca1da127046eea1de1d39b917fa53d6acde40da381a71d342e3fdbb19254d7df86704eb979ebcb905762ad78a852bc948b97aba45e99ea4fd7e0340ba70097b4884abc81c20e99276eadce78b6b5f075620424b9a83af6457cf29d9fa52b6df6bb7054711e789cbfdfd1bccf3081ed96225866598328089398b84ac237a46aeb513780c62f5ba256d9de6392fb333debe7b2ebf00571be5b6dfb5298b68c0c4f8a281db7b5a819b91297d29e586e6e78dd685f9810e830dad91e79459de4f2cc484ed9cb3a49eebe9b51975dbedfa1699e3c2fb9cd6a7e9ecde33677fd57e07847bdfb9bef75f07bb9f55f73c695a97ac188df3280a5e92f0a4f1c4adaf8f53bbab4682a9dff482e9cfb43db9dcfcda1a466cbfae58cb09a97d8336c88b124bf1cfff8a603a78299b6e6a9ffec57be9c75d73fa3a86049c56f127d5f3a6645bec138e6f9ee98510eef2dfcaba7f894ce575889ed3c89b3c1caaa37f0901d49f7ef9d46c955095899a76feb44cab15d918d6f9e924e77ef594b9af6e7a2bcbd2190758bcdc73a2cb5f7e25e9afd87f893b653e7d81b294fd6e0bfca1f5565a21f725312a2108c950f275a4c0a698ebabc567bc1d0e7b1d74b027f1dd7187cb584fccf0206d997519a61060628e2255ba437bdffb3cba91b64c87899fb19b5790ebbe92b0d55d2b0eeceed76d6e91aa53c6ae2113e33e299d531c4fe70c696050e654c13387f0f5a9006691089779741473c7c7dacc4503fe93cbf7565bbbc4f715dade96b451a21f1e36920bdd10f42f48946f8ac56ab3afafd84800b2a9fb6d643cb17cde43a619d7ceaa915e6f5d209ce489a8b47bab1eeb9a93792c6dc3b7676dbacfbf3e3cf43e2d68c79522b62530e423f3ac341333908eea556442afd4a7035384916fe5ea257e3af6655cd75ed06f3498afbed542b0ad0491373609b66dc14b70fb96868a8962a2d7ee4d16ceebdbf334a8758b9721bc879a8ff8bf9f2fbcc6f63a9718a35d8b5df1d6a38bf1c6123b43566a079d88b94f3870ea1b9ad8d94d2db14bf12db3ba58dee361949713c756937de91dffcef9fabe2f9714589d51e3a72c7c896ce9b1fde893f5ea895975986ed8766eb7c7598bf7371fdab658e7bad069a9b52f9e9761ae014c4c15c25b1f98e3e0dc1d831cbaddb6f4d3a993f2257d61f345b3cec990f79d3edeb2f5ea94016bc4c4448c8f9815e81b7433017a719d074003d7307e3d627653f5cae3753642527b0a49b76bae643e4f0cb4fe00bc7b71202d6b1ed7329f6ec4ae64ae463ba7cae1375d3ae4b36741ed88e3dbfc48efbabebde8c1eedd487fb8a9f42b66808936bb6692353b37642efd113c62f14a711f42aa00a0947ef974dbf1c9455e3c1ef175af9cbd8bb729f5a7069685775d8ba8fc3552e37e8765680a58eed27db682227eb6f9f12e9378bed42d4292a77ca86c2eb79f2d289b557e5be66535a2f7bef852cfa2dea5d97c52fee7124916123d91d6d6911564a5c2be40110b99de71e9d9a6203de69a6f7b57e54fa4af76b82daad4021a3fdf1ed0cb5f2118646dbd38f2ff5c7a107967eff06986db9e2bb8996f3dd20b7905cb8f091bfa1f9ab52307226009181925c4cf729b64c39add884cc606b5cd0ffd92167e5871ca3b85b0eda19eecd1f3876fb98ea4b0ba1df9cb8b5e5a87dc74399170e34ec1a2b29df609aa1f86f24fac697de3bdd5eec694516c5c59c6160a30d90211319e50ed9384a8c8e316331b2a64f65a2f2f72b3793163870b4fbec75e81af7d4353aed284c926575a821938c34a619bc75358126deef5ca9fe7b0a5e69b6a7c19935505c73df8f2c06ceb3c09fabd79df5e67c5f209579df95d20ad0491ca79df3ae257b8ac3e95aa302f84de6fc847c8cb147bbef136f355a240aebf1d19e1db26bab8ad4c6cc6d6e27c92d0cc8420e0f9e9cd2bab9e9ef2196e07f04997abc6c1054e015c72da67366699fc045c21974f9d6433740a3abc30b8fea46456ddc7c3adf8ff00b85cfd8e8b6ea2ead437ba951d6d5c6c73322d29be2a35e63081d752933b7938c212b1289d2775222261d18633dad1e73e5f323ff2e0ecf36784f52179e6bdc31c5c19616886a139ad0c632e47d0d32d1ba2ac6c0807fae7bd766ec53b94fab2d2ff22a93483fbb6445e72d409a398840f476eb72f1a0ae8143b70fd04676db9411782f83dee540fbe6aa82664c779605bf155030785e13d8f0bbf86951c62ecaa7e25881f491ef1ffec48fc67ecfe2b9247df77673fee6cb0f66f64fb982379d6deb13a79a3f893488596938b1bd7369f9b99832c4c3c23b04cd34cec0dc5236c4105ede9dcad1943c5813cd18525daa70fcbacda879ad6675aa6bdd5174f0b9e9b6c53e67f7771408aabf9fefe79e92c5e8a6638ae8f1e38b621647d0d51aad753ad3b5a5dffd869f18bcb9a8f73c76b452f45c2f8ef6bbc39c9a479bd1c66bb74f6e3f555db06f55a1566364ace5a637cabb4b6b5f422305086e8542d0ebfcdfff1d1e7ac9d2deab374165267cd49df53bcfaecc76a6139337e9fd029079ca503816826e13fb54ca2dd757e147bf8cb7d793ab299cb920dfd306ab4654bb7661cd83814994236d8b50cd984ccb5d210aa54dc73415c96b92d6079d19ad0da60496d2b64f3d687b8cb154565602ec004b311f96d330f172d255bc6f0c550afe9acd9927abfc7bf67c37ca527edf3f61625f6dd9d32ffe64cd0b10cc433411bc558b67cf11bc09f8f4d7f44bcdf6395fde7ea9f70c80cd66aed764d8dd0ce88679b5caed8fc9c2ee8e32d2ef5a8ed9efd1e418feb40aacf7f19f16c9a1b35357f8bf8049ea7fb91c7ba47d670ec211f547ddfac75bc552f487263f859af5dea16f91baf2104b6dac8af88eefba672312ee7fb9f96886ca51ead0fda9213f8aafb5bc4a7840f65a01cc004a510ee5ad1dcfb0fb650b747acbfff26a26f568544c0d24186f8ee4b169615fdec32de53b6d3f2813ba5b7076edaff3224797d524bba359d90f417140cb222acc5ea65d83c0e7bab279f4bd8b9f44e0e7c907865bb83d3ededf3f539ee85c3420f10ebfb0725fd91f3cd2f3cb28d9ea9911fe136e6acee0bb05a7f18bcd4b5caa9784b19f63dc0c47623730f54eb3f52b2610e9cac31bf560d29def6b778e3a6512fec915eff7cb1c18981292bc892893d581a311046dd9c132b6eb9fe354075e0caece17f573454c035ec85e973988dedccbb7041ce2801733da9f782c8d85491ddce93a943b1b647b41d02902ccebdc3c82b7901adb91cbad9d18b7915866a1d9098e1c43de52e70477048d02ab565547ad68c33c7138d30fb165f1e5e76a837bd74a4af58fbcf4e34203d0a1b56be539f2ddeb3706d3f25e663ed919a97ff8abe62a76be6bc6fa6e61aaf2c8ec01768c7eadc9bee7ceae30c31dcba975d6b558ea932e22a71a5c728d2dce05067b1d4a27dcfe0438fefcca5cb4a95efe83e63a7dc247570449dbed353e7efbad6ad8c981a2b47de139ff5bab370c969e154b9e4f07f813a9310a9c021f52b22829622dba52fc47814892fca57ded9158663b293356decac92b391c56d7ab26702767e30b9726aa51bb16b79ead28fb4e00b815b66ed75006e443a9e41bb5ead6975bd3f8326cbb4de758e7907ef62dfdbaddcb7b1fd85b3785ab27cc0128d4a8f0306ce3c92cfe0f708ffdb335bf69c55348e88597f711e5d35efb6e05bfb9d266f107bf8497e8ef403b791e1c3f23d6f83f7c7f2c7bdc63776dd4f3ab24178e30daebb616cec37cecfc9756e2809a6d131c648f88c6eaae019a9c3579c6287d7681f8b5a24bedd607ed629eae12f9e2bdf55ca28a01ebfa2b20cf30460621e22b49241d178f5884481f3cb5e062f2b5a1f90e4a66da24bcd9328d9501e5f7369ef94ab5cc9c42494d60edc51fa6d5ff843a8fd5d5ff89743ad44ca92594df4fd090a213696a51aa7c3845fb4d89b7c8ce12a9e57ea61d3651b8868e7d8af070f78ec48dfba75613df791126e81aefbbeed1911f5ee74eca37da5f0c478e8d9d4c6433984cb8ffef678888c982b7fdbd36c736ccbc9c74f65d3b6659d3f23ea63f6614974a4d62507951089c22d885256b29d96a7f196cdde15515ef5b95bed572ef4bd1b712467c7bd67f96b1674378d8f87a07fe35443630157d419d038c7fbb0486d5ff6dc908db1319271d24b7d8ec8b0cd7fa16d812828368871866b96fbd5fbc63f5db2e85991c937e1e7fe1dd93972d9013b5e56e69661f40026461bc9fde8a9a46f19da2be783f9f07695579e22609fe4b6252ef1ddbc0bbbbf687d9e316570db32c1563487f09b5e0ef117862d5301762222e99860629271f05b8d38d06658ead0e686113b9aa67ba89cb26bdd3e8fa4d64244cb955fef548f69f9c5059c0de748799e245be3b9cac822e2927319db667124ee46639bc3b462dbd6ba6f375e94842e4e5dfde5d409975afdb02d03b52c27f33f98f745c0d9ca2d2d3405f2babb8231fb446e87f1f97b776e3e7a862db3fc7a05512b633f75ab70bd7244aefa306fe9c5975202ef0d360dc8b725b06b961c6257d02b4d1c8867d3a1817257a7c80adf3e15ac7108cfcca555bde972db3b1bcf38bb19b35839f0fad1166f936c23877501f220ff81b93c06ad91f798214b63bbfc9e9cb6af0f41548c3772da35590ca55d22b6decc5d1d7b3149fb5e71e09a40fcd997da77eed62a8d84d9aee853c68fcbdd79b210ce34a9ac25093a57572d557d47cfeeac5beaf9eaa9e0bc29c3065f59c6190a303903918dd88b9b4c7624db2ee1b48de9cd4d0ca875095650d5c8b5d903717d33bc5b1635e52a1d989c72e3d397e29367db456ffe6afa124496a6f331d5243f40dd192b1a2e230badcf97acb3130f5e2af588fda4bac803875788d425867687a67dfbbb7da5fefd9443c7b565b79d3dd9e669e76d59a797b3ed7a1e3a580b98da60adba5c7c71d0acfd5bde9ab5ccb69fefbf6915976653797ef913ed65260a5c6286eb11e962d9358e3bfa3c8fef1526293c72e4bbba4522a7e07a5f6da6012775d5b1d487283eb5ff89e5b024e4b918e98ca203a0eac6bb26a5be64bb478ba936f1b410c298db9260f6969ef4023125bdbc7db8b725cf7e4b69e0c0bb1733f3a4ce5f6b2eb374713fbfd4f488c5d1be324c038689a943154e1568a3dca5823234ce6c5c26ed2da769d0b1a78bdb4fa9a71d59a2af1241315f462ec378dd24472efc8d17937fe3e4bf89a2987f3201db5ebf41726d27cf6c26b167556dfbbd0c00589c8fc1827fab8ddf45f95f76efec6f8d3ffcb2bc9754f9ebf26c935f967f74e5fe0d7f18c35eb6fe9b112b9767b418c56d9ef750f7725ad45de079a5d42b2569d5857cbdba2f426619e57273da74b5cf5af951e76b3dfc6843519c8e6ed6dc7b8290fdb0b1f7efeae73a579a7cb26abf03c7e7e8bdce598e578a5a6f1dd57f77ec99edba37793aef1fee973cb8ae75cee284055e15d62e19178d3707522ae7e3ee9ae5d00e9d4d82d21e79578288e3068737f317a7c865c4ce6f38eedfa4502deb116b7735b897b77f8d41b7cdb7fd07973e715eb9995a40986da462fa51b9e682506f2559526a6bffbe8b6f6ebcbe97568679806162eeb1602d386302d65aadbb374419f9dd1e08dcad0467c5f3e96934cdf94fc11a04ff3158473a8a7e3db77f79c292f017ed42a28ab2ff7d58f7ae65fcba3cb1ee97e5ef3feafe1ad6a4dfcd1061fa5d171d2f21debb4a99b5ec48d57adee17b492faac44efb1bed06b538a30302078e9e17bbb15b25d3ba8a4e9255b1ebb3905ab17ccebefd6591d9feea87e3660fe17e57bf9f60cf0992dab3aacb619b0902162a967eaa4709aa8b0c65aef4bbb2a56fe2695b51f950ea8b5544e6fc4d5d81b96fdccbd55df29cbae1ac667efbf62b51c39c6b2ac9c839cf8f85c7869fe8162a5f6a3925e16cea1e6cba00bffb4863d8c73c355ead66692428b3a88f939c136818237f256159881265b37aadff156ec96f73aaa3b7f8783f2bed00065a914715bb57d12b5f65a5ddda5ea31c67d9b9e6d2aab2c0a425ba857c471fd552097a343a660972b6d4cefe7e3671d5d34d5da54312451d414f00d3944635dad335e7f56b00b30ab4c3751ac81adfd8c33e950db63ff44c85b901983717a35575561570ef166c747d3abfb8a112440aeed6cc8f575a3d285f3a6c1c163bf745b4d79172d13da49263bb2415b00f520d118b3767240e1dd82f7666b352989c7a6fc8e7dde6c5ee6bcfd79e9ea9bf4ba342865886f90c30317d88903d4fc501a706f9b7070c15ef543ed850963cab079ef72e75090761717efdca575316d58589c9fdf9424ac0cf070ef2cc885b83e49f0c1c82e91a1a00b3ac246599bcc4db1ebdbbd70d956efd071652cc5e2d14f79a0f78614f7fb472185af9cabfd138b7705b9f9bf6725e29faa5804f8838a6e5f5ca00385808eb2ab34363ab50b2f2f9f39233eddd4e463d18d6352ada86a61dfeffc4424a22b2986f9791436fd96cc3ed2bdaa399b32dfad71bdd1e36cedf2f6996f6c2b6ebab2bb2ea4bf11b2295f77d0bdd9e91295cdd23ea548e8b8839e7dfec6bb74976a68232ca8be3b478f1bf6ce7868db922b5bad6f87ab1e162b27256ed71560a54aa622cdc91785046ee2dbbb4c8f220a9334b8aa9d6bbf73fe724f69fc8a0bec2236e0d39769befbf3128e88811c39d55dfccdefed9a8fdcdaac78c8df18882b467cff80e2ce569edc0fad9247625883cf6577f16f08d9efc2ce1f8c3976b73e789773f6c32b6d1baeded08ee286e68f046148b66e61ea93da5735c56a5c956b076d067db079b76bce5add735a078b5db7ce3328c04c0c488205ee15f6f3b81b559f9bc83df308fb6b974bce453e894adb0df2061d8b3c3cf62ef9491bf8a095e1fcfaab174aede7b64f4e32c0d84737f9d559b16bb8b3acccf662b81b5a01d5f8da95ea91483152a4c93feb61db3f5d2aa8791009f43820bce2c1076edf0d56a31311c6c40b4bd9bd65764de5cf8e886805e3c6d64d22ef09f9d03fd5df9ffeca41dc983f7d403e190d873607781e2d756177c31b750dea08658fba0dd215e28a501096dfbe83e5b51e068a650ebbca797ea4508fd1a570e3cc5bfa6d46fce6e35580e9581150013bc8004a93d12b222cd5dfffaa8ecc66fa9694daf5e333b4ceb24d584bd67573d34f50c9e3234dc9860d0785084ff75b32920b2539af3f26d5363e70b94bbc71615c564d61d5df3527885b8dae59373333db7ecad439655a79c12b0c39c0f552cbafd6ef742e2e682ea274bcd9a5a7a7de6de1f7a1ec5561a341040e5e75ec0da406ab741f458c3a11bf714f23f730c19ca32d78af3c52b6d79a2600b698e2c0ac8dc7548b658bc4bc3f975bf2ebfdb916e214be74fcdd8c60737bbd2e5bf68de1e444c28562b70515cc9bbce2cbf911b7ac2aa18a20b1dbbe556bb8e6788921c34285766b410601ae9221fe28ffb48169cac5e277cef909925dbec8f8e3b6fa884553a865f085fd7f9992b75ca1674672eea28d37f8f61ea77b3a6f29e4f9aca9bfde138995ca6cf80c89089830e59940400420408207f23e221137b32640242260e64084b718070600caca5ff4b6d531cfee25b1408cba09000202d9d4102b86a5c887832c4c67a923dfe048f2701c05512009492009007c2494338191722090085277d675b400280c724006c1c7b08f19000a003ad85d7cc85e8c9fee92540028001d6834cd63b2f27dee7d7752142c002069e81fe1fe2860086130306011e065bc4f04ffe8931f014940a25a138517e4a83911463d07efe36469ce18cfff95b627ff23866db0bce7d8e62ae2233cdefc8d7e57ca04b5ac564291e7e72f2d5198990002fc20bc8c41eb5413a9671188f6749213c2acecf2ba4fefc319604601c0990897d0cd7954dc8afcd07b1812400b3824102f84c5c88a3ad7f2392008c2f04d06824006b3ba662328481002ad589046036a326e023b91033f0780ac3194f860032fde7ccfc8986507431589f108063550b8ed6f527cafb331dc4347f703821dd7263afffe522bfaffa29477768e4bdf7bc9a2da6fac9cde693d2e97734c8c49e0460e7b1582e1c97f13009c0ae181531604c4454eb2400eb43633d0e8570d224802f4a7cb40c3ffa4102b0092c0a367f1605bff608c59fb148fb5333615b0924007b974a02d88227908c7d4602d8ee4dfc66f3fa2b1060332190003675daf775b319fea16e4fd6a7fd1fc948dff144fc4e7ab6bb103b04d0febabd4900db65b4c14f130db22f1eb701ab893e888304b0759000f69b3f23421542fe5b3c3831f0e90c96939100763b12c05e3f5637034f2601ec3e7408004900fbfbb1c782a22480fd2a810a0120854186408a1348a63a93d3f1780864601910e000e12000ad9b918efe8777a64220489d1ae9886a2100c2388d7a3c0960ef82b8480087f908c6f4c54799639592a96427061e4b0238b80910c0c093008e252e440a04d2d0e72480439605f7db2e448a134590047068114800872609e0e770211227387062e089a36aa0400038a1148afde8d7185bcbcd61bfeec71d205388cd015286806f5323571c274725a6b231f01048152393008e06021443154bc713211c559082caf70665bc93047024bb1089e3dca1b17b54867196211c244f03d9c67f53a9d20c7c46faa862b9681812c0e91433fc9b7f5412c0298a828d138aa38d7cdf81274bd2c5b6e3c9716274f10c3c1ebf3d63d45c14c6248ec6e1c479820470e69100ce6c3a09e0dcc8aa8ccafadc881af00f6f9000ce9b6843c9acaf2f49006723cac31917229dee440238155d885467fc68830c16d10009e0ec44bf71b153e8681be5a3be49a450509872cd2590002e0508402bea71214220054451c1c53b12e22079aa104a168d922da790008e5e566df6685dec2e440a08e1625915c1040a038f166d22015ca1b123c534904a02b878d0bac6704ba5ca4ce89a0470654380241de2c290002e8a0b51924c134bc74f2ebf88864fae5417623a9e0a613054f1b1b74900d7232a09e0707021fa8e9984c262b60f65f60309e058ed4244c986d19e88e302fa0a371f951a4b02b8b44660c0c043f2342108a48890006e730204d0454900f77c02351eb46749c3bd281695a0d08518e78c2701dcc7509548d2c54800b753dc883620804865e9927b2b819e08da539d21900aa6e3215c2c98898770230ae526b810e3682cb2032c3212c0bd07924f14a38e484773c6132110c22460c004d486a8ba581a9b929b084d72130a04ca8c43874c4dc7d31868d5b1a86a784c09e4383696967824092480478204709f7621a2e562e8433502998a49c713c9140cca35cfc25812c0dde242cc444560a2ffcf1c43349504f0b813209674bcf35d881412c013381a6258bf47d4ca134b8030ac2f0cb4c1eb28c6d2f1a334d0309dc54b2e5a9433ca0b0dc2f0a4e3e92ce0f154a025e5a325a36fe1211c4d100617d004c5205bba20c44b171413172501bc00814ae561e06923020ea2af32475f1d799304f00ad0e974082401dcefd09c8481a7d047f4c1ab462001bcaaa3e414547ee2b8f742c354b1496ecb8ba78eb8683acad16a1a09e0d5436b2301bcc9bf0d1ba81bf2ac40dd7b02e5bcd9932b2ca48dfcc8c053a91812c05b4e192ba452d9e82234517112c0db88b2fb684c31635551a90e108e0a8ed51db3ccc14c5bd31017e41fbc0eb72e04b7cecf1b17e51d1e8283bdbcc2bd23220008a0820ea34cf0a942f2a828cf47d8e0d3a592005e14e0ac0ac74327fa8342c142000d7420017ca8ddf92427452b12c01732491abee849d2800c3c996557be2c0209e0db39cafe9843f3e58f4a4a75c04f6a8d04f05d439ba39300befd2ec409b5f1354e6ea86d5243589447b4217e006d68f8bb86a6e452dae32e05ab5b934900ff32085808017d4363832818101c1b43b1e909c2a00d010274a1173264cf5084aa250403570864cf5dd2ca754230103dfa5504c62204b2670ee1f22311185b47207b16b0efbb2002b30104b2e7e3c7c9ad22309b2681ecd94cea361681d92002d9f3c3b757db4460360601066d5c8823958bc0583602d9330c2f714404c6ea10c89e9b6a7c192230d69840f64cc9bd4d1481b16e04b2e78ec6bda62230762381ec992514e6260263778e55b310bfdd5e08064509235f45600c1f81ec695225202902638c0864cfa53b03fd4460cc2a02d9d3b6e5ce561118b38e40f6741fee7c2602637208644f58dbf998088c6918ab91e3db33ac080c3a10c89edc1e79a74560309440f69410e36f168141546aa5d270f46b3981eca965d4e62b02833d04224b8330e806b31d6725dc30e89bce80a59ac6b3ef3e220c26416c643d2c0564c0606c3a0396dee74284d98d47e9b3d0279fbfa3cf87d9e1d1d2b3e90c5846e1bbd2c689daead15223b4b68051fab61f6beb87d9f78c9462b028fdb1c9a598d9e3b56114c6793b3c4abfe087da302e13f4f6e3f44f5c880011c6844f9405a12d5d47cb7a47ca5227ca12c7dee3601b6d67ef0f5262ca27e8cbd219b0ec4e947edd28fded1ff9ea99a0ef4c67c072722e44986b548b982f3fd48f951ba7c74a8ed3378ed063357ea43782b9712c49b0ab26de741c93845b67a48c04733b8dd61182d6eaf65d1d64987beb6869ca0f28c11e807904476a393fd1c2d9313df2288dbe59fd236f6f26e8dbc638e219e3e3f30fba62931aa767131da7bf3d42cfa6f223fd8a09fa65e3f43da3f4841ff8618b9aa08f44b510e94284794347e9e37fa43f3a419f3b660bdecfa82e2008c2929d60b60b0c58ae151d98cb83108e9cce8641873626f60c0a6b0e27239d41618d76a8286dc728ad28ccae432043f282303b3701060132cd890d4373488770603accaee342a482e9e3e303063a7c70604020c8807020992604b38b13a80c2c242f4605d347b299b11109cc6e05f32f43bb0c98dd85121b0bf3db8c6605e339f8f8e0860cb3af1f27a6c55263610131d42b52636363618143a3ef4d74d4c213ef158ebf578b36c25e199bc8d222cc7e2b9d01cb9bbb10ffd830ccde020b4ba0ffefa25261e19d233930cc01c2ec5f589551e1995dac71276b4c0973d8d3d1f16dc25f1f5bc2a26568ef958ea7c01c7e0498c33396e54430c75a172299884a35a61298232696e59074215198a36aea933630472903967f3369ca06adec4e5c1c0b820c3cccf1383e8e256c8c59ba65ef85ebd45b9c07963f6cb9dc5bf5aa14f7fe507a917f63827a45f2fb94c7f13f17c953e915dfb605898971494990897dbce7c82ccc0be991e63917efdc91b93d239d01e308683281b241c3522020deedbd285914e674272c8600b2f162b4af45610299d82f86e42521792a284605d1de168f429b938c56d1e9421481392ba7213e6709035648f85e7cce5bb058214b78ce96044d7a22ccd93c0609145e10401521c35cb204fc9f4da6fcc9d493085114e63222c05cfa09b404984b7bac521435309703fab13c217ef4a77d0295ce82301102514befc8c48f0dde34a73585230273ed274000cc750fad3697d5f9fd360d85b9ead219b0423e3ac81f1f2dfc23f32430b7ea8817716b43002c76da85887e5f4e85b94d584e441b71616e3c75d49df0539e86455d1b82d8c84e3077cad4231a77d97844e3e1188968dc4f7f88683c1cbf8b68dc9d3f8d68308fc488bc3c38587cd3a4f0c0b3800e714cf2701e2b02cc63ce320fcc633de6e063ba8779d68c062cf4d5206a1c2b58a1a319415198a7781a88e72960c08ab693100ff35450e3e2e8d8d1783739ac90619e87637192e7156572501d23d19cd65ca508cceb310d5e799d19b0e2f9c9bcf2fad168b078ede401120442c0df9a34fe7ba81eed1c788759669e6648c0c27cc1d3d0079f3703565a39591f7c1b282c274259e08ba5d16830bff6f4679061be5a02cc77990a8b3f42ebb9054ba8a3fddfdce9cf17c3fc9204989f8f365213bfc8684d73d00fc5b1a71a230d4d9e0dfee3bce1825f2e58fca91cfc17a7a14dfed30c58e9d977b17f9ac2ced0984673331419b072e8a4e6c8287263ff0e72ff06ec7fb144f227da8d4926be6c7bdbb58578cf2e6ce916397a5f3356a4f273b19a1c11c773a3fa4ad92c1a9a810a707f9f811e1e352f2ca048a78f06addf763f141158601f8182c63681d469a8582096012b374df60f81bd743a7d34b611c73ad069824a50681a2c087231e059c17f075482e4e934b78101cf06264b2cb80d8d0212c6a3fd4cecdf0a6e2c13084d477e212e063c3b631243acd92072dcdf442a049063ffa185c13f513cfd2fae0bd251d40b0dff01f5b0b0309d4e87256c5d4651070be3583de9cf5b81858d5963db3feb358533a76102e16d0c78f6d02413fc85303e937d3a0d7e65c02a6b278370a610048ca2706cb9179ea9349af0fe4b167be1990d7f34eaccd72362d98ee5a5d3d4aa08711a5a157160c02a5f276b55c4970a01937c9bfef77d5ba47b3a1cb53260d5d41f7c3bf6effaf60840b0a3b69c667c9e9a838a227fb4a5e8191a048c7be898468560d1e1696844b49f01ab0e4eb69118272ce138320e17b39a464d62260c58cde7fb81a8181c1b0b01202ce1c548678c0c9c59393f048064d48058586c2b0102c819f80c3c1e16cb4353733a0482548a13451016db4980c576c06a71dfafd4b2888919f80cc6d84ae0ff2fbb5c60f158880b96941df99ec6b203031f43d67cbd88ad71f33b47cb19b3676bfb3cbce51367988f5f7b2b6496ebbda4ecdbbe14141fe295dfe3239b9c8e1fc94047f37b0995bfb959019650a48d4cdf482e18c9d7258c200e58427f64c8873e449f59a1bd375f94cb77b9312cb106e282f9f527d6ed59ebf35858228ab5a6064b8ab39ed2d1a70c3c914c15148525eea2c37e89dd341a2cc99acea0c31247e269b064034a0b3a0989c212a504723a1e96b8c15adb8525ae8d4768221e968865cd7508914561897e029e064bf4c641602c2cf176746a60bc2611a2282c294480402a2cc9355a880e23596c51d06f63cb32209ae57f17b947e4935c3c6a2d322cb9928e625eb2df8528280a4b16fc9fef94a0d22560c98a7406acd6ea42a4d2c560c99a51354c8c31c9340870a08ed481aabb9b024b2d4547ee1314233b296029be118b482d9ed847014bc98d6a85e59852f309b0d4bc1f1c73d2168a1fb74ffcb078070144d6d85f2a8e40a682f6143ae830b29a0c4b05a0a5743ae83096efb2e6cb68e0c422252c554885a556ba1061a9524a2c2c4570216ec7c3523728b1102e0e84a5fcd0927b14582ac38548ff6eeaf5bb053e58aa9f4287a50e7fa7a96f631f3f758a492ba5b0f4022a2ccd35be38084b9b516069551722450496b61d5d3d632941978187a5210a2c6d38b1be47197f6b130596f619798bfec35b3b29b074e8c45b64c884168b151685a54f1360e962586ac885c862a52a961a07c2d2f62ce0423858fafec883f811238f454b8aa8e878bbef29b074d31f572361193ed662242cfd7aac5558467a84c33f52ea8d50ca084fe60f80207a1ccaa18c330196711ae750c6338e1a3fc2e176168732ebc61fa02c9bc032b1230fc659c667fc61751a96c9a5c032693f725239cac9be094e201c95ce62a38900cb3c9d60e30d9d1a3bd1c8d8f4b3cc109565bab18664675060993e96516425ff6814590d0a2ccb3e49e86fdf7fd020793a280ccbba106059675856cb8548a4ea32f093f771c0b2110458361c96d543d9c2d1c0743c0d02a9181a8896a51160d9545876910b71a29f238f28971c97804a255b4c80654f8d4b255b99404d0461591917e20e967265efb11ea0cacd642957b665e441fc88faa13850f73b7ee4b004580e034b635d8844584e60a43a13d43419787cc60e7cc60ec6f6eff42aa74380e5168c7320b77444aff693f40acb398d38c41fcc251732622e39eb9f005b2e9900cb6d9ba877cf1f802d776cfcc104b0ffce6e2b58eeebf79bad607901022c3f03963b3bbed36adadb5460793b2acb67d2f1b0bc1b8df575d21601583e9c00cb87c1f2a213cbf6accd0ccb03e1083f1bffe075863838282432781dce7ba3a7b7b757042e88b5992106dec82259190207ff9c11dcd8960758be01d2a0b16223cac4732a2cbf114d0ffe74bb038cc35260f9ea499b1dfe10b2a9149a3e3a7e6005665613383d5a2c8ce361455f9c192d960ee3a45d88306e256d34f432262d5e212ce3e302d1c61259c687719b50ec278cfe88834c68603c2b9ec0b8ed104403e3583f5006c92c068f50e871304e1d65316b271e3fbea2301e4c71d72930ee34cb6f71f726fbad3ddaf42b0a8cbbf46388c00d1360dcd038e414f85950c6354c82f20fd6f10f5e0f07fa7be1c2bdd7458607e3d68f5a2730f2977f363ac93a0ae190f2b87514b6506105b15f5b47219b022b784fb20eac707a1c630ae513181bdd1d022b3c24c00a0f7ec49821cecbdb33dc1b8ef0f6c2c18181211be0604f6f1cc93b3064c38814ac4d34539342d1009a352e85a219155668fab5148abe145851635c8a1ff4ba213c24d817b72e24c03b78d4b126daca81a427da3a498515837fb97d0756aca7c08a59136dd96d08864981de86b860ef0db8900dc1dee138ff8849bb8646a41fdd3b3415e995f4a0d9e31c29995261c527bf965ec99b022ba9fd417a431c1c1a1a1eb2de1be7131e12f4dd2ea6118e7ecdcff71cdd98842aa57a2aac14f81b8e3e5360a50b7fcad10fbbaa4638fa353fdf71a4ec06298e73a4ec4d8595867ecd9172220556b6f98123ffe088481f1f7f4f7fefe07513b0fd1e21cad721d989b6ee516165c6af11a2fc89022b97fdd0d6ba703838c2c73bfc07838c22e437069924fdac5593fc6396271556fefa6be967c553e059d67fced11f0d32c2d16f00f21d476f20a5098e3e51e159a9bfe668b624059ed5fce71cfda19f228d45c251f3fc9ea3d9d193bc68763c159e8dfb0d47c728f0ec909ff8b5271c1838e2d4c121eb587a6279f92484cc7e0de126daeaa3c2b38b7f8d1015710a3cfbe90fd24f61cb21ace204c98fb7a5b2860aabc8fda62d1a0556b1f8a12d12dac1fc101ac2bd23467d1198b22faab4430a131cbda7c22a49bfd6b4aa28055679fc738efe6877d4e2231c797a4f9923d58d93a2832a8d0aabcafc86a33c0aac1af0078e1c83e1d008bf10b45bf642adaf3939319b68ab15129b68eb1d15562dfcb53dd4665260d5477fda567048b0f746ff88756840f2f7fabe2d35fca448a446a4c26a52bf692b8602ab994db435bd4d9c5322179e9c9faabd1bcf1dd406c77307589d8700ab73c36a68ea4b45536456e23074c8cb3fd4276295a2300e42f2d627d33db73d2e215cb5582376872c7f78913f55a7ecc1de0bdb63dae58dbc2242023dcd002e6160412588987173ae8e5b27cf4f7aff45cd789beb623d3257a7d987e01bc744a564495bf45e22b622f60c4299f92c7e0176a0831accecffa49c527fab463d9dbd2e79fd368b79a54903090ba6f507d4f92ffd8e10198bcd14acf8bb6b396a37973eeab665fd31772387554c5dc6d1a3cd9ab752acde3d37601a2d8f84c0a57bf5380469a7af9f7b89c86fc9b58b0c192ccd59a7286c622ab7e6a09ebfcfc0f21b9b3e5992c29f71debfffdd611b7a13876d0ce800cb5f8a85c3ecda17e7473a29cbf408a595fcc70edbf8e7fe1e58eac4ce7beca97babdd2d73308addc04600589c8f0531d8ffd2611ba00cb2cbfe17f73d61b4b51de4257e51ae175e5774fc57e509aa949cdf1d8651a12f5e5f107db9d63d717763b87540f7056e97eaaaa13967a8d8769d756bdf3d9a79c6edee968c37a0cb9d3da27b9f6f5d5935bb5c2e7287cef5eaab83fb08363586bfab3f7cf0525012670ea652ccac4c26c6e5f055bb3b4c25ef973b1c3f53da9e2b0e58b27d7c224917afbe5690beb725cbefe09b6371a79c5e2846dcd8c6df41db79ffb169258898da3baa42ebdf0db917bb9a2cd5d077a0dfbba64e0cc3c844b7d8749d9fbf401859b196193df322d703cd96c48f85ae9d4b86b6bd791a95b369e9c0fb28a28b69ef9cf18301664deb608016bf37d71e0c959c557940e6cbb1a57cec4ade78b41244f25579b61edf6a6a6d76d63ca67d97294dbb708de30701fed8f564eb87678ad6442378d2936376699f3a9c382fec4c63ef70680783b006da4dd7f56e78e75ccc9c1758869d0f30b11ac871f35bd1999e87be191938dfd876f0b0ddd015419fd398c7b6a53efa56f77a4ee54ef96f873d9898e632cc130c13f370e4fca7ebe38703dcfdaf9f96f30ffe41ffcf4ecbc162c1bfd5c8df3b2da7c7eacbaf4fcbd1bff1ebd3706c9ffdb2fcd37a95df3958aae660c1e0aefb222e9d22e1bc5ca5ef177a540cd51faeec332c5ed1a85ea4f676ddba35fa37d9573ad7dc32ed345ac00ff770d739c8e4ea7fb4b8bdcf791934f0bb536731361eaffb234b29abd5df955fac2ab7b19c2556cfec5bbde106ffb621e058506ca45b489b4468bbef57be4367ed45d7dc3e15f56691f5a5f26d5767ae38793c31d3c2b4928c34dfe792edf7bf8bdd30e0ec5c34681054a83127ce17283930c7d007ab0e3db6463c664458e262a5afc7a45775d83bab757a2bef29e5fe68a69751abd84d0aff5c58862d0198d8d3c8e6dc35a56a2e72e27d43ecef9bb21796f6e2b4df735679c5ec55727cf075c0559d46c7982043f47bdb0566a489bd0aeb28a433fcae75303addac5c5f5fa06e5ede8cf3800ea11e0331b1ded33a8d1fa3504c6afed92132afd72203576cf6166c8adb6e9555bc252167be5bee7fe0f449a4f7d529f2d5f52fc436ef375ab75144e4e31ed1f6595107d3a2f2b744b57aec588688cdea71e0b0376ad0d0400849cb63edcaeba859e40b8f3e712be8b31f5f593d84e61d6ba79577545569633092dafa1b485b07773c3b9375b1b6559d9577845ea687b5c52a8add5ffef9eaf28550650455dd97f3a2a293b9ddd6d32f228ea722162dad9f940645b784ec5b70b92aa6bee240f74a64a1dda2f925bc1ffbb615fae4a2bc38fd3307da2cdc838481d78d8fcfbcbe92a9738570d113b83958cdd79122eab161d7994b455d089758583c367ad68b8e60da9b3de96e4a292b6bed77776e5ab0203aea2e9735b56de4a493a07ff6a4939f9d76ffaf38e9e4f5420d2d4d0d3fe7757250c93262c9cc5ebb0e19cf83afcf6a9c2fb6555bc15b82cc8a765d1cc57d4c0c4aea945dcea66c3953f5aa6258fd76a7ba0f518dc9ca6fa44bdb8181570865c52b2b719a4707e7cc674dd6d0d04be81c59dbfd5ecff1add6dafe8d36f724a7ecb7a4819de3b70d89fe95db86847a8af42ec6eddcd8eba9a2fffec54cd307a4b3d44a32d27da963f95a5b5dbfd21d5bd45e7da9d9472d7e5ec2bdc6cf4eb5b7bf490d799b84444bf2ae6e911e6a6a150a65cb977bef5505e1d2e6ca5f176e1754d1bcfbaa5db20c0c0298a03f32074e19ea482ef7d66953ace9d20a95d54830da146fabd5b3b6e1907069c9bc5d5396d48b094aa37ea1332dbf98c7b620a0b741aa7afdfd0a7f7cbbfb7d0fcc8037cb907b4e5e153a6a6cccd979e919d648759e5d9ba79a68c3870637e9baab6edcf96b3d9195cbe1abc78fe6c404d1d6cd33ad2c49a6ea362ff1c25dde7cb630fa9c5ec5ae67e339941cc6671a3994a7b4e422f752874c152e1fb354dd7a4422ffa34b2588cc30d4de56539ab1ccd43f4c90a344a9caa65fb238cfef4a71824c5d453bcf85b588f7cb8556a722dc9a3738489ceff802710eac0fb9be769ffe605a20efcaf5378343c7cedf7fa6e45780177eb2e3b367e58ea36def7431f9e7f6de9f41b821db53798693ff55f994f5ed3372fe7ef0f4cecefd0b47184ded50e8778a09af45ab62be7eb469616fdd71fe1b368590cad96452edfce66c5d9155d02b448291cd6598947f50b2e8f40bd2dba3f2dec75f0ead259e6aef693b659bc85e2d5986f10398182fa4467df656d292f543b948c846ae27ecadbd315629214e86e6af2d16087d583b77c99415e4c7c4888d7799a47fe181cd5e87cb35f15bd6ee7549da2c58a7d3fd7a79a397ff12eca73cb677973fec7a3b7f0db24cc3a2648656a80e589692641f722c090857e9ae52b5caacc5462eeb4dfa701375c78069b96368efe226e1c30098574ec2ee513bdbeb3cb72e9065def5073fa4be316e13c3c9ab0b7743cf5b6eecdc2d6c17b0d7f0915982baf909291bc4daeccab7f3f7522d021ecbb7765db43856a27b8fa47162a3e1a0c2a607e75a375d437971fe278e7d4f42b6fb9a78bec6da1b13b92e1eca6a2b709dcbe3a12f351454b72ed0d96c438af23564eb7d8900dd3df5ab0b6fb53d176e219c5eea6061aebcd0202e99c3cb4af1a4b16b19580d30c12bc88288e604bdeb98ddbe0c53c34b47d722c7ceaeb76d5eddb31c6f9cfea1dfb0e9f094a1b69609468e1f74a638f5dbcf4044e75afdfc17caedf39d06dbefcc34715ff1e933fd5c5535ad375641e3c23db1b52b9079e5865c8992c72ed5e50ce95ec67646a5ddae7d548995bcf45c0eebadfc98c4f393a4d0e84f118e3be9faf6ddcf102e52ced3f5764df2fa067eb5a75ea70d050ec4b477fd070ed12ae7a9bfbc5baa90c3345a79cb97e75a6c0fb27dd964aa579df6d8bb82e2f1a6e332a2a0f526a93455c131c93e40f644fe0937578ef50f4c254b1845a09ce02645ede2324c1dc0c4d4200ae6f194bd6da4ab7b376e5978a2f9ce85c08e6df8a84fabbdf44ccf2f9bcd85659fb225039998ad932e5ec44ce568ca1fd3ab85fb7456e70b06cf5a73f2746a90bfc99502d703772797fb24f069e57f4e00927b56fa155c9e9b207d6ca7fcff7a7a554946ba3ccf168928dce79fed5473a262d623e387af7dc959331e241d7fa57241d4d12d1309ad192c22c9ac320895f556313b223fac82975948d4bb676c6da3aa39cbe64b77e94eccc076d6881f8c9d04d729274306af0ac3bbaf34d6b45bfbbf56583f2f7b11ee8d6b258854b4be6e93c2a8a5ad585825b7babe46a46e5f6a99ca7d99e836c32f9a335e7a1d46562721da1b2e447f34dbd8e5989ccc56646af1f90ec5a64cbc546aa5c7eb3e9d323438e9fe33f9fc7611e2666ffd4b5bd602f319cb3b0f54b938575f1d006f495eeef572b0c699462292eeda2dce6baf9223d66df4b1d551106bc052eb5fea751dbeec7974b6a8d4375c19c80d304176c4eee8c74b995eaf8cd87949d9333d9e35f2eb29343d152268390d2f7971cb392568ca900e1e282d3d871938cd3aa5edc5c4296dd41281ccaa334ea52503fd77025cad78760824efcb55996f4d2e35e7f6c83bfdf7ee5b901faa04912b56add25970f83771c7900a3e678dad2762ec782282535b12cf5ca075de6ddb8f2c7c69fa7c26b8cbcd8862eb47ad3db78729b142988bb24266c7ed93359a9a3be78c5c8dc5f886e4add54c8f28ba5fde7c83cc10ebf03a1b156b5a74fc134fb631b7ed9a8d53d644e8c8d5582153396d3d09a940c0b0dcab5c3565267616774379fc12be84bbc794a40fc52ff4632e75333744a264670708cef5dc1b710d7ce32e8898a87eda8a0935ae6f280998c9c7b3cf6e43198727c0e45883449f6a0ba3056cdbc5bdec80b5ca5efdc282aa24fbfd166b0e3ce8ed64323992be4c99ff702687d0c8d5584ad3bf1a4bbd63bd5de5e75abb1ec9b977973614fb952fb6f4ff5de2fb5f8e4953b91a2b09094e91cc2e7838ff3c69d12b45be450bef6725c5455fdff1a5e2ababa8515f4e6323027a3f2c189c3b3baea14b48dafa74bb732678413c94db499023e2d3303975b7c3a4782ff1d72edafddd70f9d509bb66effe0e1b6d46da716ebdec6d94fb73fbffe7e33d88dc5493dc4f69e278efee965e5ab85b7ed329bbaf72314733535f6dfd6492fb4d6a0962dfa4372be41ac7c90277c255c53d350d408ac05320a7a37cf9ed5514c69e24e7c9eb3302d594bb5450eebfbe3e03fcdfaecfb003580cf8df99dec5e8453ff53cffabf27e05c6c35f94ebcbd4a925fcaa7ce3f69d06bf9b7e6513b85560bdd93a49353fb7494a6b1746e831c702c6a1d876e5b79ebcf147b419d7ec367bdc3e367859be6fb82af7f88e142eb9ec7589273adf9e2899afc53b58dcf1bbfa030d1c33d7c6f5a53b1fef3b3d3f4a78759d74dd9138de77b7cbf69bd9b8ef8879be38f49b6ce2f2b673164a12fc396a9f5235e44b186f6f37a8d56749e429bffb86af242309183933cb8a8b579d95e7151f78d86efe5e9ae78d85c8c5855497bbe5f03ead14c45cafa7e3a094a3fac1baccd5c7fa25edec1b3514cfcd5deee4c8b1fac2d20bcc54345f709956bef0b3cb9858f9c2be8470bb752598cb745a97ca9bed6fc181f4e5fbeeb93fbdd61c124d5d3927a41b5987e195589496714d78bd2397dda645ba552be9a488c88d38f5e379490afead1bcb30960013638ed4a79a75dc8aaa5829b541b9981ebf6c16ce3de4f66ccd859a42ca6e09a6ab3a64a6dccbac6382fde3a30acf298c2afed7c6cdc203d073e52fc9c5d1856d2ee485c7300a96aaafcb94d9b59243ea3b691e852b10310db7974d7bb34befaddd3fd79c229e6b2caccf99da7a2685bd2d64cecc1689443449089c5a9270a66ecfb7c5f854bad8fe1747973ed7fe665ceff9f298a3fab0ed5200c833492222726aaf13173ce1cee60c3e7275c9f6e72909bb0a4ef03759f4dc4c9ef3e6957786260a25bd694169454f0bffaefd5b949ff1efdb9c9d78b007a4b8e6b278c9feb0dd25e735247a63f91ea5f543c6516f37bff5bba5e4aa77f6cbeed715aa05414884220e5ed34bd3166b65cf6b1c50bfba982bb167dfeb3aa3eb360749f74e0af18edcd3736cf62646c62ee5d4cf2b7d9f9d3dd69037df85bfe2f53d05db7a24660dfcd09e7bca405a5f598639013031054834cd27d840ed65fbdce1ed29efe35d1d0d56b7a99b15dcb50c5d56e4561b95866640a688dc96eb7d31a6550a671cbf3dded3e16f5d737ffd16d17bfb9f25ee2bb2ebecd61d42abdcc8c41047ee54cc9db8539154b7fa1c3692ca79d0dba495d3337d0e8d0cdbfde14ec5e06a7e0f4697d190b05ca3f991ba47ad75067dbc2bb3de9e2bee7a2cde607d4216d9087f783317df7621578087cbcaf7a3586cf2d5c44f43c71d92e62a3b50931edd1cbfac80f2723c83f6a21e2afd79069db37467a0df6f80ca5f494616dfdede5c6953f16260752e9c45dfb55d2f89246bafba738ea543f91d2b73ec5b8467f683c7cdedbdd9db3747cd122d19d4513821c9ceddbb555066f3f58c82f9dbe78e4cfe87feb393ff06c5a11c65cd7389d5d66a7b8efbbc78710b8a14f8174cfe27222efc7e74bd1c98f3884467b638e3ce7be9aa1bb3bb8fad70b5a9ef2a983b33e50e22d51dea5bd5faad78de90fdc10e47a719853cf88557f248ee811ee73755bdecd319c3abfc5094a9652370b1b3dcf26c9c86f2de70430f8941051d67cea297596215379029e3350ac56bedc01de5e95fdff7bb7cfd2fced426223c5a417509f4dd15e77862ce6f0bb7a19d3da3607cd46367c8b2afd64bcc86eb3088de270a5620ffcc1981270f36e987e55fc8bbaa3ba810b7dc2a391ff6f2341fe41bef1c38a6d039fce93ae47fe5ca7932b2abd8f4e3f2968bd8bdc2724eab07f92f55fb9bd8149dacb64c322a3bc07ee6f900e26ad07265de1b33eea017b73825ece4dd42031b938e852df93420baa433509bc2bae38df0cfdc0ace5d5630fc499bf0799ec0f5708dfefdfb1f2cf55e57ed946c3fe3b2b269eee134552462561897338effa6dd1dac6b67e497b28fcccd356e4edf743bc55e208df3a51ccb40358009ce426abeacf7f6da35436c2567a30562fb44f8c4968ff3961cfaec9fad7ef6cdaaf076dc94f11a3dd058fa1033508f5a5770c2ba34506470aab81568ad891f72af987f39e07dc232832e8bb3477bfff60a43258844c62f4d6abc7041c820aa7ca8eb90fa4afb55a5060d73bcdfa67d01f1d22e895710f0a29dc40185561ba6733bc375f70c77cf3031d0d19cdedd7970e155c62ac7e1328e4b0093a30c79f2cc033b57c2ed6693ca296fa5a153cd3b8779a343797085bea4797d6050d8d929eb6a2b9323a0740366601daa2cb649aee0c09509983d9c117fad99eb46a096920e277bb673a08564bc28fe7353deacf9e44a32725ddba6fb2ae35dc2e2bcdb7ef657ec8f3a398b8a6fe67f6a305ce0ba93c195aa83ac53e4d7319e19b4ff63b8b017f6a60878bc2d7adecd4d34e6a9259564770c1e9cdeb46f74806ffdcf7cb029d5bbb653c2966d9f907d52407ae867cce13cb5ffc0b4ef5a3ac1cb77bd096753e04b524b4ccc815877aafbd61acf79f35e95cc8d18e0a120562619f47ba927b5aec327331478b81047932fcb2ccc17d9cab8ba8349ea7d8746ba3dfde9757bbf9be0f8d7deee71108b7de8ca57bb7585af469878f02da924cbb59cef3d4eee8cae3f24d430d3b5105996bf5de9442155f6e88b9365874f78e428a5ea55dd52b418548dbd1e71ace034f2dd364afdff91abf8feb93bcb7e76151f0788fd6fde59f69babf6def7d5fcb2fc833ef9d7bbb4dc7ebb8d727d3a533d66cbad63f39958639f2417d0698f032757575fcf5cfe85daa0af87d5708874bf6af5fb85079b7702b33c81dc5df197575665c7a8777677e3ee76b697fdaefe2dd9a9fee147bc1fdb575d7ec031d83ab4dcec6ca67db4fb01f3fd29ef2e8671399a9d5b32e38b2e8f884bcaeb9033253afeebe6846d4ffb76ffc5b194d3fbd3a50edfaa242338eb1505f72fc845db87e6ceadd83428bff2c88b6a695fafba9d05ed73f63c73388e2cad5f926960659fcb5917a0dfdafe285fd3ffc2a60fc9ee9a51f5a47cfd4739acf5f1b0ff89f5f144c4fb66d5dc01e5edade5f107dc43afcef9585ddd1961f36ace66a354f7878577536621a6974e35d816ac12d3098e1172ea911c7cb2d0acbea06cc6bae7372ecd3ea854d930b62e40caefaad6ac99d958f5b44d2023bed3e77c9fdf4deca768e323891fbe896337c64fb9ff224f5e176899b42e70f9c49fad0b6caaf165fcbd750180ad928cc80cf9c6e49c28b1e08ebc40e2eeb06d9ba9e4cca315356397a68e0cd675ab6907429891a8bab7f2c693e39ed677452b39795c783dddb08015724f29bba38efd464019d80430c146a48922e81e6cb6c87c83e860ca37bf4a27d3d6fe84fc790ff82b4fcc3159c6afdb39655d509960d27897eaf52fdc2b90adb9fdd0fcab0642f34fbfb8850815f7f21f0d292c3911f7a2e2ddbbc4c3e6f50a888bf63259cd0daf766f363f6a6d2dbe897670556fbacb894afdc646e2a0fc81072bd0b476d5b4d2da19ab92755615a4cec899b3b5fff63e0b8b38efaca52ca8575eccd6f45ba9480adb36c41f6bf964edaefe07ca225b0a45407089369d90ba03c1cb9eb23750ad5ff385d1631e98356740acf123d476fbc5b1c3590ada816ef53565986a8089b982043d894ba3088b75787ceedb967931f623b69caf77c155f8ead2e2fa6f8331b93e343a6629d2c8f398a1608d8d9f4dcf95a05012771cdd9fae4eaca7debeecb9f1fd92ea6721a879e94c4c64e9da01bfa0a94deab464608ded756266dee0caaef6edd735d0e1bdadae6063717be1e6ce050673027510d7fba73bb35d796c1a1fb1edb86bb0cb6837653342eb2f6fdf2390d6ac7ff97c0b1a780ca61778fec2f2cbd436e6a463d85f3ee7f0ac9634b8eee738ef86eac6971a430ca399da2d8e0e3912d9bb1d1005a3c53378b7b5f4377f7d12ca7f29e78a28a088b49e594c8d7b7420b7e52e44fc8b9bd17fba918a8cdc483ebc45c3ed3e59cba0d372e92a8147a106d5ff8fbb2f0f8772fdff9f194b832c85ecbb644b96905d681864295bb652b66c59b33396c328644bb678b2644d945d458821eb945df63142488591e57739ae3aaeeb774e679cbabed7399f3fe6afe79ae79ee7fdbaeff7fd7abddfaf7b1edb92ea14987071ea8c13ae760180e9cf04d615f5e48c0a3067222b9f4ca90b43032ccea3a43229571218527b78ab0ad7f3834958c44220f1fa091f2bb4b48e940f16848f4bf1f8620fdb67b3741620aa54e842cc98ffbf68b83f4edafd8018b26d41a01fbeca16b49b1d7f0f9ea4a1631fc813a937f452dd50577479dbfd963af4b542ed2903e1e09b6c6cf6db192f7866318e11ee628c2cd242cd55a415259f02081a0cee86df389283f93c5a5f0d7e00c281ef0357aa9a7a79247fb3f4411fbfc62da7abc87e72e87ee1b08b4c71f475610ea9462fbc6772180eacbdc77e5d7f6dd127cd3296529da988786d8cddbe6e96355d3d5a8ee83f51f4813d4d3b19e4dfc12c3daeb04eaec59eb8f462c2502e3d54e6c5a171a608394934a0bb22466f94d63d3c21debdc46370ddd1b53c37a0a1d4c88ac67ae3c5333462ea9b8896b97ddd223c4a9d4885c572b455285347e24a610d04b97089c99b533a10e03a873746c87d229afc9f89e81cba0c354191ae561356034037a1c489f0158ee8a745340200483ce9df7d9e8d70a0975883d1896fb55dafd259179b9e164c8c8c2f51862e03015e386fc679e32a7e7a018ff366ba48e262c226cdb6279467259342656d8873aa0950201c411330d299e5c01c1948155071a66004c910529d9849d95a363f9f365a77cc93f1550bdeb1ba8523b8f9070b99dac7423c607fc542b20d1bfa7f928510d58381cd6d53472f92b02cc97b56fdac19132a67fb8802e8275e87c355830b9b97d20d00e7e177b1728d2c27aac86cdee4918c8a0c9cf88dc67c715bfeb9879931f3e680c477c717fb411c5f66fef90bc2be304b87a3f04469d326d457b832f3631ff7c4582b8af6345de9c3802d9629c9fbf69015268922da35314f48e32dee9a5a68054ea9e5c28ed74ee79bbde5ef74b0e5ff6735a8835cffd72eff73e9758e85627e60a38e47e6120068788061eb6bb68de493538bde6225ae163e8047a05aea293e73519d2c860c2d6627484a8cf02bb25cb12ed5f418dd50d466cf2e2fb884dffbf75db4b95ae0561d9b74e57e18817cf2aefefeb238df9796c9d95a22797728ef14008209cc3223e7b33448df67642642ad385643903e51cdfab7c6a2cb863facca2ced922ca95fd1ec89006ab7ce9a57f4083569b3483c1ab8e63c56a71d04e39924b359ba669de2aae82c049c99469b64caf1e6b0adf2ea4d9aea1a4a170482c71539513d172291aba76dc9bf19922ff149130a278dbfbdcae75458de51c4a6203c9bf3ec8edf45736b939449d68b77f05eca917b8664b77f89e0f2dd843f2be5bfe06c37c7e7ce78b96d16c2888ab62d9b599468adaf465cba7611a0b32d6a7d4333c14b61de3f4bae014f78c0fde471ad8e3553db75f1f6ae681fdf6a702508077e0a7c66ab3c6278ab9e694518ed77e62502ab8a3dff86d324c4aef890476a8013cb32de018ac681adbee70ce87ea7096deb8f7346ec5dbeb77c0fb411911e9b44921462d96172c7da939a26af8a97192bc7982b9002fc5c370280c63e54fdb272c686ba0c99f0b62f5557367965c6fc8add35bb39d5efc3921c64d813204862166c896880e1716ca776bcff25f233a41b30bdcec80c8735c3931e35c071770b2abea98240b9a6a5b3b9d3d44ccfe966faf22548b424c2cb4f18762106bf0f4bba7f1bfcbb618bcc54f886de7d585c199b34fde2fdc4c68432457c0b56db58f1406447a3776a01e099106d21338488beb83c04cdc9b09c8531323fd112de6e17218321aba5da73f6fa981730f10d9a5c823e6313ae62759d5802d94e63b567b95eec225b57278582b57b434221cac0430da8a12fb0aaa8d889b1c716f82087d64eb2357f00053e788a0a826b938077718ba9af26a205e1888e00d051651e8139d4e455adcd12d9707ab3a31fc37cb050aa4eec6d18a7fc64e704deb78cc31136ee39b9380eeee4820f59d322636a7dd80eb378cc45b7a8587d2c8bfb97e7727c9c5c1180958e7bf66cb31c2832e6d92cbfeaea369350f6adcad2ce8bd0abf4592b95c061c0e76e3dc1722d98da4d869da7c7ef5629249566f45509ddeb410d21a7873529b0ef12deea3f28e1e7c3328349b0a7d30cb8238094cab4b35fdb481f893ce41145850919e25eeec40146812ad8d5ca693deda352a6aa794b5845a273b70b02edea4a2451fc3b67b00adf7bd4c4fb17b9701c9e5b8d3285fa8ad767f10ba9348773e41f50a73a8c0586d6230048340a639faec54a7a41354b76e3ed8c95c38c1b8d48a94b058dc290d8a5f348e0ba2c751b91b9c5c504e355c932f123d471500f849d183cd167ca4909d3ae706f8faf181f8caffc5971e17fe2805ad70295920b5a9d425263838bf70b46a9222cf7a61b51086dc6b593f6b3cd9c01c0b9eca50f4e2f9008b113cf6a74387448aa3f4ddea17f55e43a1c48434bdf777b6e9743481f8843008b1c9e9a4bfc3c748b5276ab41819fbb1eb64ffcfe5b8815cd1d578f267de0b5e7bdefeb694a003240bdb8f7e2ec08474d90eb95c4866ec08d596bc9d3a8fa7a029d92896c9d7205ec29347fdd9dd9939f81ede3ac2f895b350403c2412600cda3cfdba0415c57a268f97f0b94c3c42852703a600a8e0b1a093ced92e38f4bc73b2f26e020f17fc881e9fd72a0feafe480944ebcee4fca01e27a04c0c860ddfe7051bc7eb0a83d91c8d1adfb9dbff1e711afb420916c1a448168c857801663ebfd6640cdc2103ea137b53c3569c3d8290f49f778c7ee956ae23e9a77631718f70301036f1614747850728b2ccea69d31f7da90f2e649ffdf136172a4653f307ef37c7698048c667d1ab130a2d038e55cac055e5b966c0dafdc02f469b55cf2c4ce0979255defb21f7fcff3fe4d09ad01a8aae6e3c7cb3e8dbe92647b6beec6afd508d7aea6b5f9b5e80c8b8ebfb6a63a9fe19060533bfd9f58736ede03e59943dae5d4f5d2023b3bc4645f3ed569aa070c905a0f16d6c2f3357b01d618cae20eb22a3d78cfcc391345c93bc5341cd657f559adf99220d8b4e3d43e7b87425fe7cfa62fa0c12ca9156b8d0d98dbba63693e6942433225f4b7e23af4d2d9d3f09ee589fb0e851edbb7537d68feb9544df8825c8d58c2ea5d17eda9b7646f1972b912b4cd593d3daea4aa7fdd24171850060ec9717871e4d731a13fccd527f61535e6de66121077b25639db5434df9998eff74d13e47d72e531876f6bd159b2f913930ce6c1d5cd4b37d846a2ac6db891924a157d783f69d29e2630399826f8bbe4ff8f354109f9715213553d6355818a76497b47c63838ea5e780bf7a6acc0c8734c54f23d002a6496d48b75f832a1b166c472c2c331f30282ea665e66607afb910ee913fd65d5e065100ebc00385e3ff17c6653d5cf74b22d92fb26e1dbbbc76e64a8c06adadd027da92ba3ce7ae31da0141c38bdaa6bbd83f3e006a1bfa381ffb862acafb37151882a3a81d95db79a7fddf8dafb23d0f317ae425978529fd75241474e0214c14484665d32d0fec3332dd45c0587f3a1b26fc3535a9e5a3b5f6c77e10dc3ed42ee7c30c8ff0167c50f72494ff10537e788dcc75f4d89233b121ed0849324f42550e4197af1850a4a43f50038abce58ab3fdb97c36ed30123e501299fe6ebf2da57641bbd9b1dcb03099937aa4640eb830076c52e4344d96b9ef96ca6fb43274ce1cc74e33944eb0aa26068f0ac8b0f78136fc0d3d62377370a995f711432027846dafea4b6d8993d0379a682c943a9dee6f4c6a7fc49ced512f9ee8af75ccd148029733fcb51d39b14e82ee9b595c9de3ef74bf22a8e3708300c15fde8cd858d8fd584c6201ca10110620cb2dca07add2e1f089bf05473c54595d984131f2a80c8b71abffaf8b05b10efe74bc71142abc12238b050f0a189507b385a3a048c284ad097a4a0acbd6c12553ce855c37cc8189e5e7aa80a51d5e442079bfc1b18d97f6e1a9cf0ffe1657be0dd8fee6d91e55afe37e353fd0e04efac02fac9a94f4cad36da0659fcb6342bcbeed3324c371202b87a775843f50601db165de5649f4be414dc2f83bb798ffb7c8589bce20c73c435062f7e94af3b1ff42d6693ff86983dfa51675f8f6cfa87dff7b26608c02b66a237a8a21763730fbff1e4a07edfbdb3c16b9dea0a3fe7ce503094af258745da0090e5de0f61f54a56ada14cc96aaf06ecd59d079f9e66582c6e49ba37a8da1f4ef12d6653ff869845fc44cc88f9d93aedf088d92de0b4d2cc1d27ff88cc42bb64abecb0e8471e8f6a47f3087afafb7b1727ad32cd8d01f2d3495b1990cd378b4c02a26cbe16616acf95b86256a52763c96322ac75dce7aaecd66d39f02b77a22c6d0d3e838dbcb70dd36a1ea7c913a4385c6e34cb4737151de14bbcdb3bce02088ff0760d8b2d7bc35081bf853b16d7570ed2081ec58683bfa87d40bdd09775ac06278270e0388081bbbf3f5ee87295478fb88524da5eacd4e77e0955ef619a90d202c8d5c2bc28bc930b8003abfc210f30fbe4c1a3eebf92078a2f29197e4e1e10c8d58381a3626e23fdf14f1b261402da750767331a1ca2065f691dc73c5f3575ce545d4b02fc4ba7e706f5dc8695dcdedd0e7abff205414f5e1d298c72af3c26d2d1969fa2b29bf54d7f4dd6b7dbda2229b430599e9b12324445e545da2077348bdb569c4607e60760845acf00a7c6f1ecac116edeb5e2c910dfd4fb7e1b8ae9141a3b40d9329ad3cace2dbea01a920ec24152006e228e37d26766c7b91b5649635d7bc8a4d0f3062b264ca03b75cf08bb86efa3f106261307d1acfa0a595fc7f3ef89c2da7a6450cb7f527b29c7b632ddf7a0f71098e4f017687ee03607e585fc1f38125d241db744eeba6ee4396707dae590296a84d2d05b8e7ff22b1c4b22ce0eb39e05a4128e1bfa7e323d9673a572baa660bdfdf69965cbd8da159c981cff9df93922fa3dc1e5f26b05d711544c4b22cdc9e5b98756aa8c6a6b0dbea16941ff09c18584d2c4c66c57d20adc2aa917bcf4785bef50de2663c5cd2b0647e62bec3e138b02a61770236d5dce412e8564e853a3913df3dd102343bb1ad2b759d85287906d996a88300807110444eb4a9315a2e73e37f1075bf0713812703ebd328188437629b950da747e09c59f6767e3c0a37b18711e0c23cfd703fd3e335d12ddf7270e45d06b680a299e92fe1f28444500c69e4ed69878761ca577100512bed36e3630e59ddcab5eb53c245d2ef59cf82a20b3186f7a51aa8035f9f5b2b5a6522f6b475627ef73a9741706e15e969b719a6cdf7d215c07f285fcd9a4af070329c746938d422f67de7022d37f8498459f4f149ef50f71f92097c6547526895b1670999a96e9e36fe315d1379379894db6569ead7eb460763b52fa6d992fc2ef02e65b1bdf656c504bf6b4088f1ccb3c0a162b2bb465cc390cdd316fb821ee21b34c0fa1c27bca3cdcd7c667d85f51c5bf8dff0f26073e5ef870c7e0ba37b9a74ec9bd93e047493b472a050130456e64d4c360e34b77e55d8d000d4a61ddbe8b3af09a00356dea2f26d1e68a3c5fa508c68aeee4a189178edaebffbf000000ffff1ed9a0d101","nonce":"0x128cd","to":"0x8453100000000000000000000000000000000000","transactionIndex":"0x3b","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x41535904d7b20be8d733e03d95ec725d0c6ee4307d899cd00e0ca0de91533b13","s":"0x3f98c81207d078199bf74a600a77cfa389574646d63ff55ccfd194491faf7226","yParity":"0x0"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0xd3343d473f624c02045a01f4e7ab604bc56c4f42","gas":"0x156c0","gasPrice":"0x7d9f6fbd8","maxFeePerGas":"0xf49895cde","maxPriorityFeePerGas":"0xcfd6ae0","hash":"0x00b5b05c6d1a2eb8abe2c383da600516515e383fc8a29953bb6e6d167e9705b2","input":"0x9aaab64814636b3e36ce9c4227a5fb53273a04c3fdc94c8a00a4d6d5b6521571b94ec7ac00000000000000000000000000000000000000000000000000000000000028a416eb20a9e75d33ef7eeb4a0bad7413cea6c0b3a7f6ea249a93a428c0c36b52240000000000000000000000000000000000000000000000000000000000840244","nonce":"0x6c7","to":"0x1a8b038f5dff48f541e8cc658b9e186b9828a726","transactionIndex":"0x3c","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x1","r":"0xfcd40d6ec6fa94f40f866c18457c22ebf5d8e3aaf2616c4c1cee5245ce8f042c","s":"0x45e5962efdda48e38f706cf8fb7c2f722d80da6e0a1d2589b6627627b44feb82","yParity":"0x1"},{"blockHash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","blockNumber":"0x840249","from":"0x9aba7eeb134fa94dfe735205dda6ac6447d76f9b","gas":"0x3e64f","gasPrice":"0x7d9f6fbd8","maxFeePerGas":"0xf49895cde","maxPriorityFeePerGas":"0xcfd6ae0","hash":"0x6fc411f24c7b4b8d821b45de32b9edc5ac998d1ac748a98abe8e983c6f39fc19","input":"0x2e260ac30000000000000000000000000000000000000000000000000000000000000200a5c46e2dd70854a0fe8167b705eb7df71cc927469c345248c81b21a5ab2f5f19000000000000000000000000000000000000000000000000000000000000051d000000000000000000000000000000000000000000000000000000000000051e000000000000000000000000000000000000000000000000000000000084024700000000000000000000000000000000000000000000000000000001b3fd9f800000000000000000000000007ebc0d2451003f2d3bc5f6eafbb7e6bad81e250d0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000100000000000000000000000000000000000000000000000000000000640fdea40000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000002e43c000000000000000000000000000000000000000000000000000000000002e7d600000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000f6cb686532a3ece4a20f823e59cc386e1ba384d8504e902c932fa1497c4afdfb0b000000000008008402470000051e00000000000000082ffe82e7b18f0778493c989ecae1740325207ea29328999d5ae646723a4bb75a1305c8ca593670972f535ff8afda050351549ed800e9b98c805788acfe20e6ac076ae024efa64ac3ceb8f0a3cc9ce93d122f6faf8376c854da95567fb7576f762299ca2135608c7ac5a824257b50043526903fc8e01ad88006448a69a4afda4c01ad8a6ed1260582538f2311e902c1e7d5853a75b74482117f2cf920897c1b9428eaeb7e74a95d3d32db03931d157a48aaca9748bfebfecffc2d140caddc66b300000000000000000000","nonce":"0x108af","to":"0x7ebc0d2451003f2d3bc5f6eafbb7e6bad81e250d","transactionIndex":"0x3d","value":"0x0","type":"0x2","accessList":[],"chainId":"0x5","v":"0x0","r":"0x1b284cf86146fa2094c7c709cf9fcb8a177f4fde1d1c5ca112e41f82a5e973d2","s":"0x5618261c40d23e5beb7ac0bac77b16ead56e547f103f649462387c5acdf18ba5","yParity":"0x0"}],"transactionsRoot":"0x1ad3212eca045505cfc4cacf675b5fa2e7dc7b9f9cee88191464f97d1c9fbca4","uncles":[]} diff --git a/op-service/sources/testdata/data/blocks/pre-shanghai-success_metadata.json b/op-service/sources/testdata/data/blocks/pre-shanghai-success_metadata.json new file mode 100644 index 000000000000..87c9db722e78 --- /dev/null +++ b/op-service/sources/testdata/data/blocks/pre-shanghai-success_metadata.json @@ -0,0 +1 @@ +{"name":"pre-shanghai-success"} diff --git a/op-service/sources/testdata/data/headers/post-shanghai-bad-receipts_data.json b/op-service/sources/testdata/data/headers/post-shanghai-bad-receipts_data.json new file mode 100644 index 000000000000..1318d2ab45a3 --- /dev/null +++ b/op-service/sources/testdata/data/headers/post-shanghai-bad-receipts_data.json @@ -0,0 +1 @@ +{"baseFeePerGas":"0x3fb7c357","difficulty":"0x0","extraData":"0x","gasLimit":"0x1c9c380","gasUsed":"0x18f759","hash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","logsBloom":"0x020010404000001a0000021000000080001100410000100001000010040200980220400000008806200200000100000000000000000000008000000400042000000050000040000112080808800002044000040004042008800480002000000000000002020020000042002400000820000080040000000010200010020010100101212050000008000000008000001010200c80000112010000438040020400000000202400000000002002a0210402000622010000000001700144000040000000002204000000c000410105024010000808000000002004002000000261000000822200200800881000000012500400400000000000000040010000800000","miner":"0x000095e79eac4d76aab57cb2c1f091d553b36ca0","mixHash":"0x5b53dc49cbab268ef9950b1d81b5e36a1b2f1b97aee1b7ff6e4db0e06c29a8b0","nonce":"0x0000000000000000","number":"0x84161e","parentHash":"0x72d92c1498e05952988d4e79a695928a6bcbd37239f8a1734051263b4d3504b8","receiptsRoot":"0xaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa","sha3Uncles":"0x1dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347","size":"0x2a51","stateRoot":"0xc56738518b2c7854a640ae25996d2211c9ef0dd2e4dd9e59e9d9cacef39622da","timestamp":"0x64110a5c","totalDifficulty":"0xa4a470","transactions":["0x1e8f148a9aea7d8d16ea6e9446723b8f262e8bcd89c7c961d52046ebd43b4598","0xab5c870f4c367012bd763172afbfbe68fbf35336a66ae41aff3f2c9dbf4ea3f8","0xa81fd92b2d0f0bbd3cc355f869cca3243c98c5e2641db9ecf3eeabb3b13bff6a","0xa92c7b720c08c83f1a0ed7e4c163200e30a3a8c03fcc5a51e685ea20cd0cb577","0x6921b429ad2ec1e97d3457049ad2e893b5a0349beba47ca1c74a9540af75347a","0xf776b2da0b835dde05d0d8b76fd19385d61e7055036cf637f804b36dc94f2384","0x9a08d899cd14ebb930ed59fa774afdb88a22615b3a931e930931ea54d26dc0bc","0x0fe0d97e25d5eb11a33a3e8278584c3780941fc2675bdf8fc547cee3d1fd3b17","0xef47a60f57f177a683c723c658137efab66d311e1c5abbc4d74f653535144d03","0xe23a5b35faae5335adc5aca38c5d633b00438b798c2053104b8df48406c9b141","0xd8cea4ba619b317bc05d58534af73beec6c2548b31b24d4dc61c9bbd29cfa17a","0x79a4b9d90b02c768baaad305f266281213cc75062cbe99a13222cc0c4b509498","0x6790a3bbddbeb21fcb736a59b3775755051c3a6344d8390cf8ca27f2e8a814f0","0x87ec7ace5442db252b5751ffddd38dcb04b088d36b6b0e526ff25607a4293c81","0x40cb487ecffda94f97ce7fc0f7163f2f024235df2c8291169edc80dac063e6d0","0xb76bb3d88c9b30d927c45ccfcf8d5b0054411ac8501ad588822a7d04690cccf6","0x798ebe823209869347c08bd81e04fbf60e9bdfe44b1cc923215182d0cf3d4edb","0xbe68a7e02725f799a65ebb069ccc83a014ac7c40e4119bf7c220a2f6ddfee295","0xc90c3a72efe81331727fcce4b5bd4906066da314ca9a0b44023a6b09ea7e8114","0x619a6cbd43cde074d314c19623bd66d9fb1e13c158d7138775236f798dc1245e","0xca5a56cd77b9e5b0e79020cc6346edf205bc11e901984d805125f28c2e6686e6","0x999c9ddeed67c6ef6fbf02a6e977a6c1b68e18d24814e51643c7157b87a43e0a","0x47c8f5d0b3778e4c34eba7fcc356fa04a5afd954ccf484728e72c002764dd3c4","0x396797ae0ebcdb72ff1f96fd08b6128f78acc7417353f142f1a5facd425a33e6","0x454aa43d6546a6f62246826c16b7a49c6c704238c18802ef0d659922f23a573c","0x317ecb5bd19caa42a69f836d41556ebb0e0e00e1c6cd2dee230e6e6192612527","0xc879285db5ef0a6bce98021584d16f134c1dc0aed8cc988802c4f72ba6877ff6","0xecaa2d6f597608307e5084854854ba6dc1e69395e2abea14f2c6a2fa1d6faf9a","0x4dd69b69a568ff30ae439e2ded72fbd7f2e7aaa345836703663f155c749c5eed"],"transactionsRoot":"0x4a87d0cf5990b1c5bac631583e5965c2ba943858bebb2e07f74d0b697f73821a","uncles":[],"withdrawals":[{"index":"0x1170","validatorIndex":"0x38c2c","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x66edfd65"},{"index":"0x1171","validatorIndex":"0x38c2d","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6cd228e4"},{"index":"0x1172","validatorIndex":"0x38c2e","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x77f3431b"},{"index":"0x1173","validatorIndex":"0x38c2f","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6b61f268"},{"index":"0x1174","validatorIndex":"0x38c30","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6e10bb21"},{"index":"0x1175","validatorIndex":"0x38c31","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6eb115a5"},{"index":"0x1176","validatorIndex":"0x38c32","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x7caead1d"},{"index":"0x1177","validatorIndex":"0x38c33","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x772c0ddf"},{"index":"0x1178","validatorIndex":"0x38c34","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x75930a95"},{"index":"0x1179","validatorIndex":"0x38c35","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x76a4db09"},{"index":"0x117a","validatorIndex":"0x38c36","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x7e692b27"},{"index":"0x117b","validatorIndex":"0x38c37","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x72038ae6"},{"index":"0x117c","validatorIndex":"0x38c38","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6ccce352"},{"index":"0x117d","validatorIndex":"0x38c39","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x79ef6898"},{"index":"0x117e","validatorIndex":"0x38c3a","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6d58977d"},{"index":"0x117f","validatorIndex":"0x38c3b","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x76f7d208"}],"withdrawalsRoot":"0xbe712c930a0665264b025ced87cc7839eef95a3cbc26dadc93e9e185a350ad28"} diff --git a/op-service/sources/testdata/data/headers/post-shanghai-bad-receipts_metadata.json b/op-service/sources/testdata/data/headers/post-shanghai-bad-receipts_metadata.json new file mode 100644 index 000000000000..4da07c88cd37 --- /dev/null +++ b/op-service/sources/testdata/data/headers/post-shanghai-bad-receipts_metadata.json @@ -0,0 +1 @@ +{"name":"post-shanghai-bad-receipts","fail":true} diff --git a/op-service/sources/testdata/data/headers/post-shanghai-bad-transactions_data.json b/op-service/sources/testdata/data/headers/post-shanghai-bad-transactions_data.json new file mode 100644 index 000000000000..075f7f0583c5 --- /dev/null +++ b/op-service/sources/testdata/data/headers/post-shanghai-bad-transactions_data.json @@ -0,0 +1 @@ +{"baseFeePerGas":"0x3fb7c357","difficulty":"0x0","extraData":"0x","gasLimit":"0x1c9c380","gasUsed":"0x18f759","hash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","logsBloom":"0x020010404000001a0000021000000080001100410000100001000010040200980220400000008806200200000100000000000000000000008000000400042000000050000040000112080808800002044000040004042008800480002000000000000002020020000042002400000820000080040000000010200010020010100101212050000008000000008000001010200c80000112010000438040020400000000202400000000002002a0210402000622010000000001700144000040000000002204000000c000410105024010000808000000002004002000000261000000822200200800881000000012500400400000000000000040010000800000","miner":"0x000095e79eac4d76aab57cb2c1f091d553b36ca0","mixHash":"0x5b53dc49cbab268ef9950b1d81b5e36a1b2f1b97aee1b7ff6e4db0e06c29a8b0","nonce":"0x0000000000000000","number":"0x84161e","parentHash":"0x72d92c1498e05952988d4e79a695928a6bcbd37239f8a1734051263b4d3504b8","receiptsRoot":"0xaff90ae18dcc35924a4bddb68d403b8b7812c10c3ea2a114f34105c87d75bcdb","sha3Uncles":"0x1dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347","size":"0x2a51","stateRoot":"0xc56738518b2c7854a640ae25996d2211c9ef0dd2e4dd9e59e9d9cacef39622da","timestamp":"0x64110a5c","totalDifficulty":"0xa4a470","transactions":["0x1e8f148a9aea7d8d16ea6e9446723b8f262e8bcd89c7c961d52046ebd43b4598","0xab5c870f4c367012bd763172afbfbe68fbf35336a66ae41aff3f2c9dbf4ea3f8","0xa81fd92b2d0f0bbd3cc355f869cca3243c98c5e2641db9ecf3eeabb3b13bff6a","0xa92c7b720c08c83f1a0ed7e4c163200e30a3a8c03fcc5a51e685ea20cd0cb577","0x6921b429ad2ec1e97d3457049ad2e893b5a0349beba47ca1c74a9540af75347a","0xf776b2da0b835dde05d0d8b76fd19385d61e7055036cf637f804b36dc94f2384","0x9a08d899cd14ebb930ed59fa774afdb88a22615b3a931e930931ea54d26dc0bc","0x0fe0d97e25d5eb11a33a3e8278584c3780941fc2675bdf8fc547cee3d1fd3b17","0xef47a60f57f177a683c723c658137efab66d311e1c5abbc4d74f653535144d03","0xe23a5b35faae5335adc5aca38c5d633b00438b798c2053104b8df48406c9b141","0xd8cea4ba619b317bc05d58534af73beec6c2548b31b24d4dc61c9bbd29cfa17a","0x79a4b9d90b02c768baaad305f266281213cc75062cbe99a13222cc0c4b509498","0x6790a3bbddbeb21fcb736a59b3775755051c3a6344d8390cf8ca27f2e8a814f0","0x87ec7ace5442db252b5751ffddd38dcb04b088d36b6b0e526ff25607a4293c81","0x40cb487ecffda94f97ce7fc0f7163f2f024235df2c8291169edc80dac063e6d0","0xb76bb3d88c9b30d927c45ccfcf8d5b0054411ac8501ad588822a7d04690cccf6","0x798ebe823209869347c08bd81e04fbf60e9bdfe44b1cc923215182d0cf3d4edb","0xbe68a7e02725f799a65ebb069ccc83a014ac7c40e4119bf7c220a2f6ddfee295","0xc90c3a72efe81331727fcce4b5bd4906066da314ca9a0b44023a6b09ea7e8114","0x619a6cbd43cde074d314c19623bd66d9fb1e13c158d7138775236f798dc1245e","0xca5a56cd77b9e5b0e79020cc6346edf205bc11e901984d805125f28c2e6686e6","0x999c9ddeed67c6ef6fbf02a6e977a6c1b68e18d24814e51643c7157b87a43e0a","0x47c8f5d0b3778e4c34eba7fcc356fa04a5afd954ccf484728e72c002764dd3c4","0x396797ae0ebcdb72ff1f96fd08b6128f78acc7417353f142f1a5facd425a33e6","0x454aa43d6546a6f62246826c16b7a49c6c704238c18802ef0d659922f23a573c","0x317ecb5bd19caa42a69f836d41556ebb0e0e00e1c6cd2dee230e6e6192612527","0xc879285db5ef0a6bce98021584d16f134c1dc0aed8cc988802c4f72ba6877ff6","0xecaa2d6f597608307e5084854854ba6dc1e69395e2abea14f2c6a2fa1d6faf9a","0x4dd69b69a568ff30ae439e2ded72fbd7f2e7aaa345836703663f155c749c5eed"],"transactionsRoot":"0xaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa","uncles":[],"withdrawals":[{"index":"0x1170","validatorIndex":"0x38c2c","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x66edfd65"},{"index":"0x1171","validatorIndex":"0x38c2d","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6cd228e4"},{"index":"0x1172","validatorIndex":"0x38c2e","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x77f3431b"},{"index":"0x1173","validatorIndex":"0x38c2f","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6b61f268"},{"index":"0x1174","validatorIndex":"0x38c30","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6e10bb21"},{"index":"0x1175","validatorIndex":"0x38c31","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6eb115a5"},{"index":"0x1176","validatorIndex":"0x38c32","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x7caead1d"},{"index":"0x1177","validatorIndex":"0x38c33","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x772c0ddf"},{"index":"0x1178","validatorIndex":"0x38c34","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x75930a95"},{"index":"0x1179","validatorIndex":"0x38c35","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x76a4db09"},{"index":"0x117a","validatorIndex":"0x38c36","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x7e692b27"},{"index":"0x117b","validatorIndex":"0x38c37","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x72038ae6"},{"index":"0x117c","validatorIndex":"0x38c38","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6ccce352"},{"index":"0x117d","validatorIndex":"0x38c39","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x79ef6898"},{"index":"0x117e","validatorIndex":"0x38c3a","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6d58977d"},{"index":"0x117f","validatorIndex":"0x38c3b","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x76f7d208"}],"withdrawalsRoot":"0xbe712c930a0665264b025ced87cc7839eef95a3cbc26dadc93e9e185a350ad28"} diff --git a/op-service/sources/testdata/data/headers/post-shanghai-bad-transactions_metadata.json b/op-service/sources/testdata/data/headers/post-shanghai-bad-transactions_metadata.json new file mode 100644 index 000000000000..03c892fc1ea3 --- /dev/null +++ b/op-service/sources/testdata/data/headers/post-shanghai-bad-transactions_metadata.json @@ -0,0 +1 @@ +{"name":"post-shanghai-bad-transactions","fail":true} diff --git a/op-service/sources/testdata/data/headers/post-shanghai-bad-withdrawals_data.json b/op-service/sources/testdata/data/headers/post-shanghai-bad-withdrawals_data.json new file mode 100644 index 000000000000..1e0a4d51d173 --- /dev/null +++ b/op-service/sources/testdata/data/headers/post-shanghai-bad-withdrawals_data.json @@ -0,0 +1 @@ +{"baseFeePerGas":"0x3fb7c357","difficulty":"0x0","extraData":"0x","gasLimit":"0x1c9c380","gasUsed":"0x18f759","hash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","logsBloom":"0x020010404000001a0000021000000080001100410000100001000010040200980220400000008806200200000100000000000000000000008000000400042000000050000040000112080808800002044000040004042008800480002000000000000002020020000042002400000820000080040000000010200010020010100101212050000008000000008000001010200c80000112010000438040020400000000202400000000002002a0210402000622010000000001700144000040000000002204000000c000410105024010000808000000002004002000000261000000822200200800881000000012500400400000000000000040010000800000","miner":"0x000095e79eac4d76aab57cb2c1f091d553b36ca0","mixHash":"0x5b53dc49cbab268ef9950b1d81b5e36a1b2f1b97aee1b7ff6e4db0e06c29a8b0","nonce":"0x0000000000000000","number":"0x84161e","parentHash":"0x72d92c1498e05952988d4e79a695928a6bcbd37239f8a1734051263b4d3504b8","receiptsRoot":"0xaff90ae18dcc35924a4bddb68d403b8b7812c10c3ea2a114f34105c87d75bcdb","sha3Uncles":"0x1dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347","size":"0x2a51","stateRoot":"0xc56738518b2c7854a640ae25996d2211c9ef0dd2e4dd9e59e9d9cacef39622da","timestamp":"0x64110a5c","totalDifficulty":"0xa4a470","transactions":["0x1e8f148a9aea7d8d16ea6e9446723b8f262e8bcd89c7c961d52046ebd43b4598","0xab5c870f4c367012bd763172afbfbe68fbf35336a66ae41aff3f2c9dbf4ea3f8","0xa81fd92b2d0f0bbd3cc355f869cca3243c98c5e2641db9ecf3eeabb3b13bff6a","0xa92c7b720c08c83f1a0ed7e4c163200e30a3a8c03fcc5a51e685ea20cd0cb577","0x6921b429ad2ec1e97d3457049ad2e893b5a0349beba47ca1c74a9540af75347a","0xf776b2da0b835dde05d0d8b76fd19385d61e7055036cf637f804b36dc94f2384","0x9a08d899cd14ebb930ed59fa774afdb88a22615b3a931e930931ea54d26dc0bc","0x0fe0d97e25d5eb11a33a3e8278584c3780941fc2675bdf8fc547cee3d1fd3b17","0xef47a60f57f177a683c723c658137efab66d311e1c5abbc4d74f653535144d03","0xe23a5b35faae5335adc5aca38c5d633b00438b798c2053104b8df48406c9b141","0xd8cea4ba619b317bc05d58534af73beec6c2548b31b24d4dc61c9bbd29cfa17a","0x79a4b9d90b02c768baaad305f266281213cc75062cbe99a13222cc0c4b509498","0x6790a3bbddbeb21fcb736a59b3775755051c3a6344d8390cf8ca27f2e8a814f0","0x87ec7ace5442db252b5751ffddd38dcb04b088d36b6b0e526ff25607a4293c81","0x40cb487ecffda94f97ce7fc0f7163f2f024235df2c8291169edc80dac063e6d0","0xb76bb3d88c9b30d927c45ccfcf8d5b0054411ac8501ad588822a7d04690cccf6","0x798ebe823209869347c08bd81e04fbf60e9bdfe44b1cc923215182d0cf3d4edb","0xbe68a7e02725f799a65ebb069ccc83a014ac7c40e4119bf7c220a2f6ddfee295","0xc90c3a72efe81331727fcce4b5bd4906066da314ca9a0b44023a6b09ea7e8114","0x619a6cbd43cde074d314c19623bd66d9fb1e13c158d7138775236f798dc1245e","0xca5a56cd77b9e5b0e79020cc6346edf205bc11e901984d805125f28c2e6686e6","0x999c9ddeed67c6ef6fbf02a6e977a6c1b68e18d24814e51643c7157b87a43e0a","0x47c8f5d0b3778e4c34eba7fcc356fa04a5afd954ccf484728e72c002764dd3c4","0x396797ae0ebcdb72ff1f96fd08b6128f78acc7417353f142f1a5facd425a33e6","0x454aa43d6546a6f62246826c16b7a49c6c704238c18802ef0d659922f23a573c","0x317ecb5bd19caa42a69f836d41556ebb0e0e00e1c6cd2dee230e6e6192612527","0xc879285db5ef0a6bce98021584d16f134c1dc0aed8cc988802c4f72ba6877ff6","0xecaa2d6f597608307e5084854854ba6dc1e69395e2abea14f2c6a2fa1d6faf9a","0x4dd69b69a568ff30ae439e2ded72fbd7f2e7aaa345836703663f155c749c5eed"],"transactionsRoot":"0x4a87d0cf5990b1c5bac631583e5965c2ba943858bebb2e07f74d0b697f73821a","uncles":[],"withdrawals":[{"index":"0x1170","validatorIndex":"0x38c2c","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x66edfd65"},{"index":"0x1171","validatorIndex":"0x38c2d","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6cd228e4"},{"index":"0x1172","validatorIndex":"0x38c2e","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x77f3431b"},{"index":"0x1173","validatorIndex":"0x38c2f","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6b61f268"},{"index":"0x1174","validatorIndex":"0x38c30","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6e10bb21"},{"index":"0x1175","validatorIndex":"0x38c31","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6eb115a5"},{"index":"0x1176","validatorIndex":"0x38c32","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x7caead1d"},{"index":"0x1177","validatorIndex":"0x38c33","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x772c0ddf"},{"index":"0x1178","validatorIndex":"0x38c34","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x75930a95"},{"index":"0x1179","validatorIndex":"0x38c35","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x76a4db09"},{"index":"0x117a","validatorIndex":"0x38c36","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x7e692b27"},{"index":"0x117b","validatorIndex":"0x38c37","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x72038ae6"},{"index":"0x117c","validatorIndex":"0x38c38","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6ccce352"},{"index":"0x117d","validatorIndex":"0x38c39","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x79ef6898"},{"index":"0x117e","validatorIndex":"0x38c3a","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6d58977d"},{"index":"0x117f","validatorIndex":"0x38c3b","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x76f7d208"}],"withdrawalsRoot":"0xaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa"} diff --git a/op-service/sources/testdata/data/headers/post-shanghai-bad-withdrawals_metadata.json b/op-service/sources/testdata/data/headers/post-shanghai-bad-withdrawals_metadata.json new file mode 100644 index 000000000000..16d4bd5bcc0e --- /dev/null +++ b/op-service/sources/testdata/data/headers/post-shanghai-bad-withdrawals_metadata.json @@ -0,0 +1 @@ +{"name":"post-shanghai-bad-withdrawals","fail":true} diff --git a/op-service/sources/testdata/data/headers/post-shanghai-success_data.json b/op-service/sources/testdata/data/headers/post-shanghai-success_data.json new file mode 100644 index 000000000000..bbaf3af1589d --- /dev/null +++ b/op-service/sources/testdata/data/headers/post-shanghai-success_data.json @@ -0,0 +1 @@ +{"baseFeePerGas":"0x3fb7c357","difficulty":"0x0","extraData":"0x","gasLimit":"0x1c9c380","gasUsed":"0x18f759","hash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","logsBloom":"0x020010404000001a0000021000000080001100410000100001000010040200980220400000008806200200000100000000000000000000008000000400042000000050000040000112080808800002044000040004042008800480002000000000000002020020000042002400000820000080040000000010200010020010100101212050000008000000008000001010200c80000112010000438040020400000000202400000000002002a0210402000622010000000001700144000040000000002204000000c000410105024010000808000000002004002000000261000000822200200800881000000012500400400000000000000040010000800000","miner":"0x000095e79eac4d76aab57cb2c1f091d553b36ca0","mixHash":"0x5b53dc49cbab268ef9950b1d81b5e36a1b2f1b97aee1b7ff6e4db0e06c29a8b0","nonce":"0x0000000000000000","number":"0x84161e","parentHash":"0x72d92c1498e05952988d4e79a695928a6bcbd37239f8a1734051263b4d3504b8","receiptsRoot":"0xaff90ae18dcc35924a4bddb68d403b8b7812c10c3ea2a114f34105c87d75bcdb","sha3Uncles":"0x1dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347","size":"0x2a51","stateRoot":"0xc56738518b2c7854a640ae25996d2211c9ef0dd2e4dd9e59e9d9cacef39622da","timestamp":"0x64110a5c","totalDifficulty":"0xa4a470","transactions":["0x1e8f148a9aea7d8d16ea6e9446723b8f262e8bcd89c7c961d52046ebd43b4598","0xab5c870f4c367012bd763172afbfbe68fbf35336a66ae41aff3f2c9dbf4ea3f8","0xa81fd92b2d0f0bbd3cc355f869cca3243c98c5e2641db9ecf3eeabb3b13bff6a","0xa92c7b720c08c83f1a0ed7e4c163200e30a3a8c03fcc5a51e685ea20cd0cb577","0x6921b429ad2ec1e97d3457049ad2e893b5a0349beba47ca1c74a9540af75347a","0xf776b2da0b835dde05d0d8b76fd19385d61e7055036cf637f804b36dc94f2384","0x9a08d899cd14ebb930ed59fa774afdb88a22615b3a931e930931ea54d26dc0bc","0x0fe0d97e25d5eb11a33a3e8278584c3780941fc2675bdf8fc547cee3d1fd3b17","0xef47a60f57f177a683c723c658137efab66d311e1c5abbc4d74f653535144d03","0xe23a5b35faae5335adc5aca38c5d633b00438b798c2053104b8df48406c9b141","0xd8cea4ba619b317bc05d58534af73beec6c2548b31b24d4dc61c9bbd29cfa17a","0x79a4b9d90b02c768baaad305f266281213cc75062cbe99a13222cc0c4b509498","0x6790a3bbddbeb21fcb736a59b3775755051c3a6344d8390cf8ca27f2e8a814f0","0x87ec7ace5442db252b5751ffddd38dcb04b088d36b6b0e526ff25607a4293c81","0x40cb487ecffda94f97ce7fc0f7163f2f024235df2c8291169edc80dac063e6d0","0xb76bb3d88c9b30d927c45ccfcf8d5b0054411ac8501ad588822a7d04690cccf6","0x798ebe823209869347c08bd81e04fbf60e9bdfe44b1cc923215182d0cf3d4edb","0xbe68a7e02725f799a65ebb069ccc83a014ac7c40e4119bf7c220a2f6ddfee295","0xc90c3a72efe81331727fcce4b5bd4906066da314ca9a0b44023a6b09ea7e8114","0x619a6cbd43cde074d314c19623bd66d9fb1e13c158d7138775236f798dc1245e","0xca5a56cd77b9e5b0e79020cc6346edf205bc11e901984d805125f28c2e6686e6","0x999c9ddeed67c6ef6fbf02a6e977a6c1b68e18d24814e51643c7157b87a43e0a","0x47c8f5d0b3778e4c34eba7fcc356fa04a5afd954ccf484728e72c002764dd3c4","0x396797ae0ebcdb72ff1f96fd08b6128f78acc7417353f142f1a5facd425a33e6","0x454aa43d6546a6f62246826c16b7a49c6c704238c18802ef0d659922f23a573c","0x317ecb5bd19caa42a69f836d41556ebb0e0e00e1c6cd2dee230e6e6192612527","0xc879285db5ef0a6bce98021584d16f134c1dc0aed8cc988802c4f72ba6877ff6","0xecaa2d6f597608307e5084854854ba6dc1e69395e2abea14f2c6a2fa1d6faf9a","0x4dd69b69a568ff30ae439e2ded72fbd7f2e7aaa345836703663f155c749c5eed"],"transactionsRoot":"0x4a87d0cf5990b1c5bac631583e5965c2ba943858bebb2e07f74d0b697f73821a","uncles":[],"withdrawals":[{"index":"0x1170","validatorIndex":"0x38c2c","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x66edfd65"},{"index":"0x1171","validatorIndex":"0x38c2d","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6cd228e4"},{"index":"0x1172","validatorIndex":"0x38c2e","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x77f3431b"},{"index":"0x1173","validatorIndex":"0x38c2f","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6b61f268"},{"index":"0x1174","validatorIndex":"0x38c30","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6e10bb21"},{"index":"0x1175","validatorIndex":"0x38c31","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6eb115a5"},{"index":"0x1176","validatorIndex":"0x38c32","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x7caead1d"},{"index":"0x1177","validatorIndex":"0x38c33","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x772c0ddf"},{"index":"0x1178","validatorIndex":"0x38c34","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x75930a95"},{"index":"0x1179","validatorIndex":"0x38c35","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x76a4db09"},{"index":"0x117a","validatorIndex":"0x38c36","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x7e692b27"},{"index":"0x117b","validatorIndex":"0x38c37","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x72038ae6"},{"index":"0x117c","validatorIndex":"0x38c38","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6ccce352"},{"index":"0x117d","validatorIndex":"0x38c39","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x79ef6898"},{"index":"0x117e","validatorIndex":"0x38c3a","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6d58977d"},{"index":"0x117f","validatorIndex":"0x38c3b","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x76f7d208"}],"withdrawalsRoot":"0xbe712c930a0665264b025ced87cc7839eef95a3cbc26dadc93e9e185a350ad28"} diff --git a/op-service/sources/testdata/data/headers/post-shanghai-success_metadata.json b/op-service/sources/testdata/data/headers/post-shanghai-success_metadata.json new file mode 100644 index 000000000000..d6179c6587c3 --- /dev/null +++ b/op-service/sources/testdata/data/headers/post-shanghai-success_metadata.json @@ -0,0 +1 @@ +{"name":"post-shanghai-success"} diff --git a/op-service/sources/testdata/data/headers/pre-shanghai-bad-receipts_data.json b/op-service/sources/testdata/data/headers/pre-shanghai-bad-receipts_data.json new file mode 100644 index 000000000000..98d9a3304032 --- /dev/null +++ b/op-service/sources/testdata/data/headers/pre-shanghai-bad-receipts_data.json @@ -0,0 +1 @@ +{"baseFeePerGas":"0x7ccf990f8","difficulty":"0x0","extraData":"0xd883010b02846765746888676f312e32302e31856c696e7578","gasLimit":"0x1c9c380","gasUsed":"0xa79638","hash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","logsBloom":"0xb034000008010014411408c080a0018440087220211154100005a1388807241142a2504080034a00111212a47f05008520200000280202a12800538cc06488486a0141989c7800c0c848011f02249661800e08449145b040a252d18082c009000641004052c80102000804ac10901c24032000980010438a01e50a90a0d8008c138c21204040000b20425000833041028000148124c2012d0aa8d1d0548301808228002015184090000224021040d68220100210220480420308455c382a40020130dc42502986080600000115034c0401c81828490410308005610048026b822e10b4228071ba00bdd20140621b2000c02012300808084181ac308200000011","miner":"0x0000000000000000000000000000000000000000","mixHash":"0x31f0c0305fc07a93b1a33da339c79aadbe8d9811c78d2b514cd18d64e1328f25","nonce":"0x0000000000000000","number":"0x840249","parentHash":"0x2303b55af4add799b19275a491b150c1a03075395f87a7856a4e3327595ed7df","receiptsRoot":"0xaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa","sha3Uncles":"0x1dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347","size":"0xea6d","stateRoot":"0xd12bf4cf3941cf48be329a939b13d3403d326841c69cdcc9a9c13ab2f227e904","timestamp":"0x640fdeb0","totalDifficulty":"0xa4a470","transactions":["0x39c666d9b5cec429accad7b0f94f789ca2ebeb5294b8b129c1b76f552daf57d3","0x2ca7289ab3738d17e0f5093bd96c97c06c9a2ea4c22fc84a6a7fbfda93ce55ee","0xb0085de1476530de3efc6928c4683e7c40f8fac18875f74cbcc47df159de17d9","0xe01c8631c86ded63af95b8dbc0c8aac5d31254c14d6ecb4cc51d98259d838e52","0x69414a126a6f07ab5e31ad2f9069fb986b7c490e096898473873e41ece6af783","0xa2fef1133ee726533c7f190f246fede123e3706a03933c1febc92618f90d2804","0x6585ec5c4c2bbf1f683f90f58e18f3b38d875e94457fe4cbb7bc5bf6581f83af","0x1db276b864fbf01dcf8cededf8d597553ecb0eb9438edfaf2f5bd0cc93297c66","0xcbe7ed31654af4e191ca53445b82de040ae2cd92459a3f951bdcce423d780f08","0x808ba5211f03cc78a732ff0f9383c6355e63c83ae8c6035ced2ba6f7c331dc63","0xdd66f1f26672849ef54c420210f479c9f0c46924d8e9f7b210981ffe8d3fac82","0x254abb2f8cdcffe9ef62ab924312a1e4142578db87e4f7c199fd35991e92f014","0xa7b7c654e7073b8043b680b7ffc95d3f2099abaa0b0578d6f954a2a7c99404e1","0x7ccdfa698c8acf47ab9316ed078eb40819ff575bcf612c6f59f29e7726df3f96","0xa0b035ef315824a6f6a6565fa8de27042ade3af9cf0583a36dea83d6e01bf2a8","0x1ebad7f3e8cb3543d4963686a94d99f61839f666831eab9c9c1b4711de11d3d9","0x501750278e91d8b5be1ccf60e793d4bbcd9b3bb3ccc518d3634a71caeac65f48","0xd80ff8af29ae163d5811ba511e60b3a87a279f677bb3872a0f1aa6d0a226e880","0x096acab3b3fe47b149d375782d1eb00b9fef7904076d60c54b3c197b04e6bf82","0xbe9d1738af74a22400591a9a808fb01a25ab41e2e56f202dd7251eb113e8ceeb","0x0834c720e55cccd97aaf4f8fb0cb66afb9881fb6a762c0f70473ec53f98a712e","0x51a0c33c9b37245b416575bdd2751c0d8a5d8bead49585ac427bfc873d4016af","0x531c25d51ccda59aa9ea82e85c99be9dd4e285af9b8973cbab9ac4a38e26e55a","0x93ac6c08d21cb1b61ff59e5e2d6fa3f9ad54008b0a66c669199050bef219f6e3","0x3792db6dd6285f409e4281951e9f78dad16c4a78072ff1c909dfadea5658d857","0xd2d51764c01e8c0a43fbe362704388df5bacf7e5e620c3864e242530ffb3e828","0x516b0227d9e64eb6e0de6862764d40f5376b5f12fec878436fea3479b4c36bb8","0x81b0abc78b82840adb666775b182a9e292f663b64bcd35004c04436ed3c8281c","0xd0287570d431d2baea96ecc81cb890e7f4f06ab5df02f9b4067768abca19acb5","0x76ddab2674369f34946c5fa2f05e2aa8566d86235b83e808e9b27bc106e04ac7","0x34a5c74011a2c8a00103bc91bfbfd94aa99cd569be69066e4bf64d188fe8714e","0x7b9730ead1b9f59b206d0ddea87be9383ba3fc7b496c7863b0cb847889b86617","0x77166ee0409ba86bd26e7c03ad1a927abaf5af8a8a37149e725cd37512091dd6","0x3c2b6c2ae505c5c36d5f316c1fcb5f54f7346ed35ae35c93462991ded7968a68","0xf99a792837e13827b5e0a8915fb59c760babc95d242feca99a5594e64ff6b6e2","0x522313f5d923f048ae5bd0b5595c1f4fc883bc0b3cf3cb0939d3fcf8b08c829c","0x471ceb0e85af594aa56deca54cb8198567b2afd8406722ea530077aaa6b641b3","0x3e9dca502e9039ae0c6d642f62e9562ff00010c6bfbb8234a6135712ba70dfda","0xc95cac67267f4accb9b5950316ac64772f7d082bed6b712c09cf2da0bdc237b7","0xfca28fdbd13fc16daf7aec7d4a2ad2c6b5f0b2a7b0fb1d9167c09b5e115ff26e","0xc73124ca798b2f7a5df2ea4d568efab2f41b135130ea5cc41d4bcb4b5c57d5bd","0x29abb76b5e7a5ce137bf9c22474d386eb58d249f43178d2b2e15c16dfdc5ca80","0x03e5ab25a58bd44fb9dd0c698b323eab8b8363479dfcbcbb16d0a0bd983880ae","0x3c8ee80ddea7fa2d2b75e44563c10c10756f598e8ad252a49c5d3e8a5c8e6cbf","0xaffa73b68bc7ab0c3f5e28377f5ca0a5df33c0a485f64dc094b7f6ae23353203","0xc66c9c66fbc8fe97fcc16506cde7a58689af1004a18c6171cfe763bcd94f50b2","0x80fec96707519172b53790610d5800cd09a4243aca9bacfa956c56337d06f820","0x61b33bfcf11214906dcdce7d7ed83ad82f38184c03ded07f7782059d02eeedea","0x5d4138d4e28a8327e506cb012346b1b38b65f615a2b991d35cf5d4de244b3e6d","0x875a142b6dfcf10ffb71a7afe0ce4672c047fc7e162ba0383390516d6334d45d","0x79b6df832bfbd04085d0b005a6e3ad8f00fc8717eed59280aa8107268b71e7e0","0xcb2fb25d268f65dc9312e89bd3c328c9847a3c9da282026793c54a745f825ab5","0xe483d4a36ad19fd5eacb7f6d9ad3ce080ad70ac673273e710f6e3d5acbc6559c","0x0564242c37d5013b671ef4864394cc0f3924c589f8aad64118223a9af2f164f6","0x48db358e80b278c3a46c2a166339797060a40f33984a5d974992cd9722139d5d","0x69d7758db91fae31fa35ecbed4d40897c5087f45dc796cd796b8ceead21f972e","0x2951478916ecd27a8e808d08f85be4bf2c0b0e0546f21f4e309145dd96eb8df1","0xaca9028cb5d55bbf71b7bff9884a9a3b0b38a575ffc8f8807ce345cf8bd298ef","0xc7f625a19ee41a1750eac9428b4394a9a2476b8ea2d31b4c2f9f5b4fcb86cae3","0x45499074aa521ac4151138f0aad969bcc2dfc1648d22ff8c42e51c74cb77414d","0x00b5b05c6d1a2eb8abe2c383da600516515e383fc8a29953bb6e6d167e9705b2","0x6fc411f24c7b4b8d821b45de32b9edc5ac998d1ac748a98abe8e983c6f39fc19"],"transactionsRoot":"0x1ad3212eca045505cfc4cacf675b5fa2e7dc7b9f9cee88191464f97d1c9fbca4","uncles":[]} diff --git a/op-service/sources/testdata/data/headers/pre-shanghai-bad-receipts_metadata.json b/op-service/sources/testdata/data/headers/pre-shanghai-bad-receipts_metadata.json new file mode 100644 index 000000000000..7deed556e8b2 --- /dev/null +++ b/op-service/sources/testdata/data/headers/pre-shanghai-bad-receipts_metadata.json @@ -0,0 +1 @@ +{"name":"pre-shanghai-bad-receipts","fail":true} diff --git a/op-service/sources/testdata/data/headers/pre-shanghai-bad-transactions_data.json b/op-service/sources/testdata/data/headers/pre-shanghai-bad-transactions_data.json new file mode 100644 index 000000000000..70866d79962f --- /dev/null +++ b/op-service/sources/testdata/data/headers/pre-shanghai-bad-transactions_data.json @@ -0,0 +1 @@ +{"baseFeePerGas":"0x7ccf990f8","difficulty":"0x0","extraData":"0xd883010b02846765746888676f312e32302e31856c696e7578","gasLimit":"0x1c9c380","gasUsed":"0xa79638","hash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","logsBloom":"0xb034000008010014411408c080a0018440087220211154100005a1388807241142a2504080034a00111212a47f05008520200000280202a12800538cc06488486a0141989c7800c0c848011f02249661800e08449145b040a252d18082c009000641004052c80102000804ac10901c24032000980010438a01e50a90a0d8008c138c21204040000b20425000833041028000148124c2012d0aa8d1d0548301808228002015184090000224021040d68220100210220480420308455c382a40020130dc42502986080600000115034c0401c81828490410308005610048026b822e10b4228071ba00bdd20140621b2000c02012300808084181ac308200000011","miner":"0x0000000000000000000000000000000000000000","mixHash":"0x31f0c0305fc07a93b1a33da339c79aadbe8d9811c78d2b514cd18d64e1328f25","nonce":"0x0000000000000000","number":"0x840249","parentHash":"0x2303b55af4add799b19275a491b150c1a03075395f87a7856a4e3327595ed7df","receiptsRoot":"0x99da71b17ae1929db912c3315ebe349d37f2bb600454616fdde0ee90d6dbc59e","sha3Uncles":"0x1dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347","size":"0xea6d","stateRoot":"0xd12bf4cf3941cf48be329a939b13d3403d326841c69cdcc9a9c13ab2f227e904","timestamp":"0x640fdeb0","totalDifficulty":"0xa4a470","transactions":["0x39c666d9b5cec429accad7b0f94f789ca2ebeb5294b8b129c1b76f552daf57d3","0x2ca7289ab3738d17e0f5093bd96c97c06c9a2ea4c22fc84a6a7fbfda93ce55ee","0xb0085de1476530de3efc6928c4683e7c40f8fac18875f74cbcc47df159de17d9","0xe01c8631c86ded63af95b8dbc0c8aac5d31254c14d6ecb4cc51d98259d838e52","0x69414a126a6f07ab5e31ad2f9069fb986b7c490e096898473873e41ece6af783","0xa2fef1133ee726533c7f190f246fede123e3706a03933c1febc92618f90d2804","0x6585ec5c4c2bbf1f683f90f58e18f3b38d875e94457fe4cbb7bc5bf6581f83af","0x1db276b864fbf01dcf8cededf8d597553ecb0eb9438edfaf2f5bd0cc93297c66","0xcbe7ed31654af4e191ca53445b82de040ae2cd92459a3f951bdcce423d780f08","0x808ba5211f03cc78a732ff0f9383c6355e63c83ae8c6035ced2ba6f7c331dc63","0xdd66f1f26672849ef54c420210f479c9f0c46924d8e9f7b210981ffe8d3fac82","0x254abb2f8cdcffe9ef62ab924312a1e4142578db87e4f7c199fd35991e92f014","0xa7b7c654e7073b8043b680b7ffc95d3f2099abaa0b0578d6f954a2a7c99404e1","0x7ccdfa698c8acf47ab9316ed078eb40819ff575bcf612c6f59f29e7726df3f96","0xa0b035ef315824a6f6a6565fa8de27042ade3af9cf0583a36dea83d6e01bf2a8","0x1ebad7f3e8cb3543d4963686a94d99f61839f666831eab9c9c1b4711de11d3d9","0x501750278e91d8b5be1ccf60e793d4bbcd9b3bb3ccc518d3634a71caeac65f48","0xd80ff8af29ae163d5811ba511e60b3a87a279f677bb3872a0f1aa6d0a226e880","0x096acab3b3fe47b149d375782d1eb00b9fef7904076d60c54b3c197b04e6bf82","0xbe9d1738af74a22400591a9a808fb01a25ab41e2e56f202dd7251eb113e8ceeb","0x0834c720e55cccd97aaf4f8fb0cb66afb9881fb6a762c0f70473ec53f98a712e","0x51a0c33c9b37245b416575bdd2751c0d8a5d8bead49585ac427bfc873d4016af","0x531c25d51ccda59aa9ea82e85c99be9dd4e285af9b8973cbab9ac4a38e26e55a","0x93ac6c08d21cb1b61ff59e5e2d6fa3f9ad54008b0a66c669199050bef219f6e3","0x3792db6dd6285f409e4281951e9f78dad16c4a78072ff1c909dfadea5658d857","0xd2d51764c01e8c0a43fbe362704388df5bacf7e5e620c3864e242530ffb3e828","0x516b0227d9e64eb6e0de6862764d40f5376b5f12fec878436fea3479b4c36bb8","0x81b0abc78b82840adb666775b182a9e292f663b64bcd35004c04436ed3c8281c","0xd0287570d431d2baea96ecc81cb890e7f4f06ab5df02f9b4067768abca19acb5","0x76ddab2674369f34946c5fa2f05e2aa8566d86235b83e808e9b27bc106e04ac7","0x34a5c74011a2c8a00103bc91bfbfd94aa99cd569be69066e4bf64d188fe8714e","0x7b9730ead1b9f59b206d0ddea87be9383ba3fc7b496c7863b0cb847889b86617","0x77166ee0409ba86bd26e7c03ad1a927abaf5af8a8a37149e725cd37512091dd6","0x3c2b6c2ae505c5c36d5f316c1fcb5f54f7346ed35ae35c93462991ded7968a68","0xf99a792837e13827b5e0a8915fb59c760babc95d242feca99a5594e64ff6b6e2","0x522313f5d923f048ae5bd0b5595c1f4fc883bc0b3cf3cb0939d3fcf8b08c829c","0x471ceb0e85af594aa56deca54cb8198567b2afd8406722ea530077aaa6b641b3","0x3e9dca502e9039ae0c6d642f62e9562ff00010c6bfbb8234a6135712ba70dfda","0xc95cac67267f4accb9b5950316ac64772f7d082bed6b712c09cf2da0bdc237b7","0xfca28fdbd13fc16daf7aec7d4a2ad2c6b5f0b2a7b0fb1d9167c09b5e115ff26e","0xc73124ca798b2f7a5df2ea4d568efab2f41b135130ea5cc41d4bcb4b5c57d5bd","0x29abb76b5e7a5ce137bf9c22474d386eb58d249f43178d2b2e15c16dfdc5ca80","0x03e5ab25a58bd44fb9dd0c698b323eab8b8363479dfcbcbb16d0a0bd983880ae","0x3c8ee80ddea7fa2d2b75e44563c10c10756f598e8ad252a49c5d3e8a5c8e6cbf","0xaffa73b68bc7ab0c3f5e28377f5ca0a5df33c0a485f64dc094b7f6ae23353203","0xc66c9c66fbc8fe97fcc16506cde7a58689af1004a18c6171cfe763bcd94f50b2","0x80fec96707519172b53790610d5800cd09a4243aca9bacfa956c56337d06f820","0x61b33bfcf11214906dcdce7d7ed83ad82f38184c03ded07f7782059d02eeedea","0x5d4138d4e28a8327e506cb012346b1b38b65f615a2b991d35cf5d4de244b3e6d","0x875a142b6dfcf10ffb71a7afe0ce4672c047fc7e162ba0383390516d6334d45d","0x79b6df832bfbd04085d0b005a6e3ad8f00fc8717eed59280aa8107268b71e7e0","0xcb2fb25d268f65dc9312e89bd3c328c9847a3c9da282026793c54a745f825ab5","0xe483d4a36ad19fd5eacb7f6d9ad3ce080ad70ac673273e710f6e3d5acbc6559c","0x0564242c37d5013b671ef4864394cc0f3924c589f8aad64118223a9af2f164f6","0x48db358e80b278c3a46c2a166339797060a40f33984a5d974992cd9722139d5d","0x69d7758db91fae31fa35ecbed4d40897c5087f45dc796cd796b8ceead21f972e","0x2951478916ecd27a8e808d08f85be4bf2c0b0e0546f21f4e309145dd96eb8df1","0xaca9028cb5d55bbf71b7bff9884a9a3b0b38a575ffc8f8807ce345cf8bd298ef","0xc7f625a19ee41a1750eac9428b4394a9a2476b8ea2d31b4c2f9f5b4fcb86cae3","0x45499074aa521ac4151138f0aad969bcc2dfc1648d22ff8c42e51c74cb77414d","0x00b5b05c6d1a2eb8abe2c383da600516515e383fc8a29953bb6e6d167e9705b2","0x6fc411f24c7b4b8d821b45de32b9edc5ac998d1ac748a98abe8e983c6f39fc19"],"transactionsRoot":"0xaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa","uncles":[]} diff --git a/op-service/sources/testdata/data/headers/pre-shanghai-bad-transactions_metadata.json b/op-service/sources/testdata/data/headers/pre-shanghai-bad-transactions_metadata.json new file mode 100644 index 000000000000..6ae97b067cf5 --- /dev/null +++ b/op-service/sources/testdata/data/headers/pre-shanghai-bad-transactions_metadata.json @@ -0,0 +1 @@ +{"name":"pre-shanghai-bad-transactions","fail":true} diff --git a/op-service/sources/testdata/data/headers/pre-shanghai-success_data.json b/op-service/sources/testdata/data/headers/pre-shanghai-success_data.json new file mode 100644 index 000000000000..c01c454a7321 --- /dev/null +++ b/op-service/sources/testdata/data/headers/pre-shanghai-success_data.json @@ -0,0 +1 @@ +{"baseFeePerGas":"0x7ccf990f8","difficulty":"0x0","extraData":"0xd883010b02846765746888676f312e32302e31856c696e7578","gasLimit":"0x1c9c380","gasUsed":"0xa79638","hash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","logsBloom":"0xb034000008010014411408c080a0018440087220211154100005a1388807241142a2504080034a00111212a47f05008520200000280202a12800538cc06488486a0141989c7800c0c848011f02249661800e08449145b040a252d18082c009000641004052c80102000804ac10901c24032000980010438a01e50a90a0d8008c138c21204040000b20425000833041028000148124c2012d0aa8d1d0548301808228002015184090000224021040d68220100210220480420308455c382a40020130dc42502986080600000115034c0401c81828490410308005610048026b822e10b4228071ba00bdd20140621b2000c02012300808084181ac308200000011","miner":"0x0000000000000000000000000000000000000000","mixHash":"0x31f0c0305fc07a93b1a33da339c79aadbe8d9811c78d2b514cd18d64e1328f25","nonce":"0x0000000000000000","number":"0x840249","parentHash":"0x2303b55af4add799b19275a491b150c1a03075395f87a7856a4e3327595ed7df","receiptsRoot":"0x99da71b17ae1929db912c3315ebe349d37f2bb600454616fdde0ee90d6dbc59e","sha3Uncles":"0x1dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347","size":"0xea6d","stateRoot":"0xd12bf4cf3941cf48be329a939b13d3403d326841c69cdcc9a9c13ab2f227e904","timestamp":"0x640fdeb0","totalDifficulty":"0xa4a470","transactions":["0x39c666d9b5cec429accad7b0f94f789ca2ebeb5294b8b129c1b76f552daf57d3","0x2ca7289ab3738d17e0f5093bd96c97c06c9a2ea4c22fc84a6a7fbfda93ce55ee","0xb0085de1476530de3efc6928c4683e7c40f8fac18875f74cbcc47df159de17d9","0xe01c8631c86ded63af95b8dbc0c8aac5d31254c14d6ecb4cc51d98259d838e52","0x69414a126a6f07ab5e31ad2f9069fb986b7c490e096898473873e41ece6af783","0xa2fef1133ee726533c7f190f246fede123e3706a03933c1febc92618f90d2804","0x6585ec5c4c2bbf1f683f90f58e18f3b38d875e94457fe4cbb7bc5bf6581f83af","0x1db276b864fbf01dcf8cededf8d597553ecb0eb9438edfaf2f5bd0cc93297c66","0xcbe7ed31654af4e191ca53445b82de040ae2cd92459a3f951bdcce423d780f08","0x808ba5211f03cc78a732ff0f9383c6355e63c83ae8c6035ced2ba6f7c331dc63","0xdd66f1f26672849ef54c420210f479c9f0c46924d8e9f7b210981ffe8d3fac82","0x254abb2f8cdcffe9ef62ab924312a1e4142578db87e4f7c199fd35991e92f014","0xa7b7c654e7073b8043b680b7ffc95d3f2099abaa0b0578d6f954a2a7c99404e1","0x7ccdfa698c8acf47ab9316ed078eb40819ff575bcf612c6f59f29e7726df3f96","0xa0b035ef315824a6f6a6565fa8de27042ade3af9cf0583a36dea83d6e01bf2a8","0x1ebad7f3e8cb3543d4963686a94d99f61839f666831eab9c9c1b4711de11d3d9","0x501750278e91d8b5be1ccf60e793d4bbcd9b3bb3ccc518d3634a71caeac65f48","0xd80ff8af29ae163d5811ba511e60b3a87a279f677bb3872a0f1aa6d0a226e880","0x096acab3b3fe47b149d375782d1eb00b9fef7904076d60c54b3c197b04e6bf82","0xbe9d1738af74a22400591a9a808fb01a25ab41e2e56f202dd7251eb113e8ceeb","0x0834c720e55cccd97aaf4f8fb0cb66afb9881fb6a762c0f70473ec53f98a712e","0x51a0c33c9b37245b416575bdd2751c0d8a5d8bead49585ac427bfc873d4016af","0x531c25d51ccda59aa9ea82e85c99be9dd4e285af9b8973cbab9ac4a38e26e55a","0x93ac6c08d21cb1b61ff59e5e2d6fa3f9ad54008b0a66c669199050bef219f6e3","0x3792db6dd6285f409e4281951e9f78dad16c4a78072ff1c909dfadea5658d857","0xd2d51764c01e8c0a43fbe362704388df5bacf7e5e620c3864e242530ffb3e828","0x516b0227d9e64eb6e0de6862764d40f5376b5f12fec878436fea3479b4c36bb8","0x81b0abc78b82840adb666775b182a9e292f663b64bcd35004c04436ed3c8281c","0xd0287570d431d2baea96ecc81cb890e7f4f06ab5df02f9b4067768abca19acb5","0x76ddab2674369f34946c5fa2f05e2aa8566d86235b83e808e9b27bc106e04ac7","0x34a5c74011a2c8a00103bc91bfbfd94aa99cd569be69066e4bf64d188fe8714e","0x7b9730ead1b9f59b206d0ddea87be9383ba3fc7b496c7863b0cb847889b86617","0x77166ee0409ba86bd26e7c03ad1a927abaf5af8a8a37149e725cd37512091dd6","0x3c2b6c2ae505c5c36d5f316c1fcb5f54f7346ed35ae35c93462991ded7968a68","0xf99a792837e13827b5e0a8915fb59c760babc95d242feca99a5594e64ff6b6e2","0x522313f5d923f048ae5bd0b5595c1f4fc883bc0b3cf3cb0939d3fcf8b08c829c","0x471ceb0e85af594aa56deca54cb8198567b2afd8406722ea530077aaa6b641b3","0x3e9dca502e9039ae0c6d642f62e9562ff00010c6bfbb8234a6135712ba70dfda","0xc95cac67267f4accb9b5950316ac64772f7d082bed6b712c09cf2da0bdc237b7","0xfca28fdbd13fc16daf7aec7d4a2ad2c6b5f0b2a7b0fb1d9167c09b5e115ff26e","0xc73124ca798b2f7a5df2ea4d568efab2f41b135130ea5cc41d4bcb4b5c57d5bd","0x29abb76b5e7a5ce137bf9c22474d386eb58d249f43178d2b2e15c16dfdc5ca80","0x03e5ab25a58bd44fb9dd0c698b323eab8b8363479dfcbcbb16d0a0bd983880ae","0x3c8ee80ddea7fa2d2b75e44563c10c10756f598e8ad252a49c5d3e8a5c8e6cbf","0xaffa73b68bc7ab0c3f5e28377f5ca0a5df33c0a485f64dc094b7f6ae23353203","0xc66c9c66fbc8fe97fcc16506cde7a58689af1004a18c6171cfe763bcd94f50b2","0x80fec96707519172b53790610d5800cd09a4243aca9bacfa956c56337d06f820","0x61b33bfcf11214906dcdce7d7ed83ad82f38184c03ded07f7782059d02eeedea","0x5d4138d4e28a8327e506cb012346b1b38b65f615a2b991d35cf5d4de244b3e6d","0x875a142b6dfcf10ffb71a7afe0ce4672c047fc7e162ba0383390516d6334d45d","0x79b6df832bfbd04085d0b005a6e3ad8f00fc8717eed59280aa8107268b71e7e0","0xcb2fb25d268f65dc9312e89bd3c328c9847a3c9da282026793c54a745f825ab5","0xe483d4a36ad19fd5eacb7f6d9ad3ce080ad70ac673273e710f6e3d5acbc6559c","0x0564242c37d5013b671ef4864394cc0f3924c589f8aad64118223a9af2f164f6","0x48db358e80b278c3a46c2a166339797060a40f33984a5d974992cd9722139d5d","0x69d7758db91fae31fa35ecbed4d40897c5087f45dc796cd796b8ceead21f972e","0x2951478916ecd27a8e808d08f85be4bf2c0b0e0546f21f4e309145dd96eb8df1","0xaca9028cb5d55bbf71b7bff9884a9a3b0b38a575ffc8f8807ce345cf8bd298ef","0xc7f625a19ee41a1750eac9428b4394a9a2476b8ea2d31b4c2f9f5b4fcb86cae3","0x45499074aa521ac4151138f0aad969bcc2dfc1648d22ff8c42e51c74cb77414d","0x00b5b05c6d1a2eb8abe2c383da600516515e383fc8a29953bb6e6d167e9705b2","0x6fc411f24c7b4b8d821b45de32b9edc5ac998d1ac748a98abe8e983c6f39fc19"],"transactionsRoot":"0x1ad3212eca045505cfc4cacf675b5fa2e7dc7b9f9cee88191464f97d1c9fbca4","uncles":[]} diff --git a/op-service/sources/testdata/data/headers/pre-shanghai-success_metadata.json b/op-service/sources/testdata/data/headers/pre-shanghai-success_metadata.json new file mode 100644 index 000000000000..87c9db722e78 --- /dev/null +++ b/op-service/sources/testdata/data/headers/pre-shanghai-success_metadata.json @@ -0,0 +1 @@ +{"name":"pre-shanghai-success"} diff --git a/op-service/sources/testdata/gen.sh b/op-service/sources/testdata/gen.sh new file mode 100644 index 000000000000..d3a42f2440d6 --- /dev/null +++ b/op-service/sources/testdata/gen.sh @@ -0,0 +1,81 @@ +#!/bin/bash + +export ETH_RPC_URL=https://ethereum-goerli-rpc.allthatnode.com + +jq_mutate() { + local name="$1" + jq -c "$2" "$name" > "$name.tmp" && mv "$name.tmp" "$name" +} + +success_case() { + # just format the files + jq_mutate "$1" . + jq_mutate "$2" . +} + +bad_receipts_root() { + local data_file="$1" + local metadata_file="$2" + jq_mutate "$data_file" '. + {"receiptsRoot": "0xaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa"}' + jq_mutate "$metadata_file" '. + {"fail": true}' +} + +bad_withdrawals_root() { + local data_file="$1" + local metadata_file="$2" + jq_mutate "$data_file" '. + {"withdrawalsRoot": "0xaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa"}' + jq_mutate "$metadata_file" '. + {"fail": true}' +} + +bad_transactions_root() { + local data_file="$1" + local metadata_file="$2" + jq_mutate "$data_file" '. + {"transactionsRoot": "0xaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa"}' + jq_mutate "$metadata_file" '. + {"fail": true}' +} + +generate_test_vector() { + local name="$1" + local blockhash="$2" + local fulltxs="$3" + local mutation_func="$4" + + local metadata_file="" + local data_file="" + + if [[ "$fulltxs" == true ]]; then + metadata_file="data/blocks/${name}_metadata.json" + data_file="data/blocks/${name}_data.json" + else + metadata_file="data/headers/${name}_metadata.json" + data_file="data/headers/${name}_data.json" + fi + + echo "{\"name\": \"$name\"}" > "$metadata_file" + + cast rpc eth_getBlockByHash $blockhash $fulltxs > $data_file + + # Mutate data using the provided function + $mutation_func "$data_file" $metadata_file +} + +mkdir -p data/blocks +mkdir -p data/headers + +# Blocks +generate_test_vector "pre-shanghai-success" "0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663" true success_case +generate_test_vector "pre-shanghai-bad-transactions" "0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663" true bad_transactions_root +generate_test_vector "pre-shanghai-bad-receipts" "0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663" true bad_receipts_root +generate_test_vector "post-shanghai-success" "0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452" true success_case +generate_test_vector "post-shanghai-bad-withdrawals" "0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452" true bad_withdrawals_root +generate_test_vector "post-shanghai-bad-transactions" "0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452" true bad_transactions_root +generate_test_vector "post-shanghai-bad-receipts" "0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452" true bad_receipts_root + +# Headers +generate_test_vector "pre-shanghai-success" "0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663" false success_case +generate_test_vector "pre-shanghai-bad-transactions" "0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663" false bad_transactions_root +generate_test_vector "pre-shanghai-bad-receipts" "0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663" false bad_receipts_root +generate_test_vector "post-shanghai-success" "0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452" false success_case +generate_test_vector "post-shanghai-bad-withdrawals" "0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452" false bad_withdrawals_root +generate_test_vector "post-shanghai-bad-transactions" "0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452" false bad_transactions_root +generate_test_vector "post-shanghai-bad-receipts" "0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452" false bad_receipts_root diff --git a/op-service/sources/types.go b/op-service/sources/types.go index a3a9bbc949e7..2655986bb2ca 100644 --- a/op-service/sources/types.go +++ b/op-service/sources/types.go @@ -181,16 +181,38 @@ func (hdr *rpcHeader) Info(trustCache bool, mustBePostMerge bool) (eth.BlockInfo type rpcBlock struct { rpcHeader Transactions []*types.Transaction `json:"transactions"` - Withdrawals *eth.Withdrawals `json:"withdrawals,omitempty"` + Withdrawals *types.Withdrawals `json:"withdrawals,omitempty"` } func (block *rpcBlock) verify() error { if computed := block.computeBlockHash(); computed != block.Hash { return fmt.Errorf("failed to verify block hash: computed %s but RPC said %s", computed, block.Hash) } + for i, tx := range block.Transactions { + if tx == nil { + return fmt.Errorf("block tx %d is null", i) + } + } if computed := types.DeriveSha(types.Transactions(block.Transactions), trie.NewStackTrie(nil)); block.TxHash != computed { return fmt.Errorf("failed to verify transactions list: computed %s but RPC said %s", computed, block.TxHash) } + if block.WithdrawalsRoot != nil { + if block.Withdrawals == nil { + return fmt.Errorf("expected withdrawals") + } + for i, w := range *block.Withdrawals { + if w == nil { + return fmt.Errorf("block withdrawal %d is null", i) + } + } + if computed := types.DeriveSha(*block.Withdrawals, trie.NewStackTrie(nil)); *block.WithdrawalsRoot != computed { + return fmt.Errorf("failed to verify withdrawals list: computed %s but RPC said %s", computed, block.WithdrawalsRoot) + } + } else { + if block.Withdrawals != nil { + return fmt.Errorf("expected no withdrawals due to missing withdrawals-root, but got %d", len(*block.Withdrawals)) + } + } return nil } diff --git a/op-service/sources/types_test.go b/op-service/sources/types_test.go index 87bba2a98dd7..50ca8423c796 100644 --- a/op-service/sources/types_test.go +++ b/op-service/sources/types_test.go @@ -1,35 +1,75 @@ package sources import ( + "embed" "encoding/json" + "strings" "testing" "github.com/stretchr/testify/require" ) -func TestBlockJSON(t *testing.T) { - testCases := []struct { - Name string - OK bool - Data string - }{ - {Name: "pre-Shanghai good tx", OK: true, Data: `{"number":"0x840249","hash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","transactions":["0x39c666d9b5cec429accad7b0f94f789ca2ebeb5294b8b129c1b76f552daf57d3","0x2ca7289ab3738d17e0f5093bd96c97c06c9a2ea4c22fc84a6a7fbfda93ce55ee","0xb0085de1476530de3efc6928c4683e7c40f8fac18875f74cbcc47df159de17d9","0xe01c8631c86ded63af95b8dbc0c8aac5d31254c14d6ecb4cc51d98259d838e52","0x69414a126a6f07ab5e31ad2f9069fb986b7c490e096898473873e41ece6af783","0xa2fef1133ee726533c7f190f246fede123e3706a03933c1febc92618f90d2804","0x6585ec5c4c2bbf1f683f90f58e18f3b38d875e94457fe4cbb7bc5bf6581f83af","0x1db276b864fbf01dcf8cededf8d597553ecb0eb9438edfaf2f5bd0cc93297c66","0xcbe7ed31654af4e191ca53445b82de040ae2cd92459a3f951bdcce423d780f08","0x808ba5211f03cc78a732ff0f9383c6355e63c83ae8c6035ced2ba6f7c331dc63","0xdd66f1f26672849ef54c420210f479c9f0c46924d8e9f7b210981ffe8d3fac82","0x254abb2f8cdcffe9ef62ab924312a1e4142578db87e4f7c199fd35991e92f014","0xa7b7c654e7073b8043b680b7ffc95d3f2099abaa0b0578d6f954a2a7c99404e1","0x7ccdfa698c8acf47ab9316ed078eb40819ff575bcf612c6f59f29e7726df3f96","0xa0b035ef315824a6f6a6565fa8de27042ade3af9cf0583a36dea83d6e01bf2a8","0x1ebad7f3e8cb3543d4963686a94d99f61839f666831eab9c9c1b4711de11d3d9","0x501750278e91d8b5be1ccf60e793d4bbcd9b3bb3ccc518d3634a71caeac65f48","0xd80ff8af29ae163d5811ba511e60b3a87a279f677bb3872a0f1aa6d0a226e880","0x096acab3b3fe47b149d375782d1eb00b9fef7904076d60c54b3c197b04e6bf82","0xbe9d1738af74a22400591a9a808fb01a25ab41e2e56f202dd7251eb113e8ceeb","0x0834c720e55cccd97aaf4f8fb0cb66afb9881fb6a762c0f70473ec53f98a712e","0x51a0c33c9b37245b416575bdd2751c0d8a5d8bead49585ac427bfc873d4016af","0x531c25d51ccda59aa9ea82e85c99be9dd4e285af9b8973cbab9ac4a38e26e55a","0x93ac6c08d21cb1b61ff59e5e2d6fa3f9ad54008b0a66c669199050bef219f6e3","0x3792db6dd6285f409e4281951e9f78dad16c4a78072ff1c909dfadea5658d857","0xd2d51764c01e8c0a43fbe362704388df5bacf7e5e620c3864e242530ffb3e828","0x516b0227d9e64eb6e0de6862764d40f5376b5f12fec878436fea3479b4c36bb8","0x81b0abc78b82840adb666775b182a9e292f663b64bcd35004c04436ed3c8281c","0xd0287570d431d2baea96ecc81cb890e7f4f06ab5df02f9b4067768abca19acb5","0x76ddab2674369f34946c5fa2f05e2aa8566d86235b83e808e9b27bc106e04ac7","0x34a5c74011a2c8a00103bc91bfbfd94aa99cd569be69066e4bf64d188fe8714e","0x7b9730ead1b9f59b206d0ddea87be9383ba3fc7b496c7863b0cb847889b86617","0x77166ee0409ba86bd26e7c03ad1a927abaf5af8a8a37149e725cd37512091dd6","0x3c2b6c2ae505c5c36d5f316c1fcb5f54f7346ed35ae35c93462991ded7968a68","0xf99a792837e13827b5e0a8915fb59c760babc95d242feca99a5594e64ff6b6e2","0x522313f5d923f048ae5bd0b5595c1f4fc883bc0b3cf3cb0939d3fcf8b08c829c","0x471ceb0e85af594aa56deca54cb8198567b2afd8406722ea530077aaa6b641b3","0x3e9dca502e9039ae0c6d642f62e9562ff00010c6bfbb8234a6135712ba70dfda","0xc95cac67267f4accb9b5950316ac64772f7d082bed6b712c09cf2da0bdc237b7","0xfca28fdbd13fc16daf7aec7d4a2ad2c6b5f0b2a7b0fb1d9167c09b5e115ff26e","0xc73124ca798b2f7a5df2ea4d568efab2f41b135130ea5cc41d4bcb4b5c57d5bd","0x29abb76b5e7a5ce137bf9c22474d386eb58d249f43178d2b2e15c16dfdc5ca80","0x03e5ab25a58bd44fb9dd0c698b323eab8b8363479dfcbcbb16d0a0bd983880ae","0x3c8ee80ddea7fa2d2b75e44563c10c10756f598e8ad252a49c5d3e8a5c8e6cbf","0xaffa73b68bc7ab0c3f5e28377f5ca0a5df33c0a485f64dc094b7f6ae23353203","0xc66c9c66fbc8fe97fcc16506cde7a58689af1004a18c6171cfe763bcd94f50b2","0x80fec96707519172b53790610d5800cd09a4243aca9bacfa956c56337d06f820","0x61b33bfcf11214906dcdce7d7ed83ad82f38184c03ded07f7782059d02eeedea","0x5d4138d4e28a8327e506cb012346b1b38b65f615a2b991d35cf5d4de244b3e6d","0x875a142b6dfcf10ffb71a7afe0ce4672c047fc7e162ba0383390516d6334d45d","0x79b6df832bfbd04085d0b005a6e3ad8f00fc8717eed59280aa8107268b71e7e0","0xcb2fb25d268f65dc9312e89bd3c328c9847a3c9da282026793c54a745f825ab5","0xe483d4a36ad19fd5eacb7f6d9ad3ce080ad70ac673273e710f6e3d5acbc6559c","0x0564242c37d5013b671ef4864394cc0f3924c589f8aad64118223a9af2f164f6","0x48db358e80b278c3a46c2a166339797060a40f33984a5d974992cd9722139d5d","0x69d7758db91fae31fa35ecbed4d40897c5087f45dc796cd796b8ceead21f972e","0x2951478916ecd27a8e808d08f85be4bf2c0b0e0546f21f4e309145dd96eb8df1","0xaca9028cb5d55bbf71b7bff9884a9a3b0b38a575ffc8f8807ce345cf8bd298ef","0xc7f625a19ee41a1750eac9428b4394a9a2476b8ea2d31b4c2f9f5b4fcb86cae3","0x45499074aa521ac4151138f0aad969bcc2dfc1648d22ff8c42e51c74cb77414d","0x00b5b05c6d1a2eb8abe2c383da600516515e383fc8a29953bb6e6d167e9705b2","0x6fc411f24c7b4b8d821b45de32b9edc5ac998d1ac748a98abe8e983c6f39fc19"],"difficulty":"0x0","extraData":"0xd883010b02846765746888676f312e32302e31856c696e7578","gasLimit":"0x1c9c380","gasUsed":"0xa79638","logsBloom":"0xb034000008010014411408c080a0018440087220211154100005a1388807241142a2504080034a00111212a47f05008520200000280202a12800538cc06488486a0141989c7800c0c848011f02249661800e08449145b040a252d18082c009000641004052c80102000804ac10901c24032000980010438a01e50a90a0d8008c138c21204040000b20425000833041028000148124c2012d0aa8d1d0548301808228002015184090000224021040d68220100210220480420308455c382a40020130dc42502986080600000115034c0401c81828490410308005610048026b822e10b4228071ba00bdd20140621b2000c02012300808084181ac308200000011","miner":"0x0000000000000000000000000000000000000000","mixHash":"0x31f0c0305fc07a93b1a33da339c79aadbe8d9811c78d2b514cd18d64e1328f25","nonce":"0x0000000000000000","parentHash":"0x2303b55af4add799b19275a491b150c1a03075395f87a7856a4e3327595ed7df","receiptsRoot":"0x99da71b17ae1929db912c3315ebe349d37f2bb600454616fdde0ee90d6dbc59e","sha3Uncles":"0x1dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347","size":"0xea6d","stateRoot":"0xd12bf4cf3941cf48be329a939b13d3403d326841c69cdcc9a9c13ab2f227e904","timestamp":"0x640fdeb0","totalDifficulty":"0xa4a470","transactionsRoot":"0x1ad3212eca045505cfc4cacf675b5fa2e7dc7b9f9cee88191464f97d1c9fbca4","uncles":[],"baseFeePerGas":"0x7ccf990f8"} -`}, - {Name: "pre-Shanghai bad receipts root", OK: false, Data: `{"number":"0x840249","hash":"0x9ef7cd2241202b919a0e51240818a8666c73f7ce4b908931e3ae6d26d30f7663","transactions":["0x39c666d9b5cec429accad7b0f94f789ca2ebeb5294b8b129c1b76f552daf57d3","0x2ca7289ab3738d17e0f5093bd96c97c06c9a2ea4c22fc84a6a7fbfda93ce55ee","0xb0085de1476530de3efc6928c4683e7c40f8fac18875f74cbcc47df159de17d9","0xe01c8631c86ded63af95b8dbc0c8aac5d31254c14d6ecb4cc51d98259d838e52","0x69414a126a6f07ab5e31ad2f9069fb986b7c490e096898473873e41ece6af783","0xa2fef1133ee726533c7f190f246fede123e3706a03933c1febc92618f90d2804","0x6585ec5c4c2bbf1f683f90f58e18f3b38d875e94457fe4cbb7bc5bf6581f83af","0x1db276b864fbf01dcf8cededf8d597553ecb0eb9438edfaf2f5bd0cc93297c66","0xcbe7ed31654af4e191ca53445b82de040ae2cd92459a3f951bdcce423d780f08","0x808ba5211f03cc78a732ff0f9383c6355e63c83ae8c6035ced2ba6f7c331dc63","0xdd66f1f26672849ef54c420210f479c9f0c46924d8e9f7b210981ffe8d3fac82","0x254abb2f8cdcffe9ef62ab924312a1e4142578db87e4f7c199fd35991e92f014","0xa7b7c654e7073b8043b680b7ffc95d3f2099abaa0b0578d6f954a2a7c99404e1","0x7ccdfa698c8acf47ab9316ed078eb40819ff575bcf612c6f59f29e7726df3f96","0xa0b035ef315824a6f6a6565fa8de27042ade3af9cf0583a36dea83d6e01bf2a8","0x1ebad7f3e8cb3543d4963686a94d99f61839f666831eab9c9c1b4711de11d3d9","0x501750278e91d8b5be1ccf60e793d4bbcd9b3bb3ccc518d3634a71caeac65f48","0xd80ff8af29ae163d5811ba511e60b3a87a279f677bb3872a0f1aa6d0a226e880","0x096acab3b3fe47b149d375782d1eb00b9fef7904076d60c54b3c197b04e6bf82","0xbe9d1738af74a22400591a9a808fb01a25ab41e2e56f202dd7251eb113e8ceeb","0x0834c720e55cccd97aaf4f8fb0cb66afb9881fb6a762c0f70473ec53f98a712e","0x51a0c33c9b37245b416575bdd2751c0d8a5d8bead49585ac427bfc873d4016af","0x531c25d51ccda59aa9ea82e85c99be9dd4e285af9b8973cbab9ac4a38e26e55a","0x93ac6c08d21cb1b61ff59e5e2d6fa3f9ad54008b0a66c669199050bef219f6e3","0x3792db6dd6285f409e4281951e9f78dad16c4a78072ff1c909dfadea5658d857","0xd2d51764c01e8c0a43fbe362704388df5bacf7e5e620c3864e242530ffb3e828","0x516b0227d9e64eb6e0de6862764d40f5376b5f12fec878436fea3479b4c36bb8","0x81b0abc78b82840adb666775b182a9e292f663b64bcd35004c04436ed3c8281c","0xd0287570d431d2baea96ecc81cb890e7f4f06ab5df02f9b4067768abca19acb5","0x76ddab2674369f34946c5fa2f05e2aa8566d86235b83e808e9b27bc106e04ac7","0x34a5c74011a2c8a00103bc91bfbfd94aa99cd569be69066e4bf64d188fe8714e","0x7b9730ead1b9f59b206d0ddea87be9383ba3fc7b496c7863b0cb847889b86617","0x77166ee0409ba86bd26e7c03ad1a927abaf5af8a8a37149e725cd37512091dd6","0x3c2b6c2ae505c5c36d5f316c1fcb5f54f7346ed35ae35c93462991ded7968a68","0xf99a792837e13827b5e0a8915fb59c760babc95d242feca99a5594e64ff6b6e2","0x522313f5d923f048ae5bd0b5595c1f4fc883bc0b3cf3cb0939d3fcf8b08c829c","0x471ceb0e85af594aa56deca54cb8198567b2afd8406722ea530077aaa6b641b3","0x3e9dca502e9039ae0c6d642f62e9562ff00010c6bfbb8234a6135712ba70dfda","0xc95cac67267f4accb9b5950316ac64772f7d082bed6b712c09cf2da0bdc237b7","0xfca28fdbd13fc16daf7aec7d4a2ad2c6b5f0b2a7b0fb1d9167c09b5e115ff26e","0xc73124ca798b2f7a5df2ea4d568efab2f41b135130ea5cc41d4bcb4b5c57d5bd","0x29abb76b5e7a5ce137bf9c22474d386eb58d249f43178d2b2e15c16dfdc5ca80","0x03e5ab25a58bd44fb9dd0c698b323eab8b8363479dfcbcbb16d0a0bd983880ae","0x3c8ee80ddea7fa2d2b75e44563c10c10756f598e8ad252a49c5d3e8a5c8e6cbf","0xaffa73b68bc7ab0c3f5e28377f5ca0a5df33c0a485f64dc094b7f6ae23353203","0xc66c9c66fbc8fe97fcc16506cde7a58689af1004a18c6171cfe763bcd94f50b2","0x80fec96707519172b53790610d5800cd09a4243aca9bacfa956c56337d06f820","0x61b33bfcf11214906dcdce7d7ed83ad82f38184c03ded07f7782059d02eeedea","0x5d4138d4e28a8327e506cb012346b1b38b65f615a2b991d35cf5d4de244b3e6d","0x875a142b6dfcf10ffb71a7afe0ce4672c047fc7e162ba0383390516d6334d45d","0x79b6df832bfbd04085d0b005a6e3ad8f00fc8717eed59280aa8107268b71e7e0","0xcb2fb25d268f65dc9312e89bd3c328c9847a3c9da282026793c54a745f825ab5","0xe483d4a36ad19fd5eacb7f6d9ad3ce080ad70ac673273e710f6e3d5acbc6559c","0x0564242c37d5013b671ef4864394cc0f3924c589f8aad64118223a9af2f164f6","0x48db358e80b278c3a46c2a166339797060a40f33984a5d974992cd9722139d5d","0x69d7758db91fae31fa35ecbed4d40897c5087f45dc796cd796b8ceead21f972e","0x2951478916ecd27a8e808d08f85be4bf2c0b0e0546f21f4e309145dd96eb8df1","0xaca9028cb5d55bbf71b7bff9884a9a3b0b38a575ffc8f8807ce345cf8bd298ef","0xc7f625a19ee41a1750eac9428b4394a9a2476b8ea2d31b4c2f9f5b4fcb86cae3","0x45499074aa521ac4151138f0aad969bcc2dfc1648d22ff8c42e51c74cb77414d","0x00b5b05c6d1a2eb8abe2c383da600516515e383fc8a29953bb6e6d167e9705b2","0x6fc411f24c7b4b8d821b45de32b9edc5ac998d1ac748a98abe8e983c6f39fc19"],"difficulty":"0x0","extraData":"0xd883010b02846765746888676f312e32302e31856c696e7578","gasLimit":"0x1c9c380","gasUsed":"0xa79638","logsBloom":"0xb034000008010014411408c080a0018440087220211154100005a1388807241142a2504080034a00111212a47f05008520200000280202a12800538cc06488486a0141989c7800c0c848011f02249661800e08449145b040a252d18082c009000641004052c80102000804ac10901c24032000980010438a01e50a90a0d8008c138c21204040000b20425000833041028000148124c2012d0aa8d1d0548301808228002015184090000224021040d68220100210220480420308455c382a40020130dc42502986080600000115034c0401c81828490410308005610048026b822e10b4228071ba00bdd20140621b2000c02012300808084181ac308200000011","miner":"0x0000000000000000000000000000000000000000","mixHash":"0x31f0c0305fc07a93b1a33da339c79aadbe8d9811c78d2b514cd18d64e1328f25","nonce":"0x0000000000000000","parentHash":"0x2303b55af4add799b19275a491b150c1a03075395f87a7856a4e3327595ed7df","receiptsRoot":"0x99da71b17ae1929db912c3315ebe349d37f2bb600454616fdde0ee90d6dbc59f","sha3Uncles":"0x1dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347","size":"0xea6d","stateRoot":"0xd12bf4cf3941cf48be329a939b13d3403d326841c69cdcc9a9c13ab2f227e904","timestamp":"0x640fdeb0","totalDifficulty":"0xa4a470","transactionsRoot":"0x1ad3212eca045505cfc4cacf675b5fa2e7dc7b9f9cee88191464f97d1c9fbca4","uncles":[],"baseFeePerGas":"0x7ccf990f8"} -`}, - {Name: "Shanghai good tx", OK: true, Data: `{"baseFeePerGas":"0x3fb7c357","difficulty":"0x0","extraData":"0x","gasLimit":"0x1c9c380","gasUsed":"0x18f759","hash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","logsBloom":"0x020010404000001a0000021000000080001100410000100001000010040200980220400000008806200200000100000000000000000000008000000400042000000050000040000112080808800002044000040004042008800480002000000000000002020020000042002400000820000080040000000010200010020010100101212050000008000000008000001010200c80000112010000438040020400000000202400000000002002a0210402000622010000000001700144000040000000002204000000c000410105024010000808000000002004002000000261000000822200200800881000000012500400400000000000000040010000800000","miner":"0x000095e79eac4d76aab57cb2c1f091d553b36ca0","mixHash":"0x5b53dc49cbab268ef9950b1d81b5e36a1b2f1b97aee1b7ff6e4db0e06c29a8b0","nonce":"0x0000000000000000","number":"0x84161e","parentHash":"0x72d92c1498e05952988d4e79a695928a6bcbd37239f8a1734051263b4d3504b8","receiptsRoot":"0xaff90ae18dcc35924a4bddb68d403b8b7812c10c3ea2a114f34105c87d75bcdb","sha3Uncles":"0x1dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347","size":"0x2a51","stateRoot":"0xc56738518b2c7854a640ae25996d2211c9ef0dd2e4dd9e59e9d9cacef39622da","timestamp":"0x64110a5c","totalDifficulty":"0xa4a470","transactions":["0x1e8f148a9aea7d8d16ea6e9446723b8f262e8bcd89c7c961d52046ebd43b4598","0xab5c870f4c367012bd763172afbfbe68fbf35336a66ae41aff3f2c9dbf4ea3f8","0xa81fd92b2d0f0bbd3cc355f869cca3243c98c5e2641db9ecf3eeabb3b13bff6a","0xa92c7b720c08c83f1a0ed7e4c163200e30a3a8c03fcc5a51e685ea20cd0cb577","0x6921b429ad2ec1e97d3457049ad2e893b5a0349beba47ca1c74a9540af75347a","0xf776b2da0b835dde05d0d8b76fd19385d61e7055036cf637f804b36dc94f2384","0x9a08d899cd14ebb930ed59fa774afdb88a22615b3a931e930931ea54d26dc0bc","0x0fe0d97e25d5eb11a33a3e8278584c3780941fc2675bdf8fc547cee3d1fd3b17","0xef47a60f57f177a683c723c658137efab66d311e1c5abbc4d74f653535144d03","0xe23a5b35faae5335adc5aca38c5d633b00438b798c2053104b8df48406c9b141","0xd8cea4ba619b317bc05d58534af73beec6c2548b31b24d4dc61c9bbd29cfa17a","0x79a4b9d90b02c768baaad305f266281213cc75062cbe99a13222cc0c4b509498","0x6790a3bbddbeb21fcb736a59b3775755051c3a6344d8390cf8ca27f2e8a814f0","0x87ec7ace5442db252b5751ffddd38dcb04b088d36b6b0e526ff25607a4293c81","0x40cb487ecffda94f97ce7fc0f7163f2f024235df2c8291169edc80dac063e6d0","0xb76bb3d88c9b30d927c45ccfcf8d5b0054411ac8501ad588822a7d04690cccf6","0x798ebe823209869347c08bd81e04fbf60e9bdfe44b1cc923215182d0cf3d4edb","0xbe68a7e02725f799a65ebb069ccc83a014ac7c40e4119bf7c220a2f6ddfee295","0xc90c3a72efe81331727fcce4b5bd4906066da314ca9a0b44023a6b09ea7e8114","0x619a6cbd43cde074d314c19623bd66d9fb1e13c158d7138775236f798dc1245e","0xca5a56cd77b9e5b0e79020cc6346edf205bc11e901984d805125f28c2e6686e6","0x999c9ddeed67c6ef6fbf02a6e977a6c1b68e18d24814e51643c7157b87a43e0a","0x47c8f5d0b3778e4c34eba7fcc356fa04a5afd954ccf484728e72c002764dd3c4","0x396797ae0ebcdb72ff1f96fd08b6128f78acc7417353f142f1a5facd425a33e6","0x454aa43d6546a6f62246826c16b7a49c6c704238c18802ef0d659922f23a573c","0x317ecb5bd19caa42a69f836d41556ebb0e0e00e1c6cd2dee230e6e6192612527","0xc879285db5ef0a6bce98021584d16f134c1dc0aed8cc988802c4f72ba6877ff6","0xecaa2d6f597608307e5084854854ba6dc1e69395e2abea14f2c6a2fa1d6faf9a","0x4dd69b69a568ff30ae439e2ded72fbd7f2e7aaa345836703663f155c749c5eed"],"transactionsRoot":"0x4a87d0cf5990b1c5bac631583e5965c2ba943858bebb2e07f74d0b697f73821a","uncles":[],"withdrawals":[{"index":"0x1170","validatorIndex":"0x38c2c","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x66edfd65"},{"index":"0x1171","validatorIndex":"0x38c2d","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6cd228e4"},{"index":"0x1172","validatorIndex":"0x38c2e","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x77f3431b"},{"index":"0x1173","validatorIndex":"0x38c2f","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6b61f268"},{"index":"0x1174","validatorIndex":"0x38c30","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6e10bb21"},{"index":"0x1175","validatorIndex":"0x38c31","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6eb115a5"},{"index":"0x1176","validatorIndex":"0x38c32","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x7caead1d"},{"index":"0x1177","validatorIndex":"0x38c33","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x772c0ddf"},{"index":"0x1178","validatorIndex":"0x38c34","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x75930a95"},{"index":"0x1179","validatorIndex":"0x38c35","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x76a4db09"},{"index":"0x117a","validatorIndex":"0x38c36","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x7e692b27"},{"index":"0x117b","validatorIndex":"0x38c37","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x72038ae6"},{"index":"0x117c","validatorIndex":"0x38c38","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6ccce352"},{"index":"0x117d","validatorIndex":"0x38c39","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x79ef6898"},{"index":"0x117e","validatorIndex":"0x38c3a","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6d58977d"},{"index":"0x117f","validatorIndex":"0x38c3b","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x76f7d208"}],"withdrawalsRoot":"0xbe712c930a0665264b025ced87cc7839eef95a3cbc26dadc93e9e185a350ad28"} -`}, - {Name: "Shanghai bad withdrawals root", OK: false, Data: `{"baseFeePerGas":"0x3fb7c357","difficulty":"0x0","extraData":"0x","gasLimit":"0x1c9c380","gasUsed":"0x18f759","hash":"0xa16c6bcda4fdca88b5761965c4d724f7afc6a6900d9051a204e544870adb3452","logsBloom":"0x020010404000001a0000021000000080001100410000100001000010040200980220400000008806200200000100000000000000000000008000000400042000000050000040000112080808800002044000040004042008800480002000000000000002020020000042002400000820000080040000000010200010020010100101212050000008000000008000001010200c80000112010000438040020400000000202400000000002002a0210402000622010000000001700144000040000000002204000000c000410105024010000808000000002004002000000261000000822200200800881000000012500400400000000000000040010000800000","miner":"0x000095e79eac4d76aab57cb2c1f091d553b36ca0","mixHash":"0x5b53dc49cbab268ef9950b1d81b5e36a1b2f1b97aee1b7ff6e4db0e06c29a8b0","nonce":"0x0000000000000000","number":"0x84161e","parentHash":"0x72d92c1498e05952988d4e79a695928a6bcbd37239f8a1734051263b4d3504b8","receiptsRoot":"0xaff90ae18dcc35924a4bddb68d403b8b7812c10c3ea2a114f34105c87d75bcdb","sha3Uncles":"0x1dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347","size":"0x2a51","stateRoot":"0xc56738518b2c7854a640ae25996d2211c9ef0dd2e4dd9e59e9d9cacef39622da","timestamp":"0x64110a5c","totalDifficulty":"0xa4a470","transactions":["0x1e8f148a9aea7d8d16ea6e9446723b8f262e8bcd89c7c961d52046ebd43b4598","0xab5c870f4c367012bd763172afbfbe68fbf35336a66ae41aff3f2c9dbf4ea3f8","0xa81fd92b2d0f0bbd3cc355f869cca3243c98c5e2641db9ecf3eeabb3b13bff6a","0xa92c7b720c08c83f1a0ed7e4c163200e30a3a8c03fcc5a51e685ea20cd0cb577","0x6921b429ad2ec1e97d3457049ad2e893b5a0349beba47ca1c74a9540af75347a","0xf776b2da0b835dde05d0d8b76fd19385d61e7055036cf637f804b36dc94f2384","0x9a08d899cd14ebb930ed59fa774afdb88a22615b3a931e930931ea54d26dc0bc","0x0fe0d97e25d5eb11a33a3e8278584c3780941fc2675bdf8fc547cee3d1fd3b17","0xef47a60f57f177a683c723c658137efab66d311e1c5abbc4d74f653535144d03","0xe23a5b35faae5335adc5aca38c5d633b00438b798c2053104b8df48406c9b141","0xd8cea4ba619b317bc05d58534af73beec6c2548b31b24d4dc61c9bbd29cfa17a","0x79a4b9d90b02c768baaad305f266281213cc75062cbe99a13222cc0c4b509498","0x6790a3bbddbeb21fcb736a59b3775755051c3a6344d8390cf8ca27f2e8a814f0","0x87ec7ace5442db252b5751ffddd38dcb04b088d36b6b0e526ff25607a4293c81","0x40cb487ecffda94f97ce7fc0f7163f2f024235df2c8291169edc80dac063e6d0","0xb76bb3d88c9b30d927c45ccfcf8d5b0054411ac8501ad588822a7d04690cccf6","0x798ebe823209869347c08bd81e04fbf60e9bdfe44b1cc923215182d0cf3d4edb","0xbe68a7e02725f799a65ebb069ccc83a014ac7c40e4119bf7c220a2f6ddfee295","0xc90c3a72efe81331727fcce4b5bd4906066da314ca9a0b44023a6b09ea7e8114","0x619a6cbd43cde074d314c19623bd66d9fb1e13c158d7138775236f798dc1245e","0xca5a56cd77b9e5b0e79020cc6346edf205bc11e901984d805125f28c2e6686e6","0x999c9ddeed67c6ef6fbf02a6e977a6c1b68e18d24814e51643c7157b87a43e0a","0x47c8f5d0b3778e4c34eba7fcc356fa04a5afd954ccf484728e72c002764dd3c4","0x396797ae0ebcdb72ff1f96fd08b6128f78acc7417353f142f1a5facd425a33e6","0x454aa43d6546a6f62246826c16b7a49c6c704238c18802ef0d659922f23a573c","0x317ecb5bd19caa42a69f836d41556ebb0e0e00e1c6cd2dee230e6e6192612527","0xc879285db5ef0a6bce98021584d16f134c1dc0aed8cc988802c4f72ba6877ff6","0xecaa2d6f597608307e5084854854ba6dc1e69395e2abea14f2c6a2fa1d6faf9a","0x4dd69b69a568ff30ae439e2ded72fbd7f2e7aaa345836703663f155c749c5eed"],"transactionsRoot":"0x4a87d0cf5990b1c5bac631583e5965c2ba943858bebb2e07f74d0b697f73821a","uncles":[],"withdrawals":[{"index":"0x1170","validatorIndex":"0x38c2c","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x66edfd65"},{"index":"0x1171","validatorIndex":"0x38c2d","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6cd228e4"},{"index":"0x1172","validatorIndex":"0x38c2e","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x77f3431b"},{"index":"0x1173","validatorIndex":"0x38c2f","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6b61f268"},{"index":"0x1174","validatorIndex":"0x38c30","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6e10bb21"},{"index":"0x1175","validatorIndex":"0x38c31","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6eb115a5"},{"index":"0x1176","validatorIndex":"0x38c32","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x7caead1d"},{"index":"0x1177","validatorIndex":"0x38c33","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x772c0ddf"},{"index":"0x1178","validatorIndex":"0x38c34","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x75930a95"},{"index":"0x1179","validatorIndex":"0x38c35","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x76a4db09"},{"index":"0x117a","validatorIndex":"0x38c36","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x7e692b27"},{"index":"0x117b","validatorIndex":"0x38c37","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x72038ae6"},{"index":"0x117c","validatorIndex":"0x38c38","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6ccce352"},{"index":"0x117d","validatorIndex":"0x38c39","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x79ef6898"},{"index":"0x117e","validatorIndex":"0x38c3a","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x6d58977d"},{"index":"0x117f","validatorIndex":"0x38c3b","address":"0x8f0844fd51e31ff6bf5babe21dccf7328e19fd9f","amount":"0x76f7d208"}],"withdrawalsRoot":"0xbe712c930a0665264b025ced87cc7839eef95a3cbc26dadc93e9e185a350ad27"} -`}, +//go:embed testdata +var blocksTestdata embed.FS + +type testMetadata struct { + Name string `json:"name"` + Fail bool `json:"fail,omitempty"` +} + +func readJsonTestdata(t *testing.T, name string, dest any) { + f, err := blocksTestdata.Open(name) + require.NoError(t, err, "must open %q", name) + require.NoError(t, json.NewDecoder(f).Decode(dest), "must json-decode %q", name) + require.NoError(t, f.Close(), "must close %q", name) +} + +func TestBlockHeaderJSON(t *testing.T) { + headersDir, err := blocksTestdata.ReadDir("testdata/data/headers") + require.NoError(t, err) + + for _, entry := range headersDir { + if !strings.HasSuffix(entry.Name(), "_metadata.json") { + continue + } + + var metadata testMetadata + readJsonTestdata(t, "testdata/data/headers/"+entry.Name(), &metadata) + t.Run(metadata.Name, func(t *testing.T) { + var header rpcHeader + readJsonTestdata(t, "testdata/data/headers/"+strings.Replace(entry.Name(), "_metadata.json", "_data.json", 1), &header) + + h := header.computeBlockHash() + if metadata.Fail { + require.NotEqual(t, h, header.Hash, "expecting verification error") + } else { + require.Equal(t, h, header.Hash, "blockhash should verify ok") + } + }) } - for _, testCase := range testCases { - var x rpcHeader - require.NoError(t, json.Unmarshal([]byte(testCase.Data), &x)) - h := x.computeBlockHash() - if testCase.OK { - require.Equal(t, h, x.Hash, "blockhash should verify ok") - } else { - require.NotEqual(t, h, x.Hash, "expecting verification error") +} + +func TestBlockJSON(t *testing.T) { + blocksDir, err := blocksTestdata.ReadDir("testdata/data/blocks") + require.NoError(t, err) + + for _, entry := range blocksDir { + if !strings.HasSuffix(entry.Name(), "_metadata.json") { + continue } + + var metadata testMetadata + readJsonTestdata(t, "testdata/data/blocks/"+entry.Name(), &metadata) + t.Run(metadata.Name, func(t *testing.T) { + var block rpcBlock + readJsonTestdata(t, "testdata/data/blocks/"+strings.Replace(entry.Name(), "_metadata.json", "_data.json", 1), &block) + + err := block.verify() + if metadata.Fail { + require.NotNil(t, err, "expecting verification error") + } else { + require.NoError(t, err, "verification should pass") + } + }) } } From 2b57c303668de25961eb66b35143f31ba0f414ed Mon Sep 17 00:00:00 2001 From: protolambda Date: Fri, 20 Oct 2023 16:46:07 +0200 Subject: [PATCH 005/279] op-e2e: add random withdrawals to fake-proof-of-stake --- op-e2e/e2eutils/geth/fakepos.go | 25 +++++++++++++++++++++++-- 1 file changed, 23 insertions(+), 2 deletions(-) diff --git a/op-e2e/e2eutils/geth/fakepos.go b/op-e2e/e2eutils/geth/fakepos.go index 081a5b838d57..51ec48eb15d6 100644 --- a/op-e2e/e2eutils/geth/fakepos.go +++ b/op-e2e/e2eutils/geth/fakepos.go @@ -1,9 +1,9 @@ package geth import ( + "math/rand" "time" - "github.com/ethereum-optimism/optimism/op-service/clock" "github.com/ethereum/go-ethereum" "github.com/ethereum/go-ethereum/beacon/engine" "github.com/ethereum/go-ethereum/common" @@ -12,6 +12,9 @@ import ( "github.com/ethereum/go-ethereum/eth/catalyst" "github.com/ethereum/go-ethereum/event" "github.com/ethereum/go-ethereum/log" + + "github.com/ethereum-optimism/optimism/op-service/clock" + "github.com/ethereum-optimism/optimism/op-service/testutils" ) // fakePoS is a testing-only utility to attach to Geth, @@ -22,6 +25,8 @@ type fakePoS struct { log log.Logger blockTime uint64 + withdrawalsIndex uint64 + finalizedDistance uint64 safeDistance uint64 @@ -33,6 +38,7 @@ func (f *fakePoS) Start() error { if advancing, ok := f.clock.(*clock.AdvancingClock); ok { advancing.Start() } + withdrawalsRNG := rand.New(rand.NewSource(450368975843)) // avoid generating the same address as any test f.sub = event.NewSubscription(func(quit <-chan struct{}) error { // poll every half a second: enough to catch up with any block time when ticks are missed t := f.clock.NewTicker(time.Second / 2) @@ -64,6 +70,17 @@ func (f *fakePoS) Start() error { // We're a long way behind, let's skip some blocks... newBlockTime = uint64(f.clock.Now().Unix()) } + // create some random withdrawals + withdrawals := make([]*types.Withdrawal, withdrawalsRNG.Intn(4)) + for i := 0; i < len(withdrawals); i++ { + withdrawals[i] = &types.Withdrawal{ + Index: f.withdrawalsIndex + uint64(i), + Validator: withdrawalsRNG.Uint64() % 100_000_000, // 100 million fake validators + Address: testutils.RandomAddress(withdrawalsRNG), + // in gwei, consensus-layer quirk. withdraw non-zero value up to 50 ETH + Amount: uint64(withdrawalsRNG.Intn(50_000_000_000) + 1), + } + } res, err := f.engineAPI.ForkchoiceUpdatedV2(engine.ForkchoiceStateV1{ HeadBlockHash: head.Hash(), SafeBlockHash: safe.Hash(), @@ -72,7 +89,7 @@ func (f *fakePoS) Start() error { Timestamp: newBlockTime, Random: common.Hash{}, SuggestedFeeRecipient: head.Coinbase, - Withdrawals: make([]*types.Withdrawal, 0), + Withdrawals: withdrawals, }) if err != nil { f.log.Error("failed to start building L1 block", "err", err) @@ -109,6 +126,10 @@ func (f *fakePoS) Start() error { f.log.Error("failed to make built L1 block canonical", "err", err) continue } + // Increment global withdrawals index in the CL. + // The EL doesn't really care about the value, + // but it's nice to mock something consistent with the CL specs. + f.withdrawalsIndex += uint64(len(withdrawals)) case <-quit: return nil } From c01d2c02d701a0414aae98bf9c4de070b5037cad Mon Sep 17 00:00:00 2001 From: Hamdi Allam Date: Fri, 20 Oct 2023 18:38:37 -0400 Subject: [PATCH 006/279] fix(indexer) legacy processor withdrawals & latest epoch query --- indexer/database/blocks.go | 5 ++-- indexer/migrations/20230523_create_schema.sql | 28 +++---------------- .../bridge/legacy_bridge_processor.go | 11 ++++++-- 3 files changed, 15 insertions(+), 29 deletions(-) diff --git a/indexer/database/blocks.go b/indexer/database/blocks.go index 86d43e3baa77..8df6d8c35042 100644 --- a/indexer/database/blocks.go +++ b/indexer/database/blocks.go @@ -186,7 +186,6 @@ func (db *blocksDB) LatestObservedEpoch(fromL1Height *big.Int, maxL1Range uint64 var header L1BlockHeader if fromL1Height != nil { result := db.gorm.Where("number = ?", fromL1Height).Take(&header) - // TODO - Embed logging to db if result.Error != nil { if errors.Is(result.Error, gorm.ErrRecordNotFound) { return nil, nil @@ -196,7 +195,8 @@ func (db *blocksDB) LatestObservedEpoch(fromL1Height *big.Int, maxL1Range uint64 fromTimestamp = header.Timestamp } else { - result := db.gorm.Order("number desc").Take(&header) + // Take the lowest indexed L1 block to compute the lower bound + result := db.gorm.Order("number ASC").Take(&header) if result.Error != nil { if errors.Is(result.Error, gorm.ErrRecordNotFound) { return nil, nil @@ -205,6 +205,7 @@ func (db *blocksDB) LatestObservedEpoch(fromL1Height *big.Int, maxL1Range uint64 } fromL1Height = header.Number + fromTimestamp = header.Timestamp } // Upper Bound (lowest timestamp indexed between L1/L2 bounded by `maxL1Range`) diff --git a/indexer/migrations/20230523_create_schema.sql b/indexer/migrations/20230523_create_schema.sql index 67969117b81f..19f22dc38c43 100644 --- a/indexer/migrations/20230523_create_schema.sql +++ b/indexer/migrations/20230523_create_schema.sql @@ -177,26 +177,6 @@ CREATE INDEX IF NOT EXISTS l2_bridge_messages_transaction_withdrawal_hash ON l2_ CREATE INDEX IF NOT EXISTS l2_bridge_messages_from_address ON l2_bridge_messages(from_address); -- StandardBridge -CREATE TABLE IF NOT EXISTS l1_bridged_tokens ( - address VARCHAR PRIMARY KEY, - bridge_address VARCHAR NOT NULL, - - name VARCHAR NOT NULL, - symbol VARCHAR NOT NULL, - decimals INTEGER NOT NULL CHECK (decimals >= 0 AND decimals <= 18) -); -CREATE TABLE IF NOT EXISTS l2_bridged_tokens ( - address VARCHAR PRIMARY KEY, - bridge_address VARCHAR NOT NULL, - - -- L1-L2 relationship is 1 to many so this is not necessarily unique - l1_token_address VARCHAR REFERENCES l1_bridged_tokens(address) ON DELETE CASCADE, - - name VARCHAR NOT NULL, - symbol VARCHAR NOT NULL, - decimals INTEGER NOT NULL CHECK (decimals >= 0 AND decimals <= 18) -); - CREATE TABLE IF NOT EXISTS l1_bridge_deposits ( transaction_source_hash VARCHAR PRIMARY KEY REFERENCES l1_transaction_deposits(source_hash) ON DELETE CASCADE, cross_domain_message_hash VARCHAR NOT NULL UNIQUE REFERENCES l1_bridge_messages(message_hash) ON DELETE CASCADE, @@ -204,8 +184,8 @@ CREATE TABLE IF NOT EXISTS l1_bridge_deposits ( -- Deposit information from_address VARCHAR NOT NULL, to_address VARCHAR NOT NULL, - local_token_address VARCHAR NOT NULL, -- REFERENCES l1_bridged_tokens(address), uncomment me in future pr - remote_token_address VARCHAR NOT NULL, -- REFERENCES l2_bridged_tokens(address), uncomment me in future pr + local_token_address VARCHAR NOT NULL, + remote_token_address VARCHAR NOT NULL, amount UINT256 NOT NULL, data VARCHAR NOT NULL, timestamp INTEGER NOT NULL CHECK (timestamp > 0) @@ -221,8 +201,8 @@ CREATE TABLE IF NOT EXISTS l2_bridge_withdrawals ( -- Withdrawal information from_address VARCHAR NOT NULL, to_address VARCHAR NOT NULL, - local_token_address VARCHAR NOT NULL, -- REFERENCES l2_bridged_tokens(address), uncomment me in future pr - remote_token_address VARCHAR NOT NULL, -- REFERENCES l1_bridged_tokens(address), uncomment me in future pr + local_token_address VARCHAR NOT NULL, + remote_token_address VARCHAR NOT NULL, amount UINT256 NOT NULL, data VARCHAR NOT NULL, timestamp INTEGER NOT NULL CHECK (timestamp > 0) diff --git a/indexer/processors/bridge/legacy_bridge_processor.go b/indexer/processors/bridge/legacy_bridge_processor.go index d4b0abaf4d41..5410881c5107 100644 --- a/indexer/processors/bridge/legacy_bridge_processor.go +++ b/indexer/processors/bridge/legacy_bridge_processor.go @@ -162,13 +162,17 @@ func LegacyL2ProcessInitiatedBridgeEvents(log log.Logger, db *database.DB, metri log.Info("detected legacy transaction withdrawals (via L2CrossDomainMessenger)", "size", len(crossDomainSentMessages)) } + type sentMessageEvent struct { + *contracts.CrossDomainMessengerSentMessageEvent + WithdrawalHash common.Hash + } + withdrawnWEI := bigint.Zero - sentMessages := make(map[logKey]*contracts.CrossDomainMessengerSentMessageEvent, len(crossDomainSentMessages)) + sentMessages := make(map[logKey]sentMessageEvent, len(crossDomainSentMessages)) bridgeMessages := make([]database.L2BridgeMessage, len(crossDomainSentMessages)) transactionWithdrawals := make([]database.L2TransactionWithdrawal, len(crossDomainSentMessages)) for i := range crossDomainSentMessages { sentMessage := crossDomainSentMessages[i] - sentMessages[logKey{sentMessage.Event.BlockHash, sentMessage.Event.LogIndex}] = &sentMessage withdrawnWEI = new(big.Int).Add(withdrawnWEI, sentMessage.BridgeMessage.Tx.Amount) // To ensure consistency in the schema, we duplicate this as the "root" transaction withdrawal. The storage key in the message @@ -188,6 +192,7 @@ func LegacyL2ProcessInitiatedBridgeEvents(log log.Logger, db *database.DB, metri }, } + sentMessages[logKey{sentMessage.Event.BlockHash, sentMessage.Event.LogIndex}] = sentMessageEvent{&sentMessage, withdrawalHash} bridgeMessages[i] = database.L2BridgeMessage{ TransactionWithdrawalHash: withdrawalHash, BridgeMessage: sentMessage.BridgeMessage, @@ -235,7 +240,7 @@ func LegacyL2ProcessInitiatedBridgeEvents(log log.Logger, db *database.DB, metri bridgedTokens[initiatedBridge.BridgeTransfer.TokenPair.LocalTokenAddress]++ initiatedBridge.BridgeTransfer.CrossDomainMessageHash = &sentMessage.BridgeMessage.MessageHash l2BridgeWithdrawals[i] = database.L2BridgeWithdrawal{ - TransactionWithdrawalHash: sentMessage.BridgeMessage.MessageHash, + TransactionWithdrawalHash: sentMessage.WithdrawalHash, BridgeTransfer: initiatedBridge.BridgeTransfer, } } From af26334a635762d5d279ba76d3a564f528be57b1 Mon Sep 17 00:00:00 2001 From: Hamdi Allam Date: Sat, 21 Oct 2023 00:59:58 -0400 Subject: [PATCH 007/279] indexer bridge finalization --- .../processors/bridge/l1_bridge_processor.go | 33 +++------------- .../processors/bridge/l2_bridge_processor.go | 39 +++++-------------- .../bridge/legacy_bridge_processor.go | 20 ++++++---- 3 files changed, 27 insertions(+), 65 deletions(-) diff --git a/indexer/processors/bridge/l1_bridge_processor.go b/indexer/processors/bridge/l1_bridge_processor.go index 87a2d9603de0..106188b9e3f0 100644 --- a/indexer/processors/bridge/l1_bridge_processor.go +++ b/indexer/processors/bridge/l1_bridge_processor.go @@ -29,7 +29,7 @@ func L1ProcessInitiatedBridgeEvents(log log.Logger, db *database.DB, metrics L1M log.Info("detected transaction deposits", "size", len(optimismPortalTxDeposits)) } - var mintedGWEI = bigint.Zero + mintedGWEI := bigint.Zero portalDeposits := make(map[logKey]*contracts.OptimismPortalTransactionDepositEvent, len(optimismPortalTxDeposits)) transactionDeposits := make([]database.L1TransactionDeposit, len(optimismPortalTxDeposits)) for i := range optimismPortalTxDeposits { @@ -44,7 +44,6 @@ func L1ProcessInitiatedBridgeEvents(log log.Logger, db *database.DB, metrics L1M GasLimit: depositTx.GasLimit, Tx: depositTx.Tx, } - } if len(transactionDeposits) > 0 { @@ -125,6 +124,7 @@ func L1ProcessInitiatedBridgeEvents(log log.Logger, db *database.DB, metrics L1M } bridgedTokens[initiatedBridge.BridgeTransfer.TokenPair.LocalTokenAddress]++ + initiatedBridge.BridgeTransfer.CrossDomainMessageHash = &sentMessage.BridgeMessage.MessageHash bridgeDeposits[i] = database.L1BridgeDeposit{ TransactionSourceHash: portalDeposit.DepositTx.SourceHash, @@ -214,10 +214,8 @@ func L1ProcessFinalizedBridgeEvents(log log.Logger, db *database.DB, metrics L1M log.Info("detected relayed messages", "size", len(crossDomainRelayedMessages)) } - relayedMessages := make(map[logKey]*contracts.CrossDomainMessengerRelayedMessageEvent, len(crossDomainRelayedMessages)) for i := range crossDomainRelayedMessages { relayed := crossDomainRelayedMessages[i] - relayedMessages[logKey{BlockHash: relayed.Event.BlockHash, LogIndex: relayed.Event.LogIndex}] = &relayed message, err := db.BridgeMessages.L2BridgeMessage(relayed.MessageHash) if err != nil { return err @@ -236,40 +234,21 @@ func L1ProcessFinalizedBridgeEvents(log log.Logger, db *database.DB, metrics L1M } // (4) L1StandardBridge + // - Nothing actionable on the database. Since the StandardBridge is layered ontop of the + // CrossDomainMessenger, there's no need for any sanity or invariant checks as the previous step + // ensures a relayed message (finalized bridge) can be linked with a sent message (initiated bridge). finalizedBridges, err := contracts.StandardBridgeFinalizedEvents("l1", l1Contracts.L1StandardBridgeProxy, db, fromHeight, toHeight) if err != nil { return err } - if len(finalizedBridges) > 0 { - log.Info("detected finalized bridge withdrawals", "size", len(finalizedBridges)) - } finalizedTokens := make(map[common.Address]int) for i := range finalizedBridges { - // Nothing actionable on the database. However, we can treat the relayed message - // as an invariant by ensuring we can query for a deposit by the same hash finalizedBridge := finalizedBridges[i] - relayedMessage, ok := relayedMessages[logKey{finalizedBridge.Event.BlockHash, finalizedBridge.Event.LogIndex + 1}] - if !ok { - log.Error("expected RelayedMessage following BridgeFinalized event", "tx_hash", finalizedBridge.Event.TransactionHash.String()) - return fmt.Errorf("expected RelayedMessage following BridgeFinalized event. tx_hash = %s", finalizedBridge.Event.TransactionHash.String()) - } else if relayedMessage.Event.TransactionHash != finalizedBridge.Event.TransactionHash { - log.Error("correlated events tx hash mismatch", "message_tx_hash", relayedMessage.Event.TransactionHash.String(), "bridge_tx_hash", finalizedBridge.Event.TransactionHash.String()) - return fmt.Errorf("correlated events tx hash mismatch") - } - - // Since the message hash is computed from the relayed message, this ensures the deposit fields must match - withdrawal, err := db.BridgeTransfers.L2BridgeWithdrawalWithFilter(database.BridgeTransfer{CrossDomainMessageHash: &relayedMessage.MessageHash}) - if err != nil { - return err - } else if withdrawal == nil { - log.Error("missing L2StandardBridge withdrawal on L1 finalization", "tx_hash", finalizedBridge.Event.TransactionHash.String()) - return fmt.Errorf("missing L2StandardBridge withdrawal on L1 finalization. tx_hash: %s", finalizedBridge.Event.TransactionHash.String()) - } - finalizedTokens[finalizedBridge.BridgeTransfer.TokenPair.LocalTokenAddress]++ } if len(finalizedBridges) > 0 { + log.Info("detected finalized bridge withdrawals", "size", len(finalizedBridges)) for tokenAddr, size := range finalizedTokens { metrics.RecordL1FinalizedBridgeTransfers(tokenAddr, size) } diff --git a/indexer/processors/bridge/l2_bridge_processor.go b/indexer/processors/bridge/l2_bridge_processor.go index 8a23d72b8242..932b1c91c745 100644 --- a/indexer/processors/bridge/l2_bridge_processor.go +++ b/indexer/processors/bridge/l2_bridge_processor.go @@ -1,7 +1,6 @@ package bridge import ( - "errors" "fmt" "math/big" @@ -29,7 +28,7 @@ func L2ProcessInitiatedBridgeEvents(log log.Logger, db *database.DB, metrics L2M log.Info("detected transaction withdrawals", "size", len(l2ToL1MPMessagesPassed)) } - var withdrawnWEI = bigint.Zero + withdrawnWEI := bigint.Zero messagesPassed := make(map[logKey]*contracts.L2ToL1MessagePasserMessagePassed, len(l2ToL1MPMessagesPassed)) transactionWithdrawals := make([]database.L2TransactionWithdrawal, len(l2ToL1MPMessagesPassed)) for i := range l2ToL1MPMessagesPassed { @@ -122,8 +121,9 @@ func L2ProcessInitiatedBridgeEvents(log log.Logger, db *database.DB, metrics L2M return fmt.Errorf("correlated events tx hash mismatch") } - initiatedBridge.BridgeTransfer.CrossDomainMessageHash = &sentMessage.BridgeMessage.MessageHash bridgedTokens[initiatedBridge.BridgeTransfer.TokenPair.LocalTokenAddress]++ + + initiatedBridge.BridgeTransfer.CrossDomainMessageHash = &sentMessage.BridgeMessage.MessageHash bridgeWithdrawals[i] = database.L2BridgeWithdrawal{ TransactionWithdrawalHash: messagePassed.WithdrawalHash, BridgeTransfer: initiatedBridge.BridgeTransfer, @@ -158,10 +158,8 @@ func L2ProcessFinalizedBridgeEvents(log log.Logger, db *database.DB, metrics L2M log.Info("detected relayed messages", "size", len(crossDomainRelayedMessages)) } - relayedMessages := make(map[logKey]*contracts.CrossDomainMessengerRelayedMessageEvent, len(crossDomainRelayedMessages)) for i := range crossDomainRelayedMessages { relayed := crossDomainRelayedMessages[i] - relayedMessages[logKey{BlockHash: relayed.Event.BlockHash, LogIndex: relayed.Event.LogIndex}] = &relayed message, err := db.BridgeMessages.L1BridgeMessage(relayed.MessageHash) if err != nil { return err @@ -175,45 +173,26 @@ func L2ProcessFinalizedBridgeEvents(log log.Logger, db *database.DB, metrics L2M return err } } - if len(relayedMessages) > 0 { - metrics.RecordL2CrossDomainRelayedMessages(len(relayedMessages)) + if len(crossDomainRelayedMessages) > 0 { + metrics.RecordL2CrossDomainRelayedMessages(len(crossDomainRelayedMessages)) } // (2) L2StandardBridge + // - Nothing actionable on the database. Since the StandardBridge is layered ontop of the + // CrossDomainMessenger, there's no need for any sanity or invariant checks as the previous step + // ensures a relayed message (finalized bridge) can be linked with a sent message (initiated bridge). finalizedBridges, err := contracts.StandardBridgeFinalizedEvents("l2", l2Contracts.L2StandardBridge, db, fromHeight, toHeight) if err != nil { return err } - if len(finalizedBridges) > 0 { - log.Info("detected finalized bridge deposits", "size", len(finalizedBridges)) - } finalizedTokens := make(map[common.Address]int) for i := range finalizedBridges { - // Nothing actionable on the database. However, we can treat the relayed message - // as an invariant by ensuring we can query for a deposit by the same hash finalizedBridge := finalizedBridges[i] - relayedMessage, ok := relayedMessages[logKey{finalizedBridge.Event.BlockHash, finalizedBridge.Event.LogIndex + 1}] - if !ok { - log.Error("expected RelayedMessage following BridgeFinalized event", "tx_hash", finalizedBridge.Event.TransactionHash.String()) - return fmt.Errorf("expected RelayedMessage following BridgeFinalized event. tx_hash = %s", finalizedBridge.Event.TransactionHash.String()) - } else if relayedMessage.Event.TransactionHash != finalizedBridge.Event.TransactionHash { - log.Error("correlated events tx hash mismatch", "message_tx_hash", relayedMessage.Event.TransactionHash.String(), "bridge_tx_hash", finalizedBridge.Event.TransactionHash.String()) - return fmt.Errorf("correlated events tx hash mismatch") - } - - // Since the message hash is computed from the relayed message, this ensures the withdrawal fields must match - deposit, err := db.BridgeTransfers.L1BridgeDepositWithFilter(database.BridgeTransfer{CrossDomainMessageHash: &relayedMessage.MessageHash}) - if err != nil { - return err - } else if deposit == nil { - log.Error("missing L1StandardBridge deposit on L2 finalization", "tx_hash", finalizedBridge.Event.TransactionHash.String()) - return errors.New("missing L1StandardBridge deposit on L2 finalization") - } - finalizedTokens[finalizedBridge.BridgeTransfer.TokenPair.LocalTokenAddress]++ } if len(finalizedBridges) > 0 { + log.Info("detected finalized bridge deposits", "size", len(finalizedBridges)) for tokenAddr, size := range finalizedTokens { metrics.RecordL2FinalizedBridgeTransfers(tokenAddr, size) } diff --git a/indexer/processors/bridge/legacy_bridge_processor.go b/indexer/processors/bridge/legacy_bridge_processor.go index d4b0abaf4d41..00ba06f5c61b 100644 --- a/indexer/processors/bridge/legacy_bridge_processor.go +++ b/indexer/processors/bridge/legacy_bridge_processor.go @@ -330,10 +330,12 @@ func LegacyL1ProcessFinalizedBridgeEvents(log log.Logger, db *database.DB, metri log.Warn("skipped pre-regensis relayed L2CrossDomainMessenger withdrawals", "size", skippedPreRegenesisMessages) } - // (2) L2StandardBridge -- no-op for now as there's nothing actionable to do here besides - // santiy checks which is not important for legacy code. Not worth extra code pre-bedrock. - // The message status is already tracked via the relayed bridge messed through the cross domain messenger. - // - NOTE: This means we dont increment metrics for finalized bridge transfers + // (2) L1StandardBridge + // - Nothing actionable on the database. Since the StandardBridge is layered ontop of the + // CrossDomainMessenger, there's no need for any sanity or invariant checks as the previous step + // ensures a relayed message (finalized bridge) can be linked with a sent message (initiated bridge). + + // - NOTE: Ignoring metrics for pre-bedrock transfers // a-ok! return nil @@ -372,10 +374,12 @@ func LegacyL2ProcessFinalizedBridgeEvents(log log.Logger, db *database.DB, metri metrics.RecordL2CrossDomainRelayedMessages(len(crossDomainRelayedMessages)) } - // (2) L2StandardBridge -- no-op for now as there's nothing actionable to do here besides - // santiy checks which is not important for legacy code. Not worth the extra code pre-bedorck. - // The message status is already tracked via the relayed bridge messed through the cross domain messenger. - // - NOTE: This means we dont increment metrics for finalized bridge transfers + // (2) L2StandardBridge + // - Nothing actionable on the database. Since the StandardBridge is layered ontop of the + // CrossDomainMessenger, there's no need for any sanity or invariant checks as the previous step + // ensures a relayed message (finalized bridge) can be linked with a sent message (initiated bridge). + + // - NOTE: Ignoring metrics for pre-bedrock transfers // a-ok! return nil From 18d87e2ae57674ad38ea0733e3f7b58dbcf7d4e6 Mon Sep 17 00:00:00 2001 From: Michael de Hoog Date: Sat, 21 Oct 2023 12:48:02 -1000 Subject: [PATCH 008/279] Avoid incrementing the txmgr nonce if tx signing fails --- op-service/txmgr/txmgr.go | 36 ++++++++++++++++++------------------ 1 file changed, 18 insertions(+), 18 deletions(-) diff --git a/op-service/txmgr/txmgr.go b/op-service/txmgr/txmgr.go index 30bb1fbfe75e..9de7b0bb194e 100644 --- a/op-service/txmgr/txmgr.go +++ b/op-service/txmgr/txmgr.go @@ -238,23 +238,15 @@ func (m *SimpleTxManager) craftTx(ctx context.Context, candidate TxCandidate) (* rawTx.Gas = gas } - // Avoid bumping the nonce if the gas estimation fails. - nonce, err := m.nextNonce(ctx) - if err != nil { - return nil, err - } - rawTx.Nonce = nonce - - ctx, cancel := context.WithTimeout(ctx, m.cfg.NetworkTimeout) - defer cancel() - return m.cfg.Signer(ctx, m.cfg.From, types.NewTx(rawTx)) + return m.signWithNextNonce(ctx, rawTx) } -// nextNonce returns a nonce to use for the next transaction. It uses -// eth_getTransactionCount with "latest" once, and then subsequent calls simply -// increment this number. If the transaction manager is reset, it will query the -// eth_getTransactionCount nonce again. -func (m *SimpleTxManager) nextNonce(ctx context.Context) (uint64, error) { +// signWithNextNonce returns a signed transaction with the next available nonce. +// The nonce is fetched once using eth_getTransactionCount with "latest", and +// then subsequent calls simply increment this number. If the transaction manager +// is reset, it will query the eth_getTransactionCount nonce again. If signing +// fails, the nonce is not incremented. +func (m *SimpleTxManager) signWithNextNonce(ctx context.Context, rawTx *types.DynamicFeeTx) (*types.Transaction, error) { m.nonceLock.Lock() defer m.nonceLock.Unlock() @@ -265,15 +257,23 @@ func (m *SimpleTxManager) nextNonce(ctx context.Context) (uint64, error) { nonce, err := m.backend.NonceAt(childCtx, m.cfg.From, nil) if err != nil { m.metr.RPCError() - return 0, fmt.Errorf("failed to get nonce: %w", err) + return nil, fmt.Errorf("failed to get nonce: %w", err) } m.nonce = &nonce } else { *m.nonce++ } - m.metr.RecordNonce(*m.nonce) - return *m.nonce, nil + rawTx.Nonce = *m.nonce + tx, err := m.cfg.Signer(ctx, m.cfg.From, types.NewTx(rawTx)) + if err != nil { + // decrement the nonce, so we can retry signing with the same nonce next time + // signWithNextNonce is called + *m.nonce-- + } else { + m.metr.RecordNonce(*m.nonce) + } + return tx, err } // resetNonce resets the internal nonce tracking. This is called if any pending send From e7d4429cdad982e2c83ed33c0a325ec262df96de Mon Sep 17 00:00:00 2001 From: Michael de Hoog Date: Sat, 21 Oct 2023 13:03:20 -1000 Subject: [PATCH 009/279] Add test --- op-service/txmgr/txmgr_test.go | 34 ++++++++++++++++++++++++++++++++++ 1 file changed, 34 insertions(+) diff --git a/op-service/txmgr/txmgr_test.go b/op-service/txmgr/txmgr_test.go index 9bb148ac3fea..d15eae373148 100644 --- a/op-service/txmgr/txmgr_test.go +++ b/op-service/txmgr/txmgr_test.go @@ -450,6 +450,40 @@ func TestTxMgr_EstimateGasFails(t *testing.T) { require.Equal(t, lastNonce+1, tx.Nonce()) } +func TestTxMgr_SigningFails(t *testing.T) { + t.Parallel() + errorSigning := false + cfg := configWithNumConfs(1) + cfg.Signer = func(ctx context.Context, from common.Address, tx *types.Transaction) (*types.Transaction, error) { + if errorSigning { + return nil, fmt.Errorf("signer error") + } else { + return tx, nil + } + } + h := newTestHarnessWithConfig(t, cfg) + candidate := h.createTxCandidate() + + // Set the gas limit to zero to trigger gas estimation. + candidate.GasLimit = 0 + + // Craft a successful transaction. + tx, err := h.mgr.craftTx(context.Background(), candidate) + require.Nil(t, err) + lastNonce := tx.Nonce() + + // Mock signer failure. + errorSigning = true + _, err = h.mgr.craftTx(context.Background(), candidate) + require.ErrorContains(t, err, "signer error") + + // Ensure successful craft uses the correct nonce + errorSigning = false + tx, err = h.mgr.craftTx(context.Background(), candidate) + require.Nil(t, err) + require.Equal(t, lastNonce+1, tx.Nonce()) +} + // TestTxMgrOnlyOnePublicationSucceeds asserts that the tx manager will return a // receipt so long as at least one of the publications is able to succeed with a // simulated rpc failure. From a0d6c4c529acd9b54120d76ecec17426e1cf0d85 Mon Sep 17 00:00:00 2001 From: Shawn <44221603+shaspitz@users.noreply.github.com> Date: Sat, 21 Oct 2023 23:58:31 -0700 Subject: [PATCH 010/279] fix --- op-node/cmd/main.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/op-node/cmd/main.go b/op-node/cmd/main.go index adee4fd23232..ccd54946bbab 100644 --- a/op-node/cmd/main.go +++ b/op-node/cmd/main.go @@ -31,8 +31,10 @@ var ( // VersionWithMeta holds the textual version string including the metadata. var VersionWithMeta = func() string { v := version.Version - if GitCommit != "" { + if len(GitCommit) >= 8 { v += "-" + GitCommit[:8] + } else { + v += "-" + GitCommit } if GitDate != "" { v += "-" + GitDate From 31f13447ddb413d4730e90fb0c4f38055965883d Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Mon, 23 Oct 2023 10:07:43 +1000 Subject: [PATCH 011/279] Update pebble to include https://github.com/cockroachdb/pebble/pull/3005 to fix mips compatibility. --- go.mod | 7 +-- go.sum | 164 +++------------------------------------------------------ 2 files changed, 10 insertions(+), 161 deletions(-) diff --git a/go.mod b/go.mod index d22c54e27d64..a43c37b231b6 100644 --- a/go.mod +++ b/go.mod @@ -60,10 +60,11 @@ require ( github.com/btcsuite/btcd/btcutil v1.1.0 // indirect github.com/btcsuite/btclog v0.0.0-20170628155309-84c8d2346e9f // indirect github.com/cespare/xxhash/v2 v2.2.0 // indirect - github.com/cockroachdb/errors v1.9.1 // indirect + github.com/cockroachdb/errors v1.11.1 // indirect github.com/cockroachdb/logtags v0.0.0-20230118201751-21c54148d20b // indirect - github.com/cockroachdb/pebble v0.0.0-20230928194634-aa077af62593 // indirect - github.com/cockroachdb/redact v1.1.3 // indirect + // Upgraded to include https://github.com/cockroachdb/pebble/pull/3005 + github.com/cockroachdb/pebble v0.0.0-20231018212520-f6cde3fc2fa4 // indirect + github.com/cockroachdb/redact v1.1.5 // indirect github.com/cockroachdb/tokenbucket v0.0.0-20230807174530-cc333fc44b06 // indirect github.com/consensys/bavard v0.1.13 // indirect github.com/consensys/gnark-crypto v0.12.1 // indirect diff --git a/go.sum b/go.sum index 5b8dd52173fb..c8241cf12039 100644 --- a/go.sum +++ b/go.sum @@ -7,31 +7,23 @@ dmitri.shuralyov.com/html/belt v0.0.0-20180602232347-f7d459c86be0/go.mod h1:JLBr dmitri.shuralyov.com/service/change v0.0.0-20181023043359-a85b471d5412/go.mod h1:a1inKt/atXimZ4Mv927x+r7UpyzRUf4emIoiiSC2TN4= dmitri.shuralyov.com/state v0.0.0-20180228185332-28bcc343414c/go.mod h1:0PRwlb0D6DFvNNtx+9ybjezNCa8XF0xaYcETyp6rHWU= git.apache.org/thrift.git v0.0.0-20180902110319-2566ecd5d999/go.mod h1:fPE2ZNJGynbRyZ4dJvy6G277gSllfV2HJqblrnkyeyg= -github.com/AndreasBriese/bbloom v0.0.0-20190306092124-e2d15f34fcf9/go.mod h1:bOvUY6CB00SOBii9/FifXqc0awNKxLFCL/+pkDPuyl8= github.com/AndreasBriese/bbloom v0.0.0-20190825152654-46b345b51c96 h1:cTp8I5+VIoKjsnZuH8vjyaysT/ses3EvZeaV/1UkF2M= github.com/AndreasBriese/bbloom v0.0.0-20190825152654-46b345b51c96/go.mod h1:bOvUY6CB00SOBii9/FifXqc0awNKxLFCL/+pkDPuyl8= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/BurntSushi/toml v1.3.2 h1:o7IhLm0Msx3BaB+n3Ag7L8EVlByGnpq14C4YWiu/gL8= github.com/BurntSushi/toml v1.3.2/go.mod h1:CxXYINrC8qIiEnFrOxCa7Jy5BFHlXnUU2pbicEuybxQ= -github.com/CloudyKit/fastprinter v0.0.0-20200109182630-33d98a066a53/go.mod h1:+3IMCy2vIlbG1XG/0ggNQv0SvxCAIpPM5b1nCz56Xno= -github.com/CloudyKit/jet/v3 v3.0.0/go.mod h1:HKQPgSJmdK8hdoAbKUUWajkHyHo4RaU5rMdUywE7VMo= github.com/DataDog/zstd v1.5.2 h1:vUG4lAyuPCXO0TLbXvPv7EB7cNK1QV/luu55UHLrrn8= github.com/DataDog/zstd v1.5.2/go.mod h1:g4AWEaM3yOg3HYfnJ3YIawPnVdXJh9QME85blwSAmyw= -github.com/Joker/hpp v1.0.0/go.mod h1:8x5n+M1Hp5hC0g8okX3sR3vFQwynaX/UgSOM9MeBKzY= github.com/Masterminds/semver/v3 v3.1.1/go.mod h1:VPu/7SZ7ePZ3QOrcuXROw5FAcLl4a0cBrbBpGY/8hQs= github.com/Microsoft/go-winio v0.6.1 h1:9/kr64B9VUZrLm5YYwbGtUJnMgqWVOdUAXu6Migciow= github.com/Microsoft/go-winio v0.6.1/go.mod h1:LRdKpFKfdobln8UmuiYcKPot9D2v6svN5+sAH+4kjUM= -github.com/Shopify/goreferrer v0.0.0-20181106222321-ec9c9a553398/go.mod h1:a1uqRtAwp2Xwc6WNPJEufxJ7fx3npB4UV/JOLmbu5I0= github.com/VictoriaMetrics/fastcache v1.12.1 h1:i0mICQuojGDL3KblA7wUNlY5lOK6a4bwt3uRKnkZU40= github.com/VictoriaMetrics/fastcache v1.12.1/go.mod h1:tX04vaqcNoQeGLD+ra5pU5sWkuxnzWhEzLwhP9w653o= github.com/aead/siphash v1.0.1/go.mod h1:Nywa3cDsYNNK3gaciGTWPwHt0wlpNV15vwmswBAUSII= -github.com/ajg/form v1.5.1/go.mod h1:uL1WgH+h2mgNtvBq0339dVnzXdBETtL2LeUXaIv25UY= github.com/allegro/bigcache v1.2.1-0.20190218064605-e24eb225f156/go.mod h1:Cb/ax3seSYIx7SuZdm2G2xzfwmv3TPSk2ucNfQESPXM= github.com/allegro/bigcache v1.2.1 h1:hg1sY1raCwic3Vnsvje6TT7/pnZba83LeFck5NrFKSc= github.com/allegro/bigcache v1.2.1/go.mod h1:Cb/ax3seSYIx7SuZdm2G2xzfwmv3TPSk2ucNfQESPXM= github.com/anmitsu/go-shlex v0.0.0-20161002113705-648efa622239/go.mod h1:2FmKhYUyUczH0OGQWaF5ceTx0UBShxjsH6f8oGKYe2c= -github.com/armon/consul-api v0.0.0-20180202201655-eb2c6b5be1b6/go.mod h1:grANhF5doyWs3UAsr3K4I6qtAmlQcZDesFNEHPZAzj8= -github.com/aymerick/raymond v2.0.3-0.20180322193309-b565731e1464+incompatible/go.mod h1:osfaiScAUVup+UC9Nfq76eWqDhXlp+4UYaA8uhTBO6g= github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/benbjohnson/clock v1.3.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/benbjohnson/clock v1.3.5 h1:VvXlSJBzZpA/zum6Sj74hxwYI2DIxRWuNIoXAzHZz5o= @@ -65,7 +57,6 @@ github.com/btcsuite/snappy-go v1.0.0/go.mod h1:8woku9dyThutzjeg+3xrA5iCpBRH8XEEg github.com/btcsuite/websocket v0.0.0-20150119174127-31079b680792/go.mod h1:ghJtEyQwv5/p4Mg4C0fgbePVuGr935/5ddU9Z3TmDRY= github.com/btcsuite/winsvc v1.0.0/go.mod h1:jsenWakMcC0zFBFurPLEAyrnc/teJEM1O46fmI40EZs= github.com/buger/jsonparser v0.0.0-20181115193947-bf1c66bbce23/go.mod h1:bbYlZJ7hK1yFx9hf58LP0zeX7UjIGs20ufpu3evjr+s= -github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/cespare/cp v0.1.0 h1:SE+dxFebS7Iik5LK0tsi1k9ZCxEaFX4AjQmoyA+1dJk= github.com/cespare/cp v0.1.0/go.mod h1:SOGHArjBr4JWaSDEVpWpo/hNg6RoKrls6Oh40hiwW+s= github.com/cespare/xxhash v1.1.0 h1:a6HrQnmkObjyL+Gs60czilIUGqrzKutQD6XZog3p+ko= @@ -80,23 +71,19 @@ github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMn github.com/chzyer/test v0.0.0-20210722231415-061457976a23/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= github.com/cilium/ebpf v0.2.0/go.mod h1:To2CFviqOWL/M0gIMsvSMlqe7em/l1ALkX1PyjrX2Qs= github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= -github.com/cncf/udpa/go v0.0.0-20201120205902-5459f2c99403/go.mod h1:WmhPx2Nbnhtbo57+VJT5O0JRkEi1Wbu0z5j0R8u5Hbk= github.com/cockroachdb/apd v1.1.0/go.mod h1:8Sl8LxpKi29FqWXR16WEFZRNSz3SoPzUzeMeY4+DwBQ= -github.com/cockroachdb/datadriven v1.0.2/go.mod h1:a9RdTaap04u637JoCzcUoIcDmvwSUtcUFtT/C3kJlTU= github.com/cockroachdb/datadriven v1.0.3-0.20230413201302-be42291fc80f h1:otljaYPt5hWxV3MUfO5dFPFiOXg9CyG5/kCfayTqsJ4= github.com/cockroachdb/datadriven v1.0.3-0.20230413201302-be42291fc80f/go.mod h1:a9RdTaap04u637JoCzcUoIcDmvwSUtcUFtT/C3kJlTU= -github.com/cockroachdb/errors v1.9.1 h1:yFVvsI0VxmRShfawbt/laCIDy/mtTqqnvoNgiy5bEV8= -github.com/cockroachdb/errors v1.9.1/go.mod h1:2sxOtL2WIc096WSZqZ5h8fa17rdDq9HZOZLBCor4mBk= -github.com/cockroachdb/logtags v0.0.0-20211118104740-dabe8e521a4f/go.mod h1:Vz9DsVWQQhf3vs21MhPMZpMGSht7O/2vFW2xusFUVOs= +github.com/cockroachdb/errors v1.11.1 h1:xSEW75zKaKCWzR3OfxXUxgrk/NtT4G1MiOv5lWZazG8= +github.com/cockroachdb/errors v1.11.1/go.mod h1:8MUxA3Gi6b25tYlFEBGLf+D8aISL+M4MIpiWMSNRfxw= github.com/cockroachdb/logtags v0.0.0-20230118201751-21c54148d20b h1:r6VH0faHjZeQy818SGhaone5OnYfxFR/+AzdY3sf5aE= github.com/cockroachdb/logtags v0.0.0-20230118201751-21c54148d20b/go.mod h1:Vz9DsVWQQhf3vs21MhPMZpMGSht7O/2vFW2xusFUVOs= -github.com/cockroachdb/pebble v0.0.0-20230928194634-aa077af62593 h1:aPEJyR4rPBvDmeyi+l/FS/VtA00IWvjeFvjen1m1l1A= -github.com/cockroachdb/pebble v0.0.0-20230928194634-aa077af62593/go.mod h1:6hk1eMY/u5t+Cf18q5lFMUA1Rc+Sm5I6Ra1QuPyxXCo= -github.com/cockroachdb/redact v1.1.3 h1:AKZds10rFSIj7qADf0g46UixK8NNLwWTNdCIGS5wfSQ= -github.com/cockroachdb/redact v1.1.3/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg= +github.com/cockroachdb/pebble v0.0.0-20231018212520-f6cde3fc2fa4 h1:PuHFhOUMnD62r80dN+Ik5qco2drekgsUSVdcHsvllec= +github.com/cockroachdb/pebble v0.0.0-20231018212520-f6cde3fc2fa4/go.mod h1:sEHm5NOXxyiAoKWhoFxT8xMgd/f3RA6qUqQ1BXKrh2E= +github.com/cockroachdb/redact v1.1.5 h1:u1PMllDkdFfPWaNGMyLD1+so+aq3uUItthCFqzwPJ30= +github.com/cockroachdb/redact v1.1.5/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg= github.com/cockroachdb/tokenbucket v0.0.0-20230807174530-cc333fc44b06 h1:zuQyyAKVxetITBuuhv3BI9cMrmStnpT18zmgmTxunpo= github.com/cockroachdb/tokenbucket v0.0.0-20230807174530-cc333fc44b06/go.mod h1:7nc4anLGjupUW/PeY5qiNYsdNXj7zopG+eqsS7To5IQ= -github.com/codegangsta/inject v0.0.0-20150114235600-33e0aa1cb7c0/go.mod h1:4Zcjuz89kmFXt9morQgcfYZAYZ5n8WHjt81YYWIwtTM= github.com/consensys/bavard v0.1.13 h1:oLhMLOFGTLdlda/kma4VOJazblc7IM5y5QPd2A/YjhQ= github.com/consensys/bavard v0.1.13/go.mod h1:9ItSMtA/dXMAiL7BG6bqW2m3NdSEObYWoH223nGHukI= github.com/consensys/gnark-crypto v0.12.1 h1:lHH39WuuFgVHONRl3J0LRBtuYdQTumFSDtJF7HpyG8M= @@ -104,16 +91,12 @@ github.com/consensys/gnark-crypto v0.12.1/go.mod h1:v2Gy7L/4ZRosZ7Ivs+9SfUDr0f5U github.com/containerd/cgroups v0.0.0-20201119153540-4cbc285b3327/go.mod h1:ZJeTFisyysqgcCdecO57Dj79RfL0LNeGiFUqLYQRYLE= github.com/containerd/cgroups v1.1.0 h1:v8rEWFl6EoqHB+swVNjVoCJE8o3jX7e8nqBGPLaDFBM= github.com/containerd/cgroups v1.1.0/go.mod h1:6ppBcbh/NOOUU+dMKrykgaBnK9lCIBxHqJDGwsa1mIw= -github.com/coreos/etcd v3.3.10+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE= -github.com/coreos/go-etcd v2.0.0+incompatible/go.mod h1:Jez6KQU2B/sWsbdaef3ED8NzMklzPG4d5KIOhIy30Tk= -github.com/coreos/go-semver v0.2.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= github.com/coreos/go-systemd v0.0.0-20181012123002-c6f51f82210d/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= github.com/coreos/go-systemd v0.0.0-20190321100706-95778dfbb74e/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= github.com/coreos/go-systemd v0.0.0-20190719114852-fd7a80b32e1f/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= github.com/coreos/go-systemd/v22 v22.1.0/go.mod h1:xO0FLkIi5MaZafQlIrOotqXZ90ih+1atmu1JpKERPPk= github.com/coreos/go-systemd/v22 v22.5.0 h1:RrqgGjYQKalulkV8NGVIfkXQf6YYmOyiJKk8iXXhfZs= github.com/coreos/go-systemd/v22 v22.5.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= -github.com/cpuguy83/go-md2man v1.0.10/go.mod h1:SmD6nW6nTyfqj6ABTjUi3V3JVMnlJmwcJI5acqYI6dE= github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= github.com/cpuguy83/go-md2man/v2 v2.0.0/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= github.com/cpuguy83/go-md2man/v2 v2.0.2 h1:p1EgwI/C7NhT0JmVkwCD2ZBK8j4aeHQX2pMHHBfMQ6w= @@ -143,13 +126,11 @@ github.com/decred/dcrd/lru v1.0.0/go.mod h1:mxKOwFd7lFjN2GZYsiz/ecgqR6kkYAl+0pz0 github.com/deepmap/oapi-codegen v1.6.0/go.mod h1:ryDa9AgbELGeB+YEXE1dR53yAjHwFvE9iAUlWl9Al3M= github.com/deepmap/oapi-codegen v1.8.2 h1:SegyeYGcdi0jLLrpbCMoJxnUUn8GBXHsvr4rbzjuhfU= github.com/deepmap/oapi-codegen v1.8.2/go.mod h1:YLgSKSDv/bZQB7N4ws6luhozi3cEdRktEqrX88CvjIw= -github.com/dgraph-io/badger v1.6.0/go.mod h1:zwt7syl517jmP8s94KqSxTlM6IMsdhYy6psNgSztDR4= github.com/dgraph-io/badger v1.6.2 h1:mNw0qs90GVgGGWylh0umH5iag1j6n/PeJtNvL6KY/x8= github.com/dgraph-io/badger v1.6.2/go.mod h1:JW2yswe3V058sS0kZ2h/AXeDSqFjxnZcRrVH//y2UQE= github.com/dgraph-io/ristretto v0.0.2 h1:a5WaUrDa0qm0YrAAS1tUykT5El3kt62KNZZeMxQn3po= github.com/dgraph-io/ristretto v0.0.2/go.mod h1:KPxhHT9ZxKefz+PCeOGsrHpl1qZ7i70dGTu2u+Ahh6E= github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ= -github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= github.com/dlclark/regexp2 v1.4.1-0.20201116162257-a2a8dda75c91/go.mod h1:2pZnwuY/m+8K6iRw6wQdMtk+rH5tNGR1i55kozfMjCc= github.com/dlclark/regexp2 v1.7.0 h1:7lJfhqlPssTb1WQx4yvTHN0uElPEv52sbaECrAQxjAo= github.com/dlclark/regexp2 v1.7.0/go.mod h1:DHkYz0B9wPfa6wondMfaivmHpzrQ3v9q8cnmRbL6yW8= @@ -165,15 +146,9 @@ github.com/dop251/goja_nodejs v0.0.0-20210225215109-d91c329300e7/go.mod h1:hn7BA github.com/dop251/goja_nodejs v0.0.0-20211022123610-8dd9abb0616d/go.mod h1:DngW8aVqWbuLRMHItjPUyqdj+HWPvnQe8V8y1nDpIbM= github.com/dustin/go-humanize v1.0.0 h1:VSnTsYCnlFHaM2/igO1h6X3HA71jcobQuxemgkq4zYo= github.com/dustin/go-humanize v1.0.0/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= -github.com/eknkc/amber v0.0.0-20171010120322-cdade1c07385/go.mod h1:0vRUJqYpeSZifjYj7uP3BG/gKcuzL9xWVV/Y+cK33KM= github.com/elastic/gosigar v0.12.0/go.mod h1:iXRIGg2tLnu7LBdpqzyQfGDEidKCfWcCMS0WKyPWoMs= github.com/elastic/gosigar v0.14.2 h1:Dg80n8cr90OZ7x+bAax/QjoW/XqTI11RmA79ZwIm9/4= github.com/elastic/gosigar v0.14.2/go.mod h1:iXRIGg2tLnu7LBdpqzyQfGDEidKCfWcCMS0WKyPWoMs= -github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= -github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= -github.com/envoyproxy/go-control-plane v0.9.9-0.20210217033140-668b12f5399d/go.mod h1:cXg6YxExXjJnVBQHBLXeUAgxn2UodCpnH306RInaBQk= -github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= -github.com/etcd-io/bbolt v1.3.3/go.mod h1:ZF2nL25h33cCyBtcyWeZ2/I3HQOfTP+0PIEvHjkjCrw= github.com/ethereum-optimism/go-ethereum-hdwallet v0.1.3 h1:RWHKLhCrQThMfch+QJ1Z8veEq5ZO3DfIhZ7xgRP9WTc= github.com/ethereum-optimism/go-ethereum-hdwallet v0.1.3/go.mod h1:QziizLAiF0KqyLdNJYD7O5cpDlaFMNZzlxYNcWsJUxs= github.com/ethereum-optimism/op-geth v1.101301.2-0.20231018010910-b142ad43dda3 h1:WdAevEuDn3I1YPChnIlgqCXvDFx2/oRvcvDG7aU86sk= @@ -182,10 +157,8 @@ github.com/ethereum-optimism/superchain-registry/superchain v0.0.0-2023101820222 github.com/ethereum-optimism/superchain-registry/superchain v0.0.0-20231018202221-fdba3d104171/go.mod h1:/70H/KqrtKcvWvNGVj6S3rAcLC+kUPr3t2aDmYIS+Xk= github.com/ethereum/c-kzg-4844 v0.3.1 h1:sR65+68+WdnMKxseNWxSJuAv2tsUrihTpVBTfM/U5Zg= github.com/ethereum/c-kzg-4844 v0.3.1/go.mod h1:VewdlzQmpT5QSrVhbBuGoCdFJkpaJlO1aQputP83wc0= -github.com/fasthttp-contrib/websocket v0.0.0-20160511215533-1f3b11f56072/go.mod h1:duJ4Jxv5lDcvg4QuQr0oowTf7dz4/CR8NtyCooz9HL8= github.com/fatih/color v1.13.0 h1:8LOYc1KYPPmyKMuN8QV2DNRWNbLo6LZ0iLs8+mlH53w= github.com/fatih/color v1.13.0/go.mod h1:kLAiJbzzSOZDVNGyDpeOxJ47H46qBXwg5ILebYFFOfk= -github.com/fatih/structs v1.1.0/go.mod h1:9NiDSp5zOcgEDl+j00MP/WkGVPOlPRLejGD8Ga6PJ7M= github.com/felixge/fgprof v0.9.3 h1:VvyZxILNuCiUCSXtPtYmmtGvb65nqXh2QFWc0Wpf2/g= github.com/felixge/fgprof v0.9.3/go.mod h1:RdbpDgzqYVh/T9fPELJyV7EYJuHB55UTEULNun8eiPw= github.com/fjl/memsize v0.0.1 h1:+zhkb+dhUgx0/e+M8sF0QqiouvMQUiKR+QYvdxIOKcQ= @@ -200,21 +173,16 @@ github.com/fsnotify/fsnotify v1.4.9/go.mod h1:znqG4EE+3YCdAaPaxE2ZRY/06pZUdp0tY4 github.com/fsnotify/fsnotify v1.5.4/go.mod h1:OVB6XrOHzAwXMpEM7uPOzcehqUV2UqJxmVXmkdnm1bU= github.com/fsnotify/fsnotify v1.6.0 h1:n+5WquG0fcWoWp6xPWfHdbskMCQaFnG6PfBrh1Ky4HY= github.com/fsnotify/fsnotify v1.6.0/go.mod h1:sl3t1tCWJFWoRz9R8WJCbQihKKwmorjAbSClcnxKAGw= -github.com/gavv/httpexpect v2.0.0+incompatible/go.mod h1:x+9tiU1YnrOvnB725RkpoLv1M62hOWzwo5OXotisrKc= github.com/gballet/go-libpcsclite v0.0.0-20191108122812-4678299bea08 h1:f6D9Hr8xV8uYKlyuj8XIruxlh9WjVjdh1gIicAS7ays= github.com/gballet/go-libpcsclite v0.0.0-20191108122812-4678299bea08/go.mod h1:x7DCsMOv1taUwEWCzT4cmDeAkigA5/QCwUodaVOe8Ww= github.com/gballet/go-verkle v0.0.0-20230607174250-df487255f46b h1:vMT47RYsrftsHSTQhqXwC3BYflo38OLC3Y4LtXtLyU0= github.com/gballet/go-verkle v0.0.0-20230607174250-df487255f46b/go.mod h1:CDncRYVRSDqwakm282WEkjfaAj1hxU/v5RXxk5nXOiI= github.com/getkin/kin-openapi v0.53.0/go.mod h1:7Yn5whZr5kJi6t+kShccXS8ae1APpYTW6yheSwk8Yi4= github.com/getkin/kin-openapi v0.61.0/go.mod h1:7Yn5whZr5kJi6t+kShccXS8ae1APpYTW6yheSwk8Yi4= -github.com/getsentry/sentry-go v0.12.0/go.mod h1:NSap0JBYWzHND8oMbyi0+XZhUalc1TBdRL1M71JZW2c= github.com/getsentry/sentry-go v0.18.0 h1:MtBW5H9QgdcJabtZcuJG80BMOwaBpkRDZkxRkNC1sN0= github.com/getsentry/sentry-go v0.18.0/go.mod h1:Kgon4Mby+FJ7ZWHFUAZgVaIa8sxHtnRJRLTXZr51aKQ= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= -github.com/gin-contrib/sse v0.0.0-20190301062529-5545eab6dad3/go.mod h1:VJ0WA2NBN22VlZ2dKZQPAPnyWw5XTlK1KymzLKsr59s= -github.com/gin-gonic/gin v1.4.0/go.mod h1:OW2EZn3DO8Ln9oIKOvM++LBO+5UPHJJDH72/q/3rZdM= github.com/gliderlabs/ssh v0.1.1/go.mod h1:U7qILu1NlMHj9FlMhZLlkCdDnU1DBEAqr0aevW3Awn0= -github.com/go-check/check v0.0.0-20180628173108-788fd7840127/go.mod h1:9ES+weclKsC9YodN5RgxqK/VD9HM9JsCSh7rNhMZE98= github.com/go-chi/chi/v5 v5.0.0/go.mod h1:BBug9lr0cqtdAhsu6R4AAdvufI0/XBzAQSsUqJpoZOs= github.com/go-chi/chi/v5 v5.0.1/go.mod h1:DslCQbL2OYiznFReuXYUmQ2hGd1aDpCnlMNITLSKoi8= github.com/go-chi/chi/v5 v5.0.10 h1:rLz5avzKpjqxrYwXNfmjkrYYXOyLJd37pz53UFHC6vk= @@ -229,7 +197,6 @@ github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vb github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= github.com/go-logr/logr v1.2.4 h1:g01GSCwiDw2xSZfjJ2/T9M+S6pFdcNtFYsp+Y43HYDQ= github.com/go-logr/logr v1.2.4/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= -github.com/go-martini/martini v0.0.0-20170121215854-22fa46961aab/go.mod h1:/P9AEU963A2AYjv4d1V5eVL1CQbEJq6aCNHDDjibzu8= github.com/go-ole/go-ole v1.2.6 h1:/Fpf6oFPoeFik9ty7siob0G6Ke8QvQEuVcuChpwXzpY= github.com/go-ole/go-ole v1.2.6/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= github.com/go-openapi/jsonpointer v0.19.5/go.mod h1:Pl9vOtqEWErmShwVjC8pYs9cog34VGT37dQOVbmoatg= @@ -242,9 +209,6 @@ github.com/go-stack/stack v1.8.1/go.mod h1:dcoOX6HbPZSZptuspn9bctJ+N/CnF5gGygcUP github.com/go-task/slim-sprig v0.0.0-20210107165309-348f09dbbbc0/go.mod h1:fyg7847qk6SyHyPtNmDHnmrv/HOrqktSC+C9fM+CJOE= github.com/go-task/slim-sprig v0.0.0-20230315185526-52ccab3ef572 h1:tfuBGBXKqDEevZMzYi5KSi8KkcZtzBcTgAUUtapy0OI= github.com/go-task/slim-sprig v0.0.0-20230315185526-52ccab3ef572/go.mod h1:9Pwr4B2jHnOSGXyyzV8ROjYa2ojvAY6HCGYYfMoC3Ls= -github.com/gobwas/httphead v0.0.0-20180130184737-2c6c146eadee/go.mod h1:L0fX3K22YWvt/FAX9NnzrNzcI4wNYi9Yku4O0LKYflo= -github.com/gobwas/pool v0.2.0/go.mod h1:q8bcK0KcYlCgd9e7WYLm9LpyS+YeLd8JVDW6WezmKEw= -github.com/gobwas/ws v1.0.2/go.mod h1:szmBTxLgaFppYjEmNtny/v3w89xOydFnnZMcgRRu/EM= github.com/godbus/dbus/v5 v5.0.3/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= github.com/godbus/dbus/v5 v5.0.4/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= github.com/godbus/dbus/v5 v5.1.0 h1:4KLkAxT3aOY8Li4FRJe/KvhoNFFxo0m6fNuFUO8QJUk= @@ -252,15 +216,10 @@ github.com/godbus/dbus/v5 v5.1.0/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5x github.com/gofrs/flock v0.8.1 h1:+gYjHKf32LDeiEEFhQaotPbLuUXjY5ZqxKgXy7n59aw= github.com/gofrs/flock v0.8.1/go.mod h1:F1TvTiK9OcQqauNUHlbJvyl9Qa1QvF/gOUDKA14jxHU= github.com/gofrs/uuid v4.0.0+incompatible/go.mod h1:b2aQJv3Z4Fp6yNu3cdSllBxTCLRxnplIgP/c0N/04lM= -github.com/gogo/googleapis v0.0.0-20180223154316-0cd9801be74a/go.mod h1:gf4bu3Q80BeJ6H1S1vYPm8/ELATdvryBaNFGgqEef3s= -github.com/gogo/googleapis v1.4.1/go.mod h1:2lpHqI5OcWCtVElxXnPt+s8oJvMpySlOyM6xDCrzib4= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= -github.com/gogo/protobuf v1.2.0/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.3.1/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXPKa29o= github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= -github.com/gogo/status v1.1.0/go.mod h1:BFv9nrluPLmrS0EmGVvLaPNmRosr9KapBYd5/hpY1WM= -github.com/golang-jwt/jwt v3.2.2+incompatible/go.mod h1:8pz2t5EyA70fFQQSrl6XZXzqecmYZeUEB8OUGHkxJ+I= github.com/golang-jwt/jwt/v4 v4.5.0 h1:7cYmW1XlMY7h7ii7UhUyChSgS5wUJEnm9uZVTGqOWzg= github.com/golang-jwt/jwt/v4 v4.5.0/go.mod h1:m21LjoU+eqJr34lmDMbreY2eSTRJ1cv77w39/MY0Ch0= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= @@ -271,13 +230,11 @@ github.com/golang/mock v1.6.0 h1:ErTB+efbowRARo13NNdxyJji2egdxLGQhRaY+DUumQc= github.com/golang/mock v1.6.0/go.mod h1:p6yTPP+5HYm5mzsMV8JkE6ZKdX+/wYM6Hr+LicevLPs= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8= github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA= github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs= github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w= github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0= -github.com/golang/protobuf v1.4.1/go.mod h1:U8fpvMrcmy5pZrNK1lt4xCsGvpyWQ/VVv6QDs8UjoX8= github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= @@ -288,20 +245,17 @@ github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEW github.com/golang/snappy v0.0.5-0.20220116011046-fa5810519dcb h1:PBC98N2aIaM3XXiurYmW7fx4GZkL8feAMVq7nEjURHk= github.com/golang/snappy v0.0.5-0.20220116011046-fa5810519dcb/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/golangci/lint-1 v0.0.0-20181222135242-d2cdd8c08219/go.mod h1:/X8TswGSh1pIozq4ZwCfxS0WA5JGXguxk94ar/4c87Y= -github.com/gomodule/redigo v1.7.1-0.20190724094224-574c33c3df38/go.mod h1:B4C85qUVwatsJoIUNIfCRsp7qO0iAmpGFZ4EELWSbC4= github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.2/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/go-github v17.0.0+incompatible/go.mod h1:zLgOLi98H3fifZn+44m+umXrS52loVEgC2AApnigrVQ= github.com/google/go-querystring v1.0.0/go.mod h1:odCYkC5MyYFN7vkCjXpyrEuKhc/BUO6wN/zVPAxq5ck= -github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= github.com/google/gofuzz v1.2.1-0.20220503160820-4a35382e8fc8 h1:Ep/joEub9YwcjRY6ND3+Y/w0ncE540RtGatVhtZL0/Q= github.com/google/gofuzz v1.2.1-0.20220503160820-4a35382e8fc8/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= github.com/google/gopacket v1.1.19 h1:ves8RnFZPGiFnTS0uPQStjwru6uO6h+nlr9j6fL7kF8= @@ -316,14 +270,12 @@ github.com/google/pprof v0.0.0-20230817174616-7a8ec2ada47b/go.mod h1:czg5+yv1E0Z github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= github.com/google/subcommands v1.2.0/go.mod h1:ZjhPrFU+Olkh9WazFPsl27BQ4UPiG37m3yTrtFlrHVk= github.com/google/uuid v1.1.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= -github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.3.1 h1:KjJaJ9iWZ3jOFZIf1Lqf4laDRCasjl0BCmnEGxkdLb4= github.com/google/uuid v1.3.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/googleapis/gax-go v2.0.0+incompatible/go.mod h1:SFVmujtThgffbyetf+mdk2eWhX2bMyUtNHzFKcPA9HY= github.com/googleapis/gax-go/v2 v2.0.3/go.mod h1:LLvjysVCY1JZeum8Z6l8qUty8fiNwE08qbEPm1M08qg= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= github.com/gorilla/mux v1.8.0/go.mod h1:DVbg23sWSpFRCP0SfiEN6jmj59UnW/n46BH5rLB71So= -github.com/gorilla/websocket v1.4.1/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE= github.com/gorilla/websocket v1.5.0 h1:PPwGk2jz7EePpoHN/+ClbZu8SPxiqlu12wZP/3sWmnc= github.com/gorilla/websocket v1.5.0/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE= github.com/graph-gophers/graphql-go v1.3.0 h1:Eb9x/q6MFpCLz7jBCiP/WTxjSDrYLR1QY41SORZyNJ0= @@ -337,12 +289,10 @@ github.com/hashicorp/go-bexpr v0.1.11 h1:6DqdA/KBjurGby9yTY0bmkathya0lfwF2SeuubC github.com/hashicorp/go-bexpr v0.1.11/go.mod h1:f03lAo0duBlDIUMGCuad8oLcgejw4m7U+N8T+6Kz1AE= github.com/hashicorp/go-multierror v1.1.1 h1:H5DkEtf6CXdFp0N0Em5UCwQpXMWke8IA0+lD48awMYo= github.com/hashicorp/go-multierror v1.1.1/go.mod h1:iw975J/qwKPdAO1clOe2L8331t/9/fmwbPZ6JB6eMoM= -github.com/hashicorp/go-version v1.2.0/go.mod h1:fltr4n8CU8Ke44wwGCBoEymUuxUHl09ZGVZPK5anwXA= github.com/hashicorp/golang-lru/arc/v2 v2.0.5 h1:l2zaLDubNhW4XO3LnliVj0GXO3+/CGNJAg1dcN2Fpfw= github.com/hashicorp/golang-lru/arc/v2 v2.0.5/go.mod h1:ny6zBSQZi2JxIeYcv7kt2sH2PXJtirBN7RDhRpxPkxU= github.com/hashicorp/golang-lru/v2 v2.0.5 h1:wW7h1TG88eUIJ2i69gaE3uNVtEPIagzhGvHgwfx2Vm4= github.com/hashicorp/golang-lru/v2 v2.0.5/go.mod h1:QeFd9opnmA6QUJc5vARoKUSoFhyfM2/ZepoAG6RGpeM= -github.com/hashicorp/hcl v1.0.0/go.mod h1:E5yfLk+7swimpb2L/Alb/PJmXilQ/rhwaUYs4T20WEQ= github.com/holiman/billy v0.0.0-20230718173358-1c7e68d277a7 h1:3JQNjnMRil1yD0IfZKHF9GxxWKDJGj8I0IqOUol//sw= github.com/holiman/billy v0.0.0-20230718173358-1c7e68d277a7/go.mod h1:5GuXa7vkL8u9FkFuWdVvfR5ix8hRB7DbOAaYULamFpc= github.com/holiman/bloomfilter/v2 v2.0.3 h1:73e0e/V0tCydx14a0SCYS/EWCxgwLZ18CZcZKVu0fao= @@ -352,12 +302,9 @@ github.com/holiman/uint256 v1.2.3/go.mod h1:SC8Ryt4n+UBbPbIBKaG9zbbDlp4jOru9xFZm github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= github.com/huin/goupnp v1.3.0 h1:UvLUlWDNpoUdYzb2TCn+MuTWtcjXKSza2n6CBdQ0xXc= github.com/huin/goupnp v1.3.0/go.mod h1:gnGPsThkYa7bFi/KWmEysQRf48l2dvR5bxr2OFckNX8= -github.com/hydrogen18/memlistener v0.0.0-20200120041712-dcc25e7acd91/go.mod h1:qEIFzExnS6016fRpRfxrExeVn2gbClQA99gQhnIcdhE= github.com/ianlancetaylor/demangle v0.0.0-20200824232613-28f6c0f3b639/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= github.com/ianlancetaylor/demangle v0.0.0-20210905161508-09a460cdf81d/go.mod h1:aYm2/VgdVmcIU8iMfdMvDMsRAQjcfZSKFby6HOFvi/w= github.com/ianlancetaylor/demangle v0.0.0-20220319035150-800ac71e25c2/go.mod h1:aYm2/VgdVmcIU8iMfdMvDMsRAQjcfZSKFby6HOFvi/w= -github.com/imkira/go-interpol v1.1.0/go.mod h1:z0h2/2T3XF8kyEPpRgJ3kmNv+C43p+I/CoI+jC3w2iA= -github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= github.com/influxdata/influxdb-client-go/v2 v2.4.0 h1:HGBfZYStlx3Kqvsv1h2pJixbCl/jhnFtxpKFAv9Tu5k= github.com/influxdata/influxdb-client-go/v2 v2.4.0/go.mod h1:vLNHdxTJkIf2mSLvGrpj8TCcISApPoXkaxP8g9uRlW8= github.com/influxdata/influxdb1-client v0.0.0-20220302092344-a9ab5670611c h1:qSHzRbhzK8RdXOsAdfDgO49TtqC1oZ+acxPrkfTxcCs= @@ -379,11 +326,6 @@ github.com/ipfs/go-ds-leveldb v0.5.0/go.mod h1:d3XG9RUDzQ6V4SHi8+Xgj9j1XuEk1z82l github.com/ipfs/go-ipfs-delay v0.0.0-20181109222059-70721b86a9a8/go.mod h1:8SP1YXK1M1kXuc4KJZINY3TQQ03J2rwBG9QfXmbRPrw= github.com/ipfs/go-log/v2 v2.5.1 h1:1XdUzF7048prq4aBjDQQ4SL5RxftpRGdXhNRwKSAlcY= github.com/ipfs/go-log/v2 v2.5.1/go.mod h1:prSpmC1Gpllc9UYWxDiZDreBYw7zp4Iqp1kOLU9U5UI= -github.com/iris-contrib/blackfriday v2.0.0+incompatible/go.mod h1:UzZ2bDEoaSGPbkg6SAB4att1aAwTmVIx/5gCVqeyUdI= -github.com/iris-contrib/go.uuid v2.0.0+incompatible/go.mod h1:iz2lgM/1UnEf1kP0L/+fafWORmlnuysV2EMP8MW+qe0= -github.com/iris-contrib/jade v1.1.3/go.mod h1:H/geBymxJhShH5kecoiOCSssPX7QWYH7UaeZTSWddIk= -github.com/iris-contrib/pongo2 v0.0.1/go.mod h1:Ssh+00+3GAZqSQb30AvBRNxBx7rf0GqwkjqxNd0u65g= -github.com/iris-contrib/schema v0.0.1/go.mod h1:urYA3uvUNG1TIIjOSCzHr9/LmbQo8LrOcOqfqxa4hXw= github.com/jackc/chunkreader v1.0.0 h1:4s39bBR8ByfqH+DKm8rQA3E1LHZWB9XWcrz8fqaZbe0= github.com/jackc/chunkreader v1.0.0/go.mod h1:RT6O25fNZIuasFJRyZ4R/Y2BbhasbmZXF9QQ7T3kePo= github.com/jackc/chunkreader/v2 v2.0.0/go.mod h1:odVSm741yZoC3dpHEUXIqA9tQRhFrgOHwnPIn9lDKlk= @@ -449,28 +391,17 @@ github.com/jinzhu/now v1.1.5 h1:/o9tlHleP7gOFmsnYNz3RGnqzefHA47wQpKrrdTIwXQ= github.com/jinzhu/now v1.1.5/go.mod h1:d3SSVoowX0Lcu0IBviAWJpolVfI5UJVZZ7cO71lE/z8= github.com/jrick/logrotate v1.0.0/go.mod h1:LNinyqDIJnpAur+b8yyulnQw/wDuN1+BYKlTRt3OuAQ= github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= -github.com/json-iterator/go v1.1.9/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= -github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= github.com/julienschmidt/httprouter v1.3.0 h1:U0609e9tgbseu3rBINet9P48AI/D3oJs4dN7jwJOQ1U= github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8IZAc4RVcycCCAKdM= -github.com/k0kubun/colorstring v0.0.0-20150214042306-9440f1994b88/go.mod h1:3w7q1U84EfirKl04SVQ/s7nPm1ZPhiXd34z40TNz36k= github.com/karalabe/usb v0.0.3-0.20230711191512-61db3e06439c h1:AqsttAyEyIEsNz5WLRwuRwjiT5CMDUfLk6cFJDVPebs= github.com/karalabe/usb v0.0.3-0.20230711191512-61db3e06439c/go.mod h1:Od972xHfMJowv7NGVDiWVxk2zxnWgjLlJzE+F4F7AGU= -github.com/kataras/golog v0.0.10/go.mod h1:yJ8YKCmyL+nWjERB90Qwn+bdyBZsaQwU3bTVFgkFIp8= -github.com/kataras/iris/v12 v12.1.8/go.mod h1:LMYy4VlP67TQ3Zgriz8RE2h2kMZV2SgMYbq3UhfoFmE= -github.com/kataras/neffos v0.0.14/go.mod h1:8lqADm8PnbeFfL7CLXh1WHw53dG27MC3pgi2R1rmoTE= -github.com/kataras/pio v0.0.2/go.mod h1:hAoW0t9UmXi4R5Oyq5Z4irTbaTsOemSrDGUtaTl7Dro= -github.com/kataras/sitemap v0.0.5/go.mod h1:KY2eugMKiPwsJgx7+U103YZehfvNGOXURubcGyk0Bz8= github.com/kisielk/errcheck v1.2.0/go.mod h1:/BMXB+zMLi60iA8Vv6Ksmxu/1UDYcXs4uQLJ+jE2L00= github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/kkdai/bstream v0.0.0-20161212061736-f391b8402d23/go.mod h1:J+Gs4SYgM6CZQHDETBtE9HaSEkGmuNXF86RwHhHUvq4= -github.com/klauspost/compress v1.8.2/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= -github.com/klauspost/compress v1.9.7/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= github.com/klauspost/compress v1.16.7 h1:2mk3MPGNzKyxErAw8YaohYh69+pa4sIQSC0fPGCFR9I= github.com/klauspost/compress v1.16.7/go.mod h1:ntbaceVETuRiXiv4DpjP66DpAtAGkEQskQzEyD//IeE= -github.com/klauspost/cpuid v1.2.1/go.mod h1:Pj4uuM528wm8OyEC2QMXAi2YiTZ96dNQPGgoMS4s3ek= github.com/klauspost/cpuid/v2 v2.2.5 h1:0E5MSMDEoAulmXNFquVs//DdoomxaoTY1kUhbc/qbZg= github.com/klauspost/cpuid/v2 v2.2.5/go.mod h1:Lcz8mBdAVJIBVzewtcLocK12l3Y+JytZYpaMropDUws= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= @@ -492,7 +423,6 @@ github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= github.com/kylelemons/godebug v1.1.0 h1:RPNrshWIDI6G2gRW9EHilWtl7Z6Sb1BR0xunSBf0SNc= github.com/kylelemons/godebug v1.1.0/go.mod h1:9/0rRGxNHcop5bhtWyNeEfOS8JIWk580+fNqagV/RAw= github.com/labstack/echo/v4 v4.2.1/go.mod h1:AA49e0DZ8kk5jTOOCKNuPR6oTnBS0dYiM4FW1e6jwpg= -github.com/labstack/echo/v4 v4.5.0/go.mod h1:czIriw4a0C1dFun+ObrXp7ok03xON0N1awStJ6ArI7Y= github.com/labstack/gommon v0.3.0/go.mod h1:MULnywXg0yavhxWKc+lOruYdAhDwPK9wf0OL7NoOu+k= github.com/leanovate/gopter v0.2.9 h1:fQjYxZaynp97ozCzfOyOuAGOU4aU/z37zf/tOujFk7c= github.com/leanovate/gopter v0.2.9/go.mod h1:U2L/78B+KVFIx2VmW6onHJQzXtFb+p5y3y2Sh+Jxxv8= @@ -531,7 +461,6 @@ github.com/libp2p/go-reuseport v0.4.0/go.mod h1:ZtI03j/wO5hZVDFo2jKywN6bYKWLOy8S github.com/libp2p/go-yamux/v4 v4.0.1 h1:FfDR4S1wj6Bw2Pqbc8Uz7pCxeRBPbwsBbEdfwiCypkQ= github.com/libp2p/go-yamux/v4 v4.0.1/go.mod h1:NWjl8ZTLOGlozrXSOZ/HlfG++39iKNnM5wwmtQP1YB4= github.com/lunixbochs/vtclean v1.0.0/go.mod h1:pHhQNgMf3btfWnGBVipUOjRYhoOsdGqdm/+2c2E2WMI= -github.com/magiconair/properties v1.8.0/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ= github.com/mailru/easyjson v0.0.0-20190312143242-1de009706dbe/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= github.com/mailru/easyjson v0.0.0-20190614124828-94de47d64c63/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= github.com/mailru/easyjson v0.0.0-20190626092158-b2ccc519800e/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= @@ -544,7 +473,6 @@ github.com/mattn/go-colorable v0.1.6/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope github.com/mattn/go-colorable v0.1.7/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope9wVRipJSqc= github.com/mattn/go-colorable v0.1.8/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope9wVRipJSqc= github.com/mattn/go-colorable v0.1.9/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope9wVRipJSqc= -github.com/mattn/go-colorable v0.1.11/go.mod h1:u5H1YNBxpqRaxsYJYSkiCWKzEfiAb1Gb520KVy5xxl4= github.com/mattn/go-colorable v0.1.13 h1:fFA4WZxdEF4tXPZVKMLwD8oUnCTTo08duU7wxecdEvA= github.com/mattn/go-colorable v0.1.13/go.mod h1:7S9/ev0klgBDR4GtXTXX8a3vIGJpMovkB8vQcUbaXHg= github.com/mattn/go-isatty v0.0.5/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s= @@ -560,13 +488,10 @@ github.com/mattn/go-runewidth v0.0.3/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzp github.com/mattn/go-runewidth v0.0.9/go.mod h1:H031xJmbD/WCDINGzjvQ9THkh0rPKHF+m2gUSrubnMI= github.com/mattn/go-runewidth v0.0.14 h1:+xnbZSEeDbOIg5/mE6JF0w6n9duR1l3/WmbinWVwUuU= github.com/mattn/go-runewidth v0.0.14/go.mod h1:Jdepj2loyihRzMpdS35Xk/zdY8IAYHsh153qUoGf23w= -github.com/mattn/goveralls v0.0.2/go.mod h1:8d1ZMHsd7fW6IRPKQh46F2WRpyib5/X4FOpevwGNQEw= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= github.com/matttproud/golang_protobuf_extensions v1.0.4 h1:mmDVorXM7PCGKw94cs5zkfA9PSy5pEvNWRP0ET0TIVo= github.com/matttproud/golang_protobuf_extensions v1.0.4/go.mod h1:BSXmuO+STAnVfrANrmjBb36TMTDstsz7MSK+HVaYKv4= -github.com/mediocregopher/radix/v3 v3.4.2/go.mod h1:8FL3F6UQRXHXIBSPUs5h0RybMF8i4n7wVopoX3x7Bv8= github.com/microcosm-cc/bluemonday v1.0.1/go.mod h1:hsXNsILzKxV+sX77C5b8FSuKF00vh2OMYv+xgHpAMF4= -github.com/microcosm-cc/bluemonday v1.0.2/go.mod h1:iVP4YcDBq+n/5fb23BhYFvIMq/leAFZyRl6bYmGDlGc= github.com/miekg/dns v1.1.41/go.mod h1:p6aan82bvRIyn+zDIv9xYNUpwa73JcSh9BKwknJysuI= github.com/miekg/dns v1.1.55 h1:GoQ4hpsj0nFLYe+bWiCToyrBEJXkQfOOIvFGFy0lEgo= github.com/miekg/dns v1.1.55/go.mod h1:uInx36IzPl7FYnDcMeVWxj9byh7DutNykX4G9Sj60FY= @@ -580,8 +505,6 @@ github.com/minio/blake2b-simd v0.0.0-20160723061019-3f5f724cb5b1/go.mod h1:pD8Rv github.com/minio/sha256-simd v0.1.1-0.20190913151208-6de447530771/go.mod h1:B5e1o+1/KgNmWrSQK08Y6Z1Vb5pwIktudl0J58iy0KM= github.com/minio/sha256-simd v1.0.1 h1:6kaan5IFmwTNynnKKpDHe6FWHohJOHhCPchzK49dzMM= github.com/minio/sha256-simd v1.0.1/go.mod h1:Pz6AKMiUdngCLpeTL/RJY1M9rUuPMYujV5xJjtbRSN8= -github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= -github.com/mitchellh/mapstructure v1.1.2/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= github.com/mitchellh/mapstructure v1.4.1/go.mod h1:bFUtVrKA4DC2yAKiSyO/QUcy7e+RRV2QTWOzhPopBRo= github.com/mitchellh/mapstructure v1.5.0 h1:jeMsZIYE/09sWLaz43PL7Gy6RuMjD2eJVyuac5Z2hdY= github.com/mitchellh/mapstructure v1.5.0/go.mod h1:bFUtVrKA4DC2yAKiSyO/QUcy7e+RRV2QTWOzhPopBRo= @@ -590,11 +513,8 @@ github.com/mitchellh/pointerstructure v1.2.1/go.mod h1:BRAsLI5zgXmw97Lf6s25bs8oh github.com/mmcloughlin/addchain v0.4.0 h1:SobOdjm2xLj1KkXN5/n0xTIWyZA2+s99UCY1iPfkHRY= github.com/mmcloughlin/addchain v0.4.0/go.mod h1:A86O+tHqZLMNO4w6ZZ4FlVQEadcoqkyU72HC5wJ4RlU= github.com/mmcloughlin/profile v0.1.1/go.mod h1:IhHD7q1ooxgwTgjxQYkACGA77oFTDdFVejUS1/tS/qU= -github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= -github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= -github.com/moul/http2curl v1.0.0/go.mod h1:8UbvGypXm98wA/IqH45anm5Y2Z6ep6O31QGOAZ3H0fQ= github.com/mr-tron/base58 v1.1.2/go.mod h1:BinMc/sQntlIE1frQmRFPUoPA1Zkr8VRgBdjWI2mNwc= github.com/mr-tron/base58 v1.2.0 h1:T/HDJBh4ZCPbU39/+c3rRvE0uKBQlU27+QI8LJ4t64o= github.com/mr-tron/base58 v1.2.0/go.mod h1:BinMc/sQntlIE1frQmRFPUoPA1Zkr8VRgBdjWI2mNwc= @@ -626,10 +546,6 @@ github.com/naoina/go-stringutil v0.1.0 h1:rCUeRUHjBjGTSHl0VC00jUPLz8/F9dDzYI70Hz github.com/naoina/go-stringutil v0.1.0/go.mod h1:XJ2SJL9jCtBh+P9q5btrd/Ylo8XwT/h1USek5+NqSA0= github.com/naoina/toml v0.1.2-0.20170918210437-9fafd6967416 h1:shk/vn9oCoOTmwcouEdwIeOtOGA/ELRUw/GwvxwfT+0= github.com/naoina/toml v0.1.2-0.20170918210437-9fafd6967416/go.mod h1:NBIhNtsFMo3G2szEBne+bO4gS192HuIYRqfvOWb4i1E= -github.com/nats-io/jwt v0.3.0/go.mod h1:fRYCDE99xlTsqUzISS1Bi75UBJ6ljOJQOAAu5VglpSg= -github.com/nats-io/nats.go v1.9.1/go.mod h1:ZjDU1L/7fJ09jvUSRVBR2e7+RnLiiIQyqyzEE/Zbp4w= -github.com/nats-io/nkeys v0.1.0/go.mod h1:xpnFELMwJABBLVhffcfd1MZx6VsNRFpEugbxziKVo7w= -github.com/nats-io/nuid v1.0.1/go.mod h1:19wcPz3Ph3q0Jbyiqsd0kePYG7A95tJPxeL+1OSON2c= github.com/neelance/astrewrite v0.0.0-20160511093645-99348263ae86/go.mod h1:kHJEU3ofeGjhHklVoIGuVj85JJwZ6kWPaJwCIxgnFmo= github.com/neelance/sourcemap v0.0.0-20151028013722-8c68805598ab/go.mod h1:Qr6/a/Q4r9LP1IltGz7tA7iOK1WonHEYhu1HRBA7ZiM= github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A= @@ -639,7 +555,6 @@ github.com/olekukonko/tablewriter v0.0.5 h1:P2Ga83D34wi1o9J6Wh1mRuqd4mF/x/lgBS7N github.com/olekukonko/tablewriter v0.0.5/go.mod h1:hPp6KlRPjbx+hW8ykQs1w3UBbZlj6HuIJcUGPhkA7kY= github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v1.7.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= -github.com/onsi/ginkgo v1.10.3/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v1.12.1/go.mod h1:zj2OWP4+oCPe1qIXoGWkgMRwljMUYCdkwsT2108oapk= github.com/onsi/ginkgo v1.14.0/go.mod h1:iSB4RoI2tjJc9BBv4NKIKWKya62Rps+oPG/Lv9klQyY= github.com/onsi/ginkgo v1.16.4/go.mod h1:dX+/inL/fNMqNlz0e9LfyB9TswhZpCVdJM/Z6Vvnwo0= @@ -665,7 +580,6 @@ github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYr github.com/openzipkin/zipkin-go v0.1.1/go.mod h1:NtoC/o8u3JlF1lSlyPNswIbeQH9bJTmOf0Erfk+hxe8= github.com/pbnjay/memory v0.0.0-20210728143218-7b4eea64cf58 h1:onHthvaw9LFnH4t2DcNVpwGmV9E1BkGknEliJkfwQj0= github.com/pbnjay/memory v0.0.0-20210728143218-7b4eea64cf58/go.mod h1:DXv8WO4yhMYhSNPKjeNKa5WY9YCIEBRbNzFFPJbWO6Y= -github.com/pelletier/go-toml v1.2.0/go.mod h1:5z9KED0ma1S8pY6P1sdut58dfprrGBbd/94hg7ilaic= github.com/peterh/liner v1.1.1-0.20190123174540-a2c9a5303de7 h1:oYW+YCJ1pachXTQmzR3rNLYGGz4g/UgFcjb28p/viDM= github.com/peterh/liner v1.1.1-0.20190123174540-a2c9a5303de7/go.mod h1:CRroGNssyjTd/qIG2FyxByd2S8JEAZXBl4qUrZf8GS0= github.com/pingcap/errors v0.11.4 h1:lFuQV/oaUMGcD2tqt+01ROSmJs75VG1ToEOkZIZ4nE4= @@ -684,7 +598,6 @@ github.com/prometheus/client_golang v0.8.0/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXP github.com/prometheus/client_golang v1.17.0 h1:rl2sfwZMtSthVU752MqfjQozy7blglC+1SOtjMAMh+Q= github.com/prometheus/client_golang v1.17.0/go.mod h1:VeL+gMmOAxkS2IqfCq0ZmHSL+LjWfWDUmp1mBz9JgUY= github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= -github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.4.1-0.20230718164431-9a2bf3000d16 h1:v7DLqVdK4VrYkVD5diGdl4sxJurKJEMnODWRJlxV9oM= github.com/prometheus/client_model v0.4.1-0.20230718164431-9a2bf3000d16/go.mod h1:oMQmHW1/JoDwqLtg57MGgP/Fb1CJEYF2imWWhWtMkYU= github.com/prometheus/common v0.0.0-20180801064454-c7de2306084e/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro= @@ -708,7 +621,6 @@ github.com/rivo/uniseg v0.4.3 h1:utMvzDsuh3suAEnhH0RdHmoPbU648o6CvXxTx4SBMOw= github.com/rivo/uniseg v0.4.3/go.mod h1:FN3SvrM+Zdj16jyLfmOkMNblXMcoc8DfTHruCPUcx88= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/rogpeppe/go-internal v1.6.1/go.mod h1:xXDCJY+GAPziupqXw64V24skbSoqbTEfhy4qGm1nDQc= -github.com/rogpeppe/go-internal v1.8.1/go.mod h1:JeRgkft04UBgHMgCIwADu4Pn6Mtm5d4nPKWu0nJ5d+o= github.com/rogpeppe/go-internal v1.9.0/go.mod h1:WtVeX8xhTBvf0smdhujwtBcq4Qrzq/fJaraNFVN+nFs= github.com/rogpeppe/go-internal v1.10.0 h1:TMyTOH3F/DB16zRVcYyreMH6GnZZrwQVAoYjRBZyWFQ= github.com/rogpeppe/go-internal v1.10.0/go.mod h1:UQnix2H7Ngw/k4C5ijL5+65zddjncjaFoBhdsK/akog= @@ -721,9 +633,7 @@ github.com/russross/blackfriday v1.5.2/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= github.com/russross/blackfriday/v2 v2.1.0 h1:JIOH55/0cWyOuilr9/qlrm0BSXldqnqwMsf35Ld67mk= github.com/russross/blackfriday/v2 v2.1.0/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= -github.com/ryanuber/columnize v2.1.0+incompatible/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts= github.com/satori/go.uuid v1.2.0/go.mod h1:dA0hQrYB0VpLJoorglMZABFdXlWrHn1NEOzdhQKdks0= -github.com/schollz/closestmatch v2.1.0+incompatible/go.mod h1:RtP1ddjLong6gTkbtmuhtR2uUrrJOpYzYRvbcPAid+g= github.com/sergi/go-diff v1.0.0/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo= github.com/shirou/gopsutil v3.21.11+incompatible h1:+1+c1VGhc88SSonWP6foOcLhvnKlUeu/erjjvaPEYiI= github.com/shirou/gopsutil v3.21.11+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= @@ -755,18 +665,10 @@ github.com/shurcooL/webdavfs v0.0.0-20170829043945-18c3829fa133/go.mod h1:hKmq5k github.com/sirupsen/logrus v1.4.1/go.mod h1:ni0Sbl8bgC9z8RoU9G6nDWqqs/fq4eDPysMBDgk/93Q= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= github.com/sirupsen/logrus v1.7.0/go.mod h1:yWOB1SBYBC5VeMP7gHvWumXLIWorT60ONWic61uBYv0= -github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= -github.com/smartystreets/goconvey v1.6.4/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= github.com/sourcegraph/annotate v0.0.0-20160123013949-f4cad6c6324d/go.mod h1:UdhH50NIW0fCiwBSr0co2m7BnFLdv4fQTgdqdJTHFeE= github.com/sourcegraph/syntaxhighlight v0.0.0-20170531221838-bd320f5d308e/go.mod h1:HuIsMU8RRBOtsCgI77wP899iHVBQpCmg4ErYMZB+2IA= github.com/spaolacci/murmur3 v1.1.0 h1:7c1g84S4BPRrfL5Xrdp6fOJ206sU9y293DDHaoy0bLI= github.com/spaolacci/murmur3 v1.1.0/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= -github.com/spf13/afero v1.1.2/go.mod h1:j4pytiNVoe2o6bmDsKpLACNPDBIoEAkihy7loJ1B0CQ= -github.com/spf13/cast v1.3.0/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkUJE= -github.com/spf13/cobra v0.0.5/go.mod h1:3K3wKZymM7VvHMDS9+Akkh4K60UwM26emMESw8tLCHU= -github.com/spf13/jwalterweatherman v1.0.0/go.mod h1:cQK4TGJAtQXfYWX+Ddv3mKDzgVb68N+wFjFa4jdeBTo= -github.com/spf13/pflag v1.0.3/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= -github.com/spf13/viper v1.3.2/go.mod h1:ZiWeW+zYFKm7srdB9IoDzzZXaJaI5eL9QjNiN/DMA2s= github.com/status-im/keycard-go v0.2.0 h1:QDLFswOQu1r5jsycloeQh3bVU8n/NatHHaZobtDnDzA= github.com/status-im/keycard-go v0.2.0/go.mod h1:wlp8ZLbsmrF6g6WjugPAx+IzoLrkdf9+mHxBEeo3Hbg= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= @@ -798,31 +700,16 @@ github.com/tklauser/numcpus v0.6.1 h1:ng9scYS7az0Bk4OZLvrNXNSAO2Pxr1XXRAPyjhIx+F github.com/tklauser/numcpus v0.6.1/go.mod h1:1XfjsgE2zo8GVw7POkMbHENHzVg3GzmoZ9fESEdAacY= github.com/tyler-smith/go-bip39 v1.1.0 h1:5eUemwrMargf3BSLRRCalXT93Ns6pQJIjYQN2nyfOP8= github.com/tyler-smith/go-bip39 v1.1.0/go.mod h1:gUYDtqQw1JS3ZJ8UWVcGTGqqr6YIN3CWg+kkNaLt55U= -github.com/ugorji/go v1.1.4/go.mod h1:uQMGLiO92mf5W77hV/PUCpI3pbzQx3CRekS0kk+RGrc= -github.com/ugorji/go v1.1.7/go.mod h1:kZn38zHttfInRq0xu/PH0az30d+z6vm202qpg1oXVMw= -github.com/ugorji/go/codec v0.0.0-20181204163529-d75b2dcb6bc8/go.mod h1:VFNgLljTbGfSG7qAOspJ7OScBnGdDN/yBr0sguwnwf0= -github.com/ugorji/go/codec v1.1.7/go.mod h1:Ax+UKWsSmolVDwsd+7N3ZtXu+yMGCf907BLYF3GoBXY= github.com/urfave/cli v1.22.2/go.mod h1:Gos4lmkARVdJ6EkW0WaNv/tZAAMe9V7XWyB60NtXRu0= github.com/urfave/cli/v2 v2.25.7 h1:VAzn5oq403l5pHjc4OhD54+XGO9cdKVL/7lDjF+iKUs= github.com/urfave/cli/v2 v2.25.7/go.mod h1:8qnjx1vcq5s2/wpsqoZFndg2CE5tNFyrTvS6SinrnYQ= -github.com/urfave/negroni v1.0.0/go.mod h1:Meg73S6kFm/4PpbYdq35yYWoCZ9mS/YSx+lKnmiohz4= github.com/valyala/bytebufferpool v1.0.0/go.mod h1:6bBcMArwyJ5K/AmCkWv1jt77kVWyCJ6HpOuEn7z0Csc= -github.com/valyala/fasthttp v1.6.0/go.mod h1:FstJa9V+Pj9vQ7OJie2qMHdwemEDaDiSdBnvPM1Su9w= github.com/valyala/fasttemplate v1.0.1/go.mod h1:UQGH1tvbgY+Nz5t2n7tXsz52dQxojPUpymEIMZ47gx8= github.com/valyala/fasttemplate v1.2.1/go.mod h1:KHLXt3tVN2HBp8eijSv/kGJopbvo7S+qRAEEKiv+SiQ= -github.com/valyala/tcplisten v0.0.0-20161114210144-ceec8f93295a/go.mod h1:v3UYOV9WzVtRmSR+PDvWpU/qWl4Wa5LApYYX4ZtKbio= github.com/viant/assertly v0.4.8/go.mod h1:aGifi++jvCrUaklKEKT0BU95igDNaqkvz+49uaYMPRU= github.com/viant/toolbox v0.24.0/go.mod h1:OxMCG57V0PXuIP2HNQrtJf2CjqdmbrOx5EkMILuUhzM= -github.com/xeipuuv/gojsonpointer v0.0.0-20180127040702-4e3ac2762d5f/go.mod h1:N2zxlSyiKSe5eX1tZViRH5QA0qijqEDrYZiPEAiq3wU= -github.com/xeipuuv/gojsonreference v0.0.0-20180127040603-bd5ef7bd5415/go.mod h1:GwrjFmJcFw6At/Gs6z4yjiIwzuJ1/+UwLxMQDVQXShQ= -github.com/xeipuuv/gojsonschema v1.2.0/go.mod h1:anYRn/JVcOK2ZgGU+IjEV4nwlhoK5sQluxsYJ78Id3Y= -github.com/xordataexchange/crypt v0.0.3-0.20170626215501-b2862e3d0a77/go.mod h1:aYKd//L2LvnjZzWKhF00oedf4jCCReLcmhLdhm1A27Q= github.com/xrash/smetrics v0.0.0-20201216005158-039620a65673 h1:bAn7/zixMGCfxrRTfdpNzjtPYqr8smhKouy9mxVdGPU= github.com/xrash/smetrics v0.0.0-20201216005158-039620a65673/go.mod h1:N3UwUGtsrSj3ccvlPHLoLsHnpR27oXr4ZE984MbSER8= -github.com/yalp/jsonpath v0.0.0-20180802001716-5cc68e5049a0/go.mod h1:/LWChgwKmvncFJFHJ7Gvn9wZArjbV5/FppcK2fKk/tI= -github.com/yudai/gojsondiff v1.0.0/go.mod h1:AY32+k2cwILAkW1fbgxQ5mUmMiZFgLIV+FBNExI05xg= -github.com/yudai/golcs v0.0.0-20170316035057-ecda9a501e82/go.mod h1:lgjkn3NuSvDfVJdfcVVdX+jpBxNmX4rDAzaS45IcYoM= -github.com/yudai/pp v2.0.1+incompatible/go.mod h1:PuxR/8QJ7cyCkFp/aUDS+JY727OFEZkTdatxwunjIkc= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= @@ -864,16 +751,13 @@ go4.org v0.0.0-20180809161055-417644f6feb5/go.mod h1:MkTOUMDaeVYJUOUsaDXIhWPZYa1 golang.org/x/build v0.0.0-20190111050920-041ab4dc3f9d/go.mod h1:OWs+y06UdEOHN4y+MfF/py+xQ/tYqIWW03b70/CG9Rw= golang.org/x/crypto v0.0.0-20170930174604-9419663f5a44/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20181030102418-4d3f4d9ffa16/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= -golang.org/x/crypto v0.0.0-20181203042331-505ab145d0a9/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190313024323-a1f597ede03a/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190411191339-88737f569e3a/go.mod h1:WFFai1msRO1wXaEeE5yQxYXgSfI8pQAWXbQop6sCtWE= golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20190611184440-5c40567a22f8/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= -golang.org/x/crypto v0.0.0-20190701094942-4def268fd1a4/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20190820162420-60c769a6c586/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= -golang.org/x/crypto v0.0.0-20191227163750-53104e6ec876/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20200602180216-279210d13fed/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20200820211705-5c72a883971a/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= @@ -891,10 +775,8 @@ golang.org/x/exp v0.0.0-20230905200255-921286631fa9/go.mod h1:S2oDrQGGwySpoQPVqR golang.org/x/lint v0.0.0-20180702182130-06c8688daad7/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= -golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/lint v0.0.0-20200302205851-738671d3881b/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= -golang.org/x/lint v0.0.0-20210508222113-6edffad5e616/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= @@ -909,18 +791,13 @@ golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73r golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181029044818-c44066c5c816/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181106065722-10aee1819953/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20181220203305-927f97764cc3/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190313220215-9f648a60d977/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= -golang.org/x/net v0.0.0-20190327091125-710a502c58a2/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= -golang.org/x/net v0.0.0-20190503192946-f4e77d36d62c/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20190813141303-74dc4d7220e7/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20190827160401-ba9fcec4b297/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20191209160850-c0dbc17a3553/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200520004742-59133d7f0dd7/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= golang.org/x/net v0.0.0-20200813134508-3edf25e44fcc/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= @@ -930,7 +807,6 @@ golang.org/x/net v0.0.0-20210119194325-5f4716e94777/go.mod h1:m0MpNAwzfU5UDzcl9v golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= golang.org/x/net v0.0.0-20210428140749-89ef3d95e781/go.mod h1:OJAsFXCWl8Ukc7SiCT/9KSuxbyM7479/AVlXFRxuMCk= -golang.org/x/net v0.0.0-20211008194852-3b03d305991f/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20220225172249-27dd8689420f/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= golang.org/x/net v0.0.0-20220607020251-c690dde0001d/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= @@ -958,14 +834,12 @@ golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5h golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181029174526-d69651ed3497/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20181205085412-a5c9d58dba9a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190222072716-a9d3bda3a223/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190316082340-a2f829d7f35f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190403152447-81d4e9dc473e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190626221950-04f50cda93cb/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190813064441-fde4db37ae7a/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190904154756-749cb33beabd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190916202348-b4ddaad3f8a3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -987,16 +861,13 @@ golang.org/x/sys v0.0.0-20210112080510-489259a85091/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210303074136-134d130e1a04/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210403161142-5e06dd20ab57/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210630005230-0f9fa26af87c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20210927094055-39ccf1dd6fa6/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211007075335-d3039528d8ac/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211020174200-9d6173849985/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211216021012-1d35b9e2eb4e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20220209214540-3681064d5158/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220310020820-b874c991c1a5/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220412211240-33da011f77ad/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= @@ -1035,14 +906,10 @@ golang.org/x/tools v0.0.0-20180828015842-6cd1fcedba52/go.mod h1:n7NCudcB/nEzxVGm golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20181030000716-a0a13e073c7b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20181030221726-6c7e314b6563/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20181221001348-537d06c36207/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190327201419-c70d86f8b7cf/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190328211700-ab21143f2384/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190425163242-31fd60d6bfdc/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= -golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= golang.org/x/tools v0.0.0-20190621195816-6e04913cbbac/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= golang.org/x/tools v0.0.0-20190823170909-c4a336ef6a2f/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191029041327-9cc4af7d6b2c/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= @@ -1055,7 +922,6 @@ golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roY golang.org/x/tools v0.0.0-20201224043029-2b0845dc783e/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.1.1/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= -golang.org/x/tools v0.1.3/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= golang.org/x/tools v0.13.0 h1:Iey4qkscZuv0VvIt8E0neZjtPVQFSc870HQ448QgEmQ= @@ -1074,33 +940,21 @@ google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9Ywl google.golang.org/appengine v1.2.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.3.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= -google.golang.org/genproto v0.0.0-20180518175338-11a468237815/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20180831171423-11092d34479b/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20181029155118-b69ba1387ce2/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20181202183823-bd91e49a0898/go.mod h1:7Ep/1NZk928CDR8SjdVbjWNpdIf6nzjE3BTgJDr2Atg= google.golang.org/genproto v0.0.0-20190306203927-b5d61aea6440/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= -google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= -google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo= -google.golang.org/genproto v0.0.0-20210624195500-8bfb893ecb84/go.mod h1:SzzZ/N+nwJDaO1kznhnlzqS8ocJICar6hYhVyhi++24= -google.golang.org/grpc v1.12.0/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= google.golang.org/grpc v1.14.0/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= google.golang.org/grpc v1.16.0/go.mod h1:0JHn/cJsOMiMfNA9+DeHDlAU7KAAB5GDlYFpa9MZMio= google.golang.org/grpc v1.17.0/go.mod h1:6QZJwpn2B+Zp71q/5VxRsJ6NXXVCE5NRUHRo+f3cWCs= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= -google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= -google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= -google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= -google.golang.org/grpc v1.38.0/go.mod h1:NREThFqKR1f3iQ6oBuvc5LadQuXVGo9rkm5ZGrQdJfM= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miEFZTKqfCUM6K7xSMQL9OKL/b6hQv+e19PK+JZNE= google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo= -google.golang.org/protobuf v1.22.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= -google.golang.org/protobuf v1.23.1-0.20200526195155-81db48ad09cc/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= -google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= google.golang.org/protobuf v1.31.0 h1:g0LDEJHgrBl9N9r17Ru3sqWhkIx2NB67okBHPwC7hs8= @@ -1112,12 +966,8 @@ gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntN gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys= -gopkg.in/go-playground/assert.v1 v1.2.1/go.mod h1:9RXL0bg/zibRAgZUYszZSwO/z8Y/a8bDuhia5mkpMnE= -gopkg.in/go-playground/validator.v8 v8.18.2/go.mod h1:RX2a/7Ha8BgOhfk7j780h4/u/RRjR0eouCJSH80/M2Y= gopkg.in/inconshreveable/log15.v2 v2.0.0-20180818164646-67afb5ed74ec/go.mod h1:aPpfJ7XW+gOuirDoZ8gHhLh3kZ1B08FtV2bbmy7Jv3s= gopkg.in/inf.v0 v0.9.1/go.mod h1:cWUDdTG/fYaXco+Dcufb5Vnc6Gp2YChqWtbxRZE0mXw= -gopkg.in/ini.v1 v1.51.1/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= -gopkg.in/mgo.v2 v2.0.0-20180705113604-9856a29383ce/go.mod h1:yeKp02qBN3iKW1OzL3MGk2IdtZzaj7SFntXj72NppTA= gopkg.in/natefinch/lumberjack.v2 v2.0.0 h1:1Lc07Kr7qY4U2YPouBjpCLxpiyxIVoxqXgkXLknAOE8= gopkg.in/natefinch/lumberjack.v2 v2.0.0/go.mod h1:l0ndWWf7gzL7RNwBG7wST/UCcT4T24xpD6X8LsfU/+k= gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 h1:uRGJdciOHaEIrze2W8Q3AKkepLTh2hOroT7a+7czfdQ= @@ -1129,7 +979,6 @@ gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY= gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= -gopkg.in/yaml.v3 v3.0.0-20191120175047-4206685974f2/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= @@ -1142,7 +991,6 @@ grpc.go4.org v0.0.0-20170609214715-11d0a25b4919/go.mod h1:77eQGdRu53HpSqPFJFmuJd honnef.co/go/tools v0.0.0-20180728063816-88497007e858/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= lukechampine.com/blake3 v1.2.1 h1:YuqqRuaqsGV71BV/nm9xlI0MKUv4QC54jQnBChWbGnI= lukechampine.com/blake3 v1.2.1/go.mod h1:0OFRp7fBtAylGVCO40o87sbupkyIGgbpv1+M1k1LM6k= From 40d15216cf040fe211a24aafa6a097ed89061bea Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Mon, 23 Oct 2023 13:31:35 +1000 Subject: [PATCH 012/279] Update op-geth commit. --- go.mod | 2 +- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index a43c37b231b6..8a956d950283 100644 --- a/go.mod +++ b/go.mod @@ -210,7 +210,7 @@ require ( rsc.io/tmplfunc v0.0.3 // indirect ) -replace github.com/ethereum/go-ethereum v1.13.4 => github.com/ethereum-optimism/op-geth v1.101301.2-0.20231018010910-b142ad43dda3 +replace github.com/ethereum/go-ethereum v1.13.4 => github.com/ethereum-optimism/op-geth v1.101303.0-rc.2.0.20231023000909-ceb5f3201eb7 //replace github.com/ethereum-optimism/superchain-registry/superchain => ../superchain-registry/superchain //replace github.com/ethereum/go-ethereum v1.13.4 => ../go-ethereum diff --git a/go.sum b/go.sum index c8241cf12039..ec4a6c47f6dd 100644 --- a/go.sum +++ b/go.sum @@ -151,8 +151,8 @@ github.com/elastic/gosigar v0.14.2 h1:Dg80n8cr90OZ7x+bAax/QjoW/XqTI11RmA79ZwIm9/ github.com/elastic/gosigar v0.14.2/go.mod h1:iXRIGg2tLnu7LBdpqzyQfGDEidKCfWcCMS0WKyPWoMs= github.com/ethereum-optimism/go-ethereum-hdwallet v0.1.3 h1:RWHKLhCrQThMfch+QJ1Z8veEq5ZO3DfIhZ7xgRP9WTc= github.com/ethereum-optimism/go-ethereum-hdwallet v0.1.3/go.mod h1:QziizLAiF0KqyLdNJYD7O5cpDlaFMNZzlxYNcWsJUxs= -github.com/ethereum-optimism/op-geth v1.101301.2-0.20231018010910-b142ad43dda3 h1:WdAevEuDn3I1YPChnIlgqCXvDFx2/oRvcvDG7aU86sk= -github.com/ethereum-optimism/op-geth v1.101301.2-0.20231018010910-b142ad43dda3/go.mod h1:V2K+IIUITMRKVGLiPCr29RvbDaA9P7YE5JL7UrFsbD8= +github.com/ethereum-optimism/op-geth v1.101303.0-rc.2.0.20231023000909-ceb5f3201eb7 h1:4T6bQlhdoPpQDSGUO4wstmyipeX8ATMKGVISSwe4fbY= +github.com/ethereum-optimism/op-geth v1.101303.0-rc.2.0.20231023000909-ceb5f3201eb7/go.mod h1:hl28ffXoV4maInP7dvhvNgDO79Q5M3MEYrPZZO6u3W8= github.com/ethereum-optimism/superchain-registry/superchain v0.0.0-20231018202221-fdba3d104171 h1:MjCUj16JSLZRDnQQ6OOUy6Chfb4dKo7ahFceNi0RKZ8= github.com/ethereum-optimism/superchain-registry/superchain v0.0.0-20231018202221-fdba3d104171/go.mod h1:/70H/KqrtKcvWvNGVj6S3rAcLC+kUPr3t2aDmYIS+Xk= github.com/ethereum/c-kzg-4844 v0.3.1 h1:sR65+68+WdnMKxseNWxSJuAv2tsUrihTpVBTfM/U5Zg= From 0eee4b6c6de516a33fa41461100f5c304aaa5eb9 Mon Sep 17 00:00:00 2001 From: Michael de Hoog Date: Mon, 23 Oct 2023 06:25:35 -1000 Subject: [PATCH 013/279] Add the network timeout back in --- op-service/txmgr/txmgr.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/op-service/txmgr/txmgr.go b/op-service/txmgr/txmgr.go index 9de7b0bb194e..54dd5feddaac 100644 --- a/op-service/txmgr/txmgr.go +++ b/op-service/txmgr/txmgr.go @@ -265,6 +265,8 @@ func (m *SimpleTxManager) signWithNextNonce(ctx context.Context, rawTx *types.Dy } rawTx.Nonce = *m.nonce + ctx, cancel := context.WithTimeout(ctx, m.cfg.NetworkTimeout) + defer cancel() tx, err := m.cfg.Signer(ctx, m.cfg.From, types.NewTx(rawTx)) if err != nil { // decrement the nonce, so we can retry signing with the same nonce next time From 77995af78da3fadbae540df7ea93d3e1dfd6c646 Mon Sep 17 00:00:00 2001 From: "Eugene Y. Q. Shen" Date: Mon, 23 Oct 2023 13:40:27 -0400 Subject: [PATCH 014/279] specs: rename contracts to contracts-bedrock --- CONTRIBUTING.md | 4 ++-- specs/deposits.md | 6 +++--- specs/meta/devnet.md | 2 +- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index bcf0e8703602..96cd56173c13 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -181,9 +181,9 @@ You must have Python 3.x installed to run `slither`. To run `slither` locally, do: ```bash -cd packages/contracts +cd packages/contracts-bedrock pip3 install slither-analyzer -pnpm test:slither +pnpm slither ``` ## Labels diff --git a/specs/deposits.md b/specs/deposits.md index 442a4592db28..eb2602ee3a8e 100644 --- a/specs/deposits.md +++ b/specs/deposits.md @@ -308,9 +308,9 @@ A reference implementation of the L1 Attributes predeploy contract can be found [L1Block.sol]: ../packages/contracts-bedrock/src/L2/L1Block.sol -After running `pnpm build` in the `packages/contracts` directory, the bytecode to add to the genesis -file will be located in the `deployedBytecode` field of the build artifacts file at -`/packages/contracts/artifacts/contracts/L2/L1Block.sol/L1Block.json`. +After running `pnpm build` in the `packages/contracts-bedrock` directory, the bytecode to add to +the genesis file will be located in the `deployedBytecode` field of the build artifacts file at +`/packages/contracts-bedrock/forge-artifacts/L1Block.sol/L1Block.json`. ## User-Deposited Transactions diff --git a/specs/meta/devnet.md b/specs/meta/devnet.md index ad9de46e9ce9..b39a55a04d4f 100644 --- a/specs/meta/devnet.md +++ b/specs/meta/devnet.md @@ -12,7 +12,7 @@ You can spin up a local devnet via `docker compose`. For convenience, we have defined `make` targets to start and stop the devnet with a single command. To run the devnet, you will need `docker` installed. -Then, as a precondition, make sure that you have compiled the contracts by `cd`ing into `packages/contracts` +Then, as a precondition, make sure that you have compiled the contracts by `cd`ing into `packages/contracts-bedrock` and running `pnpm i` followed by `pnpm build`. You'll only need to do this if you change the contracts in the future. Then, run the following: From af539535af4f2020de281e4ceb4d97ab08fad34a Mon Sep 17 00:00:00 2001 From: protolambda Date: Mon, 23 Oct 2023 19:51:10 +0200 Subject: [PATCH 015/279] op-service: update test-gen script Co-authored-by: Adrian Sutton --- op-service/sources/testdata/gen.sh | 3 +++ 1 file changed, 3 insertions(+) diff --git a/op-service/sources/testdata/gen.sh b/op-service/sources/testdata/gen.sh index d3a42f2440d6..50612d71b81e 100644 --- a/op-service/sources/testdata/gen.sh +++ b/op-service/sources/testdata/gen.sh @@ -1,5 +1,8 @@ #!/bin/bash +set -euo pipefail +SOURCE_DIR=$(cd $(dirname "${BASH_SOURCE[0]}") && pwd) +cd "$SOURCE_DIR" export ETH_RPC_URL=https://ethereum-goerli-rpc.allthatnode.com jq_mutate() { From be50ce7e1a1380e41e1085d1244f9274d71742ac Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 23 Oct 2023 20:11:03 +0000 Subject: [PATCH 016/279] build(deps-dev): bump @wagmi/core from 1.4.4 to 1.4.5 Bumps [@wagmi/core](https://github.com/wagmi-dev/wagmi/tree/HEAD/packages/core) from 1.4.4 to 1.4.5. - [Release notes](https://github.com/wagmi-dev/wagmi/releases) - [Changelog](https://github.com/wagmi-dev/wagmi/blob/main/packages/core/CHANGELOG.md) - [Commits](https://github.com/wagmi-dev/wagmi/commits/@wagmi/core@1.4.5/packages/core) --- updated-dependencies: - dependency-name: "@wagmi/core" dependency-type: direct:development update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] --- packages/contracts-ts/package.json | 2 +- pnpm-lock.yaml | 74 +++++++++++++++--------------- 2 files changed, 38 insertions(+), 38 deletions(-) diff --git a/packages/contracts-ts/package.json b/packages/contracts-ts/package.json index cfccdfa54c55..219510f3f850 100644 --- a/packages/contracts-ts/package.json +++ b/packages/contracts-ts/package.json @@ -53,7 +53,7 @@ "@types/glob": "^8.1.0", "@vitest/coverage-istanbul": "^0.34.6", "@wagmi/cli": "^1.5.2", - "@wagmi/core": "^1.4.4", + "@wagmi/core": "^1.4.5", "abitype": "^0.10.1", "glob": "^10.3.10", "isomorphic-fetch": "^3.0.0", diff --git a/pnpm-lock.yaml b/pnpm-lock.yaml index 9f2b147ddd5e..f3b43faeb40f 100644 --- a/pnpm-lock.yaml +++ b/pnpm-lock.yaml @@ -321,10 +321,10 @@ importers: version: 0.34.6(vitest@0.34.2) '@wagmi/cli': specifier: ^1.5.2 - version: 1.5.2(@wagmi/core@1.4.4)(typescript@5.2.2)(wagmi@1.0.1) + version: 1.5.2(@wagmi/core@1.4.5)(typescript@5.2.2)(wagmi@1.0.1) '@wagmi/core': - specifier: ^1.4.4 - version: 1.4.4(react@18.2.0)(typescript@5.2.2)(viem@1.16.6) + specifier: ^1.4.5 + version: 1.4.5(react@18.2.0)(typescript@5.2.2)(viem@1.16.6) abitype: specifier: ^0.10.1 version: 0.10.1(typescript@5.2.2) @@ -4621,7 +4621,7 @@ packages: dependencies: typescript: 5.2.2 - /@wagmi/cli@1.5.2(@wagmi/core@1.4.4)(typescript@5.2.2)(wagmi@1.0.1): + /@wagmi/cli@1.5.2(@wagmi/core@1.4.5)(typescript@5.2.2)(wagmi@1.0.1): resolution: {integrity: sha512-UfLMYhW6mQBCjR8A5s01Chf9GpHzdpcuuBuzJ36QGXcMSJAxylz5ImVZWfCRV0ct1UruydjKVSW1QSI6azNxRQ==} engines: {node: '>=14'} hasBin: true @@ -4637,7 +4637,7 @@ packages: wagmi: optional: true dependencies: - '@wagmi/core': 1.4.4(react@18.2.0)(typescript@5.2.2)(viem@1.16.6) + '@wagmi/core': 1.4.5(react@18.2.0)(typescript@5.2.2)(viem@1.16.6) abitype: 0.8.7(typescript@5.2.2)(zod@3.22.3) abort-controller: 3.0.0 bundle-require: 3.1.2(esbuild@0.16.17) @@ -4702,8 +4702,8 @@ packages: - utf-8-validate - zod - /@wagmi/connectors@3.1.2(react@18.2.0)(typescript@5.2.2)(viem@1.16.6): - resolution: {integrity: sha512-IlLKErqCzQRBUcCvXGPowcczbWcvJtEG006gPsAoePNJEXCHEWoKASghgu+L/bqD7006Z6mW6zlTNjcSQJvFAg==} + /@wagmi/connectors@3.1.3(react@18.2.0)(typescript@5.2.2)(viem@1.16.6): + resolution: {integrity: sha512-UgwsQKQDFObJVJMf9pDfFoXTv710o4zrTHyhIWKBTMMkLpCMsMxN5+ZaDhBYt/BgoRinfRYQo8uwuwLhxE6Log==} peerDependencies: typescript: '>=5.0.4' viem: '>=0.3.35' @@ -4715,10 +4715,10 @@ packages: '@ledgerhq/connect-kit-loader': 1.1.0 '@safe-global/safe-apps-provider': 0.17.1(typescript@5.2.2) '@safe-global/safe-apps-sdk': 8.0.0(typescript@5.2.2) - '@walletconnect/ethereum-provider': 2.10.1(@walletconnect/modal@2.6.2) + '@walletconnect/ethereum-provider': 2.10.2(@walletconnect/modal@2.6.2) '@walletconnect/legacy-provider': 2.0.0 '@walletconnect/modal': 2.6.2(react@18.2.0) - '@walletconnect/utils': 2.10.1 + '@walletconnect/utils': 2.10.2 abitype: 0.8.7(typescript@5.2.2)(zod@3.22.3) eventemitter3: 4.0.7 typescript: 5.2.2 @@ -4763,8 +4763,8 @@ packages: - utf-8-validate - zod - /@wagmi/core@1.4.4(react@18.2.0)(typescript@5.2.2)(viem@1.16.6): - resolution: {integrity: sha512-38Pa5iH9/2zHX837vYm6M+c//BBikt6vKJqUiEgkIzSqFvlfzbxaNmwybpIxl98N5r+X22C3yHv5bboO9eP0vQ==} + /@wagmi/core@1.4.5(react@18.2.0)(typescript@5.2.2)(viem@1.16.6): + resolution: {integrity: sha512-N9luRb1Uk4tBN9kaYcQSWKE9AsRt/rvZaFt5IZech4JPzNN2sQlfhKd9GEjOXYRDqEPHdDvos7qyBKiDNTz4GA==} peerDependencies: typescript: '>=5.0.4' viem: '>=0.3.35' @@ -4772,7 +4772,7 @@ packages: typescript: optional: true dependencies: - '@wagmi/connectors': 3.1.2(react@18.2.0)(typescript@5.2.2)(viem@1.16.6) + '@wagmi/connectors': 3.1.3(react@18.2.0)(typescript@5.2.2)(viem@1.16.6) abitype: 0.8.7(typescript@5.2.2)(zod@3.22.3) eventemitter3: 4.0.7 typescript: 5.2.2 @@ -4791,8 +4791,8 @@ packages: - zod dev: true - /@walletconnect/core@2.10.1: - resolution: {integrity: sha512-WAoXfmj+Zy5q48TnrKUjmHXJCBahzKwbul+noepRZf7JDtUAZ9IOWpUjg+UPRbfK5EiWZ0TF42S6SXidf7EHoQ==} + /@walletconnect/core@2.10.2: + resolution: {integrity: sha512-JQz/xp3SLEpTeRQctdck2ugSBVEpMxoSE+lFi2voJkZop1hv6P+uqr6E4PzjFluAjeAnKlT1xvra0aFWjPWVcw==} dependencies: '@walletconnect/heartbeat': 1.2.1 '@walletconnect/jsonrpc-provider': 1.0.13 @@ -4805,8 +4805,8 @@ packages: '@walletconnect/relay-auth': 1.0.4 '@walletconnect/safe-json': 1.0.2 '@walletconnect/time': 1.0.2 - '@walletconnect/types': 2.10.1 - '@walletconnect/utils': 2.10.1 + '@walletconnect/types': 2.10.2 + '@walletconnect/utils': 2.10.2 events: 3.3.0 lodash.isequal: 4.5.0 uint8arrays: 3.1.1 @@ -4863,8 +4863,8 @@ packages: dependencies: tslib: 1.14.1 - /@walletconnect/ethereum-provider@2.10.1(@walletconnect/modal@2.6.2): - resolution: {integrity: sha512-Yhoz8EXkKzxOlBT6G+elphqCx/gkH6RxD9/ZAiy9lLc8Ng5p1gvKCVVP5zsGNE9FbkKmHd+J9JJRzn2Bw2yqtQ==} + /@walletconnect/ethereum-provider@2.10.2(@walletconnect/modal@2.6.2): + resolution: {integrity: sha512-QMYFZ6+rVq2CJLdIPdKK0j1Qm66UA27oQU5V2SrL8EVwl7wFfm0Bq7fnL+qAWeDpn612dNeNErpk/ROa1zWlWg==} peerDependencies: '@walletconnect/modal': '>=2' peerDependenciesMeta: @@ -4876,10 +4876,10 @@ packages: '@walletconnect/jsonrpc-types': 1.0.3 '@walletconnect/jsonrpc-utils': 1.0.8 '@walletconnect/modal': 2.6.2(react@18.2.0) - '@walletconnect/sign-client': 2.10.1 - '@walletconnect/types': 2.10.1 - '@walletconnect/universal-provider': 2.10.1 - '@walletconnect/utils': 2.10.1 + '@walletconnect/sign-client': 2.10.2 + '@walletconnect/types': 2.10.2 + '@walletconnect/universal-provider': 2.10.2 + '@walletconnect/utils': 2.10.2 events: 3.3.0 transitivePeerDependencies: - '@react-native-async-storage/async-storage' @@ -5102,17 +5102,17 @@ packages: dependencies: tslib: 1.14.1 - /@walletconnect/sign-client@2.10.1: - resolution: {integrity: sha512-iG3eJGi1yXeG3xGeVSSMf8wDFyx239B0prLQfy1uYDtYFb2ynnH/09oqAZyKn96W5nfQzUgM2Mz157PVdloH3Q==} + /@walletconnect/sign-client@2.10.2: + resolution: {integrity: sha512-vviSLV3f92I0bReX+OLr1HmbH0uIzYEQQFd1MzIfDk9PkfFT/LLAHhUnDaIAMkIdippqDcJia+5QEtT4JihL3Q==} dependencies: - '@walletconnect/core': 2.10.1 + '@walletconnect/core': 2.10.2 '@walletconnect/events': 1.0.1 '@walletconnect/heartbeat': 1.2.1 '@walletconnect/jsonrpc-utils': 1.0.8 '@walletconnect/logger': 2.0.1 '@walletconnect/time': 1.0.2 - '@walletconnect/types': 2.10.1 - '@walletconnect/utils': 2.10.1 + '@walletconnect/types': 2.10.2 + '@walletconnect/utils': 2.10.2 events: 3.3.0 transitivePeerDependencies: - '@react-native-async-storage/async-storage' @@ -5144,8 +5144,8 @@ packages: dependencies: tslib: 1.14.1 - /@walletconnect/types@2.10.1: - resolution: {integrity: sha512-7pccAhajQdiH2kYywjE1XI64IqRI+4ioyGy0wvz8d0UFQ/DSG3MLKR8jHf5aTOafQQ/HRLz6xvlzN4a7gIVkUQ==} + /@walletconnect/types@2.10.2: + resolution: {integrity: sha512-luNV+07Wdla4STi9AejseCQY31tzWKQ5a7C3zZZaRK/di+rFaAAb7YW04OP4klE7tw/mJRGPTlekZElmHxO8kQ==} dependencies: '@walletconnect/events': 1.0.1 '@walletconnect/heartbeat': 1.2.1 @@ -5171,17 +5171,17 @@ packages: - '@react-native-async-storage/async-storage' - lokijs - /@walletconnect/universal-provider@2.10.1: - resolution: {integrity: sha512-81QxTH/X4dRoYCz0U9iOrBYOcj7N897ONcB57wsGhEkV7Rc9htmWJq2CzeOuxvVZ+pNZkE+/aw9LrhizO1Ltxg==} + /@walletconnect/universal-provider@2.10.2: + resolution: {integrity: sha512-wFgI0LbQ3D56sgaUMsgOHCM5m8WLxiC71BGuCKQfApgsbNMVKugYVy2zWHyUyi8sqTQHI+uSaVpDev4UHq9LEw==} dependencies: '@walletconnect/jsonrpc-http-connection': 1.0.7 '@walletconnect/jsonrpc-provider': 1.0.13 '@walletconnect/jsonrpc-types': 1.0.3 '@walletconnect/jsonrpc-utils': 1.0.8 '@walletconnect/logger': 2.0.1 - '@walletconnect/sign-client': 2.10.1 - '@walletconnect/types': 2.10.1 - '@walletconnect/utils': 2.10.1 + '@walletconnect/sign-client': 2.10.2 + '@walletconnect/types': 2.10.2 + '@walletconnect/utils': 2.10.2 events: 3.3.0 transitivePeerDependencies: - '@react-native-async-storage/async-storage' @@ -5212,8 +5212,8 @@ packages: - lokijs - utf-8-validate - /@walletconnect/utils@2.10.1: - resolution: {integrity: sha512-DM0dKgm9O58l7VqJEyV2OVv16XRePhDAReI23let6WdW1dSpw/Y/A89Lp99ZJOjLm2FxyblMRF3YRaZtHwBffw==} + /@walletconnect/utils@2.10.2: + resolution: {integrity: sha512-syxXRpc2yhSknMu3IfiBGobxOY7fLfLTJuw+ppKaeO6WUdZpIit3wfuGOcc0Ms3ZPFCrGfyGOoZsCvgdXtptRg==} dependencies: '@stablelib/chacha20poly1305': 1.0.1 '@stablelib/hkdf': 1.0.1 @@ -5223,7 +5223,7 @@ packages: '@walletconnect/relay-api': 1.0.9 '@walletconnect/safe-json': 1.0.2 '@walletconnect/time': 1.0.2 - '@walletconnect/types': 2.10.1 + '@walletconnect/types': 2.10.2 '@walletconnect/window-getters': 1.0.1 '@walletconnect/window-metadata': 1.0.1 detect-browser: 5.3.0 From 034c78eeaba5af2482d3a2d2c44b82b63c6a43b8 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 23 Oct 2023 20:12:24 +0000 Subject: [PATCH 017/279] build(deps-dev): bump eslint-plugin-import from 2.28.1 to 2.29.0 Bumps [eslint-plugin-import](https://github.com/import-js/eslint-plugin-import) from 2.28.1 to 2.29.0. - [Release notes](https://github.com/import-js/eslint-plugin-import/releases) - [Changelog](https://github.com/import-js/eslint-plugin-import/blob/main/CHANGELOG.md) - [Commits](https://github.com/import-js/eslint-plugin-import/compare/v2.28.1...v2.29.0) --- updated-dependencies: - dependency-name: eslint-plugin-import dependency-type: direct:development update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] --- package.json | 2 +- pnpm-lock.yaml | 109 +++++++++++++++++++++++++++++++++++++++---------- 2 files changed, 88 insertions(+), 23 deletions(-) diff --git a/package.json b/package.json index 3ab7f0c38ecf..8cc8b7668dd0 100644 --- a/package.json +++ b/package.json @@ -52,7 +52,7 @@ "eslint": "^8.51.0", "eslint-config-prettier": "^8.3.0", "eslint-config-standard": "^16.0.3", - "eslint-plugin-import": "^2.26.0", + "eslint-plugin-import": "^2.29.0", "eslint-plugin-jsdoc": "^35.1.2", "eslint-plugin-node": "^11.1.0", "eslint-plugin-prefer-arrow": "^1.2.3", diff --git a/pnpm-lock.yaml b/pnpm-lock.yaml index 9f2b147ddd5e..aaf0f916043a 100644 --- a/pnpm-lock.yaml +++ b/pnpm-lock.yaml @@ -56,10 +56,10 @@ importers: version: 8.3.0(eslint@8.51.0) eslint-config-standard: specifier: ^16.0.3 - version: 16.0.3(eslint-plugin-import@2.28.1)(eslint-plugin-node@11.1.0)(eslint-plugin-promise@5.2.0)(eslint@8.51.0) + version: 16.0.3(eslint-plugin-import@2.29.0)(eslint-plugin-node@11.1.0)(eslint-plugin-promise@5.2.0)(eslint@8.51.0) eslint-plugin-import: - specifier: ^2.26.0 - version: 2.28.1(@typescript-eslint/parser@6.8.0)(eslint@8.51.0) + specifier: ^2.29.0 + version: 2.29.0(@typescript-eslint/parser@6.8.0)(eslint@8.51.0) eslint-plugin-jsdoc: specifier: ^35.1.2 version: 35.5.1(eslint@8.51.0) @@ -5680,6 +5680,17 @@ packages: is-string: 1.0.7 dev: true + /array-includes@3.1.7: + resolution: {integrity: sha512-dlcsNBIiWhPkHdOEEKnehA+RNUWDc4UqFtnIXU4uuYDPtA4LDkr7qip2p0VvFAEXNDr0yWZ9PJyIRiGjRLQzwQ==} + engines: {node: '>= 0.4'} + dependencies: + call-bind: 1.0.2 + define-properties: 1.2.0 + es-abstract: 1.22.1 + get-intrinsic: 1.2.1 + is-string: 1.0.7 + dev: true + /array-union@2.1.0: resolution: {integrity: sha512-HGyxoOTYUyCM6stUe6EJgnd4EoewAI7zMdfqO+kGjnlZmBDz/cR5pf8r/cR4Wq60sL/p0IkcjUEEPwS3GFrIyw==} engines: {node: '>=8'} @@ -5689,8 +5700,8 @@ packages: engines: {node: '>=12'} dev: true - /array.prototype.findlastindex@1.2.2: - resolution: {integrity: sha512-tb5thFFlUcp7NdNF6/MpDk/1r/4awWG1FIz3YqDf+/zJSTezBb+/5WViH41obXULHVpDzoiCLpJ/ZO9YbJMsdw==} + /array.prototype.findlastindex@1.2.3: + resolution: {integrity: sha512-LzLoiOMAxvy+Gd3BAq3B7VeIgPdo+Q8hthvKtXybMvRV0jrXfJM/t8mw7nNlpEcVlVUnCnM2KSX4XU5HmpodOA==} engines: {node: '>= 0.4'} dependencies: call-bind: 1.0.2 @@ -5708,6 +5719,17 @@ packages: define-properties: 1.2.0 es-abstract: 1.22.1 es-shim-unscopables: 1.0.0 + dev: false + + /array.prototype.flat@1.3.2: + resolution: {integrity: sha512-djYB+Zx2vLewY8RWlNCUdHjDXs2XOgm602S9E7P/UpHgfeHL00cRiIF+IN/G/aUJ7kGPb6yO/ErDI5V2s8iycA==} + engines: {node: '>= 0.4'} + dependencies: + call-bind: 1.0.2 + define-properties: 1.2.0 + es-abstract: 1.22.1 + es-shim-unscopables: 1.0.0 + dev: true /array.prototype.flatmap@1.3.1: resolution: {integrity: sha512-8UGn9O1FDVvMNB0UlLv4voxRMze7+FpHyF5mSMRjWHUMlpoDViniy05870VlxhfgTnLbpuwTzvD76MTtWxB/mQ==} @@ -5719,6 +5741,16 @@ packages: es-shim-unscopables: 1.0.0 dev: true + /array.prototype.flatmap@1.3.2: + resolution: {integrity: sha512-Ewyx0c9PmpcsByhSW4r+9zDU7sGjFc86qf/kKtuSCRdhfbk0SNLLkaT5qvcHnRGgc5NP/ly/y+qkXkqONX54CQ==} + engines: {node: '>= 0.4'} + dependencies: + call-bind: 1.0.2 + define-properties: 1.2.0 + es-abstract: 1.22.1 + es-shim-unscopables: 1.0.0 + dev: true + /array.prototype.tosorted@1.1.1: resolution: {integrity: sha512-pZYPXPRl2PqWcsUs6LOMn+1f1532nEoPTYowBtqLwAW+W8vSVhkIGnmOX1t/UQjD6YGI0vcD2B1U7ZFGQH9jnQ==} dependencies: @@ -7509,7 +7541,7 @@ packages: eslint: 8.51.0 dev: true - /eslint-config-standard@16.0.3(eslint-plugin-import@2.28.1)(eslint-plugin-node@11.1.0)(eslint-plugin-promise@5.2.0)(eslint@8.51.0): + /eslint-config-standard@16.0.3(eslint-plugin-import@2.29.0)(eslint-plugin-node@11.1.0)(eslint-plugin-promise@5.2.0)(eslint@8.51.0): resolution: {integrity: sha512-x4fmJL5hGqNJKGHSjnLdgA6U6h1YW/G2dW9fA+cyVur4SK6lyue8+UgNKWlZtUDTXvgKDD/Oa3GQjmB5kjtVvg==} peerDependencies: eslint: ^7.12.1 @@ -7518,7 +7550,7 @@ packages: eslint-plugin-promise: ^4.2.1 || ^5.0.0 dependencies: eslint: 8.51.0 - eslint-plugin-import: 2.28.1(@typescript-eslint/parser@6.8.0)(eslint@8.51.0) + eslint-plugin-import: 2.29.0(@typescript-eslint/parser@6.8.0)(eslint@8.51.0) eslint-plugin-node: 11.1.0(eslint@8.51.0) eslint-plugin-promise: 5.2.0(eslint@8.51.0) dev: true @@ -7527,7 +7559,7 @@ packages: resolution: {integrity: sha512-WFj2isz22JahUv+B788TlO3N6zL3nNJGU8CcZbPZvVEkBPaJdCV4vy5wyghty5ROFbCRnm132v8BScu5/1BQ8g==} dependencies: debug: 3.2.7 - is-core-module: 2.13.0 + is-core-module: 2.13.1 resolve: 1.22.4 transitivePeerDependencies: - supports-color @@ -7573,8 +7605,8 @@ packages: regexpp: 3.2.0 dev: true - /eslint-plugin-import@2.28.1(@typescript-eslint/parser@6.8.0)(eslint@8.51.0): - resolution: {integrity: sha512-9I9hFlITvOV55alzoKBI+K9q74kv0iKMeY6av5+umsNwayt59fz692daGyjR+oStBQgx6nwR9rXldDev3Clw+A==} + /eslint-plugin-import@2.29.0(@typescript-eslint/parser@6.8.0)(eslint@8.51.0): + resolution: {integrity: sha512-QPOO5NO6Odv5lpoTkddtutccQjysJuFxoPS7fAHO+9m9udNHvTCPSAMW9zGAYj8lAIdr40I8yPCdUYrncXtrwg==} engines: {node: '>=4'} peerDependencies: '@typescript-eslint/parser': '*' @@ -7584,22 +7616,22 @@ packages: optional: true dependencies: '@typescript-eslint/parser': 6.8.0(eslint@8.51.0)(typescript@5.2.2) - array-includes: 3.1.6 - array.prototype.findlastindex: 1.2.2 - array.prototype.flat: 1.3.1 - array.prototype.flatmap: 1.3.1 + array-includes: 3.1.7 + array.prototype.findlastindex: 1.2.3 + array.prototype.flat: 1.3.2 + array.prototype.flatmap: 1.3.2 debug: 3.2.7 doctrine: 2.1.0 eslint: 8.51.0 eslint-import-resolver-node: 0.3.9 eslint-module-utils: 2.8.0(@typescript-eslint/parser@6.8.0)(eslint-import-resolver-node@0.3.9)(eslint@8.51.0) - has: 1.0.3 - is-core-module: 2.13.0 + hasown: 2.0.0 + is-core-module: 2.13.1 is-glob: 4.0.3 minimatch: 3.1.2 - object.fromentries: 2.0.6 - object.groupby: 1.0.0 - object.values: 1.1.6 + object.fromentries: 2.0.7 + object.groupby: 1.0.1 + object.values: 1.1.7 semver: 6.3.1 tsconfig-paths: 3.14.2 transitivePeerDependencies: @@ -8611,6 +8643,9 @@ packages: /function-bind@1.1.1: resolution: {integrity: sha512-yIovAzMX49sF8Yl58fSCWJ5svSLuaibPxXQJFLmBObTuCr0Mf1KiPopGM9NiFjiYBCbfaa2Fh6breQ6ANVTI0A==} + /function-bind@1.1.2: + resolution: {integrity: sha512-7XHNxH7qX9xG5mIwxkhumTox/MIRNcOgDrxWsMt2pAr23WHp6MrRlN7FBSFpCpr+oVO0F744iUgR82nJMfG2SA==} + /function.prototype.name@1.1.5: resolution: {integrity: sha512-uN7m/BzVKQnCUF/iW8jYea67v++2u7m5UgENbHRtdDVclOUP+FMPlCNdmk0h/ysGyo2tavMJEDqJAkJdRa1vMA==} engines: {node: '>= 0.4'} @@ -9078,6 +9113,12 @@ packages: type-fest: 0.8.1 dev: true + /hasown@2.0.0: + resolution: {integrity: sha512-vUptKVTpIJhcczKBbgnS+RtcuYMB8+oNzPK2/Hp3hanz8JmpATdmmgLgSaadVREkDm+e2giHwY3ZRkyjSIDDFA==} + engines: {node: '>= 0.4'} + dependencies: + function-bind: 1.1.2 + /he@1.2.0: resolution: {integrity: sha512-F/1DnUGPopORZi0ni+CvrCgHQ5FyEAHRLSApuYWMmrbSwoN2Mn/7k+Gl38gJnR7yyDZk6WLXwiGod1JOWNDKGw==} hasBin: true @@ -9381,6 +9422,12 @@ packages: resolution: {integrity: sha512-Z7dk6Qo8pOCp3l4tsX2C5ZVas4V+UxwQodwZhLopL91TX8UyyHEXafPcyoeeWuLrwzHcr3igO78wNLwHJHsMCQ==} dependencies: has: 1.0.3 + dev: true + + /is-core-module@2.13.1: + resolution: {integrity: sha512-hHrIjvZsftOsvKSn2TRYl63zvxsgE0K+0mYMoH6gD4omR5IWB2KynivBQczo3+wF1cCkjzvptnI9Q0sPU66ilw==} + dependencies: + hasown: 2.0.0 /is-date-object@1.0.5: resolution: {integrity: sha512-9YQaSxsAiSwcvS33MBk3wTCVnWK+HhF8VZR2jRxehM16QcVOdHqPn4VPHmRK4lSr38n9JriurInLcP90xsYNfQ==} @@ -11458,8 +11505,17 @@ packages: es-abstract: 1.22.1 dev: true - /object.groupby@1.0.0: - resolution: {integrity: sha512-70MWG6NfRH9GnbZOikuhPPYzpUpof9iW2J9E4dW7FXTqPNb6rllE6u39SKwwiNh8lCwX3DDb5OgcKGiEBrTTyw==} + /object.fromentries@2.0.7: + resolution: {integrity: sha512-UPbPHML6sL8PI/mOqPwsH4G6iyXcCGzLin8KvEPenOZN5lpCNBZZQ+V62vdjB1mQHrmqGQt5/OJzemUA+KJmEA==} + engines: {node: '>= 0.4'} + dependencies: + call-bind: 1.0.2 + define-properties: 1.2.0 + es-abstract: 1.22.1 + dev: true + + /object.groupby@1.0.1: + resolution: {integrity: sha512-HqaQtqLnp/8Bn4GL16cj+CUYbnpe1bh0TtEaWvybszDG4tgxCJuRpV8VGuvNaI1fAnI4lUJzDG55MXcOH4JZcQ==} dependencies: call-bind: 1.0.2 define-properties: 1.2.0 @@ -11483,6 +11539,15 @@ packages: es-abstract: 1.22.1 dev: true + /object.values@1.1.7: + resolution: {integrity: sha512-aU6xnDFYT3x17e/f0IiiwlGPTy2jzMySGfUB4fq6z7CV8l85CWHDk5ErhyhpfDHhrOMwGFhSQkhMGHaIotA6Ng==} + engines: {node: '>= 0.4'} + dependencies: + call-bind: 1.0.2 + define-properties: 1.2.0 + es-abstract: 1.22.1 + dev: true + /obliterator@1.6.1: resolution: {integrity: sha512-9WXswnqINnnhOG/5SLimUlzuU1hFJUc8zkwyD59Sd+dPOMf05PmnYG/d6Q7HZ+KmgkZJa1PxRso6QdM3sTNHig==} dev: true @@ -12652,7 +12717,7 @@ packages: resolution: {integrity: sha512-PXNdCiPqDqeUou+w1C2eTQbNfxKSuMxqTCuvlmmMsk1NWHL5fRrhY6Pl0qEYYc6+QqGClco1Qj8XnjPego4wfg==} hasBin: true dependencies: - is-core-module: 2.13.0 + is-core-module: 2.13.1 path-parse: 1.0.7 supports-preserve-symlinks-flag: 1.0.0 From 99a856b7f46806919ca726393fc1824fcd92f367 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 23 Oct 2023 20:13:40 +0000 Subject: [PATCH 018/279] build(deps-dev): bump @swc/core from 1.3.92 to 1.3.94 Bumps [@swc/core](https://github.com/swc-project/swc) from 1.3.92 to 1.3.94. - [Release notes](https://github.com/swc-project/swc/releases) - [Changelog](https://github.com/swc-project/swc/blob/main/CHANGELOG.md) - [Commits](https://github.com/swc-project/swc/compare/v1.3.92...v1.3.94) --- updated-dependencies: - dependency-name: "@swc/core" dependency-type: direct:development update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] --- packages/web3js-plugin/package.json | 2 +- pnpm-lock.yaml | 80 ++++++++++++++--------------- 2 files changed, 41 insertions(+), 41 deletions(-) diff --git a/packages/web3js-plugin/package.json b/packages/web3js-plugin/package.json index 45259064724c..1d56460893e8 100644 --- a/packages/web3js-plugin/package.json +++ b/packages/web3js-plugin/package.json @@ -33,7 +33,7 @@ }, "devDependencies": { "@eth-optimism/contracts-ts": "workspace:^", - "@swc/core": "^1.3.92", + "@swc/core": "^1.3.94", "@vitest/coverage-istanbul": "^0.34.6", "tsup": "^7.2.0", "typescript": "^5.2.2", diff --git a/pnpm-lock.yaml b/pnpm-lock.yaml index 9f2b147ddd5e..dda827bc24db 100644 --- a/pnpm-lock.yaml +++ b/pnpm-lock.yaml @@ -130,7 +130,7 @@ importers: devDependencies: tsup: specifier: ^7.2.0 - version: 7.2.0(@swc/core@1.3.92)(typescript@5.2.2) + version: 7.2.0(@swc/core@1.3.94)(typescript@5.2.2) vitest: specifier: ^0.34.4 version: 0.34.4 @@ -342,7 +342,7 @@ importers: version: 22.1.0 tsup: specifier: ^7.2.0 - version: 7.2.0(@swc/core@1.3.92)(typescript@5.2.2) + version: 7.2.0(@swc/core@1.3.94)(typescript@5.2.2) typescript: specifier: ^5.2.2 version: 5.2.2 @@ -433,7 +433,7 @@ importers: version: 22.1.0 tsup: specifier: ^7.2.0 - version: 7.2.0(@swc/core@1.3.92)(typescript@5.2.2) + version: 7.2.0(@swc/core@1.3.94)(typescript@5.2.2) typescript: specifier: ^5.2.2 version: 5.2.2 @@ -557,14 +557,14 @@ importers: specifier: workspace:^ version: link:../contracts-ts '@swc/core': - specifier: ^1.3.92 - version: 1.3.92 + specifier: ^1.3.94 + version: 1.3.94 '@vitest/coverage-istanbul': specifier: ^0.34.6 version: 0.34.6(vitest@0.34.1) tsup: specifier: ^7.2.0 - version: 7.2.0(@swc/core@1.3.92)(typescript@5.2.2) + version: 7.2.0(@swc/core@1.3.94)(typescript@5.2.2) typescript: specifier: ^5.2.2 version: 5.2.2 @@ -3550,8 +3550,8 @@ packages: '@stablelib/random': 1.0.2 '@stablelib/wipe': 1.0.1 - /@swc/core-darwin-arm64@1.3.92: - resolution: {integrity: sha512-v7PqZUBtIF6Q5Cp48gqUiG8zQQnEICpnfNdoiY3xjQAglCGIQCjJIDjreZBoeZQZspB27lQN4eZ43CX18+2SnA==} + /@swc/core-darwin-arm64@1.3.94: + resolution: {integrity: sha512-KNuE6opIy/wAXiGUWLhGWhCG3wA/AdjG6eYkv6dstrAURLaQMAoD8vDfVm8pxS8FA8Kx+0Z4QiDNPqk5aKIsqg==} engines: {node: '>=10'} cpu: [arm64] os: [darwin] @@ -3559,8 +3559,8 @@ packages: dev: true optional: true - /@swc/core-darwin-x64@1.3.92: - resolution: {integrity: sha512-Q3XIgQfXyxxxms3bPN+xGgvwk0TtG9l89IomApu+yTKzaIIlf051mS+lGngjnh9L0aUiCp6ICyjDLtutWP54fw==} + /@swc/core-darwin-x64@1.3.94: + resolution: {integrity: sha512-HypemhyehQrLqXwfJv5ronD4BMAXdgMCP4Ei7rt3B6Ftmt9axwGvdwGiXxsYR9h1ncyxoVxN+coGxbNIhKhahw==} engines: {node: '>=10'} cpu: [x64] os: [darwin] @@ -3568,8 +3568,8 @@ packages: dev: true optional: true - /@swc/core-linux-arm-gnueabihf@1.3.92: - resolution: {integrity: sha512-tnOCoCpNVXC+0FCfG84PBZJyLlz0Vfj9MQhyhCvlJz9hQmvpf8nTdKH7RHrOn8VfxtUBLdVi80dXgIFgbvl7qA==} + /@swc/core-linux-arm-gnueabihf@1.3.94: + resolution: {integrity: sha512-KzKN54c7Y6X1db+bBVSXG4+bXmAPvXtDWk+TgwNJH4yYliOrnP/RKkHA5QZ9VFSnqJF06/sAO4kYBiL/aVQDBQ==} engines: {node: '>=10'} cpu: [arm] os: [linux] @@ -3577,8 +3577,8 @@ packages: dev: true optional: true - /@swc/core-linux-arm64-gnu@1.3.92: - resolution: {integrity: sha512-lFfGhX32w8h1j74Iyz0Wv7JByXIwX11OE9UxG+oT7lG0RyXkF4zKyxP8EoxfLrDXse4Oop434p95e3UNC3IfCw==} + /@swc/core-linux-arm64-gnu@1.3.94: + resolution: {integrity: sha512-iAcR8Ho0Uck/SLSrgYfXkpcGOXuN5waMZO7GlL/52QODr7GJtOfZ0H1MCZLbIFkPJp/iXoJpYgym4d/qSd477Q==} engines: {node: '>=10'} cpu: [arm64] os: [linux] @@ -3586,8 +3586,8 @@ packages: dev: true optional: true - /@swc/core-linux-arm64-musl@1.3.92: - resolution: {integrity: sha512-rOZtRcLj57MSAbiecMsqjzBcZDuaCZ8F6l6JDwGkQ7u1NYR57cqF0QDyU7RKS1Jq27Z/Vg21z5cwqoH5fLN+Sg==} + /@swc/core-linux-arm64-musl@1.3.94: + resolution: {integrity: sha512-VCHL1Mb9ENHx+sAeubSSg481MUeP9/PYzPPy9tfswunj/w35M+vEWflwK2dzQL9kUTFD3zcFTpAgsKnj6aX24w==} engines: {node: '>=10'} cpu: [arm64] os: [linux] @@ -3595,8 +3595,8 @@ packages: dev: true optional: true - /@swc/core-linux-x64-gnu@1.3.92: - resolution: {integrity: sha512-qptoMGnBL6v89x/Qpn+l1TH1Y0ed+v0qhNfAEVzZvCvzEMTFXphhlhYbDdpxbzRmCjH6GOGq7Y+xrWt9T1/ARg==} + /@swc/core-linux-x64-gnu@1.3.94: + resolution: {integrity: sha512-gjq7U6clhJi0Oel2a4gwR4MbSu+THQ2hmBNVCOSA3JjPZWZTkJXaJDpnh/r7PJxKBwUDlo0VPlwiwjepAQR2Rw==} engines: {node: '>=10'} cpu: [x64] os: [linux] @@ -3604,8 +3604,8 @@ packages: dev: true optional: true - /@swc/core-linux-x64-musl@1.3.92: - resolution: {integrity: sha512-g2KrJ43bZkCZHH4zsIV5ErojuV1OIpUHaEyW1gf7JWKaFBpWYVyubzFPvPkjcxHGLbMsEzO7w/NVfxtGMlFH/Q==} + /@swc/core-linux-x64-musl@1.3.94: + resolution: {integrity: sha512-rSylruWyeol2ujZDHmwiovupMR5ukMXivlA7DDxmQ1dFUV9HuiPknQrU5rEbI3V2V3V5RkpbEKjnADen7AeMPQ==} engines: {node: '>=10'} cpu: [x64] os: [linux] @@ -3613,8 +3613,8 @@ packages: dev: true optional: true - /@swc/core-win32-arm64-msvc@1.3.92: - resolution: {integrity: sha512-3MCRGPAYDoQ8Yyd3WsCMc8eFSyKXY5kQLyg/R5zEqA0uthomo0m0F5/fxAJMZGaSdYkU1DgF73ctOWOf+Z/EzQ==} + /@swc/core-win32-arm64-msvc@1.3.94: + resolution: {integrity: sha512-OenDUr5MQkz506ebVQq6ezoZ3GZ26nchgf5mPnwab4gx2TEiyR9zn7MdX5LWskTmOK3+FszPbGK0B5oLK6Y5yw==} engines: {node: '>=10'} cpu: [arm64] os: [win32] @@ -3622,8 +3622,8 @@ packages: dev: true optional: true - /@swc/core-win32-ia32-msvc@1.3.92: - resolution: {integrity: sha512-zqTBKQhgfWm73SVGS8FKhFYDovyRl1f5dTX1IwSKynO0qHkRCqJwauFJv/yevkpJWsI2pFh03xsRs9HncTQKSA==} + /@swc/core-win32-ia32-msvc@1.3.94: + resolution: {integrity: sha512-mi6NcmtJKnaiHAxLtVz+WzunscsEwPdA0j15DuiYVx06Xo+MdRLJj4eVBgVLwGD1AI3IqKs4MVVx2cD7n0h5mg==} engines: {node: '>=10'} cpu: [ia32] os: [win32] @@ -3631,8 +3631,8 @@ packages: dev: true optional: true - /@swc/core-win32-x64-msvc@1.3.92: - resolution: {integrity: sha512-41bE66ddr9o/Fi1FBh0sHdaKdENPTuDpv1IFHxSg0dJyM/jX8LbkjnpdInYXHBxhcLVAPraVRrNsC4SaoPw2Pg==} + /@swc/core-win32-x64-msvc@1.3.94: + resolution: {integrity: sha512-Ba0ZLcGMnqPWWF9Xa+rWhhnkpvE7XoQegMP/VCF2JIHb2ieGBC8jChO6nKRFKZjib/3wghGzxakyDQx3LDhDug==} engines: {node: '>=10'} cpu: [x64] os: [win32] @@ -3640,8 +3640,8 @@ packages: dev: true optional: true - /@swc/core@1.3.92: - resolution: {integrity: sha512-vx0vUrf4YTEw59njOJ46Ha5i0cZTMYdRHQ7KXU29efN1MxcmJH2RajWLPlvQarOP1ab9iv9cApD7SMchDyx2vA==} + /@swc/core@1.3.94: + resolution: {integrity: sha512-jTHn8UJOGgERKZLy8euEixVAzC/w/rUSuMlM3e7hxgap/TC595hSkuQwtkpL238dsuEPveD44GMy2A5UBtSvjg==} engines: {node: '>=10'} requiresBuild: true peerDependencies: @@ -3653,16 +3653,16 @@ packages: '@swc/counter': 0.1.1 '@swc/types': 0.1.5 optionalDependencies: - '@swc/core-darwin-arm64': 1.3.92 - '@swc/core-darwin-x64': 1.3.92 - '@swc/core-linux-arm-gnueabihf': 1.3.92 - '@swc/core-linux-arm64-gnu': 1.3.92 - '@swc/core-linux-arm64-musl': 1.3.92 - '@swc/core-linux-x64-gnu': 1.3.92 - '@swc/core-linux-x64-musl': 1.3.92 - '@swc/core-win32-arm64-msvc': 1.3.92 - '@swc/core-win32-ia32-msvc': 1.3.92 - '@swc/core-win32-x64-msvc': 1.3.92 + '@swc/core-darwin-arm64': 1.3.94 + '@swc/core-darwin-x64': 1.3.94 + '@swc/core-linux-arm-gnueabihf': 1.3.94 + '@swc/core-linux-arm64-gnu': 1.3.94 + '@swc/core-linux-arm64-musl': 1.3.94 + '@swc/core-linux-x64-gnu': 1.3.94 + '@swc/core-linux-x64-musl': 1.3.94 + '@swc/core-win32-arm64-msvc': 1.3.94 + '@swc/core-win32-ia32-msvc': 1.3.94 + '@swc/core-win32-x64-msvc': 1.3.94 dev: true /@swc/counter@0.1.1: @@ -13820,7 +13820,7 @@ packages: resolution: {integrity: sha512-Tyrf5mxF8Ofs1tNoxA13lFeZ2Zrbd6cKbuH3V+MQ5sb6DtBj5FjrXVsRWT8YvNAQTqNoz66dz1WsbigI22aEnw==} dev: true - /tsup@7.2.0(@swc/core@1.3.92)(typescript@5.2.2): + /tsup@7.2.0(@swc/core@1.3.94)(typescript@5.2.2): resolution: {integrity: sha512-vDHlczXbgUvY3rWvqFEbSqmC1L7woozbzngMqTtL2PGBODTtWlRwGDDawhvWzr5c1QjKe4OAKqJGfE1xeXUvtQ==} engines: {node: '>=16.14'} hasBin: true @@ -13836,7 +13836,7 @@ packages: typescript: optional: true dependencies: - '@swc/core': 1.3.92 + '@swc/core': 1.3.94 bundle-require: 4.0.1(esbuild@0.18.15) cac: 6.7.14 chokidar: 3.5.3 From 82fd1608d0f03f84a3761cc2d641a662b2c6424a Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 23 Oct 2023 20:16:20 +0000 Subject: [PATCH 019/279] build(deps): bump pino from 8.16.0 to 8.16.1 Bumps [pino](https://github.com/pinojs/pino) from 8.16.0 to 8.16.1. - [Release notes](https://github.com/pinojs/pino/releases) - [Commits](https://github.com/pinojs/pino/compare/v8.16.0...v8.16.1) --- updated-dependencies: - dependency-name: pino dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] --- packages/common-ts/package.json | 2 +- pnpm-lock.yaml | 10 +++++----- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/packages/common-ts/package.json b/packages/common-ts/package.json index 92780c7b4ef7..446d3541d30f 100644 --- a/packages/common-ts/package.json +++ b/packages/common-ts/package.json @@ -46,7 +46,7 @@ "express-prom-bundle": "^6.6.0", "lodash": "^4.17.21", "morgan": "^1.10.0", - "pino": "^8.16.0", + "pino": "^8.16.1", "pino-multi-stream": "^6.0.0", "pino-sentry": "^0.14.0", "prom-client": "^14.2.0" diff --git a/pnpm-lock.yaml b/pnpm-lock.yaml index 9f2b147ddd5e..c0c01af171fe 100644 --- a/pnpm-lock.yaml +++ b/pnpm-lock.yaml @@ -228,8 +228,8 @@ importers: specifier: ^1.10.0 version: 1.10.0 pino: - specifier: ^8.16.0 - version: 8.16.0 + specifier: ^8.16.1 + version: 8.16.1 pino-multi-stream: specifier: ^6.0.0 version: 6.0.0 @@ -4120,7 +4120,7 @@ packages: resolution: {integrity: sha512-wKoab31pknvILkxAF8ss+v9iNyhw5Iu/0jLtRkUD74cNfOOLJNnqfFKAv0r7wVaTQxRZtWrMpGfShwwBjOcgcg==} deprecated: This is a stub types definition. pino provides its own type definitions, so you do not need this installed. dependencies: - pino: 8.16.0 + pino: 8.16.1 dev: true /@types/prettier@2.3.2: @@ -11974,8 +11974,8 @@ packages: sonic-boom: 2.8.0 thread-stream: 0.15.2 - /pino@8.16.0: - resolution: {integrity: sha512-UUmvQ/7KTZt/vHjhRrnyS7h+J7qPBQnpG80V56xmIC+o9IqYmQOw/UIny9S9zYDfRBR0ClouCr464EkBMIT7Fw==} + /pino@8.16.1: + resolution: {integrity: sha512-3bKsVhBmgPjGV9pyn4fO/8RtoVDR8ssW1ev819FsRXlRNgW8gR/9Kx+gCK4UPWd4JjrRDLWpzd/pb1AyWm3MGA==} hasBin: true dependencies: atomic-sleep: 1.0.0 From 7bc9cd7eb44bce6b6d41cdaf2d44310691b31c1d Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Tue, 24 Oct 2023 07:25:13 +1000 Subject: [PATCH 020/279] op-program: Fix fpp-verify by actually setting the datadir --- op-program/verify/cmd/goerli.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/op-program/verify/cmd/goerli.go b/op-program/verify/cmd/goerli.go index c236ac7c36ae..88e1fbf0b0a4 100644 --- a/op-program/verify/cmd/goerli.go +++ b/op-program/verify/cmd/goerli.go @@ -98,7 +98,7 @@ func Run(l1RpcUrl string, l1RpcKind string, l2RpcUrl string, l2OracleAddr common l1Head := l1HeadBlock.Hash() if dataDir == "" { - dataDir, err := os.MkdirTemp("", "oracledata") + dataDir, err = os.MkdirTemp("", "oracledata") if err != nil { return fmt.Errorf("create temp dir: %w", err) } From 03c0061b9178b2d132802dcb02d0d7328d99bbe7 Mon Sep 17 00:00:00 2001 From: protolambda Date: Mon, 23 Oct 2023 23:28:43 +0200 Subject: [PATCH 021/279] devnet: set git commit/date build-args, handle op-program/challenger/node version format cases --- bedrock-devnet/devnet/__init__.py | 9 +++++++-- op-challenger/cmd/main.go | 6 +++++- op-node/cmd/main.go | 10 ++++++---- op-program/host/cmd/main.go | 6 +++++- 4 files changed, 23 insertions(+), 8 deletions(-) diff --git a/bedrock-devnet/devnet/__init__.py b/bedrock-devnet/devnet/__init__.py index 056929e2d6bf..c68968c02236 100644 --- a/bedrock-devnet/devnet/__init__.py +++ b/bedrock-devnet/devnet/__init__.py @@ -94,8 +94,13 @@ def main(): devnet_l1_genesis(paths) return - log.info('Building docker images') - run_command(['docker', 'compose', 'build', '--progress', 'plain'], cwd=paths.ops_bedrock_dir, env={ + git_commit = subprocess.run(['git', 'rev-parse', 'HEAD'], capture_output=True, text=True).stdout.strip() + git_date = subprocess.run(['git', 'show', '-s', "--format='%ct'"], capture_output=True, text=True).stdout.strip() + + log.info(f'Building docker images for git commit {git_commit} ({git_date})') + run_command(['docker', 'compose', 'build', '--progress', 'plain', + '--build-arg', f'GIT_COMMIT={git_commit}', '--build-arg', f'GIT_DATE={git_date}'], + cwd=paths.ops_bedrock_dir, env={ 'PWD': paths.ops_bedrock_dir, 'DOCKER_BUILDKIT': '1', # (should be available by default in later versions, but explicitly enable it anyway) 'COMPOSE_DOCKER_CLI_BUILD': '1' # use the docker cache diff --git a/op-challenger/cmd/main.go b/op-challenger/cmd/main.go index f33ac22d7de2..9a1e6655aaac 100644 --- a/op-challenger/cmd/main.go +++ b/op-challenger/cmd/main.go @@ -24,7 +24,11 @@ var ( var VersionWithMeta = func() string { v := version.Version if GitCommit != "" { - v += "-" + GitCommit[:8] + if len(GitCommit) >= 8 { + v += "-" + GitCommit[:8] + } else { + v += "-" + GitCommit + } } if GitDate != "" { v += "-" + GitDate diff --git a/op-node/cmd/main.go b/op-node/cmd/main.go index ccd54946bbab..bc400d7927d4 100644 --- a/op-node/cmd/main.go +++ b/op-node/cmd/main.go @@ -31,10 +31,12 @@ var ( // VersionWithMeta holds the textual version string including the metadata. var VersionWithMeta = func() string { v := version.Version - if len(GitCommit) >= 8 { - v += "-" + GitCommit[:8] - } else { - v += "-" + GitCommit + if GitCommit != "" { + if len(GitCommit) >= 8 { + v += "-" + GitCommit[:8] + } else { + v += "-" + GitCommit + } } if GitDate != "" { v += "-" + GitDate diff --git a/op-program/host/cmd/main.go b/op-program/host/cmd/main.go index cf4022c09e74..58cfda30e973 100644 --- a/op-program/host/cmd/main.go +++ b/op-program/host/cmd/main.go @@ -21,7 +21,11 @@ var ( var VersionWithMeta = func() string { v := version.Version if GitCommit != "" { - v += "-" + GitCommit[:8] + if len(GitCommit) >= 8 { + v += "-" + GitCommit[:8] + } else { + v += "-" + GitCommit + } } if GitDate != "" { v += "-" + GitDate From f27250916d209b6c776cf2031846c62113ead7e9 Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Tue, 24 Oct 2023 09:36:44 +1000 Subject: [PATCH 022/279] Remove single quotes --- bedrock-devnet/devnet/__init__.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bedrock-devnet/devnet/__init__.py b/bedrock-devnet/devnet/__init__.py index c68968c02236..a0b1afec26e8 100644 --- a/bedrock-devnet/devnet/__init__.py +++ b/bedrock-devnet/devnet/__init__.py @@ -95,7 +95,7 @@ def main(): return git_commit = subprocess.run(['git', 'rev-parse', 'HEAD'], capture_output=True, text=True).stdout.strip() - git_date = subprocess.run(['git', 'show', '-s', "--format='%ct'"], capture_output=True, text=True).stdout.strip() + git_date = subprocess.run(['git', 'show', '-s', "--format=%ct"], capture_output=True, text=True).stdout.strip() log.info(f'Building docker images for git commit {git_commit} ({git_date})') run_command(['docker', 'compose', 'build', '--progress', 'plain', From b4066f94cf288502ff66cd13df2e7399be6f0f44 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 24 Oct 2023 00:55:54 +0000 Subject: [PATCH 023/279] build(deps-dev): bump @typescript-eslint/parser from 6.8.0 to 6.9.0 Bumps [@typescript-eslint/parser](https://github.com/typescript-eslint/typescript-eslint/tree/HEAD/packages/parser) from 6.8.0 to 6.9.0. - [Release notes](https://github.com/typescript-eslint/typescript-eslint/releases) - [Changelog](https://github.com/typescript-eslint/typescript-eslint/blob/main/packages/parser/CHANGELOG.md) - [Commits](https://github.com/typescript-eslint/typescript-eslint/commits/v6.9.0/packages/parser) --- updated-dependencies: - dependency-name: "@typescript-eslint/parser" dependency-type: direct:development update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] --- package.json | 2 +- packages/contracts-bedrock/package.json | 2 +- pnpm-lock.yaml | 68 ++++++++++++------------- 3 files changed, 36 insertions(+), 36 deletions(-) diff --git a/package.json b/package.json index 8cc8b7668dd0..374220207702 100644 --- a/package.json +++ b/package.json @@ -45,7 +45,7 @@ "@types/mocha": "^10.0.3", "@types/node": "^20.8.6", "@typescript-eslint/eslint-plugin": "^6.7.5", - "@typescript-eslint/parser": "^6.8.0", + "@typescript-eslint/parser": "^6.9.0", "chai": "^4.3.10", "depcheck": "^1.4.7", "doctoc": "^2.2.0", diff --git a/packages/contracts-bedrock/package.json b/packages/contracts-bedrock/package.json index 617d3778cd67..ec14f917aedb 100644 --- a/packages/contracts-bedrock/package.json +++ b/packages/contracts-bedrock/package.json @@ -46,7 +46,7 @@ }, "devDependencies": { "@typescript-eslint/eslint-plugin": "^6.7.5", - "@typescript-eslint/parser": "^6.8.0", + "@typescript-eslint/parser": "^6.9.0", "tsx": "^3.14.0", "typescript": "^5.2.2" } diff --git a/pnpm-lock.yaml b/pnpm-lock.yaml index e25e904b6d5d..8292e5ba7ab3 100644 --- a/pnpm-lock.yaml +++ b/pnpm-lock.yaml @@ -35,10 +35,10 @@ importers: version: 20.8.6 '@typescript-eslint/eslint-plugin': specifier: ^6.7.5 - version: 6.7.5(@typescript-eslint/parser@6.8.0)(eslint@8.51.0)(typescript@5.2.2) + version: 6.7.5(@typescript-eslint/parser@6.9.0)(eslint@8.51.0)(typescript@5.2.2) '@typescript-eslint/parser': - specifier: ^6.8.0 - version: 6.8.0(eslint@8.51.0)(typescript@5.2.2) + specifier: ^6.9.0 + version: 6.9.0(eslint@8.51.0)(typescript@5.2.2) chai: specifier: ^4.3.10 version: 4.3.10 @@ -59,7 +59,7 @@ importers: version: 16.0.3(eslint-plugin-import@2.29.0)(eslint-plugin-node@11.1.0)(eslint-plugin-promise@5.2.0)(eslint@8.51.0) eslint-plugin-import: specifier: ^2.29.0 - version: 2.29.0(@typescript-eslint/parser@6.8.0)(eslint@8.51.0) + version: 2.29.0(@typescript-eslint/parser@6.9.0)(eslint@8.51.0) eslint-plugin-jsdoc: specifier: ^35.1.2 version: 35.5.1(eslint@8.51.0) @@ -269,10 +269,10 @@ importers: devDependencies: '@typescript-eslint/eslint-plugin': specifier: ^6.7.5 - version: 6.7.5(@typescript-eslint/parser@6.8.0)(eslint@8.51.0)(typescript@5.2.2) + version: 6.7.5(@typescript-eslint/parser@6.9.0)(eslint@8.51.0)(typescript@5.2.2) '@typescript-eslint/parser': - specifier: ^6.8.0 - version: 6.8.0(eslint@8.51.0)(typescript@5.2.2) + specifier: ^6.9.0 + version: 6.9.0(eslint@8.51.0)(typescript@5.2.2) tsx: specifier: ^3.14.0 version: 3.14.0 @@ -4238,7 +4238,7 @@ packages: '@types/node': 20.8.6 dev: false - /@typescript-eslint/eslint-plugin@6.7.5(@typescript-eslint/parser@6.8.0)(eslint@8.51.0)(typescript@5.2.2): + /@typescript-eslint/eslint-plugin@6.7.5(@typescript-eslint/parser@6.9.0)(eslint@8.51.0)(typescript@5.2.2): resolution: {integrity: sha512-JhtAwTRhOUcP96D0Y6KYnwig/MRQbOoLGXTON2+LlyB/N35SP9j1boai2zzwXb7ypKELXMx3DVk9UTaEq1vHEw==} engines: {node: ^16.0.0 || >=18.0.0} peerDependencies: @@ -4250,7 +4250,7 @@ packages: optional: true dependencies: '@eslint-community/regexpp': 4.6.2 - '@typescript-eslint/parser': 6.8.0(eslint@8.51.0)(typescript@5.2.2) + '@typescript-eslint/parser': 6.9.0(eslint@8.51.0)(typescript@5.2.2) '@typescript-eslint/scope-manager': 6.7.5 '@typescript-eslint/type-utils': 6.7.5(eslint@8.51.0)(typescript@5.2.2) '@typescript-eslint/utils': 6.7.5(eslint@8.51.0)(typescript@5.2.2) @@ -4267,8 +4267,8 @@ packages: - supports-color dev: true - /@typescript-eslint/parser@6.8.0(eslint@8.51.0)(typescript@5.2.2): - resolution: {integrity: sha512-5tNs6Bw0j6BdWuP8Fx+VH4G9fEPDxnVI7yH1IAPkQH5RUtvKwRoqdecAPdQXv4rSOADAaz1LFBZvZG7VbXivSg==} + /@typescript-eslint/parser@6.9.0(eslint@8.51.0)(typescript@5.2.2): + resolution: {integrity: sha512-GZmjMh4AJ/5gaH4XF2eXA8tMnHWP+Pm1mjQR2QN4Iz+j/zO04b9TOvJYOX2sCNIQHtRStKTxRY1FX7LhpJT4Gw==} engines: {node: ^16.0.0 || >=18.0.0} peerDependencies: eslint: ^7.0.0 || ^8.0.0 @@ -4277,10 +4277,10 @@ packages: typescript: optional: true dependencies: - '@typescript-eslint/scope-manager': 6.8.0 - '@typescript-eslint/types': 6.8.0 - '@typescript-eslint/typescript-estree': 6.8.0(typescript@5.2.2) - '@typescript-eslint/visitor-keys': 6.8.0 + '@typescript-eslint/scope-manager': 6.9.0 + '@typescript-eslint/types': 6.9.0 + '@typescript-eslint/typescript-estree': 6.9.0(typescript@5.2.2) + '@typescript-eslint/visitor-keys': 6.9.0 debug: 4.3.4(supports-color@8.1.1) eslint: 8.51.0 typescript: 5.2.2 @@ -4296,12 +4296,12 @@ packages: '@typescript-eslint/visitor-keys': 6.7.5 dev: true - /@typescript-eslint/scope-manager@6.8.0: - resolution: {integrity: sha512-xe0HNBVwCph7rak+ZHcFD6A+q50SMsFwcmfdjs9Kz4qDh5hWhaPhFjRs/SODEhroBI5Ruyvyz9LfwUJ624O40g==} + /@typescript-eslint/scope-manager@6.9.0: + resolution: {integrity: sha512-1R8A9Mc39n4pCCz9o79qRO31HGNDvC7UhPhv26TovDsWPBDx+Sg3rOZdCELIA3ZmNoWAuxaMOT7aWtGRSYkQxw==} engines: {node: ^16.0.0 || >=18.0.0} dependencies: - '@typescript-eslint/types': 6.8.0 - '@typescript-eslint/visitor-keys': 6.8.0 + '@typescript-eslint/types': 6.9.0 + '@typescript-eslint/visitor-keys': 6.9.0 dev: true /@typescript-eslint/type-utils@6.7.5(eslint@8.51.0)(typescript@5.2.2): @@ -4329,8 +4329,8 @@ packages: engines: {node: ^16.0.0 || >=18.0.0} dev: true - /@typescript-eslint/types@6.8.0: - resolution: {integrity: sha512-p5qOxSum7W3k+llc7owEStXlGmSl8FcGvhYt8Vjy7FqEnmkCVlM3P57XQEGj58oqaBWDQXbJDZxwUWMS/EAPNQ==} + /@typescript-eslint/types@6.9.0: + resolution: {integrity: sha512-+KB0lbkpxBkBSiVCuQvduqMJy+I1FyDbdwSpM3IoBS7APl4Bu15lStPjgBIdykdRqQNYqYNMa8Kuidax6phaEw==} engines: {node: ^16.0.0 || >=18.0.0} dev: true @@ -4355,8 +4355,8 @@ packages: - supports-color dev: true - /@typescript-eslint/typescript-estree@6.8.0(typescript@5.2.2): - resolution: {integrity: sha512-ISgV0lQ8XgW+mvv5My/+iTUdRmGspducmQcDw5JxznasXNnZn3SKNrTRuMsEXv+V/O+Lw9AGcQCfVaOPCAk/Zg==} + /@typescript-eslint/typescript-estree@6.9.0(typescript@5.2.2): + resolution: {integrity: sha512-NJM2BnJFZBEAbCfBP00zONKXvMqihZCrmwCaik0UhLr0vAgb6oguXxLX1k00oQyD+vZZ+CJn3kocvv2yxm4awQ==} engines: {node: ^16.0.0 || >=18.0.0} peerDependencies: typescript: '*' @@ -4364,8 +4364,8 @@ packages: typescript: optional: true dependencies: - '@typescript-eslint/types': 6.8.0 - '@typescript-eslint/visitor-keys': 6.8.0 + '@typescript-eslint/types': 6.9.0 + '@typescript-eslint/visitor-keys': 6.9.0 debug: 4.3.4(supports-color@8.1.1) globby: 11.1.0 is-glob: 4.0.3 @@ -4403,11 +4403,11 @@ packages: eslint-visitor-keys: 3.4.3 dev: true - /@typescript-eslint/visitor-keys@6.8.0: - resolution: {integrity: sha512-oqAnbA7c+pgOhW2OhGvxm0t1BULX5peQI/rLsNDpGM78EebV3C9IGbX5HNZabuZ6UQrYveCLjKo8Iy/lLlBkkg==} + /@typescript-eslint/visitor-keys@6.9.0: + resolution: {integrity: sha512-dGtAfqjV6RFOtIP8I0B4ZTBRrlTT8NHHlZZSchQx3qReaoDeXhYM++M4So2AgFK9ZB0emRPA6JI1HkafzA2Ibg==} engines: {node: ^16.0.0 || >=18.0.0} dependencies: - '@typescript-eslint/types': 6.8.0 + '@typescript-eslint/types': 6.9.0 eslint-visitor-keys: 3.4.3 dev: true @@ -7550,7 +7550,7 @@ packages: eslint-plugin-promise: ^4.2.1 || ^5.0.0 dependencies: eslint: 8.51.0 - eslint-plugin-import: 2.29.0(@typescript-eslint/parser@6.8.0)(eslint@8.51.0) + eslint-plugin-import: 2.29.0(@typescript-eslint/parser@6.9.0)(eslint@8.51.0) eslint-plugin-node: 11.1.0(eslint@8.51.0) eslint-plugin-promise: 5.2.0(eslint@8.51.0) dev: true @@ -7565,7 +7565,7 @@ packages: - supports-color dev: true - /eslint-module-utils@2.8.0(@typescript-eslint/parser@6.8.0)(eslint-import-resolver-node@0.3.9)(eslint@8.51.0): + /eslint-module-utils@2.8.0(@typescript-eslint/parser@6.9.0)(eslint-import-resolver-node@0.3.9)(eslint@8.51.0): resolution: {integrity: sha512-aWajIYfsqCKRDgUfjEXNN/JlrzauMuSEy5sbd7WXbtW3EH6A6MpwEh42c7qD+MqQo9QMJ6fWLAeIJynx0g6OAw==} engines: {node: '>=4'} peerDependencies: @@ -7586,7 +7586,7 @@ packages: eslint-import-resolver-webpack: optional: true dependencies: - '@typescript-eslint/parser': 6.8.0(eslint@8.51.0)(typescript@5.2.2) + '@typescript-eslint/parser': 6.9.0(eslint@8.51.0)(typescript@5.2.2) debug: 3.2.7 eslint: 8.51.0 eslint-import-resolver-node: 0.3.9 @@ -7605,7 +7605,7 @@ packages: regexpp: 3.2.0 dev: true - /eslint-plugin-import@2.29.0(@typescript-eslint/parser@6.8.0)(eslint@8.51.0): + /eslint-plugin-import@2.29.0(@typescript-eslint/parser@6.9.0)(eslint@8.51.0): resolution: {integrity: sha512-QPOO5NO6Odv5lpoTkddtutccQjysJuFxoPS7fAHO+9m9udNHvTCPSAMW9zGAYj8lAIdr40I8yPCdUYrncXtrwg==} engines: {node: '>=4'} peerDependencies: @@ -7615,7 +7615,7 @@ packages: '@typescript-eslint/parser': optional: true dependencies: - '@typescript-eslint/parser': 6.8.0(eslint@8.51.0)(typescript@5.2.2) + '@typescript-eslint/parser': 6.9.0(eslint@8.51.0)(typescript@5.2.2) array-includes: 3.1.7 array.prototype.findlastindex: 1.2.3 array.prototype.flat: 1.3.2 @@ -7624,7 +7624,7 @@ packages: doctrine: 2.1.0 eslint: 8.51.0 eslint-import-resolver-node: 0.3.9 - eslint-module-utils: 2.8.0(@typescript-eslint/parser@6.8.0)(eslint-import-resolver-node@0.3.9)(eslint@8.51.0) + eslint-module-utils: 2.8.0(@typescript-eslint/parser@6.9.0)(eslint-import-resolver-node@0.3.9)(eslint@8.51.0) hasown: 2.0.0 is-core-module: 2.13.1 is-glob: 4.0.3 From aa999b189843ad305f5c78e4050ef32db109d59a Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Tue, 24 Oct 2023 11:14:58 +1000 Subject: [PATCH 024/279] op-service: Extract shared method for formatting versions. --- endpoint-monitor/cmd/main.go | 4 +- op-batcher/cmd/main.go | 4 +- op-challenger/cmd/main.go | 19 +-------- op-heartbeat/cmd/main.go | 4 +- op-node/cmd/main.go | 18 +-------- op-program/host/cmd/main.go | 19 +-------- op-proposer/cmd/main.go | 4 +- op-service/version.go | 19 +++++++++ op-service/version_test.go | 75 ++++++++++++++++++++++++++++++++++++ op-wheel/cmd/main.go | 4 +- 10 files changed, 109 insertions(+), 61 deletions(-) create mode 100644 op-service/version.go create mode 100644 op-service/version_test.go diff --git a/endpoint-monitor/cmd/main.go b/endpoint-monitor/cmd/main.go index b31ac9359c6e..4a7599488f0d 100644 --- a/endpoint-monitor/cmd/main.go +++ b/endpoint-monitor/cmd/main.go @@ -1,9 +1,9 @@ package main import ( - "fmt" "os" + opservice "github.com/ethereum-optimism/optimism/op-service" oplog "github.com/ethereum-optimism/optimism/op-service/log" "github.com/ethereum/go-ethereum/log" "github.com/urfave/cli/v2" @@ -22,7 +22,7 @@ func main() { app := cli.NewApp() app.Flags = endpointMonitor.CLIFlags("ENDPOINT_MONITOR") - app.Version = fmt.Sprintf("%s-%s-%s", Version, GitCommit, GitDate) + app.Version = opservice.FormatVersion(Version, GitCommit, GitDate, "") app.Name = "endpoint-monitor" app.Usage = "Endpoint Monitoring Service" app.Description = "" diff --git a/op-batcher/cmd/main.go b/op-batcher/cmd/main.go index ff3579a37bf0..58803abdd656 100644 --- a/op-batcher/cmd/main.go +++ b/op-batcher/cmd/main.go @@ -1,9 +1,9 @@ package main import ( - "fmt" "os" + opservice "github.com/ethereum-optimism/optimism/op-service" "github.com/urfave/cli/v2" "github.com/ethereum-optimism/optimism/op-batcher/batcher" @@ -26,7 +26,7 @@ func main() { app := cli.NewApp() app.Flags = cliapp.ProtectFlags(flags.Flags) - app.Version = fmt.Sprintf("%s-%s-%s", Version, GitCommit, GitDate) + app.Version = opservice.FormatVersion(Version, GitCommit, GitDate, "") app.Name = "op-batcher" app.Usage = "Batch Submitter Service" app.Description = "Service for generating and submitting L2 tx batches to L1" diff --git a/op-challenger/cmd/main.go b/op-challenger/cmd/main.go index 9a1e6655aaac..7909b834ba9f 100644 --- a/op-challenger/cmd/main.go +++ b/op-challenger/cmd/main.go @@ -5,6 +5,7 @@ import ( "os" op_challenger "github.com/ethereum-optimism/optimism/op-challenger" + opservice "github.com/ethereum-optimism/optimism/op-service" "github.com/ethereum/go-ethereum/log" "github.com/urfave/cli/v2" @@ -21,23 +22,7 @@ var ( ) // VersionWithMeta holds the textual version string including the metadata. -var VersionWithMeta = func() string { - v := version.Version - if GitCommit != "" { - if len(GitCommit) >= 8 { - v += "-" + GitCommit[:8] - } else { - v += "-" + GitCommit - } - } - if GitDate != "" { - v += "-" + GitDate - } - if version.Meta != "" { - v += "-" + version.Meta - } - return v -}() +var VersionWithMeta = opservice.FormatVersion(version.Version, GitCommit, GitDate, version.Meta) func main() { args := os.Args diff --git a/op-heartbeat/cmd/main.go b/op-heartbeat/cmd/main.go index 46ee115f5c5b..2e324661e8f7 100644 --- a/op-heartbeat/cmd/main.go +++ b/op-heartbeat/cmd/main.go @@ -1,11 +1,11 @@ package main import ( - "fmt" "os" heartbeat "github.com/ethereum-optimism/optimism/op-heartbeat" "github.com/ethereum-optimism/optimism/op-heartbeat/flags" + opservice "github.com/ethereum-optimism/optimism/op-service" oplog "github.com/ethereum-optimism/optimism/op-service/log" "github.com/ethereum/go-ethereum/log" "github.com/urfave/cli/v2" @@ -22,7 +22,7 @@ func main() { app := cli.NewApp() app.Flags = flags.Flags - app.Version = fmt.Sprintf("%s-%s-%s", Version, GitCommit, GitDate) + app.Version = opservice.FormatVersion(Version, GitCommit, GitDate, "") app.Name = "op-heartbeat" app.Usage = "Heartbeat recorder" app.Description = "Service that records opt-in heartbeats from op nodes" diff --git a/op-node/cmd/main.go b/op-node/cmd/main.go index bc400d7927d4..135db4059265 100644 --- a/op-node/cmd/main.go +++ b/op-node/cmd/main.go @@ -29,23 +29,7 @@ var ( ) // VersionWithMeta holds the textual version string including the metadata. -var VersionWithMeta = func() string { - v := version.Version - if GitCommit != "" { - if len(GitCommit) >= 8 { - v += "-" + GitCommit[:8] - } else { - v += "-" + GitCommit - } - } - if GitDate != "" { - v += "-" + GitDate - } - if version.Meta != "" { - v += "-" + version.Meta - } - return v -}() +var VersionWithMeta = opservice.FormatVersion(version.Version, GitCommit, GitDate, version.Meta) func main() { // Set up logger with a default INFO level in case we fail to parse flags, diff --git a/op-program/host/cmd/main.go b/op-program/host/cmd/main.go index 58cfda30e973..19d0f188730b 100644 --- a/op-program/host/cmd/main.go +++ b/op-program/host/cmd/main.go @@ -7,6 +7,7 @@ import ( "github.com/ethereum-optimism/optimism/op-program/host/config" "github.com/ethereum-optimism/optimism/op-program/host/flags" "github.com/ethereum-optimism/optimism/op-program/host/version" + opservice "github.com/ethereum-optimism/optimism/op-service" oplog "github.com/ethereum-optimism/optimism/op-service/log" "github.com/ethereum/go-ethereum/log" "github.com/urfave/cli/v2" @@ -18,23 +19,7 @@ var ( ) // VersionWithMeta holds the textual version string including the metadata. -var VersionWithMeta = func() string { - v := version.Version - if GitCommit != "" { - if len(GitCommit) >= 8 { - v += "-" + GitCommit[:8] - } else { - v += "-" + GitCommit - } - } - if GitDate != "" { - v += "-" + GitDate - } - if version.Meta != "" { - v += "-" + version.Meta - } - return v -}() +var VersionWithMeta = opservice.FormatVersion(version.Version, GitCommit, GitDate, version.Meta) func main() { args := os.Args diff --git a/op-proposer/cmd/main.go b/op-proposer/cmd/main.go index 2bd822d5038b..10d6ad5e8d3a 100644 --- a/op-proposer/cmd/main.go +++ b/op-proposer/cmd/main.go @@ -1,9 +1,9 @@ package main import ( - "fmt" "os" + opservice "github.com/ethereum-optimism/optimism/op-service" "github.com/urfave/cli/v2" "github.com/ethereum-optimism/optimism/op-proposer/flags" @@ -26,7 +26,7 @@ func main() { app := cli.NewApp() app.Flags = cliapp.ProtectFlags(flags.Flags) - app.Version = fmt.Sprintf("%s-%s-%s", Version, GitCommit, GitDate) + app.Version = opservice.FormatVersion(Version, GitCommit, GitDate, "") app.Name = "op-proposer" app.Usage = "L2Output Submitter" app.Description = "Service for generating and submitting L2 Output checkpoints to the L2OutputOracle contract" diff --git a/op-service/version.go b/op-service/version.go new file mode 100644 index 000000000000..81a039b34d53 --- /dev/null +++ b/op-service/version.go @@ -0,0 +1,19 @@ +package op_service + +func FormatVersion(version string, gitCommit string, gitDate string, meta string) string { + v := version + if gitCommit != "" { + if len(gitCommit) >= 8 { + v += "-" + gitCommit[:8] + } else { + v += "-" + gitCommit + } + } + if gitDate != "" { + v += "-" + gitDate + } + if meta != "" { + v += "-" + meta + } + return v +} diff --git a/op-service/version_test.go b/op-service/version_test.go new file mode 100644 index 000000000000..f3f543304bfd --- /dev/null +++ b/op-service/version_test.go @@ -0,0 +1,75 @@ +package op_service + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +func TestFormatVersion(t *testing.T) { + tests := []struct { + version string + gitCommit string + gitDate string + meta string + expected string + }{ + { + version: "v1.0.0", + gitCommit: "c90a760cfaccefb60b942ffe4ccf4f9692587cec", + gitDate: "1698107786", + meta: "", + expected: "v1.0.0-c90a760c-1698107786", + }, + { + version: "v1.0.0", + gitCommit: "dev", + gitDate: "1698107786", + meta: "", + expected: "v1.0.0-dev-1698107786", + }, + { + version: "v1.0.0", + gitCommit: "", + gitDate: "1698107786", + meta: "", + expected: "v1.0.0-1698107786", + }, + { + version: "v1.0.0", + gitCommit: "dev", + gitDate: "", + meta: "", + expected: "v1.0.0-dev", + }, + { + version: "v1.0.0", + gitCommit: "", + gitDate: "", + meta: "rc.1", + expected: "v1.0.0-rc.1", + }, + { + version: "v1.0.0", + gitCommit: "", + gitDate: "", + meta: "", + expected: "v1.0.0", + }, + { + version: "v1.0.0", + gitCommit: "c90a760cfaccefb60b942ffe4ccf4f9692587cec", + gitDate: "1698107786", + meta: "beta", + expected: "v1.0.0-c90a760c-1698107786-beta", + }, + } + + for _, test := range tests { + test := test + t.Run(test.expected, func(t *testing.T) { + actual := FormatVersion(test.version, test.gitCommit, test.gitDate, test.meta) + require.Equal(t, test.expected, actual) + }) + } +} diff --git a/op-wheel/cmd/main.go b/op-wheel/cmd/main.go index f906ada46eec..29c726cddb21 100644 --- a/op-wheel/cmd/main.go +++ b/op-wheel/cmd/main.go @@ -2,9 +2,9 @@ package main import ( "errors" - "fmt" "os" + opservice "github.com/ethereum-optimism/optimism/op-service" "github.com/urfave/cli/v2" "github.com/ethereum/go-ethereum/log" @@ -21,7 +21,7 @@ var ( func main() { app := cli.NewApp() - app.Version = fmt.Sprintf("%s-%s-%s", Version, GitCommit, GitDate) + app.Version = opservice.FormatVersion(Version, GitCommit, GitDate, "") app.Name = "op-wheel" app.Usage = "Optimism Wheel is a CLI tool for the execution engine" app.Description = "Optimism Wheel is a CLI tool to direct the engine one way or the other with DB cheats and Engine API routines." From 01ec03a81fcdbcde09887c40055d94fc76f739a0 Mon Sep 17 00:00:00 2001 From: clabby Date: Mon, 23 Oct 2023 23:36:36 -0400 Subject: [PATCH 025/279] Cleanup ctb's ffi utils --- packages/contracts-bedrock/.gas-snapshot | 156 +++++++-------- packages/contracts-bedrock/.gitignore | 5 +- packages/contracts-bedrock/package.json | 15 +- .../scripts/go-ffi/README.md | 50 +++++ .../contracts-bedrock/scripts/go-ffi/bin.go | 12 ++ .../differential-testing.go | 4 +- .../trie => scripts/go-ffi}/trie.go | 28 +-- .../{differential-testing => go-ffi}/utils.go | 0 .../test-case-generator/README.md | 34 ---- .../test-case-generator/cmd/fuzz.go | 31 --- .../test-case-generator/go.mod | 26 --- .../test-case-generator/go.sum | 141 -------------- .../contracts-bedrock/test/CommonTest.t.sol | 184 +++++++++--------- 13 files changed, 255 insertions(+), 431 deletions(-) create mode 100644 packages/contracts-bedrock/scripts/go-ffi/README.md create mode 100644 packages/contracts-bedrock/scripts/go-ffi/bin.go rename packages/contracts-bedrock/scripts/{differential-testing => go-ffi}/differential-testing.go (99%) rename packages/contracts-bedrock/{test-case-generator/trie => scripts/go-ffi}/trie.go (92%) rename packages/contracts-bedrock/scripts/{differential-testing => go-ffi}/utils.go (100%) delete mode 100644 packages/contracts-bedrock/test-case-generator/README.md delete mode 100644 packages/contracts-bedrock/test-case-generator/cmd/fuzz.go delete mode 100644 packages/contracts-bedrock/test-case-generator/go.mod delete mode 100644 packages/contracts-bedrock/test-case-generator/go.sum diff --git a/packages/contracts-bedrock/.gas-snapshot b/packages/contracts-bedrock/.gas-snapshot index 363a6a8ab29d..7c2b6a4f7e29 100644 --- a/packages/contracts-bedrock/.gas-snapshot +++ b/packages/contracts-bedrock/.gas-snapshot @@ -307,81 +307,81 @@ LegacyERC20ETH_Test:test_transferFrom_doesNotExist_reverts() (gas: 12957) LegacyERC20ETH_Test:test_transfer_doesNotExist_reverts() (gas: 10755) LegacyMessagePasser_Test:test_passMessageToL1_succeeds() (gas: 34524) LibPosition_Test:test_pos_correctness_succeeds() (gas: 38689) -MIPS_Test:test_add_succeeds() (gas: 122466) -MIPS_Test:test_addiSign_succeeds() (gas: 122457) -MIPS_Test:test_addi_succeeds() (gas: 122654) -MIPS_Test:test_addu_succeeds() (gas: 122508) -MIPS_Test:test_addui_succeeds() (gas: 122716) -MIPS_Test:test_and_succeeds() (gas: 122527) -MIPS_Test:test_andi_succeeds() (gas: 122460) -MIPS_Test:test_beq_succeeds() (gas: 202893) -MIPS_Test:test_bgez_succeeds() (gas: 121753) -MIPS_Test:test_bgtz_succeeds() (gas: 121674) -MIPS_Test:test_blez_succeeds() (gas: 121630) -MIPS_Test:test_bltz_succeeds() (gas: 121773) -MIPS_Test:test_bne_succeeds() (gas: 121839) -MIPS_Test:test_branch_inDelaySlot_fails() (gas: 86048) -MIPS_Test:test_brk_succeeds() (gas: 122143) -MIPS_Test:test_clo_succeeds() (gas: 122195) -MIPS_Test:test_clone_succeeds() (gas: 122096) -MIPS_Test:test_clz_succeeds() (gas: 122666) -MIPS_Test:test_div_succeeds() (gas: 122645) -MIPS_Test:test_divu_succeeds() (gas: 122630) -MIPS_Test:test_exit_succeeds() (gas: 122145) -MIPS_Test:test_fcntl_succeeds() (gas: 204375) -MIPS_Test:test_illegal_instruction_fails() (gas: 91511) -MIPS_Test:test_invalid_root_fails() (gas: 435685) -MIPS_Test:test_jal_nonzeroRegion_succeeds() (gas: 120783) -MIPS_Test:test_jal_succeeds() (gas: 120772) -MIPS_Test:test_jalr_succeeds() (gas: 121891) -MIPS_Test:test_jr_succeeds() (gas: 121585) -MIPS_Test:test_jump_inDelaySlot_fails() (gas: 85416) -MIPS_Test:test_jump_nonzeroRegion_succeeds() (gas: 120527) -MIPS_Test:test_jump_succeeds() (gas: 120457) -MIPS_Test:test_lb_succeeds() (gas: 127698) -MIPS_Test:test_lbu_succeeds() (gas: 127596) -MIPS_Test:test_lh_succeeds() (gas: 127719) -MIPS_Test:test_lhu_succeeds() (gas: 127636) -MIPS_Test:test_ll_succeeds() (gas: 127858) -MIPS_Test:test_lui_succeeds() (gas: 121739) -MIPS_Test:test_lw_succeeds() (gas: 127487) -MIPS_Test:test_lwl_succeeds() (gas: 242138) -MIPS_Test:test_lwr_succeeds() (gas: 242426) -MIPS_Test:test_mfhi_succeeds() (gas: 122100) -MIPS_Test:test_mflo_succeeds() (gas: 122229) -MIPS_Test:test_mmap_succeeds() (gas: 119063) -MIPS_Test:test_movn_succeeds() (gas: 203565) -MIPS_Test:test_movz_succeeds() (gas: 203433) -MIPS_Test:test_mthi_succeeds() (gas: 122144) -MIPS_Test:test_mtlo_succeeds() (gas: 122252) -MIPS_Test:test_mul_succeeds() (gas: 121744) -MIPS_Test:test_mult_succeeds() (gas: 122448) -MIPS_Test:test_multu_succeeds() (gas: 122485) -MIPS_Test:test_nor_succeeds() (gas: 122577) -MIPS_Test:test_or_succeeds() (gas: 122534) -MIPS_Test:test_ori_succeeds() (gas: 122537) -MIPS_Test:test_preimage_read_succeeds() (gas: 234549) -MIPS_Test:test_preimage_write_succeeds() (gas: 127085) -MIPS_Test:test_prestate_exited_succeeds() (gas: 113326) -MIPS_Test:test_sb_succeeds() (gas: 160569) -MIPS_Test:test_sc_succeeds() (gas: 160752) -MIPS_Test:test_sh_succeeds() (gas: 160606) -MIPS_Test:test_sll_succeeds() (gas: 121705) -MIPS_Test:test_sllv_succeeds() (gas: 121934) -MIPS_Test:test_slt_succeeds() (gas: 204760) -MIPS_Test:test_sltu_succeeds() (gas: 122751) -MIPS_Test:test_sra_succeeds() (gas: 121956) -MIPS_Test:test_srav_succeeds() (gas: 122224) -MIPS_Test:test_srl_succeeds() (gas: 121787) -MIPS_Test:test_srlv_succeeds() (gas: 121952) +MIPS_Test:test_add_succeeds() (gas: 122932) +MIPS_Test:test_addiSign_succeeds() (gas: 122923) +MIPS_Test:test_addi_succeeds() (gas: 123120) +MIPS_Test:test_addu_succeeds() (gas: 122974) +MIPS_Test:test_addui_succeeds() (gas: 123182) +MIPS_Test:test_and_succeeds() (gas: 122993) +MIPS_Test:test_andi_succeeds() (gas: 122926) +MIPS_Test:test_beq_succeeds() (gas: 203359) +MIPS_Test:test_bgez_succeeds() (gas: 122219) +MIPS_Test:test_bgtz_succeeds() (gas: 122140) +MIPS_Test:test_blez_succeeds() (gas: 122096) +MIPS_Test:test_bltz_succeeds() (gas: 122239) +MIPS_Test:test_bne_succeeds() (gas: 122305) +MIPS_Test:test_branch_inDelaySlot_fails() (gas: 86514) +MIPS_Test:test_brk_succeeds() (gas: 122609) +MIPS_Test:test_clo_succeeds() (gas: 122661) +MIPS_Test:test_clone_succeeds() (gas: 122562) +MIPS_Test:test_clz_succeeds() (gas: 123132) +MIPS_Test:test_div_succeeds() (gas: 123111) +MIPS_Test:test_divu_succeeds() (gas: 123096) +MIPS_Test:test_exit_succeeds() (gas: 122611) +MIPS_Test:test_fcntl_succeeds() (gas: 204841) +MIPS_Test:test_illegal_instruction_fails() (gas: 91977) +MIPS_Test:test_invalid_root_fails() (gas: 436151) +MIPS_Test:test_jal_nonzeroRegion_succeeds() (gas: 121249) +MIPS_Test:test_jal_succeeds() (gas: 121238) +MIPS_Test:test_jalr_succeeds() (gas: 122357) +MIPS_Test:test_jr_succeeds() (gas: 122051) +MIPS_Test:test_jump_inDelaySlot_fails() (gas: 85882) +MIPS_Test:test_jump_nonzeroRegion_succeeds() (gas: 120993) +MIPS_Test:test_jump_succeeds() (gas: 120923) +MIPS_Test:test_lb_succeeds() (gas: 128164) +MIPS_Test:test_lbu_succeeds() (gas: 128062) +MIPS_Test:test_lh_succeeds() (gas: 128185) +MIPS_Test:test_lhu_succeeds() (gas: 128102) +MIPS_Test:test_ll_succeeds() (gas: 128324) +MIPS_Test:test_lui_succeeds() (gas: 122205) +MIPS_Test:test_lw_succeeds() (gas: 127953) +MIPS_Test:test_lwl_succeeds() (gas: 243070) +MIPS_Test:test_lwr_succeeds() (gas: 243358) +MIPS_Test:test_mfhi_succeeds() (gas: 122566) +MIPS_Test:test_mflo_succeeds() (gas: 122695) +MIPS_Test:test_mmap_succeeds() (gas: 119529) +MIPS_Test:test_movn_succeeds() (gas: 204031) +MIPS_Test:test_movz_succeeds() (gas: 203899) +MIPS_Test:test_mthi_succeeds() (gas: 122610) +MIPS_Test:test_mtlo_succeeds() (gas: 122718) +MIPS_Test:test_mul_succeeds() (gas: 122210) +MIPS_Test:test_mult_succeeds() (gas: 122914) +MIPS_Test:test_multu_succeeds() (gas: 122951) +MIPS_Test:test_nor_succeeds() (gas: 123043) +MIPS_Test:test_or_succeeds() (gas: 123000) +MIPS_Test:test_ori_succeeds() (gas: 123003) +MIPS_Test:test_preimage_read_succeeds() (gas: 235481) +MIPS_Test:test_preimage_write_succeeds() (gas: 127551) +MIPS_Test:test_prestate_exited_succeeds() (gas: 113792) +MIPS_Test:test_sb_succeeds() (gas: 161501) +MIPS_Test:test_sc_succeeds() (gas: 161684) +MIPS_Test:test_sh_succeeds() (gas: 161538) +MIPS_Test:test_sll_succeeds() (gas: 122171) +MIPS_Test:test_sllv_succeeds() (gas: 122400) +MIPS_Test:test_slt_succeeds() (gas: 205226) +MIPS_Test:test_sltu_succeeds() (gas: 123217) +MIPS_Test:test_sra_succeeds() (gas: 122422) +MIPS_Test:test_srav_succeeds() (gas: 122690) +MIPS_Test:test_srl_succeeds() (gas: 122253) +MIPS_Test:test_srlv_succeeds() (gas: 122418) MIPS_Test:test_step_abi_succeeds() (gas: 58467) -MIPS_Test:test_sub_succeeds() (gas: 122561) -MIPS_Test:test_subu_succeeds() (gas: 122558) -MIPS_Test:test_sw_succeeds() (gas: 160581) -MIPS_Test:test_swl_succeeds() (gas: 160642) -MIPS_Test:test_swr_succeeds() (gas: 160717) -MIPS_Test:test_xor_succeeds() (gas: 122562) -MIPS_Test:test_xori_succeeds() (gas: 122614) +MIPS_Test:test_sub_succeeds() (gas: 123027) +MIPS_Test:test_subu_succeeds() (gas: 123024) +MIPS_Test:test_sw_succeeds() (gas: 161513) +MIPS_Test:test_swl_succeeds() (gas: 161574) +MIPS_Test:test_swr_succeeds() (gas: 161649) +MIPS_Test:test_xor_succeeds() (gas: 123028) +MIPS_Test:test_xori_succeeds() (gas: 123080) MerkleTrie_get_Test:test_get_corruptedProof_reverts() (gas: 5733) MerkleTrie_get_Test:test_get_extraProofElements_reverts() (gas: 58889) MerkleTrie_get_Test:test_get_invalidDataRemainder_reverts() (gas: 35845) @@ -447,9 +447,9 @@ OptimismPortal_FinalizeWithdrawal_Test:test_finalizeWithdrawalTransaction_ifOutp OptimismPortal_FinalizeWithdrawal_Test:test_finalizeWithdrawalTransaction_ifOutputTimestampIsNotFinalized_reverts() (gas: 182306) OptimismPortal_FinalizeWithdrawal_Test:test_finalizeWithdrawalTransaction_ifWithdrawalNotProven_reverts() (gas: 41780) OptimismPortal_FinalizeWithdrawal_Test:test_finalizeWithdrawalTransaction_ifWithdrawalProofNotOldEnough_reverts() (gas: 173953) -OptimismPortal_FinalizeWithdrawal_Test:test_finalizeWithdrawalTransaction_onInsufficientGas_reverts() (gas: 180724) +OptimismPortal_FinalizeWithdrawal_Test:test_finalizeWithdrawalTransaction_onInsufficientGas_reverts() (gas: 181188) OptimismPortal_FinalizeWithdrawal_Test:test_finalizeWithdrawalTransaction_onRecentWithdrawal_reverts() (gas: 154740) -OptimismPortal_FinalizeWithdrawal_Test:test_finalizeWithdrawalTransaction_onReentrancy_reverts() (gas: 218770) +OptimismPortal_FinalizeWithdrawal_Test:test_finalizeWithdrawalTransaction_onReentrancy_reverts() (gas: 219235) OptimismPortal_FinalizeWithdrawal_Test:test_finalizeWithdrawalTransaction_onReplay_reverts() (gas: 220983) OptimismPortal_FinalizeWithdrawal_Test:test_finalizeWithdrawalTransaction_paused_reverts() (gas: 38706) OptimismPortal_FinalizeWithdrawal_Test:test_finalizeWithdrawalTransaction_provenWithdrawalHash_succeeds() (gas: 209679) @@ -660,9 +660,9 @@ SequencerFeeVault_Test:test_withdraw_toL1_succeeds() (gas: 171675) SetPrevBaseFee_Test:test_setPrevBaseFee_succeeds() (gas: 11549) StandardBridge_Stateless_Test:test_isCorrectTokenPair_succeeds() (gas: 49936) StandardBridge_Stateless_Test:test_isOptimismMintableERC20_succeeds() (gas: 33072) -Storage_Roundtrip_Test:test_setGetAddress_succeeds(bytes32,address) (runs: 64, μ: 31821, ~: 31821) +Storage_Roundtrip_Test:test_setGetAddress_succeeds(bytes32,address) (runs: 64, μ: 31510, ~: 31821) Storage_Roundtrip_Test:test_setGetBytes32_succeeds(bytes32,bytes32) (runs: 64, μ: 31598, ~: 31598) -Storage_Roundtrip_Test:test_setGetUint_succeeds(bytes32,uint256) (runs: 64, μ: 30731, ~: 31664) +Storage_Roundtrip_Test:test_setGetUint_succeeds(bytes32,uint256) (runs: 64, μ: 30420, ~: 31664) SystemConfig_Initialize_Test:test_initialize_events_succeeds() (gas: 71972) SystemConfig_Initialize_Test:test_initialize_startBlockNoop_reverts() (gas: 81247) SystemConfig_Initialize_Test:test_initialize_startBlockOverride_succeeds() (gas: 65143) diff --git a/packages/contracts-bedrock/.gitignore b/packages/contracts-bedrock/.gitignore index d82f192d9f55..61b186e0d7d9 100644 --- a/packages/contracts-bedrock/.gitignore +++ b/packages/contracts-bedrock/.gitignore @@ -9,11 +9,8 @@ typechain coverage.out .resource-metering.csv -# Tests -test-case-generator/fuzz - # Scripts -scripts/differential-testing/differential-testing +scripts/go-ffi/go-ffi # Environment Variables .envrc diff --git a/packages/contracts-bedrock/package.json b/packages/contracts-bedrock/package.json index 617d3778cd67..4446449ef874 100644 --- a/packages/contracts-bedrock/package.json +++ b/packages/contracts-bedrock/package.json @@ -15,14 +15,13 @@ "bindings:go": "pnpm clean && pnpm build && pnpm bindings:go:no-build", "prebuild": "./scripts/verify-foundry-install.sh", "build": "forge build", - "build:differential": "go build -o ./scripts/differential-testing/differential-testing ./scripts/differential-testing", - "build:fuzz": "(cd test-case-generator && go build ./cmd/fuzz.go)", + "build:go-ffi": "(cd scripts/go-ffi && go build)", "autogen:invariant-docs": "npx tsx scripts/invariant-doc-gen.ts", - "test": "pnpm build:differential && pnpm build:fuzz && forge test", - "coverage": "pnpm build:differential && pnpm build:fuzz && forge coverage", - "coverage:lcov": "pnpm build:differential && pnpm build:fuzz && forge coverage --report lcov", + "test": "pnpm build:go-ffi && forge test", + "coverage": "pnpm build:go-ffi && forge coverage", + "coverage:lcov": "pnpm build:go-ffi && forge coverage --report lcov", "gas-snapshot:no-build": "forge snapshot --no-match-test 'testDiff|testFuzz|invariant|generateArtifact'", - "gas-snapshot": "pnpm build:differential && pnpm build:fuzz && pnpm gas-snapshot:no-build", + "gas-snapshot": "pnpm build:go-ffi && pnpm gas-snapshot:no-build", "storage-snapshot": "./scripts/storage-snapshot.sh", "semver-lock": "forge script scripts/SemverLock.s.sol", "validate-deploy-configs": "./scripts/check-deploy-configs.sh", @@ -30,10 +29,10 @@ "validate-spacers": "pnpm build && pnpm validate-spacers:no-build", "slither": "./scripts/slither.sh", "slither:triage": "TRIAGE_MODE=1 ./scripts/slither.sh", - "clean": "rm -rf ./artifacts ./forge-artifacts ./cache ./tsconfig.tsbuildinfo ./tsconfig.build.tsbuildinfo ./test-case-generator/fuzz ./scripts/differential-testing/differential-testing", + "clean": "rm -rf ./artifacts ./forge-artifacts ./cache ./tsconfig.tsbuildinfo ./tsconfig.build.tsbuildinfo ./scripts/go-ffi/go-ffi", "preinstall": "npx only-allow pnpm", "pre-pr:no-build": "pnpm gas-snapshot:no-build && pnpm storage-snapshot && pnpm semver-lock && pnpm autogen:invariant-docs && pnpm lint && pnpm bindings:go", - "pre-pr": "pnpm clean && pnpm build:differential && pnpm build:fuzz && pnpm build && pnpm pre-pr:no-build", + "pre-pr": "pnpm clean && pnpm build:go-ffi && pnpm build && pnpm pre-pr:no-build", "pre-pr:full": "pnpm test && pnpm slither && pnpm validate-deploy-configs && pnpm validate-spacers && pnpm pre-pr", "lint:ts:check": "eslint . --max-warnings=0", "lint:forge-tests:check": "npx tsx scripts/forge-test-names.ts", diff --git a/packages/contracts-bedrock/scripts/go-ffi/README.md b/packages/contracts-bedrock/scripts/go-ffi/README.md new file mode 100644 index 000000000000..29dac2e83d5f --- /dev/null +++ b/packages/contracts-bedrock/scripts/go-ffi/README.md @@ -0,0 +1,50 @@ +# `ctb-go-ffi` + +A lightweight binary for utilities accessed via `forge`'s `ffi` cheatcode in the `contracts-bedrock` test suite. + +
+├── go-ffi
+│   ├── bin: `go-ffi`'s binary
+│   ├── trie: Utility for generating random merkle trie roots / inclusion proofs
+│   └── diff-testing: Utility for differential testing Solidity implementations against their respective Go implementations.
+
+ +## Usage + +To build, run `pnpm build:go-ffi` from this directory or the `contract-bedrock` package. + +### Available Modes + +#### `diff` + +> **Note** +> Variant required for diff mode. + +| Variant | Description | +| ------------------------------------- | -------------------------------------------------------------------------------------------------------------------- +| `decodeVersionedNonce` | Decodes a versioned nonce and prints the decoded arguments | +| `encodeCrossDomainMessage` | Encodes a cross domain message and prints the encoded message | +| `hashCrossDomainMessage` | Encodes and hashes a cross domain message and prints the digest | +| `hashDepositTransaction` | Encodes and hashes a deposit transaction and prints the digest | +| `encodeDepositTransaction` | RLP encodes a deposit transaction | +| `hashWithdrawal` | Hashes a withdrawal message and prints the digest | +| `hashOutputRootProof` | Hashes an output root proof and prints the digest | +| `getProveWithdrawalTransactionInputs` | Generates the inputs for a `getProveWithdrawalTransaction` call to the `OptimismPortal` given a withdrawal message | +| `cannonMemoryProof` | Computes a merkle proof of Cannon's memory | + +#### `trie` + +> **Note** +> Variant required for `trie` mode. + +| Variant | Description | +| ----------------------------- | ----------------------------------------------------------------------------------------------------------------------------------------- | +| `valid` | Generate a test case with a valid proof of inclusion for the k/v pair in the trie. | +| `extra_proof_elems` | Generate an invalid test case with an extra proof element attached to an otherwise valid proof of inclusion for the passed k/v. | +| `corrupted_proof` | Generate an invalid test case where the proof is malformed. | +| `invalid_data_remainder` | Generate an invalid test case where a random element of the proof has more bytes than the length designates within the RLP list encoding. | +| `invalid_large_internal_hash` | Generate an invalid test case where a long proof element is incorrect for the root. | +| `invalid_internal_node_hash` | Generate an invalid test case where a small proof element is incorrect for the root. | +| `prefixed_valid_key` | Generate a valid test case with a key that has been given a random prefix | +| `empty_key` | Generate a valid test case with a proof of inclusion for an empty key. | +| `partial_proof` | Generate an invalid test case with a partially correct proof | diff --git a/packages/contracts-bedrock/scripts/go-ffi/bin.go b/packages/contracts-bedrock/scripts/go-ffi/bin.go new file mode 100644 index 000000000000..26a62343a2f4 --- /dev/null +++ b/packages/contracts-bedrock/scripts/go-ffi/bin.go @@ -0,0 +1,12 @@ +package main + +import "os" + +func main() { + switch os.Args[1] { + case "diff": + DiffTestUtils() + case "trie": + FuzzTrie() + } +} diff --git a/packages/contracts-bedrock/scripts/differential-testing/differential-testing.go b/packages/contracts-bedrock/scripts/go-ffi/differential-testing.go similarity index 99% rename from packages/contracts-bedrock/scripts/differential-testing/differential-testing.go rename to packages/contracts-bedrock/scripts/go-ffi/differential-testing.go index adf19dc94119..e990f79cddb2 100644 --- a/packages/contracts-bedrock/scripts/differential-testing/differential-testing.go +++ b/packages/contracts-bedrock/scripts/go-ffi/differential-testing.go @@ -74,8 +74,8 @@ var ( } ) -func main() { - args := os.Args[1:] +func DiffTestUtils() { + args := os.Args[2:] // This command requires arguments if len(args) == 0 { diff --git a/packages/contracts-bedrock/test-case-generator/trie/trie.go b/packages/contracts-bedrock/scripts/go-ffi/trie.go similarity index 92% rename from packages/contracts-bedrock/test-case-generator/trie/trie.go rename to packages/contracts-bedrock/scripts/go-ffi/trie.go index 26b1bed0dd39..5811c1ee404e 100644 --- a/packages/contracts-bedrock/test-case-generator/trie/trie.go +++ b/packages/contracts-bedrock/scripts/go-ffi/trie.go @@ -1,15 +1,16 @@ -package trie +package main import ( "crypto/rand" "fmt" "log" "math/big" + "os" "github.com/ethereum/go-ethereum/accounts/abi" "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common/hexutil" - "github.com/ethereum/go-ethereum/ethdb/memorydb" + "github.com/ethereum/go-ethereum/core/rawdb" "github.com/ethereum/go-ethereum/rlp" "github.com/ethereum/go-ethereum/trie" ) @@ -39,7 +40,8 @@ const ( ) // Generate an abi-encoded `trieTestCase` of a specified variant -func FuzzTrie(variant string) { +func FuzzTrie() { + variant := os.Args[2] if len(variant) < 1 { log.Fatal("Must pass a variant to the trie fuzzer!") } @@ -132,8 +134,8 @@ func FuzzTrie(variant string) { // Generate a random test case for Bedrock's MerkleTrie verifier. func genTrieTestCase(selectEmptyKey bool) trieTestCase { // Create an empty merkle trie - memdb := memorydb.New() - randTrie := trie.NewEmpty(trie.NewDatabase(memdb)) + memdb := rawdb.NewMemoryDatabase() + randTrie := trie.NewEmpty(trie.NewDatabase(memdb, nil)) // Get a random number of elements to put into the trie randN := randRange(2, 1024) @@ -161,7 +163,7 @@ func genTrieTestCase(selectEmptyKey bool) trieTestCase { } // Insert the random k/v pair into the trie - if err := randTrie.TryUpdate(randKey, randValue); err != nil { + if err := randTrie.Update(randKey, randValue); err != nil { log.Fatal("Error adding key-value pair to trie") } @@ -174,7 +176,7 @@ func genTrieTestCase(selectEmptyKey bool) trieTestCase { // Generate proof for `key`'s inclusion in our trie var proof proofList - if err := randTrie.Prove(key, 0, &proof); err != nil { + if err := randTrie.Prove(key, &proof); err != nil { log.Fatal("Error creating proof for randomly selected key's inclusion in generated trie") } @@ -233,15 +235,3 @@ func randRange(min int64, max int64) int64 { return (new(big.Int).Add(r, new(big.Int).SetInt64(min))).Int64() } - -// Custom type to write the generated proof to -type proofList [][]byte - -func (n *proofList) Put(key []byte, value []byte) error { - *n = append(*n, value) - return nil -} - -func (n *proofList) Delete(key []byte) error { - panic("not supported") -} diff --git a/packages/contracts-bedrock/scripts/differential-testing/utils.go b/packages/contracts-bedrock/scripts/go-ffi/utils.go similarity index 100% rename from packages/contracts-bedrock/scripts/differential-testing/utils.go rename to packages/contracts-bedrock/scripts/go-ffi/utils.go diff --git a/packages/contracts-bedrock/test-case-generator/README.md b/packages/contracts-bedrock/test-case-generator/README.md deleted file mode 100644 index 1cd4af47c09c..000000000000 --- a/packages/contracts-bedrock/test-case-generator/README.md +++ /dev/null @@ -1,34 +0,0 @@ -# `ctb-test-case-generator` - -A lightweight input fuzzing utility used for testing various Bedrock contracts. - -
-├── test-case-generator
-│   ├── cmd: `ctb-test-case-genertor`'s binary
-│   └── trie: Utility for generating random merkle trie roots / inclusion proofs
-
- -## Usage - -To build, run `pnpm build:fuzz` from this directory or the `contract-bedrock` package. - -To generate an abi-encoded fuzz case, pass in a mode via the `-m` flag as well as an optional variant via the `-v` flag. - -### Available Modes - -#### `trie` - -> **Note** -> Variant required for `trie` mode. - -| Variant | Description | -| ----------------------------- | ----------------------------------------------------------------------------------------------------------------------------------------- | -| `valid` | Generate a test case with a valid proof of inclusion for the k/v pair in the trie. | -| `extra_proof_elems` | Generate an invalid test case with an extra proof element attached to an otherwise valid proof of inclusion for the passed k/v. | -| `corrupted_proof` | Generate an invalid test case where the proof is malformed. | -| `invalid_data_remainder` | Generate an invalid test case where a random element of the proof has more bytes than the length designates within the RLP list encoding. | -| `invalid_large_internal_hash` | Generate an invalid test case where a long proof element is incorrect for the root. | -| `invalid_internal_node_hash` | Generate an invalid test case where a small proof element is incorrect for the root. | -| `prefixed_valid_key` | Generate a valid test case with a key that has been given a random prefix | -| `empty_key` | Generate a valid test case with a proof of inclusion for an empty key. | -| `partial_proof` | Generate an invalid test case with a partially correct proof | diff --git a/packages/contracts-bedrock/test-case-generator/cmd/fuzz.go b/packages/contracts-bedrock/test-case-generator/cmd/fuzz.go deleted file mode 100644 index b1516ce77d1e..000000000000 --- a/packages/contracts-bedrock/test-case-generator/cmd/fuzz.go +++ /dev/null @@ -1,31 +0,0 @@ -package main - -import ( - "flag" - "log" - - t "github.com/ethereum-optimism/optimism/packages/contracts-bedrock/ctb-test-case-generator/trie" -) - -// Mode enum -const ( - // Enables the `trie` fuzzer - trie string = "trie" -) - -func main() { - mode := flag.String("m", "", "Fuzzer mode") - variant := flag.String("v", "", "Mode variant") - flag.Parse() - - if len(*mode) < 1 { - log.Fatal("Must pass a mode for the fuzzer!") - } - - switch *mode { - case trie: - t.FuzzTrie(*variant) - default: - log.Fatal("Invalid mode!") - } -} diff --git a/packages/contracts-bedrock/test-case-generator/go.mod b/packages/contracts-bedrock/test-case-generator/go.mod deleted file mode 100644 index 061d0da0637b..000000000000 --- a/packages/contracts-bedrock/test-case-generator/go.mod +++ /dev/null @@ -1,26 +0,0 @@ -module github.com/ethereum-optimism/optimism/packages/contracts-bedrock/ctb-test-case-generator - -go 1.20 - -require github.com/ethereum/go-ethereum v1.10.26 - -require ( - github.com/StackExchange/wmi v0.0.0-20180116203802-5d049714c4a6 // indirect - github.com/VictoriaMetrics/fastcache v1.6.0 // indirect - github.com/btcsuite/btcd/btcec/v2 v2.2.0 // indirect - github.com/cespare/xxhash/v2 v2.1.1 // indirect - github.com/decred/dcrd/dcrec/secp256k1/v4 v4.0.1 // indirect - github.com/go-ole/go-ole v1.2.1 // indirect - github.com/go-stack/stack v1.8.0 // indirect - github.com/golang/snappy v0.0.4 // indirect - github.com/mattn/go-runewidth v0.0.9 // indirect - github.com/olekukonko/tablewriter v0.0.5 // indirect - github.com/pkg/errors v0.9.1 // indirect - github.com/prometheus/tsdb v0.7.1 // indirect - github.com/shirou/gopsutil v3.21.4-0.20210419000835-c7a38de76ee5+incompatible // indirect - github.com/syndtr/goleveldb v1.0.1-0.20210819022825-2ae1ddf74ef7 // indirect - github.com/tklauser/go-sysconf v0.3.5 // indirect - github.com/tklauser/numcpus v0.2.2 // indirect - golang.org/x/crypto v0.1.0 // indirect - golang.org/x/sys v0.1.0 // indirect -) diff --git a/packages/contracts-bedrock/test-case-generator/go.sum b/packages/contracts-bedrock/test-case-generator/go.sum deleted file mode 100644 index 3f846df0ffed..000000000000 --- a/packages/contracts-bedrock/test-case-generator/go.sum +++ /dev/null @@ -1,141 +0,0 @@ -github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAElWljhcU= -github.com/StackExchange/wmi v0.0.0-20180116203802-5d049714c4a6 h1:fLjPD/aNc3UIOA6tDi6QXUemppXK3P9BI7mr2hd6gx8= -github.com/StackExchange/wmi v0.0.0-20180116203802-5d049714c4a6/go.mod h1:3eOhrUMpNV+6aFIbp5/iudMxNCF27Vw2OZgy4xEx0Fg= -github.com/VictoriaMetrics/fastcache v1.6.0 h1:C/3Oi3EiBCqufydp1neRZkqcwmEiuRT9c3fqvvgKm5o= -github.com/VictoriaMetrics/fastcache v1.6.0/go.mod h1:0qHz5QP0GMX4pfmMA/zt5RgfNuXJrTP0zS7DqpHGGTw= -github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= -github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= -github.com/allegro/bigcache v1.2.1-0.20190218064605-e24eb225f156 h1:eMwmnE/GDgah4HI848JfFxHt+iPb26b4zyfspmqY0/8= -github.com/allegro/bigcache v1.2.1-0.20190218064605-e24eb225f156/go.mod h1:Cb/ax3seSYIx7SuZdm2G2xzfwmv3TPSk2ucNfQESPXM= -github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= -github.com/btcsuite/btcd/btcec/v2 v2.2.0 h1:fzn1qaOt32TuLjFlkzYSsBC35Q3KUjT1SwPxiMSCF5k= -github.com/btcsuite/btcd/btcec/v2 v2.2.0/go.mod h1:U7MHm051Al6XmscBQ0BoNydpOTsFAn707034b5nY8zU= -github.com/btcsuite/btcd/chaincfg/chainhash v1.0.1 h1:q0rUy8C/TYNBQS1+CGKw68tLOFYSNEs0TFnxxnS9+4U= -github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghfAqPWnc= -github.com/cespare/xxhash/v2 v2.1.1 h1:6MnRN8NT7+YBpUIWxHtefFZOKTAPgGjpQSxqLNn0+qY= -github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= -github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= -github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= -github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= -github.com/decred/dcrd/crypto/blake256 v1.0.0 h1:/8DMNYp9SGi5f0w7uCm6d6M4OU2rGFK09Y2A4Xv7EE0= -github.com/decred/dcrd/crypto/blake256 v1.0.0/go.mod h1:sQl2p6Y26YV+ZOcSTP6thNdn47hh8kt6rqSlvmrXFAc= -github.com/decred/dcrd/dcrec/secp256k1/v4 v4.0.1 h1:YLtO71vCjJRCBcrPMtQ9nqBsqpA1m5sE92cU+pd5Mcc= -github.com/decred/dcrd/dcrec/secp256k1/v4 v4.0.1/go.mod h1:hyedUtir6IdtD/7lIxGeCxkaw7y45JueMRL4DIyJDKs= -github.com/dgryski/go-sip13 v0.0.0-20181026042036-e10d5fee7954/go.mod h1:vAd38F8PWV+bWy6jNmig1y/TA+kYO4g3RSRF0IAv0no= -github.com/ethereum/go-ethereum v1.10.26 h1:i/7d9RBBwiXCEuyduBQzJw/mKmnvzsN14jqBmytw72s= -github.com/ethereum/go-ethereum v1.10.26/go.mod h1:EYFyF19u3ezGLD4RqOkLq+ZCXzYbLoNDdZlMt7kyKFg= -github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= -github.com/fsnotify/fsnotify v1.4.9 h1:hsms1Qyu0jgnwNXIxa+/V/PDsU6CfLf6CNO8H7IWoS4= -github.com/fsnotify/fsnotify v1.4.9/go.mod h1:znqG4EE+3YCdAaPaxE2ZRY/06pZUdp0tY4IgpuI1SZQ= -github.com/go-kit/kit v0.8.0 h1:Wz+5lgoB0kkuqLEc6NVmwRknTKP6dTGbSqvhZtBI/j0= -github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= -github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= -github.com/go-logfmt/logfmt v0.4.0 h1:MP4Eh7ZCb31lleYCFuwm0oe4/YGak+5l1vA2NOE80nA= -github.com/go-ole/go-ole v1.2.1 h1:2lOsA72HgjxAuMlKpFiCbHTvu44PIVkZ5hqm3RSdI/E= -github.com/go-ole/go-ole v1.2.1/go.mod h1:7FAglXiTm7HKlQRDeOQ6ZNUHidzCWXuZWq/1dTyBNF8= -github.com/go-stack/stack v1.8.0 h1:5SgMzNM5HxrEjV0ww2lTmX6E2Izsfxas4+YHWRs3Lsk= -github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= -github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= -github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8= -github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA= -github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs= -github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w= -github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0= -github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= -github.com/golang/snappy v0.0.3/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= -github.com/golang/snappy v0.0.4 h1:yAGX7huGHXlcLOEtBnF4w7FQwA26wojNCwOYAEhLjQM= -github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= -github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= -github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= -github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= -github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515 h1:T+h1c/A9Gawja4Y9mFVWj2vyii2bbUNDw3kt9VxK2EY= -github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= -github.com/mattn/go-runewidth v0.0.9 h1:Lm995f3rfxdpd6TSmuVCHVb/QhupuXlYr8sCI/QdE+0= -github.com/mattn/go-runewidth v0.0.9/go.mod h1:H031xJmbD/WCDINGzjvQ9THkh0rPKHF+m2gUSrubnMI= -github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= -github.com/nxadm/tail v1.4.4 h1:DQuhQpB1tVlglWS2hLQ5OV6B5r8aGxSrPc5Qo6uTN78= -github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A= -github.com/oklog/ulid v1.3.1/go.mod h1:CirwcVhetQ6Lv90oh/F+FBtV6XMibvdAFo93nm5qn4U= -github.com/olekukonko/tablewriter v0.0.5 h1:P2Ga83D34wi1o9J6Wh1mRuqd4mF/x/lgBS7N7AbDhec= -github.com/olekukonko/tablewriter v0.0.5/go.mod h1:hPp6KlRPjbx+hW8ykQs1w3UBbZlj6HuIJcUGPhkA7kY= -github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= -github.com/onsi/ginkgo v1.12.1/go.mod h1:zj2OWP4+oCPe1qIXoGWkgMRwljMUYCdkwsT2108oapk= -github.com/onsi/ginkgo v1.14.0 h1:2mOpI4JVVPBN+WQRa0WKH2eXR+Ey+uK4n7Zj0aYpIQA= -github.com/onsi/ginkgo v1.14.0/go.mod h1:iSB4RoI2tjJc9BBv4NKIKWKya62Rps+oPG/Lv9klQyY= -github.com/onsi/gomega v1.7.1/go.mod h1:XdKZgCCFLUoM/7CFJVPcG8C1xQ1AJ0vpAezJrB7JYyY= -github.com/onsi/gomega v1.10.1 h1:o0+MgICZLuZ7xjH7Vx6zS/zcu93/BEp1VwkIW1mEXCE= -github.com/onsi/gomega v1.10.1/go.mod h1:iN09h71vgCQne3DLsj+A5owkum+a2tYe+TOCB1ybHNo= -github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= -github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= -github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= -github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= -github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= -github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= -github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= -github.com/prometheus/common v0.0.0-20181113130724-41aa239b4cce/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro= -github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= -github.com/prometheus/tsdb v0.7.1 h1:YZcsG11NqnK4czYLrWd9mpEuAJIHVQLwdrleYfszMAA= -github.com/prometheus/tsdb v0.7.1/go.mod h1:qhTCs0VvXwvX/y3TZrWD7rabWM+ijKTux40TwIPHuXU= -github.com/shirou/gopsutil v3.21.4-0.20210419000835-c7a38de76ee5+incompatible h1:Bn1aCHHRnjv4Bl16T8rcaFjYSrGrIZvpiGO6P3Q4GpU= -github.com/shirou/gopsutil v3.21.4-0.20210419000835-c7a38de76ee5+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= -github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= -github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= -github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= -github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= -github.com/stretchr/testify v1.7.2 h1:4jaiDzPyXQvSd7D0EjG45355tLlV3VOECpq10pLC+8s= -github.com/syndtr/goleveldb v1.0.1-0.20210819022825-2ae1ddf74ef7 h1:epCh84lMvA70Z7CTTCmYQn2CKbY8j86K7/FAIr141uY= -github.com/syndtr/goleveldb v1.0.1-0.20210819022825-2ae1ddf74ef7/go.mod h1:q4W45IWZaF22tdD+VEXcAWRA037jwmWEB5VWYORlTpc= -github.com/tklauser/go-sysconf v0.3.5 h1:uu3Xl4nkLzQfXNsWn15rPc/HQCJKObbt1dKJeWp3vU4= -github.com/tklauser/go-sysconf v0.3.5/go.mod h1:MkWzOF4RMCshBAMXuhXJs64Rte09mITnppBXY/rYEFI= -github.com/tklauser/numcpus v0.2.2 h1:oyhllyrScuYI6g+h/zUvNXNp1wy7x8qQy3t/piefldA= -github.com/tklauser/numcpus v0.2.2/go.mod h1:x3qojaO3uyYt0i56EW/VUYs7uBvdl2fkfZFu0T9wgjM= -golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= -golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= -golang.org/x/crypto v0.1.0 h1:MDRAIl0xIo9Io2xV565hzXHw3zVseKrJKodhohM5CjU= -golang.org/x/crypto v0.1.0/go.mod h1:RecgLatLF4+eUMCP1PoPZQb+cVrJcOPbHkTkbkB9sbw= -golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= -golang.org/x/net v0.0.0-20200520004742-59133d7f0dd7/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= -golang.org/x/net v0.0.0-20200813134508-3edf25e44fcc/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= -golang.org/x/net v0.1.0 h1:hZ/3BUoy5aId7sCpA/Tc5lt8DkFgdVS2onTpJsZ/fl0= -golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20181107165924-66b7b1311ac8/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190904154756-749cb33beabd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20191005200804-aed5e4c7ecf9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20191120155948-bd437916bb0e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200519105757-fe76b779f299/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200814200057-3d37ad5750ed/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210316164454-77fc1eacc6aa/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210324051608-47abb6519492/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.1.0 h1:kunALQeHf1/185U1i0GOB/fy1IPRDDpuoOOqRReG57U= -golang.org/x/sys v0.1.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= -golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= -golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.4.0 h1:BrVqGRd7+k1DiOgtnFvAkoQEWQvBc25ouMJM6429SFg= -golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -golang.org/x/xerrors v0.0.0-20220517211312-f3a8303e98df h1:5Pf6pFKu98ODmgnpvkJ3kFUOQGGLIzLIkbzUHp47618= -google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= -google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= -google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= -google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miEFZTKqfCUM6K7xSMQL9OKL/b6hQv+e19PK+JZNE= -google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo= -google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= -gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= -gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys= -gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 h1:uRGJdciOHaEIrze2W8Q3AKkepLTh2hOroT7a+7czfdQ= -gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw= -gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY= -gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= diff --git a/packages/contracts-bedrock/test/CommonTest.t.sol b/packages/contracts-bedrock/test/CommonTest.t.sol index 27aa1f35ee2b..56c9ded975e2 100644 --- a/packages/contracts-bedrock/test/CommonTest.t.sol +++ b/packages/contracts-bedrock/test/CommonTest.t.sol @@ -502,15 +502,16 @@ contract FFIInterface is Test { external returns (bytes32, bytes32, bytes32, bytes32, bytes[] memory) { - string[] memory cmds = new string[](8); - cmds[0] = "scripts/differential-testing/differential-testing"; - cmds[1] = "getProveWithdrawalTransactionInputs"; - cmds[2] = vm.toString(_tx.nonce); - cmds[3] = vm.toString(_tx.sender); - cmds[4] = vm.toString(_tx.target); - cmds[5] = vm.toString(_tx.value); - cmds[6] = vm.toString(_tx.gasLimit); - cmds[7] = vm.toString(_tx.data); + string[] memory cmds = new string[](9); + cmds[0] = "scripts/go-ffi/go-ffi"; + cmds[1] = "diff"; + cmds[2] = "getProveWithdrawalTransactionInputs"; + cmds[3] = vm.toString(_tx.nonce); + cmds[4] = vm.toString(_tx.sender); + cmds[5] = vm.toString(_tx.target); + cmds[6] = vm.toString(_tx.value); + cmds[7] = vm.toString(_tx.gasLimit); + cmds[8] = vm.toString(_tx.data); bytes memory result = vm.ffi(cmds); ( @@ -535,15 +536,16 @@ contract FFIInterface is Test { external returns (bytes32) { - string[] memory cmds = new string[](8); - cmds[0] = "scripts/differential-testing/differential-testing"; - cmds[1] = "hashCrossDomainMessage"; - cmds[2] = vm.toString(_nonce); - cmds[3] = vm.toString(_sender); - cmds[4] = vm.toString(_target); - cmds[5] = vm.toString(_value); - cmds[6] = vm.toString(_gasLimit); - cmds[7] = vm.toString(_data); + string[] memory cmds = new string[](9); + cmds[0] = "scripts/go-ffi/go-ffi"; + cmds[1] = "diff"; + cmds[2] = "hashCrossDomainMessage"; + cmds[3] = vm.toString(_nonce); + cmds[4] = vm.toString(_sender); + cmds[5] = vm.toString(_target); + cmds[6] = vm.toString(_value); + cmds[7] = vm.toString(_gasLimit); + cmds[8] = vm.toString(_data); bytes memory result = vm.ffi(cmds); return abi.decode(result, (bytes32)); @@ -560,15 +562,16 @@ contract FFIInterface is Test { external returns (bytes32) { - string[] memory cmds = new string[](8); - cmds[0] = "scripts/differential-testing/differential-testing"; - cmds[1] = "hashWithdrawal"; - cmds[2] = vm.toString(_nonce); - cmds[3] = vm.toString(_sender); - cmds[4] = vm.toString(_target); - cmds[5] = vm.toString(_value); - cmds[6] = vm.toString(_gasLimit); - cmds[7] = vm.toString(_data); + string[] memory cmds = new string[](9); + cmds[0] = "scripts/go-ffi/go-ffi"; + cmds[1] = "diff"; + cmds[2] = "hashWithdrawal"; + cmds[3] = vm.toString(_nonce); + cmds[4] = vm.toString(_sender); + cmds[5] = vm.toString(_target); + cmds[6] = vm.toString(_value); + cmds[7] = vm.toString(_gasLimit); + cmds[8] = vm.toString(_data); bytes memory result = vm.ffi(cmds); return abi.decode(result, (bytes32)); @@ -583,13 +586,14 @@ contract FFIInterface is Test { external returns (bytes32) { - string[] memory cmds = new string[](6); - cmds[0] = "scripts/differential-testing/differential-testing"; - cmds[1] = "hashOutputRootProof"; - cmds[2] = Strings.toHexString(uint256(_version)); - cmds[3] = Strings.toHexString(uint256(_stateRoot)); - cmds[4] = Strings.toHexString(uint256(_messagePasserStorageRoot)); - cmds[5] = Strings.toHexString(uint256(_latestBlockhash)); + string[] memory cmds = new string[](7); + cmds[0] = "scripts/go-ffi/go-ffi"; + cmds[1] = "diff"; + cmds[2] = "hashOutputRootProof"; + cmds[3] = Strings.toHexString(uint256(_version)); + cmds[4] = Strings.toHexString(uint256(_stateRoot)); + cmds[5] = Strings.toHexString(uint256(_messagePasserStorageRoot)); + cmds[6] = Strings.toHexString(uint256(_latestBlockhash)); bytes memory result = vm.ffi(cmds); return abi.decode(result, (bytes32)); @@ -607,35 +611,37 @@ contract FFIInterface is Test { external returns (bytes32) { - string[] memory cmds = new string[](10); - cmds[0] = "scripts/differential-testing/differential-testing"; - cmds[1] = "hashDepositTransaction"; - cmds[2] = "0x0000000000000000000000000000000000000000000000000000000000000000"; - cmds[3] = vm.toString(_logIndex); - cmds[4] = vm.toString(_from); - cmds[5] = vm.toString(_to); - cmds[6] = vm.toString(_mint); - cmds[7] = vm.toString(_value); - cmds[8] = vm.toString(_gas); - cmds[9] = vm.toString(_data); + string[] memory cmds = new string[](11); + cmds[0] = "scripts/go-ffi/go-ffi"; + cmds[1] = "diff"; + cmds[2] = "hashDepositTransaction"; + cmds[3] = "0x0000000000000000000000000000000000000000000000000000000000000000"; + cmds[4] = vm.toString(_logIndex); + cmds[5] = vm.toString(_from); + cmds[6] = vm.toString(_to); + cmds[7] = vm.toString(_mint); + cmds[8] = vm.toString(_value); + cmds[9] = vm.toString(_gas); + cmds[10] = vm.toString(_data); bytes memory result = vm.ffi(cmds); return abi.decode(result, (bytes32)); } function encodeDepositTransaction(Types.UserDepositTransaction calldata txn) external returns (bytes memory) { - string[] memory cmds = new string[](11); - cmds[0] = "scripts/differential-testing/differential-testing"; - cmds[1] = "encodeDepositTransaction"; - cmds[2] = vm.toString(txn.from); - cmds[3] = vm.toString(txn.to); - cmds[4] = vm.toString(txn.value); - cmds[5] = vm.toString(txn.mint); - cmds[6] = vm.toString(txn.gasLimit); - cmds[7] = vm.toString(txn.isCreation); - cmds[8] = vm.toString(txn.data); - cmds[9] = vm.toString(txn.l1BlockHash); - cmds[10] = vm.toString(txn.logIndex); + string[] memory cmds = new string[](12); + cmds[0] = "scripts/go-ffi/go-ffi"; + cmds[1] = "diff"; + cmds[2] = "encodeDepositTransaction"; + cmds[3] = vm.toString(txn.from); + cmds[4] = vm.toString(txn.to); + cmds[5] = vm.toString(txn.value); + cmds[6] = vm.toString(txn.mint); + cmds[7] = vm.toString(txn.gasLimit); + cmds[8] = vm.toString(txn.isCreation); + cmds[9] = vm.toString(txn.data); + cmds[10] = vm.toString(txn.l1BlockHash); + cmds[11] = vm.toString(txn.logIndex); bytes memory result = vm.ffi(cmds); return abi.decode(result, (bytes)); @@ -652,25 +658,27 @@ contract FFIInterface is Test { external returns (bytes memory) { - string[] memory cmds = new string[](8); - cmds[0] = "scripts/differential-testing/differential-testing"; - cmds[1] = "encodeCrossDomainMessage"; - cmds[2] = vm.toString(_nonce); - cmds[3] = vm.toString(_sender); - cmds[4] = vm.toString(_target); - cmds[5] = vm.toString(_value); - cmds[6] = vm.toString(_gasLimit); - cmds[7] = vm.toString(_data); + string[] memory cmds = new string[](9); + cmds[0] = "scripts/go-ffi/go-ffi"; + cmds[1] = "diff"; + cmds[2] = "encodeCrossDomainMessage"; + cmds[3] = vm.toString(_nonce); + cmds[4] = vm.toString(_sender); + cmds[5] = vm.toString(_target); + cmds[6] = vm.toString(_value); + cmds[7] = vm.toString(_gasLimit); + cmds[8] = vm.toString(_data); bytes memory result = vm.ffi(cmds); return abi.decode(result, (bytes)); } function decodeVersionedNonce(uint256 nonce) external returns (uint256, uint256) { - string[] memory cmds = new string[](3); - cmds[0] = "scripts/differential-testing/differential-testing"; - cmds[1] = "decodeVersionedNonce"; - cmds[2] = vm.toString(nonce); + string[] memory cmds = new string[](4); + cmds[0] = "scripts/go-ffi/go-ffi"; + cmds[1] = "diff"; + cmds[2] = "decodeVersionedNonce"; + cmds[3] = vm.toString(nonce); bytes memory result = vm.ffi(cmds); return abi.decode(result, (uint256, uint256)); @@ -680,22 +688,21 @@ contract FFIInterface is Test { external returns (bytes32, bytes memory, bytes memory, bytes[] memory) { - string[] memory cmds = new string[](5); - cmds[0] = "./test-case-generator/fuzz"; - cmds[1] = "-m"; - cmds[2] = "trie"; - cmds[3] = "-v"; - cmds[4] = variant; + string[] memory cmds = new string[](6); + cmds[0] = "./scripts/go-ffi/go-ffi"; + cmds[1] = "trie"; + cmds[2] = variant; return abi.decode(vm.ffi(cmds), (bytes32, bytes, bytes, bytes[])); } function getCannonMemoryProof(uint32 pc, uint32 insn) external returns (bytes32, bytes memory) { - string[] memory cmds = new string[](4); - cmds[0] = "scripts/differential-testing/differential-testing"; - cmds[1] = "cannonMemoryProof"; - cmds[2] = vm.toString(pc); - cmds[3] = vm.toString(insn); + string[] memory cmds = new string[](5); + cmds[0] = "scripts/go-ffi/go-ffi"; + cmds[1] = "diff"; + cmds[2] = "cannonMemoryProof"; + cmds[3] = vm.toString(pc); + cmds[4] = vm.toString(insn); bytes memory result = vm.ffi(cmds); (bytes32 memRoot, bytes memory proof) = abi.decode(result, (bytes32, bytes)); return (memRoot, proof); @@ -710,13 +717,14 @@ contract FFIInterface is Test { external returns (bytes32, bytes memory) { - string[] memory cmds = new string[](6); - cmds[0] = "scripts/differential-testing/differential-testing"; - cmds[1] = "cannonMemoryProof"; - cmds[2] = vm.toString(pc); - cmds[3] = vm.toString(insn); - cmds[4] = vm.toString(memAddr); - cmds[5] = vm.toString(memVal); + string[] memory cmds = new string[](7); + cmds[0] = "scripts/go-ffi/go-ffi"; + cmds[1] = "diff"; + cmds[2] = "cannonMemoryProof"; + cmds[3] = vm.toString(pc); + cmds[4] = vm.toString(insn); + cmds[5] = vm.toString(memAddr); + cmds[6] = vm.toString(memVal); bytes memory result = vm.ffi(cmds); (bytes32 memRoot, bytes memory proof) = abi.decode(result, (bytes32, bytes)); return (memRoot, proof); From e07b48c905a8b8ae1c34fcee7aa6e098e61c42be Mon Sep 17 00:00:00 2001 From: clabby Date: Mon, 23 Oct 2023 23:47:25 -0400 Subject: [PATCH 026/279] lint --- .../contracts-bedrock/scripts/go-ffi/trie.go | 42 ++++++++++--------- 1 file changed, 22 insertions(+), 20 deletions(-) diff --git a/packages/contracts-bedrock/scripts/go-ffi/trie.go b/packages/contracts-bedrock/scripts/go-ffi/trie.go index 5811c1ee404e..4666816f6d7a 100644 --- a/packages/contracts-bedrock/scripts/go-ffi/trie.go +++ b/packages/contracts-bedrock/scripts/go-ffi/trie.go @@ -50,12 +50,10 @@ func FuzzTrie() { switch variant { case valid: testCase = genTrieTestCase(false) - break case extraProofElems: testCase = genTrieTestCase(false) // Duplicate the last element of the proof testCase.Proof = append(testCase.Proof, [][]byte{testCase.Proof[len(testCase.Proof)-1]}...) - break case corruptedProof: testCase = genTrieTestCase(false) @@ -63,17 +61,17 @@ func FuzzTrie() { idx := randRange(0, int64(len(testCase.Proof))) encoded, _ := rlp.EncodeToBytes(testCase.Proof[idx]) testCase.Proof[idx] = encoded - break case invalidDataRemainder: testCase = genTrieTestCase(false) // Alter true length of random proof element by appending random bytes // Do not update the encoded length idx := randRange(0, int64(len(testCase.Proof))) - bytes := make([]byte, randRange(1, 512)) - rand.Read(bytes) - testCase.Proof[idx] = append(testCase.Proof[idx], bytes...) - break + b := make([]byte, randRange(1, 512)) + if _, err := rand.Read(b); err != nil { + log.Fatal("Error generating random bytes") + } + testCase.Proof[idx] = append(testCase.Proof[idx], b...) case invalidLargeInternalHash: testCase = genTrieTestCase(false) @@ -82,7 +80,9 @@ func FuzzTrie() { // bytes to overwrite. idx := randRange(1, int64(len(testCase.Proof))) b := make([]byte, 4) - rand.Read(b) + if _, err := rand.Read(b); err != nil { + log.Fatal("Error generating random bytes") + } testCase.Proof[idx] = append( testCase.Proof[idx][:20], append( @@ -90,27 +90,26 @@ func FuzzTrie() { testCase.Proof[idx][24:]..., )..., ) - - break case invalidInternalNodeHash: testCase = genTrieTestCase(false) // Assign the last proof element to an encoded list containing a // random 29 byte value b := make([]byte, 29) - rand.Read(b) + if _, err := rand.Read(b); err != nil { + log.Fatal("Error generating random bytes") + } e, _ := rlp.EncodeToBytes(b) testCase.Proof[len(testCase.Proof)-1] = append([]byte{0xc0 + 30}, e...) - break case prefixedValidKey: testCase = genTrieTestCase(false) - bytes := make([]byte, randRange(1, 16)) - rand.Read(bytes) - testCase.Key = append(bytes, testCase.Key...) - break + b := make([]byte, randRange(1, 16)) + if _, err := rand.Read(b); err != nil { + log.Fatal("Error generating random bytes") + } + testCase.Key = append(b, testCase.Key...) case emptyKey: testCase = genTrieTestCase(true) - break case partialProof: testCase = genTrieTestCase(false) @@ -122,7 +121,6 @@ func FuzzTrie() { } testCase.Proof = newProof - break default: log.Fatal("Invalid variant passed to trie fuzzer!") } @@ -154,8 +152,12 @@ func genTrieTestCase(selectEmptyKey bool) trieTestCase { // Add `randN` elements to the trie for i := int64(0); i < randN; i++ { // Randomize the contents of `randKey` and `randValue` - rand.Read(randKey) - rand.Read(randValue) + if _, err := rand.Read(randKey); err != nil { + log.Fatal("Error generating random bytes") + } + if _, err := rand.Read(randValue); err != nil { + log.Fatal("Error generating random bytes") + } // Clear the selected key if `selectEmptyKey` is true if i == randSelect && selectEmptyKey { From 96b76c381508ac771e15aa99e94d895a1b0b06f7 Mon Sep 17 00:00:00 2001 From: "Eugene Y. Q. Shen" Date: Tue, 24 Oct 2023 04:27:22 -0400 Subject: [PATCH 027/279] lint --- specs/deposits.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/specs/deposits.md b/specs/deposits.md index eb2602ee3a8e..1535fa445dc7 100644 --- a/specs/deposits.md +++ b/specs/deposits.md @@ -308,7 +308,7 @@ A reference implementation of the L1 Attributes predeploy contract can be found [L1Block.sol]: ../packages/contracts-bedrock/src/L2/L1Block.sol -After running `pnpm build` in the `packages/contracts-bedrock` directory, the bytecode to add to +After running `pnpm build` in the `packages/contracts-bedrock` directory, the bytecode to add to the genesis file will be located in the `deployedBytecode` field of the build artifacts file at `/packages/contracts-bedrock/forge-artifacts/L1Block.sol/L1Block.json`. From 81264e44eee5c0d21c9e7e86a7881b00bf3a7a1b Mon Sep 17 00:00:00 2001 From: Maurelian Date: Tue, 24 Oct 2023 10:08:18 -0400 Subject: [PATCH 028/279] docs(ctb): Document the correct approach to defining immutable variables --- packages/contracts-bedrock/STYLE_GUIDE.md | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/packages/contracts-bedrock/STYLE_GUIDE.md b/packages/contracts-bedrock/STYLE_GUIDE.md index eefcecd54c76..416e8f626a18 100644 --- a/packages/contracts-bedrock/STYLE_GUIDE.md +++ b/packages/contracts-bedrock/STYLE_GUIDE.md @@ -45,6 +45,18 @@ We also have the following custom tags: - Event parameters should NOT be prefixed with an underscore. +#### Immutable variables + +Immutable variables: + +- should be in `SCREAMING_SNAKE_CASE` +- should be `internal` +- should have a hand written getter function + +This approach clearly indicates to the developer that the value is immutable, without exposing +the non-standard casing to the interface. It also ensure that we don’t need to break the ABIs if +we switch between values being in storage and immutable. + #### Spacers We use spacer variables to account for old storage slots that are no longer being used. From 5a54745bfc794bedf051c13f7689104856855812 Mon Sep 17 00:00:00 2001 From: Hamdi Allam Date: Tue, 24 Oct 2023 11:01:43 -0400 Subject: [PATCH 029/279] legacy.withdrawal.hash --- .../processors/bridge/legacy_bridge_processor.go | 15 ++++++--------- 1 file changed, 6 insertions(+), 9 deletions(-) diff --git a/indexer/processors/bridge/legacy_bridge_processor.go b/indexer/processors/bridge/legacy_bridge_processor.go index ad7c80132314..b98819f7a0fd 100644 --- a/indexer/processors/bridge/legacy_bridge_processor.go +++ b/indexer/processors/bridge/legacy_bridge_processor.go @@ -5,7 +5,6 @@ import ( "math/big" "github.com/ethereum/go-ethereum/common" - "github.com/ethereum/go-ethereum/crypto" "github.com/ethereum/go-ethereum/log" "github.com/ethereum-optimism/optimism/indexer/bigint" @@ -40,8 +39,8 @@ func LegacyL1ProcessInitiatedBridgeEvents(log log.Logger, db *database.DB, metri ctcTxDeposits[logKey{deposit.Event.BlockHash, deposit.Event.LogIndex}] = &deposit mintedWEI = new(big.Int).Add(mintedWEI, deposit.Tx.Amount) + // We re-use the L2 Transaction hash as the source hash to remain consistent in the schema. transactionDeposits[i] = database.L1TransactionDeposit{ - // We re-use the L2 Transaction hash as the source hash to remain consistent in the schema. SourceHash: deposit.TxHash, L2TransactionHash: deposit.TxHash, InitiatedL1EventGUID: deposit.Event.GUID, @@ -175,11 +174,9 @@ func LegacyL2ProcessInitiatedBridgeEvents(log log.Logger, db *database.DB, metri sentMessage := crossDomainSentMessages[i] withdrawnWEI = new(big.Int).Add(withdrawnWEI, sentMessage.BridgeMessage.Tx.Amount) - // To ensure consistency in the schema, we duplicate this as the "root" transaction withdrawal. The storage key in the message - // passer contract is sha3(calldata + sender). The sender always being the L2CrossDomainMessenger pre-bedrock. - withdrawalHash := crypto.Keccak256Hash(append(sentMessage.MessageCalldata, l2Contracts.L2CrossDomainMessenger[:]...)) + // We re-use the L2CrossDomainMessenger message hash as the withdrawal hash to remain consistent in the schema. transactionWithdrawals[i] = database.L2TransactionWithdrawal{ - WithdrawalHash: withdrawalHash, + WithdrawalHash: sentMessage.BridgeMessage.MessageHash, InitiatedL2EventGUID: sentMessage.Event.GUID, Nonce: sentMessage.BridgeMessage.Nonce, GasLimit: sentMessage.BridgeMessage.GasLimit, @@ -192,9 +189,9 @@ func LegacyL2ProcessInitiatedBridgeEvents(log log.Logger, db *database.DB, metri }, } - sentMessages[logKey{sentMessage.Event.BlockHash, sentMessage.Event.LogIndex}] = sentMessageEvent{&sentMessage, withdrawalHash} + sentMessages[logKey{sentMessage.Event.BlockHash, sentMessage.Event.LogIndex}] = sentMessageEvent{&sentMessage, sentMessage.BridgeMessage.MessageHash} bridgeMessages[i] = database.L2BridgeMessage{ - TransactionWithdrawalHash: withdrawalHash, + TransactionWithdrawalHash: sentMessage.BridgeMessage.MessageHash, BridgeMessage: sentMessage.BridgeMessage, } } @@ -311,7 +308,7 @@ func LegacyL1ProcessFinalizedBridgeEvents(log log.Logger, db *database.DB, metri } } - // Mark the associated tx withdrawal as proven/finalized with the same event + // Mark the associated tx withdrawal as proven/finalized with the same event. The message hash is also the transaction withdrawal hash if err := db.BridgeTransactions.MarkL2TransactionWithdrawalProvenEvent(relayedMessage.MessageHash, relayedMessage.Event.GUID); err != nil { log.Error("failed to mark withdrawal as proven", "err", err) return err From 813cd01b2341c831af392ab22cd3f50f4ae60a32 Mon Sep 17 00:00:00 2001 From: protolambda Date: Tue, 24 Oct 2023 17:06:02 +0200 Subject: [PATCH 030/279] op-batcher: fix config Check method-receiver ptr --- op-batcher/batcher/config.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/op-batcher/batcher/config.go b/op-batcher/batcher/config.go index a09b7f832e53..ad7b611c9a68 100644 --- a/op-batcher/batcher/config.go +++ b/op-batcher/batcher/config.go @@ -60,8 +60,8 @@ type CLIConfig struct { RPC oprpc.CLIConfig } -func (c CLIConfig) Check() error { - // TODO: check the sanity of flags loaded directly https://github.com/ethereum-optimism/optimism/issues/7512 +func (c *CLIConfig) Check() error { + // TODO(7512): check the sanity of flags loaded directly https://github.com/ethereum-optimism/optimism/issues/7512 if err := c.MetricsConfig.Check(); err != nil { return err From 00250cb22a4bf46d40a28eba2778cdaf3016e22a Mon Sep 17 00:00:00 2001 From: clabby Date: Tue, 24 Oct 2023 12:03:44 -0400 Subject: [PATCH 031/279] @maurelian review --- .../contracts-bedrock/scripts/go-ffi/README.md | 18 ++++++++++++++++++ .../scripts/go-ffi/differential-testing.go | 3 ++- 2 files changed, 20 insertions(+), 1 deletion(-) diff --git a/packages/contracts-bedrock/scripts/go-ffi/README.md b/packages/contracts-bedrock/scripts/go-ffi/README.md index 29dac2e83d5f..8348100fb8a3 100644 --- a/packages/contracts-bedrock/scripts/go-ffi/README.md +++ b/packages/contracts-bedrock/scripts/go-ffi/README.md @@ -13,8 +13,26 @@ A lightweight binary for utilities accessed via `forge`'s `ffi` cheatcode in the To build, run `pnpm build:go-ffi` from this directory or the `contract-bedrock` package. +### In a Forge Test + +To use `go-ffi` in a forge test, simply invoke the binary via the `vm.ffi` cheatcode. + +```solidity +function myFFITest() public { + string[] memory commands = new string[](3); + commands[0] = "./scripts/go-ffi/go-ffi"; + commands[1] = "trie"; + commands[2] = "valid"; + bytes memory result = vm.ffi(commands); + + // Do something with the result of the command +} +``` + ### Available Modes +There are two modes available in `go-ffi`: `diff` and `trie`. Each are present as a subcommand to the `go-ffi` binary, with their own set of variants. + #### `diff` > **Note** diff --git a/packages/contracts-bedrock/scripts/go-ffi/differential-testing.go b/packages/contracts-bedrock/scripts/go-ffi/differential-testing.go index e990f79cddb2..6c48d31de96a 100644 --- a/packages/contracts-bedrock/scripts/go-ffi/differential-testing.go +++ b/packages/contracts-bedrock/scripts/go-ffi/differential-testing.go @@ -76,13 +76,14 @@ var ( func DiffTestUtils() { args := os.Args[2:] + variant := args[0] // This command requires arguments if len(args) == 0 { panic("Error: No arguments provided") } - switch args[0] { + switch variant { case "decodeVersionedNonce": // Parse input arguments input, ok := new(big.Int).SetString(args[1], 10) From 513f96d15e545139252c2c4f0af166d1e9866f18 Mon Sep 17 00:00:00 2001 From: Will Cory Date: Tue, 24 Oct 2023 10:07:53 -0700 Subject: [PATCH 032/279] fix: Add chain-mon to version python script --- ops/tag-service/tag-service.py | 1 + 1 file changed, 1 insertion(+) diff --git a/ops/tag-service/tag-service.py b/ops/tag-service/tag-service.py index 90b248174a96..4c226781c9ae 100755 --- a/ops/tag-service/tag-service.py +++ b/ops/tag-service/tag-service.py @@ -11,6 +11,7 @@ # Minimum version numbers for packages migrating from legacy versioning. MIN_VERSIONS = { 'ci-builder': '0.6.0', + 'chain-mon': '0.2.2' 'indexer': '0.5.0', 'op-node': '0.10.14', 'op-batcher': '0.10.14', From 2c1d75aeb5fc18c3d94e318ae8f34f81bf71e08c Mon Sep 17 00:00:00 2001 From: protolambda Date: Fri, 13 Oct 2023 00:04:49 +0200 Subject: [PATCH 033/279] specs,go.mod: protocol-version prerelease edge-case, update op-geth to pull in updated version comparison --- specs/superchain-upgrades.md | 1 + 1 file changed, 1 insertion(+) diff --git a/specs/superchain-upgrades.md b/specs/superchain-upgrades.md index 244c318622e9..d606c7eebfdd 100644 --- a/specs/superchain-upgrades.md +++ b/specs/superchain-upgrades.md @@ -128,6 +128,7 @@ Patch version changes indicate backward compatible bug fixes and improvements. Pre-releases of the protocol are proposals: these are not stable targets for production usage. A pre-release might not satisfy the intended compatibility requirements as denoted by its associated normal version. The `` must be non-zero to apply to the protocol version. +The `` `0`-value is reserved for non-prereleases, i.e. `v3.1.0` is higher than `v3.1.0-1`. Node-software may support a pre-release, but must not activate any protocol changes without the user explicitly opting in through the means of a feature-flag or configuration change. From 1672ca53f420537b09e41a136a8cfb85e4d1df66 Mon Sep 17 00:00:00 2001 From: clabby Date: Tue, 24 Oct 2023 13:47:46 -0400 Subject: [PATCH 034/279] @tynes review --- packages/contracts-bedrock/scripts/go-ffi/bin.go | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/packages/contracts-bedrock/scripts/go-ffi/bin.go b/packages/contracts-bedrock/scripts/go-ffi/bin.go index 26a62343a2f4..da7475392623 100644 --- a/packages/contracts-bedrock/scripts/go-ffi/bin.go +++ b/packages/contracts-bedrock/scripts/go-ffi/bin.go @@ -1,6 +1,9 @@ package main -import "os" +import ( + "log" + "os" +) func main() { switch os.Args[1] { @@ -8,5 +11,7 @@ func main() { DiffTestUtils() case "trie": FuzzTrie() + default: + log.Fatal("Must pass a subcommand") } } From 821dd5e9928b8ec7b9c0e3348c2472b6ee3efd81 Mon Sep 17 00:00:00 2001 From: Will Cory Date: Wed, 18 Oct 2023 15:09:02 -0700 Subject: [PATCH 035/279] feat: Throw on unknown key error --- indexer/config/config.go | 9 ++++++- indexer/config/config_test.go | 46 +++++++++++++++++++++++++++++++++++ 2 files changed, 54 insertions(+), 1 deletion(-) diff --git a/indexer/config/config.go b/indexer/config/config.go index e35e5aa6a6f7..c8ecb286a6c3 100644 --- a/indexer/config/config.go +++ b/indexer/config/config.go @@ -153,11 +153,18 @@ func LoadConfig(log log.Logger, path string) (Config, error) { data = []byte(os.ExpandEnv(string(data))) log.Debug("parsed config file", "data", string(data)) - if _, err := toml.Decode(string(data), &cfg); err != nil { + md, err := toml.Decode(string(data), &cfg) + if err != nil { log.Error("failed to decode config file", "err", err) return cfg, err } + if len(md.Undecoded()) > 0 { + log.Error("unknown fields in config file", "fields", md.Undecoded()) + err = fmt.Errorf("unknown fields in config file: %v", md.Undecoded()) + return cfg, err + } + if cfg.Chain.Preset == DevnetPresetId { preset, err := DevnetPreset() if err != nil { diff --git a/indexer/config/config_test.go b/indexer/config/config_test.go index ee6bbc216b97..4e0b0652c29d 100644 --- a/indexer/config/config_test.go +++ b/indexer/config/config_test.go @@ -257,3 +257,49 @@ func TestLocalDevnet(t *testing.T) { require.Equal(t, devnetPreset.ChainConfig.L1Contracts, conf.Chain.L1Contracts) } + +func TestThrowsOnUnknownKeys(t *testing.T) { + logger := testlog.Logger(t, log.LvlInfo) + tmpfile, err := os.CreateTemp("", "test.toml") + require.NoError(t, err) + defer os.Remove(tmpfile.Name()) + defer tmpfile.Close() + + testData := ` + [chain] + unknown_key = 420 + preset = 420 + + [rpcs] + l1-rpc = "https://l1.example.com" + l2-rpc = "https://l2.example.com" + + [db] + another_unknownKey = 420 + host = "127.0.0.1" + port = 5432 + user = "postgres" + password = "postgres" + name = "indexer" + + [http] + host = "127.0.0.1" + port = 8080 + + [metrics] + host = "127.0.0.1" + port = 7300 + ` + + data := []byte(testData) + err = os.WriteFile(tmpfile.Name(), data, 0644) + require.NoError(t, err) + defer os.Remove(tmpfile.Name()) + + err = tmpfile.Close() + require.NoError(t, err) + + _, err = LoadConfig(logger, tmpfile.Name()) + require.Error(t, err) + require.Contains(t, err.Error(), "unknown fields in config file") +} From ccf1a494a8e70c52b1b12f5827d45f86d144a9f2 Mon Sep 17 00:00:00 2001 From: Will Cory Date: Tue, 24 Oct 2023 11:22:30 -0700 Subject: [PATCH 036/279] fix: trailing comma --- ops/tag-service/tag-service.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ops/tag-service/tag-service.py b/ops/tag-service/tag-service.py index 4c226781c9ae..37d875e6abbc 100755 --- a/ops/tag-service/tag-service.py +++ b/ops/tag-service/tag-service.py @@ -11,7 +11,7 @@ # Minimum version numbers for packages migrating from legacy versioning. MIN_VERSIONS = { 'ci-builder': '0.6.0', - 'chain-mon': '0.2.2' + 'chain-mon': '0.2.2', 'indexer': '0.5.0', 'op-node': '0.10.14', 'op-batcher': '0.10.14', From 4a9bde8f5529d3e4c41aa02aedb1a147e65bb8eb Mon Sep 17 00:00:00 2001 From: Hamdi Allam Date: Tue, 24 Oct 2023 15:29:10 -0400 Subject: [PATCH 037/279] error message on tx error --- indexer/processors/bridge/legacy_bridge_processor.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/indexer/processors/bridge/legacy_bridge_processor.go b/indexer/processors/bridge/legacy_bridge_processor.go index b98819f7a0fd..a0886a019f69 100644 --- a/indexer/processors/bridge/legacy_bridge_processor.go +++ b/indexer/processors/bridge/legacy_bridge_processor.go @@ -282,7 +282,8 @@ func LegacyL1ProcessFinalizedBridgeEvents(log log.Logger, db *database.DB, metri // for OP-Mainnet pre-regensis withdrawals that no longer exist on L2. tx, err := l1Client.TxByHash(relayedMessage.Event.TransactionHash) if err != nil { - return err + log.Error("unable to query legacy relayed tx", "tx_hash", relayedMessage.Event.TransactionHash.String(), "err", err) + return fmt.Errorf("unable to query legacy relayed tx_hash = %s: %w", relayedMessage.Event.TransactionHash.String(), err) } else if tx == nil { log.Error("missing tx for relayed message", "tx_hash", relayedMessage.Event.TransactionHash.String()) return fmt.Errorf("missing tx for relayed message. tx_hash = %s", relayedMessage.Event.TransactionHash.String()) From a4ad1630fb30325c188ad5165c8bde5687fbe90e Mon Sep 17 00:00:00 2001 From: Hamdi Allam Date: Tue, 24 Oct 2023 15:29:31 -0400 Subject: [PATCH 038/279] indexer bridge api calldata check --- indexer/database/bridge_transfers.go | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/indexer/database/bridge_transfers.go b/indexer/database/bridge_transfers.go index e4d35c0d24d1..57e9dcb1b8f5 100644 --- a/indexer/database/bridge_transfers.go +++ b/indexer/database/bridge_transfers.go @@ -154,12 +154,11 @@ func (db *bridgeTransfersDB) L1BridgeDepositsByAddress(address common.Address, c cursorClause = fmt.Sprintf("l1_transaction_deposits.timestamp <= %d", txDeposit.Tx.Timestamp) } - // TODO join with l1_bridged_tokens and l2_bridged_tokens ethAddressString := predeploys.LegacyERC20ETHAddr.String() // Coalesce l1 transaction deposits that are simply ETH sends ethTransactionDeposits := db.gorm.Model(&L1TransactionDeposit{}) - ethTransactionDeposits = ethTransactionDeposits.Where(&Transaction{FromAddress: address}).Where("data = '0x' AND amount > 0") + ethTransactionDeposits = ethTransactionDeposits.Where(&Transaction{FromAddress: address}).Where("amount > 0") ethTransactionDeposits = ethTransactionDeposits.Joins("INNER JOIN l1_contract_events ON l1_contract_events.guid = initiated_l1_event_guid") ethTransactionDeposits = ethTransactionDeposits.Select(` from_address, to_address, amount, data, source_hash AS transaction_source_hash, @@ -283,7 +282,7 @@ func (db *bridgeTransfersDB) L2BridgeWithdrawalsByAddress(address common.Address // Coalesce l2 transaction withdrawals that are simply ETH sends ethTransactionWithdrawals := db.gorm.Model(&L2TransactionWithdrawal{}) - ethTransactionWithdrawals = ethTransactionWithdrawals.Where(&Transaction{FromAddress: address}).Where(`data = '0x' AND amount > 0`) + ethTransactionWithdrawals = ethTransactionWithdrawals.Where(&Transaction{FromAddress: address}).Where("AND amount > 0") ethTransactionWithdrawals = ethTransactionWithdrawals.Joins("INNER JOIN l2_contract_events ON l2_contract_events.guid = l2_transaction_withdrawals.initiated_l2_event_guid") ethTransactionWithdrawals = ethTransactionWithdrawals.Joins("LEFT JOIN l1_contract_events AS proven_l1_events ON proven_l1_events.guid = l2_transaction_withdrawals.proven_l1_event_guid") ethTransactionWithdrawals = ethTransactionWithdrawals.Joins("LEFT JOIN l1_contract_events AS finalized_l1_events ON finalized_l1_events.guid = l2_transaction_withdrawals.finalized_l1_event_guid") From a667fdda98769703e74fc57e901dba6857c96690 Mon Sep 17 00:00:00 2001 From: Hamdi Allam Date: Tue, 24 Oct 2023 15:41:56 -0400 Subject: [PATCH 039/279] extra AND --- indexer/database/bridge_transfers.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/indexer/database/bridge_transfers.go b/indexer/database/bridge_transfers.go index 57e9dcb1b8f5..543b63dbd8e9 100644 --- a/indexer/database/bridge_transfers.go +++ b/indexer/database/bridge_transfers.go @@ -282,7 +282,7 @@ func (db *bridgeTransfersDB) L2BridgeWithdrawalsByAddress(address common.Address // Coalesce l2 transaction withdrawals that are simply ETH sends ethTransactionWithdrawals := db.gorm.Model(&L2TransactionWithdrawal{}) - ethTransactionWithdrawals = ethTransactionWithdrawals.Where(&Transaction{FromAddress: address}).Where("AND amount > 0") + ethTransactionWithdrawals = ethTransactionWithdrawals.Where(&Transaction{FromAddress: address}).Where("amount > 0") ethTransactionWithdrawals = ethTransactionWithdrawals.Joins("INNER JOIN l2_contract_events ON l2_contract_events.guid = l2_transaction_withdrawals.initiated_l2_event_guid") ethTransactionWithdrawals = ethTransactionWithdrawals.Joins("LEFT JOIN l1_contract_events AS proven_l1_events ON proven_l1_events.guid = l2_transaction_withdrawals.proven_l1_event_guid") ethTransactionWithdrawals = ethTransactionWithdrawals.Joins("LEFT JOIN l1_contract_events AS finalized_l1_events ON finalized_l1_events.guid = l2_transaction_withdrawals.finalized_l1_event_guid") From cacb2c02c5442f9516366ffa978e8165ce423795 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 24 Oct 2023 20:07:36 +0000 Subject: [PATCH 040/279] build(deps-dev): bump @swc/core from 1.3.94 to 1.3.95 Bumps [@swc/core](https://github.com/swc-project/swc) from 1.3.94 to 1.3.95. - [Release notes](https://github.com/swc-project/swc/releases) - [Changelog](https://github.com/swc-project/swc/blob/main/CHANGELOG.md) - [Commits](https://github.com/swc-project/swc/compare/v1.3.94...v1.3.95) --- updated-dependencies: - dependency-name: "@swc/core" dependency-type: direct:development update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] --- packages/web3js-plugin/package.json | 2 +- pnpm-lock.yaml | 80 ++++++++++++++--------------- 2 files changed, 41 insertions(+), 41 deletions(-) diff --git a/packages/web3js-plugin/package.json b/packages/web3js-plugin/package.json index 1d56460893e8..8359a0d18acc 100644 --- a/packages/web3js-plugin/package.json +++ b/packages/web3js-plugin/package.json @@ -33,7 +33,7 @@ }, "devDependencies": { "@eth-optimism/contracts-ts": "workspace:^", - "@swc/core": "^1.3.94", + "@swc/core": "^1.3.95", "@vitest/coverage-istanbul": "^0.34.6", "tsup": "^7.2.0", "typescript": "^5.2.2", diff --git a/pnpm-lock.yaml b/pnpm-lock.yaml index e25e904b6d5d..7250c79581b1 100644 --- a/pnpm-lock.yaml +++ b/pnpm-lock.yaml @@ -130,7 +130,7 @@ importers: devDependencies: tsup: specifier: ^7.2.0 - version: 7.2.0(@swc/core@1.3.94)(typescript@5.2.2) + version: 7.2.0(@swc/core@1.3.95)(typescript@5.2.2) vitest: specifier: ^0.34.4 version: 0.34.4 @@ -342,7 +342,7 @@ importers: version: 22.1.0 tsup: specifier: ^7.2.0 - version: 7.2.0(@swc/core@1.3.94)(typescript@5.2.2) + version: 7.2.0(@swc/core@1.3.95)(typescript@5.2.2) typescript: specifier: ^5.2.2 version: 5.2.2 @@ -433,7 +433,7 @@ importers: version: 22.1.0 tsup: specifier: ^7.2.0 - version: 7.2.0(@swc/core@1.3.94)(typescript@5.2.2) + version: 7.2.0(@swc/core@1.3.95)(typescript@5.2.2) typescript: specifier: ^5.2.2 version: 5.2.2 @@ -557,14 +557,14 @@ importers: specifier: workspace:^ version: link:../contracts-ts '@swc/core': - specifier: ^1.3.94 - version: 1.3.94 + specifier: ^1.3.95 + version: 1.3.95 '@vitest/coverage-istanbul': specifier: ^0.34.6 version: 0.34.6(vitest@0.34.1) tsup: specifier: ^7.2.0 - version: 7.2.0(@swc/core@1.3.94)(typescript@5.2.2) + version: 7.2.0(@swc/core@1.3.95)(typescript@5.2.2) typescript: specifier: ^5.2.2 version: 5.2.2 @@ -3550,8 +3550,8 @@ packages: '@stablelib/random': 1.0.2 '@stablelib/wipe': 1.0.1 - /@swc/core-darwin-arm64@1.3.94: - resolution: {integrity: sha512-KNuE6opIy/wAXiGUWLhGWhCG3wA/AdjG6eYkv6dstrAURLaQMAoD8vDfVm8pxS8FA8Kx+0Z4QiDNPqk5aKIsqg==} + /@swc/core-darwin-arm64@1.3.95: + resolution: {integrity: sha512-VAuBAP3MNetO/yBIBzvorUXq7lUBwhfpJxYViSxyluMwtoQDhE/XWN598TWMwMl1ZuImb56d7eUsuFdjgY7pJw==} engines: {node: '>=10'} cpu: [arm64] os: [darwin] @@ -3559,8 +3559,8 @@ packages: dev: true optional: true - /@swc/core-darwin-x64@1.3.94: - resolution: {integrity: sha512-HypemhyehQrLqXwfJv5ronD4BMAXdgMCP4Ei7rt3B6Ftmt9axwGvdwGiXxsYR9h1ncyxoVxN+coGxbNIhKhahw==} + /@swc/core-darwin-x64@1.3.95: + resolution: {integrity: sha512-20vF2rvUsN98zGLZc+dsEdHvLoCuiYq/1B+TDeE4oolgTFDmI1jKO+m44PzWjYtKGU9QR95sZ6r/uec0QC5O4Q==} engines: {node: '>=10'} cpu: [x64] os: [darwin] @@ -3568,8 +3568,8 @@ packages: dev: true optional: true - /@swc/core-linux-arm-gnueabihf@1.3.94: - resolution: {integrity: sha512-KzKN54c7Y6X1db+bBVSXG4+bXmAPvXtDWk+TgwNJH4yYliOrnP/RKkHA5QZ9VFSnqJF06/sAO4kYBiL/aVQDBQ==} + /@swc/core-linux-arm-gnueabihf@1.3.95: + resolution: {integrity: sha512-oEudEM8PST1MRNGs+zu0cx5i9uP8TsLE4/L9HHrS07Ck0RJ3DCj3O2fU832nmLe2QxnAGPwBpSO9FntLfOiWEQ==} engines: {node: '>=10'} cpu: [arm] os: [linux] @@ -3577,8 +3577,8 @@ packages: dev: true optional: true - /@swc/core-linux-arm64-gnu@1.3.94: - resolution: {integrity: sha512-iAcR8Ho0Uck/SLSrgYfXkpcGOXuN5waMZO7GlL/52QODr7GJtOfZ0H1MCZLbIFkPJp/iXoJpYgym4d/qSd477Q==} + /@swc/core-linux-arm64-gnu@1.3.95: + resolution: {integrity: sha512-pIhFI+cuC1aYg+0NAPxwT/VRb32f2ia8oGxUjQR6aJg65gLkUYQzdwuUmpMtFR2WVf7WVFYxUnjo4UyMuyh3ng==} engines: {node: '>=10'} cpu: [arm64] os: [linux] @@ -3586,8 +3586,8 @@ packages: dev: true optional: true - /@swc/core-linux-arm64-musl@1.3.94: - resolution: {integrity: sha512-VCHL1Mb9ENHx+sAeubSSg481MUeP9/PYzPPy9tfswunj/w35M+vEWflwK2dzQL9kUTFD3zcFTpAgsKnj6aX24w==} + /@swc/core-linux-arm64-musl@1.3.95: + resolution: {integrity: sha512-ZpbTr+QZDT4OPJfjPAmScqdKKaT+wGurvMU5AhxLaf85DuL8HwUwwlL0n1oLieLc47DwIJEMuKQkYhXMqmJHlg==} engines: {node: '>=10'} cpu: [arm64] os: [linux] @@ -3595,8 +3595,8 @@ packages: dev: true optional: true - /@swc/core-linux-x64-gnu@1.3.94: - resolution: {integrity: sha512-gjq7U6clhJi0Oel2a4gwR4MbSu+THQ2hmBNVCOSA3JjPZWZTkJXaJDpnh/r7PJxKBwUDlo0VPlwiwjepAQR2Rw==} + /@swc/core-linux-x64-gnu@1.3.95: + resolution: {integrity: sha512-n9SuHEFtdfSJ+sHdNXNRuIOVprB8nbsz+08apKfdo4lEKq6IIPBBAk5kVhPhkjmg2dFVHVo4Tr/OHXM1tzWCCw==} engines: {node: '>=10'} cpu: [x64] os: [linux] @@ -3604,8 +3604,8 @@ packages: dev: true optional: true - /@swc/core-linux-x64-musl@1.3.94: - resolution: {integrity: sha512-rSylruWyeol2ujZDHmwiovupMR5ukMXivlA7DDxmQ1dFUV9HuiPknQrU5rEbI3V2V3V5RkpbEKjnADen7AeMPQ==} + /@swc/core-linux-x64-musl@1.3.95: + resolution: {integrity: sha512-L1JrVlsXU3LC0WwmVnMK9HrOT2uhHahAoPNMJnZQpc18a0paO9fqifPG8M/HjNRffMUXR199G/phJsf326UvVg==} engines: {node: '>=10'} cpu: [x64] os: [linux] @@ -3613,8 +3613,8 @@ packages: dev: true optional: true - /@swc/core-win32-arm64-msvc@1.3.94: - resolution: {integrity: sha512-OenDUr5MQkz506ebVQq6ezoZ3GZ26nchgf5mPnwab4gx2TEiyR9zn7MdX5LWskTmOK3+FszPbGK0B5oLK6Y5yw==} + /@swc/core-win32-arm64-msvc@1.3.95: + resolution: {integrity: sha512-YaP4x/aZbUyNdqCBpC2zL8b8n58MEpOUpmOIZK6G1SxGi+2ENht7gs7+iXpWPc0sy7X3YPKmSWMAuui0h8lgAA==} engines: {node: '>=10'} cpu: [arm64] os: [win32] @@ -3622,8 +3622,8 @@ packages: dev: true optional: true - /@swc/core-win32-ia32-msvc@1.3.94: - resolution: {integrity: sha512-mi6NcmtJKnaiHAxLtVz+WzunscsEwPdA0j15DuiYVx06Xo+MdRLJj4eVBgVLwGD1AI3IqKs4MVVx2cD7n0h5mg==} + /@swc/core-win32-ia32-msvc@1.3.95: + resolution: {integrity: sha512-w0u3HI916zT4BC/57gOd+AwAEjXeUlQbGJ9H4p/gzs1zkSHtoDQghVUNy3n/ZKp9KFod/95cA8mbVF9t1+6epQ==} engines: {node: '>=10'} cpu: [ia32] os: [win32] @@ -3631,8 +3631,8 @@ packages: dev: true optional: true - /@swc/core-win32-x64-msvc@1.3.94: - resolution: {integrity: sha512-Ba0ZLcGMnqPWWF9Xa+rWhhnkpvE7XoQegMP/VCF2JIHb2ieGBC8jChO6nKRFKZjib/3wghGzxakyDQx3LDhDug==} + /@swc/core-win32-x64-msvc@1.3.95: + resolution: {integrity: sha512-5RGnMt0S6gg4Gc6QtPUJ3Qs9Un4sKqccEzgH/tj7V/DVTJwKdnBKxFZfgQ34OR2Zpz7zGOn889xwsFVXspVWNA==} engines: {node: '>=10'} cpu: [x64] os: [win32] @@ -3640,8 +3640,8 @@ packages: dev: true optional: true - /@swc/core@1.3.94: - resolution: {integrity: sha512-jTHn8UJOGgERKZLy8euEixVAzC/w/rUSuMlM3e7hxgap/TC595hSkuQwtkpL238dsuEPveD44GMy2A5UBtSvjg==} + /@swc/core@1.3.95: + resolution: {integrity: sha512-PMrNeuqIusq9DPDooV3FfNEbZuTu5jKAc04N3Hm6Uk2Fl49cqElLFQ4xvl4qDmVDz97n3n/C1RE0/f6WyGPEiA==} engines: {node: '>=10'} requiresBuild: true peerDependencies: @@ -3653,16 +3653,16 @@ packages: '@swc/counter': 0.1.1 '@swc/types': 0.1.5 optionalDependencies: - '@swc/core-darwin-arm64': 1.3.94 - '@swc/core-darwin-x64': 1.3.94 - '@swc/core-linux-arm-gnueabihf': 1.3.94 - '@swc/core-linux-arm64-gnu': 1.3.94 - '@swc/core-linux-arm64-musl': 1.3.94 - '@swc/core-linux-x64-gnu': 1.3.94 - '@swc/core-linux-x64-musl': 1.3.94 - '@swc/core-win32-arm64-msvc': 1.3.94 - '@swc/core-win32-ia32-msvc': 1.3.94 - '@swc/core-win32-x64-msvc': 1.3.94 + '@swc/core-darwin-arm64': 1.3.95 + '@swc/core-darwin-x64': 1.3.95 + '@swc/core-linux-arm-gnueabihf': 1.3.95 + '@swc/core-linux-arm64-gnu': 1.3.95 + '@swc/core-linux-arm64-musl': 1.3.95 + '@swc/core-linux-x64-gnu': 1.3.95 + '@swc/core-linux-x64-musl': 1.3.95 + '@swc/core-win32-arm64-msvc': 1.3.95 + '@swc/core-win32-ia32-msvc': 1.3.95 + '@swc/core-win32-x64-msvc': 1.3.95 dev: true /@swc/counter@0.1.1: @@ -13885,7 +13885,7 @@ packages: resolution: {integrity: sha512-Tyrf5mxF8Ofs1tNoxA13lFeZ2Zrbd6cKbuH3V+MQ5sb6DtBj5FjrXVsRWT8YvNAQTqNoz66dz1WsbigI22aEnw==} dev: true - /tsup@7.2.0(@swc/core@1.3.94)(typescript@5.2.2): + /tsup@7.2.0(@swc/core@1.3.95)(typescript@5.2.2): resolution: {integrity: sha512-vDHlczXbgUvY3rWvqFEbSqmC1L7woozbzngMqTtL2PGBODTtWlRwGDDawhvWzr5c1QjKe4OAKqJGfE1xeXUvtQ==} engines: {node: '>=16.14'} hasBin: true @@ -13901,7 +13901,7 @@ packages: typescript: optional: true dependencies: - '@swc/core': 1.3.94 + '@swc/core': 1.3.95 bundle-require: 4.0.1(esbuild@0.18.15) cac: 6.7.14 chokidar: 3.5.3 From bc58f02f6464bb5fd59918316f6d95d2fc31eeda Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 24 Oct 2023 20:11:45 +0000 Subject: [PATCH 041/279] build(deps): bump @sentry/node from 7.74.1 to 7.75.0 Bumps [@sentry/node](https://github.com/getsentry/sentry-javascript) from 7.74.1 to 7.75.0. - [Release notes](https://github.com/getsentry/sentry-javascript/releases) - [Changelog](https://github.com/getsentry/sentry-javascript/blob/develop/CHANGELOG.md) - [Commits](https://github.com/getsentry/sentry-javascript/compare/7.74.1...7.75.0) --- updated-dependencies: - dependency-name: "@sentry/node" dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] --- packages/common-ts/package.json | 2 +- pnpm-lock.yaml | 53 +++++++++++++++------------------ 2 files changed, 25 insertions(+), 30 deletions(-) diff --git a/packages/common-ts/package.json b/packages/common-ts/package.json index 446d3541d30f..82500bbd4109 100644 --- a/packages/common-ts/package.json +++ b/packages/common-ts/package.json @@ -35,7 +35,7 @@ }, "dependencies": { "@eth-optimism/core-utils": "workspace:*", - "@sentry/node": "^7.74.1", + "@sentry/node": "^7.75.0", "bcfg": "^0.2.1", "body-parser": "^1.20.2", "commander": "^11.1.0", diff --git a/pnpm-lock.yaml b/pnpm-lock.yaml index e25e904b6d5d..7c82f8567ed1 100644 --- a/pnpm-lock.yaml +++ b/pnpm-lock.yaml @@ -195,8 +195,8 @@ importers: specifier: workspace:* version: link:../core-utils '@sentry/node': - specifier: ^7.74.1 - version: 7.74.1 + specifier: ^7.75.0 + version: 7.75.0 bcfg: specifier: ^0.2.1 version: 0.2.1 @@ -3282,14 +3282,13 @@ packages: '@noble/hashes': 1.3.2 '@scure/base': 1.1.3 - /@sentry-internal/tracing@7.74.1: - resolution: {integrity: sha512-nNaiZreQxCitG2PzYPaC7XtyA9OMsETGYMKAtiK4p62/uTmeYbsBva9BoNx1XeiHRwbrVQYRMKQ9nV5e2jS4/A==} + /@sentry-internal/tracing@7.75.0: + resolution: {integrity: sha512-/j4opF/jB9j8qnSiQK75/lFLtkfqXS5/MoOKc2KWK/pOaf15W+6uJzGQ8jRBHLYd9dDg6AyqsF48Wqy561/mNg==} engines: {node: '>=8'} dependencies: - '@sentry/core': 7.74.1 - '@sentry/types': 7.74.1 - '@sentry/utils': 7.74.1 - tslib: 2.6.2 + '@sentry/core': 7.75.0 + '@sentry/types': 7.75.0 + '@sentry/utils': 7.75.0 dev: false /@sentry/core@5.30.0: @@ -3303,13 +3302,12 @@ packages: tslib: 1.14.1 dev: true - /@sentry/core@7.74.1: - resolution: {integrity: sha512-LvEhOSfdIvwkr+PdlrT/aA/iOLhkXrSkvjqAQyogE4ddCWeYfS0NoirxNt1EaxMBAWKhYZRqzkA7WA4LDLbzlA==} + /@sentry/core@7.75.0: + resolution: {integrity: sha512-vXg3cdJgwzP24oTS9zFCgLW4MgTkMZqXx+ESRq7gTD9qJTpcmAmYT+Ckmvebg8K6DBThV6+0v61r50na2+XdrA==} engines: {node: '>=8'} dependencies: - '@sentry/types': 7.74.1 - '@sentry/utils': 7.74.1 - tslib: 2.6.2 + '@sentry/types': 7.75.0 + '@sentry/utils': 7.75.0 dev: false /@sentry/hub@5.30.0: @@ -3347,18 +3345,15 @@ packages: - supports-color dev: true - /@sentry/node@7.74.1: - resolution: {integrity: sha512-aMUQ2LFZF64FBr+cgjAqjT4OkpYBIC9lyWI8QqjEHqNho5+LGu18/iVrJPD4fgs4UhGdCuAiQjpC36MbmnIDZA==} + /@sentry/node@7.75.0: + resolution: {integrity: sha512-z5Xanf9QeTd4YrEuZiJfvtAy2C874Zg4KpurEo3okJ8uYjnbXMsQ3EwVHbKEoYSwE3ExTrqOggPfk2NNSJIECA==} engines: {node: '>=8'} dependencies: - '@sentry-internal/tracing': 7.74.1 - '@sentry/core': 7.74.1 - '@sentry/types': 7.74.1 - '@sentry/utils': 7.74.1 - cookie: 0.5.0 + '@sentry-internal/tracing': 7.75.0 + '@sentry/core': 7.75.0 + '@sentry/types': 7.75.0 + '@sentry/utils': 7.75.0 https-proxy-agent: 5.0.1 - lru_map: 0.3.3 - tslib: 2.6.2 transitivePeerDependencies: - supports-color dev: false @@ -3379,8 +3374,8 @@ packages: engines: {node: '>=6'} dev: true - /@sentry/types@7.74.1: - resolution: {integrity: sha512-2jIuPc+YKvXqZETwr2E8VYnsH1zsSUR/wkIvg1uTVeVNyoowJv+YsOtCdeGyL2AwiotUBSPKu7O1Lz0kq5rMOQ==} + /@sentry/types@7.75.0: + resolution: {integrity: sha512-xG8OLADxG7HpGhMxrF4v4tKq/v/gqmLsTZ858R51pz0xCWM8SK6ZSWOKudkAGBIpRjI6RUHMnkBtRAN2aKDOkQ==} engines: {node: '>=8'} dev: false @@ -3392,12 +3387,11 @@ packages: tslib: 1.14.1 dev: true - /@sentry/utils@7.74.1: - resolution: {integrity: sha512-qUsqufuHYcy5gFhLZslLxA5kcEOkkODITXW3c7D+x+8iP/AJqa8v8CeUCVNS7RetHCuIeWAbbTClC4c411EwQg==} + /@sentry/utils@7.75.0: + resolution: {integrity: sha512-UHWKeevhUNRp+mAWDbMVFOMgseoq8t/xFgdUywO/2PC14qZKRBH+0k1BKoNkp5sOzDT06ETj2w6wYoYhy6i+dA==} engines: {node: '>=8'} dependencies: - '@sentry/types': 7.74.1 - tslib: 2.6.2 + '@sentry/types': 7.75.0 dev: false /@sinclair/typebox@0.27.8: @@ -10488,6 +10482,7 @@ packages: /lru_map@0.3.3: resolution: {integrity: sha512-Pn9cox5CsMYngeDbmChANltQl+5pi6XmTrraMSzhPmMBbmgcxmqWry0U3PGapCU1yB4/LqCcom7qhHZiF/jGfQ==} + dev: true /ltgt@2.2.1: resolution: {integrity: sha512-AI2r85+4MquTw9ZYqabu4nMwy9Oftlfa/e/52t9IjtfG+mGBbTNdAoZ3RQKLHR6r0wQnwZnPIEh/Ya6XTWAKNA==} @@ -12008,7 +12003,7 @@ packages: engines: {node: '>=10'} hasBin: true dependencies: - '@sentry/node': 7.74.1 + '@sentry/node': 7.75.0 commander: 2.20.3 pumpify: 2.0.1 split2: 3.2.2 From 0958c63c9628a943bf01ae48afec0af9b31f7978 Mon Sep 17 00:00:00 2001 From: Mark Tyneway Date: Tue, 24 Oct 2023 14:13:07 -0600 Subject: [PATCH 042/279] contracts-bedrock: add protocol versions to hh deploy config Adds the two new protocol versions config values to the `hardhat` deploy config. This ensures that the config is up to date and can be consumed by the tests. --- packages/contracts-bedrock/deploy-config/hardhat.json | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/packages/contracts-bedrock/deploy-config/hardhat.json b/packages/contracts-bedrock/deploy-config/hardhat.json index d52ffe5b01e5..ab5279412ef4 100644 --- a/packages/contracts-bedrock/deploy-config/hardhat.json +++ b/packages/contracts-bedrock/deploy-config/hardhat.json @@ -39,5 +39,7 @@ "eip1559Denominator": 50, "eip1559Elasticity": 10, "l2GenesisRegolithTimeOffset": "0x0", - "systemConfigStartBlock": 0 + "systemConfigStartBlock": 0, + "requiredProtocolVersion": "0x0000000000000000000000000000000000000000000000000000000000000000", + "recommendedProtocolVersion": "0x0000000000000000000000000000000000000000000000000000000000000000" } From a36e3ecd8ac99aa4c166c51658064048b962d35b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 24 Oct 2023 20:26:00 +0000 Subject: [PATCH 043/279] build(deps-dev): bump @types/node from 20.8.6 to 20.8.8 Bumps [@types/node](https://github.com/DefinitelyTyped/DefinitelyTyped/tree/HEAD/types/node) from 20.8.6 to 20.8.8. - [Release notes](https://github.com/DefinitelyTyped/DefinitelyTyped/releases) - [Commits](https://github.com/DefinitelyTyped/DefinitelyTyped/commits/HEAD/types/node) --- updated-dependencies: - dependency-name: "@types/node" dependency-type: direct:development update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] --- package.json | 2 +- packages/core-utils/package.json | 2 +- packages/sdk/package.json | 2 +- pnpm-lock.yaml | 126 +++++++++++++++---------------- 4 files changed, 66 insertions(+), 66 deletions(-) diff --git a/package.json b/package.json index 374220207702..e3c8d6778437 100644 --- a/package.json +++ b/package.json @@ -43,7 +43,7 @@ "@types/chai": "^4.3.8", "@types/chai-as-promised": "^7.1.4", "@types/mocha": "^10.0.3", - "@types/node": "^20.8.6", + "@types/node": "^20.8.8", "@typescript-eslint/eslint-plugin": "^6.7.5", "@typescript-eslint/parser": "^6.9.0", "chai": "^4.3.10", diff --git a/packages/core-utils/package.json b/packages/core-utils/package.json index 21594366365b..44194a74da0c 100644 --- a/packages/core-utils/package.json +++ b/packages/core-utils/package.json @@ -49,7 +49,7 @@ "node-fetch": "^2.6.7" }, "devDependencies": { - "@types/node": "^20.8.6", + "@types/node": "^20.8.8", "mocha": "^10.2.0" } } diff --git a/packages/sdk/package.json b/packages/sdk/package.json index be3244dd9edd..5e2ac2a16f34 100644 --- a/packages/sdk/package.json +++ b/packages/sdk/package.json @@ -44,7 +44,7 @@ "@types/chai": "^4.3.8", "@types/chai-as-promised": "^7.1.5", "@types/mocha": "^10.0.3", - "@types/node": "^20.8.6", + "@types/node": "^20.8.8", "chai-as-promised": "^7.1.1", "ethereum-waffle": "^4.0.10", "ethers": "^5.7.2", diff --git a/pnpm-lock.yaml b/pnpm-lock.yaml index 8292e5ba7ab3..d66bd85d3ab0 100644 --- a/pnpm-lock.yaml +++ b/pnpm-lock.yaml @@ -31,8 +31,8 @@ importers: specifier: ^10.0.3 version: 10.0.3 '@types/node': - specifier: ^20.8.6 - version: 20.8.6 + specifier: ^20.8.8 + version: 20.8.8 '@typescript-eslint/eslint-plugin': specifier: ^6.7.5 version: 6.7.5(@typescript-eslint/parser@6.9.0)(eslint@8.51.0)(typescript@5.2.2) @@ -184,7 +184,7 @@ importers: version: 2.18.2(ts-node@10.9.1)(typescript@5.2.2) ts-node: specifier: ^10.9.1 - version: 10.9.1(@types/node@20.8.6)(typescript@5.2.2) + version: 10.9.1(@types/node@20.8.8)(typescript@5.2.2) tsx: specifier: ^3.14.0 version: 3.14.0 @@ -348,7 +348,7 @@ importers: version: 5.2.2 vite: specifier: ^4.4.10 - version: 4.4.10(@types/node@20.8.6) + version: 4.4.10(@types/node@20.8.8) vitest: specifier: ^0.34.2 version: 0.34.2(jsdom@22.1.0) @@ -399,8 +399,8 @@ importers: version: 2.6.7 devDependencies: '@types/node': - specifier: ^20.8.6 - version: 20.8.6 + specifier: ^20.8.8 + version: 20.8.8 mocha: specifier: ^10.2.0 version: 10.2.0 @@ -442,7 +442,7 @@ importers: version: 1.16.6(typescript@5.2.2)(zod@3.22.4) vite: specifier: ^4.4.10 - version: 4.4.10(@types/node@20.8.6) + version: 4.4.10(@types/node@20.8.8) vitest: specifier: ^0.34.2 version: 0.34.2(jsdom@22.1.0) @@ -493,8 +493,8 @@ importers: specifier: ^10.0.3 version: 10.0.3 '@types/node': - specifier: ^20.8.6 - version: 20.8.6 + specifier: ^20.8.8 + version: 20.8.8 chai-as-promised: specifier: ^7.1.1 version: 7.1.1(chai@4.3.10) @@ -521,7 +521,7 @@ importers: version: 15.1.0 ts-node: specifier: ^10.9.1 - version: 10.9.1(@types/node@20.8.6)(typescript@5.2.2) + version: 10.9.1(@types/node@20.8.8)(typescript@5.2.2) typedoc: specifier: ^0.25.2 version: 0.25.2(typescript@5.2.2) @@ -573,7 +573,7 @@ importers: version: 1.16.6(typescript@5.2.2)(zod@3.22.4) vite: specifier: ^4.4.10 - version: 4.4.10(@types/node@20.8.6) + version: 4.4.10(@types/node@20.8.8) vitest: specifier: ^0.34.1 version: 0.34.1 @@ -3897,20 +3897,20 @@ packages: /@types/bn.js@4.11.6: resolution: {integrity: sha512-pqr857jrp2kPuO9uRjZ3PwnJTjoQy+fcdxvBTvHm6dkmEL9q+hDD/2j/0ELOBPtPnS8LjCX0gI9nbl8lVkadpg==} dependencies: - '@types/node': 20.8.6 + '@types/node': 20.8.8 dev: true /@types/bn.js@5.1.0: resolution: {integrity: sha512-QSSVYj7pYFN49kW77o2s9xTCwZ8F2xLbjLLSEVh8D2F4JUhZtPAGOFLTD+ffqksBx/u4cE/KImFjyhqCjn/LIA==} dependencies: - '@types/node': 20.8.6 + '@types/node': 20.8.8 dev: true /@types/body-parser@1.19.1: resolution: {integrity: sha512-a6bTJ21vFOGIkwM0kzh9Yr89ziVxq4vYH2fQ6N8AeipEzai/cFK6aGMArIkUeIdRIgpwQa+2bXiLuUJCpSf2Cg==} dependencies: '@types/connect': 3.4.35 - '@types/node': 20.8.6 + '@types/node': 20.8.8 dev: true /@types/chai-as-promised@7.1.5: @@ -3943,7 +3943,7 @@ packages: /@types/connect@3.4.35: resolution: {integrity: sha512-cdeYyv4KWoEgpBISTxWvqYsVy444DOqehiF3fM3ne10AmJ62RSyNkUnxMJXHQWRQQX2eR94m5y1IZyDwBjV9FQ==} dependencies: - '@types/node': 20.8.6 + '@types/node': 20.8.8 /@types/dateformat@5.0.0: resolution: {integrity: sha512-SZg4JdHIWHQGEokbYGZSDvo5wA4TLYPXaqhigs/wH+REDOejcJzgH+qyY+HtEUtWOZxEUkbhbdYPqQDiEgrXeA==} @@ -3957,7 +3957,7 @@ packages: /@types/express-serve-static-core@4.17.35: resolution: {integrity: sha512-wALWQwrgiB2AWTT91CB62b6Yt0sNHpznUXeZEcnPU3DRdlDIz74x8Qg1UUYKSVFi+va5vKOLYRBI1bRKiLLKIg==} dependencies: - '@types/node': 20.8.6 + '@types/node': 20.8.8 '@types/qs': 6.9.7 '@types/range-parser': 1.2.4 '@types/send': 0.17.1 @@ -3976,7 +3976,7 @@ packages: resolution: {integrity: sha512-IO+MJPVhoqz+28h1qLAcBEH2+xHMK6MTyHJc7MTnnYb6wsoLR29POVGJ7LycmVXIqyy/4/2ShP5sUwTXuOwb/w==} dependencies: '@types/minimatch': 5.1.2 - '@types/node': 20.8.5 + '@types/node': 20.8.6 dev: true /@types/is-ci@3.0.0: @@ -4003,7 +4003,7 @@ packages: dependencies: '@types/abstract-leveldown': 5.0.2 '@types/level-errors': 3.0.0 - '@types/node': 20.8.6 + '@types/node': 20.8.8 dev: true /@types/lru-cache@5.1.1: @@ -4035,7 +4035,7 @@ packages: /@types/mkdirp@0.5.2: resolution: {integrity: sha512-U5icWpv7YnZYGsN4/cmh3WD2onMY0aJIiTE6+51TwJCttdHvtCYmkBNOobHlXwrJRL0nkH9jH4kD+1FAdMN4Tg==} dependencies: - '@types/node': 20.8.6 + '@types/node': 20.8.8 dev: true /@types/mocha@10.0.3: @@ -4045,7 +4045,7 @@ packages: /@types/morgan@1.9.6: resolution: {integrity: sha512-xfKogz5WcKww2DAiVT9zxMgrqQt+Shq8tDVeLT+otoj6dJnkRkyJxMF51mHtUc3JCPKGk5x1EBU0buuGpfftlQ==} dependencies: - '@types/node': 20.8.5 + '@types/node': 20.8.6 dev: true /@types/ms@0.7.31: @@ -4054,7 +4054,7 @@ packages: /@types/node-fetch@2.6.4: resolution: {integrity: sha512-1ZX9fcN4Rvkvgv4E6PAY5WXUFWFcRWxZa3EW83UjycOB9ljJCedb2CupIP4RZMEwF/M3eTcCihbBRgwtGbg5Rg==} dependencies: - '@types/node': 20.8.6 + '@types/node': 20.8.8 form-data: 3.0.1 dev: true @@ -4065,14 +4065,14 @@ packages: /@types/node@12.20.55: resolution: {integrity: sha512-J8xLz7q2OFulZ2cyGTLE1TbbZcjpno7FaN6zdJNrgAdrJ+DZzh/uFR6YrTb4C+nXakvud8Q4+rbhoIWlYQbUFQ==} - /@types/node@20.8.5: - resolution: {integrity: sha512-SPlobFgbidfIeOYlzXiEjSYeIJiOCthv+9tSQVpvk4PAdIIc+2SmjNVzWXk9t0Y7dl73Zdf+OgXKHX9XtkqUpw==} + /@types/node@20.8.6: + resolution: {integrity: sha512-eWO4K2Ji70QzKUqRy6oyJWUeB7+g2cRagT3T/nxYibYcT4y2BDL8lqolRXjTHmkZCdJfIPaY73KbJAZmcryxTQ==} dependencies: undici-types: 5.25.3 dev: true - /@types/node@20.8.6: - resolution: {integrity: sha512-eWO4K2Ji70QzKUqRy6oyJWUeB7+g2cRagT3T/nxYibYcT4y2BDL8lqolRXjTHmkZCdJfIPaY73KbJAZmcryxTQ==} + /@types/node@20.8.8: + resolution: {integrity: sha512-YRsdVxq6OaLfmR9Hy816IMp33xOBjfyOgUd77ehqg96CFywxAPbDbXvAsuN2KVg2HOT8Eh6uAfU+l4WffwPVrQ==} dependencies: undici-types: 5.25.3 @@ -4086,7 +4086,7 @@ packages: /@types/pbkdf2@3.1.0: resolution: {integrity: sha512-Cf63Rv7jCQ0LaL8tNXmEyqTHuIJxRdlS5vMh1mj5voN4+QFhVZnlZruezqpWYDiJ8UTzhP0VmeLXCmBk66YrMQ==} dependencies: - '@types/node': 20.8.6 + '@types/node': 20.8.8 dev: true /@types/pino-multi-stream@5.1.4: @@ -4104,13 +4104,13 @@ packages: /@types/pino-std-serializers@2.4.1: resolution: {integrity: sha512-17XcksO47M24IVTVKPeAByWUd3Oez7EbIjXpSbzMPhXVzgjGtrOa49gKBwxH9hb8dKv58OelsWQ+A1G1l9S3wQ==} dependencies: - '@types/node': 20.8.6 + '@types/node': 20.8.8 dev: true /@types/pino@6.3.11: resolution: {integrity: sha512-S7+fLONqSpHeW9d7TApUqO6VN47KYgOXhCNKwGBVLHObq8HhaAYlVqUNdfnvoXjCMiwE5xcPm/5R2ZUh8bgaXQ==} dependencies: - '@types/node': 20.8.6 + '@types/node': 20.8.8 '@types/pino-pretty': 4.7.1 '@types/pino-std-serializers': 2.4.1 sonic-boom: 2.8.0 @@ -4156,7 +4156,7 @@ packages: /@types/readable-stream@2.3.15: resolution: {integrity: sha512-oM5JSKQCcICF1wvGgmecmHldZ48OZamtMxcGGVICOJA8o8cahXC1zEVAif8iwoc5j8etxFaRFnf095+CDsuoFQ==} dependencies: - '@types/node': 20.8.6 + '@types/node': 20.8.8 safe-buffer: 5.1.2 dev: true @@ -4167,7 +4167,7 @@ packages: /@types/secp256k1@4.0.3: resolution: {integrity: sha512-Da66lEIFeIz9ltsdMZcpQvmrmmoqrfju8pm1BH8WbYjZSwUgCwXLb9C+9XYogwBITnbsSaMdVPb2ekf7TV+03w==} dependencies: - '@types/node': 20.8.6 + '@types/node': 20.8.8 dev: true /@types/seedrandom@3.0.1: @@ -4186,14 +4186,14 @@ packages: resolution: {integrity: sha512-Cwo8LE/0rnvX7kIIa3QHCkcuF21c05Ayb0ZfxPiv0W8VRiZiNW/WuRupHKpqqGVGf7SUA44QSOUKaEd9lIrd/Q==} dependencies: '@types/mime': 1.3.2 - '@types/node': 20.8.6 + '@types/node': 20.8.8 dev: true /@types/serve-static@1.13.10: resolution: {integrity: sha512-nCkHGI4w7ZgAdNkrEu0bv+4xNV/XDqW+DydknebMOQwkpDGx8G+HTlj7R7ABI8i8nKxVw0wtKPi1D+lPOkh4YQ==} dependencies: '@types/mime': 1.3.2 - '@types/node': 20.8.6 + '@types/node': 20.8.8 dev: true /@types/sinon-chai@3.2.5: @@ -4230,12 +4230,12 @@ packages: /@types/ws@7.4.7: resolution: {integrity: sha512-JQbbmxZTZehdc2iszGKs5oC3NFnjeay7mtAWrdt7qNtAVK0g19muApzAy4bm9byz79xa2ZnO/BOBC2R8RC5Lww==} dependencies: - '@types/node': 20.8.6 + '@types/node': 20.8.8 /@types/ws@8.5.3: resolution: {integrity: sha512-6YOoWjruKj1uLf3INHH7D3qTXwFfEsg1kf3c0uDdSBJwfa/llkwIjrAGV7j7mVgGNbzTQ3HiHKKDXl6bJPD97w==} dependencies: - '@types/node': 20.8.6 + '@types/node': 20.8.8 dev: false /@typescript-eslint/eslint-plugin@6.7.5(@typescript-eslint/parser@6.9.0)(eslint@8.51.0)(typescript@5.2.2): @@ -9037,7 +9037,7 @@ packages: solc: 0.7.3(debug@4.3.4) source-map-support: 0.5.21 stacktrace-parser: 0.1.10 - ts-node: 10.9.1(@types/node@20.8.6)(typescript@5.2.2) + ts-node: 10.9.1(@types/node@20.8.8)(typescript@5.2.2) tsort: 0.0.1 typescript: 5.2.2 undici: 5.24.0 @@ -13795,7 +13795,7 @@ packages: tsconfig-paths: 3.14.2 dev: true - /ts-node@10.9.1(@types/node@20.8.6)(typescript@5.2.2): + /ts-node@10.9.1(@types/node@20.8.8)(typescript@5.2.2): resolution: {integrity: sha512-NtVysVPkxxrwFGUUxGYhfux8k78pQB3JqYBXlLRZgdGUqTO5wU/UyHop5p70iEbGhB7q5KmiZiU0Y3KlJrScEw==} hasBin: true peerDependencies: @@ -13814,7 +13814,7 @@ packages: '@tsconfig/node12': 1.0.11 '@tsconfig/node14': 1.0.3 '@tsconfig/node16': 1.0.4 - '@types/node': 20.8.6 + '@types/node': 20.8.8 acorn: 8.10.0 acorn-walk: 8.2.0 arg: 4.1.3 @@ -14415,7 +14415,7 @@ packages: - utf-8-validate - zod - /vite-node@0.34.1(@types/node@20.8.6): + /vite-node@0.34.1(@types/node@20.8.8): resolution: {integrity: sha512-odAZAL9xFMuAg8aWd7nSPT+hU8u2r9gU3LRm9QKjxBEF2rRdWpMuqkrkjvyVQEdNFiBctqr2Gg4uJYizm5Le6w==} engines: {node: '>=v14.18.0'} hasBin: true @@ -14425,7 +14425,7 @@ packages: mlly: 1.4.0 pathe: 1.1.1 picocolors: 1.0.0 - vite: 4.4.10(@types/node@20.8.6) + vite: 4.4.10(@types/node@20.8.8) transitivePeerDependencies: - '@types/node' - less @@ -14437,7 +14437,7 @@ packages: - terser dev: true - /vite-node@0.34.2(@types/node@20.8.5): + /vite-node@0.34.2(@types/node@20.8.6): resolution: {integrity: sha512-JtW249Zm3FB+F7pQfH56uWSdlltCo1IOkZW5oHBzeQo0iX4jtC7o1t9aILMGd9kVekXBP2lfJBEQt9rBh07ebA==} engines: {node: '>=v14.18.0'} hasBin: true @@ -14447,7 +14447,7 @@ packages: mlly: 1.4.0 pathe: 1.1.1 picocolors: 1.0.0 - vite: 4.4.10(@types/node@20.8.5) + vite: 4.4.10(@types/node@20.8.6) transitivePeerDependencies: - '@types/node' - less @@ -14459,7 +14459,7 @@ packages: - terser dev: true - /vite-node@0.34.2(@types/node@20.8.6): + /vite-node@0.34.2(@types/node@20.8.8): resolution: {integrity: sha512-JtW249Zm3FB+F7pQfH56uWSdlltCo1IOkZW5oHBzeQo0iX4jtC7o1t9aILMGd9kVekXBP2lfJBEQt9rBh07ebA==} engines: {node: '>=v14.18.0'} hasBin: true @@ -14469,7 +14469,7 @@ packages: mlly: 1.4.0 pathe: 1.1.1 picocolors: 1.0.0 - vite: 4.4.10(@types/node@20.8.6) + vite: 4.4.10(@types/node@20.8.8) transitivePeerDependencies: - '@types/node' - less @@ -14481,7 +14481,7 @@ packages: - terser dev: true - /vite-node@0.34.4(@types/node@20.8.5): + /vite-node@0.34.4(@types/node@20.8.6): resolution: {integrity: sha512-ho8HtiLc+nsmbwZMw8SlghESEE3KxJNp04F/jPUCLVvaURwt0d+r9LxEqCX5hvrrOQ0GSyxbYr5ZfRYhQ0yVKQ==} engines: {node: '>=v14.18.0'} hasBin: true @@ -14491,7 +14491,7 @@ packages: mlly: 1.4.0 pathe: 1.1.1 picocolors: 1.0.0 - vite: 4.4.10(@types/node@20.8.5) + vite: 4.4.10(@types/node@20.8.6) transitivePeerDependencies: - '@types/node' - less @@ -14503,7 +14503,7 @@ packages: - terser dev: true - /vite@4.4.10(@types/node@20.8.5): + /vite@4.4.10(@types/node@20.8.6): resolution: {integrity: sha512-TzIjiqx9BEXF8yzYdF2NTf1kFFbjMjUSV0LFZ3HyHoI3SGSPLnnFUKiIQtL3gl2AjHvMrprOvQ3amzaHgQlAxw==} engines: {node: ^14.18.0 || >=16.0.0} hasBin: true @@ -14531,7 +14531,7 @@ packages: terser: optional: true dependencies: - '@types/node': 20.8.5 + '@types/node': 20.8.6 esbuild: 0.18.15 postcss: 8.4.27 rollup: 3.28.0 @@ -14539,7 +14539,7 @@ packages: fsevents: 2.3.3 dev: true - /vite@4.4.10(@types/node@20.8.6): + /vite@4.4.10(@types/node@20.8.8): resolution: {integrity: sha512-TzIjiqx9BEXF8yzYdF2NTf1kFFbjMjUSV0LFZ3HyHoI3SGSPLnnFUKiIQtL3gl2AjHvMrprOvQ3amzaHgQlAxw==} engines: {node: ^14.18.0 || >=16.0.0} hasBin: true @@ -14567,7 +14567,7 @@ packages: terser: optional: true dependencies: - '@types/node': 20.8.6 + '@types/node': 20.8.8 esbuild: 0.18.15 postcss: 8.4.27 rollup: 3.28.0 @@ -14575,7 +14575,7 @@ packages: fsevents: 2.3.3 dev: true - /vite@4.4.9(@types/node@20.8.5): + /vite@4.4.9(@types/node@20.8.6): resolution: {integrity: sha512-2mbUn2LlUmNASWwSCNSJ/EG2HuSRTnVNaydp6vMCm5VIqJsjMfbIWtbH2kDuwUVW5mMUKKZvGPX/rqeqVvv1XA==} engines: {node: ^14.18.0 || >=16.0.0} hasBin: true @@ -14603,7 +14603,7 @@ packages: terser: optional: true dependencies: - '@types/node': 20.8.5 + '@types/node': 20.8.6 esbuild: 0.18.20 postcss: 8.4.27 rollup: 3.28.0 @@ -14644,7 +14644,7 @@ packages: dependencies: '@types/chai': 4.3.8 '@types/chai-subset': 1.3.3 - '@types/node': 20.8.6 + '@types/node': 20.8.8 '@vitest/expect': 0.34.1 '@vitest/runner': 0.34.1 '@vitest/snapshot': 0.34.1 @@ -14663,8 +14663,8 @@ packages: strip-literal: 1.0.1 tinybench: 2.5.0 tinypool: 0.7.0 - vite: 4.4.10(@types/node@20.8.6) - vite-node: 0.34.1(@types/node@20.8.6) + vite: 4.4.10(@types/node@20.8.8) + vite-node: 0.34.1(@types/node@20.8.8) why-is-node-running: 2.2.2 transitivePeerDependencies: - less @@ -14709,7 +14709,7 @@ packages: dependencies: '@types/chai': 4.3.8 '@types/chai-subset': 1.3.3 - '@types/node': 20.8.6 + '@types/node': 20.8.8 '@vitest/expect': 0.34.2 '@vitest/runner': 0.34.2 '@vitest/snapshot': 0.34.2 @@ -14728,8 +14728,8 @@ packages: strip-literal: 1.0.1 tinybench: 2.5.0 tinypool: 0.7.0 - vite: 4.4.10(@types/node@20.8.6) - vite-node: 0.34.2(@types/node@20.8.6) + vite: 4.4.10(@types/node@20.8.8) + vite-node: 0.34.2(@types/node@20.8.8) why-is-node-running: 2.2.2 transitivePeerDependencies: - less @@ -14774,7 +14774,7 @@ packages: dependencies: '@types/chai': 4.3.7 '@types/chai-subset': 1.3.3 - '@types/node': 20.8.5 + '@types/node': 20.8.6 '@vitest/expect': 0.34.2 '@vitest/runner': 0.34.2 '@vitest/snapshot': 0.34.2 @@ -14794,8 +14794,8 @@ packages: strip-literal: 1.0.1 tinybench: 2.5.0 tinypool: 0.7.0 - vite: 4.4.10(@types/node@20.8.5) - vite-node: 0.34.2(@types/node@20.8.5) + vite: 4.4.10(@types/node@20.8.6) + vite-node: 0.34.2(@types/node@20.8.6) why-is-node-running: 2.2.2 transitivePeerDependencies: - less @@ -14840,7 +14840,7 @@ packages: dependencies: '@types/chai': 4.3.7 '@types/chai-subset': 1.3.3 - '@types/node': 20.8.5 + '@types/node': 20.8.6 '@vitest/expect': 0.34.4 '@vitest/runner': 0.34.4 '@vitest/snapshot': 0.34.4 @@ -14859,8 +14859,8 @@ packages: strip-literal: 1.0.1 tinybench: 2.5.0 tinypool: 0.7.0 - vite: 4.4.9(@types/node@20.8.5) - vite-node: 0.34.4(@types/node@20.8.5) + vite: 4.4.9(@types/node@20.8.6) + vite-node: 0.34.4(@types/node@20.8.6) why-is-node-running: 2.2.2 transitivePeerDependencies: - less From 98eb885f5003ee5e6b9bbd532a42bba2ad39cb4b Mon Sep 17 00:00:00 2001 From: Will Cory Date: Tue, 24 Oct 2023 13:26:47 -0700 Subject: [PATCH 044/279] chore: Update node version to 20 LTS --- .changeset/clean-guests-guess.md | 5 +++++ .nvmrc | 2 +- ops/docker/Dockerfile.packages | 2 +- 3 files changed, 7 insertions(+), 2 deletions(-) create mode 100644 .changeset/clean-guests-guess.md diff --git a/.changeset/clean-guests-guess.md b/.changeset/clean-guests-guess.md new file mode 100644 index 000000000000..9c5a9986951a --- /dev/null +++ b/.changeset/clean-guests-guess.md @@ -0,0 +1,5 @@ +--- +'@eth-optimism/chain-mon': patch +--- + +Bump node version to LTS node 20.9.0 diff --git a/.nvmrc b/.nvmrc index 860cc5000ae6..805b5a4e00bf 100644 --- a/.nvmrc +++ b/.nvmrc @@ -1 +1 @@ -v18.17.1 +v20.9.0 diff --git a/ops/docker/Dockerfile.packages b/ops/docker/Dockerfile.packages index 2fdf67bde525..cb82dc365011 100644 --- a/ops/docker/Dockerfile.packages +++ b/ops/docker/Dockerfile.packages @@ -35,7 +35,7 @@ FROM us-docker.pkg.dev/oplabs-tools-artifacts/images/ci-builder:latest as foundr # we use it rather than alpine because it's not much # bigger and alpine is often missing packages for node applications # alpine is not officially supported by node.js -FROM node:18.17.1-bullseye-slim as base +FROM node:20.9.0-bullseye-slim as base # Base: install deps RUN apt-get update && apt-get install -y \ From a71b4c63fea32c4b8425d5c411d61c99347d1d02 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 24 Oct 2023 20:26:56 +0000 Subject: [PATCH 045/279] build(deps-dev): bump eslint from 8.51.0 to 8.52.0 Bumps [eslint](https://github.com/eslint/eslint) from 8.51.0 to 8.52.0. - [Release notes](https://github.com/eslint/eslint/releases) - [Changelog](https://github.com/eslint/eslint/blob/main/CHANGELOG.md) - [Commits](https://github.com/eslint/eslint/compare/v8.51.0...v8.52.0) --- updated-dependencies: - dependency-name: eslint dependency-type: direct:development update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] --- package.json | 2 +- pnpm-lock.yaml | 270 ++++++++++++++++++++++++++++++++++++++++--------- 2 files changed, 221 insertions(+), 51 deletions(-) diff --git a/package.json b/package.json index 374220207702..f5ba1bc45106 100644 --- a/package.json +++ b/package.json @@ -49,7 +49,7 @@ "chai": "^4.3.10", "depcheck": "^1.4.7", "doctoc": "^2.2.0", - "eslint": "^8.51.0", + "eslint": "^8.52.0", "eslint-config-prettier": "^8.3.0", "eslint-config-standard": "^16.0.3", "eslint-plugin-import": "^2.29.0", diff --git a/pnpm-lock.yaml b/pnpm-lock.yaml index 8292e5ba7ab3..9e5b0e42e151 100644 --- a/pnpm-lock.yaml +++ b/pnpm-lock.yaml @@ -17,7 +17,7 @@ importers: devDependencies: '@babel/eslint-parser': specifier: ^7.18.2 - version: 7.22.15(@babel/core@7.22.10)(eslint@8.51.0) + version: 7.22.15(@babel/core@7.22.10)(eslint@8.52.0) '@changesets/changelog-github': specifier: ^0.4.8 version: 0.4.8 @@ -35,10 +35,10 @@ importers: version: 20.8.6 '@typescript-eslint/eslint-plugin': specifier: ^6.7.5 - version: 6.7.5(@typescript-eslint/parser@6.9.0)(eslint@8.51.0)(typescript@5.2.2) + version: 6.7.5(@typescript-eslint/parser@6.9.0)(eslint@8.52.0)(typescript@5.2.2) '@typescript-eslint/parser': specifier: ^6.9.0 - version: 6.9.0(eslint@8.51.0)(typescript@5.2.2) + version: 6.9.0(eslint@8.52.0)(typescript@5.2.2) chai: specifier: ^4.3.10 version: 4.3.10 @@ -49,38 +49,38 @@ importers: specifier: ^2.2.0 version: 2.2.1 eslint: - specifier: ^8.51.0 - version: 8.51.0 + specifier: ^8.52.0 + version: 8.52.0 eslint-config-prettier: specifier: ^8.3.0 - version: 8.3.0(eslint@8.51.0) + version: 8.3.0(eslint@8.52.0) eslint-config-standard: specifier: ^16.0.3 - version: 16.0.3(eslint-plugin-import@2.29.0)(eslint-plugin-node@11.1.0)(eslint-plugin-promise@5.2.0)(eslint@8.51.0) + version: 16.0.3(eslint-plugin-import@2.29.0)(eslint-plugin-node@11.1.0)(eslint-plugin-promise@5.2.0)(eslint@8.52.0) eslint-plugin-import: specifier: ^2.29.0 - version: 2.29.0(@typescript-eslint/parser@6.9.0)(eslint@8.51.0) + version: 2.29.0(@typescript-eslint/parser@6.9.0)(eslint@8.52.0) eslint-plugin-jsdoc: specifier: ^35.1.2 - version: 35.5.1(eslint@8.51.0) + version: 35.5.1(eslint@8.52.0) eslint-plugin-node: specifier: ^11.1.0 - version: 11.1.0(eslint@8.51.0) + version: 11.1.0(eslint@8.52.0) eslint-plugin-prefer-arrow: specifier: ^1.2.3 - version: 1.2.3(eslint@8.51.0) + version: 1.2.3(eslint@8.52.0) eslint-plugin-prettier: specifier: ^4.0.0 - version: 4.2.1(eslint-config-prettier@8.3.0)(eslint@8.51.0)(prettier@2.8.8) + version: 4.2.1(eslint-config-prettier@8.3.0)(eslint@8.52.0)(prettier@2.8.8) eslint-plugin-promise: specifier: ^5.1.0 - version: 5.2.0(eslint@8.51.0) + version: 5.2.0(eslint@8.52.0) eslint-plugin-react: specifier: ^7.24.0 - version: 7.33.2(eslint@8.51.0) + version: 7.33.2(eslint@8.52.0) eslint-plugin-unicorn: specifier: ^48.0.1 - version: 48.0.1(eslint@8.51.0) + version: 48.0.1(eslint@8.52.0) husky: specifier: ^8.0.3 version: 8.0.3 @@ -646,7 +646,7 @@ packages: - supports-color dev: true - /@babel/eslint-parser@7.22.15(@babel/core@7.22.10)(eslint@8.51.0): + /@babel/eslint-parser@7.22.15(@babel/core@7.22.10)(eslint@8.52.0): resolution: {integrity: sha512-yc8OOBIQk1EcRrpizuARSQS0TWAcOMpEJ1aafhNznaeYkeL+OhqnDObGFylB8ka8VFF/sZc+S4RzHyO+3LjQxg==} engines: {node: ^10.13.0 || ^12.13.0 || >=14.0.0} peerDependencies: @@ -655,7 +655,7 @@ packages: dependencies: '@babel/core': 7.22.10 '@nicolo-ribaudo/eslint-scope-5-internals': 5.1.1-v1 - eslint: 8.51.0 + eslint: 8.52.0 eslint-visitor-keys: 2.1.0 semver: 6.3.1 dev: true @@ -1833,6 +1833,16 @@ packages: eslint-visitor-keys: 3.4.3 dev: true + /@eslint-community/eslint-utils@4.4.0(eslint@8.52.0): + resolution: {integrity: sha512-1/sA4dwrzBAyeUoQ6oxahHKmrZvsnLCg4RfxW3ZFGGmQkSNQPFNLV9CUEFQP1x9EYXHTo5p6xdhZM1Ne9p/AfA==} + engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0} + peerDependencies: + eslint: ^6.0.0 || ^7.0.0 || >=8.0.0 + dependencies: + eslint: 8.52.0 + eslint-visitor-keys: 3.4.3 + dev: true + /@eslint-community/regexpp@4.6.2: resolution: {integrity: sha512-pPTNuaAG3QMH+buKyBIGJs3g/S5y0caxw0ygM3YyE6yJFySwiGGSzA+mM3KJ8QQvzeLh3blwgSonkFjgQdxzMw==} engines: {node: ^12.0.0 || ^14.0.0 || >=16.0.0} @@ -1860,6 +1870,11 @@ packages: engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0} dev: true + /@eslint/js@8.52.0: + resolution: {integrity: sha512-mjZVbpaeMZludF2fsWLD0Z9gCref1Tk4i9+wddjRvpUNqqcndPkBD09N/Mapey0b3jaXbLm2kICwFv2E64QinA==} + engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0} + dev: true + /@eth-optimism/contracts-periphery@1.0.8: resolution: {integrity: sha512-n8a9rmlMxl1lWSiC1zHUlr5Qk6qy85nhsmSgpU12El1WY75MOIPknSTQKj+yJhEmrTtI0PViWlKfgviY09pwUg==} dev: false @@ -2439,6 +2454,17 @@ packages: - supports-color dev: true + /@humanwhocodes/config-array@0.11.13: + resolution: {integrity: sha512-JSBDMiDKSzQVngfRjOdFXgFfklaXI4K9nLF49Auh21lmBWRLIK3+xTErTWD4KU54pb6coM6ESE7Awz/FNU3zgQ==} + engines: {node: '>=10.10.0'} + dependencies: + '@humanwhocodes/object-schema': 2.0.1 + debug: 4.3.4(supports-color@8.1.1) + minimatch: 3.1.2 + transitivePeerDependencies: + - supports-color + dev: true + /@humanwhocodes/module-importer@1.0.1: resolution: {integrity: sha512-bxveV4V8v5Yb4ncFTT3rPSgZBOpCkjfK0y4oVVVJwIuDVBRMDXrPyXRL988i5ap9m9bnyEEjWfm5WkBmtffLfA==} engines: {node: '>=12.22'} @@ -2448,6 +2474,10 @@ packages: resolution: {integrity: sha512-ZnQMnLV4e7hDlUvw8H+U8ASL02SS2Gn6+9Ac3wGGLIe7+je2AeAOxPY+izIPJDfFDb7eDjev0Us8MO1iFRN8hA==} dev: true + /@humanwhocodes/object-schema@2.0.1: + resolution: {integrity: sha512-dvuCeX5fC9dXgJn9t+X5atfmgQAzUOWqS1254Gh0m6i8wKd10ebXkfNKiRK+1GWi/yTvvLDHpoxLr0xxxeslWw==} + dev: true + /@isaacs/cliui@8.0.2: resolution: {integrity: sha512-O8jcjabXaleOG9DQ0+ARXWZBTfnP4WNAqzuiJK7ll44AmxGKv/J2M4TPjxjY3znBCfvBXFzucm1twdyFybFqEA==} engines: {node: '>=12'} @@ -4267,6 +4297,35 @@ packages: - supports-color dev: true + /@typescript-eslint/eslint-plugin@6.7.5(@typescript-eslint/parser@6.9.0)(eslint@8.52.0)(typescript@5.2.2): + resolution: {integrity: sha512-JhtAwTRhOUcP96D0Y6KYnwig/MRQbOoLGXTON2+LlyB/N35SP9j1boai2zzwXb7ypKELXMx3DVk9UTaEq1vHEw==} + engines: {node: ^16.0.0 || >=18.0.0} + peerDependencies: + '@typescript-eslint/parser': ^6.0.0 || ^6.0.0-alpha + eslint: ^7.0.0 || ^8.0.0 + typescript: '*' + peerDependenciesMeta: + typescript: + optional: true + dependencies: + '@eslint-community/regexpp': 4.6.2 + '@typescript-eslint/parser': 6.9.0(eslint@8.52.0)(typescript@5.2.2) + '@typescript-eslint/scope-manager': 6.7.5 + '@typescript-eslint/type-utils': 6.7.5(eslint@8.52.0)(typescript@5.2.2) + '@typescript-eslint/utils': 6.7.5(eslint@8.52.0)(typescript@5.2.2) + '@typescript-eslint/visitor-keys': 6.7.5 + debug: 4.3.4(supports-color@8.1.1) + eslint: 8.52.0 + graphemer: 1.4.0 + ignore: 5.2.4 + natural-compare: 1.4.0 + semver: 7.5.4 + ts-api-utils: 1.0.1(typescript@5.2.2) + typescript: 5.2.2 + transitivePeerDependencies: + - supports-color + dev: true + /@typescript-eslint/parser@6.9.0(eslint@8.51.0)(typescript@5.2.2): resolution: {integrity: sha512-GZmjMh4AJ/5gaH4XF2eXA8tMnHWP+Pm1mjQR2QN4Iz+j/zO04b9TOvJYOX2sCNIQHtRStKTxRY1FX7LhpJT4Gw==} engines: {node: ^16.0.0 || >=18.0.0} @@ -4288,6 +4347,27 @@ packages: - supports-color dev: true + /@typescript-eslint/parser@6.9.0(eslint@8.52.0)(typescript@5.2.2): + resolution: {integrity: sha512-GZmjMh4AJ/5gaH4XF2eXA8tMnHWP+Pm1mjQR2QN4Iz+j/zO04b9TOvJYOX2sCNIQHtRStKTxRY1FX7LhpJT4Gw==} + engines: {node: ^16.0.0 || >=18.0.0} + peerDependencies: + eslint: ^7.0.0 || ^8.0.0 + typescript: '*' + peerDependenciesMeta: + typescript: + optional: true + dependencies: + '@typescript-eslint/scope-manager': 6.9.0 + '@typescript-eslint/types': 6.9.0 + '@typescript-eslint/typescript-estree': 6.9.0(typescript@5.2.2) + '@typescript-eslint/visitor-keys': 6.9.0 + debug: 4.3.4(supports-color@8.1.1) + eslint: 8.52.0 + typescript: 5.2.2 + transitivePeerDependencies: + - supports-color + dev: true + /@typescript-eslint/scope-manager@6.7.5: resolution: {integrity: sha512-GAlk3eQIwWOJeb9F7MKQ6Jbah/vx1zETSDw8likab/eFcqkjSD7BI75SDAeC5N2L0MmConMoPvTsmkrg71+B1A==} engines: {node: ^16.0.0 || >=18.0.0} @@ -4324,6 +4404,26 @@ packages: - supports-color dev: true + /@typescript-eslint/type-utils@6.7.5(eslint@8.52.0)(typescript@5.2.2): + resolution: {integrity: sha512-Gs0qos5wqxnQrvpYv+pf3XfcRXW6jiAn9zE/K+DlmYf6FcpxeNYN0AIETaPR7rHO4K2UY+D0CIbDP9Ut0U4m1g==} + engines: {node: ^16.0.0 || >=18.0.0} + peerDependencies: + eslint: ^7.0.0 || ^8.0.0 + typescript: '*' + peerDependenciesMeta: + typescript: + optional: true + dependencies: + '@typescript-eslint/typescript-estree': 6.7.5(typescript@5.2.2) + '@typescript-eslint/utils': 6.7.5(eslint@8.52.0)(typescript@5.2.2) + debug: 4.3.4(supports-color@8.1.1) + eslint: 8.52.0 + ts-api-utils: 1.0.1(typescript@5.2.2) + typescript: 5.2.2 + transitivePeerDependencies: + - supports-color + dev: true + /@typescript-eslint/types@6.7.5: resolution: {integrity: sha512-WboQBlOXtdj1tDFPyIthpKrUb+kZf2VroLZhxKa/VlwLlLyqv/PwUNgL30BlTVZV1Wu4Asu2mMYPqarSO4L5ZQ==} engines: {node: ^16.0.0 || >=18.0.0} @@ -4395,6 +4495,25 @@ packages: - typescript dev: true + /@typescript-eslint/utils@6.7.5(eslint@8.52.0)(typescript@5.2.2): + resolution: {integrity: sha512-pfRRrH20thJbzPPlPc4j0UNGvH1PjPlhlCMq4Yx7EGjV7lvEeGX0U6MJYe8+SyFutWgSHsdbJ3BXzZccYggezA==} + engines: {node: ^16.0.0 || >=18.0.0} + peerDependencies: + eslint: ^7.0.0 || ^8.0.0 + dependencies: + '@eslint-community/eslint-utils': 4.4.0(eslint@8.52.0) + '@types/json-schema': 7.0.12 + '@types/semver': 7.5.0 + '@typescript-eslint/scope-manager': 6.7.5 + '@typescript-eslint/types': 6.7.5 + '@typescript-eslint/typescript-estree': 6.7.5(typescript@5.2.2) + eslint: 8.52.0 + semver: 7.5.4 + transitivePeerDependencies: + - supports-color + - typescript + dev: true + /@typescript-eslint/visitor-keys@6.7.5: resolution: {integrity: sha512-3MaWdDZtLlsexZzDSdQWsFQ9l9nL8B80Z4fImSpyllFC/KLqWQRdEcB+gGGO+N3Q2uL40EsG66wZLsohPxNXvg==} engines: {node: ^16.0.0 || >=18.0.0} @@ -4411,6 +4530,10 @@ packages: eslint-visitor-keys: 3.4.3 dev: true + /@ungap/structured-clone@1.2.0: + resolution: {integrity: sha512-zuVdFrMJiuCDQUMCzQaD6KL28MjnqqN8XnAqiEq9PNm/hCPTSGfrXCOfwj1ow4LFb/tNymJPwsNbVePc1xFqrQ==} + dev: true + /@vitest/coverage-istanbul@0.34.6(vitest@0.34.1): resolution: {integrity: sha512-5KaBNZPDSk2ybavC3rZ1pWGniw7sJ5usuwVGRUYzJwiBfWvnLpuUer7bjw7qUCRGdKJXrBgb/Dsgif9rkwMX/A==} peerDependencies: @@ -7532,16 +7655,16 @@ packages: engines: {node: '>=10'} dev: true - /eslint-config-prettier@8.3.0(eslint@8.51.0): + /eslint-config-prettier@8.3.0(eslint@8.52.0): resolution: {integrity: sha512-BgZuLUSeKzvlL/VUjx/Yb787VQ26RU3gGjA3iiFvdsp/2bMfVIWUVP7tjxtjS0e+HP409cPlPvNkQloz8C91ew==} hasBin: true peerDependencies: eslint: '>=7.0.0' dependencies: - eslint: 8.51.0 + eslint: 8.52.0 dev: true - /eslint-config-standard@16.0.3(eslint-plugin-import@2.29.0)(eslint-plugin-node@11.1.0)(eslint-plugin-promise@5.2.0)(eslint@8.51.0): + /eslint-config-standard@16.0.3(eslint-plugin-import@2.29.0)(eslint-plugin-node@11.1.0)(eslint-plugin-promise@5.2.0)(eslint@8.52.0): resolution: {integrity: sha512-x4fmJL5hGqNJKGHSjnLdgA6U6h1YW/G2dW9fA+cyVur4SK6lyue8+UgNKWlZtUDTXvgKDD/Oa3GQjmB5kjtVvg==} peerDependencies: eslint: ^7.12.1 @@ -7549,10 +7672,10 @@ packages: eslint-plugin-node: ^11.1.0 eslint-plugin-promise: ^4.2.1 || ^5.0.0 dependencies: - eslint: 8.51.0 - eslint-plugin-import: 2.29.0(@typescript-eslint/parser@6.9.0)(eslint@8.51.0) - eslint-plugin-node: 11.1.0(eslint@8.51.0) - eslint-plugin-promise: 5.2.0(eslint@8.51.0) + eslint: 8.52.0 + eslint-plugin-import: 2.29.0(@typescript-eslint/parser@6.9.0)(eslint@8.52.0) + eslint-plugin-node: 11.1.0(eslint@8.52.0) + eslint-plugin-promise: 5.2.0(eslint@8.52.0) dev: true /eslint-import-resolver-node@0.3.9: @@ -7565,7 +7688,7 @@ packages: - supports-color dev: true - /eslint-module-utils@2.8.0(@typescript-eslint/parser@6.9.0)(eslint-import-resolver-node@0.3.9)(eslint@8.51.0): + /eslint-module-utils@2.8.0(@typescript-eslint/parser@6.9.0)(eslint-import-resolver-node@0.3.9)(eslint@8.52.0): resolution: {integrity: sha512-aWajIYfsqCKRDgUfjEXNN/JlrzauMuSEy5sbd7WXbtW3EH6A6MpwEh42c7qD+MqQo9QMJ6fWLAeIJynx0g6OAw==} engines: {node: '>=4'} peerDependencies: @@ -7586,26 +7709,26 @@ packages: eslint-import-resolver-webpack: optional: true dependencies: - '@typescript-eslint/parser': 6.9.0(eslint@8.51.0)(typescript@5.2.2) + '@typescript-eslint/parser': 6.9.0(eslint@8.52.0)(typescript@5.2.2) debug: 3.2.7 - eslint: 8.51.0 + eslint: 8.52.0 eslint-import-resolver-node: 0.3.9 transitivePeerDependencies: - supports-color dev: true - /eslint-plugin-es@3.0.1(eslint@8.51.0): + /eslint-plugin-es@3.0.1(eslint@8.52.0): resolution: {integrity: sha512-GUmAsJaN4Fc7Gbtl8uOBlayo2DqhwWvEzykMHSCZHU3XdJ+NSzzZcVhXh3VxX5icqQ+oQdIEawXX8xkR3mIFmQ==} engines: {node: '>=8.10.0'} peerDependencies: eslint: '>=4.19.1' dependencies: - eslint: 8.51.0 + eslint: 8.52.0 eslint-utils: 2.1.0 regexpp: 3.2.0 dev: true - /eslint-plugin-import@2.29.0(@typescript-eslint/parser@6.9.0)(eslint@8.51.0): + /eslint-plugin-import@2.29.0(@typescript-eslint/parser@6.9.0)(eslint@8.52.0): resolution: {integrity: sha512-QPOO5NO6Odv5lpoTkddtutccQjysJuFxoPS7fAHO+9m9udNHvTCPSAMW9zGAYj8lAIdr40I8yPCdUYrncXtrwg==} engines: {node: '>=4'} peerDependencies: @@ -7615,16 +7738,16 @@ packages: '@typescript-eslint/parser': optional: true dependencies: - '@typescript-eslint/parser': 6.9.0(eslint@8.51.0)(typescript@5.2.2) + '@typescript-eslint/parser': 6.9.0(eslint@8.52.0)(typescript@5.2.2) array-includes: 3.1.7 array.prototype.findlastindex: 1.2.3 array.prototype.flat: 1.3.2 array.prototype.flatmap: 1.3.2 debug: 3.2.7 doctrine: 2.1.0 - eslint: 8.51.0 + eslint: 8.52.0 eslint-import-resolver-node: 0.3.9 - eslint-module-utils: 2.8.0(@typescript-eslint/parser@6.9.0)(eslint-import-resolver-node@0.3.9)(eslint@8.51.0) + eslint-module-utils: 2.8.0(@typescript-eslint/parser@6.9.0)(eslint-import-resolver-node@0.3.9)(eslint@8.52.0) hasown: 2.0.0 is-core-module: 2.13.1 is-glob: 4.0.3 @@ -7640,7 +7763,7 @@ packages: - supports-color dev: true - /eslint-plugin-jsdoc@35.5.1(eslint@8.51.0): + /eslint-plugin-jsdoc@35.5.1(eslint@8.52.0): resolution: {integrity: sha512-pPYPWtsykwVEue1tYEyoppBj4dgF7XicF67tLLLraY6RQYBq7qMKjUHji19+hfiTtYKKBD0YfeK8hgjPAE5viw==} engines: {node: '>=12'} peerDependencies: @@ -7649,7 +7772,7 @@ packages: '@es-joy/jsdoccomment': 0.9.0-alpha.1 comment-parser: 1.1.6-beta.0 debug: 4.3.4(supports-color@8.1.1) - eslint: 8.51.0 + eslint: 8.52.0 esquery: 1.4.0 jsdoc-type-pratt-parser: 1.1.1 lodash: 4.17.21 @@ -7660,14 +7783,14 @@ packages: - supports-color dev: true - /eslint-plugin-node@11.1.0(eslint@8.51.0): + /eslint-plugin-node@11.1.0(eslint@8.52.0): resolution: {integrity: sha512-oUwtPJ1W0SKD0Tr+wqu92c5xuCeQqB3hSCHasn/ZgjFdA9iDGNkNf2Zi9ztY7X+hNuMib23LNGRm6+uN+KLE3g==} engines: {node: '>=8.10.0'} peerDependencies: eslint: '>=5.16.0' dependencies: - eslint: 8.51.0 - eslint-plugin-es: 3.0.1(eslint@8.51.0) + eslint: 8.52.0 + eslint-plugin-es: 3.0.1(eslint@8.52.0) eslint-utils: 2.1.0 ignore: 5.2.4 minimatch: 3.1.2 @@ -7675,15 +7798,15 @@ packages: semver: 6.3.1 dev: true - /eslint-plugin-prefer-arrow@1.2.3(eslint@8.51.0): + /eslint-plugin-prefer-arrow@1.2.3(eslint@8.52.0): resolution: {integrity: sha512-J9I5PKCOJretVuiZRGvPQxCbllxGAV/viI20JO3LYblAodofBxyMnZAJ+WGeClHgANnSJberTNoFWWjrWKBuXQ==} peerDependencies: eslint: '>=2.0.0' dependencies: - eslint: 8.51.0 + eslint: 8.52.0 dev: true - /eslint-plugin-prettier@4.2.1(eslint-config-prettier@8.3.0)(eslint@8.51.0)(prettier@2.8.8): + /eslint-plugin-prettier@4.2.1(eslint-config-prettier@8.3.0)(eslint@8.52.0)(prettier@2.8.8): resolution: {integrity: sha512-f/0rXLXUt0oFYs8ra4w49wYZBG5GKZpAYsJSm6rnYL5uVDjd+zowwMwVZHnAjf4edNrKpCDYfXDgmRE/Ak7QyQ==} engines: {node: '>=12.0.0'} peerDependencies: @@ -7694,22 +7817,22 @@ packages: eslint-config-prettier: optional: true dependencies: - eslint: 8.51.0 - eslint-config-prettier: 8.3.0(eslint@8.51.0) + eslint: 8.52.0 + eslint-config-prettier: 8.3.0(eslint@8.52.0) prettier: 2.8.8 prettier-linter-helpers: 1.0.0 dev: true - /eslint-plugin-promise@5.2.0(eslint@8.51.0): + /eslint-plugin-promise@5.2.0(eslint@8.52.0): resolution: {integrity: sha512-SftLb1pUG01QYq2A/hGAWfDRXqYD82zE7j7TopDOyNdU+7SvvoXREls/+PRTY17vUXzXnZA/zfnyKgRH6x4JJw==} engines: {node: ^10.12.0 || >=12.0.0} peerDependencies: eslint: ^7.0.0 dependencies: - eslint: 8.51.0 + eslint: 8.52.0 dev: true - /eslint-plugin-react@7.33.2(eslint@8.51.0): + /eslint-plugin-react@7.33.2(eslint@8.52.0): resolution: {integrity: sha512-73QQMKALArI8/7xGLNI/3LylrEYrlKZSb5C9+q3OtOewTnMQi5cT+aE9E41sLCmli3I9PGGmD1yiZydyo4FEPw==} engines: {node: '>=4'} peerDependencies: @@ -7720,7 +7843,7 @@ packages: array.prototype.tosorted: 1.1.1 doctrine: 2.1.0 es-iterator-helpers: 1.0.13 - eslint: 8.51.0 + eslint: 8.52.0 estraverse: 5.3.0 jsx-ast-utils: 3.2.0 minimatch: 3.1.2 @@ -7734,17 +7857,17 @@ packages: string.prototype.matchall: 4.0.8 dev: true - /eslint-plugin-unicorn@48.0.1(eslint@8.51.0): + /eslint-plugin-unicorn@48.0.1(eslint@8.52.0): resolution: {integrity: sha512-FW+4r20myG/DqFcCSzoumaddKBicIPeFnTrifon2mWIzlfyvzwyqZjqVP7m4Cqr/ZYisS2aiLghkUWaPg6vtCw==} engines: {node: '>=16'} peerDependencies: eslint: '>=8.44.0' dependencies: '@babel/helper-validator-identifier': 7.22.5 - '@eslint-community/eslint-utils': 4.4.0(eslint@8.51.0) + '@eslint-community/eslint-utils': 4.4.0(eslint@8.52.0) ci-info: 3.8.0 clean-regexp: 1.0.0 - eslint: 8.51.0 + eslint: 8.52.0 esquery: 1.5.0 indent-string: 4.0.0 is-builtin-module: 3.2.1 @@ -7842,6 +7965,53 @@ packages: - supports-color dev: true + /eslint@8.52.0: + resolution: {integrity: sha512-zh/JHnaixqHZsolRB/w9/02akBk9EPrOs9JwcTP2ek7yL5bVvXuRariiaAjjoJ5DvuwQ1WAE/HsMz+w17YgBCg==} + engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0} + hasBin: true + dependencies: + '@eslint-community/eslint-utils': 4.4.0(eslint@8.52.0) + '@eslint-community/regexpp': 4.6.2 + '@eslint/eslintrc': 2.1.2 + '@eslint/js': 8.52.0 + '@humanwhocodes/config-array': 0.11.13 + '@humanwhocodes/module-importer': 1.0.1 + '@nodelib/fs.walk': 1.2.8 + '@ungap/structured-clone': 1.2.0 + ajv: 6.12.6 + chalk: 4.1.2 + cross-spawn: 7.0.3 + debug: 4.3.4(supports-color@8.1.1) + doctrine: 3.0.0 + escape-string-regexp: 4.0.0 + eslint-scope: 7.2.2 + eslint-visitor-keys: 3.4.3 + espree: 9.6.1 + esquery: 1.5.0 + esutils: 2.0.3 + fast-deep-equal: 3.1.3 + file-entry-cache: 6.0.1 + find-up: 5.0.0 + glob-parent: 6.0.2 + globals: 13.21.0 + graphemer: 1.4.0 + ignore: 5.2.4 + imurmurhash: 0.1.4 + is-glob: 4.0.3 + is-path-inside: 3.0.3 + js-yaml: 4.1.0 + json-stable-stringify-without-jsonify: 1.0.1 + levn: 0.4.1 + lodash.merge: 4.6.2 + minimatch: 3.1.2 + natural-compare: 1.4.0 + optionator: 0.9.3 + strip-ansi: 6.0.1 + text-table: 0.2.0 + transitivePeerDependencies: + - supports-color + dev: true + /espree@9.6.1: resolution: {integrity: sha512-oruZaFkjorTpF32kDSI5/75ViwGeZginGGy2NoOSg3Q9bnwlnmDm4HLnkl0RE3n+njDXR037aY1+x58Z/zFdwQ==} engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0} From 2f51283b7dc993286f52a70fcf75ad0c61f4d579 Mon Sep 17 00:00:00 2001 From: Mark Tyneway Date: Tue, 24 Oct 2023 14:33:31 -0600 Subject: [PATCH 046/279] ci-builder: use node version 20 node.js v20 is now LTS, this updates `ci-builder` to use the LTS version as part of the docker image. --- ops/docker/ci-builder/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ops/docker/ci-builder/Dockerfile b/ops/docker/ci-builder/Dockerfile index 6b3c14c4b638..4f7cf7234978 100644 --- a/ops/docker/ci-builder/Dockerfile +++ b/ops/docker/ci-builder/Dockerfile @@ -78,7 +78,7 @@ COPY --from=echidna-test /usr/local/bin/echidna-test /usr/local/bin/echidna-test COPY .nvmrc .nvmrc -ENV NODE_MAJOR=18 +ENV NODE_MAJOR=20 ENV SLITHER_VERSION=0.10.0 # note: python3 package in apt is python 3.9, while base image already has python 3.11 From 2375e5839f7762d7a8ff04f389b41a1f721816c8 Mon Sep 17 00:00:00 2001 From: Tei Im Date: Mon, 18 Sep 2023 16:44:05 +0900 Subject: [PATCH 047/279] Implement span batch derivation --- .../batch_decoder/reassemble/reassemble.go | 8 +- op-node/rollup/derive/attributes_queue.go | 4 +- .../rollup/derive/attributes_queue_test.go | 6 +- op-node/rollup/derive/batch_queue.go | 137 +- op-node/rollup/derive/batch_queue_test.go | 647 +++++++++- op-node/rollup/derive/batches.go | 271 +++- op-node/rollup/derive/batches_test.go | 1119 ++++++++++++++++- op-node/rollup/derive/channel.go | 22 +- op-node/rollup/derive/channel_in_reader.go | 30 +- op-node/rollup/derive/engine_queue.go | 1 + op-node/rollup/derive/pipeline.go | 2 +- op-program/client/l2/engine.go | 8 + op-service/testutils/mock_l2.go | 18 +- 13 files changed, 2075 insertions(+), 198 deletions(-) diff --git a/op-node/cmd/batch_decoder/reassemble/reassemble.go b/op-node/cmd/batch_decoder/reassemble/reassemble.go index 18ebba79f15e..8472ba8c03ca 100644 --- a/op-node/cmd/batch_decoder/reassemble/reassemble.go +++ b/op-node/cmd/batch_decoder/reassemble/reassemble.go @@ -107,14 +107,18 @@ func processFrames(cfg *rollup.Config, id derive.ChannelID, frames []FrameWithMe var batches []derive.SingularBatch invalidBatches := false if ch.IsReady() { - br, err := derive.BatchReader(cfg, ch.Reader(), eth.L1BlockRef{}) + br, err := derive.BatchReader(ch.Reader()) if err == nil { for batch, err := br(); err != io.EOF; batch, err = br() { if err != nil { fmt.Printf("Error reading batch for channel %v. Err: %v\n", id.String(), err) invalidBatches = true } else { - batches = append(batches, batch.Batch.SingularBatch) + if batch.BatchType != derive.SingularBatchType { + batches = append(batches, batch.SingularBatch) + } else { + fmt.Printf("batch-type %d is not supported", batch.BatchType) + } } } } else { diff --git a/op-node/rollup/derive/attributes_queue.go b/op-node/rollup/derive/attributes_queue.go index 016c39d696b8..023fbb3dccda 100644 --- a/op-node/rollup/derive/attributes_queue.go +++ b/op-node/rollup/derive/attributes_queue.go @@ -32,7 +32,7 @@ type AttributesQueue struct { config *rollup.Config builder AttributesBuilder prev *BatchQueue - batch *BatchData + batch *SingularBatch } func NewAttributesQueue(log log.Logger, cfg *rollup.Config, builder AttributesBuilder, prev *BatchQueue) *AttributesQueue { @@ -71,7 +71,7 @@ func (aq *AttributesQueue) NextAttributes(ctx context.Context, l2SafeHead eth.L2 // createNextAttributes transforms a batch into a payload attributes. This sets `NoTxPool` and appends the batched transactions // to the attributes transaction list -func (aq *AttributesQueue) createNextAttributes(ctx context.Context, batch *BatchData, l2SafeHead eth.L2BlockRef) (*eth.PayloadAttributes, error) { +func (aq *AttributesQueue) createNextAttributes(ctx context.Context, batch *SingularBatch, l2SafeHead eth.L2BlockRef) (*eth.PayloadAttributes, error) { // sanity check parent hash if batch.ParentHash != l2SafeHead.Hash { return nil, NewResetError(fmt.Errorf("valid batch has bad parent hash %s, expected %s", batch.ParentHash, l2SafeHead.Hash)) diff --git a/op-node/rollup/derive/attributes_queue_test.go b/op-node/rollup/derive/attributes_queue_test.go index fef240c576bc..c5b2cde5eca6 100644 --- a/op-node/rollup/derive/attributes_queue_test.go +++ b/op-node/rollup/derive/attributes_queue_test.go @@ -42,13 +42,13 @@ func TestAttributesQueue(t *testing.T) { safeHead.L1Origin = l1Info.ID() safeHead.Time = l1Info.InfoTime - batch := NewSingularBatchData(SingularBatch{ + batch := SingularBatch{ ParentHash: safeHead.Hash, EpochNum: rollup.Epoch(l1Info.InfoNum), EpochHash: l1Info.InfoHash, Timestamp: safeHead.Time + cfg.BlockTime, Transactions: []eth.Data{eth.Data("foobar"), eth.Data("example")}, - }) + } parentL1Cfg := eth.SystemConfig{ BatcherAddr: common.Address{42}, @@ -80,7 +80,7 @@ func TestAttributesQueue(t *testing.T) { aq := NewAttributesQueue(testlog.Logger(t, log.LvlError), cfg, attrBuilder, nil) - actual, err := aq.createNextAttributes(context.Background(), batch, safeHead) + actual, err := aq.createNextAttributes(context.Background(), &batch, safeHead) require.NoError(t, err) require.Equal(t, attrs, *actual) diff --git a/op-node/rollup/derive/batch_queue.go b/op-node/rollup/derive/batch_queue.go index 5f57299879cb..30dd1441924b 100644 --- a/op-node/rollup/derive/batch_queue.go +++ b/op-node/rollup/derive/batch_queue.go @@ -29,7 +29,12 @@ import ( type NextBatchProvider interface { Origin() eth.L1BlockRef - NextBatch(ctx context.Context) (*BatchData, error) + NextBatch(ctx context.Context) (Batch, error) +} + +type SafeBlockFetcher interface { + L2BlockRefByNumber(context.Context, uint64) (eth.L2BlockRef, error) + PayloadByNumber(context.Context, uint64) (*eth.ExecutionPayload, error) } // BatchQueue contains a set of batches for every L1 block. @@ -42,16 +47,22 @@ type BatchQueue struct { l1Blocks []eth.L1BlockRef - // batches in order of when we've first seen them, grouped by L2 timestamp - batches map[uint64][]*BatchWithL1InclusionBlock + // batches in order of when we've first seen them + batches []*BatchWithL1InclusionBlock + + // nextSpan is cached SingularBatches derived from SpanBatch + nextSpan []*SingularBatch + + l2 SafeBlockFetcher } // NewBatchQueue creates a BatchQueue, which should be Reset(origin) before use. -func NewBatchQueue(log log.Logger, cfg *rollup.Config, prev NextBatchProvider) *BatchQueue { +func NewBatchQueue(log log.Logger, cfg *rollup.Config, prev NextBatchProvider, l2 SafeBlockFetcher) *BatchQueue { return &BatchQueue{ log: log, config: cfg, prev: prev, + l2: l2, } } @@ -59,7 +70,29 @@ func (bq *BatchQueue) Origin() eth.L1BlockRef { return bq.prev.Origin() } -func (bq *BatchQueue) NextBatch(ctx context.Context, safeL2Head eth.L2BlockRef) (*BatchData, error) { +func (bq *BatchQueue) popNextBatch(safeL2Head eth.L2BlockRef) *SingularBatch { + nextBatch := bq.nextSpan[0] + bq.nextSpan = bq.nextSpan[1:] + // Must set ParentHash before return. we can use safeL2Head because the parentCheck is verified in CheckBatch(). + nextBatch.ParentHash = safeL2Head.Hash + return nextBatch +} + +func (bq *BatchQueue) advanceEpoch(nextBatch *SingularBatch) { + if nextBatch.GetEpochNum() == rollup.Epoch(bq.l1Blocks[0].Number)+1 { + // Advance epoch if necessary + bq.l1Blocks = bq.l1Blocks[1:] + } +} + +func (bq *BatchQueue) NextBatch(ctx context.Context, safeL2Head eth.L2BlockRef) (*SingularBatch, error) { + if len(bq.nextSpan) > 0 { + // If there are cached singular batches, pop first one and return. + nextBatch := bq.popNextBatch(safeL2Head) + bq.advanceEpoch(nextBatch) + return nextBatch, nil + } + // Note: We use the origin that we will have to determine if it's behind. This is important // because it's the future origin that gets saved into the l1Blocks array. // We always update the origin of this stage if it is not the same so after the update code @@ -89,7 +122,7 @@ func (bq *BatchQueue) NextBatch(ctx context.Context, safeL2Head eth.L2BlockRef) } else if err != nil { return nil, err } else if !originBehind { - bq.AddBatch(batch, safeL2Head) + bq.AddBatch(ctx, batch, safeL2Head) } // Skip adding data unless we are up to date with the origin, but do fully @@ -111,43 +144,70 @@ func (bq *BatchQueue) NextBatch(ctx context.Context, safeL2Head eth.L2BlockRef) } else if err != nil { return nil, err } - return batch, nil + + var nextBatch *SingularBatch + switch batch.GetBatchType() { + case SingularBatchType: + singularBatch, ok := batch.(*SingularBatch) + if !ok { + return nil, NewCriticalError(errors.New("failed type assertion to SingularBatch")) + } + nextBatch = singularBatch + case SpanBatchType: + spanBatch, ok := batch.(*SpanBatch) + if !ok { + return nil, NewCriticalError(errors.New("failed type assertion to SpanBatch")) + } + // If next batch is SpanBatch, convert it to SingularBatches. + singularBatches, err := spanBatch.GetSingularBatches(bq.l1Blocks, safeL2Head) + if err != nil { + return nil, NewCriticalError(err) + } + bq.nextSpan = singularBatches + nextBatch = bq.popNextBatch(safeL2Head) + default: + return nil, NewCriticalError(fmt.Errorf("unrecognized batch type: %d", batch.GetBatchType())) + } + + bq.advanceEpoch(nextBatch) + return nextBatch, nil } func (bq *BatchQueue) Reset(ctx context.Context, base eth.L1BlockRef, _ eth.SystemConfig) error { // Copy over the Origin from the next stage // It is set in the engine queue (two stages away) such that the L2 Safe Head origin is the progress bq.origin = base - bq.batches = make(map[uint64][]*BatchWithL1InclusionBlock) + bq.batches = []*BatchWithL1InclusionBlock{} // Include the new origin as an origin to build on // Note: This is only for the initialization case. During normal resets we will later // throw out this block. bq.l1Blocks = bq.l1Blocks[:0] bq.l1Blocks = append(bq.l1Blocks, base) + bq.nextSpan = bq.nextSpan[:0] return io.EOF } -func (bq *BatchQueue) AddBatch(batch *BatchData, l2SafeHead eth.L2BlockRef) { +func (bq *BatchQueue) AddBatch(ctx context.Context, batch Batch, l2SafeHead eth.L2BlockRef) { if len(bq.l1Blocks) == 0 { - panic(fmt.Errorf("cannot add batch with timestamp %d, no origin was prepared", batch.Timestamp)) + panic(fmt.Errorf("cannot add batch with timestamp %d, no origin was prepared", batch.GetTimestamp())) } data := BatchWithL1InclusionBlock{ L1InclusionBlock: bq.origin, Batch: batch, } - validity := CheckBatch(bq.config, bq.log, bq.l1Blocks, l2SafeHead, &data) + validity := CheckBatch(ctx, bq.config, bq.log, bq.l1Blocks, l2SafeHead, &data, bq.l2) if validity == BatchDrop { return // if we do drop the batch, CheckBatch will log the drop reason with WARN level. } - bq.log.Debug("Adding batch", "batch_timestamp", batch.Timestamp, "parent_hash", batch.ParentHash, "batch_epoch", batch.Epoch(), "txs", len(batch.Transactions)) - bq.batches[batch.Timestamp] = append(bq.batches[batch.Timestamp], &data) + batch.LogContext(bq.log).Debug("Adding batch") + bq.batches = append(bq.batches, &data) } // deriveNextBatch derives the next batch to apply on top of the current L2 safe head, // following the validity rules imposed on consecutive batches, // based on currently available buffered batch and L1 origin information. // If no batch can be derived yet, then (nil, io.EOF) is returned. -func (bq *BatchQueue) deriveNextBatch(ctx context.Context, outOfData bool, l2SafeHead eth.L2BlockRef) (*BatchData, error) { +func (bq *BatchQueue) deriveNextBatch(ctx context.Context, outOfData bool, l2SafeHead eth.L2BlockRef) (Batch, error) { if len(bq.l1Blocks) == 0 { return nil, NewCriticalError(errors.New("cannot derive next batch, no origin was prepared")) } @@ -170,19 +230,15 @@ func (bq *BatchQueue) deriveNextBatch(ctx context.Context, outOfData bool, l2Saf // Go over all batches, in order of inclusion, and find the first batch we can accept. // We filter in-place by only remembering the batches that may be processed in the future, or those we are undecided on. var remaining []*BatchWithL1InclusionBlock - candidates := bq.batches[nextTimestamp] batchLoop: - for i, batch := range candidates { - validity := CheckBatch(bq.config, bq.log.New("batch_index", i), bq.l1Blocks, l2SafeHead, batch) + for i, batch := range bq.batches { + validity := CheckBatch(ctx, bq.config, bq.log.New("batch_index", i), bq.l1Blocks, l2SafeHead, batch, bq.l2) switch validity { case BatchFuture: - return nil, NewCriticalError(fmt.Errorf("found batch with timestamp %d marked as future batch, but expected timestamp %d", batch.Batch.Timestamp, nextTimestamp)) + remaining = append(remaining, batch) + continue case BatchDrop: - bq.log.Warn("dropping batch", - "batch_timestamp", batch.Batch.Timestamp, - "parent_hash", batch.Batch.ParentHash, - "batch_epoch", batch.Batch.Epoch(), - "txs", len(batch.Batch.Transactions), + batch.Batch.LogContext(bq.log).Warn("dropping batch", "l2_safe_head", l2SafeHead.ID(), "l2_safe_head_time", l2SafeHead.Time, ) @@ -191,29 +247,20 @@ batchLoop: nextBatch = batch // don't keep the current batch in the remaining items since we are processing it now, // but retain every batch we didn't get to yet. - remaining = append(remaining, candidates[i+1:]...) + remaining = append(remaining, bq.batches[i+1:]...) break batchLoop case BatchUndecided: - remaining = append(remaining, batch) - bq.batches[nextTimestamp] = remaining + remaining = append(remaining, bq.batches[i:]...) + bq.batches = remaining return nil, io.EOF default: return nil, NewCriticalError(fmt.Errorf("unknown batch validity type: %d", validity)) } } - // clean up if we remove the final batch for this timestamp - if len(remaining) == 0 { - delete(bq.batches, nextTimestamp) - } else { - bq.batches[nextTimestamp] = remaining - } + bq.batches = remaining if nextBatch != nil { - // advance epoch if necessary - if nextBatch.Batch.EpochNum == rollup.Epoch(epoch.Number)+1 { - bq.l1Blocks = bq.l1Blocks[1:] - } - bq.log.Info("Found next batch", "epoch", epoch, "batch_epoch", nextBatch.Batch.EpochNum, "batch_timestamp", nextBatch.Batch.Timestamp) + nextBatch.Batch.LogContext(bq.log).Info("Found next batch") return nextBatch.Batch, nil } @@ -243,15 +290,13 @@ batchLoop: // batch to ensure that we at least have one batch per epoch. if nextTimestamp < nextEpoch.Time || firstOfEpoch { bq.log.Info("Generating next batch", "epoch", epoch, "timestamp", nextTimestamp) - return NewSingularBatchData( - SingularBatch{ - ParentHash: l2SafeHead.Hash, - EpochNum: rollup.Epoch(epoch.Number), - EpochHash: epoch.Hash, - Timestamp: nextTimestamp, - Transactions: nil, - }, - ), nil + return &SingularBatch{ + ParentHash: l2SafeHead.Hash, + EpochNum: rollup.Epoch(epoch.Number), + EpochHash: epoch.Hash, + Timestamp: nextTimestamp, + Transactions: nil, + }, nil } // At this point we have auto generated every batch for the current epoch diff --git a/op-node/rollup/derive/batch_queue_test.go b/op-node/rollup/derive/batch_queue_test.go index 8bad5f22a6a6..dee33b866177 100644 --- a/op-node/rollup/derive/batch_queue_test.go +++ b/op-node/rollup/derive/batch_queue_test.go @@ -3,10 +3,14 @@ package derive import ( "context" "encoding/binary" + "errors" "io" + "math/big" "math/rand" "testing" + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common/hexutil" "github.com/ethereum/go-ethereum/log" @@ -20,7 +24,7 @@ import ( type fakeBatchQueueInput struct { i int - batches []*BatchData + batches []Batch errors []error origin eth.L1BlockRef } @@ -29,7 +33,7 @@ func (f *fakeBatchQueueInput) Origin() eth.L1BlockRef { return f.origin } -func (f *fakeBatchQueueInput) NextBatch(ctx context.Context) (*BatchData, error) { +func (f *fakeBatchQueueInput) NextBatch(ctx context.Context) (Batch, error) { if f.i >= len(f.batches) { return nil, io.EOF } @@ -45,16 +49,74 @@ func mockHash(time uint64, layer uint8) common.Hash { return hash } -func b(timestamp uint64, epoch eth.L1BlockRef) *BatchData { +func b(chainId *big.Int, timestamp uint64, epoch eth.L1BlockRef) *SingularBatch { rng := rand.New(rand.NewSource(int64(timestamp))) - data := testutils.RandomData(rng, 20) - return NewSingularBatchData(SingularBatch{ + signer := types.NewLondonSigner(chainId) + tx := testutils.RandomTx(rng, new(big.Int).SetUint64(rng.Uint64()), signer) + txData, _ := tx.MarshalBinary() + return &SingularBatch{ ParentHash: mockHash(timestamp-2, 2), Timestamp: timestamp, EpochNum: rollup.Epoch(epoch.Number), EpochHash: epoch.Hash, - Transactions: []hexutil.Bytes{data}, - }) + Transactions: []hexutil.Bytes{txData}, + } +} + +func buildSpanBatches(t *testing.T, parent *eth.L2BlockRef, singularBatches []*SingularBatch, blockCounts []int, chainId *big.Int) []Batch { + var spanBatches []Batch + idx := 0 + for _, count := range blockCounts { + span := NewSpanBatch(singularBatches[idx : idx+count]) + spanBatches = append(spanBatches, span) + idx += count + } + return spanBatches +} + +func getSpanBatchTime(batchType int) *uint64 { + minTs := uint64(0) + if batchType == SpanBatchType { + return &minTs + } + return nil +} + +func l1InfoDepositTx(t *testing.T, l1BlockNum uint64) hexutil.Bytes { + l1Info := L1BlockInfo{ + Number: l1BlockNum, + BaseFee: big.NewInt(0), + } + infoData, err := l1Info.MarshalBinary() + require.NoError(t, err) + depositTx := &types.DepositTx{ + Data: infoData, + } + txData, err := types.NewTx(depositTx).MarshalBinary() + require.NoError(t, err) + return txData +} + +func singularBatchToPayload(t *testing.T, batch *SingularBatch, blockNumber uint64) eth.ExecutionPayload { + txs := []hexutil.Bytes{l1InfoDepositTx(t, uint64(batch.EpochNum))} + txs = append(txs, batch.Transactions...) + return eth.ExecutionPayload{ + BlockHash: mockHash(batch.Timestamp, 2), + ParentHash: batch.ParentHash, + BlockNumber: hexutil.Uint64(blockNumber), + Timestamp: hexutil.Uint64(batch.Timestamp), + Transactions: txs, + } +} + +func singularBatchToBlockRef(t *testing.T, batch *SingularBatch, blockNumber uint64) eth.L2BlockRef { + return eth.L2BlockRef{ + Hash: mockHash(batch.Timestamp, 2), + Number: blockNumber, + ParentHash: batch.ParentHash, + Time: batch.Timestamp, + L1Origin: eth.BlockID{Hash: batch.EpochHash, Number: uint64(batch.EpochNum)}, + } } func L1Chain(l1Times []uint64) []eth.L1BlockRef { @@ -73,10 +135,37 @@ func L1Chain(l1Times []uint64) []eth.L1BlockRef { return out } -// TestBatchQueueNewOrigin tests that the batch queue properly saves the new origin +func TestBatchQueue(t *testing.T) { + tests := []struct { + name string + f func(t *testing.T, batchType int) + }{ + {"BatchQueueNewOrigin", BatchQueueNewOrigin}, + {"BatchQueueEager", BatchQueueEager}, + {"BatchQueueInvalidInternalAdvance", BatchQueueInvalidInternalAdvance}, + {"BatchQueueMissing", BatchQueueMissing}, + {"BatchQueueAdvancedEpoch", BatchQueueAdvancedEpoch}, + {"BatchQueueShuffle", BatchQueueShuffle}, + } + for _, test := range tests { + test := test + t.Run(test.name+"_SingularBatch", func(t *testing.T) { + test.f(t, SingularBatchType) + }) + } + + for _, test := range tests { + test := test + t.Run(test.name+"_SpanBatch", func(t *testing.T) { + test.f(t, SpanBatchType) + }) + } +} + +// BatchQueueNewOrigin tests that the batch queue properly saves the new origin // when the safehead's origin is ahead of the pipeline's origin (as is after a reset). // This issue was fixed in https://github.com/ethereum-optimism/optimism/pull/3694 -func TestBatchQueueNewOrigin(t *testing.T) { +func BatchQueueNewOrigin(t *testing.T, batchType int) { log := testlog.Logger(t, log.LvlCrit) l1 := L1Chain([]uint64{10, 15, 20, 25}) safeHead := eth.L2BlockRef{ @@ -94,15 +183,16 @@ func TestBatchQueueNewOrigin(t *testing.T) { BlockTime: 2, MaxSequencerDrift: 600, SeqWindowSize: 2, + SpanBatchTime: getSpanBatchTime(batchType), } input := &fakeBatchQueueInput{ - batches: []*BatchData{nil}, + batches: []Batch{nil}, errors: []error{io.EOF}, origin: l1[0], } - bq := NewBatchQueue(log, cfg, input) + bq := NewBatchQueue(log, cfg, input, nil) _ = bq.Reset(context.Background(), l1[0], eth.SystemConfig{}) require.Equal(t, []eth.L1BlockRef{l1[0]}, bq.l1Blocks) @@ -133,11 +223,12 @@ func TestBatchQueueNewOrigin(t *testing.T) { require.Equal(t, l1[2], bq.origin) } -// TestBatchQueueEager adds a bunch of contiguous batches and asserts that +// BatchQueueEager adds a bunch of contiguous batches and asserts that // enough calls to `NextBatch` return all of those batches. -func TestBatchQueueEager(t *testing.T) { +func BatchQueueEager(t *testing.T, batchType int) { log := testlog.Logger(t, log.LvlCrit) l1 := L1Chain([]uint64{10, 20, 30}) + chainId := big.NewInt(1234) safeHead := eth.L2BlockRef{ Hash: mockHash(10, 2), Number: 0, @@ -153,41 +244,69 @@ func TestBatchQueueEager(t *testing.T) { BlockTime: 2, MaxSequencerDrift: 600, SeqWindowSize: 30, + SpanBatchTime: getSpanBatchTime(batchType), + L2ChainID: chainId, } - batches := []*BatchData{b(12, l1[0]), b(14, l1[0]), b(16, l1[0]), b(18, l1[0]), b(20, l1[0]), b(22, l1[0]), b(24, l1[1]), nil} - errors := []error{nil, nil, nil, nil, nil, nil, nil, io.EOF} + // expected output of BatchQueue.NextBatch() + expectedOutputBatches := []*SingularBatch{ + b(cfg.L2ChainID, 12, l1[0]), + b(cfg.L2ChainID, 14, l1[0]), + b(cfg.L2ChainID, 16, l1[0]), + b(cfg.L2ChainID, 18, l1[0]), + b(cfg.L2ChainID, 20, l1[0]), + b(cfg.L2ChainID, 22, l1[0]), + nil, + } + // expected error of BatchQueue.NextBatch() + expectedOutputErrors := []error{nil, nil, nil, nil, nil, nil, io.EOF} + // errors will be returned by fakeBatchQueueInput.NextBatch() + inputErrors := expectedOutputErrors + // batches will be returned by fakeBatchQueueInput + var inputBatches []Batch + if batchType == SpanBatchType { + spanBlockCounts := []int{1, 2, 3} + inputErrors = []error{nil, nil, nil, io.EOF} + inputBatches = buildSpanBatches(t, &safeHead, expectedOutputBatches, spanBlockCounts, chainId) + inputBatches = append(inputBatches, nil) + } else { + for _, singularBatch := range expectedOutputBatches { + inputBatches = append(inputBatches, singularBatch) + } + } input := &fakeBatchQueueInput{ - batches: batches, - errors: errors, + batches: inputBatches, + errors: inputErrors, origin: l1[0], } - bq := NewBatchQueue(log, cfg, input) + bq := NewBatchQueue(log, cfg, input, nil) _ = bq.Reset(context.Background(), l1[0], eth.SystemConfig{}) // Advance the origin input.origin = l1[1] - for i := 0; i < len(batches); i++ { + for i := 0; i < len(expectedOutputBatches); i++ { b, e := bq.NextBatch(context.Background(), safeHead) - require.ErrorIs(t, e, errors[i]) - require.Equal(t, batches[i], b) - - if b != nil { + require.ErrorIs(t, e, expectedOutputErrors[i]) + if b == nil { + require.Nil(t, expectedOutputBatches[i]) + } else { + require.Equal(t, expectedOutputBatches[i], b) safeHead.Number += 1 - safeHead.Time += 2 + safeHead.Time += cfg.BlockTime safeHead.Hash = mockHash(b.Timestamp, 2) safeHead.L1Origin = b.Epoch() } } } -// TestBatchQueueInvalidInternalAdvance asserts that we do not miss an epoch when generating batches. +// BatchQueueInvalidInternalAdvance asserts that we do not miss an epoch when generating batches. // This is a regression test for CLI-3378. -func TestBatchQueueInvalidInternalAdvance(t *testing.T) { +func BatchQueueInvalidInternalAdvance(t *testing.T, batchType int) { log := testlog.Logger(t, log.LvlTrace) l1 := L1Chain([]uint64{10, 15, 20, 25, 30}) + chainId := big.NewInt(1234) safeHead := eth.L2BlockRef{ Hash: mockHash(10, 2), Number: 0, @@ -203,27 +322,54 @@ func TestBatchQueueInvalidInternalAdvance(t *testing.T) { BlockTime: 2, MaxSequencerDrift: 600, SeqWindowSize: 2, + SpanBatchTime: getSpanBatchTime(batchType), + L2ChainID: chainId, } - batches := []*BatchData{b(12, l1[0]), b(14, l1[0]), b(16, l1[0]), b(18, l1[0]), b(20, l1[0]), b(22, l1[0]), nil} - errors := []error{nil, nil, nil, nil, nil, nil, io.EOF} + // expected output of BatchQueue.NextBatch() + expectedOutputBatches := []*SingularBatch{ + b(cfg.L2ChainID, 12, l1[0]), + b(cfg.L2ChainID, 14, l1[0]), + b(cfg.L2ChainID, 16, l1[0]), + b(cfg.L2ChainID, 18, l1[0]), + b(cfg.L2ChainID, 20, l1[0]), + b(cfg.L2ChainID, 22, l1[0]), + nil, + } + // expected error of BatchQueue.NextBatch() + expectedOutputErrors := []error{nil, nil, nil, nil, nil, nil, io.EOF} + // errors will be returned by fakeBatchQueueInput.NextBatch() + inputErrors := expectedOutputErrors + // batches will be returned by fakeBatchQueueInput + var inputBatches []Batch + if batchType == SpanBatchType { + spanBlockCounts := []int{1, 2, 3} + inputErrors = []error{nil, nil, nil, io.EOF} + inputBatches = buildSpanBatches(t, &safeHead, expectedOutputBatches, spanBlockCounts, chainId) + inputBatches = append(inputBatches, nil) + } else { + for _, singularBatch := range expectedOutputBatches { + inputBatches = append(inputBatches, singularBatch) + } + } input := &fakeBatchQueueInput{ - batches: batches, - errors: errors, + batches: inputBatches, + errors: inputErrors, origin: l1[0], } - bq := NewBatchQueue(log, cfg, input) + bq := NewBatchQueue(log, cfg, input, nil) _ = bq.Reset(context.Background(), l1[0], eth.SystemConfig{}) // Load continuous batches for epoch 0 - for i := 0; i < len(batches); i++ { + for i := 0; i < len(expectedOutputBatches); i++ { b, e := bq.NextBatch(context.Background(), safeHead) - require.ErrorIs(t, e, errors[i]) - require.Equal(t, batches[i], b) - - if b != nil { + require.ErrorIs(t, e, expectedOutputErrors[i]) + if b == nil { + require.Nil(t, expectedOutputBatches[i]) + } else { + require.Equal(t, expectedOutputBatches[i], b) safeHead.Number += 1 safeHead.Time += 2 safeHead.Hash = mockHash(b.Timestamp, 2) @@ -276,9 +422,10 @@ func TestBatchQueueInvalidInternalAdvance(t *testing.T) { } -func TestBatchQueueMissing(t *testing.T) { +func BatchQueueMissing(t *testing.T, batchType int) { log := testlog.Logger(t, log.LvlCrit) l1 := L1Chain([]uint64{10, 15, 20, 25}) + chainId := big.NewInt(1234) safeHead := eth.L2BlockRef{ Hash: mockHash(10, 2), Number: 0, @@ -294,30 +441,49 @@ func TestBatchQueueMissing(t *testing.T) { BlockTime: 2, MaxSequencerDrift: 600, SeqWindowSize: 2, + SpanBatchTime: getSpanBatchTime(batchType), + L2ChainID: chainId, } - // The batches at 18 and 20 are skipped to stop 22 from being eagerly processed. + // The inputBatches at 18 and 20 are skipped to stop 22 from being eagerly processed. // This test checks that batch timestamp 12 & 14 are created, 16 is used, and 18 is advancing the epoch. - // Due to the large sequencer time drift 16 is perfectly valid to have epoch 0 as origin. - batches := []*BatchData{b(16, l1[0]), b(22, l1[1])} - errors := []error{nil, nil} + // Due to the large sequencer time drift 16 is perfectly valid to have epoch 0 as origin.a + + // expected output of BatchQueue.NextBatch() + expectedOutputBatches := []*SingularBatch{ + b(cfg.L2ChainID, 16, l1[0]), + b(cfg.L2ChainID, 22, l1[1]), + } + // errors will be returned by fakeBatchQueueInput.NextBatch() + inputErrors := []error{nil, nil} + // batches will be returned by fakeBatchQueueInput + var inputBatches []Batch + if batchType == SpanBatchType { + spanBlockCounts := []int{1, 1} + inputErrors = []error{nil, nil, nil, io.EOF} + inputBatches = buildSpanBatches(t, &safeHead, expectedOutputBatches, spanBlockCounts, chainId) + } else { + for _, singularBatch := range expectedOutputBatches { + inputBatches = append(inputBatches, singularBatch) + } + } input := &fakeBatchQueueInput{ - batches: batches, - errors: errors, + batches: inputBatches, + errors: inputErrors, origin: l1[0], } - bq := NewBatchQueue(log, cfg, input) + bq := NewBatchQueue(log, cfg, input, nil) _ = bq.Reset(context.Background(), l1[0], eth.SystemConfig{}) - for i := 0; i < len(batches); i++ { + for i := 0; i < len(expectedOutputBatches); i++ { b, e := bq.NextBatch(context.Background(), safeHead) require.ErrorIs(t, e, NotEnoughData) require.Nil(t, b) } - // advance origin. Underlying stage still has no more batches + // advance origin. Underlying stage still has no more inputBatches // This is not enough to auto advance yet input.origin = l1[1] b, e := bq.NextBatch(context.Background(), safeHead) @@ -331,7 +497,7 @@ func TestBatchQueueMissing(t *testing.T) { b, e = bq.NextBatch(context.Background(), safeHead) require.Nil(t, e) require.Equal(t, b.Timestamp, uint64(12)) - require.Empty(t, b.SingularBatch.Transactions) + require.Empty(t, b.Transactions) require.Equal(t, rollup.Epoch(0), b.EpochNum) safeHead.Number += 1 safeHead.Time += 2 @@ -341,7 +507,7 @@ func TestBatchQueueMissing(t *testing.T) { b, e = bq.NextBatch(context.Background(), safeHead) require.Nil(t, e) require.Equal(t, b.Timestamp, uint64(14)) - require.Empty(t, b.SingularBatch.Transactions) + require.Empty(t, b.Transactions) require.Equal(t, rollup.Epoch(0), b.EpochNum) safeHead.Number += 1 safeHead.Time += 2 @@ -350,7 +516,7 @@ func TestBatchQueueMissing(t *testing.T) { // Check for the inputted batch at t = 16 b, e = bq.NextBatch(context.Background(), safeHead) require.Nil(t, e) - require.Equal(t, b, batches[0]) + require.Equal(t, b, expectedOutputBatches[0]) require.Equal(t, rollup.Epoch(0), b.EpochNum) safeHead.Number += 1 safeHead.Time += 2 @@ -367,6 +533,387 @@ func TestBatchQueueMissing(t *testing.T) { b, e = bq.NextBatch(context.Background(), safeHead) require.Nil(t, e) require.Equal(t, b.Timestamp, uint64(18)) - require.Empty(t, b.SingularBatch.Transactions) + require.Empty(t, b.Transactions) require.Equal(t, rollup.Epoch(1), b.EpochNum) } + +// BatchQueueAdvancedEpoch tests that batch queue derives consecutive valid batches with advancing epochs. +// Batch queue's l1blocks list should be updated along epochs. +func BatchQueueAdvancedEpoch(t *testing.T, batchType int) { + log := testlog.Logger(t, log.LvlCrit) + l1 := L1Chain([]uint64{0, 6, 12, 18, 24}) // L1 block time: 6s + chainId := big.NewInt(1234) + safeHead := eth.L2BlockRef{ + Hash: mockHash(4, 2), + Number: 0, + ParentHash: common.Hash{}, + Time: 4, + L1Origin: l1[0].ID(), + SequenceNumber: 0, + } + cfg := &rollup.Config{ + Genesis: rollup.Genesis{ + L2Time: 10, + }, + BlockTime: 2, + MaxSequencerDrift: 600, + SeqWindowSize: 30, + SpanBatchTime: getSpanBatchTime(batchType), + L2ChainID: chainId, + } + + // expected output of BatchQueue.NextBatch() + expectedOutputBatches := []*SingularBatch{ + // 3 L2 blocks per L1 block + b(cfg.L2ChainID, 6, l1[1]), + b(cfg.L2ChainID, 8, l1[1]), + b(cfg.L2ChainID, 10, l1[1]), + b(cfg.L2ChainID, 12, l1[2]), + b(cfg.L2ChainID, 14, l1[2]), + b(cfg.L2ChainID, 16, l1[2]), + b(cfg.L2ChainID, 18, l1[3]), + b(cfg.L2ChainID, 20, l1[3]), + b(cfg.L2ChainID, 22, l1[3]), + nil, + } + // expected error of BatchQueue.NextBatch() + expectedOutputErrors := []error{nil, nil, nil, nil, nil, nil, nil, nil, nil, io.EOF} + // errors will be returned by fakeBatchQueueInput.NextBatch() + inputErrors := expectedOutputErrors + // batches will be returned by fakeBatchQueueInput + var inputBatches []Batch + if batchType == SpanBatchType { + spanBlockCounts := []int{2, 2, 2, 3} + inputErrors = []error{nil, nil, nil, nil, io.EOF} + inputBatches = buildSpanBatches(t, &safeHead, expectedOutputBatches, spanBlockCounts, chainId) + inputBatches = append(inputBatches, nil) + } else { + for _, singularBatch := range expectedOutputBatches { + inputBatches = append(inputBatches, singularBatch) + } + } + + // ChannelInReader origin number + inputOriginNumber := 2 + input := &fakeBatchQueueInput{ + batches: inputBatches, + errors: inputErrors, + origin: l1[inputOriginNumber], + } + + bq := NewBatchQueue(log, cfg, input, nil) + _ = bq.Reset(context.Background(), l1[1], eth.SystemConfig{}) + + for i := 0; i < len(expectedOutputBatches); i++ { + expectedOutput := expectedOutputBatches[i] + if expectedOutput != nil && uint64(expectedOutput.EpochNum) == l1[inputOriginNumber].Number { + // Advance ChannelInReader origin if needed + inputOriginNumber += 1 + input.origin = l1[inputOriginNumber] + } + b, e := bq.NextBatch(context.Background(), safeHead) + require.ErrorIs(t, e, expectedOutputErrors[i]) + if b == nil { + require.Nil(t, expectedOutput) + } else { + require.Equal(t, expectedOutput, b) + require.Equal(t, bq.l1Blocks[0].Number, uint64(b.EpochNum)) + safeHead.Number += 1 + safeHead.Time += cfg.BlockTime + safeHead.Hash = mockHash(b.Timestamp, 2) + safeHead.L1Origin = b.Epoch() + } + } +} + +// BatchQueueShuffle tests batch queue can reorder shuffled valid batches +func BatchQueueShuffle(t *testing.T, batchType int) { + log := testlog.Logger(t, log.LvlCrit) + l1 := L1Chain([]uint64{0, 6, 12, 18, 24}) // L1 block time: 6s + chainId := big.NewInt(1234) + safeHead := eth.L2BlockRef{ + Hash: mockHash(4, 2), + Number: 0, + ParentHash: common.Hash{}, + Time: 4, + L1Origin: l1[0].ID(), + SequenceNumber: 0, + } + cfg := &rollup.Config{ + Genesis: rollup.Genesis{ + L2Time: 10, + }, + BlockTime: 2, + MaxSequencerDrift: 600, + SeqWindowSize: 30, + SpanBatchTime: getSpanBatchTime(batchType), + L2ChainID: chainId, + } + + // expected output of BatchQueue.NextBatch() + expectedOutputBatches := []*SingularBatch{ + // 3 L2 blocks per L1 block + b(cfg.L2ChainID, 6, l1[1]), + b(cfg.L2ChainID, 8, l1[1]), + b(cfg.L2ChainID, 10, l1[1]), + b(cfg.L2ChainID, 12, l1[2]), + b(cfg.L2ChainID, 14, l1[2]), + b(cfg.L2ChainID, 16, l1[2]), + b(cfg.L2ChainID, 18, l1[3]), + b(cfg.L2ChainID, 20, l1[3]), + b(cfg.L2ChainID, 22, l1[3]), + } + // expected error of BatchQueue.NextBatch() + expectedOutputErrors := []error{nil, nil, nil, nil, nil, nil, nil, nil, nil, io.EOF} + // errors will be returned by fakeBatchQueueInput.NextBatch() + inputErrors := expectedOutputErrors + // batches will be returned by fakeBatchQueueInput + var inputBatches []Batch + if batchType == SpanBatchType { + spanBlockCounts := []int{2, 2, 2, 3} + inputErrors = []error{nil, nil, nil, nil, io.EOF} + inputBatches = buildSpanBatches(t, &safeHead, expectedOutputBatches, spanBlockCounts, chainId) + } else { + for _, singularBatch := range expectedOutputBatches { + inputBatches = append(inputBatches, singularBatch) + } + } + + // Shuffle the order of input batches + rand.Shuffle(len(inputBatches), func(i, j int) { + inputBatches[i], inputBatches[j] = inputBatches[j], inputBatches[i] + }) + inputBatches = append(inputBatches, nil) + + // ChannelInReader origin number + inputOriginNumber := 2 + input := &fakeBatchQueueInput{ + batches: inputBatches, + errors: inputErrors, + origin: l1[inputOriginNumber], + } + + bq := NewBatchQueue(log, cfg, input, nil) + _ = bq.Reset(context.Background(), l1[1], eth.SystemConfig{}) + + for i := 0; i < len(expectedOutputBatches); i++ { + expectedOutput := expectedOutputBatches[i] + if expectedOutput != nil && uint64(expectedOutput.EpochNum) == l1[inputOriginNumber].Number { + // Advance ChannelInReader origin if needed + inputOriginNumber += 1 + input.origin = l1[inputOriginNumber] + } + var b *SingularBatch + var e error + for j := 0; j < len(expectedOutputBatches); j++ { + // Multiple NextBatch() executions may be required because the order of input is shuffled + b, e = bq.NextBatch(context.Background(), safeHead) + if !errors.Is(e, NotEnoughData) { + break + } + } + require.ErrorIs(t, e, expectedOutputErrors[i]) + if b == nil { + require.Nil(t, expectedOutput) + } else { + require.Equal(t, expectedOutput, b) + require.Equal(t, bq.l1Blocks[0].Number, uint64(b.EpochNum)) + safeHead.Number += 1 + safeHead.Time += cfg.BlockTime + safeHead.Hash = mockHash(b.Timestamp, 2) + safeHead.L1Origin = b.Epoch() + } + } +} + +func TestBatchQueueOverlappingSpanBatch(t *testing.T) { + log := testlog.Logger(t, log.LvlCrit) + l1 := L1Chain([]uint64{10, 20, 30}) + chainId := big.NewInt(1234) + safeHead := eth.L2BlockRef{ + Hash: mockHash(10, 2), + Number: 0, + ParentHash: common.Hash{}, + Time: 10, + L1Origin: l1[0].ID(), + SequenceNumber: 0, + } + cfg := &rollup.Config{ + Genesis: rollup.Genesis{ + L2Time: 10, + }, + BlockTime: 2, + MaxSequencerDrift: 600, + SeqWindowSize: 30, + SpanBatchTime: getSpanBatchTime(SpanBatchType), + L2ChainID: chainId, + } + + // expected output of BatchQueue.NextBatch() + expectedOutputBatches := []*SingularBatch{ + b(cfg.L2ChainID, 12, l1[0]), + b(cfg.L2ChainID, 14, l1[0]), + b(cfg.L2ChainID, 16, l1[0]), + b(cfg.L2ChainID, 18, l1[0]), + b(cfg.L2ChainID, 20, l1[0]), + b(cfg.L2ChainID, 22, l1[0]), + nil, + } + // expected error of BatchQueue.NextBatch() + expectedOutputErrors := []error{nil, nil, nil, nil, nil, nil, io.EOF} + // errors will be returned by fakeBatchQueueInput.NextBatch() + inputErrors := []error{nil, nil, nil, nil, io.EOF} + + // batches will be returned by fakeBatchQueueInput + var inputBatches []Batch + batchSize := 3 + for i := 0; i < len(expectedOutputBatches)-batchSize; i++ { + inputBatches = append(inputBatches, NewSpanBatch(expectedOutputBatches[i:i+batchSize])) + } + inputBatches = append(inputBatches, nil) + + input := &fakeBatchQueueInput{ + batches: inputBatches, + errors: inputErrors, + origin: l1[0], + } + + l2Client := testutils.MockL2Client{} + var nilErr error + for i, batch := range expectedOutputBatches { + if batch != nil { + blockRef := singularBatchToBlockRef(t, batch, uint64(i+1)) + payload := singularBatchToPayload(t, batch, uint64(i+1)) + l2Client.Mock.On("L2BlockRefByNumber", uint64(i+1)).Times(9999).Return(blockRef, &nilErr) + l2Client.Mock.On("PayloadByNumber", uint64(i+1)).Times(9999).Return(&payload, &nilErr) + } + } + + bq := NewBatchQueue(log, cfg, input, &l2Client) + _ = bq.Reset(context.Background(), l1[0], eth.SystemConfig{}) + // Advance the origin + input.origin = l1[1] + + for i := 0; i < len(expectedOutputBatches); i++ { + b, e := bq.NextBatch(context.Background(), safeHead) + require.ErrorIs(t, e, expectedOutputErrors[i]) + if b == nil { + require.Nil(t, expectedOutputBatches[i]) + } else { + require.Equal(t, expectedOutputBatches[i], b) + safeHead.Number += 1 + safeHead.Time += cfg.BlockTime + safeHead.Hash = mockHash(b.Timestamp, 2) + safeHead.L1Origin = b.Epoch() + } + } +} + +func TestBatchQueueComplex(t *testing.T) { + log := testlog.Logger(t, log.LvlCrit) + l1 := L1Chain([]uint64{0, 6, 12, 18, 24}) // L1 block time: 6s + chainId := big.NewInt(1234) + safeHead := eth.L2BlockRef{ + Hash: mockHash(4, 2), + Number: 0, + ParentHash: common.Hash{}, + Time: 4, + L1Origin: l1[0].ID(), + SequenceNumber: 0, + } + cfg := &rollup.Config{ + Genesis: rollup.Genesis{ + L2Time: 10, + }, + BlockTime: 2, + MaxSequencerDrift: 600, + SeqWindowSize: 30, + SpanBatchTime: getSpanBatchTime(SpanBatchType), + L2ChainID: chainId, + } + + // expected output of BatchQueue.NextBatch() + expectedOutputBatches := []*SingularBatch{ + // 3 L2 blocks per L1 block + b(cfg.L2ChainID, 6, l1[1]), + b(cfg.L2ChainID, 8, l1[1]), + b(cfg.L2ChainID, 10, l1[1]), + b(cfg.L2ChainID, 12, l1[2]), + b(cfg.L2ChainID, 14, l1[2]), + b(cfg.L2ChainID, 16, l1[2]), + b(cfg.L2ChainID, 18, l1[3]), + b(cfg.L2ChainID, 20, l1[3]), + b(cfg.L2ChainID, 22, l1[3]), + } + // expected error of BatchQueue.NextBatch() + expectedOutputErrors := []error{nil, nil, nil, nil, nil, nil, nil, nil, nil, io.EOF} + // errors will be returned by fakeBatchQueueInput.NextBatch() + inputErrors := []error{nil, nil, nil, nil, nil, nil, io.EOF} + // batches will be returned by fakeBatchQueueInput + inputBatches := []Batch{ + NewSpanBatch(expectedOutputBatches[0:2]), // 6, 8 + expectedOutputBatches[2], // 10 + NewSpanBatch(expectedOutputBatches[1:4]), // 8, 10, 12 + expectedOutputBatches[4], // 14 + NewSpanBatch(expectedOutputBatches[4:6]), // 14, 16 + NewSpanBatch(expectedOutputBatches[6:9]), // 18, 20, 22 + } + + // Shuffle the order of input batches + rand.Shuffle(len(inputBatches), func(i, j int) { + inputBatches[i], inputBatches[j] = inputBatches[j], inputBatches[i] + }) + + inputBatches = append(inputBatches, nil) + + // ChannelInReader origin number + inputOriginNumber := 2 + input := &fakeBatchQueueInput{ + batches: inputBatches, + errors: inputErrors, + origin: l1[inputOriginNumber], + } + + l2Client := testutils.MockL2Client{} + var nilErr error + for i, batch := range expectedOutputBatches { + if batch != nil { + blockRef := singularBatchToBlockRef(t, batch, uint64(i+1)) + payload := singularBatchToPayload(t, batch, uint64(i+1)) + l2Client.Mock.On("L2BlockRefByNumber", uint64(i+1)).Times(9999).Return(blockRef, &nilErr) + l2Client.Mock.On("PayloadByNumber", uint64(i+1)).Times(9999).Return(&payload, &nilErr) + } + } + + bq := NewBatchQueue(log, cfg, input, &l2Client) + _ = bq.Reset(context.Background(), l1[1], eth.SystemConfig{}) + + for i := 0; i < len(expectedOutputBatches); i++ { + expectedOutput := expectedOutputBatches[i] + if expectedOutput != nil && uint64(expectedOutput.EpochNum) == l1[inputOriginNumber].Number { + // Advance ChannelInReader origin if needed + inputOriginNumber += 1 + input.origin = l1[inputOriginNumber] + } + var b *SingularBatch + var e error + for j := 0; j < len(expectedOutputBatches); j++ { + // Multiple NextBatch() executions may be required because the order of input is shuffled + b, e = bq.NextBatch(context.Background(), safeHead) + if !errors.Is(e, NotEnoughData) { + break + } + } + require.ErrorIs(t, e, expectedOutputErrors[i]) + if b == nil { + require.Nil(t, expectedOutput) + } else { + require.Equal(t, expectedOutput, b) + require.Equal(t, bq.l1Blocks[0].Number, uint64(b.EpochNum)) + safeHead.Number += 1 + safeHead.Time += cfg.BlockTime + safeHead.Hash = mockHash(b.Timestamp, 2) + safeHead.L1Origin = b.Epoch() + } + } +} diff --git a/op-node/rollup/derive/batches.go b/op-node/rollup/derive/batches.go index 7790e471d9dd..73020fa4079f 100644 --- a/op-node/rollup/derive/batches.go +++ b/op-node/rollup/derive/batches.go @@ -1,6 +1,9 @@ package derive import ( + "bytes" + "context" + "github.com/ethereum-optimism/optimism/op-node/rollup" "github.com/ethereum-optimism/optimism/op-service/eth" "github.com/ethereum/go-ethereum/core/types" @@ -9,7 +12,7 @@ import ( type BatchWithL1InclusionBlock struct { L1InclusionBlock eth.L1BlockRef - Batch *BatchData + Batch Batch } type BatchValidity uint8 @@ -28,14 +31,37 @@ const ( // CheckBatch checks if the given batch can be applied on top of the given l2SafeHead, given the contextual L1 blocks the batch was included in. // The first entry of the l1Blocks should match the origin of the l2SafeHead. One or more consecutive l1Blocks should be provided. // In case of only a single L1 block, the decision whether a batch is valid may have to stay undecided. -func CheckBatch(cfg *rollup.Config, log log.Logger, l1Blocks []eth.L1BlockRef, l2SafeHead eth.L2BlockRef, batch *BatchWithL1InclusionBlock) BatchValidity { +func CheckBatch(ctx context.Context, cfg *rollup.Config, log log.Logger, l1Blocks []eth.L1BlockRef, + l2SafeHead eth.L2BlockRef, batch *BatchWithL1InclusionBlock, l2Fetcher SafeBlockFetcher) BatchValidity { + switch batch.Batch.GetBatchType() { + case SingularBatchType: + singularBatch, ok := batch.Batch.(*SingularBatch) + if !ok { + log.Error("failed type assertion to SingularBatch") + return BatchDrop + } + return checkSingularBatch(cfg, log, l1Blocks, l2SafeHead, singularBatch, batch.L1InclusionBlock) + case SpanBatchType: + spanBatch, ok := batch.Batch.(*SpanBatch) + if !ok { + log.Error("failed type assertion to SpanBatch") + return BatchDrop + } + if !cfg.IsSpanBatch(batch.Batch.GetTimestamp()) { + log.Warn("received SpanBatch before SpanBatch hard fork") + return BatchDrop + } + return checkSpanBatch(ctx, cfg, log, l1Blocks, l2SafeHead, spanBatch, batch.L1InclusionBlock, l2Fetcher) + default: + log.Warn("unrecognized batch type: %d", batch.Batch.GetBatchType()) + return BatchDrop + } +} + +// checkSingularBatch implements SingularBatch validation rule. +func checkSingularBatch(cfg *rollup.Config, log log.Logger, l1Blocks []eth.L1BlockRef, l2SafeHead eth.L2BlockRef, batch *SingularBatch, l1InclusionBlock eth.L1BlockRef) BatchValidity { // add details to the log - log = log.New( - "batch_timestamp", batch.Batch.Timestamp, - "parent_hash", batch.Batch.ParentHash, - "batch_epoch", batch.Batch.Epoch(), - "txs", len(batch.Batch.Transactions), - ) + log = batch.LogContext(log) // sanity check we have consistent inputs if len(l1Blocks) == 0 { @@ -45,36 +71,36 @@ func CheckBatch(cfg *rollup.Config, log log.Logger, l1Blocks []eth.L1BlockRef, l epoch := l1Blocks[0] nextTimestamp := l2SafeHead.Time + cfg.BlockTime - if batch.Batch.Timestamp > nextTimestamp { + if batch.Timestamp > nextTimestamp { log.Trace("received out-of-order batch for future processing after next batch", "next_timestamp", nextTimestamp) return BatchFuture } - if batch.Batch.Timestamp < nextTimestamp { + if batch.Timestamp < nextTimestamp { log.Warn("dropping batch with old timestamp", "min_timestamp", nextTimestamp) return BatchDrop } // dependent on above timestamp check. If the timestamp is correct, then it must build on top of the safe head. - if batch.Batch.ParentHash != l2SafeHead.Hash { + if batch.ParentHash != l2SafeHead.Hash { log.Warn("ignoring batch with mismatching parent hash", "current_safe_head", l2SafeHead.Hash) return BatchDrop } // Filter out batches that were included too late. - if uint64(batch.Batch.EpochNum)+cfg.SeqWindowSize < batch.L1InclusionBlock.Number { + if uint64(batch.EpochNum)+cfg.SeqWindowSize < l1InclusionBlock.Number { log.Warn("batch was included too late, sequence window expired") return BatchDrop } // Check the L1 origin of the batch batchOrigin := epoch - if uint64(batch.Batch.EpochNum) < epoch.Number { + if uint64(batch.EpochNum) < epoch.Number { log.Warn("dropped batch, epoch is too old", "minimum", epoch.ID()) // batch epoch too old return BatchDrop - } else if uint64(batch.Batch.EpochNum) == epoch.Number { + } else if uint64(batch.EpochNum) == epoch.Number { // Batch is sticking to the current epoch, continue. - } else if uint64(batch.Batch.EpochNum) == epoch.Number+1 { + } else if uint64(batch.EpochNum) == epoch.Number+1 { // With only 1 l1Block we cannot look at the next L1 Origin. // Note: This means that we are unable to determine validity of a batch // without more information. In this case we should bail out until we have @@ -90,19 +116,19 @@ func CheckBatch(cfg *rollup.Config, log log.Logger, l1Blocks []eth.L1BlockRef, l return BatchDrop } - if batch.Batch.EpochHash != batchOrigin.Hash { + if batch.EpochHash != batchOrigin.Hash { log.Warn("batch is for different L1 chain, epoch hash does not match", "expected", batchOrigin.ID()) return BatchDrop } - if batch.Batch.Timestamp < batchOrigin.Time { - log.Warn("batch timestamp is less than L1 origin timestamp", "l2_timestamp", batch.Batch.Timestamp, "l1_timestamp", batchOrigin.Time, "origin", batchOrigin.ID()) + if batch.Timestamp < batchOrigin.Time { + log.Warn("batch timestamp is less than L1 origin timestamp", "l2_timestamp", batch.Timestamp, "l1_timestamp", batchOrigin.Time, "origin", batchOrigin.ID()) return BatchDrop } // Check if we ran out of sequencer time drift - if max := batchOrigin.Time + cfg.MaxSequencerDrift; batch.Batch.Timestamp > max { - if len(batch.Batch.Transactions) == 0 { + if max := batchOrigin.Time + cfg.MaxSequencerDrift; batch.Timestamp > max { + if len(batch.Transactions) == 0 { // If the sequencer is co-operating by producing an empty batch, // then allow the batch if it was the right thing to do to maintain the L2 time >= L1 time invariant. // We only check batches that do not advance the epoch, to ensure epoch advancement regardless of time drift is allowed. @@ -112,7 +138,7 @@ func CheckBatch(cfg *rollup.Config, log log.Logger, l1Blocks []eth.L1BlockRef, l return BatchUndecided } nextOrigin := l1Blocks[1] - if batch.Batch.Timestamp >= nextOrigin.Time { // check if the next L1 origin could have been adopted + if batch.Timestamp >= nextOrigin.Time { // check if the next L1 origin could have been adopted log.Info("batch exceeded sequencer time drift without adopting next origin, and next L1 origin would have been valid") return BatchDrop } else { @@ -128,7 +154,7 @@ func CheckBatch(cfg *rollup.Config, log log.Logger, l1Blocks []eth.L1BlockRef, l } // We can do this check earlier, but it's a more intensive one, so we do this last. - for i, txBytes := range batch.Batch.Transactions { + for i, txBytes := range batch.Transactions { if len(txBytes) == 0 { log.Warn("transaction data must not be empty, but found empty tx", "tx_index", i) return BatchDrop @@ -141,3 +167,204 @@ func CheckBatch(cfg *rollup.Config, log log.Logger, l1Blocks []eth.L1BlockRef, l return BatchAccept } + +// checkSpanBatch implements SpanBatch validation rule. +func checkSpanBatch(ctx context.Context, cfg *rollup.Config, log log.Logger, l1Blocks []eth.L1BlockRef, l2SafeHead eth.L2BlockRef, + batch *SpanBatch, l1InclusionBlock eth.L1BlockRef, l2Fetcher SafeBlockFetcher) BatchValidity { + // add details to the log + log = batch.LogContext(log) + + // sanity check we have consistent inputs + if len(l1Blocks) == 0 { + log.Warn("missing L1 block input, cannot proceed with batch checking") + return BatchUndecided + } + epoch := l1Blocks[0] + + nextTimestamp := l2SafeHead.Time + cfg.BlockTime + + if batch.GetTimestamp() > nextTimestamp { + log.Trace("received out-of-order batch for future processing after next batch", "next_timestamp", nextTimestamp) + return BatchFuture + } + if batch.GetBlockTimestamp(batch.GetBlockCount()-1) < nextTimestamp { + log.Warn("span batch has no new blocks after safe head") + return BatchDrop + } + + // finding parent block of the span batch. + // if the span batch does not overlap the current safe chain, parentBLock should be l2SafeHead. + parentNum := l2SafeHead.Number + parentBlock := l2SafeHead + if batch.GetTimestamp() < nextTimestamp { + if batch.GetTimestamp() > l2SafeHead.Time { + // batch timestamp cannot be between safe head and next timestamp + log.Warn("batch has misaligned timestamp") + return BatchDrop + } + if (l2SafeHead.Time-batch.GetTimestamp())%cfg.BlockTime != 0 { + log.Warn("batch has misaligned timestamp") + return BatchDrop + } + parentNum = l2SafeHead.Number - (l2SafeHead.Time-batch.GetTimestamp())/cfg.BlockTime - 1 + var err error + parentBlock, err = l2Fetcher.L2BlockRefByNumber(ctx, parentNum) + if err != nil { + log.Error("failed to fetch L2 block", "number", parentNum, "err", err) + // unable to validate the batch for now. retry later. + return BatchUndecided + } + } + if !batch.CheckParentHash(parentBlock.Hash) { + log.Warn("ignoring batch with mismatching parent hash", "parent_block", parentBlock.Hash) + return BatchDrop + } + + startEpochNum := uint64(batch.GetStartEpochNum()) + + // Filter out batches that were included too late. + if startEpochNum+cfg.SeqWindowSize < l1InclusionBlock.Number { + log.Warn("batch was included too late, sequence window expired") + return BatchDrop + } + + // Check the L1 origin of the batch + if startEpochNum > parentBlock.L1Origin.Number+1 { + log.Warn("batch is for future epoch too far ahead, while it has the next timestamp, so it must be invalid", "current_epoch", epoch.ID()) + return BatchDrop + } + + endEpochNum := batch.GetBlockEpochNum(batch.GetBlockCount() - 1) + originChecked := false + for _, l1Block := range l1Blocks { + if l1Block.Number == endEpochNum { + if !batch.CheckOriginHash(l1Block.Hash) { + log.Warn("batch is for different L1 chain, epoch hash does not match", "expected", l1Block.Hash) + return BatchDrop + } + originChecked = true + break + } + } + if !originChecked { + log.Info("need more l1 blocks to check entire origins of span batch") + return BatchUndecided + } + + if startEpochNum < parentBlock.L1Origin.Number { + log.Warn("dropped batch, epoch is too old", "minimum", parentBlock.ID()) + return BatchDrop + } + + originIdx := 0 + originAdvanced := false + if startEpochNum == parentBlock.L1Origin.Number+1 { + originAdvanced = true + } + + for i := 0; i < batch.GetBlockCount(); i++ { + if batch.GetBlockTimestamp(i) <= l2SafeHead.Time { + continue + } + var l1Origin eth.L1BlockRef + for j := originIdx; j < len(l1Blocks); j++ { + if batch.GetBlockEpochNum(i) == l1Blocks[j].Number { + l1Origin = l1Blocks[j] + originIdx = j + break + } + + } + if i > 0 { + originAdvanced = false + if batch.GetBlockEpochNum(i) > batch.GetBlockEpochNum(i-1) { + originAdvanced = true + } + } + blockTimestamp := batch.GetBlockTimestamp(i) + if blockTimestamp < l1Origin.Time { + log.Warn("block timestamp is less than L1 origin timestamp", "l2_timestamp", blockTimestamp, "l1_timestamp", l1Origin.Time, "origin", l1Origin.ID()) + return BatchDrop + } + + // Check if we ran out of sequencer time drift + if max := l1Origin.Time + cfg.MaxSequencerDrift; blockTimestamp > max { + if len(batch.GetBlockTransactions(i)) == 0 { + // If the sequencer is co-operating by producing an empty batch, + // then allow the batch if it was the right thing to do to maintain the L2 time >= L1 time invariant. + // We only check batches that do not advance the epoch, to ensure epoch advancement regardless of time drift is allowed. + if !originAdvanced { + if originIdx+1 >= len(l1Blocks) { + log.Info("without the next L1 origin we cannot determine yet if this empty batch that exceeds the time drift is still valid") + return BatchUndecided + } + if blockTimestamp >= l1Blocks[originIdx+1].Time { // check if the next L1 origin could have been adopted + log.Info("batch exceeded sequencer time drift without adopting next origin, and next L1 origin would have been valid") + return BatchDrop + } else { + log.Info("continuing with empty batch before late L1 block to preserve L2 time invariant") + } + } + } else { + // If the sequencer is ignoring the time drift rule, then drop the batch and force an empty batch instead, + // as the sequencer is not allowed to include anything past this point without moving to the next epoch. + log.Warn("batch exceeded sequencer time drift, sequencer must adopt new L1 origin to include transactions again", "max_time", max) + return BatchDrop + } + } + + for i, txBytes := range batch.GetBlockTransactions(i) { + if len(txBytes) == 0 { + log.Warn("transaction data must not be empty, but found empty tx", "tx_index", i) + return BatchDrop + } + if txBytes[0] == types.DepositTxType { + log.Warn("sequencers may not embed any deposits into batch data, but found tx that has one", "tx_index", i) + return BatchDrop + } + } + } + + // Check overlapped blocks + if batch.GetTimestamp() < nextTimestamp { + for i := uint64(0); i < l2SafeHead.Number-parentNum; i++ { + safeBlockNum := parentNum + i + 1 + safeBlockPayload, err := l2Fetcher.PayloadByNumber(ctx, safeBlockNum) + if err != nil { + log.Error("failed to fetch L2 block payload", "number", parentNum, "err", err) + // unable to validate the batch for now. retry later. + return BatchUndecided + } + safeBlockTxs := safeBlockPayload.Transactions + batchTxs := batch.GetBlockTransactions(int(i)) + // execution payload has deposit TXs, but batch does not. + depositCount := 0 + for _, tx := range safeBlockTxs { + if tx[0] == types.DepositTxType { + depositCount++ + } + } + if len(safeBlockTxs)-depositCount != len(batchTxs) { + log.Warn("overlapped block's tx count does not match", "safeBlockTxs", len(safeBlockTxs), "batchTxs", len(batchTxs)) + return BatchDrop + } + for j := 0; j < len(batchTxs); j++ { + if !bytes.Equal(safeBlockTxs[j+depositCount], batchTxs[j]) { + log.Warn("overlapped block's transaction does not match") + return BatchDrop + } + } + safeBlockRef, err := PayloadToBlockRef(safeBlockPayload, &cfg.Genesis) + if err != nil { + log.Error("failed to extract L2BlockRef from execution payload", "hash", safeBlockPayload.BlockHash, "err", err) + return BatchDrop + } + if safeBlockRef.L1Origin.Number != batch.GetBlockEpochNum(int(i)) { + log.Warn("overlapped block's L1 origin number does not match") + return BatchDrop + } + } + } + + return BatchAccept +} diff --git a/op-node/rollup/derive/batches_test.go b/op-node/rollup/derive/batches_test.go index cabcb6e4e147..a3948fa1a27a 100644 --- a/op-node/rollup/derive/batches_test.go +++ b/op-node/rollup/derive/batches_test.go @@ -1,6 +1,9 @@ package derive import ( + "context" + "errors" + "math/big" "math/rand" "testing" @@ -24,10 +27,19 @@ type ValidBatchTestCase struct { Expected BatchValidity } +type SpanBatchHardForkTestCase struct { + Name string + L1Blocks []eth.L1BlockRef + L2SafeHead eth.L2BlockRef + Batch BatchWithL1InclusionBlock + Expected BatchValidity + SpanBatchTime uint64 +} + var HashA = common.Hash{0x0a} var HashB = common.Hash{0x0b} -func TestValidBatch(t *testing.T) { +func TestValidSingularBatch(t *testing.T) { conf := rollup.Config{ Genesis: rollup.Genesis{ L2Time: 31, // a genesis time that itself does not align to make it more interesting @@ -174,13 +186,13 @@ func TestValidBatch(t *testing.T) { L2SafeHead: l2A0, Batch: BatchWithL1InclusionBlock{ L1InclusionBlock: l1B, - Batch: NewSingularBatchData(SingularBatch{ + Batch: &SingularBatch{ ParentHash: l2A1.ParentHash, EpochNum: rollup.Epoch(l2A1.L1Origin.Number), EpochHash: l2A1.L1Origin.Hash, Timestamp: l2A1.Time, Transactions: nil, - }), + }, }, Expected: BatchUndecided, }, @@ -190,13 +202,13 @@ func TestValidBatch(t *testing.T) { L2SafeHead: l2A0, Batch: BatchWithL1InclusionBlock{ L1InclusionBlock: l1B, - Batch: NewSingularBatchData(SingularBatch{ + Batch: &SingularBatch{ ParentHash: l2A1.ParentHash, EpochNum: rollup.Epoch(l2A1.L1Origin.Number), EpochHash: l2A1.L1Origin.Hash, Timestamp: l2A1.Time + 1, // 1 too high Transactions: nil, - }), + }, }, Expected: BatchFuture, }, @@ -206,13 +218,13 @@ func TestValidBatch(t *testing.T) { L2SafeHead: l2A0, Batch: BatchWithL1InclusionBlock{ L1InclusionBlock: l1B, - Batch: NewSingularBatchData(SingularBatch{ + Batch: &SingularBatch{ ParentHash: l2A1.ParentHash, EpochNum: rollup.Epoch(l2A1.L1Origin.Number), EpochHash: l2A1.L1Origin.Hash, Timestamp: l2A0.Time, // repeating the same time Transactions: nil, - }), + }, }, Expected: BatchDrop, }, @@ -222,13 +234,13 @@ func TestValidBatch(t *testing.T) { L2SafeHead: l2A0, Batch: BatchWithL1InclusionBlock{ L1InclusionBlock: l1B, - Batch: NewSingularBatchData(SingularBatch{ + Batch: &SingularBatch{ ParentHash: l2A1.ParentHash, EpochNum: rollup.Epoch(l2A1.L1Origin.Number), EpochHash: l2A1.L1Origin.Hash, Timestamp: l2A1.Time - 1, // block time is 2, so this is 1 too low Transactions: nil, - }), + }, }, Expected: BatchDrop, }, @@ -238,13 +250,13 @@ func TestValidBatch(t *testing.T) { L2SafeHead: l2A0, Batch: BatchWithL1InclusionBlock{ L1InclusionBlock: l1B, - Batch: NewSingularBatchData(SingularBatch{ + Batch: &SingularBatch{ ParentHash: testutils.RandomHash(rng), EpochNum: rollup.Epoch(l2A1.L1Origin.Number), EpochHash: l2A1.L1Origin.Hash, Timestamp: l2A1.Time, Transactions: nil, - }), + }, }, Expected: BatchDrop, }, @@ -254,13 +266,13 @@ func TestValidBatch(t *testing.T) { L2SafeHead: l2A0, Batch: BatchWithL1InclusionBlock{ L1InclusionBlock: l1F, // included in 5th block after epoch of batch, while seq window is 4 - Batch: NewSingularBatchData(SingularBatch{ + Batch: &SingularBatch{ ParentHash: l2A1.ParentHash, EpochNum: rollup.Epoch(l2A1.L1Origin.Number), EpochHash: l2A1.L1Origin.Hash, Timestamp: l2A1.Time, Transactions: nil, - }), + }, }, Expected: BatchDrop, }, @@ -270,13 +282,13 @@ func TestValidBatch(t *testing.T) { L2SafeHead: l2B0, // we already moved on to B Batch: BatchWithL1InclusionBlock{ L1InclusionBlock: l1C, - Batch: NewSingularBatchData(SingularBatch{ + Batch: &SingularBatch{ ParentHash: l2B0.Hash, // build on top of safe head to continue EpochNum: rollup.Epoch(l2A3.L1Origin.Number), // epoch A is no longer valid EpochHash: l2A3.L1Origin.Hash, Timestamp: l2B0.Time + conf.BlockTime, // pass the timestamp check to get too epoch check Transactions: nil, - }), + }, }, Expected: BatchDrop, }, @@ -286,13 +298,13 @@ func TestValidBatch(t *testing.T) { L2SafeHead: l2A3, Batch: BatchWithL1InclusionBlock{ L1InclusionBlock: l1C, - Batch: NewSingularBatchData(SingularBatch{ + Batch: &SingularBatch{ ParentHash: l2B0.ParentHash, EpochNum: rollup.Epoch(l2B0.L1Origin.Number), EpochHash: l2B0.L1Origin.Hash, Timestamp: l2B0.Time, Transactions: nil, - }), + }, }, Expected: BatchUndecided, }, @@ -302,13 +314,13 @@ func TestValidBatch(t *testing.T) { L2SafeHead: l2A3, Batch: BatchWithL1InclusionBlock{ L1InclusionBlock: l1D, - Batch: NewSingularBatchData(SingularBatch{ + Batch: &SingularBatch{ ParentHash: l2B0.ParentHash, EpochNum: rollup.Epoch(l1C.Number), // invalid, we need to adopt epoch B before C EpochHash: l1C.Hash, Timestamp: l2B0.Time, Transactions: nil, - }), + }, }, Expected: BatchDrop, }, @@ -318,13 +330,13 @@ func TestValidBatch(t *testing.T) { L2SafeHead: l2A3, Batch: BatchWithL1InclusionBlock{ L1InclusionBlock: l1C, - Batch: NewSingularBatchData(SingularBatch{ + Batch: &SingularBatch{ ParentHash: l2B0.ParentHash, EpochNum: rollup.Epoch(l2B0.L1Origin.Number), EpochHash: l1A.Hash, // invalid, epoch hash should be l1B Timestamp: l2B0.Time, Transactions: nil, - }), + }, }, Expected: BatchDrop, }, @@ -334,13 +346,13 @@ func TestValidBatch(t *testing.T) { L2SafeHead: l2A3, Batch: BatchWithL1InclusionBlock{ L1InclusionBlock: l1B, - Batch: NewSingularBatchData(SingularBatch{ // we build l2A4, which has a timestamp of 2*4 = 8 higher than l2A0 + Batch: &SingularBatch{ // we build l2A4, which has a timestamp of 2*4 = 8 higher than l2A0 ParentHash: l2A4.ParentHash, EpochNum: rollup.Epoch(l2A4.L1Origin.Number), EpochHash: l2A4.L1Origin.Hash, Timestamp: l2A4.Time, Transactions: []hexutil.Bytes{[]byte("sequencer should not include this tx")}, - }), + }, }, Expected: BatchDrop, }, @@ -350,13 +362,13 @@ func TestValidBatch(t *testing.T) { L2SafeHead: l2X0, Batch: BatchWithL1InclusionBlock{ L1InclusionBlock: l1Z, - Batch: NewSingularBatchData(SingularBatch{ + Batch: &SingularBatch{ ParentHash: l2Y0.ParentHash, EpochNum: rollup.Epoch(l2Y0.L1Origin.Number), EpochHash: l2Y0.L1Origin.Hash, Timestamp: l2Y0.Time, // valid, but more than 6 ahead of l1Y.Time Transactions: []hexutil.Bytes{[]byte("sequencer should not include this tx")}, - }), + }, }, Expected: BatchDrop, }, @@ -366,13 +378,13 @@ func TestValidBatch(t *testing.T) { L2SafeHead: l2A3, Batch: BatchWithL1InclusionBlock{ L1InclusionBlock: l1BLate, - Batch: NewSingularBatchData(SingularBatch{ // l2A4 time < l1BLate time, so we cannot adopt origin B yet + Batch: &SingularBatch{ // l2A4 time < l1BLate time, so we cannot adopt origin B yet ParentHash: l2A4.ParentHash, EpochNum: rollup.Epoch(l2A4.L1Origin.Number), EpochHash: l2A4.L1Origin.Hash, Timestamp: l2A4.Time, Transactions: nil, - }), + }, }, Expected: BatchAccept, // accepted because empty & preserving L2 time invariant }, @@ -382,13 +394,13 @@ func TestValidBatch(t *testing.T) { L2SafeHead: l2X0, Batch: BatchWithL1InclusionBlock{ L1InclusionBlock: l1Z, - Batch: NewSingularBatchData(SingularBatch{ + Batch: &SingularBatch{ ParentHash: l2Y0.ParentHash, EpochNum: rollup.Epoch(l2Y0.L1Origin.Number), EpochHash: l2Y0.L1Origin.Hash, Timestamp: l2Y0.Time, // valid, but more than 6 ahead of l1Y.Time Transactions: nil, - }), + }, }, Expected: BatchAccept, // accepted because empty & still advancing epoch }, @@ -398,13 +410,13 @@ func TestValidBatch(t *testing.T) { L2SafeHead: l2A3, Batch: BatchWithL1InclusionBlock{ L1InclusionBlock: l1B, - Batch: NewSingularBatchData(SingularBatch{ // we build l2A4, which has a timestamp of 2*4 = 8 higher than l2A0 + Batch: &SingularBatch{ // we build l2A4, which has a timestamp of 2*4 = 8 higher than l2A0 ParentHash: l2A4.ParentHash, EpochNum: rollup.Epoch(l2A4.L1Origin.Number), EpochHash: l2A4.L1Origin.Hash, Timestamp: l2A4.Time, Transactions: nil, - }), + }, }, Expected: BatchUndecided, // we have to wait till the next epoch is in sight to check the time }, @@ -414,13 +426,13 @@ func TestValidBatch(t *testing.T) { L2SafeHead: l2A3, Batch: BatchWithL1InclusionBlock{ L1InclusionBlock: l1C, - Batch: NewSingularBatchData(SingularBatch{ // we build l2A4, which has a timestamp of 2*4 = 8 higher than l2A0 + Batch: &SingularBatch{ // we build l2A4, which has a timestamp of 2*4 = 8 higher than l2A0 ParentHash: l2A4.ParentHash, EpochNum: rollup.Epoch(l2A4.L1Origin.Number), EpochHash: l2A4.L1Origin.Hash, Timestamp: l2A4.Time, Transactions: nil, - }), + }, }, Expected: BatchDrop, // dropped because it could have advanced the epoch to B }, @@ -430,7 +442,7 @@ func TestValidBatch(t *testing.T) { L2SafeHead: l2A0, Batch: BatchWithL1InclusionBlock{ L1InclusionBlock: l1B, - Batch: NewSingularBatchData(SingularBatch{ + Batch: &SingularBatch{ ParentHash: l2A1.ParentHash, EpochNum: rollup.Epoch(l2A1.L1Origin.Number), EpochHash: l2A1.L1Origin.Hash, @@ -438,7 +450,7 @@ func TestValidBatch(t *testing.T) { Transactions: []hexutil.Bytes{ []byte{}, // empty tx data }, - }), + }, }, Expected: BatchDrop, }, @@ -448,7 +460,7 @@ func TestValidBatch(t *testing.T) { L2SafeHead: l2A0, Batch: BatchWithL1InclusionBlock{ L1InclusionBlock: l1B, - Batch: NewSingularBatchData(SingularBatch{ + Batch: &SingularBatch{ ParentHash: l2A1.ParentHash, EpochNum: rollup.Epoch(l2A1.L1Origin.Number), EpochHash: l2A1.L1Origin.Hash, @@ -456,7 +468,7 @@ func TestValidBatch(t *testing.T) { Transactions: []hexutil.Bytes{ []byte{types.DepositTxType, 0}, // piece of data alike to a deposit }, - }), + }, }, Expected: BatchDrop, }, @@ -466,7 +478,7 @@ func TestValidBatch(t *testing.T) { L2SafeHead: l2A0, Batch: BatchWithL1InclusionBlock{ L1InclusionBlock: l1B, - Batch: NewSingularBatchData(SingularBatch{ + Batch: &SingularBatch{ ParentHash: l2A1.ParentHash, EpochNum: rollup.Epoch(l2A1.L1Origin.Number), EpochHash: l2A1.L1Origin.Hash, @@ -475,7 +487,7 @@ func TestValidBatch(t *testing.T) { []byte{0x02, 0x42, 0x13, 0x37}, []byte{0x02, 0xde, 0xad, 0xbe, 0xef}, }, - }), + }, }, Expected: BatchAccept, }, @@ -485,7 +497,7 @@ func TestValidBatch(t *testing.T) { L2SafeHead: l2A3, Batch: BatchWithL1InclusionBlock{ L1InclusionBlock: l1C, - Batch: NewSingularBatchData(SingularBatch{ + Batch: &SingularBatch{ ParentHash: l2B0.ParentHash, EpochNum: rollup.Epoch(l2B0.L1Origin.Number), EpochHash: l2B0.L1Origin.Hash, @@ -494,7 +506,7 @@ func TestValidBatch(t *testing.T) { []byte{0x02, 0x42, 0x13, 0x37}, []byte{0x02, 0xde, 0xad, 0xbe, 0xef}, }, - }), + }, }, Expected: BatchAccept, }, @@ -504,13 +516,13 @@ func TestValidBatch(t *testing.T) { L2SafeHead: l2A2, Batch: BatchWithL1InclusionBlock{ L1InclusionBlock: l1B, - Batch: NewSingularBatchData(SingularBatch{ // we build l2B0', which starts a new epoch too early + Batch: &SingularBatch{ // we build l2B0', which starts a new epoch too early ParentHash: l2A2.Hash, EpochNum: rollup.Epoch(l2B0.L1Origin.Number), EpochHash: l2B0.L1Origin.Hash, Timestamp: l2A2.Time + conf.BlockTime, Transactions: nil, - }), + }, }, Expected: BatchDrop, }, @@ -521,7 +533,1026 @@ func TestValidBatch(t *testing.T) { for _, testCase := range testCases { t.Run(testCase.Name, func(t *testing.T) { - validity := CheckBatch(&conf, logger, testCase.L1Blocks, testCase.L2SafeHead, &testCase.Batch) + ctx := context.Background() + validity := CheckBatch(ctx, &conf, logger, testCase.L1Blocks, testCase.L2SafeHead, &testCase.Batch, nil) + require.Equal(t, testCase.Expected, validity, "batch check must return expected validity level") + }) + } +} + +func TestValidSpanBatch(t *testing.T) { + minTs := uint64(0) + conf := rollup.Config{ + Genesis: rollup.Genesis{ + L2Time: 31, // a genesis time that itself does not align to make it more interesting + }, + BlockTime: 2, + SeqWindowSize: 4, + MaxSequencerDrift: 6, + SpanBatchTime: &minTs, + // other config fields are ignored and can be left empty. + } + + rng := rand.New(rand.NewSource(1234)) + chainId := new(big.Int).SetUint64(rng.Uint64()) + signer := types.NewLondonSigner(chainId) + randTx := testutils.RandomTx(rng, new(big.Int).SetUint64(rng.Uint64()), signer) + randTxData, _ := randTx.MarshalBinary() + l1A := testutils.RandomBlockRef(rng) + l1B := eth.L1BlockRef{ + Hash: testutils.RandomHash(rng), + Number: l1A.Number + 1, + ParentHash: l1A.Hash, + Time: l1A.Time + 7, + } + l1C := eth.L1BlockRef{ + Hash: testutils.RandomHash(rng), + Number: l1B.Number + 1, + ParentHash: l1B.Hash, + Time: l1B.Time + 7, + } + l1D := eth.L1BlockRef{ + Hash: testutils.RandomHash(rng), + Number: l1C.Number + 1, + ParentHash: l1C.Hash, + Time: l1C.Time + 7, + } + l1E := eth.L1BlockRef{ + Hash: testutils.RandomHash(rng), + Number: l1D.Number + 1, + ParentHash: l1D.Hash, + Time: l1D.Time + 7, + } + l1F := eth.L1BlockRef{ + Hash: testutils.RandomHash(rng), + Number: l1E.Number + 1, + ParentHash: l1E.Hash, + Time: l1E.Time + 7, + } + + l2A0 := eth.L2BlockRef{ + Hash: testutils.RandomHash(rng), + Number: 100, + ParentHash: testutils.RandomHash(rng), + Time: l1A.Time, + L1Origin: l1A.ID(), + SequenceNumber: 0, + } + + l2A1 := eth.L2BlockRef{ + Hash: testutils.RandomHash(rng), + Number: l2A0.Number + 1, + ParentHash: l2A0.Hash, + Time: l2A0.Time + conf.BlockTime, + L1Origin: l1A.ID(), + SequenceNumber: 1, + } + + l2A2 := eth.L2BlockRef{ + Hash: testutils.RandomHash(rng), + Number: l2A1.Number + 1, + ParentHash: l2A1.Hash, + Time: l2A1.Time + conf.BlockTime, + L1Origin: l1A.ID(), + SequenceNumber: 2, + } + + l2A3 := eth.L2BlockRef{ + Hash: testutils.RandomHash(rng), + Number: l2A2.Number + 1, + ParentHash: l2A2.Hash, + Time: l2A2.Time + conf.BlockTime, + L1Origin: l1A.ID(), + SequenceNumber: 3, + } + + l2B0 := eth.L2BlockRef{ + Hash: testutils.RandomHash(rng), + Number: l2A3.Number + 1, + ParentHash: l2A3.Hash, + Time: l2A3.Time + conf.BlockTime, // 8 seconds larger than l1A0, 1 larger than origin + L1Origin: l1B.ID(), + SequenceNumber: 0, + } + + l1X := eth.L1BlockRef{ + Hash: testutils.RandomHash(rng), + Number: 42, + ParentHash: testutils.RandomHash(rng), + Time: 10_000, + } + l1Y := eth.L1BlockRef{ + Hash: testutils.RandomHash(rng), + Number: l1X.Number + 1, + ParentHash: l1X.Hash, + Time: l1X.Time + 12, + } + l1Z := eth.L1BlockRef{ + Hash: testutils.RandomHash(rng), + Number: l1Y.Number + 1, + ParentHash: l1Y.Hash, + Time: l1Y.Time + 12, + } + l2X0 := eth.L2BlockRef{ + Hash: testutils.RandomHash(rng), + Number: 1000, + ParentHash: testutils.RandomHash(rng), + Time: 10_000 + 12 + 6 - 1, // add one block, and you get ahead of next l1 block by more than the drift + L1Origin: l1X.ID(), + SequenceNumber: 0, + } + l2Y0 := eth.L2BlockRef{ + Hash: testutils.RandomHash(rng), + Number: l2X0.Number + 1, + ParentHash: l2X0.Hash, + Time: l2X0.Time + conf.BlockTime, // exceeds sequencer time drift, forced to be empty block + L1Origin: l1Y.ID(), + SequenceNumber: 0, + } + + l2A4 := eth.L2BlockRef{ + Hash: testutils.RandomHash(rng), + Number: l2A3.Number + 1, + ParentHash: l2A3.Hash, + Time: l2A3.Time + conf.BlockTime, // 4*2 = 8, higher than seq time drift + L1Origin: l1A.ID(), + SequenceNumber: 4, + } + + l1BLate := eth.L1BlockRef{ + Hash: testutils.RandomHash(rng), + Number: l1A.Number + 1, + ParentHash: l1A.Hash, + Time: l2A4.Time + 1, // too late for l2A4 to adopt yet + } + + testCases := []ValidBatchTestCase{ + { + Name: "missing L1 info", + L1Blocks: []eth.L1BlockRef{}, + L2SafeHead: l2A0, + Batch: BatchWithL1InclusionBlock{ + L1InclusionBlock: l1B, + Batch: NewSpanBatch([]*SingularBatch{ + { + ParentHash: l2A1.ParentHash, + EpochNum: rollup.Epoch(l2A1.L1Origin.Number), + EpochHash: l2A1.L1Origin.Hash, + Timestamp: l2A1.Time, + Transactions: nil, + }, + }), + }, + Expected: BatchUndecided, + }, + { + Name: "future timestamp", + L1Blocks: []eth.L1BlockRef{l1A, l1B, l1C}, + L2SafeHead: l2A0, + Batch: BatchWithL1InclusionBlock{ + L1InclusionBlock: l1B, + Batch: NewSpanBatch([]*SingularBatch{ + { + ParentHash: l2A1.ParentHash, + EpochNum: rollup.Epoch(l2A1.L1Origin.Number), + EpochHash: l2A1.L1Origin.Hash, + Timestamp: l2A1.Time + 1, // 1 too high + Transactions: nil, + }, + }), + }, + Expected: BatchFuture, + }, + { + Name: "old timestamp", + L1Blocks: []eth.L1BlockRef{l1A, l1B, l1C}, + L2SafeHead: l2A0, + Batch: BatchWithL1InclusionBlock{ + L1InclusionBlock: l1B, + Batch: NewSpanBatch([]*SingularBatch{ + { + ParentHash: l2A1.ParentHash, + EpochNum: rollup.Epoch(l2A1.L1Origin.Number), + EpochHash: l2A1.L1Origin.Hash, + Timestamp: l2A0.Time, // repeating the same time + Transactions: nil, + }, + }), + }, + Expected: BatchDrop, + }, + { + Name: "misaligned timestamp", + L1Blocks: []eth.L1BlockRef{l1A, l1B, l1C}, + L2SafeHead: l2A0, + Batch: BatchWithL1InclusionBlock{ + L1InclusionBlock: l1B, + Batch: NewSpanBatch([]*SingularBatch{ + { + ParentHash: l2A1.ParentHash, + EpochNum: rollup.Epoch(l2A1.L1Origin.Number), + EpochHash: l2A1.L1Origin.Hash, + Timestamp: l2A1.Time - 1, // block time is 2, so this is 1 too low + Transactions: nil, + }, + }), + }, + Expected: BatchDrop, + }, + { + Name: "invalid parent block hash", + L1Blocks: []eth.L1BlockRef{l1A, l1B, l1C}, + L2SafeHead: l2A0, + Batch: BatchWithL1InclusionBlock{ + L1InclusionBlock: l1B, + Batch: NewSpanBatch([]*SingularBatch{ + { + ParentHash: testutils.RandomHash(rng), + EpochNum: rollup.Epoch(l2A1.L1Origin.Number), + EpochHash: l2A1.L1Origin.Hash, + Timestamp: l2A1.Time, + Transactions: nil, + }, + }), + }, + Expected: BatchDrop, + }, + { + Name: "sequence window expired", + L1Blocks: []eth.L1BlockRef{l1A, l1B, l1C, l1D, l1E, l1F}, + L2SafeHead: l2A0, + Batch: BatchWithL1InclusionBlock{ + L1InclusionBlock: l1F, + Batch: NewSpanBatch([]*SingularBatch{ + { + ParentHash: l2A1.ParentHash, + EpochNum: rollup.Epoch(l2A1.L1Origin.Number), + EpochHash: l2A1.L1Origin.Hash, + Timestamp: l2A1.Time, + Transactions: nil, + }, + }), + }, + Expected: BatchDrop, + }, + { + Name: "epoch too old, but good parent hash and timestamp", // repeat of now outdated l2A3 data + L1Blocks: []eth.L1BlockRef{l1B, l1C, l1D}, + L2SafeHead: l2B0, // we already moved on to B + Batch: BatchWithL1InclusionBlock{ + L1InclusionBlock: l1C, + Batch: NewSpanBatch([]*SingularBatch{ + { + ParentHash: l2B0.Hash, // build on top of safe head to continue + EpochNum: rollup.Epoch(l2A3.L1Origin.Number), // epoch A is no longer valid + EpochHash: l2A3.L1Origin.Hash, + Timestamp: l2B0.Time + conf.BlockTime, // pass the timestamp check to get too epoch check + Transactions: nil, + }, + { + EpochNum: rollup.Epoch(l1B.Number), + EpochHash: l1B.Hash, // pass the l1 origin check + Timestamp: l2B0.Time + conf.BlockTime*2, + Transactions: nil, + }, + }), + }, + Expected: BatchDrop, + }, + { + Name: "insufficient L1 info for eager derivation", + L1Blocks: []eth.L1BlockRef{l1A}, // don't know about l1B yet + L2SafeHead: l2A3, + Batch: BatchWithL1InclusionBlock{ + L1InclusionBlock: l1C, + Batch: NewSpanBatch([]*SingularBatch{ + { + ParentHash: l2B0.ParentHash, + EpochNum: rollup.Epoch(l2B0.L1Origin.Number), + EpochHash: l2B0.L1Origin.Hash, + Timestamp: l2B0.Time, + Transactions: nil, + }, + }), + }, + Expected: BatchUndecided, + }, + { + Name: "epoch too new", + L1Blocks: []eth.L1BlockRef{l1A, l1B, l1C, l1D}, + L2SafeHead: l2A3, + Batch: BatchWithL1InclusionBlock{ + L1InclusionBlock: l1D, + Batch: NewSpanBatch([]*SingularBatch{ + { + ParentHash: l2B0.ParentHash, + EpochNum: rollup.Epoch(l1C.Number), // invalid, we need to adopt epoch B before C + EpochHash: l1C.Hash, + Timestamp: l2B0.Time, + Transactions: nil, + }, + }), + }, + Expected: BatchDrop, + }, + { + Name: "epoch hash wrong", + L1Blocks: []eth.L1BlockRef{l1A, l1B}, + L2SafeHead: l2A3, + Batch: BatchWithL1InclusionBlock{ + L1InclusionBlock: l1C, + Batch: NewSpanBatch([]*SingularBatch{ + { + ParentHash: l2B0.ParentHash, + EpochNum: rollup.Epoch(l2B0.L1Origin.Number), + EpochHash: l1A.Hash, // invalid, epoch hash should be l1B + Timestamp: l2B0.Time, + Transactions: nil, + }, + }), + }, + Expected: BatchDrop, + }, + { + Name: "epoch hash wrong - long span", + L1Blocks: []eth.L1BlockRef{l1A, l1B}, + L2SafeHead: l2A2, + Batch: BatchWithL1InclusionBlock{ + L1InclusionBlock: l1C, + Batch: NewSpanBatch([]*SingularBatch{ + { // valid batch + ParentHash: l2A3.ParentHash, + EpochNum: rollup.Epoch(l2A3.L1Origin.Number), + EpochHash: l1A.Hash, + Timestamp: l2A3.Time, + Transactions: nil, + }, + { + ParentHash: l2B0.ParentHash, + EpochNum: rollup.Epoch(l2B0.L1Origin.Number), + EpochHash: l1A.Hash, // invalid, epoch hash should be l1B + Timestamp: l2B0.Time, + Transactions: nil, + }, + }), + }, + Expected: BatchDrop, + }, + { + Name: "sequencer time drift on same epoch with non-empty txs", + L1Blocks: []eth.L1BlockRef{l1A, l1B}, + L2SafeHead: l2A3, + Batch: BatchWithL1InclusionBlock{ + L1InclusionBlock: l1B, + Batch: NewSpanBatch([]*SingularBatch{ + { // we build l2A4, which has a timestamp of 2*4 = 8 higher than l2A0 + ParentHash: l2A4.ParentHash, + EpochNum: rollup.Epoch(l2A4.L1Origin.Number), + EpochHash: l2A4.L1Origin.Hash, + Timestamp: l2A4.Time, + Transactions: []hexutil.Bytes{randTxData}, + }, + }), + }, + Expected: BatchDrop, + }, + { + Name: "sequencer time drift on same epoch with non-empty txs - long span", + L1Blocks: []eth.L1BlockRef{l1A, l1B}, + L2SafeHead: l2A2, + Batch: BatchWithL1InclusionBlock{ + L1InclusionBlock: l1B, + Batch: NewSpanBatch([]*SingularBatch{ + { // valid batch + ParentHash: l2A3.ParentHash, + EpochNum: rollup.Epoch(l2A3.L1Origin.Number), + EpochHash: l2A3.L1Origin.Hash, + Timestamp: l2A3.Time, + Transactions: []hexutil.Bytes{randTxData}, + }, + { // we build l2A4, which has a timestamp of 2*4 = 8 higher than l2A0 + ParentHash: l2A4.ParentHash, + EpochNum: rollup.Epoch(l2A4.L1Origin.Number), + EpochHash: l2A4.L1Origin.Hash, + Timestamp: l2A4.Time, + Transactions: []hexutil.Bytes{randTxData}, + }, + }), + }, + Expected: BatchDrop, + }, + { + Name: "sequencer time drift on changing epoch with non-empty txs", + L1Blocks: []eth.L1BlockRef{l1X, l1Y, l1Z}, + L2SafeHead: l2X0, + Batch: BatchWithL1InclusionBlock{ + L1InclusionBlock: l1Z, + Batch: NewSpanBatch([]*SingularBatch{ + { + ParentHash: l2Y0.ParentHash, + EpochNum: rollup.Epoch(l2Y0.L1Origin.Number), + EpochHash: l2Y0.L1Origin.Hash, + Timestamp: l2Y0.Time, // valid, but more than 6 ahead of l1Y.Time + Transactions: []hexutil.Bytes{randTxData}, + }, + }), + }, + Expected: BatchDrop, + }, + { + Name: "sequencer time drift on same epoch with empty txs and late next epoch", + L1Blocks: []eth.L1BlockRef{l1A, l1BLate}, + L2SafeHead: l2A3, + Batch: BatchWithL1InclusionBlock{ + L1InclusionBlock: l1BLate, + Batch: NewSpanBatch([]*SingularBatch{ + { // l2A4 time < l1BLate time, so we cannot adopt origin B yet + ParentHash: l2A4.ParentHash, + EpochNum: rollup.Epoch(l2A4.L1Origin.Number), + EpochHash: l2A4.L1Origin.Hash, + Timestamp: l2A4.Time, + Transactions: nil, + }, + }), + }, + Expected: BatchAccept, // accepted because empty & preserving L2 time invariant + }, + { + Name: "sequencer time drift on changing epoch with empty txs", + L1Blocks: []eth.L1BlockRef{l1X, l1Y, l1Z}, + L2SafeHead: l2X0, + Batch: BatchWithL1InclusionBlock{ + L1InclusionBlock: l1Z, + Batch: NewSpanBatch([]*SingularBatch{ + { + ParentHash: l2Y0.ParentHash, + EpochNum: rollup.Epoch(l2Y0.L1Origin.Number), + EpochHash: l2Y0.L1Origin.Hash, + Timestamp: l2Y0.Time, // valid, but more than 6 ahead of l1Y.Time + Transactions: nil, + }, + }), + }, + Expected: BatchAccept, // accepted because empty & still advancing epoch + }, + { + Name: "sequencer time drift on same epoch with empty txs and no next epoch in sight yet", + L1Blocks: []eth.L1BlockRef{l1A}, + L2SafeHead: l2A3, + Batch: BatchWithL1InclusionBlock{ + L1InclusionBlock: l1B, + Batch: NewSpanBatch([]*SingularBatch{ + { // we build l2A4, which has a timestamp of 2*4 = 8 higher than l2A0 + ParentHash: l2A4.ParentHash, + EpochNum: rollup.Epoch(l2A4.L1Origin.Number), + EpochHash: l2A4.L1Origin.Hash, + Timestamp: l2A4.Time, + Transactions: nil, + }, + }), + }, + Expected: BatchUndecided, // we have to wait till the next epoch is in sight to check the time + }, + { + Name: "sequencer time drift on same epoch with empty txs and no next epoch in sight yet - long span", + L1Blocks: []eth.L1BlockRef{l1A}, + L2SafeHead: l2A2, + Batch: BatchWithL1InclusionBlock{ + L1InclusionBlock: l1B, + Batch: NewSpanBatch([]*SingularBatch{ + { // valid batch + ParentHash: l2A3.ParentHash, + EpochNum: rollup.Epoch(l2A3.L1Origin.Number), + EpochHash: l2A3.L1Origin.Hash, + Timestamp: l2A3.Time, + Transactions: nil, + }, + { // we build l2A4, which has a timestamp of 2*4 = 8 higher than l2A0 + ParentHash: l2A4.ParentHash, + EpochNum: rollup.Epoch(l2A4.L1Origin.Number), + EpochHash: l2A4.L1Origin.Hash, + Timestamp: l2A4.Time, + Transactions: nil, + }, + }), + }, + Expected: BatchUndecided, // we have to wait till the next epoch is in sight to check the time + }, + { + Name: "sequencer time drift on same epoch with empty txs and but in-sight epoch that invalidates it", + L1Blocks: []eth.L1BlockRef{l1A, l1B, l1C}, + L2SafeHead: l2A3, + Batch: BatchWithL1InclusionBlock{ + L1InclusionBlock: l1C, + Batch: NewSpanBatch([]*SingularBatch{ + { // we build l2A4, which has a timestamp of 2*4 = 8 higher than l2A0 + ParentHash: l2A4.ParentHash, + EpochNum: rollup.Epoch(l2A4.L1Origin.Number), + EpochHash: l2A4.L1Origin.Hash, + Timestamp: l2A4.Time, + Transactions: nil, + }, + }), + }, + Expected: BatchDrop, // dropped because it could have advanced the epoch to B + }, + { + Name: "sequencer time drift on same epoch with empty txs and but in-sight epoch that invalidates it - long span", + L1Blocks: []eth.L1BlockRef{l1A, l1B, l1C}, + L2SafeHead: l2A2, + Batch: BatchWithL1InclusionBlock{ + L1InclusionBlock: l1C, + Batch: NewSpanBatch([]*SingularBatch{ + { // valid batch + ParentHash: l2A3.ParentHash, + EpochNum: rollup.Epoch(l2A3.L1Origin.Number), + EpochHash: l2A3.L1Origin.Hash, + Timestamp: l2A3.Time, + Transactions: nil, + }, + { // we build l2A4, which has a timestamp of 2*4 = 8 higher than l2A0 + ParentHash: l2A4.ParentHash, + EpochNum: rollup.Epoch(l2A4.L1Origin.Number), + EpochHash: l2A4.L1Origin.Hash, + Timestamp: l2A4.Time, + Transactions: nil, + }, + }), + }, + Expected: BatchDrop, // dropped because it could have advanced the epoch to B + }, + { + Name: "empty tx included", + L1Blocks: []eth.L1BlockRef{l1A, l1B}, + L2SafeHead: l2A0, + Batch: BatchWithL1InclusionBlock{ + L1InclusionBlock: l1B, + Batch: NewSpanBatch([]*SingularBatch{ + { + ParentHash: l2A1.ParentHash, + EpochNum: rollup.Epoch(l2A1.L1Origin.Number), + EpochHash: l2A1.L1Origin.Hash, + Timestamp: l2A1.Time, + Transactions: []hexutil.Bytes{ + []byte{}, // empty tx data + }, + }, + }), + }, + Expected: BatchDrop, + }, + { + Name: "deposit tx included", + L1Blocks: []eth.L1BlockRef{l1A, l1B}, + L2SafeHead: l2A0, + Batch: BatchWithL1InclusionBlock{ + L1InclusionBlock: l1B, + Batch: NewSpanBatch([]*SingularBatch{ + { + ParentHash: l2A1.ParentHash, + EpochNum: rollup.Epoch(l2A1.L1Origin.Number), + EpochHash: l2A1.L1Origin.Hash, + Timestamp: l2A1.Time, + Transactions: []hexutil.Bytes{ + []byte{types.DepositTxType, 0}, // piece of data alike to a deposit + }, + }, + }), + }, + Expected: BatchDrop, + }, + { + Name: "valid batch same epoch", + L1Blocks: []eth.L1BlockRef{l1A, l1B}, + L2SafeHead: l2A0, + Batch: BatchWithL1InclusionBlock{ + L1InclusionBlock: l1B, + Batch: NewSpanBatch([]*SingularBatch{ + { + ParentHash: l2A1.ParentHash, + EpochNum: rollup.Epoch(l2A1.L1Origin.Number), + EpochHash: l2A1.L1Origin.Hash, + Timestamp: l2A1.Time, + Transactions: []hexutil.Bytes{randTxData}, + }, + }), + }, + Expected: BatchAccept, + }, + { + Name: "valid batch changing epoch", + L1Blocks: []eth.L1BlockRef{l1A, l1B, l1C}, + L2SafeHead: l2A3, + Batch: BatchWithL1InclusionBlock{ + L1InclusionBlock: l1C, + Batch: NewSpanBatch([]*SingularBatch{ + { + ParentHash: l2B0.ParentHash, + EpochNum: rollup.Epoch(l2B0.L1Origin.Number), + EpochHash: l2B0.L1Origin.Hash, + Timestamp: l2B0.Time, + Transactions: []hexutil.Bytes{randTxData}, + }, + }), + }, + Expected: BatchAccept, + }, + { + Name: "batch with L2 time before L1 time", + L1Blocks: []eth.L1BlockRef{l1A, l1B, l1C}, + L2SafeHead: l2A2, + Batch: BatchWithL1InclusionBlock{ + L1InclusionBlock: l1B, + Batch: NewSpanBatch([]*SingularBatch{ + { // we build l2B0, which starts a new epoch too early + ParentHash: l2A2.Hash, + EpochNum: rollup.Epoch(l2B0.L1Origin.Number), + EpochHash: l2B0.L1Origin.Hash, + Timestamp: l2A2.Time + conf.BlockTime, + Transactions: nil, + }, + }), + }, + Expected: BatchDrop, + }, + { + Name: "batch with L2 time before L1 time - long span", + L1Blocks: []eth.L1BlockRef{l1A, l1B, l1C}, + L2SafeHead: l2A1, + Batch: BatchWithL1InclusionBlock{ + L1InclusionBlock: l1B, + Batch: NewSpanBatch([]*SingularBatch{ + { // valid batch + ParentHash: l2A1.Hash, + EpochNum: rollup.Epoch(l2A2.L1Origin.Number), + EpochHash: l2A2.L1Origin.Hash, + Timestamp: l2A2.Time, + Transactions: nil, + }, + { // we build l2B0, which starts a new epoch too early + ParentHash: l2A2.Hash, + EpochNum: rollup.Epoch(l2B0.L1Origin.Number), + EpochHash: l2B0.L1Origin.Hash, + Timestamp: l2A2.Time + conf.BlockTime, + Transactions: nil, + }, + }), + }, + Expected: BatchDrop, + }, + { + Name: "valid overlapping batch", + L1Blocks: []eth.L1BlockRef{l1A, l1B}, + L2SafeHead: l2A2, + Batch: BatchWithL1InclusionBlock{ + L1InclusionBlock: l1B, + Batch: NewSpanBatch([]*SingularBatch{ + { + ParentHash: l2A1.Hash, + EpochNum: rollup.Epoch(l2A2.L1Origin.Number), + EpochHash: l2A2.L1Origin.Hash, + Timestamp: l2A2.Time, + Transactions: nil, + }, + { + ParentHash: l2A2.Hash, + EpochNum: rollup.Epoch(l2A3.L1Origin.Number), + EpochHash: l2A3.L1Origin.Hash, + Timestamp: l2A3.Time, + Transactions: nil, + }, + }), + }, + Expected: BatchAccept, + }, + { + Name: "longer overlapping batch", + L1Blocks: []eth.L1BlockRef{l1A, l1B}, + L2SafeHead: l2A2, + Batch: BatchWithL1InclusionBlock{ + L1InclusionBlock: l1B, + Batch: NewSpanBatch([]*SingularBatch{ + { + ParentHash: l2A0.Hash, + EpochNum: rollup.Epoch(l2A1.L1Origin.Number), + EpochHash: l2A1.L1Origin.Hash, + Timestamp: l2A1.Time, + Transactions: nil, + }, + { + ParentHash: l2A1.Hash, + EpochNum: rollup.Epoch(l2A2.L1Origin.Number), + EpochHash: l2A2.L1Origin.Hash, + Timestamp: l2A2.Time, + Transactions: nil, + }, + { + ParentHash: l2A2.Hash, + EpochNum: rollup.Epoch(l2A3.L1Origin.Number), + EpochHash: l2A3.L1Origin.Hash, + Timestamp: l2A3.Time, + Transactions: nil, + }, + }), + }, + Expected: BatchAccept, + }, + { + Name: "fully overlapping batch", + L1Blocks: []eth.L1BlockRef{l1A, l1B}, + L2SafeHead: l2A2, + Batch: BatchWithL1InclusionBlock{ + L1InclusionBlock: l1B, + Batch: NewSpanBatch([]*SingularBatch{ + { + ParentHash: l2A0.Hash, + EpochNum: rollup.Epoch(l2A1.L1Origin.Number), + EpochHash: l2A1.L1Origin.Hash, + Timestamp: l2A1.Time, + Transactions: nil, + }, + { + ParentHash: l2A1.Hash, + EpochNum: rollup.Epoch(l2A2.L1Origin.Number), + EpochHash: l2A2.L1Origin.Hash, + Timestamp: l2A2.Time, + Transactions: nil, + }, + }), + }, + Expected: BatchDrop, + }, + { + Name: "overlapping batch with invalid parent hash", + L1Blocks: []eth.L1BlockRef{l1A, l1B}, + L2SafeHead: l2A2, + Batch: BatchWithL1InclusionBlock{ + L1InclusionBlock: l1B, + Batch: NewSpanBatch([]*SingularBatch{ + { + ParentHash: l2A0.Hash, + EpochNum: rollup.Epoch(l2A2.L1Origin.Number), + EpochHash: l2A2.L1Origin.Hash, + Timestamp: l2A2.Time, + Transactions: nil, + }, + { + ParentHash: l2A2.Hash, + EpochNum: rollup.Epoch(l2A3.L1Origin.Number), + EpochHash: l2A3.L1Origin.Hash, + Timestamp: l2A3.Time, + Transactions: nil, + }, + }), + }, + Expected: BatchDrop, + }, + { + Name: "overlapping batch with invalid origin number", + L1Blocks: []eth.L1BlockRef{l1A, l1B}, + L2SafeHead: l2A2, + Batch: BatchWithL1InclusionBlock{ + L1InclusionBlock: l1B, + Batch: NewSpanBatch([]*SingularBatch{ + { + ParentHash: l2A1.Hash, + EpochNum: rollup.Epoch(l2A2.L1Origin.Number) + 1, + EpochHash: l2A2.L1Origin.Hash, + Timestamp: l2A2.Time, + Transactions: nil, + }, + { + ParentHash: l2A2.Hash, + EpochNum: rollup.Epoch(l2A3.L1Origin.Number), + EpochHash: l2A3.L1Origin.Hash, + Timestamp: l2A3.Time, + Transactions: nil, + }, + }), + }, + Expected: BatchDrop, + }, + { + Name: "overlapping batch with invalid tx", + L1Blocks: []eth.L1BlockRef{l1A, l1B}, + L2SafeHead: l2A2, + Batch: BatchWithL1InclusionBlock{ + L1InclusionBlock: l1B, + Batch: NewSpanBatch([]*SingularBatch{ + { + ParentHash: l2A1.Hash, + EpochNum: rollup.Epoch(l2A2.L1Origin.Number), + EpochHash: l2A2.L1Origin.Hash, + Timestamp: l2A2.Time, + Transactions: []hexutil.Bytes{randTxData}, + }, + { + ParentHash: l2A2.Hash, + EpochNum: rollup.Epoch(l2A3.L1Origin.Number), + EpochHash: l2A3.L1Origin.Hash, + Timestamp: l2A3.Time, + Transactions: nil, + }, + }), + }, + Expected: BatchDrop, + }, + { + Name: "overlapping batch l2 fetcher error", + L1Blocks: []eth.L1BlockRef{l1A, l1B}, + L2SafeHead: l2A1, + Batch: BatchWithL1InclusionBlock{ + L1InclusionBlock: l1B, + Batch: NewSpanBatch([]*SingularBatch{ + { + ParentHash: l2A0.ParentHash, + EpochNum: rollup.Epoch(l2A0.L1Origin.Number), + EpochHash: l2A0.L1Origin.Hash, + Timestamp: l2A0.Time, + Transactions: nil, + }, + { + ParentHash: l2A0.Hash, + EpochNum: rollup.Epoch(l2A1.L1Origin.Number), + EpochHash: l2A1.L1Origin.Hash, + Timestamp: l2A1.Time, + Transactions: nil, + }, + { + ParentHash: l2A1.Hash, + EpochNum: rollup.Epoch(l2A2.L1Origin.Number), + EpochHash: l2A2.L1Origin.Hash, + Timestamp: l2A2.Time, + Transactions: nil, + }, + }), + }, + Expected: BatchUndecided, + }, + } + + // Log level can be increased for debugging purposes + logger := testlog.Logger(t, log.LvlError) + + l2Client := testutils.MockL2Client{} + var nilErr error + // will be return error for block #99 (parent of l2A0) + tempErr := errors.New("temp error") + l2Client.Mock.On("L2BlockRefByNumber", l2A0.Number-1).Times(9999).Return(eth.L2BlockRef{}, &tempErr) + l2Client.Mock.On("PayloadByNumber", l2A0.Number-1).Times(9999).Return(nil, &tempErr) + + // make payloads for L2 blocks and set as expected return value of MockL2Client + for _, l2Block := range []eth.L2BlockRef{l2A0, l2A1, l2A2, l2A3, l2A4, l2B0} { + l2Client.ExpectL2BlockRefByNumber(l2Block.Number, l2Block, nil) + txData := l1InfoDepositTx(t, l2Block.L1Origin.Number) + payload := eth.ExecutionPayload{ + ParentHash: l2Block.ParentHash, + BlockNumber: hexutil.Uint64(l2Block.Number), + Timestamp: hexutil.Uint64(l2Block.Time), + BlockHash: l2Block.Hash, + Transactions: []hexutil.Bytes{txData}, + } + l2Client.Mock.On("L2BlockRefByNumber", l2Block.Number).Times(9999).Return(l2Block, &nilErr) + l2Client.Mock.On("PayloadByNumber", l2Block.Number).Times(9999).Return(&payload, &nilErr) + } + + for _, testCase := range testCases { + t.Run(testCase.Name, func(t *testing.T) { + ctx := context.Background() + validity := CheckBatch(ctx, &conf, logger, testCase.L1Blocks, testCase.L2SafeHead, &testCase.Batch, &l2Client) + require.Equal(t, testCase.Expected, validity, "batch check must return expected validity level") + }) + } +} + +func TestSpanBatchHardFork(t *testing.T) { + minTs := uint64(0) + conf := rollup.Config{ + Genesis: rollup.Genesis{ + L2Time: 31, // a genesis time that itself does not align to make it more interesting + }, + BlockTime: 2, + SeqWindowSize: 4, + MaxSequencerDrift: 6, + SpanBatchTime: &minTs, + // other config fields are ignored and can be left empty. + } + + rng := rand.New(rand.NewSource(1234)) + chainId := new(big.Int).SetUint64(rng.Uint64()) + signer := types.NewLondonSigner(chainId) + randTx := testutils.RandomTx(rng, new(big.Int).SetUint64(rng.Uint64()), signer) + randTxData, _ := randTx.MarshalBinary() + l1A := testutils.RandomBlockRef(rng) + l1B := eth.L1BlockRef{ + Hash: testutils.RandomHash(rng), + Number: l1A.Number + 1, + ParentHash: l1A.Hash, + Time: l1A.Time + 7, + } + + l2A0 := eth.L2BlockRef{ + Hash: testutils.RandomHash(rng), + Number: 100, + ParentHash: testutils.RandomHash(rng), + Time: l1A.Time, + L1Origin: l1A.ID(), + SequenceNumber: 0, + } + + l2A1 := eth.L2BlockRef{ + Hash: testutils.RandomHash(rng), + Number: l2A0.Number + 1, + ParentHash: l2A0.Hash, + Time: l2A0.Time + conf.BlockTime, + L1Origin: l1A.ID(), + SequenceNumber: 1, + } + + testCases := []SpanBatchHardForkTestCase{ + { + Name: "singular batch before hard fork", + L1Blocks: []eth.L1BlockRef{l1A, l1B}, + L2SafeHead: l2A0, + Batch: BatchWithL1InclusionBlock{ + L1InclusionBlock: l1B, + Batch: &SingularBatch{ + ParentHash: l2A1.ParentHash, + EpochNum: rollup.Epoch(l2A1.L1Origin.Number), + EpochHash: l2A1.L1Origin.Hash, + Timestamp: l2A1.Time, + Transactions: []hexutil.Bytes{randTxData}, + }, + }, + SpanBatchTime: l2A1.Time + 2, + Expected: BatchAccept, + }, + { + Name: "span batch before hard fork", + L1Blocks: []eth.L1BlockRef{l1A, l1B}, + L2SafeHead: l2A0, + Batch: BatchWithL1InclusionBlock{ + L1InclusionBlock: l1B, + Batch: NewSpanBatch([]*SingularBatch{ + { + ParentHash: l2A1.ParentHash, + EpochNum: rollup.Epoch(l2A1.L1Origin.Number), + EpochHash: l2A1.L1Origin.Hash, + Timestamp: l2A1.Time, + Transactions: []hexutil.Bytes{randTxData}, + }, + }), + }, + SpanBatchTime: l2A1.Time + 2, + Expected: BatchDrop, + }, + { + Name: "singular batch after hard fork", + L1Blocks: []eth.L1BlockRef{l1A, l1B}, + L2SafeHead: l2A0, + Batch: BatchWithL1InclusionBlock{ + L1InclusionBlock: l1B, + Batch: &SingularBatch{ + ParentHash: l2A1.ParentHash, + EpochNum: rollup.Epoch(l2A1.L1Origin.Number), + EpochHash: l2A1.L1Origin.Hash, + Timestamp: l2A1.Time, + Transactions: []hexutil.Bytes{randTxData}, + }, + }, + SpanBatchTime: l2A1.Time - 2, + Expected: BatchAccept, + }, + { + Name: "span batch after hard fork", + L1Blocks: []eth.L1BlockRef{l1A, l1B}, + L2SafeHead: l2A0, + Batch: BatchWithL1InclusionBlock{ + L1InclusionBlock: l1B, + Batch: NewSpanBatch([]*SingularBatch{ + { + ParentHash: l2A1.ParentHash, + EpochNum: rollup.Epoch(l2A1.L1Origin.Number), + EpochHash: l2A1.L1Origin.Hash, + Timestamp: l2A1.Time, + Transactions: []hexutil.Bytes{randTxData}, + }, + }), + }, + SpanBatchTime: l2A1.Time - 2, + Expected: BatchAccept, + }, + } + + // Log level can be increased for debugging purposes + logger := testlog.Logger(t, log.LvlInfo) + + for _, testCase := range testCases { + t.Run(testCase.Name, func(t *testing.T) { + rcfg := conf + rcfg.SpanBatchTime = &testCase.SpanBatchTime + ctx := context.Background() + validity := CheckBatch(ctx, &rcfg, logger, testCase.L1Blocks, testCase.L2SafeHead, &testCase.Batch, nil) require.Equal(t, testCase.Expected, validity, "batch check must return expected validity level") }) } diff --git a/op-node/rollup/derive/channel.go b/op-node/rollup/derive/channel.go index 1d9a85cc9027..7bd67f184995 100644 --- a/op-node/rollup/derive/channel.go +++ b/op-node/rollup/derive/channel.go @@ -6,8 +6,6 @@ import ( "fmt" "io" - "github.com/ethereum-optimism/optimism/op-node/rollup" - "github.com/ethereum-optimism/optimism/op-service/eth" "github.com/ethereum/go-ethereum/rlp" ) @@ -146,7 +144,9 @@ func (ch *Channel) Reader() io.Reader { // BatchReader provides a function that iteratively consumes batches from the reader. // The L1Inclusion block is also provided at creation time. -func BatchReader(cfg *rollup.Config, r io.Reader, l1InclusionBlock eth.L1BlockRef) (func() (BatchWithL1InclusionBlock, error), error) { +// Warning: the batch reader can read every batch-type. +// The caller of the batch-reader should filter the results. +func BatchReader(r io.Reader) (func() (*BatchData, error), error) { // Setup decompressor stage + RLP reader zr, err := zlib.NewReader(r) if err != nil { @@ -154,17 +154,11 @@ func BatchReader(cfg *rollup.Config, r io.Reader, l1InclusionBlock eth.L1BlockRe } rlpReader := rlp.NewStream(zr, MaxRLPBytesPerChannel) // Read each batch iteratively - return func() (BatchWithL1InclusionBlock, error) { - ret := BatchWithL1InclusionBlock{ - L1InclusionBlock: l1InclusionBlock, - } - err := rlpReader.Decode(&ret.Batch) - if err != nil { - return ret, err - } - if ret.Batch.BatchType == SpanBatchType && !cfg.IsSpanBatch(ret.L1InclusionBlock.Time) { - return ret, fmt.Errorf("cannot accept span-batch in L1 block with time %d", ret.L1InclusionBlock.Time) + return func() (*BatchData, error) { + var batchData BatchData + if err = rlpReader.Decode(&batchData); err != nil { + return nil, err } - return ret, nil + return &batchData, nil }, nil } diff --git a/op-node/rollup/derive/channel_in_reader.go b/op-node/rollup/derive/channel_in_reader.go index 7487326d99bf..1553bd9a6656 100644 --- a/op-node/rollup/derive/channel_in_reader.go +++ b/op-node/rollup/derive/channel_in_reader.go @@ -3,12 +3,12 @@ package derive import ( "bytes" "context" + "fmt" "io" - "github.com/ethereum-optimism/optimism/op-node/rollup" - "github.com/ethereum/go-ethereum/log" + "github.com/ethereum-optimism/optimism/op-node/rollup" "github.com/ethereum-optimism/optimism/op-service/eth" ) @@ -21,7 +21,7 @@ type ChannelInReader struct { cfg *rollup.Config - nextBatchFn func() (BatchWithL1InclusionBlock, error) + nextBatchFn func() (*BatchData, error) prev *ChannelBank @@ -46,7 +46,7 @@ func (cr *ChannelInReader) Origin() eth.L1BlockRef { // TODO: Take full channel for better logging func (cr *ChannelInReader) WriteChannel(data []byte) error { - if f, err := BatchReader(cr.cfg, bytes.NewBuffer(data), cr.Origin()); err == nil { + if f, err := BatchReader(bytes.NewBuffer(data)); err == nil { cr.nextBatchFn = f cr.metrics.RecordChannelInputBytes(len(data)) return nil @@ -65,7 +65,7 @@ func (cr *ChannelInReader) NextChannel() { // NextBatch pulls out the next batch from the channel if it has it. // It returns io.EOF when it cannot make any more progress. // It will return a temporary error if it needs to be called again to advance some internal state. -func (cr *ChannelInReader) NextBatch(ctx context.Context) (*BatchData, error) { +func (cr *ChannelInReader) NextBatch(ctx context.Context) (Batch, error) { if cr.nextBatchFn == nil { if data, err := cr.prev.NextData(ctx); err == io.EOF { return nil, io.EOF @@ -80,7 +80,7 @@ func (cr *ChannelInReader) NextBatch(ctx context.Context) (*BatchData, error) { // TODO: can batch be non nil while err == io.EOF // This depends on the behavior of rlp.Stream - batch, err := cr.nextBatchFn() + batchData, err := cr.nextBatchFn() if err == io.EOF { cr.NextChannel() return nil, NotEnoughData @@ -89,7 +89,23 @@ func (cr *ChannelInReader) NextBatch(ctx context.Context) (*BatchData, error) { cr.NextChannel() return nil, NotEnoughData } - return batch.Batch, nil + switch batchData.BatchType { + case SingularBatchType: + return &batchData.SingularBatch, nil + case SpanBatchType: + if origin := cr.Origin(); !cr.cfg.IsSpanBatch(origin.Time) { + return nil, NewTemporaryError(fmt.Errorf("cannot accept span batch in L1 block %s at time %d", origin, origin.Time)) + } + // If the batch type is Span batch, derive block inputs from RawSpanBatch. + spanBatch, err := batchData.RawSpanBatch.derive(cr.cfg.BlockTime, cr.cfg.Genesis.L2Time, cr.cfg.L2ChainID) + if err != nil { + return nil, err + } + return spanBatch, nil + default: + // error is bubbled up to user, but pipeline can skip the batch and continue after. + return nil, NewTemporaryError(fmt.Errorf("unrecognized batch type: %w", err)) + } } func (cr *ChannelInReader) Reset(ctx context.Context, _ eth.L1BlockRef, _ eth.SystemConfig) error { diff --git a/op-node/rollup/derive/engine_queue.go b/op-node/rollup/derive/engine_queue.go index 775ab64a8959..b8cd3ae5917f 100644 --- a/op-node/rollup/derive/engine_queue.go +++ b/op-node/rollup/derive/engine_queue.go @@ -35,6 +35,7 @@ type Engine interface { PayloadByNumber(context.Context, uint64) (*eth.ExecutionPayload, error) L2BlockRefByLabel(ctx context.Context, label eth.BlockLabel) (eth.L2BlockRef, error) L2BlockRefByHash(ctx context.Context, l2Hash common.Hash) (eth.L2BlockRef, error) + L2BlockRefByNumber(ctx context.Context, num uint64) (eth.L2BlockRef, error) SystemConfigL2Fetcher } diff --git a/op-node/rollup/derive/pipeline.go b/op-node/rollup/derive/pipeline.go index 7b017515b763..d054939f151d 100644 --- a/op-node/rollup/derive/pipeline.go +++ b/op-node/rollup/derive/pipeline.go @@ -90,7 +90,7 @@ func NewDerivationPipeline(log log.Logger, cfg *rollup.Config, l1Fetcher L1Fetch frameQueue := NewFrameQueue(log, l1Src) bank := NewChannelBank(log, cfg, frameQueue, l1Fetcher, metrics) chInReader := NewChannelInReader(cfg, log, bank, metrics) - batchQueue := NewBatchQueue(log, cfg, chInReader) + batchQueue := NewBatchQueue(log, cfg, chInReader, engine) attrBuilder := NewFetchingAttributesBuilder(cfg, l1Fetcher, engine) attributesQueue := NewAttributesQueue(log, cfg, attrBuilder, batchQueue) diff --git a/op-program/client/l2/engine.go b/op-program/client/l2/engine.go index 3ed4476b71ad..e6ffadebf470 100644 --- a/op-program/client/l2/engine.go +++ b/op-program/client/l2/engine.go @@ -109,6 +109,14 @@ func (o *OracleEngine) L2BlockRefByHash(ctx context.Context, l2Hash common.Hash) return derive.L2BlockToBlockRef(block, &o.rollupCfg.Genesis) } +func (o *OracleEngine) L2BlockRefByNumber(ctx context.Context, n uint64) (eth.L2BlockRef, error) { + hash := o.backend.GetCanonicalHash(n) + if hash == (common.Hash{}) { + return eth.L2BlockRef{}, ErrNotFound + } + return o.L2BlockRefByHash(ctx, hash) +} + func (o *OracleEngine) SystemConfigByL2Hash(ctx context.Context, hash common.Hash) (eth.SystemConfig, error) { payload, err := o.PayloadByHash(ctx, hash) if err != nil { diff --git a/op-service/testutils/mock_l2.go b/op-service/testutils/mock_l2.go index 18f5f4c82b1c..6b01b401a402 100644 --- a/op-service/testutils/mock_l2.go +++ b/op-service/testutils/mock_l2.go @@ -3,9 +3,8 @@ package testutils import ( "context" - "github.com/ethereum/go-ethereum/common" - "github.com/ethereum-optimism/optimism/op-service/eth" + "github.com/ethereum/go-ethereum/common" ) type MockL2Client struct { @@ -13,7 +12,8 @@ type MockL2Client struct { } func (c *MockL2Client) L2BlockRefByLabel(ctx context.Context, label eth.BlockLabel) (eth.L2BlockRef, error) { - return c.Mock.MethodCalled("L2BlockRefByLabel", label).Get(0).(eth.L2BlockRef), nil + out := c.Mock.MethodCalled("L2BlockRefByLabel", label) + return out[0].(eth.L2BlockRef), *out[1].(*error) } func (m *MockL2Client) ExpectL2BlockRefByLabel(label eth.BlockLabel, ref eth.L2BlockRef, err error) { @@ -21,7 +21,8 @@ func (m *MockL2Client) ExpectL2BlockRefByLabel(label eth.BlockLabel, ref eth.L2B } func (c *MockL2Client) L2BlockRefByNumber(ctx context.Context, num uint64) (eth.L2BlockRef, error) { - return c.Mock.MethodCalled("L2BlockRefByNumber", num).Get(0).(eth.L2BlockRef), nil + out := c.Mock.MethodCalled("L2BlockRefByNumber", num) + return out[0].(eth.L2BlockRef), *out[1].(*error) } func (m *MockL2Client) ExpectL2BlockRefByNumber(num uint64, ref eth.L2BlockRef, err error) { @@ -29,7 +30,8 @@ func (m *MockL2Client) ExpectL2BlockRefByNumber(num uint64, ref eth.L2BlockRef, } func (c *MockL2Client) L2BlockRefByHash(ctx context.Context, hash common.Hash) (eth.L2BlockRef, error) { - return c.Mock.MethodCalled("L2BlockRefByHash", hash).Get(0).(eth.L2BlockRef), nil + out := c.Mock.MethodCalled("L2BlockRefByHash", hash) + return out[0].(eth.L2BlockRef), *out[1].(*error) } func (m *MockL2Client) ExpectL2BlockRefByHash(hash common.Hash, ref eth.L2BlockRef, err error) { @@ -37,7 +39,8 @@ func (m *MockL2Client) ExpectL2BlockRefByHash(hash common.Hash, ref eth.L2BlockR } func (m *MockL2Client) SystemConfigByL2Hash(ctx context.Context, hash common.Hash) (eth.SystemConfig, error) { - return m.Mock.MethodCalled("SystemConfigByL2Hash", hash).Get(0).(eth.SystemConfig), nil + out := m.Mock.MethodCalled("SystemConfigByL2Hash", hash) + return out[0].(eth.SystemConfig), *out[1].(*error) } func (m *MockL2Client) ExpectSystemConfigByL2Hash(hash common.Hash, cfg eth.SystemConfig, err error) { @@ -45,7 +48,8 @@ func (m *MockL2Client) ExpectSystemConfigByL2Hash(hash common.Hash, cfg eth.Syst } func (m *MockL2Client) OutputV0AtBlock(ctx context.Context, blockHash common.Hash) (*eth.OutputV0, error) { - return m.Mock.MethodCalled("OutputV0AtBlock", blockHash).Get(0).(*eth.OutputV0), nil + out := m.Mock.MethodCalled("OutputV0AtBlock", blockHash) + return out[0].(*eth.OutputV0), *out[1].(*error) } func (m *MockL2Client) ExpectOutputV0AtBlock(blockHash common.Hash, output *eth.OutputV0, err error) { From 1267baddd031bd4eece30f05ca85f2fa81541bc3 Mon Sep 17 00:00:00 2001 From: Tei Im Date: Tue, 10 Oct 2023 19:30:46 +0900 Subject: [PATCH 048/279] Apply suggestions from code reivews Rename advanceEpoch() to advanceEpochMaybe() Set number of calls for L2 client mock and assert them in batch queue tests --- op-node/rollup/derive/batch_queue.go | 6 +-- op-node/rollup/derive/batch_queue_test.go | 52 ++++++++++++++++++----- 2 files changed, 45 insertions(+), 13 deletions(-) diff --git a/op-node/rollup/derive/batch_queue.go b/op-node/rollup/derive/batch_queue.go index 30dd1441924b..9805b1ee0103 100644 --- a/op-node/rollup/derive/batch_queue.go +++ b/op-node/rollup/derive/batch_queue.go @@ -78,7 +78,7 @@ func (bq *BatchQueue) popNextBatch(safeL2Head eth.L2BlockRef) *SingularBatch { return nextBatch } -func (bq *BatchQueue) advanceEpoch(nextBatch *SingularBatch) { +func (bq *BatchQueue) advanceEpochMaybe(nextBatch *SingularBatch) { if nextBatch.GetEpochNum() == rollup.Epoch(bq.l1Blocks[0].Number)+1 { // Advance epoch if necessary bq.l1Blocks = bq.l1Blocks[1:] @@ -89,7 +89,7 @@ func (bq *BatchQueue) NextBatch(ctx context.Context, safeL2Head eth.L2BlockRef) if len(bq.nextSpan) > 0 { // If there are cached singular batches, pop first one and return. nextBatch := bq.popNextBatch(safeL2Head) - bq.advanceEpoch(nextBatch) + bq.advanceEpochMaybe(nextBatch) return nextBatch, nil } @@ -169,7 +169,7 @@ func (bq *BatchQueue) NextBatch(ctx context.Context, safeL2Head eth.L2BlockRef) return nil, NewCriticalError(fmt.Errorf("unrecognized batch type: %d", batch.GetBatchType())) } - bq.advanceEpoch(nextBatch) + bq.advanceEpochMaybe(nextBatch) return nextBatch, nil } diff --git a/op-node/rollup/derive/batch_queue_test.go b/op-node/rollup/derive/batch_queue_test.go index dee33b866177..f2fbb1b65602 100644 --- a/op-node/rollup/derive/batch_queue_test.go +++ b/op-node/rollup/derive/batch_queue_test.go @@ -771,6 +771,13 @@ func TestBatchQueueOverlappingSpanBatch(t *testing.T) { inputBatches = append(inputBatches, NewSpanBatch(expectedOutputBatches[i:i+batchSize])) } inputBatches = append(inputBatches, nil) + // inputBatches: + // [ + // [12, 14, 16], // No overlap + // [14, 16, 18], // overlapped blocks: 14, 16 + // [16, 18, 20], // overlapped blocks: 16, 18 + // [18, 20, 22], // overlapped blocks: 18, 20 + // ] input := &fakeBatchQueueInput{ batches: inputBatches, @@ -784,8 +791,21 @@ func TestBatchQueueOverlappingSpanBatch(t *testing.T) { if batch != nil { blockRef := singularBatchToBlockRef(t, batch, uint64(i+1)) payload := singularBatchToPayload(t, batch, uint64(i+1)) - l2Client.Mock.On("L2BlockRefByNumber", uint64(i+1)).Times(9999).Return(blockRef, &nilErr) - l2Client.Mock.On("PayloadByNumber", uint64(i+1)).Times(9999).Return(&payload, &nilErr) + if i < 3 { + // In CheckBatch(), "L2BlockRefByNumber" is called when fetching the parent block of overlapped span batch + // so blocks at 12, 14, 16 should be called. + // CheckBatch() is called twice for a batch - before pushing to the queue, after popping from the queue + l2Client.Mock.On("L2BlockRefByNumber", uint64(i+1)).Times(2).Return(blockRef, &nilErr) + } + if i == 1 || i == 4 { + // In CheckBatch(), "PayloadByNumber" is called when fetching the overlapped blocks. + // blocks at 14, 20 are included in overlapped blocks once. + // CheckBatch() is called twice for a batch - before adding to the queue, after getting from the queue + l2Client.Mock.On("PayloadByNumber", uint64(i+1)).Times(2).Return(&payload, &nilErr) + } else if i == 2 || i == 3 { + // blocks at 16, 18 are included in overlapped blocks twice. + l2Client.Mock.On("PayloadByNumber", uint64(i+1)).Times(4).Return(&payload, &nilErr) + } } } @@ -807,6 +827,8 @@ func TestBatchQueueOverlappingSpanBatch(t *testing.T) { safeHead.L1Origin = b.Epoch() } } + + l2Client.Mock.AssertExpectations(t) } func TestBatchQueueComplex(t *testing.T) { @@ -851,12 +873,12 @@ func TestBatchQueueComplex(t *testing.T) { inputErrors := []error{nil, nil, nil, nil, nil, nil, io.EOF} // batches will be returned by fakeBatchQueueInput inputBatches := []Batch{ - NewSpanBatch(expectedOutputBatches[0:2]), // 6, 8 - expectedOutputBatches[2], // 10 - NewSpanBatch(expectedOutputBatches[1:4]), // 8, 10, 12 - expectedOutputBatches[4], // 14 - NewSpanBatch(expectedOutputBatches[4:6]), // 14, 16 - NewSpanBatch(expectedOutputBatches[6:9]), // 18, 20, 22 + NewSpanBatch(expectedOutputBatches[0:2]), // [6, 8] - no overlap + expectedOutputBatches[2], // [10] - no overlap + NewSpanBatch(expectedOutputBatches[1:4]), // [8, 10, 12] - overlapped blocks: 8 or 8, 10 + expectedOutputBatches[4], // [14] - no overlap + NewSpanBatch(expectedOutputBatches[4:6]), // [14, 16] - overlapped blocks: nothing or 14 + NewSpanBatch(expectedOutputBatches[6:9]), // [18, 20, 22] - no overlap } // Shuffle the order of input batches @@ -880,8 +902,16 @@ func TestBatchQueueComplex(t *testing.T) { if batch != nil { blockRef := singularBatchToBlockRef(t, batch, uint64(i+1)) payload := singularBatchToPayload(t, batch, uint64(i+1)) - l2Client.Mock.On("L2BlockRefByNumber", uint64(i+1)).Times(9999).Return(blockRef, &nilErr) - l2Client.Mock.On("PayloadByNumber", uint64(i+1)).Times(9999).Return(&payload, &nilErr) + if i == 0 || i == 3 { + // In CheckBatch(), "L2BlockRefByNumber" is called when fetching the parent block of overlapped span batch + // so blocks at 6, 8 could be called, depends on the order of batches + l2Client.Mock.On("L2BlockRefByNumber", uint64(i+1)).Return(blockRef, &nilErr).Maybe() + } + if i == 1 || i == 2 || i == 4 { + // In CheckBatch(), "PayloadByNumber" is called when fetching the overlapped blocks. + // so blocks at 14, 20 could be called, depends on the order of batches + l2Client.Mock.On("PayloadByNumber", uint64(i+1)).Return(&payload, &nilErr).Maybe() + } } } @@ -916,4 +946,6 @@ func TestBatchQueueComplex(t *testing.T) { safeHead.L1Origin = b.Epoch() } } + + l2Client.Mock.AssertExpectations(t) } From b81df0e69134f3054b967e76c079f5c8fda857b3 Mon Sep 17 00:00:00 2001 From: protolambda Date: Tue, 10 Oct 2023 23:35:59 +0200 Subject: [PATCH 049/279] op-node: sanity check l1Blocks length, fix import --- op-node/rollup/derive/batch_queue.go | 3 +++ op-node/rollup/derive/batch_queue_test.go | 3 +-- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/op-node/rollup/derive/batch_queue.go b/op-node/rollup/derive/batch_queue.go index 9805b1ee0103..5f388a71a927 100644 --- a/op-node/rollup/derive/batch_queue.go +++ b/op-node/rollup/derive/batch_queue.go @@ -79,6 +79,9 @@ func (bq *BatchQueue) popNextBatch(safeL2Head eth.L2BlockRef) *SingularBatch { } func (bq *BatchQueue) advanceEpochMaybe(nextBatch *SingularBatch) { + if len(bq.l1Blocks) == 0 { + return + } if nextBatch.GetEpochNum() == rollup.Epoch(bq.l1Blocks[0].Number)+1 { // Advance epoch if necessary bq.l1Blocks = bq.l1Blocks[1:] diff --git a/op-node/rollup/derive/batch_queue_test.go b/op-node/rollup/derive/batch_queue_test.go index f2fbb1b65602..3580f78c40d6 100644 --- a/op-node/rollup/derive/batch_queue_test.go +++ b/op-node/rollup/derive/batch_queue_test.go @@ -9,10 +9,9 @@ import ( "math/rand" "testing" - "github.com/ethereum/go-ethereum/core/types" - "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/log" "github.com/stretchr/testify/require" From 66007fb4641d60e5a6eaedcecf4eabbeea71505a Mon Sep 17 00:00:00 2001 From: Hamdi Allam Date: Tue, 24 Oct 2023 16:50:46 -0400 Subject: [PATCH 050/279] indexer FilterLog client-side consistency --- indexer/etl/etl.go | 26 +++++++++++---- indexer/node/client.go | 72 ++++++++++++++++++++++++++---------------- indexer/node/mocks.go | 4 +-- 3 files changed, 65 insertions(+), 37 deletions(-) diff --git a/indexer/etl/etl.go b/indexer/etl/etl.go index c69d1a6e7397..f11c69ec7862 100644 --- a/indexer/etl/etl.go +++ b/indexer/etl/etl.go @@ -3,6 +3,7 @@ package etl import ( "context" "errors" + "fmt" "math/big" "time" @@ -105,22 +106,33 @@ func (etl *ETL) processBatch(headers []types.Header) error { } headersWithLog := make(map[common.Hash]bool, len(headers)) - logs, err := etl.EthClient.FilterLogs(ethereum.FilterQuery{FromBlock: firstHeader.Number, ToBlock: lastHeader.Number, Addresses: etl.contracts}) + filterQuery := ethereum.FilterQuery{FromBlock: firstHeader.Number, ToBlock: lastHeader.Number, Addresses: etl.contracts} + logs, err := etl.EthClient.FilterLogs(filterQuery) if err != nil { batchLog.Info("failed to extract logs", "err", err) return err } - if len(logs) > 0 { - batchLog.Info("detected logs", "size", len(logs)) + + if logs.ToBlockHeader.Number.Cmp(lastHeader.Number) != 0 { + // Warn and simply wait for the provider to synchronize state + batchLog.Warn("mismatch in FilterLog#ToBlock number", "queried_to_block_number", lastHeader.Number, "reported_to_block_number", logs.ToBlockHeader.Number) + return fmt.Errorf("mismatch in FilterLog#ToBlock number") + } else if logs.ToBlockHeader.Hash() != lastHeader.Hash() { + batchLog.Error("mismatch in FitlerLog#ToBlock block hash!!!", "queried_to_block_hash", lastHeader.Hash().String(), "reported_to_block_hash", logs.ToBlockHeader.Hash().String()) + return fmt.Errorf("mismatch in FitlerLog#ToBlock block hash!!!") + } + + if len(logs.Logs) > 0 { + batchLog.Info("detected logs", "size", len(logs.Logs)) } - for i := range logs { - log := logs[i] + for i := range logs.Logs { + log := logs.Logs[i] if _, ok := headerMap[log.BlockHash]; !ok { // NOTE. Definitely an error state if the none of the headers were re-orged out in between // the blocks and logs retrieval operations. Unlikely as long as the confirmation depth has // been appropriately set or when we get to natively handling reorgs. - batchLog.Error("log found with block hash not in the batch", "block_hash", logs[i].BlockHash, "log_index", logs[i].Index) + batchLog.Error("log found with block hash not in the batch", "block_hash", logs.Logs[i].BlockHash, "log_index", logs.Logs[i].Index) return errors.New("parsed log with a block hash not in the batch") } @@ -130,6 +142,6 @@ func (etl *ETL) processBatch(headers []types.Header) error { // ensure we use unique downstream references for the etl batch headersRef := headers - etl.etlBatches <- ETLBatch{Logger: batchLog, Headers: headersRef, HeaderMap: headerMap, Logs: logs, HeadersWithLog: headersWithLog} + etl.etlBatches <- ETLBatch{Logger: batchLog, Headers: headersRef, HeaderMap: headerMap, Logs: logs.Logs, HeadersWithLog: headersWithLog} return nil } diff --git a/indexer/node/client.go b/indexer/node/client.go index 6e974e0b792c..363ddc26f30c 100644 --- a/indexer/node/client.go +++ b/indexer/node/client.go @@ -39,7 +39,7 @@ type EthClient interface { TxByHash(common.Hash) (*types.Transaction, error) StorageHash(common.Address, *big.Int) (common.Hash, error) - FilterLogs(ethereum.FilterQuery) ([]types.Log, error) + FilterLogs(ethereum.FilterQuery) (Logs, error) } type clnt struct { @@ -122,15 +122,12 @@ func (c *clnt) BlockHeadersByRange(startHeight, endHeight *big.Int) ([]types.Hea } count := new(big.Int).Sub(endHeight, startHeight).Uint64() + 1 + headers := make([]types.Header, count) batchElems := make([]rpc.BatchElem, count) + for i := uint64(0); i < count; i++ { height := new(big.Int).Add(startHeight, new(big.Int).SetUint64(i)) - batchElems[i] = rpc.BatchElem{ - Method: "eth_getBlockByNumber", - Args: []interface{}{toBlockNumArg(height), false}, - Result: new(types.Header), - Error: nil, - } + batchElems[i] = rpc.BatchElem{Method: "eth_getBlockByNumber", Args: []interface{}{toBlockNumArg(height), false}, Result: &headers[i]} } ctxwt, cancel := context.WithTimeout(context.Background(), defaultRequestTimeout) @@ -144,23 +141,21 @@ func (c *clnt) BlockHeadersByRange(startHeight, endHeight *big.Int) ([]types.Hea // - Ensure integrity that they build on top of each other // - Truncate out headers that do not exist (endHeight > "latest") size := 0 - headers := make([]types.Header, count) for i, batchElem := range batchElems { if batchElem.Error != nil { - return nil, batchElem.Error + if size == 0 { + return nil, batchElem.Error + } else { + break // try return whatever headers are available + } } else if batchElem.Result == nil { break } - header, ok := batchElem.Result.(*types.Header) - if !ok { - return nil, fmt.Errorf("unable to transform rpc response %v into types.Header", batchElem.Result) - } - if i > 0 && header.ParentHash != headers[i-1].Hash() { - return nil, fmt.Errorf("queried header %s does not follow parent %s", header.Hash(), headers[i-1].Hash()) + if i > 0 && headers[i].ParentHash != headers[i-1].Hash() { + return nil, fmt.Errorf("queried header %s does not follow parent %s", headers[i].Hash(), headers[i-1].Hash()) } - headers[i] = *header size = size + 1 } @@ -197,19 +192,43 @@ func (c *clnt) StorageHash(address common.Address, blockNumber *big.Int) (common return proof.StorageHash, nil } -// FilterLogs returns logs that fit the query parameters -func (c *clnt) FilterLogs(query ethereum.FilterQuery) ([]types.Log, error) { - ctxwt, cancel := context.WithTimeout(context.Background(), defaultRequestTimeout) - defer cancel() +type Logs struct { + Logs []types.Log + ToBlockHeader *types.Header +} - var result []types.Log +// FilterLogs returns logs that fit the query parameters. The underlying request is a batch +// request including `eth_getBlockHeaderByNumber` to allow the caller to check that connected +// node has the state necessary to fulfill this request +func (c *clnt) FilterLogs(query ethereum.FilterQuery) (Logs, error) { arg, err := toFilterArg(query) if err != nil { - return nil, err + return Logs{}, err } - err = c.rpc.CallContext(ctxwt, &result, "eth_getLogs", arg) - return result, err + var logs []types.Log + var header types.Header + + batchElems := make([]rpc.BatchElem, 2) + batchElems[0] = rpc.BatchElem{Method: "eth_getBlockByNumber", Args: []interface{}{toBlockNumArg(query.ToBlock), false}, Result: &header} + batchElems[1] = rpc.BatchElem{Method: "eth_getLogs", Args: []interface{}{arg}, Result: &logs} + + ctxwt, cancel := context.WithTimeout(context.Background(), defaultRequestTimeout) + defer cancel() + err = c.rpc.BatchCallContext(ctxwt, batchElems) + if err != nil { + return Logs{}, err + } + + if batchElems[0].Error != nil { + return Logs{}, fmt.Errorf("unable to query for the `FilterQuery#ToBlock` header: %w", batchElems[0].Error) + } + + if batchElems[1].Error != nil { + return Logs{}, fmt.Errorf("unable to query logs: %w", batchElems[1].Error) + } + + return Logs{Logs: logs, ToBlockHeader: &header}, nil } // Modeled off op-service/client.go. We can refactor this once the client/metrics portion @@ -262,10 +281,7 @@ func toBlockNumArg(number *big.Int) string { } func toFilterArg(q ethereum.FilterQuery) (interface{}, error) { - arg := map[string]interface{}{ - "address": q.Addresses, - "topics": q.Topics, - } + arg := map[string]interface{}{"address": q.Addresses, "topics": q.Topics} if q.BlockHash != nil { arg["blockHash"] = *q.BlockHash if q.FromBlock != nil || q.ToBlock != nil { diff --git a/indexer/node/mocks.go b/indexer/node/mocks.go index 115a81767b2e..b18196307ee1 100644 --- a/indexer/node/mocks.go +++ b/indexer/node/mocks.go @@ -41,7 +41,7 @@ func (m *MockEthClient) StorageHash(address common.Address, blockNumber *big.Int return args.Get(0).(common.Hash), args.Error(1) } -func (m *MockEthClient) FilterLogs(query ethereum.FilterQuery) ([]types.Log, error) { +func (m *MockEthClient) FilterLogs(query ethereum.FilterQuery) (Logs, error) { args := m.Called(query) - return args.Get(0).([]types.Log), args.Error(1) + return args.Get(0).(Logs), args.Error(1) } From d5f9ebfee20434b7154713622412ec0add0c8ffc Mon Sep 17 00:00:00 2001 From: protolambda Date: Tue, 24 Oct 2023 23:00:55 +0200 Subject: [PATCH 051/279] op-node: disallow empty span batches, implement review suggestions --- op-node/rollup/derive/batch_queue.go | 12 +++++-- op-node/rollup/derive/span_batch.go | 10 +++++- op-node/rollup/derive/span_batch_test.go | 42 ++++++++++++++++++++++-- 3 files changed, 57 insertions(+), 7 deletions(-) diff --git a/op-node/rollup/derive/batch_queue.go b/op-node/rollup/derive/batch_queue.go index 5f388a71a927..a340bc171bff 100644 --- a/op-node/rollup/derive/batch_queue.go +++ b/op-node/rollup/derive/batch_queue.go @@ -70,7 +70,12 @@ func (bq *BatchQueue) Origin() eth.L1BlockRef { return bq.prev.Origin() } +// popNextBatch pops the next batch from the current queued up span-batch nextSpan. +// The queue must be non-empty, or the function will panic. func (bq *BatchQueue) popNextBatch(safeL2Head eth.L2BlockRef) *SingularBatch { + if len(bq.nextSpan) == 0 { + panic("popping non-existent span-batch, invalid state") + } nextBatch := bq.nextSpan[0] bq.nextSpan = bq.nextSpan[1:] // Must set ParentHash before return. we can use safeL2Head because the parentCheck is verified in CheckBatch(). @@ -78,7 +83,7 @@ func (bq *BatchQueue) popNextBatch(safeL2Head eth.L2BlockRef) *SingularBatch { return nextBatch } -func (bq *BatchQueue) advanceEpochMaybe(nextBatch *SingularBatch) { +func (bq *BatchQueue) maybeAdvanceEpoch(nextBatch *SingularBatch) { if len(bq.l1Blocks) == 0 { return } @@ -92,7 +97,7 @@ func (bq *BatchQueue) NextBatch(ctx context.Context, safeL2Head eth.L2BlockRef) if len(bq.nextSpan) > 0 { // If there are cached singular batches, pop first one and return. nextBatch := bq.popNextBatch(safeL2Head) - bq.advanceEpochMaybe(nextBatch) + bq.maybeAdvanceEpoch(nextBatch) return nextBatch, nil } @@ -167,12 +172,13 @@ func (bq *BatchQueue) NextBatch(ctx context.Context, safeL2Head eth.L2BlockRef) return nil, NewCriticalError(err) } bq.nextSpan = singularBatches + // span-batches are non-empty, so the below pop is safe. nextBatch = bq.popNextBatch(safeL2Head) default: return nil, NewCriticalError(fmt.Errorf("unrecognized batch type: %d", batch.GetBatchType())) } - bq.advanceEpochMaybe(nextBatch) + bq.maybeAdvanceEpoch(nextBatch) return nextBatch, nil } diff --git a/op-node/rollup/derive/span_batch.go b/op-node/rollup/derive/span_batch.go index 81baf158bfee..d68d542ff957 100644 --- a/op-node/rollup/derive/span_batch.go +++ b/op-node/rollup/derive/span_batch.go @@ -27,6 +27,8 @@ import ( var ErrTooBigSpanBatchFieldSize = errors.New("batch would cause field bytes to go over limit") +var ErrEmptySpanBatch = errors.New("span-batch must not be empty") + type spanBatchPrefix struct { relTimestamp uint64 // Relative timestamp of the first block l1OriginNum uint64 // L1 origin number @@ -139,10 +141,13 @@ func (bp *spanBatchPrefix) decodePrefix(r *bytes.Reader) error { // decodeBlockCount parses data into bp.blockCount func (bp *spanBatchPayload) decodeBlockCount(r *bytes.Reader) error { blockCount, err := binary.ReadUvarint(r) - bp.blockCount = blockCount if err != nil { return fmt.Errorf("failed to read block count: %w", err) } + bp.blockCount = blockCount + if blockCount == 0 { + return ErrEmptySpanBatch + } return nil } @@ -362,6 +367,9 @@ func (b *RawSpanBatch) encodeBytes() ([]byte, error) { // derive converts RawSpanBatch into SpanBatch, which has a list of spanBatchElement. // We need chain config constants to derive values for making payload attributes. func (b *RawSpanBatch) derive(blockTime, genesisTimestamp uint64, chainID *big.Int) (*SpanBatch, error) { + if b.blockCount == 0 { + return nil, ErrEmptySpanBatch + } blockOriginNums := make([]uint64, b.blockCount) l1OriginBlockNumber := b.l1OriginNum for i := int(b.blockCount) - 1; i >= 0; i-- { diff --git a/op-node/rollup/derive/span_batch_test.go b/op-node/rollup/derive/span_batch_test.go index 3348b358a1cf..d2a0ac3f8d54 100644 --- a/op-node/rollup/derive/span_batch_test.go +++ b/op-node/rollup/derive/span_batch_test.go @@ -6,12 +6,15 @@ import ( "math/rand" "testing" - "github.com/ethereum-optimism/optimism/op-node/rollup" - "github.com/ethereum-optimism/optimism/op-service/testutils" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/rlp" - "github.com/stretchr/testify/assert" + + "github.com/ethereum-optimism/optimism/op-node/rollup" + "github.com/ethereum-optimism/optimism/op-service/testutils" ) func TestSpanBatchForBatchInterface(t *testing.T) { @@ -33,6 +36,39 @@ func TestSpanBatchForBatchInterface(t *testing.T) { assert.True(t, spanBatch.CheckParentHash(singularBatches[0].ParentHash)) } +func TestEmptySpanBatch(t *testing.T) { + rng := rand.New(rand.NewSource(0x77556691)) + chainID := big.NewInt(rng.Int63n(1000)) + spanTxs, err := newSpanBatchTxs(nil, chainID) + require.NoError(t, err) + + rawSpanBatch := RawSpanBatch{ + spanBatchPrefix: spanBatchPrefix{ + relTimestamp: uint64(rng.Uint32()), + l1OriginNum: rng.Uint64(), + parentCheck: testutils.RandomData(rng, 20), + l1OriginCheck: testutils.RandomData(rng, 20), + }, + spanBatchPayload: spanBatchPayload{ + blockCount: 0, + originBits: big.NewInt(0), + blockTxCounts: []uint64{}, + txs: spanTxs, + }, + } + + var buf bytes.Buffer + err = rawSpanBatch.encodeBlockCount(&buf) + assert.NoError(t, err) + + result := buf.Bytes() + r := bytes.NewReader(result) + var sb RawSpanBatch + + err = sb.decodeBlockCount(r) + require.ErrorIs(t, err, ErrEmptySpanBatch) +} + func TestSpanBatchOriginBits(t *testing.T) { rng := rand.New(rand.NewSource(0x77665544)) chainID := big.NewInt(rng.Int63n(1000)) From b36a8be32f80f5a36388fe8bc00b2343a3d4586d Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Wed, 25 Oct 2023 07:03:33 +1000 Subject: [PATCH 052/279] op-service: Update CalcBaseFee call --- op-e2e/actions/l1_miner.go | 2 +- op-program/client/l2/engineapi/block_processor.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/op-e2e/actions/l1_miner.go b/op-e2e/actions/l1_miner.go index 70c5059a16c0..c9d085877bf6 100644 --- a/op-e2e/actions/l1_miner.go +++ b/op-e2e/actions/l1_miner.go @@ -67,7 +67,7 @@ func (s *L1Miner) ActL1StartBlock(timeDelta uint64) Action { MixDigest: common.Hash{}, // TODO: maybe randomize this (prev-randao value) } if s.l1Cfg.Config.IsLondon(header.Number) { - header.BaseFee = eip1559.CalcBaseFee(s.l1Cfg.Config, parent) + header.BaseFee = eip1559.CalcBaseFee(s.l1Cfg.Config, parent, header.Time) // At the transition, double the gas limit so the gas target is equal to the old gas limit. if !s.l1Cfg.Config.IsLondon(parent.Number) { header.GasLimit = parent.GasLimit * s.l1Cfg.Config.ElasticityMultiplier() diff --git a/op-program/client/l2/engineapi/block_processor.go b/op-program/client/l2/engineapi/block_processor.go index 22c6e1441401..c6966e0d94c2 100644 --- a/op-program/client/l2/engineapi/block_processor.go +++ b/op-program/client/l2/engineapi/block_processor.go @@ -68,7 +68,7 @@ func NewBlockProcessorFromHeader(provider BlockDataProvider, h *types.Header) (* return nil, fmt.Errorf("get parent state: %w", err) } header.Number = new(big.Int).Add(parentHeader.Number, common.Big1) - header.BaseFee = eip1559.CalcBaseFee(provider.Config(), parentHeader) + header.BaseFee = eip1559.CalcBaseFee(provider.Config(), parentHeader, header.Time) header.GasUsed = 0 gasPool := new(core.GasPool).AddGas(header.GasLimit) return &BlockProcessor{ From 2f5eeea3514cc40192400bd6f8607ad44de71016 Mon Sep 17 00:00:00 2001 From: protolambda Date: Tue, 24 Oct 2023 23:34:01 +0200 Subject: [PATCH 053/279] specs: disallow empty span-batches --- specs/span-batches.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/specs/span-batches.md b/specs/span-batches.md index 9b1b17db5f99..3314a19aa457 100644 --- a/specs/span-batches.md +++ b/specs/span-batches.md @@ -94,7 +94,7 @@ Where: - `l1_origin_check`: the block hash of the last L1 origin is referenced. The hash is truncated to 20 bytes for efficiency, i.e. `span_end.l1_origin.hash[:20]`. - `payload = block_count ++ origin_bits ++ block_tx_counts ++ txs`: - - `block_count`: `uvarint` number of L2 blocks. + - `block_count`: `uvarint` number of L2 blocks. This is at least 1, empty span batches are invalid. - `origin_bits`: bitlist of `block_count` bits, right-padded to a multiple of 8 bits: 1 bit per L2 block, indicating if the L1 origin changed this L2 block. - `block_tx_counts`: for each block, a `uvarint` of `len(block.transactions)`. @@ -130,6 +130,7 @@ Where: - `prefix = rel_timestamp ++ l1_origin_num ++ parent_check ++ l1_origin_check`: - Identical to `batch_version` 1 - `payload = block_count ++ origin_bits ++ block_tx_counts ++ txs ++ fee_recipients`: + - An empty span-batch, i.e. with `block_count == 0`, is invalid and must not be processed. - Every field definition identical to `batch_version` 1 except that `fee_recipients` is added to support more decentralized sequencing. - `fee_recipients = fee_recipients_idxs + fee_recipients_set` From 8fb22c5d0ca12ebefb0cc70dd66c25029cb88129 Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Wed, 25 Oct 2023 08:30:34 +1000 Subject: [PATCH 054/279] Revert "op-service: Update CalcBaseFee call" This reverts commit b36a8be32f80f5a36388fe8bc00b2343a3d4586d. --- op-e2e/actions/l1_miner.go | 2 +- op-program/client/l2/engineapi/block_processor.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/op-e2e/actions/l1_miner.go b/op-e2e/actions/l1_miner.go index c9d085877bf6..70c5059a16c0 100644 --- a/op-e2e/actions/l1_miner.go +++ b/op-e2e/actions/l1_miner.go @@ -67,7 +67,7 @@ func (s *L1Miner) ActL1StartBlock(timeDelta uint64) Action { MixDigest: common.Hash{}, // TODO: maybe randomize this (prev-randao value) } if s.l1Cfg.Config.IsLondon(header.Number) { - header.BaseFee = eip1559.CalcBaseFee(s.l1Cfg.Config, parent, header.Time) + header.BaseFee = eip1559.CalcBaseFee(s.l1Cfg.Config, parent) // At the transition, double the gas limit so the gas target is equal to the old gas limit. if !s.l1Cfg.Config.IsLondon(parent.Number) { header.GasLimit = parent.GasLimit * s.l1Cfg.Config.ElasticityMultiplier() diff --git a/op-program/client/l2/engineapi/block_processor.go b/op-program/client/l2/engineapi/block_processor.go index c6966e0d94c2..22c6e1441401 100644 --- a/op-program/client/l2/engineapi/block_processor.go +++ b/op-program/client/l2/engineapi/block_processor.go @@ -68,7 +68,7 @@ func NewBlockProcessorFromHeader(provider BlockDataProvider, h *types.Header) (* return nil, fmt.Errorf("get parent state: %w", err) } header.Number = new(big.Int).Add(parentHeader.Number, common.Big1) - header.BaseFee = eip1559.CalcBaseFee(provider.Config(), parentHeader, header.Time) + header.BaseFee = eip1559.CalcBaseFee(provider.Config(), parentHeader) header.GasUsed = 0 gasPool := new(core.GasPool).AddGas(header.GasLimit) return &BlockProcessor{ From aaca815ef8c2ecb441ef3e003d19d355a3e9b8f7 Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Wed, 25 Oct 2023 08:30:39 +1000 Subject: [PATCH 055/279] Use op-geth version with upstream 1.13.4 but without 1559 denominator change. --- go.mod | 2 +- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index 370e4dddcd44..3713a12490c6 100644 --- a/go.mod +++ b/go.mod @@ -209,7 +209,7 @@ require ( rsc.io/tmplfunc v0.0.3 // indirect ) -replace github.com/ethereum/go-ethereum v1.13.1 => github.com/ethereum-optimism/op-geth v1.101303.0-rc.2.0.20231024175019-29cd9a353f83 +replace github.com/ethereum/go-ethereum v1.13.1 => github.com/ethereum-optimism/op-geth v1.101303.0-rc.2.0.20231024150425-5023660bf92d //replace github.com/ethereum-optimism/superchain-registry/superchain => ../superchain-registry/superchain //replace github.com/ethereum/go-ethereum v1.13.1 => ../go-ethereum diff --git a/go.sum b/go.sum index 45a78c60555e..1627f592ddca 100644 --- a/go.sum +++ b/go.sum @@ -151,8 +151,8 @@ github.com/elastic/gosigar v0.14.2 h1:Dg80n8cr90OZ7x+bAax/QjoW/XqTI11RmA79ZwIm9/ github.com/elastic/gosigar v0.14.2/go.mod h1:iXRIGg2tLnu7LBdpqzyQfGDEidKCfWcCMS0WKyPWoMs= github.com/ethereum-optimism/go-ethereum-hdwallet v0.1.3 h1:RWHKLhCrQThMfch+QJ1Z8veEq5ZO3DfIhZ7xgRP9WTc= github.com/ethereum-optimism/go-ethereum-hdwallet v0.1.3/go.mod h1:QziizLAiF0KqyLdNJYD7O5cpDlaFMNZzlxYNcWsJUxs= -github.com/ethereum-optimism/op-geth v1.101303.0-rc.2.0.20231024175019-29cd9a353f83 h1:RFKnTUJqbYM8+dueFcGPdOY0ycrOhxp0HQJyy2OYzvc= -github.com/ethereum-optimism/op-geth v1.101303.0-rc.2.0.20231024175019-29cd9a353f83/go.mod h1:hl28ffXoV4maInP7dvhvNgDO79Q5M3MEYrPZZO6u3W8= +github.com/ethereum-optimism/op-geth v1.101303.0-rc.2.0.20231024150425-5023660bf92d h1:5dptu9FNKPcZ3o5h2incAbQWH411Mw4HKlsZ4sF4llY= +github.com/ethereum-optimism/op-geth v1.101303.0-rc.2.0.20231024150425-5023660bf92d/go.mod h1:hl28ffXoV4maInP7dvhvNgDO79Q5M3MEYrPZZO6u3W8= github.com/ethereum-optimism/superchain-registry/superchain v0.0.0-20231018202221-fdba3d104171 h1:MjCUj16JSLZRDnQQ6OOUy6Chfb4dKo7ahFceNi0RKZ8= github.com/ethereum-optimism/superchain-registry/superchain v0.0.0-20231018202221-fdba3d104171/go.mod h1:/70H/KqrtKcvWvNGVj6S3rAcLC+kUPr3t2aDmYIS+Xk= github.com/ethereum/c-kzg-4844 v0.3.1 h1:sR65+68+WdnMKxseNWxSJuAv2tsUrihTpVBTfM/U5Zg= From 24ecfea5264850dd010b0a646b843d569f6c3941 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 24 Oct 2023 22:42:22 +0000 Subject: [PATCH 056/279] build(deps-dev): bump vite from 4.4.10 to 4.5.0 Bumps [vite](https://github.com/vitejs/vite/tree/HEAD/packages/vite) from 4.4.10 to 4.5.0. - [Release notes](https://github.com/vitejs/vite/releases) - [Changelog](https://github.com/vitejs/vite/blob/v4.5.0/packages/vite/CHANGELOG.md) - [Commits](https://github.com/vitejs/vite/commits/v4.5.0/packages/vite) --- updated-dependencies: - dependency-name: vite dependency-type: direct:development update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] --- packages/contracts-ts/package.json | 2 +- packages/fee-estimation/package.json | 2 +- packages/web3js-plugin/package.json | 2 +- pnpm-lock.yaml | 213 +++------------------------ 4 files changed, 27 insertions(+), 192 deletions(-) diff --git a/packages/contracts-ts/package.json b/packages/contracts-ts/package.json index 219510f3f850..47679477d2ba 100644 --- a/packages/contracts-ts/package.json +++ b/packages/contracts-ts/package.json @@ -61,7 +61,7 @@ "jsdom": "^22.1.0", "tsup": "^7.2.0", "typescript": "^5.2.2", - "vite": "^4.4.10", + "vite": "^4.5.0", "vitest": "^0.34.2" }, "peerDependencies": { diff --git a/packages/fee-estimation/package.json b/packages/fee-estimation/package.json index 067bc5da2fe6..65098be171ff 100644 --- a/packages/fee-estimation/package.json +++ b/packages/fee-estimation/package.json @@ -45,7 +45,7 @@ "tsup": "^7.2.0", "typescript": "^5.2.2", "viem": "^1.16.6", - "vite": "^4.4.10", + "vite": "^4.5.0", "vitest": "^0.34.2" }, "peerDependencies": { diff --git a/packages/web3js-plugin/package.json b/packages/web3js-plugin/package.json index 8359a0d18acc..930553db3cd2 100644 --- a/packages/web3js-plugin/package.json +++ b/packages/web3js-plugin/package.json @@ -38,7 +38,7 @@ "tsup": "^7.2.0", "typescript": "^5.2.2", "viem": "^1.16.6", - "vite": "^4.4.10", + "vite": "^4.5.0", "vitest": "^0.34.1", "zod": "^3.22.4" }, diff --git a/pnpm-lock.yaml b/pnpm-lock.yaml index 3224072a4f9d..d14c58918309 100644 --- a/pnpm-lock.yaml +++ b/pnpm-lock.yaml @@ -269,10 +269,10 @@ importers: devDependencies: '@typescript-eslint/eslint-plugin': specifier: ^6.7.5 - version: 6.7.5(@typescript-eslint/parser@6.9.0)(eslint@8.51.0)(typescript@5.2.2) + version: 6.7.5(@typescript-eslint/parser@6.9.0)(eslint@8.52.0)(typescript@5.2.2) '@typescript-eslint/parser': specifier: ^6.9.0 - version: 6.9.0(eslint@8.51.0)(typescript@5.2.2) + version: 6.9.0(eslint@8.52.0)(typescript@5.2.2) tsx: specifier: ^3.14.0 version: 3.14.0 @@ -347,8 +347,8 @@ importers: specifier: ^5.2.2 version: 5.2.2 vite: - specifier: ^4.4.10 - version: 4.4.10(@types/node@20.8.8) + specifier: ^4.5.0 + version: 4.5.0(@types/node@20.8.8) vitest: specifier: ^0.34.2 version: 0.34.2(jsdom@22.1.0) @@ -441,8 +441,8 @@ importers: specifier: ^1.16.6 version: 1.16.6(typescript@5.2.2)(zod@3.22.4) vite: - specifier: ^4.4.10 - version: 4.4.10(@types/node@20.8.8) + specifier: ^4.5.0 + version: 4.5.0(@types/node@20.8.8) vitest: specifier: ^0.34.2 version: 0.34.2(jsdom@22.1.0) @@ -572,8 +572,8 @@ importers: specifier: ^1.16.6 version: 1.16.6(typescript@5.2.2)(zod@3.22.4) vite: - specifier: ^4.4.10 - version: 4.4.10(@types/node@20.8.8) + specifier: ^4.5.0 + version: 4.5.0(@types/node@20.8.8) vitest: specifier: ^0.34.1 version: 0.34.1 @@ -1823,16 +1823,6 @@ packages: dev: true optional: true - /@eslint-community/eslint-utils@4.4.0(eslint@8.51.0): - resolution: {integrity: sha512-1/sA4dwrzBAyeUoQ6oxahHKmrZvsnLCg4RfxW3ZFGGmQkSNQPFNLV9CUEFQP1x9EYXHTo5p6xdhZM1Ne9p/AfA==} - engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0} - peerDependencies: - eslint: ^6.0.0 || ^7.0.0 || >=8.0.0 - dependencies: - eslint: 8.51.0 - eslint-visitor-keys: 3.4.3 - dev: true - /@eslint-community/eslint-utils@4.4.0(eslint@8.52.0): resolution: {integrity: sha512-1/sA4dwrzBAyeUoQ6oxahHKmrZvsnLCg4RfxW3ZFGGmQkSNQPFNLV9CUEFQP1x9EYXHTo5p6xdhZM1Ne9p/AfA==} engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0} @@ -1865,11 +1855,6 @@ packages: - supports-color dev: true - /@eslint/js@8.51.0: - resolution: {integrity: sha512-HxjQ8Qn+4SI3/AFv6sOrDB+g6PpUTDwSJiQqOrnneEk8L71161srI9gjzzZvYVbzHiVg/BvcH95+cK/zfIt4pg==} - engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0} - dev: true - /@eslint/js@8.52.0: resolution: {integrity: sha512-mjZVbpaeMZludF2fsWLD0Z9gCref1Tk4i9+wddjRvpUNqqcndPkBD09N/Mapey0b3jaXbLm2kICwFv2E64QinA==} engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0} @@ -2443,17 +2428,6 @@ packages: '@trufflesuite/bigint-buffer': 1.1.9 dev: true - /@humanwhocodes/config-array@0.11.11: - resolution: {integrity: sha512-N2brEuAadi0CcdeMXUkhbZB84eskAc8MEX1By6qEchoVywSgXPIjou4rYsl0V3Hj0ZnuGycGCjdNgockbzeWNA==} - engines: {node: '>=10.10.0'} - dependencies: - '@humanwhocodes/object-schema': 1.2.1 - debug: 4.3.4(supports-color@8.1.1) - minimatch: 3.1.2 - transitivePeerDependencies: - - supports-color - dev: true - /@humanwhocodes/config-array@0.11.13: resolution: {integrity: sha512-JSBDMiDKSzQVngfRjOdFXgFfklaXI4K9nLF49Auh21lmBWRLIK3+xTErTWD4KU54pb6coM6ESE7Awz/FNU3zgQ==} engines: {node: '>=10.10.0'} @@ -2470,10 +2444,6 @@ packages: engines: {node: '>=12.22'} dev: true - /@humanwhocodes/object-schema@1.2.1: - resolution: {integrity: sha512-ZnQMnLV4e7hDlUvw8H+U8ASL02SS2Gn6+9Ac3wGGLIe7+je2AeAOxPY+izIPJDfFDb7eDjev0Us8MO1iFRN8hA==} - dev: true - /@humanwhocodes/object-schema@2.0.1: resolution: {integrity: sha512-dvuCeX5fC9dXgJn9t+X5atfmgQAzUOWqS1254Gh0m6i8wKd10ebXkfNKiRK+1GWi/yTvvLDHpoxLr0xxxeslWw==} dev: true @@ -4262,35 +4232,6 @@ packages: '@types/node': 20.8.8 dev: false - /@typescript-eslint/eslint-plugin@6.7.5(@typescript-eslint/parser@6.9.0)(eslint@8.51.0)(typescript@5.2.2): - resolution: {integrity: sha512-JhtAwTRhOUcP96D0Y6KYnwig/MRQbOoLGXTON2+LlyB/N35SP9j1boai2zzwXb7ypKELXMx3DVk9UTaEq1vHEw==} - engines: {node: ^16.0.0 || >=18.0.0} - peerDependencies: - '@typescript-eslint/parser': ^6.0.0 || ^6.0.0-alpha - eslint: ^7.0.0 || ^8.0.0 - typescript: '*' - peerDependenciesMeta: - typescript: - optional: true - dependencies: - '@eslint-community/regexpp': 4.6.2 - '@typescript-eslint/parser': 6.9.0(eslint@8.51.0)(typescript@5.2.2) - '@typescript-eslint/scope-manager': 6.7.5 - '@typescript-eslint/type-utils': 6.7.5(eslint@8.51.0)(typescript@5.2.2) - '@typescript-eslint/utils': 6.7.5(eslint@8.51.0)(typescript@5.2.2) - '@typescript-eslint/visitor-keys': 6.7.5 - debug: 4.3.4(supports-color@8.1.1) - eslint: 8.51.0 - graphemer: 1.4.0 - ignore: 5.2.4 - natural-compare: 1.4.0 - semver: 7.5.4 - ts-api-utils: 1.0.1(typescript@5.2.2) - typescript: 5.2.2 - transitivePeerDependencies: - - supports-color - dev: true - /@typescript-eslint/eslint-plugin@6.7.5(@typescript-eslint/parser@6.9.0)(eslint@8.52.0)(typescript@5.2.2): resolution: {integrity: sha512-JhtAwTRhOUcP96D0Y6KYnwig/MRQbOoLGXTON2+LlyB/N35SP9j1boai2zzwXb7ypKELXMx3DVk9UTaEq1vHEw==} engines: {node: ^16.0.0 || >=18.0.0} @@ -4320,27 +4261,6 @@ packages: - supports-color dev: true - /@typescript-eslint/parser@6.9.0(eslint@8.51.0)(typescript@5.2.2): - resolution: {integrity: sha512-GZmjMh4AJ/5gaH4XF2eXA8tMnHWP+Pm1mjQR2QN4Iz+j/zO04b9TOvJYOX2sCNIQHtRStKTxRY1FX7LhpJT4Gw==} - engines: {node: ^16.0.0 || >=18.0.0} - peerDependencies: - eslint: ^7.0.0 || ^8.0.0 - typescript: '*' - peerDependenciesMeta: - typescript: - optional: true - dependencies: - '@typescript-eslint/scope-manager': 6.9.0 - '@typescript-eslint/types': 6.9.0 - '@typescript-eslint/typescript-estree': 6.9.0(typescript@5.2.2) - '@typescript-eslint/visitor-keys': 6.9.0 - debug: 4.3.4(supports-color@8.1.1) - eslint: 8.51.0 - typescript: 5.2.2 - transitivePeerDependencies: - - supports-color - dev: true - /@typescript-eslint/parser@6.9.0(eslint@8.52.0)(typescript@5.2.2): resolution: {integrity: sha512-GZmjMh4AJ/5gaH4XF2eXA8tMnHWP+Pm1mjQR2QN4Iz+j/zO04b9TOvJYOX2sCNIQHtRStKTxRY1FX7LhpJT4Gw==} engines: {node: ^16.0.0 || >=18.0.0} @@ -4378,26 +4298,6 @@ packages: '@typescript-eslint/visitor-keys': 6.9.0 dev: true - /@typescript-eslint/type-utils@6.7.5(eslint@8.51.0)(typescript@5.2.2): - resolution: {integrity: sha512-Gs0qos5wqxnQrvpYv+pf3XfcRXW6jiAn9zE/K+DlmYf6FcpxeNYN0AIETaPR7rHO4K2UY+D0CIbDP9Ut0U4m1g==} - engines: {node: ^16.0.0 || >=18.0.0} - peerDependencies: - eslint: ^7.0.0 || ^8.0.0 - typescript: '*' - peerDependenciesMeta: - typescript: - optional: true - dependencies: - '@typescript-eslint/typescript-estree': 6.7.5(typescript@5.2.2) - '@typescript-eslint/utils': 6.7.5(eslint@8.51.0)(typescript@5.2.2) - debug: 4.3.4(supports-color@8.1.1) - eslint: 8.51.0 - ts-api-utils: 1.0.1(typescript@5.2.2) - typescript: 5.2.2 - transitivePeerDependencies: - - supports-color - dev: true - /@typescript-eslint/type-utils@6.7.5(eslint@8.52.0)(typescript@5.2.2): resolution: {integrity: sha512-Gs0qos5wqxnQrvpYv+pf3XfcRXW6jiAn9zE/K+DlmYf6FcpxeNYN0AIETaPR7rHO4K2UY+D0CIbDP9Ut0U4m1g==} engines: {node: ^16.0.0 || >=18.0.0} @@ -4470,25 +4370,6 @@ packages: - supports-color dev: true - /@typescript-eslint/utils@6.7.5(eslint@8.51.0)(typescript@5.2.2): - resolution: {integrity: sha512-pfRRrH20thJbzPPlPc4j0UNGvH1PjPlhlCMq4Yx7EGjV7lvEeGX0U6MJYe8+SyFutWgSHsdbJ3BXzZccYggezA==} - engines: {node: ^16.0.0 || >=18.0.0} - peerDependencies: - eslint: ^7.0.0 || ^8.0.0 - dependencies: - '@eslint-community/eslint-utils': 4.4.0(eslint@8.51.0) - '@types/json-schema': 7.0.12 - '@types/semver': 7.5.0 - '@typescript-eslint/scope-manager': 6.7.5 - '@typescript-eslint/types': 6.7.5 - '@typescript-eslint/typescript-estree': 6.7.5(typescript@5.2.2) - eslint: 8.51.0 - semver: 7.5.4 - transitivePeerDependencies: - - supports-color - - typescript - dev: true - /@typescript-eslint/utils@6.7.5(eslint@8.52.0)(typescript@5.2.2): resolution: {integrity: sha512-pfRRrH20thJbzPPlPc4j0UNGvH1PjPlhlCMq4Yx7EGjV7lvEeGX0U6MJYe8+SyFutWgSHsdbJ3BXzZccYggezA==} engines: {node: ^16.0.0 || >=18.0.0} @@ -7913,52 +7794,6 @@ packages: engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0} dev: true - /eslint@8.51.0: - resolution: {integrity: sha512-2WuxRZBrlwnXi+/vFSJyjMqrNjtJqiasMzehF0shoLaW7DzS3/9Yvrmq5JiT66+pNjiX4UBnLDiKHcWAr/OInA==} - engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0} - hasBin: true - dependencies: - '@eslint-community/eslint-utils': 4.4.0(eslint@8.51.0) - '@eslint-community/regexpp': 4.6.2 - '@eslint/eslintrc': 2.1.2 - '@eslint/js': 8.51.0 - '@humanwhocodes/config-array': 0.11.11 - '@humanwhocodes/module-importer': 1.0.1 - '@nodelib/fs.walk': 1.2.8 - ajv: 6.12.6 - chalk: 4.1.2 - cross-spawn: 7.0.3 - debug: 4.3.4(supports-color@8.1.1) - doctrine: 3.0.0 - escape-string-regexp: 4.0.0 - eslint-scope: 7.2.2 - eslint-visitor-keys: 3.4.3 - espree: 9.6.1 - esquery: 1.5.0 - esutils: 2.0.3 - fast-deep-equal: 3.1.3 - file-entry-cache: 6.0.1 - find-up: 5.0.0 - glob-parent: 6.0.2 - globals: 13.21.0 - graphemer: 1.4.0 - ignore: 5.2.4 - imurmurhash: 0.1.4 - is-glob: 4.0.3 - is-path-inside: 3.0.3 - js-yaml: 4.1.0 - json-stable-stringify-without-jsonify: 1.0.1 - levn: 0.4.1 - lodash.merge: 4.6.2 - minimatch: 3.1.2 - natural-compare: 1.4.0 - optionator: 0.9.3 - strip-ansi: 6.0.1 - text-table: 0.2.0 - transitivePeerDependencies: - - supports-color - dev: true - /eslint@8.52.0: resolution: {integrity: sha512-zh/JHnaixqHZsolRB/w9/02akBk9EPrOs9JwcTP2ek7yL5bVvXuRariiaAjjoJ5DvuwQ1WAE/HsMz+w17YgBCg==} engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0} @@ -14590,7 +14425,7 @@ packages: mlly: 1.4.0 pathe: 1.1.1 picocolors: 1.0.0 - vite: 4.4.10(@types/node@20.8.8) + vite: 4.5.0(@types/node@20.8.8) transitivePeerDependencies: - '@types/node' - less @@ -14612,7 +14447,7 @@ packages: mlly: 1.4.0 pathe: 1.1.1 picocolors: 1.0.0 - vite: 4.4.10(@types/node@20.8.6) + vite: 4.5.0(@types/node@20.8.6) transitivePeerDependencies: - '@types/node' - less @@ -14634,7 +14469,7 @@ packages: mlly: 1.4.0 pathe: 1.1.1 picocolors: 1.0.0 - vite: 4.4.10(@types/node@20.8.8) + vite: 4.5.0(@types/node@20.8.8) transitivePeerDependencies: - '@types/node' - less @@ -14656,7 +14491,7 @@ packages: mlly: 1.4.0 pathe: 1.1.1 picocolors: 1.0.0 - vite: 4.4.10(@types/node@20.8.6) + vite: 4.5.0(@types/node@20.8.6) transitivePeerDependencies: - '@types/node' - less @@ -14697,15 +14532,15 @@ packages: optional: true dependencies: '@types/node': 20.8.6 - esbuild: 0.18.15 + esbuild: 0.18.20 postcss: 8.4.27 rollup: 3.28.0 optionalDependencies: fsevents: 2.3.3 dev: true - /vite@4.4.10(@types/node@20.8.8): - resolution: {integrity: sha512-TzIjiqx9BEXF8yzYdF2NTf1kFFbjMjUSV0LFZ3HyHoI3SGSPLnnFUKiIQtL3gl2AjHvMrprOvQ3amzaHgQlAxw==} + /vite@4.5.0(@types/node@20.8.6): + resolution: {integrity: sha512-ulr8rNLA6rkyFAlVWw2q5YJ91v098AFQ2R0PRFwPzREXOUJQPtFUG0t+/ZikhaOCDqFoDhN6/v8Sq0o4araFAw==} engines: {node: ^14.18.0 || >=16.0.0} hasBin: true peerDependencies: @@ -14732,16 +14567,16 @@ packages: terser: optional: true dependencies: - '@types/node': 20.8.8 - esbuild: 0.18.15 + '@types/node': 20.8.6 + esbuild: 0.18.20 postcss: 8.4.27 rollup: 3.28.0 optionalDependencies: fsevents: 2.3.3 dev: true - /vite@4.4.9(@types/node@20.8.6): - resolution: {integrity: sha512-2mbUn2LlUmNASWwSCNSJ/EG2HuSRTnVNaydp6vMCm5VIqJsjMfbIWtbH2kDuwUVW5mMUKKZvGPX/rqeqVvv1XA==} + /vite@4.5.0(@types/node@20.8.8): + resolution: {integrity: sha512-ulr8rNLA6rkyFAlVWw2q5YJ91v098AFQ2R0PRFwPzREXOUJQPtFUG0t+/ZikhaOCDqFoDhN6/v8Sq0o4araFAw==} engines: {node: ^14.18.0 || >=16.0.0} hasBin: true peerDependencies: @@ -14768,7 +14603,7 @@ packages: terser: optional: true dependencies: - '@types/node': 20.8.6 + '@types/node': 20.8.8 esbuild: 0.18.20 postcss: 8.4.27 rollup: 3.28.0 @@ -14828,7 +14663,7 @@ packages: strip-literal: 1.0.1 tinybench: 2.5.0 tinypool: 0.7.0 - vite: 4.4.10(@types/node@20.8.8) + vite: 4.5.0(@types/node@20.8.8) vite-node: 0.34.1(@types/node@20.8.8) why-is-node-running: 2.2.2 transitivePeerDependencies: @@ -14893,7 +14728,7 @@ packages: strip-literal: 1.0.1 tinybench: 2.5.0 tinypool: 0.7.0 - vite: 4.4.10(@types/node@20.8.8) + vite: 4.5.0(@types/node@20.8.8) vite-node: 0.34.2(@types/node@20.8.8) why-is-node-running: 2.2.2 transitivePeerDependencies: @@ -14959,7 +14794,7 @@ packages: strip-literal: 1.0.1 tinybench: 2.5.0 tinypool: 0.7.0 - vite: 4.4.10(@types/node@20.8.6) + vite: 4.5.0(@types/node@20.8.6) vite-node: 0.34.2(@types/node@20.8.6) why-is-node-running: 2.2.2 transitivePeerDependencies: @@ -15024,7 +14859,7 @@ packages: strip-literal: 1.0.1 tinybench: 2.5.0 tinypool: 0.7.0 - vite: 4.4.9(@types/node@20.8.6) + vite: 4.4.10(@types/node@20.8.6) vite-node: 0.34.4(@types/node@20.8.6) why-is-node-running: 2.2.2 transitivePeerDependencies: From e09d3d74c148abc9a9c4971bb6935c94df369eae Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 24 Oct 2023 22:42:26 +0000 Subject: [PATCH 057/279] build(deps-dev): bump @types/morgan from 1.9.6 to 1.9.7 Bumps [@types/morgan](https://github.com/DefinitelyTyped/DefinitelyTyped/tree/HEAD/types/morgan) from 1.9.6 to 1.9.7. - [Release notes](https://github.com/DefinitelyTyped/DefinitelyTyped/releases) - [Commits](https://github.com/DefinitelyTyped/DefinitelyTyped/commits/HEAD/types/morgan) --- updated-dependencies: - dependency-name: "@types/morgan" dependency-type: direct:development update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] --- packages/common-ts/package.json | 2 +- pnpm-lock.yaml | 179 ++------------------------------ 2 files changed, 8 insertions(+), 173 deletions(-) diff --git a/packages/common-ts/package.json b/packages/common-ts/package.json index 82500bbd4109..7f7a67e4ce1f 100644 --- a/packages/common-ts/package.json +++ b/packages/common-ts/package.json @@ -55,7 +55,7 @@ "@ethersproject/abstract-provider": "^5.7.0", "@ethersproject/abstract-signer": "^5.7.0", "@types/express": "^4.17.19", - "@types/morgan": "^1.9.6", + "@types/morgan": "^1.9.7", "@types/pino": "^7.0.5", "@types/pino-multi-stream": "^5.1.4", "chai": "^4.3.10", diff --git a/pnpm-lock.yaml b/pnpm-lock.yaml index 3224072a4f9d..746c86911bd3 100644 --- a/pnpm-lock.yaml +++ b/pnpm-lock.yaml @@ -250,8 +250,8 @@ importers: specifier: ^4.17.19 version: 4.17.19 '@types/morgan': - specifier: ^1.9.6 - version: 1.9.6 + specifier: ^1.9.7 + version: 1.9.7 '@types/pino': specifier: ^7.0.5 version: 7.0.5 @@ -269,10 +269,10 @@ importers: devDependencies: '@typescript-eslint/eslint-plugin': specifier: ^6.7.5 - version: 6.7.5(@typescript-eslint/parser@6.9.0)(eslint@8.51.0)(typescript@5.2.2) + version: 6.7.5(@typescript-eslint/parser@6.9.0)(eslint@8.52.0)(typescript@5.2.2) '@typescript-eslint/parser': specifier: ^6.9.0 - version: 6.9.0(eslint@8.51.0)(typescript@5.2.2) + version: 6.9.0(eslint@8.52.0)(typescript@5.2.2) tsx: specifier: ^3.14.0 version: 3.14.0 @@ -1823,16 +1823,6 @@ packages: dev: true optional: true - /@eslint-community/eslint-utils@4.4.0(eslint@8.51.0): - resolution: {integrity: sha512-1/sA4dwrzBAyeUoQ6oxahHKmrZvsnLCg4RfxW3ZFGGmQkSNQPFNLV9CUEFQP1x9EYXHTo5p6xdhZM1Ne9p/AfA==} - engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0} - peerDependencies: - eslint: ^6.0.0 || ^7.0.0 || >=8.0.0 - dependencies: - eslint: 8.51.0 - eslint-visitor-keys: 3.4.3 - dev: true - /@eslint-community/eslint-utils@4.4.0(eslint@8.52.0): resolution: {integrity: sha512-1/sA4dwrzBAyeUoQ6oxahHKmrZvsnLCg4RfxW3ZFGGmQkSNQPFNLV9CUEFQP1x9EYXHTo5p6xdhZM1Ne9p/AfA==} engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0} @@ -1865,11 +1855,6 @@ packages: - supports-color dev: true - /@eslint/js@8.51.0: - resolution: {integrity: sha512-HxjQ8Qn+4SI3/AFv6sOrDB+g6PpUTDwSJiQqOrnneEk8L71161srI9gjzzZvYVbzHiVg/BvcH95+cK/zfIt4pg==} - engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0} - dev: true - /@eslint/js@8.52.0: resolution: {integrity: sha512-mjZVbpaeMZludF2fsWLD0Z9gCref1Tk4i9+wddjRvpUNqqcndPkBD09N/Mapey0b3jaXbLm2kICwFv2E64QinA==} engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0} @@ -2443,17 +2428,6 @@ packages: '@trufflesuite/bigint-buffer': 1.1.9 dev: true - /@humanwhocodes/config-array@0.11.11: - resolution: {integrity: sha512-N2brEuAadi0CcdeMXUkhbZB84eskAc8MEX1By6qEchoVywSgXPIjou4rYsl0V3Hj0ZnuGycGCjdNgockbzeWNA==} - engines: {node: '>=10.10.0'} - dependencies: - '@humanwhocodes/object-schema': 1.2.1 - debug: 4.3.4(supports-color@8.1.1) - minimatch: 3.1.2 - transitivePeerDependencies: - - supports-color - dev: true - /@humanwhocodes/config-array@0.11.13: resolution: {integrity: sha512-JSBDMiDKSzQVngfRjOdFXgFfklaXI4K9nLF49Auh21lmBWRLIK3+xTErTWD4KU54pb6coM6ESE7Awz/FNU3zgQ==} engines: {node: '>=10.10.0'} @@ -2470,10 +2444,6 @@ packages: engines: {node: '>=12.22'} dev: true - /@humanwhocodes/object-schema@1.2.1: - resolution: {integrity: sha512-ZnQMnLV4e7hDlUvw8H+U8ASL02SS2Gn6+9Ac3wGGLIe7+je2AeAOxPY+izIPJDfFDb7eDjev0Us8MO1iFRN8hA==} - dev: true - /@humanwhocodes/object-schema@2.0.1: resolution: {integrity: sha512-dvuCeX5fC9dXgJn9t+X5atfmgQAzUOWqS1254Gh0m6i8wKd10ebXkfNKiRK+1GWi/yTvvLDHpoxLr0xxxeslWw==} dev: true @@ -4066,10 +4036,10 @@ packages: resolution: {integrity: sha512-RsOPImTriV/OE4A9qKjMtk2MnXiuLLbcO3nCXK+kvq4nr0iMfFgpjaX3MPLb6f7+EL1FGSelYvuJMV6REH+ZPQ==} dev: true - /@types/morgan@1.9.6: - resolution: {integrity: sha512-xfKogz5WcKww2DAiVT9zxMgrqQt+Shq8tDVeLT+otoj6dJnkRkyJxMF51mHtUc3JCPKGk5x1EBU0buuGpfftlQ==} + /@types/morgan@1.9.7: + resolution: {integrity: sha512-4sJFBUBrIZkP5EvMm1L6VCXp3SQe8dnXqlVpe1jsmTjS1JQVmSjnpMNs8DosQd6omBi/K7BSKJ6z/Mc3ki0K9g==} dependencies: - '@types/node': 20.8.6 + '@types/node': 20.8.8 dev: true /@types/ms@0.7.31: @@ -4262,35 +4232,6 @@ packages: '@types/node': 20.8.8 dev: false - /@typescript-eslint/eslint-plugin@6.7.5(@typescript-eslint/parser@6.9.0)(eslint@8.51.0)(typescript@5.2.2): - resolution: {integrity: sha512-JhtAwTRhOUcP96D0Y6KYnwig/MRQbOoLGXTON2+LlyB/N35SP9j1boai2zzwXb7ypKELXMx3DVk9UTaEq1vHEw==} - engines: {node: ^16.0.0 || >=18.0.0} - peerDependencies: - '@typescript-eslint/parser': ^6.0.0 || ^6.0.0-alpha - eslint: ^7.0.0 || ^8.0.0 - typescript: '*' - peerDependenciesMeta: - typescript: - optional: true - dependencies: - '@eslint-community/regexpp': 4.6.2 - '@typescript-eslint/parser': 6.9.0(eslint@8.51.0)(typescript@5.2.2) - '@typescript-eslint/scope-manager': 6.7.5 - '@typescript-eslint/type-utils': 6.7.5(eslint@8.51.0)(typescript@5.2.2) - '@typescript-eslint/utils': 6.7.5(eslint@8.51.0)(typescript@5.2.2) - '@typescript-eslint/visitor-keys': 6.7.5 - debug: 4.3.4(supports-color@8.1.1) - eslint: 8.51.0 - graphemer: 1.4.0 - ignore: 5.2.4 - natural-compare: 1.4.0 - semver: 7.5.4 - ts-api-utils: 1.0.1(typescript@5.2.2) - typescript: 5.2.2 - transitivePeerDependencies: - - supports-color - dev: true - /@typescript-eslint/eslint-plugin@6.7.5(@typescript-eslint/parser@6.9.0)(eslint@8.52.0)(typescript@5.2.2): resolution: {integrity: sha512-JhtAwTRhOUcP96D0Y6KYnwig/MRQbOoLGXTON2+LlyB/N35SP9j1boai2zzwXb7ypKELXMx3DVk9UTaEq1vHEw==} engines: {node: ^16.0.0 || >=18.0.0} @@ -4320,27 +4261,6 @@ packages: - supports-color dev: true - /@typescript-eslint/parser@6.9.0(eslint@8.51.0)(typescript@5.2.2): - resolution: {integrity: sha512-GZmjMh4AJ/5gaH4XF2eXA8tMnHWP+Pm1mjQR2QN4Iz+j/zO04b9TOvJYOX2sCNIQHtRStKTxRY1FX7LhpJT4Gw==} - engines: {node: ^16.0.0 || >=18.0.0} - peerDependencies: - eslint: ^7.0.0 || ^8.0.0 - typescript: '*' - peerDependenciesMeta: - typescript: - optional: true - dependencies: - '@typescript-eslint/scope-manager': 6.9.0 - '@typescript-eslint/types': 6.9.0 - '@typescript-eslint/typescript-estree': 6.9.0(typescript@5.2.2) - '@typescript-eslint/visitor-keys': 6.9.0 - debug: 4.3.4(supports-color@8.1.1) - eslint: 8.51.0 - typescript: 5.2.2 - transitivePeerDependencies: - - supports-color - dev: true - /@typescript-eslint/parser@6.9.0(eslint@8.52.0)(typescript@5.2.2): resolution: {integrity: sha512-GZmjMh4AJ/5gaH4XF2eXA8tMnHWP+Pm1mjQR2QN4Iz+j/zO04b9TOvJYOX2sCNIQHtRStKTxRY1FX7LhpJT4Gw==} engines: {node: ^16.0.0 || >=18.0.0} @@ -4378,26 +4298,6 @@ packages: '@typescript-eslint/visitor-keys': 6.9.0 dev: true - /@typescript-eslint/type-utils@6.7.5(eslint@8.51.0)(typescript@5.2.2): - resolution: {integrity: sha512-Gs0qos5wqxnQrvpYv+pf3XfcRXW6jiAn9zE/K+DlmYf6FcpxeNYN0AIETaPR7rHO4K2UY+D0CIbDP9Ut0U4m1g==} - engines: {node: ^16.0.0 || >=18.0.0} - peerDependencies: - eslint: ^7.0.0 || ^8.0.0 - typescript: '*' - peerDependenciesMeta: - typescript: - optional: true - dependencies: - '@typescript-eslint/typescript-estree': 6.7.5(typescript@5.2.2) - '@typescript-eslint/utils': 6.7.5(eslint@8.51.0)(typescript@5.2.2) - debug: 4.3.4(supports-color@8.1.1) - eslint: 8.51.0 - ts-api-utils: 1.0.1(typescript@5.2.2) - typescript: 5.2.2 - transitivePeerDependencies: - - supports-color - dev: true - /@typescript-eslint/type-utils@6.7.5(eslint@8.52.0)(typescript@5.2.2): resolution: {integrity: sha512-Gs0qos5wqxnQrvpYv+pf3XfcRXW6jiAn9zE/K+DlmYf6FcpxeNYN0AIETaPR7rHO4K2UY+D0CIbDP9Ut0U4m1g==} engines: {node: ^16.0.0 || >=18.0.0} @@ -4470,25 +4370,6 @@ packages: - supports-color dev: true - /@typescript-eslint/utils@6.7.5(eslint@8.51.0)(typescript@5.2.2): - resolution: {integrity: sha512-pfRRrH20thJbzPPlPc4j0UNGvH1PjPlhlCMq4Yx7EGjV7lvEeGX0U6MJYe8+SyFutWgSHsdbJ3BXzZccYggezA==} - engines: {node: ^16.0.0 || >=18.0.0} - peerDependencies: - eslint: ^7.0.0 || ^8.0.0 - dependencies: - '@eslint-community/eslint-utils': 4.4.0(eslint@8.51.0) - '@types/json-schema': 7.0.12 - '@types/semver': 7.5.0 - '@typescript-eslint/scope-manager': 6.7.5 - '@typescript-eslint/types': 6.7.5 - '@typescript-eslint/typescript-estree': 6.7.5(typescript@5.2.2) - eslint: 8.51.0 - semver: 7.5.4 - transitivePeerDependencies: - - supports-color - - typescript - dev: true - /@typescript-eslint/utils@6.7.5(eslint@8.52.0)(typescript@5.2.2): resolution: {integrity: sha512-pfRRrH20thJbzPPlPc4j0UNGvH1PjPlhlCMq4Yx7EGjV7lvEeGX0U6MJYe8+SyFutWgSHsdbJ3BXzZccYggezA==} engines: {node: ^16.0.0 || >=18.0.0} @@ -7913,52 +7794,6 @@ packages: engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0} dev: true - /eslint@8.51.0: - resolution: {integrity: sha512-2WuxRZBrlwnXi+/vFSJyjMqrNjtJqiasMzehF0shoLaW7DzS3/9Yvrmq5JiT66+pNjiX4UBnLDiKHcWAr/OInA==} - engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0} - hasBin: true - dependencies: - '@eslint-community/eslint-utils': 4.4.0(eslint@8.51.0) - '@eslint-community/regexpp': 4.6.2 - '@eslint/eslintrc': 2.1.2 - '@eslint/js': 8.51.0 - '@humanwhocodes/config-array': 0.11.11 - '@humanwhocodes/module-importer': 1.0.1 - '@nodelib/fs.walk': 1.2.8 - ajv: 6.12.6 - chalk: 4.1.2 - cross-spawn: 7.0.3 - debug: 4.3.4(supports-color@8.1.1) - doctrine: 3.0.0 - escape-string-regexp: 4.0.0 - eslint-scope: 7.2.2 - eslint-visitor-keys: 3.4.3 - espree: 9.6.1 - esquery: 1.5.0 - esutils: 2.0.3 - fast-deep-equal: 3.1.3 - file-entry-cache: 6.0.1 - find-up: 5.0.0 - glob-parent: 6.0.2 - globals: 13.21.0 - graphemer: 1.4.0 - ignore: 5.2.4 - imurmurhash: 0.1.4 - is-glob: 4.0.3 - is-path-inside: 3.0.3 - js-yaml: 4.1.0 - json-stable-stringify-without-jsonify: 1.0.1 - levn: 0.4.1 - lodash.merge: 4.6.2 - minimatch: 3.1.2 - natural-compare: 1.4.0 - optionator: 0.9.3 - strip-ansi: 6.0.1 - text-table: 0.2.0 - transitivePeerDependencies: - - supports-color - dev: true - /eslint@8.52.0: resolution: {integrity: sha512-zh/JHnaixqHZsolRB/w9/02akBk9EPrOs9JwcTP2ek7yL5bVvXuRariiaAjjoJ5DvuwQ1WAE/HsMz+w17YgBCg==} engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0} From e46624424b3f6d0016918760373bec64436c5726 Mon Sep 17 00:00:00 2001 From: "github-actions[bot]" Date: Tue, 24 Oct 2023 23:51:16 +0000 Subject: [PATCH 058/279] Version Packages --- .changeset/clean-guests-guess.md | 5 ----- packages/chain-mon/CHANGELOG.md | 6 ++++++ packages/chain-mon/package.json | 2 +- 3 files changed, 7 insertions(+), 6 deletions(-) delete mode 100644 .changeset/clean-guests-guess.md diff --git a/.changeset/clean-guests-guess.md b/.changeset/clean-guests-guess.md deleted file mode 100644 index 9c5a9986951a..000000000000 --- a/.changeset/clean-guests-guess.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -'@eth-optimism/chain-mon': patch ---- - -Bump node version to LTS node 20.9.0 diff --git a/packages/chain-mon/CHANGELOG.md b/packages/chain-mon/CHANGELOG.md index 0585ce35ce45..7d989cd321f0 100644 --- a/packages/chain-mon/CHANGELOG.md +++ b/packages/chain-mon/CHANGELOG.md @@ -1,5 +1,11 @@ # @eth-optimism/drippie-mon +## 0.5.2 + +### Patch Changes + +- [#7824](https://github.com/ethereum-optimism/optimism/pull/7824) [`98eb885f5`](https://github.com/ethereum-optimism/optimism/commit/98eb885f5003ee5e6b9bbd532a42bba2ad39cb4b) Thanks [@roninjin10](https://github.com/roninjin10)! - Bump node version to LTS node 20.9.0 + ## 0.5.1 ### Patch Changes diff --git a/packages/chain-mon/package.json b/packages/chain-mon/package.json index 195af10095ce..33fc16d63883 100644 --- a/packages/chain-mon/package.json +++ b/packages/chain-mon/package.json @@ -1,7 +1,7 @@ { "private": true, "name": "@eth-optimism/chain-mon", - "version": "0.5.1", + "version": "0.5.2", "description": "[Optimism] Chain monitoring services", "main": "dist/index", "types": "dist/index", From 1ef194e0ab80423f79a6c764f320d18ca6d8c0df Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 24 Oct 2023 23:52:01 +0000 Subject: [PATCH 059/279] build(deps-dev): bump @typescript-eslint/eslint-plugin Bumps [@typescript-eslint/eslint-plugin](https://github.com/typescript-eslint/typescript-eslint/tree/HEAD/packages/eslint-plugin) from 6.7.5 to 6.9.0. - [Release notes](https://github.com/typescript-eslint/typescript-eslint/releases) - [Changelog](https://github.com/typescript-eslint/typescript-eslint/blob/main/packages/eslint-plugin/CHANGELOG.md) - [Commits](https://github.com/typescript-eslint/typescript-eslint/commits/v6.9.0/packages/eslint-plugin) --- updated-dependencies: - dependency-name: "@typescript-eslint/eslint-plugin" dependency-type: direct:development update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] --- package.json | 2 +- packages/contracts-bedrock/package.json | 2 +- pnpm-lock.yaml | 80 ++++++------------------- 3 files changed, 21 insertions(+), 63 deletions(-) diff --git a/package.json b/package.json index dad50328938e..9f6fc137195f 100644 --- a/package.json +++ b/package.json @@ -44,7 +44,7 @@ "@types/chai-as-promised": "^7.1.4", "@types/mocha": "^10.0.3", "@types/node": "^20.8.8", - "@typescript-eslint/eslint-plugin": "^6.7.5", + "@typescript-eslint/eslint-plugin": "^6.9.0", "@typescript-eslint/parser": "^6.9.0", "chai": "^4.3.10", "depcheck": "^1.4.7", diff --git a/packages/contracts-bedrock/package.json b/packages/contracts-bedrock/package.json index 5298a83105c9..88f7e05e4700 100644 --- a/packages/contracts-bedrock/package.json +++ b/packages/contracts-bedrock/package.json @@ -44,7 +44,7 @@ "lint": "pnpm lint:fix && pnpm lint:check" }, "devDependencies": { - "@typescript-eslint/eslint-plugin": "^6.7.5", + "@typescript-eslint/eslint-plugin": "^6.9.0", "@typescript-eslint/parser": "^6.9.0", "tsx": "^3.14.0", "typescript": "^5.2.2" diff --git a/pnpm-lock.yaml b/pnpm-lock.yaml index d14c58918309..8245cbec60bb 100644 --- a/pnpm-lock.yaml +++ b/pnpm-lock.yaml @@ -34,8 +34,8 @@ importers: specifier: ^20.8.8 version: 20.8.8 '@typescript-eslint/eslint-plugin': - specifier: ^6.7.5 - version: 6.7.5(@typescript-eslint/parser@6.9.0)(eslint@8.52.0)(typescript@5.2.2) + specifier: ^6.9.0 + version: 6.9.0(@typescript-eslint/parser@6.9.0)(eslint@8.52.0)(typescript@5.2.2) '@typescript-eslint/parser': specifier: ^6.9.0 version: 6.9.0(eslint@8.52.0)(typescript@5.2.2) @@ -268,8 +268,8 @@ importers: packages/contracts-bedrock: devDependencies: '@typescript-eslint/eslint-plugin': - specifier: ^6.7.5 - version: 6.7.5(@typescript-eslint/parser@6.9.0)(eslint@8.52.0)(typescript@5.2.2) + specifier: ^6.9.0 + version: 6.9.0(@typescript-eslint/parser@6.9.0)(eslint@8.52.0)(typescript@5.2.2) '@typescript-eslint/parser': specifier: ^6.9.0 version: 6.9.0(eslint@8.52.0)(typescript@5.2.2) @@ -4232,8 +4232,8 @@ packages: '@types/node': 20.8.8 dev: false - /@typescript-eslint/eslint-plugin@6.7.5(@typescript-eslint/parser@6.9.0)(eslint@8.52.0)(typescript@5.2.2): - resolution: {integrity: sha512-JhtAwTRhOUcP96D0Y6KYnwig/MRQbOoLGXTON2+LlyB/N35SP9j1boai2zzwXb7ypKELXMx3DVk9UTaEq1vHEw==} + /@typescript-eslint/eslint-plugin@6.9.0(@typescript-eslint/parser@6.9.0)(eslint@8.52.0)(typescript@5.2.2): + resolution: {integrity: sha512-lgX7F0azQwRPB7t7WAyeHWVfW1YJ9NIgd9mvGhfQpRY56X6AVf8mwM8Wol+0z4liE7XX3QOt8MN1rUKCfSjRIA==} engines: {node: ^16.0.0 || >=18.0.0} peerDependencies: '@typescript-eslint/parser': ^6.0.0 || ^6.0.0-alpha @@ -4245,10 +4245,10 @@ packages: dependencies: '@eslint-community/regexpp': 4.6.2 '@typescript-eslint/parser': 6.9.0(eslint@8.52.0)(typescript@5.2.2) - '@typescript-eslint/scope-manager': 6.7.5 - '@typescript-eslint/type-utils': 6.7.5(eslint@8.52.0)(typescript@5.2.2) - '@typescript-eslint/utils': 6.7.5(eslint@8.52.0)(typescript@5.2.2) - '@typescript-eslint/visitor-keys': 6.7.5 + '@typescript-eslint/scope-manager': 6.9.0 + '@typescript-eslint/type-utils': 6.9.0(eslint@8.52.0)(typescript@5.2.2) + '@typescript-eslint/utils': 6.9.0(eslint@8.52.0)(typescript@5.2.2) + '@typescript-eslint/visitor-keys': 6.9.0 debug: 4.3.4(supports-color@8.1.1) eslint: 8.52.0 graphemer: 1.4.0 @@ -4282,14 +4282,6 @@ packages: - supports-color dev: true - /@typescript-eslint/scope-manager@6.7.5: - resolution: {integrity: sha512-GAlk3eQIwWOJeb9F7MKQ6Jbah/vx1zETSDw8likab/eFcqkjSD7BI75SDAeC5N2L0MmConMoPvTsmkrg71+B1A==} - engines: {node: ^16.0.0 || >=18.0.0} - dependencies: - '@typescript-eslint/types': 6.7.5 - '@typescript-eslint/visitor-keys': 6.7.5 - dev: true - /@typescript-eslint/scope-manager@6.9.0: resolution: {integrity: sha512-1R8A9Mc39n4pCCz9o79qRO31HGNDvC7UhPhv26TovDsWPBDx+Sg3rOZdCELIA3ZmNoWAuxaMOT7aWtGRSYkQxw==} engines: {node: ^16.0.0 || >=18.0.0} @@ -4298,8 +4290,8 @@ packages: '@typescript-eslint/visitor-keys': 6.9.0 dev: true - /@typescript-eslint/type-utils@6.7.5(eslint@8.52.0)(typescript@5.2.2): - resolution: {integrity: sha512-Gs0qos5wqxnQrvpYv+pf3XfcRXW6jiAn9zE/K+DlmYf6FcpxeNYN0AIETaPR7rHO4K2UY+D0CIbDP9Ut0U4m1g==} + /@typescript-eslint/type-utils@6.9.0(eslint@8.52.0)(typescript@5.2.2): + resolution: {integrity: sha512-XXeahmfbpuhVbhSOROIzJ+b13krFmgtc4GlEuu1WBT+RpyGPIA4Y/eGnXzjbDj5gZLzpAXO/sj+IF/x2GtTMjQ==} engines: {node: ^16.0.0 || >=18.0.0} peerDependencies: eslint: ^7.0.0 || ^8.0.0 @@ -4308,8 +4300,8 @@ packages: typescript: optional: true dependencies: - '@typescript-eslint/typescript-estree': 6.7.5(typescript@5.2.2) - '@typescript-eslint/utils': 6.7.5(eslint@8.52.0)(typescript@5.2.2) + '@typescript-eslint/typescript-estree': 6.9.0(typescript@5.2.2) + '@typescript-eslint/utils': 6.9.0(eslint@8.52.0)(typescript@5.2.2) debug: 4.3.4(supports-color@8.1.1) eslint: 8.52.0 ts-api-utils: 1.0.1(typescript@5.2.2) @@ -4318,37 +4310,11 @@ packages: - supports-color dev: true - /@typescript-eslint/types@6.7.5: - resolution: {integrity: sha512-WboQBlOXtdj1tDFPyIthpKrUb+kZf2VroLZhxKa/VlwLlLyqv/PwUNgL30BlTVZV1Wu4Asu2mMYPqarSO4L5ZQ==} - engines: {node: ^16.0.0 || >=18.0.0} - dev: true - /@typescript-eslint/types@6.9.0: resolution: {integrity: sha512-+KB0lbkpxBkBSiVCuQvduqMJy+I1FyDbdwSpM3IoBS7APl4Bu15lStPjgBIdykdRqQNYqYNMa8Kuidax6phaEw==} engines: {node: ^16.0.0 || >=18.0.0} dev: true - /@typescript-eslint/typescript-estree@6.7.5(typescript@5.2.2): - resolution: {integrity: sha512-NhJiJ4KdtwBIxrKl0BqG1Ur+uw7FiOnOThcYx9DpOGJ/Abc9z2xNzLeirCG02Ig3vkvrc2qFLmYSSsaITbKjlg==} - engines: {node: ^16.0.0 || >=18.0.0} - peerDependencies: - typescript: '*' - peerDependenciesMeta: - typescript: - optional: true - dependencies: - '@typescript-eslint/types': 6.7.5 - '@typescript-eslint/visitor-keys': 6.7.5 - debug: 4.3.4(supports-color@8.1.1) - globby: 11.1.0 - is-glob: 4.0.3 - semver: 7.5.4 - ts-api-utils: 1.0.1(typescript@5.2.2) - typescript: 5.2.2 - transitivePeerDependencies: - - supports-color - dev: true - /@typescript-eslint/typescript-estree@6.9.0(typescript@5.2.2): resolution: {integrity: sha512-NJM2BnJFZBEAbCfBP00zONKXvMqihZCrmwCaik0UhLr0vAgb6oguXxLX1k00oQyD+vZZ+CJn3kocvv2yxm4awQ==} engines: {node: ^16.0.0 || >=18.0.0} @@ -4370,8 +4336,8 @@ packages: - supports-color dev: true - /@typescript-eslint/utils@6.7.5(eslint@8.52.0)(typescript@5.2.2): - resolution: {integrity: sha512-pfRRrH20thJbzPPlPc4j0UNGvH1PjPlhlCMq4Yx7EGjV7lvEeGX0U6MJYe8+SyFutWgSHsdbJ3BXzZccYggezA==} + /@typescript-eslint/utils@6.9.0(eslint@8.52.0)(typescript@5.2.2): + resolution: {integrity: sha512-5Wf+Jsqya7WcCO8me504FBigeQKVLAMPmUzYgDbWchINNh1KJbxCgVya3EQ2MjvJMVeXl3pofRmprqX6mfQkjQ==} engines: {node: ^16.0.0 || >=18.0.0} peerDependencies: eslint: ^7.0.0 || ^8.0.0 @@ -4379,9 +4345,9 @@ packages: '@eslint-community/eslint-utils': 4.4.0(eslint@8.52.0) '@types/json-schema': 7.0.12 '@types/semver': 7.5.0 - '@typescript-eslint/scope-manager': 6.7.5 - '@typescript-eslint/types': 6.7.5 - '@typescript-eslint/typescript-estree': 6.7.5(typescript@5.2.2) + '@typescript-eslint/scope-manager': 6.9.0 + '@typescript-eslint/types': 6.9.0 + '@typescript-eslint/typescript-estree': 6.9.0(typescript@5.2.2) eslint: 8.52.0 semver: 7.5.4 transitivePeerDependencies: @@ -4389,14 +4355,6 @@ packages: - typescript dev: true - /@typescript-eslint/visitor-keys@6.7.5: - resolution: {integrity: sha512-3MaWdDZtLlsexZzDSdQWsFQ9l9nL8B80Z4fImSpyllFC/KLqWQRdEcB+gGGO+N3Q2uL40EsG66wZLsohPxNXvg==} - engines: {node: ^16.0.0 || >=18.0.0} - dependencies: - '@typescript-eslint/types': 6.7.5 - eslint-visitor-keys: 3.4.3 - dev: true - /@typescript-eslint/visitor-keys@6.9.0: resolution: {integrity: sha512-dGtAfqjV6RFOtIP8I0B4ZTBRrlTT8NHHlZZSchQx3qReaoDeXhYM++M4So2AgFK9ZB0emRPA6JI1HkafzA2Ibg==} engines: {node: ^16.0.0 || >=18.0.0} From 78917284539d28f44eb5268533a79ac0886a653e Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 25 Oct 2023 00:20:45 +0000 Subject: [PATCH 060/279] build(deps-dev): bump @types/pino-multi-stream from 5.1.4 to 5.1.5 Bumps [@types/pino-multi-stream](https://github.com/DefinitelyTyped/DefinitelyTyped/tree/HEAD/types/pino-multi-stream) from 5.1.4 to 5.1.5. - [Release notes](https://github.com/DefinitelyTyped/DefinitelyTyped/releases) - [Commits](https://github.com/DefinitelyTyped/DefinitelyTyped/commits/HEAD/types/pino-multi-stream) --- updated-dependencies: - dependency-name: "@types/pino-multi-stream" dependency-type: direct:development update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] --- packages/common-ts/package.json | 2 +- pnpm-lock.yaml | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/packages/common-ts/package.json b/packages/common-ts/package.json index 7f7a67e4ce1f..72060a89145c 100644 --- a/packages/common-ts/package.json +++ b/packages/common-ts/package.json @@ -57,7 +57,7 @@ "@types/express": "^4.17.19", "@types/morgan": "^1.9.7", "@types/pino": "^7.0.5", - "@types/pino-multi-stream": "^5.1.4", + "@types/pino-multi-stream": "^5.1.5", "chai": "^4.3.10", "supertest": "^6.3.3" } diff --git a/pnpm-lock.yaml b/pnpm-lock.yaml index e6263969dd84..171d6f276c4f 100644 --- a/pnpm-lock.yaml +++ b/pnpm-lock.yaml @@ -256,8 +256,8 @@ importers: specifier: ^7.0.5 version: 7.0.5 '@types/pino-multi-stream': - specifier: ^5.1.4 - version: 5.1.4 + specifier: ^5.1.5 + version: 5.1.5 chai: specifier: ^4.3.10 version: 4.3.10 @@ -4083,8 +4083,8 @@ packages: '@types/node': 20.8.8 dev: true - /@types/pino-multi-stream@5.1.4: - resolution: {integrity: sha512-+UrFCcIx4cUo473GwCt5/v8m0r+W08iuaphu8L5bx76zcShvQRWPtKkRxImUCLXW9bHpPFHAJZKANMP/vAbycA==} + /@types/pino-multi-stream@5.1.5: + resolution: {integrity: sha512-md4VfNYACuZZVgBbYqv9D4msKIyLmIdyf/kdjT7HG/xhjM18pKseDGylME0nPvNR66bpYQ2i4v4zPWe8FzwZPA==} dependencies: '@types/pino': 6.3.11 dev: true From f1a4e7f202ccd5136f4920a1b14182df2b5390a0 Mon Sep 17 00:00:00 2001 From: Mark Tyneway Date: Tue, 24 Oct 2023 18:55:13 -0600 Subject: [PATCH 061/279] contracts-bedrock: dispute game setup fix Prevent `setUp` from being called twice in the dispute game tests. `setUp` is called from within `init` which in some cases resulted in the top level `setUp` being called twice. Now `init` is no longer responsible for calling `setUp`. Ran into this issue when porting the common test script to use the deploy script to set up the forge unit tests. --- packages/contracts-bedrock/.gas-snapshot | 4 ++-- .../contracts-bedrock/test/FaultDisputeGame.t.sol | 13 +++++++++++-- 2 files changed, 13 insertions(+), 4 deletions(-) diff --git a/packages/contracts-bedrock/.gas-snapshot b/packages/contracts-bedrock/.gas-snapshot index 7c2b6a4f7e29..ac878dee55d9 100644 --- a/packages/contracts-bedrock/.gas-snapshot +++ b/packages/contracts-bedrock/.gas-snapshot @@ -660,9 +660,9 @@ SequencerFeeVault_Test:test_withdraw_toL1_succeeds() (gas: 171675) SetPrevBaseFee_Test:test_setPrevBaseFee_succeeds() (gas: 11549) StandardBridge_Stateless_Test:test_isCorrectTokenPair_succeeds() (gas: 49936) StandardBridge_Stateless_Test:test_isOptimismMintableERC20_succeeds() (gas: 33072) -Storage_Roundtrip_Test:test_setGetAddress_succeeds(bytes32,address) (runs: 64, μ: 31510, ~: 31821) +Storage_Roundtrip_Test:test_setGetAddress_succeeds(bytes32,address) (runs: 64, μ: 31821, ~: 31821) Storage_Roundtrip_Test:test_setGetBytes32_succeeds(bytes32,bytes32) (runs: 64, μ: 31598, ~: 31598) -Storage_Roundtrip_Test:test_setGetUint_succeeds(bytes32,uint256) (runs: 64, μ: 30420, ~: 31664) +Storage_Roundtrip_Test:test_setGetUint_succeeds(bytes32,uint256) (runs: 64, μ: 30731, ~: 31664) SystemConfig_Initialize_Test:test_initialize_events_succeeds() (gas: 71972) SystemConfig_Initialize_Test:test_initialize_startBlockNoop_reverts() (gas: 81247) SystemConfig_Initialize_Test:test_initialize_startBlockOverride_succeeds() (gas: 65143) diff --git a/packages/contracts-bedrock/test/FaultDisputeGame.t.sol b/packages/contracts-bedrock/test/FaultDisputeGame.t.sol index 674e7d335e97..e56665e1e784 100644 --- a/packages/contracts-bedrock/test/FaultDisputeGame.t.sol +++ b/packages/contracts-bedrock/test/FaultDisputeGame.t.sol @@ -32,8 +32,6 @@ contract FaultDisputeGame_Init is DisputeGameFactory_Init { event Move(uint256 indexed parentIndex, Claim indexed pivot, address indexed claimant); function init(Claim rootClaim, Claim absolutePrestate) public { - super.setUp(); - // Set the time to a realistic date. vm.warp(1690906994); @@ -82,6 +80,7 @@ contract FaultDisputeGame_Test is FaultDisputeGame_Init { Claim internal constant ABSOLUTE_PRESTATE = Claim.wrap(bytes32((uint256(3) << 248) | uint256(0))); function setUp() public override { + super.setUp(); super.init(ROOT_CLAIM, ABSOLUTE_PRESTATE); } @@ -755,6 +754,7 @@ contract OneVsOne_Arena is FaultDisputeGame_Init { contract FaultDisputeGame_ResolvesCorrectly_IncorrectRoot1 is OneVsOne_Arena { function setUp() public override { + super.setUp(); GamePlayer honest = new HonestPlayer(ABSOLUTE_PRESTATE); GamePlayer dishonest = new VariableDivergentPlayer(ABSOLUTE_PRESTATE, 16, 0); super.init(dishonest, honest, 15); @@ -777,6 +777,7 @@ contract FaultDisputeGame_ResolvesCorrectly_IncorrectRoot1 is OneVsOne_Arena { contract FaultDisputeGame_ResolvesCorrectly_CorrectRoot1 is OneVsOne_Arena { function setUp() public override { + super.setUp(); GamePlayer honest = new HonestPlayer(ABSOLUTE_PRESTATE); GamePlayer dishonest = new VariableDivergentPlayer(ABSOLUTE_PRESTATE, 16, 0); super.init(honest, dishonest, 15); @@ -799,6 +800,7 @@ contract FaultDisputeGame_ResolvesCorrectly_CorrectRoot1 is OneVsOne_Arena { contract FaultDisputeGame_ResolvesCorrectly_IncorrectRoot2 is OneVsOne_Arena { function setUp() public override { + super.setUp(); GamePlayer honest = new HonestPlayer(ABSOLUTE_PRESTATE); GamePlayer dishonest = new VariableDivergentPlayer(ABSOLUTE_PRESTATE, 16, 7); super.init(dishonest, honest, 15); @@ -821,6 +823,7 @@ contract FaultDisputeGame_ResolvesCorrectly_IncorrectRoot2 is OneVsOne_Arena { contract FaultDisputeGame_ResolvesCorrectly_CorrectRoot2 is OneVsOne_Arena { function setUp() public override { + super.setUp(); GamePlayer honest = new HonestPlayer(ABSOLUTE_PRESTATE); GamePlayer dishonest = new VariableDivergentPlayer(ABSOLUTE_PRESTATE, 16, 7); super.init(honest, dishonest, 15); @@ -843,6 +846,7 @@ contract FaultDisputeGame_ResolvesCorrectly_CorrectRoot2 is OneVsOne_Arena { contract FaultDisputeGame_ResolvesCorrectly_IncorrectRoot3 is OneVsOne_Arena { function setUp() public override { + super.setUp(); GamePlayer honest = new HonestPlayer(ABSOLUTE_PRESTATE); GamePlayer dishonest = new VariableDivergentPlayer(ABSOLUTE_PRESTATE, 16, 2); super.init(dishonest, honest, 15); @@ -865,6 +869,7 @@ contract FaultDisputeGame_ResolvesCorrectly_IncorrectRoot3 is OneVsOne_Arena { contract FaultDisputeGame_ResolvesCorrectly_CorrectRoot3 is OneVsOne_Arena { function setUp() public override { + super.setUp(); GamePlayer honest = new HonestPlayer(ABSOLUTE_PRESTATE); GamePlayer dishonest = new VariableDivergentPlayer(ABSOLUTE_PRESTATE, 16, 2); super.init(honest, dishonest, 15); @@ -887,6 +892,7 @@ contract FaultDisputeGame_ResolvesCorrectly_CorrectRoot3 is OneVsOne_Arena { contract FaultDisputeGame_ResolvesCorrectly_IncorrectRoot4 is OneVsOne_Arena { function setUp() public override { + super.setUp(); GamePlayer honest = new HonestPlayer_HalfTrace(ABSOLUTE_PRESTATE); GamePlayer dishonest = new VariableDivergentPlayer(ABSOLUTE_PRESTATE, 8, 5); super.init(dishonest, honest, 7); @@ -909,6 +915,7 @@ contract FaultDisputeGame_ResolvesCorrectly_IncorrectRoot4 is OneVsOne_Arena { contract FaultDisputeGame_ResolvesCorrectly_CorrectRoot4 is OneVsOne_Arena { function setUp() public override { + super.setUp(); GamePlayer honest = new HonestPlayer_HalfTrace(ABSOLUTE_PRESTATE); GamePlayer dishonest = new VariableDivergentPlayer(ABSOLUTE_PRESTATE, 8, 5); super.init(honest, dishonest, 7); @@ -931,6 +938,7 @@ contract FaultDisputeGame_ResolvesCorrectly_CorrectRoot4 is OneVsOne_Arena { contract FaultDisputeGame_ResolvesCorrectly_IncorrectRoot5 is OneVsOne_Arena { function setUp() public override { + super.setUp(); GamePlayer honest = new HonestPlayer_QuarterTrace(ABSOLUTE_PRESTATE); GamePlayer dishonest = new VariableDivergentPlayer(ABSOLUTE_PRESTATE, 4, 3); super.init(dishonest, honest, 3); @@ -953,6 +961,7 @@ contract FaultDisputeGame_ResolvesCorrectly_IncorrectRoot5 is OneVsOne_Arena { contract FaultDisputeGame_ResolvesCorrectly_CorrectRoot5 is OneVsOne_Arena { function setUp() public override { + super.setUp(); GamePlayer honest = new HonestPlayer_QuarterTrace(ABSOLUTE_PRESTATE); GamePlayer dishonest = new VariableDivergentPlayer(ABSOLUTE_PRESTATE, 4, 3); super.init(honest, dishonest, 3); From 460bb3f31cadf0bdc6b5f70c9a57ace068e06554 Mon Sep 17 00:00:00 2001 From: Tei Im Date: Mon, 18 Sep 2023 16:48:09 +0900 Subject: [PATCH 062/279] Implement span batch submission for op-batcher --- op-batcher/batcher/channel.go | 4 +- op-batcher/batcher/channel_builder.go | 17 +- op-batcher/batcher/channel_builder_test.go | 253 ++++++++++++++------- op-batcher/batcher/channel_manager.go | 38 +++- op-batcher/batcher/channel_manager_test.go | 123 +++++++--- op-batcher/batcher/channel_test.go | 13 +- op-batcher/batcher/config.go | 3 + op-batcher/batcher/driver.go | 41 +++- op-batcher/batcher/service.go | 1 + op-batcher/flags/flags.go | 7 + op-e2e/actions/l2_batcher.go | 2 +- op-e2e/setup.go | 8 +- op-node/rollup/derive/channel_out.go | 173 +++++++++++--- op-node/rollup/derive/channel_out_test.go | 22 +- ops-bedrock/docker-compose.yml | 1 + 15 files changed, 524 insertions(+), 182 deletions(-) diff --git a/op-batcher/batcher/channel.go b/op-batcher/batcher/channel.go index d42679eb34d5..cd44f31a9c46 100644 --- a/op-batcher/batcher/channel.go +++ b/op-batcher/batcher/channel.go @@ -26,8 +26,8 @@ type channel struct { confirmedTransactions map[txID]eth.BlockID } -func newChannel(log log.Logger, metr metrics.Metricer, cfg ChannelConfig) (*channel, error) { - cb, err := newChannelBuilder(cfg) +func newChannel(log log.Logger, metr metrics.Metricer, cfg ChannelConfig, spanBatchBuilder *derive.SpanBatchBuilder) (*channel, error) { + cb, err := newChannelBuilder(cfg, spanBatchBuilder) if err != nil { return nil, fmt.Errorf("creating new channel: %w", err) } diff --git a/op-batcher/batcher/channel_builder.go b/op-batcher/batcher/channel_builder.go index 551e5d96e672..33e236c2dd93 100644 --- a/op-batcher/batcher/channel_builder.go +++ b/op-batcher/batcher/channel_builder.go @@ -58,6 +58,9 @@ type ChannelConfig struct { // CompressorConfig contains the configuration for creating new compressors. CompressorConfig compressor.Config + + // BatchType indicates whether the channel uses SingularBatch or SpanBatch. + BatchType uint } // Check validates the [ChannelConfig] parameters. @@ -83,6 +86,10 @@ func (cc *ChannelConfig) Check() error { return fmt.Errorf("max frame size %d is less than the minimum 23", cc.MaxFrameSize) } + if cc.BatchType > derive.SpanBatchType { + return fmt.Errorf("unrecognized batch type: %d", cc.BatchType) + } + return nil } @@ -127,12 +134,12 @@ type channelBuilder struct { // newChannelBuilder creates a new channel builder or returns an error if the // channel out could not be created. -func newChannelBuilder(cfg ChannelConfig) (*channelBuilder, error) { +func newChannelBuilder(cfg ChannelConfig, spanBatchBuilder *derive.SpanBatchBuilder) (*channelBuilder, error) { c, err := cfg.CompressorConfig.NewCompressor() if err != nil { return nil, err } - co, err := derive.NewChannelOut(c) + co, err := derive.NewChannelOut(c, cfg.BatchType, spanBatchBuilder) if err != nil { return nil, err } @@ -194,12 +201,12 @@ func (c *channelBuilder) AddBlock(block *types.Block) (derive.L1BlockInfo, error return derive.L1BlockInfo{}, c.FullErr() } - batch, l1info, err := derive.BlockToBatch(block) + batch, l1info, err := derive.BlockToSingularBatch(block) if err != nil { return l1info, fmt.Errorf("converting block to batch: %w", err) } - if _, err = c.co.AddBatch(batch); errors.Is(err, derive.ErrTooManyRLPBytes) || errors.Is(err, derive.CompressorFullErr) { + if _, err = c.co.AddSingularBatch(batch); errors.Is(err, derive.ErrTooManyRLPBytes) || errors.Is(err, derive.CompressorFullErr) { c.setFullErr(err) return l1info, c.FullErr() } else if err != nil { @@ -252,7 +259,7 @@ func (c *channelBuilder) updateDurationTimeout(l1BlockNum uint64) { // derived from the batch's origin L1 block. The timeout is only moved forward // if the derived sequencer window timeout is earlier than the currently set // timeout. -func (c *channelBuilder) updateSwTimeout(batch *derive.BatchData) { +func (c *channelBuilder) updateSwTimeout(batch *derive.SingularBatch) { timeout := uint64(batch.EpochNum) + c.cfg.SeqWindowSize - c.cfg.SubSafetyMargin c.updateTimeout(timeout, ErrSeqWindowClose) } diff --git a/op-batcher/batcher/channel_builder_test.go b/op-batcher/batcher/channel_builder_test.go index 45ee5cef0bf0..51c3d76af380 100644 --- a/op-batcher/batcher/channel_builder_test.go +++ b/op-batcher/batcher/channel_builder_test.go @@ -32,6 +32,15 @@ var defaultTestChannelConfig = ChannelConfig{ TargetNumFrames: 1, ApproxComprRatio: 0.4, }, + BatchType: derive.SingularBatchType, +} + +func getSpanBatchBuilder(batchType uint) *derive.SpanBatchBuilder { + if batchType == derive.SpanBatchType { + chainId := big.NewInt(1234) + return derive.NewSpanBatchBuilder(uint64(0), uint64(0), chainId) + } + return nil } // TestChannelConfig_Check tests the [ChannelConfig] [Check] function. @@ -158,8 +167,9 @@ func newMiniL2BlockWithNumberParent(numTx int, number *big.Int, parent common.Ha // addTooManyBlocks adds blocks to the channel until it hits an error, // which is presumably ErrTooManyRLPBytes. func addTooManyBlocks(cb *channelBuilder) error { + rng := rand.New(rand.NewSource(1234)) for i := 0; i < 10_000; i++ { - block := newMiniL2Block(100) + block, _ := dtest.RandomL2Block(rng, 1000) _, err := cb.AddBlock(block) if err != nil { return err @@ -178,7 +188,7 @@ func FuzzDurationTimeoutZeroMaxChannelDuration(f *testing.F) { f.Fuzz(func(t *testing.T, l1BlockNum uint64) { channelConfig := defaultTestChannelConfig channelConfig.MaxChannelDuration = 0 - cb, err := newChannelBuilder(channelConfig) + cb, err := newChannelBuilder(channelConfig, nil) require.NoError(t, err) cb.timeout = 0 cb.updateDurationTimeout(l1BlockNum) @@ -201,7 +211,7 @@ func FuzzChannelBuilder_DurationZero(f *testing.F) { // Create the channel builder channelConfig := defaultTestChannelConfig channelConfig.MaxChannelDuration = maxChannelDuration - cb, err := newChannelBuilder(channelConfig) + cb, err := newChannelBuilder(channelConfig, nil) require.NoError(t, err) // Whenever the timeout is set to 0, the channel builder should have a duration timeout @@ -228,7 +238,7 @@ func FuzzDurationTimeoutMaxChannelDuration(f *testing.F) { // Create the channel builder channelConfig := defaultTestChannelConfig channelConfig.MaxChannelDuration = maxChannelDuration - cb, err := newChannelBuilder(channelConfig) + cb, err := newChannelBuilder(channelConfig, nil) require.NoError(t, err) // Whenever the timeout is greater than the l1BlockNum, @@ -262,7 +272,7 @@ func FuzzChannelCloseTimeout(f *testing.F) { channelConfig := defaultTestChannelConfig channelConfig.ChannelTimeout = channelTimeout channelConfig.SubSafetyMargin = subSafetyMargin - cb, err := newChannelBuilder(channelConfig) + cb, err := newChannelBuilder(channelConfig, nil) require.NoError(t, err) // Check the timeout @@ -290,7 +300,7 @@ func FuzzChannelZeroCloseTimeout(f *testing.F) { channelConfig := defaultTestChannelConfig channelConfig.ChannelTimeout = channelTimeout channelConfig.SubSafetyMargin = subSafetyMargin - cb, err := newChannelBuilder(channelConfig) + cb, err := newChannelBuilder(channelConfig, nil) require.NoError(t, err) // Check the timeout @@ -317,16 +327,12 @@ func FuzzSeqWindowClose(f *testing.F) { channelConfig := defaultTestChannelConfig channelConfig.SeqWindowSize = seqWindowSize channelConfig.SubSafetyMargin = subSafetyMargin - cb, err := newChannelBuilder(channelConfig) + cb, err := newChannelBuilder(channelConfig, nil) require.NoError(t, err) // Check the timeout cb.timeout = timeout - cb.updateSwTimeout(derive.NewSingularBatchData( - derive.SingularBatch{ - EpochNum: rollup.Epoch(epochNum), - }, - )) + cb.updateSwTimeout(&derive.SingularBatch{EpochNum: rollup.Epoch(epochNum)}) calculatedTimeout := epochNum + seqWindowSize - subSafetyMargin if timeout > calculatedTimeout && calculatedTimeout != 0 { cb.checkTimeout(calculatedTimeout) @@ -349,16 +355,12 @@ func FuzzSeqWindowZeroTimeoutClose(f *testing.F) { channelConfig := defaultTestChannelConfig channelConfig.SeqWindowSize = seqWindowSize channelConfig.SubSafetyMargin = subSafetyMargin - cb, err := newChannelBuilder(channelConfig) + cb, err := newChannelBuilder(channelConfig, nil) require.NoError(t, err) // Check the timeout cb.timeout = 0 - cb.updateSwTimeout(derive.NewSingularBatchData( - derive.SingularBatch{ - EpochNum: rollup.Epoch(epochNum), - }, - )) + cb.updateSwTimeout(&derive.SingularBatch{EpochNum: rollup.Epoch(epochNum)}) calculatedTimeout := epochNum + seqWindowSize - subSafetyMargin cb.checkTimeout(calculatedTimeout) if cb.timeout != 0 { @@ -367,12 +369,40 @@ func FuzzSeqWindowZeroTimeoutClose(f *testing.F) { }) } +func TestChannelBuilderBatchType(t *testing.T) { + tests := []struct { + name string + f func(t *testing.T, batchType uint) + }{ + {"ChannelBuilder_MaxRLPBytesPerChannel", ChannelBuilder_MaxRLPBytesPerChannel}, + {"ChannelBuilder_OutputFramesMaxFrameIndex", ChannelBuilder_OutputFramesMaxFrameIndex}, + {"ChannelBuilder_AddBlock", ChannelBuilder_AddBlock}, + {"ChannelBuilder_Reset", ChannelBuilder_Reset}, + {"ChannelBuilder_PendingFrames_TotalFrames", ChannelBuilder_PendingFrames_TotalFrames}, + {"ChannelBuilder_InputBytes", ChannelBuilder_InputBytes}, + {"ChannelBuilder_OutputBytes", ChannelBuilder_OutputBytes}, + } + for _, test := range tests { + test := test + t.Run(test.name+"_SingularBatch", func(t *testing.T) { + test.f(t, derive.SingularBatchType) + }) + } + + for _, test := range tests { + test := test + t.Run(test.name+"_SpanBatch", func(t *testing.T) { + test.f(t, derive.SpanBatchType) + }) + } +} + // TestChannelBuilder_NextFrame tests calling NextFrame on a ChannelBuilder with only one frame func TestChannelBuilder_NextFrame(t *testing.T) { channelConfig := defaultTestChannelConfig // Create a new channel builder - cb, err := newChannelBuilder(channelConfig) + cb, err := newChannelBuilder(channelConfig, nil) require.NoError(t, err) // Mock the internals of `channelBuilder.outputFrame` @@ -412,14 +442,14 @@ func TestChannelBuilder_OutputWrongFramePanic(t *testing.T) { channelConfig := defaultTestChannelConfig // Construct a channel builder - cb, err := newChannelBuilder(channelConfig) + cb, err := newChannelBuilder(channelConfig, nil) require.NoError(t, err) // Mock the internals of `channelBuilder.outputFrame` // to construct a single frame c, err := channelConfig.CompressorConfig.NewCompressor() require.NoError(t, err) - co, err := derive.NewChannelOut(c) + co, err := derive.NewChannelOut(c, derive.SingularBatchType, nil) require.NoError(t, err) var buf bytes.Buffer fn, err := co.OutputFrame(&buf, channelConfig.MaxFrameSize) @@ -445,7 +475,7 @@ func TestChannelBuilder_OutputFramesWorks(t *testing.T) { channelConfig.MaxFrameSize = 24 // Construct the channel builder - cb, err := newChannelBuilder(channelConfig) + cb, err := newChannelBuilder(channelConfig, nil) require.NoError(t, err) require.False(t, cb.IsFull()) require.Equal(t, 0, cb.PendingFrames()) @@ -480,17 +510,68 @@ func TestChannelBuilder_OutputFramesWorks(t *testing.T) { } } -// TestChannelBuilder_MaxRLPBytesPerChannel tests the [channelBuilder.OutputFrames] +// TestChannelBuilder_OutputFramesWorks tests the [ChannelBuilder] OutputFrames is successful. +func TestChannelBuilder_OutputFramesWorks_SpanBatch(t *testing.T) { + channelConfig := defaultTestChannelConfig + channelConfig.MaxFrameSize = 24 + channelConfig.CompressorConfig.TargetFrameSize = 50 + channelConfig.BatchType = derive.SpanBatchType + + // Construct the channel builder + cb, err := newChannelBuilder(channelConfig, getSpanBatchBuilder(derive.SpanBatchType)) + require.NoError(t, err) + require.False(t, cb.IsFull()) + require.Equal(t, 0, cb.PendingFrames()) + + // Calling OutputFrames without having called [AddBlock] + // should return no error + require.NoError(t, cb.OutputFrames()) + + // There should be no ready bytes yet + require.Equal(t, 0, cb.co.ReadyBytes()) + + // fill up + for { + err = addMiniBlock(cb) + if err == nil { + require.False(t, cb.IsFull()) + // There should be no ready bytes until the channel is full + require.Equal(t, cb.co.ReadyBytes(), 0) + } else { + require.ErrorIs(t, err, derive.CompressorFullErr) + break + } + } + + require.True(t, cb.IsFull()) + // Check how many ready bytes + // There should be more than the max frame size ready + require.Greater(t, uint64(cb.co.ReadyBytes()), channelConfig.MaxFrameSize) + require.Equal(t, 0, cb.PendingFrames()) + + // We should be able to output the frames + require.NoError(t, cb.OutputFrames()) + + // There should be many frames in the channel builder now + require.Greater(t, cb.PendingFrames(), 1) + for i := 0; i < cb.numFrames-1; i++ { + require.Len(t, cb.frames[i].data, int(channelConfig.MaxFrameSize)) + } + require.LessOrEqual(t, len(cb.frames[len(cb.frames)-1].data), int(channelConfig.MaxFrameSize)) +} + +// ChannelBuilder_MaxRLPBytesPerChannel tests the [channelBuilder.OutputFrames] // function errors when the max RLP bytes per channel is reached. -func TestChannelBuilder_MaxRLPBytesPerChannel(t *testing.T) { +func ChannelBuilder_MaxRLPBytesPerChannel(t *testing.T, batchType uint) { t.Parallel() channelConfig := defaultTestChannelConfig channelConfig.MaxFrameSize = derive.MaxRLPBytesPerChannel * 2 channelConfig.CompressorConfig.TargetFrameSize = derive.MaxRLPBytesPerChannel * 2 channelConfig.CompressorConfig.ApproxComprRatio = 1 + channelConfig.BatchType = batchType // Construct the channel builder - cb, err := newChannelBuilder(channelConfig) + cb, err := newChannelBuilder(channelConfig, getSpanBatchBuilder(batchType)) require.NoError(t, err) // Add a block that overflows the [ChannelOut] @@ -498,61 +579,55 @@ func TestChannelBuilder_MaxRLPBytesPerChannel(t *testing.T) { require.ErrorIs(t, err, derive.ErrTooManyRLPBytes) } -// TestChannelBuilder_OutputFramesMaxFrameIndex tests the [ChannelBuilder.OutputFrames] +// ChannelBuilder_OutputFramesMaxFrameIndex tests the [ChannelBuilder.OutputFrames] // function errors when the max frame index is reached. -func TestChannelBuilder_OutputFramesMaxFrameIndex(t *testing.T) { +func ChannelBuilder_OutputFramesMaxFrameIndex(t *testing.T, batchType uint) { channelConfig := defaultTestChannelConfig channelConfig.MaxFrameSize = 24 - channelConfig.CompressorConfig.TargetNumFrames = math.MaxInt + channelConfig.CompressorConfig.TargetNumFrames = 6000 channelConfig.CompressorConfig.TargetFrameSize = 24 - channelConfig.CompressorConfig.ApproxComprRatio = 0 + channelConfig.CompressorConfig.ApproxComprRatio = 1 + channelConfig.BatchType = batchType + + rng := rand.New(rand.NewSource(123)) // Continuously add blocks until the max frame index is reached // This should cause the [channelBuilder.OutputFrames] function // to error - cb, err := newChannelBuilder(channelConfig) + cb, err := newChannelBuilder(channelConfig, getSpanBatchBuilder(batchType)) require.NoError(t, err) require.False(t, cb.IsFull()) require.Equal(t, 0, cb.PendingFrames()) for { - lBlock := types.NewBlock(&types.Header{ - BaseFee: common.Big0, - Difficulty: common.Big0, - Number: common.Big0, - }, nil, nil, nil, trie.NewStackTrie(nil)) - l1InfoTx, _ := derive.L1InfoDeposit(0, eth.BlockToInfo(lBlock), eth.SystemConfig{}, false) - txs := []*types.Transaction{types.NewTx(l1InfoTx)} - a := types.NewBlock(&types.Header{ - Number: big.NewInt(0), - }, txs, nil, nil, trie.NewStackTrie(nil)) + a, _ := dtest.RandomL2Block(rng, 1) _, err = cb.AddBlock(a) - require.NoError(t, cb.co.Flush()) if cb.IsFull() { fullErr := cb.FullErr() - require.ErrorIs(t, fullErr, ErrMaxFrameIndex) + require.ErrorIs(t, fullErr, derive.CompressorFullErr) break } require.NoError(t, err) - _ = cb.OutputFrames() - // Flushing so we can construct new frames - _ = cb.co.Flush() } + + _ = cb.OutputFrames() + require.ErrorIs(t, cb.FullErr(), ErrMaxFrameIndex) } -// TestChannelBuilder_AddBlock tests the AddBlock function -func TestChannelBuilder_AddBlock(t *testing.T) { +// ChannelBuilder_AddBlock tests the AddBlock function +func ChannelBuilder_AddBlock(t *testing.T, batchType uint) { channelConfig := defaultTestChannelConfig + channelConfig.BatchType = batchType // Lower the max frame size so that we can batch - channelConfig.MaxFrameSize = 30 + channelConfig.MaxFrameSize = 20 // Configure the Input Threshold params so we observe a full channel - channelConfig.CompressorConfig.TargetFrameSize = 30 + channelConfig.CompressorConfig.TargetFrameSize = 20 channelConfig.CompressorConfig.TargetNumFrames = 2 channelConfig.CompressorConfig.ApproxComprRatio = 1 // Construct the channel builder - cb, err := newChannelBuilder(channelConfig) + cb, err := newChannelBuilder(channelConfig, getSpanBatchBuilder(batchType)) require.NoError(t, err) // Add a nonsense block to the channel builder @@ -560,7 +635,11 @@ func TestChannelBuilder_AddBlock(t *testing.T) { require.NoError(t, cb.co.Flush()) // Check the fields reset in the AddBlock function - require.Equal(t, 74, cb.co.InputBytes()) + expectedInputBytes := 74 + if batchType == derive.SpanBatchType { + expectedInputBytes = 47 + } + require.Equal(t, expectedInputBytes, cb.co.InputBytes()) require.Equal(t, 1, len(cb.blocks)) require.Equal(t, 0, len(cb.frames)) require.True(t, cb.IsFull()) @@ -570,14 +649,18 @@ func TestChannelBuilder_AddBlock(t *testing.T) { require.ErrorIs(t, addMiniBlock(cb), derive.CompressorFullErr) } -// TestChannelBuilder_Reset tests the [Reset] function -func TestChannelBuilder_Reset(t *testing.T) { +// ChannelBuilder_Reset tests the [Reset] function +func ChannelBuilder_Reset(t *testing.T, batchType uint) { channelConfig := defaultTestChannelConfig + channelConfig.BatchType = batchType // Lower the max frame size so that we can batch channelConfig.MaxFrameSize = 24 + channelConfig.CompressorConfig.TargetNumFrames = 1 + channelConfig.CompressorConfig.TargetFrameSize = 24 + channelConfig.CompressorConfig.ApproxComprRatio = 1 - cb, err := newChannelBuilder(channelConfig) + cb, err := newChannelBuilder(channelConfig, getSpanBatchBuilder(batchType)) require.NoError(t, err) // Add a nonsense block to the channel builder @@ -590,20 +673,16 @@ func TestChannelBuilder_Reset(t *testing.T) { // Timeout should be updated in the AddBlock internal call to `updateSwTimeout` timeout := uint64(100) + cb.cfg.SeqWindowSize - cb.cfg.SubSafetyMargin require.Equal(t, timeout, cb.timeout) - require.NoError(t, cb.fullErr) + require.Error(t, cb.fullErr) // Output frames so we can set the channel builder frames require.NoError(t, cb.OutputFrames()) - // Add another block to increment the block count - require.NoError(t, addMiniBlock(cb)) - require.NoError(t, cb.co.Flush()) - // Check the fields reset in the Reset function - require.Equal(t, 2, len(cb.blocks)) - require.Greater(t, len(cb.frames), 1) + require.Equal(t, 1, len(cb.blocks)) require.Equal(t, timeout, cb.timeout) - require.NoError(t, cb.fullErr) + require.Error(t, cb.fullErr) + require.Greater(t, len(cb.frames), 1) // Reset the channel builder require.NoError(t, cb.Reset()) @@ -622,7 +701,7 @@ func TestBuilderRegisterL1Block(t *testing.T) { channelConfig := defaultTestChannelConfig // Construct the channel builder - cb, err := newChannelBuilder(channelConfig) + cb, err := newChannelBuilder(channelConfig, nil) require.NoError(t, err) // Assert params modified in RegisterL1Block @@ -645,7 +724,7 @@ func TestBuilderRegisterL1BlockZeroMaxChannelDuration(t *testing.T) { channelConfig.MaxChannelDuration = 0 // Construct the channel builder - cb, err := newChannelBuilder(channelConfig) + cb, err := newChannelBuilder(channelConfig, nil) require.NoError(t, err) // Assert params modified in RegisterL1Block @@ -666,7 +745,7 @@ func TestFramePublished(t *testing.T) { channelConfig := defaultTestChannelConfig // Construct the channel builder - cb, err := newChannelBuilder(channelConfig) + cb, err := newChannelBuilder(channelConfig, nil) require.NoError(t, err) // Let's say the block number is fed in as 100 @@ -682,7 +761,7 @@ func TestFramePublished(t *testing.T) { require.Equal(t, uint64(1000), cb.timeout) } -func TestChannelBuilder_PendingFrames_TotalFrames(t *testing.T) { +func ChannelBuilder_PendingFrames_TotalFrames(t *testing.T, batchType uint) { const tnf = 8 rng := rand.New(rand.NewSource(94572314)) require := require.New(t) @@ -691,7 +770,8 @@ func TestChannelBuilder_PendingFrames_TotalFrames(t *testing.T) { cfg.MaxFrameSize = 1000 cfg.CompressorConfig.TargetNumFrames = tnf cfg.CompressorConfig.Kind = "shadow" - cb, err := newChannelBuilder(cfg) + cfg.BatchType = batchType + cb, err := newChannelBuilder(cfg, getSpanBatchBuilder(batchType)) require.NoError(err) // initial builder should be empty @@ -725,25 +805,40 @@ func TestChannelBuilder_PendingFrames_TotalFrames(t *testing.T) { } } -func TestChannelBuilder_InputBytes(t *testing.T) { +func ChannelBuilder_InputBytes(t *testing.T, batchType uint) { require := require.New(t) rng := rand.New(rand.NewSource(4982432)) - cb, _ := defaultChannelBuilderSetup(t) + cfg := defaultTestChannelConfig + cfg.BatchType = batchType + spanBatchBuilder := getSpanBatchBuilder(batchType) + cb, err := newChannelBuilder(cfg, getSpanBatchBuilder(batchType)) + require.NoError(err) require.Zero(cb.InputBytes()) var l int for i := 0; i < 5; i++ { block := newMiniL2Block(rng.Intn(32)) - l += blockBatchRlpSize(t, block) - + if batchType == derive.SingularBatchType { + l += blockBatchRlpSize(t, block) + } else { + singularBatch, _, err := derive.BlockToSingularBatch(block) + require.NoError(err) + spanBatchBuilder.AppendSingularBatch(singularBatch) + rawSpanBatch, err := spanBatchBuilder.GetRawSpanBatch() + require.NoError(err) + batch := derive.NewSpanBatchData(*rawSpanBatch) + var buf bytes.Buffer + require.NoError(batch.EncodeRLP(&buf)) + l = buf.Len() + } _, err := cb.AddBlock(block) require.NoError(err) require.Equal(cb.InputBytes(), l) } } -func TestChannelBuilder_OutputBytes(t *testing.T) { +func ChannelBuilder_OutputBytes(t *testing.T, batchType uint) { require := require.New(t) rng := rand.New(rand.NewSource(9860372)) cfg := defaultTestChannelConfig @@ -751,7 +846,8 @@ func TestChannelBuilder_OutputBytes(t *testing.T) { cfg.MaxFrameSize = 1000 cfg.CompressorConfig.TargetNumFrames = 16 cfg.CompressorConfig.ApproxComprRatio = 1.0 - cb, err := newChannelBuilder(cfg) + cfg.BatchType = batchType + cb, err := newChannelBuilder(cfg, getSpanBatchBuilder(batchType)) require.NoError(err, "newChannelBuilder") require.Zero(cb.OutputBytes()) @@ -778,17 +874,10 @@ func TestChannelBuilder_OutputBytes(t *testing.T) { require.Equal(cb.OutputBytes(), flen) } -func defaultChannelBuilderSetup(t *testing.T) (*channelBuilder, ChannelConfig) { - t.Helper() - cfg := defaultTestChannelConfig - cb, err := newChannelBuilder(cfg) - require.NoError(t, err, "newChannelBuilder") - return cb, cfg -} - func blockBatchRlpSize(t *testing.T, b *types.Block) int { t.Helper() - batch, _, err := derive.BlockToBatch(b) + singularBatch, _, err := derive.BlockToSingularBatch(b) + batch := derive.NewSingularBatchData(*singularBatch) require.NoError(t, err) var buf bytes.Buffer require.NoError(t, batch.EncodeRLP(&buf), "RLP-encoding batch") diff --git a/op-batcher/batcher/channel_manager.go b/op-batcher/batcher/channel_manager.go index 052b9dbe1e3c..dc00fab364e9 100644 --- a/op-batcher/batcher/channel_manager.go +++ b/op-batcher/batcher/channel_manager.go @@ -7,6 +7,7 @@ import ( "sync" "github.com/ethereum-optimism/optimism/op-batcher/metrics" + "github.com/ethereum-optimism/optimism/op-node/rollup" "github.com/ethereum-optimism/optimism/op-node/rollup/derive" "github.com/ethereum-optimism/optimism/op-service/eth" "github.com/ethereum/go-ethereum/common" @@ -28,12 +29,16 @@ type channelManager struct { log log.Logger metr metrics.Metricer cfg ChannelConfig + rcfg *rollup.Config // All blocks since the last request for new tx data. blocks []*types.Block // last block hash - for reorg detection tip common.Hash + // last block added to channel. nil at first. + lastProcessedBlock *eth.L2BlockRef + // channel to write new block data to currentChannel *channel // channels to read frame data from, for writing batches onchain @@ -45,18 +50,21 @@ type channelManager struct { closed bool } -func NewChannelManager(log log.Logger, metr metrics.Metricer, cfg ChannelConfig) *channelManager { +func NewChannelManager(log log.Logger, metr metrics.Metricer, cfg ChannelConfig, rcfg *rollup.Config) *channelManager { return &channelManager{ - log: log, - metr: metr, - cfg: cfg, - txChannels: make(map[txID]*channel), + log: log, + metr: metr, + cfg: cfg, + rcfg: rcfg, + txChannels: make(map[txID]*channel), + lastProcessedBlock: nil, } } // Clear clears the entire state of the channel manager. -// It is intended to be used after an L2 reorg. -func (s *channelManager) Clear() { +// It is intended to be used before launching op-batcher and after an L2 reorg. +// Must set lastProcessedBlock as current L2 safe head fetched from L2 node. +func (s *channelManager) Clear(safeHead *eth.L2BlockRef) { s.mu.Lock() defer s.mu.Unlock() s.log.Trace("clearing channel manager state") @@ -66,6 +74,7 @@ func (s *channelManager) Clear() { s.currentChannel = nil s.channelQueue = nil s.txChannels = make(map[txID]*channel) + s.lastProcessedBlock = safeHead } // TxFailed records a transaction as failed. It will attempt to resubmit the data @@ -195,7 +204,19 @@ func (s *channelManager) ensureChannelWithSpace(l1Head eth.BlockID) error { return nil } - pc, err := newChannel(s.log, s.metr, s.cfg) + var spanBatchBuilder *derive.SpanBatchBuilder + if s.cfg.BatchType == derive.SpanBatchType { + if s.lastProcessedBlock == nil { + // TODO: we can remove "lastProcessedBlock" if we change the data-builder + // to append a singular-batch *with* the L2 metadata such as the L1-block-info seq-number; + // this helps determine whether or not the L1 origin changed in the first block of the span, + // without having to remember the last block from before the span. + return errors.New("last block is not initialized") + } + // Pass the current lastProcessedBlock as the parent + spanBatchBuilder = derive.NewSpanBatchBuilder(s.lastProcessedBlock.L1Origin.Number, s.rcfg.Genesis.L2Time, s.rcfg.L2ChainID) + } + pc, err := newChannel(s.log, s.metr, s.cfg, spanBatchBuilder) if err != nil { return fmt.Errorf("creating new channel: %w", err) } @@ -241,6 +262,7 @@ func (s *channelManager) processBlocks() error { blocksAdded += 1 latestL2ref = l2BlockRefFromBlockAndL1Info(block, l1info) s.metr.RecordL2BlockInChannel(block) + s.lastProcessedBlock = &latestL2ref // current block got added but channel is now full if s.currentChannel.IsFull() { break diff --git a/op-batcher/batcher/channel_manager_test.go b/op-batcher/batcher/channel_manager_test.go index 4621dde1bdb9..b68d13e4cc8b 100644 --- a/op-batcher/batcher/channel_manager_test.go +++ b/op-batcher/batcher/channel_manager_test.go @@ -9,21 +9,53 @@ import ( "github.com/ethereum-optimism/optimism/op-batcher/compressor" "github.com/ethereum-optimism/optimism/op-batcher/metrics" + "github.com/ethereum-optimism/optimism/op-node/rollup" "github.com/ethereum-optimism/optimism/op-node/rollup/derive" derivetest "github.com/ethereum-optimism/optimism/op-node/rollup/derive/test" "github.com/ethereum-optimism/optimism/op-service/eth" "github.com/ethereum-optimism/optimism/op-service/testlog" + "github.com/ethereum-optimism/optimism/op-service/testutils" "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/log" "github.com/stretchr/testify/require" ) -// TestChannelManagerReturnsErrReorg ensures that the channel manager +func TestChannelManagerBatchType(t *testing.T) { + tests := []struct { + name string + f func(t *testing.T, batchType uint) + }{ + {"ChannelManagerReturnsErrReorg", ChannelManagerReturnsErrReorg}, + {"ChannelManagerReturnsErrReorgWhenDrained", ChannelManagerReturnsErrReorgWhenDrained}, + {"ChannelManager_Clear", ChannelManager_Clear}, + {"ChannelManager_TxResend", ChannelManager_TxResend}, + {"ChannelManagerCloseBeforeFirstUse", ChannelManagerCloseBeforeFirstUse}, + {"ChannelManagerCloseNoPendingChannel", ChannelManagerCloseNoPendingChannel}, + {"ChannelManagerClosePendingChannel", ChannelManagerClosePendingChannel}, + {"ChannelManagerCloseAllTxsFailed", ChannelManagerCloseAllTxsFailed}, + } + for _, test := range tests { + test := test + t.Run(test.name+"_SingularBatch", func(t *testing.T) { + test.f(t, derive.SingularBatchType) + }) + } + + for _, test := range tests { + test := test + t.Run(test.name+"_SpanBatch", func(t *testing.T) { + test.f(t, derive.SpanBatchType) + }) + } +} + +// ChannelManagerReturnsErrReorg ensures that the channel manager // detects a reorg when it has cached L1 blocks. -func TestChannelManagerReturnsErrReorg(t *testing.T) { +func ChannelManagerReturnsErrReorg(t *testing.T, batchType uint) { log := testlog.Logger(t, log.LvlCrit) - m := NewChannelManager(log, metrics.NoopMetrics, ChannelConfig{}) + m := NewChannelManager(log, metrics.NoopMetrics, ChannelConfig{BatchType: batchType}, &rollup.Config{}) + m.Clear(ð.L2BlockRef{}) a := types.NewBlock(&types.Header{ Number: big.NewInt(0), @@ -49,9 +81,9 @@ func TestChannelManagerReturnsErrReorg(t *testing.T) { require.Equal(t, []*types.Block{a, b, c}, m.blocks) } -// TestChannelManagerReturnsErrReorgWhenDrained ensures that the channel manager +// ChannelManagerReturnsErrReorgWhenDrained ensures that the channel manager // detects a reorg even if it does not have any blocks inside it. -func TestChannelManagerReturnsErrReorgWhenDrained(t *testing.T) { +func ChannelManagerReturnsErrReorgWhenDrained(t *testing.T, batchType uint) { log := testlog.Logger(t, log.LvlCrit) m := NewChannelManager(log, metrics.NoopMetrics, ChannelConfig{ @@ -61,7 +93,11 @@ func TestChannelManagerReturnsErrReorgWhenDrained(t *testing.T) { TargetNumFrames: 1, ApproxComprRatio: 1.0, }, - }) + BatchType: batchType, + }, + &rollup.Config{}, + ) + m.Clear(ð.L2BlockRef{}) a := newMiniL2Block(0) x := newMiniL2BlockWithNumberParent(0, big.NewInt(1), common.Hash{0xff}) @@ -76,8 +112,8 @@ func TestChannelManagerReturnsErrReorgWhenDrained(t *testing.T) { require.ErrorIs(t, m.AddL2Block(x), ErrReorg) } -// TestChannelManager_Clear tests clearing the channel manager. -func TestChannelManager_Clear(t *testing.T) { +// ChannelManager_Clear tests clearing the channel manager. +func ChannelManager_Clear(t *testing.T, batchType uint) { require := require.New(t) // Create a channel manager @@ -96,7 +132,10 @@ func TestChannelManager_Clear(t *testing.T) { TargetNumFrames: 1, ApproxComprRatio: 1.0, }, - }) + BatchType: batchType, + }, + &rollup.Config{}, + ) // Channel Manager state should be empty by default require.Empty(m.blocks) @@ -104,6 +143,9 @@ func TestChannelManager_Clear(t *testing.T) { require.Nil(m.currentChannel) require.Empty(m.channelQueue) require.Empty(m.txChannels) + require.Nil(m.lastProcessedBlock) + // Set the last block + m.Clear(ð.L2BlockRef{}) // Add a block to the channel manager a, _ := derivetest.RandomL2Block(rng, 4) @@ -143,7 +185,8 @@ func TestChannelManager_Clear(t *testing.T) { require.Equal(b.Hash(), m.tip) // Clear the channel manager - m.Clear() + safeHead := testutils.RandomL2BlockRef(rng) + m.Clear(&safeHead) // Check that the entire channel manager state cleared require.Empty(m.blocks) @@ -151,9 +194,10 @@ func TestChannelManager_Clear(t *testing.T) { require.Nil(m.currentChannel) require.Empty(m.channelQueue) require.Empty(m.txChannels) + require.Equal(m.lastProcessedBlock, &safeHead) } -func TestChannelManager_TxResend(t *testing.T) { +func ChannelManager_TxResend(t *testing.T, batchType uint) { require := require.New(t) rng := rand.New(rand.NewSource(time.Now().UnixNano())) log := testlog.Logger(t, log.LvlError) @@ -165,7 +209,11 @@ func TestChannelManager_TxResend(t *testing.T) { TargetNumFrames: 1, ApproxComprRatio: 1.0, }, - }) + BatchType: batchType, + }, + &rollup.Config{}, + ) + m.Clear(ð.L2BlockRef{}) a, _ := derivetest.RandomL2Block(rng, 4) @@ -195,9 +243,9 @@ func TestChannelManager_TxResend(t *testing.T) { require.Len(fs, 1) } -// TestChannelManagerCloseBeforeFirstUse ensures that the channel manager +// ChannelManagerCloseBeforeFirstUse ensures that the channel manager // will not produce any frames if closed immediately. -func TestChannelManagerCloseBeforeFirstUse(t *testing.T) { +func ChannelManagerCloseBeforeFirstUse(t *testing.T, batchType uint) { require := require.New(t) rng := rand.New(rand.NewSource(time.Now().UnixNano())) log := testlog.Logger(t, log.LvlCrit) @@ -209,7 +257,11 @@ func TestChannelManagerCloseBeforeFirstUse(t *testing.T) { TargetFrameSize: 0, ApproxComprRatio: 1.0, }, - }) + BatchType: batchType, + }, + &rollup.Config{}, + ) + m.Clear(ð.L2BlockRef{}) a, _ := derivetest.RandomL2Block(rng, 4) @@ -222,10 +274,10 @@ func TestChannelManagerCloseBeforeFirstUse(t *testing.T) { require.ErrorIs(err, io.EOF, "Expected closed channel manager to contain no tx data") } -// TestChannelManagerCloseNoPendingChannel ensures that the channel manager +// ChannelManagerCloseNoPendingChannel ensures that the channel manager // can gracefully close with no pending channels, and will not emit any new // channel frames. -func TestChannelManagerCloseNoPendingChannel(t *testing.T) { +func ChannelManagerCloseNoPendingChannel(t *testing.T, batchType uint) { require := require.New(t) log := testlog.Logger(t, log.LvlCrit) m := NewChannelManager(log, metrics.NoopMetrics, @@ -237,7 +289,11 @@ func TestChannelManagerCloseNoPendingChannel(t *testing.T) { TargetNumFrames: 1, ApproxComprRatio: 1.0, }, - }) + BatchType: batchType, + }, + &rollup.Config{}, + ) + m.Clear(ð.L2BlockRef{}) a := newMiniL2Block(0) b := newMiniL2BlockWithNumberParent(0, big.NewInt(1), a.Hash()) @@ -261,10 +317,10 @@ func TestChannelManagerCloseNoPendingChannel(t *testing.T) { require.ErrorIs(err, io.EOF, "Expected closed channel manager to return no new tx data") } -// TestChannelManagerCloseNoPendingChannel ensures that the channel manager +// ChannelManagerCloseNoPendingChannel ensures that the channel manager // can gracefully close with a pending channel, and will not produce any // new channel frames after this point. -func TestChannelManagerClosePendingChannel(t *testing.T) { +func ChannelManagerClosePendingChannel(t *testing.T, batchType uint) { require := require.New(t) log := testlog.Logger(t, log.LvlCrit) m := NewChannelManager(log, metrics.NoopMetrics, @@ -272,13 +328,23 @@ func TestChannelManagerClosePendingChannel(t *testing.T) { MaxFrameSize: 1000, ChannelTimeout: 1000, CompressorConfig: compressor.Config{ - TargetNumFrames: 100, + TargetNumFrames: 1, TargetFrameSize: 1000, ApproxComprRatio: 1.0, }, - }) - - a := newMiniL2Block(50_000) + BatchType: batchType, + }, + &rollup.Config{}, + ) + m.Clear(ð.L2BlockRef{}) + + numTx := 50000 + if batchType == derive.SpanBatchType { + // Adjust number of txs to make 2 frames + // Encoding empty txs as span batch requires more data size because span batch encodes tx signature to fixed length + numTx = 20000 + } + a := newMiniL2Block(numTx) b := newMiniL2BlockWithNumberParent(10, big.NewInt(1), a.Hash()) err := m.AddL2Block(a) @@ -306,10 +372,10 @@ func TestChannelManagerClosePendingChannel(t *testing.T) { require.ErrorIs(err, io.EOF, "Expected closed channel manager to produce no more tx data") } -// TestChannelManagerCloseAllTxsFailed ensures that the channel manager +// ChannelManagerCloseAllTxsFailed ensures that the channel manager // can gracefully close after producing transaction frames if none of these // have successfully landed on chain. -func TestChannelManagerCloseAllTxsFailed(t *testing.T) { +func ChannelManagerCloseAllTxsFailed(t *testing.T, batchType uint) { require := require.New(t) log := testlog.Logger(t, log.LvlCrit) m := NewChannelManager(log, metrics.NoopMetrics, @@ -321,7 +387,10 @@ func TestChannelManagerCloseAllTxsFailed(t *testing.T) { TargetFrameSize: 1000, ApproxComprRatio: 1.0, }, - }) + BatchType: batchType, + }, &rollup.Config{}, + ) + m.Clear(ð.L2BlockRef{}) a := newMiniL2Block(50_000) diff --git a/op-batcher/batcher/channel_test.go b/op-batcher/batcher/channel_test.go index 0c5ce3b83bab..26f64204bc73 100644 --- a/op-batcher/batcher/channel_test.go +++ b/op-batcher/batcher/channel_test.go @@ -5,6 +5,7 @@ import ( "testing" "github.com/ethereum-optimism/optimism/op-batcher/metrics" + "github.com/ethereum-optimism/optimism/op-node/rollup" "github.com/ethereum-optimism/optimism/op-node/rollup/derive" "github.com/ethereum-optimism/optimism/op-service/eth" "github.com/ethereum-optimism/optimism/op-service/testlog" @@ -20,7 +21,8 @@ func TestChannelTimeout(t *testing.T) { log := testlog.Logger(t, log.LvlCrit) m := NewChannelManager(log, metrics.NoopMetrics, ChannelConfig{ ChannelTimeout: 100, - }) + }, &rollup.Config{}) + m.Clear(ð.L2BlockRef{}) // Pending channel is nil so is cannot be timed out require.Nil(t, m.currentChannel) @@ -61,7 +63,8 @@ func TestChannelTimeout(t *testing.T) { // TestChannelNextTxData checks the nextTxData function. func TestChannelNextTxData(t *testing.T) { log := testlog.Logger(t, log.LvlCrit) - m := NewChannelManager(log, metrics.NoopMetrics, ChannelConfig{}) + m := NewChannelManager(log, metrics.NoopMetrics, ChannelConfig{}, &rollup.Config{}) + m.Clear(ð.L2BlockRef{}) // Nil pending channel should return EOF returnedTxData, err := m.nextTxData(nil) @@ -109,7 +112,8 @@ func TestChannelTxConfirmed(t *testing.T) { // channels on confirmation. This would result in [TxConfirmed] // clearing confirmed transactions, and reseting the pendingChannels map ChannelTimeout: 10, - }) + }, &rollup.Config{}) + m.Clear(ð.L2BlockRef{}) // Let's add a valid pending transaction to the channel manager // So we can demonstrate that TxConfirmed's correctness @@ -157,7 +161,8 @@ func TestChannelTxConfirmed(t *testing.T) { func TestChannelTxFailed(t *testing.T) { // Create a channel manager log := testlog.Logger(t, log.LvlCrit) - m := NewChannelManager(log, metrics.NoopMetrics, ChannelConfig{}) + m := NewChannelManager(log, metrics.NoopMetrics, ChannelConfig{}, &rollup.Config{}) + m.Clear(ð.L2BlockRef{}) // Let's add a valid pending transaction to the channel // manager so we can demonstrate correctness diff --git a/op-batcher/batcher/config.go b/op-batcher/batcher/config.go index ad7b611c9a68..d4f34461f02d 100644 --- a/op-batcher/batcher/config.go +++ b/op-batcher/batcher/config.go @@ -52,6 +52,8 @@ type CLIConfig struct { Stopped bool + BatchType uint + TxMgrConfig txmgr.CLIConfig LogConfig oplog.CLIConfig MetricsConfig opmetrics.CLIConfig @@ -93,6 +95,7 @@ func NewConfig(ctx *cli.Context) *CLIConfig { MaxChannelDuration: ctx.Uint64(flags.MaxChannelDurationFlag.Name), MaxL1TxSize: ctx.Uint64(flags.MaxL1TxSizeBytesFlag.Name), Stopped: ctx.Bool(flags.StoppedFlag.Name), + BatchType: ctx.Uint(flags.BatchTypeFlag.Name), TxMgrConfig: txmgr.ReadCLIConfig(ctx), LogConfig: oplog.ReadCLIConfig(ctx), MetricsConfig: opmetrics.ReadCLIConfig(ctx), diff --git a/op-batcher/batcher/driver.go b/op-batcher/batcher/driver.go index bcffef6a9edf..d45ae3153780 100644 --- a/op-batcher/batcher/driver.go +++ b/op-batcher/batcher/driver.go @@ -4,6 +4,7 @@ import ( "context" "errors" "fmt" + "github.com/ethereum-optimism/optimism/op-node/rollup/derive" "io" "math/big" _ "net/http/pprof" @@ -16,7 +17,6 @@ import ( "github.com/ethereum-optimism/optimism/op-batcher/metrics" "github.com/ethereum-optimism/optimism/op-node/rollup" - "github.com/ethereum-optimism/optimism/op-node/rollup/derive" "github.com/ethereum-optimism/optimism/op-service/eth" "github.com/ethereum-optimism/optimism/op-service/txmgr" ) @@ -74,7 +74,7 @@ type BatchSubmitter struct { func NewBatchSubmitter(setup DriverSetup) *BatchSubmitter { return &BatchSubmitter{ DriverSetup: setup, - state: NewChannelManager(setup.Log, setup.Metr, setup.Channel), + state: NewChannelManager(setup.Log, setup.Metr, setup.Channel, setup.RollupCfg), } } @@ -91,7 +91,11 @@ func (l *BatchSubmitter) StartBatchSubmitting() error { l.shutdownCtx, l.cancelShutdownCtx = context.WithCancel(context.Background()) l.killCtx, l.cancelKillCtx = context.WithCancel(context.Background()) - l.state.Clear() + syncStatus, err := fetchSyncStatus(l.shutdownCtx, l.RollupClient, l.Cfg.NetworkTimeout) + if err != nil { + return err + } + l.state.Clear(&syncStatus.SafeL2) l.lastStoredBlock = eth.BlockID{} l.wg.Add(1) @@ -201,15 +205,9 @@ func (l *BatchSubmitter) loadBlockIntoState(ctx context.Context, blockNumber uin // calculateL2BlockRangeToStore determines the range (start,end] that should be loaded into the local state. // It also takes care of initializing some local state (i.e. will modify l.lastStoredBlock in certain conditions) func (l *BatchSubmitter) calculateL2BlockRangeToStore(ctx context.Context) (eth.BlockID, eth.BlockID, error) { - ctx, cancel := context.WithTimeout(ctx, l.Cfg.NetworkTimeout) - defer cancel() - syncStatus, err := l.RollupClient.SyncStatus(ctx) - // Ensure that we have the sync status + syncStatus, err := fetchSyncStatus(ctx, l.RollupClient, l.Cfg.NetworkTimeout) if err != nil { - return eth.BlockID{}, eth.BlockID{}, fmt.Errorf("failed to get sync status: %w", err) - } - if syncStatus.HeadL1 == (eth.L1BlockRef{}) { - return eth.BlockID{}, eth.BlockID{}, errors.New("empty sync status") + return eth.BlockID{}, eth.BlockID{}, err } // Check last stored to see if it needs to be set on startup OR set if is lagged behind. @@ -259,7 +257,12 @@ func (l *BatchSubmitter) loop() { l.Log.Error("error closing the channel manager to handle a L2 reorg", "err", err) } l.publishStateToL1(queue, receiptsCh, true) - l.state.Clear() + if syncStatus, err := fetchSyncStatus(l.shutdownCtx, l.RollupClient, l.Cfg.NetworkTimeout); err == nil { + l.state.Clear(&syncStatus.SafeL2) + } else { + // if fetchSyncStatus failed, ErrReorg will be returned again + l.Log.Error("error fetching sync status from L2 node", "err", err) + } continue } l.publishStateToL1(queue, receiptsCh, false) @@ -395,3 +398,17 @@ func (l *BatchSubmitter) l1Tip(ctx context.Context) (eth.L1BlockRef, error) { } return eth.InfoToL1BlockRef(eth.HeaderBlockInfo(head)), nil } + +func fetchSyncStatus(ctx context.Context, rollupNode RollupClient, timeout time.Duration) (*eth.SyncStatus, error) { + ctx, cancel := context.WithTimeout(ctx, timeout) + defer cancel() + syncStatus, err := rollupNode.SyncStatus(ctx) + // Ensure that we have the sync status + if err != nil { + return ð.SyncStatus{}, fmt.Errorf("failed to get sync status: %w", err) + } + if syncStatus.SafeL2 == (eth.L2BlockRef{}) { + return ð.SyncStatus{}, errors.New("empty sync status") + } + return syncStatus, nil +} diff --git a/op-batcher/batcher/service.go b/op-batcher/batcher/service.go index 22e8dd0e2fb8..1c9a493d0c7e 100644 --- a/op-batcher/batcher/service.go +++ b/op-batcher/batcher/service.go @@ -173,6 +173,7 @@ func (bs *BatcherService) initChannelConfig(cfg *CLIConfig) error { SubSafetyMargin: cfg.SubSafetyMargin, MaxFrameSize: cfg.MaxL1TxSize - 1, // subtract 1 byte for version CompressorConfig: cfg.CompressorConfig.Config(), + BatchType: cfg.BatchType, } if err := bs.Channel.Check(); err != nil { return fmt.Errorf("invalid channel configuration: %w", err) diff --git a/op-batcher/flags/flags.go b/op-batcher/flags/flags.go index 9b854f8d6bbb..5e752d9d2748 100644 --- a/op-batcher/flags/flags.go +++ b/op-batcher/flags/flags.go @@ -76,6 +76,12 @@ var ( Usage: "Initialize the batcher in a stopped state. The batcher can be started using the admin_startBatcher RPC", EnvVars: prefixEnvVars("STOPPED"), } + BatchTypeFlag = &cli.UintFlag{ + Name: "batch-type", + Usage: "The batch type. 0 for SingularBatch and 1 for SpanBatch.", + Value: 0, + EnvVars: prefixEnvVars("BATCH_TYPE"), + } // Legacy Flags SequencerHDPathFlag = txmgr.SequencerHDPathFlag ) @@ -94,6 +100,7 @@ var optionalFlags = []cli.Flag{ MaxL1TxSizeBytesFlag, StoppedFlag, SequencerHDPathFlag, + BatchTypeFlag, } func init() { diff --git a/op-e2e/actions/l2_batcher.go b/op-e2e/actions/l2_batcher.go index 57d5864571f0..c1b82cbae14a 100644 --- a/op-e2e/actions/l2_batcher.go +++ b/op-e2e/actions/l2_batcher.go @@ -140,7 +140,7 @@ func (s *L2Batcher) Buffer(t Testing) error { ApproxComprRatio: 1, }) require.NoError(t, e, "failed to create compressor") - ch, err = derive.NewChannelOut(c) + ch, err = derive.NewChannelOut(c, derive.SingularBatchType, nil) } require.NoError(t, err, "failed to create channel") s.l2ChannelOut = ch diff --git a/op-e2e/setup.go b/op-e2e/setup.go index fe70a263985e..40db06f94391 100644 --- a/op-e2e/setup.go +++ b/op-e2e/setup.go @@ -49,6 +49,7 @@ import ( "github.com/ethereum-optimism/optimism/op-node/p2p" "github.com/ethereum-optimism/optimism/op-node/p2p/store" "github.com/ethereum-optimism/optimism/op-node/rollup" + "github.com/ethereum-optimism/optimism/op-node/rollup/derive" "github.com/ethereum-optimism/optimism/op-node/rollup/driver" proposermetrics "github.com/ethereum-optimism/optimism/op-proposer/metrics" l2os "github.com/ethereum-optimism/optimism/op-proposer/proposer" @@ -679,6 +680,10 @@ func (cfg SystemConfig) Start(t *testing.T, _opts ...SystemConfigOption) (*Syste return nil, fmt.Errorf("unable to start l2 output submitter: %w", err) } + batchType := derive.SingularBatchType + if os.Getenv("OP_E2E_USE_SPAN_BATCH") == "true" { + batchType = derive.SpanBatchType + } batcherCLIConfig := &bss.CLIConfig{ L1EthRpc: sys.EthInstances["l1"].WSEndpoint(), L2EthRpc: sys.EthInstances["sequencer"].WSEndpoint(), @@ -698,7 +703,8 @@ func (cfg SystemConfig) Start(t *testing.T, _opts ...SystemConfigOption) (*Syste Level: log.LvlInfo, Format: oplog.FormatText, }, - Stopped: sys.cfg.DisableBatcher, // Batch submitter may be enabled later + Stopped: sys.cfg.DisableBatcher, // Batch submitter may be enabled later + BatchType: uint(batchType), } // Batch Submitter batcher, err := bss.BatcherServiceFromCLIConfig(context.Background(), "0.0.1", batcherCLIConfig, sys.cfg.Loggers["batcher"]) diff --git a/op-node/rollup/derive/channel_out.go b/op-node/rollup/derive/channel_out.go index 63e7cc38bc62..b0e36d6091d7 100644 --- a/op-node/rollup/derive/channel_out.go +++ b/op-node/rollup/derive/channel_out.go @@ -48,6 +48,13 @@ type Compressor interface { FullErr() error } +type ChannelOutReader interface { + io.Writer + io.Reader + Reset() + Len() int +} + type ChannelOut struct { id ChannelID // Frame ID of the next frame to emit. Increment after emitting @@ -57,20 +64,35 @@ type ChannelOut struct { // Compressor stage. Write input data to it compress Compressor - + // closed indicates if the channel is closed closed bool + // batchType indicates whether this channel uses SingularBatch or SpanBatch + batchType uint + // spanBatchBuilder contains information requires to build SpanBatch + spanBatchBuilder *SpanBatchBuilder + // reader contains compressed data for making output frames + reader ChannelOutReader } func (co *ChannelOut) ID() ChannelID { return co.id } -func NewChannelOut(compress Compressor) (*ChannelOut, error) { +func NewChannelOut(compress Compressor, batchType uint, spanBatchBuilder *SpanBatchBuilder) (*ChannelOut, error) { + // If the channel uses SingularBatch, use compressor directly as its reader + var reader ChannelOutReader = compress + if batchType == SpanBatchType { + // If the channel uses SpanBatch, create empty buffer for reader + reader = &bytes.Buffer{} + } c := &ChannelOut{ - id: ChannelID{}, // TODO: use GUID here instead of fully random data - frame: 0, - rlpLength: 0, - compress: compress, + id: ChannelID{}, // TODO: use GUID here instead of fully random data + frame: 0, + rlpLength: 0, + compress: compress, + batchType: batchType, + spanBatchBuilder: spanBatchBuilder, + reader: reader, } _, err := rand.Read(c.id[:]) if err != nil { @@ -85,7 +107,11 @@ func (co *ChannelOut) Reset() error { co.frame = 0 co.rlpLength = 0 co.compress.Reset() + co.reader.Reset() co.closed = false + if co.spanBatchBuilder != nil { + co.spanBatchBuilder.Reset() + } _, err := rand.Read(co.id[:]) return err } @@ -99,30 +125,41 @@ func (co *ChannelOut) AddBlock(block *types.Block) (uint64, error) { return 0, errors.New("already closed") } - batch, _, err := BlockToBatch(block) + batch, _, err := BlockToSingularBatch(block) if err != nil { return 0, err } - return co.AddBatch(batch) + return co.AddSingularBatch(batch) } -// AddBatch adds a batch to the channel. It returns the RLP encoded byte size +// AddSingularBatch adds a batch to the channel. It returns the RLP encoded byte size // and an error if there is a problem adding the batch. The only sentinel error // that it returns is ErrTooManyRLPBytes. If this error is returned, the channel // should be closed and a new one should be made. // -// AddBatch should be used together with BlockToBatch if you need to access the +// AddSingularBatch should be used together with BlockToSingularBatch if you need to access the // BatchData before adding a block to the channel. It isn't possible to access // the batch data with AddBlock. -func (co *ChannelOut) AddBatch(batch *BatchData) (uint64, error) { +func (co *ChannelOut) AddSingularBatch(batch *SingularBatch) (uint64, error) { if co.closed { return 0, errors.New("already closed") } + switch co.batchType { + case SingularBatchType: + return co.writeSingularBatch(batch) + case SpanBatchType: + return co.writeSpanBatch(batch) + default: + return 0, fmt.Errorf("unrecognized batch type: %d", co.batchType) + } +} + +func (co *ChannelOut) writeSingularBatch(batch *SingularBatch) (uint64, error) { + var buf bytes.Buffer // We encode to a temporary buffer to determine the encoded length to // ensure that the total size of all RLP elements is less than or equal to MAX_RLP_BYTES_PER_CHANNEL - var buf bytes.Buffer - if err := rlp.Encode(&buf, batch); err != nil { + if err := rlp.Encode(&buf, NewSingularBatchData(*batch)); err != nil { return 0, err } if co.rlpLength+buf.Len() > MaxRLPBytesPerChannel { @@ -136,6 +173,70 @@ func (co *ChannelOut) AddBatch(batch *BatchData) (uint64, error) { return uint64(written), err } +// writeSpanBatch appends a SingularBatch to the channel's SpanBatch. +// A channel can have only one SpanBatch. And compressed results should not be accessible until the channel is closed, since the prefix and payload can be changed. +// So it resets channel contents and rewrites the entire SpanBatch each time, and compressed results are copied to reader after the channel is closed. +// It makes we can only get frames once the channel is full or closed, in the case of SpanBatch. +func (co *ChannelOut) writeSpanBatch(batch *SingularBatch) (uint64, error) { + if co.FullErr() != nil { + // channel is already full + return 0, co.FullErr() + } + var buf bytes.Buffer + // Append Singular batch to its span batch builder + co.spanBatchBuilder.AppendSingularBatch(batch) + // Convert Span batch to RawSpanBatch + rawSpanBatch, err := co.spanBatchBuilder.GetRawSpanBatch() + if err != nil { + return 0, fmt.Errorf("failed to convert SpanBatch into RawSpanBatch: %w", err) + } + // Encode RawSpanBatch into bytes + if err = rlp.Encode(&buf, NewSpanBatchData(*rawSpanBatch)); err != nil { + return 0, fmt.Errorf("failed to encode RawSpanBatch into bytes: %w", err) + } + co.rlpLength = 0 + // Ensure that the total size of all RLP elements is less than or equal to MAX_RLP_BYTES_PER_CHANNEL + if co.rlpLength+buf.Len() > MaxRLPBytesPerChannel { + return 0, fmt.Errorf("could not add %d bytes to channel of %d bytes, max is %d. err: %w", + buf.Len(), co.rlpLength, MaxRLPBytesPerChannel, ErrTooManyRLPBytes) + } + co.rlpLength = buf.Len() + + if co.spanBatchBuilder.GetBlockCount() > 1 { + // Flush compressed data into reader to preserve current result. + // If the channel is full after this block is appended, we should use preserved data. + if err := co.compress.Flush(); err != nil { + return 0, fmt.Errorf("failed to flush compressor: %w", err) + } + _, err = io.Copy(co.reader, co.compress) + if err != nil { + // Must reset reader to avoid partial output + co.reader.Reset() + return 0, fmt.Errorf("failed to copy compressed data to reader: %w", err) + } + } + + // Reset compressor to rewrite the entire span batch + co.compress.Reset() + // Avoid using io.Copy here, because we need all or nothing + written, err := co.compress.Write(buf.Bytes()) + if co.compress.FullErr() != nil { + err = co.compress.FullErr() + if co.spanBatchBuilder.GetBlockCount() == 1 { + // Do not return CompressorFullErr for the first block in the batch + // In this case, reader must be empty. then the contents of compressor will be copied to reader when the channel is closed. + err = nil + } + // If there are more than one blocks in the channel, reader should have data that preserves previous compression result before adding this block. + // So, as a result, this block is not added to the channel and the channel will be closed. + return uint64(written), err + } + + // If compressor is not full yet, reader must be reset to avoid submitting invalid frames + co.reader.Reset() + return uint64(written), err +} + // InputBytes returns the total amount of RLP-encoded input bytes. func (co *ChannelOut) InputBytes() int { return co.rlpLength @@ -145,13 +246,24 @@ func (co *ChannelOut) InputBytes() int { // Use `Flush` or `Close` to move data from the compression buffer into the ready buffer if more bytes // are needed. Add blocks may add to the ready buffer, but it is not guaranteed due to the compression stage. func (co *ChannelOut) ReadyBytes() int { - return co.compress.Len() + return co.reader.Len() } // Flush flushes the internal compression stage to the ready buffer. It enables pulling a larger & more // complete frame. It reduces the compression efficiency. func (co *ChannelOut) Flush() error { - return co.compress.Flush() + if err := co.compress.Flush(); err != nil { + return err + } + if co.batchType == SpanBatchType && co.closed && co.ReadyBytes() == 0 && co.compress.Len() > 0 { + _, err := io.Copy(co.reader, co.compress) + if err != nil { + // Must reset reader to avoid partial output + co.reader.Reset() + return fmt.Errorf("failed to flush compressed data to reader: %w", err) + } + } + return nil } func (co *ChannelOut) FullErr() error { @@ -163,6 +275,11 @@ func (co *ChannelOut) Close() error { return errors.New("already closed") } co.closed = true + if co.batchType == SpanBatchType { + if err := co.Flush(); err != nil { + return err + } + } return co.compress.Close() } @@ -186,8 +303,8 @@ func (co *ChannelOut) OutputFrame(w *bytes.Buffer, maxSize uint64) (uint16, erro // Copy data from the local buffer into the frame data buffer maxDataSize := maxSize - FrameV0OverHeadSize - if maxDataSize > uint64(co.compress.Len()) { - maxDataSize = uint64(co.compress.Len()) + if maxDataSize > uint64(co.ReadyBytes()) { + maxDataSize = uint64(co.ReadyBytes()) // If we are closed & will not spill past the current frame // mark it is the final frame of the channel. if co.closed { @@ -196,7 +313,7 @@ func (co *ChannelOut) OutputFrame(w *bytes.Buffer, maxSize uint64) (uint16, erro } f.Data = make([]byte, maxDataSize) - if _, err := io.ReadFull(co.compress, f.Data); err != nil { + if _, err := io.ReadFull(co.reader, f.Data); err != nil { return 0, err } @@ -213,8 +330,8 @@ func (co *ChannelOut) OutputFrame(w *bytes.Buffer, maxSize uint64) (uint16, erro } } -// BlockToBatch transforms a block into a batch object that can easily be RLP encoded. -func BlockToBatch(block *types.Block) (*BatchData, L1BlockInfo, error) { +// BlockToSingularBatch transforms a block into a batch object that can easily be RLP encoded. +func BlockToSingularBatch(block *types.Block) (*SingularBatch, L1BlockInfo, error) { opaqueTxs := make([]hexutil.Bytes, 0, len(block.Transactions())) for i, tx := range block.Transactions() { if tx.Type() == types.DepositTxType { @@ -238,15 +355,13 @@ func BlockToBatch(block *types.Block) (*BatchData, L1BlockInfo, error) { return nil, l1Info, fmt.Errorf("could not parse the L1 Info deposit: %w", err) } - return NewSingularBatchData( - SingularBatch{ - ParentHash: block.ParentHash(), - EpochNum: rollup.Epoch(l1Info.Number), - EpochHash: l1Info.BlockHash, - Timestamp: block.Time(), - Transactions: opaqueTxs, - }, - ), l1Info, nil + return &SingularBatch{ + ParentHash: block.ParentHash(), + EpochNum: rollup.Epoch(l1Info.Number), + EpochHash: l1Info.BlockHash, + Timestamp: block.Time(), + Transactions: opaqueTxs, + }, l1Info, nil } // ForceCloseTxData generates the transaction data for a transaction which will force close diff --git a/op-node/rollup/derive/channel_out_test.go b/op-node/rollup/derive/channel_out_test.go index 73f07d762884..0a01da76d0a3 100644 --- a/op-node/rollup/derive/channel_out_test.go +++ b/op-node/rollup/derive/channel_out_test.go @@ -29,7 +29,7 @@ func (s *nonCompressor) FullErr() error { } func TestChannelOutAddBlock(t *testing.T) { - cout, err := NewChannelOut(&nonCompressor{}) + cout, err := NewChannelOut(&nonCompressor{}, SingularBatchType, nil) require.NoError(t, err) t.Run("returns err if first tx is not an l1info tx", func(t *testing.T) { @@ -50,7 +50,7 @@ func TestChannelOutAddBlock(t *testing.T) { // max size that is below the fixed frame size overhead of 23, will return // an error. func TestOutputFrameSmallMaxSize(t *testing.T) { - cout, err := NewChannelOut(&nonCompressor{}) + cout, err := NewChannelOut(&nonCompressor{}, SingularBatchType, nil) require.NoError(t, err) // Call OutputFrame with the range of small max size values that err @@ -97,42 +97,42 @@ func TestForceCloseTxData(t *testing.T) { output: "", }, { - frames: []Frame{Frame{FrameNumber: 0, IsLast: false}, Frame{ID: id, FrameNumber: 1, IsLast: true}}, + frames: []Frame{{FrameNumber: 0, IsLast: false}, {ID: id, FrameNumber: 1, IsLast: true}}, errors: true, output: "", }, { - frames: []Frame{Frame{ID: id, FrameNumber: 0, IsLast: false}}, + frames: []Frame{{ID: id, FrameNumber: 0, IsLast: false}}, errors: false, output: "00deadbeefdeadbeefdeadbeefdeadbeef00000000000001", }, { - frames: []Frame{Frame{ID: id, FrameNumber: 0, IsLast: true}}, + frames: []Frame{{ID: id, FrameNumber: 0, IsLast: true}}, errors: false, output: "00", }, { - frames: []Frame{Frame{ID: id, FrameNumber: 1, IsLast: false}}, + frames: []Frame{{ID: id, FrameNumber: 1, IsLast: false}}, errors: false, output: "00deadbeefdeadbeefdeadbeefdeadbeef00000000000001", }, { - frames: []Frame{Frame{ID: id, FrameNumber: 1, IsLast: true}}, + frames: []Frame{{ID: id, FrameNumber: 1, IsLast: true}}, errors: false, output: "00deadbeefdeadbeefdeadbeefdeadbeef00000000000000", }, { - frames: []Frame{Frame{ID: id, FrameNumber: 2, IsLast: true}}, + frames: []Frame{{ID: id, FrameNumber: 2, IsLast: true}}, errors: false, output: "00deadbeefdeadbeefdeadbeefdeadbeef00000000000000deadbeefdeadbeefdeadbeefdeadbeef00010000000000", }, { - frames: []Frame{Frame{ID: id, FrameNumber: 1, IsLast: false}, Frame{ID: id, FrameNumber: 3, IsLast: true}}, + frames: []Frame{{ID: id, FrameNumber: 1, IsLast: false}, {ID: id, FrameNumber: 3, IsLast: true}}, errors: false, output: "00deadbeefdeadbeefdeadbeefdeadbeef00000000000000deadbeefdeadbeefdeadbeefdeadbeef00020000000000", }, { - frames: []Frame{Frame{ID: id, FrameNumber: 1, IsLast: false}, Frame{ID: id, FrameNumber: 3, IsLast: true}, Frame{ID: id, FrameNumber: 5, IsLast: true}}, + frames: []Frame{{ID: id, FrameNumber: 1, IsLast: false}, {ID: id, FrameNumber: 3, IsLast: true}, {ID: id, FrameNumber: 5, IsLast: true}}, errors: false, output: "00deadbeefdeadbeefdeadbeefdeadbeef00000000000000deadbeefdeadbeefdeadbeefdeadbeef00020000000000", }, @@ -152,6 +152,6 @@ func TestForceCloseTxData(t *testing.T) { func TestBlockToBatchValidity(t *testing.T) { block := new(types.Block) - _, _, err := BlockToBatch(block) + _, _, err := BlockToSingularBatch(block) require.ErrorContains(t, err, "has no transactions") } diff --git a/ops-bedrock/docker-compose.yml b/ops-bedrock/docker-compose.yml index 73d3a54a827f..4819d90aac2a 100644 --- a/ops-bedrock/docker-compose.yml +++ b/ops-bedrock/docker-compose.yml @@ -155,6 +155,7 @@ services: OP_BATCHER_PPROF_ENABLED: "true" OP_BATCHER_METRICS_ENABLED: "true" OP_BATCHER_RPC_ENABLE_ADMIN: "true" + OP_BATCHER_BATCH_TYPE: 0 artifact-server: depends_on: From 6191eea720aed8223e733721fe2fadd0bcf51bde Mon Sep 17 00:00:00 2001 From: Tei Im Date: Thu, 12 Oct 2023 18:50:34 +0900 Subject: [PATCH 063/279] Split ChannelOut to singular and span channel out --- op-batcher/batcher/channel_builder.go | 4 +- op-batcher/batcher/channel_builder_test.go | 2 +- op-e2e/actions/garbage_channel_out.go | 7 +- op-e2e/actions/l2_batcher.go | 2 +- op-node/rollup/derive/channel_out.go | 223 ++++++-------------- op-node/rollup/derive/channel_out_test.go | 4 +- op-node/rollup/derive/span_channel_out.go | 229 +++++++++++++++++++++ 7 files changed, 308 insertions(+), 163 deletions(-) create mode 100644 op-node/rollup/derive/span_channel_out.go diff --git a/op-batcher/batcher/channel_builder.go b/op-batcher/batcher/channel_builder.go index 33e236c2dd93..81bce734b522 100644 --- a/op-batcher/batcher/channel_builder.go +++ b/op-batcher/batcher/channel_builder.go @@ -121,7 +121,7 @@ type channelBuilder struct { // guaranteed to be a ChannelFullError wrapping the specific reason. fullErr error // current channel - co *derive.ChannelOut + co derive.ChannelOut // list of blocks in the channel. Saved in case the channel must be rebuilt blocks []*types.Block // frames data queue, to be send as txs @@ -139,7 +139,7 @@ func newChannelBuilder(cfg ChannelConfig, spanBatchBuilder *derive.SpanBatchBuil if err != nil { return nil, err } - co, err := derive.NewChannelOut(c, cfg.BatchType, spanBatchBuilder) + co, err := derive.NewChannelOut(cfg.BatchType, c, spanBatchBuilder) if err != nil { return nil, err } diff --git a/op-batcher/batcher/channel_builder_test.go b/op-batcher/batcher/channel_builder_test.go index 51c3d76af380..fa8512039d8a 100644 --- a/op-batcher/batcher/channel_builder_test.go +++ b/op-batcher/batcher/channel_builder_test.go @@ -449,7 +449,7 @@ func TestChannelBuilder_OutputWrongFramePanic(t *testing.T) { // to construct a single frame c, err := channelConfig.CompressorConfig.NewCompressor() require.NoError(t, err) - co, err := derive.NewChannelOut(c, derive.SingularBatchType, nil) + co, err := derive.NewChannelOut(derive.SingularBatchType, c, nil) require.NoError(t, err) var buf bytes.Buffer fn, err := co.OutputFrame(&buf, channelConfig.MaxFrameSize) diff --git a/op-e2e/actions/garbage_channel_out.go b/op-e2e/actions/garbage_channel_out.go index a2e5535b1074..d412f80d5566 100644 --- a/op-e2e/actions/garbage_channel_out.go +++ b/op-e2e/actions/garbage_channel_out.go @@ -61,8 +61,11 @@ type ChannelOutIface interface { OutputFrame(w *bytes.Buffer, maxSize uint64) (uint16, error) } -// Compile-time check for ChannelOutIface interface implementation for the ChannelOut type. -var _ ChannelOutIface = (*derive.ChannelOut)(nil) +// Compile-time check for ChannelOutIface interface implementation for the SingularChannelOut type. +var _ ChannelOutIface = (*derive.SingularChannelOut)(nil) + +// Compile-time check for ChannelOutIface interface implementation for the SpanChannelOut type. +var _ ChannelOutIface = (*derive.SpanChannelOut)(nil) // Compile-time check for ChannelOutIface interface implementation for the GarbageChannelOut type. var _ ChannelOutIface = (*GarbageChannelOut)(nil) diff --git a/op-e2e/actions/l2_batcher.go b/op-e2e/actions/l2_batcher.go index c1b82cbae14a..7431e7985bda 100644 --- a/op-e2e/actions/l2_batcher.go +++ b/op-e2e/actions/l2_batcher.go @@ -140,7 +140,7 @@ func (s *L2Batcher) Buffer(t Testing) error { ApproxComprRatio: 1, }) require.NoError(t, e, "failed to create compressor") - ch, err = derive.NewChannelOut(c, derive.SingularBatchType, nil) + ch, err = derive.NewChannelOut(derive.SingularBatchType, c, nil) } require.NoError(t, err, "failed to create channel") s.l2ChannelOut = ch diff --git a/op-node/rollup/derive/channel_out.go b/op-node/rollup/derive/channel_out.go index b0e36d6091d7..d2af4be76e28 100644 --- a/op-node/rollup/derive/channel_out.go +++ b/op-node/rollup/derive/channel_out.go @@ -48,14 +48,31 @@ type Compressor interface { FullErr() error } -type ChannelOutReader interface { - io.Writer - io.Reader - Reset() - Len() int +type ChannelOut interface { + ID() ChannelID + Reset() error + AddBlock(*types.Block) (uint64, error) + AddSingularBatch(*SingularBatch) (uint64, error) + InputBytes() int + ReadyBytes() int + Flush() error + FullErr() error + Close() error + OutputFrame(*bytes.Buffer, uint64) (uint16, error) +} + +func NewChannelOut(batchType uint, compress Compressor, spanBatchBuilder *SpanBatchBuilder) (ChannelOut, error) { + switch batchType { + case SingularBatchType: + return NewSingularChannelOut(compress) + case SpanBatchType: + return NewSpanChannelOut(compress, spanBatchBuilder) + default: + return nil, fmt.Errorf("unrecognized batch type: %d", batchType) + } } -type ChannelOut struct { +type SingularChannelOut struct { id ChannelID // Frame ID of the next frame to emit. Increment after emitting frame uint64 @@ -64,35 +81,20 @@ type ChannelOut struct { // Compressor stage. Write input data to it compress Compressor - // closed indicates if the channel is closed + closed bool - // batchType indicates whether this channel uses SingularBatch or SpanBatch - batchType uint - // spanBatchBuilder contains information requires to build SpanBatch - spanBatchBuilder *SpanBatchBuilder - // reader contains compressed data for making output frames - reader ChannelOutReader } -func (co *ChannelOut) ID() ChannelID { +func (co *SingularChannelOut) ID() ChannelID { return co.id } -func NewChannelOut(compress Compressor, batchType uint, spanBatchBuilder *SpanBatchBuilder) (*ChannelOut, error) { - // If the channel uses SingularBatch, use compressor directly as its reader - var reader ChannelOutReader = compress - if batchType == SpanBatchType { - // If the channel uses SpanBatch, create empty buffer for reader - reader = &bytes.Buffer{} - } - c := &ChannelOut{ - id: ChannelID{}, // TODO: use GUID here instead of fully random data - frame: 0, - rlpLength: 0, - compress: compress, - batchType: batchType, - spanBatchBuilder: spanBatchBuilder, - reader: reader, +func NewSingularChannelOut(compress Compressor) (*SingularChannelOut, error) { + c := &SingularChannelOut{ + id: ChannelID{}, // TODO: use GUID here instead of fully random data + frame: 0, + rlpLength: 0, + compress: compress, } _, err := rand.Read(c.id[:]) if err != nil { @@ -102,16 +104,12 @@ func NewChannelOut(compress Compressor, batchType uint, spanBatchBuilder *SpanBa return c, nil } -// TODO: reuse ChannelOut for performance -func (co *ChannelOut) Reset() error { +// TODO: reuse SingularChannelOut for performance +func (co *SingularChannelOut) Reset() error { co.frame = 0 co.rlpLength = 0 co.compress.Reset() - co.reader.Reset() co.closed = false - if co.spanBatchBuilder != nil { - co.spanBatchBuilder.Reset() - } _, err := rand.Read(co.id[:]) return err } @@ -120,7 +118,7 @@ func (co *ChannelOut) Reset() error { // and an error if there is a problem adding the block. The only sentinel error // that it returns is ErrTooManyRLPBytes. If this error is returned, the channel // should be closed and a new one should be made. -func (co *ChannelOut) AddBlock(block *types.Block) (uint64, error) { +func (co *SingularChannelOut) AddBlock(block *types.Block) (uint64, error) { if co.closed { return 0, errors.New("already closed") } @@ -137,28 +135,17 @@ func (co *ChannelOut) AddBlock(block *types.Block) (uint64, error) { // that it returns is ErrTooManyRLPBytes. If this error is returned, the channel // should be closed and a new one should be made. // -// AddSingularBatch should be used together with BlockToSingularBatch if you need to access the +// AddSingularBatch should be used together with BlockToBatch if you need to access the // BatchData before adding a block to the channel. It isn't possible to access // the batch data with AddBlock. -func (co *ChannelOut) AddSingularBatch(batch *SingularBatch) (uint64, error) { +func (co *SingularChannelOut) AddSingularBatch(batch *SingularBatch) (uint64, error) { if co.closed { return 0, errors.New("already closed") } - switch co.batchType { - case SingularBatchType: - return co.writeSingularBatch(batch) - case SpanBatchType: - return co.writeSpanBatch(batch) - default: - return 0, fmt.Errorf("unrecognized batch type: %d", co.batchType) - } -} - -func (co *ChannelOut) writeSingularBatch(batch *SingularBatch) (uint64, error) { - var buf bytes.Buffer // We encode to a temporary buffer to determine the encoded length to // ensure that the total size of all RLP elements is less than or equal to MAX_RLP_BYTES_PER_CHANNEL + var buf bytes.Buffer if err := rlp.Encode(&buf, NewSingularBatchData(*batch)); err != nil { return 0, err } @@ -173,113 +160,33 @@ func (co *ChannelOut) writeSingularBatch(batch *SingularBatch) (uint64, error) { return uint64(written), err } -// writeSpanBatch appends a SingularBatch to the channel's SpanBatch. -// A channel can have only one SpanBatch. And compressed results should not be accessible until the channel is closed, since the prefix and payload can be changed. -// So it resets channel contents and rewrites the entire SpanBatch each time, and compressed results are copied to reader after the channel is closed. -// It makes we can only get frames once the channel is full or closed, in the case of SpanBatch. -func (co *ChannelOut) writeSpanBatch(batch *SingularBatch) (uint64, error) { - if co.FullErr() != nil { - // channel is already full - return 0, co.FullErr() - } - var buf bytes.Buffer - // Append Singular batch to its span batch builder - co.spanBatchBuilder.AppendSingularBatch(batch) - // Convert Span batch to RawSpanBatch - rawSpanBatch, err := co.spanBatchBuilder.GetRawSpanBatch() - if err != nil { - return 0, fmt.Errorf("failed to convert SpanBatch into RawSpanBatch: %w", err) - } - // Encode RawSpanBatch into bytes - if err = rlp.Encode(&buf, NewSpanBatchData(*rawSpanBatch)); err != nil { - return 0, fmt.Errorf("failed to encode RawSpanBatch into bytes: %w", err) - } - co.rlpLength = 0 - // Ensure that the total size of all RLP elements is less than or equal to MAX_RLP_BYTES_PER_CHANNEL - if co.rlpLength+buf.Len() > MaxRLPBytesPerChannel { - return 0, fmt.Errorf("could not add %d bytes to channel of %d bytes, max is %d. err: %w", - buf.Len(), co.rlpLength, MaxRLPBytesPerChannel, ErrTooManyRLPBytes) - } - co.rlpLength = buf.Len() - - if co.spanBatchBuilder.GetBlockCount() > 1 { - // Flush compressed data into reader to preserve current result. - // If the channel is full after this block is appended, we should use preserved data. - if err := co.compress.Flush(); err != nil { - return 0, fmt.Errorf("failed to flush compressor: %w", err) - } - _, err = io.Copy(co.reader, co.compress) - if err != nil { - // Must reset reader to avoid partial output - co.reader.Reset() - return 0, fmt.Errorf("failed to copy compressed data to reader: %w", err) - } - } - - // Reset compressor to rewrite the entire span batch - co.compress.Reset() - // Avoid using io.Copy here, because we need all or nothing - written, err := co.compress.Write(buf.Bytes()) - if co.compress.FullErr() != nil { - err = co.compress.FullErr() - if co.spanBatchBuilder.GetBlockCount() == 1 { - // Do not return CompressorFullErr for the first block in the batch - // In this case, reader must be empty. then the contents of compressor will be copied to reader when the channel is closed. - err = nil - } - // If there are more than one blocks in the channel, reader should have data that preserves previous compression result before adding this block. - // So, as a result, this block is not added to the channel and the channel will be closed. - return uint64(written), err - } - - // If compressor is not full yet, reader must be reset to avoid submitting invalid frames - co.reader.Reset() - return uint64(written), err -} - // InputBytes returns the total amount of RLP-encoded input bytes. -func (co *ChannelOut) InputBytes() int { +func (co *SingularChannelOut) InputBytes() int { return co.rlpLength } // ReadyBytes returns the number of bytes that the channel out can immediately output into a frame. // Use `Flush` or `Close` to move data from the compression buffer into the ready buffer if more bytes // are needed. Add blocks may add to the ready buffer, but it is not guaranteed due to the compression stage. -func (co *ChannelOut) ReadyBytes() int { - return co.reader.Len() +func (co *SingularChannelOut) ReadyBytes() int { + return co.compress.Len() } // Flush flushes the internal compression stage to the ready buffer. It enables pulling a larger & more // complete frame. It reduces the compression efficiency. -func (co *ChannelOut) Flush() error { - if err := co.compress.Flush(); err != nil { - return err - } - if co.batchType == SpanBatchType && co.closed && co.ReadyBytes() == 0 && co.compress.Len() > 0 { - _, err := io.Copy(co.reader, co.compress) - if err != nil { - // Must reset reader to avoid partial output - co.reader.Reset() - return fmt.Errorf("failed to flush compressed data to reader: %w", err) - } - } - return nil +func (co *SingularChannelOut) Flush() error { + return co.compress.Flush() } -func (co *ChannelOut) FullErr() error { +func (co *SingularChannelOut) FullErr() error { return co.compress.FullErr() } -func (co *ChannelOut) Close() error { +func (co *SingularChannelOut) Close() error { if co.closed { return errors.New("already closed") } co.closed = true - if co.batchType == SpanBatchType { - if err := co.Flush(); err != nil { - return err - } - } return co.compress.Close() } @@ -290,30 +197,15 @@ func (co *ChannelOut) Close() error { // Returns io.EOF when the channel is closed & there are no more frames. // Returns nil if there is still more buffered data. // Returns an error if it ran into an error during processing. -func (co *ChannelOut) OutputFrame(w *bytes.Buffer, maxSize uint64) (uint16, error) { - f := Frame{ - ID: co.id, - FrameNumber: uint16(co.frame), - } - +func (co *SingularChannelOut) OutputFrame(w *bytes.Buffer, maxSize uint64) (uint16, error) { // Check that the maxSize is large enough for the frame overhead size. if maxSize < FrameV0OverHeadSize { return 0, ErrMaxFrameSizeTooSmall } - // Copy data from the local buffer into the frame data buffer - maxDataSize := maxSize - FrameV0OverHeadSize - if maxDataSize > uint64(co.ReadyBytes()) { - maxDataSize = uint64(co.ReadyBytes()) - // If we are closed & will not spill past the current frame - // mark it is the final frame of the channel. - if co.closed { - f.IsLast = true - } - } - f.Data = make([]byte, maxDataSize) + f := createEmptyFrame(co.id, co.frame, co.ReadyBytes(), co.closed, maxSize) - if _, err := io.ReadFull(co.reader, f.Data); err != nil { + if _, err := io.ReadFull(co.compress, f.Data); err != nil { return 0, err } @@ -418,3 +310,24 @@ func ForceCloseTxData(frames []Frame) ([]byte, error) { return out.Bytes(), nil } + +// createEmptyFrame creates new empty Frame with given information. Frame data must be copied from ChannelOut. +func createEmptyFrame(id ChannelID, frame uint64, readyBytes int, closed bool, maxSize uint64) *Frame { + f := Frame{ + ID: id, + FrameNumber: uint16(frame), + } + + // Copy data from the local buffer into the frame data buffer + maxDataSize := maxSize - FrameV0OverHeadSize + if maxDataSize > uint64(readyBytes) { + maxDataSize = uint64(readyBytes) + // If we are closed & will not spill past the current frame + // mark it is the final frame of the channel. + if closed { + f.IsLast = true + } + } + f.Data = make([]byte, maxDataSize) + return &f +} diff --git a/op-node/rollup/derive/channel_out_test.go b/op-node/rollup/derive/channel_out_test.go index 0a01da76d0a3..0eef4a241c1b 100644 --- a/op-node/rollup/derive/channel_out_test.go +++ b/op-node/rollup/derive/channel_out_test.go @@ -29,7 +29,7 @@ func (s *nonCompressor) FullErr() error { } func TestChannelOutAddBlock(t *testing.T) { - cout, err := NewChannelOut(&nonCompressor{}, SingularBatchType, nil) + cout, err := NewChannelOut(SingularBatchType, &nonCompressor{}, nil) require.NoError(t, err) t.Run("returns err if first tx is not an l1info tx", func(t *testing.T) { @@ -50,7 +50,7 @@ func TestChannelOutAddBlock(t *testing.T) { // max size that is below the fixed frame size overhead of 23, will return // an error. func TestOutputFrameSmallMaxSize(t *testing.T) { - cout, err := NewChannelOut(&nonCompressor{}, SingularBatchType, nil) + cout, err := NewChannelOut(SingularBatchType, &nonCompressor{}, nil) require.NoError(t, err) // Call OutputFrame with the range of small max size values that err diff --git a/op-node/rollup/derive/span_channel_out.go b/op-node/rollup/derive/span_channel_out.go new file mode 100644 index 000000000000..d074a19c29ef --- /dev/null +++ b/op-node/rollup/derive/span_channel_out.go @@ -0,0 +1,229 @@ +package derive + +import ( + "bytes" + "crypto/rand" + "errors" + "fmt" + "io" + + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/rlp" +) + +type SpanChannelOut struct { + id ChannelID + // Frame ID of the next frame to emit. Increment after emitting + frame uint64 + // rlpLength is the uncompressed size of the channel. Must be less than MAX_RLP_BYTES_PER_CHANNEL + rlpLength int + + // Compressor stage. Write input data to it + compress Compressor + // closed indicates if the channel is closed + closed bool + // spanBatchBuilder contains information requires to build SpanBatch + spanBatchBuilder *SpanBatchBuilder + // reader contains compressed data for making output frames + reader *bytes.Buffer +} + +func (co *SpanChannelOut) ID() ChannelID { + return co.id +} + +func NewSpanChannelOut(compress Compressor, spanBatchBuilder *SpanBatchBuilder) (*SpanChannelOut, error) { + c := &SpanChannelOut{ + id: ChannelID{}, // TODO: use GUID here instead of fully random data + frame: 0, + rlpLength: 0, + compress: compress, + spanBatchBuilder: spanBatchBuilder, + reader: &bytes.Buffer{}, + } + _, err := rand.Read(c.id[:]) + if err != nil { + return nil, err + } + + return c, nil +} + +// TODO: reuse ChannelOut for performance +func (co *SpanChannelOut) Reset() error { + co.frame = 0 + co.rlpLength = 0 + co.compress.Reset() + co.reader.Reset() + co.closed = false + co.spanBatchBuilder.Reset() + _, err := rand.Read(co.id[:]) + return err +} + +// AddBlock adds a block to the channel. It returns the RLP encoded byte size +// and an error if there is a problem adding the block. The only sentinel error +// that it returns is ErrTooManyRLPBytes. If this error is returned, the channel +// should be closed and a new one should be made. +func (co *SpanChannelOut) AddBlock(block *types.Block) (uint64, error) { + if co.closed { + return 0, errors.New("already closed") + } + + batch, _, err := BlockToSingularBatch(block) + if err != nil { + return 0, err + } + return co.AddSingularBatch(batch) +} + +// AddSingularBatch adds a batch to the channel. It returns the RLP encoded byte size +// and an error if there is a problem adding the batch. The only sentinel error +// that it returns is ErrTooManyRLPBytes. If this error is returned, the channel +// should be closed and a new one should be made. +// +// AddSingularBatch should be used together with BlockToSingularBatch if you need to access the +// BatchData before adding a block to the channel. It isn't possible to access +// the batch data with AddBlock. +// +// SingularBatch is appended to the channel's SpanBatch. +// A channel can have only one SpanBatch. And compressed results should not be accessible until the channel is closed, since the prefix and payload can be changed. +// So it resets channel contents and rewrites the entire SpanBatch each time, and compressed results are copied to reader after the channel is closed. +// It makes we can only get frames once the channel is full or closed, in the case of SpanBatch. +func (co *SpanChannelOut) AddSingularBatch(batch *SingularBatch) (uint64, error) { + if co.closed { + return 0, errors.New("already closed") + } + if co.FullErr() != nil { + // channel is already full + return 0, co.FullErr() + } + var buf bytes.Buffer + // Append Singular batch to its span batch builder + co.spanBatchBuilder.AppendSingularBatch(batch) + // Convert Span batch to RawSpanBatch + rawSpanBatch, err := co.spanBatchBuilder.GetRawSpanBatch() + if err != nil { + return 0, fmt.Errorf("failed to convert SpanBatch into RawSpanBatch: %w", err) + } + // Encode RawSpanBatch into bytes + if err = rlp.Encode(&buf, NewSpanBatchData(*rawSpanBatch)); err != nil { + return 0, fmt.Errorf("failed to encode RawSpanBatch into bytes: %w", err) + } + // Ensure that the total size of all RLP elements is less than or equal to MAX_RLP_BYTES_PER_CHANNEL + if buf.Len() > MaxRLPBytesPerChannel { + return 0, fmt.Errorf("could not add %d bytes to channel of %d bytes, max is %d. err: %w", + buf.Len(), co.rlpLength, MaxRLPBytesPerChannel, ErrTooManyRLPBytes) + } + co.rlpLength = buf.Len() + + if co.spanBatchBuilder.GetBlockCount() > 1 { + // Flush compressed data into reader to preserve current result. + // If the channel is full after this block is appended, we should use preserved data. + if err := co.compress.Flush(); err != nil { + return 0, fmt.Errorf("failed to flush compressor: %w", err) + } + _, err = io.Copy(co.reader, co.compress) + if err != nil { + // Must reset reader to avoid partial output + co.reader.Reset() + return 0, fmt.Errorf("failed to copy compressed data to reader: %w", err) + } + } + + // Reset compressor to rewrite the entire span batch + co.compress.Reset() + // Avoid using io.Copy here, because we need all or nothing + written, err := co.compress.Write(buf.Bytes()) + if co.compress.FullErr() != nil { + err = co.compress.FullErr() + if co.spanBatchBuilder.GetBlockCount() == 1 { + // Do not return CompressorFullErr for the first block in the batch + // In this case, reader must be empty. then the contents of compressor will be copied to reader when the channel is closed. + err = nil + } + // If there are more than one blocks in the channel, reader should have data that preserves previous compression result before adding this block. + // So, as a result, this block is not added to the channel and the channel will be closed. + return uint64(written), err + } + + // If compressor is not full yet, reader must be reset to avoid submitting invalid frames + co.reader.Reset() + return uint64(written), err +} + +// InputBytes returns the total amount of RLP-encoded input bytes. +func (co *SpanChannelOut) InputBytes() int { + return co.rlpLength +} + +// ReadyBytes returns the number of bytes that the channel out can immediately output into a frame. +// Use `Flush` or `Close` to move data from the compression buffer into the ready buffer if more bytes +// are needed. Add blocks may add to the ready buffer, but it is not guaranteed due to the compression stage. +func (co *SpanChannelOut) ReadyBytes() int { + return co.reader.Len() +} + +// Flush flushes the internal compression stage to the ready buffer. It enables pulling a larger & more +// complete frame. It reduces the compression efficiency. +func (co *SpanChannelOut) Flush() error { + if err := co.compress.Flush(); err != nil { + return err + } + if co.closed && co.ReadyBytes() == 0 && co.compress.Len() > 0 { + _, err := io.Copy(co.reader, co.compress) + if err != nil { + // Must reset reader to avoid partial output + co.reader.Reset() + return fmt.Errorf("failed to flush compressed data to reader: %w", err) + } + } + return nil +} + +func (co *SpanChannelOut) FullErr() error { + return co.compress.FullErr() +} + +func (co *SpanChannelOut) Close() error { + if co.closed { + return errors.New("already closed") + } + co.closed = true + if err := co.Flush(); err != nil { + return err + } + return co.compress.Close() +} + +// OutputFrame writes a frame to w with a given max size and returns the frame +// number. +// Use `ReadyBytes`, `Flush`, and `Close` to modify the ready buffer. +// Returns an error if the `maxSize` < FrameV0OverHeadSize. +// Returns io.EOF when the channel is closed & there are no more frames. +// Returns nil if there is still more buffered data. +// Returns an error if it ran into an error during processing. +func (co *SpanChannelOut) OutputFrame(w *bytes.Buffer, maxSize uint64) (uint16, error) { + // Check that the maxSize is large enough for the frame overhead size. + if maxSize < FrameV0OverHeadSize { + return 0, ErrMaxFrameSizeTooSmall + } + + f := createEmptyFrame(co.id, co.frame, co.ReadyBytes(), co.closed, maxSize) + + if _, err := io.ReadFull(co.reader, f.Data); err != nil { + return 0, err + } + + if err := f.MarshalBinary(w); err != nil { + return 0, err + } + + co.frame += 1 + fn := f.FrameNumber + if f.IsLast { + return fn, io.EOF + } else { + return fn, nil + } +} From 33983267003e8c7d33d22495fe78c4f14e2ea1c7 Mon Sep 17 00:00:00 2001 From: protolambda Date: Wed, 25 Oct 2023 03:15:00 +0200 Subject: [PATCH 064/279] op-batcher: fix lint --- op-batcher/batcher/driver.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/op-batcher/batcher/driver.go b/op-batcher/batcher/driver.go index d45ae3153780..d61de4b2b3aa 100644 --- a/op-batcher/batcher/driver.go +++ b/op-batcher/batcher/driver.go @@ -4,7 +4,6 @@ import ( "context" "errors" "fmt" - "github.com/ethereum-optimism/optimism/op-node/rollup/derive" "io" "math/big" _ "net/http/pprof" @@ -17,6 +16,7 @@ import ( "github.com/ethereum-optimism/optimism/op-batcher/metrics" "github.com/ethereum-optimism/optimism/op-node/rollup" + "github.com/ethereum-optimism/optimism/op-node/rollup/derive" "github.com/ethereum-optimism/optimism/op-service/eth" "github.com/ethereum-optimism/optimism/op-service/txmgr" ) From 5f5db7775660d1d5a00f396b48ebd11e16cfe0ba Mon Sep 17 00:00:00 2001 From: Tei Im Date: Wed, 25 Oct 2023 15:01:24 +0900 Subject: [PATCH 065/279] Use sequencer number to calculate origin bit of span batch Remove lastProcessedBlock from ChannelManager Add seqNum arg to ChannelOut AddSingularBatch interface --- op-batcher/batcher/channel.go | 5 +- op-batcher/batcher/channel_builder.go | 9 +++- op-batcher/batcher/channel_builder_test.go | 45 ++++++++-------- op-batcher/batcher/channel_manager.go | 32 +++-------- op-batcher/batcher/channel_manager_test.go | 63 +++++++++++----------- op-batcher/batcher/channel_test.go | 8 +-- op-batcher/batcher/driver.go | 37 ++++--------- op-node/rollup/derive/channel_out.go | 8 +-- op-node/rollup/derive/span_batch.go | 19 +++---- op-node/rollup/derive/span_batch_test.go | 8 +-- op-node/rollup/derive/span_channel_out.go | 8 +-- op-node/rollup/derive/test/random.go | 11 ++++ 12 files changed, 117 insertions(+), 136 deletions(-) diff --git a/op-batcher/batcher/channel.go b/op-batcher/batcher/channel.go index cd44f31a9c46..1d1eb2d5f568 100644 --- a/op-batcher/batcher/channel.go +++ b/op-batcher/batcher/channel.go @@ -5,6 +5,7 @@ import ( "math" "github.com/ethereum-optimism/optimism/op-batcher/metrics" + "github.com/ethereum-optimism/optimism/op-node/rollup" "github.com/ethereum-optimism/optimism/op-node/rollup/derive" "github.com/ethereum-optimism/optimism/op-service/eth" "github.com/ethereum/go-ethereum/core/types" @@ -26,8 +27,8 @@ type channel struct { confirmedTransactions map[txID]eth.BlockID } -func newChannel(log log.Logger, metr metrics.Metricer, cfg ChannelConfig, spanBatchBuilder *derive.SpanBatchBuilder) (*channel, error) { - cb, err := newChannelBuilder(cfg, spanBatchBuilder) +func newChannel(log log.Logger, metr metrics.Metricer, cfg ChannelConfig, rcfg *rollup.Config) (*channel, error) { + cb, err := newChannelBuilder(cfg, rcfg) if err != nil { return nil, fmt.Errorf("creating new channel: %w", err) } diff --git a/op-batcher/batcher/channel_builder.go b/op-batcher/batcher/channel_builder.go index 81bce734b522..e336c90ad3d5 100644 --- a/op-batcher/batcher/channel_builder.go +++ b/op-batcher/batcher/channel_builder.go @@ -8,6 +8,7 @@ import ( "math" "github.com/ethereum-optimism/optimism/op-batcher/compressor" + "github.com/ethereum-optimism/optimism/op-node/rollup" "github.com/ethereum-optimism/optimism/op-node/rollup/derive" "github.com/ethereum/go-ethereum/core/types" ) @@ -134,11 +135,15 @@ type channelBuilder struct { // newChannelBuilder creates a new channel builder or returns an error if the // channel out could not be created. -func newChannelBuilder(cfg ChannelConfig, spanBatchBuilder *derive.SpanBatchBuilder) (*channelBuilder, error) { +func newChannelBuilder(cfg ChannelConfig, rcfg *rollup.Config) (*channelBuilder, error) { c, err := cfg.CompressorConfig.NewCompressor() if err != nil { return nil, err } + var spanBatchBuilder *derive.SpanBatchBuilder + if cfg.BatchType == derive.SpanBatchType { + spanBatchBuilder = derive.NewSpanBatchBuilder(rcfg.Genesis.L2Time, rcfg.L2ChainID) + } co, err := derive.NewChannelOut(cfg.BatchType, c, spanBatchBuilder) if err != nil { return nil, err @@ -206,7 +211,7 @@ func (c *channelBuilder) AddBlock(block *types.Block) (derive.L1BlockInfo, error return l1info, fmt.Errorf("converting block to batch: %w", err) } - if _, err = c.co.AddSingularBatch(batch); errors.Is(err, derive.ErrTooManyRLPBytes) || errors.Is(err, derive.CompressorFullErr) { + if _, err = c.co.AddSingularBatch(batch, l1info.SequenceNumber); errors.Is(err, derive.ErrTooManyRLPBytes) || errors.Is(err, derive.CompressorFullErr) { c.setFullErr(err) return l1info, c.FullErr() } else if err != nil { diff --git a/op-batcher/batcher/channel_builder_test.go b/op-batcher/batcher/channel_builder_test.go index fa8512039d8a..db8e74e2e932 100644 --- a/op-batcher/batcher/channel_builder_test.go +++ b/op-batcher/batcher/channel_builder_test.go @@ -35,12 +35,9 @@ var defaultTestChannelConfig = ChannelConfig{ BatchType: derive.SingularBatchType, } -func getSpanBatchBuilder(batchType uint) *derive.SpanBatchBuilder { - if batchType == derive.SpanBatchType { - chainId := big.NewInt(1234) - return derive.NewSpanBatchBuilder(uint64(0), uint64(0), chainId) - } - return nil +var defaultTestRollupConfig = rollup.Config{ + Genesis: rollup.Genesis{L2: eth.BlockID{Number: 0}}, + L2ChainID: big.NewInt(1234), } // TestChannelConfig_Check tests the [ChannelConfig] [Check] function. @@ -169,7 +166,7 @@ func newMiniL2BlockWithNumberParent(numTx int, number *big.Int, parent common.Ha func addTooManyBlocks(cb *channelBuilder) error { rng := rand.New(rand.NewSource(1234)) for i := 0; i < 10_000; i++ { - block, _ := dtest.RandomL2Block(rng, 1000) + block := dtest.RandomL2BlockWithChainId(rng, 1000, defaultTestRollupConfig.L2ChainID) _, err := cb.AddBlock(block) if err != nil { return err @@ -518,7 +515,7 @@ func TestChannelBuilder_OutputFramesWorks_SpanBatch(t *testing.T) { channelConfig.BatchType = derive.SpanBatchType // Construct the channel builder - cb, err := newChannelBuilder(channelConfig, getSpanBatchBuilder(derive.SpanBatchType)) + cb, err := newChannelBuilder(channelConfig, &defaultTestRollupConfig) require.NoError(t, err) require.False(t, cb.IsFull()) require.Equal(t, 0, cb.PendingFrames()) @@ -571,7 +568,7 @@ func ChannelBuilder_MaxRLPBytesPerChannel(t *testing.T, batchType uint) { channelConfig.BatchType = batchType // Construct the channel builder - cb, err := newChannelBuilder(channelConfig, getSpanBatchBuilder(batchType)) + cb, err := newChannelBuilder(channelConfig, &defaultTestRollupConfig) require.NoError(t, err) // Add a block that overflows the [ChannelOut] @@ -594,12 +591,12 @@ func ChannelBuilder_OutputFramesMaxFrameIndex(t *testing.T, batchType uint) { // Continuously add blocks until the max frame index is reached // This should cause the [channelBuilder.OutputFrames] function // to error - cb, err := newChannelBuilder(channelConfig, getSpanBatchBuilder(batchType)) + cb, err := newChannelBuilder(channelConfig, &defaultTestRollupConfig) require.NoError(t, err) require.False(t, cb.IsFull()) require.Equal(t, 0, cb.PendingFrames()) for { - a, _ := dtest.RandomL2Block(rng, 1) + a := dtest.RandomL2BlockWithChainId(rng, 1, defaultTestRollupConfig.L2ChainID) _, err = cb.AddBlock(a) if cb.IsFull() { fullErr := cb.FullErr() @@ -627,7 +624,7 @@ func ChannelBuilder_AddBlock(t *testing.T, batchType uint) { channelConfig.CompressorConfig.ApproxComprRatio = 1 // Construct the channel builder - cb, err := newChannelBuilder(channelConfig, getSpanBatchBuilder(batchType)) + cb, err := newChannelBuilder(channelConfig, &defaultTestRollupConfig) require.NoError(t, err) // Add a nonsense block to the channel builder @@ -660,7 +657,7 @@ func ChannelBuilder_Reset(t *testing.T, batchType uint) { channelConfig.CompressorConfig.TargetFrameSize = 24 channelConfig.CompressorConfig.ApproxComprRatio = 1 - cb, err := newChannelBuilder(channelConfig, getSpanBatchBuilder(batchType)) + cb, err := newChannelBuilder(channelConfig, &defaultTestRollupConfig) require.NoError(t, err) // Add a nonsense block to the channel builder @@ -771,7 +768,7 @@ func ChannelBuilder_PendingFrames_TotalFrames(t *testing.T, batchType uint) { cfg.CompressorConfig.TargetNumFrames = tnf cfg.CompressorConfig.Kind = "shadow" cfg.BatchType = batchType - cb, err := newChannelBuilder(cfg, getSpanBatchBuilder(batchType)) + cb, err := newChannelBuilder(cfg, &defaultTestRollupConfig) require.NoError(err) // initial builder should be empty @@ -780,7 +777,7 @@ func ChannelBuilder_PendingFrames_TotalFrames(t *testing.T, batchType uint) { // fill up for { - block, _ := dtest.RandomL2Block(rng, 4) + block := dtest.RandomL2BlockWithChainId(rng, 4, defaultTestRollupConfig.L2ChainID) _, err := cb.AddBlock(block) if cb.IsFull() { break @@ -810,21 +807,25 @@ func ChannelBuilder_InputBytes(t *testing.T, batchType uint) { rng := rand.New(rand.NewSource(4982432)) cfg := defaultTestChannelConfig cfg.BatchType = batchType - spanBatchBuilder := getSpanBatchBuilder(batchType) - cb, err := newChannelBuilder(cfg, getSpanBatchBuilder(batchType)) + var spanBatchBuilder *derive.SpanBatchBuilder + if batchType == derive.SpanBatchType { + chainId := big.NewInt(1234) + spanBatchBuilder = derive.NewSpanBatchBuilder(uint64(0), chainId) + } + cb, err := newChannelBuilder(cfg, &defaultTestRollupConfig) require.NoError(err) require.Zero(cb.InputBytes()) var l int for i := 0; i < 5; i++ { - block := newMiniL2Block(rng.Intn(32)) + block := dtest.RandomL2BlockWithChainId(rng, rng.Intn(32), defaultTestRollupConfig.L2ChainID) if batchType == derive.SingularBatchType { l += blockBatchRlpSize(t, block) } else { - singularBatch, _, err := derive.BlockToSingularBatch(block) + singularBatch, l1Info, err := derive.BlockToSingularBatch(block) require.NoError(err) - spanBatchBuilder.AppendSingularBatch(singularBatch) + spanBatchBuilder.AppendSingularBatch(singularBatch, l1Info.SequenceNumber) rawSpanBatch, err := spanBatchBuilder.GetRawSpanBatch() require.NoError(err) batch := derive.NewSpanBatchData(*rawSpanBatch) @@ -847,13 +848,13 @@ func ChannelBuilder_OutputBytes(t *testing.T, batchType uint) { cfg.CompressorConfig.TargetNumFrames = 16 cfg.CompressorConfig.ApproxComprRatio = 1.0 cfg.BatchType = batchType - cb, err := newChannelBuilder(cfg, getSpanBatchBuilder(batchType)) + cb, err := newChannelBuilder(cfg, &defaultTestRollupConfig) require.NoError(err, "newChannelBuilder") require.Zero(cb.OutputBytes()) for { - block, _ := dtest.RandomL2Block(rng, rng.Intn(32)) + block := dtest.RandomL2BlockWithChainId(rng, rng.Intn(32), defaultTestRollupConfig.L2ChainID) _, err := cb.AddBlock(block) if errors.Is(err, derive.CompressorFullErr) { break diff --git a/op-batcher/batcher/channel_manager.go b/op-batcher/batcher/channel_manager.go index dc00fab364e9..5829e3dc448b 100644 --- a/op-batcher/batcher/channel_manager.go +++ b/op-batcher/batcher/channel_manager.go @@ -36,9 +36,6 @@ type channelManager struct { // last block hash - for reorg detection tip common.Hash - // last block added to channel. nil at first. - lastProcessedBlock *eth.L2BlockRef - // channel to write new block data to currentChannel *channel // channels to read frame data from, for writing batches onchain @@ -52,19 +49,18 @@ type channelManager struct { func NewChannelManager(log log.Logger, metr metrics.Metricer, cfg ChannelConfig, rcfg *rollup.Config) *channelManager { return &channelManager{ - log: log, - metr: metr, - cfg: cfg, - rcfg: rcfg, - txChannels: make(map[txID]*channel), - lastProcessedBlock: nil, + log: log, + metr: metr, + cfg: cfg, + rcfg: rcfg, + txChannels: make(map[txID]*channel), } } // Clear clears the entire state of the channel manager. // It is intended to be used before launching op-batcher and after an L2 reorg. // Must set lastProcessedBlock as current L2 safe head fetched from L2 node. -func (s *channelManager) Clear(safeHead *eth.L2BlockRef) { +func (s *channelManager) Clear() { s.mu.Lock() defer s.mu.Unlock() s.log.Trace("clearing channel manager state") @@ -74,7 +70,6 @@ func (s *channelManager) Clear(safeHead *eth.L2BlockRef) { s.currentChannel = nil s.channelQueue = nil s.txChannels = make(map[txID]*channel) - s.lastProcessedBlock = safeHead } // TxFailed records a transaction as failed. It will attempt to resubmit the data @@ -204,19 +199,7 @@ func (s *channelManager) ensureChannelWithSpace(l1Head eth.BlockID) error { return nil } - var spanBatchBuilder *derive.SpanBatchBuilder - if s.cfg.BatchType == derive.SpanBatchType { - if s.lastProcessedBlock == nil { - // TODO: we can remove "lastProcessedBlock" if we change the data-builder - // to append a singular-batch *with* the L2 metadata such as the L1-block-info seq-number; - // this helps determine whether or not the L1 origin changed in the first block of the span, - // without having to remember the last block from before the span. - return errors.New("last block is not initialized") - } - // Pass the current lastProcessedBlock as the parent - spanBatchBuilder = derive.NewSpanBatchBuilder(s.lastProcessedBlock.L1Origin.Number, s.rcfg.Genesis.L2Time, s.rcfg.L2ChainID) - } - pc, err := newChannel(s.log, s.metr, s.cfg, spanBatchBuilder) + pc, err := newChannel(s.log, s.metr, s.cfg, s.rcfg) if err != nil { return fmt.Errorf("creating new channel: %w", err) } @@ -262,7 +245,6 @@ func (s *channelManager) processBlocks() error { blocksAdded += 1 latestL2ref = l2BlockRefFromBlockAndL1Info(block, l1info) s.metr.RecordL2BlockInChannel(block) - s.lastProcessedBlock = &latestL2ref // current block got added but channel is now full if s.currentChannel.IsFull() { break diff --git a/op-batcher/batcher/channel_manager_test.go b/op-batcher/batcher/channel_manager_test.go index b68d13e4cc8b..eac3cff44d11 100644 --- a/op-batcher/batcher/channel_manager_test.go +++ b/op-batcher/batcher/channel_manager_test.go @@ -14,7 +14,6 @@ import ( derivetest "github.com/ethereum-optimism/optimism/op-node/rollup/derive/test" "github.com/ethereum-optimism/optimism/op-service/eth" "github.com/ethereum-optimism/optimism/op-service/testlog" - "github.com/ethereum-optimism/optimism/op-service/testutils" "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/log" @@ -55,7 +54,7 @@ func TestChannelManagerBatchType(t *testing.T) { func ChannelManagerReturnsErrReorg(t *testing.T, batchType uint) { log := testlog.Logger(t, log.LvlCrit) m := NewChannelManager(log, metrics.NoopMetrics, ChannelConfig{BatchType: batchType}, &rollup.Config{}) - m.Clear(ð.L2BlockRef{}) + m.Clear() a := types.NewBlock(&types.Header{ Number: big.NewInt(0), @@ -97,7 +96,7 @@ func ChannelManagerReturnsErrReorgWhenDrained(t *testing.T, batchType uint) { }, &rollup.Config{}, ) - m.Clear(ð.L2BlockRef{}) + m.Clear() a := newMiniL2Block(0) x := newMiniL2BlockWithNumberParent(0, big.NewInt(1), common.Hash{0xff}) @@ -134,7 +133,7 @@ func ChannelManager_Clear(t *testing.T, batchType uint) { }, BatchType: batchType, }, - &rollup.Config{}, + &defaultTestRollupConfig, ) // Channel Manager state should be empty by default @@ -143,12 +142,11 @@ func ChannelManager_Clear(t *testing.T, batchType uint) { require.Nil(m.currentChannel) require.Empty(m.channelQueue) require.Empty(m.txChannels) - require.Nil(m.lastProcessedBlock) // Set the last block - m.Clear(ð.L2BlockRef{}) + m.Clear() // Add a block to the channel manager - a, _ := derivetest.RandomL2Block(rng, 4) + a := derivetest.RandomL2BlockWithChainId(rng, 4, defaultTestRollupConfig.L2ChainID) newL1Tip := a.Hash() l1BlockID := eth.BlockID{ Hash: a.Hash(), @@ -185,8 +183,7 @@ func ChannelManager_Clear(t *testing.T, batchType uint) { require.Equal(b.Hash(), m.tip) // Clear the channel manager - safeHead := testutils.RandomL2BlockRef(rng) - m.Clear(&safeHead) + m.Clear() // Check that the entire channel manager state cleared require.Empty(m.blocks) @@ -194,7 +191,6 @@ func ChannelManager_Clear(t *testing.T, batchType uint) { require.Nil(m.currentChannel) require.Empty(m.channelQueue) require.Empty(m.txChannels) - require.Equal(m.lastProcessedBlock, &safeHead) } func ChannelManager_TxResend(t *testing.T, batchType uint) { @@ -211,11 +207,11 @@ func ChannelManager_TxResend(t *testing.T, batchType uint) { }, BatchType: batchType, }, - &rollup.Config{}, + &defaultTestRollupConfig, ) - m.Clear(ð.L2BlockRef{}) + m.Clear() - a, _ := derivetest.RandomL2Block(rng, 4) + a := derivetest.RandomL2BlockWithChainId(rng, 4, defaultTestRollupConfig.L2ChainID) require.NoError(m.AddL2Block(a)) @@ -259,11 +255,11 @@ func ChannelManagerCloseBeforeFirstUse(t *testing.T, batchType uint) { }, BatchType: batchType, }, - &rollup.Config{}, + &defaultTestRollupConfig, ) - m.Clear(ð.L2BlockRef{}) + m.Clear() - a, _ := derivetest.RandomL2Block(rng, 4) + a := derivetest.RandomL2BlockWithChainId(rng, 4, defaultTestRollupConfig.L2ChainID) m.Close() @@ -291,9 +287,9 @@ func ChannelManagerCloseNoPendingChannel(t *testing.T, batchType uint) { }, BatchType: batchType, }, - &rollup.Config{}, + &defaultTestRollupConfig, ) - m.Clear(ð.L2BlockRef{}) + m.Clear() a := newMiniL2Block(0) b := newMiniL2BlockWithNumberParent(0, big.NewInt(1), a.Hash()) @@ -322,30 +318,30 @@ func ChannelManagerCloseNoPendingChannel(t *testing.T, batchType uint) { // new channel frames after this point. func ChannelManagerClosePendingChannel(t *testing.T, batchType uint) { require := require.New(t) + rng := rand.New(rand.NewSource(time.Now().UnixNano())) log := testlog.Logger(t, log.LvlCrit) m := NewChannelManager(log, metrics.NoopMetrics, ChannelConfig{ - MaxFrameSize: 1000, + MaxFrameSize: 10000, ChannelTimeout: 1000, CompressorConfig: compressor.Config{ TargetNumFrames: 1, - TargetFrameSize: 1000, + TargetFrameSize: 10000, ApproxComprRatio: 1.0, }, BatchType: batchType, }, - &rollup.Config{}, + &defaultTestRollupConfig, ) - m.Clear(ð.L2BlockRef{}) + m.Clear() - numTx := 50000 - if batchType == derive.SpanBatchType { - // Adjust number of txs to make 2 frames - // Encoding empty txs as span batch requires more data size because span batch encodes tx signature to fixed length - numTx = 20000 - } - a := newMiniL2Block(numTx) - b := newMiniL2BlockWithNumberParent(10, big.NewInt(1), a.Hash()) + numTx := 20 // Adjust number of txs to make 2 frames + a := derivetest.RandomL2BlockWithChainId(rng, numTx, defaultTestRollupConfig.L2ChainID) + b := derivetest.RandomL2BlockWithChainId(rng, 10, defaultTestRollupConfig.L2ChainID) + bHeader := b.Header() + bHeader.Number = new(big.Int).Add(a.Number(), big.NewInt(1)) + bHeader.ParentHash = a.Hash() + b = b.WithSeal(bHeader) err := m.AddL2Block(a) require.NoError(err, "Failed to add L2 block") @@ -377,6 +373,7 @@ func ChannelManagerClosePendingChannel(t *testing.T, batchType uint) { // have successfully landed on chain. func ChannelManagerCloseAllTxsFailed(t *testing.T, batchType uint) { require := require.New(t) + rng := rand.New(rand.NewSource(time.Now().UnixNano())) log := testlog.Logger(t, log.LvlCrit) m := NewChannelManager(log, metrics.NoopMetrics, ChannelConfig{ @@ -388,11 +385,11 @@ func ChannelManagerCloseAllTxsFailed(t *testing.T, batchType uint) { ApproxComprRatio: 1.0, }, BatchType: batchType, - }, &rollup.Config{}, + }, &defaultTestRollupConfig, ) - m.Clear(ð.L2BlockRef{}) + m.Clear() - a := newMiniL2Block(50_000) + a := derivetest.RandomL2BlockWithChainId(rng, 50000, defaultTestRollupConfig.L2ChainID) err := m.AddL2Block(a) require.NoError(err, "Failed to add L2 block") diff --git a/op-batcher/batcher/channel_test.go b/op-batcher/batcher/channel_test.go index 26f64204bc73..138416069767 100644 --- a/op-batcher/batcher/channel_test.go +++ b/op-batcher/batcher/channel_test.go @@ -22,7 +22,7 @@ func TestChannelTimeout(t *testing.T) { m := NewChannelManager(log, metrics.NoopMetrics, ChannelConfig{ ChannelTimeout: 100, }, &rollup.Config{}) - m.Clear(ð.L2BlockRef{}) + m.Clear() // Pending channel is nil so is cannot be timed out require.Nil(t, m.currentChannel) @@ -64,7 +64,7 @@ func TestChannelTimeout(t *testing.T) { func TestChannelNextTxData(t *testing.T) { log := testlog.Logger(t, log.LvlCrit) m := NewChannelManager(log, metrics.NoopMetrics, ChannelConfig{}, &rollup.Config{}) - m.Clear(ð.L2BlockRef{}) + m.Clear() // Nil pending channel should return EOF returnedTxData, err := m.nextTxData(nil) @@ -113,7 +113,7 @@ func TestChannelTxConfirmed(t *testing.T) { // clearing confirmed transactions, and reseting the pendingChannels map ChannelTimeout: 10, }, &rollup.Config{}) - m.Clear(ð.L2BlockRef{}) + m.Clear() // Let's add a valid pending transaction to the channel manager // So we can demonstrate that TxConfirmed's correctness @@ -162,7 +162,7 @@ func TestChannelTxFailed(t *testing.T) { // Create a channel manager log := testlog.Logger(t, log.LvlCrit) m := NewChannelManager(log, metrics.NoopMetrics, ChannelConfig{}, &rollup.Config{}) - m.Clear(ð.L2BlockRef{}) + m.Clear() // Let's add a valid pending transaction to the channel // manager so we can demonstrate correctness diff --git a/op-batcher/batcher/driver.go b/op-batcher/batcher/driver.go index d61de4b2b3aa..54633060dac4 100644 --- a/op-batcher/batcher/driver.go +++ b/op-batcher/batcher/driver.go @@ -91,11 +91,7 @@ func (l *BatchSubmitter) StartBatchSubmitting() error { l.shutdownCtx, l.cancelShutdownCtx = context.WithCancel(context.Background()) l.killCtx, l.cancelKillCtx = context.WithCancel(context.Background()) - syncStatus, err := fetchSyncStatus(l.shutdownCtx, l.RollupClient, l.Cfg.NetworkTimeout) - if err != nil { - return err - } - l.state.Clear(&syncStatus.SafeL2) + l.state.Clear() l.lastStoredBlock = eth.BlockID{} l.wg.Add(1) @@ -205,9 +201,15 @@ func (l *BatchSubmitter) loadBlockIntoState(ctx context.Context, blockNumber uin // calculateL2BlockRangeToStore determines the range (start,end] that should be loaded into the local state. // It also takes care of initializing some local state (i.e. will modify l.lastStoredBlock in certain conditions) func (l *BatchSubmitter) calculateL2BlockRangeToStore(ctx context.Context) (eth.BlockID, eth.BlockID, error) { - syncStatus, err := fetchSyncStatus(ctx, l.RollupClient, l.Cfg.NetworkTimeout) + ctx, cancel := context.WithTimeout(ctx, l.Cfg.NetworkTimeout) + defer cancel() + syncStatus, err := l.RollupClient.SyncStatus(ctx) + // Ensure that we have the sync status if err != nil { - return eth.BlockID{}, eth.BlockID{}, err + return eth.BlockID{}, eth.BlockID{}, fmt.Errorf("failed to get sync status: %w", err) + } + if syncStatus.HeadL1 == (eth.L1BlockRef{}) { + return eth.BlockID{}, eth.BlockID{}, errors.New("empty sync status") } // Check last stored to see if it needs to be set on startup OR set if is lagged behind. @@ -257,12 +259,7 @@ func (l *BatchSubmitter) loop() { l.Log.Error("error closing the channel manager to handle a L2 reorg", "err", err) } l.publishStateToL1(queue, receiptsCh, true) - if syncStatus, err := fetchSyncStatus(l.shutdownCtx, l.RollupClient, l.Cfg.NetworkTimeout); err == nil { - l.state.Clear(&syncStatus.SafeL2) - } else { - // if fetchSyncStatus failed, ErrReorg will be returned again - l.Log.Error("error fetching sync status from L2 node", "err", err) - } + l.state.Clear() continue } l.publishStateToL1(queue, receiptsCh, false) @@ -398,17 +395,3 @@ func (l *BatchSubmitter) l1Tip(ctx context.Context) (eth.L1BlockRef, error) { } return eth.InfoToL1BlockRef(eth.HeaderBlockInfo(head)), nil } - -func fetchSyncStatus(ctx context.Context, rollupNode RollupClient, timeout time.Duration) (*eth.SyncStatus, error) { - ctx, cancel := context.WithTimeout(ctx, timeout) - defer cancel() - syncStatus, err := rollupNode.SyncStatus(ctx) - // Ensure that we have the sync status - if err != nil { - return ð.SyncStatus{}, fmt.Errorf("failed to get sync status: %w", err) - } - if syncStatus.SafeL2 == (eth.L2BlockRef{}) { - return ð.SyncStatus{}, errors.New("empty sync status") - } - return syncStatus, nil -} diff --git a/op-node/rollup/derive/channel_out.go b/op-node/rollup/derive/channel_out.go index d2af4be76e28..642c32ac06aa 100644 --- a/op-node/rollup/derive/channel_out.go +++ b/op-node/rollup/derive/channel_out.go @@ -52,7 +52,7 @@ type ChannelOut interface { ID() ChannelID Reset() error AddBlock(*types.Block) (uint64, error) - AddSingularBatch(*SingularBatch) (uint64, error) + AddSingularBatch(*SingularBatch, uint64) (uint64, error) InputBytes() int ReadyBytes() int Flush() error @@ -123,11 +123,11 @@ func (co *SingularChannelOut) AddBlock(block *types.Block) (uint64, error) { return 0, errors.New("already closed") } - batch, _, err := BlockToSingularBatch(block) + batch, l1Info, err := BlockToSingularBatch(block) if err != nil { return 0, err } - return co.AddSingularBatch(batch) + return co.AddSingularBatch(batch, l1Info.SequenceNumber) } // AddSingularBatch adds a batch to the channel. It returns the RLP encoded byte size @@ -138,7 +138,7 @@ func (co *SingularChannelOut) AddBlock(block *types.Block) (uint64, error) { // AddSingularBatch should be used together with BlockToBatch if you need to access the // BatchData before adding a block to the channel. It isn't possible to access // the batch data with AddBlock. -func (co *SingularChannelOut) AddSingularBatch(batch *SingularBatch) (uint64, error) { +func (co *SingularChannelOut) AddSingularBatch(batch *SingularBatch, _ uint64) (uint64, error) { if co.closed { return 0, errors.New("already closed") } diff --git a/op-node/rollup/derive/span_batch.go b/op-node/rollup/derive/span_batch.go index d68d542ff957..770eb58b7008 100644 --- a/op-node/rollup/derive/span_batch.go +++ b/op-node/rollup/derive/span_batch.go @@ -597,31 +597,32 @@ func NewSpanBatch(singularBatches []*SingularBatch) *SpanBatch { // SpanBatchBuilder is a utility type to build a SpanBatch by adding a SingularBatch one by one. // makes easier to stack SingularBatches and convert to RawSpanBatch for encoding. type SpanBatchBuilder struct { - parentEpoch uint64 genesisTimestamp uint64 chainID *big.Int spanBatch *SpanBatch + originChangedBit uint } -func NewSpanBatchBuilder(parentEpoch uint64, genesisTimestamp uint64, chainID *big.Int) *SpanBatchBuilder { +func NewSpanBatchBuilder(genesisTimestamp uint64, chainID *big.Int) *SpanBatchBuilder { return &SpanBatchBuilder{ - parentEpoch: parentEpoch, genesisTimestamp: genesisTimestamp, chainID: chainID, spanBatch: &SpanBatch{}, } } -func (b *SpanBatchBuilder) AppendSingularBatch(singularBatch *SingularBatch) { +func (b *SpanBatchBuilder) AppendSingularBatch(singularBatch *SingularBatch, seqNum uint64) { + if b.GetBlockCount() == 0 { + b.originChangedBit = 0 + if seqNum == 0 { + b.originChangedBit = 1 + } + } b.spanBatch.AppendSingularBatch(singularBatch) } func (b *SpanBatchBuilder) GetRawSpanBatch() (*RawSpanBatch, error) { - originChangedBit := 0 - if uint64(b.spanBatch.GetStartEpochNum()) != b.parentEpoch { - originChangedBit = 1 - } - raw, err := b.spanBatch.ToRawSpanBatch(uint(originChangedBit), b.genesisTimestamp, b.chainID) + raw, err := b.spanBatch.ToRawSpanBatch(b.originChangedBit, b.genesisTimestamp, b.chainID) if err != nil { return nil, err } diff --git a/op-node/rollup/derive/span_batch_test.go b/op-node/rollup/derive/span_batch_test.go index d2a0ac3f8d54..a4929467ac35 100644 --- a/op-node/rollup/derive/span_batch_test.go +++ b/op-node/rollup/derive/span_batch_test.go @@ -493,16 +493,16 @@ func TestSpanBatchBuilder(t *testing.T) { } genesisTimeStamp := 1 + singularBatches[0].Timestamp - 128 - parentEpoch := uint64(singularBatches[0].EpochNum) + var seqNum uint64 = 1 if originChangedBit == 1 { - parentEpoch -= 1 + seqNum = 0 } - spanBatchBuilder := NewSpanBatchBuilder(parentEpoch, genesisTimeStamp, chainID) + spanBatchBuilder := NewSpanBatchBuilder(genesisTimeStamp, chainID) assert.Equal(t, 0, spanBatchBuilder.GetBlockCount()) for i := 0; i < len(singularBatches); i++ { - spanBatchBuilder.AppendSingularBatch(singularBatches[i]) + spanBatchBuilder.AppendSingularBatch(singularBatches[i], seqNum) assert.Equal(t, i+1, spanBatchBuilder.GetBlockCount()) assert.Equal(t, singularBatches[0].ParentHash.Bytes()[:20], spanBatchBuilder.spanBatch.parentCheck) assert.Equal(t, singularBatches[i].EpochHash.Bytes()[:20], spanBatchBuilder.spanBatch.l1OriginCheck) diff --git a/op-node/rollup/derive/span_channel_out.go b/op-node/rollup/derive/span_channel_out.go index d074a19c29ef..608257b7bbb3 100644 --- a/op-node/rollup/derive/span_channel_out.go +++ b/op-node/rollup/derive/span_channel_out.go @@ -70,11 +70,11 @@ func (co *SpanChannelOut) AddBlock(block *types.Block) (uint64, error) { return 0, errors.New("already closed") } - batch, _, err := BlockToSingularBatch(block) + batch, l1Info, err := BlockToSingularBatch(block) if err != nil { return 0, err } - return co.AddSingularBatch(batch) + return co.AddSingularBatch(batch, l1Info.SequenceNumber) } // AddSingularBatch adds a batch to the channel. It returns the RLP encoded byte size @@ -90,7 +90,7 @@ func (co *SpanChannelOut) AddBlock(block *types.Block) (uint64, error) { // A channel can have only one SpanBatch. And compressed results should not be accessible until the channel is closed, since the prefix and payload can be changed. // So it resets channel contents and rewrites the entire SpanBatch each time, and compressed results are copied to reader after the channel is closed. // It makes we can only get frames once the channel is full or closed, in the case of SpanBatch. -func (co *SpanChannelOut) AddSingularBatch(batch *SingularBatch) (uint64, error) { +func (co *SpanChannelOut) AddSingularBatch(batch *SingularBatch, seqNum uint64) (uint64, error) { if co.closed { return 0, errors.New("already closed") } @@ -100,7 +100,7 @@ func (co *SpanChannelOut) AddSingularBatch(batch *SingularBatch) (uint64, error) } var buf bytes.Buffer // Append Singular batch to its span batch builder - co.spanBatchBuilder.AppendSingularBatch(batch) + co.spanBatchBuilder.AppendSingularBatch(batch, seqNum) // Convert Span batch to RawSpanBatch rawSpanBatch, err := co.spanBatchBuilder.GetRawSpanBatch() if err != nil { diff --git a/op-node/rollup/derive/test/random.go b/op-node/rollup/derive/test/random.go index 83ab5835808f..5724fd023f31 100644 --- a/op-node/rollup/derive/test/random.go +++ b/op-node/rollup/derive/test/random.go @@ -1,6 +1,7 @@ package test import ( + "math/big" "math/rand" "github.com/ethereum-optimism/optimism/op-node/rollup/derive" @@ -21,3 +22,13 @@ func RandomL2Block(rng *rand.Rand, txCount int) (*types.Block, []*types.Receipt) } return testutils.RandomBlockPrependTxs(rng, txCount, types.NewTx(l1InfoTx)) } + +func RandomL2BlockWithChainId(rng *rand.Rand, txCount int, chainId *big.Int) *types.Block { + signer := types.NewLondonSigner(chainId) + block, _ := RandomL2Block(rng, 0) + txs := []*types.Transaction{block.Transactions()[0]} // L1 info deposit TX + for i := 0; i < txCount; i++ { + txs = append(txs, testutils.RandomTx(rng, big.NewInt(int64(rng.Uint32())), signer)) + } + return block.WithBody(txs, nil) +} From c622719a9362996d99afa089c82f90fc2e281b6a Mon Sep 17 00:00:00 2001 From: protolambda Date: Tue, 24 Oct 2023 16:20:42 +0200 Subject: [PATCH 066/279] ops: buildx bake and fix ci docker publish/release --- .circleci/config.yml | 456 +++++++++++++++------------------------ Makefile | 13 +- docker-bake.hcl | 170 +++++++++++++++ proxyd/Dockerfile.ignore | 3 + 4 files changed, 359 insertions(+), 283 deletions(-) create mode 100644 docker-bake.hcl create mode 100644 proxyd/Dockerfile.ignore diff --git a/.circleci/config.yml b/.circleci/config.yml index 41f738d70c31..f9a33fdfabb3 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -148,18 +148,13 @@ jobs: environment: DOCKER_BUILDKIT: 1 parameters: - docker_name: - description: Docker image name - type: string docker_tags: - description: Docker image tags as csv - type: string - docker_file: - description: Path to Dockerfile + description: Docker image tags, comma-separated type: string - docker_context: - description: Docker build context + docker_name: + description: "Docker buildx bake target" type: string + default: "" registry: description: Docker registry type: string @@ -168,42 +163,50 @@ jobs: description: Docker repo type: string default: "oplabs-tools-artifacts/images" - build_args: - description: Docker build args - type: string - default: "" - load_base_image: - description: Load docker image as base - type: string - default: "" save_image_tag: - description: Save docker image as + description: Save docker image with given tag type: string default: "" + platforms: + description: Platforms to build for, comma-separated + type: string + default: "linux/amd64" + publish: + description: Publish the docker image (multi-platform, all tags) + type: boolean + default: false + release: + description: Run the release script + type: boolean + default: false machine: image: ubuntu-2204:2022.07.1 resource_class: medium - docker_layer_caching: true + docker_layer_caching: true # we rely on this for faster builds, and actively warm it up for builds with common stages steps: - checkout - - restore_cache: - name: Restore docker build cache - key: docker-build-cache - attach_workspace: at: /tmp/docker_images - run: command: mkdir -p /tmp/docker_images - when: - condition: "<>" + condition: "<>" steps: - - run: - name: Load OP-Stack Go base image - command: | - docker load < "/tmp/docker_images/<>.tar" - - run: - name: build args - command: | - echo "build args: <>" + - gcp-cli/install + - when: + condition: + or: + - "<>" + - "<>" + steps: + - gcp-oidc-authenticate + # Below is CircleCI recommended way of specifying nameservers on an Ubuntu box: + # https://support.circleci.com/hc/en-us/articles/7323511028251-How-to-set-custom-DNS-on-Ubuntu-based-images-using-netplan + - run: sudo sed -i '13 i \ \ \ \ \ \ \ \ \ \ \ \ nameservers:' /etc/netplan/50-cloud-init.yaml + - run: sudo sed -i '14 i \ \ \ \ \ \ \ \ \ \ \ \ \ \ \ addresses:' /etc/netplan/50-cloud-init.yaml + - run: sudo sed -i "s/addresses:/ addresses":" [8.8.8.8, 8.8.4.4] /g" /etc/netplan/50-cloud-init.yaml + - run: cat /etc/netplan/50-cloud-init.yaml + - run: sudo netplan apply - run: name: Build command: | @@ -212,15 +215,43 @@ jobs: if [[ -v DOCKER_HUB_READ_ONLY_TOKEN ]]; then echo "$DOCKER_HUB_READ_ONLY_TOKEN" | docker login -u "$DOCKER_HUB_READ_ONLY_USER" --password-stdin fi - IMAGE_BASE="<>/<>/<>" - DOCKER_TAGS=$(echo -ne <> | sed "s/,/\n/g" | sed "s/[^a-zA-Z0-9\n]/-/g" | sed -e "s|^|-t ${IMAGE_BASE}:|") - IMAGE_BASE_PREFIX="<>/<>" - IMAGE_BASE="$IMAGE_BASE_PREFIX/<>" - docker build --progress plain \ - <> \ - $(echo -ne $DOCKER_TAGS | tr '\n' ' ') \ - -f <> \ - <> + + export REGISTRY="<>" + export REPOSITORY="<>" + export IMAGE_TAGS="<>" + export GIT_COMMIT="$(git rev-parse HEAD)" + export GIT_DATE="$(git show -s --format='%ct')" + export GIT_VERSION="<>" + export PLATFORMS="<>" + + # Create, start (bootstrap) and use a *named* docker builder + # This allows us to cross-build multi-platform, + # and naming allows us to use the DLC (docker-layer-cache) + docker buildx create --driver=docker-container --name=buildx-build --bootstrap --use + + DOCKER_OUTPUT_DESTINATION="--load" + # if we are publishing, change the destination + if [ "<>" == "true" ]; then + DOCKER_OUTPUT_DESTINATION="--push" + echo "Building for platforms $PLATFORMS and then publishing to registry" + else + if [[ $PLATFORMS == *,* ]]; then + echo "ERROR: cannot perform multi-arch build while also loading the result into regular docker" + exit 1 + else + echo "Running single-platform $PLATFORMS build and loading into docker" + fi + fi + + # Let them cook! + docker buildx bake \ + --progress plain \ + --builder=buildx-build \ + -f docker-bake.hcl \ + $DOCKER_OUTPUT_DESTINATION \ + <> + + no_output_timeout: 45m - when: condition: "<>" steps: @@ -233,130 +264,28 @@ jobs: root: /tmp/docker_images paths: # only write the one file, to avoid concurrent workspace-file additions - "<>.tar" - - save_cache: - name: Save docker build cache - key: docker-build-cache - paths: - - "/tmp/docker-build-cache" + - when: + condition: "<>" + steps: + - run: + name: Publish + command: | + gcloud auth configure-docker <> + IMAGE_BASE="<>/<>/<>" + # tags, without the '-t ' here, so we can loop over them + DOCKER_TAGS=$(echo -ne <> | sed "s/,/\n/g" | sed "s/[^a-zA-Z0-9\n]/-/g" | sed -e "s|^|${IMAGE_BASE}:|") + for docker_image_tag in $DOCKER_TAGS; do + docker image push $docker_image_tag + done + no_output_timeout: 45m + - when: + condition: "<>" + steps: + - run: + name: Tag + command: | + ./ops/scripts/ci-docker-tag-op-stack-release.sh <>/<> $CIRCLE_TAG $CIRCLE_SHA1 - docker-publish: - environment: - DOCKER_BUILDKIT: 1 - parameters: - docker_name: - description: Docker image name - type: string - docker_tags: - description: Docker image tags as csv - type: string - docker_file: - description: Path to Dockerfile - type: string - docker_context: - description: Docker build context - type: string - default: "." - docker_target: - description: "target build stage" - type: string - default: "" - registry: - description: Docker registry - type: string - default: "us-docker.pkg.dev" - repo: - description: Docker repo - type: string - default: "oplabs-tools-artifacts/images" - platforms: - description: Platforms to build for - type: string - default: "linux/amd64" - machine: - image: ubuntu-2204:2022.07.1 - resource_class: medium - steps: - - gcp-oidc-authenticate - # Below is CircleCI recommended way of specifying nameservers on an Ubuntu box: - # https://support.circleci.com/hc/en-us/articles/7323511028251-How-to-set-custom-DNS-on-Ubuntu-based-images-using-netplan - - run: sudo sed -i '13 i \ \ \ \ \ \ \ \ \ \ \ \ nameservers:' /etc/netplan/50-cloud-init.yaml - - run: sudo sed -i '14 i \ \ \ \ \ \ \ \ \ \ \ \ \ \ \ addresses:' /etc/netplan/50-cloud-init.yaml - - run: sudo sed -i "s/addresses:/ addresses":" [8.8.8.8, 8.8.4.4] /g" /etc/netplan/50-cloud-init.yaml - - run: cat /etc/netplan/50-cloud-init.yaml - - run: sudo netplan apply - - checkout - - run: - name: Build & Publish - command: | - gcloud auth configure-docker <> - IMAGE_BASE="<>/<>/<>" - DOCKER_TAGS=$(echo -ne <> | sed "s/,/\n/g" | sed "s/[^a-zA-Z0-9\n]/-/g" | sed -e "s|^|-t ${IMAGE_BASE}:|") - docker context create buildx-build - docker buildx create --use buildx-build - docker buildx build --progress plain --platform=<> --target "<>" --push \ - $(echo -ne $DOCKER_TAGS | tr '\n' ' ') \ - -f <> \ - <> - - docker-release: - environment: - DOCKER_BUILDKIT: 1 - parameters: - docker_name: - description: Docker image name - type: string - docker_tags: - description: Docker image tags as csv - type: string - docker_file: - description: Path to Dockerfile - type: string - docker_context: - description: Docker build context - type: string - docker_target: - description: "target build stage" - type: string - default: "" - registry: - description: Docker registry - type: string - default: "us-docker.pkg.dev" - repo: - description: Docker repo - type: string - default: "oplabs-tools-artifacts/images" - platforms: - description: Platforms to build for - type: string - default: "linux/amd64" - machine: - image: ubuntu-2204:2022.07.1 - resource_class: medium - steps: - - gcp-cli/install - - gcp-oidc-authenticate - - checkout - - run: - name: Configure Docker - command: | - gcloud auth configure-docker <> - - run: - name: Build & Publish - command: | - IMAGE_BASE="<>/<>/<>" - DOCKER_TAGS=$(echo -ne <> | sed "s/,/\n/g" | sed "s/[^a-zA-Z0-9\n]/-/g" | sed -e "s|^|-t ${IMAGE_BASE}:|") - docker context create buildx-build - docker buildx create --use buildx-build - docker buildx build --progress plain --platform=<> --target "<>" --push \ - $(echo -ne $DOCKER_TAGS | tr '\n' ' ') \ - -f <> \ - <> - no_output_timeout: 45m - - run: - name: Tag - command: | - ./ops/scripts/ci-docker-tag-op-stack-release.sh <>/<> $CIRCLE_TAG $CIRCLE_SHA1 contracts-bedrock-coverage: docker: @@ -1099,12 +1028,12 @@ jobs: command: | IMAGE_BASE_PREFIX="us-docker.pkg.dev/oplabs-tools-artifacts/images" # Load from previous docker-build job - docker load < "/tmp/workspace/op_stack_go.tar" + docker load < "/tmp/workspace/op-stack-go.tar" docker load < "/tmp/workspace/op-node.tar" docker load < "/tmp/workspace/op-proposer.tar" docker load < "/tmp/workspace/op-batcher.tar" # rename to the tags that the docker-compose of the devnet expects - docker tag "$IMAGE_BASE_PREFIX/op_stack_go:<>" "$IMAGE_BASE_PREFIX/op_stack_go:devnet" + docker tag "$IMAGE_BASE_PREFIX/op-stack-go:<>" "$IMAGE_BASE_PREFIX/op-stack-go:devnet" docker tag "$IMAGE_BASE_PREFIX/op-node:<>" "$IMAGE_BASE_PREFIX/op-node:devnet" docker tag "$IMAGE_BASE_PREFIX/op-proposer:<>" "$IMAGE_BASE_PREFIX/op-proposer:devnet" docker tag "$IMAGE_BASE_PREFIX/op-batcher:<>" "$IMAGE_BASE_PREFIX/op-batcher:devnet" @@ -1473,71 +1402,46 @@ workflows: - op-e2e-WS-tests - op-e2e-HTTP-tests - op-e2e-ext-geth-tests - - docker-build: + - docker-build: # just to warm up the cache (other jobs run in parallel) name: op-stack-go-docker-build - docker_file: ops/docker/op-stack-go/Dockerfile - docker_name: op_stack_go + docker_name: op-stack-go docker_tags: <>,<> - docker_context: . - build_args: "--build-arg GIT_COMMIT=$(git rev-parse HEAD) --build-arg GIT_DATE=$(git show -s --format='%ct')" - save_image_tag: <> # other images builds below depend on this image, so we save it to the workspace - docker-build: name: op-node-docker-build - docker_file: op-node/Dockerfile docker_name: op-node docker_tags: <>,<> - load_base_image: "op_stack_go" - docker_context: . requires: ['op-stack-go-docker-build'] - build_args: --build-arg OP_STACK_GO_BUILDER="$IMAGE_BASE_PREFIX/op_stack_go:<>" save_image_tag: <> # for devnet later - docker-build: name: op-batcher-docker-build - docker_file: op-batcher/Dockerfile docker_name: op-batcher docker_tags: <>,<> - load_base_image: "op_stack_go" - docker_context: . requires: ['op-stack-go-docker-build'] - build_args: --build-arg OP_STACK_GO_BUILDER="$IMAGE_BASE_PREFIX/op_stack_go:<>" save_image_tag: <> # for devnet later - docker-build: name: op-program-docker-build - docker_file: op-program/Dockerfile docker_name: op-program docker_tags: <>,<> - load_base_image: "op_stack_go" - docker_context: . requires: ['op-stack-go-docker-build'] - build_args: --build-arg OP_STACK_GO_BUILDER="$IMAGE_BASE_PREFIX/op_stack_go:<>" + save_image_tag: <> # for devnet later - docker-build: name: op-proposer-docker-build - docker_file: op-proposer/Dockerfile docker_name: op-proposer docker_tags: <>,<> - load_base_image: "op_stack_go" - docker_context: . requires: ['op-stack-go-docker-build'] - build_args: --build-arg OP_STACK_GO_BUILDER="$IMAGE_BASE_PREFIX/op_stack_go:<>" save_image_tag: <> # for devnet later - docker-build: name: op-challenger-docker-build - docker_file: op-challenger/Dockerfile docker_name: op-challenger docker_tags: <>,<> - load_base_image: "op_stack_go" - docker_context: . requires: ['op-stack-go-docker-build'] - build_args: --build-arg OP_STACK_GO_BUILDER="$IMAGE_BASE_PREFIX/op_stack_go:<>" + save_image_tag: <> # for devnet later - docker-build: name: op-heartbeat-docker-build - docker_file: op-heartbeat/Dockerfile docker_name: op-heartbeat docker_tags: <>,<> - load_base_image: "op_stack_go" - docker_context: . requires: ['op-stack-go-docker-build'] - build_args: --build-arg OP_STACK_GO_BUILDER="$IMAGE_BASE_PREFIX/op_stack_go:<>" + save_image_tag: <> # for devnet later - cannon-prestate: requires: ["op-stack-go-lint"] - devnet: @@ -1550,16 +1454,12 @@ workflows: - cannon-prestate - docker-build: name: indexer-docker-build - docker_file: indexer/Dockerfile docker_name: indexer docker_tags: <>,<> - docker_context: . - docker-build: name: ufm-metamask-docker-build - docker_file: ufm-test-services/metamask/Dockerfile docker_name: ufm-metamask docker_tags: <>,<> - docker_context: ufm-test-services/metamask - check-generated-mocks-op-node - check-generated-mocks-op-service - cannon-go-lint-and-test @@ -1577,172 +1477,159 @@ workflows: only: /^(proxyd|indexer|ci-builder|ufm-[a-z0-9\-]*|op-[a-z0-9\-]*)\/v.*/ branches: ignore: /.*/ - - docker-release: + - docker-build: # just to warm up the cache (other jobs run in parallel) + name: op-stack-go-docker-build-release + docker_name: op-stack-go + docker_tags: <> + platforms: "linux/amd64,linux/arm64" + requires: + - hold + - docker-build: name: op-heartbeat-release filters: tags: only: /^op-heartbeat\/v.*/ branches: ignore: /.*/ - docker_file: op-heartbeat/Dockerfile docker_name: op-heartbeat docker_tags: <>,<> - docker_context: . + requires: ['op-stack-go-docker-build-release'] platforms: "linux/amd64,linux/arm64" + publish: true + release: true context: - oplabs-gcr-release - requires: - - hold - - docker-release: + - docker-build: name: op-node-docker-release filters: tags: only: /^op-node\/v.*/ branches: ignore: /.*/ - docker_file: op-node/Dockerfile docker_name: op-node docker_tags: <>,<> - docker_context: . + requires: ['op-stack-go-docker-build-release'] platforms: "linux/amd64,linux/arm64" + publish: true + release: true context: - oplabs-gcr-release - requires: - - hold - - docker-release: + - docker-build: name: op-batcher-docker-release filters: tags: only: /^op-batcher\/v.*/ branches: ignore: /.*/ - docker_file: op-batcher/Dockerfile docker_name: op-batcher docker_tags: <>,<> - docker_context: . + requires: ['op-stack-go-docker-build-release'] platforms: "linux/amd64,linux/arm64" + publish: true + release: true context: - oplabs-gcr-release - requires: - - hold - - docker-release: + - docker-build: name: op-proposer-docker-release filters: tags: only: /^op-proposer\/v.*/ branches: ignore: /.*/ - docker_file: op-proposer/Dockerfile docker_name: op-proposer docker_tags: <>,<> - docker_context: . + requires: ['op-stack-go-docker-build-release'] platforms: "linux/amd64,linux/arm64" + publish: true + release: true context: - oplabs-gcr-release - requires: - - hold - - docker-release: + - docker-build: name: op-challenger-docker-release filters: tags: only: /^op-challenger\/v.*/ branches: ignore: /.*/ - docker_file: op-challenger/Dockerfile docker_name: op-challenger docker_tags: <>,<> - docker_context: . + requires: ['op-stack-go-docker-build-release'] platforms: "linux/amd64,linux/arm64" + publish: true + release: true context: - oplabs-gcr-release - requires: - - hold - docker-build: - name: op-migrate-docker-release - filters: - tags: - only: /^op-migrate\/v.*/ - branches: - ignore: /.*/ - docker_file: op-chain-ops/Dockerfile - docker_name: op-migrate - docker_tags: <>,<> - docker_context: . - context: - - oplabs-gcr-release - requires: - - hold - - docker-release: name: op-ufm-docker-release filters: tags: only: /^op-ufm\/v.*/ branches: ignore: /.*/ - docker_file: op-ufm/Dockerfile docker_name: op-ufm docker_tags: <>,<> - docker_context: . + publish: true + release: true context: - oplabs-gcr-release requires: - hold - - docker-release: + - docker-build: name: proxyd-docker-release filters: tags: only: /^proxyd\/v.*/ branches: ignore: /.*/ - docker_file: proxyd/Dockerfile docker_name: proxyd docker_tags: <>,<> - docker_context: . + publish: true + release: true context: - oplabs-gcr-release requires: - hold - - docker-release: + - docker-build: name: indexer-docker-release filters: tags: only: /^indexer\/v.*/ branches: ignore: /.*/ - docker_file: indexer/Dockerfile docker_name: indexer docker_tags: <>,<> - docker_context: . + publish: true + release: true context: - oplabs-gcr-release requires: - hold - - docker-release: + - docker-build: name: ci-builder-docker-release filters: tags: only: /^ci-builder\/v.*/ branches: ignore: /.*/ - docker_file: ./ops/docker/ci-builder/Dockerfile docker_name: ci-builder docker_tags: <>,latest - docker_context: . + publish: true + release: true context: - oplabs-gcr requires: - hold - - docker-release: + - docker-build: name: ufm-metamask-docker-release filters: tags: only: /^ufm-metamask\/v.*/ branches: ignore: /.*/ - docker_file: ./ufm-test-services/metamask/Dockerfile docker_name: ufm-metamask docker_tags: <>,latest - docker_context: ./ufm-test-services/metamask + publish: true + release: true context: - oplabs-gcr requires: @@ -1777,75 +1664,86 @@ workflows: when: equal: [ build_hourly, <> ] jobs: - - docker-publish: + - docker-build: # just to warm up the cache (other jobs run in parallel) + name: op-stack-go-docker-build-publish + docker_name: op-stack-go + docker_tags: <>,<> + platforms: "linux/amd64,linux/arm64" + context: + - oplabs-gcr + - docker-build: name: op-node-docker-publish docker_name: op-node - docker_file: op-node/Dockerfile docker_tags: <>,<> + requires: [ 'op-stack-go-docker-build-publish' ] + platforms: "linux/amd64,linux/arm64" + publish: true context: - oplabs-gcr - platforms: "linux/amd64,linux/arm64" - - docker-publish: + - docker-build: name: op-batcher-docker-publish - docker_file: op-batcher/Dockerfile docker_name: op-batcher docker_tags: <>,<> + requires: [ 'op-stack-go-docker-build-publish' ] + platforms: "linux/amd64,linux/arm64" + publish: true context: - oplabs-gcr - platforms: "linux/amd64,linux/arm64" - - docker-publish: + - docker-build: name: op-program-docker-publish - docker_file: op-program/Dockerfile docker_name: op-program docker_tags: <>,<> + requires: [ 'op-stack-go-docker-build-publish' ] + platforms: "linux/amd64,linux/arm64" + publish: true context: - oplabs-gcr - platforms: "linux/amd64,linux/arm64" - - docker-publish: + - docker-build: name: op-proposer-docker-publish - docker_file: op-proposer/Dockerfile docker_name: op-proposer docker_tags: <>,<> + requires: [ 'op-stack-go-docker-build-publish' ] + platforms: "linux/amd64,linux/arm64" + publish: true context: - oplabs-gcr - platforms: "linux/amd64,linux/arm64" - - docker-publish: + - docker-build: name: op-challenger-docker-publish - docker_file: op-challenger/Dockerfile docker_name: op-challenger docker_tags: <>,<> + requires: [ 'op-stack-go-docker-build-publish' ] + platforms: "linux/amd64,linux/arm64" + publish: true context: - oplabs-gcr - platforms: "linux/amd64,linux/arm64" - - docker-publish: + - docker-build: name: op-heartbeat-docker-publish - docker_file: op-heartbeat/Dockerfile docker_name: op-heartbeat docker_tags: <>,<> + requires: [ 'op-stack-go-docker-build-publish' ] + platforms: "linux/amd64,linux/arm64" + publish: true context: - oplabs-gcr - platforms: "linux/amd64,linux/arm64" - - docker-publish: + - docker-build: name: indexer-docker-publish - docker_file: indexer/Dockerfile docker_name: indexer docker_tags: <>,<> + publish: true context: - oplabs-gcr platforms: "linux/amd64,linux/arm64" - - docker-publish: + - docker-build: name: chain-mon-docker-publish - docker_file: ./ops/docker/Dockerfile.packages docker_name: chain-mon docker_tags: <>,<> - docker_target: wd-mon + publish: true context: - oplabs-gcr - - docker-publish: + - docker-build: name: ufm-metamask-docker-publish - docker_file: ufm-test-services/metamask/Dockerfile docker_name: ufm-metamask - docker_context: ufm-test-services/metamask docker_tags: <>,<> + publish: true context: - oplabs-gcr diff --git a/Makefile b/Makefile index d8a9ff296425..d4ad574ee421 100644 --- a/Makefile +++ b/Makefile @@ -25,10 +25,15 @@ ci-builder: docker build -t ci-builder -f ops/docker/ci-builder/Dockerfile . golang-docker: - DOCKER_BUILDKIT=1 docker build -t op-stack-go \ - --build-arg GIT_COMMIT=$$(git rev-parse HEAD) \ - --build-arg GIT_DATE=$$(git show -s --format='%ct') \ - -f ops/docker/op-stack-go/Dockerfile . + # We don't use a buildx builder here, and just load directly into regular docker, for convenience. + GIT_COMMIT=$$(git rev-parse HEAD) \ + GIT_DATE=$$(git show -s --format='%ct') \ + IMAGE_TAGS=$$GIT_COMMIT,latest \ + docker buildx bake \ + --progress plain \ + --load \ + -f docker-bake.hcl \ + op-node op-batcher op-proposer op-challenger .PHONY: golang-docker submodules: diff --git a/docker-bake.hcl b/docker-bake.hcl new file mode 100644 index 000000000000..d794b93dc9d9 --- /dev/null +++ b/docker-bake.hcl @@ -0,0 +1,170 @@ +variable "REGISTRY" { + default = "us-docker.pkg.dev" +} + +variable "REPOSITORY" { + default = "oplabs-tools-artifacts/images" +} + +variable "GIT_COMMIT" { + default = "dev" +} + +variable "GIT_DATE" { + default = "0" +} + +variable "GIT_VERSION" { + default = "docker" // original default as set in proxyd file, not used by full go stack, yet +} + +variable "IMAGE_TAGS" { + default = "${GIT_COMMIT}" // split by "," +} + +variable "PLATFORMS" { + // You can override this as "linux/amd64,linux/arm64". + // Only a specify a single platform when `--load` ing into docker. + // Multi-platform is supported when outputting to disk or pushing to a registry. + // Multi-platform builds can be tested locally with: --set="*.output=type=image,push=false" + default = "linux/amd64" +} + +target "op-stack-go" { + dockerfile = "ops/docker/op-stack-go/Dockerfile" + context = "." + args = { + GIT_COMMIT = "${GIT_COMMIT}" + GIT_DATE = "${GIT_DATE}" + } + platforms = split(",", PLATFORMS) + tags = [for tag in split(",", IMAGE_TAGS) : "${REGISTRY}/${REPOSITORY}/op_stack_go:${tag}"] +} + +target "op-node" { + dockerfile = "Dockerfile" + context = "./op-node" + args = { + OP_STACK_GO_BUILDER = "op_stack_go" + } + contexts = { + op_stack_go: "target:op-stack-go" + } + platforms = split(",", PLATFORMS) + tags = [for tag in split(",", IMAGE_TAGS) : "${REGISTRY}/${REPOSITORY}/op-node:${tag}"] +} + +target "op-batcher" { + dockerfile = "Dockerfile" + context = "./op-batcher" + args = { + OP_STACK_GO_BUILDER = "op_stack_go" + } + contexts = { + op_stack_go: "target:op-stack-go" + } + platforms = split(",", PLATFORMS) + tags = [for tag in split(",", IMAGE_TAGS) : "${REGISTRY}/${REPOSITORY}/op-batcher:${tag}"] +} + +target "op-proposer" { + dockerfile = "Dockerfile" + context = "./op-proposer" + args = { + OP_STACK_GO_BUILDER = "op_stack_go" + } + contexts = { + op_stack_go: "target:op-stack-go" + } + platforms = split(",", PLATFORMS) + tags = [for tag in split(",", IMAGE_TAGS) : "${REGISTRY}/${REPOSITORY}/op-proposer:${tag}"] +} + +target "op-challenger" { + dockerfile = "Dockerfile" + context = "./op-challenger" + args = { + OP_STACK_GO_BUILDER = "op_stack_go" + } + contexts = { + op_stack_go: "target:op-stack-go" + } + platforms = split(",", PLATFORMS) + tags = [for tag in split(",", IMAGE_TAGS) : "${REGISTRY}/${REPOSITORY}/op-challenger:${tag}"] +} + +target "op-heartbeat" { + dockerfile = "Dockerfile" + context = "./op-heartbeat" + args = { + OP_STACK_GO_BUILDER = "op_stack_go" + } + contexts = { + op_stack_go: "target:op-stack-go" + } + platforms = split(",", PLATFORMS) + tags = [for tag in split(",", IMAGE_TAGS) : "${REGISTRY}/${REPOSITORY}/op-heartbeat:${tag}"] +} + +target "proxyd" { + dockerfile = "Dockerfile" + context = "./proxyd" + args = { + // proxyd dockerfile has no _ in the args + GITCOMMIT = "${GIT_COMMIT}" + GITDATE = "${GIT_DATE}" + GITVERSION = "${GIT_VERSION}" + } + platforms = split(",", PLATFORMS) + tags = [for tag in split(",", IMAGE_TAGS) : "${REGISTRY}/${REPOSITORY}/proxyd:${tag}"] +} + +target "indexer" { + dockerfile = "./indexer/Dockerfile" + context = "./" + args = { + // proxyd dockerfile has no _ in the args + GITCOMMIT = "${GIT_COMMIT}" + GITDATE = "${GIT_DATE}" + GITVERSION = "${GIT_VERSION}" + } + platforms = split(",", PLATFORMS) + tags = [for tag in split(",", IMAGE_TAGS) : "${REGISTRY}/${REPOSITORY}/proxyd:${tag}"] +} + +target "ufm-metamask" { + dockerfile = "Dockerfile" + context = "./ufm-test-services/metamask" + args = { + // proxyd dockerfile has no _ in the args + GITCOMMIT = "${GIT_COMMIT}" + GITDATE = "${GIT_DATE}" + GITVERSION = "${GIT_VERSION}" + } + platforms = split(",", PLATFORMS) + tags = [for tag in split(",", IMAGE_TAGS) : "${REGISTRY}/${REPOSITORY}/ufm-metamask:${tag}"] +} + +type "chain-mon" { + dockerfile = "./ops/docker/Dockerfile.packages" + context = "." + args = { + // proxyd dockerfile has no _ in the args + GITCOMMIT = "${GIT_COMMIT}" + GITDATE = "${GIT_DATE}" + GITVERSION = "${GIT_VERSION}" + } + // this is a multi-stage build, where each stage is a possible output target, but wd-mon is all we publish + target = "wd-mon" + platforms = split(",", PLATFORMS) + tags = [for tag in split(",", IMAGE_TAGS) : "${REGISTRY}/${REPOSITORY}/chain-mon:${tag}"] +} + +type "ci-builder" { + dockerfile = "Dockerfile" + context = "ops/docker/ci-builder" + platforms = split(",", PLATFORMS) + tags = [for tag in split(",", IMAGE_TAGS) : "${REGISTRY}/${REPOSITORY}/chain-mon:${tag}"] +} + + diff --git a/proxyd/Dockerfile.ignore b/proxyd/Dockerfile.ignore new file mode 100644 index 000000000000..eac1d0bc0b26 --- /dev/null +++ b/proxyd/Dockerfile.ignore @@ -0,0 +1,3 @@ +# ignore everything but proxyd, proxyd defines all its dependencies in the go.mod +* +!/proxyd From 6050d330ed017da1e36092201a5b185744fc92ec Mon Sep 17 00:00:00 2001 From: Hamdi Allam Date: Wed, 25 Oct 2023 11:00:25 -0400 Subject: [PATCH 067/279] comment update --- indexer/node/client.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/indexer/node/client.go b/indexer/node/client.go index 363ddc26f30c..be046f930b48 100644 --- a/indexer/node/client.go +++ b/indexer/node/client.go @@ -198,7 +198,7 @@ type Logs struct { } // FilterLogs returns logs that fit the query parameters. The underlying request is a batch -// request including `eth_getBlockHeaderByNumber` to allow the caller to check that connected +// request including `eth_getBlockByNumber` to allow the caller to check that connected // node has the state necessary to fulfill this request func (c *clnt) FilterLogs(query ethereum.FilterQuery) (Logs, error) { arg, err := toFilterArg(query) From 8358328132512ca974b5bd73dc181c1ca4d623fb Mon Sep 17 00:00:00 2001 From: protolambda Date: Wed, 25 Oct 2023 16:36:07 +0200 Subject: [PATCH 068/279] ops: add op-program docker target --- .circleci/config.yml | 1 + Makefile | 2 +- docker-bake.hcl | 35 +++++++++++++++++++++++----------- op-batcher/Dockerfile | 2 +- op-challenger/Dockerfile | 2 +- op-heartbeat/Dockerfile | 2 +- op-node/Dockerfile | 2 +- op-program/Dockerfile | 2 +- op-proposer/Dockerfile | 2 +- op-wheel/Dockerfile | 2 +- ops-bedrock/docker-compose.yml | 8 ++++---- 11 files changed, 37 insertions(+), 23 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index f9a33fdfabb3..55f571c8b3bb 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -1406,6 +1406,7 @@ workflows: name: op-stack-go-docker-build docker_name: op-stack-go docker_tags: <>,<> + save_image_tag: <> # for devnet later - docker-build: name: op-node-docker-build docker_name: op-node diff --git a/Makefile b/Makefile index d4ad574ee421..8afbb12550db 100644 --- a/Makefile +++ b/Makefile @@ -1,7 +1,7 @@ COMPOSEFLAGS=-d ITESTS_L2_HOST=http://localhost:9545 BEDROCK_TAGS_REMOTE?=origin -OP_STACK_GO_BUILDER?=us-docker.pkg.dev/oplabs-tools-artifacts/images/op_stack_go:latest +OP_STACK_GO_BUILDER?=us-docker.pkg.dev/oplabs-tools-artifacts/images/op-stack-go:latest build: build-go build-ts .PHONY: build diff --git a/docker-bake.hcl b/docker-bake.hcl index d794b93dc9d9..5f5f53123b65 100644 --- a/docker-bake.hcl +++ b/docker-bake.hcl @@ -38,17 +38,17 @@ target "op-stack-go" { GIT_DATE = "${GIT_DATE}" } platforms = split(",", PLATFORMS) - tags = [for tag in split(",", IMAGE_TAGS) : "${REGISTRY}/${REPOSITORY}/op_stack_go:${tag}"] + tags = [for tag in split(",", IMAGE_TAGS) : "${REGISTRY}/${REPOSITORY}/op-stack-go:${tag}"] } target "op-node" { dockerfile = "Dockerfile" context = "./op-node" args = { - OP_STACK_GO_BUILDER = "op_stack_go" + OP_STACK_GO_BUILDER = "op-stack-go" } contexts = { - op_stack_go: "target:op-stack-go" + op-stack-go: "target:op-stack-go" } platforms = split(",", PLATFORMS) tags = [for tag in split(",", IMAGE_TAGS) : "${REGISTRY}/${REPOSITORY}/op-node:${tag}"] @@ -58,10 +58,10 @@ target "op-batcher" { dockerfile = "Dockerfile" context = "./op-batcher" args = { - OP_STACK_GO_BUILDER = "op_stack_go" + OP_STACK_GO_BUILDER = "op-stack-go" } contexts = { - op_stack_go: "target:op-stack-go" + op-stack-go: "target:op-stack-go" } platforms = split(",", PLATFORMS) tags = [for tag in split(",", IMAGE_TAGS) : "${REGISTRY}/${REPOSITORY}/op-batcher:${tag}"] @@ -71,10 +71,10 @@ target "op-proposer" { dockerfile = "Dockerfile" context = "./op-proposer" args = { - OP_STACK_GO_BUILDER = "op_stack_go" + OP_STACK_GO_BUILDER = "op-stack-go" } contexts = { - op_stack_go: "target:op-stack-go" + op-stack-go: "target:op-stack-go" } platforms = split(",", PLATFORMS) tags = [for tag in split(",", IMAGE_TAGS) : "${REGISTRY}/${REPOSITORY}/op-proposer:${tag}"] @@ -84,10 +84,10 @@ target "op-challenger" { dockerfile = "Dockerfile" context = "./op-challenger" args = { - OP_STACK_GO_BUILDER = "op_stack_go" + OP_STACK_GO_BUILDER = "op-stack-go" } contexts = { - op_stack_go: "target:op-stack-go" + op-stack-go: "target:op-stack-go" } platforms = split(",", PLATFORMS) tags = [for tag in split(",", IMAGE_TAGS) : "${REGISTRY}/${REPOSITORY}/op-challenger:${tag}"] @@ -97,15 +97,28 @@ target "op-heartbeat" { dockerfile = "Dockerfile" context = "./op-heartbeat" args = { - OP_STACK_GO_BUILDER = "op_stack_go" + OP_STACK_GO_BUILDER = "op-stack-go" } contexts = { - op_stack_go: "target:op-stack-go" + op-stack-go: "target:op-stack-go" } platforms = split(",", PLATFORMS) tags = [for tag in split(",", IMAGE_TAGS) : "${REGISTRY}/${REPOSITORY}/op-heartbeat:${tag}"] } +target "op-program" { + dockerfile = "Dockerfile" + context = "./op-program" + args = { + OP_STACK_GO_BUILDER = "op-stack-go" + } + contexts = { + op-stack-go: "target:op-stack-go" + } + platforms = split(",", PLATFORMS) + tags = [for tag in split(",", IMAGE_TAGS) : "${REGISTRY}/${REPOSITORY}/op-program:${tag}"] +} + target "proxyd" { dockerfile = "Dockerfile" context = "./proxyd" diff --git a/op-batcher/Dockerfile b/op-batcher/Dockerfile index 028050b7e54c..6732ecc65678 100644 --- a/op-batcher/Dockerfile +++ b/op-batcher/Dockerfile @@ -1,4 +1,4 @@ -ARG OP_STACK_GO_BUILDER=us-docker.pkg.dev/oplabs-tools-artifacts/images/op_stack_go:latest +ARG OP_STACK_GO_BUILDER=us-docker.pkg.dev/oplabs-tools-artifacts/images/op-stack-go:latest FROM $OP_STACK_GO_BUILDER as builder # See "make golang-docker" and /ops/docker/op-stack-go diff --git a/op-challenger/Dockerfile b/op-challenger/Dockerfile index abd2ce3bf850..ad57e2b52b6f 100644 --- a/op-challenger/Dockerfile +++ b/op-challenger/Dockerfile @@ -1,4 +1,4 @@ -ARG OP_STACK_GO_BUILDER=us-docker.pkg.dev/oplabs-tools-artifacts/images/op_stack_go:latest +ARG OP_STACK_GO_BUILDER=us-docker.pkg.dev/oplabs-tools-artifacts/images/op-stack-go:latest FROM $OP_STACK_GO_BUILDER as builder # See "make golang-docker" and /ops/docker/op-stack-go diff --git a/op-heartbeat/Dockerfile b/op-heartbeat/Dockerfile index 4f54b518a591..2e3996cde666 100644 --- a/op-heartbeat/Dockerfile +++ b/op-heartbeat/Dockerfile @@ -1,4 +1,4 @@ -ARG OP_STACK_GO_BUILDER=us-docker.pkg.dev/oplabs-tools-artifacts/images/op_stack_go:latest +ARG OP_STACK_GO_BUILDER=us-docker.pkg.dev/oplabs-tools-artifacts/images/op-stack-go:latest FROM $OP_STACK_GO_BUILDER as builder # See "make golang-docker" and /ops/docker/op-stack-go diff --git a/op-node/Dockerfile b/op-node/Dockerfile index 91903cc5cbf8..0cd7f16d9c1d 100644 --- a/op-node/Dockerfile +++ b/op-node/Dockerfile @@ -1,4 +1,4 @@ -ARG OP_STACK_GO_BUILDER=us-docker.pkg.dev/oplabs-tools-artifacts/images/op_stack_go:latest +ARG OP_STACK_GO_BUILDER=us-docker.pkg.dev/oplabs-tools-artifacts/images/op-stack-go:latest FROM $OP_STACK_GO_BUILDER as builder # See "make golang-docker" and /ops/docker/op-stack-go diff --git a/op-program/Dockerfile b/op-program/Dockerfile index 61f896c789e0..91688b51607d 100644 --- a/op-program/Dockerfile +++ b/op-program/Dockerfile @@ -1,4 +1,4 @@ -ARG OP_STACK_GO_BUILDER=us-docker.pkg.dev/oplabs-tools-artifacts/images/op_stack_go:latest +ARG OP_STACK_GO_BUILDER=us-docker.pkg.dev/oplabs-tools-artifacts/images/op-stack-go:latest FROM $OP_STACK_GO_BUILDER as builder # See "make golang-docker" and /ops/docker/op-stack-go diff --git a/op-proposer/Dockerfile b/op-proposer/Dockerfile index 298eefe5097b..8eb4b7cc0ebb 100644 --- a/op-proposer/Dockerfile +++ b/op-proposer/Dockerfile @@ -1,4 +1,4 @@ -ARG OP_STACK_GO_BUILDER=us-docker.pkg.dev/oplabs-tools-artifacts/images/op_stack_go:latest +ARG OP_STACK_GO_BUILDER=us-docker.pkg.dev/oplabs-tools-artifacts/images/op-stack-go:latest FROM $OP_STACK_GO_BUILDER as builder # See "make golang-docker" and /ops/docker/op-stack-go diff --git a/op-wheel/Dockerfile b/op-wheel/Dockerfile index f6f5ef936e71..30f7a8a3713a 100644 --- a/op-wheel/Dockerfile +++ b/op-wheel/Dockerfile @@ -1,4 +1,4 @@ -ARG OP_STACK_GO_BUILDER=us-docker.pkg.dev/oplabs-tools-artifacts/images/op_stack_go:latest +ARG OP_STACK_GO_BUILDER=us-docker.pkg.dev/oplabs-tools-artifacts/images/op-stack-go:latest FROM $OP_STACK_GO_BUILDER as builder # See "make golang-docker" and /ops/docker/op-stack-go FROM alpine:3.18 diff --git a/ops-bedrock/docker-compose.yml b/ops-bedrock/docker-compose.yml index 73d3a54a827f..e80350b67376 100644 --- a/ops-bedrock/docker-compose.yml +++ b/ops-bedrock/docker-compose.yml @@ -18,7 +18,7 @@ services: args: GIT_COMMIT: "dev" GIT_DATE: "0" - image: us-docker.pkg.dev/oplabs-tools-artifacts/images/op_stack_go:devnet + image: us-docker.pkg.dev/oplabs-tools-artifacts/images/op-stack-go:devnet entrypoint: ["echo", "build complete"] l1: @@ -59,7 +59,7 @@ services: context: ../ dockerfile: ./op-node/Dockerfile args: - OP_STACK_GO_BUILDER: us-docker.pkg.dev/oplabs-tools-artifacts/images/op_stack_go:devnet + OP_STACK_GO_BUILDER: us-docker.pkg.dev/oplabs-tools-artifacts/images/op-stack-go:devnet image: us-docker.pkg.dev/oplabs-tools-artifacts/images/op-node:devnet command: > op-node @@ -107,7 +107,7 @@ services: context: ../ dockerfile: ./op-proposer/Dockerfile args: - OP_STACK_GO_BUILDER: us-docker.pkg.dev/oplabs-tools-artifacts/images/op_stack_go:devnet + OP_STACK_GO_BUILDER: us-docker.pkg.dev/oplabs-tools-artifacts/images/op-stack-go:devnet image: us-docker.pkg.dev/oplabs-tools-artifacts/images/op-proposer:devnet ports: - "6062:6060" @@ -136,7 +136,7 @@ services: context: ../ dockerfile: ./op-batcher/Dockerfile args: - OP_STACK_GO_BUILDER: us-docker.pkg.dev/oplabs-tools-artifacts/images/op_stack_go:devnet + OP_STACK_GO_BUILDER: us-docker.pkg.dev/oplabs-tools-artifacts/images/op-stack-go:devnet image: us-docker.pkg.dev/oplabs-tools-artifacts/images/op-batcher:devnet ports: - "6061:6060" From 284f3a4c61546019a76c3d3d27b0d174bc9fa8f8 Mon Sep 17 00:00:00 2001 From: protolambda Date: Mon, 23 Oct 2023 10:16:06 +0200 Subject: [PATCH 069/279] devnet: set recommit time to 100ms --- ops-bedrock/docker-compose.yml | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/ops-bedrock/docker-compose.yml b/ops-bedrock/docker-compose.yml index 73d3a54a827f..691fed8deafe 100644 --- a/ops-bedrock/docker-compose.yml +++ b/ops-bedrock/docker-compose.yml @@ -33,6 +33,8 @@ services: - "l1_data:/db" - "${PWD}/../.devnet/genesis-l1.json:/genesis.json" - "${PWD}/test-jwt-secret.txt:/config/test-jwt-secret.txt" + environment: + GETH_MINER_RECOMMIT: 100ms l2: build: @@ -49,6 +51,8 @@ services: - "/bin/sh" - "/entrypoint.sh" - "--authrpc.jwtsecret=/config/test-jwt-secret.txt" + environment: + GETH_MINER_RECOMMIT: 100ms op-node: depends_on: From 1a563c71b0ee8174cd4d1e6512d86f88bc627b46 Mon Sep 17 00:00:00 2001 From: protolambda Date: Mon, 23 Oct 2023 13:16:36 +0200 Subject: [PATCH 070/279] ci: skip second docker build in devnet branch of workflow --- .circleci/config.yml | 1 + bedrock-devnet/devnet/__init__.py | 20 ++++++++++++-------- 2 files changed, 13 insertions(+), 8 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index 41f738d70c31..0dcd0f3db831 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -1031,6 +1031,7 @@ jobs: image: ubuntu-2204:2022.10.2 environment: DOCKER_BUILDKIT: 1 + DEVNET_NO_BUILD: 'true' steps: - checkout - check-changed: diff --git a/bedrock-devnet/devnet/__init__.py b/bedrock-devnet/devnet/__init__.py index a0b1afec26e8..463e9f6d316d 100644 --- a/bedrock-devnet/devnet/__init__.py +++ b/bedrock-devnet/devnet/__init__.py @@ -97,14 +97,18 @@ def main(): git_commit = subprocess.run(['git', 'rev-parse', 'HEAD'], capture_output=True, text=True).stdout.strip() git_date = subprocess.run(['git', 'show', '-s', "--format=%ct"], capture_output=True, text=True).stdout.strip() - log.info(f'Building docker images for git commit {git_commit} ({git_date})') - run_command(['docker', 'compose', 'build', '--progress', 'plain', - '--build-arg', f'GIT_COMMIT={git_commit}', '--build-arg', f'GIT_DATE={git_date}'], - cwd=paths.ops_bedrock_dir, env={ - 'PWD': paths.ops_bedrock_dir, - 'DOCKER_BUILDKIT': '1', # (should be available by default in later versions, but explicitly enable it anyway) - 'COMPOSE_DOCKER_CLI_BUILD': '1' # use the docker cache - }) + # CI loads the images from workspace, and does not otherwise know the images are good as-is + if os.getenv('DEVNET_NO_BUILD') == "true": + log.info('Skipping docker images build') + else: + log.info(f'Building docker images for git commit {git_commit} ({git_date})') + run_command(['docker', 'compose', 'build', '--progress', 'plain', + '--build-arg', f'GIT_COMMIT={git_commit}', '--build-arg', f'GIT_DATE={git_date}'], + cwd=paths.ops_bedrock_dir, env={ + 'PWD': paths.ops_bedrock_dir, + 'DOCKER_BUILDKIT': '1', # (should be available by default in later versions, but explicitly enable it anyway) + 'COMPOSE_DOCKER_CLI_BUILD': '1' # use the docker cache + }) log.info('Devnet starting') devnet_deploy(paths) From 54b70f51b61b671aa2abe138dcc0a3159c6513de Mon Sep 17 00:00:00 2001 From: protolambda Date: Mon, 23 Oct 2023 15:40:48 +0200 Subject: [PATCH 071/279] devnet: eth/erc20 test in parallel with different signers and annotated output --- bedrock-devnet/devnet/__init__.py | 67 ++++++++++++++++++++++++----- packages/sdk/hardhat.config.ts | 22 +++++++++- packages/sdk/tasks/deposit-erc20.ts | 16 ++++--- packages/sdk/tasks/deposit-eth.ts | 9 ++-- 4 files changed, 93 insertions(+), 21 deletions(-) diff --git a/bedrock-devnet/devnet/__init__.py b/bedrock-devnet/devnet/__init__.py index 463e9f6d316d..54a98747bb2e 100644 --- a/bedrock-devnet/devnet/__init__.py +++ b/bedrock-devnet/devnet/__init__.py @@ -10,6 +10,9 @@ import shutil import http.client from multiprocessing import Process, Queue +import concurrent.futures +from collections import namedtuple + import devnet.log_setup @@ -301,6 +304,10 @@ def wait_for_rpc_server(url): log.info(f'Waiting for RPC server at {url}') time.sleep(1) + +CommandPreset = namedtuple('Command', ['name', 'args', 'cwd', 'timeout']) + + def devnet_test(paths): # Check the L2 config run_command( @@ -308,17 +315,57 @@ def devnet_test(paths): cwd=paths.ops_chain_ops, ) - run_command( - ['npx', 'hardhat', 'deposit-erc20', '--network', 'devnetL1', '--l1-contracts-json-path', paths.addresses_json_path], - cwd=paths.sdk_dir, - timeout=8*60, - ) + # Run the two commands with different signers, so the ethereum nonce management does not conflict + # And do not use devnet system addresses, to avoid breaking fee-estimation or nonce values. + run_commands([ + CommandPreset('erc20-test', + ['npx', 'hardhat', 'deposit-erc20', '--network', 'devnetL1', + '--l1-contracts-json-path', paths.addresses_json_path, '--signer-index', '14'], + cwd=paths.sdk_dir, timeout=8*60), + CommandPreset('eth-test', + ['npx', 'hardhat', 'deposit-eth', '--network', 'devnetL1', + '--l1-contracts-json-path', paths.addresses_json_path, '--signer-index', '15'], + cwd=paths.sdk_dir, timeout=8*60) + ], max_workers=2) + + +def run_commands(commands: list[CommandPreset], max_workers=2): + with concurrent.futures.ThreadPoolExecutor(max_workers=max_workers) as executor: + futures = [executor.submit(run_command_preset, cmd) for cmd in commands] + + for future in concurrent.futures.as_completed(futures): + result = future.result() + if result: + print(result.stdout) + + +def run_command_preset(command: CommandPreset): + with subprocess.Popen(command.args, cwd=command.cwd, + stdout=subprocess.PIPE, stderr=subprocess.PIPE, text=True) as proc: + try: + # Live output processing + for line in proc.stdout: + # Annotate and print the line with timestamp and command name + timestamp = datetime.datetime.utcnow().strftime('%H:%M:%S.%f') + # Annotate and print the line with the timestamp + print(f"[{timestamp}][{command.name}] {line}", end='') + + stdout, stderr = proc.communicate(timeout=command.timeout) + + if proc.returncode != 0: + raise RuntimeError(f"Command '{' '.join(command.args)}' failed with return code {proc.returncode}: {stderr}") + + except subprocess.TimeoutExpired: + raise RuntimeError(f"Command '{' '.join(command.args)}' timed out!") + + except Exception as e: + raise RuntimeError(f"Error executing '{' '.join(command.args)}': {e}") + + finally: + # Ensure process is terminated + proc.kill() + return proc.returncode - run_command( - ['npx', 'hardhat', 'deposit-eth', '--network', 'devnetL1', '--l1-contracts-json-path', paths.addresses_json_path], - cwd=paths.sdk_dir, - timeout=8*60, - ) def run_command(args, check=True, shell=False, cwd=None, env=None, timeout=None): env = env if env else {} diff --git a/packages/sdk/hardhat.config.ts b/packages/sdk/hardhat.config.ts index 7b3fedc4b780..717317b8a933 100644 --- a/packages/sdk/hardhat.config.ts +++ b/packages/sdk/hardhat.config.ts @@ -24,7 +24,27 @@ const config: HardhatUserConfig = { devnetL1: { url: 'http://localhost:8545', accounts: [ - 'ac0974bec39a17e36ba4a6b4d238ff944bacb478cbed5efcae784d7bf4f2ff80', + // warning: keys 0 - 12 (incl) are used by the system + 'ac0974bec39a17e36ba4a6b4d238ff944bacb478cbed5efcae784d7bf4f2ff80', // 0 + '59c6995e998f97a5a0044966f0945389dc9e86dae88c7a8412f4603b6b78690d', // 1 + '5de4111afa1a4b94908f83103eb1f1706367c2e68ca870fc3fb9a804cdab365a', // 2 + '7c852118294e51e653712a81e05800f419141751be58f605c371e15141b007a6', // 3 + '47e179ec197488593b187f80a00eb0da91f1b9d0b13f8733639f19c30a34926a', // 4 + '8b3a350cf5c34c9194ca85829a2df0ec3153be0318b5e2d3348e872092edffba', // 5 + '92db14e403b83dfe3df233f83dfa3a0d7096f21ca9b0d6d6b8d88b2b4ec1564e', // 6 + '4bbbf85ce3377467afe5d46f804f221813b2bb87f24d81f60f1fcdbf7cbf4356', // 7 + 'dbda1821b80551c9d65939329250298aa3472ba22feea921c0cf5d620ea67b97', // 8 + '2a871d0798f97d79848a013d4936a73bf4cc922c825d33c1cf7073dff6d409c6', // 9 + 'f214f2b2cd398c806f84e317254e0f0b801d0643303237d97a22a48e01628897', // 10 + '701b615bbdfb9de65240bc28bd21bbc0d996645a3dd57e7b12bc2bdf6f192c82', // 11 + 'a267530f49f8280200edf313ee7af6b827f2a8bce2897751d06a843f644967b1', // 12 + '47c99abed3324a2707c28affff1267e45918ec8c3f20b8aa892e8b065d2942dd', // 13 + 'c526ee95bf44d8fc405a158bb884d9d1238d99f0612e9f33d006bb0789009aaa', // 14 + '8166f546bab6da521a8369cab06c5d2b9e46670292d85c875ee9ec20e84ffb61', // 15 + 'ea6c44ac03bff858b476bba40716402b03e41b8e97e276d1baec7c37d42484a0', // 16 + '689af8efa8c651a91ad287602527f3af2fe9f6501a7ac4b061667b5a93e037fd', // 17 + 'de9be858da4a475276426320d5e9262ecfc3ba460bfac56360bfa6c4c28b4ee0', // 18 + 'df57089febbacf7ba0bc227dafbffa9fc08a93fdc68e1e42411a14efcf23656e', // 19 ], }, hivenet: { diff --git a/packages/sdk/tasks/deposit-erc20.ts b/packages/sdk/tasks/deposit-erc20.ts index d96cd60f4e1e..cad8bd5d16c4 100644 --- a/packages/sdk/tasks/deposit-erc20.ts +++ b/packages/sdk/tasks/deposit-erc20.ts @@ -2,6 +2,7 @@ import { promises as fs } from 'fs' import { task, types } from 'hardhat/config' import { HardhatRuntimeEnvironment } from 'hardhat/types' +import { SignerWithAddress } from '@nomiclabs/hardhat-ethers/signers' import '@nomiclabs/hardhat-ethers' import 'hardhat-deploy' import { Event, Contract, Wallet, providers, utils, ethers } from 'ethers' @@ -27,11 +28,9 @@ import { const deployWETH9 = async ( hre: HardhatRuntimeEnvironment, + signer: SignerWithAddress, wrap: boolean ): Promise => { - const signers = await hre.ethers.getSigners() - const signer = signers[0] - const Factory__WETH9 = new hre.ethers.ContractFactory( Artifact__WETH9.abi, Artifact__WETH9.bytecode.object, @@ -117,13 +116,16 @@ task('deposit-erc20', 'Deposits WETH9 onto L2.') '', types.string ) + .addOptionalParam('signerIndex', 'Index of signer to use', 0, types.int) .setAction(async (args, hre) => { const signers = await hre.ethers.getSigners() if (signers.length === 0) { throw new Error('No configured signers') } - // Use the first configured signer for simplicity - const signer = signers[0] + if (args.signerIndex < 0 || signers.length <= args.signerIndex) { + throw new Error('Invalid signer index') + } + const signer = signers[args.signerIndex] const address = await signer.getAddress() console.log(`Using signer ${address}`) @@ -137,7 +139,7 @@ task('deposit-erc20', 'Deposits WETH9 onto L2.') const l2Provider = new providers.StaticJsonRpcProvider(args.l2ProviderUrl) const l2Signer = new hre.ethers.Wallet( - hre.network.config.accounts[0], + hre.network.config.accounts[args.signerIndex], l2Provider ) @@ -219,7 +221,7 @@ task('deposit-erc20', 'Deposits WETH9 onto L2.') console.log(params) console.log('Deploying WETH9 to L1') - const WETH9 = await deployWETH9(hre, true) + const WETH9 = await deployWETH9(hre, signer, true) console.log(`Deployed to ${WETH9.address}`) console.log('Creating L2 WETH9') diff --git a/packages/sdk/tasks/deposit-eth.ts b/packages/sdk/tasks/deposit-eth.ts index fe298a617699..62a9539f8ded 100644 --- a/packages/sdk/tasks/deposit-eth.ts +++ b/packages/sdk/tasks/deposit-eth.ts @@ -50,14 +50,17 @@ task('deposit-eth', 'Deposits ether to L2.') '', types.string ) + .addOptionalParam('signerIndex', 'Index of signer to use', 0, types.int) .addOptionalParam('withdrawAmount', 'Amount to withdraw', '', types.string) .setAction(async (args, hre) => { const signers = await hre.ethers.getSigners() if (signers.length === 0) { throw new Error('No configured signers') } - // Use the first configured signer for simplicity - const signer = signers[0] + if (args.signerIndex < 0 || signers.length <= args.signerIndex) { + throw new Error('Invalid signer index') + } + const signer = signers[args.signerIndex] const address = await signer.getAddress() console.log(`Using signer ${address}`) @@ -81,7 +84,7 @@ task('deposit-eth', 'Deposits ether to L2.') : amount.div(2) const l2Signer = new hre.ethers.Wallet( - hre.network.config.accounts[0], + hre.network.config.accounts[args.signerIndex], l2Provider ) From efabb10a7a2fc4af6f56bb184a3cf669b914b668 Mon Sep 17 00:00:00 2001 From: protolambda Date: Wed, 25 Oct 2023 17:43:07 +0200 Subject: [PATCH 072/279] devnet: increase l2OutputOracleSubmissionInterval to 10 for proposer to not fall behind --- packages/contracts-bedrock/deploy-config/devnetL1-template.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/packages/contracts-bedrock/deploy-config/devnetL1-template.json b/packages/contracts-bedrock/deploy-config/devnetL1-template.json index 10badc81e0df..76a51bacd7fa 100644 --- a/packages/contracts-bedrock/deploy-config/devnetL1-template.json +++ b/packages/contracts-bedrock/deploy-config/devnetL1-template.json @@ -11,7 +11,7 @@ "cliqueSignerAddress": "0xf39Fd6e51aad88F6F4ce6aB8827279cffFb92266", "l1UseClique": true, "l1StartingBlockTag": "earliest", - "l2OutputOracleSubmissionInterval": 6, + "l2OutputOracleSubmissionInterval": 10, "l2OutputOracleStartingTimestamp": 0, "l2OutputOracleStartingBlockNumber": 0, "l2OutputOracleProposer": "0x70997970C51812dc3A010C7d01b50e0d17dc79C8", From ad96f1c2ea404444335c9b723669a9246317f6c5 Mon Sep 17 00:00:00 2001 From: Will Cory Date: Wed, 25 Oct 2023 08:47:21 -0700 Subject: [PATCH 073/279] fix: use 20.8.1 instead of 20.9.0 for bullseye slim image --- ops/docker/Dockerfile.packages | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ops/docker/Dockerfile.packages b/ops/docker/Dockerfile.packages index cb82dc365011..d84a3726ac35 100644 --- a/ops/docker/Dockerfile.packages +++ b/ops/docker/Dockerfile.packages @@ -35,7 +35,7 @@ FROM us-docker.pkg.dev/oplabs-tools-artifacts/images/ci-builder:latest as foundr # we use it rather than alpine because it's not much # bigger and alpine is often missing packages for node applications # alpine is not officially supported by node.js -FROM node:20.9.0-bullseye-slim as base +FROM node:20.8.1-bullseye-slim as base # Base: install deps RUN apt-get update && apt-get install -y \ From c2cc1996e6438b980e77ebccba7fce06213b5c04 Mon Sep 17 00:00:00 2001 From: inphi Date: Wed, 25 Oct 2023 10:46:36 -0400 Subject: [PATCH 074/279] ci: re-enable ctb codecov And increase the resource class of the contract-bedrock-tests job --- .circleci/config.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index 41f738d70c31..314af130995c 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -361,7 +361,7 @@ jobs: contracts-bedrock-coverage: docker: - image: us-docker.pkg.dev/oplabs-tools-artifacts/images/ci-builder:latest - resource_class: large + resource_class: xlarge steps: - checkout - check-changed: @@ -372,7 +372,7 @@ jobs: working_directory: packages/contracts-bedrock - run: name: test and generate coverage - command: pnpm coverage:lcov || true + command: pnpm coverage:lcov no_output_timeout: 18m environment: FOUNDRY_PROFILE: ci From 4ed9bf02c7c1feab8f3716388cc23e3b7cd1be43 Mon Sep 17 00:00:00 2001 From: Will Cory Date: Wed, 25 Oct 2023 09:40:22 -0700 Subject: [PATCH 075/279] chore(chain-mon): Add dev scripts for chain mon --- packages/chain-mon/package.json | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/packages/chain-mon/package.json b/packages/chain-mon/package.json index 33fc16d63883..829c69f46321 100644 --- a/packages/chain-mon/package.json +++ b/packages/chain-mon/package.json @@ -9,6 +9,12 @@ "dist/*" ], "scripts": { + "dev:balance-mon": "tsx watch ./src/balance-mon/service.ts", + "dev:wallet-mon": "tsx watch ./src/wallet-mon/service.ts", + "dev:drippie-mon": "tsx watch ./src/drippie-mon/service.ts", + "dev:wd-mon": "tsx watch ./src/wd-mon/service.ts", + "dev:fault-mon": "tsx watch ./src/fault-mon/service.ts", + "dev:replica-mon": "tsx watch ./src/replica-mon/service.ts", "start:balance-mon": "tsx ./src/balance-mon/service.ts", "start:wallet-mon": "tsx ./src/wallet-mon/service.ts", "start:drippie-mon": "tsx ./src/drippie-mon/service.ts", From 64005ab47f7fcd5c71811e4c344bfdf49b28ece9 Mon Sep 17 00:00:00 2001 From: inphi Date: Wed, 25 Oct 2023 12:10:55 -0400 Subject: [PATCH 076/279] ci: fix ctb path in codecov.yml --- codecov.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/codecov.yml b/codecov.yml index 557c0f65aedf..88f4b4ee729b 100644 --- a/codecov.yml +++ b/codecov.yml @@ -22,7 +22,7 @@ flag_management: individual_flags: - name: contracts-bedrock-tests paths: - - packages/contracts-bedrock/contracts + - packages/contracts-bedrock statuses: - type: patch target: 100% From d2ad25efde8b5ffe310e16baaff4e97dc4a4ae53 Mon Sep 17 00:00:00 2001 From: Luis Marcano Date: Wed, 25 Oct 2023 12:04:19 -0400 Subject: [PATCH 077/279] fix race condition fix: import formatting; LastUpdate as atomic.Int64 fix ci/cd --- op-node/p2p/store/scorebook.go | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/op-node/p2p/store/scorebook.go b/op-node/p2p/store/scorebook.go index 81def53685fc..68043ef8a9e1 100644 --- a/op-node/p2p/store/scorebook.go +++ b/op-node/p2p/store/scorebook.go @@ -2,6 +2,7 @@ package store import ( "context" + "sync/atomic" "time" "github.com/ethereum-optimism/optimism/op-service/clock" @@ -17,17 +18,18 @@ const ( var scoresBase = ds.NewKey("/peers/scores") +// LastUpdate requires atomic update operations. Use the helper functions SetLastUpdated and LastUpdated to modify and access this field. type scoreRecord struct { - PeerScores PeerScores `json:"peerScores"` LastUpdate int64 `json:"lastUpdate"` // unix timestamp in seconds + PeerScores PeerScores `json:"peerScores"` } func (s *scoreRecord) SetLastUpdated(t time.Time) { - s.LastUpdate = t.Unix() + atomic.StoreInt64(&s.LastUpdate, t.Unix()) } func (s *scoreRecord) LastUpdated() time.Time { - return time.Unix(s.LastUpdate, 0) + return time.Unix(atomic.LoadInt64(&s.LastUpdate), 0) } func (s *scoreRecord) MarshalBinary() (data []byte, err error) { From cc1829df01949d4399ac06600b161422f2b9a6d6 Mon Sep 17 00:00:00 2001 From: protolambda Date: Wed, 25 Oct 2023 22:01:44 +0200 Subject: [PATCH 078/279] ops: fix docker bake --- docker-bake.hcl | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker-bake.hcl b/docker-bake.hcl index 5f5f53123b65..afdc5996d25c 100644 --- a/docker-bake.hcl +++ b/docker-bake.hcl @@ -142,7 +142,7 @@ target "indexer" { GITVERSION = "${GIT_VERSION}" } platforms = split(",", PLATFORMS) - tags = [for tag in split(",", IMAGE_TAGS) : "${REGISTRY}/${REPOSITORY}/proxyd:${tag}"] + tags = [for tag in split(",", IMAGE_TAGS) : "${REGISTRY}/${REPOSITORY}/indexer:${tag}"] } target "ufm-metamask" { @@ -177,7 +177,7 @@ type "ci-builder" { dockerfile = "Dockerfile" context = "ops/docker/ci-builder" platforms = split(",", PLATFORMS) - tags = [for tag in split(",", IMAGE_TAGS) : "${REGISTRY}/${REPOSITORY}/chain-mon:${tag}"] + tags = [for tag in split(",", IMAGE_TAGS) : "${REGISTRY}/${REPOSITORY}/ci-builder:${tag}"] } From 9407d90cf0b08487353d9c06e3a8d43ca272ea69 Mon Sep 17 00:00:00 2001 From: protolambda Date: Wed, 25 Oct 2023 22:02:50 +0200 Subject: [PATCH 079/279] github: add docker-bake codeowner --- .github/CODEOWNERS | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/.github/CODEOWNERS b/.github/CODEOWNERS index 5c51a08822e9..a142b138a992 100644 --- a/.github/CODEOWNERS +++ b/.github/CODEOWNERS @@ -25,9 +25,10 @@ /ops-bedrock @ethereum-optimism/go-reviewers # Ops -/.circleci @ethereum-optimism/infra-reviewers -/.github @ethereum-optimism/infra-reviewers -/ops @ethereum-optimism/infra-reviewers +/.circleci @ethereum-optimism/infra-reviewers +/.github @ethereum-optimism/infra-reviewers +/ops @ethereum-optimism/infra-reviewers +/docker-bake.hcl @ethereum-optimism/infra-reviewers # Misc /proxyd @ethereum-optimism/infra-reviewers From 5e7e08c61ff19cbc5b9e962daf30fabd06016140 Mon Sep 17 00:00:00 2001 From: protolambda Date: Wed, 25 Oct 2023 22:57:49 +0200 Subject: [PATCH 080/279] ops: fix image branch format, filter out special characters --- .circleci/config.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index f5d8c330fb0d..9b3835a12ab4 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -218,7 +218,7 @@ jobs: export REGISTRY="<>" export REPOSITORY="<>" - export IMAGE_TAGS="<>" + export IMAGE_TAGS=$(echo "<>" | sed "s/[^a-zA-Z0-9\n,]/-/g") export GIT_COMMIT="$(git rev-parse HEAD)" export GIT_DATE="$(git show -s --format='%ct')" export GIT_VERSION="<>" From 50b4767162bbebb5a7decceeebb7d1cf57db4317 Mon Sep 17 00:00:00 2001 From: protolambda Date: Wed, 25 Oct 2023 23:10:58 +0200 Subject: [PATCH 081/279] op-node: span-batch logging stye fixes Co-authored-by: Adrian Sutton --- op-node/rollup/derive/batch_queue.go | 2 +- op-node/rollup/derive/batches.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/op-node/rollup/derive/batch_queue.go b/op-node/rollup/derive/batch_queue.go index a340bc171bff..b12f08c54fbd 100644 --- a/op-node/rollup/derive/batch_queue.go +++ b/op-node/rollup/derive/batch_queue.go @@ -247,7 +247,7 @@ batchLoop: remaining = append(remaining, batch) continue case BatchDrop: - batch.Batch.LogContext(bq.log).Warn("dropping batch", + batch.Batch.LogContext(bq.log).Warn("Dropping batch", "l2_safe_head", l2SafeHead.ID(), "l2_safe_head_time", l2SafeHead.Time, ) diff --git a/op-node/rollup/derive/batches.go b/op-node/rollup/derive/batches.go index 73020fa4079f..e618fcc7433a 100644 --- a/op-node/rollup/derive/batches.go +++ b/op-node/rollup/derive/batches.go @@ -53,7 +53,7 @@ func CheckBatch(ctx context.Context, cfg *rollup.Config, log log.Logger, l1Block } return checkSpanBatch(ctx, cfg, log, l1Blocks, l2SafeHead, spanBatch, batch.L1InclusionBlock, l2Fetcher) default: - log.Warn("unrecognized batch type: %d", batch.Batch.GetBatchType()) + log.Warn("Unrecognized batch type: %d", batch.Batch.GetBatchType()) return BatchDrop } } From 6416a98cfc6b4d3650cf97f10d70639f0e32fc21 Mon Sep 17 00:00:00 2001 From: protolambda Date: Wed, 25 Oct 2023 23:27:36 +0200 Subject: [PATCH 082/279] ops: add missing quotes --- .circleci/config.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index 9b3835a12ab4..3209e8f16fbb 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -218,7 +218,7 @@ jobs: export REGISTRY="<>" export REPOSITORY="<>" - export IMAGE_TAGS=$(echo "<>" | sed "s/[^a-zA-Z0-9\n,]/-/g") + export IMAGE_TAGS="$(echo -ne "<>" | sed "s/[^a-zA-Z0-9\n,]/-/g")" export GIT_COMMIT="$(git rev-parse HEAD)" export GIT_DATE="$(git show -s --format='%ct')" export GIT_VERSION="<>" @@ -273,7 +273,7 @@ jobs: gcloud auth configure-docker <> IMAGE_BASE="<>/<>/<>" # tags, without the '-t ' here, so we can loop over them - DOCKER_TAGS=$(echo -ne <> | sed "s/,/\n/g" | sed "s/[^a-zA-Z0-9\n]/-/g" | sed -e "s|^|${IMAGE_BASE}:|") + DOCKER_TAGS="$(echo -ne "<>" | sed "s/,/\n/g" | sed "s/[^a-zA-Z0-9\n]/-/g" | sed -e "s|^|${IMAGE_BASE}:|")" for docker_image_tag in $DOCKER_TAGS; do docker image push $docker_image_tag done From a172c7d962f3fd632c73c4ea0d84999d6a53a1e5 Mon Sep 17 00:00:00 2001 From: Mark Tyneway Date: Wed, 25 Oct 2023 14:54:14 -0600 Subject: [PATCH 083/279] contracts-bedrock: scripts absolute import Import the scripts files using relative absolute imports from the project root. This makes the file much more portable and follows the conventions that the rest of the repo uses. --- packages/contracts-bedrock/scripts/Deploy.s.sol | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/packages/contracts-bedrock/scripts/Deploy.s.sol b/packages/contracts-bedrock/scripts/Deploy.s.sol index ca494ccb8a9b..c293825a359d 100644 --- a/packages/contracts-bedrock/scripts/Deploy.s.sol +++ b/packages/contracts-bedrock/scripts/Deploy.s.sol @@ -10,8 +10,8 @@ import { Safe } from "safe-contracts/Safe.sol"; import { SafeProxyFactory } from "safe-contracts/proxies/SafeProxyFactory.sol"; import { Enum as SafeOps } from "safe-contracts/common/Enum.sol"; -import { Deployer } from "./Deployer.sol"; -import { DeployConfig } from "./DeployConfig.s.sol"; +import { Deployer } from "scripts/Deployer.sol"; +import { DeployConfig } from "scripts/DeployConfig.s.sol"; import { Safe } from "safe-contracts/Safe.sol"; import { SafeProxyFactory } from "safe-contracts/proxies/SafeProxyFactory.sol"; @@ -37,7 +37,7 @@ import { L1ERC721Bridge } from "src/L1/L1ERC721Bridge.sol"; import { ProtocolVersions, ProtocolVersion } from "src/L1/ProtocolVersions.sol"; import { StorageSetter } from "src/universal/StorageSetter.sol"; import { Predeploys } from "src/libraries/Predeploys.sol"; -import { Chains } from "./Chains.sol"; +import { Chains } from "scripts/Chains.sol"; import { IBigStepper } from "src/dispute/interfaces/IBigStepper.sol"; import { IPreimageOracle } from "src/cannon/interfaces/IPreimageOracle.sol"; From a1af32dcb5de34141137614af339cf9305e89d8a Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 25 Oct 2023 22:18:47 +0000 Subject: [PATCH 084/279] build(deps): bump viem from 1.16.6 to 1.17.1 Bumps [viem](https://github.com/wagmi-dev/viem) from 1.16.6 to 1.17.1. - [Release notes](https://github.com/wagmi-dev/viem/releases) - [Commits](https://github.com/wagmi-dev/viem/compare/viem@1.16.6...viem@1.17.1) --- updated-dependencies: - dependency-name: viem dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] --- packages/contracts-ts/package.json | 2 +- packages/fee-estimation/package.json | 2 +- packages/sdk/package.json | 2 +- packages/web3js-plugin/package.json | 2 +- pnpm-lock.yaml | 62 ++++++++++++++-------------- 5 files changed, 35 insertions(+), 35 deletions(-) diff --git a/packages/contracts-ts/package.json b/packages/contracts-ts/package.json index 47679477d2ba..5fbd9a45925d 100644 --- a/packages/contracts-ts/package.json +++ b/packages/contracts-ts/package.json @@ -82,6 +82,6 @@ "change-case": "4.1.2", "react": "^18.2.0", "react-dom": "^18.2.0", - "viem": "^1.16.6" + "viem": "^1.17.1" } } diff --git a/packages/fee-estimation/package.json b/packages/fee-estimation/package.json index 65098be171ff..28d611323354 100644 --- a/packages/fee-estimation/package.json +++ b/packages/fee-estimation/package.json @@ -44,7 +44,7 @@ "jsdom": "^22.1.0", "tsup": "^7.2.0", "typescript": "^5.2.2", - "viem": "^1.16.6", + "viem": "^1.17.1", "vite": "^4.5.0", "vitest": "^0.34.2" }, diff --git a/packages/sdk/package.json b/packages/sdk/package.json index 5e2ac2a16f34..6c079d72fcca 100644 --- a/packages/sdk/package.json +++ b/packages/sdk/package.json @@ -56,7 +56,7 @@ "ts-node": "^10.9.1", "typedoc": "^0.25.2", "typescript": "^5.2.2", - "viem": "^1.16.6", + "viem": "^1.17.1", "vitest": "^0.34.2", "zod": "^3.22.4" }, diff --git a/packages/web3js-plugin/package.json b/packages/web3js-plugin/package.json index 930553db3cd2..92222f2a37a8 100644 --- a/packages/web3js-plugin/package.json +++ b/packages/web3js-plugin/package.json @@ -37,7 +37,7 @@ "@vitest/coverage-istanbul": "^0.34.6", "tsup": "^7.2.0", "typescript": "^5.2.2", - "viem": "^1.16.6", + "viem": "^1.17.1", "vite": "^4.5.0", "vitest": "^0.34.1", "zod": "^3.22.4" diff --git a/pnpm-lock.yaml b/pnpm-lock.yaml index 475c824d1ca2..d358ea25a96d 100644 --- a/pnpm-lock.yaml +++ b/pnpm-lock.yaml @@ -298,11 +298,11 @@ importers: specifier: ^18.2.0 version: 18.2.0(react@18.2.0) viem: - specifier: ^1.16.6 - version: 1.16.6(typescript@5.2.2)(zod@3.22.4) + specifier: ^1.17.1 + version: 1.17.1(typescript@5.2.2)(zod@3.22.4) wagmi: specifier: '>1.0.0' - version: 1.0.1(react-dom@18.2.0)(react@18.2.0)(typescript@5.2.2)(viem@1.16.6) + version: 1.0.1(react-dom@18.2.0)(react@18.2.0)(typescript@5.2.2)(viem@1.17.1) devDependencies: '@eth-optimism/contracts-bedrock': specifier: workspace:* @@ -324,7 +324,7 @@ importers: version: 1.5.2(@wagmi/core@1.4.5)(typescript@5.2.2)(wagmi@1.0.1) '@wagmi/core': specifier: ^1.4.5 - version: 1.4.5(react@18.2.0)(typescript@5.2.2)(viem@1.16.6) + version: 1.4.5(react@18.2.0)(typescript@5.2.2)(viem@1.17.1) abitype: specifier: ^0.10.1 version: 0.10.1(typescript@5.2.2) @@ -438,8 +438,8 @@ importers: specifier: ^5.2.2 version: 5.2.2 viem: - specifier: ^1.16.6 - version: 1.16.6(typescript@5.2.2)(zod@3.22.4) + specifier: ^1.17.1 + version: 1.17.1(typescript@5.2.2)(zod@3.22.4) vite: specifier: ^4.5.0 version: 4.5.0(@types/node@20.8.8) @@ -529,8 +529,8 @@ importers: specifier: ^5.2.2 version: 5.2.2 viem: - specifier: ^1.16.6 - version: 1.16.6(typescript@5.2.2)(zod@3.22.4) + specifier: ^1.17.1 + version: 1.17.1(typescript@5.2.2)(zod@3.22.4) vitest: specifier: ^0.34.2 version: 0.34.2 @@ -569,8 +569,8 @@ importers: specifier: ^5.2.2 version: 5.2.2 viem: - specifier: ^1.16.6 - version: 1.16.6(typescript@5.2.2)(zod@3.22.4) + specifier: ^1.17.1 + version: 1.17.1(typescript@5.2.2)(zod@3.22.4) vite: specifier: ^4.5.0 version: 4.5.0(@types/node@20.8.8) @@ -3228,7 +3228,7 @@ packages: resolution: {integrity: sha512-gYw0ki/EAuV1oSyMxpqandHjnthZjYYy+YWpTAzf8BqfXM3ItcZLpjxfg+3+mXW8HIO+3jw6T9iiqEXsqHaMMw==} dependencies: '@safe-global/safe-gateway-typescript-sdk': 3.7.3 - viem: 1.16.6(typescript@5.2.2)(zod@3.22.4) + viem: 1.17.1(typescript@5.2.2)(zod@3.22.4) transitivePeerDependencies: - bufferutil - encoding @@ -4593,7 +4593,7 @@ packages: wagmi: optional: true dependencies: - '@wagmi/core': 1.4.5(react@18.2.0)(typescript@5.2.2)(viem@1.16.6) + '@wagmi/core': 1.4.5(react@18.2.0)(typescript@5.2.2)(viem@1.17.1) abitype: 0.8.7(typescript@5.2.2)(zod@3.22.3) abort-controller: 3.0.0 bundle-require: 3.1.2(esbuild@0.16.17) @@ -4615,15 +4615,15 @@ packages: picocolors: 1.0.0 prettier: 2.8.8 typescript: 5.2.2 - viem: 1.16.6(typescript@5.2.2)(zod@3.22.3) - wagmi: 1.0.1(react-dom@18.2.0)(react@18.2.0)(typescript@5.2.2)(viem@1.16.6) + viem: 1.17.1(typescript@5.2.2)(zod@3.22.3) + wagmi: 1.0.1(react-dom@18.2.0)(react@18.2.0)(typescript@5.2.2)(viem@1.17.1) zod: 3.22.3 transitivePeerDependencies: - bufferutil - utf-8-validate dev: true - /@wagmi/connectors@1.0.1(@wagmi/chains@0.2.22)(react@18.2.0)(typescript@5.2.2)(viem@1.16.6): + /@wagmi/connectors@1.0.1(@wagmi/chains@0.2.22)(react@18.2.0)(typescript@5.2.2)(viem@1.17.1): resolution: {integrity: sha512-fl01vym19DE1uoE+MlASw5zo3Orr/YXlJRjOKLaKYtV+Q7jOLY4TwHgq7sEMs+JYOvFICFBEAlWNNxidr51AqQ==} peerDependencies: '@wagmi/chains': '>=0.2.0' @@ -4646,7 +4646,7 @@ packages: abitype: 0.8.1(typescript@5.2.2) eventemitter3: 4.0.7 typescript: 5.2.2 - viem: 1.16.6(typescript@5.2.2)(zod@3.22.4) + viem: 1.17.1(typescript@5.2.2)(zod@3.22.4) transitivePeerDependencies: - '@react-native-async-storage/async-storage' - bufferutil @@ -4658,7 +4658,7 @@ packages: - utf-8-validate - zod - /@wagmi/connectors@3.1.3(react@18.2.0)(typescript@5.2.2)(viem@1.16.6): + /@wagmi/connectors@3.1.3(react@18.2.0)(typescript@5.2.2)(viem@1.17.1): resolution: {integrity: sha512-UgwsQKQDFObJVJMf9pDfFoXTv710o4zrTHyhIWKBTMMkLpCMsMxN5+ZaDhBYt/BgoRinfRYQo8uwuwLhxE6Log==} peerDependencies: typescript: '>=5.0.4' @@ -4678,7 +4678,7 @@ packages: abitype: 0.8.7(typescript@5.2.2)(zod@3.22.3) eventemitter3: 4.0.7 typescript: 5.2.2 - viem: 1.16.6(typescript@5.2.2)(zod@3.22.4) + viem: 1.17.1(typescript@5.2.2)(zod@3.22.4) transitivePeerDependencies: - '@react-native-async-storage/async-storage' - '@types/react' @@ -4691,7 +4691,7 @@ packages: - zod dev: true - /@wagmi/core@1.0.1(react@18.2.0)(typescript@5.2.2)(viem@1.16.6): + /@wagmi/core@1.0.1(react@18.2.0)(typescript@5.2.2)(viem@1.17.1): resolution: {integrity: sha512-Zzg4Ob92QMF9NsC+z5/8JZjMn3NCCnwVWGJlv79qRX9mp5Ku40OzJNvqDnjcSGjshe6H0L/KtFZAqTlmu8lT7w==} peerDependencies: typescript: '>=4.9.4' @@ -4701,11 +4701,11 @@ packages: optional: true dependencies: '@wagmi/chains': 0.2.22(typescript@5.2.2) - '@wagmi/connectors': 1.0.1(@wagmi/chains@0.2.22)(react@18.2.0)(typescript@5.2.2)(viem@1.16.6) + '@wagmi/connectors': 1.0.1(@wagmi/chains@0.2.22)(react@18.2.0)(typescript@5.2.2)(viem@1.17.1) abitype: 0.8.1(typescript@5.2.2) eventemitter3: 4.0.7 typescript: 5.2.2 - viem: 1.16.6(typescript@5.2.2)(zod@3.22.4) + viem: 1.17.1(typescript@5.2.2)(zod@3.22.4) zustand: 4.3.9(react@18.2.0) transitivePeerDependencies: - '@react-native-async-storage/async-storage' @@ -4719,7 +4719,7 @@ packages: - utf-8-validate - zod - /@wagmi/core@1.4.5(react@18.2.0)(typescript@5.2.2)(viem@1.16.6): + /@wagmi/core@1.4.5(react@18.2.0)(typescript@5.2.2)(viem@1.17.1): resolution: {integrity: sha512-N9luRb1Uk4tBN9kaYcQSWKE9AsRt/rvZaFt5IZech4JPzNN2sQlfhKd9GEjOXYRDqEPHdDvos7qyBKiDNTz4GA==} peerDependencies: typescript: '>=5.0.4' @@ -4728,11 +4728,11 @@ packages: typescript: optional: true dependencies: - '@wagmi/connectors': 3.1.3(react@18.2.0)(typescript@5.2.2)(viem@1.16.6) + '@wagmi/connectors': 3.1.3(react@18.2.0)(typescript@5.2.2)(viem@1.17.1) abitype: 0.8.7(typescript@5.2.2)(zod@3.22.3) eventemitter3: 4.0.7 typescript: 5.2.2 - viem: 1.16.6(typescript@5.2.2)(zod@3.22.4) + viem: 1.17.1(typescript@5.2.2)(zod@3.22.4) zustand: 4.3.9(react@18.2.0) transitivePeerDependencies: - '@react-native-async-storage/async-storage' @@ -14328,8 +14328,8 @@ packages: vfile-message: 2.0.4 dev: true - /viem@1.16.6(typescript@5.2.2)(zod@3.22.3): - resolution: {integrity: sha512-jcWcFQ+xzIfDwexwPJRvCuCRJKEkK9iHTStG7mpU5MmuSBpACs4nATBDyXNFtUiyYTFzLlVEwWkt68K0nCSImg==} + /viem@1.17.1(typescript@5.2.2)(zod@3.22.3): + resolution: {integrity: sha512-MSbrfntjgIMKPUPdNJ1pnwT1pDfnOzJnKSLqpafw1q+1k6k6M/jxn09g3WbKefIKIok122DcbmviMow+4FqkAg==} peerDependencies: typescript: '>=5.0.4' peerDependenciesMeta: @@ -14351,8 +14351,8 @@ packages: - zod dev: true - /viem@1.16.6(typescript@5.2.2)(zod@3.22.4): - resolution: {integrity: sha512-jcWcFQ+xzIfDwexwPJRvCuCRJKEkK9iHTStG7mpU5MmuSBpACs4nATBDyXNFtUiyYTFzLlVEwWkt68K0nCSImg==} + /viem@1.17.1(typescript@5.2.2)(zod@3.22.4): + resolution: {integrity: sha512-MSbrfntjgIMKPUPdNJ1pnwT1pDfnOzJnKSLqpafw1q+1k6k6M/jxn09g3WbKefIKIok122DcbmviMow+4FqkAg==} peerDependencies: typescript: '>=5.0.4' peerDependenciesMeta: @@ -14845,7 +14845,7 @@ packages: xml-name-validator: 4.0.0 dev: true - /wagmi@1.0.1(react-dom@18.2.0)(react@18.2.0)(typescript@5.2.2)(viem@1.16.6): + /wagmi@1.0.1(react-dom@18.2.0)(react@18.2.0)(typescript@5.2.2)(viem@1.17.1): resolution: {integrity: sha512-+2UkZG9eA3tKqXj1wvlvI8mL0Bcff7Tf5CKfUOyQsdKcY+J5rfwYYya25G+jja57umpHFtfxRaL7xDkNjehrRg==} peerDependencies: react: '>=17.0.0' @@ -14858,12 +14858,12 @@ packages: '@tanstack/query-sync-storage-persister': 4.29.25 '@tanstack/react-query': 4.29.25(react-dom@18.2.0)(react@18.2.0) '@tanstack/react-query-persist-client': 4.29.25(@tanstack/react-query@4.29.25) - '@wagmi/core': 1.0.1(react@18.2.0)(typescript@5.2.2)(viem@1.16.6) + '@wagmi/core': 1.0.1(react@18.2.0)(typescript@5.2.2)(viem@1.17.1) abitype: 0.8.1(typescript@5.2.2) react: 18.2.0 typescript: 5.2.2 use-sync-external-store: 1.2.0(react@18.2.0) - viem: 1.16.6(typescript@5.2.2)(zod@3.22.4) + viem: 1.17.1(typescript@5.2.2)(zod@3.22.4) transitivePeerDependencies: - '@react-native-async-storage/async-storage' - bufferutil From 36b0f4e1264c1abc4c9191e8aa4a435f6e283138 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 25 Oct 2023 22:21:07 +0000 Subject: [PATCH 085/279] build(deps-dev): bump hardhat from 2.18.2 to 2.18.3 Bumps [hardhat](https://github.com/nomiclabs/hardhat) from 2.18.2 to 2.18.3. - [Release notes](https://github.com/nomiclabs/hardhat/releases) - [Commits](https://github.com/nomiclabs/hardhat/compare/hardhat@2.18.2...hardhat@2.18.3) --- updated-dependencies: - dependency-name: hardhat dependency-type: direct:development update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] --- packages/chain-mon/package.json | 2 +- packages/sdk/package.json | 2 +- pnpm-lock.yaml | 36 ++++++++++++++++----------------- 3 files changed, 20 insertions(+), 20 deletions(-) diff --git a/packages/chain-mon/package.json b/packages/chain-mon/package.json index 829c69f46321..dcd3f4ad8965 100644 --- a/packages/chain-mon/package.json +++ b/packages/chain-mon/package.json @@ -59,7 +59,7 @@ "@ethersproject/abstract-provider": "^5.7.0", "@nomiclabs/hardhat-ethers": "^2.2.3", "@nomiclabs/hardhat-waffle": "^2.0.6", - "hardhat": "^2.18.2", + "hardhat": "^2.18.3", "ts-node": "^10.9.1", "tsx": "^3.14.0" } diff --git a/packages/sdk/package.json b/packages/sdk/package.json index 5e2ac2a16f34..cbf651cfd58f 100644 --- a/packages/sdk/package.json +++ b/packages/sdk/package.json @@ -48,7 +48,7 @@ "chai-as-promised": "^7.1.1", "ethereum-waffle": "^4.0.10", "ethers": "^5.7.2", - "hardhat": "^2.18.2", + "hardhat": "^2.18.3", "hardhat-deploy": "^0.11.43", "isomorphic-fetch": "^3.0.0", "mocha": "^10.2.0", diff --git a/pnpm-lock.yaml b/pnpm-lock.yaml index 475c824d1ca2..353d6bf50b62 100644 --- a/pnpm-lock.yaml +++ b/pnpm-lock.yaml @@ -175,13 +175,13 @@ importers: version: 5.7.0 '@nomiclabs/hardhat-ethers': specifier: ^2.2.3 - version: 2.2.3(ethers@5.7.2)(hardhat@2.18.2) + version: 2.2.3(ethers@5.7.2)(hardhat@2.18.3) '@nomiclabs/hardhat-waffle': specifier: ^2.0.6 - version: 2.0.6(@nomiclabs/hardhat-ethers@2.2.3)(@types/sinon-chai@3.2.5)(ethereum-waffle@4.0.10)(ethers@5.7.2)(hardhat@2.18.2) + version: 2.0.6(@nomiclabs/hardhat-ethers@2.2.3)(@types/sinon-chai@3.2.5)(ethereum-waffle@4.0.10)(ethers@5.7.2)(hardhat@2.18.3) hardhat: - specifier: ^2.18.2 - version: 2.18.2(ts-node@10.9.1)(typescript@5.2.2) + specifier: ^2.18.3 + version: 2.18.3(ts-node@10.9.1)(typescript@5.2.2) ts-node: specifier: ^10.9.1 version: 10.9.1(@types/node@20.8.8)(typescript@5.2.2) @@ -479,10 +479,10 @@ importers: version: 5.7.0 '@nomiclabs/hardhat-ethers': specifier: ^2.2.3 - version: 2.2.3(ethers@5.7.2)(hardhat@2.18.2) + version: 2.2.3(ethers@5.7.2)(hardhat@2.18.3) '@nomiclabs/hardhat-waffle': specifier: ^2.0.1 - version: 2.0.1(@nomiclabs/hardhat-ethers@2.2.3)(ethereum-waffle@4.0.10)(ethers@5.7.2)(hardhat@2.18.2) + version: 2.0.1(@nomiclabs/hardhat-ethers@2.2.3)(ethereum-waffle@4.0.10)(ethers@5.7.2)(hardhat@2.18.3) '@types/chai': specifier: ^4.3.8 version: 4.3.8 @@ -505,8 +505,8 @@ importers: specifier: ^5.7.2 version: 5.7.2 hardhat: - specifier: ^2.18.2 - version: 2.18.2(ts-node@10.9.1)(typescript@5.2.2) + specifier: ^2.18.3 + version: 2.18.3(ts-node@10.9.1)(typescript@5.2.2) hardhat-deploy: specifier: ^0.11.43 version: 0.11.43 @@ -2969,17 +2969,17 @@ packages: '@nomicfoundation/solidity-analyzer-win32-x64-msvc': 0.1.1 dev: true - /@nomiclabs/hardhat-ethers@2.2.3(ethers@5.7.2)(hardhat@2.18.2): + /@nomiclabs/hardhat-ethers@2.2.3(ethers@5.7.2)(hardhat@2.18.3): resolution: {integrity: sha512-YhzPdzb612X591FOe68q+qXVXGG2ANZRvDo0RRUtimev85rCrAlv/TLMEZw5c+kq9AbzocLTVX/h2jVIFPL9Xg==} peerDependencies: ethers: ^5.0.0 hardhat: ^2.0.0 dependencies: ethers: 5.7.2 - hardhat: 2.18.2(ts-node@10.9.1)(typescript@5.2.2) + hardhat: 2.18.3(ts-node@10.9.1)(typescript@5.2.2) dev: true - /@nomiclabs/hardhat-waffle@2.0.1(@nomiclabs/hardhat-ethers@2.2.3)(ethereum-waffle@4.0.10)(ethers@5.7.2)(hardhat@2.18.2): + /@nomiclabs/hardhat-waffle@2.0.1(@nomiclabs/hardhat-ethers@2.2.3)(ethereum-waffle@4.0.10)(ethers@5.7.2)(hardhat@2.18.3): resolution: {integrity: sha512-2YR2V5zTiztSH9n8BYWgtv3Q+EL0N5Ltm1PAr5z20uAY4SkkfylJ98CIqt18XFvxTD5x4K2wKBzddjV9ViDAZQ==} peerDependencies: '@nomiclabs/hardhat-ethers': ^2.0.0 @@ -2987,15 +2987,15 @@ packages: ethers: ^5.0.0 hardhat: ^2.0.0 dependencies: - '@nomiclabs/hardhat-ethers': 2.2.3(ethers@5.7.2)(hardhat@2.18.2) + '@nomiclabs/hardhat-ethers': 2.2.3(ethers@5.7.2)(hardhat@2.18.3) '@types/sinon-chai': 3.2.5 '@types/web3': 1.0.19 ethereum-waffle: 4.0.10(@ensdomains/ens@0.4.5)(@ensdomains/resolver@0.2.4)(@ethersproject/abi@5.7.0)(@ethersproject/providers@5.7.2)(ethers@5.7.2)(typescript@5.2.2) ethers: 5.7.2 - hardhat: 2.18.2(ts-node@10.9.1)(typescript@5.2.2) + hardhat: 2.18.3(ts-node@10.9.1)(typescript@5.2.2) dev: true - /@nomiclabs/hardhat-waffle@2.0.6(@nomiclabs/hardhat-ethers@2.2.3)(@types/sinon-chai@3.2.5)(ethereum-waffle@4.0.10)(ethers@5.7.2)(hardhat@2.18.2): + /@nomiclabs/hardhat-waffle@2.0.6(@nomiclabs/hardhat-ethers@2.2.3)(@types/sinon-chai@3.2.5)(ethereum-waffle@4.0.10)(ethers@5.7.2)(hardhat@2.18.3): resolution: {integrity: sha512-+Wz0hwmJGSI17B+BhU/qFRZ1l6/xMW82QGXE/Gi+WTmwgJrQefuBs1lIf7hzQ1hLk6hpkvb/zwcNkpVKRYTQYg==} peerDependencies: '@nomiclabs/hardhat-ethers': ^2.0.0 @@ -3004,11 +3004,11 @@ packages: ethers: ^5.0.0 hardhat: ^2.0.0 dependencies: - '@nomiclabs/hardhat-ethers': 2.2.3(ethers@5.7.2)(hardhat@2.18.2) + '@nomiclabs/hardhat-ethers': 2.2.3(ethers@5.7.2)(hardhat@2.18.3) '@types/sinon-chai': 3.2.5 ethereum-waffle: 4.0.10(@ensdomains/ens@0.4.5)(@ensdomains/resolver@0.2.4)(@ethersproject/abi@5.7.0)(@ethersproject/providers@5.7.2)(ethers@5.7.2)(typescript@5.2.2) ethers: 5.7.2 - hardhat: 2.18.2(ts-node@10.9.1)(typescript@5.2.2) + hardhat: 2.18.3(ts-node@10.9.1)(typescript@5.2.2) dev: true /@nrwl/nx-cloud@16.5.2: @@ -8938,8 +8938,8 @@ packages: - utf-8-validate dev: true - /hardhat@2.18.2(ts-node@10.9.1)(typescript@5.2.2): - resolution: {integrity: sha512-lUVmJg7DsKcUCDpqv57CJl6vHqo/1PeHSfM3+WIa8UtRKmXyVTj1qQK01TDiuetkZBVg9Dn52qU+ZwaJQynaKA==} + /hardhat@2.18.3(ts-node@10.9.1)(typescript@5.2.2): + resolution: {integrity: sha512-JuYaTG+4ZHVjEHCW5Hn6jCHH3LpO75dtgznZpM/dLv12RcSlw/xHbeQh3FAsGahQr1epKryZcZEMHvztVZHe0g==} hasBin: true peerDependencies: ts-node: '*' From fe46b7ead573761bd89267341298925f86093877 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 25 Oct 2023 22:23:22 +0000 Subject: [PATCH 086/279] build(deps-dev): bump nx from 17.0.1 to 17.0.2 Bumps [nx](https://github.com/nrwl/nx/tree/HEAD/packages/nx) from 17.0.1 to 17.0.2. - [Release notes](https://github.com/nrwl/nx/releases) - [Commits](https://github.com/nrwl/nx/commits/17.0.2/packages/nx) --- updated-dependencies: - dependency-name: nx dependency-type: direct:development update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] --- package.json | 2 +- pnpm-lock.yaml | 90 +++++++++++++++++++++----------------------------- 2 files changed, 39 insertions(+), 53 deletions(-) diff --git a/package.json b/package.json index 9f6fc137195f..c30e072afdc5 100644 --- a/package.json +++ b/package.json @@ -65,7 +65,7 @@ "markdownlint": "^0.31.0", "markdownlint-cli2": "0.4.0", "mocha": "^10.2.0", - "nx": "17.0.1", + "nx": "17.0.2", "nyc": "^15.1.0", "patch-package": "^8.0.0", "prettier": "^2.8.0", diff --git a/pnpm-lock.yaml b/pnpm-lock.yaml index 475c824d1ca2..cb1ac203ae15 100644 --- a/pnpm-lock.yaml +++ b/pnpm-lock.yaml @@ -97,8 +97,8 @@ importers: specifier: ^10.2.0 version: 10.2.0 nx: - specifier: 17.0.1 - version: 17.0.1 + specifier: 17.0.2 + version: 17.0.2 nx-cloud: specifier: latest version: 16.5.2 @@ -3019,11 +3019,11 @@ packages: - debug dev: true - /@nrwl/tao@17.0.1: - resolution: {integrity: sha512-UtqLJi3kCtGFY4voZ78FfQ/t3av7F9gYgWkG9/WquDmXzUMNufPgsnIX4wwzvyeifwy1ocEA0APsUfRX63d5rw==} + /@nrwl/tao@17.0.2: + resolution: {integrity: sha512-H+htIRzQR6Ibael34rhQkpNkpFFFmaSTsIzdqkBqL4j5+FzSpZh67NJnWSY8vsYQGQL8Ncc+MHvpQC+7pyfgGw==} hasBin: true dependencies: - nx: 17.0.1 + nx: 17.0.2 tslib: 2.6.2 transitivePeerDependencies: - '@swc-node/register' @@ -3031,8 +3031,8 @@ packages: - debug dev: true - /@nx/nx-darwin-arm64@17.0.1: - resolution: {integrity: sha512-Ksgrtck+vZtdh4mFw+IF4ECh4VGQktuHZNT7MwzKQTzLc7gQgjt3OyC97rDWoTb8qgUNc857Hor6ZAZpqHkItA==} + /@nx/nx-darwin-arm64@17.0.2: + resolution: {integrity: sha512-OSZLRfV8VplYPEqMcIg3mbAsJXlXEHKrdlJ0KUTk8Hih2+wl7cxuSEwG7X7qfBUOz+ognxaqicL+hueNrgwjlQ==} engines: {node: '>= 10'} cpu: [arm64] os: [darwin] @@ -3040,8 +3040,8 @@ packages: dev: true optional: true - /@nx/nx-darwin-x64@17.0.1: - resolution: {integrity: sha512-QBGRDwh9xxBAeOI9WUgiihOqnJ3WUlNtIajf3rsH0A9UpU2hPg/7rTSzLRSJHScEPeB9ICLRoiCRnvoZ1iuQMQ==} + /@nx/nx-darwin-x64@17.0.2: + resolution: {integrity: sha512-olGt5R2dWYwdl1+I2RfJ8LdZO1elqhr9yDPnMIx//ZuN6T6wJA+Wdp2P3qpM1bY0F1lI/6AgjqzRyrTLUZ9cDA==} engines: {node: '>= 10'} cpu: [x64] os: [darwin] @@ -3049,8 +3049,8 @@ packages: dev: true optional: true - /@nx/nx-freebsd-x64@17.0.1: - resolution: {integrity: sha512-r7cCVBdypTK73b5uPiewIQSq9klSEabcYJkhOP5H0Kzdb3n5G8eJrQBLopoXXqnkxIZnfzvIV/ATssr7nIGJJg==} + /@nx/nx-freebsd-x64@17.0.2: + resolution: {integrity: sha512-+mta0J2G2byd+rfZ275oZs0aYXC/s92nI9ySBFQFQZnKJ6bsAagdZHe+uETsnE4xdhFXD8kvNMJU1WTGlyFyjg==} engines: {node: '>= 10'} cpu: [x64] os: [freebsd] @@ -3058,8 +3058,8 @@ packages: dev: true optional: true - /@nx/nx-linux-arm-gnueabihf@17.0.1: - resolution: {integrity: sha512-s0EGuEMvib+yxji+A/C5BDzOe03GczVtB8ArSKWUxwSGq2jv7ybTXgZK0VdJc0odhzXu6zpwy6mEm1ZAGfshDw==} + /@nx/nx-linux-arm-gnueabihf@17.0.2: + resolution: {integrity: sha512-m80CmxHHyNAJ8j/0rkjc0hg/eGQlf6V2sLsV+gEJkz2sTEEdgSOK4DvnWcZRWO/SWBnqigxoHX4Kf5TH1nmoHA==} engines: {node: '>= 10'} cpu: [arm] os: [linux] @@ -3067,8 +3067,8 @@ packages: dev: true optional: true - /@nx/nx-linux-arm64-gnu@17.0.1: - resolution: {integrity: sha512-gB4gdBrnXh4XXqk6RC+GBDnDNZ3UnCnQRTvyVZUtlDZnJ7j9FYbNTZB+m/zoQGweBpCDYAYnpx+bJ7sK1lVwlg==} + /@nx/nx-linux-arm64-gnu@17.0.2: + resolution: {integrity: sha512-AsD1H6wt68MK1u6vkmtNaFaxDMcyuk6dpo5kq1YT9cfUd614ys3qMUjVp3P2CXxzXh+0UDZeGrc6qotNKOkpJw==} engines: {node: '>= 10'} cpu: [arm64] os: [linux] @@ -3076,8 +3076,8 @@ packages: dev: true optional: true - /@nx/nx-linux-arm64-musl@17.0.1: - resolution: {integrity: sha512-h2258DDRwV10bp3ygmaBxLVnV/gUXo9fQc5yBLgJD679RcyhtQNIW76pCtqqJAb/ErzxJAj6s90YxQdtnnScqQ==} + /@nx/nx-linux-arm64-musl@17.0.2: + resolution: {integrity: sha512-f8pUFoZHBFQtHnopHgTEuwIiu0Rzem0dD7iK8SyyBy/lRAADtHCAHxaPAG+iatHAJ9h4DFIB50k9ybYxDtH2mg==} engines: {node: '>= 10'} cpu: [arm64] os: [linux] @@ -3085,8 +3085,8 @@ packages: dev: true optional: true - /@nx/nx-linux-x64-gnu@17.0.1: - resolution: {integrity: sha512-KrmkK3CcROGgjO5yOtHgVa9Add2UTjrJnPGFcBES4nm6+7ZDCDwll7ukYCsPy6cwx6hhC5aHrKCARc0oTRmjOQ==} + /@nx/nx-linux-x64-gnu@17.0.2: + resolution: {integrity: sha512-PISrHjLTxv5w8bz50vPZH6puYos88xu28o4IbVyYWrUrhoFsAx9Zbn1D6gWDPMSaKJU32v1l+5bTciQjQJU8fQ==} engines: {node: '>= 10'} cpu: [x64] os: [linux] @@ -3094,8 +3094,8 @@ packages: dev: true optional: true - /@nx/nx-linux-x64-musl@17.0.1: - resolution: {integrity: sha512-qMWAPb+dH50bkM/LgCLhgOMC1Ie0A7lEl4KsJGQcib7CE9l7ntdRso9osJGkBtmidpohL5LbfRb3W4DJunz6Vg==} + /@nx/nx-linux-x64-musl@17.0.2: + resolution: {integrity: sha512-2wsqyBRjsxmAjxW+0lnGFtJLTk+AxgW7gjMv8NgLK8P1bc/sJYQB+g0o5op2z+szXRG3Noi0RZ9C0fG39EPFZw==} engines: {node: '>= 10'} cpu: [x64] os: [linux] @@ -3103,8 +3103,8 @@ packages: dev: true optional: true - /@nx/nx-win32-arm64-msvc@17.0.1: - resolution: {integrity: sha512-nVsZIqUnqYeo3fkrxjivy2mHW+wbORaJPv2donxPJDCFjXgn1gPsQenKl1wlkeetw+TDCOLQmt09wMkDfpFO2w==} + /@nx/nx-win32-arm64-msvc@17.0.2: + resolution: {integrity: sha512-Sc3sQUcS5xdk05PABe/knG6orG5rmHZdSUj6SMRpvYfN2tM3ziNn6/wCF/LJoW6n70OxrOEXXwLSRK/5WigXbA==} engines: {node: '>= 10'} cpu: [arm64] os: [win32] @@ -3112,8 +3112,8 @@ packages: dev: true optional: true - /@nx/nx-win32-x64-msvc@17.0.1: - resolution: {integrity: sha512-0W1Hk7tbAOrou6ScTRo6hlZiIIBt0ECdn6rALEDWN4lNKheu0pb2gHZO/hSYMwGj3XUpNvApvUCCdcSTBjegRQ==} + /@nx/nx-win32-x64-msvc@17.0.2: + resolution: {integrity: sha512-XhET0BDk6fbvTBCs7m5gZii8+2WhLpiC1sZchJw4LAJN2VJBiy3I3xnvpQYGFOAWaCb/iUGpuN/qP/NlQ+LNgA==} engines: {node: '>= 10'} cpu: [x64] os: [win32] @@ -3121,15 +3121,6 @@ packages: dev: true optional: true - /@parcel/watcher@2.0.4: - resolution: {integrity: sha512-cTDi+FUDBIUOBKEtj+nhiJ71AZVlkAsQFuGQTun5tV9mwQBQgZvhCzG+URPQc8myeN32yRVZEfVAPCs1RW+Jvg==} - engines: {node: '>= 10.0.0'} - requiresBuild: true - dependencies: - node-addon-api: 3.2.1 - node-gyp-build: 4.6.0 - dev: true - /@pedrouid/environment@1.0.1: resolution: {integrity: sha512-HaW78NszGzRZd9SeoI3JD11JqY+lubnaOx7Pewj5pfjqWXOEATpeKIFb9Z4t2WBUK2iryiXX3lzWwmYWgUL0Ug==} @@ -11167,10 +11158,6 @@ packages: /node-addon-api@2.0.2: resolution: {integrity: sha512-Ntyt4AIXyaLIuMHF6IOoTakB3K+RWxwtsHNRxllEoA6vPwP9o4866g6YWDLUdnucilZhmkxiHwHr11gAENw+QA==} - /node-addon-api@3.2.1: - resolution: {integrity: sha512-mmcei9JghVNDYydghQmeDX8KoAm0FAiYyIcUt/N4nhyAipB17pllZQDOJD2fotxABnt4Mdz+dKTO7eftLg4d0A==} - dev: true - /node-domexception@1.0.0: resolution: {integrity: sha512-/jKZoMpw0F8GRwl4/eLROPA3cfcXtLApP0QzLmUT/HuPCZWyB7IY9ZrMeKw2O/nFIqPQB3PVM9aYm0F312AXDQ==} engines: {node: '>=10.5.0'} @@ -11313,8 +11300,8 @@ packages: - debug dev: true - /nx@17.0.1: - resolution: {integrity: sha512-OA5M0oJmVAujXjjbXTN9zBG9fG0B2efaKICkfsrFy/g74QhdCxnzvEiGjO2CUQbx5nn/l8sJv0ApEBwfX+F24Q==} + /nx@17.0.2: + resolution: {integrity: sha512-utk9ufxLlRd210nEV6cKjMLVK0gup2ZMlNT41lLgUX/gp3Q59G1NkyLo3o29DxBh3AhNJ9q5MKgybmzDNdpudA==} hasBin: true requiresBuild: true peerDependencies: @@ -11326,8 +11313,7 @@ packages: '@swc/core': optional: true dependencies: - '@nrwl/tao': 17.0.1 - '@parcel/watcher': 2.0.4 + '@nrwl/tao': 17.0.2 '@yarnpkg/lockfile': 1.1.0 '@yarnpkg/parsers': 3.0.0-rc.46 '@zkochan/js-yaml': 0.0.6 @@ -11363,16 +11349,16 @@ packages: yargs: 17.7.2 yargs-parser: 21.1.1 optionalDependencies: - '@nx/nx-darwin-arm64': 17.0.1 - '@nx/nx-darwin-x64': 17.0.1 - '@nx/nx-freebsd-x64': 17.0.1 - '@nx/nx-linux-arm-gnueabihf': 17.0.1 - '@nx/nx-linux-arm64-gnu': 17.0.1 - '@nx/nx-linux-arm64-musl': 17.0.1 - '@nx/nx-linux-x64-gnu': 17.0.1 - '@nx/nx-linux-x64-musl': 17.0.1 - '@nx/nx-win32-arm64-msvc': 17.0.1 - '@nx/nx-win32-x64-msvc': 17.0.1 + '@nx/nx-darwin-arm64': 17.0.2 + '@nx/nx-darwin-x64': 17.0.2 + '@nx/nx-freebsd-x64': 17.0.2 + '@nx/nx-linux-arm-gnueabihf': 17.0.2 + '@nx/nx-linux-arm64-gnu': 17.0.2 + '@nx/nx-linux-arm64-musl': 17.0.2 + '@nx/nx-linux-x64-gnu': 17.0.2 + '@nx/nx-linux-x64-musl': 17.0.2 + '@nx/nx-win32-arm64-msvc': 17.0.2 + '@nx/nx-win32-x64-msvc': 17.0.2 transitivePeerDependencies: - debug dev: true From 4f78aa4eea5896f75c9973b41c933715dea03bfc Mon Sep 17 00:00:00 2001 From: Felipe Andrade Date: Wed, 25 Oct 2023 14:07:07 -0700 Subject: [PATCH 087/279] feat(op-service): new latency metric --- op-service/metrics/http.go | 26 +++++++++++++++++++++++++- 1 file changed, 25 insertions(+), 1 deletion(-) diff --git a/op-service/metrics/http.go b/op-service/metrics/http.go index f9102ac44f90..342b7857adb6 100644 --- a/op-service/metrics/http.go +++ b/op-service/metrics/http.go @@ -44,21 +44,41 @@ func (n *noopHTTPRecorder) RecordHTTPRequest(*HTTPParams) {} func (n *noopHTTPRecorder) RecordHTTPResponse(*HTTPParams) {} type PromHTTPRecorder struct { - HTTPRequestDuration *prometheus.HistogramVec + // HTTPRequestDuration is the old metric for request latency + // it was created with too tight buckets: [.005, .01, .025, .05, .1, .25, .5, 1, 2.5, 5, 10] + // in order to preserve backward compatibility we are keeping this metric for now + // and it will be removed when services opt in to HTTPRequestLatency + // Deprecated: HTTPRequestDuration is deprecated + HTTPRequestDuration *prometheus.HistogramVec + + // HTTPRequestLatency measures request execution latency in *seconds* + // buckets are: [.025, .05, .1, .25, .5, 1, 2.5, 5, 10, 25, 50] + HTTPRequestLatency *prometheus.HistogramVec + HTTPResponseSize *prometheus.HistogramVec HTTPInflightRequests *prometheus.GaugeVec HTTPRequests *prometheus.CounterVec HTTPResponses *prometheus.CounterVec } +var LatencyBuckets = []float64{.025, .05, .1, .25, .5, 1, 2.5, 5, 10, 25, 50} + func NewPromHTTPRecorder(r *prometheus.Registry, ns string) HTTPRecorder { return &PromHTTPRecorder{ + // TODO: remove this in the future when services opted in to HTTPRequestLatency HTTPRequestDuration: promauto.With(r).NewHistogramVec(prometheus.HistogramOpts{ Namespace: ns, Name: "http_request_duration_ms", Help: "Tracks HTTP request durations, in ms", Buckets: prometheus.DefBuckets, }, httpLabels), + HTTPRequestLatency: promauto.With(r).NewHistogramVec(prometheus.HistogramOpts{ + Namespace: ns, + Name: "http_request_latency_seconds", + Help: "Tracks HTTP request execution latency, in seconds", + Buckets: LatencyBuckets, + }, httpLabels), + HTTPResponseSize: promauto.With(r).NewHistogramVec(prometheus.HistogramOpts{ Namespace: ns, Name: "http_response_size", @@ -84,8 +104,12 @@ func NewPromHTTPRecorder(r *prometheus.Registry, ns string) HTTPRecorder { } func (p *PromHTTPRecorder) RecordHTTPRequestDuration(params *HTTPParams, dur time.Duration) { + // TODO: remove this in the future when services opted in to new metric p.HTTPRequestDuration.WithLabelValues(params.Method, strconv.Itoa(params.StatusCode)). Observe(float64(dur.Milliseconds())) + + p.HTTPRequestLatency.WithLabelValues(params.Method, strconv.Itoa(params.StatusCode)). + Observe(dur.Seconds()) } func (p *PromHTTPRecorder) RecordHTTPResponseSize(params *HTTPParams, size int) { From 3dff59f8eac0e67a0da6a80347c4f94303635f87 Mon Sep 17 00:00:00 2001 From: Felipe Andrade Date: Wed, 25 Oct 2023 14:17:36 -0700 Subject: [PATCH 088/279] semgrep --- op-service/metrics/http.go | 1 + 1 file changed, 1 insertion(+) diff --git a/op-service/metrics/http.go b/op-service/metrics/http.go index 342b7857adb6..f16bbec01e9f 100644 --- a/op-service/metrics/http.go +++ b/op-service/metrics/http.go @@ -65,6 +65,7 @@ var LatencyBuckets = []float64{.025, .05, .1, .25, .5, 1, 2.5, 5, 10, 25, 50} func NewPromHTTPRecorder(r *prometheus.Registry, ns string) HTTPRecorder { return &PromHTTPRecorder{ + // nosemgrep: todos_require_linear // TODO: remove this in the future when services opted in to HTTPRequestLatency HTTPRequestDuration: promauto.With(r).NewHistogramVec(prometheus.HistogramOpts{ Namespace: ns, From 9789fa603a68e3605636c190aa27af66f41ec968 Mon Sep 17 00:00:00 2001 From: Felipe Andrade Date: Wed, 25 Oct 2023 14:43:15 -0700 Subject: [PATCH 089/279] add linear to todo --- op-service/metrics/http.go | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/op-service/metrics/http.go b/op-service/metrics/http.go index f16bbec01e9f..5d606ccf1b42 100644 --- a/op-service/metrics/http.go +++ b/op-service/metrics/http.go @@ -61,12 +61,11 @@ type PromHTTPRecorder struct { HTTPResponses *prometheus.CounterVec } -var LatencyBuckets = []float64{.025, .05, .1, .25, .5, 1, 2.5, 5, 10, 25, 50} +var LatencyBuckets = []float64{.025, .05, .1, .25, .5, 1, 2.5, 5, 10, 25, 50, 100} func NewPromHTTPRecorder(r *prometheus.Registry, ns string) HTTPRecorder { return &PromHTTPRecorder{ - // nosemgrep: todos_require_linear - // TODO: remove this in the future when services opted in to HTTPRequestLatency + // TODO(INF-509): remove this in the future when services opted in to HTTPRequestLatency HTTPRequestDuration: promauto.With(r).NewHistogramVec(prometheus.HistogramOpts{ Namespace: ns, Name: "http_request_duration_ms", From 412ac55e5365907ea001351404239cb3f7f9c91f Mon Sep 17 00:00:00 2001 From: Felipe Andrade Date: Wed, 25 Oct 2023 15:02:19 -0700 Subject: [PATCH 090/279] also to other todo --- op-service/metrics/http.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/op-service/metrics/http.go b/op-service/metrics/http.go index 5d606ccf1b42..8640fe02a6f2 100644 --- a/op-service/metrics/http.go +++ b/op-service/metrics/http.go @@ -104,7 +104,7 @@ func NewPromHTTPRecorder(r *prometheus.Registry, ns string) HTTPRecorder { } func (p *PromHTTPRecorder) RecordHTTPRequestDuration(params *HTTPParams, dur time.Duration) { - // TODO: remove this in the future when services opted in to new metric + // TODO(INF-509): remove this in the future when services opted in to new metric p.HTTPRequestDuration.WithLabelValues(params.Method, strconv.Itoa(params.StatusCode)). Observe(float64(dur.Milliseconds())) From 1ea3bba36b595c7ee17d4f5a3547df1c6f62ae80 Mon Sep 17 00:00:00 2001 From: Mark Tyneway Date: Wed, 25 Oct 2023 14:27:46 -0600 Subject: [PATCH 091/279] contracts-bedrock: pseudorandom blocknumber in portal fuzz test Set a pseudorandom `block.number` when fuzzing the `OptimismPortal` deposit transaction codepath. This is a small fix required for the migration to using the deploy script to set up the unit tests. This prevents underflows with computing the `_prevBlockNum` and also adds additional converage since previously the block number was not being set dynamically. Breaking this out into its own smaller PR so that the deploy script migration can be smaller. --- packages/contracts-bedrock/test/OptimismPortal.t.sol | 2 ++ 1 file changed, 2 insertions(+) diff --git a/packages/contracts-bedrock/test/OptimismPortal.t.sol b/packages/contracts-bedrock/test/OptimismPortal.t.sol index 256e5c6359fc..f5f0d0a86e91 100644 --- a/packages/contracts-bedrock/test/OptimismPortal.t.sol +++ b/packages/contracts-bedrock/test/OptimismPortal.t.sol @@ -997,6 +997,8 @@ contract OptimismPortalResourceFuzz_Test is Portal_Initializer { vm.assume(((_maxResourceLimit / _elasticityMultiplier) * _elasticityMultiplier) == _maxResourceLimit); _prevBoughtGas = uint64(bound(_prevBoughtGas, 0, _maxResourceLimit - _gasLimit)); _blockDiff = uint8(bound(_blockDiff, 0, 3)); + // Pick a pseudorandom block number + vm.roll(uint256(keccak256(abi.encode(_blockDiff))) % uint256(type(uint16).max) + uint256(_blockDiff)); // Create a resource config to mock the call to the system config with ResourceMetering.ResourceConfig memory rcfg = ResourceMetering.ResourceConfig({ From 5cbccfd162ccf7734362fc6a0eff81d978719d10 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 25 Oct 2023 23:23:09 +0000 Subject: [PATCH 092/279] build(deps-dev): bump @types/node from 20.8.8 to 20.8.9 Bumps [@types/node](https://github.com/DefinitelyTyped/DefinitelyTyped/tree/HEAD/types/node) from 20.8.8 to 20.8.9. - [Release notes](https://github.com/DefinitelyTyped/DefinitelyTyped/releases) - [Commits](https://github.com/DefinitelyTyped/DefinitelyTyped/commits/HEAD/types/node) --- updated-dependencies: - dependency-name: "@types/node" dependency-type: direct:development update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] --- package.json | 2 +- packages/core-utils/package.json | 2 +- packages/sdk/package.json | 2 +- pnpm-lock.yaml | 130 ++++++++++++++++--------------- 4 files changed, 70 insertions(+), 66 deletions(-) diff --git a/package.json b/package.json index c30e072afdc5..9a94eb9d18ff 100644 --- a/package.json +++ b/package.json @@ -43,7 +43,7 @@ "@types/chai": "^4.3.8", "@types/chai-as-promised": "^7.1.4", "@types/mocha": "^10.0.3", - "@types/node": "^20.8.8", + "@types/node": "^20.8.9", "@typescript-eslint/eslint-plugin": "^6.9.0", "@typescript-eslint/parser": "^6.9.0", "chai": "^4.3.10", diff --git a/packages/core-utils/package.json b/packages/core-utils/package.json index 44194a74da0c..ec7ad4020352 100644 --- a/packages/core-utils/package.json +++ b/packages/core-utils/package.json @@ -49,7 +49,7 @@ "node-fetch": "^2.6.7" }, "devDependencies": { - "@types/node": "^20.8.8", + "@types/node": "^20.8.9", "mocha": "^10.2.0" } } diff --git a/packages/sdk/package.json b/packages/sdk/package.json index cbf651cfd58f..be729bed2af0 100644 --- a/packages/sdk/package.json +++ b/packages/sdk/package.json @@ -44,7 +44,7 @@ "@types/chai": "^4.3.8", "@types/chai-as-promised": "^7.1.5", "@types/mocha": "^10.0.3", - "@types/node": "^20.8.8", + "@types/node": "^20.8.9", "chai-as-promised": "^7.1.1", "ethereum-waffle": "^4.0.10", "ethers": "^5.7.2", diff --git a/pnpm-lock.yaml b/pnpm-lock.yaml index a511fe527501..3b54280963db 100644 --- a/pnpm-lock.yaml +++ b/pnpm-lock.yaml @@ -31,8 +31,8 @@ importers: specifier: ^10.0.3 version: 10.0.3 '@types/node': - specifier: ^20.8.8 - version: 20.8.8 + specifier: ^20.8.9 + version: 20.8.9 '@typescript-eslint/eslint-plugin': specifier: ^6.9.0 version: 6.9.0(@typescript-eslint/parser@6.9.0)(eslint@8.52.0)(typescript@5.2.2) @@ -184,7 +184,7 @@ importers: version: 2.18.3(ts-node@10.9.1)(typescript@5.2.2) ts-node: specifier: ^10.9.1 - version: 10.9.1(@types/node@20.8.8)(typescript@5.2.2) + version: 10.9.1(@types/node@20.8.9)(typescript@5.2.2) tsx: specifier: ^3.14.0 version: 3.14.0 @@ -348,7 +348,7 @@ importers: version: 5.2.2 vite: specifier: ^4.5.0 - version: 4.5.0(@types/node@20.8.8) + version: 4.5.0(@types/node@20.8.9) vitest: specifier: ^0.34.2 version: 0.34.2(jsdom@22.1.0) @@ -399,8 +399,8 @@ importers: version: 2.6.7 devDependencies: '@types/node': - specifier: ^20.8.8 - version: 20.8.8 + specifier: ^20.8.9 + version: 20.8.9 mocha: specifier: ^10.2.0 version: 10.2.0 @@ -442,7 +442,7 @@ importers: version: 1.16.6(typescript@5.2.2)(zod@3.22.4) vite: specifier: ^4.5.0 - version: 4.5.0(@types/node@20.8.8) + version: 4.5.0(@types/node@20.8.9) vitest: specifier: ^0.34.2 version: 0.34.2(jsdom@22.1.0) @@ -493,8 +493,8 @@ importers: specifier: ^10.0.3 version: 10.0.3 '@types/node': - specifier: ^20.8.8 - version: 20.8.8 + specifier: ^20.8.9 + version: 20.8.9 chai-as-promised: specifier: ^7.1.1 version: 7.1.1(chai@4.3.10) @@ -521,7 +521,7 @@ importers: version: 15.1.0 ts-node: specifier: ^10.9.1 - version: 10.9.1(@types/node@20.8.8)(typescript@5.2.2) + version: 10.9.1(@types/node@20.8.9)(typescript@5.2.2) typedoc: specifier: ^0.25.2 version: 0.25.2(typescript@5.2.2) @@ -573,7 +573,7 @@ importers: version: 1.16.6(typescript@5.2.2)(zod@3.22.4) vite: specifier: ^4.5.0 - version: 4.5.0(@types/node@20.8.8) + version: 4.5.0(@types/node@20.8.9) vitest: specifier: ^0.34.1 version: 0.34.1 @@ -3882,20 +3882,20 @@ packages: /@types/bn.js@4.11.6: resolution: {integrity: sha512-pqr857jrp2kPuO9uRjZ3PwnJTjoQy+fcdxvBTvHm6dkmEL9q+hDD/2j/0ELOBPtPnS8LjCX0gI9nbl8lVkadpg==} dependencies: - '@types/node': 20.8.8 + '@types/node': 20.8.9 dev: true /@types/bn.js@5.1.0: resolution: {integrity: sha512-QSSVYj7pYFN49kW77o2s9xTCwZ8F2xLbjLLSEVh8D2F4JUhZtPAGOFLTD+ffqksBx/u4cE/KImFjyhqCjn/LIA==} dependencies: - '@types/node': 20.8.8 + '@types/node': 20.8.9 dev: true /@types/body-parser@1.19.1: resolution: {integrity: sha512-a6bTJ21vFOGIkwM0kzh9Yr89ziVxq4vYH2fQ6N8AeipEzai/cFK6aGMArIkUeIdRIgpwQa+2bXiLuUJCpSf2Cg==} dependencies: '@types/connect': 3.4.35 - '@types/node': 20.8.8 + '@types/node': 20.8.9 dev: true /@types/chai-as-promised@7.1.5: @@ -3928,7 +3928,7 @@ packages: /@types/connect@3.4.35: resolution: {integrity: sha512-cdeYyv4KWoEgpBISTxWvqYsVy444DOqehiF3fM3ne10AmJ62RSyNkUnxMJXHQWRQQX2eR94m5y1IZyDwBjV9FQ==} dependencies: - '@types/node': 20.8.8 + '@types/node': 20.8.9 /@types/dateformat@5.0.0: resolution: {integrity: sha512-SZg4JdHIWHQGEokbYGZSDvo5wA4TLYPXaqhigs/wH+REDOejcJzgH+qyY+HtEUtWOZxEUkbhbdYPqQDiEgrXeA==} @@ -3942,7 +3942,7 @@ packages: /@types/express-serve-static-core@4.17.35: resolution: {integrity: sha512-wALWQwrgiB2AWTT91CB62b6Yt0sNHpznUXeZEcnPU3DRdlDIz74x8Qg1UUYKSVFi+va5vKOLYRBI1bRKiLLKIg==} dependencies: - '@types/node': 20.8.8 + '@types/node': 20.8.9 '@types/qs': 6.9.7 '@types/range-parser': 1.2.4 '@types/send': 0.17.1 @@ -3961,7 +3961,7 @@ packages: resolution: {integrity: sha512-IO+MJPVhoqz+28h1qLAcBEH2+xHMK6MTyHJc7MTnnYb6wsoLR29POVGJ7LycmVXIqyy/4/2ShP5sUwTXuOwb/w==} dependencies: '@types/minimatch': 5.1.2 - '@types/node': 20.8.6 + '@types/node': 20.8.8 dev: true /@types/is-ci@3.0.0: @@ -3988,7 +3988,7 @@ packages: dependencies: '@types/abstract-leveldown': 5.0.2 '@types/level-errors': 3.0.0 - '@types/node': 20.8.8 + '@types/node': 20.8.9 dev: true /@types/lru-cache@5.1.1: @@ -4020,7 +4020,7 @@ packages: /@types/mkdirp@0.5.2: resolution: {integrity: sha512-U5icWpv7YnZYGsN4/cmh3WD2onMY0aJIiTE6+51TwJCttdHvtCYmkBNOobHlXwrJRL0nkH9jH4kD+1FAdMN4Tg==} dependencies: - '@types/node': 20.8.8 + '@types/node': 20.8.9 dev: true /@types/mocha@10.0.3: @@ -4039,7 +4039,7 @@ packages: /@types/node-fetch@2.6.4: resolution: {integrity: sha512-1ZX9fcN4Rvkvgv4E6PAY5WXUFWFcRWxZa3EW83UjycOB9ljJCedb2CupIP4RZMEwF/M3eTcCihbBRgwtGbg5Rg==} dependencies: - '@types/node': 20.8.8 + '@types/node': 20.8.9 form-data: 3.0.1 dev: true @@ -4050,16 +4050,16 @@ packages: /@types/node@12.20.55: resolution: {integrity: sha512-J8xLz7q2OFulZ2cyGTLE1TbbZcjpno7FaN6zdJNrgAdrJ+DZzh/uFR6YrTb4C+nXakvud8Q4+rbhoIWlYQbUFQ==} - /@types/node@20.8.6: - resolution: {integrity: sha512-eWO4K2Ji70QzKUqRy6oyJWUeB7+g2cRagT3T/nxYibYcT4y2BDL8lqolRXjTHmkZCdJfIPaY73KbJAZmcryxTQ==} + /@types/node@20.8.8: + resolution: {integrity: sha512-YRsdVxq6OaLfmR9Hy816IMp33xOBjfyOgUd77ehqg96CFywxAPbDbXvAsuN2KVg2HOT8Eh6uAfU+l4WffwPVrQ==} dependencies: undici-types: 5.25.3 dev: true - /@types/node@20.8.8: - resolution: {integrity: sha512-YRsdVxq6OaLfmR9Hy816IMp33xOBjfyOgUd77ehqg96CFywxAPbDbXvAsuN2KVg2HOT8Eh6uAfU+l4WffwPVrQ==} + /@types/node@20.8.9: + resolution: {integrity: sha512-UzykFsT3FhHb1h7yD4CA4YhBHq545JC0YnEz41xkipN88eKQtL6rSgocL5tbAP6Ola9Izm/Aw4Ora8He4x0BHg==} dependencies: - undici-types: 5.25.3 + undici-types: 5.26.5 /@types/normalize-package-data@2.4.1: resolution: {integrity: sha512-Gj7cI7z+98M282Tqmp2K5EIsoouUEzbBJhQQzDE3jSIRk6r9gsz0oUokqIUR4u1R3dMHo0pDHM7sNOHyhulypw==} @@ -4071,7 +4071,7 @@ packages: /@types/pbkdf2@3.1.0: resolution: {integrity: sha512-Cf63Rv7jCQ0LaL8tNXmEyqTHuIJxRdlS5vMh1mj5voN4+QFhVZnlZruezqpWYDiJ8UTzhP0VmeLXCmBk66YrMQ==} dependencies: - '@types/node': 20.8.8 + '@types/node': 20.8.9 dev: true /@types/pino-multi-stream@5.1.5: @@ -4089,13 +4089,13 @@ packages: /@types/pino-std-serializers@2.4.1: resolution: {integrity: sha512-17XcksO47M24IVTVKPeAByWUd3Oez7EbIjXpSbzMPhXVzgjGtrOa49gKBwxH9hb8dKv58OelsWQ+A1G1l9S3wQ==} dependencies: - '@types/node': 20.8.8 + '@types/node': 20.8.9 dev: true /@types/pino@6.3.11: resolution: {integrity: sha512-S7+fLONqSpHeW9d7TApUqO6VN47KYgOXhCNKwGBVLHObq8HhaAYlVqUNdfnvoXjCMiwE5xcPm/5R2ZUh8bgaXQ==} dependencies: - '@types/node': 20.8.8 + '@types/node': 20.8.9 '@types/pino-pretty': 4.7.1 '@types/pino-std-serializers': 2.4.1 sonic-boom: 2.8.0 @@ -4141,7 +4141,7 @@ packages: /@types/readable-stream@2.3.15: resolution: {integrity: sha512-oM5JSKQCcICF1wvGgmecmHldZ48OZamtMxcGGVICOJA8o8cahXC1zEVAif8iwoc5j8etxFaRFnf095+CDsuoFQ==} dependencies: - '@types/node': 20.8.8 + '@types/node': 20.8.9 safe-buffer: 5.1.2 dev: true @@ -4152,7 +4152,7 @@ packages: /@types/secp256k1@4.0.3: resolution: {integrity: sha512-Da66lEIFeIz9ltsdMZcpQvmrmmoqrfju8pm1BH8WbYjZSwUgCwXLb9C+9XYogwBITnbsSaMdVPb2ekf7TV+03w==} dependencies: - '@types/node': 20.8.8 + '@types/node': 20.8.9 dev: true /@types/seedrandom@3.0.1: @@ -4171,14 +4171,14 @@ packages: resolution: {integrity: sha512-Cwo8LE/0rnvX7kIIa3QHCkcuF21c05Ayb0ZfxPiv0W8VRiZiNW/WuRupHKpqqGVGf7SUA44QSOUKaEd9lIrd/Q==} dependencies: '@types/mime': 1.3.2 - '@types/node': 20.8.8 + '@types/node': 20.8.9 dev: true /@types/serve-static@1.13.10: resolution: {integrity: sha512-nCkHGI4w7ZgAdNkrEu0bv+4xNV/XDqW+DydknebMOQwkpDGx8G+HTlj7R7ABI8i8nKxVw0wtKPi1D+lPOkh4YQ==} dependencies: '@types/mime': 1.3.2 - '@types/node': 20.8.8 + '@types/node': 20.8.9 dev: true /@types/sinon-chai@3.2.5: @@ -4215,12 +4215,12 @@ packages: /@types/ws@7.4.7: resolution: {integrity: sha512-JQbbmxZTZehdc2iszGKs5oC3NFnjeay7mtAWrdt7qNtAVK0g19muApzAy4bm9byz79xa2ZnO/BOBC2R8RC5Lww==} dependencies: - '@types/node': 20.8.8 + '@types/node': 20.8.9 /@types/ws@8.5.3: resolution: {integrity: sha512-6YOoWjruKj1uLf3INHH7D3qTXwFfEsg1kf3c0uDdSBJwfa/llkwIjrAGV7j7mVgGNbzTQ3HiHKKDXl6bJPD97w==} dependencies: - '@types/node': 20.8.8 + '@types/node': 20.8.9 dev: false /@typescript-eslint/eslint-plugin@6.9.0(@typescript-eslint/parser@6.9.0)(eslint@8.52.0)(typescript@5.2.2): @@ -8985,7 +8985,7 @@ packages: solc: 0.7.3(debug@4.3.4) source-map-support: 0.5.21 stacktrace-parser: 0.1.10 - ts-node: 10.9.1(@types/node@20.8.8)(typescript@5.2.2) + ts-node: 10.9.1(@types/node@20.8.9)(typescript@5.2.2) tsort: 0.0.1 typescript: 5.2.2 undici: 5.24.0 @@ -13739,7 +13739,7 @@ packages: tsconfig-paths: 3.14.2 dev: true - /ts-node@10.9.1(@types/node@20.8.8)(typescript@5.2.2): + /ts-node@10.9.1(@types/node@20.8.9)(typescript@5.2.2): resolution: {integrity: sha512-NtVysVPkxxrwFGUUxGYhfux8k78pQB3JqYBXlLRZgdGUqTO5wU/UyHop5p70iEbGhB7q5KmiZiU0Y3KlJrScEw==} hasBin: true peerDependencies: @@ -13758,7 +13758,7 @@ packages: '@tsconfig/node12': 1.0.11 '@tsconfig/node14': 1.0.3 '@tsconfig/node16': 1.0.4 - '@types/node': 20.8.8 + '@types/node': 20.8.9 acorn: 8.10.0 acorn-walk: 8.2.0 arg: 4.1.3 @@ -14081,6 +14081,10 @@ packages: /undici-types@5.25.3: resolution: {integrity: sha512-Ga1jfYwRn7+cP9v8auvEXN1rX3sWqlayd4HP7OKk4mZWylEmu3KzXDUGrQUN6Ol7qo1gPvB2e5gX6udnyEPgdA==} + dev: true + + /undici-types@5.26.5: + resolution: {integrity: sha512-JlCMO+ehdEIKqlFxk6IfVoAUVmgz7cU7zD/h9XZ0qzeosSHmUJVOzSQvvYSYWXkFXC+IfLKSIffhv0sVZup6pA==} /undici@5.24.0: resolution: {integrity: sha512-OKlckxBjFl0oXxcj9FU6oB8fDAaiRUq+D8jrFWGmOfI/gIyjk/IeS75LMzgYKUaeHzLUcYvf9bbJGSrUwTfwwQ==} @@ -14359,7 +14363,7 @@ packages: - utf-8-validate - zod - /vite-node@0.34.1(@types/node@20.8.8): + /vite-node@0.34.1(@types/node@20.8.9): resolution: {integrity: sha512-odAZAL9xFMuAg8aWd7nSPT+hU8u2r9gU3LRm9QKjxBEF2rRdWpMuqkrkjvyVQEdNFiBctqr2Gg4uJYizm5Le6w==} engines: {node: '>=v14.18.0'} hasBin: true @@ -14369,7 +14373,7 @@ packages: mlly: 1.4.0 pathe: 1.1.1 picocolors: 1.0.0 - vite: 4.5.0(@types/node@20.8.8) + vite: 4.5.0(@types/node@20.8.9) transitivePeerDependencies: - '@types/node' - less @@ -14381,7 +14385,7 @@ packages: - terser dev: true - /vite-node@0.34.2(@types/node@20.8.6): + /vite-node@0.34.2(@types/node@20.8.8): resolution: {integrity: sha512-JtW249Zm3FB+F7pQfH56uWSdlltCo1IOkZW5oHBzeQo0iX4jtC7o1t9aILMGd9kVekXBP2lfJBEQt9rBh07ebA==} engines: {node: '>=v14.18.0'} hasBin: true @@ -14391,7 +14395,7 @@ packages: mlly: 1.4.0 pathe: 1.1.1 picocolors: 1.0.0 - vite: 4.5.0(@types/node@20.8.6) + vite: 4.5.0(@types/node@20.8.8) transitivePeerDependencies: - '@types/node' - less @@ -14403,7 +14407,7 @@ packages: - terser dev: true - /vite-node@0.34.2(@types/node@20.8.8): + /vite-node@0.34.2(@types/node@20.8.9): resolution: {integrity: sha512-JtW249Zm3FB+F7pQfH56uWSdlltCo1IOkZW5oHBzeQo0iX4jtC7o1t9aILMGd9kVekXBP2lfJBEQt9rBh07ebA==} engines: {node: '>=v14.18.0'} hasBin: true @@ -14413,7 +14417,7 @@ packages: mlly: 1.4.0 pathe: 1.1.1 picocolors: 1.0.0 - vite: 4.5.0(@types/node@20.8.8) + vite: 4.5.0(@types/node@20.8.9) transitivePeerDependencies: - '@types/node' - less @@ -14425,7 +14429,7 @@ packages: - terser dev: true - /vite-node@0.34.4(@types/node@20.8.6): + /vite-node@0.34.4(@types/node@20.8.8): resolution: {integrity: sha512-ho8HtiLc+nsmbwZMw8SlghESEE3KxJNp04F/jPUCLVvaURwt0d+r9LxEqCX5hvrrOQ0GSyxbYr5ZfRYhQ0yVKQ==} engines: {node: '>=v14.18.0'} hasBin: true @@ -14435,7 +14439,7 @@ packages: mlly: 1.4.0 pathe: 1.1.1 picocolors: 1.0.0 - vite: 4.5.0(@types/node@20.8.6) + vite: 4.5.0(@types/node@20.8.8) transitivePeerDependencies: - '@types/node' - less @@ -14447,7 +14451,7 @@ packages: - terser dev: true - /vite@4.4.10(@types/node@20.8.6): + /vite@4.4.10(@types/node@20.8.8): resolution: {integrity: sha512-TzIjiqx9BEXF8yzYdF2NTf1kFFbjMjUSV0LFZ3HyHoI3SGSPLnnFUKiIQtL3gl2AjHvMrprOvQ3amzaHgQlAxw==} engines: {node: ^14.18.0 || >=16.0.0} hasBin: true @@ -14475,7 +14479,7 @@ packages: terser: optional: true dependencies: - '@types/node': 20.8.6 + '@types/node': 20.8.8 esbuild: 0.18.20 postcss: 8.4.27 rollup: 3.28.0 @@ -14483,7 +14487,7 @@ packages: fsevents: 2.3.3 dev: true - /vite@4.5.0(@types/node@20.8.6): + /vite@4.5.0(@types/node@20.8.8): resolution: {integrity: sha512-ulr8rNLA6rkyFAlVWw2q5YJ91v098AFQ2R0PRFwPzREXOUJQPtFUG0t+/ZikhaOCDqFoDhN6/v8Sq0o4araFAw==} engines: {node: ^14.18.0 || >=16.0.0} hasBin: true @@ -14511,7 +14515,7 @@ packages: terser: optional: true dependencies: - '@types/node': 20.8.6 + '@types/node': 20.8.8 esbuild: 0.18.20 postcss: 8.4.27 rollup: 3.28.0 @@ -14519,7 +14523,7 @@ packages: fsevents: 2.3.3 dev: true - /vite@4.5.0(@types/node@20.8.8): + /vite@4.5.0(@types/node@20.8.9): resolution: {integrity: sha512-ulr8rNLA6rkyFAlVWw2q5YJ91v098AFQ2R0PRFwPzREXOUJQPtFUG0t+/ZikhaOCDqFoDhN6/v8Sq0o4araFAw==} engines: {node: ^14.18.0 || >=16.0.0} hasBin: true @@ -14547,7 +14551,7 @@ packages: terser: optional: true dependencies: - '@types/node': 20.8.8 + '@types/node': 20.8.9 esbuild: 0.18.20 postcss: 8.4.27 rollup: 3.28.0 @@ -14588,7 +14592,7 @@ packages: dependencies: '@types/chai': 4.3.8 '@types/chai-subset': 1.3.3 - '@types/node': 20.8.8 + '@types/node': 20.8.9 '@vitest/expect': 0.34.1 '@vitest/runner': 0.34.1 '@vitest/snapshot': 0.34.1 @@ -14607,8 +14611,8 @@ packages: strip-literal: 1.0.1 tinybench: 2.5.0 tinypool: 0.7.0 - vite: 4.5.0(@types/node@20.8.8) - vite-node: 0.34.1(@types/node@20.8.8) + vite: 4.5.0(@types/node@20.8.9) + vite-node: 0.34.1(@types/node@20.8.9) why-is-node-running: 2.2.2 transitivePeerDependencies: - less @@ -14653,7 +14657,7 @@ packages: dependencies: '@types/chai': 4.3.8 '@types/chai-subset': 1.3.3 - '@types/node': 20.8.8 + '@types/node': 20.8.9 '@vitest/expect': 0.34.2 '@vitest/runner': 0.34.2 '@vitest/snapshot': 0.34.2 @@ -14672,8 +14676,8 @@ packages: strip-literal: 1.0.1 tinybench: 2.5.0 tinypool: 0.7.0 - vite: 4.5.0(@types/node@20.8.8) - vite-node: 0.34.2(@types/node@20.8.8) + vite: 4.5.0(@types/node@20.8.9) + vite-node: 0.34.2(@types/node@20.8.9) why-is-node-running: 2.2.2 transitivePeerDependencies: - less @@ -14718,7 +14722,7 @@ packages: dependencies: '@types/chai': 4.3.7 '@types/chai-subset': 1.3.3 - '@types/node': 20.8.6 + '@types/node': 20.8.8 '@vitest/expect': 0.34.2 '@vitest/runner': 0.34.2 '@vitest/snapshot': 0.34.2 @@ -14738,8 +14742,8 @@ packages: strip-literal: 1.0.1 tinybench: 2.5.0 tinypool: 0.7.0 - vite: 4.5.0(@types/node@20.8.6) - vite-node: 0.34.2(@types/node@20.8.6) + vite: 4.5.0(@types/node@20.8.8) + vite-node: 0.34.2(@types/node@20.8.8) why-is-node-running: 2.2.2 transitivePeerDependencies: - less @@ -14784,7 +14788,7 @@ packages: dependencies: '@types/chai': 4.3.7 '@types/chai-subset': 1.3.3 - '@types/node': 20.8.6 + '@types/node': 20.8.8 '@vitest/expect': 0.34.4 '@vitest/runner': 0.34.4 '@vitest/snapshot': 0.34.4 @@ -14803,8 +14807,8 @@ packages: strip-literal: 1.0.1 tinybench: 2.5.0 tinypool: 0.7.0 - vite: 4.4.10(@types/node@20.8.6) - vite-node: 0.34.4(@types/node@20.8.6) + vite: 4.4.10(@types/node@20.8.8) + vite-node: 0.34.4(@types/node@20.8.8) why-is-node-running: 2.2.2 transitivePeerDependencies: - less From c58bf1c14a5141f89d56f575fa0f75ad1d051c2c Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Thu, 26 Oct 2023 09:26:31 +1000 Subject: [PATCH 093/279] op-e2e: Increase timeouts for cannon Some tests are failing because the challenger can't generate the cannon trace before the timeout. Better to have long timeouts than flaky tests. --- op-e2e/e2eutils/disputegame/game_helper.go | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/op-e2e/e2eutils/disputegame/game_helper.go b/op-e2e/e2eutils/disputegame/game_helper.go index 14279aa00f83..5563d437fc59 100644 --- a/op-e2e/e2eutils/disputegame/game_helper.go +++ b/op-e2e/e2eutils/disputegame/game_helper.go @@ -18,6 +18,8 @@ import ( "github.com/stretchr/testify/require" ) +const defaultTimeout = 5 * time.Minute + type FaultGameHelper struct { t *testing.T require *require.Assertions @@ -42,7 +44,7 @@ func (g *FaultGameHelper) GameDuration(ctx context.Context) time.Duration { // This does not check that the number of claims is exactly the specified count to avoid intermittent failures // where a challenger posts an additional claim before this method sees the number of claims it was waiting for. func (g *FaultGameHelper) WaitForClaimCount(ctx context.Context, count int64) { - ctx, cancel := context.WithTimeout(ctx, 2*time.Minute) + ctx, cancel := context.WithTimeout(ctx, defaultTimeout) defer cancel() err := wait.For(ctx, time.Second, func() (bool, error) { actual, err := g.game.ClaimDataLen(&bind.CallOpts{Context: ctx}) @@ -70,7 +72,7 @@ func (g *FaultGameHelper) MaxDepth(ctx context.Context) int64 { } func (g *FaultGameHelper) waitForClaim(ctx context.Context, errorMsg string, predicate func(claim ContractClaim) bool) { - timedCtx, cancel := context.WithTimeout(ctx, 2*time.Minute) + timedCtx, cancel := context.WithTimeout(ctx, defaultTimeout) defer cancel() err := wait.For(timedCtx, time.Second, func() (bool, error) { count, err := g.game.ClaimDataLen(&bind.CallOpts{Context: timedCtx}) @@ -95,7 +97,7 @@ func (g *FaultGameHelper) waitForClaim(ctx context.Context, errorMsg string, pre } func (g *FaultGameHelper) waitForNoClaim(ctx context.Context, errorMsg string, predicate func(claim ContractClaim) bool) { - timedCtx, cancel := context.WithTimeout(ctx, 3*time.Minute) + timedCtx, cancel := context.WithTimeout(ctx, defaultTimeout) defer cancel() err := wait.For(timedCtx, time.Second, func() (bool, error) { count, err := g.game.ClaimDataLen(&bind.CallOpts{Context: timedCtx}) @@ -193,7 +195,7 @@ func (g *FaultGameHelper) Status(ctx context.Context) Status { func (g *FaultGameHelper) WaitForGameStatus(ctx context.Context, expected Status) { g.t.Logf("Waiting for game %v to have status %v", g.addr, expected) - timedCtx, cancel := context.WithTimeout(ctx, time.Minute) + timedCtx, cancel := context.WithTimeout(ctx, defaultTimeout) defer cancel() err := wait.For(timedCtx, time.Second, func() (bool, error) { ctx, cancel := context.WithTimeout(timedCtx, 30*time.Second) @@ -302,7 +304,7 @@ func (g *FaultGameHelper) ChallengeRootClaim(ctx context.Context, performMove Mo } func (g *FaultGameHelper) WaitForNewClaim(ctx context.Context, checkPoint int64) (int64, error) { - timedCtx, cancel := context.WithTimeout(ctx, 2*time.Minute) + timedCtx, cancel := context.WithTimeout(ctx, defaultTimeout) defer cancel() var newClaimLen int64 err := wait.For(timedCtx, time.Second, func() (bool, error) { From 20fa6c579355e751cfd562c636bf897fc3fde17a Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Thu, 26 Oct 2023 10:12:18 +1000 Subject: [PATCH 094/279] op-challenger: Unsubscribe from l1 heads on shutdown Avoids leaving a background function running that could trigger logging. --- op-challenger/game/monitor.go | 1 + op-challenger/game/monitor_test.go | 3 ++- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/op-challenger/game/monitor.go b/op-challenger/game/monitor.go index b6ddc64adcea..a8c9c694b20c 100644 --- a/op-challenger/game/monitor.go +++ b/op-challenger/game/monitor.go @@ -141,6 +141,7 @@ func (m *gameMonitor) MonitorGames(ctx context.Context) error { for { select { case <-ctx.Done(): + m.l1HeadsSub.Unsubscribe() return nil case err, ok := <-m.l1HeadsSub.Err(): if !ok { diff --git a/op-challenger/game/monitor_test.go b/op-challenger/game/monitor_test.go index b869dad350f6..7295fdb74a3d 100644 --- a/op-challenger/game/monitor_test.go +++ b/op-challenger/game/monitor_test.go @@ -100,7 +100,8 @@ func TestMonitorGames(t *testing.T) { defer cancel() go func() { - waitErr := wait.For(context.Background(), 100*time.Millisecond, func() (bool, error) { + // Wait for the subscription to be created + waitErr := wait.For(context.Background(), 5*time.Second, func() (bool, error) { return mockHeadSource.sub != nil, nil }) require.NoError(t, waitErr) From 7a6ae801a6a841bb60b5a7abbc7f192f1b50d05f Mon Sep 17 00:00:00 2001 From: Mark Tyneway Date: Wed, 25 Oct 2023 19:14:55 -0600 Subject: [PATCH 095/279] contracts-bedrock: modern import style in tests Migrate to the modern import style in the tests where the path relative from the project root is used. This makes the code much more portable and easier to modify without the dot dot syntax. There should be very little old relative style imports left in the codebase if any. There are no functional changes with this commit. --- .../test/AddressAliasHelper.t.sol | 2 +- .../test/AdminFaucetAuthModule.t.sol | 6 +++--- .../contracts-bedrock/test/AssetReceiver.t.sol | 6 +++--- .../test/AttestationStation.t.sol | 2 +- .../contracts-bedrock/test/BenchmarkTest.t.sol | 6 +++--- packages/contracts-bedrock/test/Bytes.t.sol | 2 +- .../test/CheckBalanceHigh.t.sol | 2 +- .../test/CheckBalanceLow.t.sol | 2 +- .../test/CheckGelatoLow.t.sol | 2 +- .../contracts-bedrock/test/CheckTrue.t.sol | 2 +- packages/contracts-bedrock/test/Clones.t.sol | 2 +- .../test/CrossDomainMessenger.t.sol | 4 ++-- .../test/CrossDomainOwnable.t.sol | 2 +- .../test/CrossDomainOwnable2.t.sol | 6 +++--- .../test/CrossDomainOwnable3.t.sol | 10 +++++----- .../test/DelayedVetoable.t.sol | 2 +- .../test/DeployerWhitelist.t.sol | 4 ++-- .../test/DisputeGameFactory.t.sol | 12 ++++++------ packages/contracts-bedrock/test/Drippie.t.sol | 8 ++++---- packages/contracts-bedrock/test/Encoding.t.sol | 8 ++++---- packages/contracts-bedrock/test/Faucet.t.sol | 6 +++--- .../test/FaultDisputeGame.t.sol | 2 +- packages/contracts-bedrock/test/FeeVault.t.sol | 12 ++++++------ .../test/GasPriceOracle.t.sol | 8 ++++---- .../test/GovernanceToken.t.sol | 4 ++-- packages/contracts-bedrock/test/Hashing.t.sol | 10 +++++----- packages/contracts-bedrock/test/Helpers.sol | 4 ++-- packages/contracts-bedrock/test/L1Block.t.sol | 4 ++-- .../contracts-bedrock/test/L1BlockNumber.t.sol | 6 +++--- .../test/L1CrossDomainMessenger.t.sol | 18 +++++++++--------- .../test/L1ERC721Bridge.t.sol | 8 ++++---- .../test/L1StandardBridge.t.sol | 2 +- .../test/L2CrossDomainMessenger.t.sol | 16 ++++++++-------- .../test/L2ERC721Bridge.t.sol | 8 ++++---- .../test/L2OutputOracle.t.sol | 2 +- .../test/L2StandardBridge.t.sol | 16 ++++++++-------- .../test/L2ToL1MessagePasser.t.sol | 8 ++++---- .../test/LegacyERC20ETH.t.sol | 6 +++--- .../test/LegacyMessagePasser.t.sol | 6 +++--- packages/contracts-bedrock/test/LibClock.t.sol | 2 +- .../contracts-bedrock/test/LibPosition.t.sol | 2 +- .../contracts-bedrock/test/MerkleTrie.t.sol | 4 ++-- .../contracts-bedrock/test/MintManager.t.sol | 6 +++--- .../test/OptimismMintableERC20.t.sol | 4 ++-- .../test/OptimismMintableERC20Factory.t.sol | 4 ++-- .../test/OptimismMintableERC721.t.sol | 4 ++-- .../test/OptimismMintableERC721Factory.t.sol | 6 +++--- .../test/OptimismPortal.t.sol | 2 +- packages/contracts-bedrock/test/Optimist.t.sol | 10 +++++----- .../test/OptimistAllowlist.t.sol | 10 +++++----- .../test/OptimistInviter.t.sol | 12 ++++++------ .../test/ProtocolVersions.t.sol | 2 +- packages/contracts-bedrock/test/Proxy.t.sol | 2 +- .../contracts-bedrock/test/ProxyAdmin.t.sol | 12 ++++++------ .../contracts-bedrock/test/RLPReader.t.sol | 4 ++-- .../contracts-bedrock/test/RLPWriter.t.sol | 4 ++-- .../test/ResolvedDelegateProxy.t.sol | 4 ++-- .../test/ResourceMetering.t.sol | 6 +++--- packages/contracts-bedrock/test/SafeCall.t.sol | 4 ++-- packages/contracts-bedrock/test/Semver.t.sol | 6 +++--- .../test/SequencerFeeVault.t.sol | 10 +++++----- .../test/StandardBridge.t.sol | 6 +++--- .../contracts-bedrock/test/SystemConfig.t.sol | 10 +++++----- .../contracts-bedrock/test/Transactor.t.sol | 6 +++--- .../contracts-bedrock/test/TransferOnion.t.sol | 2 +- .../test/invariants/AddressAliasHelper.t.sol | 2 +- .../test/invariants/Burn.Eth.t.sol | 2 +- .../test/invariants/Burn.Gas.t.sol | 2 +- .../test/invariants/CrossDomainMessenger.t.sol | 16 ++++++++-------- .../test/invariants/Encoding.t.sol | 2 +- .../test/invariants/Hashing.t.sol | 4 ++-- .../test/invariants/L2OutputOracle.t.sol | 4 ++-- .../test/invariants/OptimismPortal.t.sol | 18 +++++++++--------- .../test/invariants/ResourceMetering.t.sol | 8 ++++---- .../test/invariants/SafeCall.t.sol | 2 +- .../test/invariants/SystemConfig.t.sol | 8 ++++---- 76 files changed, 224 insertions(+), 224 deletions(-) diff --git a/packages/contracts-bedrock/test/AddressAliasHelper.t.sol b/packages/contracts-bedrock/test/AddressAliasHelper.t.sol index d0c5440a5511..45211879c17f 100644 --- a/packages/contracts-bedrock/test/AddressAliasHelper.t.sol +++ b/packages/contracts-bedrock/test/AddressAliasHelper.t.sol @@ -2,7 +2,7 @@ pragma solidity 0.8.15; import { Test } from "forge-std/Test.sol"; -import { AddressAliasHelper } from "../src/vendor/AddressAliasHelper.sol"; +import { AddressAliasHelper } from "src/vendor/AddressAliasHelper.sol"; contract AddressAliasHelper_applyAndUndo_Test is Test { /// @notice Tests that applying and then undoing an alias results in the original address. diff --git a/packages/contracts-bedrock/test/AdminFaucetAuthModule.t.sol b/packages/contracts-bedrock/test/AdminFaucetAuthModule.t.sol index 3a59ba7e70a4..340bbfe4aa83 100644 --- a/packages/contracts-bedrock/test/AdminFaucetAuthModule.t.sol +++ b/packages/contracts-bedrock/test/AdminFaucetAuthModule.t.sol @@ -2,9 +2,9 @@ pragma solidity 0.8.15; import { Test } from "forge-std/Test.sol"; -import { AdminFaucetAuthModule } from "../src/periphery/faucet/authmodules/AdminFaucetAuthModule.sol"; -import { Faucet } from "../src/periphery/faucet/Faucet.sol"; -import { FaucetHelper } from "./Helpers.sol"; +import { AdminFaucetAuthModule } from "src/periphery/faucet/authmodules/AdminFaucetAuthModule.sol"; +import { Faucet } from "src/periphery/faucet/Faucet.sol"; +import { FaucetHelper } from "test/Helpers.sol"; /// @title AdminFaucetAuthModuleTest /// @notice Tests the AdminFaucetAuthModule contract. diff --git a/packages/contracts-bedrock/test/AssetReceiver.t.sol b/packages/contracts-bedrock/test/AssetReceiver.t.sol index a7f3693423c7..87c14e2608dd 100644 --- a/packages/contracts-bedrock/test/AssetReceiver.t.sol +++ b/packages/contracts-bedrock/test/AssetReceiver.t.sol @@ -3,9 +3,9 @@ pragma solidity 0.8.15; // Testing utilities import { Test } from "forge-std/Test.sol"; -import { TestERC20 } from "./Helpers.sol"; -import { TestERC721 } from "./Helpers.sol"; -import { AssetReceiver } from "../src/periphery/AssetReceiver.sol"; +import { TestERC20 } from "test/Helpers.sol"; +import { TestERC721 } from "test/Helpers.sol"; +import { AssetReceiver } from "src/periphery/AssetReceiver.sol"; contract AssetReceiver_Initializer is Test { address alice = address(128); diff --git a/packages/contracts-bedrock/test/AttestationStation.t.sol b/packages/contracts-bedrock/test/AttestationStation.t.sol index 5d17fcd0e14f..4c9b72254d34 100644 --- a/packages/contracts-bedrock/test/AttestationStation.t.sol +++ b/packages/contracts-bedrock/test/AttestationStation.t.sol @@ -3,7 +3,7 @@ pragma solidity 0.8.15; /* Testing utilities */ import { Test } from "forge-std/Test.sol"; -import { AttestationStation } from "../src/periphery/op-nft/AttestationStation.sol"; +import { AttestationStation } from "src/periphery/op-nft/AttestationStation.sol"; contract AttestationStation_Initializer is Test { address alice_attestor = address(128); diff --git a/packages/contracts-bedrock/test/BenchmarkTest.t.sol b/packages/contracts-bedrock/test/BenchmarkTest.t.sol index 64b9d982cf7d..780162b66b0d 100644 --- a/packages/contracts-bedrock/test/BenchmarkTest.t.sol +++ b/packages/contracts-bedrock/test/BenchmarkTest.t.sol @@ -4,9 +4,9 @@ pragma solidity 0.8.15; // Testing utilities import { Test } from "forge-std/Test.sol"; import { Vm } from "forge-std/Vm.sol"; -import "./CommonTest.t.sol"; -import { CrossDomainMessenger } from "../src/universal/CrossDomainMessenger.sol"; -import { ResourceMetering } from "../src/L1/ResourceMetering.sol"; +import "test/CommonTest.t.sol"; +import { CrossDomainMessenger } from "src/universal/CrossDomainMessenger.sol"; +import { ResourceMetering } from "src/L1/ResourceMetering.sol"; // Free function for setting the prevBaseFee param in the OptimismPortal. function setPrevBaseFee(Vm _vm, address _op, uint128 _prevBaseFee) { diff --git a/packages/contracts-bedrock/test/Bytes.t.sol b/packages/contracts-bedrock/test/Bytes.t.sol index 79174d029667..766cf7e01f5b 100644 --- a/packages/contracts-bedrock/test/Bytes.t.sol +++ b/packages/contracts-bedrock/test/Bytes.t.sol @@ -5,7 +5,7 @@ pragma solidity 0.8.15; import { Test } from "forge-std/Test.sol"; // Target contract -import { Bytes } from "../src/libraries/Bytes.sol"; +import { Bytes } from "src/libraries/Bytes.sol"; contract Bytes_slice_Test is Test { /// @notice Tests that the `slice` function works as expected when starting from index 0. diff --git a/packages/contracts-bedrock/test/CheckBalanceHigh.t.sol b/packages/contracts-bedrock/test/CheckBalanceHigh.t.sol index 07505b85f9e5..3102f560f3c8 100644 --- a/packages/contracts-bedrock/test/CheckBalanceHigh.t.sol +++ b/packages/contracts-bedrock/test/CheckBalanceHigh.t.sol @@ -2,7 +2,7 @@ pragma solidity 0.8.15; import { Test } from "forge-std/Test.sol"; -import { CheckBalanceHigh } from "../src/periphery/drippie/dripchecks/CheckBalanceHigh.sol"; +import { CheckBalanceHigh } from "src/periphery/drippie/dripchecks/CheckBalanceHigh.sol"; /// @title CheckBalanceHighTest /// @notice Tests the CheckBalanceHigh contract via fuzzing both the success case diff --git a/packages/contracts-bedrock/test/CheckBalanceLow.t.sol b/packages/contracts-bedrock/test/CheckBalanceLow.t.sol index 71bcf199ff71..0345fa248a8a 100644 --- a/packages/contracts-bedrock/test/CheckBalanceLow.t.sol +++ b/packages/contracts-bedrock/test/CheckBalanceLow.t.sol @@ -2,7 +2,7 @@ pragma solidity 0.8.15; import { Test } from "forge-std/Test.sol"; -import { CheckBalanceLow } from "../src/periphery/drippie/dripchecks/CheckBalanceLow.sol"; +import { CheckBalanceLow } from "src/periphery/drippie/dripchecks/CheckBalanceLow.sol"; /// @title CheckBalanceLowTest /// @notice Tests the CheckBalanceLow contract via fuzzing both the success case diff --git a/packages/contracts-bedrock/test/CheckGelatoLow.t.sol b/packages/contracts-bedrock/test/CheckGelatoLow.t.sol index a89f49251f68..fb213742c917 100644 --- a/packages/contracts-bedrock/test/CheckGelatoLow.t.sol +++ b/packages/contracts-bedrock/test/CheckGelatoLow.t.sol @@ -2,7 +2,7 @@ pragma solidity 0.8.15; import { Test } from "forge-std/Test.sol"; -import { CheckGelatoLow, IGelatoTreasury } from "../src/periphery/drippie/dripchecks/CheckGelatoLow.sol"; +import { CheckGelatoLow, IGelatoTreasury } from "src/periphery/drippie/dripchecks/CheckGelatoLow.sol"; /// @title MockGelatoTreasury /// @notice Mocks the Gelato treasury for testing purposes. Allows arbitrary diff --git a/packages/contracts-bedrock/test/CheckTrue.t.sol b/packages/contracts-bedrock/test/CheckTrue.t.sol index c0b1016fa842..2c7e4be070a2 100644 --- a/packages/contracts-bedrock/test/CheckTrue.t.sol +++ b/packages/contracts-bedrock/test/CheckTrue.t.sol @@ -2,7 +2,7 @@ pragma solidity 0.8.15; import { Test } from "forge-std/Test.sol"; -import { CheckTrue } from "../src/periphery/drippie/dripchecks/CheckTrue.sol"; +import { CheckTrue } from "src/periphery/drippie/dripchecks/CheckTrue.sol"; /// @title CheckTrueTest /// @notice Ensures that the CheckTrue DripCheck contract always returns true. diff --git a/packages/contracts-bedrock/test/Clones.t.sol b/packages/contracts-bedrock/test/Clones.t.sol index e207b246ad67..4b8802c287e5 100644 --- a/packages/contracts-bedrock/test/Clones.t.sol +++ b/packages/contracts-bedrock/test/Clones.t.sol @@ -5,7 +5,7 @@ import "forge-std/Test.sol"; import { ClonesWithImmutableArgs } from "@cwia/ClonesWithImmutableArgs.sol"; -import { Clone } from "../src/libraries/Clone.sol"; +import { Clone } from "src/libraries/Clone.sol"; contract ExampleClone is Clone { uint256 argOffset; diff --git a/packages/contracts-bedrock/test/CrossDomainMessenger.t.sol b/packages/contracts-bedrock/test/CrossDomainMessenger.t.sol index 712fa6da5e3c..f525adc21c1a 100644 --- a/packages/contracts-bedrock/test/CrossDomainMessenger.t.sol +++ b/packages/contracts-bedrock/test/CrossDomainMessenger.t.sol @@ -2,8 +2,8 @@ pragma solidity 0.8.15; // Testing utilities -import { Messenger_Initializer, Reverter, CallerCaller, CommonTest } from "./CommonTest.t.sol"; -import { L1CrossDomainMessenger } from "../src/L1/L1CrossDomainMessenger.sol"; +import { Messenger_Initializer, Reverter, CallerCaller, CommonTest } from "test/CommonTest.t.sol"; +import { L1CrossDomainMessenger } from "src/L1/L1CrossDomainMessenger.sol"; // Libraries import { Predeploys } from "../src/libraries/Predeploys.sol"; diff --git a/packages/contracts-bedrock/test/CrossDomainOwnable.t.sol b/packages/contracts-bedrock/test/CrossDomainOwnable.t.sol index 51394e369592..27c8757066f7 100644 --- a/packages/contracts-bedrock/test/CrossDomainOwnable.t.sol +++ b/packages/contracts-bedrock/test/CrossDomainOwnable.t.sol @@ -3,7 +3,7 @@ pragma solidity 0.8.15; // Testing utilities import { Vm, VmSafe } from "forge-std/Vm.sol"; -import { CommonTest, Portal_Initializer } from "./CommonTest.t.sol"; +import { CommonTest, Portal_Initializer } from "test/CommonTest.t.sol"; // Libraries import { Bytes32AddressLib } from "@rari-capital/solmate/src/utils/Bytes32AddressLib.sol"; diff --git a/packages/contracts-bedrock/test/CrossDomainOwnable2.t.sol b/packages/contracts-bedrock/test/CrossDomainOwnable2.t.sol index 2eb3082d9dfe..c059872c784a 100644 --- a/packages/contracts-bedrock/test/CrossDomainOwnable2.t.sol +++ b/packages/contracts-bedrock/test/CrossDomainOwnable2.t.sol @@ -2,11 +2,11 @@ pragma solidity 0.8.15; // Testing utilities -import { CommonTest, Messenger_Initializer } from "./CommonTest.t.sol"; +import { CommonTest, Messenger_Initializer } from "test/CommonTest.t.sol"; // Libraries -import { Hashing } from "../src/libraries/Hashing.sol"; -import { Encoding } from "../src/libraries/Encoding.sol"; +import { Hashing } from "src/libraries/Hashing.sol"; +import { Encoding } from "src/libraries/Encoding.sol"; import { Bytes32AddressLib } from "@rari-capital/solmate/src/utils/Bytes32AddressLib.sol"; // Target contract dependencies diff --git a/packages/contracts-bedrock/test/CrossDomainOwnable3.t.sol b/packages/contracts-bedrock/test/CrossDomainOwnable3.t.sol index b2a7ff522c17..dbe2c9af3b4c 100644 --- a/packages/contracts-bedrock/test/CrossDomainOwnable3.t.sol +++ b/packages/contracts-bedrock/test/CrossDomainOwnable3.t.sol @@ -2,18 +2,18 @@ pragma solidity 0.8.15; // Testing utilities -import { CommonTest, Messenger_Initializer } from "./CommonTest.t.sol"; +import { CommonTest, Messenger_Initializer } from "test/CommonTest.t.sol"; // Libraries -import { Hashing } from "../src/libraries/Hashing.sol"; -import { Encoding } from "../src/libraries/Encoding.sol"; +import { Hashing } from "src/libraries/Hashing.sol"; +import { Encoding } from "src/libraries/Encoding.sol"; import { Bytes32AddressLib } from "@rari-capital/solmate/src/utils/Bytes32AddressLib.sol"; // Target contract dependencies -import { AddressAliasHelper } from "../src/vendor/AddressAliasHelper.sol"; +import { AddressAliasHelper } from "src/vendor/AddressAliasHelper.sol"; // Target contract -import { CrossDomainOwnable3 } from "../src/L2/CrossDomainOwnable3.sol"; +import { CrossDomainOwnable3 } from "src/L2/CrossDomainOwnable3.sol"; contract XDomainSetter3 is CrossDomainOwnable3 { uint256 public value; diff --git a/packages/contracts-bedrock/test/DelayedVetoable.t.sol b/packages/contracts-bedrock/test/DelayedVetoable.t.sol index cf891069da2b..27d0670a5875 100644 --- a/packages/contracts-bedrock/test/DelayedVetoable.t.sol +++ b/packages/contracts-bedrock/test/DelayedVetoable.t.sol @@ -1,7 +1,7 @@ // SPDX-License-Identifier: MIT pragma solidity 0.8.15; -import { CommonTest } from "./CommonTest.t.sol"; +import { CommonTest } from "test/CommonTest.t.sol"; import { DelayedVetoable } from "src/L1/DelayedVetoable.sol"; contract DelayedVetoable_Init is CommonTest { diff --git a/packages/contracts-bedrock/test/DeployerWhitelist.t.sol b/packages/contracts-bedrock/test/DeployerWhitelist.t.sol index 0d8eda5dcd49..a3096fd373f8 100644 --- a/packages/contracts-bedrock/test/DeployerWhitelist.t.sol +++ b/packages/contracts-bedrock/test/DeployerWhitelist.t.sol @@ -2,10 +2,10 @@ pragma solidity 0.8.15; // Testing utilities -import { CommonTest } from "./CommonTest.t.sol"; +import { CommonTest } from "test/CommonTest.t.sol"; // Target contract -import { DeployerWhitelist } from "../src/legacy/DeployerWhitelist.sol"; +import { DeployerWhitelist } from "src/legacy/DeployerWhitelist.sol"; contract DeployerWhitelist_Test is CommonTest { DeployerWhitelist list; diff --git a/packages/contracts-bedrock/test/DisputeGameFactory.t.sol b/packages/contracts-bedrock/test/DisputeGameFactory.t.sol index 1f81c8b58b3d..a745b73fb04c 100644 --- a/packages/contracts-bedrock/test/DisputeGameFactory.t.sol +++ b/packages/contracts-bedrock/test/DisputeGameFactory.t.sol @@ -1,14 +1,14 @@ // SPDX-License-Identifier: MIT pragma solidity ^0.8.15; -import "../src/libraries/DisputeTypes.sol"; -import "../src/libraries/DisputeErrors.sol"; +import "src/libraries/DisputeTypes.sol"; +import "src/libraries/DisputeErrors.sol"; import { Test } from "forge-std/Test.sol"; -import { DisputeGameFactory } from "../src/dispute/DisputeGameFactory.sol"; -import { IDisputeGame } from "../src/dispute/interfaces/IDisputeGame.sol"; -import { Proxy } from "../src/universal/Proxy.sol"; -import { L2OutputOracle_Initializer } from "./CommonTest.t.sol"; +import { DisputeGameFactory } from "src/dispute/DisputeGameFactory.sol"; +import { IDisputeGame } from "src/dispute/interfaces/IDisputeGame.sol"; +import { Proxy } from "src/universal/Proxy.sol"; +import { L2OutputOracle_Initializer } from "test/CommonTest.t.sol"; contract DisputeGameFactory_Init is L2OutputOracle_Initializer { DisputeGameFactory factory; diff --git a/packages/contracts-bedrock/test/Drippie.t.sol b/packages/contracts-bedrock/test/Drippie.t.sol index 33a69d861111..758011448608 100644 --- a/packages/contracts-bedrock/test/Drippie.t.sol +++ b/packages/contracts-bedrock/test/Drippie.t.sol @@ -2,10 +2,10 @@ pragma solidity 0.8.15; import { Test } from "forge-std/Test.sol"; -import { Drippie } from "../src/periphery/drippie/Drippie.sol"; -import { IDripCheck } from "../src/periphery/drippie/IDripCheck.sol"; -import { CheckTrue } from "../src/periphery/drippie/dripchecks/CheckTrue.sol"; -import { SimpleStorage } from "./Helpers.sol"; +import { Drippie } from "src/periphery/drippie/Drippie.sol"; +import { IDripCheck } from "src/periphery/drippie/IDripCheck.sol"; +import { CheckTrue } from "src/periphery/drippie/dripchecks/CheckTrue.sol"; +import { SimpleStorage } from "test/Helpers.sol"; /// @title TestDrippie /// @notice This is a wrapper contract around Drippie used for testing. diff --git a/packages/contracts-bedrock/test/Encoding.t.sol b/packages/contracts-bedrock/test/Encoding.t.sol index f2261801bb25..14ac3c81a0f7 100644 --- a/packages/contracts-bedrock/test/Encoding.t.sol +++ b/packages/contracts-bedrock/test/Encoding.t.sol @@ -2,14 +2,14 @@ pragma solidity 0.8.15; // Testing utilities -import { CommonTest } from "./CommonTest.t.sol"; +import { CommonTest } from "test/CommonTest.t.sol"; // Libraries -import { Types } from "../src/libraries/Types.sol"; -import { LegacyCrossDomainUtils } from "../src/libraries/LegacyCrossDomainUtils.sol"; +import { Types } from "src/libraries/Types.sol"; +import { LegacyCrossDomainUtils } from "src/libraries/LegacyCrossDomainUtils.sol"; // Target contract -import { Encoding } from "../src/libraries/Encoding.sol"; +import { Encoding } from "src/libraries/Encoding.sol"; contract Encoding_Test is CommonTest { /// @dev Tests encoding and decoding a nonce and version. diff --git a/packages/contracts-bedrock/test/Faucet.t.sol b/packages/contracts-bedrock/test/Faucet.t.sol index 3f3ca41a8963..3c4da9c98798 100644 --- a/packages/contracts-bedrock/test/Faucet.t.sol +++ b/packages/contracts-bedrock/test/Faucet.t.sol @@ -2,9 +2,9 @@ pragma solidity 0.8.15; import { Test } from "forge-std/Test.sol"; -import { Faucet } from "../src/periphery/faucet/Faucet.sol"; -import { AdminFaucetAuthModule } from "../src/periphery/faucet/authmodules/AdminFaucetAuthModule.sol"; -import { FaucetHelper } from "./Helpers.sol"; +import { Faucet } from "src/periphery/faucet/Faucet.sol"; +import { AdminFaucetAuthModule } from "src/periphery/faucet/authmodules/AdminFaucetAuthModule.sol"; +import { FaucetHelper } from "test/Helpers.sol"; contract Faucet_Initializer is Test { event Drip(string indexed authModule, bytes32 indexed userId, uint256 amount, address indexed recipient); diff --git a/packages/contracts-bedrock/test/FaultDisputeGame.t.sol b/packages/contracts-bedrock/test/FaultDisputeGame.t.sol index e56665e1e784..40297b6cc8c0 100644 --- a/packages/contracts-bedrock/test/FaultDisputeGame.t.sol +++ b/packages/contracts-bedrock/test/FaultDisputeGame.t.sol @@ -3,7 +3,7 @@ pragma solidity ^0.8.15; import { Test } from "forge-std/Test.sol"; import { Vm } from "forge-std/Vm.sol"; -import { DisputeGameFactory_Init } from "./DisputeGameFactory.t.sol"; +import { DisputeGameFactory_Init } from "test/DisputeGameFactory.t.sol"; import { DisputeGameFactory } from "src/dispute/DisputeGameFactory.sol"; import { FaultDisputeGame } from "src/dispute/FaultDisputeGame.sol"; import { L2OutputOracle } from "src/L1/L2OutputOracle.sol"; diff --git a/packages/contracts-bedrock/test/FeeVault.t.sol b/packages/contracts-bedrock/test/FeeVault.t.sol index dede09e762f6..3b877840935c 100644 --- a/packages/contracts-bedrock/test/FeeVault.t.sol +++ b/packages/contracts-bedrock/test/FeeVault.t.sol @@ -2,18 +2,18 @@ pragma solidity 0.8.15; // Testing utilities -import { Bridge_Initializer } from "./CommonTest.t.sol"; -import { BaseFeeVault } from "../src/L2/BaseFeeVault.sol"; -import { StandardBridge } from "../src/universal/StandardBridge.sol"; +import { Bridge_Initializer } from "test/CommonTest.t.sol"; +import { BaseFeeVault } from "src/L2/BaseFeeVault.sol"; +import { StandardBridge } from "src/universal/StandardBridge.sol"; // Libraries -import { Predeploys } from "../src/libraries/Predeploys.sol"; +import { Predeploys } from "src/libraries/Predeploys.sol"; // Target contract dependencies -import { FeeVault } from "../src/universal/FeeVault.sol"; +import { FeeVault } from "src/universal/FeeVault.sol"; // Target contract -import { L1FeeVault } from "../src/L2/L1FeeVault.sol"; +import { L1FeeVault } from "src/L2/L1FeeVault.sol"; // Test the implementations of the FeeVault contract FeeVault_Test is Bridge_Initializer { diff --git a/packages/contracts-bedrock/test/GasPriceOracle.t.sol b/packages/contracts-bedrock/test/GasPriceOracle.t.sol index 7a69ed1bf806..26e571ecb550 100644 --- a/packages/contracts-bedrock/test/GasPriceOracle.t.sol +++ b/packages/contracts-bedrock/test/GasPriceOracle.t.sol @@ -2,14 +2,14 @@ pragma solidity 0.8.15; // Testing utilities -import { CommonTest } from "./CommonTest.t.sol"; +import { CommonTest } from "test/CommonTest.t.sol"; // Target contract dependencies -import { L1Block } from "../src/L2/L1Block.sol"; -import { Predeploys } from "../src/libraries/Predeploys.sol"; +import { L1Block } from "src/L2/L1Block.sol"; +import { Predeploys } from "src/libraries/Predeploys.sol"; // Target contract -import { GasPriceOracle } from "../src/L2/GasPriceOracle.sol"; +import { GasPriceOracle } from "src/L2/GasPriceOracle.sol"; contract GasPriceOracle_Test is CommonTest { event OverheadUpdated(uint256); diff --git a/packages/contracts-bedrock/test/GovernanceToken.t.sol b/packages/contracts-bedrock/test/GovernanceToken.t.sol index c1b4b9595424..ec3c7902af7e 100644 --- a/packages/contracts-bedrock/test/GovernanceToken.t.sol +++ b/packages/contracts-bedrock/test/GovernanceToken.t.sol @@ -2,10 +2,10 @@ pragma solidity 0.8.15; // Testing utilities -import { CommonTest } from "./CommonTest.t.sol"; +import { CommonTest } from "test/CommonTest.t.sol"; // Target contract -import { GovernanceToken } from "../src/governance/GovernanceToken.sol"; +import { GovernanceToken } from "src/governance/GovernanceToken.sol"; contract GovernanceToken_Test is CommonTest { address constant owner = address(0x1234); diff --git a/packages/contracts-bedrock/test/Hashing.t.sol b/packages/contracts-bedrock/test/Hashing.t.sol index 33b8a10c5f41..fc79c24422bd 100644 --- a/packages/contracts-bedrock/test/Hashing.t.sol +++ b/packages/contracts-bedrock/test/Hashing.t.sol @@ -2,15 +2,15 @@ pragma solidity 0.8.15; // Testing utilities -import { CommonTest } from "./CommonTest.t.sol"; +import { CommonTest } from "test/CommonTest.t.sol"; // Libraries -import { Types } from "../src/libraries/Types.sol"; -import { Encoding } from "../src/libraries/Encoding.sol"; -import { LegacyCrossDomainUtils } from "../src/libraries/LegacyCrossDomainUtils.sol"; +import { Types } from "src/libraries/Types.sol"; +import { Encoding } from "src/libraries/Encoding.sol"; +import { LegacyCrossDomainUtils } from "src/libraries/LegacyCrossDomainUtils.sol"; // Target contract -import { Hashing } from "../src/libraries/Hashing.sol"; +import { Hashing } from "src/libraries/Hashing.sol"; contract Hashing_hashDepositSource_Test is CommonTest { /// @notice Tests that hashDepositSource returns the correct hash in a simple case. diff --git a/packages/contracts-bedrock/test/Helpers.sol b/packages/contracts-bedrock/test/Helpers.sol index 13a16e7af9f1..a27da6e2b2fb 100644 --- a/packages/contracts-bedrock/test/Helpers.sol +++ b/packages/contracts-bedrock/test/Helpers.sol @@ -4,11 +4,11 @@ pragma solidity ^0.8.0; import { ERC20 } from "@rari-capital/solmate/src/tokens/ERC20.sol"; import { ERC721 } from "@rari-capital/solmate/src/tokens/ERC721.sol"; import { ECDSA } from "@openzeppelin/contracts/utils/cryptography/ECDSA.sol"; -import { OptimistInviter } from "../src/periphery/op-nft/OptimistInviter.sol"; +import { OptimistInviter } from "src/periphery/op-nft/OptimistInviter.sol"; import { IERC1271 } from "@openzeppelin/contracts/interfaces/IERC1271.sol"; import { Ownable } from "@openzeppelin/contracts/access/Ownable.sol"; import { ECDSAUpgradeable } from "@openzeppelin/contracts-upgradeable/utils/cryptography/ECDSAUpgradeable.sol"; -import { AdminFaucetAuthModule } from "../src/periphery/faucet/authmodules/AdminFaucetAuthModule.sol"; +import { AdminFaucetAuthModule } from "src/periphery/faucet/authmodules/AdminFaucetAuthModule.sol"; contract TestERC20 is ERC20 { constructor() ERC20("TEST", "TST", 18) { } diff --git a/packages/contracts-bedrock/test/L1Block.t.sol b/packages/contracts-bedrock/test/L1Block.t.sol index 940db8494001..f44f606a6270 100644 --- a/packages/contracts-bedrock/test/L1Block.t.sol +++ b/packages/contracts-bedrock/test/L1Block.t.sol @@ -2,10 +2,10 @@ pragma solidity 0.8.15; // Testing utilities -import { CommonTest } from "./CommonTest.t.sol"; +import { CommonTest } from "test/CommonTest.t.sol"; // Target contract -import { L1Block } from "../src/L2/L1Block.sol"; +import { L1Block } from "src/L2/L1Block.sol"; contract L1BlockTest is CommonTest { L1Block lb; diff --git a/packages/contracts-bedrock/test/L1BlockNumber.t.sol b/packages/contracts-bedrock/test/L1BlockNumber.t.sol index b80965feea23..8ac6ab140d6b 100644 --- a/packages/contracts-bedrock/test/L1BlockNumber.t.sol +++ b/packages/contracts-bedrock/test/L1BlockNumber.t.sol @@ -5,11 +5,11 @@ pragma solidity 0.8.15; import { Test } from "forge-std/Test.sol"; // Target contract dependencies -import { L1Block } from "../src/L2/L1Block.sol"; -import { Predeploys } from "../src/libraries/Predeploys.sol"; +import { L1Block } from "src/L2/L1Block.sol"; +import { Predeploys } from "src/libraries/Predeploys.sol"; // Target contract -import { L1BlockNumber } from "../src/legacy/L1BlockNumber.sol"; +import { L1BlockNumber } from "src/legacy/L1BlockNumber.sol"; contract L1BlockNumberTest is Test { L1Block lb; diff --git a/packages/contracts-bedrock/test/L1CrossDomainMessenger.t.sol b/packages/contracts-bedrock/test/L1CrossDomainMessenger.t.sol index 5090cebb8156..77f966f310d6 100644 --- a/packages/contracts-bedrock/test/L1CrossDomainMessenger.t.sol +++ b/packages/contracts-bedrock/test/L1CrossDomainMessenger.t.sol @@ -2,21 +2,21 @@ pragma solidity 0.8.15; // Testing utilities -import { Messenger_Initializer, Reverter, ConfigurableCaller } from "./CommonTest.t.sol"; -import { L2OutputOracle_Initializer } from "./L2OutputOracle.t.sol"; +import { Messenger_Initializer, Reverter, ConfigurableCaller } from "test/CommonTest.t.sol"; +import { L2OutputOracle_Initializer } from "test/L2OutputOracle.t.sol"; // Libraries -import { AddressAliasHelper } from "../src/vendor/AddressAliasHelper.sol"; -import { Predeploys } from "../src/libraries/Predeploys.sol"; -import { Hashing } from "../src/libraries/Hashing.sol"; -import { Encoding } from "../src/libraries/Encoding.sol"; +import { AddressAliasHelper } from "src/vendor/AddressAliasHelper.sol"; +import { Predeploys } from "src/libraries/Predeploys.sol"; +import { Hashing } from "src/libraries/Hashing.sol"; +import { Encoding } from "src/libraries/Encoding.sol"; // Target contract dependencies -import { L2OutputOracle } from "../src/L1/L2OutputOracle.sol"; -import { OptimismPortal } from "../src/L1/OptimismPortal.sol"; +import { L2OutputOracle } from "src/L1/L2OutputOracle.sol"; +import { OptimismPortal } from "src/L1/OptimismPortal.sol"; // Target contract -import { L1CrossDomainMessenger } from "../src/L1/L1CrossDomainMessenger.sol"; +import { L1CrossDomainMessenger } from "src/L1/L1CrossDomainMessenger.sol"; contract L1CrossDomainMessenger_Test is Messenger_Initializer { /// @dev The receiver address diff --git a/packages/contracts-bedrock/test/L1ERC721Bridge.t.sol b/packages/contracts-bedrock/test/L1ERC721Bridge.t.sol index 658237ad4ec0..b57c8b7ef70d 100644 --- a/packages/contracts-bedrock/test/L1ERC721Bridge.t.sol +++ b/packages/contracts-bedrock/test/L1ERC721Bridge.t.sol @@ -2,15 +2,15 @@ pragma solidity 0.8.15; // Testing utilities -import { ERC721Bridge_Initializer } from "./CommonTest.t.sol"; +import { ERC721Bridge_Initializer } from "test/CommonTest.t.sol"; import { ERC721 } from "@openzeppelin/contracts/token/ERC721/ERC721.sol"; // Target contract dependencies -import { L2ERC721Bridge } from "../src/L2/L2ERC721Bridge.sol"; -import { Predeploys } from "../src/libraries/Predeploys.sol"; +import { L2ERC721Bridge } from "src/L2/L2ERC721Bridge.sol"; +import { Predeploys } from "src/libraries/Predeploys.sol"; // Target contract -import { L1ERC721Bridge } from "../src/L1/L1ERC721Bridge.sol"; +import { L1ERC721Bridge } from "src/L1/L1ERC721Bridge.sol"; /// @dev Test ERC721 contract. contract TestERC721 is ERC721 { diff --git a/packages/contracts-bedrock/test/L1StandardBridge.t.sol b/packages/contracts-bedrock/test/L1StandardBridge.t.sol index 18dc292946d7..c8966dd8b62f 100644 --- a/packages/contracts-bedrock/test/L1StandardBridge.t.sol +++ b/packages/contracts-bedrock/test/L1StandardBridge.t.sol @@ -4,7 +4,7 @@ pragma solidity 0.8.15; // Testing utilities import { stdStorage, StdStorage } from "forge-std/Test.sol"; import { ERC20 } from "@openzeppelin/contracts/token/ERC20/ERC20.sol"; -import { Bridge_Initializer } from "./CommonTest.t.sol"; +import { Bridge_Initializer } from "test/CommonTest.t.sol"; // Libraries import { Predeploys } from "src/libraries/Predeploys.sol"; diff --git a/packages/contracts-bedrock/test/L2CrossDomainMessenger.t.sol b/packages/contracts-bedrock/test/L2CrossDomainMessenger.t.sol index b84d10ff8631..ead28756af6d 100644 --- a/packages/contracts-bedrock/test/L2CrossDomainMessenger.t.sol +++ b/packages/contracts-bedrock/test/L2CrossDomainMessenger.t.sol @@ -2,20 +2,20 @@ pragma solidity 0.8.15; // Testing utilities -import { Messenger_Initializer, Reverter, ConfigurableCaller } from "./CommonTest.t.sol"; +import { Messenger_Initializer, Reverter, ConfigurableCaller } from "test/CommonTest.t.sol"; // Libraries -import { Hashing } from "../src/libraries/Hashing.sol"; -import { Encoding } from "../src/libraries/Encoding.sol"; -import { Types } from "../src/libraries/Types.sol"; +import { Hashing } from "src/libraries/Hashing.sol"; +import { Encoding } from "src/libraries/Encoding.sol"; +import { Types } from "src/libraries/Types.sol"; // Target contract dependencies -import { L2ToL1MessagePasser } from "../src/L2/L2ToL1MessagePasser.sol"; -import { AddressAliasHelper } from "../src/vendor/AddressAliasHelper.sol"; -import { L1CrossDomainMessenger } from "../src/L1/L1CrossDomainMessenger.sol"; +import { L2ToL1MessagePasser } from "src/L2/L2ToL1MessagePasser.sol"; +import { AddressAliasHelper } from "src/vendor/AddressAliasHelper.sol"; +import { L1CrossDomainMessenger } from "src/L1/L1CrossDomainMessenger.sol"; // Target contract -import { L2CrossDomainMessenger } from "../src/L2/L2CrossDomainMessenger.sol"; +import { L2CrossDomainMessenger } from "src/L2/L2CrossDomainMessenger.sol"; contract L2CrossDomainMessenger_Test is Messenger_Initializer { /// @dev Receiver address for testing diff --git a/packages/contracts-bedrock/test/L2ERC721Bridge.t.sol b/packages/contracts-bedrock/test/L2ERC721Bridge.t.sol index 7c91f9e57744..03567f3beae1 100644 --- a/packages/contracts-bedrock/test/L2ERC721Bridge.t.sol +++ b/packages/contracts-bedrock/test/L2ERC721Bridge.t.sol @@ -2,15 +2,15 @@ pragma solidity 0.8.15; // Testing utilities -import { ERC721Bridge_Initializer } from "./CommonTest.t.sol"; +import { ERC721Bridge_Initializer } from "test/CommonTest.t.sol"; // Target contract dependencies import { ERC721 } from "@openzeppelin/contracts/token/ERC721/ERC721.sol"; -import { L1ERC721Bridge } from "../src/L1/L1ERC721Bridge.sol"; -import { OptimismMintableERC721 } from "../src/universal/OptimismMintableERC721.sol"; +import { L1ERC721Bridge } from "src/L1/L1ERC721Bridge.sol"; +import { OptimismMintableERC721 } from "src/universal/OptimismMintableERC721.sol"; // Target contract -import { L2ERC721Bridge } from "../src/L2/L2ERC721Bridge.sol"; +import { L2ERC721Bridge } from "src/L2/L2ERC721Bridge.sol"; contract TestERC721 is ERC721 { constructor() ERC721("Test", "TST") { } diff --git a/packages/contracts-bedrock/test/L2OutputOracle.t.sol b/packages/contracts-bedrock/test/L2OutputOracle.t.sol index f6c521f11ce6..623fc7aa224c 100644 --- a/packages/contracts-bedrock/test/L2OutputOracle.t.sol +++ b/packages/contracts-bedrock/test/L2OutputOracle.t.sol @@ -3,7 +3,7 @@ pragma solidity 0.8.15; // Testing utilities import { stdError } from "forge-std/Test.sol"; -import { L2OutputOracle_Initializer, NextImpl } from "./CommonTest.t.sol"; +import { L2OutputOracle_Initializer, NextImpl } from "test/CommonTest.t.sol"; // Libraries import { Types } from "src/libraries/Types.sol"; diff --git a/packages/contracts-bedrock/test/L2StandardBridge.t.sol b/packages/contracts-bedrock/test/L2StandardBridge.t.sol index b9f31b3abcd7..be7dd6ff639c 100644 --- a/packages/contracts-bedrock/test/L2StandardBridge.t.sol +++ b/packages/contracts-bedrock/test/L2StandardBridge.t.sol @@ -3,20 +3,20 @@ pragma solidity 0.8.15; // Testing utilities // Target contract is imported by the `Bridge_Initializer` -import { Bridge_Initializer } from "./CommonTest.t.sol"; +import { Bridge_Initializer } from "test/CommonTest.t.sol"; import { stdStorage, StdStorage } from "forge-std/Test.sol"; -import { CrossDomainMessenger } from "../src/universal/CrossDomainMessenger.sol"; -import { L2ToL1MessagePasser } from "../src/L2/L2ToL1MessagePasser.sol"; +import { CrossDomainMessenger } from "src/universal/CrossDomainMessenger.sol"; +import { L2ToL1MessagePasser } from "src/L2/L2ToL1MessagePasser.sol"; import { ERC20 } from "@openzeppelin/contracts/token/ERC20/ERC20.sol"; // Libraries -import { Hashing } from "../src/libraries/Hashing.sol"; -import { Types } from "../src/libraries/Types.sol"; +import { Hashing } from "src/libraries/Hashing.sol"; +import { Types } from "src/libraries/Types.sol"; // Target contract dependencies -import { Predeploys } from "../src/libraries/Predeploys.sol"; -import { StandardBridge } from "../src/universal/StandardBridge.sol"; -import { OptimismMintableERC20 } from "../src/universal/OptimismMintableERC20.sol"; +import { Predeploys } from "src/libraries/Predeploys.sol"; +import { StandardBridge } from "src/universal/StandardBridge.sol"; +import { OptimismMintableERC20 } from "src/universal/OptimismMintableERC20.sol"; contract L2StandardBridge_Test is Bridge_Initializer { using stdStorage for StdStorage; diff --git a/packages/contracts-bedrock/test/L2ToL1MessagePasser.t.sol b/packages/contracts-bedrock/test/L2ToL1MessagePasser.t.sol index 01ebac687c1c..9e62c74944b4 100644 --- a/packages/contracts-bedrock/test/L2ToL1MessagePasser.t.sol +++ b/packages/contracts-bedrock/test/L2ToL1MessagePasser.t.sol @@ -2,14 +2,14 @@ pragma solidity 0.8.15; // Testing utilities -import { CommonTest } from "./CommonTest.t.sol"; +import { CommonTest } from "test/CommonTest.t.sol"; // Libraries -import { Types } from "../src/libraries/Types.sol"; -import { Hashing } from "../src/libraries/Hashing.sol"; +import { Types } from "src/libraries/Types.sol"; +import { Hashing } from "src/libraries/Hashing.sol"; // Target contract -import { L2ToL1MessagePasser } from "../src/L2/L2ToL1MessagePasser.sol"; +import { L2ToL1MessagePasser } from "src/L2/L2ToL1MessagePasser.sol"; contract L2ToL1MessagePasserTest is CommonTest { L2ToL1MessagePasser messagePasser; diff --git a/packages/contracts-bedrock/test/LegacyERC20ETH.t.sol b/packages/contracts-bedrock/test/LegacyERC20ETH.t.sol index 22c801de4ab2..126fdfe661e2 100644 --- a/packages/contracts-bedrock/test/LegacyERC20ETH.t.sol +++ b/packages/contracts-bedrock/test/LegacyERC20ETH.t.sol @@ -2,13 +2,13 @@ pragma solidity 0.8.15; // Testing utilities -import { CommonTest } from "./CommonTest.t.sol"; +import { CommonTest } from "test/CommonTest.t.sol"; // Target contract dependencies -import { Predeploys } from "../src/libraries/Predeploys.sol"; +import { Predeploys } from "src/libraries/Predeploys.sol"; // Target contract -import { LegacyERC20ETH } from "../src/legacy/LegacyERC20ETH.sol"; +import { LegacyERC20ETH } from "src/legacy/LegacyERC20ETH.sol"; contract LegacyERC20ETH_Test is CommonTest { LegacyERC20ETH eth; diff --git a/packages/contracts-bedrock/test/LegacyMessagePasser.t.sol b/packages/contracts-bedrock/test/LegacyMessagePasser.t.sol index a5914779034d..abf841e3e79e 100644 --- a/packages/contracts-bedrock/test/LegacyMessagePasser.t.sol +++ b/packages/contracts-bedrock/test/LegacyMessagePasser.t.sol @@ -2,13 +2,13 @@ pragma solidity 0.8.15; // Testing utilities -import { CommonTest } from "./CommonTest.t.sol"; +import { CommonTest } from "test/CommonTest.t.sol"; // Testing contract dependencies -import { Predeploys } from "../src/libraries/Predeploys.sol"; +import { Predeploys } from "src/libraries/Predeploys.sol"; // Target contract -import { LegacyMessagePasser } from "../src/legacy/LegacyMessagePasser.sol"; +import { LegacyMessagePasser } from "src/legacy/LegacyMessagePasser.sol"; contract LegacyMessagePasser_Test is CommonTest { LegacyMessagePasser messagePasser; diff --git a/packages/contracts-bedrock/test/LibClock.t.sol b/packages/contracts-bedrock/test/LibClock.t.sol index cc8fc6606902..41b5ae184976 100644 --- a/packages/contracts-bedrock/test/LibClock.t.sol +++ b/packages/contracts-bedrock/test/LibClock.t.sol @@ -3,7 +3,7 @@ pragma solidity ^0.8.15; import { Test } from "forge-std/Test.sol"; import { LibClock } from "../src/dispute/lib/LibClock.sol"; -import "../src/libraries/DisputeTypes.sol"; +import "src/libraries/DisputeTypes.sol"; /// @notice Tests for `LibClock` contract LibClock_Test is Test { diff --git a/packages/contracts-bedrock/test/LibPosition.t.sol b/packages/contracts-bedrock/test/LibPosition.t.sol index 4aa2dc010267..a60f5e2bafca 100644 --- a/packages/contracts-bedrock/test/LibPosition.t.sol +++ b/packages/contracts-bedrock/test/LibPosition.t.sol @@ -3,7 +3,7 @@ pragma solidity ^0.8.15; import { Test } from "forge-std/Test.sol"; import { LibPosition } from "../src/dispute/lib/LibPosition.sol"; -import "../src/libraries/DisputeTypes.sol"; +import "src/libraries/DisputeTypes.sol"; /// @notice Tests for `LibPosition` contract LibPosition_Test is Test { diff --git a/packages/contracts-bedrock/test/MerkleTrie.t.sol b/packages/contracts-bedrock/test/MerkleTrie.t.sol index f6c559bc4644..59673f06e2ae 100644 --- a/packages/contracts-bedrock/test/MerkleTrie.t.sol +++ b/packages/contracts-bedrock/test/MerkleTrie.t.sol @@ -1,8 +1,8 @@ // SPDX-License-Identifier: MIT pragma solidity 0.8.15; -import { CommonTest } from "./CommonTest.t.sol"; -import { MerkleTrie } from "../src/libraries/trie/MerkleTrie.sol"; +import { CommonTest } from "test/CommonTest.t.sol"; +import { MerkleTrie } from "src/libraries/trie/MerkleTrie.sol"; contract MerkleTrie_get_Test is CommonTest { function test_get_validProof1_succeeds() external { diff --git a/packages/contracts-bedrock/test/MintManager.t.sol b/packages/contracts-bedrock/test/MintManager.t.sol index 66578642294f..92fceec0ae1a 100644 --- a/packages/contracts-bedrock/test/MintManager.t.sol +++ b/packages/contracts-bedrock/test/MintManager.t.sol @@ -2,13 +2,13 @@ pragma solidity 0.8.15; // Testing utilities -import { CommonTest } from "./CommonTest.t.sol"; +import { CommonTest } from "test/CommonTest.t.sol"; // Target contract dependencies -import { GovernanceToken } from "../src/governance/GovernanceToken.sol"; +import { GovernanceToken } from "src/governance/GovernanceToken.sol"; // Target contract -import { MintManager } from "../src/governance/MintManager.sol"; +import { MintManager } from "src/governance/MintManager.sol"; contract MintManager_Initializer is CommonTest { address constant owner = address(0x1234); diff --git a/packages/contracts-bedrock/test/OptimismMintableERC20.t.sol b/packages/contracts-bedrock/test/OptimismMintableERC20.t.sol index ef6c5468b703..40233eded0f8 100644 --- a/packages/contracts-bedrock/test/OptimismMintableERC20.t.sol +++ b/packages/contracts-bedrock/test/OptimismMintableERC20.t.sol @@ -1,8 +1,8 @@ // SPDX-License-Identifier: MIT pragma solidity 0.8.15; -import { Bridge_Initializer } from "./CommonTest.t.sol"; -import { ILegacyMintableERC20, IOptimismMintableERC20 } from "../src/universal/IOptimismMintableERC20.sol"; +import { Bridge_Initializer } from "test/CommonTest.t.sol"; +import { ILegacyMintableERC20, IOptimismMintableERC20 } from "src/universal/IOptimismMintableERC20.sol"; import { IERC165 } from "@openzeppelin/contracts/utils/introspection/IERC165.sol"; contract OptimismMintableERC20_Test is Bridge_Initializer { diff --git a/packages/contracts-bedrock/test/OptimismMintableERC20Factory.t.sol b/packages/contracts-bedrock/test/OptimismMintableERC20Factory.t.sol index 64b0a7de5b98..a09ea0f94df7 100644 --- a/packages/contracts-bedrock/test/OptimismMintableERC20Factory.t.sol +++ b/packages/contracts-bedrock/test/OptimismMintableERC20Factory.t.sol @@ -1,8 +1,8 @@ // SPDX-License-Identifier: MIT pragma solidity 0.8.15; -import { OptimismMintableERC20 } from "../src/universal/OptimismMintableERC20.sol"; -import { Bridge_Initializer } from "./CommonTest.t.sol"; +import { OptimismMintableERC20 } from "src/universal/OptimismMintableERC20.sol"; +import { Bridge_Initializer } from "test/CommonTest.t.sol"; contract OptimismMintableTokenFactory_Test is Bridge_Initializer { event StandardL2TokenCreated(address indexed remoteToken, address indexed localToken); diff --git a/packages/contracts-bedrock/test/OptimismMintableERC721.t.sol b/packages/contracts-bedrock/test/OptimismMintableERC721.t.sol index 8935617cd49f..5aac93b2eb4c 100644 --- a/packages/contracts-bedrock/test/OptimismMintableERC721.t.sol +++ b/packages/contracts-bedrock/test/OptimismMintableERC721.t.sol @@ -5,8 +5,8 @@ import { ERC721, IERC721 } from "@openzeppelin/contracts/token/ERC721/ERC721.sol import { IERC721Enumerable } from "@openzeppelin/contracts/token/ERC721/extensions/ERC721Enumerable.sol"; import { IERC165 } from "@openzeppelin/contracts/utils/introspection/IERC165.sol"; import { Strings } from "@openzeppelin/contracts/utils/Strings.sol"; -import { ERC721Bridge_Initializer } from "./CommonTest.t.sol"; -import { OptimismMintableERC721, IOptimismMintableERC721 } from "../src/universal/OptimismMintableERC721.sol"; +import { ERC721Bridge_Initializer } from "test/CommonTest.t.sol"; +import { OptimismMintableERC721, IOptimismMintableERC721 } from "src/universal/OptimismMintableERC721.sol"; contract OptimismMintableERC721_Test is ERC721Bridge_Initializer { ERC721 internal L1NFT; diff --git a/packages/contracts-bedrock/test/OptimismMintableERC721Factory.t.sol b/packages/contracts-bedrock/test/OptimismMintableERC721Factory.t.sol index 4d8b523706bd..ee70e3ff9f3b 100644 --- a/packages/contracts-bedrock/test/OptimismMintableERC721Factory.t.sol +++ b/packages/contracts-bedrock/test/OptimismMintableERC721Factory.t.sol @@ -2,9 +2,9 @@ pragma solidity 0.8.15; import { ERC721 } from "@openzeppelin/contracts/token/ERC721/ERC721.sol"; -import { ERC721Bridge_Initializer } from "./CommonTest.t.sol"; -import { OptimismMintableERC721 } from "../src/universal/OptimismMintableERC721.sol"; -import { OptimismMintableERC721Factory } from "../src/universal/OptimismMintableERC721Factory.sol"; +import { ERC721Bridge_Initializer } from "test/CommonTest.t.sol"; +import { OptimismMintableERC721 } from "src/universal/OptimismMintableERC721.sol"; +import { OptimismMintableERC721Factory } from "src/universal/OptimismMintableERC721Factory.sol"; contract OptimismMintableERC721Factory_Test is ERC721Bridge_Initializer { OptimismMintableERC721Factory internal factory; diff --git a/packages/contracts-bedrock/test/OptimismPortal.t.sol b/packages/contracts-bedrock/test/OptimismPortal.t.sol index 256e5c6359fc..30f134e6c9b5 100644 --- a/packages/contracts-bedrock/test/OptimismPortal.t.sol +++ b/packages/contracts-bedrock/test/OptimismPortal.t.sol @@ -3,7 +3,7 @@ pragma solidity 0.8.15; // Testing utilities import { stdError } from "forge-std/Test.sol"; -import { Portal_Initializer, CommonTest, NextImpl } from "./CommonTest.t.sol"; +import { Portal_Initializer, CommonTest, NextImpl } from "test/CommonTest.t.sol"; // Libraries import { Types } from "src/libraries/Types.sol"; diff --git a/packages/contracts-bedrock/test/Optimist.t.sol b/packages/contracts-bedrock/test/Optimist.t.sol index f08656bf20da..a7a84e2b9854 100644 --- a/packages/contracts-bedrock/test/Optimist.t.sol +++ b/packages/contracts-bedrock/test/Optimist.t.sol @@ -3,11 +3,11 @@ pragma solidity >=0.6.2 <0.9.0; // Testing utilities import { Test } from "forge-std/Test.sol"; -import { AttestationStation } from "../src/periphery/op-nft/AttestationStation.sol"; -import { Optimist } from "../src/periphery/op-nft/Optimist.sol"; -import { OptimistAllowlist } from "../src/periphery/op-nft/OptimistAllowlist.sol"; -import { OptimistInviter } from "../src/periphery/op-nft/OptimistInviter.sol"; -import { OptimistInviterHelper } from "./Helpers.sol"; +import { AttestationStation } from "src/periphery/op-nft/AttestationStation.sol"; +import { Optimist } from "src/periphery/op-nft/Optimist.sol"; +import { OptimistAllowlist } from "src/periphery/op-nft/OptimistAllowlist.sol"; +import { OptimistInviter } from "src/periphery/op-nft/OptimistInviter.sol"; +import { OptimistInviterHelper } from "test/Helpers.sol"; import { Strings } from "@openzeppelin/contracts/utils/Strings.sol"; import { IERC721 } from "@openzeppelin/contracts/token/ERC721/IERC721.sol"; diff --git a/packages/contracts-bedrock/test/OptimistAllowlist.t.sol b/packages/contracts-bedrock/test/OptimistAllowlist.t.sol index 4a3d59db14c5..d7f53ff8c23c 100644 --- a/packages/contracts-bedrock/test/OptimistAllowlist.t.sol +++ b/packages/contracts-bedrock/test/OptimistAllowlist.t.sol @@ -3,11 +3,11 @@ pragma solidity 0.8.15; // Testing utilities import { Test } from "forge-std/Test.sol"; -import { AttestationStation } from "../src/periphery/op-nft/AttestationStation.sol"; -import { OptimistAllowlist } from "../src/periphery/op-nft/OptimistAllowlist.sol"; -import { OptimistInviter } from "../src/periphery/op-nft/OptimistInviter.sol"; -import { OptimistInviterHelper } from "./Helpers.sol"; -import { OptimistConstants } from "../src/periphery/op-nft/libraries/OptimistConstants.sol"; +import { AttestationStation } from "src/periphery/op-nft/AttestationStation.sol"; +import { OptimistAllowlist } from "src/periphery/op-nft/OptimistAllowlist.sol"; +import { OptimistInviter } from "src/periphery/op-nft/OptimistInviter.sol"; +import { OptimistInviterHelper } from "test/Helpers.sol"; +import { OptimistConstants } from "src/periphery/op-nft/libraries/OptimistConstants.sol"; contract OptimistAllowlist_Initializer is Test { event AttestationCreated(address indexed creator, address indexed about, bytes32 indexed key, bytes val); diff --git a/packages/contracts-bedrock/test/OptimistInviter.t.sol b/packages/contracts-bedrock/test/OptimistInviter.t.sol index 6c73f0688925..677dceb7d151 100644 --- a/packages/contracts-bedrock/test/OptimistInviter.t.sol +++ b/packages/contracts-bedrock/test/OptimistInviter.t.sol @@ -3,13 +3,13 @@ pragma solidity 0.8.15; // Testing utilities import { Test } from "forge-std/Test.sol"; -import { AttestationStation } from "../src/periphery/op-nft/AttestationStation.sol"; -import { OptimistInviter } from "../src/periphery/op-nft/OptimistInviter.sol"; -import { Optimist } from "../src/periphery/op-nft/Optimist.sol"; +import { AttestationStation } from "src/periphery/op-nft/AttestationStation.sol"; +import { OptimistInviter } from "src/periphery/op-nft/OptimistInviter.sol"; +import { Optimist } from "src/periphery/op-nft/Optimist.sol"; import { Strings } from "@openzeppelin/contracts/utils/Strings.sol"; -import { TestERC1271Wallet } from "./Helpers.sol"; -import { OptimistInviterHelper } from "./Helpers.sol"; -import { OptimistConstants } from "../src/periphery/op-nft/libraries/OptimistConstants.sol"; +import { TestERC1271Wallet } from "test/Helpers.sol"; +import { OptimistInviterHelper } from "test/Helpers.sol"; +import { OptimistConstants } from "src/periphery/op-nft/libraries/OptimistConstants.sol"; contract OptimistInviter_Initializer is Test { event InviteClaimed(address indexed issuer, address indexed claimer); diff --git a/packages/contracts-bedrock/test/ProtocolVersions.t.sol b/packages/contracts-bedrock/test/ProtocolVersions.t.sol index 5c01bafb5e46..814c36ec9639 100644 --- a/packages/contracts-bedrock/test/ProtocolVersions.t.sol +++ b/packages/contracts-bedrock/test/ProtocolVersions.t.sol @@ -2,7 +2,7 @@ pragma solidity 0.8.15; // Testing utilities -import { CommonTest } from "./CommonTest.t.sol"; +import { CommonTest } from "test/CommonTest.t.sol"; // Libraries import { Constants } from "src/libraries/Constants.sol"; diff --git a/packages/contracts-bedrock/test/Proxy.t.sol b/packages/contracts-bedrock/test/Proxy.t.sol index 8474f50aea77..fa794dd295b2 100644 --- a/packages/contracts-bedrock/test/Proxy.t.sol +++ b/packages/contracts-bedrock/test/Proxy.t.sol @@ -2,7 +2,7 @@ pragma solidity 0.8.15; import { Test } from "forge-std/Test.sol"; -import { Proxy } from "../src/universal/Proxy.sol"; +import { Proxy } from "src/universal/Proxy.sol"; import { Bytes32AddressLib } from "@rari-capital/solmate/src/utils/Bytes32AddressLib.sol"; contract SimpleStorage { diff --git a/packages/contracts-bedrock/test/ProxyAdmin.t.sol b/packages/contracts-bedrock/test/ProxyAdmin.t.sol index 49b290f25bb9..083ba04c2fe5 100644 --- a/packages/contracts-bedrock/test/ProxyAdmin.t.sol +++ b/packages/contracts-bedrock/test/ProxyAdmin.t.sol @@ -2,12 +2,12 @@ pragma solidity 0.8.15; import { Test } from "forge-std/Test.sol"; -import { Proxy } from "../src/universal/Proxy.sol"; -import { ProxyAdmin } from "../src/universal/ProxyAdmin.sol"; -import { SimpleStorage } from "./Proxy.t.sol"; -import { L1ChugSplashProxy } from "../src/legacy/L1ChugSplashProxy.sol"; -import { ResolvedDelegateProxy } from "../src/legacy/ResolvedDelegateProxy.sol"; -import { AddressManager } from "../src/legacy/AddressManager.sol"; +import { Proxy } from "src/universal/Proxy.sol"; +import { ProxyAdmin } from "src/universal/ProxyAdmin.sol"; +import { SimpleStorage } from "test/Proxy.t.sol"; +import { L1ChugSplashProxy } from "src/legacy/L1ChugSplashProxy.sol"; +import { ResolvedDelegateProxy } from "src/legacy/ResolvedDelegateProxy.sol"; +import { AddressManager } from "src/legacy/AddressManager.sol"; contract ProxyAdmin_Test is Test { address alice = address(64); diff --git a/packages/contracts-bedrock/test/RLPReader.t.sol b/packages/contracts-bedrock/test/RLPReader.t.sol index 2276370a69d2..17fe69c817c0 100644 --- a/packages/contracts-bedrock/test/RLPReader.t.sol +++ b/packages/contracts-bedrock/test/RLPReader.t.sol @@ -2,8 +2,8 @@ pragma solidity 0.8.15; import { stdError } from "forge-std/Test.sol"; -import { CommonTest } from "./CommonTest.t.sol"; -import { RLPReader } from "../src/libraries/rlp/RLPReader.sol"; +import { CommonTest } from "test/CommonTest.t.sol"; +import { RLPReader } from "src/libraries/rlp/RLPReader.sol"; contract RLPReader_readBytes_Test is CommonTest { function test_readBytes_bytestring00_succeeds() external { diff --git a/packages/contracts-bedrock/test/RLPWriter.t.sol b/packages/contracts-bedrock/test/RLPWriter.t.sol index 6b3468c81ae9..b1735578ad44 100644 --- a/packages/contracts-bedrock/test/RLPWriter.t.sol +++ b/packages/contracts-bedrock/test/RLPWriter.t.sol @@ -1,8 +1,8 @@ // SPDX-License-Identifier: MIT pragma solidity 0.8.15; -import { RLPWriter } from "../src/libraries/rlp/RLPWriter.sol"; -import { CommonTest } from "./CommonTest.t.sol"; +import { RLPWriter } from "src/libraries/rlp/RLPWriter.sol"; +import { CommonTest } from "test/CommonTest.t.sol"; contract RLPWriter_writeString_Test is CommonTest { function test_writeString_empty_succeeds() external { diff --git a/packages/contracts-bedrock/test/ResolvedDelegateProxy.t.sol b/packages/contracts-bedrock/test/ResolvedDelegateProxy.t.sol index d740bc385964..33a81bc2ca0e 100644 --- a/packages/contracts-bedrock/test/ResolvedDelegateProxy.t.sol +++ b/packages/contracts-bedrock/test/ResolvedDelegateProxy.t.sol @@ -5,10 +5,10 @@ pragma solidity 0.8.15; import { Test } from "forge-std/Test.sol"; // Target contract dependencies -import { AddressManager } from "../src/legacy/AddressManager.sol"; +import { AddressManager } from "src/legacy/AddressManager.sol"; // Target contract -import { ResolvedDelegateProxy } from "../src/legacy/ResolvedDelegateProxy.sol"; +import { ResolvedDelegateProxy } from "src/legacy/ResolvedDelegateProxy.sol"; contract ResolvedDelegateProxy_Test is Test { AddressManager internal addressManager; diff --git a/packages/contracts-bedrock/test/ResourceMetering.t.sol b/packages/contracts-bedrock/test/ResourceMetering.t.sol index 2493896b8055..01fcab1c9264 100644 --- a/packages/contracts-bedrock/test/ResourceMetering.t.sol +++ b/packages/contracts-bedrock/test/ResourceMetering.t.sol @@ -5,13 +5,13 @@ pragma solidity 0.8.15; import { Test } from "forge-std/Test.sol"; // Libraries -import { Constants } from "../src/libraries/Constants.sol"; +import { Constants } from "src/libraries/Constants.sol"; // Target contract dependencies -import { Proxy } from "../src/universal/Proxy.sol"; +import { Proxy } from "src/universal/Proxy.sol"; // Target contract -import { ResourceMetering } from "../src/L1/ResourceMetering.sol"; +import { ResourceMetering } from "src/L1/ResourceMetering.sol"; contract MeterUser is ResourceMetering { ResourceMetering.ResourceConfig public innerConfig; diff --git a/packages/contracts-bedrock/test/SafeCall.t.sol b/packages/contracts-bedrock/test/SafeCall.t.sol index 308344787414..550e7dd982b4 100644 --- a/packages/contracts-bedrock/test/SafeCall.t.sol +++ b/packages/contracts-bedrock/test/SafeCall.t.sol @@ -2,10 +2,10 @@ pragma solidity 0.8.15; // Testing utilities -import { CommonTest } from "./CommonTest.t.sol"; +import { CommonTest } from "test/CommonTest.t.sol"; // Target contract -import { SafeCall } from "../src/libraries/SafeCall.sol"; +import { SafeCall } from "src/libraries/SafeCall.sol"; contract SafeCall_Test is CommonTest { /// @dev Tests that the `send` function succeeds. diff --git a/packages/contracts-bedrock/test/Semver.t.sol b/packages/contracts-bedrock/test/Semver.t.sol index c15044eb3fa2..08d0dae64f06 100644 --- a/packages/contracts-bedrock/test/Semver.t.sol +++ b/packages/contracts-bedrock/test/Semver.t.sol @@ -1,9 +1,9 @@ // SPDX-License-Identifier: MIT pragma solidity 0.8.15; -import { CommonTest } from "./CommonTest.t.sol"; -import { Semver } from "../src/universal/Semver.sol"; -import { Proxy } from "../src/universal/Proxy.sol"; +import { CommonTest } from "test/CommonTest.t.sol"; +import { Semver } from "src/universal/Semver.sol"; +import { Proxy } from "src/universal/Proxy.sol"; /// @notice Test the Semver contract that is used for semantic versioning /// of various contracts. diff --git a/packages/contracts-bedrock/test/SequencerFeeVault.t.sol b/packages/contracts-bedrock/test/SequencerFeeVault.t.sol index 026b80e21d57..9c6b391a9194 100644 --- a/packages/contracts-bedrock/test/SequencerFeeVault.t.sol +++ b/packages/contracts-bedrock/test/SequencerFeeVault.t.sol @@ -2,17 +2,17 @@ pragma solidity 0.8.15; // Testing utilities -import { FeeVault_Initializer, Reverter } from "./CommonTest.t.sol"; -import { StandardBridge } from "../src/universal/StandardBridge.sol"; +import { FeeVault_Initializer, Reverter } from "test/CommonTest.t.sol"; +import { StandardBridge } from "src/universal/StandardBridge.sol"; // Libraries -import { Predeploys } from "../src/libraries/Predeploys.sol"; +import { Predeploys } from "src/libraries/Predeploys.sol"; // Target contract dependencies -import { FeeVault } from "../src/universal/FeeVault.sol"; +import { FeeVault } from "src/universal/FeeVault.sol"; // Target contract -import { SequencerFeeVault } from "../src/L2/SequencerFeeVault.sol"; +import { SequencerFeeVault } from "src/L2/SequencerFeeVault.sol"; contract SequencerFeeVault_Test is FeeVault_Initializer { /// @dev Sets up the test suite. diff --git a/packages/contracts-bedrock/test/StandardBridge.t.sol b/packages/contracts-bedrock/test/StandardBridge.t.sol index fd96e09f2c67..000d2334ae34 100644 --- a/packages/contracts-bedrock/test/StandardBridge.t.sol +++ b/packages/contracts-bedrock/test/StandardBridge.t.sol @@ -1,9 +1,9 @@ // SPDX-License-Identifier: MIT pragma solidity 0.8.15; -import { StandardBridge } from "../src/universal/StandardBridge.sol"; -import { CommonTest } from "./CommonTest.t.sol"; -import { OptimismMintableERC20, ILegacyMintableERC20 } from "../src/universal/OptimismMintableERC20.sol"; +import { StandardBridge } from "src/universal/StandardBridge.sol"; +import { CommonTest } from "test/CommonTest.t.sol"; +import { OptimismMintableERC20, ILegacyMintableERC20 } from "src/universal/OptimismMintableERC20.sol"; import { ERC20 } from "@openzeppelin/contracts/token/ERC20/ERC20.sol"; /// @title StandardBridgeTester diff --git a/packages/contracts-bedrock/test/SystemConfig.t.sol b/packages/contracts-bedrock/test/SystemConfig.t.sol index bb355fc4146c..b6cb18cf7d7c 100644 --- a/packages/contracts-bedrock/test/SystemConfig.t.sol +++ b/packages/contracts-bedrock/test/SystemConfig.t.sol @@ -2,17 +2,17 @@ pragma solidity 0.8.15; // Testing utilities -import { CommonTest } from "./CommonTest.t.sol"; +import { CommonTest } from "test/CommonTest.t.sol"; // Libraries -import { Constants } from "../src/libraries/Constants.sol"; +import { Constants } from "src/libraries/Constants.sol"; // Target contract dependencies -import { ResourceMetering } from "../src/L1/ResourceMetering.sol"; -import { Proxy } from "../src/universal/Proxy.sol"; +import { ResourceMetering } from "src/L1/ResourceMetering.sol"; +import { Proxy } from "src/universal/Proxy.sol"; // Target contract -import { SystemConfig } from "../src/L1/SystemConfig.sol"; +import { SystemConfig } from "src/L1/SystemConfig.sol"; contract SystemConfig_Init is CommonTest { SystemConfig sysConf; diff --git a/packages/contracts-bedrock/test/Transactor.t.sol b/packages/contracts-bedrock/test/Transactor.t.sol index 3eefd78287cc..70c02e53d813 100644 --- a/packages/contracts-bedrock/test/Transactor.t.sol +++ b/packages/contracts-bedrock/test/Transactor.t.sol @@ -3,9 +3,9 @@ pragma solidity 0.8.15; // Testing utilities import { Test } from "forge-std/Test.sol"; -import { CallRecorder } from "./Helpers.sol"; -import { Reverter } from "./Helpers.sol"; -import { Transactor } from "../src/periphery/Transactor.sol"; +import { CallRecorder } from "test/Helpers.sol"; +import { Reverter } from "test/Helpers.sol"; +import { Transactor } from "src/periphery/Transactor.sol"; contract Transactor_Initializer is Test { address alice = address(128); diff --git a/packages/contracts-bedrock/test/TransferOnion.t.sol b/packages/contracts-bedrock/test/TransferOnion.t.sol index 88fe1dc60ab4..27947e82c20e 100644 --- a/packages/contracts-bedrock/test/TransferOnion.t.sol +++ b/packages/contracts-bedrock/test/TransferOnion.t.sol @@ -6,7 +6,7 @@ import { Test } from "forge-std/Test.sol"; import { ERC20 } from "@openzeppelin/contracts/token/ERC20/ERC20.sol"; // Target contract -import { TransferOnion } from "../src/periphery/TransferOnion.sol"; +import { TransferOnion } from "src/periphery/TransferOnion.sol"; /// @title TransferOnionTest /// @notice Test coverage of TransferOnion diff --git a/packages/contracts-bedrock/test/invariants/AddressAliasHelper.t.sol b/packages/contracts-bedrock/test/invariants/AddressAliasHelper.t.sol index 9c5d5f205a69..fd5bcc8c7cad 100644 --- a/packages/contracts-bedrock/test/invariants/AddressAliasHelper.t.sol +++ b/packages/contracts-bedrock/test/invariants/AddressAliasHelper.t.sol @@ -3,7 +3,7 @@ pragma solidity 0.8.15; import { Test } from "forge-std/Test.sol"; import { StdInvariant } from "forge-std/StdInvariant.sol"; -import { AddressAliasHelper } from "../../src/vendor/AddressAliasHelper.sol"; +import { AddressAliasHelper } from "src/vendor/AddressAliasHelper.sol"; contract AddressAliasHelper_Converter { bool public failedRoundtrip; diff --git a/packages/contracts-bedrock/test/invariants/Burn.Eth.t.sol b/packages/contracts-bedrock/test/invariants/Burn.Eth.t.sol index 49f2a0249908..0fdfcdbbb194 100644 --- a/packages/contracts-bedrock/test/invariants/Burn.Eth.t.sol +++ b/packages/contracts-bedrock/test/invariants/Burn.Eth.t.sol @@ -6,7 +6,7 @@ import { Test } from "forge-std/Test.sol"; import { Vm } from "forge-std/Vm.sol"; import { StdInvariant } from "forge-std/StdInvariant.sol"; -import { Burn } from "../../src/libraries/Burn.sol"; +import { Burn } from "src/libraries/Burn.sol"; contract Burn_EthBurner is StdUtils { Vm internal vm; diff --git a/packages/contracts-bedrock/test/invariants/Burn.Gas.t.sol b/packages/contracts-bedrock/test/invariants/Burn.Gas.t.sol index 8e973245a419..4e9145a08d77 100644 --- a/packages/contracts-bedrock/test/invariants/Burn.Gas.t.sol +++ b/packages/contracts-bedrock/test/invariants/Burn.Gas.t.sol @@ -6,7 +6,7 @@ import { Test } from "forge-std/Test.sol"; import { Vm } from "forge-std/Vm.sol"; import { StdInvariant } from "forge-std/StdInvariant.sol"; -import { Burn } from "../../src/libraries/Burn.sol"; +import { Burn } from "src/libraries/Burn.sol"; contract Burn_GasBurner is StdUtils { Vm internal vm; diff --git a/packages/contracts-bedrock/test/invariants/CrossDomainMessenger.t.sol b/packages/contracts-bedrock/test/invariants/CrossDomainMessenger.t.sol index 1fb358e85c03..6eef617f5095 100644 --- a/packages/contracts-bedrock/test/invariants/CrossDomainMessenger.t.sol +++ b/packages/contracts-bedrock/test/invariants/CrossDomainMessenger.t.sol @@ -3,14 +3,14 @@ pragma solidity 0.8.15; import { StdUtils } from "forge-std/StdUtils.sol"; import { Vm } from "forge-std/Vm.sol"; -import { OptimismPortal } from "../../src/L1/OptimismPortal.sol"; -import { L1CrossDomainMessenger } from "../../src/L1/L1CrossDomainMessenger.sol"; -import { Messenger_Initializer } from "../CommonTest.t.sol"; -import { Types } from "../../src/libraries/Types.sol"; -import { Predeploys } from "../../src/libraries/Predeploys.sol"; -import { Constants } from "../../src/libraries/Constants.sol"; -import { Encoding } from "../../src/libraries/Encoding.sol"; -import { Hashing } from "../../src/libraries/Hashing.sol"; +import { OptimismPortal } from "src/L1/OptimismPortal.sol"; +import { L1CrossDomainMessenger } from "src/L1/L1CrossDomainMessenger.sol"; +import { Messenger_Initializer } from "test/CommonTest.t.sol"; +import { Types } from "src/libraries/Types.sol"; +import { Predeploys } from "src/libraries/Predeploys.sol"; +import { Constants } from "src/libraries/Constants.sol"; +import { Encoding } from "src/libraries/Encoding.sol"; +import { Hashing } from "src/libraries/Hashing.sol"; contract RelayActor is StdUtils { // Storage slot of the l2Sender diff --git a/packages/contracts-bedrock/test/invariants/Encoding.t.sol b/packages/contracts-bedrock/test/invariants/Encoding.t.sol index 405ac3781515..566c82903e01 100644 --- a/packages/contracts-bedrock/test/invariants/Encoding.t.sol +++ b/packages/contracts-bedrock/test/invariants/Encoding.t.sol @@ -3,7 +3,7 @@ pragma solidity 0.8.15; import { Test } from "forge-std/Test.sol"; import { StdInvariant } from "forge-std/StdInvariant.sol"; -import { Encoding } from "../../src/libraries/Encoding.sol"; +import { Encoding } from "src/libraries/Encoding.sol"; contract Encoding_Converter { bool public failedRoundtripAToB; diff --git a/packages/contracts-bedrock/test/invariants/Hashing.t.sol b/packages/contracts-bedrock/test/invariants/Hashing.t.sol index 729f177be30a..c972148f12b0 100644 --- a/packages/contracts-bedrock/test/invariants/Hashing.t.sol +++ b/packages/contracts-bedrock/test/invariants/Hashing.t.sol @@ -3,8 +3,8 @@ pragma solidity 0.8.15; import { Test } from "forge-std/Test.sol"; import { StdInvariant } from "forge-std/StdInvariant.sol"; -import { Encoding } from "../../src/libraries/Encoding.sol"; -import { Hashing } from "../../src/libraries/Hashing.sol"; +import { Encoding } from "src/libraries/Encoding.sol"; +import { Hashing } from "src/libraries/Hashing.sol"; contract Hash_CrossDomainHasher { bool public failedCrossDomainHashHighVersion; diff --git a/packages/contracts-bedrock/test/invariants/L2OutputOracle.t.sol b/packages/contracts-bedrock/test/invariants/L2OutputOracle.t.sol index 8423e5a9b547..79d637fca9d6 100644 --- a/packages/contracts-bedrock/test/invariants/L2OutputOracle.t.sol +++ b/packages/contracts-bedrock/test/invariants/L2OutputOracle.t.sol @@ -1,8 +1,8 @@ // SPDX-License-Identifier: MIT pragma solidity 0.8.15; -import { L2OutputOracle_Initializer } from "../CommonTest.t.sol"; -import { L2OutputOracle } from "../../src/L1/L2OutputOracle.sol"; +import { L2OutputOracle_Initializer } from "test/CommonTest.t.sol"; +import { L2OutputOracle } from "src/L1/L2OutputOracle.sol"; import { Vm } from "forge-std/Vm.sol"; contract L2OutputOracle_Proposer { diff --git a/packages/contracts-bedrock/test/invariants/OptimismPortal.t.sol b/packages/contracts-bedrock/test/invariants/OptimismPortal.t.sol index 145a3b9e56b2..f0ac1e197843 100644 --- a/packages/contracts-bedrock/test/invariants/OptimismPortal.t.sol +++ b/packages/contracts-bedrock/test/invariants/OptimismPortal.t.sol @@ -4,15 +4,15 @@ pragma solidity 0.8.15; import { StdUtils } from "forge-std/Test.sol"; import { Vm } from "forge-std/Vm.sol"; -import { OptimismPortal } from "../../src/L1/OptimismPortal.sol"; -import { L2OutputOracle } from "../../src/L1/L2OutputOracle.sol"; -import { AddressAliasHelper } from "../../src/vendor/AddressAliasHelper.sol"; -import { SystemConfig } from "../../src/L1/SystemConfig.sol"; -import { ResourceMetering } from "../../src/L1/ResourceMetering.sol"; -import { Constants } from "../../src/libraries/Constants.sol"; - -import { Portal_Initializer } from "../CommonTest.t.sol"; -import { Types } from "../../src/libraries/Types.sol"; +import { OptimismPortal } from "src/L1/OptimismPortal.sol"; +import { L2OutputOracle } from "src/L1/L2OutputOracle.sol"; +import { AddressAliasHelper } from "src/vendor/AddressAliasHelper.sol"; +import { SystemConfig } from "src/L1/SystemConfig.sol"; +import { ResourceMetering } from "src/L1/ResourceMetering.sol"; +import { Constants } from "src/libraries/Constants.sol"; + +import { Portal_Initializer } from "test/CommonTest.t.sol"; +import { Types } from "src/libraries/Types.sol"; contract OptimismPortal_Depositor is StdUtils, ResourceMetering { Vm internal vm; diff --git a/packages/contracts-bedrock/test/invariants/ResourceMetering.t.sol b/packages/contracts-bedrock/test/invariants/ResourceMetering.t.sol index 3f2ffce7199d..866861331219 100644 --- a/packages/contracts-bedrock/test/invariants/ResourceMetering.t.sol +++ b/packages/contracts-bedrock/test/invariants/ResourceMetering.t.sol @@ -6,10 +6,10 @@ import { Test } from "forge-std/Test.sol"; import { StdUtils } from "forge-std/StdUtils.sol"; import { StdInvariant } from "forge-std/StdInvariant.sol"; -import { Arithmetic } from "../../src/libraries/Arithmetic.sol"; -import { ResourceMetering } from "../../src/L1/ResourceMetering.sol"; -import { Proxy } from "../../src/universal/Proxy.sol"; -import { Constants } from "../../src/libraries/Constants.sol"; +import { Arithmetic } from "src/libraries/Arithmetic.sol"; +import { ResourceMetering } from "src/L1/ResourceMetering.sol"; +import { Proxy } from "src/universal/Proxy.sol"; +import { Constants } from "src/libraries/Constants.sol"; contract ResourceMetering_User is StdUtils, ResourceMetering { bool public failedMaxGasPerBlock; diff --git a/packages/contracts-bedrock/test/invariants/SafeCall.t.sol b/packages/contracts-bedrock/test/invariants/SafeCall.t.sol index bb2bdc570200..66fbc0be115b 100644 --- a/packages/contracts-bedrock/test/invariants/SafeCall.t.sol +++ b/packages/contracts-bedrock/test/invariants/SafeCall.t.sol @@ -4,7 +4,7 @@ pragma solidity 0.8.15; import { Test } from "forge-std/Test.sol"; import { StdUtils } from "forge-std/StdUtils.sol"; import { Vm } from "forge-std/Vm.sol"; -import { SafeCall } from "../../src/libraries/SafeCall.sol"; +import { SafeCall } from "src/libraries/SafeCall.sol"; contract SafeCall_Succeeds_Invariants is Test { SafeCaller_Actor actor; diff --git a/packages/contracts-bedrock/test/invariants/SystemConfig.t.sol b/packages/contracts-bedrock/test/invariants/SystemConfig.t.sol index eb54e0f5ae54..436148cd609c 100644 --- a/packages/contracts-bedrock/test/invariants/SystemConfig.t.sol +++ b/packages/contracts-bedrock/test/invariants/SystemConfig.t.sol @@ -2,10 +2,10 @@ pragma solidity 0.8.15; import { Test } from "forge-std/Test.sol"; -import { SystemConfig } from "../../src/L1/SystemConfig.sol"; -import { Proxy } from "../../src/universal/Proxy.sol"; -import { ResourceMetering } from "../../src/L1/ResourceMetering.sol"; -import { Constants } from "../../src/libraries/Constants.sol"; +import { SystemConfig } from "src/L1/SystemConfig.sol"; +import { Proxy } from "src/universal/Proxy.sol"; +import { ResourceMetering } from "src/L1/ResourceMetering.sol"; +import { Constants } from "src/libraries/Constants.sol"; contract SystemConfig_GasLimitLowerBound_Invariant is Test { struct FuzzInterface { From 0ec7e153700b994205a8d7e9bd6267796bf2fd05 Mon Sep 17 00:00:00 2001 From: Mark Tyneway Date: Wed, 25 Oct 2023 19:20:52 -0600 Subject: [PATCH 096/279] contracts-bedrock: delete dead rlp code This code is no longer used so we should delete it to reduce compile times slightly. It is a library for computing contract addresses, taken from the solmate issues. It may be in modern versions of solmate if we ever end up needing it again. --- packages/contracts-bedrock/test/RLP.t.sol | 46 ----------------------- 1 file changed, 46 deletions(-) delete mode 100644 packages/contracts-bedrock/test/RLP.t.sol diff --git a/packages/contracts-bedrock/test/RLP.t.sol b/packages/contracts-bedrock/test/RLP.t.sol deleted file mode 100644 index fadd5b21bec1..000000000000 --- a/packages/contracts-bedrock/test/RLP.t.sol +++ /dev/null @@ -1,46 +0,0 @@ -// SPDX-License-Identifier: Unlicense -pragma solidity ^0.8.0; - -import { Bytes32AddressLib } from "@rari-capital/solmate/src/utils/Bytes32AddressLib.sol"; - -/// @title LibRLP -/// @notice Via https://github.com/Rari-Capital/solmate/issues/207. -library LibRLP { - using Bytes32AddressLib for bytes32; - - function computeAddress(address deployer, uint256 nonce) internal pure returns (address) { - // The integer zero is treated as an empty byte string, and as a result it only has a length prefix, 0x80, - // computed via 0x80 + 0. - // A one byte integer uses its own value as its length prefix, there is no additional "0x80 + length" prefix - // that comes before it. - if (nonce == 0x00) { - return keccak256(abi.encodePacked(bytes1(0xd6), bytes1(0x94), deployer, bytes1(0x80))).fromLast20Bytes(); - } - if (nonce <= 0x7f) { - return keccak256(abi.encodePacked(bytes1(0xd6), bytes1(0x94), deployer, uint8(nonce))).fromLast20Bytes(); - } - - // Nonces greater than 1 byte all follow a consistent encoding scheme, where each value is preceded by a prefix - // of 0x80 + length. - if (nonce <= type(uint8).max) { - return keccak256(abi.encodePacked(bytes1(0xd7), bytes1(0x94), deployer, bytes1(0x81), uint8(nonce))) - .fromLast20Bytes(); - } - if (nonce <= type(uint16).max) { - return keccak256(abi.encodePacked(bytes1(0xd8), bytes1(0x94), deployer, bytes1(0x82), uint16(nonce))) - .fromLast20Bytes(); - } - if (nonce <= type(uint24).max) { - return keccak256(abi.encodePacked(bytes1(0xd9), bytes1(0x94), deployer, bytes1(0x83), uint24(nonce))) - .fromLast20Bytes(); - } - - // More details about RLP encoding can be found here: https://eth.wiki/fundamentals/rlp - // 0xda = 0xc0 (short RLP prefix) + 0x16 (length of: 0x94 ++ proxy ++ 0x84 ++ nonce) - // 0x94 = 0x80 + 0x14 (0x14 = the length of an address, 20 bytes, in hex) - // 0x84 = 0x80 + 0x04 (0x04 = the bytes length of the nonce, 4 bytes, in hex) - // We assume nobody can have a nonce large enough to require more than 32 bytes. - return keccak256(abi.encodePacked(bytes1(0xda), bytes1(0x94), deployer, bytes1(0x84), uint32(nonce))) - .fromLast20Bytes(); - } -} From 37a0c5d8c39be66a0e848ec7fcc1942e636a935d Mon Sep 17 00:00:00 2001 From: Mark Tyneway Date: Wed, 25 Oct 2023 19:25:44 -0600 Subject: [PATCH 097/279] contracts-bedrock: delete dead eas upgrade script All upgraes in the future will go through the `superchain-ops` repo. Delete this script to reduce compile time of the contracts. All new chains will have EAS in their genesis. Chains that do not have EAS already should use the `superchain-ops` repo to create a deploy script. --- .../scripts/upgrades/EASUpgrade.s.sol | 130 ------------------ 1 file changed, 130 deletions(-) delete mode 100644 packages/contracts-bedrock/scripts/upgrades/EASUpgrade.s.sol diff --git a/packages/contracts-bedrock/scripts/upgrades/EASUpgrade.s.sol b/packages/contracts-bedrock/scripts/upgrades/EASUpgrade.s.sol deleted file mode 100644 index 3eb8db286721..000000000000 --- a/packages/contracts-bedrock/scripts/upgrades/EASUpgrade.s.sol +++ /dev/null @@ -1,130 +0,0 @@ -// SPDX-License-Identifier: MIT -pragma solidity ^0.8.0; - -import { console2 as console } from "forge-std/console2.sol"; -import { SafeBuilder } from "../universal/SafeBuilder.sol"; -import { IGnosisSafe, Enum } from "../interfaces/IGnosisSafe.sol"; -import { IMulticall3 } from "forge-std/interfaces/IMulticall3.sol"; -import { Predeploys } from "../../src/libraries/Predeploys.sol"; -import { ProxyAdmin } from "../../src/universal/ProxyAdmin.sol"; -import { Deployer } from "../Deployer.sol"; - -/// @title EASUpgrader -/// @notice Upgrades the EAS predeploys. -contract EASUpgrader is SafeBuilder, Deployer { - /// @notice The proxy admin predeploy on L2. - ProxyAdmin immutable PROXY_ADMIN = ProxyAdmin(Predeploys.PROXY_ADMIN); - - /// @notice Represents the EAS contracts predeploys - struct ContractSet { - address EAS; - address SchemaRegistry; - } - - /// @notice A mapping of chainid to a ContractSet of implementations. - mapping(uint256 => ContractSet) internal implementations; - - /// @notice A mapping of chainid to ContractSet of proxy addresses. - mapping(uint256 => ContractSet) internal proxies; - - /// @notice The expected versions for the contracts to be upgraded to. - string internal constant EAS_Version = "1.0.0"; - string internal constant SchemaRegistry_Version = "1.0.0"; - - /// @notice Place the contract addresses in storage so they can be used when building calldata. - function setUp() public override { - super.setUp(); - - implementations[OP_GOERLI] = - ContractSet({ EAS: getAddress("EAS"), SchemaRegistry: getAddress("SchemaRegistry") }); - - proxies[OP_GOERLI] = ContractSet({ EAS: Predeploys.EAS, SchemaRegistry: Predeploys.SCHEMA_REGISTRY }); - } - - /// @notice - function name() public pure override returns (string memory) { - return "EASUpgrader"; - } - - /// @notice Follow up assertions to ensure that the script ran to completion. - function _postCheck() internal view override { - ContractSet memory prox = getProxies(); - require(_versionHash(prox.EAS) == keccak256(bytes(EAS_Version)), "EAS"); - require(_versionHash(prox.SchemaRegistry) == keccak256(bytes(SchemaRegistry_Version)), "SchemaRegistry"); - - // Check that the codehashes of all implementations match the proxies set implementations. - ContractSet memory impl = getImplementations(); - require(PROXY_ADMIN.getProxyImplementation(prox.EAS).codehash == impl.EAS.codehash); - require(PROXY_ADMIN.getProxyImplementation(prox.SchemaRegistry).codehash == impl.SchemaRegistry.codehash); - } - - /// @notice Test coverage of the logic. Should only run on goerli but other chains - /// could be added. - function test_script_succeeds() external skipWhenNotForking { - address _safe; - address _proxyAdmin; - - if (block.chainid == OP_GOERLI) { - _safe = 0xE534ccA2753aCFbcDBCeB2291F596fc60495257e; - _proxyAdmin = 0x4200000000000000000000000000000000000018; - } - - require(_safe != address(0) && _proxyAdmin != address(0)); - - address[] memory owners = IGnosisSafe(payable(_safe)).getOwners(); - - for (uint256 i; i < owners.length; i++) { - address owner = owners[i]; - vm.startBroadcast(owner); - bool success = _run(_safe, _proxyAdmin); - vm.stopBroadcast(); - - if (success) { - console.log("tx success"); - break; - } - } - - _postCheck(); - } - - /// @notice Builds the calldata that the multisig needs to make for the upgrade to happen. - /// A total of 9 calls are made to the proxy admin to upgrade the implementations - /// of the predeploys. - function buildCalldata(address _proxyAdmin) internal view override returns (bytes memory) { - IMulticall3.Call3[] memory calls = new IMulticall3.Call3[](2); - - ContractSet memory impl = getImplementations(); - ContractSet memory prox = getProxies(); - - // Upgrade EAS - calls[0] = IMulticall3.Call3({ - target: _proxyAdmin, - allowFailure: false, - callData: abi.encodeCall(ProxyAdmin.upgrade, (payable(prox.EAS), impl.EAS)) - }); - - // Upgrade SchemaRegistry - calls[1] = IMulticall3.Call3({ - target: _proxyAdmin, - allowFailure: false, - callData: abi.encodeCall(ProxyAdmin.upgrade, (payable(prox.SchemaRegistry), impl.SchemaRegistry)) - }); - - return abi.encodeCall(IMulticall3.aggregate3, (calls)); - } - - /// @notice Returns the ContractSet that represents the implementations for a given network. - function getImplementations() internal view returns (ContractSet memory) { - ContractSet memory set = implementations[block.chainid]; - require(set.EAS != address(0), "no implementations for this network"); - return set; - } - - /// @notice Returns the ContractSet that represents the proxies for a given network. - function getProxies() internal view returns (ContractSet memory) { - ContractSet memory set = proxies[block.chainid]; - require(set.EAS != address(0), "no proxies for this network"); - return set; - } -} From 3d61fe8f2ca02169d7e436553dbe52098e5ee512 Mon Sep 17 00:00:00 2001 From: bendanzhentan <455462586@qq.com> Date: Wed, 25 Oct 2023 14:46:03 +0800 Subject: [PATCH 098/279] fix: string encode error TS2345 --- packages/common-ts/src/base-service/base-service-v2.ts | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/packages/common-ts/src/base-service/base-service-v2.ts b/packages/common-ts/src/base-service/base-service-v2.ts index e6b34a0d44f1..79672e9b41e6 100644 --- a/packages/common-ts/src/base-service/base-service-v2.ts +++ b/packages/common-ts/src/base-service/base-service-v2.ts @@ -336,7 +336,8 @@ export abstract class BaseServiceV2< app.use( bodyParser.json({ verify: (req, res, buf, encoding) => { - ;(req as any).rawBody = buf?.toString(encoding || 'utf8') || '' + ;(req as any).rawBody = + buf?.toString((encoding as BufferEncoding) || 'utf8') || '' }, ...(this.params.bodyParserParams ?? {}), }) From 08c3a506dc967b6b99a81016ec907326af33f8d8 Mon Sep 17 00:00:00 2001 From: Mark Tyneway Date: Wed, 25 Oct 2023 19:37:41 -0600 Subject: [PATCH 099/279] op-bindings: regenerate --- op-bindings/bindings/mips_more.go | 2 +- op-bindings/bindings/preimageoracle_more.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/op-bindings/bindings/mips_more.go b/op-bindings/bindings/mips_more.go index d6c26b36bba1..e7983bf62efd 100644 --- a/op-bindings/bindings/mips_more.go +++ b/op-bindings/bindings/mips_more.go @@ -15,7 +15,7 @@ var MIPSStorageLayout = new(solc.StorageLayout) var MIPSDeployedBin = "0x608060405234801561001057600080fd5b50600436106100415760003560e01c8063155633fe146100465780637dc0d1d01461006b578063836e7b32146100af575b600080fd5b610051634000000081565b60405163ffffffff90911681526020015b60405180910390f35b60405173ffffffffffffffffffffffffffffffffffffffff7f0000000000000000000000000000000000000000000000000000000000000000168152602001610062565b6100c26100bd366004611d2e565b6100d0565b604051908152602001610062565b60006100da611c5b565b608081146100e757600080fd5b604051610600146100f757600080fd5b6084871461010457600080fd5b6101a4851461011257600080fd5b8635608052602087013560a052604087013560e090811c60c09081526044890135821c82526048890135821c61010052604c890135821c610120526050890135821c61014052605489013590911c61016052605888013560f890811c610180526059890135901c6101a052605a880135901c6101c0526102006101e0819052606288019060005b60208110156101bd57823560e01c8252600490920191602090910190600101610199565b505050806101200151156101db576101d361061b565b915050610612565b6101408101805160010167ffffffffffffffff16905260608101516000906102039082610737565b9050603f601a82901c16600281148061022257508063ffffffff166003145b156102775760006002836303ffffff1663ffffffff16901b846080015163f00000001617905061026c8263ffffffff1660021461026057601f610263565b60005b60ff16826107f3565b945050505050610612565b6101608301516000908190601f601086901c81169190601587901c16602081106102a3576102a3611da2565b602002015192508063ffffffff851615806102c457508463ffffffff16601c145b156102fb578661016001518263ffffffff16602081106102e6576102e6611da2565b6020020151925050601f600b86901c166103b7565b60208563ffffffff16101561035d578463ffffffff16600c148061032557508463ffffffff16600d145b8061033657508463ffffffff16600e145b15610347578561ffff1692506103b7565b6103568661ffff1660106108e4565b92506103b7565b60288563ffffffff1610158061037957508463ffffffff166022145b8061038a57508463ffffffff166026145b156103b7578661016001518263ffffffff16602081106103ac576103ac611da2565b602002015192508190505b60048563ffffffff16101580156103d4575060088563ffffffff16105b806103e557508463ffffffff166001145b15610404576103f685878487610957565b975050505050505050610612565b63ffffffff6000602087831610610469576104248861ffff1660106108e4565b9095019463fffffffc861661043a816001610737565b915060288863ffffffff161015801561045a57508763ffffffff16603014155b1561046757809250600093505b505b600061047789888885610b67565b63ffffffff9081169150603f8a1690891615801561049c575060088163ffffffff1610155b80156104ae5750601c8163ffffffff16105b1561058b578063ffffffff16600814806104ce57508063ffffffff166009145b15610505576104f38163ffffffff166008146104ea57856104ed565b60005b896107f3565b9b505050505050505050505050610612565b8063ffffffff16600a03610525576104f3858963ffffffff8a16156112f7565b8063ffffffff16600b03610546576104f3858963ffffffff8a1615156112f7565b8063ffffffff16600c0361055d576104f38d6113dd565b60108163ffffffff161015801561057a5750601c8163ffffffff16105b1561058b576104f381898988611914565b8863ffffffff1660381480156105a6575063ffffffff861615155b156105db5760018b61016001518763ffffffff16602081106105ca576105ca611da2565b63ffffffff90921660209290920201525b8363ffffffff1663ffffffff146105f8576105f884600184611b0e565b610604858360016112f7565b9b5050505050505050505050505b95945050505050565b60408051608051815260a051602082015260dc519181019190915260fc51604482015261011c51604882015261013c51604c82015261015c51605082015261017c5160548201526101805161019f5160588301526101a0516101bf5160598401526101d851605a840152600092610200929091606283019190855b60208110156106ba57601c8601518452602090950194600490930192600101610696565b506000835283830384a06000945080600181146106da5760039550610702565b8280156106f257600181146106fb5760029650610700565b60009650610700565b600196505b505b50505081900390207effffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff1660f89190911b17919050565b60008061074383611bb2565b9050600384161561075357600080fd5b6020810190358460051c8160005b601b8110156107b95760208501943583821c6001168015610789576001811461079e576107af565b600084815260208390526040902093506107af565b600082815260208590526040902093505b5050600101610761565b5060805191508181146107d457630badf00d60005260206000fd5b5050601f94909416601c0360031b9390931c63ffffffff169392505050565b60006107fd611c5b565b60809050806060015160040163ffffffff16816080015163ffffffff1614610886576040517f08c379a000000000000000000000000000000000000000000000000000000000815260206004820152601260248201527f6a756d7020696e2064656c617920736c6f74000000000000000000000000000060448201526064015b60405180910390fd5b60608101805160808301805163ffffffff9081169093528583169052908516156108dc57806008018261016001518663ffffffff16602081106108cb576108cb611da2565b63ffffffff90921660209290920201525b61061261061b565b600063ffffffff8381167fffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff80850183169190911c821615159160016020869003821681901b830191861691821b92911b0182610941576000610943565b815b90861663ffffffff16179250505092915050565b6000610961611c5b565b608090506000816060015160040163ffffffff16826080015163ffffffff16146109e7576040517f08c379a000000000000000000000000000000000000000000000000000000000815260206004820152601460248201527f6272616e636820696e2064656c617920736c6f74000000000000000000000000604482015260640161087d565b8663ffffffff1660041480610a0257508663ffffffff166005145b15610a7e5760008261016001518663ffffffff1660208110610a2657610a26611da2565b602002015190508063ffffffff168563ffffffff16148015610a4e57508763ffffffff166004145b80610a7657508063ffffffff168563ffffffff1614158015610a7657508763ffffffff166005145b915050610afb565b8663ffffffff16600603610a9b5760008460030b13159050610afb565b8663ffffffff16600703610ab75760008460030b139050610afb565b8663ffffffff16600103610afb57601f601087901c166000819003610ae05760008560030b1291505b8063ffffffff16600103610af95760008560030b121591505b505b606082018051608084015163ffffffff169091528115610b41576002610b268861ffff1660106108e4565b63ffffffff90811690911b8201600401166080840152610b53565b60808301805160040163ffffffff1690525b610b5b61061b565b98975050505050505050565b6000603f601a86901c16801580610b96575060088163ffffffff1610158015610b965750600f8163ffffffff16105b15610fec57603f86168160088114610bdd5760098114610be657600a8114610bef57600b8114610bf857600c8114610c0157600d8114610c0a57600e8114610c1357610c18565b60209150610c18565b60219150610c18565b602a9150610c18565b602b9150610c18565b60249150610c18565b60259150610c18565b602691505b508063ffffffff16600003610c3f5750505063ffffffff8216601f600686901c161b6112ef565b8063ffffffff16600203610c655750505063ffffffff8216601f600686901c161c6112ef565b8063ffffffff16600303610c9b57601f600688901c16610c9163ffffffff8716821c60208390036108e4565b93505050506112ef565b8063ffffffff16600403610cbd5750505063ffffffff8216601f84161b6112ef565b8063ffffffff16600603610cdf5750505063ffffffff8216601f84161c6112ef565b8063ffffffff16600703610d1257610d098663ffffffff168663ffffffff16901c876020036108e4565b925050506112ef565b8063ffffffff16600803610d2a5785925050506112ef565b8063ffffffff16600903610d425785925050506112ef565b8063ffffffff16600a03610d5a5785925050506112ef565b8063ffffffff16600b03610d725785925050506112ef565b8063ffffffff16600c03610d8a5785925050506112ef565b8063ffffffff16600f03610da25785925050506112ef565b8063ffffffff16601003610dba5785925050506112ef565b8063ffffffff16601103610dd25785925050506112ef565b8063ffffffff16601203610dea5785925050506112ef565b8063ffffffff16601303610e025785925050506112ef565b8063ffffffff16601803610e1a5785925050506112ef565b8063ffffffff16601903610e325785925050506112ef565b8063ffffffff16601a03610e4a5785925050506112ef565b8063ffffffff16601b03610e625785925050506112ef565b8063ffffffff16602003610e7b575050508282016112ef565b8063ffffffff16602103610e94575050508282016112ef565b8063ffffffff16602203610ead575050508183036112ef565b8063ffffffff16602303610ec6575050508183036112ef565b8063ffffffff16602403610edf575050508282166112ef565b8063ffffffff16602503610ef8575050508282176112ef565b8063ffffffff16602603610f11575050508282186112ef565b8063ffffffff16602703610f2b57505050828217196112ef565b8063ffffffff16602a03610f5c578460030b8660030b12610f4d576000610f50565b60015b60ff16925050506112ef565b8063ffffffff16602b03610f84578463ffffffff168663ffffffff1610610f4d576000610f50565b6040517f08c379a000000000000000000000000000000000000000000000000000000000815260206004820152601360248201527f696e76616c696420696e737472756374696f6e00000000000000000000000000604482015260640161087d565b50610f84565b8063ffffffff16601c0361107057603f86166002819003611012575050508282026112ef565b8063ffffffff166020148061102d57508063ffffffff166021145b15610fe6578063ffffffff16602003611044579419945b60005b6380000000871615611066576401fffffffe600197881b169601611047565b92506112ef915050565b8063ffffffff16600f0361109257505065ffffffff0000601083901b166112ef565b8063ffffffff166020036110ce576110c68560031660080260180363ffffffff168463ffffffff16901c60ff1660086108e4565b9150506112ef565b8063ffffffff16602103611103576110c68560021660080260100363ffffffff168463ffffffff16901c61ffff1660106108e4565b8063ffffffff1660220361113257505063ffffffff60086003851602811681811b198416918316901b176112ef565b8063ffffffff1660230361114957829150506112ef565b8063ffffffff1660240361117b578460031660080260180363ffffffff168363ffffffff16901c60ff169150506112ef565b8063ffffffff166025036111ae578460021660080260100363ffffffff168363ffffffff16901c61ffff169150506112ef565b8063ffffffff166026036111e057505063ffffffff60086003851602601803811681811c198416918316901c176112ef565b8063ffffffff1660280361121657505060ff63ffffffff60086003861602601803811682811b9091188316918416901b176112ef565b8063ffffffff1660290361124d57505061ffff63ffffffff60086002861602601003811682811b9091188316918416901b176112ef565b8063ffffffff16602a0361127c57505063ffffffff60086003851602811681811c198316918416901c176112ef565b8063ffffffff16602b0361129357839150506112ef565b8063ffffffff16602e036112c557505063ffffffff60086003851602601803811681811b198316918416901b176112ef565b8063ffffffff166030036112dc57829150506112ef565b8063ffffffff16603803610f8457839150505b949350505050565b6000611301611c5b565b506080602063ffffffff861610611374576040517f08c379a000000000000000000000000000000000000000000000000000000000815260206004820152600e60248201527f76616c6964207265676973746572000000000000000000000000000000000000604482015260640161087d565b63ffffffff8516158015906113865750825b156113ba57838161016001518663ffffffff16602081106113a9576113a9611da2565b63ffffffff90921660209290920201525b60808101805163ffffffff8082166060850152600490910116905261061261061b565b60006113e7611c5b565b506101e051604081015160808083015160a084015160c09094015191936000928392919063ffffffff8616610ffa036114615781610fff81161561143057610fff811661100003015b8363ffffffff166000036114575760e08801805163ffffffff83820116909152955061145b565b8395505b506118d3565b8563ffffffff16610fcd0361147c57634000000094506118d3565b8563ffffffff166110180361149457600194506118d3565b8563ffffffff16611096036114ca57600161012088015260ff83166101008801526114bd61061b565b9998505050505050505050565b8563ffffffff16610fa3036117365763ffffffff8316156118d3577ffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffb63ffffffff8416016116f05760006115258363fffffffc166001610737565b60208901519091508060001a60010361159457604080516000838152336020528d83526060902091527effffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff167f01000000000000000000000000000000000000000000000000000000000000001790505b6040808a015190517fe03110e10000000000000000000000000000000000000000000000000000000081526004810183905263ffffffff9091166024820152600090819073ffffffffffffffffffffffffffffffffffffffff7f0000000000000000000000000000000000000000000000000000000000000000169063e03110e1906044016040805180830381865afa158015611635573d6000803e3d6000fd5b505050506040513d601f19601f820116820180604052508101906116599190611dd1565b91509150600386168060040382811015611671578092505b508186101561167e578591505b8260088302610100031c9250826008828460040303021b9250600180600883600403021b036001806008858560040303021b039150811981169050838119871617955050506116d58663fffffffc16600186611b0e565b60408b018051820163ffffffff169052975061173192505050565b7ffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffd63ffffffff841601611725578094506118d3565b63ffffffff9450600993505b6118d3565b8563ffffffff16610fa4036118275763ffffffff831660011480611760575063ffffffff83166002145b80611771575063ffffffff83166004145b1561177e578094506118d3565b7ffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffa63ffffffff8416016117255760006117be8363fffffffc166001610737565b602089015190915060038416600403838110156117d9578093505b83900360089081029290921c7fffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff600193850293841b0116911b176020880152600060408801529350836118d3565b8563ffffffff16610fd7036118d3578163ffffffff166003036118c75763ffffffff8316158061185d575063ffffffff83166005145b8061186e575063ffffffff83166003145b1561187c57600094506118d3565b63ffffffff831660011480611897575063ffffffff83166002145b806118a8575063ffffffff83166006145b806118b9575063ffffffff83166004145b1561172557600194506118d3565b63ffffffff9450601693505b6101608701805163ffffffff808816604090920191909152905185821660e09091015260808801805180831660608b015260040190911690526114bd61061b565b600061191e611c5b565b506080600063ffffffff871660100361193c575060c0810151611aa5565b8663ffffffff1660110361195b5763ffffffff861660c0830152611aa5565b8663ffffffff16601203611974575060a0810151611aa5565b8663ffffffff166013036119935763ffffffff861660a0830152611aa5565b8663ffffffff166018036119c75763ffffffff600387810b9087900b02602081901c821660c08501521660a0830152611aa5565b8663ffffffff166019036119f85763ffffffff86811681871602602081901c821660c08501521660a0830152611aa5565b8663ffffffff16601a03611a4e578460030b8660030b81611a1b57611a1b611df5565b0763ffffffff1660c0830152600385810b9087900b81611a3d57611a3d611df5565b0563ffffffff1660a0830152611aa5565b8663ffffffff16601b03611aa5578463ffffffff168663ffffffff1681611a7757611a77611df5565b0663ffffffff90811660c084015285811690871681611a9857611a98611df5565b0463ffffffff1660a08301525b63ffffffff841615611ae057808261016001518563ffffffff1660208110611acf57611acf611da2565b63ffffffff90921660209290920201525b60808201805163ffffffff80821660608601526004909101169052611b0361061b565b979650505050505050565b6000611b1983611bb2565b90506003841615611b2957600080fd5b6020810190601f8516601c0360031b83811b913563ffffffff90911b1916178460051c60005b601b811015611ba75760208401933582821c6001168015611b775760018114611b8c57611b9d565b60008581526020839052604090209450611b9d565b600082815260208690526040902094505b5050600101611b4f565b505060805250505050565b60ff8116610380026101a4810190369061052401811015611c55576040517f08c379a000000000000000000000000000000000000000000000000000000000815260206004820152602360248201527f636865636b207468617420746865726520697320656e6f7567682063616c6c6460448201527f6174610000000000000000000000000000000000000000000000000000000000606482015260840161087d565b50919050565b6040805161018081018252600080825260208201819052918101829052606081018290526080810182905260a0810182905260c0810182905260e08101829052610100810182905261012081018290526101408101919091526101608101611cc1611cc6565b905290565b6040518061040001604052806020906020820280368337509192915050565b60008083601f840112611cf757600080fd5b50813567ffffffffffffffff811115611d0f57600080fd5b602083019150836020828501011115611d2757600080fd5b9250929050565b600080600080600060608688031215611d4657600080fd5b853567ffffffffffffffff80821115611d5e57600080fd5b611d6a89838a01611ce5565b90975095506020880135915080821115611d8357600080fd5b50611d9088828901611ce5565b96999598509660400135949350505050565b7f4e487b7100000000000000000000000000000000000000000000000000000000600052603260045260246000fd5b60008060408385031215611de457600080fd5b505080516020909101519092909150565b7f4e487b7100000000000000000000000000000000000000000000000000000000600052601260045260246000fdfea164736f6c634300080f000a" -var MIPSDeployedSourceMap = "1131:40054:128:-:0;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;1710:45;;1745:10;1710:45;;;;;188:10:287;176:23;;;158:42;;146:2;131:18;1710:45:128;;;;;;;;2448:99;;;412:42:287;2534:6:128;400:55:287;382:74;;370:2;355:18;2448:99:128;211:251:287;26025:6379:128;;;;;;:::i;:::-;;:::i;:::-;;;1755:25:287;;;1743:2;1728:18;26025:6379:128;1609:177:287;26025:6379:128;26128:7;26171:18;;:::i;:::-;26318:4;26311:5;26308:15;26298:134;;26412:1;26409;26402:12;26298:134;26468:4;26462:11;26475:10;26459:27;26449:136;;26565:1;26562;26555:12;26449:136;26634:3;26615:17;26612:26;26602:151;;26733:1;26730;26723:12;26602:151;26798:3;26783:13;26780:22;26770:146;;26896:1;26893;26886:12;26770:146;27176:24;;27521:4;27222:20;27579:2;27280:21;;27176:24;27338:18;27222:20;27280:21;;;27176:24;27153:21;27149:52;;;27338:18;27222:20;;;27280:21;;;27176:24;27149:52;;27222:20;;27280:21;;;27176:24;27149:52;;27338:18;27222:20;27280:21;;;27176:24;27149:52;;27338:18;27222:20;27280:21;;;27176:24;27149:52;;27338:18;27222:20;27280:21;;;27176:24;27149:52;;;27338:18;27222:20;27280:21;;;27176:24;27153:21;27149:52;;;27338:18;27222:20;27280:21;;;27176:24;27149:52;;27338:18;27222:20;27280:21;;;27176:24;27149:52;;27338:18;27222:20;28197:10;27338:18;28187:21;;;27280;;;;28295:1;28280:77;28305:2;28302:1;28299:9;28280:77;;;27176:24;;27153:21;27149:52;27222:20;;28353:1;27280:21;;;;27164:2;27338:18;;;;28323:1;28316:9;28280:77;;;28284:14;;;28435:5;:12;;;28431:71;;;28474:13;:11;:13::i;:::-;28467:20;;;;;28431:71;28516:10;;;:15;;28530:1;28516:15;;;;;28601:8;;;;-1:-1:-1;;28593:20:128;;-1:-1:-1;28593:7:128;:20::i;:::-;28579:34;-1:-1:-1;28643:10:128;28651:2;28643:10;;;;28720:1;28710:11;;;:26;;;28725:6;:11;;28735:1;28725:11;28710:26;28706:310;;;28866:13;28935:1;28913:4;28920:10;28913:17;28912:24;;;;28883:5;:12;;;28898:10;28883:25;28882:54;28866:70;;28961:40;28972:6;:11;;28982:1;28972:11;:20;;28990:2;28972:20;;;28986:1;28972:20;28961:40;;28994:6;28961:10;:40::i;:::-;28954:47;;;;;;;;28706:310;29265:15;;;;29060:9;;;;29197:4;29191:2;29183:10;;;29182:19;;;29265:15;29290:2;29282:10;;;29281:19;29265:36;;;;;;;:::i;:::-;;;;;;-1:-1:-1;29330:5:128;29354:11;;;;;:29;;;29369:6;:14;;29379:4;29369:14;29354:29;29350:832;;;29446:5;:15;;;29462:5;29446:22;;;;;;;;;:::i;:::-;;;;;;-1:-1:-1;;29509:4:128;29503:2;29495:10;;;29494:19;29350:832;;;29547:4;29538:6;:13;;;29534:648;;;29668:6;:13;;29678:3;29668:13;:30;;;;29685:6;:13;;29695:3;29685:13;29668:30;:47;;;;29702:6;:13;;29712:3;29702:13;29668:47;29664:253;;;29778:4;29785:6;29778:13;29773:18;;29534:648;;29664:253;29877:21;29880:4;29887:6;29880:13;29895:2;29877;:21::i;:::-;29872:26;;29534:648;;;29951:4;29941:6;:14;;;;:32;;;;29959:6;:14;;29969:4;29959:14;29941:32;:50;;;;29977:6;:14;;29987:4;29977:14;29941:50;29937:245;;;30061:5;:15;;;30077:5;30061:22;;;;;;;;;:::i;:::-;;;;;30056:27;;30162:5;30154:13;;29937:245;30211:1;30201:6;:11;;;;:25;;;;;30225:1;30216:6;:10;;;30201:25;30200:42;;;;30231:6;:11;;30241:1;30231:11;30200:42;30196:125;;;30269:37;30282:6;30290:4;30296:5;30303:2;30269:12;:37::i;:::-;30262:44;;;;;;;;;;;30196:125;30354:13;30335:16;30506:4;30496:14;;;;30492:446;;30575:21;30578:4;30585:6;30578:13;30593:2;30575;:21::i;:::-;30569:27;;;;30633:10;30628:15;;30667:16;30628:15;30681:1;30667:7;:16::i;:::-;30661:22;;30715:4;30705:6;:14;;;;:32;;;;;30723:6;:14;;30733:4;30723:14;;30705:32;30701:223;;;30802:4;30790:16;;30904:1;30896:9;;30701:223;30512:426;30492:446;30971:10;30984:26;30992:4;30998:2;31002;31006:3;30984:7;:26::i;:::-;31013:10;30984:39;;;;-1:-1:-1;31109:4:128;31102:11;;;31141;;;:24;;;;;31164:1;31156:4;:9;;;;31141:24;:39;;;;;31176:4;31169;:11;;;31141:39;31137:860;;;31204:4;:9;;31212:1;31204:9;:22;;;;31217:4;:9;;31225:1;31217:9;31204:22;31200:144;;;31288:37;31299:4;:9;;31307:1;31299:9;:21;;31315:5;31299:21;;;31311:1;31299:21;31322:2;31288:10;:37::i;:::-;31281:44;;;;;;;;;;;;;;;31200:144;31366:4;:11;;31374:3;31366:11;31362:121;;31436:28;31445:5;31452:2;31456:7;;;;31436:8;:28::i;31362:121::-;31504:4;:11;;31512:3;31504:11;31500:121;;31574:28;31583:5;31590:2;31594:7;;;;;31574:8;:28::i;31500:121::-;31691:4;:11;;31699:3;31691:11;31687:93;;31733:28;31747:13;31733;:28::i;31687:93::-;31883:4;31875;:12;;;;:27;;;;;31898:4;31891;:11;;;31875:27;31871:112;;;31933:31;31944:4;31950:2;31954;31958:5;31933:10;:31::i;31871:112::-;32057:6;:14;;32067:4;32057:14;:28;;;;-1:-1:-1;32075:10:128;;;;;32057:28;32053:93;;;32130:1;32105:5;:15;;;32121:5;32105:22;;;;;;;;;:::i;:::-;:26;;;;:22;;;;;;:26;32053:93;32192:9;:26;;32205:13;32192:26;32188:92;;32238:27;32247:9;32258:1;32261:3;32238:8;:27::i;:::-;32361:26;32370:5;32377:3;32382:4;32361:8;:26::i;:::-;32354:33;;;;;;;;;;;;;26025:6379;;;;;;;;:::o;3087:2334::-;3634:4;3628:11;;3550:4;3353:31;3342:43;;3413:13;3353:31;3752:2;3452:13;;3342:43;3359:24;3353:31;3452:13;;;3342:43;;;;3359:24;3353:31;3452:13;;;3342:43;3359:24;3353:31;3452:13;;;3342:43;3359:24;3353:31;3452:13;;;3342:43;3359:24;3353:31;3452:13;;;3342:43;3359:24;3353:31;3452:13;;;3342:43;3413:13;4180:11;3359:24;3353:31;3452:13;;;3342:43;3413:13;4275:11;3359:24;3353:31;3452:13;;;3342:43;3359:24;3353:31;3452:13;;;3342:43;3128:12;;4415:13;;3628:11;;3452:13;;;;4180:11;3128:12;4495:84;4520:2;4517:1;4514:9;4495:84;;;3369:13;3359:24;;3353:31;3342:43;;3373:2;3413:13;;;;4575:1;3452:13;;;;4538:1;4531:9;4495:84;;;4499:14;4642:1;4638:2;4631:13;4737:5;4733:2;4729:14;4722:5;4717:27;4811:1;4797:15;;4832:6;4856:1;4851:273;;;;5191:1;5181:11;;4825:369;;4851:273;4883:8;4941:22;;;;5020:1;5015:22;;;;5107:1;5097:11;;4876:234;;4941:22;4960:1;4950:11;;4941:22;;5015;5034:1;5024:11;;4876:234;;4825:369;-1:-1:-1;;;5317:14:128;;;5300:32;;5360:19;5356:30;5392:3;5388:16;;;;5353:52;;3087:2334;-1:-1:-1;3087:2334:128:o;21746:1831::-;21819:11;21930:14;21947:24;21959:11;21947;:24::i;:::-;21930:41;;22079:1;22072:5;22068:13;22065:33;;;22094:1;22091;22084:12;22065:33;22227:2;22215:15;;;22168:20;22657:5;22654:1;22650:13;22692:4;22728:1;22713:343;22738:2;22735:1;22732:9;22713:343;;;22861:2;22849:15;;;22798:20;22896:12;;;22910:1;22892:20;22933:42;;;;23001:1;22996:42;;;;22885:153;;22933:42;22391:1;22384:12;;;22424:2;22417:13;;;22469:2;22456:16;;22942:31;;22933:42;;22996;22391:1;22384:12;;;22424:2;22417:13;;;22469:2;22456:16;;23005:31;;22885:153;-1:-1:-1;;22756:1:128;22749:9;22713:343;;;22717:14;23166:4;23160:11;23145:26;;23252:7;23246:4;23243:17;23233:124;;23294:10;23291:1;23284:21;23336:2;23333:1;23326:13;23233:124;-1:-1:-1;;23484:2:128;23473:14;;;;23461:10;23457:31;23454:1;23450:39;23518:16;;;;23536:10;23514:33;;21746:1831;-1:-1:-1;;;21746:1831:128:o;18856:823::-;18925:12;19012:18;;:::i;:::-;19080:4;19071:13;;19132:5;:8;;;19143:1;19132:12;19116:28;;:5;:12;;;:28;;;19112:95;;19164:28;;;;;2182:2:287;19164:28:128;;;2164:21:287;2221:2;2201:18;;;2194:30;2260:20;2240:18;;;2233:48;2298:18;;19164:28:128;;;;;;;;19112:95;19296:8;;;;;19329:12;;;;;19318:23;;;;;;;19355:20;;;;;19296:8;19487:13;;;19483:90;;19548:6;19557:1;19548:10;19520:5;:15;;;19536:8;19520:25;;;;;;;;;:::i;:::-;:38;;;;:25;;;;;;:38;19483:90;19649:13;:11;:13::i;2645:339::-;2706:11;2770:18;;;;2779:8;;;;2770:18;;;;;;2769:25;;;;;2786:1;2833:2;:9;;;2827:16;;;;;2826:22;;2825:32;;;;;;;2887:9;;2886:15;2769:25;2944:21;;2964:1;2944:21;;;2955:6;2944:21;2929:11;;;;;:37;;-1:-1:-1;;;2645:339:128;;;;:::o;13732:2026::-;13829:12;13915:18;;:::i;:::-;13983:4;13974:13;;14015:17;14075:5;:8;;;14086:1;14075:12;14059:28;;:5;:12;;;:28;;;14055:97;;14107:30;;;;;2529:2:287;14107:30:128;;;2511:21:287;2568:2;2548:18;;;2541:30;2607:22;2587:18;;;2580:50;2647:18;;14107:30:128;2327:344:287;14055:97:128;14222:7;:12;;14233:1;14222:12;:28;;;;14238:7;:12;;14249:1;14238:12;14222:28;14218:947;;;14270:9;14282:5;:15;;;14298:6;14282:23;;;;;;;;;:::i;:::-;;;;;14270:35;;14346:2;14339:9;;:3;:9;;;:25;;;;;14352:7;:12;;14363:1;14352:12;14339:25;14338:58;;;;14377:2;14370:9;;:3;:9;;;;:25;;;;;14383:7;:12;;14394:1;14383:12;14370:25;14323:73;;14252:159;14218:947;;;14508:7;:12;;14519:1;14508:12;14504:661;;14569:1;14561:3;14555:15;;;;14540:30;;14504:661;;;14673:7;:12;;14684:1;14673:12;14669:496;;14733:1;14726:3;14720:14;;;14705:29;;14669:496;;;14854:7;:12;;14865:1;14854:12;14850:315;;14942:4;14936:2;14927:11;;;14926:20;14912:10;14969:8;;;14965:84;;15029:1;15022:3;15016:14;;;15001:29;;14965:84;15070:3;:8;;15077:1;15070:8;15066:85;;15131:1;15123:3;15117:15;;;;15102:30;;15066:85;14868:297;14850:315;15241:8;;;;;15319:12;;;;15308:23;;;;;15475:178;;;;15566:1;15540:22;15543:5;15551:6;15543:14;15559:2;15540;:22::i;:::-;:27;;;;;;;15526:42;;15535:1;15526:42;15511:57;:12;;;:57;15475:178;;;15622:12;;;;;15637:1;15622:16;15607:31;;;;15475:178;15728:13;:11;:13::i;:::-;15721:20;13732:2026;-1:-1:-1;;;;;;;;13732:2026:128:o;32450:8733::-;32537:10;32599;32607:2;32599:10;;;;32638:11;;;:44;;;32664:1;32654:6;:11;;;;:27;;;;;32678:3;32669:6;:12;;;32654:27;32634:8490;;;32723:4;32716:11;;32847:6;32907:3;32902:25;;;;32982:3;32977:25;;;;33056:3;33051:25;;;;33131:3;33126:25;;;;33205:3;33200:25;;;;33278:3;33273:25;;;;33352:3;33347:25;;;;32840:532;;32902:25;32921:4;32913:12;;32902:25;;32977;32996:4;32988:12;;32977:25;;33051;33070:4;33062:12;;33051:25;;33126;33145:4;33137:12;;33126:25;;33200;33219:4;33211:12;;33200:25;;33273;33292:4;33284:12;;33273:25;;33347;33366:4;33358:12;;32840:532;;33435:4;:12;;33443:4;33435:12;33431:4023;;-1:-1:-1;;;33486:9:128;33478:26;;33499:4;33494:1;33486:9;;;33485:18;33478:26;33471:33;;33431:4023;33572:4;:12;;33580:4;33572:12;33568:3886;;-1:-1:-1;;;33623:9:128;33615:26;;33636:4;33631:1;33623:9;;;33622:18;33615:26;33608:33;;33568:3886;33709:4;:12;;33717:4;33709:12;33705:3749;;33774:4;33769:1;33761:9;;;33760:18;33807:27;33761:9;33810:11;;;;33823:2;:10;;;33807:2;:27::i;:::-;33800:34;;;;;;;33705:3749;33903:4;:12;;33911:4;33903:12;33899:3555;;-1:-1:-1;;;33946:17:128;;;33958:4;33953:9;;33946:17;33939:24;;33899:3555;34032:4;:11;;34040:3;34032:11;34028:3426;;-1:-1:-1;;;34074:17:128;;;34086:4;34081:9;;34074:17;34067:24;;34028:3426;34160:4;:12;;34168:4;34160:12;34156:3298;;34203:21;34212:2;34206:8;;:2;:8;;;;34221:2;34216;:7;34203:2;:21::i;:::-;34196:28;;;;;;34156:3298;34473:4;:12;;34481:4;34473:12;34469:2985;;34516:2;34509:9;;;;;;34469:2985;34587:4;:12;;34595:4;34587:12;34583:2871;;34630:2;34623:9;;;;;;34583:2871;34701:4;:12;;34709:4;34701:12;34697:2757;;34744:2;34737:9;;;;;;34697:2757;34815:4;:12;;34823:4;34815:12;34811:2643;;34858:2;34851:9;;;;;;34811:2643;34932:4;:12;;34940:4;34932:12;34928:2526;;34975:2;34968:9;;;;;;34928:2526;35092:4;:12;;35100:4;35092:12;35088:2366;;35135:2;35128:9;;;;;;35088:2366;35206:4;:12;;35214:4;35206:12;35202:2252;;35249:2;35242:9;;;;;;35202:2252;35320:4;:12;;35328:4;35320:12;35316:2138;;35363:2;35356:9;;;;;;35316:2138;35434:4;:12;;35442:4;35434:12;35430:2024;;35477:2;35470:9;;;;;;35430:2024;35548:4;:12;;35556:4;35548:12;35544:1910;;35591:2;35584:9;;;;;;35544:1910;35662:4;:12;;35670:4;35662:12;35658:1796;;35705:2;35698:9;;;;;;35658:1796;35777:4;:12;;35785:4;35777:12;35773:1681;;35820:2;35813:9;;;;;;35773:1681;35890:4;:12;;35898:4;35890:12;35886:1568;;35933:2;35926:9;;;;;;35886:1568;36004:4;:12;;36012:4;36004:12;36000:1454;;36047:2;36040:9;;;;;;36000:1454;36196:4;:12;;36204:4;36196:12;36192:1262;;-1:-1:-1;;;36240:7:128;;;36232:16;;36192:1262;36317:4;:12;;36325:4;36317:12;36313:1141;;-1:-1:-1;;;36361:7:128;;;36353:16;;36313:1141;36437:4;:12;;36445:4;36437:12;36433:1021;;-1:-1:-1;;;36481:7:128;;;36473:16;;36433:1021;36558:4;:12;;36566:4;36558:12;36554:900;;-1:-1:-1;;;36602:7:128;;;36594:16;;36554:900;36678:4;:12;;36686:4;36678:12;36674:780;;-1:-1:-1;;;36722:7:128;;;36714:16;;36674:780;36797:4;:12;;36805:4;36797:12;36793:661;;-1:-1:-1;;;36841:7:128;;;36833:16;;36793:661;36917:4;:12;;36925:4;36917:12;36913:541;;-1:-1:-1;;;36961:7:128;;;36953:16;;36913:541;37037:4;:12;;37045:4;37037:12;37033:421;;-1:-1:-1;;;37082:7:128;;;37080:10;37073:17;;37033:421;37159:4;:12;;37167:4;37159:12;37155:299;;37220:2;37202:21;;37208:2;37202:21;;;:29;;37230:1;37202:29;;;37226:1;37202:29;37195:36;;;;;;;;37155:299;37301:4;:12;;37309:4;37301:12;37297:157;;37349:2;37344:7;;:2;:7;;;:15;;37358:1;37344:15;;37297:157;37406:29;;;;;2878:2:287;37406:29:128;;;2860:21:287;2917:2;2897:18;;;2890:30;2956:21;2936:18;;;2929:49;2995:18;;37406:29:128;2676:343:287;37297:157:128;32684:4784;32634:8490;;;37524:6;:14;;37534:4;37524:14;37520:3590;;37583:4;37576:11;;37658:3;37650:11;;;37646:549;;-1:-1:-1;;;37703:21:128;;;37689:36;;37646:549;37810:4;:12;;37818:4;37810:12;:28;;;;37826:4;:12;;37834:4;37826:12;37810:28;37806:389;;;37870:4;:12;;37878:4;37870:12;37866:83;;37919:3;;;37866:83;37974:8;38012:127;38024:10;38019:15;;:20;38012:127;;38104:8;38071:3;38104:8;;;;;38071:3;38012:127;;;38171:1;-1:-1:-1;38164:8:128;;-1:-1:-1;;38164:8:128;37520:3590;38262:6;:14;;38272:4;38262:14;38258:2852;;-1:-1:-1;;38307:8:128;38313:2;38307:8;;;;38300:15;;38258:2852;38382:6;:14;;38392:4;38382:14;38378:2732;;38427:42;38445:2;38450:1;38445:6;38455:1;38444:12;38439:2;:17;38431:26;;:3;:26;;;;38461:4;38430:35;38467:1;38427:2;:42::i;:::-;38420:49;;;;;38378:2732;38536:6;:14;;38546:4;38536:14;38532:2578;;38581:45;38599:2;38604:1;38599:6;38609:1;38598:12;38593:2;:17;38585:26;;:3;:26;;;;38615:6;38584:37;38623:2;38581;:45::i;38532:2578::-;38694:6;:14;;38704:4;38694:14;38690:2420;;-1:-1:-1;;38745:21:128;38764:1;38759;38754:6;;38753:12;38745:21;;38802:36;;;38873:5;38868:10;;38745:21;;;;;38867:18;38860:25;;38690:2420;38952:6;:14;;38962:4;38952:14;38948:2162;;38997:3;38990:10;;;;;38948:2162;39068:6;:14;;39078:4;39068:14;39064:2046;;39128:2;39133:1;39128:6;39138:1;39127:12;39122:2;:17;39114:26;;:3;:26;;;;39144:4;39113:35;39106:42;;;;;39064:2046;39217:6;:14;;39227:4;39217:14;39213:1897;;39277:2;39282:1;39277:6;39287:1;39276:12;39271:2;:17;39263:26;;:3;:26;;;;39293:6;39262:37;39255:44;;;;;39213:1897;39368:6;:14;;39378:4;39368:14;39364:1746;;-1:-1:-1;;39419:26:128;39443:1;39438;39433:6;;39432:12;39427:2;:17;39419:26;;39481:41;;;39557:5;39552:10;;39419:26;;;;;39551:18;39544:25;;39364:1746;39637:6;:14;;39647:4;39637:14;39633:1477;;-1:-1:-1;;39694:4:128;39688:34;39720:1;39715;39710:6;;39709:12;39704:2;:17;39688:34;;39778:27;;;39758:48;;;39836:10;;39689:9;;;39688:34;;39835:18;39828:25;;39633:1477;39921:6;:14;;39931:4;39921:14;39917:1193;;-1:-1:-1;;39978:6:128;39972:36;40006:1;40001;39996:6;;39995:12;39990:2;:17;39972:36;;40064:29;;;40044:50;;;40124:10;;39973:11;;;39972:36;;40123:18;40116:25;;39917:1193;40210:6;:14;;40220:4;40210:14;40206:904;;-1:-1:-1;;40261:20:128;40279:1;40274;40269:6;;40268:12;40261:20;;40317:36;;;40389:5;40383:11;;40261:20;;;;;40382:19;40375:26;;40206:904;40469:6;:14;;40479:4;40469:14;40465:645;;40514:2;40507:9;;;;;40465:645;40585:6;:14;;40595:4;40585:14;40581:529;;-1:-1:-1;;40636:25:128;40659:1;40654;40649:6;;40648:12;40643:2;:17;40636:25;;40697:41;;;40774:5;40768:11;;40636:25;;;;;40767:19;40760:26;;40581:529;40853:6;:14;;40863:4;40853:14;40849:261;;40898:3;40891:10;;;;;40849:261;40968:6;:14;;40978:4;40968:14;40964:146;;41013:2;41006:9;;;32450:8733;;;;;;;:::o;19960:782::-;20046:12;20133:18;;:::i;:::-;-1:-1:-1;20201:4:128;20308:2;20296:14;;;;20288:41;;;;;;;3226:2:287;20288:41:128;;;3208:21:287;3265:2;3245:18;;;3238:30;3304:16;3284:18;;;3277:44;3338:18;;20288:41:128;3024:338:287;20288:41:128;20425:14;;;;;;;:30;;;20443:12;20425:30;20421:102;;;20504:4;20475:5;:15;;;20491:9;20475:26;;;;;;;;;:::i;:::-;:33;;;;:26;;;;;;:33;20421:102;20578:12;;;;;20567:23;;;;:8;;;:23;20634:1;20619:16;;;20604:31;;;20712:13;:11;:13::i;5582:7764::-;5646:12;5732:18;;:::i;:::-;-1:-1:-1;5910:15:128;;:18;;;;5800:4;6070:18;;;;6114;;;;6158;;;;;5800:4;;5890:17;;;;6070:18;6114;6248;;;6262:4;6248:18;6244:6792;;6298:2;6327:4;6322:9;;:14;6318:144;;6438:4;6433:9;;6425:4;:18;6419:24;6318:144;6483:2;:7;;6489:1;6483:7;6479:161;;6519:10;;;;;6551:16;;;;;;;;6519:10;-1:-1:-1;6479:161:128;;;6619:2;6614:7;;6479:161;6268:386;6244:6792;;;6756:10;:18;;6770:4;6756:18;6752:6284;;1745:10;6794:14;;6752:6284;;;6892:10;:18;;6906:4;6892:18;6888:6148;;6935:1;6930:6;;6888:6148;;;7060:10;:18;;7074:4;7060:18;7056:5980;;7113:4;7098:12;;;:19;7135:26;;;:14;;;:26;7186:13;:11;:13::i;:::-;7179:20;5582:7764;-1:-1:-1;;;;;;;;;5582:7764:128:o;7056:5980::-;7325:10;:18;;7339:4;7325:18;7321:5715;;7476:14;;;7472:2723;7321:5715;7472:2723;7646:22;;;;;7642:2553;;7771:10;7784:27;7792:2;7797:10;7792:15;7809:1;7784:7;:27::i;:::-;7895:17;;;;7771:40;;-1:-1:-1;7895:17:128;7873:19;8045:14;8064:1;8039:26;8035:146;;1676:4:129;1670:11;;1533:21;1787:15;;;1828:8;1822:4;1815:22;1850:27;;;1996:4;1983:18;;2098:17;;2003:19;1979:44;2025:11;1976:61;8093:65:128;;8035:146;8267:20;;;;;8234:54;;;;;;;;3540:25:287;;;8234:54:128;3601:23:287;;;3581:18;;;3574:51;8203:11:128;;;;8234:19;:6;:19;;;;3513:18:287;;8234:54:128;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;:::i;:::-;8202:86;;;;8515:1;8511:2;8507:10;8612:9;8609:1;8605:17;8694:6;8687:5;8684:17;8681:40;;;8714:5;8704:15;;8681:40;;8797:6;8793:2;8790:14;8787:34;;;8817:2;8807:12;;8787:34;8923:3;8918:1;8910:6;8906:14;8901:3;8897:24;8893:34;8886:41;;9023:3;9019:1;9007:9;8998:6;8995:1;8991:14;8987:30;8983:38;8979:48;8972:55;;9178:1;9174;9170;9158:9;9155:1;9151:17;9147:25;9143:33;9139:41;9305:1;9301;9297;9288:6;9276:9;9273:1;9269:17;9265:30;9261:38;9257:46;9253:54;9235:72;;9436:10;9432:15;9426:4;9422:26;9414:34;;9552:3;9544:4;9540:9;9535:3;9531:19;9528:28;9521:35;;;;9698:33;9707:2;9712:10;9707:15;9724:1;9727:3;9698:8;:33::i;:::-;9753:20;;;:38;;;;;;;;;-1:-1:-1;7642:2553:128;;-1:-1:-1;;;7642:2553:128;;9910:18;;;;;9906:289;;10080:2;10075:7;;7321:5715;;9906:289;10134:10;10129:15;;2053:3;10166:10;;9906:289;7321:5715;;;10324:10;:18;;10338:4;10324:18;10320:2716;;10478:15;;;1824:1;10478:15;;:34;;-1:-1:-1;10497:15:128;;;1859:1;10497:15;10478:34;:57;;;-1:-1:-1;10516:19:128;;;1936:1;10516:19;10478:57;10474:1593;;;10564:2;10559:7;;10320:2716;;10474:1593;10690:23;;;;;10686:1381;;10737:10;10750:27;10758:2;10763:10;10758:15;10775:1;10750:7;:27::i;:::-;10853:17;;;;10737:40;;-1:-1:-1;11096:1:128;11088:10;;11190:1;11186:17;11265:13;;;11262:32;;;11287:5;11281:11;;11262:32;11573:14;;;11379:1;11569:22;;;11565:32;;;;11462:26;11486:1;11371:10;;;11466:18;;;11462:26;11561:43;11367:20;;11669:12;11797:17;;;:23;11865:1;11842:20;;;:24;11375:2;-1:-1:-1;11375:2:128;7321:5715;;10320:2716;12269:10;:18;;12283:4;12269:18;12265:771;;12379:2;:7;;12385:1;12379:7;12375:647;;12472:14;;;;;:40;;-1:-1:-1;12490:22:128;;;1978:1;12490:22;12472:40;:62;;;-1:-1:-1;12516:18:128;;;1897:1;12516:18;12472:62;12468:404;;;12567:1;12562:6;;12375:647;;12468:404;12613:15;;;1824:1;12613:15;;:34;;-1:-1:-1;12632:15:128;;;1859:1;12632:15;12613:34;:61;;;-1:-1:-1;12651:23:128;;;2021:1;12651:23;12613:61;:84;;;-1:-1:-1;12678:19:128;;;1936:1;12678:19;12613:84;12609:263;;;12730:1;12725:6;;7321:5715;;12375:647;12923:10;12918:15;;2087:4;12955:11;;12375:647;13111:15;;;;;:23;;;;:18;;;;:23;;;;13148:15;;:23;;;:18;;;;:23;-1:-1:-1;13237:12:128;;;;13226:23;;;:8;;;:23;13293:1;13278:16;13263:31;;;;;13316:13;:11;:13::i;16084:2480::-;16178:12;16264:18;;:::i;:::-;-1:-1:-1;16332:4:128;16364:10;16472:13;;;16481:4;16472:13;16468:1705;;-1:-1:-1;16511:8:128;;;;16468:1705;;;16630:5;:13;;16639:4;16630:13;16626:1547;;16663:14;;;:8;;;:14;16626:1547;;;16793:5;:13;;16802:4;16793:13;16789:1384;;-1:-1:-1;16832:8:128;;;;16789:1384;;;16951:5;:13;;16960:4;16951:13;16947:1226;;16984:14;;;:8;;;:14;16947:1226;;;17125:5;:13;;17134:4;17125:13;17121:1052;;17252:9;17198:17;17178;;;17198;;;;17178:37;17259:2;17252:9;;;;;17234:8;;;:28;17280:22;:8;;;:22;17121:1052;;;17439:5;:13;;17448:4;17439:13;17435:738;;17506:11;17492;;;17506;;;17492:25;17561:2;17554:9;;;;;17536:8;;;:28;17582:22;:8;;;:22;17435:738;;;17763:5;:13;;17772:4;17763:13;17759:414;;17833:3;17814:23;;17820:3;17814:23;;;;;;;:::i;:::-;;17796:42;;:8;;;:42;17874:23;;;;;;;;;;;;;:::i;:::-;;17856:42;;:8;;;:42;17759:414;;;18067:5;:13;;18076:4;18067:13;18063:110;;18117:3;18111:9;;:3;:9;;;;;;;:::i;:::-;;18100:20;;;;:8;;;:20;18149:9;;;;;;;;;;;:::i;:::-;;18138:20;;:8;;;:20;18063:110;18266:14;;;;18262:85;;18329:3;18300:5;:15;;;18316:9;18300:26;;;;;;;;;:::i;:::-;:32;;;;:26;;;;;;:32;18262:85;18401:12;;;;;18390:23;;;;:8;;;:23;18457:1;18442:16;;;18427:31;;;18534:13;:11;:13::i;:::-;18527:20;16084:2480;-1:-1:-1;;;;;;;16084:2480:128:o;23913:1654::-;24089:14;24106:24;24118:11;24106;:24::i;:::-;24089:41;;24238:1;24231:5;24227:13;24224:33;;;24253:1;24250;24243:12;24224:33;24392:2;24586:15;;;24411:2;24400:14;;24388:10;24384:31;24381:1;24377:39;24542:16;;;24327:20;;24527:10;24516:22;;;24512:27;24502:38;24499:60;25028:5;25025:1;25021:13;25099:1;25084:343;25109:2;25106:1;25103:9;25084:343;;;25232:2;25220:15;;;25169:20;25267:12;;;25281:1;25263:20;25304:42;;;;25372:1;25367:42;;;;25256:153;;25304:42;22391:1;22384:12;;;22424:2;22417:13;;;22469:2;22456:16;;25313:31;;25304:42;;25367;22391:1;22384:12;;;22424:2;22417:13;;;22469:2;22456:16;;25376:31;;25256:153;-1:-1:-1;;25127:1:128;25120:9;25084:343;;;-1:-1:-1;;25526:4:128;25519:18;-1:-1:-1;;;;23913:1654:128:o;20946:586::-;21268:20;;;21292:7;21268:32;21261:3;:40;;;21374:14;;21429:17;;21423:24;;;21415:72;;;;;;;4277:2:287;21415:72:128;;;4259:21:287;4316:2;4296:18;;;4289:30;4355:34;4335:18;;;4328:62;4426:5;4406:18;;;4399:33;4449:19;;21415:72:128;4075:399:287;21415:72:128;21501:14;20946:586;;;:::o;-1:-1:-1:-;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;:::i;:::-;;;;:::o;:::-;;;;;;;;;;;;;;;;;;;;;;;;:::o;467:347:287:-;518:8;528:6;582:3;575:4;567:6;563:17;559:27;549:55;;600:1;597;590:12;549:55;-1:-1:-1;623:20:287;;666:18;655:30;;652:50;;;698:1;695;688:12;652:50;735:4;727:6;723:17;711:29;;787:3;780:4;771:6;763;759:19;755:30;752:39;749:59;;;804:1;801;794:12;749:59;467:347;;;;;:::o;819:785::-;918:6;926;934;942;950;1003:2;991:9;982:7;978:23;974:32;971:52;;;1019:1;1016;1009:12;971:52;1059:9;1046:23;1088:18;1129:2;1121:6;1118:14;1115:34;;;1145:1;1142;1135:12;1115:34;1184:58;1234:7;1225:6;1214:9;1210:22;1184:58;:::i;:::-;1261:8;;-1:-1:-1;1158:84:287;-1:-1:-1;1349:2:287;1334:18;;1321:32;;-1:-1:-1;1365:16:287;;;1362:36;;;1394:1;1391;1384:12;1362:36;;1433:60;1485:7;1474:8;1463:9;1459:24;1433:60;:::i;:::-;819:785;;;;-1:-1:-1;1512:8:287;1594:2;1579:18;1566:32;;819:785;-1:-1:-1;;;;819:785:287:o;1791:184::-;1843:77;1840:1;1833:88;1940:4;1937:1;1930:15;1964:4;1961:1;1954:15;3636:245;3715:6;3723;3776:2;3764:9;3755:7;3751:23;3747:32;3744:52;;;3792:1;3789;3782:12;3744:52;-1:-1:-1;;3815:16:287;;3871:2;3856:18;;;3850:25;3815:16;;3850:25;;-1:-1:-1;3636:245:287:o;3886:184::-;3938:77;3935:1;3928:88;4035:4;4032:1;4025:15;4059:4;4056:1;4049:15" +var MIPSDeployedSourceMap = "1131:40054:127:-:0;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;1710:45;;1745:10;1710:45;;;;;188:10:286;176:23;;;158:42;;146:2;131:18;1710:45:127;;;;;;;;2448:99;;;412:42:286;2534:6:127;400:55:286;382:74;;370:2;355:18;2448:99:127;211:251:286;26025:6379:127;;;;;;:::i;:::-;;:::i;:::-;;;1755:25:286;;;1743:2;1728:18;26025:6379:127;1609:177:286;26025:6379:127;26128:7;26171:18;;:::i;:::-;26318:4;26311:5;26308:15;26298:134;;26412:1;26409;26402:12;26298:134;26468:4;26462:11;26475:10;26459:27;26449:136;;26565:1;26562;26555:12;26449:136;26634:3;26615:17;26612:26;26602:151;;26733:1;26730;26723:12;26602:151;26798:3;26783:13;26780:22;26770:146;;26896:1;26893;26886:12;26770:146;27176:24;;27521:4;27222:20;27579:2;27280:21;;27176:24;27338:18;27222:20;27280:21;;;27176:24;27153:21;27149:52;;;27338:18;27222:20;;;27280:21;;;27176:24;27149:52;;27222:20;;27280:21;;;27176:24;27149:52;;27338:18;27222:20;27280:21;;;27176:24;27149:52;;27338:18;27222:20;27280:21;;;27176:24;27149:52;;27338:18;27222:20;27280:21;;;27176:24;27149:52;;;27338:18;27222:20;27280:21;;;27176:24;27153:21;27149:52;;;27338:18;27222:20;27280:21;;;27176:24;27149:52;;27338:18;27222:20;27280:21;;;27176:24;27149:52;;27338:18;27222:20;28197:10;27338:18;28187:21;;;27280;;;;28295:1;28280:77;28305:2;28302:1;28299:9;28280:77;;;27176:24;;27153:21;27149:52;27222:20;;28353:1;27280:21;;;;27164:2;27338:18;;;;28323:1;28316:9;28280:77;;;28284:14;;;28435:5;:12;;;28431:71;;;28474:13;:11;:13::i;:::-;28467:20;;;;;28431:71;28516:10;;;:15;;28530:1;28516:15;;;;;28601:8;;;;-1:-1:-1;;28593:20:127;;-1:-1:-1;28593:7:127;:20::i;:::-;28579:34;-1:-1:-1;28643:10:127;28651:2;28643:10;;;;28720:1;28710:11;;;:26;;;28725:6;:11;;28735:1;28725:11;28710:26;28706:310;;;28866:13;28935:1;28913:4;28920:10;28913:17;28912:24;;;;28883:5;:12;;;28898:10;28883:25;28882:54;28866:70;;28961:40;28972:6;:11;;28982:1;28972:11;:20;;28990:2;28972:20;;;28986:1;28972:20;28961:40;;28994:6;28961:10;:40::i;:::-;28954:47;;;;;;;;28706:310;29265:15;;;;29060:9;;;;29197:4;29191:2;29183:10;;;29182:19;;;29265:15;29290:2;29282:10;;;29281:19;29265:36;;;;;;;:::i;:::-;;;;;;-1:-1:-1;29330:5:127;29354:11;;;;;:29;;;29369:6;:14;;29379:4;29369:14;29354:29;29350:832;;;29446:5;:15;;;29462:5;29446:22;;;;;;;;;:::i;:::-;;;;;;-1:-1:-1;;29509:4:127;29503:2;29495:10;;;29494:19;29350:832;;;29547:4;29538:6;:13;;;29534:648;;;29668:6;:13;;29678:3;29668:13;:30;;;;29685:6;:13;;29695:3;29685:13;29668:30;:47;;;;29702:6;:13;;29712:3;29702:13;29668:47;29664:253;;;29778:4;29785:6;29778:13;29773:18;;29534:648;;29664:253;29877:21;29880:4;29887:6;29880:13;29895:2;29877;:21::i;:::-;29872:26;;29534:648;;;29951:4;29941:6;:14;;;;:32;;;;29959:6;:14;;29969:4;29959:14;29941:32;:50;;;;29977:6;:14;;29987:4;29977:14;29941:50;29937:245;;;30061:5;:15;;;30077:5;30061:22;;;;;;;;;:::i;:::-;;;;;30056:27;;30162:5;30154:13;;29937:245;30211:1;30201:6;:11;;;;:25;;;;;30225:1;30216:6;:10;;;30201:25;30200:42;;;;30231:6;:11;;30241:1;30231:11;30200:42;30196:125;;;30269:37;30282:6;30290:4;30296:5;30303:2;30269:12;:37::i;:::-;30262:44;;;;;;;;;;;30196:125;30354:13;30335:16;30506:4;30496:14;;;;30492:446;;30575:21;30578:4;30585:6;30578:13;30593:2;30575;:21::i;:::-;30569:27;;;;30633:10;30628:15;;30667:16;30628:15;30681:1;30667:7;:16::i;:::-;30661:22;;30715:4;30705:6;:14;;;;:32;;;;;30723:6;:14;;30733:4;30723:14;;30705:32;30701:223;;;30802:4;30790:16;;30904:1;30896:9;;30701:223;30512:426;30492:446;30971:10;30984:26;30992:4;30998:2;31002;31006:3;30984:7;:26::i;:::-;31013:10;30984:39;;;;-1:-1:-1;31109:4:127;31102:11;;;31141;;;:24;;;;;31164:1;31156:4;:9;;;;31141:24;:39;;;;;31176:4;31169;:11;;;31141:39;31137:860;;;31204:4;:9;;31212:1;31204:9;:22;;;;31217:4;:9;;31225:1;31217:9;31204:22;31200:144;;;31288:37;31299:4;:9;;31307:1;31299:9;:21;;31315:5;31299:21;;;31311:1;31299:21;31322:2;31288:10;:37::i;:::-;31281:44;;;;;;;;;;;;;;;31200:144;31366:4;:11;;31374:3;31366:11;31362:121;;31436:28;31445:5;31452:2;31456:7;;;;31436:8;:28::i;31362:121::-;31504:4;:11;;31512:3;31504:11;31500:121;;31574:28;31583:5;31590:2;31594:7;;;;;31574:8;:28::i;31500:121::-;31691:4;:11;;31699:3;31691:11;31687:93;;31733:28;31747:13;31733;:28::i;31687:93::-;31883:4;31875;:12;;;;:27;;;;;31898:4;31891;:11;;;31875:27;31871:112;;;31933:31;31944:4;31950:2;31954;31958:5;31933:10;:31::i;31871:112::-;32057:6;:14;;32067:4;32057:14;:28;;;;-1:-1:-1;32075:10:127;;;;;32057:28;32053:93;;;32130:1;32105:5;:15;;;32121:5;32105:22;;;;;;;;;:::i;:::-;:26;;;;:22;;;;;;:26;32053:93;32192:9;:26;;32205:13;32192:26;32188:92;;32238:27;32247:9;32258:1;32261:3;32238:8;:27::i;:::-;32361:26;32370:5;32377:3;32382:4;32361:8;:26::i;:::-;32354:33;;;;;;;;;;;;;26025:6379;;;;;;;;:::o;3087:2334::-;3634:4;3628:11;;3550:4;3353:31;3342:43;;3413:13;3353:31;3752:2;3452:13;;3342:43;3359:24;3353:31;3452:13;;;3342:43;;;;3359:24;3353:31;3452:13;;;3342:43;3359:24;3353:31;3452:13;;;3342:43;3359:24;3353:31;3452:13;;;3342:43;3359:24;3353:31;3452:13;;;3342:43;3359:24;3353:31;3452:13;;;3342:43;3413:13;4180:11;3359:24;3353:31;3452:13;;;3342:43;3413:13;4275:11;3359:24;3353:31;3452:13;;;3342:43;3359:24;3353:31;3452:13;;;3342:43;3128:12;;4415:13;;3628:11;;3452:13;;;;4180:11;3128:12;4495:84;4520:2;4517:1;4514:9;4495:84;;;3369:13;3359:24;;3353:31;3342:43;;3373:2;3413:13;;;;4575:1;3452:13;;;;4538:1;4531:9;4495:84;;;4499:14;4642:1;4638:2;4631:13;4737:5;4733:2;4729:14;4722:5;4717:27;4811:1;4797:15;;4832:6;4856:1;4851:273;;;;5191:1;5181:11;;4825:369;;4851:273;4883:8;4941:22;;;;5020:1;5015:22;;;;5107:1;5097:11;;4876:234;;4941:22;4960:1;4950:11;;4941:22;;5015;5034:1;5024:11;;4876:234;;4825:369;-1:-1:-1;;;5317:14:127;;;5300:32;;5360:19;5356:30;5392:3;5388:16;;;;5353:52;;3087:2334;-1:-1:-1;3087:2334:127:o;21746:1831::-;21819:11;21930:14;21947:24;21959:11;21947;:24::i;:::-;21930:41;;22079:1;22072:5;22068:13;22065:33;;;22094:1;22091;22084:12;22065:33;22227:2;22215:15;;;22168:20;22657:5;22654:1;22650:13;22692:4;22728:1;22713:343;22738:2;22735:1;22732:9;22713:343;;;22861:2;22849:15;;;22798:20;22896:12;;;22910:1;22892:20;22933:42;;;;23001:1;22996:42;;;;22885:153;;22933:42;22391:1;22384:12;;;22424:2;22417:13;;;22469:2;22456:16;;22942:31;;22933:42;;22996;22391:1;22384:12;;;22424:2;22417:13;;;22469:2;22456:16;;23005:31;;22885:153;-1:-1:-1;;22756:1:127;22749:9;22713:343;;;22717:14;23166:4;23160:11;23145:26;;23252:7;23246:4;23243:17;23233:124;;23294:10;23291:1;23284:21;23336:2;23333:1;23326:13;23233:124;-1:-1:-1;;23484:2:127;23473:14;;;;23461:10;23457:31;23454:1;23450:39;23518:16;;;;23536:10;23514:33;;21746:1831;-1:-1:-1;;;21746:1831:127:o;18856:823::-;18925:12;19012:18;;:::i;:::-;19080:4;19071:13;;19132:5;:8;;;19143:1;19132:12;19116:28;;:5;:12;;;:28;;;19112:95;;19164:28;;;;;2182:2:286;19164:28:127;;;2164:21:286;2221:2;2201:18;;;2194:30;2260:20;2240:18;;;2233:48;2298:18;;19164:28:127;;;;;;;;19112:95;19296:8;;;;;19329:12;;;;;19318:23;;;;;;;19355:20;;;;;19296:8;19487:13;;;19483:90;;19548:6;19557:1;19548:10;19520:5;:15;;;19536:8;19520:25;;;;;;;;;:::i;:::-;:38;;;;:25;;;;;;:38;19483:90;19649:13;:11;:13::i;2645:339::-;2706:11;2770:18;;;;2779:8;;;;2770:18;;;;;;2769:25;;;;;2786:1;2833:2;:9;;;2827:16;;;;;2826:22;;2825:32;;;;;;;2887:9;;2886:15;2769:25;2944:21;;2964:1;2944:21;;;2955:6;2944:21;2929:11;;;;;:37;;-1:-1:-1;;;2645:339:127;;;;:::o;13732:2026::-;13829:12;13915:18;;:::i;:::-;13983:4;13974:13;;14015:17;14075:5;:8;;;14086:1;14075:12;14059:28;;:5;:12;;;:28;;;14055:97;;14107:30;;;;;2529:2:286;14107:30:127;;;2511:21:286;2568:2;2548:18;;;2541:30;2607:22;2587:18;;;2580:50;2647:18;;14107:30:127;2327:344:286;14055:97:127;14222:7;:12;;14233:1;14222:12;:28;;;;14238:7;:12;;14249:1;14238:12;14222:28;14218:947;;;14270:9;14282:5;:15;;;14298:6;14282:23;;;;;;;;;:::i;:::-;;;;;14270:35;;14346:2;14339:9;;:3;:9;;;:25;;;;;14352:7;:12;;14363:1;14352:12;14339:25;14338:58;;;;14377:2;14370:9;;:3;:9;;;;:25;;;;;14383:7;:12;;14394:1;14383:12;14370:25;14323:73;;14252:159;14218:947;;;14508:7;:12;;14519:1;14508:12;14504:661;;14569:1;14561:3;14555:15;;;;14540:30;;14504:661;;;14673:7;:12;;14684:1;14673:12;14669:496;;14733:1;14726:3;14720:14;;;14705:29;;14669:496;;;14854:7;:12;;14865:1;14854:12;14850:315;;14942:4;14936:2;14927:11;;;14926:20;14912:10;14969:8;;;14965:84;;15029:1;15022:3;15016:14;;;15001:29;;14965:84;15070:3;:8;;15077:1;15070:8;15066:85;;15131:1;15123:3;15117:15;;;;15102:30;;15066:85;14868:297;14850:315;15241:8;;;;;15319:12;;;;15308:23;;;;;15475:178;;;;15566:1;15540:22;15543:5;15551:6;15543:14;15559:2;15540;:22::i;:::-;:27;;;;;;;15526:42;;15535:1;15526:42;15511:57;:12;;;:57;15475:178;;;15622:12;;;;;15637:1;15622:16;15607:31;;;;15475:178;15728:13;:11;:13::i;:::-;15721:20;13732:2026;-1:-1:-1;;;;;;;;13732:2026:127:o;32450:8733::-;32537:10;32599;32607:2;32599:10;;;;32638:11;;;:44;;;32664:1;32654:6;:11;;;;:27;;;;;32678:3;32669:6;:12;;;32654:27;32634:8490;;;32723:4;32716:11;;32847:6;32907:3;32902:25;;;;32982:3;32977:25;;;;33056:3;33051:25;;;;33131:3;33126:25;;;;33205:3;33200:25;;;;33278:3;33273:25;;;;33352:3;33347:25;;;;32840:532;;32902:25;32921:4;32913:12;;32902:25;;32977;32996:4;32988:12;;32977:25;;33051;33070:4;33062:12;;33051:25;;33126;33145:4;33137:12;;33126:25;;33200;33219:4;33211:12;;33200:25;;33273;33292:4;33284:12;;33273:25;;33347;33366:4;33358:12;;32840:532;;33435:4;:12;;33443:4;33435:12;33431:4023;;-1:-1:-1;;;33486:9:127;33478:26;;33499:4;33494:1;33486:9;;;33485:18;33478:26;33471:33;;33431:4023;33572:4;:12;;33580:4;33572:12;33568:3886;;-1:-1:-1;;;33623:9:127;33615:26;;33636:4;33631:1;33623:9;;;33622:18;33615:26;33608:33;;33568:3886;33709:4;:12;;33717:4;33709:12;33705:3749;;33774:4;33769:1;33761:9;;;33760:18;33807:27;33761:9;33810:11;;;;33823:2;:10;;;33807:2;:27::i;:::-;33800:34;;;;;;;33705:3749;33903:4;:12;;33911:4;33903:12;33899:3555;;-1:-1:-1;;;33946:17:127;;;33958:4;33953:9;;33946:17;33939:24;;33899:3555;34032:4;:11;;34040:3;34032:11;34028:3426;;-1:-1:-1;;;34074:17:127;;;34086:4;34081:9;;34074:17;34067:24;;34028:3426;34160:4;:12;;34168:4;34160:12;34156:3298;;34203:21;34212:2;34206:8;;:2;:8;;;;34221:2;34216;:7;34203:2;:21::i;:::-;34196:28;;;;;;34156:3298;34473:4;:12;;34481:4;34473:12;34469:2985;;34516:2;34509:9;;;;;;34469:2985;34587:4;:12;;34595:4;34587:12;34583:2871;;34630:2;34623:9;;;;;;34583:2871;34701:4;:12;;34709:4;34701:12;34697:2757;;34744:2;34737:9;;;;;;34697:2757;34815:4;:12;;34823:4;34815:12;34811:2643;;34858:2;34851:9;;;;;;34811:2643;34932:4;:12;;34940:4;34932:12;34928:2526;;34975:2;34968:9;;;;;;34928:2526;35092:4;:12;;35100:4;35092:12;35088:2366;;35135:2;35128:9;;;;;;35088:2366;35206:4;:12;;35214:4;35206:12;35202:2252;;35249:2;35242:9;;;;;;35202:2252;35320:4;:12;;35328:4;35320:12;35316:2138;;35363:2;35356:9;;;;;;35316:2138;35434:4;:12;;35442:4;35434:12;35430:2024;;35477:2;35470:9;;;;;;35430:2024;35548:4;:12;;35556:4;35548:12;35544:1910;;35591:2;35584:9;;;;;;35544:1910;35662:4;:12;;35670:4;35662:12;35658:1796;;35705:2;35698:9;;;;;;35658:1796;35777:4;:12;;35785:4;35777:12;35773:1681;;35820:2;35813:9;;;;;;35773:1681;35890:4;:12;;35898:4;35890:12;35886:1568;;35933:2;35926:9;;;;;;35886:1568;36004:4;:12;;36012:4;36004:12;36000:1454;;36047:2;36040:9;;;;;;36000:1454;36196:4;:12;;36204:4;36196:12;36192:1262;;-1:-1:-1;;;36240:7:127;;;36232:16;;36192:1262;36317:4;:12;;36325:4;36317:12;36313:1141;;-1:-1:-1;;;36361:7:127;;;36353:16;;36313:1141;36437:4;:12;;36445:4;36437:12;36433:1021;;-1:-1:-1;;;36481:7:127;;;36473:16;;36433:1021;36558:4;:12;;36566:4;36558:12;36554:900;;-1:-1:-1;;;36602:7:127;;;36594:16;;36554:900;36678:4;:12;;36686:4;36678:12;36674:780;;-1:-1:-1;;;36722:7:127;;;36714:16;;36674:780;36797:4;:12;;36805:4;36797:12;36793:661;;-1:-1:-1;;;36841:7:127;;;36833:16;;36793:661;36917:4;:12;;36925:4;36917:12;36913:541;;-1:-1:-1;;;36961:7:127;;;36953:16;;36913:541;37037:4;:12;;37045:4;37037:12;37033:421;;-1:-1:-1;;;37082:7:127;;;37080:10;37073:17;;37033:421;37159:4;:12;;37167:4;37159:12;37155:299;;37220:2;37202:21;;37208:2;37202:21;;;:29;;37230:1;37202:29;;;37226:1;37202:29;37195:36;;;;;;;;37155:299;37301:4;:12;;37309:4;37301:12;37297:157;;37349:2;37344:7;;:2;:7;;;:15;;37358:1;37344:15;;37297:157;37406:29;;;;;2878:2:286;37406:29:127;;;2860:21:286;2917:2;2897:18;;;2890:30;2956:21;2936:18;;;2929:49;2995:18;;37406:29:127;2676:343:286;37297:157:127;32684:4784;32634:8490;;;37524:6;:14;;37534:4;37524:14;37520:3590;;37583:4;37576:11;;37658:3;37650:11;;;37646:549;;-1:-1:-1;;;37703:21:127;;;37689:36;;37646:549;37810:4;:12;;37818:4;37810:12;:28;;;;37826:4;:12;;37834:4;37826:12;37810:28;37806:389;;;37870:4;:12;;37878:4;37870:12;37866:83;;37919:3;;;37866:83;37974:8;38012:127;38024:10;38019:15;;:20;38012:127;;38104:8;38071:3;38104:8;;;;;38071:3;38012:127;;;38171:1;-1:-1:-1;38164:8:127;;-1:-1:-1;;38164:8:127;37520:3590;38262:6;:14;;38272:4;38262:14;38258:2852;;-1:-1:-1;;38307:8:127;38313:2;38307:8;;;;38300:15;;38258:2852;38382:6;:14;;38392:4;38382:14;38378:2732;;38427:42;38445:2;38450:1;38445:6;38455:1;38444:12;38439:2;:17;38431:26;;:3;:26;;;;38461:4;38430:35;38467:1;38427:2;:42::i;:::-;38420:49;;;;;38378:2732;38536:6;:14;;38546:4;38536:14;38532:2578;;38581:45;38599:2;38604:1;38599:6;38609:1;38598:12;38593:2;:17;38585:26;;:3;:26;;;;38615:6;38584:37;38623:2;38581;:45::i;38532:2578::-;38694:6;:14;;38704:4;38694:14;38690:2420;;-1:-1:-1;;38745:21:127;38764:1;38759;38754:6;;38753:12;38745:21;;38802:36;;;38873:5;38868:10;;38745:21;;;;;38867:18;38860:25;;38690:2420;38952:6;:14;;38962:4;38952:14;38948:2162;;38997:3;38990:10;;;;;38948:2162;39068:6;:14;;39078:4;39068:14;39064:2046;;39128:2;39133:1;39128:6;39138:1;39127:12;39122:2;:17;39114:26;;:3;:26;;;;39144:4;39113:35;39106:42;;;;;39064:2046;39217:6;:14;;39227:4;39217:14;39213:1897;;39277:2;39282:1;39277:6;39287:1;39276:12;39271:2;:17;39263:26;;:3;:26;;;;39293:6;39262:37;39255:44;;;;;39213:1897;39368:6;:14;;39378:4;39368:14;39364:1746;;-1:-1:-1;;39419:26:127;39443:1;39438;39433:6;;39432:12;39427:2;:17;39419:26;;39481:41;;;39557:5;39552:10;;39419:26;;;;;39551:18;39544:25;;39364:1746;39637:6;:14;;39647:4;39637:14;39633:1477;;-1:-1:-1;;39694:4:127;39688:34;39720:1;39715;39710:6;;39709:12;39704:2;:17;39688:34;;39778:27;;;39758:48;;;39836:10;;39689:9;;;39688:34;;39835:18;39828:25;;39633:1477;39921:6;:14;;39931:4;39921:14;39917:1193;;-1:-1:-1;;39978:6:127;39972:36;40006:1;40001;39996:6;;39995:12;39990:2;:17;39972:36;;40064:29;;;40044:50;;;40124:10;;39973:11;;;39972:36;;40123:18;40116:25;;39917:1193;40210:6;:14;;40220:4;40210:14;40206:904;;-1:-1:-1;;40261:20:127;40279:1;40274;40269:6;;40268:12;40261:20;;40317:36;;;40389:5;40383:11;;40261:20;;;;;40382:19;40375:26;;40206:904;40469:6;:14;;40479:4;40469:14;40465:645;;40514:2;40507:9;;;;;40465:645;40585:6;:14;;40595:4;40585:14;40581:529;;-1:-1:-1;;40636:25:127;40659:1;40654;40649:6;;40648:12;40643:2;:17;40636:25;;40697:41;;;40774:5;40768:11;;40636:25;;;;;40767:19;40760:26;;40581:529;40853:6;:14;;40863:4;40853:14;40849:261;;40898:3;40891:10;;;;;40849:261;40968:6;:14;;40978:4;40968:14;40964:146;;41013:2;41006:9;;;32450:8733;;;;;;;:::o;19960:782::-;20046:12;20133:18;;:::i;:::-;-1:-1:-1;20201:4:127;20308:2;20296:14;;;;20288:41;;;;;;;3226:2:286;20288:41:127;;;3208:21:286;3265:2;3245:18;;;3238:30;3304:16;3284:18;;;3277:44;3338:18;;20288:41:127;3024:338:286;20288:41:127;20425:14;;;;;;;:30;;;20443:12;20425:30;20421:102;;;20504:4;20475:5;:15;;;20491:9;20475:26;;;;;;;;;:::i;:::-;:33;;;;:26;;;;;;:33;20421:102;20578:12;;;;;20567:23;;;;:8;;;:23;20634:1;20619:16;;;20604:31;;;20712:13;:11;:13::i;5582:7764::-;5646:12;5732:18;;:::i;:::-;-1:-1:-1;5910:15:127;;:18;;;;5800:4;6070:18;;;;6114;;;;6158;;;;;5800:4;;5890:17;;;;6070:18;6114;6248;;;6262:4;6248:18;6244:6792;;6298:2;6327:4;6322:9;;:14;6318:144;;6438:4;6433:9;;6425:4;:18;6419:24;6318:144;6483:2;:7;;6489:1;6483:7;6479:161;;6519:10;;;;;6551:16;;;;;;;;6519:10;-1:-1:-1;6479:161:127;;;6619:2;6614:7;;6479:161;6268:386;6244:6792;;;6756:10;:18;;6770:4;6756:18;6752:6284;;1745:10;6794:14;;6752:6284;;;6892:10;:18;;6906:4;6892:18;6888:6148;;6935:1;6930:6;;6888:6148;;;7060:10;:18;;7074:4;7060:18;7056:5980;;7113:4;7098:12;;;:19;7135:26;;;:14;;;:26;7186:13;:11;:13::i;:::-;7179:20;5582:7764;-1:-1:-1;;;;;;;;;5582:7764:127:o;7056:5980::-;7325:10;:18;;7339:4;7325:18;7321:5715;;7476:14;;;7472:2723;7321:5715;7472:2723;7646:22;;;;;7642:2553;;7771:10;7784:27;7792:2;7797:10;7792:15;7809:1;7784:7;:27::i;:::-;7895:17;;;;7771:40;;-1:-1:-1;7895:17:127;7873:19;8045:14;8064:1;8039:26;8035:146;;1676:4:128;1670:11;;1533:21;1787:15;;;1828:8;1822:4;1815:22;1850:27;;;1996:4;1983:18;;2098:17;;2003:19;1979:44;2025:11;1976:61;8093:65:127;;8035:146;8267:20;;;;;8234:54;;;;;;;;3540:25:286;;;8234:54:127;3601:23:286;;;3581:18;;;3574:51;8203:11:127;;;;8234:19;:6;:19;;;;3513:18:286;;8234:54:127;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;:::i;:::-;8202:86;;;;8515:1;8511:2;8507:10;8612:9;8609:1;8605:17;8694:6;8687:5;8684:17;8681:40;;;8714:5;8704:15;;8681:40;;8797:6;8793:2;8790:14;8787:34;;;8817:2;8807:12;;8787:34;8923:3;8918:1;8910:6;8906:14;8901:3;8897:24;8893:34;8886:41;;9023:3;9019:1;9007:9;8998:6;8995:1;8991:14;8987:30;8983:38;8979:48;8972:55;;9178:1;9174;9170;9158:9;9155:1;9151:17;9147:25;9143:33;9139:41;9305:1;9301;9297;9288:6;9276:9;9273:1;9269:17;9265:30;9261:38;9257:46;9253:54;9235:72;;9436:10;9432:15;9426:4;9422:26;9414:34;;9552:3;9544:4;9540:9;9535:3;9531:19;9528:28;9521:35;;;;9698:33;9707:2;9712:10;9707:15;9724:1;9727:3;9698:8;:33::i;:::-;9753:20;;;:38;;;;;;;;;-1:-1:-1;7642:2553:127;;-1:-1:-1;;;7642:2553:127;;9910:18;;;;;9906:289;;10080:2;10075:7;;7321:5715;;9906:289;10134:10;10129:15;;2053:3;10166:10;;9906:289;7321:5715;;;10324:10;:18;;10338:4;10324:18;10320:2716;;10478:15;;;1824:1;10478:15;;:34;;-1:-1:-1;10497:15:127;;;1859:1;10497:15;10478:34;:57;;;-1:-1:-1;10516:19:127;;;1936:1;10516:19;10478:57;10474:1593;;;10564:2;10559:7;;10320:2716;;10474:1593;10690:23;;;;;10686:1381;;10737:10;10750:27;10758:2;10763:10;10758:15;10775:1;10750:7;:27::i;:::-;10853:17;;;;10737:40;;-1:-1:-1;11096:1:127;11088:10;;11190:1;11186:17;11265:13;;;11262:32;;;11287:5;11281:11;;11262:32;11573:14;;;11379:1;11569:22;;;11565:32;;;;11462:26;11486:1;11371:10;;;11466:18;;;11462:26;11561:43;11367:20;;11669:12;11797:17;;;:23;11865:1;11842:20;;;:24;11375:2;-1:-1:-1;11375:2:127;7321:5715;;10320:2716;12269:10;:18;;12283:4;12269:18;12265:771;;12379:2;:7;;12385:1;12379:7;12375:647;;12472:14;;;;;:40;;-1:-1:-1;12490:22:127;;;1978:1;12490:22;12472:40;:62;;;-1:-1:-1;12516:18:127;;;1897:1;12516:18;12472:62;12468:404;;;12567:1;12562:6;;12375:647;;12468:404;12613:15;;;1824:1;12613:15;;:34;;-1:-1:-1;12632:15:127;;;1859:1;12632:15;12613:34;:61;;;-1:-1:-1;12651:23:127;;;2021:1;12651:23;12613:61;:84;;;-1:-1:-1;12678:19:127;;;1936:1;12678:19;12613:84;12609:263;;;12730:1;12725:6;;7321:5715;;12375:647;12923:10;12918:15;;2087:4;12955:11;;12375:647;13111:15;;;;;:23;;;;:18;;;;:23;;;;13148:15;;:23;;;:18;;;;:23;-1:-1:-1;13237:12:127;;;;13226:23;;;:8;;;:23;13293:1;13278:16;13263:31;;;;;13316:13;:11;:13::i;16084:2480::-;16178:12;16264:18;;:::i;:::-;-1:-1:-1;16332:4:127;16364:10;16472:13;;;16481:4;16472:13;16468:1705;;-1:-1:-1;16511:8:127;;;;16468:1705;;;16630:5;:13;;16639:4;16630:13;16626:1547;;16663:14;;;:8;;;:14;16626:1547;;;16793:5;:13;;16802:4;16793:13;16789:1384;;-1:-1:-1;16832:8:127;;;;16789:1384;;;16951:5;:13;;16960:4;16951:13;16947:1226;;16984:14;;;:8;;;:14;16947:1226;;;17125:5;:13;;17134:4;17125:13;17121:1052;;17252:9;17198:17;17178;;;17198;;;;17178:37;17259:2;17252:9;;;;;17234:8;;;:28;17280:22;:8;;;:22;17121:1052;;;17439:5;:13;;17448:4;17439:13;17435:738;;17506:11;17492;;;17506;;;17492:25;17561:2;17554:9;;;;;17536:8;;;:28;17582:22;:8;;;:22;17435:738;;;17763:5;:13;;17772:4;17763:13;17759:414;;17833:3;17814:23;;17820:3;17814:23;;;;;;;:::i;:::-;;17796:42;;:8;;;:42;17874:23;;;;;;;;;;;;;:::i;:::-;;17856:42;;:8;;;:42;17759:414;;;18067:5;:13;;18076:4;18067:13;18063:110;;18117:3;18111:9;;:3;:9;;;;;;;:::i;:::-;;18100:20;;;;:8;;;:20;18149:9;;;;;;;;;;;:::i;:::-;;18138:20;;:8;;;:20;18063:110;18266:14;;;;18262:85;;18329:3;18300:5;:15;;;18316:9;18300:26;;;;;;;;;:::i;:::-;:32;;;;:26;;;;;;:32;18262:85;18401:12;;;;;18390:23;;;;:8;;;:23;18457:1;18442:16;;;18427:31;;;18534:13;:11;:13::i;:::-;18527:20;16084:2480;-1:-1:-1;;;;;;;16084:2480:127:o;23913:1654::-;24089:14;24106:24;24118:11;24106;:24::i;:::-;24089:41;;24238:1;24231:5;24227:13;24224:33;;;24253:1;24250;24243:12;24224:33;24392:2;24586:15;;;24411:2;24400:14;;24388:10;24384:31;24381:1;24377:39;24542:16;;;24327:20;;24527:10;24516:22;;;24512:27;24502:38;24499:60;25028:5;25025:1;25021:13;25099:1;25084:343;25109:2;25106:1;25103:9;25084:343;;;25232:2;25220:15;;;25169:20;25267:12;;;25281:1;25263:20;25304:42;;;;25372:1;25367:42;;;;25256:153;;25304:42;22391:1;22384:12;;;22424:2;22417:13;;;22469:2;22456:16;;25313:31;;25304:42;;25367;22391:1;22384:12;;;22424:2;22417:13;;;22469:2;22456:16;;25376:31;;25256:153;-1:-1:-1;;25127:1:127;25120:9;25084:343;;;-1:-1:-1;;25526:4:127;25519:18;-1:-1:-1;;;;23913:1654:127:o;20946:586::-;21268:20;;;21292:7;21268:32;21261:3;:40;;;21374:14;;21429:17;;21423:24;;;21415:72;;;;;;;4277:2:286;21415:72:127;;;4259:21:286;4316:2;4296:18;;;4289:30;4355:34;4335:18;;;4328:62;4426:5;4406:18;;;4399:33;4449:19;;21415:72:127;4075:399:286;21415:72:127;21501:14;20946:586;;;:::o;-1:-1:-1:-;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;:::i;:::-;;;;:::o;:::-;;;;;;;;;;;;;;;;;;;;;;;;:::o;467:347:286:-;518:8;528:6;582:3;575:4;567:6;563:17;559:27;549:55;;600:1;597;590:12;549:55;-1:-1:-1;623:20:286;;666:18;655:30;;652:50;;;698:1;695;688:12;652:50;735:4;727:6;723:17;711:29;;787:3;780:4;771:6;763;759:19;755:30;752:39;749:59;;;804:1;801;794:12;749:59;467:347;;;;;:::o;819:785::-;918:6;926;934;942;950;1003:2;991:9;982:7;978:23;974:32;971:52;;;1019:1;1016;1009:12;971:52;1059:9;1046:23;1088:18;1129:2;1121:6;1118:14;1115:34;;;1145:1;1142;1135:12;1115:34;1184:58;1234:7;1225:6;1214:9;1210:22;1184:58;:::i;:::-;1261:8;;-1:-1:-1;1158:84:286;-1:-1:-1;1349:2:286;1334:18;;1321:32;;-1:-1:-1;1365:16:286;;;1362:36;;;1394:1;1391;1384:12;1362:36;;1433:60;1485:7;1474:8;1463:9;1459:24;1433:60;:::i;:::-;819:785;;;;-1:-1:-1;1512:8:286;1594:2;1579:18;1566:32;;819:785;-1:-1:-1;;;;819:785:286:o;1791:184::-;1843:77;1840:1;1833:88;1940:4;1937:1;1930:15;1964:4;1961:1;1954:15;3636:245;3715:6;3723;3776:2;3764:9;3755:7;3751:23;3747:32;3744:52;;;3792:1;3789;3782:12;3744:52;-1:-1:-1;;3815:16:286;;3871:2;3856:18;;;3850:25;3815:16;;3850:25;;-1:-1:-1;3636:245:286:o;3886:184::-;3938:77;3935:1;3928:88;4035:4;4032:1;4025:15;4059:4;4056:1;4049:15" func init() { if err := json.Unmarshal([]byte(MIPSStorageLayoutJSON), MIPSStorageLayout); err != nil { diff --git a/op-bindings/bindings/preimageoracle_more.go b/op-bindings/bindings/preimageoracle_more.go index 2c53d11a0d7f..54abd4665e9b 100644 --- a/op-bindings/bindings/preimageoracle_more.go +++ b/op-bindings/bindings/preimageoracle_more.go @@ -15,7 +15,7 @@ var PreimageOracleStorageLayout = new(solc.StorageLayout) var PreimageOracleDeployedBin = "0x608060405234801561001057600080fd5b50600436106100725760003560e01c8063e03110e111610050578063e03110e114610106578063e15926111461012e578063fef2b4ed1461014357600080fd5b806361238bde146100775780638542cf50146100b5578063c0c220c9146100f3575b600080fd5b6100a26100853660046104df565b600160209081526000928352604080842090915290825290205481565b6040519081526020015b60405180910390f35b6100e36100c33660046104df565b600260209081526000928352604080842090915290825290205460ff1681565b60405190151581526020016100ac565b6100a2610101366004610501565b610163565b6101196101143660046104df565b610238565b604080519283526020830191909152016100ac565b61014161013c36600461053c565b610329565b005b6100a26101513660046105b8565b60006020819052908152604090205481565b600061016f8686610432565b905061017c836008610600565b8211806101895750602083115b156101c0576040517ffe25498700000000000000000000000000000000000000000000000000000000815260040160405180910390fd5b6000602081815260c085901b82526008959095528251828252600286526040808320858452875280832080547fffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff001660019081179091558484528752808320948352938652838220558181529384905292205592915050565b6000828152600260209081526040808320848452909152812054819060ff166102c1576040517f08c379a000000000000000000000000000000000000000000000000000000000815260206004820152601460248201527f7072652d696d616765206d757374206578697374000000000000000000000000604482015260640160405180910390fd5b50600083815260208181526040909120546102dd816008610600565b6102e8856020610600565b1061030657836102f9826008610600565b6103039190610618565b91505b506000938452600160209081526040808620948652939052919092205492909150565b604435600080600883018611156103485763fe2549876000526004601cfd5b60c083901b6080526088838682378087017ffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff80151908490207effffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff167f02000000000000000000000000000000000000000000000000000000000000001760008181526002602090815260408083208b8452825280832080547fffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff0016600190811790915584845282528083209a83529981528982209390935590815290819052959095209190915550505050565b7f01000000000000000000000000000000000000000000000000000000000000007effffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff8316176104d8818360408051600093845233602052918152606090922091527effffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff167f01000000000000000000000000000000000000000000000000000000000000001790565b9392505050565b600080604083850312156104f257600080fd5b50508035926020909101359150565b600080600080600060a0868803121561051957600080fd5b505083359560208501359550604085013594606081013594506080013592509050565b60008060006040848603121561055157600080fd5b83359250602084013567ffffffffffffffff8082111561057057600080fd5b818601915086601f83011261058457600080fd5b81358181111561059357600080fd5b8760208285010111156105a557600080fd5b6020830194508093505050509250925092565b6000602082840312156105ca57600080fd5b5035919050565b7f4e487b7100000000000000000000000000000000000000000000000000000000600052601160045260246000fd5b60008219821115610613576106136105d1565b500190565b60008282101561062a5761062a6105d1565b50039056fea164736f6c634300080f000a" -var PreimageOracleDeployedSourceMap = "306:3911:130:-:0;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;537:68;;;;;;:::i;:::-;;;;;;;;;;;;;;;;;;;;;;;;;;;;413:25:287;;;401:2;386:18;537:68:130;;;;;;;;680:66;;;;;;:::i;:::-;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;614:14:287;;607:22;589:41;;577:2;562:18;680:66:130;449:187:287;1367:1211:130;;;;;;:::i;:::-;;:::i;789:536::-;;;;;;:::i;:::-;;:::i;:::-;;;;1274:25:287;;;1330:2;1315:18;;1308:34;;;;1247:18;789:536:130;1100:248:287;2620:1595:130;;;;;;:::i;:::-;;:::i;:::-;;419:50;;;;;;:::i;:::-;;;;;;;;;;;;;;;1367:1211;1560:12;1665:51;1694:6;1702:13;1665:28;:51::i;:::-;1658:58;-1:-1:-1;1810:9:130;:5;1818:1;1810:9;:::i;:::-;1796:11;:23;:37;;;;1831:2;1823:5;:10;1796:37;1792:90;;;1856:15;;;;;;;;;;;;;;1792:90;1951:12;2051:4;2044:18;;;2152:3;2148:15;;;2135:29;;2184:4;2177:19;;;;2286:18;;2376:20;;;:14;:20;;;;;;:33;;;;;;;;:40;;;;2412:4;2376:40;;;;;;2426:19;;;;;;;;:32;;;;;;;;;:39;2542:21;;;;;;;;;:29;2391:4;1367:1211;-1:-1:-1;;1367:1211:130:o;789:536::-;865:12;914:20;;;:14;:20;;;;;;;;:29;;;;;;;;;865:12;;914:29;;906:62;;;;;;;2908:2:287;906:62:130;;;2890:21:287;2947:2;2927:18;;;2920:30;2986:22;2966:18;;;2959:50;3026:18;;906:62:130;;;;;;;;-1:-1:-1;1099:14:130;1116:21;;;1087:2;1116:21;;;;;;;;1167:10;1116:21;1176:1;1167:10;:::i;:::-;1151:12;:7;1161:2;1151:12;:::i;:::-;:26;1147:87;;1216:7;1203:10;:6;1212:1;1203:10;:::i;:::-;:20;;;;:::i;:::-;1193:30;;1147:87;-1:-1:-1;1290:19:130;;;;:13;:19;;;;;;;;:28;;;;;;;;;;;;789:536;;-1:-1:-1;789:536:130:o;2620:1595::-;2916:4;2903:18;2721:12;;3045:1;3035:12;;3019:29;;3016:210;;;3120:10;3117:1;3110:21;3210:1;3204:4;3197:15;3016:210;3469:3;3465:14;;;3369:4;3453:27;3500:11;3474:4;3619:16;3500:11;3601:41;3832:29;;;3836:11;3832:29;3826:36;3884:20;;;;4031:19;4024:27;4053:11;4021:44;4084:19;;;;4062:1;4084:19;;;;;;;;:32;;;;;;;;:39;;;;4119:4;4084:39;;;;;;4133:18;;;;;;;;:31;;;;;;;;;:38;;;;4181:20;;;;;;;;;;;:27;;;;-1:-1:-1;;;;2620:1595:130:o;552:449:129:-;835:11;860:19;848:32;;832:49;965:29;832:49;980:13;1676:4;1670:11;;1533:21;1787:15;;;1828:8;1822:4;1815:22;1850:27;;;1996:4;1983:18;;;2098:17;;2003:19;1979:44;2025:11;1976:61;;1455:676;965:29;958:36;552:449;-1:-1:-1;;;552:449:129:o;14:248:287:-;82:6;90;143:2;131:9;122:7;118:23;114:32;111:52;;;159:1;156;149:12;111:52;-1:-1:-1;;182:23:287;;;252:2;237:18;;;224:32;;-1:-1:-1;14:248:287:o;641:454::-;736:6;744;752;760;768;821:3;809:9;800:7;796:23;792:33;789:53;;;838:1;835;828:12;789:53;-1:-1:-1;;861:23:287;;;931:2;916:18;;903:32;;-1:-1:-1;982:2:287;967:18;;954:32;;1033:2;1018:18;;1005:32;;-1:-1:-1;1084:3:287;1069:19;1056:33;;-1:-1:-1;641:454:287;-1:-1:-1;641:454:287:o;1353:659::-;1432:6;1440;1448;1501:2;1489:9;1480:7;1476:23;1472:32;1469:52;;;1517:1;1514;1507:12;1469:52;1553:9;1540:23;1530:33;;1614:2;1603:9;1599:18;1586:32;1637:18;1678:2;1670:6;1667:14;1664:34;;;1694:1;1691;1684:12;1664:34;1732:6;1721:9;1717:22;1707:32;;1777:7;1770:4;1766:2;1762:13;1758:27;1748:55;;1799:1;1796;1789:12;1748:55;1839:2;1826:16;1865:2;1857:6;1854:14;1851:34;;;1881:1;1878;1871:12;1851:34;1926:7;1921:2;1912:6;1908:2;1904:15;1900:24;1897:37;1894:57;;;1947:1;1944;1937:12;1894:57;1978:2;1974;1970:11;1960:21;;2000:6;1990:16;;;;;1353:659;;;;;:::o;2017:180::-;2076:6;2129:2;2117:9;2108:7;2104:23;2100:32;2097:52;;;2145:1;2142;2135:12;2097:52;-1:-1:-1;2168:23:287;;2017:180;-1:-1:-1;2017:180:287:o;2384:184::-;2436:77;2433:1;2426:88;2533:4;2530:1;2523:15;2557:4;2554:1;2547:15;2573:128;2613:3;2644:1;2640:6;2637:1;2634:13;2631:39;;;2650:18;;:::i;:::-;-1:-1:-1;2686:9:287;;2573:128::o;3055:125::-;3095:4;3123:1;3120;3117:8;3114:34;;;3128:18;;:::i;:::-;-1:-1:-1;3165:9:287;;3055:125::o" +var PreimageOracleDeployedSourceMap = "306:3911:129:-:0;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;537:68;;;;;;:::i;:::-;;;;;;;;;;;;;;;;;;;;;;;;;;;;413:25:286;;;401:2;386:18;537:68:129;;;;;;;;680:66;;;;;;:::i;:::-;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;614:14:286;;607:22;589:41;;577:2;562:18;680:66:129;449:187:286;1367:1211:129;;;;;;:::i;:::-;;:::i;789:536::-;;;;;;:::i;:::-;;:::i;:::-;;;;1274:25:286;;;1330:2;1315:18;;1308:34;;;;1247:18;789:536:129;1100:248:286;2620:1595:129;;;;;;:::i;:::-;;:::i;:::-;;419:50;;;;;;:::i;:::-;;;;;;;;;;;;;;;1367:1211;1560:12;1665:51;1694:6;1702:13;1665:28;:51::i;:::-;1658:58;-1:-1:-1;1810:9:129;:5;1818:1;1810:9;:::i;:::-;1796:11;:23;:37;;;;1831:2;1823:5;:10;1796:37;1792:90;;;1856:15;;;;;;;;;;;;;;1792:90;1951:12;2051:4;2044:18;;;2152:3;2148:15;;;2135:29;;2184:4;2177:19;;;;2286:18;;2376:20;;;:14;:20;;;;;;:33;;;;;;;;:40;;;;2412:4;2376:40;;;;;;2426:19;;;;;;;;:32;;;;;;;;;:39;2542:21;;;;;;;;;:29;2391:4;1367:1211;-1:-1:-1;;1367:1211:129:o;789:536::-;865:12;914:20;;;:14;:20;;;;;;;;:29;;;;;;;;;865:12;;914:29;;906:62;;;;;;;2908:2:286;906:62:129;;;2890:21:286;2947:2;2927:18;;;2920:30;2986:22;2966:18;;;2959:50;3026:18;;906:62:129;;;;;;;;-1:-1:-1;1099:14:129;1116:21;;;1087:2;1116:21;;;;;;;;1167:10;1116:21;1176:1;1167:10;:::i;:::-;1151:12;:7;1161:2;1151:12;:::i;:::-;:26;1147:87;;1216:7;1203:10;:6;1212:1;1203:10;:::i;:::-;:20;;;;:::i;:::-;1193:30;;1147:87;-1:-1:-1;1290:19:129;;;;:13;:19;;;;;;;;:28;;;;;;;;;;;;789:536;;-1:-1:-1;789:536:129:o;2620:1595::-;2916:4;2903:18;2721:12;;3045:1;3035:12;;3019:29;;3016:210;;;3120:10;3117:1;3110:21;3210:1;3204:4;3197:15;3016:210;3469:3;3465:14;;;3369:4;3453:27;3500:11;3474:4;3619:16;3500:11;3601:41;3832:29;;;3836:11;3832:29;3826:36;3884:20;;;;4031:19;4024:27;4053:11;4021:44;4084:19;;;;4062:1;4084:19;;;;;;;;:32;;;;;;;;:39;;;;4119:4;4084:39;;;;;;4133:18;;;;;;;;:31;;;;;;;;;:38;;;;4181:20;;;;;;;;;;;:27;;;;-1:-1:-1;;;;2620:1595:129:o;552:449:128:-;835:11;860:19;848:32;;832:49;965:29;832:49;980:13;1676:4;1670:11;;1533:21;1787:15;;;1828:8;1822:4;1815:22;1850:27;;;1996:4;1983:18;;;2098:17;;2003:19;1979:44;2025:11;1976:61;;1455:676;965:29;958:36;552:449;-1:-1:-1;;;552:449:128:o;14:248:286:-;82:6;90;143:2;131:9;122:7;118:23;114:32;111:52;;;159:1;156;149:12;111:52;-1:-1:-1;;182:23:286;;;252:2;237:18;;;224:32;;-1:-1:-1;14:248:286:o;641:454::-;736:6;744;752;760;768;821:3;809:9;800:7;796:23;792:33;789:53;;;838:1;835;828:12;789:53;-1:-1:-1;;861:23:286;;;931:2;916:18;;903:32;;-1:-1:-1;982:2:286;967:18;;954:32;;1033:2;1018:18;;1005:32;;-1:-1:-1;1084:3:286;1069:19;1056:33;;-1:-1:-1;641:454:286;-1:-1:-1;641:454:286:o;1353:659::-;1432:6;1440;1448;1501:2;1489:9;1480:7;1476:23;1472:32;1469:52;;;1517:1;1514;1507:12;1469:52;1553:9;1540:23;1530:33;;1614:2;1603:9;1599:18;1586:32;1637:18;1678:2;1670:6;1667:14;1664:34;;;1694:1;1691;1684:12;1664:34;1732:6;1721:9;1717:22;1707:32;;1777:7;1770:4;1766:2;1762:13;1758:27;1748:55;;1799:1;1796;1789:12;1748:55;1839:2;1826:16;1865:2;1857:6;1854:14;1851:34;;;1881:1;1878;1871:12;1851:34;1926:7;1921:2;1912:6;1908:2;1904:15;1900:24;1897:37;1894:57;;;1947:1;1944;1937:12;1894:57;1978:2;1974;1970:11;1960:21;;2000:6;1990:16;;;;;1353:659;;;;;:::o;2017:180::-;2076:6;2129:2;2117:9;2108:7;2104:23;2100:32;2097:52;;;2145:1;2142;2135:12;2097:52;-1:-1:-1;2168:23:286;;2017:180;-1:-1:-1;2017:180:286:o;2384:184::-;2436:77;2433:1;2426:88;2533:4;2530:1;2523:15;2557:4;2554:1;2547:15;2573:128;2613:3;2644:1;2640:6;2637:1;2634:13;2631:39;;;2650:18;;:::i;:::-;-1:-1:-1;2686:9:286;;2573:128::o;3055:125::-;3095:4;3123:1;3120;3117:8;3114:34;;;3128:18;;:::i;:::-;-1:-1:-1;3165:9:286;;3055:125::o" func init() { if err := json.Unmarshal([]byte(PreimageOracleStorageLayoutJSON), PreimageOracleStorageLayout); err != nil { From 798f054993dd4e2dde968e3d48dd1fe1f477bb20 Mon Sep 17 00:00:00 2001 From: Mark Tyneway Date: Wed, 25 Oct 2023 19:23:16 -0600 Subject: [PATCH 100/279] contracts-bedrock: delete dead delete output code We have migrated to using `superchain-ops` for any sort of safe interactions. This code will likely need to be ported over to the `superchain-ops` repo in the future. It is not necessary for now, so delete it. --- .../scripts/outputs/DeleteOutput.s.sol | 152 ------------------ .../scripts/outputs/README.md | 38 ----- 2 files changed, 190 deletions(-) delete mode 100644 packages/contracts-bedrock/scripts/outputs/DeleteOutput.s.sol delete mode 100644 packages/contracts-bedrock/scripts/outputs/README.md diff --git a/packages/contracts-bedrock/scripts/outputs/DeleteOutput.s.sol b/packages/contracts-bedrock/scripts/outputs/DeleteOutput.s.sol deleted file mode 100644 index 2e79e88000ac..000000000000 --- a/packages/contracts-bedrock/scripts/outputs/DeleteOutput.s.sol +++ /dev/null @@ -1,152 +0,0 @@ -// SPDX-License-Identifier: MIT -pragma solidity 0.8.15; - -import { console } from "forge-std/console.sol"; -import { IMulticall3 } from "forge-std/interfaces/IMulticall3.sol"; - -import { LibSort } from "../libraries/LibSort.sol"; -import { IGnosisSafe, Enum } from "../interfaces/IGnosisSafe.sol"; -import { SafeBuilder } from "../universal/SafeBuilder.sol"; - -import { Types } from "../../src/libraries/Types.sol"; -import { FeeVault } from "../../src/universal/FeeVault.sol"; -import { L2OutputOracle } from "../../src/L1/L2OutputOracle.sol"; -import { Predeploys } from "../../src/libraries/Predeploys.sol"; - -/// @title DeleteOutput -/// @notice Deletes an output root from the L2OutputOracle. -/// @notice Example usage is provided in the README documentation. -contract DeleteOutput is SafeBuilder { - /// @notice A set of contract addresses for the script. - struct ContractSet { - address Safe; - address ProxyAdmin; - address L2OutputOracleProxy; - } - - /// @notice A mapping of chainid to a ContractSet. - mapping(uint256 => ContractSet) internal _contracts; - - /// @notice The l2 output index we will delete. - uint256 internal index; - - /// @notice The address of the L2OutputOracle to target. - address internal oracle; - - /// @notice Place the contract addresses in storage for ux. - function setUp() external { - _contracts[GOERLI] = ContractSet({ - Safe: 0xBc1233d0C3e6B5d53Ab455cF65A6623F6dCd7e4f, - ProxyAdmin: 0x01d3670863c3F4b24D7b107900f0b75d4BbC6e0d, - L2OutputOracleProxy: 0xE6Dfba0953616Bacab0c9A8ecb3a9BBa77FC15c0 - }); - } - - /// @notice Returns the ContractSet for the defined block chainid. - /// @dev Reverts if no ContractSet is defined. - function contracts() public view returns (ContractSet memory) { - ContractSet memory cs = _contracts[block.chainid]; - if (cs.Safe == address(0) || cs.ProxyAdmin == address(0) || cs.L2OutputOracleProxy == address(0)) { - revert("Missing Contract Set for the given block.chainid"); - } - return cs; - } - - /// @notice Executes the gnosis safe transaction to delete an L2 Output Root. - function run(uint256 _index) external returns (bool) { - address _safe = contracts().Safe; - address _proxyAdmin = contracts().ProxyAdmin; - index = _index; - return run(_safe, _proxyAdmin); - } - - /// @notice Follow up assertions to ensure that the script ran to completion. - function _postCheck() internal view override { - L2OutputOracle l2oo = L2OutputOracle(contracts().L2OutputOracleProxy); - Types.OutputProposal memory proposal = l2oo.getL2Output(index); - require(proposal.l2BlockNumber == 0, "DeleteOutput: Output deletion failed."); - } - - /// @notice Test coverage of the script. - function test_script_succeeds() external skipWhenNotForking { - uint256 _index = getLatestIndex(); - require(_index != 0, "DeleteOutput: No outputs to delete."); - - index = _index; - - address safe = contracts().Safe; - require(safe != address(0), "DeleteOutput: Invalid safe address."); - - address proxyAdmin = contracts().ProxyAdmin; - require(proxyAdmin != address(0), "DeleteOutput: Invalid proxy admin address."); - - address[] memory owners = IGnosisSafe(payable(safe)).getOwners(); - - for (uint256 i; i < owners.length; i++) { - address owner = owners[i]; - vm.startBroadcast(owner); - bool success = _run(safe, proxyAdmin); - vm.stopBroadcast(); - - if (success) { - console.log("tx success"); - break; - } - } - - _postCheck(); - } - - function buildCalldata(address) internal view override returns (bytes memory) { - IMulticall3.Call3[] memory calls = new IMulticall3.Call3[](1); - - calls[0] = IMulticall3.Call3({ - target: oracle, - allowFailure: false, - callData: abi.encodeCall(L2OutputOracle.deleteL2Outputs, (index)) - }); - - return abi.encodeCall(IMulticall3.aggregate3, (calls)); - } - - /// @notice Computes the safe transaction hash. - function computeSafeTransactionHash(uint256 _index) public returns (bytes32) { - ContractSet memory cs = contracts(); - address _safe = cs.Safe; - address _proxyAdmin = cs.ProxyAdmin; - index = _index; - oracle = cs.L2OutputOracleProxy; - - return _getTransactionHash(_safe, _proxyAdmin); - } - - /// @notice Returns the challenger for the L2OutputOracle. - function getChallenger() public view returns (address) { - L2OutputOracle l2oo = L2OutputOracle(contracts().L2OutputOracleProxy); - return l2oo.CHALLENGER(); - } - - /// @notice Returns the L2 Block Number for the given index. - function getL2BlockNumber(uint256 _index) public view returns (uint256) { - L2OutputOracle l2oo = L2OutputOracle(contracts().L2OutputOracleProxy); - return l2oo.getL2Output(_index).l2BlockNumber; - } - - /// @notice Returns the output root for the given index. - function getOutputFromIndex(uint256 _index) public view returns (bytes32) { - L2OutputOracle l2oo = L2OutputOracle(contracts().L2OutputOracleProxy); - return l2oo.getL2Output(_index).outputRoot; - } - - /// @notice Returns the output root with the corresponding to the L2 Block Number. - function getOutputFromL2BlockNumber(uint256 l2BlockNumber) public view returns (bytes32) { - L2OutputOracle l2oo = L2OutputOracle(contracts().L2OutputOracleProxy); - return l2oo.getL2OutputAfter(l2BlockNumber).outputRoot; - } - - /// @notice Returns the latest l2 output index. - function getLatestIndex() public view returns (uint256) { - L2OutputOracle l2oo = L2OutputOracle(contracts().L2OutputOracleProxy); - return l2oo.latestOutputIndex(); - } -} diff --git a/packages/contracts-bedrock/scripts/outputs/README.md b/packages/contracts-bedrock/scripts/outputs/README.md deleted file mode 100644 index 24621bc43921..000000000000 --- a/packages/contracts-bedrock/scripts/outputs/README.md +++ /dev/null @@ -1,38 +0,0 @@ -## L2 Output Oracle Scripts - -A collection of scripts to interact with the L2OutputOracle. - -### Output Deletion - -[DeleteOutput](./DeleteOutput.s.sol) contains a variety of functions that deal -with deleting an output root from the [L2OutputOracle](../../contracts/L1/L2OutputOracle.sol). - -To delete an output root, the script can be run as follows, where `` is -the index of the posted output to delete. - -```bash -$ forge script scripts/output/DeleteOutput.s.sol \ - --sig "run(uint256)" \ - --rpc-url $ETH_RPC_URL \ - --broadcast \ - --private-key $PRIVATE_KEY \ - -``` - -To find and confirm the output index, there are a variety of helper functions that -can be run using the script `--sig` flag, passing the function signatures in as arguments. -These are outlined below. - -### Retrieving an L2 Block Number - -The output's associated L2 block number can be retrieved using the following command, where -`` is the index of the output in the [L2OutputOracle](../../contracts/L1/L2OutputOracle.sol). - -```bash -$ forge script scripts/output/DeleteOutput.s.sol \ - --sig "getL2BlockNumber(uint256)" \ - --rpc-url $ETH_RPC_URL \ - -``` - - From bd23c0aa618f21b15debc4aaab2dc11a21cd0a16 Mon Sep 17 00:00:00 2001 From: Mark Tyneway Date: Wed, 25 Oct 2023 20:20:34 -0600 Subject: [PATCH 101/279] op-bindings: regenerate --- op-bindings/bindings/mips_more.go | 2 +- op-bindings/bindings/preimageoracle_more.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/op-bindings/bindings/mips_more.go b/op-bindings/bindings/mips_more.go index d6c26b36bba1..e7983bf62efd 100644 --- a/op-bindings/bindings/mips_more.go +++ b/op-bindings/bindings/mips_more.go @@ -15,7 +15,7 @@ var MIPSStorageLayout = new(solc.StorageLayout) var MIPSDeployedBin = "0x608060405234801561001057600080fd5b50600436106100415760003560e01c8063155633fe146100465780637dc0d1d01461006b578063836e7b32146100af575b600080fd5b610051634000000081565b60405163ffffffff90911681526020015b60405180910390f35b60405173ffffffffffffffffffffffffffffffffffffffff7f0000000000000000000000000000000000000000000000000000000000000000168152602001610062565b6100c26100bd366004611d2e565b6100d0565b604051908152602001610062565b60006100da611c5b565b608081146100e757600080fd5b604051610600146100f757600080fd5b6084871461010457600080fd5b6101a4851461011257600080fd5b8635608052602087013560a052604087013560e090811c60c09081526044890135821c82526048890135821c61010052604c890135821c610120526050890135821c61014052605489013590911c61016052605888013560f890811c610180526059890135901c6101a052605a880135901c6101c0526102006101e0819052606288019060005b60208110156101bd57823560e01c8252600490920191602090910190600101610199565b505050806101200151156101db576101d361061b565b915050610612565b6101408101805160010167ffffffffffffffff16905260608101516000906102039082610737565b9050603f601a82901c16600281148061022257508063ffffffff166003145b156102775760006002836303ffffff1663ffffffff16901b846080015163f00000001617905061026c8263ffffffff1660021461026057601f610263565b60005b60ff16826107f3565b945050505050610612565b6101608301516000908190601f601086901c81169190601587901c16602081106102a3576102a3611da2565b602002015192508063ffffffff851615806102c457508463ffffffff16601c145b156102fb578661016001518263ffffffff16602081106102e6576102e6611da2565b6020020151925050601f600b86901c166103b7565b60208563ffffffff16101561035d578463ffffffff16600c148061032557508463ffffffff16600d145b8061033657508463ffffffff16600e145b15610347578561ffff1692506103b7565b6103568661ffff1660106108e4565b92506103b7565b60288563ffffffff1610158061037957508463ffffffff166022145b8061038a57508463ffffffff166026145b156103b7578661016001518263ffffffff16602081106103ac576103ac611da2565b602002015192508190505b60048563ffffffff16101580156103d4575060088563ffffffff16105b806103e557508463ffffffff166001145b15610404576103f685878487610957565b975050505050505050610612565b63ffffffff6000602087831610610469576104248861ffff1660106108e4565b9095019463fffffffc861661043a816001610737565b915060288863ffffffff161015801561045a57508763ffffffff16603014155b1561046757809250600093505b505b600061047789888885610b67565b63ffffffff9081169150603f8a1690891615801561049c575060088163ffffffff1610155b80156104ae5750601c8163ffffffff16105b1561058b578063ffffffff16600814806104ce57508063ffffffff166009145b15610505576104f38163ffffffff166008146104ea57856104ed565b60005b896107f3565b9b505050505050505050505050610612565b8063ffffffff16600a03610525576104f3858963ffffffff8a16156112f7565b8063ffffffff16600b03610546576104f3858963ffffffff8a1615156112f7565b8063ffffffff16600c0361055d576104f38d6113dd565b60108163ffffffff161015801561057a5750601c8163ffffffff16105b1561058b576104f381898988611914565b8863ffffffff1660381480156105a6575063ffffffff861615155b156105db5760018b61016001518763ffffffff16602081106105ca576105ca611da2565b63ffffffff90921660209290920201525b8363ffffffff1663ffffffff146105f8576105f884600184611b0e565b610604858360016112f7565b9b5050505050505050505050505b95945050505050565b60408051608051815260a051602082015260dc519181019190915260fc51604482015261011c51604882015261013c51604c82015261015c51605082015261017c5160548201526101805161019f5160588301526101a0516101bf5160598401526101d851605a840152600092610200929091606283019190855b60208110156106ba57601c8601518452602090950194600490930192600101610696565b506000835283830384a06000945080600181146106da5760039550610702565b8280156106f257600181146106fb5760029650610700565b60009650610700565b600196505b505b50505081900390207effffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff1660f89190911b17919050565b60008061074383611bb2565b9050600384161561075357600080fd5b6020810190358460051c8160005b601b8110156107b95760208501943583821c6001168015610789576001811461079e576107af565b600084815260208390526040902093506107af565b600082815260208590526040902093505b5050600101610761565b5060805191508181146107d457630badf00d60005260206000fd5b5050601f94909416601c0360031b9390931c63ffffffff169392505050565b60006107fd611c5b565b60809050806060015160040163ffffffff16816080015163ffffffff1614610886576040517f08c379a000000000000000000000000000000000000000000000000000000000815260206004820152601260248201527f6a756d7020696e2064656c617920736c6f74000000000000000000000000000060448201526064015b60405180910390fd5b60608101805160808301805163ffffffff9081169093528583169052908516156108dc57806008018261016001518663ffffffff16602081106108cb576108cb611da2565b63ffffffff90921660209290920201525b61061261061b565b600063ffffffff8381167fffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff80850183169190911c821615159160016020869003821681901b830191861691821b92911b0182610941576000610943565b815b90861663ffffffff16179250505092915050565b6000610961611c5b565b608090506000816060015160040163ffffffff16826080015163ffffffff16146109e7576040517f08c379a000000000000000000000000000000000000000000000000000000000815260206004820152601460248201527f6272616e636820696e2064656c617920736c6f74000000000000000000000000604482015260640161087d565b8663ffffffff1660041480610a0257508663ffffffff166005145b15610a7e5760008261016001518663ffffffff1660208110610a2657610a26611da2565b602002015190508063ffffffff168563ffffffff16148015610a4e57508763ffffffff166004145b80610a7657508063ffffffff168563ffffffff1614158015610a7657508763ffffffff166005145b915050610afb565b8663ffffffff16600603610a9b5760008460030b13159050610afb565b8663ffffffff16600703610ab75760008460030b139050610afb565b8663ffffffff16600103610afb57601f601087901c166000819003610ae05760008560030b1291505b8063ffffffff16600103610af95760008560030b121591505b505b606082018051608084015163ffffffff169091528115610b41576002610b268861ffff1660106108e4565b63ffffffff90811690911b8201600401166080840152610b53565b60808301805160040163ffffffff1690525b610b5b61061b565b98975050505050505050565b6000603f601a86901c16801580610b96575060088163ffffffff1610158015610b965750600f8163ffffffff16105b15610fec57603f86168160088114610bdd5760098114610be657600a8114610bef57600b8114610bf857600c8114610c0157600d8114610c0a57600e8114610c1357610c18565b60209150610c18565b60219150610c18565b602a9150610c18565b602b9150610c18565b60249150610c18565b60259150610c18565b602691505b508063ffffffff16600003610c3f5750505063ffffffff8216601f600686901c161b6112ef565b8063ffffffff16600203610c655750505063ffffffff8216601f600686901c161c6112ef565b8063ffffffff16600303610c9b57601f600688901c16610c9163ffffffff8716821c60208390036108e4565b93505050506112ef565b8063ffffffff16600403610cbd5750505063ffffffff8216601f84161b6112ef565b8063ffffffff16600603610cdf5750505063ffffffff8216601f84161c6112ef565b8063ffffffff16600703610d1257610d098663ffffffff168663ffffffff16901c876020036108e4565b925050506112ef565b8063ffffffff16600803610d2a5785925050506112ef565b8063ffffffff16600903610d425785925050506112ef565b8063ffffffff16600a03610d5a5785925050506112ef565b8063ffffffff16600b03610d725785925050506112ef565b8063ffffffff16600c03610d8a5785925050506112ef565b8063ffffffff16600f03610da25785925050506112ef565b8063ffffffff16601003610dba5785925050506112ef565b8063ffffffff16601103610dd25785925050506112ef565b8063ffffffff16601203610dea5785925050506112ef565b8063ffffffff16601303610e025785925050506112ef565b8063ffffffff16601803610e1a5785925050506112ef565b8063ffffffff16601903610e325785925050506112ef565b8063ffffffff16601a03610e4a5785925050506112ef565b8063ffffffff16601b03610e625785925050506112ef565b8063ffffffff16602003610e7b575050508282016112ef565b8063ffffffff16602103610e94575050508282016112ef565b8063ffffffff16602203610ead575050508183036112ef565b8063ffffffff16602303610ec6575050508183036112ef565b8063ffffffff16602403610edf575050508282166112ef565b8063ffffffff16602503610ef8575050508282176112ef565b8063ffffffff16602603610f11575050508282186112ef565b8063ffffffff16602703610f2b57505050828217196112ef565b8063ffffffff16602a03610f5c578460030b8660030b12610f4d576000610f50565b60015b60ff16925050506112ef565b8063ffffffff16602b03610f84578463ffffffff168663ffffffff1610610f4d576000610f50565b6040517f08c379a000000000000000000000000000000000000000000000000000000000815260206004820152601360248201527f696e76616c696420696e737472756374696f6e00000000000000000000000000604482015260640161087d565b50610f84565b8063ffffffff16601c0361107057603f86166002819003611012575050508282026112ef565b8063ffffffff166020148061102d57508063ffffffff166021145b15610fe6578063ffffffff16602003611044579419945b60005b6380000000871615611066576401fffffffe600197881b169601611047565b92506112ef915050565b8063ffffffff16600f0361109257505065ffffffff0000601083901b166112ef565b8063ffffffff166020036110ce576110c68560031660080260180363ffffffff168463ffffffff16901c60ff1660086108e4565b9150506112ef565b8063ffffffff16602103611103576110c68560021660080260100363ffffffff168463ffffffff16901c61ffff1660106108e4565b8063ffffffff1660220361113257505063ffffffff60086003851602811681811b198416918316901b176112ef565b8063ffffffff1660230361114957829150506112ef565b8063ffffffff1660240361117b578460031660080260180363ffffffff168363ffffffff16901c60ff169150506112ef565b8063ffffffff166025036111ae578460021660080260100363ffffffff168363ffffffff16901c61ffff169150506112ef565b8063ffffffff166026036111e057505063ffffffff60086003851602601803811681811c198416918316901c176112ef565b8063ffffffff1660280361121657505060ff63ffffffff60086003861602601803811682811b9091188316918416901b176112ef565b8063ffffffff1660290361124d57505061ffff63ffffffff60086002861602601003811682811b9091188316918416901b176112ef565b8063ffffffff16602a0361127c57505063ffffffff60086003851602811681811c198316918416901c176112ef565b8063ffffffff16602b0361129357839150506112ef565b8063ffffffff16602e036112c557505063ffffffff60086003851602601803811681811b198316918416901b176112ef565b8063ffffffff166030036112dc57829150506112ef565b8063ffffffff16603803610f8457839150505b949350505050565b6000611301611c5b565b506080602063ffffffff861610611374576040517f08c379a000000000000000000000000000000000000000000000000000000000815260206004820152600e60248201527f76616c6964207265676973746572000000000000000000000000000000000000604482015260640161087d565b63ffffffff8516158015906113865750825b156113ba57838161016001518663ffffffff16602081106113a9576113a9611da2565b63ffffffff90921660209290920201525b60808101805163ffffffff8082166060850152600490910116905261061261061b565b60006113e7611c5b565b506101e051604081015160808083015160a084015160c09094015191936000928392919063ffffffff8616610ffa036114615781610fff81161561143057610fff811661100003015b8363ffffffff166000036114575760e08801805163ffffffff83820116909152955061145b565b8395505b506118d3565b8563ffffffff16610fcd0361147c57634000000094506118d3565b8563ffffffff166110180361149457600194506118d3565b8563ffffffff16611096036114ca57600161012088015260ff83166101008801526114bd61061b565b9998505050505050505050565b8563ffffffff16610fa3036117365763ffffffff8316156118d3577ffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffb63ffffffff8416016116f05760006115258363fffffffc166001610737565b60208901519091508060001a60010361159457604080516000838152336020528d83526060902091527effffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff167f01000000000000000000000000000000000000000000000000000000000000001790505b6040808a015190517fe03110e10000000000000000000000000000000000000000000000000000000081526004810183905263ffffffff9091166024820152600090819073ffffffffffffffffffffffffffffffffffffffff7f0000000000000000000000000000000000000000000000000000000000000000169063e03110e1906044016040805180830381865afa158015611635573d6000803e3d6000fd5b505050506040513d601f19601f820116820180604052508101906116599190611dd1565b91509150600386168060040382811015611671578092505b508186101561167e578591505b8260088302610100031c9250826008828460040303021b9250600180600883600403021b036001806008858560040303021b039150811981169050838119871617955050506116d58663fffffffc16600186611b0e565b60408b018051820163ffffffff169052975061173192505050565b7ffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffd63ffffffff841601611725578094506118d3565b63ffffffff9450600993505b6118d3565b8563ffffffff16610fa4036118275763ffffffff831660011480611760575063ffffffff83166002145b80611771575063ffffffff83166004145b1561177e578094506118d3565b7ffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffa63ffffffff8416016117255760006117be8363fffffffc166001610737565b602089015190915060038416600403838110156117d9578093505b83900360089081029290921c7fffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff600193850293841b0116911b176020880152600060408801529350836118d3565b8563ffffffff16610fd7036118d3578163ffffffff166003036118c75763ffffffff8316158061185d575063ffffffff83166005145b8061186e575063ffffffff83166003145b1561187c57600094506118d3565b63ffffffff831660011480611897575063ffffffff83166002145b806118a8575063ffffffff83166006145b806118b9575063ffffffff83166004145b1561172557600194506118d3565b63ffffffff9450601693505b6101608701805163ffffffff808816604090920191909152905185821660e09091015260808801805180831660608b015260040190911690526114bd61061b565b600061191e611c5b565b506080600063ffffffff871660100361193c575060c0810151611aa5565b8663ffffffff1660110361195b5763ffffffff861660c0830152611aa5565b8663ffffffff16601203611974575060a0810151611aa5565b8663ffffffff166013036119935763ffffffff861660a0830152611aa5565b8663ffffffff166018036119c75763ffffffff600387810b9087900b02602081901c821660c08501521660a0830152611aa5565b8663ffffffff166019036119f85763ffffffff86811681871602602081901c821660c08501521660a0830152611aa5565b8663ffffffff16601a03611a4e578460030b8660030b81611a1b57611a1b611df5565b0763ffffffff1660c0830152600385810b9087900b81611a3d57611a3d611df5565b0563ffffffff1660a0830152611aa5565b8663ffffffff16601b03611aa5578463ffffffff168663ffffffff1681611a7757611a77611df5565b0663ffffffff90811660c084015285811690871681611a9857611a98611df5565b0463ffffffff1660a08301525b63ffffffff841615611ae057808261016001518563ffffffff1660208110611acf57611acf611da2565b63ffffffff90921660209290920201525b60808201805163ffffffff80821660608601526004909101169052611b0361061b565b979650505050505050565b6000611b1983611bb2565b90506003841615611b2957600080fd5b6020810190601f8516601c0360031b83811b913563ffffffff90911b1916178460051c60005b601b811015611ba75760208401933582821c6001168015611b775760018114611b8c57611b9d565b60008581526020839052604090209450611b9d565b600082815260208690526040902094505b5050600101611b4f565b505060805250505050565b60ff8116610380026101a4810190369061052401811015611c55576040517f08c379a000000000000000000000000000000000000000000000000000000000815260206004820152602360248201527f636865636b207468617420746865726520697320656e6f7567682063616c6c6460448201527f6174610000000000000000000000000000000000000000000000000000000000606482015260840161087d565b50919050565b6040805161018081018252600080825260208201819052918101829052606081018290526080810182905260a0810182905260c0810182905260e08101829052610100810182905261012081018290526101408101919091526101608101611cc1611cc6565b905290565b6040518061040001604052806020906020820280368337509192915050565b60008083601f840112611cf757600080fd5b50813567ffffffffffffffff811115611d0f57600080fd5b602083019150836020828501011115611d2757600080fd5b9250929050565b600080600080600060608688031215611d4657600080fd5b853567ffffffffffffffff80821115611d5e57600080fd5b611d6a89838a01611ce5565b90975095506020880135915080821115611d8357600080fd5b50611d9088828901611ce5565b96999598509660400135949350505050565b7f4e487b7100000000000000000000000000000000000000000000000000000000600052603260045260246000fd5b60008060408385031215611de457600080fd5b505080516020909101519092909150565b7f4e487b7100000000000000000000000000000000000000000000000000000000600052601260045260246000fdfea164736f6c634300080f000a" -var MIPSDeployedSourceMap = "1131:40054:128:-:0;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;1710:45;;1745:10;1710:45;;;;;188:10:287;176:23;;;158:42;;146:2;131:18;1710:45:128;;;;;;;;2448:99;;;412:42:287;2534:6:128;400:55:287;382:74;;370:2;355:18;2448:99:128;211:251:287;26025:6379:128;;;;;;:::i;:::-;;:::i;:::-;;;1755:25:287;;;1743:2;1728:18;26025:6379:128;1609:177:287;26025:6379:128;26128:7;26171:18;;:::i;:::-;26318:4;26311:5;26308:15;26298:134;;26412:1;26409;26402:12;26298:134;26468:4;26462:11;26475:10;26459:27;26449:136;;26565:1;26562;26555:12;26449:136;26634:3;26615:17;26612:26;26602:151;;26733:1;26730;26723:12;26602:151;26798:3;26783:13;26780:22;26770:146;;26896:1;26893;26886:12;26770:146;27176:24;;27521:4;27222:20;27579:2;27280:21;;27176:24;27338:18;27222:20;27280:21;;;27176:24;27153:21;27149:52;;;27338:18;27222:20;;;27280:21;;;27176:24;27149:52;;27222:20;;27280:21;;;27176:24;27149:52;;27338:18;27222:20;27280:21;;;27176:24;27149:52;;27338:18;27222:20;27280:21;;;27176:24;27149:52;;27338:18;27222:20;27280:21;;;27176:24;27149:52;;;27338:18;27222:20;27280:21;;;27176:24;27153:21;27149:52;;;27338:18;27222:20;27280:21;;;27176:24;27149:52;;27338:18;27222:20;27280:21;;;27176:24;27149:52;;27338:18;27222:20;28197:10;27338:18;28187:21;;;27280;;;;28295:1;28280:77;28305:2;28302:1;28299:9;28280:77;;;27176:24;;27153:21;27149:52;27222:20;;28353:1;27280:21;;;;27164:2;27338:18;;;;28323:1;28316:9;28280:77;;;28284:14;;;28435:5;:12;;;28431:71;;;28474:13;:11;:13::i;:::-;28467:20;;;;;28431:71;28516:10;;;:15;;28530:1;28516:15;;;;;28601:8;;;;-1:-1:-1;;28593:20:128;;-1:-1:-1;28593:7:128;:20::i;:::-;28579:34;-1:-1:-1;28643:10:128;28651:2;28643:10;;;;28720:1;28710:11;;;:26;;;28725:6;:11;;28735:1;28725:11;28710:26;28706:310;;;28866:13;28935:1;28913:4;28920:10;28913:17;28912:24;;;;28883:5;:12;;;28898:10;28883:25;28882:54;28866:70;;28961:40;28972:6;:11;;28982:1;28972:11;:20;;28990:2;28972:20;;;28986:1;28972:20;28961:40;;28994:6;28961:10;:40::i;:::-;28954:47;;;;;;;;28706:310;29265:15;;;;29060:9;;;;29197:4;29191:2;29183:10;;;29182:19;;;29265:15;29290:2;29282:10;;;29281:19;29265:36;;;;;;;:::i;:::-;;;;;;-1:-1:-1;29330:5:128;29354:11;;;;;:29;;;29369:6;:14;;29379:4;29369:14;29354:29;29350:832;;;29446:5;:15;;;29462:5;29446:22;;;;;;;;;:::i;:::-;;;;;;-1:-1:-1;;29509:4:128;29503:2;29495:10;;;29494:19;29350:832;;;29547:4;29538:6;:13;;;29534:648;;;29668:6;:13;;29678:3;29668:13;:30;;;;29685:6;:13;;29695:3;29685:13;29668:30;:47;;;;29702:6;:13;;29712:3;29702:13;29668:47;29664:253;;;29778:4;29785:6;29778:13;29773:18;;29534:648;;29664:253;29877:21;29880:4;29887:6;29880:13;29895:2;29877;:21::i;:::-;29872:26;;29534:648;;;29951:4;29941:6;:14;;;;:32;;;;29959:6;:14;;29969:4;29959:14;29941:32;:50;;;;29977:6;:14;;29987:4;29977:14;29941:50;29937:245;;;30061:5;:15;;;30077:5;30061:22;;;;;;;;;:::i;:::-;;;;;30056:27;;30162:5;30154:13;;29937:245;30211:1;30201:6;:11;;;;:25;;;;;30225:1;30216:6;:10;;;30201:25;30200:42;;;;30231:6;:11;;30241:1;30231:11;30200:42;30196:125;;;30269:37;30282:6;30290:4;30296:5;30303:2;30269:12;:37::i;:::-;30262:44;;;;;;;;;;;30196:125;30354:13;30335:16;30506:4;30496:14;;;;30492:446;;30575:21;30578:4;30585:6;30578:13;30593:2;30575;:21::i;:::-;30569:27;;;;30633:10;30628:15;;30667:16;30628:15;30681:1;30667:7;:16::i;:::-;30661:22;;30715:4;30705:6;:14;;;;:32;;;;;30723:6;:14;;30733:4;30723:14;;30705:32;30701:223;;;30802:4;30790:16;;30904:1;30896:9;;30701:223;30512:426;30492:446;30971:10;30984:26;30992:4;30998:2;31002;31006:3;30984:7;:26::i;:::-;31013:10;30984:39;;;;-1:-1:-1;31109:4:128;31102:11;;;31141;;;:24;;;;;31164:1;31156:4;:9;;;;31141:24;:39;;;;;31176:4;31169;:11;;;31141:39;31137:860;;;31204:4;:9;;31212:1;31204:9;:22;;;;31217:4;:9;;31225:1;31217:9;31204:22;31200:144;;;31288:37;31299:4;:9;;31307:1;31299:9;:21;;31315:5;31299:21;;;31311:1;31299:21;31322:2;31288:10;:37::i;:::-;31281:44;;;;;;;;;;;;;;;31200:144;31366:4;:11;;31374:3;31366:11;31362:121;;31436:28;31445:5;31452:2;31456:7;;;;31436:8;:28::i;31362:121::-;31504:4;:11;;31512:3;31504:11;31500:121;;31574:28;31583:5;31590:2;31594:7;;;;;31574:8;:28::i;31500:121::-;31691:4;:11;;31699:3;31691:11;31687:93;;31733:28;31747:13;31733;:28::i;31687:93::-;31883:4;31875;:12;;;;:27;;;;;31898:4;31891;:11;;;31875:27;31871:112;;;31933:31;31944:4;31950:2;31954;31958:5;31933:10;:31::i;31871:112::-;32057:6;:14;;32067:4;32057:14;:28;;;;-1:-1:-1;32075:10:128;;;;;32057:28;32053:93;;;32130:1;32105:5;:15;;;32121:5;32105:22;;;;;;;;;:::i;:::-;:26;;;;:22;;;;;;:26;32053:93;32192:9;:26;;32205:13;32192:26;32188:92;;32238:27;32247:9;32258:1;32261:3;32238:8;:27::i;:::-;32361:26;32370:5;32377:3;32382:4;32361:8;:26::i;:::-;32354:33;;;;;;;;;;;;;26025:6379;;;;;;;;:::o;3087:2334::-;3634:4;3628:11;;3550:4;3353:31;3342:43;;3413:13;3353:31;3752:2;3452:13;;3342:43;3359:24;3353:31;3452:13;;;3342:43;;;;3359:24;3353:31;3452:13;;;3342:43;3359:24;3353:31;3452:13;;;3342:43;3359:24;3353:31;3452:13;;;3342:43;3359:24;3353:31;3452:13;;;3342:43;3359:24;3353:31;3452:13;;;3342:43;3413:13;4180:11;3359:24;3353:31;3452:13;;;3342:43;3413:13;4275:11;3359:24;3353:31;3452:13;;;3342:43;3359:24;3353:31;3452:13;;;3342:43;3128:12;;4415:13;;3628:11;;3452:13;;;;4180:11;3128:12;4495:84;4520:2;4517:1;4514:9;4495:84;;;3369:13;3359:24;;3353:31;3342:43;;3373:2;3413:13;;;;4575:1;3452:13;;;;4538:1;4531:9;4495:84;;;4499:14;4642:1;4638:2;4631:13;4737:5;4733:2;4729:14;4722:5;4717:27;4811:1;4797:15;;4832:6;4856:1;4851:273;;;;5191:1;5181:11;;4825:369;;4851:273;4883:8;4941:22;;;;5020:1;5015:22;;;;5107:1;5097:11;;4876:234;;4941:22;4960:1;4950:11;;4941:22;;5015;5034:1;5024:11;;4876:234;;4825:369;-1:-1:-1;;;5317:14:128;;;5300:32;;5360:19;5356:30;5392:3;5388:16;;;;5353:52;;3087:2334;-1:-1:-1;3087:2334:128:o;21746:1831::-;21819:11;21930:14;21947:24;21959:11;21947;:24::i;:::-;21930:41;;22079:1;22072:5;22068:13;22065:33;;;22094:1;22091;22084:12;22065:33;22227:2;22215:15;;;22168:20;22657:5;22654:1;22650:13;22692:4;22728:1;22713:343;22738:2;22735:1;22732:9;22713:343;;;22861:2;22849:15;;;22798:20;22896:12;;;22910:1;22892:20;22933:42;;;;23001:1;22996:42;;;;22885:153;;22933:42;22391:1;22384:12;;;22424:2;22417:13;;;22469:2;22456:16;;22942:31;;22933:42;;22996;22391:1;22384:12;;;22424:2;22417:13;;;22469:2;22456:16;;23005:31;;22885:153;-1:-1:-1;;22756:1:128;22749:9;22713:343;;;22717:14;23166:4;23160:11;23145:26;;23252:7;23246:4;23243:17;23233:124;;23294:10;23291:1;23284:21;23336:2;23333:1;23326:13;23233:124;-1:-1:-1;;23484:2:128;23473:14;;;;23461:10;23457:31;23454:1;23450:39;23518:16;;;;23536:10;23514:33;;21746:1831;-1:-1:-1;;;21746:1831:128:o;18856:823::-;18925:12;19012:18;;:::i;:::-;19080:4;19071:13;;19132:5;:8;;;19143:1;19132:12;19116:28;;:5;:12;;;:28;;;19112:95;;19164:28;;;;;2182:2:287;19164:28:128;;;2164:21:287;2221:2;2201:18;;;2194:30;2260:20;2240:18;;;2233:48;2298:18;;19164:28:128;;;;;;;;19112:95;19296:8;;;;;19329:12;;;;;19318:23;;;;;;;19355:20;;;;;19296:8;19487:13;;;19483:90;;19548:6;19557:1;19548:10;19520:5;:15;;;19536:8;19520:25;;;;;;;;;:::i;:::-;:38;;;;:25;;;;;;:38;19483:90;19649:13;:11;:13::i;2645:339::-;2706:11;2770:18;;;;2779:8;;;;2770:18;;;;;;2769:25;;;;;2786:1;2833:2;:9;;;2827:16;;;;;2826:22;;2825:32;;;;;;;2887:9;;2886:15;2769:25;2944:21;;2964:1;2944:21;;;2955:6;2944:21;2929:11;;;;;:37;;-1:-1:-1;;;2645:339:128;;;;:::o;13732:2026::-;13829:12;13915:18;;:::i;:::-;13983:4;13974:13;;14015:17;14075:5;:8;;;14086:1;14075:12;14059:28;;:5;:12;;;:28;;;14055:97;;14107:30;;;;;2529:2:287;14107:30:128;;;2511:21:287;2568:2;2548:18;;;2541:30;2607:22;2587:18;;;2580:50;2647:18;;14107:30:128;2327:344:287;14055:97:128;14222:7;:12;;14233:1;14222:12;:28;;;;14238:7;:12;;14249:1;14238:12;14222:28;14218:947;;;14270:9;14282:5;:15;;;14298:6;14282:23;;;;;;;;;:::i;:::-;;;;;14270:35;;14346:2;14339:9;;:3;:9;;;:25;;;;;14352:7;:12;;14363:1;14352:12;14339:25;14338:58;;;;14377:2;14370:9;;:3;:9;;;;:25;;;;;14383:7;:12;;14394:1;14383:12;14370:25;14323:73;;14252:159;14218:947;;;14508:7;:12;;14519:1;14508:12;14504:661;;14569:1;14561:3;14555:15;;;;14540:30;;14504:661;;;14673:7;:12;;14684:1;14673:12;14669:496;;14733:1;14726:3;14720:14;;;14705:29;;14669:496;;;14854:7;:12;;14865:1;14854:12;14850:315;;14942:4;14936:2;14927:11;;;14926:20;14912:10;14969:8;;;14965:84;;15029:1;15022:3;15016:14;;;15001:29;;14965:84;15070:3;:8;;15077:1;15070:8;15066:85;;15131:1;15123:3;15117:15;;;;15102:30;;15066:85;14868:297;14850:315;15241:8;;;;;15319:12;;;;15308:23;;;;;15475:178;;;;15566:1;15540:22;15543:5;15551:6;15543:14;15559:2;15540;:22::i;:::-;:27;;;;;;;15526:42;;15535:1;15526:42;15511:57;:12;;;:57;15475:178;;;15622:12;;;;;15637:1;15622:16;15607:31;;;;15475:178;15728:13;:11;:13::i;:::-;15721:20;13732:2026;-1:-1:-1;;;;;;;;13732:2026:128:o;32450:8733::-;32537:10;32599;32607:2;32599:10;;;;32638:11;;;:44;;;32664:1;32654:6;:11;;;;:27;;;;;32678:3;32669:6;:12;;;32654:27;32634:8490;;;32723:4;32716:11;;32847:6;32907:3;32902:25;;;;32982:3;32977:25;;;;33056:3;33051:25;;;;33131:3;33126:25;;;;33205:3;33200:25;;;;33278:3;33273:25;;;;33352:3;33347:25;;;;32840:532;;32902:25;32921:4;32913:12;;32902:25;;32977;32996:4;32988:12;;32977:25;;33051;33070:4;33062:12;;33051:25;;33126;33145:4;33137:12;;33126:25;;33200;33219:4;33211:12;;33200:25;;33273;33292:4;33284:12;;33273:25;;33347;33366:4;33358:12;;32840:532;;33435:4;:12;;33443:4;33435:12;33431:4023;;-1:-1:-1;;;33486:9:128;33478:26;;33499:4;33494:1;33486:9;;;33485:18;33478:26;33471:33;;33431:4023;33572:4;:12;;33580:4;33572:12;33568:3886;;-1:-1:-1;;;33623:9:128;33615:26;;33636:4;33631:1;33623:9;;;33622:18;33615:26;33608:33;;33568:3886;33709:4;:12;;33717:4;33709:12;33705:3749;;33774:4;33769:1;33761:9;;;33760:18;33807:27;33761:9;33810:11;;;;33823:2;:10;;;33807:2;:27::i;:::-;33800:34;;;;;;;33705:3749;33903:4;:12;;33911:4;33903:12;33899:3555;;-1:-1:-1;;;33946:17:128;;;33958:4;33953:9;;33946:17;33939:24;;33899:3555;34032:4;:11;;34040:3;34032:11;34028:3426;;-1:-1:-1;;;34074:17:128;;;34086:4;34081:9;;34074:17;34067:24;;34028:3426;34160:4;:12;;34168:4;34160:12;34156:3298;;34203:21;34212:2;34206:8;;:2;:8;;;;34221:2;34216;:7;34203:2;:21::i;:::-;34196:28;;;;;;34156:3298;34473:4;:12;;34481:4;34473:12;34469:2985;;34516:2;34509:9;;;;;;34469:2985;34587:4;:12;;34595:4;34587:12;34583:2871;;34630:2;34623:9;;;;;;34583:2871;34701:4;:12;;34709:4;34701:12;34697:2757;;34744:2;34737:9;;;;;;34697:2757;34815:4;:12;;34823:4;34815:12;34811:2643;;34858:2;34851:9;;;;;;34811:2643;34932:4;:12;;34940:4;34932:12;34928:2526;;34975:2;34968:9;;;;;;34928:2526;35092:4;:12;;35100:4;35092:12;35088:2366;;35135:2;35128:9;;;;;;35088:2366;35206:4;:12;;35214:4;35206:12;35202:2252;;35249:2;35242:9;;;;;;35202:2252;35320:4;:12;;35328:4;35320:12;35316:2138;;35363:2;35356:9;;;;;;35316:2138;35434:4;:12;;35442:4;35434:12;35430:2024;;35477:2;35470:9;;;;;;35430:2024;35548:4;:12;;35556:4;35548:12;35544:1910;;35591:2;35584:9;;;;;;35544:1910;35662:4;:12;;35670:4;35662:12;35658:1796;;35705:2;35698:9;;;;;;35658:1796;35777:4;:12;;35785:4;35777:12;35773:1681;;35820:2;35813:9;;;;;;35773:1681;35890:4;:12;;35898:4;35890:12;35886:1568;;35933:2;35926:9;;;;;;35886:1568;36004:4;:12;;36012:4;36004:12;36000:1454;;36047:2;36040:9;;;;;;36000:1454;36196:4;:12;;36204:4;36196:12;36192:1262;;-1:-1:-1;;;36240:7:128;;;36232:16;;36192:1262;36317:4;:12;;36325:4;36317:12;36313:1141;;-1:-1:-1;;;36361:7:128;;;36353:16;;36313:1141;36437:4;:12;;36445:4;36437:12;36433:1021;;-1:-1:-1;;;36481:7:128;;;36473:16;;36433:1021;36558:4;:12;;36566:4;36558:12;36554:900;;-1:-1:-1;;;36602:7:128;;;36594:16;;36554:900;36678:4;:12;;36686:4;36678:12;36674:780;;-1:-1:-1;;;36722:7:128;;;36714:16;;36674:780;36797:4;:12;;36805:4;36797:12;36793:661;;-1:-1:-1;;;36841:7:128;;;36833:16;;36793:661;36917:4;:12;;36925:4;36917:12;36913:541;;-1:-1:-1;;;36961:7:128;;;36953:16;;36913:541;37037:4;:12;;37045:4;37037:12;37033:421;;-1:-1:-1;;;37082:7:128;;;37080:10;37073:17;;37033:421;37159:4;:12;;37167:4;37159:12;37155:299;;37220:2;37202:21;;37208:2;37202:21;;;:29;;37230:1;37202:29;;;37226:1;37202:29;37195:36;;;;;;;;37155:299;37301:4;:12;;37309:4;37301:12;37297:157;;37349:2;37344:7;;:2;:7;;;:15;;37358:1;37344:15;;37297:157;37406:29;;;;;2878:2:287;37406:29:128;;;2860:21:287;2917:2;2897:18;;;2890:30;2956:21;2936:18;;;2929:49;2995:18;;37406:29:128;2676:343:287;37297:157:128;32684:4784;32634:8490;;;37524:6;:14;;37534:4;37524:14;37520:3590;;37583:4;37576:11;;37658:3;37650:11;;;37646:549;;-1:-1:-1;;;37703:21:128;;;37689:36;;37646:549;37810:4;:12;;37818:4;37810:12;:28;;;;37826:4;:12;;37834:4;37826:12;37810:28;37806:389;;;37870:4;:12;;37878:4;37870:12;37866:83;;37919:3;;;37866:83;37974:8;38012:127;38024:10;38019:15;;:20;38012:127;;38104:8;38071:3;38104:8;;;;;38071:3;38012:127;;;38171:1;-1:-1:-1;38164:8:128;;-1:-1:-1;;38164:8:128;37520:3590;38262:6;:14;;38272:4;38262:14;38258:2852;;-1:-1:-1;;38307:8:128;38313:2;38307:8;;;;38300:15;;38258:2852;38382:6;:14;;38392:4;38382:14;38378:2732;;38427:42;38445:2;38450:1;38445:6;38455:1;38444:12;38439:2;:17;38431:26;;:3;:26;;;;38461:4;38430:35;38467:1;38427:2;:42::i;:::-;38420:49;;;;;38378:2732;38536:6;:14;;38546:4;38536:14;38532:2578;;38581:45;38599:2;38604:1;38599:6;38609:1;38598:12;38593:2;:17;38585:26;;:3;:26;;;;38615:6;38584:37;38623:2;38581;:45::i;38532:2578::-;38694:6;:14;;38704:4;38694:14;38690:2420;;-1:-1:-1;;38745:21:128;38764:1;38759;38754:6;;38753:12;38745:21;;38802:36;;;38873:5;38868:10;;38745:21;;;;;38867:18;38860:25;;38690:2420;38952:6;:14;;38962:4;38952:14;38948:2162;;38997:3;38990:10;;;;;38948:2162;39068:6;:14;;39078:4;39068:14;39064:2046;;39128:2;39133:1;39128:6;39138:1;39127:12;39122:2;:17;39114:26;;:3;:26;;;;39144:4;39113:35;39106:42;;;;;39064:2046;39217:6;:14;;39227:4;39217:14;39213:1897;;39277:2;39282:1;39277:6;39287:1;39276:12;39271:2;:17;39263:26;;:3;:26;;;;39293:6;39262:37;39255:44;;;;;39213:1897;39368:6;:14;;39378:4;39368:14;39364:1746;;-1:-1:-1;;39419:26:128;39443:1;39438;39433:6;;39432:12;39427:2;:17;39419:26;;39481:41;;;39557:5;39552:10;;39419:26;;;;;39551:18;39544:25;;39364:1746;39637:6;:14;;39647:4;39637:14;39633:1477;;-1:-1:-1;;39694:4:128;39688:34;39720:1;39715;39710:6;;39709:12;39704:2;:17;39688:34;;39778:27;;;39758:48;;;39836:10;;39689:9;;;39688:34;;39835:18;39828:25;;39633:1477;39921:6;:14;;39931:4;39921:14;39917:1193;;-1:-1:-1;;39978:6:128;39972:36;40006:1;40001;39996:6;;39995:12;39990:2;:17;39972:36;;40064:29;;;40044:50;;;40124:10;;39973:11;;;39972:36;;40123:18;40116:25;;39917:1193;40210:6;:14;;40220:4;40210:14;40206:904;;-1:-1:-1;;40261:20:128;40279:1;40274;40269:6;;40268:12;40261:20;;40317:36;;;40389:5;40383:11;;40261:20;;;;;40382:19;40375:26;;40206:904;40469:6;:14;;40479:4;40469:14;40465:645;;40514:2;40507:9;;;;;40465:645;40585:6;:14;;40595:4;40585:14;40581:529;;-1:-1:-1;;40636:25:128;40659:1;40654;40649:6;;40648:12;40643:2;:17;40636:25;;40697:41;;;40774:5;40768:11;;40636:25;;;;;40767:19;40760:26;;40581:529;40853:6;:14;;40863:4;40853:14;40849:261;;40898:3;40891:10;;;;;40849:261;40968:6;:14;;40978:4;40968:14;40964:146;;41013:2;41006:9;;;32450:8733;;;;;;;:::o;19960:782::-;20046:12;20133:18;;:::i;:::-;-1:-1:-1;20201:4:128;20308:2;20296:14;;;;20288:41;;;;;;;3226:2:287;20288:41:128;;;3208:21:287;3265:2;3245:18;;;3238:30;3304:16;3284:18;;;3277:44;3338:18;;20288:41:128;3024:338:287;20288:41:128;20425:14;;;;;;;:30;;;20443:12;20425:30;20421:102;;;20504:4;20475:5;:15;;;20491:9;20475:26;;;;;;;;;:::i;:::-;:33;;;;:26;;;;;;:33;20421:102;20578:12;;;;;20567:23;;;;:8;;;:23;20634:1;20619:16;;;20604:31;;;20712:13;:11;:13::i;5582:7764::-;5646:12;5732:18;;:::i;:::-;-1:-1:-1;5910:15:128;;:18;;;;5800:4;6070:18;;;;6114;;;;6158;;;;;5800:4;;5890:17;;;;6070:18;6114;6248;;;6262:4;6248:18;6244:6792;;6298:2;6327:4;6322:9;;:14;6318:144;;6438:4;6433:9;;6425:4;:18;6419:24;6318:144;6483:2;:7;;6489:1;6483:7;6479:161;;6519:10;;;;;6551:16;;;;;;;;6519:10;-1:-1:-1;6479:161:128;;;6619:2;6614:7;;6479:161;6268:386;6244:6792;;;6756:10;:18;;6770:4;6756:18;6752:6284;;1745:10;6794:14;;6752:6284;;;6892:10;:18;;6906:4;6892:18;6888:6148;;6935:1;6930:6;;6888:6148;;;7060:10;:18;;7074:4;7060:18;7056:5980;;7113:4;7098:12;;;:19;7135:26;;;:14;;;:26;7186:13;:11;:13::i;:::-;7179:20;5582:7764;-1:-1:-1;;;;;;;;;5582:7764:128:o;7056:5980::-;7325:10;:18;;7339:4;7325:18;7321:5715;;7476:14;;;7472:2723;7321:5715;7472:2723;7646:22;;;;;7642:2553;;7771:10;7784:27;7792:2;7797:10;7792:15;7809:1;7784:7;:27::i;:::-;7895:17;;;;7771:40;;-1:-1:-1;7895:17:128;7873:19;8045:14;8064:1;8039:26;8035:146;;1676:4:129;1670:11;;1533:21;1787:15;;;1828:8;1822:4;1815:22;1850:27;;;1996:4;1983:18;;2098:17;;2003:19;1979:44;2025:11;1976:61;8093:65:128;;8035:146;8267:20;;;;;8234:54;;;;;;;;3540:25:287;;;8234:54:128;3601:23:287;;;3581:18;;;3574:51;8203:11:128;;;;8234:19;:6;:19;;;;3513:18:287;;8234:54:128;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;:::i;:::-;8202:86;;;;8515:1;8511:2;8507:10;8612:9;8609:1;8605:17;8694:6;8687:5;8684:17;8681:40;;;8714:5;8704:15;;8681:40;;8797:6;8793:2;8790:14;8787:34;;;8817:2;8807:12;;8787:34;8923:3;8918:1;8910:6;8906:14;8901:3;8897:24;8893:34;8886:41;;9023:3;9019:1;9007:9;8998:6;8995:1;8991:14;8987:30;8983:38;8979:48;8972:55;;9178:1;9174;9170;9158:9;9155:1;9151:17;9147:25;9143:33;9139:41;9305:1;9301;9297;9288:6;9276:9;9273:1;9269:17;9265:30;9261:38;9257:46;9253:54;9235:72;;9436:10;9432:15;9426:4;9422:26;9414:34;;9552:3;9544:4;9540:9;9535:3;9531:19;9528:28;9521:35;;;;9698:33;9707:2;9712:10;9707:15;9724:1;9727:3;9698:8;:33::i;:::-;9753:20;;;:38;;;;;;;;;-1:-1:-1;7642:2553:128;;-1:-1:-1;;;7642:2553:128;;9910:18;;;;;9906:289;;10080:2;10075:7;;7321:5715;;9906:289;10134:10;10129:15;;2053:3;10166:10;;9906:289;7321:5715;;;10324:10;:18;;10338:4;10324:18;10320:2716;;10478:15;;;1824:1;10478:15;;:34;;-1:-1:-1;10497:15:128;;;1859:1;10497:15;10478:34;:57;;;-1:-1:-1;10516:19:128;;;1936:1;10516:19;10478:57;10474:1593;;;10564:2;10559:7;;10320:2716;;10474:1593;10690:23;;;;;10686:1381;;10737:10;10750:27;10758:2;10763:10;10758:15;10775:1;10750:7;:27::i;:::-;10853:17;;;;10737:40;;-1:-1:-1;11096:1:128;11088:10;;11190:1;11186:17;11265:13;;;11262:32;;;11287:5;11281:11;;11262:32;11573:14;;;11379:1;11569:22;;;11565:32;;;;11462:26;11486:1;11371:10;;;11466:18;;;11462:26;11561:43;11367:20;;11669:12;11797:17;;;:23;11865:1;11842:20;;;:24;11375:2;-1:-1:-1;11375:2:128;7321:5715;;10320:2716;12269:10;:18;;12283:4;12269:18;12265:771;;12379:2;:7;;12385:1;12379:7;12375:647;;12472:14;;;;;:40;;-1:-1:-1;12490:22:128;;;1978:1;12490:22;12472:40;:62;;;-1:-1:-1;12516:18:128;;;1897:1;12516:18;12472:62;12468:404;;;12567:1;12562:6;;12375:647;;12468:404;12613:15;;;1824:1;12613:15;;:34;;-1:-1:-1;12632:15:128;;;1859:1;12632:15;12613:34;:61;;;-1:-1:-1;12651:23:128;;;2021:1;12651:23;12613:61;:84;;;-1:-1:-1;12678:19:128;;;1936:1;12678:19;12613:84;12609:263;;;12730:1;12725:6;;7321:5715;;12375:647;12923:10;12918:15;;2087:4;12955:11;;12375:647;13111:15;;;;;:23;;;;:18;;;;:23;;;;13148:15;;:23;;;:18;;;;:23;-1:-1:-1;13237:12:128;;;;13226:23;;;:8;;;:23;13293:1;13278:16;13263:31;;;;;13316:13;:11;:13::i;16084:2480::-;16178:12;16264:18;;:::i;:::-;-1:-1:-1;16332:4:128;16364:10;16472:13;;;16481:4;16472:13;16468:1705;;-1:-1:-1;16511:8:128;;;;16468:1705;;;16630:5;:13;;16639:4;16630:13;16626:1547;;16663:14;;;:8;;;:14;16626:1547;;;16793:5;:13;;16802:4;16793:13;16789:1384;;-1:-1:-1;16832:8:128;;;;16789:1384;;;16951:5;:13;;16960:4;16951:13;16947:1226;;16984:14;;;:8;;;:14;16947:1226;;;17125:5;:13;;17134:4;17125:13;17121:1052;;17252:9;17198:17;17178;;;17198;;;;17178:37;17259:2;17252:9;;;;;17234:8;;;:28;17280:22;:8;;;:22;17121:1052;;;17439:5;:13;;17448:4;17439:13;17435:738;;17506:11;17492;;;17506;;;17492:25;17561:2;17554:9;;;;;17536:8;;;:28;17582:22;:8;;;:22;17435:738;;;17763:5;:13;;17772:4;17763:13;17759:414;;17833:3;17814:23;;17820:3;17814:23;;;;;;;:::i;:::-;;17796:42;;:8;;;:42;17874:23;;;;;;;;;;;;;:::i;:::-;;17856:42;;:8;;;:42;17759:414;;;18067:5;:13;;18076:4;18067:13;18063:110;;18117:3;18111:9;;:3;:9;;;;;;;:::i;:::-;;18100:20;;;;:8;;;:20;18149:9;;;;;;;;;;;:::i;:::-;;18138:20;;:8;;;:20;18063:110;18266:14;;;;18262:85;;18329:3;18300:5;:15;;;18316:9;18300:26;;;;;;;;;:::i;:::-;:32;;;;:26;;;;;;:32;18262:85;18401:12;;;;;18390:23;;;;:8;;;:23;18457:1;18442:16;;;18427:31;;;18534:13;:11;:13::i;:::-;18527:20;16084:2480;-1:-1:-1;;;;;;;16084:2480:128:o;23913:1654::-;24089:14;24106:24;24118:11;24106;:24::i;:::-;24089:41;;24238:1;24231:5;24227:13;24224:33;;;24253:1;24250;24243:12;24224:33;24392:2;24586:15;;;24411:2;24400:14;;24388:10;24384:31;24381:1;24377:39;24542:16;;;24327:20;;24527:10;24516:22;;;24512:27;24502:38;24499:60;25028:5;25025:1;25021:13;25099:1;25084:343;25109:2;25106:1;25103:9;25084:343;;;25232:2;25220:15;;;25169:20;25267:12;;;25281:1;25263:20;25304:42;;;;25372:1;25367:42;;;;25256:153;;25304:42;22391:1;22384:12;;;22424:2;22417:13;;;22469:2;22456:16;;25313:31;;25304:42;;25367;22391:1;22384:12;;;22424:2;22417:13;;;22469:2;22456:16;;25376:31;;25256:153;-1:-1:-1;;25127:1:128;25120:9;25084:343;;;-1:-1:-1;;25526:4:128;25519:18;-1:-1:-1;;;;23913:1654:128:o;20946:586::-;21268:20;;;21292:7;21268:32;21261:3;:40;;;21374:14;;21429:17;;21423:24;;;21415:72;;;;;;;4277:2:287;21415:72:128;;;4259:21:287;4316:2;4296:18;;;4289:30;4355:34;4335:18;;;4328:62;4426:5;4406:18;;;4399:33;4449:19;;21415:72:128;4075:399:287;21415:72:128;21501:14;20946:586;;;:::o;-1:-1:-1:-;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;:::i;:::-;;;;:::o;:::-;;;;;;;;;;;;;;;;;;;;;;;;:::o;467:347:287:-;518:8;528:6;582:3;575:4;567:6;563:17;559:27;549:55;;600:1;597;590:12;549:55;-1:-1:-1;623:20:287;;666:18;655:30;;652:50;;;698:1;695;688:12;652:50;735:4;727:6;723:17;711:29;;787:3;780:4;771:6;763;759:19;755:30;752:39;749:59;;;804:1;801;794:12;749:59;467:347;;;;;:::o;819:785::-;918:6;926;934;942;950;1003:2;991:9;982:7;978:23;974:32;971:52;;;1019:1;1016;1009:12;971:52;1059:9;1046:23;1088:18;1129:2;1121:6;1118:14;1115:34;;;1145:1;1142;1135:12;1115:34;1184:58;1234:7;1225:6;1214:9;1210:22;1184:58;:::i;:::-;1261:8;;-1:-1:-1;1158:84:287;-1:-1:-1;1349:2:287;1334:18;;1321:32;;-1:-1:-1;1365:16:287;;;1362:36;;;1394:1;1391;1384:12;1362:36;;1433:60;1485:7;1474:8;1463:9;1459:24;1433:60;:::i;:::-;819:785;;;;-1:-1:-1;1512:8:287;1594:2;1579:18;1566:32;;819:785;-1:-1:-1;;;;819:785:287:o;1791:184::-;1843:77;1840:1;1833:88;1940:4;1937:1;1930:15;1964:4;1961:1;1954:15;3636:245;3715:6;3723;3776:2;3764:9;3755:7;3751:23;3747:32;3744:52;;;3792:1;3789;3782:12;3744:52;-1:-1:-1;;3815:16:287;;3871:2;3856:18;;;3850:25;3815:16;;3850:25;;-1:-1:-1;3636:245:287:o;3886:184::-;3938:77;3935:1;3928:88;4035:4;4032:1;4025:15;4059:4;4056:1;4049:15" +var MIPSDeployedSourceMap = "1131:40054:127:-:0;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;1710:45;;1745:10;1710:45;;;;;188:10:286;176:23;;;158:42;;146:2;131:18;1710:45:127;;;;;;;;2448:99;;;412:42:286;2534:6:127;400:55:286;382:74;;370:2;355:18;2448:99:127;211:251:286;26025:6379:127;;;;;;:::i;:::-;;:::i;:::-;;;1755:25:286;;;1743:2;1728:18;26025:6379:127;1609:177:286;26025:6379:127;26128:7;26171:18;;:::i;:::-;26318:4;26311:5;26308:15;26298:134;;26412:1;26409;26402:12;26298:134;26468:4;26462:11;26475:10;26459:27;26449:136;;26565:1;26562;26555:12;26449:136;26634:3;26615:17;26612:26;26602:151;;26733:1;26730;26723:12;26602:151;26798:3;26783:13;26780:22;26770:146;;26896:1;26893;26886:12;26770:146;27176:24;;27521:4;27222:20;27579:2;27280:21;;27176:24;27338:18;27222:20;27280:21;;;27176:24;27153:21;27149:52;;;27338:18;27222:20;;;27280:21;;;27176:24;27149:52;;27222:20;;27280:21;;;27176:24;27149:52;;27338:18;27222:20;27280:21;;;27176:24;27149:52;;27338:18;27222:20;27280:21;;;27176:24;27149:52;;27338:18;27222:20;27280:21;;;27176:24;27149:52;;;27338:18;27222:20;27280:21;;;27176:24;27153:21;27149:52;;;27338:18;27222:20;27280:21;;;27176:24;27149:52;;27338:18;27222:20;27280:21;;;27176:24;27149:52;;27338:18;27222:20;28197:10;27338:18;28187:21;;;27280;;;;28295:1;28280:77;28305:2;28302:1;28299:9;28280:77;;;27176:24;;27153:21;27149:52;27222:20;;28353:1;27280:21;;;;27164:2;27338:18;;;;28323:1;28316:9;28280:77;;;28284:14;;;28435:5;:12;;;28431:71;;;28474:13;:11;:13::i;:::-;28467:20;;;;;28431:71;28516:10;;;:15;;28530:1;28516:15;;;;;28601:8;;;;-1:-1:-1;;28593:20:127;;-1:-1:-1;28593:7:127;:20::i;:::-;28579:34;-1:-1:-1;28643:10:127;28651:2;28643:10;;;;28720:1;28710:11;;;:26;;;28725:6;:11;;28735:1;28725:11;28710:26;28706:310;;;28866:13;28935:1;28913:4;28920:10;28913:17;28912:24;;;;28883:5;:12;;;28898:10;28883:25;28882:54;28866:70;;28961:40;28972:6;:11;;28982:1;28972:11;:20;;28990:2;28972:20;;;28986:1;28972:20;28961:40;;28994:6;28961:10;:40::i;:::-;28954:47;;;;;;;;28706:310;29265:15;;;;29060:9;;;;29197:4;29191:2;29183:10;;;29182:19;;;29265:15;29290:2;29282:10;;;29281:19;29265:36;;;;;;;:::i;:::-;;;;;;-1:-1:-1;29330:5:127;29354:11;;;;;:29;;;29369:6;:14;;29379:4;29369:14;29354:29;29350:832;;;29446:5;:15;;;29462:5;29446:22;;;;;;;;;:::i;:::-;;;;;;-1:-1:-1;;29509:4:127;29503:2;29495:10;;;29494:19;29350:832;;;29547:4;29538:6;:13;;;29534:648;;;29668:6;:13;;29678:3;29668:13;:30;;;;29685:6;:13;;29695:3;29685:13;29668:30;:47;;;;29702:6;:13;;29712:3;29702:13;29668:47;29664:253;;;29778:4;29785:6;29778:13;29773:18;;29534:648;;29664:253;29877:21;29880:4;29887:6;29880:13;29895:2;29877;:21::i;:::-;29872:26;;29534:648;;;29951:4;29941:6;:14;;;;:32;;;;29959:6;:14;;29969:4;29959:14;29941:32;:50;;;;29977:6;:14;;29987:4;29977:14;29941:50;29937:245;;;30061:5;:15;;;30077:5;30061:22;;;;;;;;;:::i;:::-;;;;;30056:27;;30162:5;30154:13;;29937:245;30211:1;30201:6;:11;;;;:25;;;;;30225:1;30216:6;:10;;;30201:25;30200:42;;;;30231:6;:11;;30241:1;30231:11;30200:42;30196:125;;;30269:37;30282:6;30290:4;30296:5;30303:2;30269:12;:37::i;:::-;30262:44;;;;;;;;;;;30196:125;30354:13;30335:16;30506:4;30496:14;;;;30492:446;;30575:21;30578:4;30585:6;30578:13;30593:2;30575;:21::i;:::-;30569:27;;;;30633:10;30628:15;;30667:16;30628:15;30681:1;30667:7;:16::i;:::-;30661:22;;30715:4;30705:6;:14;;;;:32;;;;;30723:6;:14;;30733:4;30723:14;;30705:32;30701:223;;;30802:4;30790:16;;30904:1;30896:9;;30701:223;30512:426;30492:446;30971:10;30984:26;30992:4;30998:2;31002;31006:3;30984:7;:26::i;:::-;31013:10;30984:39;;;;-1:-1:-1;31109:4:127;31102:11;;;31141;;;:24;;;;;31164:1;31156:4;:9;;;;31141:24;:39;;;;;31176:4;31169;:11;;;31141:39;31137:860;;;31204:4;:9;;31212:1;31204:9;:22;;;;31217:4;:9;;31225:1;31217:9;31204:22;31200:144;;;31288:37;31299:4;:9;;31307:1;31299:9;:21;;31315:5;31299:21;;;31311:1;31299:21;31322:2;31288:10;:37::i;:::-;31281:44;;;;;;;;;;;;;;;31200:144;31366:4;:11;;31374:3;31366:11;31362:121;;31436:28;31445:5;31452:2;31456:7;;;;31436:8;:28::i;31362:121::-;31504:4;:11;;31512:3;31504:11;31500:121;;31574:28;31583:5;31590:2;31594:7;;;;;31574:8;:28::i;31500:121::-;31691:4;:11;;31699:3;31691:11;31687:93;;31733:28;31747:13;31733;:28::i;31687:93::-;31883:4;31875;:12;;;;:27;;;;;31898:4;31891;:11;;;31875:27;31871:112;;;31933:31;31944:4;31950:2;31954;31958:5;31933:10;:31::i;31871:112::-;32057:6;:14;;32067:4;32057:14;:28;;;;-1:-1:-1;32075:10:127;;;;;32057:28;32053:93;;;32130:1;32105:5;:15;;;32121:5;32105:22;;;;;;;;;:::i;:::-;:26;;;;:22;;;;;;:26;32053:93;32192:9;:26;;32205:13;32192:26;32188:92;;32238:27;32247:9;32258:1;32261:3;32238:8;:27::i;:::-;32361:26;32370:5;32377:3;32382:4;32361:8;:26::i;:::-;32354:33;;;;;;;;;;;;;26025:6379;;;;;;;;:::o;3087:2334::-;3634:4;3628:11;;3550:4;3353:31;3342:43;;3413:13;3353:31;3752:2;3452:13;;3342:43;3359:24;3353:31;3452:13;;;3342:43;;;;3359:24;3353:31;3452:13;;;3342:43;3359:24;3353:31;3452:13;;;3342:43;3359:24;3353:31;3452:13;;;3342:43;3359:24;3353:31;3452:13;;;3342:43;3359:24;3353:31;3452:13;;;3342:43;3413:13;4180:11;3359:24;3353:31;3452:13;;;3342:43;3413:13;4275:11;3359:24;3353:31;3452:13;;;3342:43;3359:24;3353:31;3452:13;;;3342:43;3128:12;;4415:13;;3628:11;;3452:13;;;;4180:11;3128:12;4495:84;4520:2;4517:1;4514:9;4495:84;;;3369:13;3359:24;;3353:31;3342:43;;3373:2;3413:13;;;;4575:1;3452:13;;;;4538:1;4531:9;4495:84;;;4499:14;4642:1;4638:2;4631:13;4737:5;4733:2;4729:14;4722:5;4717:27;4811:1;4797:15;;4832:6;4856:1;4851:273;;;;5191:1;5181:11;;4825:369;;4851:273;4883:8;4941:22;;;;5020:1;5015:22;;;;5107:1;5097:11;;4876:234;;4941:22;4960:1;4950:11;;4941:22;;5015;5034:1;5024:11;;4876:234;;4825:369;-1:-1:-1;;;5317:14:127;;;5300:32;;5360:19;5356:30;5392:3;5388:16;;;;5353:52;;3087:2334;-1:-1:-1;3087:2334:127:o;21746:1831::-;21819:11;21930:14;21947:24;21959:11;21947;:24::i;:::-;21930:41;;22079:1;22072:5;22068:13;22065:33;;;22094:1;22091;22084:12;22065:33;22227:2;22215:15;;;22168:20;22657:5;22654:1;22650:13;22692:4;22728:1;22713:343;22738:2;22735:1;22732:9;22713:343;;;22861:2;22849:15;;;22798:20;22896:12;;;22910:1;22892:20;22933:42;;;;23001:1;22996:42;;;;22885:153;;22933:42;22391:1;22384:12;;;22424:2;22417:13;;;22469:2;22456:16;;22942:31;;22933:42;;22996;22391:1;22384:12;;;22424:2;22417:13;;;22469:2;22456:16;;23005:31;;22885:153;-1:-1:-1;;22756:1:127;22749:9;22713:343;;;22717:14;23166:4;23160:11;23145:26;;23252:7;23246:4;23243:17;23233:124;;23294:10;23291:1;23284:21;23336:2;23333:1;23326:13;23233:124;-1:-1:-1;;23484:2:127;23473:14;;;;23461:10;23457:31;23454:1;23450:39;23518:16;;;;23536:10;23514:33;;21746:1831;-1:-1:-1;;;21746:1831:127:o;18856:823::-;18925:12;19012:18;;:::i;:::-;19080:4;19071:13;;19132:5;:8;;;19143:1;19132:12;19116:28;;:5;:12;;;:28;;;19112:95;;19164:28;;;;;2182:2:286;19164:28:127;;;2164:21:286;2221:2;2201:18;;;2194:30;2260:20;2240:18;;;2233:48;2298:18;;19164:28:127;;;;;;;;19112:95;19296:8;;;;;19329:12;;;;;19318:23;;;;;;;19355:20;;;;;19296:8;19487:13;;;19483:90;;19548:6;19557:1;19548:10;19520:5;:15;;;19536:8;19520:25;;;;;;;;;:::i;:::-;:38;;;;:25;;;;;;:38;19483:90;19649:13;:11;:13::i;2645:339::-;2706:11;2770:18;;;;2779:8;;;;2770:18;;;;;;2769:25;;;;;2786:1;2833:2;:9;;;2827:16;;;;;2826:22;;2825:32;;;;;;;2887:9;;2886:15;2769:25;2944:21;;2964:1;2944:21;;;2955:6;2944:21;2929:11;;;;;:37;;-1:-1:-1;;;2645:339:127;;;;:::o;13732:2026::-;13829:12;13915:18;;:::i;:::-;13983:4;13974:13;;14015:17;14075:5;:8;;;14086:1;14075:12;14059:28;;:5;:12;;;:28;;;14055:97;;14107:30;;;;;2529:2:286;14107:30:127;;;2511:21:286;2568:2;2548:18;;;2541:30;2607:22;2587:18;;;2580:50;2647:18;;14107:30:127;2327:344:286;14055:97:127;14222:7;:12;;14233:1;14222:12;:28;;;;14238:7;:12;;14249:1;14238:12;14222:28;14218:947;;;14270:9;14282:5;:15;;;14298:6;14282:23;;;;;;;;;:::i;:::-;;;;;14270:35;;14346:2;14339:9;;:3;:9;;;:25;;;;;14352:7;:12;;14363:1;14352:12;14339:25;14338:58;;;;14377:2;14370:9;;:3;:9;;;;:25;;;;;14383:7;:12;;14394:1;14383:12;14370:25;14323:73;;14252:159;14218:947;;;14508:7;:12;;14519:1;14508:12;14504:661;;14569:1;14561:3;14555:15;;;;14540:30;;14504:661;;;14673:7;:12;;14684:1;14673:12;14669:496;;14733:1;14726:3;14720:14;;;14705:29;;14669:496;;;14854:7;:12;;14865:1;14854:12;14850:315;;14942:4;14936:2;14927:11;;;14926:20;14912:10;14969:8;;;14965:84;;15029:1;15022:3;15016:14;;;15001:29;;14965:84;15070:3;:8;;15077:1;15070:8;15066:85;;15131:1;15123:3;15117:15;;;;15102:30;;15066:85;14868:297;14850:315;15241:8;;;;;15319:12;;;;15308:23;;;;;15475:178;;;;15566:1;15540:22;15543:5;15551:6;15543:14;15559:2;15540;:22::i;:::-;:27;;;;;;;15526:42;;15535:1;15526:42;15511:57;:12;;;:57;15475:178;;;15622:12;;;;;15637:1;15622:16;15607:31;;;;15475:178;15728:13;:11;:13::i;:::-;15721:20;13732:2026;-1:-1:-1;;;;;;;;13732:2026:127:o;32450:8733::-;32537:10;32599;32607:2;32599:10;;;;32638:11;;;:44;;;32664:1;32654:6;:11;;;;:27;;;;;32678:3;32669:6;:12;;;32654:27;32634:8490;;;32723:4;32716:11;;32847:6;32907:3;32902:25;;;;32982:3;32977:25;;;;33056:3;33051:25;;;;33131:3;33126:25;;;;33205:3;33200:25;;;;33278:3;33273:25;;;;33352:3;33347:25;;;;32840:532;;32902:25;32921:4;32913:12;;32902:25;;32977;32996:4;32988:12;;32977:25;;33051;33070:4;33062:12;;33051:25;;33126;33145:4;33137:12;;33126:25;;33200;33219:4;33211:12;;33200:25;;33273;33292:4;33284:12;;33273:25;;33347;33366:4;33358:12;;32840:532;;33435:4;:12;;33443:4;33435:12;33431:4023;;-1:-1:-1;;;33486:9:127;33478:26;;33499:4;33494:1;33486:9;;;33485:18;33478:26;33471:33;;33431:4023;33572:4;:12;;33580:4;33572:12;33568:3886;;-1:-1:-1;;;33623:9:127;33615:26;;33636:4;33631:1;33623:9;;;33622:18;33615:26;33608:33;;33568:3886;33709:4;:12;;33717:4;33709:12;33705:3749;;33774:4;33769:1;33761:9;;;33760:18;33807:27;33761:9;33810:11;;;;33823:2;:10;;;33807:2;:27::i;:::-;33800:34;;;;;;;33705:3749;33903:4;:12;;33911:4;33903:12;33899:3555;;-1:-1:-1;;;33946:17:127;;;33958:4;33953:9;;33946:17;33939:24;;33899:3555;34032:4;:11;;34040:3;34032:11;34028:3426;;-1:-1:-1;;;34074:17:127;;;34086:4;34081:9;;34074:17;34067:24;;34028:3426;34160:4;:12;;34168:4;34160:12;34156:3298;;34203:21;34212:2;34206:8;;:2;:8;;;;34221:2;34216;:7;34203:2;:21::i;:::-;34196:28;;;;;;34156:3298;34473:4;:12;;34481:4;34473:12;34469:2985;;34516:2;34509:9;;;;;;34469:2985;34587:4;:12;;34595:4;34587:12;34583:2871;;34630:2;34623:9;;;;;;34583:2871;34701:4;:12;;34709:4;34701:12;34697:2757;;34744:2;34737:9;;;;;;34697:2757;34815:4;:12;;34823:4;34815:12;34811:2643;;34858:2;34851:9;;;;;;34811:2643;34932:4;:12;;34940:4;34932:12;34928:2526;;34975:2;34968:9;;;;;;34928:2526;35092:4;:12;;35100:4;35092:12;35088:2366;;35135:2;35128:9;;;;;;35088:2366;35206:4;:12;;35214:4;35206:12;35202:2252;;35249:2;35242:9;;;;;;35202:2252;35320:4;:12;;35328:4;35320:12;35316:2138;;35363:2;35356:9;;;;;;35316:2138;35434:4;:12;;35442:4;35434:12;35430:2024;;35477:2;35470:9;;;;;;35430:2024;35548:4;:12;;35556:4;35548:12;35544:1910;;35591:2;35584:9;;;;;;35544:1910;35662:4;:12;;35670:4;35662:12;35658:1796;;35705:2;35698:9;;;;;;35658:1796;35777:4;:12;;35785:4;35777:12;35773:1681;;35820:2;35813:9;;;;;;35773:1681;35890:4;:12;;35898:4;35890:12;35886:1568;;35933:2;35926:9;;;;;;35886:1568;36004:4;:12;;36012:4;36004:12;36000:1454;;36047:2;36040:9;;;;;;36000:1454;36196:4;:12;;36204:4;36196:12;36192:1262;;-1:-1:-1;;;36240:7:127;;;36232:16;;36192:1262;36317:4;:12;;36325:4;36317:12;36313:1141;;-1:-1:-1;;;36361:7:127;;;36353:16;;36313:1141;36437:4;:12;;36445:4;36437:12;36433:1021;;-1:-1:-1;;;36481:7:127;;;36473:16;;36433:1021;36558:4;:12;;36566:4;36558:12;36554:900;;-1:-1:-1;;;36602:7:127;;;36594:16;;36554:900;36678:4;:12;;36686:4;36678:12;36674:780;;-1:-1:-1;;;36722:7:127;;;36714:16;;36674:780;36797:4;:12;;36805:4;36797:12;36793:661;;-1:-1:-1;;;36841:7:127;;;36833:16;;36793:661;36917:4;:12;;36925:4;36917:12;36913:541;;-1:-1:-1;;;36961:7:127;;;36953:16;;36913:541;37037:4;:12;;37045:4;37037:12;37033:421;;-1:-1:-1;;;37082:7:127;;;37080:10;37073:17;;37033:421;37159:4;:12;;37167:4;37159:12;37155:299;;37220:2;37202:21;;37208:2;37202:21;;;:29;;37230:1;37202:29;;;37226:1;37202:29;37195:36;;;;;;;;37155:299;37301:4;:12;;37309:4;37301:12;37297:157;;37349:2;37344:7;;:2;:7;;;:15;;37358:1;37344:15;;37297:157;37406:29;;;;;2878:2:286;37406:29:127;;;2860:21:286;2917:2;2897:18;;;2890:30;2956:21;2936:18;;;2929:49;2995:18;;37406:29:127;2676:343:286;37297:157:127;32684:4784;32634:8490;;;37524:6;:14;;37534:4;37524:14;37520:3590;;37583:4;37576:11;;37658:3;37650:11;;;37646:549;;-1:-1:-1;;;37703:21:127;;;37689:36;;37646:549;37810:4;:12;;37818:4;37810:12;:28;;;;37826:4;:12;;37834:4;37826:12;37810:28;37806:389;;;37870:4;:12;;37878:4;37870:12;37866:83;;37919:3;;;37866:83;37974:8;38012:127;38024:10;38019:15;;:20;38012:127;;38104:8;38071:3;38104:8;;;;;38071:3;38012:127;;;38171:1;-1:-1:-1;38164:8:127;;-1:-1:-1;;38164:8:127;37520:3590;38262:6;:14;;38272:4;38262:14;38258:2852;;-1:-1:-1;;38307:8:127;38313:2;38307:8;;;;38300:15;;38258:2852;38382:6;:14;;38392:4;38382:14;38378:2732;;38427:42;38445:2;38450:1;38445:6;38455:1;38444:12;38439:2;:17;38431:26;;:3;:26;;;;38461:4;38430:35;38467:1;38427:2;:42::i;:::-;38420:49;;;;;38378:2732;38536:6;:14;;38546:4;38536:14;38532:2578;;38581:45;38599:2;38604:1;38599:6;38609:1;38598:12;38593:2;:17;38585:26;;:3;:26;;;;38615:6;38584:37;38623:2;38581;:45::i;38532:2578::-;38694:6;:14;;38704:4;38694:14;38690:2420;;-1:-1:-1;;38745:21:127;38764:1;38759;38754:6;;38753:12;38745:21;;38802:36;;;38873:5;38868:10;;38745:21;;;;;38867:18;38860:25;;38690:2420;38952:6;:14;;38962:4;38952:14;38948:2162;;38997:3;38990:10;;;;;38948:2162;39068:6;:14;;39078:4;39068:14;39064:2046;;39128:2;39133:1;39128:6;39138:1;39127:12;39122:2;:17;39114:26;;:3;:26;;;;39144:4;39113:35;39106:42;;;;;39064:2046;39217:6;:14;;39227:4;39217:14;39213:1897;;39277:2;39282:1;39277:6;39287:1;39276:12;39271:2;:17;39263:26;;:3;:26;;;;39293:6;39262:37;39255:44;;;;;39213:1897;39368:6;:14;;39378:4;39368:14;39364:1746;;-1:-1:-1;;39419:26:127;39443:1;39438;39433:6;;39432:12;39427:2;:17;39419:26;;39481:41;;;39557:5;39552:10;;39419:26;;;;;39551:18;39544:25;;39364:1746;39637:6;:14;;39647:4;39637:14;39633:1477;;-1:-1:-1;;39694:4:127;39688:34;39720:1;39715;39710:6;;39709:12;39704:2;:17;39688:34;;39778:27;;;39758:48;;;39836:10;;39689:9;;;39688:34;;39835:18;39828:25;;39633:1477;39921:6;:14;;39931:4;39921:14;39917:1193;;-1:-1:-1;;39978:6:127;39972:36;40006:1;40001;39996:6;;39995:12;39990:2;:17;39972:36;;40064:29;;;40044:50;;;40124:10;;39973:11;;;39972:36;;40123:18;40116:25;;39917:1193;40210:6;:14;;40220:4;40210:14;40206:904;;-1:-1:-1;;40261:20:127;40279:1;40274;40269:6;;40268:12;40261:20;;40317:36;;;40389:5;40383:11;;40261:20;;;;;40382:19;40375:26;;40206:904;40469:6;:14;;40479:4;40469:14;40465:645;;40514:2;40507:9;;;;;40465:645;40585:6;:14;;40595:4;40585:14;40581:529;;-1:-1:-1;;40636:25:127;40659:1;40654;40649:6;;40648:12;40643:2;:17;40636:25;;40697:41;;;40774:5;40768:11;;40636:25;;;;;40767:19;40760:26;;40581:529;40853:6;:14;;40863:4;40853:14;40849:261;;40898:3;40891:10;;;;;40849:261;40968:6;:14;;40978:4;40968:14;40964:146;;41013:2;41006:9;;;32450:8733;;;;;;;:::o;19960:782::-;20046:12;20133:18;;:::i;:::-;-1:-1:-1;20201:4:127;20308:2;20296:14;;;;20288:41;;;;;;;3226:2:286;20288:41:127;;;3208:21:286;3265:2;3245:18;;;3238:30;3304:16;3284:18;;;3277:44;3338:18;;20288:41:127;3024:338:286;20288:41:127;20425:14;;;;;;;:30;;;20443:12;20425:30;20421:102;;;20504:4;20475:5;:15;;;20491:9;20475:26;;;;;;;;;:::i;:::-;:33;;;;:26;;;;;;:33;20421:102;20578:12;;;;;20567:23;;;;:8;;;:23;20634:1;20619:16;;;20604:31;;;20712:13;:11;:13::i;5582:7764::-;5646:12;5732:18;;:::i;:::-;-1:-1:-1;5910:15:127;;:18;;;;5800:4;6070:18;;;;6114;;;;6158;;;;;5800:4;;5890:17;;;;6070:18;6114;6248;;;6262:4;6248:18;6244:6792;;6298:2;6327:4;6322:9;;:14;6318:144;;6438:4;6433:9;;6425:4;:18;6419:24;6318:144;6483:2;:7;;6489:1;6483:7;6479:161;;6519:10;;;;;6551:16;;;;;;;;6519:10;-1:-1:-1;6479:161:127;;;6619:2;6614:7;;6479:161;6268:386;6244:6792;;;6756:10;:18;;6770:4;6756:18;6752:6284;;1745:10;6794:14;;6752:6284;;;6892:10;:18;;6906:4;6892:18;6888:6148;;6935:1;6930:6;;6888:6148;;;7060:10;:18;;7074:4;7060:18;7056:5980;;7113:4;7098:12;;;:19;7135:26;;;:14;;;:26;7186:13;:11;:13::i;:::-;7179:20;5582:7764;-1:-1:-1;;;;;;;;;5582:7764:127:o;7056:5980::-;7325:10;:18;;7339:4;7325:18;7321:5715;;7476:14;;;7472:2723;7321:5715;7472:2723;7646:22;;;;;7642:2553;;7771:10;7784:27;7792:2;7797:10;7792:15;7809:1;7784:7;:27::i;:::-;7895:17;;;;7771:40;;-1:-1:-1;7895:17:127;7873:19;8045:14;8064:1;8039:26;8035:146;;1676:4:128;1670:11;;1533:21;1787:15;;;1828:8;1822:4;1815:22;1850:27;;;1996:4;1983:18;;2098:17;;2003:19;1979:44;2025:11;1976:61;8093:65:127;;8035:146;8267:20;;;;;8234:54;;;;;;;;3540:25:286;;;8234:54:127;3601:23:286;;;3581:18;;;3574:51;8203:11:127;;;;8234:19;:6;:19;;;;3513:18:286;;8234:54:127;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;:::i;:::-;8202:86;;;;8515:1;8511:2;8507:10;8612:9;8609:1;8605:17;8694:6;8687:5;8684:17;8681:40;;;8714:5;8704:15;;8681:40;;8797:6;8793:2;8790:14;8787:34;;;8817:2;8807:12;;8787:34;8923:3;8918:1;8910:6;8906:14;8901:3;8897:24;8893:34;8886:41;;9023:3;9019:1;9007:9;8998:6;8995:1;8991:14;8987:30;8983:38;8979:48;8972:55;;9178:1;9174;9170;9158:9;9155:1;9151:17;9147:25;9143:33;9139:41;9305:1;9301;9297;9288:6;9276:9;9273:1;9269:17;9265:30;9261:38;9257:46;9253:54;9235:72;;9436:10;9432:15;9426:4;9422:26;9414:34;;9552:3;9544:4;9540:9;9535:3;9531:19;9528:28;9521:35;;;;9698:33;9707:2;9712:10;9707:15;9724:1;9727:3;9698:8;:33::i;:::-;9753:20;;;:38;;;;;;;;;-1:-1:-1;7642:2553:127;;-1:-1:-1;;;7642:2553:127;;9910:18;;;;;9906:289;;10080:2;10075:7;;7321:5715;;9906:289;10134:10;10129:15;;2053:3;10166:10;;9906:289;7321:5715;;;10324:10;:18;;10338:4;10324:18;10320:2716;;10478:15;;;1824:1;10478:15;;:34;;-1:-1:-1;10497:15:127;;;1859:1;10497:15;10478:34;:57;;;-1:-1:-1;10516:19:127;;;1936:1;10516:19;10478:57;10474:1593;;;10564:2;10559:7;;10320:2716;;10474:1593;10690:23;;;;;10686:1381;;10737:10;10750:27;10758:2;10763:10;10758:15;10775:1;10750:7;:27::i;:::-;10853:17;;;;10737:40;;-1:-1:-1;11096:1:127;11088:10;;11190:1;11186:17;11265:13;;;11262:32;;;11287:5;11281:11;;11262:32;11573:14;;;11379:1;11569:22;;;11565:32;;;;11462:26;11486:1;11371:10;;;11466:18;;;11462:26;11561:43;11367:20;;11669:12;11797:17;;;:23;11865:1;11842:20;;;:24;11375:2;-1:-1:-1;11375:2:127;7321:5715;;10320:2716;12269:10;:18;;12283:4;12269:18;12265:771;;12379:2;:7;;12385:1;12379:7;12375:647;;12472:14;;;;;:40;;-1:-1:-1;12490:22:127;;;1978:1;12490:22;12472:40;:62;;;-1:-1:-1;12516:18:127;;;1897:1;12516:18;12472:62;12468:404;;;12567:1;12562:6;;12375:647;;12468:404;12613:15;;;1824:1;12613:15;;:34;;-1:-1:-1;12632:15:127;;;1859:1;12632:15;12613:34;:61;;;-1:-1:-1;12651:23:127;;;2021:1;12651:23;12613:61;:84;;;-1:-1:-1;12678:19:127;;;1936:1;12678:19;12613:84;12609:263;;;12730:1;12725:6;;7321:5715;;12375:647;12923:10;12918:15;;2087:4;12955:11;;12375:647;13111:15;;;;;:23;;;;:18;;;;:23;;;;13148:15;;:23;;;:18;;;;:23;-1:-1:-1;13237:12:127;;;;13226:23;;;:8;;;:23;13293:1;13278:16;13263:31;;;;;13316:13;:11;:13::i;16084:2480::-;16178:12;16264:18;;:::i;:::-;-1:-1:-1;16332:4:127;16364:10;16472:13;;;16481:4;16472:13;16468:1705;;-1:-1:-1;16511:8:127;;;;16468:1705;;;16630:5;:13;;16639:4;16630:13;16626:1547;;16663:14;;;:8;;;:14;16626:1547;;;16793:5;:13;;16802:4;16793:13;16789:1384;;-1:-1:-1;16832:8:127;;;;16789:1384;;;16951:5;:13;;16960:4;16951:13;16947:1226;;16984:14;;;:8;;;:14;16947:1226;;;17125:5;:13;;17134:4;17125:13;17121:1052;;17252:9;17198:17;17178;;;17198;;;;17178:37;17259:2;17252:9;;;;;17234:8;;;:28;17280:22;:8;;;:22;17121:1052;;;17439:5;:13;;17448:4;17439:13;17435:738;;17506:11;17492;;;17506;;;17492:25;17561:2;17554:9;;;;;17536:8;;;:28;17582:22;:8;;;:22;17435:738;;;17763:5;:13;;17772:4;17763:13;17759:414;;17833:3;17814:23;;17820:3;17814:23;;;;;;;:::i;:::-;;17796:42;;:8;;;:42;17874:23;;;;;;;;;;;;;:::i;:::-;;17856:42;;:8;;;:42;17759:414;;;18067:5;:13;;18076:4;18067:13;18063:110;;18117:3;18111:9;;:3;:9;;;;;;;:::i;:::-;;18100:20;;;;:8;;;:20;18149:9;;;;;;;;;;;:::i;:::-;;18138:20;;:8;;;:20;18063:110;18266:14;;;;18262:85;;18329:3;18300:5;:15;;;18316:9;18300:26;;;;;;;;;:::i;:::-;:32;;;;:26;;;;;;:32;18262:85;18401:12;;;;;18390:23;;;;:8;;;:23;18457:1;18442:16;;;18427:31;;;18534:13;:11;:13::i;:::-;18527:20;16084:2480;-1:-1:-1;;;;;;;16084:2480:127:o;23913:1654::-;24089:14;24106:24;24118:11;24106;:24::i;:::-;24089:41;;24238:1;24231:5;24227:13;24224:33;;;24253:1;24250;24243:12;24224:33;24392:2;24586:15;;;24411:2;24400:14;;24388:10;24384:31;24381:1;24377:39;24542:16;;;24327:20;;24527:10;24516:22;;;24512:27;24502:38;24499:60;25028:5;25025:1;25021:13;25099:1;25084:343;25109:2;25106:1;25103:9;25084:343;;;25232:2;25220:15;;;25169:20;25267:12;;;25281:1;25263:20;25304:42;;;;25372:1;25367:42;;;;25256:153;;25304:42;22391:1;22384:12;;;22424:2;22417:13;;;22469:2;22456:16;;25313:31;;25304:42;;25367;22391:1;22384:12;;;22424:2;22417:13;;;22469:2;22456:16;;25376:31;;25256:153;-1:-1:-1;;25127:1:127;25120:9;25084:343;;;-1:-1:-1;;25526:4:127;25519:18;-1:-1:-1;;;;23913:1654:127:o;20946:586::-;21268:20;;;21292:7;21268:32;21261:3;:40;;;21374:14;;21429:17;;21423:24;;;21415:72;;;;;;;4277:2:286;21415:72:127;;;4259:21:286;4316:2;4296:18;;;4289:30;4355:34;4335:18;;;4328:62;4426:5;4406:18;;;4399:33;4449:19;;21415:72:127;4075:399:286;21415:72:127;21501:14;20946:586;;;:::o;-1:-1:-1:-;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;:::i;:::-;;;;:::o;:::-;;;;;;;;;;;;;;;;;;;;;;;;:::o;467:347:286:-;518:8;528:6;582:3;575:4;567:6;563:17;559:27;549:55;;600:1;597;590:12;549:55;-1:-1:-1;623:20:286;;666:18;655:30;;652:50;;;698:1;695;688:12;652:50;735:4;727:6;723:17;711:29;;787:3;780:4;771:6;763;759:19;755:30;752:39;749:59;;;804:1;801;794:12;749:59;467:347;;;;;:::o;819:785::-;918:6;926;934;942;950;1003:2;991:9;982:7;978:23;974:32;971:52;;;1019:1;1016;1009:12;971:52;1059:9;1046:23;1088:18;1129:2;1121:6;1118:14;1115:34;;;1145:1;1142;1135:12;1115:34;1184:58;1234:7;1225:6;1214:9;1210:22;1184:58;:::i;:::-;1261:8;;-1:-1:-1;1158:84:286;-1:-1:-1;1349:2:286;1334:18;;1321:32;;-1:-1:-1;1365:16:286;;;1362:36;;;1394:1;1391;1384:12;1362:36;;1433:60;1485:7;1474:8;1463:9;1459:24;1433:60;:::i;:::-;819:785;;;;-1:-1:-1;1512:8:286;1594:2;1579:18;1566:32;;819:785;-1:-1:-1;;;;819:785:286:o;1791:184::-;1843:77;1840:1;1833:88;1940:4;1937:1;1930:15;1964:4;1961:1;1954:15;3636:245;3715:6;3723;3776:2;3764:9;3755:7;3751:23;3747:32;3744:52;;;3792:1;3789;3782:12;3744:52;-1:-1:-1;;3815:16:286;;3871:2;3856:18;;;3850:25;3815:16;;3850:25;;-1:-1:-1;3636:245:286:o;3886:184::-;3938:77;3935:1;3928:88;4035:4;4032:1;4025:15;4059:4;4056:1;4049:15" func init() { if err := json.Unmarshal([]byte(MIPSStorageLayoutJSON), MIPSStorageLayout); err != nil { diff --git a/op-bindings/bindings/preimageoracle_more.go b/op-bindings/bindings/preimageoracle_more.go index 2c53d11a0d7f..54abd4665e9b 100644 --- a/op-bindings/bindings/preimageoracle_more.go +++ b/op-bindings/bindings/preimageoracle_more.go @@ -15,7 +15,7 @@ var PreimageOracleStorageLayout = new(solc.StorageLayout) var PreimageOracleDeployedBin = "0x608060405234801561001057600080fd5b50600436106100725760003560e01c8063e03110e111610050578063e03110e114610106578063e15926111461012e578063fef2b4ed1461014357600080fd5b806361238bde146100775780638542cf50146100b5578063c0c220c9146100f3575b600080fd5b6100a26100853660046104df565b600160209081526000928352604080842090915290825290205481565b6040519081526020015b60405180910390f35b6100e36100c33660046104df565b600260209081526000928352604080842090915290825290205460ff1681565b60405190151581526020016100ac565b6100a2610101366004610501565b610163565b6101196101143660046104df565b610238565b604080519283526020830191909152016100ac565b61014161013c36600461053c565b610329565b005b6100a26101513660046105b8565b60006020819052908152604090205481565b600061016f8686610432565b905061017c836008610600565b8211806101895750602083115b156101c0576040517ffe25498700000000000000000000000000000000000000000000000000000000815260040160405180910390fd5b6000602081815260c085901b82526008959095528251828252600286526040808320858452875280832080547fffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff001660019081179091558484528752808320948352938652838220558181529384905292205592915050565b6000828152600260209081526040808320848452909152812054819060ff166102c1576040517f08c379a000000000000000000000000000000000000000000000000000000000815260206004820152601460248201527f7072652d696d616765206d757374206578697374000000000000000000000000604482015260640160405180910390fd5b50600083815260208181526040909120546102dd816008610600565b6102e8856020610600565b1061030657836102f9826008610600565b6103039190610618565b91505b506000938452600160209081526040808620948652939052919092205492909150565b604435600080600883018611156103485763fe2549876000526004601cfd5b60c083901b6080526088838682378087017ffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff80151908490207effffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff167f02000000000000000000000000000000000000000000000000000000000000001760008181526002602090815260408083208b8452825280832080547fffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff0016600190811790915584845282528083209a83529981528982209390935590815290819052959095209190915550505050565b7f01000000000000000000000000000000000000000000000000000000000000007effffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff8316176104d8818360408051600093845233602052918152606090922091527effffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff167f01000000000000000000000000000000000000000000000000000000000000001790565b9392505050565b600080604083850312156104f257600080fd5b50508035926020909101359150565b600080600080600060a0868803121561051957600080fd5b505083359560208501359550604085013594606081013594506080013592509050565b60008060006040848603121561055157600080fd5b83359250602084013567ffffffffffffffff8082111561057057600080fd5b818601915086601f83011261058457600080fd5b81358181111561059357600080fd5b8760208285010111156105a557600080fd5b6020830194508093505050509250925092565b6000602082840312156105ca57600080fd5b5035919050565b7f4e487b7100000000000000000000000000000000000000000000000000000000600052601160045260246000fd5b60008219821115610613576106136105d1565b500190565b60008282101561062a5761062a6105d1565b50039056fea164736f6c634300080f000a" -var PreimageOracleDeployedSourceMap = "306:3911:130:-:0;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;537:68;;;;;;:::i;:::-;;;;;;;;;;;;;;;;;;;;;;;;;;;;413:25:287;;;401:2;386:18;537:68:130;;;;;;;;680:66;;;;;;:::i;:::-;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;614:14:287;;607:22;589:41;;577:2;562:18;680:66:130;449:187:287;1367:1211:130;;;;;;:::i;:::-;;:::i;789:536::-;;;;;;:::i;:::-;;:::i;:::-;;;;1274:25:287;;;1330:2;1315:18;;1308:34;;;;1247:18;789:536:130;1100:248:287;2620:1595:130;;;;;;:::i;:::-;;:::i;:::-;;419:50;;;;;;:::i;:::-;;;;;;;;;;;;;;;1367:1211;1560:12;1665:51;1694:6;1702:13;1665:28;:51::i;:::-;1658:58;-1:-1:-1;1810:9:130;:5;1818:1;1810:9;:::i;:::-;1796:11;:23;:37;;;;1831:2;1823:5;:10;1796:37;1792:90;;;1856:15;;;;;;;;;;;;;;1792:90;1951:12;2051:4;2044:18;;;2152:3;2148:15;;;2135:29;;2184:4;2177:19;;;;2286:18;;2376:20;;;:14;:20;;;;;;:33;;;;;;;;:40;;;;2412:4;2376:40;;;;;;2426:19;;;;;;;;:32;;;;;;;;;:39;2542:21;;;;;;;;;:29;2391:4;1367:1211;-1:-1:-1;;1367:1211:130:o;789:536::-;865:12;914:20;;;:14;:20;;;;;;;;:29;;;;;;;;;865:12;;914:29;;906:62;;;;;;;2908:2:287;906:62:130;;;2890:21:287;2947:2;2927:18;;;2920:30;2986:22;2966:18;;;2959:50;3026:18;;906:62:130;;;;;;;;-1:-1:-1;1099:14:130;1116:21;;;1087:2;1116:21;;;;;;;;1167:10;1116:21;1176:1;1167:10;:::i;:::-;1151:12;:7;1161:2;1151:12;:::i;:::-;:26;1147:87;;1216:7;1203:10;:6;1212:1;1203:10;:::i;:::-;:20;;;;:::i;:::-;1193:30;;1147:87;-1:-1:-1;1290:19:130;;;;:13;:19;;;;;;;;:28;;;;;;;;;;;;789:536;;-1:-1:-1;789:536:130:o;2620:1595::-;2916:4;2903:18;2721:12;;3045:1;3035:12;;3019:29;;3016:210;;;3120:10;3117:1;3110:21;3210:1;3204:4;3197:15;3016:210;3469:3;3465:14;;;3369:4;3453:27;3500:11;3474:4;3619:16;3500:11;3601:41;3832:29;;;3836:11;3832:29;3826:36;3884:20;;;;4031:19;4024:27;4053:11;4021:44;4084:19;;;;4062:1;4084:19;;;;;;;;:32;;;;;;;;:39;;;;4119:4;4084:39;;;;;;4133:18;;;;;;;;:31;;;;;;;;;:38;;;;4181:20;;;;;;;;;;;:27;;;;-1:-1:-1;;;;2620:1595:130:o;552:449:129:-;835:11;860:19;848:32;;832:49;965:29;832:49;980:13;1676:4;1670:11;;1533:21;1787:15;;;1828:8;1822:4;1815:22;1850:27;;;1996:4;1983:18;;;2098:17;;2003:19;1979:44;2025:11;1976:61;;1455:676;965:29;958:36;552:449;-1:-1:-1;;;552:449:129:o;14:248:287:-;82:6;90;143:2;131:9;122:7;118:23;114:32;111:52;;;159:1;156;149:12;111:52;-1:-1:-1;;182:23:287;;;252:2;237:18;;;224:32;;-1:-1:-1;14:248:287:o;641:454::-;736:6;744;752;760;768;821:3;809:9;800:7;796:23;792:33;789:53;;;838:1;835;828:12;789:53;-1:-1:-1;;861:23:287;;;931:2;916:18;;903:32;;-1:-1:-1;982:2:287;967:18;;954:32;;1033:2;1018:18;;1005:32;;-1:-1:-1;1084:3:287;1069:19;1056:33;;-1:-1:-1;641:454:287;-1:-1:-1;641:454:287:o;1353:659::-;1432:6;1440;1448;1501:2;1489:9;1480:7;1476:23;1472:32;1469:52;;;1517:1;1514;1507:12;1469:52;1553:9;1540:23;1530:33;;1614:2;1603:9;1599:18;1586:32;1637:18;1678:2;1670:6;1667:14;1664:34;;;1694:1;1691;1684:12;1664:34;1732:6;1721:9;1717:22;1707:32;;1777:7;1770:4;1766:2;1762:13;1758:27;1748:55;;1799:1;1796;1789:12;1748:55;1839:2;1826:16;1865:2;1857:6;1854:14;1851:34;;;1881:1;1878;1871:12;1851:34;1926:7;1921:2;1912:6;1908:2;1904:15;1900:24;1897:37;1894:57;;;1947:1;1944;1937:12;1894:57;1978:2;1974;1970:11;1960:21;;2000:6;1990:16;;;;;1353:659;;;;;:::o;2017:180::-;2076:6;2129:2;2117:9;2108:7;2104:23;2100:32;2097:52;;;2145:1;2142;2135:12;2097:52;-1:-1:-1;2168:23:287;;2017:180;-1:-1:-1;2017:180:287:o;2384:184::-;2436:77;2433:1;2426:88;2533:4;2530:1;2523:15;2557:4;2554:1;2547:15;2573:128;2613:3;2644:1;2640:6;2637:1;2634:13;2631:39;;;2650:18;;:::i;:::-;-1:-1:-1;2686:9:287;;2573:128::o;3055:125::-;3095:4;3123:1;3120;3117:8;3114:34;;;3128:18;;:::i;:::-;-1:-1:-1;3165:9:287;;3055:125::o" +var PreimageOracleDeployedSourceMap = "306:3911:129:-:0;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;537:68;;;;;;:::i;:::-;;;;;;;;;;;;;;;;;;;;;;;;;;;;413:25:286;;;401:2;386:18;537:68:129;;;;;;;;680:66;;;;;;:::i;:::-;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;614:14:286;;607:22;589:41;;577:2;562:18;680:66:129;449:187:286;1367:1211:129;;;;;;:::i;:::-;;:::i;789:536::-;;;;;;:::i;:::-;;:::i;:::-;;;;1274:25:286;;;1330:2;1315:18;;1308:34;;;;1247:18;789:536:129;1100:248:286;2620:1595:129;;;;;;:::i;:::-;;:::i;:::-;;419:50;;;;;;:::i;:::-;;;;;;;;;;;;;;;1367:1211;1560:12;1665:51;1694:6;1702:13;1665:28;:51::i;:::-;1658:58;-1:-1:-1;1810:9:129;:5;1818:1;1810:9;:::i;:::-;1796:11;:23;:37;;;;1831:2;1823:5;:10;1796:37;1792:90;;;1856:15;;;;;;;;;;;;;;1792:90;1951:12;2051:4;2044:18;;;2152:3;2148:15;;;2135:29;;2184:4;2177:19;;;;2286:18;;2376:20;;;:14;:20;;;;;;:33;;;;;;;;:40;;;;2412:4;2376:40;;;;;;2426:19;;;;;;;;:32;;;;;;;;;:39;2542:21;;;;;;;;;:29;2391:4;1367:1211;-1:-1:-1;;1367:1211:129:o;789:536::-;865:12;914:20;;;:14;:20;;;;;;;;:29;;;;;;;;;865:12;;914:29;;906:62;;;;;;;2908:2:286;906:62:129;;;2890:21:286;2947:2;2927:18;;;2920:30;2986:22;2966:18;;;2959:50;3026:18;;906:62:129;;;;;;;;-1:-1:-1;1099:14:129;1116:21;;;1087:2;1116:21;;;;;;;;1167:10;1116:21;1176:1;1167:10;:::i;:::-;1151:12;:7;1161:2;1151:12;:::i;:::-;:26;1147:87;;1216:7;1203:10;:6;1212:1;1203:10;:::i;:::-;:20;;;;:::i;:::-;1193:30;;1147:87;-1:-1:-1;1290:19:129;;;;:13;:19;;;;;;;;:28;;;;;;;;;;;;789:536;;-1:-1:-1;789:536:129:o;2620:1595::-;2916:4;2903:18;2721:12;;3045:1;3035:12;;3019:29;;3016:210;;;3120:10;3117:1;3110:21;3210:1;3204:4;3197:15;3016:210;3469:3;3465:14;;;3369:4;3453:27;3500:11;3474:4;3619:16;3500:11;3601:41;3832:29;;;3836:11;3832:29;3826:36;3884:20;;;;4031:19;4024:27;4053:11;4021:44;4084:19;;;;4062:1;4084:19;;;;;;;;:32;;;;;;;;:39;;;;4119:4;4084:39;;;;;;4133:18;;;;;;;;:31;;;;;;;;;:38;;;;4181:20;;;;;;;;;;;:27;;;;-1:-1:-1;;;;2620:1595:129:o;552:449:128:-;835:11;860:19;848:32;;832:49;965:29;832:49;980:13;1676:4;1670:11;;1533:21;1787:15;;;1828:8;1822:4;1815:22;1850:27;;;1996:4;1983:18;;;2098:17;;2003:19;1979:44;2025:11;1976:61;;1455:676;965:29;958:36;552:449;-1:-1:-1;;;552:449:128:o;14:248:286:-;82:6;90;143:2;131:9;122:7;118:23;114:32;111:52;;;159:1;156;149:12;111:52;-1:-1:-1;;182:23:286;;;252:2;237:18;;;224:32;;-1:-1:-1;14:248:286:o;641:454::-;736:6;744;752;760;768;821:3;809:9;800:7;796:23;792:33;789:53;;;838:1;835;828:12;789:53;-1:-1:-1;;861:23:286;;;931:2;916:18;;903:32;;-1:-1:-1;982:2:286;967:18;;954:32;;1033:2;1018:18;;1005:32;;-1:-1:-1;1084:3:286;1069:19;1056:33;;-1:-1:-1;641:454:286;-1:-1:-1;641:454:286:o;1353:659::-;1432:6;1440;1448;1501:2;1489:9;1480:7;1476:23;1472:32;1469:52;;;1517:1;1514;1507:12;1469:52;1553:9;1540:23;1530:33;;1614:2;1603:9;1599:18;1586:32;1637:18;1678:2;1670:6;1667:14;1664:34;;;1694:1;1691;1684:12;1664:34;1732:6;1721:9;1717:22;1707:32;;1777:7;1770:4;1766:2;1762:13;1758:27;1748:55;;1799:1;1796;1789:12;1748:55;1839:2;1826:16;1865:2;1857:6;1854:14;1851:34;;;1881:1;1878;1871:12;1851:34;1926:7;1921:2;1912:6;1908:2;1904:15;1900:24;1897:37;1894:57;;;1947:1;1944;1937:12;1894:57;1978:2;1974;1970:11;1960:21;;2000:6;1990:16;;;;;1353:659;;;;;:::o;2017:180::-;2076:6;2129:2;2117:9;2108:7;2104:23;2100:32;2097:52;;;2145:1;2142;2135:12;2097:52;-1:-1:-1;2168:23:286;;2017:180;-1:-1:-1;2017:180:286:o;2384:184::-;2436:77;2433:1;2426:88;2533:4;2530:1;2523:15;2557:4;2554:1;2547:15;2573:128;2613:3;2644:1;2640:6;2637:1;2634:13;2631:39;;;2650:18;;:::i;:::-;-1:-1:-1;2686:9:286;;2573:128::o;3055:125::-;3095:4;3123:1;3120;3117:8;3114:34;;;3128:18;;:::i;:::-;-1:-1:-1;3165:9:286;;3055:125::o" func init() { if err := json.Unmarshal([]byte(PreimageOracleStorageLayoutJSON), PreimageOracleStorageLayout); err != nil { From 329bec4998ac21ffd445c74a717f05d15def6eee Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Thu, 26 Oct 2023 12:22:50 +1000 Subject: [PATCH 102/279] op-e2e: Add waiter to ActL1IncludeTx This mirrors a change to the L2 block builder to wait for the tx pool to have a processable transaction. The txpool does promotion async so a simple check leads to race conditions. Fixes intermittency in TestFinalizeWhileSyncing --- op-e2e/actions/l1_miner.go | 21 ++++++++++++++++----- 1 file changed, 16 insertions(+), 5 deletions(-) diff --git a/op-e2e/actions/l1_miner.go b/op-e2e/actions/l1_miner.go index 70c5059a16c0..fecfb1532aaf 100644 --- a/op-e2e/actions/l1_miner.go +++ b/op-e2e/actions/l1_miner.go @@ -1,8 +1,11 @@ package actions import ( + "context" "math/big" + "time" + "github.com/ethereum-optimism/optimism/op-e2e/e2eutils/wait" "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/consensus/misc/eip1559" "github.com/ethereum/go-ethereum/core" @@ -95,11 +98,19 @@ func (s *L1Miner) ActL1IncludeTx(from common.Address) Action { t.InvalidAction("no tx inclusion when not building l1 block") return } - i := s.pendingIndices[from] - txs, q := s.eth.TxPool().ContentFrom(from) - if uint64(len(txs)) <= i { - t.Fatalf("no pending txs from %s, and have %d unprocessable queued txs from this account", from, len(q)) - } + var i uint64 + var txs []*types.Transaction + var q []*types.Transaction + // Wait for the tx to be in the pending tx queue + ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) + defer cancel() + err := wait.For(ctx, time.Second, func() (bool, error) { + i = s.pendingIndices[from] + txs, q = s.eth.TxPool().ContentFrom(from) + return uint64(len(txs)) > i, nil + }) + require.NoError(t, err, + "no pending txs from %s, and have %d unprocessable queued txs from this account: %w", from, len(q), err) tx := txs[i] s.IncludeTx(t, tx) s.pendingIndices[from] = i + 1 // won't retry the tx From ef3ecc2e585cd372940d3fce64d291abead5a0fb Mon Sep 17 00:00:00 2001 From: Joshua Gutow Date: Wed, 25 Oct 2023 22:45:52 -0700 Subject: [PATCH 103/279] Enable Canyon in the devnet This does several things to enable Canyon 1. Update to op-geth which has all canyon changes enabled in it 2. Thread the new 1559 config change through the devnet config 3. Fix some EIP-1559 calculations where were use CalcBaseFee 4. Add the PostCanyonDenominator in several deploy configs --- go.mod | 2 +- go.sum | 4 ++-- op-chain-ops/genesis/config.go | 5 +++++ op-chain-ops/genesis/genesis.go | 9 +++++++-- .../genesis/testdata/test-deploy-config-full.json | 1 + op-e2e/actions/l1_miner.go | 2 +- op-program/client/l2/engineapi/block_processor.go | 2 +- .../deploy-config/devnetL1-template.json | 4 +++- .../contracts-bedrock/deploy-config/getting-started.json | 1 + packages/contracts-bedrock/deploy-config/goerli.json | 1 + packages/contracts-bedrock/deploy-config/hardhat.json | 1 + 11 files changed, 24 insertions(+), 8 deletions(-) diff --git a/go.mod b/go.mod index 3713a12490c6..370e4dddcd44 100644 --- a/go.mod +++ b/go.mod @@ -209,7 +209,7 @@ require ( rsc.io/tmplfunc v0.0.3 // indirect ) -replace github.com/ethereum/go-ethereum v1.13.1 => github.com/ethereum-optimism/op-geth v1.101303.0-rc.2.0.20231024150425-5023660bf92d +replace github.com/ethereum/go-ethereum v1.13.1 => github.com/ethereum-optimism/op-geth v1.101303.0-rc.2.0.20231024175019-29cd9a353f83 //replace github.com/ethereum-optimism/superchain-registry/superchain => ../superchain-registry/superchain //replace github.com/ethereum/go-ethereum v1.13.1 => ../go-ethereum diff --git a/go.sum b/go.sum index 1627f592ddca..45a78c60555e 100644 --- a/go.sum +++ b/go.sum @@ -151,8 +151,8 @@ github.com/elastic/gosigar v0.14.2 h1:Dg80n8cr90OZ7x+bAax/QjoW/XqTI11RmA79ZwIm9/ github.com/elastic/gosigar v0.14.2/go.mod h1:iXRIGg2tLnu7LBdpqzyQfGDEidKCfWcCMS0WKyPWoMs= github.com/ethereum-optimism/go-ethereum-hdwallet v0.1.3 h1:RWHKLhCrQThMfch+QJ1Z8veEq5ZO3DfIhZ7xgRP9WTc= github.com/ethereum-optimism/go-ethereum-hdwallet v0.1.3/go.mod h1:QziizLAiF0KqyLdNJYD7O5cpDlaFMNZzlxYNcWsJUxs= -github.com/ethereum-optimism/op-geth v1.101303.0-rc.2.0.20231024150425-5023660bf92d h1:5dptu9FNKPcZ3o5h2incAbQWH411Mw4HKlsZ4sF4llY= -github.com/ethereum-optimism/op-geth v1.101303.0-rc.2.0.20231024150425-5023660bf92d/go.mod h1:hl28ffXoV4maInP7dvhvNgDO79Q5M3MEYrPZZO6u3W8= +github.com/ethereum-optimism/op-geth v1.101303.0-rc.2.0.20231024175019-29cd9a353f83 h1:RFKnTUJqbYM8+dueFcGPdOY0ycrOhxp0HQJyy2OYzvc= +github.com/ethereum-optimism/op-geth v1.101303.0-rc.2.0.20231024175019-29cd9a353f83/go.mod h1:hl28ffXoV4maInP7dvhvNgDO79Q5M3MEYrPZZO6u3W8= github.com/ethereum-optimism/superchain-registry/superchain v0.0.0-20231018202221-fdba3d104171 h1:MjCUj16JSLZRDnQQ6OOUy6Chfb4dKo7ahFceNi0RKZ8= github.com/ethereum-optimism/superchain-registry/superchain v0.0.0-20231018202221-fdba3d104171/go.mod h1:/70H/KqrtKcvWvNGVj6S3rAcLC+kUPr3t2aDmYIS+Xk= github.com/ethereum/c-kzg-4844 v0.3.1 h1:sR65+68+WdnMKxseNWxSJuAv2tsUrihTpVBTfM/U5Zg= diff --git a/op-chain-ops/genesis/config.go b/op-chain-ops/genesis/config.go index b706f59dd7cd..0e1e4dcfa581 100644 --- a/op-chain-ops/genesis/config.go +++ b/op-chain-ops/genesis/config.go @@ -185,6 +185,8 @@ type DeployConfig struct { EIP1559Elasticity uint64 `json:"eip1559Elasticity"` // EIP1559Denominator is the denominator of EIP1559 base fee market. EIP1559Denominator uint64 `json:"eip1559Denominator"` + // EIP1559DenominatorCanyon is the denominator of EIP1559 base fee market when Canyon is active. + EIP1559DenominatorCanyon uint64 `json:"eip1559DenominatorCanyon"` // SystemConfigStartBlock represents the block at which the op-node should start syncing // from. It is an override to set this value on legacy networks where it is not set by // default. It can be removed once all networks have this value set in their storage. @@ -318,6 +320,9 @@ func (d *DeployConfig) Check() error { if d.EIP1559Denominator == 0 { return fmt.Errorf("%w: EIP1559Denominator cannot be 0", ErrInvalidDeployConfig) } + if d.L2GenesisCanyonTimeOffset != nil && d.EIP1559DenominatorCanyon == 0 { + return fmt.Errorf("%w: EIP1559DenominatorCanyon cannot be 0 if Canyon is activated", ErrInvalidDeployConfig) + } if d.EIP1559Elasticity == 0 { return fmt.Errorf("%w: EIP1559Elasticity cannot be 0", ErrInvalidDeployConfig) } diff --git a/op-chain-ops/genesis/genesis.go b/op-chain-ops/genesis/genesis.go index adae3c489ce5..7a4df6b1ad49 100644 --- a/op-chain-ops/genesis/genesis.go +++ b/op-chain-ops/genesis/genesis.go @@ -31,6 +31,10 @@ func NewL2Genesis(config *DeployConfig, block *types.Block) (*core.Genesis, erro if eip1559Denom == 0 { eip1559Denom = 50 } + eip1559DenomCanyon := config.EIP1559DenominatorCanyon + if eip1559DenomCanyon == 0 { + eip1559DenomCanyon = 250 + } eip1559Elasticity := config.EIP1559Elasticity if eip1559Elasticity == 0 { eip1559Elasticity = 10 @@ -61,8 +65,9 @@ func NewL2Genesis(config *DeployConfig, block *types.Block) (*core.Genesis, erro CanyonTime: config.CanyonTime(block.Time()), ShanghaiTime: config.CanyonTime(block.Time()), Optimism: ¶ms.OptimismConfig{ - EIP1559Denominator: eip1559Denom, - EIP1559Elasticity: eip1559Elasticity, + EIP1559Denominator: eip1559Denom, + EIP1559Elasticity: eip1559Elasticity, + EIP1559DenominatorPostCanyon: eip1559DenomCanyon, }, } diff --git a/op-chain-ops/genesis/testdata/test-deploy-config-full.json b/op-chain-ops/genesis/testdata/test-deploy-config-full.json index e496fce29090..d2f7b980ddae 100644 --- a/op-chain-ops/genesis/testdata/test-deploy-config-full.json +++ b/op-chain-ops/genesis/testdata/test-deploy-config-full.json @@ -62,6 +62,7 @@ "governanceTokenOwner": "0x0000000000000000000000000000000000000333", "deploymentWaitConfirmations": 1, "eip1559Denominator": 8, + "eip1559DenominatorCanyon": 12, "eip1559Elasticity": 2, "fundDevAccounts": true, "faultGameAbsolutePrestate": "0x0000000000000000000000000000000000000000000000000000000000000000", diff --git a/op-e2e/actions/l1_miner.go b/op-e2e/actions/l1_miner.go index 70c5059a16c0..c9d085877bf6 100644 --- a/op-e2e/actions/l1_miner.go +++ b/op-e2e/actions/l1_miner.go @@ -67,7 +67,7 @@ func (s *L1Miner) ActL1StartBlock(timeDelta uint64) Action { MixDigest: common.Hash{}, // TODO: maybe randomize this (prev-randao value) } if s.l1Cfg.Config.IsLondon(header.Number) { - header.BaseFee = eip1559.CalcBaseFee(s.l1Cfg.Config, parent) + header.BaseFee = eip1559.CalcBaseFee(s.l1Cfg.Config, parent, header.Time) // At the transition, double the gas limit so the gas target is equal to the old gas limit. if !s.l1Cfg.Config.IsLondon(parent.Number) { header.GasLimit = parent.GasLimit * s.l1Cfg.Config.ElasticityMultiplier() diff --git a/op-program/client/l2/engineapi/block_processor.go b/op-program/client/l2/engineapi/block_processor.go index 22c6e1441401..c6966e0d94c2 100644 --- a/op-program/client/l2/engineapi/block_processor.go +++ b/op-program/client/l2/engineapi/block_processor.go @@ -68,7 +68,7 @@ func NewBlockProcessorFromHeader(provider BlockDataProvider, h *types.Header) (* return nil, fmt.Errorf("get parent state: %w", err) } header.Number = new(big.Int).Add(parentHeader.Number, common.Big1) - header.BaseFee = eip1559.CalcBaseFee(provider.Config(), parentHeader) + header.BaseFee = eip1559.CalcBaseFee(provider.Config(), parentHeader, header.Time) header.GasUsed = 0 gasPool := new(core.GasPool).AddGas(header.GasLimit) return &BlockProcessor{ diff --git a/packages/contracts-bedrock/deploy-config/devnetL1-template.json b/packages/contracts-bedrock/deploy-config/devnetL1-template.json index 10badc81e0df..c7acc73768f0 100644 --- a/packages/contracts-bedrock/deploy-config/devnetL1-template.json +++ b/packages/contracts-bedrock/deploy-config/devnetL1-template.json @@ -32,7 +32,7 @@ "portalGuardian": "0xa0Ee7A142d267C1f36714E4a8F75612F20a79720", "finalizationPeriodSeconds": 2, "fundDevAccounts": true, - "l2GenesisBlockBaseFeePerGas": "0x1", + "l2GenesisBlockBaseFeePerGas": "1", "gasPriceOracleOverhead": 2100, "gasPriceOracleScalar": 1000000, "enableGovernance": true, @@ -40,10 +40,12 @@ "governanceTokenName": "Optimism", "governanceTokenOwner": "0xa0Ee7A142d267C1f36714E4a8F75612F20a79720", "eip1559Denominator": 50, + "eip1559DenominatorCanyon": 250, "eip1559Elasticity": 6, "l1GenesisBlockTimestamp": "0x64c811bf", "l2GenesisRegolithTimeOffset": "0x0", "l2GenesisSpanBatchTimeOffset": "0x0", + "l2GenesisCanyonTimeOffset": "0x40", "faultGameAbsolutePrestate": "0x03c7ae758795765c6664a5d39bf63841c71ff191e9189522bad8ebff5d4eca98", "faultGameMaxDepth": 30, "faultGameMaxDuration": 1200, diff --git a/packages/contracts-bedrock/deploy-config/getting-started.json b/packages/contracts-bedrock/deploy-config/getting-started.json index 1757c3d049fa..7ac03cdc652d 100644 --- a/packages/contracts-bedrock/deploy-config/getting-started.json +++ b/packages/contracts-bedrock/deploy-config/getting-started.json @@ -51,6 +51,7 @@ "l2GenesisRegolithTimeOffset": "0x0", "eip1559Denominator": 50, + "eip1559DenominatorCanyon": 250, "eip1559Elasticity": 10, "systemConfigStartBlock": 0, diff --git a/packages/contracts-bedrock/deploy-config/goerli.json b/packages/contracts-bedrock/deploy-config/goerli.json index 1fa80b8c75ad..5c084b4fd36c 100644 --- a/packages/contracts-bedrock/deploy-config/goerli.json +++ b/packages/contracts-bedrock/deploy-config/goerli.json @@ -37,6 +37,7 @@ "l2GenesisBlockGasLimit": "0x2faf080", "l2GenesisBlockBaseFeePerGas": "0x3b9aca00", "eip1559Denominator": 50, + "eip1559DenominatorCanyon": 250, "eip1559Elasticity": 10, "systemConfigStartBlock": 8300214, "requiredProtocolVersion": "0x0000000000000000000000000000000000000000000000000000000000000000", diff --git a/packages/contracts-bedrock/deploy-config/hardhat.json b/packages/contracts-bedrock/deploy-config/hardhat.json index ab5279412ef4..3a59eb60ccb8 100644 --- a/packages/contracts-bedrock/deploy-config/hardhat.json +++ b/packages/contracts-bedrock/deploy-config/hardhat.json @@ -37,6 +37,7 @@ "governanceTokenOwner": "0x9965507D1a55bcC2695C58ba16FB37d819B0A4dc", "finalizationPeriodSeconds": 2, "eip1559Denominator": 50, + "eip1559DenominatorCanyon": 250, "eip1559Elasticity": 10, "l2GenesisRegolithTimeOffset": "0x0", "systemConfigStartBlock": 0, From 30871d7fdf746d3917fa79b9feb77eb4172e4762 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 26 Oct 2023 07:28:29 +0000 Subject: [PATCH 104/279] build(deps): bump github.com/onsi/gomega from 1.28.1 to 1.29.0 Bumps [github.com/onsi/gomega](https://github.com/onsi/gomega) from 1.28.1 to 1.29.0. - [Release notes](https://github.com/onsi/gomega/releases) - [Changelog](https://github.com/onsi/gomega/blob/master/CHANGELOG.md) - [Commits](https://github.com/onsi/gomega/compare/v1.28.1...v1.29.0) --- updated-dependencies: - dependency-name: github.com/onsi/gomega dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] --- go.mod | 2 +- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index 3713a12490c6..f9733ad04b61 100644 --- a/go.mod +++ b/go.mod @@ -33,7 +33,7 @@ require ( github.com/multiformats/go-multiaddr v0.12.0 github.com/multiformats/go-multiaddr-dns v0.3.1 github.com/olekukonko/tablewriter v0.0.5 - github.com/onsi/gomega v1.28.1 + github.com/onsi/gomega v1.29.0 github.com/pkg/errors v0.9.1 github.com/pkg/profile v1.7.0 github.com/prometheus/client_golang v1.17.0 diff --git a/go.sum b/go.sum index 1627f592ddca..ee04405cff83 100644 --- a/go.sum +++ b/go.sum @@ -569,8 +569,8 @@ github.com/onsi/gomega v1.7.1/go.mod h1:XdKZgCCFLUoM/7CFJVPcG8C1xQ1AJ0vpAezJrB7J github.com/onsi/gomega v1.10.1/go.mod h1:iN09h71vgCQne3DLsj+A5owkum+a2tYe+TOCB1ybHNo= github.com/onsi/gomega v1.17.0/go.mod h1:HnhC7FXeEQY45zxNK3PPoIUhzk/80Xly9PcubAlGdZY= github.com/onsi/gomega v1.19.0/go.mod h1:LY+I3pBVzYsTBU1AnDwOSxaYi9WoWiqgwooUqq9yPro= -github.com/onsi/gomega v1.28.1 h1:MijcGUbfYuznzK/5R4CPNoUP/9Xvuo20sXfEm6XxoTA= -github.com/onsi/gomega v1.28.1/go.mod h1:9sxs+SwGrKI0+PWe4Fxa9tFQQBG5xSsSbMXOI8PPpoQ= +github.com/onsi/gomega v1.29.0 h1:KIA/t2t5UBzoirT4H9tsML45GEbo3ouUnBHsCfD2tVg= +github.com/onsi/gomega v1.29.0/go.mod h1:9sxs+SwGrKI0+PWe4Fxa9tFQQBG5xSsSbMXOI8PPpoQ= github.com/opencontainers/runtime-spec v1.0.2/go.mod h1:jwyrGlmzljRJv/Fgzds9SsS/C5hL+LL3ko9hs6T5lQ0= github.com/opencontainers/runtime-spec v1.1.0 h1:HHUyrt9mwHUjtasSbXSMvs4cyFxh+Bll4AjJ9odEGpg= github.com/opencontainers/runtime-spec v1.1.0/go.mod h1:jwyrGlmzljRJv/Fgzds9SsS/C5hL+LL3ko9hs6T5lQ0= From 68e012cd5238d6c9777b319019ee7574442374c7 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 26 Oct 2023 07:28:34 +0000 Subject: [PATCH 105/279] build(deps): bump gorm.io/driver/postgres from 1.5.3 to 1.5.4 Bumps [gorm.io/driver/postgres](https://github.com/go-gorm/postgres) from 1.5.3 to 1.5.4. - [Commits](https://github.com/go-gorm/postgres/compare/v1.5.3...v1.5.4) --- updated-dependencies: - dependency-name: gorm.io/driver/postgres dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] --- go.mod | 2 +- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index 3713a12490c6..99f6d68739de 100644 --- a/go.mod +++ b/go.mod @@ -44,7 +44,7 @@ require ( golang.org/x/sync v0.4.0 golang.org/x/term v0.13.0 golang.org/x/time v0.3.0 - gorm.io/driver/postgres v1.5.3 + gorm.io/driver/postgres v1.5.4 gorm.io/gorm v1.25.5 ) diff --git a/go.sum b/go.sum index 1627f592ddca..649bcb92a4de 100644 --- a/go.sum +++ b/go.sum @@ -982,8 +982,8 @@ gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -gorm.io/driver/postgres v1.5.3 h1:qKGY5CPHOuj47K/VxbCXJfFvIUeqMSXXadqdCY+MbBU= -gorm.io/driver/postgres v1.5.3/go.mod h1:F+LtvlFhZT7UBiA81mC9W6Su3D4WUhSboc/36QZU0gk= +gorm.io/driver/postgres v1.5.4 h1:Iyrp9Meh3GmbSuyIAGyjkN+n9K+GHX9b9MqsTL4EJCo= +gorm.io/driver/postgres v1.5.4/go.mod h1:Bgo89+h0CRcdA33Y6frlaHHVuTdOf87pmyzwW9C/BH0= gorm.io/gorm v1.25.5 h1:zR9lOiiYf09VNh5Q1gphfyia1JpiClIWG9hQaxB/mls= gorm.io/gorm v1.25.5/go.mod h1:hbnx/Oo0ChWMn1BIhpy1oYozzpM15i4YPuHDmfYtwg8= grpc.go4.org v0.0.0-20170609214715-11d0a25b4919/go.mod h1:77eQGdRu53HpSqPFJFmuJdjuHRquDANNeA4x7B8WQ9o= From 9c0960cc8944e42620562034aaeea20262d7a6cb Mon Sep 17 00:00:00 2001 From: protolambda Date: Thu, 26 Oct 2023 15:10:28 +0200 Subject: [PATCH 106/279] ops: handle warm-up output case during docker publish correctly --- .circleci/config.yml | 17 ++++++++++++----- 1 file changed, 12 insertions(+), 5 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index f1e0212e0087..747de52041a4 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -229,17 +229,24 @@ jobs: # and naming allows us to use the DLC (docker-layer-cache) docker buildx create --driver=docker-container --name=buildx-build --bootstrap --use - DOCKER_OUTPUT_DESTINATION="--load" - # if we are publishing, change the destination + DOCKER_OUTPUT_DESTINATION="" if [ "<>" == "true" ]; then - DOCKER_OUTPUT_DESTINATION="--push" echo "Building for platforms $PLATFORMS and then publishing to registry" + DOCKER_OUTPUT_DESTINATION="--push" + if [ "<>" != "" ]; then + echo "ERROR: cannot save image to docker when publishing to registry" + exit 1 + fi else - if [[ $PLATFORMS == *,* ]]; then - echo "ERROR: cannot perform multi-arch build while also loading the result into regular docker" + if [ "<>" == "" ]; then + echo "Running $PLATFORMS build without destination (cache warm-up)" + DOCKER_OUTPUT_DESTINATION="" + elif [[ $PLATFORMS == *,* ]]; then + echo "ERROR: cannot perform multi-arch (platforms: $PLATFORMS) build while also loading the result into regular docker" exit 1 else echo "Running single-platform $PLATFORMS build and loading into docker" + DOCKER_OUTPUT_DESTINATION="--load" fi fi From 0fc2b1dc54e62a66b96de48a8bb5d77d1c562907 Mon Sep 17 00:00:00 2001 From: Joshua Gutow Date: Thu, 26 Oct 2023 07:24:44 -0700 Subject: [PATCH 107/279] Update packages/contracts-bedrock/deploy-config/devnetL1-template.json Co-authored-by: Adrian Sutton --- packages/contracts-bedrock/deploy-config/devnetL1-template.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/packages/contracts-bedrock/deploy-config/devnetL1-template.json b/packages/contracts-bedrock/deploy-config/devnetL1-template.json index c7acc73768f0..f05de9180ea5 100644 --- a/packages/contracts-bedrock/deploy-config/devnetL1-template.json +++ b/packages/contracts-bedrock/deploy-config/devnetL1-template.json @@ -32,7 +32,7 @@ "portalGuardian": "0xa0Ee7A142d267C1f36714E4a8F75612F20a79720", "finalizationPeriodSeconds": 2, "fundDevAccounts": true, - "l2GenesisBlockBaseFeePerGas": "1", + "l2GenesisBlockBaseFeePerGas": "0x1", "gasPriceOracleOverhead": 2100, "gasPriceOracleScalar": 1000000, "enableGovernance": true, From 8b964040fba52a310fa4d9f9711ee53b5e02e61a Mon Sep 17 00:00:00 2001 From: Mark Tyneway Date: Thu, 26 Oct 2023 10:38:01 -0600 Subject: [PATCH 108/279] op-bindings: regenerate --- op-bindings/bindings/mips_more.go | 2 +- op-bindings/bindings/preimageoracle_more.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/op-bindings/bindings/mips_more.go b/op-bindings/bindings/mips_more.go index e7983bf62efd..75570153b367 100644 --- a/op-bindings/bindings/mips_more.go +++ b/op-bindings/bindings/mips_more.go @@ -15,7 +15,7 @@ var MIPSStorageLayout = new(solc.StorageLayout) var MIPSDeployedBin = "0x608060405234801561001057600080fd5b50600436106100415760003560e01c8063155633fe146100465780637dc0d1d01461006b578063836e7b32146100af575b600080fd5b610051634000000081565b60405163ffffffff90911681526020015b60405180910390f35b60405173ffffffffffffffffffffffffffffffffffffffff7f0000000000000000000000000000000000000000000000000000000000000000168152602001610062565b6100c26100bd366004611d2e565b6100d0565b604051908152602001610062565b60006100da611c5b565b608081146100e757600080fd5b604051610600146100f757600080fd5b6084871461010457600080fd5b6101a4851461011257600080fd5b8635608052602087013560a052604087013560e090811c60c09081526044890135821c82526048890135821c61010052604c890135821c610120526050890135821c61014052605489013590911c61016052605888013560f890811c610180526059890135901c6101a052605a880135901c6101c0526102006101e0819052606288019060005b60208110156101bd57823560e01c8252600490920191602090910190600101610199565b505050806101200151156101db576101d361061b565b915050610612565b6101408101805160010167ffffffffffffffff16905260608101516000906102039082610737565b9050603f601a82901c16600281148061022257508063ffffffff166003145b156102775760006002836303ffffff1663ffffffff16901b846080015163f00000001617905061026c8263ffffffff1660021461026057601f610263565b60005b60ff16826107f3565b945050505050610612565b6101608301516000908190601f601086901c81169190601587901c16602081106102a3576102a3611da2565b602002015192508063ffffffff851615806102c457508463ffffffff16601c145b156102fb578661016001518263ffffffff16602081106102e6576102e6611da2565b6020020151925050601f600b86901c166103b7565b60208563ffffffff16101561035d578463ffffffff16600c148061032557508463ffffffff16600d145b8061033657508463ffffffff16600e145b15610347578561ffff1692506103b7565b6103568661ffff1660106108e4565b92506103b7565b60288563ffffffff1610158061037957508463ffffffff166022145b8061038a57508463ffffffff166026145b156103b7578661016001518263ffffffff16602081106103ac576103ac611da2565b602002015192508190505b60048563ffffffff16101580156103d4575060088563ffffffff16105b806103e557508463ffffffff166001145b15610404576103f685878487610957565b975050505050505050610612565b63ffffffff6000602087831610610469576104248861ffff1660106108e4565b9095019463fffffffc861661043a816001610737565b915060288863ffffffff161015801561045a57508763ffffffff16603014155b1561046757809250600093505b505b600061047789888885610b67565b63ffffffff9081169150603f8a1690891615801561049c575060088163ffffffff1610155b80156104ae5750601c8163ffffffff16105b1561058b578063ffffffff16600814806104ce57508063ffffffff166009145b15610505576104f38163ffffffff166008146104ea57856104ed565b60005b896107f3565b9b505050505050505050505050610612565b8063ffffffff16600a03610525576104f3858963ffffffff8a16156112f7565b8063ffffffff16600b03610546576104f3858963ffffffff8a1615156112f7565b8063ffffffff16600c0361055d576104f38d6113dd565b60108163ffffffff161015801561057a5750601c8163ffffffff16105b1561058b576104f381898988611914565b8863ffffffff1660381480156105a6575063ffffffff861615155b156105db5760018b61016001518763ffffffff16602081106105ca576105ca611da2565b63ffffffff90921660209290920201525b8363ffffffff1663ffffffff146105f8576105f884600184611b0e565b610604858360016112f7565b9b5050505050505050505050505b95945050505050565b60408051608051815260a051602082015260dc519181019190915260fc51604482015261011c51604882015261013c51604c82015261015c51605082015261017c5160548201526101805161019f5160588301526101a0516101bf5160598401526101d851605a840152600092610200929091606283019190855b60208110156106ba57601c8601518452602090950194600490930192600101610696565b506000835283830384a06000945080600181146106da5760039550610702565b8280156106f257600181146106fb5760029650610700565b60009650610700565b600196505b505b50505081900390207effffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff1660f89190911b17919050565b60008061074383611bb2565b9050600384161561075357600080fd5b6020810190358460051c8160005b601b8110156107b95760208501943583821c6001168015610789576001811461079e576107af565b600084815260208390526040902093506107af565b600082815260208590526040902093505b5050600101610761565b5060805191508181146107d457630badf00d60005260206000fd5b5050601f94909416601c0360031b9390931c63ffffffff169392505050565b60006107fd611c5b565b60809050806060015160040163ffffffff16816080015163ffffffff1614610886576040517f08c379a000000000000000000000000000000000000000000000000000000000815260206004820152601260248201527f6a756d7020696e2064656c617920736c6f74000000000000000000000000000060448201526064015b60405180910390fd5b60608101805160808301805163ffffffff9081169093528583169052908516156108dc57806008018261016001518663ffffffff16602081106108cb576108cb611da2565b63ffffffff90921660209290920201525b61061261061b565b600063ffffffff8381167fffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff80850183169190911c821615159160016020869003821681901b830191861691821b92911b0182610941576000610943565b815b90861663ffffffff16179250505092915050565b6000610961611c5b565b608090506000816060015160040163ffffffff16826080015163ffffffff16146109e7576040517f08c379a000000000000000000000000000000000000000000000000000000000815260206004820152601460248201527f6272616e636820696e2064656c617920736c6f74000000000000000000000000604482015260640161087d565b8663ffffffff1660041480610a0257508663ffffffff166005145b15610a7e5760008261016001518663ffffffff1660208110610a2657610a26611da2565b602002015190508063ffffffff168563ffffffff16148015610a4e57508763ffffffff166004145b80610a7657508063ffffffff168563ffffffff1614158015610a7657508763ffffffff166005145b915050610afb565b8663ffffffff16600603610a9b5760008460030b13159050610afb565b8663ffffffff16600703610ab75760008460030b139050610afb565b8663ffffffff16600103610afb57601f601087901c166000819003610ae05760008560030b1291505b8063ffffffff16600103610af95760008560030b121591505b505b606082018051608084015163ffffffff169091528115610b41576002610b268861ffff1660106108e4565b63ffffffff90811690911b8201600401166080840152610b53565b60808301805160040163ffffffff1690525b610b5b61061b565b98975050505050505050565b6000603f601a86901c16801580610b96575060088163ffffffff1610158015610b965750600f8163ffffffff16105b15610fec57603f86168160088114610bdd5760098114610be657600a8114610bef57600b8114610bf857600c8114610c0157600d8114610c0a57600e8114610c1357610c18565b60209150610c18565b60219150610c18565b602a9150610c18565b602b9150610c18565b60249150610c18565b60259150610c18565b602691505b508063ffffffff16600003610c3f5750505063ffffffff8216601f600686901c161b6112ef565b8063ffffffff16600203610c655750505063ffffffff8216601f600686901c161c6112ef565b8063ffffffff16600303610c9b57601f600688901c16610c9163ffffffff8716821c60208390036108e4565b93505050506112ef565b8063ffffffff16600403610cbd5750505063ffffffff8216601f84161b6112ef565b8063ffffffff16600603610cdf5750505063ffffffff8216601f84161c6112ef565b8063ffffffff16600703610d1257610d098663ffffffff168663ffffffff16901c876020036108e4565b925050506112ef565b8063ffffffff16600803610d2a5785925050506112ef565b8063ffffffff16600903610d425785925050506112ef565b8063ffffffff16600a03610d5a5785925050506112ef565b8063ffffffff16600b03610d725785925050506112ef565b8063ffffffff16600c03610d8a5785925050506112ef565b8063ffffffff16600f03610da25785925050506112ef565b8063ffffffff16601003610dba5785925050506112ef565b8063ffffffff16601103610dd25785925050506112ef565b8063ffffffff16601203610dea5785925050506112ef565b8063ffffffff16601303610e025785925050506112ef565b8063ffffffff16601803610e1a5785925050506112ef565b8063ffffffff16601903610e325785925050506112ef565b8063ffffffff16601a03610e4a5785925050506112ef565b8063ffffffff16601b03610e625785925050506112ef565b8063ffffffff16602003610e7b575050508282016112ef565b8063ffffffff16602103610e94575050508282016112ef565b8063ffffffff16602203610ead575050508183036112ef565b8063ffffffff16602303610ec6575050508183036112ef565b8063ffffffff16602403610edf575050508282166112ef565b8063ffffffff16602503610ef8575050508282176112ef565b8063ffffffff16602603610f11575050508282186112ef565b8063ffffffff16602703610f2b57505050828217196112ef565b8063ffffffff16602a03610f5c578460030b8660030b12610f4d576000610f50565b60015b60ff16925050506112ef565b8063ffffffff16602b03610f84578463ffffffff168663ffffffff1610610f4d576000610f50565b6040517f08c379a000000000000000000000000000000000000000000000000000000000815260206004820152601360248201527f696e76616c696420696e737472756374696f6e00000000000000000000000000604482015260640161087d565b50610f84565b8063ffffffff16601c0361107057603f86166002819003611012575050508282026112ef565b8063ffffffff166020148061102d57508063ffffffff166021145b15610fe6578063ffffffff16602003611044579419945b60005b6380000000871615611066576401fffffffe600197881b169601611047565b92506112ef915050565b8063ffffffff16600f0361109257505065ffffffff0000601083901b166112ef565b8063ffffffff166020036110ce576110c68560031660080260180363ffffffff168463ffffffff16901c60ff1660086108e4565b9150506112ef565b8063ffffffff16602103611103576110c68560021660080260100363ffffffff168463ffffffff16901c61ffff1660106108e4565b8063ffffffff1660220361113257505063ffffffff60086003851602811681811b198416918316901b176112ef565b8063ffffffff1660230361114957829150506112ef565b8063ffffffff1660240361117b578460031660080260180363ffffffff168363ffffffff16901c60ff169150506112ef565b8063ffffffff166025036111ae578460021660080260100363ffffffff168363ffffffff16901c61ffff169150506112ef565b8063ffffffff166026036111e057505063ffffffff60086003851602601803811681811c198416918316901c176112ef565b8063ffffffff1660280361121657505060ff63ffffffff60086003861602601803811682811b9091188316918416901b176112ef565b8063ffffffff1660290361124d57505061ffff63ffffffff60086002861602601003811682811b9091188316918416901b176112ef565b8063ffffffff16602a0361127c57505063ffffffff60086003851602811681811c198316918416901c176112ef565b8063ffffffff16602b0361129357839150506112ef565b8063ffffffff16602e036112c557505063ffffffff60086003851602601803811681811b198316918416901b176112ef565b8063ffffffff166030036112dc57829150506112ef565b8063ffffffff16603803610f8457839150505b949350505050565b6000611301611c5b565b506080602063ffffffff861610611374576040517f08c379a000000000000000000000000000000000000000000000000000000000815260206004820152600e60248201527f76616c6964207265676973746572000000000000000000000000000000000000604482015260640161087d565b63ffffffff8516158015906113865750825b156113ba57838161016001518663ffffffff16602081106113a9576113a9611da2565b63ffffffff90921660209290920201525b60808101805163ffffffff8082166060850152600490910116905261061261061b565b60006113e7611c5b565b506101e051604081015160808083015160a084015160c09094015191936000928392919063ffffffff8616610ffa036114615781610fff81161561143057610fff811661100003015b8363ffffffff166000036114575760e08801805163ffffffff83820116909152955061145b565b8395505b506118d3565b8563ffffffff16610fcd0361147c57634000000094506118d3565b8563ffffffff166110180361149457600194506118d3565b8563ffffffff16611096036114ca57600161012088015260ff83166101008801526114bd61061b565b9998505050505050505050565b8563ffffffff16610fa3036117365763ffffffff8316156118d3577ffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffb63ffffffff8416016116f05760006115258363fffffffc166001610737565b60208901519091508060001a60010361159457604080516000838152336020528d83526060902091527effffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff167f01000000000000000000000000000000000000000000000000000000000000001790505b6040808a015190517fe03110e10000000000000000000000000000000000000000000000000000000081526004810183905263ffffffff9091166024820152600090819073ffffffffffffffffffffffffffffffffffffffff7f0000000000000000000000000000000000000000000000000000000000000000169063e03110e1906044016040805180830381865afa158015611635573d6000803e3d6000fd5b505050506040513d601f19601f820116820180604052508101906116599190611dd1565b91509150600386168060040382811015611671578092505b508186101561167e578591505b8260088302610100031c9250826008828460040303021b9250600180600883600403021b036001806008858560040303021b039150811981169050838119871617955050506116d58663fffffffc16600186611b0e565b60408b018051820163ffffffff169052975061173192505050565b7ffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffd63ffffffff841601611725578094506118d3565b63ffffffff9450600993505b6118d3565b8563ffffffff16610fa4036118275763ffffffff831660011480611760575063ffffffff83166002145b80611771575063ffffffff83166004145b1561177e578094506118d3565b7ffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffa63ffffffff8416016117255760006117be8363fffffffc166001610737565b602089015190915060038416600403838110156117d9578093505b83900360089081029290921c7fffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff600193850293841b0116911b176020880152600060408801529350836118d3565b8563ffffffff16610fd7036118d3578163ffffffff166003036118c75763ffffffff8316158061185d575063ffffffff83166005145b8061186e575063ffffffff83166003145b1561187c57600094506118d3565b63ffffffff831660011480611897575063ffffffff83166002145b806118a8575063ffffffff83166006145b806118b9575063ffffffff83166004145b1561172557600194506118d3565b63ffffffff9450601693505b6101608701805163ffffffff808816604090920191909152905185821660e09091015260808801805180831660608b015260040190911690526114bd61061b565b600061191e611c5b565b506080600063ffffffff871660100361193c575060c0810151611aa5565b8663ffffffff1660110361195b5763ffffffff861660c0830152611aa5565b8663ffffffff16601203611974575060a0810151611aa5565b8663ffffffff166013036119935763ffffffff861660a0830152611aa5565b8663ffffffff166018036119c75763ffffffff600387810b9087900b02602081901c821660c08501521660a0830152611aa5565b8663ffffffff166019036119f85763ffffffff86811681871602602081901c821660c08501521660a0830152611aa5565b8663ffffffff16601a03611a4e578460030b8660030b81611a1b57611a1b611df5565b0763ffffffff1660c0830152600385810b9087900b81611a3d57611a3d611df5565b0563ffffffff1660a0830152611aa5565b8663ffffffff16601b03611aa5578463ffffffff168663ffffffff1681611a7757611a77611df5565b0663ffffffff90811660c084015285811690871681611a9857611a98611df5565b0463ffffffff1660a08301525b63ffffffff841615611ae057808261016001518563ffffffff1660208110611acf57611acf611da2565b63ffffffff90921660209290920201525b60808201805163ffffffff80821660608601526004909101169052611b0361061b565b979650505050505050565b6000611b1983611bb2565b90506003841615611b2957600080fd5b6020810190601f8516601c0360031b83811b913563ffffffff90911b1916178460051c60005b601b811015611ba75760208401933582821c6001168015611b775760018114611b8c57611b9d565b60008581526020839052604090209450611b9d565b600082815260208690526040902094505b5050600101611b4f565b505060805250505050565b60ff8116610380026101a4810190369061052401811015611c55576040517f08c379a000000000000000000000000000000000000000000000000000000000815260206004820152602360248201527f636865636b207468617420746865726520697320656e6f7567682063616c6c6460448201527f6174610000000000000000000000000000000000000000000000000000000000606482015260840161087d565b50919050565b6040805161018081018252600080825260208201819052918101829052606081018290526080810182905260a0810182905260c0810182905260e08101829052610100810182905261012081018290526101408101919091526101608101611cc1611cc6565b905290565b6040518061040001604052806020906020820280368337509192915050565b60008083601f840112611cf757600080fd5b50813567ffffffffffffffff811115611d0f57600080fd5b602083019150836020828501011115611d2757600080fd5b9250929050565b600080600080600060608688031215611d4657600080fd5b853567ffffffffffffffff80821115611d5e57600080fd5b611d6a89838a01611ce5565b90975095506020880135915080821115611d8357600080fd5b50611d9088828901611ce5565b96999598509660400135949350505050565b7f4e487b7100000000000000000000000000000000000000000000000000000000600052603260045260246000fd5b60008060408385031215611de457600080fd5b505080516020909101519092909150565b7f4e487b7100000000000000000000000000000000000000000000000000000000600052601260045260246000fdfea164736f6c634300080f000a" -var MIPSDeployedSourceMap = "1131:40054:127:-:0;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;1710:45;;1745:10;1710:45;;;;;188:10:286;176:23;;;158:42;;146:2;131:18;1710:45:127;;;;;;;;2448:99;;;412:42:286;2534:6:127;400:55:286;382:74;;370:2;355:18;2448:99:127;211:251:286;26025:6379:127;;;;;;:::i;:::-;;:::i;:::-;;;1755:25:286;;;1743:2;1728:18;26025:6379:127;1609:177:286;26025:6379:127;26128:7;26171:18;;:::i;:::-;26318:4;26311:5;26308:15;26298:134;;26412:1;26409;26402:12;26298:134;26468:4;26462:11;26475:10;26459:27;26449:136;;26565:1;26562;26555:12;26449:136;26634:3;26615:17;26612:26;26602:151;;26733:1;26730;26723:12;26602:151;26798:3;26783:13;26780:22;26770:146;;26896:1;26893;26886:12;26770:146;27176:24;;27521:4;27222:20;27579:2;27280:21;;27176:24;27338:18;27222:20;27280:21;;;27176:24;27153:21;27149:52;;;27338:18;27222:20;;;27280:21;;;27176:24;27149:52;;27222:20;;27280:21;;;27176:24;27149:52;;27338:18;27222:20;27280:21;;;27176:24;27149:52;;27338:18;27222:20;27280:21;;;27176:24;27149:52;;27338:18;27222:20;27280:21;;;27176:24;27149:52;;;27338:18;27222:20;27280:21;;;27176:24;27153:21;27149:52;;;27338:18;27222:20;27280:21;;;27176:24;27149:52;;27338:18;27222:20;27280:21;;;27176:24;27149:52;;27338:18;27222:20;28197:10;27338:18;28187:21;;;27280;;;;28295:1;28280:77;28305:2;28302:1;28299:9;28280:77;;;27176:24;;27153:21;27149:52;27222:20;;28353:1;27280:21;;;;27164:2;27338:18;;;;28323:1;28316:9;28280:77;;;28284:14;;;28435:5;:12;;;28431:71;;;28474:13;:11;:13::i;:::-;28467:20;;;;;28431:71;28516:10;;;:15;;28530:1;28516:15;;;;;28601:8;;;;-1:-1:-1;;28593:20:127;;-1:-1:-1;28593:7:127;:20::i;:::-;28579:34;-1:-1:-1;28643:10:127;28651:2;28643:10;;;;28720:1;28710:11;;;:26;;;28725:6;:11;;28735:1;28725:11;28710:26;28706:310;;;28866:13;28935:1;28913:4;28920:10;28913:17;28912:24;;;;28883:5;:12;;;28898:10;28883:25;28882:54;28866:70;;28961:40;28972:6;:11;;28982:1;28972:11;:20;;28990:2;28972:20;;;28986:1;28972:20;28961:40;;28994:6;28961:10;:40::i;:::-;28954:47;;;;;;;;28706:310;29265:15;;;;29060:9;;;;29197:4;29191:2;29183:10;;;29182:19;;;29265:15;29290:2;29282:10;;;29281:19;29265:36;;;;;;;:::i;:::-;;;;;;-1:-1:-1;29330:5:127;29354:11;;;;;:29;;;29369:6;:14;;29379:4;29369:14;29354:29;29350:832;;;29446:5;:15;;;29462:5;29446:22;;;;;;;;;:::i;:::-;;;;;;-1:-1:-1;;29509:4:127;29503:2;29495:10;;;29494:19;29350:832;;;29547:4;29538:6;:13;;;29534:648;;;29668:6;:13;;29678:3;29668:13;:30;;;;29685:6;:13;;29695:3;29685:13;29668:30;:47;;;;29702:6;:13;;29712:3;29702:13;29668:47;29664:253;;;29778:4;29785:6;29778:13;29773:18;;29534:648;;29664:253;29877:21;29880:4;29887:6;29880:13;29895:2;29877;:21::i;:::-;29872:26;;29534:648;;;29951:4;29941:6;:14;;;;:32;;;;29959:6;:14;;29969:4;29959:14;29941:32;:50;;;;29977:6;:14;;29987:4;29977:14;29941:50;29937:245;;;30061:5;:15;;;30077:5;30061:22;;;;;;;;;:::i;:::-;;;;;30056:27;;30162:5;30154:13;;29937:245;30211:1;30201:6;:11;;;;:25;;;;;30225:1;30216:6;:10;;;30201:25;30200:42;;;;30231:6;:11;;30241:1;30231:11;30200:42;30196:125;;;30269:37;30282:6;30290:4;30296:5;30303:2;30269:12;:37::i;:::-;30262:44;;;;;;;;;;;30196:125;30354:13;30335:16;30506:4;30496:14;;;;30492:446;;30575:21;30578:4;30585:6;30578:13;30593:2;30575;:21::i;:::-;30569:27;;;;30633:10;30628:15;;30667:16;30628:15;30681:1;30667:7;:16::i;:::-;30661:22;;30715:4;30705:6;:14;;;;:32;;;;;30723:6;:14;;30733:4;30723:14;;30705:32;30701:223;;;30802:4;30790:16;;30904:1;30896:9;;30701:223;30512:426;30492:446;30971:10;30984:26;30992:4;30998:2;31002;31006:3;30984:7;:26::i;:::-;31013:10;30984:39;;;;-1:-1:-1;31109:4:127;31102:11;;;31141;;;:24;;;;;31164:1;31156:4;:9;;;;31141:24;:39;;;;;31176:4;31169;:11;;;31141:39;31137:860;;;31204:4;:9;;31212:1;31204:9;:22;;;;31217:4;:9;;31225:1;31217:9;31204:22;31200:144;;;31288:37;31299:4;:9;;31307:1;31299:9;:21;;31315:5;31299:21;;;31311:1;31299:21;31322:2;31288:10;:37::i;:::-;31281:44;;;;;;;;;;;;;;;31200:144;31366:4;:11;;31374:3;31366:11;31362:121;;31436:28;31445:5;31452:2;31456:7;;;;31436:8;:28::i;31362:121::-;31504:4;:11;;31512:3;31504:11;31500:121;;31574:28;31583:5;31590:2;31594:7;;;;;31574:8;:28::i;31500:121::-;31691:4;:11;;31699:3;31691:11;31687:93;;31733:28;31747:13;31733;:28::i;31687:93::-;31883:4;31875;:12;;;;:27;;;;;31898:4;31891;:11;;;31875:27;31871:112;;;31933:31;31944:4;31950:2;31954;31958:5;31933:10;:31::i;31871:112::-;32057:6;:14;;32067:4;32057:14;:28;;;;-1:-1:-1;32075:10:127;;;;;32057:28;32053:93;;;32130:1;32105:5;:15;;;32121:5;32105:22;;;;;;;;;:::i;:::-;:26;;;;:22;;;;;;:26;32053:93;32192:9;:26;;32205:13;32192:26;32188:92;;32238:27;32247:9;32258:1;32261:3;32238:8;:27::i;:::-;32361:26;32370:5;32377:3;32382:4;32361:8;:26::i;:::-;32354:33;;;;;;;;;;;;;26025:6379;;;;;;;;:::o;3087:2334::-;3634:4;3628:11;;3550:4;3353:31;3342:43;;3413:13;3353:31;3752:2;3452:13;;3342:43;3359:24;3353:31;3452:13;;;3342:43;;;;3359:24;3353:31;3452:13;;;3342:43;3359:24;3353:31;3452:13;;;3342:43;3359:24;3353:31;3452:13;;;3342:43;3359:24;3353:31;3452:13;;;3342:43;3359:24;3353:31;3452:13;;;3342:43;3413:13;4180:11;3359:24;3353:31;3452:13;;;3342:43;3413:13;4275:11;3359:24;3353:31;3452:13;;;3342:43;3359:24;3353:31;3452:13;;;3342:43;3128:12;;4415:13;;3628:11;;3452:13;;;;4180:11;3128:12;4495:84;4520:2;4517:1;4514:9;4495:84;;;3369:13;3359:24;;3353:31;3342:43;;3373:2;3413:13;;;;4575:1;3452:13;;;;4538:1;4531:9;4495:84;;;4499:14;4642:1;4638:2;4631:13;4737:5;4733:2;4729:14;4722:5;4717:27;4811:1;4797:15;;4832:6;4856:1;4851:273;;;;5191:1;5181:11;;4825:369;;4851:273;4883:8;4941:22;;;;5020:1;5015:22;;;;5107:1;5097:11;;4876:234;;4941:22;4960:1;4950:11;;4941:22;;5015;5034:1;5024:11;;4876:234;;4825:369;-1:-1:-1;;;5317:14:127;;;5300:32;;5360:19;5356:30;5392:3;5388:16;;;;5353:52;;3087:2334;-1:-1:-1;3087:2334:127:o;21746:1831::-;21819:11;21930:14;21947:24;21959:11;21947;:24::i;:::-;21930:41;;22079:1;22072:5;22068:13;22065:33;;;22094:1;22091;22084:12;22065:33;22227:2;22215:15;;;22168:20;22657:5;22654:1;22650:13;22692:4;22728:1;22713:343;22738:2;22735:1;22732:9;22713:343;;;22861:2;22849:15;;;22798:20;22896:12;;;22910:1;22892:20;22933:42;;;;23001:1;22996:42;;;;22885:153;;22933:42;22391:1;22384:12;;;22424:2;22417:13;;;22469:2;22456:16;;22942:31;;22933:42;;22996;22391:1;22384:12;;;22424:2;22417:13;;;22469:2;22456:16;;23005:31;;22885:153;-1:-1:-1;;22756:1:127;22749:9;22713:343;;;22717:14;23166:4;23160:11;23145:26;;23252:7;23246:4;23243:17;23233:124;;23294:10;23291:1;23284:21;23336:2;23333:1;23326:13;23233:124;-1:-1:-1;;23484:2:127;23473:14;;;;23461:10;23457:31;23454:1;23450:39;23518:16;;;;23536:10;23514:33;;21746:1831;-1:-1:-1;;;21746:1831:127:o;18856:823::-;18925:12;19012:18;;:::i;:::-;19080:4;19071:13;;19132:5;:8;;;19143:1;19132:12;19116:28;;:5;:12;;;:28;;;19112:95;;19164:28;;;;;2182:2:286;19164:28:127;;;2164:21:286;2221:2;2201:18;;;2194:30;2260:20;2240:18;;;2233:48;2298:18;;19164:28:127;;;;;;;;19112:95;19296:8;;;;;19329:12;;;;;19318:23;;;;;;;19355:20;;;;;19296:8;19487:13;;;19483:90;;19548:6;19557:1;19548:10;19520:5;:15;;;19536:8;19520:25;;;;;;;;;:::i;:::-;:38;;;;:25;;;;;;:38;19483:90;19649:13;:11;:13::i;2645:339::-;2706:11;2770:18;;;;2779:8;;;;2770:18;;;;;;2769:25;;;;;2786:1;2833:2;:9;;;2827:16;;;;;2826:22;;2825:32;;;;;;;2887:9;;2886:15;2769:25;2944:21;;2964:1;2944:21;;;2955:6;2944:21;2929:11;;;;;:37;;-1:-1:-1;;;2645:339:127;;;;:::o;13732:2026::-;13829:12;13915:18;;:::i;:::-;13983:4;13974:13;;14015:17;14075:5;:8;;;14086:1;14075:12;14059:28;;:5;:12;;;:28;;;14055:97;;14107:30;;;;;2529:2:286;14107:30:127;;;2511:21:286;2568:2;2548:18;;;2541:30;2607:22;2587:18;;;2580:50;2647:18;;14107:30:127;2327:344:286;14055:97:127;14222:7;:12;;14233:1;14222:12;:28;;;;14238:7;:12;;14249:1;14238:12;14222:28;14218:947;;;14270:9;14282:5;:15;;;14298:6;14282:23;;;;;;;;;:::i;:::-;;;;;14270:35;;14346:2;14339:9;;:3;:9;;;:25;;;;;14352:7;:12;;14363:1;14352:12;14339:25;14338:58;;;;14377:2;14370:9;;:3;:9;;;;:25;;;;;14383:7;:12;;14394:1;14383:12;14370:25;14323:73;;14252:159;14218:947;;;14508:7;:12;;14519:1;14508:12;14504:661;;14569:1;14561:3;14555:15;;;;14540:30;;14504:661;;;14673:7;:12;;14684:1;14673:12;14669:496;;14733:1;14726:3;14720:14;;;14705:29;;14669:496;;;14854:7;:12;;14865:1;14854:12;14850:315;;14942:4;14936:2;14927:11;;;14926:20;14912:10;14969:8;;;14965:84;;15029:1;15022:3;15016:14;;;15001:29;;14965:84;15070:3;:8;;15077:1;15070:8;15066:85;;15131:1;15123:3;15117:15;;;;15102:30;;15066:85;14868:297;14850:315;15241:8;;;;;15319:12;;;;15308:23;;;;;15475:178;;;;15566:1;15540:22;15543:5;15551:6;15543:14;15559:2;15540;:22::i;:::-;:27;;;;;;;15526:42;;15535:1;15526:42;15511:57;:12;;;:57;15475:178;;;15622:12;;;;;15637:1;15622:16;15607:31;;;;15475:178;15728:13;:11;:13::i;:::-;15721:20;13732:2026;-1:-1:-1;;;;;;;;13732:2026:127:o;32450:8733::-;32537:10;32599;32607:2;32599:10;;;;32638:11;;;:44;;;32664:1;32654:6;:11;;;;:27;;;;;32678:3;32669:6;:12;;;32654:27;32634:8490;;;32723:4;32716:11;;32847:6;32907:3;32902:25;;;;32982:3;32977:25;;;;33056:3;33051:25;;;;33131:3;33126:25;;;;33205:3;33200:25;;;;33278:3;33273:25;;;;33352:3;33347:25;;;;32840:532;;32902:25;32921:4;32913:12;;32902:25;;32977;32996:4;32988:12;;32977:25;;33051;33070:4;33062:12;;33051:25;;33126;33145:4;33137:12;;33126:25;;33200;33219:4;33211:12;;33200:25;;33273;33292:4;33284:12;;33273:25;;33347;33366:4;33358:12;;32840:532;;33435:4;:12;;33443:4;33435:12;33431:4023;;-1:-1:-1;;;33486:9:127;33478:26;;33499:4;33494:1;33486:9;;;33485:18;33478:26;33471:33;;33431:4023;33572:4;:12;;33580:4;33572:12;33568:3886;;-1:-1:-1;;;33623:9:127;33615:26;;33636:4;33631:1;33623:9;;;33622:18;33615:26;33608:33;;33568:3886;33709:4;:12;;33717:4;33709:12;33705:3749;;33774:4;33769:1;33761:9;;;33760:18;33807:27;33761:9;33810:11;;;;33823:2;:10;;;33807:2;:27::i;:::-;33800:34;;;;;;;33705:3749;33903:4;:12;;33911:4;33903:12;33899:3555;;-1:-1:-1;;;33946:17:127;;;33958:4;33953:9;;33946:17;33939:24;;33899:3555;34032:4;:11;;34040:3;34032:11;34028:3426;;-1:-1:-1;;;34074:17:127;;;34086:4;34081:9;;34074:17;34067:24;;34028:3426;34160:4;:12;;34168:4;34160:12;34156:3298;;34203:21;34212:2;34206:8;;:2;:8;;;;34221:2;34216;:7;34203:2;:21::i;:::-;34196:28;;;;;;34156:3298;34473:4;:12;;34481:4;34473:12;34469:2985;;34516:2;34509:9;;;;;;34469:2985;34587:4;:12;;34595:4;34587:12;34583:2871;;34630:2;34623:9;;;;;;34583:2871;34701:4;:12;;34709:4;34701:12;34697:2757;;34744:2;34737:9;;;;;;34697:2757;34815:4;:12;;34823:4;34815:12;34811:2643;;34858:2;34851:9;;;;;;34811:2643;34932:4;:12;;34940:4;34932:12;34928:2526;;34975:2;34968:9;;;;;;34928:2526;35092:4;:12;;35100:4;35092:12;35088:2366;;35135:2;35128:9;;;;;;35088:2366;35206:4;:12;;35214:4;35206:12;35202:2252;;35249:2;35242:9;;;;;;35202:2252;35320:4;:12;;35328:4;35320:12;35316:2138;;35363:2;35356:9;;;;;;35316:2138;35434:4;:12;;35442:4;35434:12;35430:2024;;35477:2;35470:9;;;;;;35430:2024;35548:4;:12;;35556:4;35548:12;35544:1910;;35591:2;35584:9;;;;;;35544:1910;35662:4;:12;;35670:4;35662:12;35658:1796;;35705:2;35698:9;;;;;;35658:1796;35777:4;:12;;35785:4;35777:12;35773:1681;;35820:2;35813:9;;;;;;35773:1681;35890:4;:12;;35898:4;35890:12;35886:1568;;35933:2;35926:9;;;;;;35886:1568;36004:4;:12;;36012:4;36004:12;36000:1454;;36047:2;36040:9;;;;;;36000:1454;36196:4;:12;;36204:4;36196:12;36192:1262;;-1:-1:-1;;;36240:7:127;;;36232:16;;36192:1262;36317:4;:12;;36325:4;36317:12;36313:1141;;-1:-1:-1;;;36361:7:127;;;36353:16;;36313:1141;36437:4;:12;;36445:4;36437:12;36433:1021;;-1:-1:-1;;;36481:7:127;;;36473:16;;36433:1021;36558:4;:12;;36566:4;36558:12;36554:900;;-1:-1:-1;;;36602:7:127;;;36594:16;;36554:900;36678:4;:12;;36686:4;36678:12;36674:780;;-1:-1:-1;;;36722:7:127;;;36714:16;;36674:780;36797:4;:12;;36805:4;36797:12;36793:661;;-1:-1:-1;;;36841:7:127;;;36833:16;;36793:661;36917:4;:12;;36925:4;36917:12;36913:541;;-1:-1:-1;;;36961:7:127;;;36953:16;;36913:541;37037:4;:12;;37045:4;37037:12;37033:421;;-1:-1:-1;;;37082:7:127;;;37080:10;37073:17;;37033:421;37159:4;:12;;37167:4;37159:12;37155:299;;37220:2;37202:21;;37208:2;37202:21;;;:29;;37230:1;37202:29;;;37226:1;37202:29;37195:36;;;;;;;;37155:299;37301:4;:12;;37309:4;37301:12;37297:157;;37349:2;37344:7;;:2;:7;;;:15;;37358:1;37344:15;;37297:157;37406:29;;;;;2878:2:286;37406:29:127;;;2860:21:286;2917:2;2897:18;;;2890:30;2956:21;2936:18;;;2929:49;2995:18;;37406:29:127;2676:343:286;37297:157:127;32684:4784;32634:8490;;;37524:6;:14;;37534:4;37524:14;37520:3590;;37583:4;37576:11;;37658:3;37650:11;;;37646:549;;-1:-1:-1;;;37703:21:127;;;37689:36;;37646:549;37810:4;:12;;37818:4;37810:12;:28;;;;37826:4;:12;;37834:4;37826:12;37810:28;37806:389;;;37870:4;:12;;37878:4;37870:12;37866:83;;37919:3;;;37866:83;37974:8;38012:127;38024:10;38019:15;;:20;38012:127;;38104:8;38071:3;38104:8;;;;;38071:3;38012:127;;;38171:1;-1:-1:-1;38164:8:127;;-1:-1:-1;;38164:8:127;37520:3590;38262:6;:14;;38272:4;38262:14;38258:2852;;-1:-1:-1;;38307:8:127;38313:2;38307:8;;;;38300:15;;38258:2852;38382:6;:14;;38392:4;38382:14;38378:2732;;38427:42;38445:2;38450:1;38445:6;38455:1;38444:12;38439:2;:17;38431:26;;:3;:26;;;;38461:4;38430:35;38467:1;38427:2;:42::i;:::-;38420:49;;;;;38378:2732;38536:6;:14;;38546:4;38536:14;38532:2578;;38581:45;38599:2;38604:1;38599:6;38609:1;38598:12;38593:2;:17;38585:26;;:3;:26;;;;38615:6;38584:37;38623:2;38581;:45::i;38532:2578::-;38694:6;:14;;38704:4;38694:14;38690:2420;;-1:-1:-1;;38745:21:127;38764:1;38759;38754:6;;38753:12;38745:21;;38802:36;;;38873:5;38868:10;;38745:21;;;;;38867:18;38860:25;;38690:2420;38952:6;:14;;38962:4;38952:14;38948:2162;;38997:3;38990:10;;;;;38948:2162;39068:6;:14;;39078:4;39068:14;39064:2046;;39128:2;39133:1;39128:6;39138:1;39127:12;39122:2;:17;39114:26;;:3;:26;;;;39144:4;39113:35;39106:42;;;;;39064:2046;39217:6;:14;;39227:4;39217:14;39213:1897;;39277:2;39282:1;39277:6;39287:1;39276:12;39271:2;:17;39263:26;;:3;:26;;;;39293:6;39262:37;39255:44;;;;;39213:1897;39368:6;:14;;39378:4;39368:14;39364:1746;;-1:-1:-1;;39419:26:127;39443:1;39438;39433:6;;39432:12;39427:2;:17;39419:26;;39481:41;;;39557:5;39552:10;;39419:26;;;;;39551:18;39544:25;;39364:1746;39637:6;:14;;39647:4;39637:14;39633:1477;;-1:-1:-1;;39694:4:127;39688:34;39720:1;39715;39710:6;;39709:12;39704:2;:17;39688:34;;39778:27;;;39758:48;;;39836:10;;39689:9;;;39688:34;;39835:18;39828:25;;39633:1477;39921:6;:14;;39931:4;39921:14;39917:1193;;-1:-1:-1;;39978:6:127;39972:36;40006:1;40001;39996:6;;39995:12;39990:2;:17;39972:36;;40064:29;;;40044:50;;;40124:10;;39973:11;;;39972:36;;40123:18;40116:25;;39917:1193;40210:6;:14;;40220:4;40210:14;40206:904;;-1:-1:-1;;40261:20:127;40279:1;40274;40269:6;;40268:12;40261:20;;40317:36;;;40389:5;40383:11;;40261:20;;;;;40382:19;40375:26;;40206:904;40469:6;:14;;40479:4;40469:14;40465:645;;40514:2;40507:9;;;;;40465:645;40585:6;:14;;40595:4;40585:14;40581:529;;-1:-1:-1;;40636:25:127;40659:1;40654;40649:6;;40648:12;40643:2;:17;40636:25;;40697:41;;;40774:5;40768:11;;40636:25;;;;;40767:19;40760:26;;40581:529;40853:6;:14;;40863:4;40853:14;40849:261;;40898:3;40891:10;;;;;40849:261;40968:6;:14;;40978:4;40968:14;40964:146;;41013:2;41006:9;;;32450:8733;;;;;;;:::o;19960:782::-;20046:12;20133:18;;:::i;:::-;-1:-1:-1;20201:4:127;20308:2;20296:14;;;;20288:41;;;;;;;3226:2:286;20288:41:127;;;3208:21:286;3265:2;3245:18;;;3238:30;3304:16;3284:18;;;3277:44;3338:18;;20288:41:127;3024:338:286;20288:41:127;20425:14;;;;;;;:30;;;20443:12;20425:30;20421:102;;;20504:4;20475:5;:15;;;20491:9;20475:26;;;;;;;;;:::i;:::-;:33;;;;:26;;;;;;:33;20421:102;20578:12;;;;;20567:23;;;;:8;;;:23;20634:1;20619:16;;;20604:31;;;20712:13;:11;:13::i;5582:7764::-;5646:12;5732:18;;:::i;:::-;-1:-1:-1;5910:15:127;;:18;;;;5800:4;6070:18;;;;6114;;;;6158;;;;;5800:4;;5890:17;;;;6070:18;6114;6248;;;6262:4;6248:18;6244:6792;;6298:2;6327:4;6322:9;;:14;6318:144;;6438:4;6433:9;;6425:4;:18;6419:24;6318:144;6483:2;:7;;6489:1;6483:7;6479:161;;6519:10;;;;;6551:16;;;;;;;;6519:10;-1:-1:-1;6479:161:127;;;6619:2;6614:7;;6479:161;6268:386;6244:6792;;;6756:10;:18;;6770:4;6756:18;6752:6284;;1745:10;6794:14;;6752:6284;;;6892:10;:18;;6906:4;6892:18;6888:6148;;6935:1;6930:6;;6888:6148;;;7060:10;:18;;7074:4;7060:18;7056:5980;;7113:4;7098:12;;;:19;7135:26;;;:14;;;:26;7186:13;:11;:13::i;:::-;7179:20;5582:7764;-1:-1:-1;;;;;;;;;5582:7764:127:o;7056:5980::-;7325:10;:18;;7339:4;7325:18;7321:5715;;7476:14;;;7472:2723;7321:5715;7472:2723;7646:22;;;;;7642:2553;;7771:10;7784:27;7792:2;7797:10;7792:15;7809:1;7784:7;:27::i;:::-;7895:17;;;;7771:40;;-1:-1:-1;7895:17:127;7873:19;8045:14;8064:1;8039:26;8035:146;;1676:4:128;1670:11;;1533:21;1787:15;;;1828:8;1822:4;1815:22;1850:27;;;1996:4;1983:18;;2098:17;;2003:19;1979:44;2025:11;1976:61;8093:65:127;;8035:146;8267:20;;;;;8234:54;;;;;;;;3540:25:286;;;8234:54:127;3601:23:286;;;3581:18;;;3574:51;8203:11:127;;;;8234:19;:6;:19;;;;3513:18:286;;8234:54:127;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;:::i;:::-;8202:86;;;;8515:1;8511:2;8507:10;8612:9;8609:1;8605:17;8694:6;8687:5;8684:17;8681:40;;;8714:5;8704:15;;8681:40;;8797:6;8793:2;8790:14;8787:34;;;8817:2;8807:12;;8787:34;8923:3;8918:1;8910:6;8906:14;8901:3;8897:24;8893:34;8886:41;;9023:3;9019:1;9007:9;8998:6;8995:1;8991:14;8987:30;8983:38;8979:48;8972:55;;9178:1;9174;9170;9158:9;9155:1;9151:17;9147:25;9143:33;9139:41;9305:1;9301;9297;9288:6;9276:9;9273:1;9269:17;9265:30;9261:38;9257:46;9253:54;9235:72;;9436:10;9432:15;9426:4;9422:26;9414:34;;9552:3;9544:4;9540:9;9535:3;9531:19;9528:28;9521:35;;;;9698:33;9707:2;9712:10;9707:15;9724:1;9727:3;9698:8;:33::i;:::-;9753:20;;;:38;;;;;;;;;-1:-1:-1;7642:2553:127;;-1:-1:-1;;;7642:2553:127;;9910:18;;;;;9906:289;;10080:2;10075:7;;7321:5715;;9906:289;10134:10;10129:15;;2053:3;10166:10;;9906:289;7321:5715;;;10324:10;:18;;10338:4;10324:18;10320:2716;;10478:15;;;1824:1;10478:15;;:34;;-1:-1:-1;10497:15:127;;;1859:1;10497:15;10478:34;:57;;;-1:-1:-1;10516:19:127;;;1936:1;10516:19;10478:57;10474:1593;;;10564:2;10559:7;;10320:2716;;10474:1593;10690:23;;;;;10686:1381;;10737:10;10750:27;10758:2;10763:10;10758:15;10775:1;10750:7;:27::i;:::-;10853:17;;;;10737:40;;-1:-1:-1;11096:1:127;11088:10;;11190:1;11186:17;11265:13;;;11262:32;;;11287:5;11281:11;;11262:32;11573:14;;;11379:1;11569:22;;;11565:32;;;;11462:26;11486:1;11371:10;;;11466:18;;;11462:26;11561:43;11367:20;;11669:12;11797:17;;;:23;11865:1;11842:20;;;:24;11375:2;-1:-1:-1;11375:2:127;7321:5715;;10320:2716;12269:10;:18;;12283:4;12269:18;12265:771;;12379:2;:7;;12385:1;12379:7;12375:647;;12472:14;;;;;:40;;-1:-1:-1;12490:22:127;;;1978:1;12490:22;12472:40;:62;;;-1:-1:-1;12516:18:127;;;1897:1;12516:18;12472:62;12468:404;;;12567:1;12562:6;;12375:647;;12468:404;12613:15;;;1824:1;12613:15;;:34;;-1:-1:-1;12632:15:127;;;1859:1;12632:15;12613:34;:61;;;-1:-1:-1;12651:23:127;;;2021:1;12651:23;12613:61;:84;;;-1:-1:-1;12678:19:127;;;1936:1;12678:19;12613:84;12609:263;;;12730:1;12725:6;;7321:5715;;12375:647;12923:10;12918:15;;2087:4;12955:11;;12375:647;13111:15;;;;;:23;;;;:18;;;;:23;;;;13148:15;;:23;;;:18;;;;:23;-1:-1:-1;13237:12:127;;;;13226:23;;;:8;;;:23;13293:1;13278:16;13263:31;;;;;13316:13;:11;:13::i;16084:2480::-;16178:12;16264:18;;:::i;:::-;-1:-1:-1;16332:4:127;16364:10;16472:13;;;16481:4;16472:13;16468:1705;;-1:-1:-1;16511:8:127;;;;16468:1705;;;16630:5;:13;;16639:4;16630:13;16626:1547;;16663:14;;;:8;;;:14;16626:1547;;;16793:5;:13;;16802:4;16793:13;16789:1384;;-1:-1:-1;16832:8:127;;;;16789:1384;;;16951:5;:13;;16960:4;16951:13;16947:1226;;16984:14;;;:8;;;:14;16947:1226;;;17125:5;:13;;17134:4;17125:13;17121:1052;;17252:9;17198:17;17178;;;17198;;;;17178:37;17259:2;17252:9;;;;;17234:8;;;:28;17280:22;:8;;;:22;17121:1052;;;17439:5;:13;;17448:4;17439:13;17435:738;;17506:11;17492;;;17506;;;17492:25;17561:2;17554:9;;;;;17536:8;;;:28;17582:22;:8;;;:22;17435:738;;;17763:5;:13;;17772:4;17763:13;17759:414;;17833:3;17814:23;;17820:3;17814:23;;;;;;;:::i;:::-;;17796:42;;:8;;;:42;17874:23;;;;;;;;;;;;;:::i;:::-;;17856:42;;:8;;;:42;17759:414;;;18067:5;:13;;18076:4;18067:13;18063:110;;18117:3;18111:9;;:3;:9;;;;;;;:::i;:::-;;18100:20;;;;:8;;;:20;18149:9;;;;;;;;;;;:::i;:::-;;18138:20;;:8;;;:20;18063:110;18266:14;;;;18262:85;;18329:3;18300:5;:15;;;18316:9;18300:26;;;;;;;;;:::i;:::-;:32;;;;:26;;;;;;:32;18262:85;18401:12;;;;;18390:23;;;;:8;;;:23;18457:1;18442:16;;;18427:31;;;18534:13;:11;:13::i;:::-;18527:20;16084:2480;-1:-1:-1;;;;;;;16084:2480:127:o;23913:1654::-;24089:14;24106:24;24118:11;24106;:24::i;:::-;24089:41;;24238:1;24231:5;24227:13;24224:33;;;24253:1;24250;24243:12;24224:33;24392:2;24586:15;;;24411:2;24400:14;;24388:10;24384:31;24381:1;24377:39;24542:16;;;24327:20;;24527:10;24516:22;;;24512:27;24502:38;24499:60;25028:5;25025:1;25021:13;25099:1;25084:343;25109:2;25106:1;25103:9;25084:343;;;25232:2;25220:15;;;25169:20;25267:12;;;25281:1;25263:20;25304:42;;;;25372:1;25367:42;;;;25256:153;;25304:42;22391:1;22384:12;;;22424:2;22417:13;;;22469:2;22456:16;;25313:31;;25304:42;;25367;22391:1;22384:12;;;22424:2;22417:13;;;22469:2;22456:16;;25376:31;;25256:153;-1:-1:-1;;25127:1:127;25120:9;25084:343;;;-1:-1:-1;;25526:4:127;25519:18;-1:-1:-1;;;;23913:1654:127:o;20946:586::-;21268:20;;;21292:7;21268:32;21261:3;:40;;;21374:14;;21429:17;;21423:24;;;21415:72;;;;;;;4277:2:286;21415:72:127;;;4259:21:286;4316:2;4296:18;;;4289:30;4355:34;4335:18;;;4328:62;4426:5;4406:18;;;4399:33;4449:19;;21415:72:127;4075:399:286;21415:72:127;21501:14;20946:586;;;:::o;-1:-1:-1:-;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;:::i;:::-;;;;:::o;:::-;;;;;;;;;;;;;;;;;;;;;;;;:::o;467:347:286:-;518:8;528:6;582:3;575:4;567:6;563:17;559:27;549:55;;600:1;597;590:12;549:55;-1:-1:-1;623:20:286;;666:18;655:30;;652:50;;;698:1;695;688:12;652:50;735:4;727:6;723:17;711:29;;787:3;780:4;771:6;763;759:19;755:30;752:39;749:59;;;804:1;801;794:12;749:59;467:347;;;;;:::o;819:785::-;918:6;926;934;942;950;1003:2;991:9;982:7;978:23;974:32;971:52;;;1019:1;1016;1009:12;971:52;1059:9;1046:23;1088:18;1129:2;1121:6;1118:14;1115:34;;;1145:1;1142;1135:12;1115:34;1184:58;1234:7;1225:6;1214:9;1210:22;1184:58;:::i;:::-;1261:8;;-1:-1:-1;1158:84:286;-1:-1:-1;1349:2:286;1334:18;;1321:32;;-1:-1:-1;1365:16:286;;;1362:36;;;1394:1;1391;1384:12;1362:36;;1433:60;1485:7;1474:8;1463:9;1459:24;1433:60;:::i;:::-;819:785;;;;-1:-1:-1;1512:8:286;1594:2;1579:18;1566:32;;819:785;-1:-1:-1;;;;819:785:286:o;1791:184::-;1843:77;1840:1;1833:88;1940:4;1937:1;1930:15;1964:4;1961:1;1954:15;3636:245;3715:6;3723;3776:2;3764:9;3755:7;3751:23;3747:32;3744:52;;;3792:1;3789;3782:12;3744:52;-1:-1:-1;;3815:16:286;;3871:2;3856:18;;;3850:25;3815:16;;3850:25;;-1:-1:-1;3636:245:286:o;3886:184::-;3938:77;3935:1;3928:88;4035:4;4032:1;4025:15;4059:4;4056:1;4049:15" +var MIPSDeployedSourceMap = "1131:40054:126:-:0;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;1710:45;;1745:10;1710:45;;;;;188:10:285;176:23;;;158:42;;146:2;131:18;1710:45:126;;;;;;;;2448:99;;;412:42:285;2534:6:126;400:55:285;382:74;;370:2;355:18;2448:99:126;211:251:285;26025:6379:126;;;;;;:::i;:::-;;:::i;:::-;;;1755:25:285;;;1743:2;1728:18;26025:6379:126;1609:177:285;26025:6379:126;26128:7;26171:18;;:::i;:::-;26318:4;26311:5;26308:15;26298:134;;26412:1;26409;26402:12;26298:134;26468:4;26462:11;26475:10;26459:27;26449:136;;26565:1;26562;26555:12;26449:136;26634:3;26615:17;26612:26;26602:151;;26733:1;26730;26723:12;26602:151;26798:3;26783:13;26780:22;26770:146;;26896:1;26893;26886:12;26770:146;27176:24;;27521:4;27222:20;27579:2;27280:21;;27176:24;27338:18;27222:20;27280:21;;;27176:24;27153:21;27149:52;;;27338:18;27222:20;;;27280:21;;;27176:24;27149:52;;27222:20;;27280:21;;;27176:24;27149:52;;27338:18;27222:20;27280:21;;;27176:24;27149:52;;27338:18;27222:20;27280:21;;;27176:24;27149:52;;27338:18;27222:20;27280:21;;;27176:24;27149:52;;;27338:18;27222:20;27280:21;;;27176:24;27153:21;27149:52;;;27338:18;27222:20;27280:21;;;27176:24;27149:52;;27338:18;27222:20;27280:21;;;27176:24;27149:52;;27338:18;27222:20;28197:10;27338:18;28187:21;;;27280;;;;28295:1;28280:77;28305:2;28302:1;28299:9;28280:77;;;27176:24;;27153:21;27149:52;27222:20;;28353:1;27280:21;;;;27164:2;27338:18;;;;28323:1;28316:9;28280:77;;;28284:14;;;28435:5;:12;;;28431:71;;;28474:13;:11;:13::i;:::-;28467:20;;;;;28431:71;28516:10;;;:15;;28530:1;28516:15;;;;;28601:8;;;;-1:-1:-1;;28593:20:126;;-1:-1:-1;28593:7:126;:20::i;:::-;28579:34;-1:-1:-1;28643:10:126;28651:2;28643:10;;;;28720:1;28710:11;;;:26;;;28725:6;:11;;28735:1;28725:11;28710:26;28706:310;;;28866:13;28935:1;28913:4;28920:10;28913:17;28912:24;;;;28883:5;:12;;;28898:10;28883:25;28882:54;28866:70;;28961:40;28972:6;:11;;28982:1;28972:11;:20;;28990:2;28972:20;;;28986:1;28972:20;28961:40;;28994:6;28961:10;:40::i;:::-;28954:47;;;;;;;;28706:310;29265:15;;;;29060:9;;;;29197:4;29191:2;29183:10;;;29182:19;;;29265:15;29290:2;29282:10;;;29281:19;29265:36;;;;;;;:::i;:::-;;;;;;-1:-1:-1;29330:5:126;29354:11;;;;;:29;;;29369:6;:14;;29379:4;29369:14;29354:29;29350:832;;;29446:5;:15;;;29462:5;29446:22;;;;;;;;;:::i;:::-;;;;;;-1:-1:-1;;29509:4:126;29503:2;29495:10;;;29494:19;29350:832;;;29547:4;29538:6;:13;;;29534:648;;;29668:6;:13;;29678:3;29668:13;:30;;;;29685:6;:13;;29695:3;29685:13;29668:30;:47;;;;29702:6;:13;;29712:3;29702:13;29668:47;29664:253;;;29778:4;29785:6;29778:13;29773:18;;29534:648;;29664:253;29877:21;29880:4;29887:6;29880:13;29895:2;29877;:21::i;:::-;29872:26;;29534:648;;;29951:4;29941:6;:14;;;;:32;;;;29959:6;:14;;29969:4;29959:14;29941:32;:50;;;;29977:6;:14;;29987:4;29977:14;29941:50;29937:245;;;30061:5;:15;;;30077:5;30061:22;;;;;;;;;:::i;:::-;;;;;30056:27;;30162:5;30154:13;;29937:245;30211:1;30201:6;:11;;;;:25;;;;;30225:1;30216:6;:10;;;30201:25;30200:42;;;;30231:6;:11;;30241:1;30231:11;30200:42;30196:125;;;30269:37;30282:6;30290:4;30296:5;30303:2;30269:12;:37::i;:::-;30262:44;;;;;;;;;;;30196:125;30354:13;30335:16;30506:4;30496:14;;;;30492:446;;30575:21;30578:4;30585:6;30578:13;30593:2;30575;:21::i;:::-;30569:27;;;;30633:10;30628:15;;30667:16;30628:15;30681:1;30667:7;:16::i;:::-;30661:22;;30715:4;30705:6;:14;;;;:32;;;;;30723:6;:14;;30733:4;30723:14;;30705:32;30701:223;;;30802:4;30790:16;;30904:1;30896:9;;30701:223;30512:426;30492:446;30971:10;30984:26;30992:4;30998:2;31002;31006:3;30984:7;:26::i;:::-;31013:10;30984:39;;;;-1:-1:-1;31109:4:126;31102:11;;;31141;;;:24;;;;;31164:1;31156:4;:9;;;;31141:24;:39;;;;;31176:4;31169;:11;;;31141:39;31137:860;;;31204:4;:9;;31212:1;31204:9;:22;;;;31217:4;:9;;31225:1;31217:9;31204:22;31200:144;;;31288:37;31299:4;:9;;31307:1;31299:9;:21;;31315:5;31299:21;;;31311:1;31299:21;31322:2;31288:10;:37::i;:::-;31281:44;;;;;;;;;;;;;;;31200:144;31366:4;:11;;31374:3;31366:11;31362:121;;31436:28;31445:5;31452:2;31456:7;;;;31436:8;:28::i;31362:121::-;31504:4;:11;;31512:3;31504:11;31500:121;;31574:28;31583:5;31590:2;31594:7;;;;;31574:8;:28::i;31500:121::-;31691:4;:11;;31699:3;31691:11;31687:93;;31733:28;31747:13;31733;:28::i;31687:93::-;31883:4;31875;:12;;;;:27;;;;;31898:4;31891;:11;;;31875:27;31871:112;;;31933:31;31944:4;31950:2;31954;31958:5;31933:10;:31::i;31871:112::-;32057:6;:14;;32067:4;32057:14;:28;;;;-1:-1:-1;32075:10:126;;;;;32057:28;32053:93;;;32130:1;32105:5;:15;;;32121:5;32105:22;;;;;;;;;:::i;:::-;:26;;;;:22;;;;;;:26;32053:93;32192:9;:26;;32205:13;32192:26;32188:92;;32238:27;32247:9;32258:1;32261:3;32238:8;:27::i;:::-;32361:26;32370:5;32377:3;32382:4;32361:8;:26::i;:::-;32354:33;;;;;;;;;;;;;26025:6379;;;;;;;;:::o;3087:2334::-;3634:4;3628:11;;3550:4;3353:31;3342:43;;3413:13;3353:31;3752:2;3452:13;;3342:43;3359:24;3353:31;3452:13;;;3342:43;;;;3359:24;3353:31;3452:13;;;3342:43;3359:24;3353:31;3452:13;;;3342:43;3359:24;3353:31;3452:13;;;3342:43;3359:24;3353:31;3452:13;;;3342:43;3359:24;3353:31;3452:13;;;3342:43;3413:13;4180:11;3359:24;3353:31;3452:13;;;3342:43;3413:13;4275:11;3359:24;3353:31;3452:13;;;3342:43;3359:24;3353:31;3452:13;;;3342:43;3128:12;;4415:13;;3628:11;;3452:13;;;;4180:11;3128:12;4495:84;4520:2;4517:1;4514:9;4495:84;;;3369:13;3359:24;;3353:31;3342:43;;3373:2;3413:13;;;;4575:1;3452:13;;;;4538:1;4531:9;4495:84;;;4499:14;4642:1;4638:2;4631:13;4737:5;4733:2;4729:14;4722:5;4717:27;4811:1;4797:15;;4832:6;4856:1;4851:273;;;;5191:1;5181:11;;4825:369;;4851:273;4883:8;4941:22;;;;5020:1;5015:22;;;;5107:1;5097:11;;4876:234;;4941:22;4960:1;4950:11;;4941:22;;5015;5034:1;5024:11;;4876:234;;4825:369;-1:-1:-1;;;5317:14:126;;;5300:32;;5360:19;5356:30;5392:3;5388:16;;;;5353:52;;3087:2334;-1:-1:-1;3087:2334:126:o;21746:1831::-;21819:11;21930:14;21947:24;21959:11;21947;:24::i;:::-;21930:41;;22079:1;22072:5;22068:13;22065:33;;;22094:1;22091;22084:12;22065:33;22227:2;22215:15;;;22168:20;22657:5;22654:1;22650:13;22692:4;22728:1;22713:343;22738:2;22735:1;22732:9;22713:343;;;22861:2;22849:15;;;22798:20;22896:12;;;22910:1;22892:20;22933:42;;;;23001:1;22996:42;;;;22885:153;;22933:42;22391:1;22384:12;;;22424:2;22417:13;;;22469:2;22456:16;;22942:31;;22933:42;;22996;22391:1;22384:12;;;22424:2;22417:13;;;22469:2;22456:16;;23005:31;;22885:153;-1:-1:-1;;22756:1:126;22749:9;22713:343;;;22717:14;23166:4;23160:11;23145:26;;23252:7;23246:4;23243:17;23233:124;;23294:10;23291:1;23284:21;23336:2;23333:1;23326:13;23233:124;-1:-1:-1;;23484:2:126;23473:14;;;;23461:10;23457:31;23454:1;23450:39;23518:16;;;;23536:10;23514:33;;21746:1831;-1:-1:-1;;;21746:1831:126:o;18856:823::-;18925:12;19012:18;;:::i;:::-;19080:4;19071:13;;19132:5;:8;;;19143:1;19132:12;19116:28;;:5;:12;;;:28;;;19112:95;;19164:28;;;;;2182:2:285;19164:28:126;;;2164:21:285;2221:2;2201:18;;;2194:30;2260:20;2240:18;;;2233:48;2298:18;;19164:28:126;;;;;;;;19112:95;19296:8;;;;;19329:12;;;;;19318:23;;;;;;;19355:20;;;;;19296:8;19487:13;;;19483:90;;19548:6;19557:1;19548:10;19520:5;:15;;;19536:8;19520:25;;;;;;;;;:::i;:::-;:38;;;;:25;;;;;;:38;19483:90;19649:13;:11;:13::i;2645:339::-;2706:11;2770:18;;;;2779:8;;;;2770:18;;;;;;2769:25;;;;;2786:1;2833:2;:9;;;2827:16;;;;;2826:22;;2825:32;;;;;;;2887:9;;2886:15;2769:25;2944:21;;2964:1;2944:21;;;2955:6;2944:21;2929:11;;;;;:37;;-1:-1:-1;;;2645:339:126;;;;:::o;13732:2026::-;13829:12;13915:18;;:::i;:::-;13983:4;13974:13;;14015:17;14075:5;:8;;;14086:1;14075:12;14059:28;;:5;:12;;;:28;;;14055:97;;14107:30;;;;;2529:2:285;14107:30:126;;;2511:21:285;2568:2;2548:18;;;2541:30;2607:22;2587:18;;;2580:50;2647:18;;14107:30:126;2327:344:285;14055:97:126;14222:7;:12;;14233:1;14222:12;:28;;;;14238:7;:12;;14249:1;14238:12;14222:28;14218:947;;;14270:9;14282:5;:15;;;14298:6;14282:23;;;;;;;;;:::i;:::-;;;;;14270:35;;14346:2;14339:9;;:3;:9;;;:25;;;;;14352:7;:12;;14363:1;14352:12;14339:25;14338:58;;;;14377:2;14370:9;;:3;:9;;;;:25;;;;;14383:7;:12;;14394:1;14383:12;14370:25;14323:73;;14252:159;14218:947;;;14508:7;:12;;14519:1;14508:12;14504:661;;14569:1;14561:3;14555:15;;;;14540:30;;14504:661;;;14673:7;:12;;14684:1;14673:12;14669:496;;14733:1;14726:3;14720:14;;;14705:29;;14669:496;;;14854:7;:12;;14865:1;14854:12;14850:315;;14942:4;14936:2;14927:11;;;14926:20;14912:10;14969:8;;;14965:84;;15029:1;15022:3;15016:14;;;15001:29;;14965:84;15070:3;:8;;15077:1;15070:8;15066:85;;15131:1;15123:3;15117:15;;;;15102:30;;15066:85;14868:297;14850:315;15241:8;;;;;15319:12;;;;15308:23;;;;;15475:178;;;;15566:1;15540:22;15543:5;15551:6;15543:14;15559:2;15540;:22::i;:::-;:27;;;;;;;15526:42;;15535:1;15526:42;15511:57;:12;;;:57;15475:178;;;15622:12;;;;;15637:1;15622:16;15607:31;;;;15475:178;15728:13;:11;:13::i;:::-;15721:20;13732:2026;-1:-1:-1;;;;;;;;13732:2026:126:o;32450:8733::-;32537:10;32599;32607:2;32599:10;;;;32638:11;;;:44;;;32664:1;32654:6;:11;;;;:27;;;;;32678:3;32669:6;:12;;;32654:27;32634:8490;;;32723:4;32716:11;;32847:6;32907:3;32902:25;;;;32982:3;32977:25;;;;33056:3;33051:25;;;;33131:3;33126:25;;;;33205:3;33200:25;;;;33278:3;33273:25;;;;33352:3;33347:25;;;;32840:532;;32902:25;32921:4;32913:12;;32902:25;;32977;32996:4;32988:12;;32977:25;;33051;33070:4;33062:12;;33051:25;;33126;33145:4;33137:12;;33126:25;;33200;33219:4;33211:12;;33200:25;;33273;33292:4;33284:12;;33273:25;;33347;33366:4;33358:12;;32840:532;;33435:4;:12;;33443:4;33435:12;33431:4023;;-1:-1:-1;;;33486:9:126;33478:26;;33499:4;33494:1;33486:9;;;33485:18;33478:26;33471:33;;33431:4023;33572:4;:12;;33580:4;33572:12;33568:3886;;-1:-1:-1;;;33623:9:126;33615:26;;33636:4;33631:1;33623:9;;;33622:18;33615:26;33608:33;;33568:3886;33709:4;:12;;33717:4;33709:12;33705:3749;;33774:4;33769:1;33761:9;;;33760:18;33807:27;33761:9;33810:11;;;;33823:2;:10;;;33807:2;:27::i;:::-;33800:34;;;;;;;33705:3749;33903:4;:12;;33911:4;33903:12;33899:3555;;-1:-1:-1;;;33946:17:126;;;33958:4;33953:9;;33946:17;33939:24;;33899:3555;34032:4;:11;;34040:3;34032:11;34028:3426;;-1:-1:-1;;;34074:17:126;;;34086:4;34081:9;;34074:17;34067:24;;34028:3426;34160:4;:12;;34168:4;34160:12;34156:3298;;34203:21;34212:2;34206:8;;:2;:8;;;;34221:2;34216;:7;34203:2;:21::i;:::-;34196:28;;;;;;34156:3298;34473:4;:12;;34481:4;34473:12;34469:2985;;34516:2;34509:9;;;;;;34469:2985;34587:4;:12;;34595:4;34587:12;34583:2871;;34630:2;34623:9;;;;;;34583:2871;34701:4;:12;;34709:4;34701:12;34697:2757;;34744:2;34737:9;;;;;;34697:2757;34815:4;:12;;34823:4;34815:12;34811:2643;;34858:2;34851:9;;;;;;34811:2643;34932:4;:12;;34940:4;34932:12;34928:2526;;34975:2;34968:9;;;;;;34928:2526;35092:4;:12;;35100:4;35092:12;35088:2366;;35135:2;35128:9;;;;;;35088:2366;35206:4;:12;;35214:4;35206:12;35202:2252;;35249:2;35242:9;;;;;;35202:2252;35320:4;:12;;35328:4;35320:12;35316:2138;;35363:2;35356:9;;;;;;35316:2138;35434:4;:12;;35442:4;35434:12;35430:2024;;35477:2;35470:9;;;;;;35430:2024;35548:4;:12;;35556:4;35548:12;35544:1910;;35591:2;35584:9;;;;;;35544:1910;35662:4;:12;;35670:4;35662:12;35658:1796;;35705:2;35698:9;;;;;;35658:1796;35777:4;:12;;35785:4;35777:12;35773:1681;;35820:2;35813:9;;;;;;35773:1681;35890:4;:12;;35898:4;35890:12;35886:1568;;35933:2;35926:9;;;;;;35886:1568;36004:4;:12;;36012:4;36004:12;36000:1454;;36047:2;36040:9;;;;;;36000:1454;36196:4;:12;;36204:4;36196:12;36192:1262;;-1:-1:-1;;;36240:7:126;;;36232:16;;36192:1262;36317:4;:12;;36325:4;36317:12;36313:1141;;-1:-1:-1;;;36361:7:126;;;36353:16;;36313:1141;36437:4;:12;;36445:4;36437:12;36433:1021;;-1:-1:-1;;;36481:7:126;;;36473:16;;36433:1021;36558:4;:12;;36566:4;36558:12;36554:900;;-1:-1:-1;;;36602:7:126;;;36594:16;;36554:900;36678:4;:12;;36686:4;36678:12;36674:780;;-1:-1:-1;;;36722:7:126;;;36714:16;;36674:780;36797:4;:12;;36805:4;36797:12;36793:661;;-1:-1:-1;;;36841:7:126;;;36833:16;;36793:661;36917:4;:12;;36925:4;36917:12;36913:541;;-1:-1:-1;;;36961:7:126;;;36953:16;;36913:541;37037:4;:12;;37045:4;37037:12;37033:421;;-1:-1:-1;;;37082:7:126;;;37080:10;37073:17;;37033:421;37159:4;:12;;37167:4;37159:12;37155:299;;37220:2;37202:21;;37208:2;37202:21;;;:29;;37230:1;37202:29;;;37226:1;37202:29;37195:36;;;;;;;;37155:299;37301:4;:12;;37309:4;37301:12;37297:157;;37349:2;37344:7;;:2;:7;;;:15;;37358:1;37344:15;;37297:157;37406:29;;;;;2878:2:285;37406:29:126;;;2860:21:285;2917:2;2897:18;;;2890:30;2956:21;2936:18;;;2929:49;2995:18;;37406:29:126;2676:343:285;37297:157:126;32684:4784;32634:8490;;;37524:6;:14;;37534:4;37524:14;37520:3590;;37583:4;37576:11;;37658:3;37650:11;;;37646:549;;-1:-1:-1;;;37703:21:126;;;37689:36;;37646:549;37810:4;:12;;37818:4;37810:12;:28;;;;37826:4;:12;;37834:4;37826:12;37810:28;37806:389;;;37870:4;:12;;37878:4;37870:12;37866:83;;37919:3;;;37866:83;37974:8;38012:127;38024:10;38019:15;;:20;38012:127;;38104:8;38071:3;38104:8;;;;;38071:3;38012:127;;;38171:1;-1:-1:-1;38164:8:126;;-1:-1:-1;;38164:8:126;37520:3590;38262:6;:14;;38272:4;38262:14;38258:2852;;-1:-1:-1;;38307:8:126;38313:2;38307:8;;;;38300:15;;38258:2852;38382:6;:14;;38392:4;38382:14;38378:2732;;38427:42;38445:2;38450:1;38445:6;38455:1;38444:12;38439:2;:17;38431:26;;:3;:26;;;;38461:4;38430:35;38467:1;38427:2;:42::i;:::-;38420:49;;;;;38378:2732;38536:6;:14;;38546:4;38536:14;38532:2578;;38581:45;38599:2;38604:1;38599:6;38609:1;38598:12;38593:2;:17;38585:26;;:3;:26;;;;38615:6;38584:37;38623:2;38581;:45::i;38532:2578::-;38694:6;:14;;38704:4;38694:14;38690:2420;;-1:-1:-1;;38745:21:126;38764:1;38759;38754:6;;38753:12;38745:21;;38802:36;;;38873:5;38868:10;;38745:21;;;;;38867:18;38860:25;;38690:2420;38952:6;:14;;38962:4;38952:14;38948:2162;;38997:3;38990:10;;;;;38948:2162;39068:6;:14;;39078:4;39068:14;39064:2046;;39128:2;39133:1;39128:6;39138:1;39127:12;39122:2;:17;39114:26;;:3;:26;;;;39144:4;39113:35;39106:42;;;;;39064:2046;39217:6;:14;;39227:4;39217:14;39213:1897;;39277:2;39282:1;39277:6;39287:1;39276:12;39271:2;:17;39263:26;;:3;:26;;;;39293:6;39262:37;39255:44;;;;;39213:1897;39368:6;:14;;39378:4;39368:14;39364:1746;;-1:-1:-1;;39419:26:126;39443:1;39438;39433:6;;39432:12;39427:2;:17;39419:26;;39481:41;;;39557:5;39552:10;;39419:26;;;;;39551:18;39544:25;;39364:1746;39637:6;:14;;39647:4;39637:14;39633:1477;;-1:-1:-1;;39694:4:126;39688:34;39720:1;39715;39710:6;;39709:12;39704:2;:17;39688:34;;39778:27;;;39758:48;;;39836:10;;39689:9;;;39688:34;;39835:18;39828:25;;39633:1477;39921:6;:14;;39931:4;39921:14;39917:1193;;-1:-1:-1;;39978:6:126;39972:36;40006:1;40001;39996:6;;39995:12;39990:2;:17;39972:36;;40064:29;;;40044:50;;;40124:10;;39973:11;;;39972:36;;40123:18;40116:25;;39917:1193;40210:6;:14;;40220:4;40210:14;40206:904;;-1:-1:-1;;40261:20:126;40279:1;40274;40269:6;;40268:12;40261:20;;40317:36;;;40389:5;40383:11;;40261:20;;;;;40382:19;40375:26;;40206:904;40469:6;:14;;40479:4;40469:14;40465:645;;40514:2;40507:9;;;;;40465:645;40585:6;:14;;40595:4;40585:14;40581:529;;-1:-1:-1;;40636:25:126;40659:1;40654;40649:6;;40648:12;40643:2;:17;40636:25;;40697:41;;;40774:5;40768:11;;40636:25;;;;;40767:19;40760:26;;40581:529;40853:6;:14;;40863:4;40853:14;40849:261;;40898:3;40891:10;;;;;40849:261;40968:6;:14;;40978:4;40968:14;40964:146;;41013:2;41006:9;;;32450:8733;;;;;;;:::o;19960:782::-;20046:12;20133:18;;:::i;:::-;-1:-1:-1;20201:4:126;20308:2;20296:14;;;;20288:41;;;;;;;3226:2:285;20288:41:126;;;3208:21:285;3265:2;3245:18;;;3238:30;3304:16;3284:18;;;3277:44;3338:18;;20288:41:126;3024:338:285;20288:41:126;20425:14;;;;;;;:30;;;20443:12;20425:30;20421:102;;;20504:4;20475:5;:15;;;20491:9;20475:26;;;;;;;;;:::i;:::-;:33;;;;:26;;;;;;:33;20421:102;20578:12;;;;;20567:23;;;;:8;;;:23;20634:1;20619:16;;;20604:31;;;20712:13;:11;:13::i;5582:7764::-;5646:12;5732:18;;:::i;:::-;-1:-1:-1;5910:15:126;;:18;;;;5800:4;6070:18;;;;6114;;;;6158;;;;;5800:4;;5890:17;;;;6070:18;6114;6248;;;6262:4;6248:18;6244:6792;;6298:2;6327:4;6322:9;;:14;6318:144;;6438:4;6433:9;;6425:4;:18;6419:24;6318:144;6483:2;:7;;6489:1;6483:7;6479:161;;6519:10;;;;;6551:16;;;;;;;;6519:10;-1:-1:-1;6479:161:126;;;6619:2;6614:7;;6479:161;6268:386;6244:6792;;;6756:10;:18;;6770:4;6756:18;6752:6284;;1745:10;6794:14;;6752:6284;;;6892:10;:18;;6906:4;6892:18;6888:6148;;6935:1;6930:6;;6888:6148;;;7060:10;:18;;7074:4;7060:18;7056:5980;;7113:4;7098:12;;;:19;7135:26;;;:14;;;:26;7186:13;:11;:13::i;:::-;7179:20;5582:7764;-1:-1:-1;;;;;;;;;5582:7764:126:o;7056:5980::-;7325:10;:18;;7339:4;7325:18;7321:5715;;7476:14;;;7472:2723;7321:5715;7472:2723;7646:22;;;;;7642:2553;;7771:10;7784:27;7792:2;7797:10;7792:15;7809:1;7784:7;:27::i;:::-;7895:17;;;;7771:40;;-1:-1:-1;7895:17:126;7873:19;8045:14;8064:1;8039:26;8035:146;;1676:4:127;1670:11;;1533:21;1787:15;;;1828:8;1822:4;1815:22;1850:27;;;1996:4;1983:18;;2098:17;;2003:19;1979:44;2025:11;1976:61;8093:65:126;;8035:146;8267:20;;;;;8234:54;;;;;;;;3540:25:285;;;8234:54:126;3601:23:285;;;3581:18;;;3574:51;8203:11:126;;;;8234:19;:6;:19;;;;3513:18:285;;8234:54:126;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;:::i;:::-;8202:86;;;;8515:1;8511:2;8507:10;8612:9;8609:1;8605:17;8694:6;8687:5;8684:17;8681:40;;;8714:5;8704:15;;8681:40;;8797:6;8793:2;8790:14;8787:34;;;8817:2;8807:12;;8787:34;8923:3;8918:1;8910:6;8906:14;8901:3;8897:24;8893:34;8886:41;;9023:3;9019:1;9007:9;8998:6;8995:1;8991:14;8987:30;8983:38;8979:48;8972:55;;9178:1;9174;9170;9158:9;9155:1;9151:17;9147:25;9143:33;9139:41;9305:1;9301;9297;9288:6;9276:9;9273:1;9269:17;9265:30;9261:38;9257:46;9253:54;9235:72;;9436:10;9432:15;9426:4;9422:26;9414:34;;9552:3;9544:4;9540:9;9535:3;9531:19;9528:28;9521:35;;;;9698:33;9707:2;9712:10;9707:15;9724:1;9727:3;9698:8;:33::i;:::-;9753:20;;;:38;;;;;;;;;-1:-1:-1;7642:2553:126;;-1:-1:-1;;;7642:2553:126;;9910:18;;;;;9906:289;;10080:2;10075:7;;7321:5715;;9906:289;10134:10;10129:15;;2053:3;10166:10;;9906:289;7321:5715;;;10324:10;:18;;10338:4;10324:18;10320:2716;;10478:15;;;1824:1;10478:15;;:34;;-1:-1:-1;10497:15:126;;;1859:1;10497:15;10478:34;:57;;;-1:-1:-1;10516:19:126;;;1936:1;10516:19;10478:57;10474:1593;;;10564:2;10559:7;;10320:2716;;10474:1593;10690:23;;;;;10686:1381;;10737:10;10750:27;10758:2;10763:10;10758:15;10775:1;10750:7;:27::i;:::-;10853:17;;;;10737:40;;-1:-1:-1;11096:1:126;11088:10;;11190:1;11186:17;11265:13;;;11262:32;;;11287:5;11281:11;;11262:32;11573:14;;;11379:1;11569:22;;;11565:32;;;;11462:26;11486:1;11371:10;;;11466:18;;;11462:26;11561:43;11367:20;;11669:12;11797:17;;;:23;11865:1;11842:20;;;:24;11375:2;-1:-1:-1;11375:2:126;7321:5715;;10320:2716;12269:10;:18;;12283:4;12269:18;12265:771;;12379:2;:7;;12385:1;12379:7;12375:647;;12472:14;;;;;:40;;-1:-1:-1;12490:22:126;;;1978:1;12490:22;12472:40;:62;;;-1:-1:-1;12516:18:126;;;1897:1;12516:18;12472:62;12468:404;;;12567:1;12562:6;;12375:647;;12468:404;12613:15;;;1824:1;12613:15;;:34;;-1:-1:-1;12632:15:126;;;1859:1;12632:15;12613:34;:61;;;-1:-1:-1;12651:23:126;;;2021:1;12651:23;12613:61;:84;;;-1:-1:-1;12678:19:126;;;1936:1;12678:19;12613:84;12609:263;;;12730:1;12725:6;;7321:5715;;12375:647;12923:10;12918:15;;2087:4;12955:11;;12375:647;13111:15;;;;;:23;;;;:18;;;;:23;;;;13148:15;;:23;;;:18;;;;:23;-1:-1:-1;13237:12:126;;;;13226:23;;;:8;;;:23;13293:1;13278:16;13263:31;;;;;13316:13;:11;:13::i;16084:2480::-;16178:12;16264:18;;:::i;:::-;-1:-1:-1;16332:4:126;16364:10;16472:13;;;16481:4;16472:13;16468:1705;;-1:-1:-1;16511:8:126;;;;16468:1705;;;16630:5;:13;;16639:4;16630:13;16626:1547;;16663:14;;;:8;;;:14;16626:1547;;;16793:5;:13;;16802:4;16793:13;16789:1384;;-1:-1:-1;16832:8:126;;;;16789:1384;;;16951:5;:13;;16960:4;16951:13;16947:1226;;16984:14;;;:8;;;:14;16947:1226;;;17125:5;:13;;17134:4;17125:13;17121:1052;;17252:9;17198:17;17178;;;17198;;;;17178:37;17259:2;17252:9;;;;;17234:8;;;:28;17280:22;:8;;;:22;17121:1052;;;17439:5;:13;;17448:4;17439:13;17435:738;;17506:11;17492;;;17506;;;17492:25;17561:2;17554:9;;;;;17536:8;;;:28;17582:22;:8;;;:22;17435:738;;;17763:5;:13;;17772:4;17763:13;17759:414;;17833:3;17814:23;;17820:3;17814:23;;;;;;;:::i;:::-;;17796:42;;:8;;;:42;17874:23;;;;;;;;;;;;;:::i;:::-;;17856:42;;:8;;;:42;17759:414;;;18067:5;:13;;18076:4;18067:13;18063:110;;18117:3;18111:9;;:3;:9;;;;;;;:::i;:::-;;18100:20;;;;:8;;;:20;18149:9;;;;;;;;;;;:::i;:::-;;18138:20;;:8;;;:20;18063:110;18266:14;;;;18262:85;;18329:3;18300:5;:15;;;18316:9;18300:26;;;;;;;;;:::i;:::-;:32;;;;:26;;;;;;:32;18262:85;18401:12;;;;;18390:23;;;;:8;;;:23;18457:1;18442:16;;;18427:31;;;18534:13;:11;:13::i;:::-;18527:20;16084:2480;-1:-1:-1;;;;;;;16084:2480:126:o;23913:1654::-;24089:14;24106:24;24118:11;24106;:24::i;:::-;24089:41;;24238:1;24231:5;24227:13;24224:33;;;24253:1;24250;24243:12;24224:33;24392:2;24586:15;;;24411:2;24400:14;;24388:10;24384:31;24381:1;24377:39;24542:16;;;24327:20;;24527:10;24516:22;;;24512:27;24502:38;24499:60;25028:5;25025:1;25021:13;25099:1;25084:343;25109:2;25106:1;25103:9;25084:343;;;25232:2;25220:15;;;25169:20;25267:12;;;25281:1;25263:20;25304:42;;;;25372:1;25367:42;;;;25256:153;;25304:42;22391:1;22384:12;;;22424:2;22417:13;;;22469:2;22456:16;;25313:31;;25304:42;;25367;22391:1;22384:12;;;22424:2;22417:13;;;22469:2;22456:16;;25376:31;;25256:153;-1:-1:-1;;25127:1:126;25120:9;25084:343;;;-1:-1:-1;;25526:4:126;25519:18;-1:-1:-1;;;;23913:1654:126:o;20946:586::-;21268:20;;;21292:7;21268:32;21261:3;:40;;;21374:14;;21429:17;;21423:24;;;21415:72;;;;;;;4277:2:285;21415:72:126;;;4259:21:285;4316:2;4296:18;;;4289:30;4355:34;4335:18;;;4328:62;4426:5;4406:18;;;4399:33;4449:19;;21415:72:126;4075:399:285;21415:72:126;21501:14;20946:586;;;:::o;-1:-1:-1:-;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;:::i;:::-;;;;:::o;:::-;;;;;;;;;;;;;;;;;;;;;;;;:::o;467:347:285:-;518:8;528:6;582:3;575:4;567:6;563:17;559:27;549:55;;600:1;597;590:12;549:55;-1:-1:-1;623:20:285;;666:18;655:30;;652:50;;;698:1;695;688:12;652:50;735:4;727:6;723:17;711:29;;787:3;780:4;771:6;763;759:19;755:30;752:39;749:59;;;804:1;801;794:12;749:59;467:347;;;;;:::o;819:785::-;918:6;926;934;942;950;1003:2;991:9;982:7;978:23;974:32;971:52;;;1019:1;1016;1009:12;971:52;1059:9;1046:23;1088:18;1129:2;1121:6;1118:14;1115:34;;;1145:1;1142;1135:12;1115:34;1184:58;1234:7;1225:6;1214:9;1210:22;1184:58;:::i;:::-;1261:8;;-1:-1:-1;1158:84:285;-1:-1:-1;1349:2:285;1334:18;;1321:32;;-1:-1:-1;1365:16:285;;;1362:36;;;1394:1;1391;1384:12;1362:36;;1433:60;1485:7;1474:8;1463:9;1459:24;1433:60;:::i;:::-;819:785;;;;-1:-1:-1;1512:8:285;1594:2;1579:18;1566:32;;819:785;-1:-1:-1;;;;819:785:285:o;1791:184::-;1843:77;1840:1;1833:88;1940:4;1937:1;1930:15;1964:4;1961:1;1954:15;3636:245;3715:6;3723;3776:2;3764:9;3755:7;3751:23;3747:32;3744:52;;;3792:1;3789;3782:12;3744:52;-1:-1:-1;;3815:16:285;;3871:2;3856:18;;;3850:25;3815:16;;3850:25;;-1:-1:-1;3636:245:285:o;3886:184::-;3938:77;3935:1;3928:88;4035:4;4032:1;4025:15;4059:4;4056:1;4049:15" func init() { if err := json.Unmarshal([]byte(MIPSStorageLayoutJSON), MIPSStorageLayout); err != nil { diff --git a/op-bindings/bindings/preimageoracle_more.go b/op-bindings/bindings/preimageoracle_more.go index 54abd4665e9b..87c5194ee3bc 100644 --- a/op-bindings/bindings/preimageoracle_more.go +++ b/op-bindings/bindings/preimageoracle_more.go @@ -15,7 +15,7 @@ var PreimageOracleStorageLayout = new(solc.StorageLayout) var PreimageOracleDeployedBin = "0x608060405234801561001057600080fd5b50600436106100725760003560e01c8063e03110e111610050578063e03110e114610106578063e15926111461012e578063fef2b4ed1461014357600080fd5b806361238bde146100775780638542cf50146100b5578063c0c220c9146100f3575b600080fd5b6100a26100853660046104df565b600160209081526000928352604080842090915290825290205481565b6040519081526020015b60405180910390f35b6100e36100c33660046104df565b600260209081526000928352604080842090915290825290205460ff1681565b60405190151581526020016100ac565b6100a2610101366004610501565b610163565b6101196101143660046104df565b610238565b604080519283526020830191909152016100ac565b61014161013c36600461053c565b610329565b005b6100a26101513660046105b8565b60006020819052908152604090205481565b600061016f8686610432565b905061017c836008610600565b8211806101895750602083115b156101c0576040517ffe25498700000000000000000000000000000000000000000000000000000000815260040160405180910390fd5b6000602081815260c085901b82526008959095528251828252600286526040808320858452875280832080547fffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff001660019081179091558484528752808320948352938652838220558181529384905292205592915050565b6000828152600260209081526040808320848452909152812054819060ff166102c1576040517f08c379a000000000000000000000000000000000000000000000000000000000815260206004820152601460248201527f7072652d696d616765206d757374206578697374000000000000000000000000604482015260640160405180910390fd5b50600083815260208181526040909120546102dd816008610600565b6102e8856020610600565b1061030657836102f9826008610600565b6103039190610618565b91505b506000938452600160209081526040808620948652939052919092205492909150565b604435600080600883018611156103485763fe2549876000526004601cfd5b60c083901b6080526088838682378087017ffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff80151908490207effffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff167f02000000000000000000000000000000000000000000000000000000000000001760008181526002602090815260408083208b8452825280832080547fffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff0016600190811790915584845282528083209a83529981528982209390935590815290819052959095209190915550505050565b7f01000000000000000000000000000000000000000000000000000000000000007effffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff8316176104d8818360408051600093845233602052918152606090922091527effffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff167f01000000000000000000000000000000000000000000000000000000000000001790565b9392505050565b600080604083850312156104f257600080fd5b50508035926020909101359150565b600080600080600060a0868803121561051957600080fd5b505083359560208501359550604085013594606081013594506080013592509050565b60008060006040848603121561055157600080fd5b83359250602084013567ffffffffffffffff8082111561057057600080fd5b818601915086601f83011261058457600080fd5b81358181111561059357600080fd5b8760208285010111156105a557600080fd5b6020830194508093505050509250925092565b6000602082840312156105ca57600080fd5b5035919050565b7f4e487b7100000000000000000000000000000000000000000000000000000000600052601160045260246000fd5b60008219821115610613576106136105d1565b500190565b60008282101561062a5761062a6105d1565b50039056fea164736f6c634300080f000a" -var PreimageOracleDeployedSourceMap = "306:3911:129:-:0;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;537:68;;;;;;:::i;:::-;;;;;;;;;;;;;;;;;;;;;;;;;;;;413:25:286;;;401:2;386:18;537:68:129;;;;;;;;680:66;;;;;;:::i;:::-;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;614:14:286;;607:22;589:41;;577:2;562:18;680:66:129;449:187:286;1367:1211:129;;;;;;:::i;:::-;;:::i;789:536::-;;;;;;:::i;:::-;;:::i;:::-;;;;1274:25:286;;;1330:2;1315:18;;1308:34;;;;1247:18;789:536:129;1100:248:286;2620:1595:129;;;;;;:::i;:::-;;:::i;:::-;;419:50;;;;;;:::i;:::-;;;;;;;;;;;;;;;1367:1211;1560:12;1665:51;1694:6;1702:13;1665:28;:51::i;:::-;1658:58;-1:-1:-1;1810:9:129;:5;1818:1;1810:9;:::i;:::-;1796:11;:23;:37;;;;1831:2;1823:5;:10;1796:37;1792:90;;;1856:15;;;;;;;;;;;;;;1792:90;1951:12;2051:4;2044:18;;;2152:3;2148:15;;;2135:29;;2184:4;2177:19;;;;2286:18;;2376:20;;;:14;:20;;;;;;:33;;;;;;;;:40;;;;2412:4;2376:40;;;;;;2426:19;;;;;;;;:32;;;;;;;;;:39;2542:21;;;;;;;;;:29;2391:4;1367:1211;-1:-1:-1;;1367:1211:129:o;789:536::-;865:12;914:20;;;:14;:20;;;;;;;;:29;;;;;;;;;865:12;;914:29;;906:62;;;;;;;2908:2:286;906:62:129;;;2890:21:286;2947:2;2927:18;;;2920:30;2986:22;2966:18;;;2959:50;3026:18;;906:62:129;;;;;;;;-1:-1:-1;1099:14:129;1116:21;;;1087:2;1116:21;;;;;;;;1167:10;1116:21;1176:1;1167:10;:::i;:::-;1151:12;:7;1161:2;1151:12;:::i;:::-;:26;1147:87;;1216:7;1203:10;:6;1212:1;1203:10;:::i;:::-;:20;;;;:::i;:::-;1193:30;;1147:87;-1:-1:-1;1290:19:129;;;;:13;:19;;;;;;;;:28;;;;;;;;;;;;789:536;;-1:-1:-1;789:536:129:o;2620:1595::-;2916:4;2903:18;2721:12;;3045:1;3035:12;;3019:29;;3016:210;;;3120:10;3117:1;3110:21;3210:1;3204:4;3197:15;3016:210;3469:3;3465:14;;;3369:4;3453:27;3500:11;3474:4;3619:16;3500:11;3601:41;3832:29;;;3836:11;3832:29;3826:36;3884:20;;;;4031:19;4024:27;4053:11;4021:44;4084:19;;;;4062:1;4084:19;;;;;;;;:32;;;;;;;;:39;;;;4119:4;4084:39;;;;;;4133:18;;;;;;;;:31;;;;;;;;;:38;;;;4181:20;;;;;;;;;;;:27;;;;-1:-1:-1;;;;2620:1595:129:o;552:449:128:-;835:11;860:19;848:32;;832:49;965:29;832:49;980:13;1676:4;1670:11;;1533:21;1787:15;;;1828:8;1822:4;1815:22;1850:27;;;1996:4;1983:18;;;2098:17;;2003:19;1979:44;2025:11;1976:61;;1455:676;965:29;958:36;552:449;-1:-1:-1;;;552:449:128:o;14:248:286:-;82:6;90;143:2;131:9;122:7;118:23;114:32;111:52;;;159:1;156;149:12;111:52;-1:-1:-1;;182:23:286;;;252:2;237:18;;;224:32;;-1:-1:-1;14:248:286:o;641:454::-;736:6;744;752;760;768;821:3;809:9;800:7;796:23;792:33;789:53;;;838:1;835;828:12;789:53;-1:-1:-1;;861:23:286;;;931:2;916:18;;903:32;;-1:-1:-1;982:2:286;967:18;;954:32;;1033:2;1018:18;;1005:32;;-1:-1:-1;1084:3:286;1069:19;1056:33;;-1:-1:-1;641:454:286;-1:-1:-1;641:454:286:o;1353:659::-;1432:6;1440;1448;1501:2;1489:9;1480:7;1476:23;1472:32;1469:52;;;1517:1;1514;1507:12;1469:52;1553:9;1540:23;1530:33;;1614:2;1603:9;1599:18;1586:32;1637:18;1678:2;1670:6;1667:14;1664:34;;;1694:1;1691;1684:12;1664:34;1732:6;1721:9;1717:22;1707:32;;1777:7;1770:4;1766:2;1762:13;1758:27;1748:55;;1799:1;1796;1789:12;1748:55;1839:2;1826:16;1865:2;1857:6;1854:14;1851:34;;;1881:1;1878;1871:12;1851:34;1926:7;1921:2;1912:6;1908:2;1904:15;1900:24;1897:37;1894:57;;;1947:1;1944;1937:12;1894:57;1978:2;1974;1970:11;1960:21;;2000:6;1990:16;;;;;1353:659;;;;;:::o;2017:180::-;2076:6;2129:2;2117:9;2108:7;2104:23;2100:32;2097:52;;;2145:1;2142;2135:12;2097:52;-1:-1:-1;2168:23:286;;2017:180;-1:-1:-1;2017:180:286:o;2384:184::-;2436:77;2433:1;2426:88;2533:4;2530:1;2523:15;2557:4;2554:1;2547:15;2573:128;2613:3;2644:1;2640:6;2637:1;2634:13;2631:39;;;2650:18;;:::i;:::-;-1:-1:-1;2686:9:286;;2573:128::o;3055:125::-;3095:4;3123:1;3120;3117:8;3114:34;;;3128:18;;:::i;:::-;-1:-1:-1;3165:9:286;;3055:125::o" +var PreimageOracleDeployedSourceMap = "306:3911:128:-:0;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;537:68;;;;;;:::i;:::-;;;;;;;;;;;;;;;;;;;;;;;;;;;;413:25:285;;;401:2;386:18;537:68:128;;;;;;;;680:66;;;;;;:::i;:::-;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;614:14:285;;607:22;589:41;;577:2;562:18;680:66:128;449:187:285;1367:1211:128;;;;;;:::i;:::-;;:::i;789:536::-;;;;;;:::i;:::-;;:::i;:::-;;;;1274:25:285;;;1330:2;1315:18;;1308:34;;;;1247:18;789:536:128;1100:248:285;2620:1595:128;;;;;;:::i;:::-;;:::i;:::-;;419:50;;;;;;:::i;:::-;;;;;;;;;;;;;;;1367:1211;1560:12;1665:51;1694:6;1702:13;1665:28;:51::i;:::-;1658:58;-1:-1:-1;1810:9:128;:5;1818:1;1810:9;:::i;:::-;1796:11;:23;:37;;;;1831:2;1823:5;:10;1796:37;1792:90;;;1856:15;;;;;;;;;;;;;;1792:90;1951:12;2051:4;2044:18;;;2152:3;2148:15;;;2135:29;;2184:4;2177:19;;;;2286:18;;2376:20;;;:14;:20;;;;;;:33;;;;;;;;:40;;;;2412:4;2376:40;;;;;;2426:19;;;;;;;;:32;;;;;;;;;:39;2542:21;;;;;;;;;:29;2391:4;1367:1211;-1:-1:-1;;1367:1211:128:o;789:536::-;865:12;914:20;;;:14;:20;;;;;;;;:29;;;;;;;;;865:12;;914:29;;906:62;;;;;;;2908:2:285;906:62:128;;;2890:21:285;2947:2;2927:18;;;2920:30;2986:22;2966:18;;;2959:50;3026:18;;906:62:128;;;;;;;;-1:-1:-1;1099:14:128;1116:21;;;1087:2;1116:21;;;;;;;;1167:10;1116:21;1176:1;1167:10;:::i;:::-;1151:12;:7;1161:2;1151:12;:::i;:::-;:26;1147:87;;1216:7;1203:10;:6;1212:1;1203:10;:::i;:::-;:20;;;;:::i;:::-;1193:30;;1147:87;-1:-1:-1;1290:19:128;;;;:13;:19;;;;;;;;:28;;;;;;;;;;;;789:536;;-1:-1:-1;789:536:128:o;2620:1595::-;2916:4;2903:18;2721:12;;3045:1;3035:12;;3019:29;;3016:210;;;3120:10;3117:1;3110:21;3210:1;3204:4;3197:15;3016:210;3469:3;3465:14;;;3369:4;3453:27;3500:11;3474:4;3619:16;3500:11;3601:41;3832:29;;;3836:11;3832:29;3826:36;3884:20;;;;4031:19;4024:27;4053:11;4021:44;4084:19;;;;4062:1;4084:19;;;;;;;;:32;;;;;;;;:39;;;;4119:4;4084:39;;;;;;4133:18;;;;;;;;:31;;;;;;;;;:38;;;;4181:20;;;;;;;;;;;:27;;;;-1:-1:-1;;;;2620:1595:128:o;552:449:127:-;835:11;860:19;848:32;;832:49;965:29;832:49;980:13;1676:4;1670:11;;1533:21;1787:15;;;1828:8;1822:4;1815:22;1850:27;;;1996:4;1983:18;;;2098:17;;2003:19;1979:44;2025:11;1976:61;;1455:676;965:29;958:36;552:449;-1:-1:-1;;;552:449:127:o;14:248:285:-;82:6;90;143:2;131:9;122:7;118:23;114:32;111:52;;;159:1;156;149:12;111:52;-1:-1:-1;;182:23:285;;;252:2;237:18;;;224:32;;-1:-1:-1;14:248:285:o;641:454::-;736:6;744;752;760;768;821:3;809:9;800:7;796:23;792:33;789:53;;;838:1;835;828:12;789:53;-1:-1:-1;;861:23:285;;;931:2;916:18;;903:32;;-1:-1:-1;982:2:285;967:18;;954:32;;1033:2;1018:18;;1005:32;;-1:-1:-1;1084:3:285;1069:19;1056:33;;-1:-1:-1;641:454:285;-1:-1:-1;641:454:285:o;1353:659::-;1432:6;1440;1448;1501:2;1489:9;1480:7;1476:23;1472:32;1469:52;;;1517:1;1514;1507:12;1469:52;1553:9;1540:23;1530:33;;1614:2;1603:9;1599:18;1586:32;1637:18;1678:2;1670:6;1667:14;1664:34;;;1694:1;1691;1684:12;1664:34;1732:6;1721:9;1717:22;1707:32;;1777:7;1770:4;1766:2;1762:13;1758:27;1748:55;;1799:1;1796;1789:12;1748:55;1839:2;1826:16;1865:2;1857:6;1854:14;1851:34;;;1881:1;1878;1871:12;1851:34;1926:7;1921:2;1912:6;1908:2;1904:15;1900:24;1897:37;1894:57;;;1947:1;1944;1937:12;1894:57;1978:2;1974;1970:11;1960:21;;2000:6;1990:16;;;;;1353:659;;;;;:::o;2017:180::-;2076:6;2129:2;2117:9;2108:7;2104:23;2100:32;2097:52;;;2145:1;2142;2135:12;2097:52;-1:-1:-1;2168:23:285;;2017:180;-1:-1:-1;2017:180:285:o;2384:184::-;2436:77;2433:1;2426:88;2533:4;2530:1;2523:15;2557:4;2554:1;2547:15;2573:128;2613:3;2644:1;2640:6;2637:1;2634:13;2631:39;;;2650:18;;:::i;:::-;-1:-1:-1;2686:9:285;;2573:128::o;3055:125::-;3095:4;3123:1;3120;3117:8;3114:34;;;3128:18;;:::i;:::-;-1:-1:-1;3165:9:285;;3055:125::o" func init() { if err := json.Unmarshal([]byte(PreimageOracleStorageLayoutJSON), PreimageOracleStorageLayout); err != nil { From d42577120629dfe167abff9256819a8ec6d2e3e8 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 26 Oct 2023 16:50:58 +0000 Subject: [PATCH 109/279] build(deps): bump github.com/google/uuid from 1.3.1 to 1.4.0 Bumps [github.com/google/uuid](https://github.com/google/uuid) from 1.3.1 to 1.4.0. - [Release notes](https://github.com/google/uuid/releases) - [Changelog](https://github.com/google/uuid/blob/master/CHANGELOG.md) - [Commits](https://github.com/google/uuid/compare/v1.3.1...v1.4.0) --- updated-dependencies: - dependency-name: github.com/google/uuid dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] --- go.mod | 2 +- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index 0c1231d9d041..eb86a741e197 100644 --- a/go.mod +++ b/go.mod @@ -16,7 +16,7 @@ require ( github.com/golang/snappy v0.0.5-0.20220116011046-fa5810519dcb github.com/google/go-cmp v0.6.0 github.com/google/gofuzz v1.2.1-0.20220503160820-4a35382e8fc8 - github.com/google/uuid v1.3.1 + github.com/google/uuid v1.4.0 github.com/hashicorp/go-multierror v1.1.1 github.com/hashicorp/golang-lru/v2 v2.0.5 github.com/holiman/uint256 v1.2.3 diff --git a/go.sum b/go.sum index bb949ec67bae..84e5b9ca917b 100644 --- a/go.sum +++ b/go.sum @@ -270,8 +270,8 @@ github.com/google/pprof v0.0.0-20230817174616-7a8ec2ada47b/go.mod h1:czg5+yv1E0Z github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= github.com/google/subcommands v1.2.0/go.mod h1:ZjhPrFU+Olkh9WazFPsl27BQ4UPiG37m3yTrtFlrHVk= github.com/google/uuid v1.1.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= -github.com/google/uuid v1.3.1 h1:KjJaJ9iWZ3jOFZIf1Lqf4laDRCasjl0BCmnEGxkdLb4= -github.com/google/uuid v1.3.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +github.com/google/uuid v1.4.0 h1:MtMxsa51/r9yyhkyLsVeVt0B+BGQZzpQiTQ4eHZ8bc4= +github.com/google/uuid v1.4.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/googleapis/gax-go v2.0.0+incompatible/go.mod h1:SFVmujtThgffbyetf+mdk2eWhX2bMyUtNHzFKcPA9HY= github.com/googleapis/gax-go/v2 v2.0.3/go.mod h1:LLvjysVCY1JZeum8Z6l8qUty8fiNwE08qbEPm1M08qg= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= From ae67c7ced53630243554b6dbe9515a7eda526d41 Mon Sep 17 00:00:00 2001 From: Tushar Shah Date: Thu, 26 Oct 2023 11:01:01 -0600 Subject: [PATCH 110/279] Verify Initializer Volues in Contract & Genesis Match - add match check-values-match job that searches using grep - define workflow to verify initializer value matches between contract and genesis --- .circleci/config.yml | 46 +++++++++++++++++++++++++++++++++++++++++++- 1 file changed, 45 insertions(+), 1 deletion(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index f1e0212e0087..b29b3f3de3e9 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -1222,6 +1222,46 @@ jobs: name: check-generated-mocks command: make generate-mocks-op-service && git diff --exit-code + check-values-match: + parameters: + pattern_file1: + type: string + default: "uint8 internal constant INITIALIZER =" + pattern_file2: + type: string + default: "const initializedValue =" + file1_path: + type: string + default: "packages/contracts-bedrock/src/libraries/Constants.sol" + file2_path: + type: string + default: "op-chain-ops/genesis/config.go" + + docker: + - image: cimg/base:2020.01 + + steps: + - checkout + - run: + name: Extract value from file1 + command: | + VALUE1=$(grep '<< parameters.pattern_file1 >>' << parameters.file1_path >> | awk -F'=' '{print $2}' | tr -d ' ;') + echo "export VALUE1=$VALUE1" >> $BASH_ENV + - run: + name: Extract value from file2 + command: | + VALUE2=$(grep '<< parameters.pattern_file2 >>' << parameters.file2_path >> | awk -F'=' '{print $2}' | tr -d ' ;') + echo "export VALUE2=$VALUE2" >> $BASH_ENV + - run: + name: Compare values + command: | + if [ "$VALUE1" != "$VALUE2" ]; then + echo "Error: Values from file1 ($VALUE1) and file2 ($VALUE2) don't match." + exit 1 + else + echo "Values match!" + fi + workflows: main: when: @@ -1466,7 +1506,11 @@ workflows: - check-generated-mocks-op-service - cannon-go-lint-and-test - cannon-build-test-vectors - + - check-values-match: + pattern_file1: "uint8 internal constant INITIALIZER =" + pattern_file2: "const initializedValue =" + file1_path: "packages/contracts-bedrock/src/libraries/Constants.sol" + file2_path: "op-chain-ops/genesis/config.go" release: when: not: From 3f41ff95a8b3ed5b25276f85dd46a65f467c415a Mon Sep 17 00:00:00 2001 From: Tushar Shah Date: Thu, 26 Oct 2023 11:05:06 -0600 Subject: [PATCH 111/279] docker image & fmt --- .circleci/config.yml | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index b29b3f3de3e9..09bf97c48270 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -1236,10 +1236,8 @@ jobs: file2_path: type: string default: "op-chain-ops/genesis/config.go" - docker: - - image: cimg/base:2020.01 - + - image: us-docker.pkg.dev/oplabs-tools-artifacts/images/ci-builder:latest steps: - checkout - run: From 93f38e6d3c9435be73726fd2b055595b40914b1c Mon Sep 17 00:00:00 2001 From: Tushar Shah Date: Thu, 26 Oct 2023 11:08:41 -0600 Subject: [PATCH 112/279] defaults --- .circleci/config.yml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index 09bf97c48270..0455c533c3d3 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -1226,16 +1226,16 @@ jobs: parameters: pattern_file1: type: string - default: "uint8 internal constant INITIALIZER =" + default: "" pattern_file2: type: string - default: "const initializedValue =" + default: "" file1_path: type: string - default: "packages/contracts-bedrock/src/libraries/Constants.sol" + default: "" file2_path: type: string - default: "op-chain-ops/genesis/config.go" + default: "" docker: - image: us-docker.pkg.dev/oplabs-tools-artifacts/images/ci-builder:latest steps: From 03f5fd0c27a410b0b2fc721ea4949a0078b4849b Mon Sep 17 00:00:00 2001 From: Tushar Shah Date: Thu, 26 Oct 2023 11:41:01 -0600 Subject: [PATCH 113/279] add awk script to ignore single and multi line comments & require only 1 match --- .circleci/config.yml | 29 +++++++++++++++++++++++++++-- 1 file changed, 27 insertions(+), 2 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index 0455c533c3d3..2512ef11d3ce 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -1240,16 +1240,40 @@ jobs: - image: us-docker.pkg.dev/oplabs-tools-artifacts/images/ci-builder:latest steps: - checkout + - run: + name: Filter comments and match only 1 script + command: | + SCRIPT=$(cat <<'EOF' + BEGIN { in_comment = 0; matches = 0; } + /^ *\/\*/ { in_comment = 1; } + in_comment && /\*\// { in_comment = 0; next; } + !in_comment && !/^ *\/\// && /PATTERN/ { matches++; matched_line = $0; } + END { + if (matches == 1) { + print matched_line; + } else if (matches > 1) { + print "Multiple matches found. Exiting."; + exit 1; + } else { + print "No matches found. Exiting."; + exit 1; + } + } + EOF + ) + echo "export SCRIPT=$SCRIPT" >> $BASH_ENV - run: name: Extract value from file1 command: | - VALUE1=$(grep '<< parameters.pattern_file1 >>' << parameters.file1_path >> | awk -F'=' '{print $2}' | tr -d ' ;') + VALUE1=$(echo "$SCRIPT" | awk -v PATTERN='<< parameters.pattern_file1 >>' -f- << parameters.file1_path >> | awk -F'=' '{print $2}' | tr -d ' ;') echo "export VALUE1=$VALUE1" >> $BASH_ENV + - run: name: Extract value from file2 command: | - VALUE2=$(grep '<< parameters.pattern_file2 >>' << parameters.file2_path >> | awk -F'=' '{print $2}' | tr -d ' ;') + VALUE2=$(echo "$SCRIPT" | awk -v PATTERN='<< parameters.pattern_file2 >>' -f- << parameters.file2_path >> | awk -F'=' '{print $2}' | tr -d ' ;') echo "export VALUE2=$VALUE2" >> $BASH_ENV + - run: name: Compare values command: | @@ -1260,6 +1284,7 @@ jobs: echo "Values match!" fi + workflows: main: when: From 27009c135f544aeb25b9445df0b4a00d464903b0 Mon Sep 17 00:00:00 2001 From: Tushar Shah Date: Thu, 26 Oct 2023 11:42:40 -0600 Subject: [PATCH 114/279] indent --- .circleci/config.yml | 36 ++++++++++++++++++------------------ 1 file changed, 18 insertions(+), 18 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index 2512ef11d3ce..b77c29d41718 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -1244,24 +1244,24 @@ jobs: name: Filter comments and match only 1 script command: | SCRIPT=$(cat <<'EOF' - BEGIN { in_comment = 0; matches = 0; } - /^ *\/\*/ { in_comment = 1; } - in_comment && /\*\// { in_comment = 0; next; } - !in_comment && !/^ *\/\// && /PATTERN/ { matches++; matched_line = $0; } - END { - if (matches == 1) { - print matched_line; - } else if (matches > 1) { - print "Multiple matches found. Exiting."; - exit 1; - } else { - print "No matches found. Exiting."; - exit 1; - } - } - EOF - ) - echo "export SCRIPT=$SCRIPT" >> $BASH_ENV + BEGIN { in_comment = 0; matches = 0; } + /^ *\/\*/ { in_comment = 1; } + in_comment && /\*\// { in_comment = 0; next; } + !in_comment && !/^ *\/\// && /PATTERN/ { matches++; matched_line = $0; } + END { + if (matches == 1) { + print matched_line; + } else if (matches > 1) { + print "Multiple matches found. Exiting."; + exit 1; + } else { + print "No matches found. Exiting."; + exit 1; + } + } + EOF + ) + echo "export SCRIPT=$SCRIPT" >> $BASH_ENV - run: name: Extract value from file1 command: | From fb12a4ac4493a4237bdb2102e53e5866d2e97e3e Mon Sep 17 00:00:00 2001 From: Tushar Shah Date: Thu, 26 Oct 2023 11:48:41 -0600 Subject: [PATCH 115/279] CircleCI didn't like the << in my last script, switching to string literal --- .circleci/config.yml | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index b77c29d41718..0b900433ab2d 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -1243,7 +1243,7 @@ jobs: - run: name: Filter comments and match only 1 script command: | - SCRIPT=$(cat <<'EOF' + AWK_SCRIPT=' BEGIN { in_comment = 0; matches = 0; } /^ *\/\*/ { in_comment = 1; } in_comment && /\*\// { in_comment = 0; next; } @@ -1258,10 +1258,8 @@ jobs: print "No matches found. Exiting."; exit 1; } - } - EOF - ) - echo "export SCRIPT=$SCRIPT" >> $BASH_ENV + }' + echo "export SCRIPT=$AWK_SCRIPT" >> $BASH_ENV - run: name: Extract value from file1 command: | From cd85e3d5a49148cd0217baa89229b154c8a10369 Mon Sep 17 00:00:00 2001 From: Will Cory Date: Wed, 25 Oct 2023 09:40:57 -0700 Subject: [PATCH 116/279] chore: Improve chain-mon docker build --- ops/docker/Dockerfile.packages | 24 ++++++++++++++++++------ 1 file changed, 18 insertions(+), 6 deletions(-) diff --git a/ops/docker/Dockerfile.packages b/ops/docker/Dockerfile.packages index d84a3726ac35..18bf17ddf270 100644 --- a/ops/docker/Dockerfile.packages +++ b/ops/docker/Dockerfile.packages @@ -79,26 +79,38 @@ RUN git submodule update --init --recursive RUN pnpm build +FROM base as chain-mon +WORKDIR /opt/optimism/packages/chain-mon +ENTRYPOINT ["pnpm", "run"] + +# TODO keeping the rest of these here for now because they are being used +# but we should really delete them we only need one image FROM base as replica-mon WORKDIR /opt/optimism/packages/chain-mon -ENTRYPOINT ["pnpm", "run", "start:replica-mon"] +ENTRYPOINT ["pnpm", "run"] +CMD ["start:replica-mon"] FROM base as balance-mon WORKDIR /opt/optimism/packages/chain-mon -ENTRYPOINT ["pnpm", "run", "start:balance-mon"] +ENTRYPOINT ["pnpm", "run"] +CMD ["start:balance-mon"] FROM base as drippie-mon WORKDIR /opt/optimism/packages/chain-mon -ENTRYPOINT ["pnpm", "run", "start:drippie-mon"] +ENTRYPOINT ["pnpm", "run"] +CMD ["start:drippie-mon"] FROM base as wd-mon WORKDIR /opt/optimism/packages/chain-mon -ENTRYPOINT ["pnpm", "run", "start:wd-mon"] +ENTRYPOINT ["pnpm", "run"] +CMD ["start:wd-mon"] FROM base as wallet-mon WORKDIR /opt/optimism/packages/chain-mon -ENTRYPOINT ["pnpm", "run", "start:wallet-mon"] +ENTRYPOINT ["pnpm", "run"] +CMD ["start:wallet-mon"] from base as fault-mon WORKDIR /opt/optimism/packages/chain-mon -ENTRYPOINT ["pnpm", "run", "start:fault-mon"] +ENTRYPOINT ["pnpm", "run"] +CMD ["start:fault-mon"] From e02becb534145a8c89c5bf5b415aee892044d934 Mon Sep 17 00:00:00 2001 From: Will Cory Date: Thu, 26 Oct 2023 10:49:31 -0700 Subject: [PATCH 117/279] move entrypoint to builder image --- ops/docker/Dockerfile.packages | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/ops/docker/Dockerfile.packages b/ops/docker/Dockerfile.packages index 18bf17ddf270..e05febf6081e 100644 --- a/ops/docker/Dockerfile.packages +++ b/ops/docker/Dockerfile.packages @@ -79,38 +79,33 @@ RUN git submodule update --init --recursive RUN pnpm build +ENTRYPOINT ["pnpm", "run"] + FROM base as chain-mon WORKDIR /opt/optimism/packages/chain-mon -ENTRYPOINT ["pnpm", "run"] # TODO keeping the rest of these here for now because they are being used # but we should really delete them we only need one image FROM base as replica-mon WORKDIR /opt/optimism/packages/chain-mon -ENTRYPOINT ["pnpm", "run"] CMD ["start:replica-mon"] FROM base as balance-mon WORKDIR /opt/optimism/packages/chain-mon -ENTRYPOINT ["pnpm", "run"] CMD ["start:balance-mon"] FROM base as drippie-mon WORKDIR /opt/optimism/packages/chain-mon -ENTRYPOINT ["pnpm", "run"] CMD ["start:drippie-mon"] FROM base as wd-mon WORKDIR /opt/optimism/packages/chain-mon -ENTRYPOINT ["pnpm", "run"] CMD ["start:wd-mon"] FROM base as wallet-mon WORKDIR /opt/optimism/packages/chain-mon -ENTRYPOINT ["pnpm", "run"] CMD ["start:wallet-mon"] from base as fault-mon WORKDIR /opt/optimism/packages/chain-mon -ENTRYPOINT ["pnpm", "run"] CMD ["start:fault-mon"] From d64776935f8982cecb87f5e4ec989edf095c8483 Mon Sep 17 00:00:00 2001 From: Tushar Shah Date: Thu, 26 Oct 2023 11:58:03 -0600 Subject: [PATCH 118/279] quote kind --- .circleci/config.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index 0b900433ab2d..225112fc1bb1 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -1243,7 +1243,7 @@ jobs: - run: name: Filter comments and match only 1 script command: | - AWK_SCRIPT=' + AWK_SCRIPT=" BEGIN { in_comment = 0; matches = 0; } /^ *\/\*/ { in_comment = 1; } in_comment && /\*\// { in_comment = 0; next; } @@ -1258,7 +1258,7 @@ jobs: print "No matches found. Exiting."; exit 1; } - }' + }" echo "export SCRIPT=$AWK_SCRIPT" >> $BASH_ENV - run: name: Extract value from file1 From ef1dfdb00d9a3b494b52aa78102fc5451c560ed7 Mon Sep 17 00:00:00 2001 From: Tushar Shah Date: Thu, 26 Oct 2023 12:06:10 -0600 Subject: [PATCH 119/279] inline script --- .circleci/config.yml | 18 +----------------- 1 file changed, 1 insertion(+), 17 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index 225112fc1bb1..19449a6fe57d 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -1243,23 +1243,7 @@ jobs: - run: name: Filter comments and match only 1 script command: | - AWK_SCRIPT=" - BEGIN { in_comment = 0; matches = 0; } - /^ *\/\*/ { in_comment = 1; } - in_comment && /\*\// { in_comment = 0; next; } - !in_comment && !/^ *\/\// && /PATTERN/ { matches++; matched_line = $0; } - END { - if (matches == 1) { - print matched_line; - } else if (matches > 1) { - print "Multiple matches found. Exiting."; - exit 1; - } else { - print "No matches found. Exiting."; - exit 1; - } - }" - echo "export SCRIPT=$AWK_SCRIPT" >> $BASH_ENV + SCRIPT='BEGIN { in_comment = 0; matches = 0; } /^ *\/\*/ { in_comment = 1; } in_comment && /\*\// { in_comment = 0; next; } !in_comment && !/^ *\/\// && $0 ~ PATTERN { matches++; matched_line = $0; } END { if (matches == 1) { print matched_line; } else if (matches > 1) { print "Multiple matches found. Exiting."; exit 1; } else { print "No matches found. Exiting."; exit 1; } }' - run: name: Extract value from file1 command: | From b63da6564b344a40060773dad2041ec24b9da369 Mon Sep 17 00:00:00 2001 From: Joshua Gutow Date: Thu, 26 Oct 2023 11:10:10 -0700 Subject: [PATCH 120/279] op-node: Pull in Canyon Time from superchain registry --- go.mod | 4 ++-- go.sum | 8 ++++---- op-chain-ops/genesis/genesis.go | 6 +++--- op-node/flags/flags.go | 2 +- op-node/rollup/superchain.go | 3 ++- 5 files changed, 12 insertions(+), 11 deletions(-) diff --git a/go.mod b/go.mod index eb86a741e197..7b15580851dd 100644 --- a/go.mod +++ b/go.mod @@ -8,7 +8,7 @@ require ( github.com/btcsuite/btcd/chaincfg/chainhash v1.0.1 github.com/decred/dcrd/dcrec/secp256k1/v4 v4.2.0 github.com/ethereum-optimism/go-ethereum-hdwallet v0.1.3 - github.com/ethereum-optimism/superchain-registry/superchain v0.0.0-20231018202221-fdba3d104171 + github.com/ethereum-optimism/superchain-registry/superchain v0.0.0-20231026175037-2cff0d130e74 github.com/ethereum/go-ethereum v1.13.1 github.com/fsnotify/fsnotify v1.7.0 github.com/go-chi/chi/v5 v5.0.10 @@ -209,7 +209,7 @@ require ( rsc.io/tmplfunc v0.0.3 // indirect ) -replace github.com/ethereum/go-ethereum v1.13.1 => github.com/ethereum-optimism/op-geth v1.101303.0-rc.2.0.20231024175019-29cd9a353f83 +replace github.com/ethereum/go-ethereum v1.13.1 => github.com/ethereum-optimism/op-geth v1.101303.0-rc.2.0.20231026180835-94fbbd04522e //replace github.com/ethereum-optimism/superchain-registry/superchain => ../superchain-registry/superchain //replace github.com/ethereum/go-ethereum v1.13.1 => ../go-ethereum diff --git a/go.sum b/go.sum index 84e5b9ca917b..87fbcbcfa36f 100644 --- a/go.sum +++ b/go.sum @@ -151,10 +151,10 @@ github.com/elastic/gosigar v0.14.2 h1:Dg80n8cr90OZ7x+bAax/QjoW/XqTI11RmA79ZwIm9/ github.com/elastic/gosigar v0.14.2/go.mod h1:iXRIGg2tLnu7LBdpqzyQfGDEidKCfWcCMS0WKyPWoMs= github.com/ethereum-optimism/go-ethereum-hdwallet v0.1.3 h1:RWHKLhCrQThMfch+QJ1Z8veEq5ZO3DfIhZ7xgRP9WTc= github.com/ethereum-optimism/go-ethereum-hdwallet v0.1.3/go.mod h1:QziizLAiF0KqyLdNJYD7O5cpDlaFMNZzlxYNcWsJUxs= -github.com/ethereum-optimism/op-geth v1.101303.0-rc.2.0.20231024175019-29cd9a353f83 h1:RFKnTUJqbYM8+dueFcGPdOY0ycrOhxp0HQJyy2OYzvc= -github.com/ethereum-optimism/op-geth v1.101303.0-rc.2.0.20231024175019-29cd9a353f83/go.mod h1:hl28ffXoV4maInP7dvhvNgDO79Q5M3MEYrPZZO6u3W8= -github.com/ethereum-optimism/superchain-registry/superchain v0.0.0-20231018202221-fdba3d104171 h1:MjCUj16JSLZRDnQQ6OOUy6Chfb4dKo7ahFceNi0RKZ8= -github.com/ethereum-optimism/superchain-registry/superchain v0.0.0-20231018202221-fdba3d104171/go.mod h1:/70H/KqrtKcvWvNGVj6S3rAcLC+kUPr3t2aDmYIS+Xk= +github.com/ethereum-optimism/op-geth v1.101303.0-rc.2.0.20231026180835-94fbbd04522e h1:5ucLyIBCwo07ejZOKFY+6QbCqbLgITHWVqkmLoO6604= +github.com/ethereum-optimism/op-geth v1.101303.0-rc.2.0.20231026180835-94fbbd04522e/go.mod h1:m6GrpSyAe1zdFLJlSctgYKSXUdHwj/yfq2WSOc5vs2A= +github.com/ethereum-optimism/superchain-registry/superchain v0.0.0-20231026175037-2cff0d130e74 h1:02gXBD+Cas7xj9rpkke5wD1+vpfYxyF/+31M5tosP9A= +github.com/ethereum-optimism/superchain-registry/superchain v0.0.0-20231026175037-2cff0d130e74/go.mod h1:/70H/KqrtKcvWvNGVj6S3rAcLC+kUPr3t2aDmYIS+Xk= github.com/ethereum/c-kzg-4844 v0.3.1 h1:sR65+68+WdnMKxseNWxSJuAv2tsUrihTpVBTfM/U5Zg= github.com/ethereum/c-kzg-4844 v0.3.1/go.mod h1:VewdlzQmpT5QSrVhbBuGoCdFJkpaJlO1aQputP83wc0= github.com/fatih/color v1.13.0 h1:8LOYc1KYPPmyKMuN8QV2DNRWNbLo6LZ0iLs8+mlH53w= diff --git a/op-chain-ops/genesis/genesis.go b/op-chain-ops/genesis/genesis.go index 7a4df6b1ad49..c4aef7d94317 100644 --- a/op-chain-ops/genesis/genesis.go +++ b/op-chain-ops/genesis/genesis.go @@ -65,9 +65,9 @@ func NewL2Genesis(config *DeployConfig, block *types.Block) (*core.Genesis, erro CanyonTime: config.CanyonTime(block.Time()), ShanghaiTime: config.CanyonTime(block.Time()), Optimism: ¶ms.OptimismConfig{ - EIP1559Denominator: eip1559Denom, - EIP1559Elasticity: eip1559Elasticity, - EIP1559DenominatorPostCanyon: eip1559DenomCanyon, + EIP1559Denominator: eip1559Denom, + EIP1559Elasticity: eip1559Elasticity, + EIP1559DenominatorCanyon: eip1559DenomCanyon, }, } diff --git a/op-node/flags/flags.go b/op-node/flags/flags.go index 6c5b2fe78f31..8f0247949137 100644 --- a/op-node/flags/flags.go +++ b/op-node/flags/flags.go @@ -256,7 +256,7 @@ var ( CanyonOverrideFlag = &cli.Uint64Flag{ Name: "override.canyon", Usage: "Manually specify the Canyon fork timestamp, overriding the bundled setting", - Hidden: true, + Hidden: false, } ) diff --git a/op-node/rollup/superchain.go b/op-node/rollup/superchain.go index 2235a0051e86..88d802f5cf13 100644 --- a/op-node/rollup/superchain.go +++ b/op-node/rollup/superchain.go @@ -12,7 +12,7 @@ import ( "github.com/ethereum-optimism/superchain-registry/superchain" ) -var OPStackSupport = params.ProtocolVersionV0{Build: [8]byte{}, Major: 3, Minor: 1, Patch: 0, PreRelease: 1}.Encode() +var OPStackSupport = params.ProtocolVersionV0{Build: [8]byte{}, Major: 4, Minor: 0, Patch: 0, PreRelease: 1}.Encode() const ( opMainnet = 10 @@ -98,6 +98,7 @@ func LoadOPStackRollupConfig(chainID uint64) (*Config, error) { L1ChainID: new(big.Int).SetUint64(superChain.Config.L1.ChainID), L2ChainID: new(big.Int).SetUint64(chConfig.ChainID), RegolithTime: ®olithTime, + CanyonTime: superChain.Config.CanyonTime, BatchInboxAddress: common.Address(chConfig.BatchInboxAddr), DepositContractAddress: depositContractAddress, L1SystemConfigAddress: common.Address(chConfig.SystemConfigAddr), From fafcb05e4256d8161b6c7d4f53b9cc7782f825eb Mon Sep 17 00:00:00 2001 From: Tushar Shah Date: Thu, 26 Oct 2023 12:10:16 -0600 Subject: [PATCH 121/279] export script to bash env --- .circleci/config.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.circleci/config.yml b/.circleci/config.yml index 19449a6fe57d..c40cafe92761 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -1244,6 +1244,7 @@ jobs: name: Filter comments and match only 1 script command: | SCRIPT='BEGIN { in_comment = 0; matches = 0; } /^ *\/\*/ { in_comment = 1; } in_comment && /\*\// { in_comment = 0; next; } !in_comment && !/^ *\/\// && $0 ~ PATTERN { matches++; matched_line = $0; } END { if (matches == 1) { print matched_line; } else if (matches > 1) { print "Multiple matches found. Exiting."; exit 1; } else { print "No matches found. Exiting."; exit 1; } }' + echo "export SCRIPT='$SCRIPT'" >> $BASH_ENV - run: name: Extract value from file1 command: | From b5914297b38121c36eab686f80ab70d302be3c2c Mon Sep 17 00:00:00 2001 From: Tushar Shah Date: Thu, 26 Oct 2023 12:31:46 -0600 Subject: [PATCH 122/279] handle awk error --- .circleci/config.yml | 18 +++++++++++++++++- 1 file changed, 17 insertions(+), 1 deletion(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index c40cafe92761..184a4acf1146 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -1248,13 +1248,29 @@ jobs: - run: name: Extract value from file1 command: | - VALUE1=$(echo "$SCRIPT" | awk -v PATTERN='<< parameters.pattern_file1 >>' -f- << parameters.file1_path >> | awk -F'=' '{print $2}' | tr -d ' ;') + VALUE1_MATCH=$(echo "$SCRIPT" | awk -v PATTERN='<< parameters.pattern_file1 >>' -f- "<< parameters.file1_path >>") + + if [ $? -ne 0 ]; then + exit 1 + fi + VALUE1=$(echo "$VALUE1_MATCH" | awk -F'=' '{print $2}' | tr -d ' ;') + echo "Value:" + echo "$VALUE1" echo "export VALUE1=$VALUE1" >> $BASH_ENV - run: name: Extract value from file2 command: | VALUE2=$(echo "$SCRIPT" | awk -v PATTERN='<< parameters.pattern_file2 >>' -f- << parameters.file2_path >> | awk -F'=' '{print $2}' | tr -d ' ;') + VALUE2_MATCH=$(echo "$SCRIPT" | awk -v PATTERN='<< parameters.pattern_file2 >>' -f- "<< parameters.file2_path >>") + + if [ $? -ne 0 ]; then + exit 1 + fi + + VALUE2=$(echo "$VALUE2_MATCH" | awk -F'=' '{print $2}' | tr -d ' ;') + echo "Value:" + echo "$VALUE2" echo "export VALUE2=$VALUE2" >> $BASH_ENV - run: From ed4a4868acaf670a7464083d9d73eedbd5511327 Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Fri, 27 Oct 2023 05:02:38 +1000 Subject: [PATCH 123/279] ci: Add filters so op-stack-go-docker-build-release builds on tags. --- .circleci/config.yml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/.circleci/config.yml b/.circleci/config.yml index 747de52041a4..72fec84fe746 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -1488,6 +1488,11 @@ workflows: ignore: /.*/ - docker-build: # just to warm up the cache (other jobs run in parallel) name: op-stack-go-docker-build-release + filters: + tags: + only: /^(proxyd|indexer|ci-builder|ufm-[a-z0-9\-]*|op-[a-z0-9\-]*)\/v.*/ + branches: + ignore: /.*/ docker_name: op-stack-go docker_tags: <> platforms: "linux/amd64,linux/arm64" From af5ad1d2aa6f1a9aa29b1a8fe93989b1104d0ca8 Mon Sep 17 00:00:00 2001 From: Mark Tyneway Date: Wed, 25 Oct 2023 20:15:49 -0600 Subject: [PATCH 124/279] contracts-bedrock: fix test legibility Adds a new helper library called `EIP1967Helper` that can get the admin and impl storage slots from a `Proxy` implementing ERC1967. This is more helpful to use than hardcoded magic values such as `multisig` because it is not clear who the multisig is since its value is assigned in a different file. We want to decouple the value from a magic value and set it to exactly what we want it to be which is the admin. This will work in all cases no matter what the admin is since it dynamically pulls the value from storage for the tests. --- packages/contracts-bedrock/.gas-snapshot | 2 +- packages/contracts-bedrock/test/CommonTest.t.sol | 13 +++++++++++++ .../contracts-bedrock/test/L2OutputOracle.t.sol | 4 ++-- .../test/invariants/OptimismPortal.t.sol | 13 +++++++------ 4 files changed, 23 insertions(+), 9 deletions(-) diff --git a/packages/contracts-bedrock/.gas-snapshot b/packages/contracts-bedrock/.gas-snapshot index ac878dee55d9..5b6747f72e62 100644 --- a/packages/contracts-bedrock/.gas-snapshot +++ b/packages/contracts-bedrock/.gas-snapshot @@ -249,7 +249,7 @@ L2OutputOracleUpgradeable_Test:test_initValuesOnImpl_succeeds() (gas: 23902) L2OutputOracleUpgradeable_Test:test_initValuesOnProxy_succeeds() (gas: 46800) L2OutputOracleUpgradeable_Test:test_initializeImpl_alreadyInitialized_reverts() (gas: 15216) L2OutputOracleUpgradeable_Test:test_initializeProxy_alreadyInitialized_reverts() (gas: 20216) -L2OutputOracleUpgradeable_Test:test_upgrading_succeeds() (gas: 191455) +L2OutputOracleUpgradeable_Test:test_upgrading_succeeds() (gas: 187875) L2OutputOracle_constructor_Test:test_constructor_l2BlockTimeZero_reverts() (gas: 39022) L2OutputOracle_constructor_Test:test_constructor_submissionInterval_reverts() (gas: 39032) L2OutputOracle_constructor_Test:test_constructor_succeeds() (gas: 51777) diff --git a/packages/contracts-bedrock/test/CommonTest.t.sol b/packages/contracts-bedrock/test/CommonTest.t.sol index 56c9ded975e2..a3a64b6d9b22 100644 --- a/packages/contracts-bedrock/test/CommonTest.t.sol +++ b/packages/contracts-bedrock/test/CommonTest.t.sol @@ -3,6 +3,7 @@ pragma solidity 0.8.15; // Testing utilities import { Test, StdUtils } from "forge-std/Test.sol"; +import { Vm } from "forge-std/Vm.sol"; import { L2OutputOracle } from "src/L1/L2OutputOracle.sol"; import { L2ToL1MessagePasser } from "src/L2/L2ToL1MessagePasser.sol"; import { L1StandardBridge } from "src/L1/L1StandardBridge.sol"; @@ -731,6 +732,18 @@ contract FFIInterface is Test { } } +library EIP1967Helper { + Vm internal constant vm = Vm(0x7109709ECfa91a80626fF3989D68f67F5b1DD12D); + + function getAdmin(address _proxy) internal view returns (address) { + return address(uint160(uint256(vm.load(address(_proxy), Constants.PROXY_OWNER_ADDRESS)))); + } + + function getImplementation(address _proxy) internal view returns (address) { + return address(uint160(uint256(vm.load(address(_proxy), Constants.PROXY_IMPLEMENTATION_ADDRESS)))); + } +} + // Used for testing a future upgrade beyond the current implementations. // We include some variables so that we can sanity check accessing storage values after an upgrade. contract NextImpl is Initializable { diff --git a/packages/contracts-bedrock/test/L2OutputOracle.t.sol b/packages/contracts-bedrock/test/L2OutputOracle.t.sol index 623fc7aa224c..52b87b450aa8 100644 --- a/packages/contracts-bedrock/test/L2OutputOracle.t.sol +++ b/packages/contracts-bedrock/test/L2OutputOracle.t.sol @@ -3,7 +3,7 @@ pragma solidity 0.8.15; // Testing utilities import { stdError } from "forge-std/Test.sol"; -import { L2OutputOracle_Initializer, NextImpl } from "test/CommonTest.t.sol"; +import { L2OutputOracle_Initializer, NextImpl, EIP1967Helper } from "test/CommonTest.t.sol"; // Libraries import { Types } from "src/libraries/Types.sol"; @@ -463,7 +463,7 @@ contract L2OutputOracleUpgradeable_Test is L2OutputOracle_Initializer { assertEq(bytes32(0), slot21Before); NextImpl nextImpl = new NextImpl(); - vm.startPrank(multisig); + vm.startPrank(EIP1967Helper.getAdmin(address(proxy))); proxy.upgradeToAndCall( address(nextImpl), abi.encodeWithSelector(NextImpl.initialize.selector, Constants.INITIALIZER + 1) ); diff --git a/packages/contracts-bedrock/test/invariants/OptimismPortal.t.sol b/packages/contracts-bedrock/test/invariants/OptimismPortal.t.sol index f0ac1e197843..af38e6536bb8 100644 --- a/packages/contracts-bedrock/test/invariants/OptimismPortal.t.sol +++ b/packages/contracts-bedrock/test/invariants/OptimismPortal.t.sol @@ -12,6 +12,7 @@ import { ResourceMetering } from "src/L1/ResourceMetering.sol"; import { Constants } from "src/libraries/Constants.sol"; import { Portal_Initializer } from "test/CommonTest.t.sol"; +import { EIP1967Helper } from "test/CommonTest.t.sol"; import { Types } from "src/libraries/Types.sol"; contract OptimismPortal_Depositor is StdUtils, ResourceMetering { @@ -158,8 +159,8 @@ contract OptimismPortal_CannotTimeTravel is OptimismPortal_Invariant_Harness { // Set the target contract to the portal proxy targetContract(address(op)); - // Exclude the proxy multisig from the senders so that the proxy cannot be upgraded - excludeSender(address(multisig)); + // Exclude the proxy admin from the senders so that the proxy cannot be upgraded + excludeSender(EIP1967Helper.getAdmin(address(op))); } /// @custom:invariant `finalizeWithdrawalTransaction` should revert if the finalization @@ -188,8 +189,8 @@ contract OptimismPortal_CannotFinalizeTwice is OptimismPortal_Invariant_Harness // Set the target contract to the portal proxy targetContract(address(op)); - // Exclude the proxy multisig from the senders so that the proxy cannot be upgraded - excludeSender(address(multisig)); + // Exclude the proxy admin from the senders so that the proxy cannot be upgraded + excludeSender(EIP1967Helper.getAdmin(address(op))); } /// @custom:invariant `finalizeWithdrawalTransaction` should revert if the withdrawal @@ -215,8 +216,8 @@ contract OptimismPortal_CanAlwaysFinalizeAfterWindow is OptimismPortal_Invariant // Set the target contract to the portal proxy targetContract(address(op)); - // Exclude the proxy multisig from the senders so that the proxy cannot be upgraded - excludeSender(address(multisig)); + // Exclude the proxy admin from the senders so that the proxy cannot be upgraded + excludeSender(EIP1967Helper.getAdmin(address(op))); } /// @custom:invariant A withdrawal should **always** be able to be finalized From 78c6c7f32b5f82dc3131583d6773d84e08b7e91e Mon Sep 17 00:00:00 2001 From: Mark Tyneway Date: Thu, 26 Oct 2023 08:57:21 -0600 Subject: [PATCH 125/279] invariant-docs: regenerate --- .../contracts-bedrock/invariant-docs/OptimismPortal.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/packages/contracts-bedrock/invariant-docs/OptimismPortal.md b/packages/contracts-bedrock/invariant-docs/OptimismPortal.md index d9f5bf7d8a97..c13d0212128d 100644 --- a/packages/contracts-bedrock/invariant-docs/OptimismPortal.md +++ b/packages/contracts-bedrock/invariant-docs/OptimismPortal.md @@ -1,21 +1,21 @@ # `OptimismPortal` Invariants ## Deposits of any value should always succeed unless `_to` = `address(0)` or `_isCreation` = `true`. -**Test:** [`OptimismPortal.t.sol#L147`](../test/invariants/OptimismPortal.t.sol#L147) +**Test:** [`OptimismPortal.t.sol#L148`](../test/invariants/OptimismPortal.t.sol#L148) All deposits, barring creation transactions and transactions sent to `address(0)`, should always succeed. ## `finalizeWithdrawalTransaction` should revert if the finalization period has not elapsed. -**Test:** [`OptimismPortal.t.sol#L170`](../test/invariants/OptimismPortal.t.sol#L170) +**Test:** [`OptimismPortal.t.sol#L171`](../test/invariants/OptimismPortal.t.sol#L171) A withdrawal that has been proven should not be able to be finalized until after the finalization period has elapsed. ## `finalizeWithdrawalTransaction` should revert if the withdrawal has already been finalized. -**Test:** [`OptimismPortal.t.sol#L200`](../test/invariants/OptimismPortal.t.sol#L200) +**Test:** [`OptimismPortal.t.sol#L201`](../test/invariants/OptimismPortal.t.sol#L201) Ensures that there is no chain of calls that can be made that allows a withdrawal to be finalized twice. ## A withdrawal should **always** be able to be finalized `FINALIZATION_PERIOD_SECONDS` after it was successfully proven. -**Test:** [`OptimismPortal.t.sol#L229`](../test/invariants/OptimismPortal.t.sol#L229) +**Test:** [`OptimismPortal.t.sol#L230`](../test/invariants/OptimismPortal.t.sol#L230) This invariant asserts that there is no chain of calls that can be made that will prevent a withdrawal from being finalized exactly `FINALIZATION_PERIOD_SECONDS` after it was successfully proven. \ No newline at end of file From 43f9f7ae24ff734a2ab4e0c034129d5df477020e Mon Sep 17 00:00:00 2001 From: Mark Tyneway Date: Thu, 26 Oct 2023 13:52:50 -0600 Subject: [PATCH 126/279] contracts-bedrock: gas snapshot --- packages/contracts-bedrock/.gas-snapshot | 2 -- 1 file changed, 2 deletions(-) diff --git a/packages/contracts-bedrock/.gas-snapshot b/packages/contracts-bedrock/.gas-snapshot index 5b6747f72e62..449fd2dcba8c 100644 --- a/packages/contracts-bedrock/.gas-snapshot +++ b/packages/contracts-bedrock/.gas-snapshot @@ -49,7 +49,6 @@ CrossDomainOwnable_Test:test_onlyOwner_succeeds() (gas: 34883) DelayedVetoable_Getters_Test:test_getters() (gas: 24466) DelayedVetoable_Getters_TestFail:test_getters_notZeroAddress_reverts() (gas: 36220) DelayedVetoable_HandleCall_TestFail:test_handleCall_unauthorizedInitiation_reverts() (gas: 21867) -DeleteOutput:test_script_succeeds() (gas: 3100) DeployerWhitelist_Test:test_owner_succeeds() (gas: 7582) DeployerWhitelist_Test:test_storageSlots_succeeds() (gas: 33395) DisputeGameFactory_Owner_Test:test_owner_succeeds() (gas: 12581) @@ -77,7 +76,6 @@ Drippie_Test:test_status_unauthorized_reverts() (gas: 167388) Drippie_Test:test_trigger_oneFunction_succeeds() (gas: 338226) Drippie_Test:test_trigger_twoFunctions_succeeds() (gas: 491907) Drippie_Test:test_twice_inOneInterval_reverts() (gas: 303933) -EASUpgrader:test_script_succeeds() (gas: 3078) FaucetTest:test_authAdmin_drip_succeeds() (gas: 366107) FaucetTest:test_drip_afterTimeout_succeeds() (gas: 447891) FaucetTest:test_drip_beforeTimeout_reverts() (gas: 378884) From 16b0527c6cbaf79e5e7a761a39a6aa91bd31e3fe Mon Sep 17 00:00:00 2001 From: Tushar Shah Date: Thu, 26 Oct 2023 12:56:43 -0600 Subject: [PATCH 127/279] remove old line --- .circleci/config.yml | 1 - 1 file changed, 1 deletion(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index 184a4acf1146..9288400aa475 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -1261,7 +1261,6 @@ jobs: - run: name: Extract value from file2 command: | - VALUE2=$(echo "$SCRIPT" | awk -v PATTERN='<< parameters.pattern_file2 >>' -f- << parameters.file2_path >> | awk -F'=' '{print $2}' | tr -d ' ;') VALUE2_MATCH=$(echo "$SCRIPT" | awk -v PATTERN='<< parameters.pattern_file2 >>' -f- "<< parameters.file2_path >>") if [ $? -ne 0 ]; then From e65c212241e89d9a3a5266df83fdfac40dbbee30 Mon Sep 17 00:00:00 2001 From: Tushar Shah Date: Thu, 26 Oct 2023 12:57:20 -0600 Subject: [PATCH 128/279] format --- .circleci/config.yml | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index 9288400aa475..d87803690962 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -1249,10 +1249,10 @@ jobs: name: Extract value from file1 command: | VALUE1_MATCH=$(echo "$SCRIPT" | awk -v PATTERN='<< parameters.pattern_file1 >>' -f- "<< parameters.file1_path >>") - if [ $? -ne 0 ]; then exit 1 fi + VALUE1=$(echo "$VALUE1_MATCH" | awk -F'=' '{print $2}' | tr -d ' ;') echo "Value:" echo "$VALUE1" @@ -1262,7 +1262,6 @@ jobs: name: Extract value from file2 command: | VALUE2_MATCH=$(echo "$SCRIPT" | awk -v PATTERN='<< parameters.pattern_file2 >>' -f- "<< parameters.file2_path >>") - if [ $? -ne 0 ]; then exit 1 fi From 5821f294912ba44b172943e34924f27f171089b2 Mon Sep 17 00:00:00 2001 From: Tushar Shah Date: Thu, 26 Oct 2023 13:31:10 -0600 Subject: [PATCH 129/279] Move value match and check logic to script --- .circleci/config.yml | 42 +-------------- ops/scripts/ci-match-values-between-files.sh | 57 ++++++++++++++++++++ 2 files changed, 59 insertions(+), 40 deletions(-) create mode 100755 ops/scripts/ci-match-values-between-files.sh diff --git a/.circleci/config.yml b/.circleci/config.yml index d87803690962..087a296f38f9 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -1241,47 +1241,9 @@ jobs: steps: - checkout - run: - name: Filter comments and match only 1 script + name: Verify Values Match command: | - SCRIPT='BEGIN { in_comment = 0; matches = 0; } /^ *\/\*/ { in_comment = 1; } in_comment && /\*\// { in_comment = 0; next; } !in_comment && !/^ *\/\// && $0 ~ PATTERN { matches++; matched_line = $0; } END { if (matches == 1) { print matched_line; } else if (matches > 1) { print "Multiple matches found. Exiting."; exit 1; } else { print "No matches found. Exiting."; exit 1; } }' - echo "export SCRIPT='$SCRIPT'" >> $BASH_ENV - - run: - name: Extract value from file1 - command: | - VALUE1_MATCH=$(echo "$SCRIPT" | awk -v PATTERN='<< parameters.pattern_file1 >>' -f- "<< parameters.file1_path >>") - if [ $? -ne 0 ]; then - exit 1 - fi - - VALUE1=$(echo "$VALUE1_MATCH" | awk -F'=' '{print $2}' | tr -d ' ;') - echo "Value:" - echo "$VALUE1" - echo "export VALUE1=$VALUE1" >> $BASH_ENV - - - run: - name: Extract value from file2 - command: | - VALUE2_MATCH=$(echo "$SCRIPT" | awk -v PATTERN='<< parameters.pattern_file2 >>' -f- "<< parameters.file2_path >>") - if [ $? -ne 0 ]; then - exit 1 - fi - - VALUE2=$(echo "$VALUE2_MATCH" | awk -F'=' '{print $2}' | tr -d ' ;') - echo "Value:" - echo "$VALUE2" - echo "export VALUE2=$VALUE2" >> $BASH_ENV - - - run: - name: Compare values - command: | - if [ "$VALUE1" != "$VALUE2" ]; then - echo "Error: Values from file1 ($VALUE1) and file2 ($VALUE2) don't match." - exit 1 - else - echo "Values match!" - fi - - + ./ops/scripts/ci-match-values-between-files.sh "<< parameters.file1_path >>" "<< parameters.pattern_file1 >>" "<< parameters.file2_path >>" "<< parameters.pattern_file2 >>" workflows: main: when: diff --git a/ops/scripts/ci-match-values-between-files.sh b/ops/scripts/ci-match-values-between-files.sh new file mode 100755 index 000000000000..53d680cd3197 --- /dev/null +++ b/ops/scripts/ci-match-values-between-files.sh @@ -0,0 +1,57 @@ +#!/usr/bin/env bash + +set -euo pipefail + +FILE1=$1 +PATTERN1=$2 +FILE2=$3 +PATTERN2=$4 + +# shellcheck disable=SC2016 +SCRIPT=' +BEGIN { + in_comment = 0; + matches = 0; +} + +/^ *\/\*/ { + in_comment = 1; +} + +in_comment && /\*\// { + in_comment = 0; + next; +} + +!in_comment && !/^ *\/\// && $0 ~ PATTERN { + matches++; + matched_line = $0; +} + +END { + if (matches == 1) { + print matched_line; + } else if (matches > 1) { + print "Multiple matches found. Exiting."; + exit 1; + } else { + print "No matches found. Exiting."; + exit 1; + } +}' + +VALUE1_MATCH=$(echo "$SCRIPT" | awk -v PATTERN="$PATTERN1" -f- "$FILE1") +VALUE1=$(echo "$VALUE1_MATCH" | awk -F'=' '{print $2}' | tr -d ' ;') +echo "Value from File 1: $VALUE1" + +VALUE2_MATCH=$(echo "$SCRIPT" | awk -v PATTERN="$PATTERN2" -f- "$FILE2") +VALUE2=$(echo "$VALUE2_MATCH" | awk -F'=' '{print $2}' | tr -d ' ;') +echo "Value from File 2: $VALUE2" + +if [ "$VALUE1" != "$VALUE2" ]; then + echo "Error: Values from file1 ($VALUE1) and file2 ($VALUE2) don't match." + exit 1 +fi + +echo "Values match!" + From 31d1b6f59645f988246b0be06897e1fca4f0ab82 Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Fri, 27 Oct 2023 06:03:05 +1000 Subject: [PATCH 130/279] ci: Use pipeline.git.tag not pipeline.git.branch when doing releases. --- .circleci/config.yml | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index 72fec84fe746..9afdb33a19b9 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -1506,7 +1506,7 @@ workflows: branches: ignore: /.*/ docker_name: op-heartbeat - docker_tags: <>,<> + docker_tags: <>,<> requires: ['op-stack-go-docker-build-release'] platforms: "linux/amd64,linux/arm64" publish: true @@ -1521,7 +1521,7 @@ workflows: branches: ignore: /.*/ docker_name: op-node - docker_tags: <>,<> + docker_tags: <>,<> requires: ['op-stack-go-docker-build-release'] platforms: "linux/amd64,linux/arm64" publish: true @@ -1536,7 +1536,7 @@ workflows: branches: ignore: /.*/ docker_name: op-batcher - docker_tags: <>,<> + docker_tags: <>,<> requires: ['op-stack-go-docker-build-release'] platforms: "linux/amd64,linux/arm64" publish: true @@ -1551,7 +1551,7 @@ workflows: branches: ignore: /.*/ docker_name: op-proposer - docker_tags: <>,<> + docker_tags: <>,<> requires: ['op-stack-go-docker-build-release'] platforms: "linux/amd64,linux/arm64" publish: true @@ -1566,7 +1566,7 @@ workflows: branches: ignore: /.*/ docker_name: op-challenger - docker_tags: <>,<> + docker_tags: <>,<> requires: ['op-stack-go-docker-build-release'] platforms: "linux/amd64,linux/arm64" publish: true @@ -1581,7 +1581,7 @@ workflows: branches: ignore: /.*/ docker_name: op-ufm - docker_tags: <>,<> + docker_tags: <>,<> publish: true release: true context: @@ -1596,7 +1596,7 @@ workflows: branches: ignore: /.*/ docker_name: proxyd - docker_tags: <>,<> + docker_tags: <>,<> publish: true release: true context: @@ -1611,7 +1611,7 @@ workflows: branches: ignore: /.*/ docker_name: indexer - docker_tags: <>,<> + docker_tags: <>,<> publish: true release: true context: From fdc2193e25791eaf06284f26bb6634c09a4365fc Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Fri, 27 Oct 2023 06:55:24 +1000 Subject: [PATCH 131/279] op-e2e: Disable cannon tests when using HTTP instead of WS Fault dispute tests take longer to execute when polling with HTTP so disable the longer running cannon tests on HTTP. Cannon itself is unaffected by the connection type and the challenger is tested with HTTP via the alphabet game tests. --- op-e2e/faultproof_test.go | 14 +++++++------- op-e2e/helper.go | 11 ++++++++++- op-e2e/op_geth.go | 2 +- op-e2e/setup.go | 10 ++++++---- 4 files changed, 24 insertions(+), 13 deletions(-) diff --git a/op-e2e/faultproof_test.go b/op-e2e/faultproof_test.go index cd38f67af8e3..a191532139d6 100644 --- a/op-e2e/faultproof_test.go +++ b/op-e2e/faultproof_test.go @@ -18,7 +18,7 @@ import ( ) func TestMultipleCannonGames(t *testing.T) { - InitParallel(t) + InitParallel(t, SkipIfHTTP) ctx := context.Background() sys, l1Client := startFaultDisputeSystem(t) @@ -78,7 +78,7 @@ func TestMultipleCannonGames(t *testing.T) { } func TestMultipleGameTypes(t *testing.T) { - InitParallel(t) + InitParallel(t, SkipIfHTTP) ctx := context.Background() sys, l1Client := startFaultDisputeSystem(t) @@ -277,7 +277,7 @@ func TestChallengerCompleteExhaustiveDisputeGame(t *testing.T) { } func TestCannonDisputeGame(t *testing.T) { - InitParallel(t) + InitParallel(t, SkipIfHTTP) tests := []struct { name string @@ -328,7 +328,7 @@ func TestCannonDisputeGame(t *testing.T) { } func TestCannonDefendStep(t *testing.T) { - InitParallel(t) + InitParallel(t, SkipIfHTTP) ctx := context.Background() sys, l1Client := startFaultDisputeSystem(t) @@ -370,7 +370,7 @@ func TestCannonDefendStep(t *testing.T) { } func TestCannonProposedOutputRootInvalid(t *testing.T) { - InitParallel(t) + InitParallel(t, SkipIfHTTP) // honestStepsFail attempts to perform both an attack and defend step using the correct trace. honestStepsFail := func(ctx context.Context, game *disputegame.CannonGameHelper, correctTrace *disputegame.HonestHelper, parentClaimIdx int64) { // Attack step should fail @@ -448,7 +448,7 @@ func TestCannonProposedOutputRootInvalid(t *testing.T) { } func TestCannonPoisonedPostState(t *testing.T) { - InitParallel(t) + InitParallel(t, SkipIfHTTP) ctx := context.Background() sys, l1Client := startFaultDisputeSystem(t) @@ -558,7 +558,7 @@ func setupDisputeGameForInvalidOutputRoot(t *testing.T, outputRoot common.Hash) } func TestCannonChallengeWithCorrectRoot(t *testing.T) { - InitParallel(t) + InitParallel(t, SkipIfHTTP) ctx := context.Background() sys, l1Client := startFaultDisputeSystem(t) diff --git a/op-e2e/helper.go b/op-e2e/helper.go index 04f5a4c0feac..eafeeb576e31 100644 --- a/op-e2e/helper.go +++ b/op-e2e/helper.go @@ -7,9 +7,18 @@ import ( var enableParallelTesting bool = os.Getenv("OP_E2E_DISABLE_PARALLEL") != "true" -func InitParallel(t *testing.T) { +func InitParallel(t *testing.T, opts ...func(t *testing.T)) { t.Helper() if enableParallelTesting { t.Parallel() } + for _, opt := range opts { + opt(t) + } +} + +func SkipIfHTTP(t *testing.T) { + if UseHTTP() { + t.Skip("Skipping test because HTTP connection is in use") + } } diff --git a/op-e2e/op_geth.go b/op-e2e/op_geth.go index 8a1bf8bbbe61..6b7eef31b48b 100644 --- a/op-e2e/op_geth.go +++ b/op-e2e/op_geth.go @@ -102,7 +102,7 @@ func NewOpGeth(t *testing.T, ctx context.Context, cfg *SystemConfig) (*OpGeth, e ) require.Nil(t, err) - l2Client, err := ethclient.Dial(node.HTTPEndpoint()) + l2Client, err := ethclient.Dial(selectEndpoint(node)) require.Nil(t, err) genesisPayload, err := eth.BlockAsPayload(l2GenesisBlock, cfg.DeployConfig.CanyonTime(l2GenesisBlock.Time())) diff --git a/op-e2e/setup.go b/op-e2e/setup.go index fe70a263985e..6f65a2b51916 100644 --- a/op-e2e/setup.go +++ b/op-e2e/setup.go @@ -758,9 +758,12 @@ func (sys *System) newMockNetPeer() (host.Host, error) { return sys.Mocknet.AddPeerWithPeerstore(p, eps) } +func UseHTTP() bool { + return os.Getenv("OP_E2E_USE_HTTP") == "true" +} + func selectEndpoint(node EthInstance) string { - useHTTP := os.Getenv("OP_E2E_USE_HTTP") == "true" - if useHTTP { + if UseHTTP() { log.Info("using HTTP client") return node.HTTPEndpoint() } @@ -785,9 +788,8 @@ type WSOrHTTPEndpoint interface { } func configureL2(rollupNodeCfg *rollupNode.Config, l2Node WSOrHTTPEndpoint, jwtSecret [32]byte) { - useHTTP := os.Getenv("OP_E2E_USE_HTTP") == "true" l2EndpointConfig := l2Node.WSAuthEndpoint() - if useHTTP { + if UseHTTP() { l2EndpointConfig = l2Node.HTTPAuthEndpoint() } From 8ec6427a3a9c09f3c0a2c90130ebf66c6ff7bef9 Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Fri, 27 Oct 2023 08:32:06 +1000 Subject: [PATCH 132/279] ci: Configure docker auth before attempting to publish. --- .circleci/config.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.circleci/config.yml b/.circleci/config.yml index 9afdb33a19b9..82bc88eb0f06 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -231,6 +231,7 @@ jobs: DOCKER_OUTPUT_DESTINATION="" if [ "<>" == "true" ]; then + gcloud auth configure-docker <> echo "Building for platforms $PLATFORMS and then publishing to registry" DOCKER_OUTPUT_DESTINATION="--push" if [ "<>" != "" ]; then From ab6d0be9c16c6af2042ee6f75895f960eb00f8b7 Mon Sep 17 00:00:00 2001 From: Kevin Kz Date: Thu, 26 Oct 2023 16:47:05 -0600 Subject: [PATCH 133/279] Specify that the recognized batch submitter account is stored in the System Configuration. --- specs/glossary.md | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/specs/glossary.md b/specs/glossary.md index cb96c8922e07..fc954bc5534f 100644 --- a/specs/glossary.md +++ b/specs/glossary.md @@ -471,9 +471,8 @@ channels available. These transactions carry one or more full frames, which may channel's frame may be split between multiple batcher transactions. When submitted to Ethereum calldata, the batcher transaction's receiver must be the sequencer inbox address. The -transaction must also be signed by a recognized batch submitter account. - -> **TODO** specify where these recognized batch submitter accounts are stored +transaction must also be signed by a recognized batch submitter account. The recognized batch submitter account +is stored in the [System Configuration][system-config]. ## Channel Timeout From ef2ceff26c5b81d8b325a34fabc4100cf7f1cd5f Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Fri, 27 Oct 2023 08:53:17 +1000 Subject: [PATCH 134/279] ci: Allow . and / in docker tags. Remove the publish step because all tags are added by the docker buildx step already. --- .circleci/config.yml | 16 +--------------- 1 file changed, 1 insertion(+), 15 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index 82bc88eb0f06..118113b542a3 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -218,7 +218,7 @@ jobs: export REGISTRY="<>" export REPOSITORY="<>" - export IMAGE_TAGS="$(echo -ne "<>" | sed "s/[^a-zA-Z0-9\n,]/-/g")" + export IMAGE_TAGS="$(echo -ne "<>" | sed "s/[^a-zA-Z0-9\n,./]/-/g")" export GIT_COMMIT="$(git rev-parse HEAD)" export GIT_DATE="$(git show -s --format='%ct')" export GIT_VERSION="<>" @@ -272,20 +272,6 @@ jobs: root: /tmp/docker_images paths: # only write the one file, to avoid concurrent workspace-file additions - "<>.tar" - - when: - condition: "<>" - steps: - - run: - name: Publish - command: | - gcloud auth configure-docker <> - IMAGE_BASE="<>/<>/<>" - # tags, without the '-t ' here, so we can loop over them - DOCKER_TAGS="$(echo -ne "<>" | sed "s/,/\n/g" | sed "s/[^a-zA-Z0-9\n]/-/g" | sed -e "s|^|${IMAGE_BASE}:|")" - for docker_image_tag in $DOCKER_TAGS; do - docker image push $docker_image_tag - done - no_output_timeout: 45m - when: condition: "<>" steps: From 6a7360bc6dd3fb86be7c641bc7803212e4ca5b03 Mon Sep 17 00:00:00 2001 From: Zach Howard Date: Thu, 26 Oct 2023 18:55:56 -0400 Subject: [PATCH 135/279] Fixes docker-bake build targets for ci-builder and chain-mon --- docker-bake.hcl | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docker-bake.hcl b/docker-bake.hcl index afdc5996d25c..d808f751b273 100644 --- a/docker-bake.hcl +++ b/docker-bake.hcl @@ -158,7 +158,7 @@ target "ufm-metamask" { tags = [for tag in split(",", IMAGE_TAGS) : "${REGISTRY}/${REPOSITORY}/ufm-metamask:${tag}"] } -type "chain-mon" { +target "chain-mon" { dockerfile = "./ops/docker/Dockerfile.packages" context = "." args = { @@ -173,9 +173,9 @@ type "chain-mon" { tags = [for tag in split(",", IMAGE_TAGS) : "${REGISTRY}/${REPOSITORY}/chain-mon:${tag}"] } -type "ci-builder" { - dockerfile = "Dockerfile" - context = "ops/docker/ci-builder" +target "ci-builder" { + dockerfile = "./ops/docker/ci-builder/Dockerfile" + context = "." platforms = split(",", PLATFORMS) tags = [for tag in split(",", IMAGE_TAGS) : "${REGISTRY}/${REPOSITORY}/ci-builder:${tag}"] } From 485f9c659164b0c3906f9683d9e2748b646c5ebc Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Fri, 27 Oct 2023 08:59:11 +1000 Subject: [PATCH 136/279] ci: Disallow . and / in docker tags again. Remove the git tag from the release tags to add and allow the release script to add it instead. --- .circleci/config.yml | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index 118113b542a3..e9b5021d5aa7 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -218,7 +218,7 @@ jobs: export REGISTRY="<>" export REPOSITORY="<>" - export IMAGE_TAGS="$(echo -ne "<>" | sed "s/[^a-zA-Z0-9\n,./]/-/g")" + export IMAGE_TAGS="$(echo -ne "<>" | sed "s/[^a-zA-Z0-9\n,]/-/g")" export GIT_COMMIT="$(git rev-parse HEAD)" export GIT_DATE="$(git show -s --format='%ct')" export GIT_VERSION="<>" @@ -1493,7 +1493,7 @@ workflows: branches: ignore: /.*/ docker_name: op-heartbeat - docker_tags: <>,<> + docker_tags: <> requires: ['op-stack-go-docker-build-release'] platforms: "linux/amd64,linux/arm64" publish: true @@ -1508,7 +1508,7 @@ workflows: branches: ignore: /.*/ docker_name: op-node - docker_tags: <>,<> + docker_tags: <> requires: ['op-stack-go-docker-build-release'] platforms: "linux/amd64,linux/arm64" publish: true @@ -1523,7 +1523,7 @@ workflows: branches: ignore: /.*/ docker_name: op-batcher - docker_tags: <>,<> + docker_tags: <> requires: ['op-stack-go-docker-build-release'] platforms: "linux/amd64,linux/arm64" publish: true @@ -1538,7 +1538,7 @@ workflows: branches: ignore: /.*/ docker_name: op-proposer - docker_tags: <>,<> + docker_tags: <> requires: ['op-stack-go-docker-build-release'] platforms: "linux/amd64,linux/arm64" publish: true @@ -1553,7 +1553,7 @@ workflows: branches: ignore: /.*/ docker_name: op-challenger - docker_tags: <>,<> + docker_tags: <> requires: ['op-stack-go-docker-build-release'] platforms: "linux/amd64,linux/arm64" publish: true @@ -1568,7 +1568,7 @@ workflows: branches: ignore: /.*/ docker_name: op-ufm - docker_tags: <>,<> + docker_tags: <> publish: true release: true context: @@ -1583,7 +1583,7 @@ workflows: branches: ignore: /.*/ docker_name: proxyd - docker_tags: <>,<> + docker_tags: <> publish: true release: true context: @@ -1598,7 +1598,7 @@ workflows: branches: ignore: /.*/ docker_name: indexer - docker_tags: <>,<> + docker_tags: <> publish: true release: true context: From b62235a3365968409a0b0b0dbea68a8229948d25 Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Fri, 27 Oct 2023 09:23:57 +1000 Subject: [PATCH 137/279] op-e2e: Use a short wait for new claim when performing all possible dishonest moves. The timeout is expected to be reached sometimes because the challenger has chosen not to respond, so we don't want the default long timeout. --- op-e2e/e2eutils/disputegame/dishonest_helper.go | 5 ++++- op-e2e/e2eutils/disputegame/game_helper.go | 5 ++++- 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/op-e2e/e2eutils/disputegame/dishonest_helper.go b/op-e2e/e2eutils/disputegame/dishonest_helper.go index fde8b817843e..6b6f1912c18e 100644 --- a/op-e2e/e2eutils/disputegame/dishonest_helper.go +++ b/op-e2e/e2eutils/disputegame/dishonest_helper.go @@ -3,6 +3,7 @@ package disputegame import ( "context" "errors" + "time" "github.com/ethereum-optimism/optimism/op-challenger/game/fault/types" "github.com/ethereum/go-ethereum/common" @@ -96,7 +97,9 @@ func (d *DishonestHelper) ExhaustDishonestClaims(ctx context.Context) { var numClaimsSeen int64 for { - newCount, err := d.WaitForNewClaim(ctx, numClaimsSeen) + // Use a short timeout since we don't know the challenger will respond, + // and this is only designed for the alphabet game where the response should be fast. + newCount, err := d.waitForNewClaim(ctx, numClaimsSeen, 30*time.Second) if errors.Is(err, context.DeadlineExceeded) { // we assume that the honest challenger has stopped responding // There's nothing to respond to. diff --git a/op-e2e/e2eutils/disputegame/game_helper.go b/op-e2e/e2eutils/disputegame/game_helper.go index 5563d437fc59..6ce31f2f9c76 100644 --- a/op-e2e/e2eutils/disputegame/game_helper.go +++ b/op-e2e/e2eutils/disputegame/game_helper.go @@ -304,7 +304,10 @@ func (g *FaultGameHelper) ChallengeRootClaim(ctx context.Context, performMove Mo } func (g *FaultGameHelper) WaitForNewClaim(ctx context.Context, checkPoint int64) (int64, error) { - timedCtx, cancel := context.WithTimeout(ctx, defaultTimeout) + return g.waitForNewClaim(ctx, checkPoint, defaultTimeout) +} +func (g *FaultGameHelper) waitForNewClaim(ctx context.Context, checkPoint int64, timeout time.Duration) (int64, error) { + timedCtx, cancel := context.WithTimeout(ctx, timeout) defer cancel() var newClaimLen int64 err := wait.For(timedCtx, time.Second, func() (bool, error) { From 40858b5a39819cce8bf9e7962fd510cd76a6d678 Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Fri, 27 Oct 2023 10:09:04 +1000 Subject: [PATCH 138/279] op-e2e: Fix race condition with removing included tx from the pool The tx pool updates async, so the list of pending transactions may contain transactions included in a block if the tx pool update hasn't completed. Filter any transactions with nonces that are too low to avoid this race condition. --- op-e2e/actions/l1_miner.go | 23 ++++------------- op-e2e/actions/l2_engine.go | 21 ++-------------- op-e2e/actions/tx_helper.go | 49 +++++++++++++++++++++++++++++++++++++ 3 files changed, 56 insertions(+), 37 deletions(-) create mode 100644 op-e2e/actions/tx_helper.go diff --git a/op-e2e/actions/l1_miner.go b/op-e2e/actions/l1_miner.go index 812d35dfb3f9..8ba3ee857b67 100644 --- a/op-e2e/actions/l1_miner.go +++ b/op-e2e/actions/l1_miner.go @@ -1,11 +1,8 @@ package actions import ( - "context" "math/big" - "time" - "github.com/ethereum-optimism/optimism/op-e2e/e2eutils/wait" "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/consensus/misc/eip1559" "github.com/ethereum/go-ethereum/core" @@ -98,22 +95,12 @@ func (s *L1Miner) ActL1IncludeTx(from common.Address) Action { t.InvalidAction("no tx inclusion when not building l1 block") return } - var i uint64 - var txs []*types.Transaction - var q []*types.Transaction - // Wait for the tx to be in the pending tx queue - ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) - defer cancel() - err := wait.For(ctx, time.Second, func() (bool, error) { - i = s.pendingIndices[from] - txs, q = s.eth.TxPool().ContentFrom(from) - return uint64(len(txs)) > i, nil - }) - require.NoError(t, err, - "no pending txs from %s, and have %d unprocessable queued txs from this account: %w", from, len(q), err) - tx := txs[i] + getPendingIndex := func(from common.Address) uint64 { + return s.pendingIndices[from] + } + tx := firstValidTx(t, from, getPendingIndex, s.eth.TxPool().ContentFrom, s.EthClient().NonceAt) s.IncludeTx(t, tx) - s.pendingIndices[from] = i + 1 // won't retry the tx + s.pendingIndices[from] = s.pendingIndices[from] + 1 // won't retry the tx } } diff --git a/op-e2e/actions/l2_engine.go b/op-e2e/actions/l2_engine.go index 33669ec6afcc..d1230ec8b7c5 100644 --- a/op-e2e/actions/l2_engine.go +++ b/op-e2e/actions/l2_engine.go @@ -1,12 +1,9 @@ package actions import ( - "context" "errors" - "time" "github.com/ethereum-optimism/optimism/op-e2e/e2eutils" - "github.com/ethereum-optimism/optimism/op-e2e/e2eutils/wait" "github.com/ethereum-optimism/optimism/op-program/client/l2/engineapi" "github.com/stretchr/testify/require" @@ -179,22 +176,8 @@ func (e *L2Engine) ActL2IncludeTx(from common.Address) Action { return } - var i uint64 - var txs []*types.Transaction - var q []*types.Transaction - // Wait for the tx to be in the pending tx queue - ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) - defer cancel() - err := wait.For(ctx, time.Second, func() (bool, error) { - i = e.engineApi.PendingIndices(from) - txs, q = e.eth.TxPool().ContentFrom(from) - return uint64(len(txs)) > i, nil - }) - require.NoError(t, err, - "no pending txs from %s, and have %d unprocessable queued txs from this account: %w", from, len(q), err) - - tx := txs[i] - err = e.engineApi.IncludeTx(tx, from) + tx := firstValidTx(t, from, e.engineApi.PendingIndices, e.eth.TxPool().ContentFrom, e.EthClient().NonceAt) + err := e.engineApi.IncludeTx(tx, from) if errors.Is(err, engineapi.ErrNotBuildingBlock) { t.InvalidAction(err.Error()) } else if errors.Is(err, engineapi.ErrUsesTooMuchGas) { diff --git a/op-e2e/actions/tx_helper.go b/op-e2e/actions/tx_helper.go new file mode 100644 index 000000000000..e63c1a224822 --- /dev/null +++ b/op-e2e/actions/tx_helper.go @@ -0,0 +1,49 @@ +package actions + +import ( + "context" + "math/big" + "time" + + "github.com/ethereum-optimism/optimism/op-e2e/e2eutils/wait" + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/core/types" + "github.com/stretchr/testify/require" +) + +// firstValidTx finds the first transaction that is valid for inclusion from the specified address. +// It uses a waiter and filtering of already included transactions to avoid race conditions with the async +// updates to the transaction pool. +func firstValidTx( + t Testing, + from common.Address, + pendingIndices func(common.Address) uint64, + contentFrom func(common.Address) ([]*types.Transaction, []*types.Transaction), + nonceAt func(context.Context, common.Address, *big.Int) (uint64, error), +) *types.Transaction { + var i uint64 + var txs []*types.Transaction + var q []*types.Transaction + // Wait for the tx to be in the pending tx queue + ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) + defer cancel() + err := wait.For(ctx, time.Second, func() (bool, error) { + i = pendingIndices(from) + txs, q = contentFrom(from) + // Remove any transactions that have already been included in the head block + // The tx pool only prunes included transactions async so they may still be in the list + nonce, err := nonceAt(ctx, from, nil) + if err != nil { + return false, err + } + for len(txs) > 0 && txs[0].Nonce() < nonce { + t.Logf("Removing already included transaction from list of length %v", len(txs)) + txs = txs[1:] + } + return uint64(len(txs)) > i, nil + }) + require.NoError(t, err, + "no pending txs from %s, and have %d unprocessable queued txs from this account: %w", from, len(q), err) + + return txs[i] +} From 9f388881064bcc4e15635293c7ff1f91d62f1ef1 Mon Sep 17 00:00:00 2001 From: Mark Tyneway Date: Thu, 26 Oct 2023 18:54:34 -0600 Subject: [PATCH 139/279] ci-builder: go version env var Makes setting the go version via the env var `GO_VERSION` possible. Previously it hardcoded the version that it was unpacking but used the env var in other locations. Now setting the env var for `GO_VERSION` will allow the user to configure whatever version they want to include in the image. --- ops/docker/ci-builder/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ops/docker/ci-builder/Dockerfile b/ops/docker/ci-builder/Dockerfile index 4f7cf7234978..ab9f0565c258 100644 --- a/ops/docker/ci-builder/Dockerfile +++ b/ops/docker/ci-builder/Dockerfile @@ -39,7 +39,7 @@ ENV GO_VERSION=1.21.1 # Fetch go manually, rather than using a Go base image, so we can copy the installation into the final stage RUN curl -sL https://go.dev/dl/go$GO_VERSION.linux-amd64.tar.gz -o go$GO_VERSION.linux-amd64.tar.gz && \ - tar -C /usr/local/ -xzvf go1.21.1.linux-amd64.tar.gz + tar -C /usr/local/ -xzvf go$GO_VERSION.linux-amd64.tar.gz ENV GOPATH=/go ENV PATH=/usr/local/go/bin:$GOPATH/bin:$PATH From ea77d5a1999aba8e97b95ee06cf9f54023bbbcce Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Fri, 27 Oct 2023 11:05:50 +1000 Subject: [PATCH 140/279] ci: Notify slack when semgrep or docker publish fails on develop branch --- .circleci/config.yml | 42 +++++++++++++++++++++++++++++++----------- 1 file changed, 31 insertions(+), 11 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index e9b5021d5aa7..0e4ff1745cef 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -58,6 +58,19 @@ commands: cd ops/check-changed pip3 install -r requirements.txt python3 main.py "<>" + notify-failures-on-develop: + description: "Notify Slack" + parameters: + channel: + type: string + default: C03N11M0BBN + steps: + - slack/notify: + channel: << parameters.channel >> + event: fail + template: basic_fail_1 + branch_pattern: develop + jobs: cannon-go-lint-and-test: docker: @@ -260,6 +273,10 @@ jobs: <> no_output_timeout: 45m + - when: + condition: "<>" + steps: + - notify-failures-on-develop - when: condition: "<>" steps: @@ -585,10 +602,7 @@ jobs: - run: name: Check TODO issues command: ./ops/scripts/todo-checker.sh --verbose - - slack/notify: - channel: C03N11M0BBN - event: fail - template: basic_fail_1 + - notify-failures-on-develop bedrock-markdown: machine: @@ -632,10 +646,8 @@ jobs: name: link lint command: | make bedrock-markdown-links - - slack/notify: + - notify-failures-on-develop: channel: C055R639XT9 #notify-link-check - event: fail - template: basic_fail_1 fuzz-golang: parameters: @@ -1114,6 +1126,7 @@ jobs: - run: name: "Semgrep scan" command: semgrep ci + - notify-failures-on-develop go-mod-download: docker: @@ -1173,10 +1186,7 @@ jobs: command: | make verify-goerli working_directory: op-program - - slack/notify: - channel: C03N11M0BBN - event: fail - template: basic_fail_1 + - notify-failures-on-develop op-program-compat: docker: @@ -1672,6 +1682,7 @@ workflows: platforms: "linux/amd64,linux/arm64" context: - oplabs-gcr + - slack - docker-build: name: op-node-docker-publish docker_name: op-node @@ -1681,6 +1692,7 @@ workflows: publish: true context: - oplabs-gcr + - slack - docker-build: name: op-batcher-docker-publish docker_name: op-batcher @@ -1690,6 +1702,7 @@ workflows: publish: true context: - oplabs-gcr + - slack - docker-build: name: op-program-docker-publish docker_name: op-program @@ -1699,6 +1712,7 @@ workflows: publish: true context: - oplabs-gcr + - slack - docker-build: name: op-proposer-docker-publish docker_name: op-proposer @@ -1708,6 +1722,7 @@ workflows: publish: true context: - oplabs-gcr + - slack - docker-build: name: op-challenger-docker-publish docker_name: op-challenger @@ -1717,6 +1732,7 @@ workflows: publish: true context: - oplabs-gcr + - slack - docker-build: name: op-heartbeat-docker-publish docker_name: op-heartbeat @@ -1726,6 +1742,7 @@ workflows: publish: true context: - oplabs-gcr + - slack - docker-build: name: indexer-docker-publish docker_name: indexer @@ -1733,6 +1750,7 @@ workflows: publish: true context: - oplabs-gcr + - slack platforms: "linux/amd64,linux/arm64" - docker-build: name: chain-mon-docker-publish @@ -1741,6 +1759,7 @@ workflows: publish: true context: - oplabs-gcr + - slack - docker-build: name: ufm-metamask-docker-publish docker_name: ufm-metamask @@ -1748,3 +1767,4 @@ workflows: publish: true context: - oplabs-gcr + - slack From 2ec6f8fc8bec825b72d95ec105a44c6a3ee88818 Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Fri, 27 Oct 2023 11:22:46 +1000 Subject: [PATCH 141/279] op-e2e: Disable cannon tests for external geth Moves choice of running cannon tests to a separate env var rather so it is possible to run cannon tests with HTTP when desired. --- .circleci/config.yml | 9 ++++++++- op-e2e/faultproof_test.go | 14 +++++++------- op-e2e/helper.go | 6 +++--- 3 files changed, 18 insertions(+), 11 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index e9b5021d5aa7..beaffef8fb97 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -772,6 +772,10 @@ jobs: target: description: The make target to execute type: string + cannon_enabled: + description: Whether to include cannon tests + default: true + type: boolean docker: - image: us-docker.pkg.dev/oplabs-tools-artifacts/images/ci-builder:latest resource_class: xlarge @@ -797,7 +801,8 @@ jobs: command: go tool dist list | grep mips - run: name: run tests - command: + command: | + export OP_E2E_CANNON_ENABLED="<>" # Note: We don't use circle CI test splits because we need to split by test name, not by package. There is an additional # constraint that gotestsum does not currently (nor likely will) accept files from different pacakges when building. JUNIT_FILE=/tmp/test-results/<>_<>.xml make <> @@ -1361,6 +1366,7 @@ workflows: name: op-e2e-HTTP-tests module: op-e2e target: test-http + cannon_enabled: false requires: - op-stack-go-lint - devnet-allocs @@ -1368,6 +1374,7 @@ workflows: name: op-e2e-ext-geth-tests module: op-e2e target: test-external-geth + cannon_enabled: false requires: - op-stack-go-lint - devnet-allocs diff --git a/op-e2e/faultproof_test.go b/op-e2e/faultproof_test.go index a191532139d6..e6b2c755aeba 100644 --- a/op-e2e/faultproof_test.go +++ b/op-e2e/faultproof_test.go @@ -18,7 +18,7 @@ import ( ) func TestMultipleCannonGames(t *testing.T) { - InitParallel(t, SkipIfHTTP) + InitParallel(t, UsesCannon) ctx := context.Background() sys, l1Client := startFaultDisputeSystem(t) @@ -78,7 +78,7 @@ func TestMultipleCannonGames(t *testing.T) { } func TestMultipleGameTypes(t *testing.T) { - InitParallel(t, SkipIfHTTP) + InitParallel(t, UsesCannon) ctx := context.Background() sys, l1Client := startFaultDisputeSystem(t) @@ -277,7 +277,7 @@ func TestChallengerCompleteExhaustiveDisputeGame(t *testing.T) { } func TestCannonDisputeGame(t *testing.T) { - InitParallel(t, SkipIfHTTP) + InitParallel(t, UsesCannon) tests := []struct { name string @@ -328,7 +328,7 @@ func TestCannonDisputeGame(t *testing.T) { } func TestCannonDefendStep(t *testing.T) { - InitParallel(t, SkipIfHTTP) + InitParallel(t, UsesCannon) ctx := context.Background() sys, l1Client := startFaultDisputeSystem(t) @@ -370,7 +370,7 @@ func TestCannonDefendStep(t *testing.T) { } func TestCannonProposedOutputRootInvalid(t *testing.T) { - InitParallel(t, SkipIfHTTP) + InitParallel(t, UsesCannon) // honestStepsFail attempts to perform both an attack and defend step using the correct trace. honestStepsFail := func(ctx context.Context, game *disputegame.CannonGameHelper, correctTrace *disputegame.HonestHelper, parentClaimIdx int64) { // Attack step should fail @@ -448,7 +448,7 @@ func TestCannonProposedOutputRootInvalid(t *testing.T) { } func TestCannonPoisonedPostState(t *testing.T) { - InitParallel(t, SkipIfHTTP) + InitParallel(t, UsesCannon) ctx := context.Background() sys, l1Client := startFaultDisputeSystem(t) @@ -558,7 +558,7 @@ func setupDisputeGameForInvalidOutputRoot(t *testing.T, outputRoot common.Hash) } func TestCannonChallengeWithCorrectRoot(t *testing.T) { - InitParallel(t, SkipIfHTTP) + InitParallel(t, UsesCannon) ctx := context.Background() sys, l1Client := startFaultDisputeSystem(t) diff --git a/op-e2e/helper.go b/op-e2e/helper.go index eafeeb576e31..0b36458d4780 100644 --- a/op-e2e/helper.go +++ b/op-e2e/helper.go @@ -17,8 +17,8 @@ func InitParallel(t *testing.T, opts ...func(t *testing.T)) { } } -func SkipIfHTTP(t *testing.T) { - if UseHTTP() { - t.Skip("Skipping test because HTTP connection is in use") +func UsesCannon(t *testing.T) { + if os.Getenv("OP_E2E_CANNON_ENABLED") == "false" { + t.Skip("Skipping cannon test") } } From eb7e94ea27d4616ffdeb3a5272ab930920be8cf6 Mon Sep 17 00:00:00 2001 From: Michael de Hoog Date: Thu, 26 Oct 2023 15:40:45 -1000 Subject: [PATCH 142/279] Missing envvars setting for --override.canyon flag --- op-node/flags/flags.go | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/op-node/flags/flags.go b/op-node/flags/flags.go index 8f0247949137..c660db759acd 100644 --- a/op-node/flags/flags.go +++ b/op-node/flags/flags.go @@ -254,9 +254,10 @@ var ( EnvVars: prefixEnvVars("ROLLUP_LOAD_PROTOCOL_VERSIONS"), } CanyonOverrideFlag = &cli.Uint64Flag{ - Name: "override.canyon", - Usage: "Manually specify the Canyon fork timestamp, overriding the bundled setting", - Hidden: false, + Name: "override.canyon", + Usage: "Manually specify the Canyon fork timestamp, overriding the bundled setting", + EnvVars: prefixEnvVars("OVERRIDE_CANYON"), + Hidden: false, } ) From d9e44b5694c1c8a943b1195968c3eaeb6340b4a6 Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Fri, 27 Oct 2023 11:48:15 +1000 Subject: [PATCH 143/279] op-e2e: Wait for external clients to terminate after Kill() Otherwise the client may write to a directory the test tries to delete causing failures. --- op-e2e/external.go | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/op-e2e/external.go b/op-e2e/external.go index aaeaa4448184..0efdf4deb600 100644 --- a/op-e2e/external.go +++ b/op-e2e/external.go @@ -2,6 +2,7 @@ package op_e2e import ( "encoding/json" + "errors" "math/big" "os" "os/exec" @@ -51,6 +52,11 @@ func (eec *ExternalEthClient) Close() error { select { case <-time.After(5 * time.Second): eec.Session.Kill() + select { + case <-time.After(30 * time.Second): + return errors.New("external client failed to terminate") + case <-eec.Session.Exited: + } case <-eec.Session.Exited: } return nil From 17de450d52f71cc0bd86d5c777348381a5b9f7c1 Mon Sep 17 00:00:00 2001 From: Lukas Rosario <36800180+lukasrosario@users.noreply.github.com> Date: Fri, 27 Oct 2023 05:02:57 -0400 Subject: [PATCH 144/279] add timestamp to op-indexer withdrawals response --- indexer/api/routes/withdrawals.go | 1 + 1 file changed, 1 insertion(+) diff --git a/indexer/api/routes/withdrawals.go b/indexer/api/routes/withdrawals.go index 1fc2e1d84166..7c3ed399c00e 100644 --- a/indexer/api/routes/withdrawals.go +++ b/indexer/api/routes/withdrawals.go @@ -16,6 +16,7 @@ func newWithdrawalResponse(withdrawals *database.L2BridgeWithdrawalsResponse) mo item := models.WithdrawalItem{ Guid: withdrawal.L2BridgeWithdrawal.TransactionWithdrawalHash.String(), L2BlockHash: withdrawal.L2BlockHash.String(), + Timestamp: withdrawal.L2BridgeWithdrawal.Tx.Timestamp, From: withdrawal.L2BridgeWithdrawal.Tx.FromAddress.String(), To: withdrawal.L2BridgeWithdrawal.Tx.ToAddress.String(), TransactionHash: withdrawal.L2TransactionHash.String(), From 1a521c94c44804b9972fe9046b3f672bac81f006 Mon Sep 17 00:00:00 2001 From: Jason Yellick Date: Fri, 27 Oct 2023 11:09:18 -0400 Subject: [PATCH 145/279] Parameterize python binary in Makefile Although ideally, all python minor versions (3.x) would be backwards compatible, unfortunately that is not the case. Consequently, sometimes developers will have multiple python builds available on their systems, not just a single `python3`. This change simply makes the python binary name a parameter of the Makefile so that affected developers may pass: ``` PYTHON=python-3.9 make ``` Or similar. --- Makefile | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/Makefile b/Makefile index 8afbb12550db..01a42d947702 100644 --- a/Makefile +++ b/Makefile @@ -3,6 +3,9 @@ ITESTS_L2_HOST=http://localhost:9545 BEDROCK_TAGS_REMOTE?=origin OP_STACK_GO_BUILDER?=us-docker.pkg.dev/oplabs-tools-artifacts/images/op-stack-go:latest +# Requires at least Python v3.9; specify a minor version below if needed +PYTHON?=python3 + build: build-go build-ts .PHONY: build @@ -114,14 +117,14 @@ pre-devnet: devnet-up: pre-devnet ./ops/scripts/newer-file.sh .devnet/allocs-l1.json ./packages/contracts-bedrock \ || make devnet-allocs - PYTHONPATH=./bedrock-devnet python3 ./bedrock-devnet/main.py --monorepo-dir=. + PYTHONPATH=./bedrock-devnet $(PYTHON) ./bedrock-devnet/main.py --monorepo-dir=. .PHONY: devnet-up # alias for devnet-up devnet-up-deploy: devnet-up devnet-test: pre-devnet - PYTHONPATH=./bedrock-devnet python3 ./bedrock-devnet/main.py --monorepo-dir=. --test + PYTHONPATH=./bedrock-devnet $(PYTHON) ./bedrock-devnet/main.py --monorepo-dir=. --test .PHONY: devnet-test devnet-down: @@ -137,7 +140,7 @@ devnet-clean: .PHONY: devnet-clean devnet-allocs: pre-devnet - PYTHONPATH=./bedrock-devnet python3 ./bedrock-devnet/main.py --monorepo-dir=. --allocs + PYTHONPATH=./bedrock-devnet $(PYTHON) ./bedrock-devnet/main.py --monorepo-dir=. --allocs devnet-logs: @(cd ./ops-bedrock && docker compose logs -f) From 5393e64d3102baee8ade9015ce7e7a77689c1f5b Mon Sep 17 00:00:00 2001 From: inphi Date: Fri, 27 Oct 2023 11:35:22 -0400 Subject: [PATCH 146/279] ci: enable codecov comments for contracts --- codecov.yml | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/codecov.yml b/codecov.yml index 88f4b4ee729b..953756f48752 100644 --- a/codecov.yml +++ b/codecov.yml @@ -1,4 +1,12 @@ -comment: false +codecov: + require_ci_to_pass: false +comment: + layout: "diff, flags, files" + behavior: default + require_changes: false + flags: + - contracts-bedrock-tests + ignore: - "op-e2e" - "**/*.t.sol" @@ -13,6 +21,7 @@ coverage: threshold: 0% # coverage is not allowed to reduce vs. the PR base base: auto informational: true + enabled: true project: default: informational: true From 89c014831d9ff8624ee22ac6034997267ea5cc8f Mon Sep 17 00:00:00 2001 From: clabby Date: Fri, 27 Oct 2023 18:35:47 -0400 Subject: [PATCH 147/279] Change revision of nightly foundry in devnet CI --- .circleci/config.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index c669ebcac3bd..16a9fb0f5037 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -989,7 +989,7 @@ jobs: command: | curl -L https://foundry.paradigm.xyz | bash source $HOME/.bashrc - foundryup + foundryup --version nightly-aa257c2fb50814dfc5da4b3688cd3b95b5e3844d echo 'export PATH=$HOME/.foundry/bin:$PATH' >> $BASH_ENV source $HOME/.bashrc forge --version From 80d619ab8bd4b4246336c97bbf662efb49754a5e Mon Sep 17 00:00:00 2001 From: clabby Date: Thu, 26 Oct 2023 20:52:26 -0400 Subject: [PATCH 148/279] PoC init --- op-service/rethdb-reader/.gitignore | 2 + op-service/rethdb-reader/Cargo.lock | 7449 +++++++++++++++++++++++++++ op-service/rethdb-reader/Cargo.toml | 12 + op-service/rethdb-reader/README.md | 4 + op-service/rethdb-reader/src/lib.rs | 102 + op-service/sources/receipts.go | 28 +- op-service/sources/reth_db.go | 62 + op-service/sources/reth_db_test.go | 22 + 8 files changed, 7679 insertions(+), 2 deletions(-) create mode 100644 op-service/rethdb-reader/.gitignore create mode 100644 op-service/rethdb-reader/Cargo.lock create mode 100644 op-service/rethdb-reader/Cargo.toml create mode 100644 op-service/rethdb-reader/README.md create mode 100644 op-service/rethdb-reader/src/lib.rs create mode 100644 op-service/sources/reth_db.go create mode 100644 op-service/sources/reth_db_test.go diff --git a/op-service/rethdb-reader/.gitignore b/op-service/rethdb-reader/.gitignore new file mode 100644 index 000000000000..9004537baafa --- /dev/null +++ b/op-service/rethdb-reader/.gitignore @@ -0,0 +1,2 @@ +# Target +target/ diff --git a/op-service/rethdb-reader/Cargo.lock b/op-service/rethdb-reader/Cargo.lock new file mode 100644 index 000000000000..f0fe36055823 --- /dev/null +++ b/op-service/rethdb-reader/Cargo.lock @@ -0,0 +1,7449 @@ +# This file is automatically @generated by Cargo. +# It is not intended for manual editing. +version = 3 + +[[package]] +name = "addr2line" +version = "0.21.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8a30b2e23b9e17a9f90641c7ab1549cd9b44f296d3ccbf309d2863cfe398a0cb" +dependencies = [ + "gimli", +] + +[[package]] +name = "adler" +version = "1.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f26201604c87b1e01bd3d98f8d5d9a8fcbb815e8cedb41ffccbeb4bf593a35fe" + +[[package]] +name = "aead" +version = "0.4.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0b613b8e1e3cf911a086f53f03bf286f52fd7a7258e4fa606f0ef220d39d8877" +dependencies = [ + "generic-array", +] + +[[package]] +name = "aes" +version = "0.7.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9e8b47f52ea9bae42228d07ec09eb676433d7c4ed1ebdf0f1d1c29ed446f1ab8" +dependencies = [ + "cfg-if", + "cipher 0.3.0", + "cpufeatures", + "ctr 0.8.0", + "opaque-debug", +] + +[[package]] +name = "aes" +version = "0.8.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ac1f845298e95f983ff1944b728ae08b8cebab80d684f0a832ed0fc74dfa27e2" +dependencies = [ + "cfg-if", + "cipher 0.4.4", + "cpufeatures", +] + +[[package]] +name = "aes-gcm" +version = "0.9.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bc3be92e19a7ef47457b8e6f90707e12b6ac5d20c6f3866584fa3be0787d839f" +dependencies = [ + "aead", + "aes 0.7.5", + "cipher 0.3.0", + "ctr 0.7.0", + "ghash", + "subtle", +] + +[[package]] +name = "ahash" +version = "0.8.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "91429305e9f0a25f6205c5b8e0d2db09e0708a7a6df0f42212bb56c32c8ac97a" +dependencies = [ + "cfg-if", + "getrandom 0.2.10", + "once_cell", + "version_check", + "zerocopy", +] + +[[package]] +name = "aho-corasick" +version = "0.7.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cc936419f96fa211c1b9166887b38e5e40b19958e5b895be7c1f93adec7071ac" +dependencies = [ + "memchr", +] + +[[package]] +name = "aho-corasick" +version = "1.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b2969dcb958b36655471fc61f7e416fa76033bdd4bfed0678d8fee1e2d07a1f0" +dependencies = [ + "memchr", +] + +[[package]] +name = "alloc-no-stdlib" +version = "2.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cc7bb162ec39d46ab1ca8c77bf72e890535becd1751bb45f64c597edb4c8c6b3" + +[[package]] +name = "alloc-stdlib" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "94fb8275041c72129eb51b7d0322c29b8387a0386127718b096429201a5d6ece" +dependencies = [ + "alloc-no-stdlib", +] + +[[package]] +name = "allocator-api2" +version = "0.2.16" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0942ffc6dcaadf03badf6e6a2d0228460359d5e34b57ccdc720b7382dfbd5ec5" + +[[package]] +name = "alloy-dyn-abi" +version = "0.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e4e0daba57ddaba12dc9b21f608b843251f3de017f94a431dca4e7f4f72e5ba9" +dependencies = [ + "alloy-json-abi", + "alloy-primitives", + "alloy-sol-type-parser", + "alloy-sol-types", + "const-hex", + "derive_more", + "itoa", + "serde", + "serde_json", +] + +[[package]] +name = "alloy-json-abi" +version = "0.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "63c9319ad8b2b623c6a3ac15899f8ffb71479224762dbaedc385c16efbb6cfe3" +dependencies = [ + "alloy-primitives", + "alloy-sol-type-parser", + "serde", + "serde_json", +] + +[[package]] +name = "alloy-primitives" +version = "0.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a0628ec0ba5b98b3370bb6be17b12f23bfce8ee4ad83823325a20546d9b03b78" +dependencies = [ + "alloy-rlp", + "arbitrary", + "bytes", + "cfg-if", + "const-hex", + "derive_arbitrary", + "derive_more", + "getrandom 0.2.10", + "hex-literal", + "itoa", + "proptest", + "proptest-derive", + "rand 0.8.5", + "ruint", + "serde", + "tiny-keccak", +] + +[[package]] +name = "alloy-rlp" +version = "0.3.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cc0fac0fc16baf1f63f78b47c3d24718f3619b0714076f6a02957d808d52cbef" +dependencies = [ + "alloy-rlp-derive", + "arrayvec", + "bytes", + "smol_str", +] + +[[package]] +name = "alloy-rlp-derive" +version = "0.3.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c0391754c09fab4eae3404d19d0d297aa1c670c1775ab51d8a5312afeca23157" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.38", +] + +[[package]] +name = "alloy-sol-macro" +version = "0.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8a98ad1696a2e17f010ae8e43e9f2a1e930ed176a8e3ff77acfeff6dfb07b42c" +dependencies = [ + "const-hex", + "dunce", + "heck", + "proc-macro-error", + "proc-macro2", + "quote", + "syn 2.0.38", + "syn-solidity", + "tiny-keccak", +] + +[[package]] +name = "alloy-sol-type-parser" +version = "0.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "81c61ccc29e7c58bf16a2f780898852348183f58b127bde03ced6d07ad544787" +dependencies = [ + "winnow", +] + +[[package]] +name = "alloy-sol-types" +version = "0.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "98d7107bed88e8f09f0ddcc3335622d87bfb6821f3e0c7473329fb1cfad5e015" +dependencies = [ + "alloy-primitives", + "alloy-sol-macro", + "const-hex", + "serde", +] + +[[package]] +name = "android-tzdata" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e999941b234f3131b00bc13c22d06e8c5ff726d1b6318ac7eb276997bbb4fef0" + +[[package]] +name = "android_system_properties" +version = "0.1.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "819e7219dbd41043ac279b19830f2efc897156490d7fd6ea916720117ee66311" +dependencies = [ + "libc", +] + +[[package]] +name = "anstream" +version = "0.6.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2ab91ebe16eb252986481c5b62f6098f3b698a45e34b5b98200cf20dd2484a44" +dependencies = [ + "anstyle", + "anstyle-parse", + "anstyle-query", + "anstyle-wincon", + "colorchoice", + "utf8parse", +] + +[[package]] +name = "anstyle" +version = "1.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7079075b41f533b8c61d2a4d073c4676e1f8b249ff94a393b0595db304e0dd87" + +[[package]] +name = "anstyle-parse" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "317b9a89c1868f5ea6ff1d9539a69f45dffc21ce321ac1fd1160dfa48c8e2140" +dependencies = [ + "utf8parse", +] + +[[package]] +name = "anstyle-query" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5ca11d4be1bab0c8bc8734a9aa7bf4ee8316d462a08c6ac5052f888fef5b494b" +dependencies = [ + "windows-sys 0.48.0", +] + +[[package]] +name = "anstyle-wincon" +version = "3.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f0699d10d2f4d628a98ee7b57b289abbc98ff3bad977cb3152709d4bf2330628" +dependencies = [ + "anstyle", + "windows-sys 0.48.0", +] + +[[package]] +name = "anyhow" +version = "1.0.75" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a4668cab20f66d8d020e1fbc0ebe47217433c1b6c8f2040faf858554e394ace6" + +[[package]] +name = "aquamarine" +version = "0.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "df752953c49ce90719c7bf1fc587bc8227aed04732ea0c0f85e5397d7fdbd1a1" +dependencies = [ + "include_dir", + "itertools 0.10.5", + "proc-macro-error", + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "arbitrary" +version = "1.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a2e1373abdaa212b704512ec2bd8b26bd0b7d5c3f70117411a5d9a451383c859" +dependencies = [ + "derive_arbitrary", +] + +[[package]] +name = "ark-ff" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6b3235cc41ee7a12aaaf2c575a2ad7b46713a8a50bda2fc3b003a04845c05dd6" +dependencies = [ + "ark-ff-asm 0.3.0", + "ark-ff-macros 0.3.0", + "ark-serialize 0.3.0", + "ark-std 0.3.0", + "derivative", + "num-bigint", + "num-traits", + "paste", + "rustc_version 0.3.3", + "zeroize", +] + +[[package]] +name = "ark-ff" +version = "0.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ec847af850f44ad29048935519032c33da8aa03340876d351dfab5660d2966ba" +dependencies = [ + "ark-ff-asm 0.4.2", + "ark-ff-macros 0.4.2", + "ark-serialize 0.4.2", + "ark-std 0.4.0", + "derivative", + "digest 0.10.7", + "itertools 0.10.5", + "num-bigint", + "num-traits", + "paste", + "rustc_version 0.4.0", + "zeroize", +] + +[[package]] +name = "ark-ff-asm" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "db02d390bf6643fb404d3d22d31aee1c4bc4459600aef9113833d17e786c6e44" +dependencies = [ + "quote", + "syn 1.0.109", +] + +[[package]] +name = "ark-ff-asm" +version = "0.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3ed4aa4fe255d0bc6d79373f7e31d2ea147bcf486cba1be5ba7ea85abdb92348" +dependencies = [ + "quote", + "syn 1.0.109", +] + +[[package]] +name = "ark-ff-macros" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "db2fd794a08ccb318058009eefdf15bcaaaaf6f8161eb3345f907222bac38b20" +dependencies = [ + "num-bigint", + "num-traits", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "ark-ff-macros" +version = "0.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7abe79b0e4288889c4574159ab790824d0033b9fdcb2a112a3182fac2e514565" +dependencies = [ + "num-bigint", + "num-traits", + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "ark-serialize" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1d6c2b318ee6e10f8c2853e73a83adc0ccb88995aa978d8a3408d492ab2ee671" +dependencies = [ + "ark-std 0.3.0", + "digest 0.9.0", +] + +[[package]] +name = "ark-serialize" +version = "0.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "adb7b85a02b83d2f22f89bd5cac66c9c89474240cb6207cb1efc16d098e822a5" +dependencies = [ + "ark-std 0.4.0", + "digest 0.10.7", + "num-bigint", +] + +[[package]] +name = "ark-std" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1df2c09229cbc5a028b1d70e00fdb2acee28b1055dfb5ca73eea49c5a25c4e7c" +dependencies = [ + "num-traits", + "rand 0.8.5", +] + +[[package]] +name = "ark-std" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "94893f1e0c6eeab764ade8dc4c0db24caf4fe7cbbaafc0eba0a9030f447b5185" +dependencies = [ + "num-traits", + "rand 0.8.5", +] + +[[package]] +name = "arrayvec" +version = "0.7.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711" + +[[package]] +name = "async-compression" +version = "0.4.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f658e2baef915ba0f26f1f7c42bfb8e12f532a01f449a090ded75ae7a07e9ba2" +dependencies = [ + "brotli", + "flate2", + "futures-core", + "memchr", + "pin-project-lite", + "tokio", + "zstd 0.13.0", + "zstd-safe 7.0.0", +] + +[[package]] +name = "async-lock" +version = "2.8.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "287272293e9d8c41773cec55e365490fe034813a2f172f502d6ddcf75b2f582b" +dependencies = [ + "event-listener", +] + +[[package]] +name = "async-trait" +version = "0.1.74" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a66537f1bb974b254c98ed142ff995236e81b9d0fe4db0575f46612cb15eb0f9" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.38", +] + +[[package]] +name = "atomic-polyfill" +version = "0.1.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e3ff7eb3f316534d83a8a2c3d1674ace8a5a71198eba31e2e2b597833f699b28" +dependencies = [ + "critical-section", +] + +[[package]] +name = "atomic-polyfill" +version = "1.0.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8cf2bce30dfe09ef0bfaef228b9d414faaf7e563035494d7fe092dba54b300f4" +dependencies = [ + "critical-section", +] + +[[package]] +name = "attohttpc" +version = "0.16.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fdb8867f378f33f78a811a8eb9bf108ad99430d7aad43315dd9319c827ef6247" +dependencies = [ + "http", + "log", + "url", + "wildmatch", +] + +[[package]] +name = "aurora-engine-modexp" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bfacad86e9e138fca0670949eb8ed4ffdf73a55bded8887efe0863cd1a3a6f70" +dependencies = [ + "hex", + "num", +] + +[[package]] +name = "auto_impl" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fee3da8ef1276b0bee5dd1c7258010d8fffd31801447323115a25560e1327b89" +dependencies = [ + "proc-macro-error", + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "autocfg" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d468802bab17cbc0cc575e9b053f41e72aa36bfa6b7f55e3529ffa43161b97fa" + +[[package]] +name = "backon" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0c1a6197b2120bb2185a267f6515038558b019e92b832bb0320e96d66268dcf9" +dependencies = [ + "fastrand 1.9.0", + "futures-core", + "pin-project", + "tokio", +] + +[[package]] +name = "backtrace" +version = "0.3.69" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2089b7e3f35b9dd2d0ed921ead4f6d318c27680d4a5bd167b3ee120edb105837" +dependencies = [ + "addr2line", + "cc", + "cfg-if", + "libc", + "miniz_oxide", + "object", + "rustc-demangle", +] + +[[package]] +name = "base16ct" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4c7f02d4ea65f2c1853089ffd8d2787bdbc63de2f0d29dedbcf8ccdfa0ccd4cf" + +[[package]] +name = "base64" +version = "0.13.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9e1b586273c5702936fe7b7d6896644d8be71e6314cfe09d3167c95f712589e8" + +[[package]] +name = "base64" +version = "0.21.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "35636a1494ede3b646cc98f74f8e62c773a38a659ebc777a2cf26b9b74171df9" + +[[package]] +name = "base64ct" +version = "1.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8c3c1a368f70d6cf7302d78f8f7093da241fb8e8807c05cc9e51a125895a6d5b" + +[[package]] +name = "beef" +version = "0.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3a8241f3ebb85c056b509d4327ad0358fbbba6ffb340bf388f26350aeda225b1" +dependencies = [ + "serde", +] + +[[package]] +name = "bincode" +version = "1.3.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b1f45e9417d87227c7a56d22e471c6206462cba514c7590c09aff4cf6d1ddcad" +dependencies = [ + "serde", +] + +[[package]] +name = "bindgen" +version = "0.64.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c4243e6031260db77ede97ad86c27e501d646a27ab57b59a574f725d98ab1fb4" +dependencies = [ + "bitflags 1.3.2", + "cexpr", + "clang-sys", + "lazy_static", + "lazycell", + "peeking_take_while", + "proc-macro2", + "quote", + "regex", + "rustc-hash", + "shlex", + "syn 1.0.109", +] + +[[package]] +name = "bindgen" +version = "0.66.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f2b84e06fc203107bfbad243f4aba2af864eb7db3b1cf46ea0a023b0b433d2a7" +dependencies = [ + "bitflags 2.4.1", + "cexpr", + "clang-sys", + "lazy_static", + "lazycell", + "log", + "peeking_take_while", + "prettyplease", + "proc-macro2", + "quote", + "regex", + "rustc-hash", + "shlex", + "syn 2.0.38", + "which", +] + +[[package]] +name = "bindgen" +version = "0.68.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "726e4313eb6ec35d2730258ad4e15b547ee75d6afaa1361a922e78e59b7d8078" +dependencies = [ + "bitflags 2.4.1", + "cexpr", + "clang-sys", + "lazy_static", + "lazycell", + "peeking_take_while", + "proc-macro2", + "quote", + "regex", + "rustc-hash", + "shlex", + "syn 2.0.38", +] + +[[package]] +name = "binout" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "288c7b1c00556959bb7dc822d8adad4a30edd0d3a1fcc6839515792b8f300e5f" + +[[package]] +name = "bit-set" +version = "0.5.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0700ddab506f33b20a03b13996eccd309a48e5ff77d0d95926aa0210fb4e95f1" +dependencies = [ + "bit-vec", +] + +[[package]] +name = "bit-vec" +version = "0.6.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "349f9b6a179ed607305526ca489b34ad0a41aed5f7980fa90eb03160b69598fb" + +[[package]] +name = "bitflags" +version = "1.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a" + +[[package]] +name = "bitflags" +version = "2.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "327762f6e5a765692301e5bb513e0d9fef63be86bbc14528052b1cd3e6f03e07" +dependencies = [ + "arbitrary", + "serde", +] + +[[package]] +name = "bitm" +version = "0.2.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7becd9fb525c1c507eb025ec37129a0d9320aee17c841085a48101f4f18c0d27" +dependencies = [ + "dyn_size_of", +] + +[[package]] +name = "bitvec" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1bc2832c24239b0141d5674bb9174f9d68a8b5b3f2753311927c172ca46f7e9c" +dependencies = [ + "funty", + "radium", + "serde", + "tap", + "wyz", +] + +[[package]] +name = "block-buffer" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4152116fd6e9dadb291ae18fc1ec3575ed6d84c29642d97890f4b4a3417297e4" +dependencies = [ + "generic-array", +] + +[[package]] +name = "block-buffer" +version = "0.10.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3078c7629b62d3f0439517fa394996acacc5cbc91c5a20d8c658e77abd503a71" +dependencies = [ + "generic-array", +] + +[[package]] +name = "block-padding" +version = "0.3.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a8894febbff9f758034a5b8e12d87918f56dfc64a8e1fe757d65e29041538d93" +dependencies = [ + "generic-array", +] + +[[package]] +name = "blst" +version = "0.3.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c94087b935a822949d3291a9989ad2b2051ea141eda0fd4e478a75f6aa3e604b" +dependencies = [ + "cc", + "glob", + "threadpool", + "zeroize", +] + +[[package]] +name = "boa_ast" +version = "0.17.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "73498e9b2f0aa7db74977afa4d594657611e90587abf0dd564c0b55b4a130163" +dependencies = [ + "bitflags 2.4.1", + "boa_interner", + "boa_macros", + "indexmap 2.0.2", + "num-bigint", + "rustc-hash", +] + +[[package]] +name = "boa_engine" +version = "0.17.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "16377479d5d6d33896e7acdd1cc698d04a8f72004025bbbddf47558cd29146a6" +dependencies = [ + "bitflags 2.4.1", + "boa_ast", + "boa_gc", + "boa_icu_provider", + "boa_interner", + "boa_macros", + "boa_parser", + "boa_profiler", + "chrono", + "dashmap", + "fast-float", + "icu_normalizer", + "indexmap 2.0.2", + "itertools 0.11.0", + "num-bigint", + "num-integer", + "num-traits", + "num_enum 0.6.1", + "once_cell", + "pollster", + "rand 0.8.5", + "regress", + "rustc-hash", + "ryu-js", + "serde", + "serde_json", + "sptr", + "static_assertions", + "tap", + "thin-vec", + "thiserror", +] + +[[package]] +name = "boa_gc" +version = "0.17.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c97b44beaef9d4452342d117d94607fdfa8d474280f1ba0fd97853834e3a49b2" +dependencies = [ + "boa_macros", + "boa_profiler", + "thin-vec", +] + +[[package]] +name = "boa_icu_provider" +version = "0.17.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b30e52e34e451dd0bfc2c654a9a43ed34b0073dbd4ae3394b40313edda8627aa" +dependencies = [ + "icu_collections", + "icu_normalizer", + "icu_properties", + "icu_provider", + "icu_provider_adapters", + "icu_provider_blob", + "once_cell", +] + +[[package]] +name = "boa_interner" +version = "0.17.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f3e5afa991908cfbe79bd3109b824e473a1dc5f74f31fab91bb44c9e245daa77" +dependencies = [ + "boa_gc", + "boa_macros", + "hashbrown 0.14.2", + "indexmap 2.0.2", + "once_cell", + "phf", + "rustc-hash", + "static_assertions", +] + +[[package]] +name = "boa_macros" +version = "0.17.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "005fa0c5bd20805466dda55eb34cd709bb31a2592bb26927b47714eeed6914d8" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.38", + "synstructure", +] + +[[package]] +name = "boa_parser" +version = "0.17.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9e09afb035377a9044443b598187a7d34cd13164617182a4d7c348522ee3f052" +dependencies = [ + "bitflags 2.4.1", + "boa_ast", + "boa_icu_provider", + "boa_interner", + "boa_macros", + "boa_profiler", + "fast-float", + "icu_locid", + "icu_properties", + "icu_provider", + "icu_provider_macros", + "num-bigint", + "num-traits", + "once_cell", + "regress", + "rustc-hash", + "tinystr", +] + +[[package]] +name = "boa_profiler" +version = "0.17.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3190f92dfe48224adc92881c620f08ccf37ff62b91a094bb357fe53bd5e84647" + +[[package]] +name = "boyer-moore-magiclen" +version = "0.2.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "116d76fee857b03ecdd95d5f9555e46aa0cd34e5bb348a520e9445d151182a7e" +dependencies = [ + "debug-helper", +] + +[[package]] +name = "brotli" +version = "3.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "516074a47ef4bce09577a3b379392300159ce5b1ba2e501ff1c819950066100f" +dependencies = [ + "alloc-no-stdlib", + "alloc-stdlib", + "brotli-decompressor", +] + +[[package]] +name = "brotli-decompressor" +version = "2.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "da74e2b81409b1b743f8f0c62cc6254afefb8b8e50bbfe3735550f7aeefa3448" +dependencies = [ + "alloc-no-stdlib", + "alloc-stdlib", +] + +[[package]] +name = "bumpalo" +version = "3.14.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7f30e7476521f6f8af1a1c4c0b8cc94f0bee37d91763d0ca2665f299b6cd8aec" + +[[package]] +name = "byte-slice-cast" +version = "1.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c3ac9f8b63eca6fd385229b3675f6cc0dc5c8a5c8a54a59d4f52ffd670d87b0c" + +[[package]] +name = "byteorder" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1fd0f2584146f6f2ef48085050886acf353beff7305ebd1ae69500e27c67f64b" + +[[package]] +name = "bytes" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a2bd12c1caf447e69cd4528f47f94d203fd2582878ecb9e9465484c4148a8223" +dependencies = [ + "serde", +] + +[[package]] +name = "c-kzg" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ac926d808fb72fe09ebf471a091d6d72918876ccf0b4989766093d2d0d24a0ef" +dependencies = [ + "bindgen 0.66.1", + "blst", + "cc", + "glob", + "hex", + "libc", + "serde", +] + +[[package]] +name = "cassowary" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "df8670b8c7b9dae1793364eafadf7239c40d669904660c5960d74cfd80b46a53" + +[[package]] +name = "cc" +version = "1.0.83" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f1174fb0b6ec23863f8b971027804a42614e347eafb0a95bf0b12cdae21fc4d0" +dependencies = [ + "jobserver", + "libc", +] + +[[package]] +name = "cexpr" +version = "0.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6fac387a98bb7c37292057cffc56d62ecb629900026402633ae9160df93a8766" +dependencies = [ + "nom", +] + +[[package]] +name = "cfg-if" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" + +[[package]] +name = "chrono" +version = "0.4.31" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7f2c685bad3eb3d45a01354cedb7d5faa66194d1d58ba6e267a8de788f79db38" +dependencies = [ + "android-tzdata", + "iana-time-zone", + "js-sys", + "num-traits", + "serde", + "wasm-bindgen", + "windows-targets 0.48.5", +] + +[[package]] +name = "cipher" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7ee52072ec15386f770805afd189a01c8841be8696bed250fa2f13c4c0d6dfb7" +dependencies = [ + "generic-array", +] + +[[package]] +name = "cipher" +version = "0.4.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "773f3b9af64447d2ce9850330c473515014aa235e6a783b02db81ff39e4a3dad" +dependencies = [ + "crypto-common", + "inout", +] + +[[package]] +name = "clang-sys" +version = "1.6.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c688fc74432808e3eb684cae8830a86be1d66a2bd58e1f248ed0960a590baf6f" +dependencies = [ + "glob", + "libc", + "libloading", +] + +[[package]] +name = "clap" +version = "4.4.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ac495e00dcec98c83465d5ad66c5c4fabd652fd6686e7c6269b117e729a6f17b" +dependencies = [ + "clap_builder", + "clap_derive", +] + +[[package]] +name = "clap_builder" +version = "4.4.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c77ed9a32a62e6ca27175d00d29d05ca32e396ea1eb5fb01d8256b669cec7663" +dependencies = [ + "anstream", + "anstyle", + "clap_lex", + "strsim 0.10.0", +] + +[[package]] +name = "clap_derive" +version = "4.4.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cf9804afaaf59a91e75b022a30fb7229a7901f60c755489cc61c9b423b836442" +dependencies = [ + "heck", + "proc-macro2", + "quote", + "syn 2.0.38", +] + +[[package]] +name = "clap_lex" +version = "0.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "702fc72eb24e5a1e48ce58027a675bc24edd52096d5397d4aea7c6dd9eca0bd1" + +[[package]] +name = "cobs" +version = "0.2.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "67ba02a97a2bd10f4b59b25c7973101c79642302776489e030cd13cdab09ed15" + +[[package]] +name = "codecs-derive" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "convert_case 0.6.0", + "parity-scale-codec", + "proc-macro2", + "quote", + "serde", + "syn 2.0.38", +] + +[[package]] +name = "colorchoice" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "acbf1af155f9b9ef647e42cdc158db4b64a1b61f743629225fde6f3e0be2a7c7" + +[[package]] +name = "comfy-table" +version = "7.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7c64043d6c7b7a4c58e39e7efccfdea7b93d885a795d0c054a69dbbf4dd52686" +dependencies = [ + "crossterm 0.27.0", + "strum", + "strum_macros", + "unicode-width", +] + +[[package]] +name = "confy" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e37668cb35145dcfaa1931a5f37fde375eeae8068b4c0d2f289da28a270b2d2c" +dependencies = [ + "directories", + "serde", + "thiserror", + "toml 0.5.11", +] + +[[package]] +name = "const-hex" +version = "1.9.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c37be52ef5e3b394db27a2341010685ad5103c72ac15ce2e9420a7e8f93f342c" +dependencies = [ + "cfg-if", + "cpufeatures", + "hex", + "serde", +] + +[[package]] +name = "const-oid" +version = "0.9.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "28c122c3980598d243d63d9a704629a2d748d101f278052ff068be5a4423ab6f" + +[[package]] +name = "const-str" +version = "0.5.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "aca749d3d3f5b87a0d6100509879f9cf486ab510803a4a4e1001da1ff61c2bd6" + +[[package]] +name = "convert_case" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6245d59a3e82a7fc217c5828a6692dbc6dfb63a0c8c90495621f7b9d79704a0e" + +[[package]] +name = "convert_case" +version = "0.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ec182b0ca2f35d8fc196cf3404988fd8b8c739a4d270ff118a398feb0cbec1ca" +dependencies = [ + "unicode-segmentation", +] + +[[package]] +name = "core-foundation" +version = "0.9.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "194a7a9e6de53fa55116934067c844d9d749312f75c6f6d0980e8c252f8c2146" +dependencies = [ + "core-foundation-sys", + "libc", +] + +[[package]] +name = "core-foundation-sys" +version = "0.8.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e496a50fda8aacccc86d7529e2c1e0892dbd0f898a6b5645b5561b89c3210efa" + +[[package]] +name = "cpufeatures" +version = "0.2.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ce420fe07aecd3e67c5f910618fe65e94158f6dcc0adf44e00d69ce2bdfe0fd0" +dependencies = [ + "libc", +] + +[[package]] +name = "crc" +version = "3.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "86ec7a15cbe22e59248fc7eadb1907dab5ba09372595da4d73dd805ed4417dfe" +dependencies = [ + "crc-catalog", +] + +[[package]] +name = "crc-catalog" +version = "2.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9cace84e55f07e7301bae1c519df89cdad8cc3cd868413d3fdbdeca9ff3db484" + +[[package]] +name = "crc32fast" +version = "1.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b540bd8bc810d3885c6ea91e2018302f68baba2129ab3e88f32389ee9370880d" +dependencies = [ + "cfg-if", +] + +[[package]] +name = "critical-section" +version = "1.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7059fff8937831a9ae6f0fe4d658ffabf58f2ca96aa9dec1c889f936f705f216" + +[[package]] +name = "crossbeam-channel" +version = "0.5.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a33c2bf77f2df06183c3aa30d1e96c0695a313d4f9c453cc3762a6db39f99200" +dependencies = [ + "cfg-if", + "crossbeam-utils", +] + +[[package]] +name = "crossbeam-deque" +version = "0.8.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ce6fd6f855243022dcecf8702fef0c297d4338e226845fe067f6341ad9fa0cef" +dependencies = [ + "cfg-if", + "crossbeam-epoch", + "crossbeam-utils", +] + +[[package]] +name = "crossbeam-epoch" +version = "0.9.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ae211234986c545741a7dc064309f67ee1e5ad243d0e48335adc0484d960bcc7" +dependencies = [ + "autocfg", + "cfg-if", + "crossbeam-utils", + "memoffset", + "scopeguard", +] + +[[package]] +name = "crossbeam-utils" +version = "0.8.16" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5a22b2d63d4d1dc0b7f1b6b2747dd0088008a9be28b6ddf0b1e7d335e3037294" +dependencies = [ + "cfg-if", +] + +[[package]] +name = "crossterm" +version = "0.25.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e64e6c0fbe2c17357405f7c758c1ef960fce08bdfb2c03d88d2a18d7e09c4b67" +dependencies = [ + "bitflags 1.3.2", + "crossterm_winapi", + "libc", + "mio", + "parking_lot 0.12.1", + "signal-hook", + "signal-hook-mio", + "winapi", +] + +[[package]] +name = "crossterm" +version = "0.27.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f476fe445d41c9e991fd07515a6f463074b782242ccf4a5b7b1d1012e70824df" +dependencies = [ + "bitflags 2.4.1", + "crossterm_winapi", + "libc", + "mio", + "parking_lot 0.12.1", + "signal-hook", + "signal-hook-mio", + "winapi", +] + +[[package]] +name = "crossterm_winapi" +version = "0.9.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "acdd7c62a3665c7f6830a51635d9ac9b23ed385797f70a83bb8bafe9c572ab2b" +dependencies = [ + "winapi", +] + +[[package]] +name = "crunchy" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7a81dae078cea95a014a339291cec439d2f232ebe854a9d672b796c6afafa9b7" + +[[package]] +name = "crypto-bigint" +version = "0.5.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "740fe28e594155f10cfc383984cbefd529d7396050557148f79cb0f621204124" +dependencies = [ + "generic-array", + "rand_core 0.6.4", + "subtle", + "zeroize", +] + +[[package]] +name = "crypto-common" +version = "0.1.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1bfb12502f3fc46cca1bb51ac28df9d618d813cdc3d2f25b9fe775a34af26bb3" +dependencies = [ + "generic-array", + "typenum", +] + +[[package]] +name = "ctr" +version = "0.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a232f92a03f37dd7d7dd2adc67166c77e9cd88de5b019b9a9eecfaeaf7bfd481" +dependencies = [ + "cipher 0.3.0", +] + +[[package]] +name = "ctr" +version = "0.8.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "049bb91fb4aaf0e3c7efa6cd5ef877dbbbd15b39dad06d9948de4ec8a75761ea" +dependencies = [ + "cipher 0.3.0", +] + +[[package]] +name = "ctr" +version = "0.9.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0369ee1ad671834580515889b80f2ea915f23b8be8d0daa4bbaf2ac5c7590835" +dependencies = [ + "cipher 0.4.4", +] + +[[package]] +name = "cuckoofilter" +version = "0.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b810a8449931679f64cd7eef1bbd0fa315801b6d5d9cdc1ace2804d6529eee18" +dependencies = [ + "byteorder", + "fnv", + "rand 0.7.3", + "serde", + "serde_bytes", + "serde_derive", +] + +[[package]] +name = "curve25519-dalek" +version = "4.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e89b8c6a2e4b1f45971ad09761aafb85514a84744b67a95e32c3cc1352d1f65c" +dependencies = [ + "cfg-if", + "cpufeatures", + "curve25519-dalek-derive", + "digest 0.10.7", + "fiat-crypto", + "platforms", + "rustc_version 0.4.0", + "subtle", + "zeroize", +] + +[[package]] +name = "curve25519-dalek-derive" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "83fdaf97f4804dcebfa5862639bc9ce4121e82140bec2a987ac5140294865b5b" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.38", +] + +[[package]] +name = "darling" +version = "0.10.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0d706e75d87e35569db781a9b5e2416cff1236a47ed380831f959382ccd5f858" +dependencies = [ + "darling_core 0.10.2", + "darling_macro 0.10.2", +] + +[[package]] +name = "darling" +version = "0.20.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0209d94da627ab5605dcccf08bb18afa5009cfbef48d8a8b7d7bdbc79be25c5e" +dependencies = [ + "darling_core 0.20.3", + "darling_macro 0.20.3", +] + +[[package]] +name = "darling_core" +version = "0.10.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f0c960ae2da4de88a91b2d920c2a7233b400bc33cb28453a2987822d8392519b" +dependencies = [ + "fnv", + "ident_case", + "proc-macro2", + "quote", + "strsim 0.9.3", + "syn 1.0.109", +] + +[[package]] +name = "darling_core" +version = "0.20.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "177e3443818124b357d8e76f53be906d60937f0d3a90773a664fa63fa253e621" +dependencies = [ + "fnv", + "ident_case", + "proc-macro2", + "quote", + "strsim 0.10.0", + "syn 2.0.38", +] + +[[package]] +name = "darling_macro" +version = "0.10.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d9b5a2f4ac4969822c62224815d069952656cadc7084fdca9751e6d959189b72" +dependencies = [ + "darling_core 0.10.2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "darling_macro" +version = "0.20.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "836a9bbc7ad63342d6d6e7b815ccab164bc77a2d95d84bc3117a8c0d5c98e2d5" +dependencies = [ + "darling_core 0.20.3", + "quote", + "syn 2.0.38", +] + +[[package]] +name = "dashmap" +version = "5.5.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "978747c1d849a7d2ee5e8adc0159961c48fb7e5db2f06af6723b80123bb53856" +dependencies = [ + "cfg-if", + "hashbrown 0.14.2", + "lock_api", + "once_cell", + "parking_lot_core 0.9.9", +] + +[[package]] +name = "data-encoding" +version = "2.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c2e66c9d817f1720209181c316d28635c050fa304f9c79e47a520882661b7308" + +[[package]] +name = "debug-helper" +version = "0.3.13" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f578e8e2c440e7297e008bb5486a3a8a194775224bbc23729b0dbdfaeebf162e" + +[[package]] +name = "delay_map" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e4355c25cbf99edcb6b4a0e906f6bdc6956eda149e84455bea49696429b2f8e8" +dependencies = [ + "futures", + "tokio-util", +] + +[[package]] +name = "der" +version = "0.7.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fffa369a668c8af7dbf8b5e56c9f744fbd399949ed171606040001947de40b1c" +dependencies = [ + "const-oid", + "zeroize", +] + +[[package]] +name = "deranged" +version = "0.3.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0f32d04922c60427da6f9fef14d042d9edddef64cb9d4ce0d64d0685fbeb1fd3" +dependencies = [ + "powerfmt", + "serde", +] + +[[package]] +name = "derivative" +version = "2.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fcc3dd5e9e9c0b295d6e1e4d811fb6f157d5ffd784b8d202fc62eac8035a770b" +dependencies = [ + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "derive_arbitrary" +version = "1.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "53e0efad4403bfc52dc201159c4b842a246a14b98c64b55dfd0f2d89729dfeb8" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.38", +] + +[[package]] +name = "derive_builder" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a2658621297f2cf68762a6f7dc0bb7e1ff2cfd6583daef8ee0fed6f7ec468ec0" +dependencies = [ + "darling 0.10.2", + "derive_builder_core", + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "derive_builder_core" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2791ea3e372c8495c0bc2033991d76b512cd799d07491fbd6890124db9458bef" +dependencies = [ + "darling 0.10.2", + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "derive_more" +version = "0.99.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4fb810d30a7c1953f91334de7244731fc3f3c10d7fe163338a35b9f640960321" +dependencies = [ + "convert_case 0.4.0", + "proc-macro2", + "quote", + "rustc_version 0.4.0", + "syn 1.0.109", +] + +[[package]] +name = "diff" +version = "0.1.13" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "56254986775e3233ffa9c4d7d3faaf6d36a2c09d30b20687e9f88bc8bafc16c8" + +[[package]] +name = "digest" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d3dd60d1080a57a05ab032377049e0591415d2b31afd7028356dbf3cc6dcb066" +dependencies = [ + "generic-array", +] + +[[package]] +name = "digest" +version = "0.10.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9ed9a281f7bc9b7576e61468ba615a66a5c8cfdff42420a70aa82701a3b1e292" +dependencies = [ + "block-buffer 0.10.4", + "const-oid", + "crypto-common", + "subtle", +] + +[[package]] +name = "directories" +version = "4.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f51c5d4ddabd36886dd3e1438cb358cdcb0d7c499cb99cb4ac2e38e18b5cb210" +dependencies = [ + "dirs-sys 0.3.7", +] + +[[package]] +name = "dirs" +version = "5.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "44c45a9d03d6676652bcb5e724c7e988de1acad23a711b5217ab9cbecbec2225" +dependencies = [ + "dirs-sys 0.4.1", +] + +[[package]] +name = "dirs-next" +version = "2.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b98cf8ebf19c3d1b223e151f99a4f9f0690dca41414773390fc824184ac833e1" +dependencies = [ + "cfg-if", + "dirs-sys-next", +] + +[[package]] +name = "dirs-sys" +version = "0.3.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1b1d1d91c932ef41c0f2663aa8b0ca0342d444d842c06914aa0a7e352d0bada6" +dependencies = [ + "libc", + "redox_users", + "winapi", +] + +[[package]] +name = "dirs-sys" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "520f05a5cbd335fae5a99ff7a6ab8627577660ee5cfd6a94a6a929b52ff0321c" +dependencies = [ + "libc", + "option-ext", + "redox_users", + "windows-sys 0.48.0", +] + +[[package]] +name = "dirs-sys-next" +version = "0.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4ebda144c4fe02d1f7ea1a7d9641b6fc6b580adcfa024ae48797ecdeb6825b4d" +dependencies = [ + "libc", + "redox_users", + "winapi", +] + +[[package]] +name = "discv5" +version = "0.3.1" +source = "git+https://github.com/sigp/discv5?rev=f289bbd4c57d499bb1bdb393af3c249600a1c662#f289bbd4c57d499bb1bdb393af3c249600a1c662" +dependencies = [ + "aes 0.7.5", + "aes-gcm", + "arrayvec", + "delay_map", + "enr", + "fnv", + "futures", + "hashlink", + "hex", + "hkdf", + "lazy_static", + "lru 0.12.0", + "more-asserts", + "parking_lot 0.11.2", + "rand 0.8.5", + "rlp", + "smallvec", + "socket2 0.4.10", + "tokio", + "tracing", + "tracing-subscriber", + "uint", + "zeroize", +] + +[[package]] +name = "displaydoc" +version = "0.2.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "487585f4d0c6655fe74905e2504d8ad6908e4db67f744eb140876906c2f3175d" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.38", +] + +[[package]] +name = "dns-lookup" +version = "1.0.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "53ecafc952c4528d9b51a458d1a8904b81783feff9fde08ab6ed2545ff396872" +dependencies = [ + "cfg-if", + "libc", + "socket2 0.4.10", + "winapi", +] + +[[package]] +name = "dunce" +version = "1.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "56ce8c6da7551ec6c462cbaf3bfbc75131ebbfa1c944aeaa9dab51ca1c5f0c3b" + +[[package]] +name = "dyn-clone" +version = "1.0.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "23d2f3407d9a573d666de4b5bdf10569d73ca9478087346697dcbae6244bfbcd" + +[[package]] +name = "dyn_size_of" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6c8adcce29eef18ae1369bbd268fd56bf98144e80281315e9d4a82e34df001c7" + +[[package]] +name = "ecdsa" +version = "0.16.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a4b1e0c257a9e9f25f90ff76d7a68360ed497ee519c8e428d1825ef0000799d4" +dependencies = [ + "der", + "digest 0.10.7", + "elliptic-curve", + "rfc6979", + "signature", + "spki", +] + +[[package]] +name = "ed25519" +version = "2.2.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "115531babc129696a58c64a4fef0a8bf9e9698629fb97e9e40767d235cfbcd53" +dependencies = [ + "pkcs8", + "signature", +] + +[[package]] +name = "ed25519-dalek" +version = "2.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7277392b266383ef8396db7fdeb1e77b6c52fed775f5df15bb24f35b72156980" +dependencies = [ + "curve25519-dalek", + "ed25519", + "rand_core 0.6.4", + "serde", + "sha2", + "zeroize", +] + +[[package]] +name = "educe" +version = "0.4.23" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0f0042ff8246a363dbe77d2ceedb073339e85a804b9a47636c6e016a9a32c05f" +dependencies = [ + "enum-ordinalize", + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "either" +version = "1.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a26ae43d7bcc3b814de94796a5e736d4029efb0ee900c12e2d54c993ad1a1e07" + +[[package]] +name = "elliptic-curve" +version = "0.13.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d97ca172ae9dc9f9b779a6e3a65d308f2af74e5b8c921299075bdb4a0370e914" +dependencies = [ + "base16ct", + "crypto-bigint", + "digest 0.10.7", + "ff", + "generic-array", + "group", + "pkcs8", + "rand_core 0.6.4", + "sec1", + "subtle", + "zeroize", +] + +[[package]] +name = "embedded-io" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ef1a6892d9eef45c8fa6b9e0086428a2cca8491aca8f787c534a3d6d0bcb3ced" + +[[package]] +name = "endian-type" +version = "0.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c34f04666d835ff5d62e058c3995147c06f42fe86ff053337632bca83e42702d" + +[[package]] +name = "enr" +version = "0.9.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fe81b5c06ecfdbc71dd845216f225f53b62a10cb8a16c946836a3467f701d05b" +dependencies = [ + "base64 0.21.5", + "bytes", + "ed25519-dalek", + "hex", + "k256", + "log", + "rand 0.8.5", + "rlp", + "secp256k1 0.27.0", + "serde", + "sha3", + "zeroize", +] + +[[package]] +name = "enum-as-inner" +version = "0.3.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "570d109b813e904becc80d8d5da38376818a143348413f7149f1340fe04754d4" +dependencies = [ + "heck", + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "enum-as-inner" +version = "0.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5ffccbb6966c05b32ef8fbac435df276c4ae4d3dc55a8cd0eb9745e6c12f546a" +dependencies = [ + "heck", + "proc-macro2", + "quote", + "syn 2.0.38", +] + +[[package]] +name = "enum-ordinalize" +version = "3.1.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1bf1fa3f06bbff1ea5b1a9c7b14aa992a39657db60a2759457328d7e058f49ee" +dependencies = [ + "num-bigint", + "num-traits", + "proc-macro2", + "quote", + "syn 2.0.38", +] + +[[package]] +name = "enumn" +version = "0.1.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c2ad8cef1d801a4686bfd8919f0b30eac4c8e48968c437a6405ded4fb5272d2b" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.38", +] + +[[package]] +name = "equivalent" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5443807d6dff69373d433ab9ef5378ad8df50ca6298caf15de6e52e24aaf54d5" + +[[package]] +name = "errno" +version = "0.2.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f639046355ee4f37944e44f60642c6f3a7efa3cf6b78c78a0d989a8ce6c396a1" +dependencies = [ + "errno-dragonfly", + "libc", + "winapi", +] + +[[package]] +name = "errno" +version = "0.3.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ac3e13f66a2f95e32a39eaa81f6b95d42878ca0e1db0c7543723dfe12557e860" +dependencies = [ + "libc", + "windows-sys 0.48.0", +] + +[[package]] +name = "errno-dragonfly" +version = "0.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "aa68f1b12764fab894d2755d2518754e71b4fd80ecfb822714a1206c2aab39bf" +dependencies = [ + "cc", + "libc", +] + +[[package]] +name = "event-listener" +version = "2.5.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0206175f82b8d6bf6652ff7d71a1e27fd2e4efde587fd368662814d6ec1d9ce0" + +[[package]] +name = "eyre" +version = "0.6.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4c2b6b5a29c02cdc822728b7d7b8ae1bab3e3b05d44522770ddd49722eeac7eb" +dependencies = [ + "indenter", + "once_cell", +] + +[[package]] +name = "fast-float" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "95765f67b4b18863968b4a1bd5bb576f732b29a4a28c7cd84c09fa3e2875f33c" + +[[package]] +name = "fastrand" +version = "1.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e51093e27b0797c359783294ca4f0a911c270184cb10f85783b118614a1501be" +dependencies = [ + "instant", +] + +[[package]] +name = "fastrand" +version = "2.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "25cbce373ec4653f1a01a31e8a5e5ec0c622dc27ff9c4e6606eefef5cbbed4a5" + +[[package]] +name = "fastrlp" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "139834ddba373bbdd213dffe02c8d110508dcf1726c2be27e8d1f7d7e1856418" +dependencies = [ + "arrayvec", + "auto_impl", + "bytes", +] + +[[package]] +name = "fdlimit" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2c4c9e43643f5a3be4ca5b67d26b98031ff9db6806c3440ae32e02e3ceac3f1b" +dependencies = [ + "libc", +] + +[[package]] +name = "ff" +version = "0.13.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ded41244b729663b1e574f1b4fb731469f69f79c17667b5d776b16cda0479449" +dependencies = [ + "rand_core 0.6.4", + "subtle", +] + +[[package]] +name = "fiat-crypto" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d0870c84016d4b481be5c9f323c24f65e31e901ae618f0e80f4308fb00de1d2d" + +[[package]] +name = "fixed-hash" +version = "0.8.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "835c052cb0c08c1acf6ffd71c022172e18723949c8282f2b9f27efbc51e64534" +dependencies = [ + "byteorder", + "rand 0.8.5", + "rustc-hex", + "static_assertions", +] + +[[package]] +name = "flate2" +version = "1.0.28" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "46303f565772937ffe1d394a4fac6f411c6013172fadde9dcdb1e147a086940e" +dependencies = [ + "crc32fast", + "miniz_oxide", +] + +[[package]] +name = "fnv" +version = "1.0.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3f9eec918d3f24069decb9af1554cad7c880e2da24a9afd88aca000531ab82c1" + +[[package]] +name = "form_urlencoded" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a62bc1cf6f830c2ec14a513a9fb124d0a213a629668a4186f329db21fe045652" +dependencies = [ + "percent-encoding", +] + +[[package]] +name = "funty" +version = "2.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e6d5a32815ae3f33302d95fdcb2ce17862f8c65363dcfd29360480ba1001fc9c" + +[[package]] +name = "futures" +version = "0.3.29" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "da0290714b38af9b4a7b094b8a37086d1b4e61f2df9122c3cad2577669145335" +dependencies = [ + "futures-channel", + "futures-core", + "futures-executor", + "futures-io", + "futures-sink", + "futures-task", + "futures-util", +] + +[[package]] +name = "futures-channel" +version = "0.3.29" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ff4dd66668b557604244583e3e1e1eada8c5c2e96a6d0d6653ede395b78bbacb" +dependencies = [ + "futures-core", + "futures-sink", +] + +[[package]] +name = "futures-core" +version = "0.3.29" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "eb1d22c66e66d9d72e1758f0bd7d4fd0bee04cad842ee34587d68c07e45d088c" + +[[package]] +name = "futures-executor" +version = "0.3.29" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0f4fb8693db0cf099eadcca0efe2a5a22e4550f98ed16aba6c48700da29597bc" +dependencies = [ + "futures-core", + "futures-task", + "futures-util", +] + +[[package]] +name = "futures-io" +version = "0.3.29" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8bf34a163b5c4c52d0478a4d757da8fb65cabef42ba90515efee0f6f9fa45aaa" + +[[package]] +name = "futures-macro" +version = "0.3.29" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "53b153fd91e4b0147f4aced87be237c98248656bb01050b96bf3ee89220a8ddb" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.38", +] + +[[package]] +name = "futures-sink" +version = "0.3.29" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e36d3378ee38c2a36ad710c5d30c2911d752cb941c00c72dbabfb786a7970817" + +[[package]] +name = "futures-task" +version = "0.3.29" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "efd193069b0ddadc69c46389b740bbccdd97203899b48d09c5f7969591d6bae2" + +[[package]] +name = "futures-timer" +version = "3.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e64b03909df88034c26dc1547e8970b91f98bdb65165d6a4e9110d94263dbb2c" +dependencies = [ + "gloo-timers", + "send_wrapper", +] + +[[package]] +name = "futures-util" +version = "0.3.29" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a19526d624e703a3179b3d322efec918b6246ea0fa51d41124525f00f1cc8104" +dependencies = [ + "futures-channel", + "futures-core", + "futures-io", + "futures-macro", + "futures-sink", + "futures-task", + "memchr", + "pin-project-lite", + "pin-utils", + "slab", +] + +[[package]] +name = "generic-array" +version = "0.14.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "85649ca51fd72272d7821adaf274ad91c288277713d9c18820d8499a7ff69e9a" +dependencies = [ + "typenum", + "version_check", + "zeroize", +] + +[[package]] +name = "getrandom" +version = "0.1.16" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8fc3cb4d91f53b50155bdcfd23f6a4c39ae1969c2ae85982b135750cccaf5fce" +dependencies = [ + "cfg-if", + "libc", + "wasi 0.9.0+wasi-snapshot-preview1", +] + +[[package]] +name = "getrandom" +version = "0.2.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "be4136b2a15dd319360be1c07d9933517ccf0be8f16bf62a3bee4f0d618df427" +dependencies = [ + "cfg-if", + "libc", + "wasi 0.11.0+wasi-snapshot-preview1", +] + +[[package]] +name = "ghash" +version = "0.4.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1583cc1656d7839fd3732b80cf4f38850336cdb9b8ded1cd399ca62958de3c99" +dependencies = [ + "opaque-debug", + "polyval", +] + +[[package]] +name = "gimli" +version = "0.28.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6fb8d784f27acf97159b40fc4db5ecd8aa23b9ad5ef69cdd136d3bc80665f0c0" + +[[package]] +name = "glob" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d2fabcfbdc87f4758337ca535fb41a6d701b65693ce38287d856d1674551ec9b" + +[[package]] +name = "gloo-net" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8ac9e8288ae2c632fa9f8657ac70bfe38a1530f345282d7ba66a1f70b72b7dc4" +dependencies = [ + "futures-channel", + "futures-core", + "futures-sink", + "gloo-utils", + "http", + "js-sys", + "pin-project", + "serde", + "serde_json", + "thiserror", + "wasm-bindgen", + "wasm-bindgen-futures", + "web-sys", +] + +[[package]] +name = "gloo-timers" +version = "0.2.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9b995a66bb87bebce9a0f4a95aed01daca4872c050bfcb21653361c03bc35e5c" +dependencies = [ + "futures-channel", + "futures-core", + "js-sys", + "wasm-bindgen", +] + +[[package]] +name = "gloo-utils" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0b5555354113b18c547c1d3a98fbf7fb32a9ff4f6fa112ce823a21641a0ba3aa" +dependencies = [ + "js-sys", + "serde", + "serde_json", + "wasm-bindgen", + "web-sys", +] + +[[package]] +name = "group" +version = "0.13.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f0f9ef7462f7c099f518d754361858f86d8a07af53ba9af0fe635bbccb151a63" +dependencies = [ + "ff", + "rand_core 0.6.4", + "subtle", +] + +[[package]] +name = "h2" +version = "0.3.21" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "91fc23aa11be92976ef4729127f1a74adf36d8436f7816b185d18df956790833" +dependencies = [ + "bytes", + "fnv", + "futures-core", + "futures-sink", + "futures-util", + "http", + "indexmap 1.9.3", + "slab", + "tokio", + "tokio-util", + "tracing", +] + +[[package]] +name = "hash32" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b0c35f58762feb77d74ebe43bdbc3210f09be9fe6742234d573bacc26ed92b67" +dependencies = [ + "byteorder", +] + +[[package]] +name = "hashbrown" +version = "0.12.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8a9ee70c43aaf417c914396645a0fa852624801b24ebb7ae78fe8272889ac888" + +[[package]] +name = "hashbrown" +version = "0.13.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "43a3c133739dddd0d2990f9a4bdf8eb4b21ef50e4851ca85ab661199821d510e" +dependencies = [ + "ahash", +] + +[[package]] +name = "hashbrown" +version = "0.14.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f93e7192158dbcda357bdec5fb5788eebf8bbac027f3f33e719d29135ae84156" +dependencies = [ + "ahash", + "allocator-api2", + "serde", +] + +[[package]] +name = "hashlink" +version = "0.8.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e8094feaf31ff591f651a2664fb9cfd92bba7a60ce3197265e9482ebe753c8f7" +dependencies = [ + "hashbrown 0.14.2", +] + +[[package]] +name = "hdrhistogram" +version = "7.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7f19b9f54f7c7f55e31401bb647626ce0cf0f67b0004982ce815b3ee72a02aa8" +dependencies = [ + "byteorder", + "num-traits", +] + +[[package]] +name = "heapless" +version = "0.7.16" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "db04bc24a18b9ea980628ecf00e6c0264f3c1426dac36c00cb49b6fbad8b0743" +dependencies = [ + "atomic-polyfill 0.1.11", + "hash32", + "rustc_version 0.4.0", + "serde", + "spin 0.9.8", + "stable_deref_trait", +] + +[[package]] +name = "heck" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "95505c38b4572b2d910cecb0281560f54b440a19336cbbcb27bf6ce6adc6f5a8" + +[[package]] +name = "hermit-abi" +version = "0.3.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d77f7ec81a6d05a3abb01ab6eb7590f6083d08449fe5a1c8b1e620283546ccb7" + +[[package]] +name = "hex" +version = "0.4.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7f24254aa9a54b5c858eaee2f5bccdb46aaf0e486a595ed5fd8f86ba55232a70" +dependencies = [ + "serde", +] + +[[package]] +name = "hex-literal" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6fe2267d4ed49bc07b63801559be28c718ea06c4738b7a03c94df7386d2cde46" + +[[package]] +name = "hkdf" +version = "0.12.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "791a029f6b9fc27657f6f188ec6e5e43f6911f6f878e0dc5501396e09809d437" +dependencies = [ + "hmac", +] + +[[package]] +name = "hmac" +version = "0.12.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6c49c37c09c17a53d937dfbb742eb3a961d65a994e6bcdcf37e7399d0cc8ab5e" +dependencies = [ + "digest 0.10.7", +] + +[[package]] +name = "home" +version = "0.5.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5444c27eef6923071f7ebcc33e3444508466a76f7a2b93da00ed6e19f30c1ddb" +dependencies = [ + "windows-sys 0.48.0", +] + +[[package]] +name = "hostname" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3c731c3e10504cc8ed35cfe2f1db4c9274c3d35fa486e3b31df46f068ef3e867" +dependencies = [ + "libc", + "match_cfg", + "winapi", +] + +[[package]] +name = "http" +version = "0.2.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bd6effc99afb63425aff9b05836f029929e345a6148a14b7ecd5ab67af944482" +dependencies = [ + "bytes", + "fnv", + "itoa", +] + +[[package]] +name = "http-body" +version = "0.4.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d5f38f16d184e36f2408a55281cd658ecbd3ca05cce6d6510a176eca393e26d1" +dependencies = [ + "bytes", + "http", + "pin-project-lite", +] + +[[package]] +name = "http-range-header" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "add0ab9360ddbd88cfeb3bd9574a1d85cfdfa14db10b3e21d3700dbc4328758f" + +[[package]] +name = "httparse" +version = "1.8.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d897f394bad6a705d5f4104762e116a75639e470d80901eed05a860a95cb1904" + +[[package]] +name = "httpdate" +version = "1.0.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "df3b46402a9d5adb4c86a0cf463f42e19994e3ee891101b1841f30a545cb49a9" + +[[package]] +name = "human_bytes" +version = "0.4.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "91f255a4535024abf7640cb288260811fc14794f62b063652ed349f9a6c2348e" + +[[package]] +name = "humantime" +version = "2.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9a3a5bfb195931eeb336b2a7b4d761daec841b97f947d34394601737a7bba5e4" + +[[package]] +name = "humantime-serde" +version = "1.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "57a3db5ea5923d99402c94e9feb261dc5ee9b4efa158b0315f788cf549cc200c" +dependencies = [ + "humantime", + "serde", +] + +[[package]] +name = "hyper" +version = "0.14.27" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ffb1cfd654a8219eaef89881fdb3bb3b1cdc5fa75ded05d6933b2b382e395468" +dependencies = [ + "bytes", + "futures-channel", + "futures-core", + "futures-util", + "h2", + "http", + "http-body", + "httparse", + "httpdate", + "itoa", + "pin-project-lite", + "socket2 0.4.10", + "tokio", + "tower-service", + "tracing", + "want", +] + +[[package]] +name = "hyper-rustls" +version = "0.24.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ec3efd23720e2049821a693cbc7e65ea87c72f1c58ff2f9522ff332b1491e590" +dependencies = [ + "futures-util", + "http", + "hyper", + "log", + "rustls", + "rustls-native-certs", + "tokio", + "tokio-rustls", +] + +[[package]] +name = "hyper-system-resolver" +version = "0.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6eea26c5d0b6ab9d72219f65000af310f042a740926f7b2fa3553e774036e2e7" +dependencies = [ + "derive_builder", + "dns-lookup", + "hyper", + "tokio", + "tower-service", + "tracing", +] + +[[package]] +name = "iana-time-zone" +version = "0.1.58" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8326b86b6cff230b97d0d312a6c40a60726df3332e721f72a1b035f451663b20" +dependencies = [ + "android_system_properties", + "core-foundation-sys", + "iana-time-zone-haiku", + "js-sys", + "wasm-bindgen", + "windows-core", +] + +[[package]] +name = "iana-time-zone-haiku" +version = "0.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f31827a206f56af32e590ba56d5d2d085f558508192593743f16b2306495269f" +dependencies = [ + "cc", +] + +[[package]] +name = "icu_collections" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ef8302d8dfd6044d3ddb3f807a5ef3d7bbca9a574959c6d6e4dc39aa7012d0d5" +dependencies = [ + "displaydoc", + "serde", + "yoke", + "zerofrom", + "zerovec", +] + +[[package]] +name = "icu_locid" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3003f85dccfc0e238ff567693248c59153a46f4e6125ba4020b973cef4d1d335" +dependencies = [ + "displaydoc", + "litemap", + "serde", + "tinystr", + "writeable", + "zerovec", +] + +[[package]] +name = "icu_normalizer" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "652869735c9fb9f5a64ba180ee16f2c848390469c116deef517ecc53f4343598" +dependencies = [ + "displaydoc", + "icu_collections", + "icu_properties", + "icu_provider", + "serde", + "smallvec", + "utf16_iter", + "utf8_iter", + "write16", + "zerovec", +] + +[[package]] +name = "icu_properties" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ce0e1aa26851f16c9e04412a5911c86b7f8768dac8f8d4c5f1c568a7e5d7a434" +dependencies = [ + "displaydoc", + "icu_collections", + "icu_provider", + "serde", + "tinystr", + "zerovec", +] + +[[package]] +name = "icu_provider" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8dc312a7b6148f7dfe098047ae2494d12d4034f48ade58d4f353000db376e305" +dependencies = [ + "displaydoc", + "icu_locid", + "icu_provider_macros", + "postcard", + "serde", + "stable_deref_trait", + "writeable", + "yoke", + "zerofrom", + "zerovec", +] + +[[package]] +name = "icu_provider_adapters" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f4ae1e2bd0c41728b77e7c46e9afdec5e2127d1eedacc684724667d50c126bd3" +dependencies = [ + "icu_locid", + "icu_provider", + "serde", + "tinystr", + "yoke", + "zerovec", +] + +[[package]] +name = "icu_provider_blob" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fd364c9a01f791a4bc04a74cf2a1d01d9f6926a40fd5ae1c28004e1e70d8338b" +dependencies = [ + "icu_provider", + "postcard", + "serde", + "writeable", + "yoke", + "zerovec", +] + +[[package]] +name = "icu_provider_macros" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dd8b728b9421e93eff1d9f8681101b78fa745e0748c95c655c83f337044a7e10" +dependencies = [ + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "ident_case" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b9e0384b61958566e926dc50660321d12159025e767c18e043daf26b70104c39" + +[[package]] +name = "idna" +version = "0.2.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "418a0a6fab821475f634efe3ccc45c013f742efe03d853e8d3355d5cb850ecf8" +dependencies = [ + "matches", + "unicode-bidi", + "unicode-normalization", +] + +[[package]] +name = "idna" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7d20d6b07bfbc108882d88ed8e37d39636dcc260e15e30c45e6ba089610b917c" +dependencies = [ + "unicode-bidi", + "unicode-normalization", +] + +[[package]] +name = "igd" +version = "0.12.0" +source = "git+https://github.com/stevefan1999-personal/rust-igd?rev=c2d1f83eb1612a462962453cb0703bc93258b173#c2d1f83eb1612a462962453cb0703bc93258b173" +dependencies = [ + "attohttpc", + "bytes", + "futures", + "http", + "hyper", + "log", + "rand 0.8.5", + "tokio", + "url", + "xmltree", +] + +[[package]] +name = "impl-codec" +version = "0.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ba6a270039626615617f3f36d15fc827041df3b78c439da2cadfa47455a77f2f" +dependencies = [ + "parity-scale-codec", +] + +[[package]] +name = "impl-trait-for-tuples" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "11d7a9f6330b71fea57921c9b61c47ee6e84f72d394754eff6163ae67e7395eb" +dependencies = [ + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "include_dir" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "18762faeff7122e89e0857b02f7ce6fcc0d101d5e9ad2ad7846cc01d61b7f19e" +dependencies = [ + "include_dir_macros", +] + +[[package]] +name = "include_dir_macros" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b139284b5cf57ecfa712bcc66950bb635b31aff41c188e8a4cfc758eca374a3f" +dependencies = [ + "proc-macro2", + "quote", +] + +[[package]] +name = "indenter" +version = "0.3.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ce23b50ad8242c51a442f3ff322d56b02f08852c77e4c0b4d3fd684abc89c683" + +[[package]] +name = "indexmap" +version = "1.9.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bd070e393353796e801d209ad339e89596eb4c8d430d18ede6a1cced8fafbd99" +dependencies = [ + "autocfg", + "hashbrown 0.12.3", + "serde", +] + +[[package]] +name = "indexmap" +version = "2.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8adf3ddd720272c6ea8bf59463c04e0f93d0bbf7c5439b691bca2987e0270897" +dependencies = [ + "equivalent", + "hashbrown 0.14.2", + "serde", +] + +[[package]] +name = "inout" +version = "0.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a0c10553d664a4d0bcff9f4215d0aac67a639cc68ef660840afe309b807bc9f5" +dependencies = [ + "block-padding", + "generic-array", +] + +[[package]] +name = "instant" +version = "0.1.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7a5bbe824c507c5da5956355e86a746d82e0e1464f65d862cc5e71da70e94b2c" +dependencies = [ + "cfg-if", +] + +[[package]] +name = "io-lifetimes" +version = "1.0.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "eae7b9aee968036d54dce06cebaefd919e4472e753296daccd6d344e3e2df0c2" +dependencies = [ + "hermit-abi", + "libc", + "windows-sys 0.48.0", +] + +[[package]] +name = "ipconfig" +version = "0.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b58db92f96b720de98181bbbe63c831e87005ab460c1bf306eb2622b4707997f" +dependencies = [ + "socket2 0.5.5", + "widestring", + "windows-sys 0.48.0", + "winreg", +] + +[[package]] +name = "ipnet" +version = "2.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8f518f335dce6725a761382244631d86cf0ccb2863413590b31338feb467f9c3" + +[[package]] +name = "iri-string" +version = "0.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "21859b667d66a4c1dacd9df0863b3efb65785474255face87f5bca39dd8407c0" +dependencies = [ + "memchr", + "serde", +] + +[[package]] +name = "itertools" +version = "0.10.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b0fd2260e829bddf4cb6ea802289de2f86d6a7a690192fbe91b3f46e0f2c8473" +dependencies = [ + "either", +] + +[[package]] +name = "itertools" +version = "0.11.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b1c173a5686ce8bfa551b3563d0c2170bf24ca44da99c7ca4bfdab5418c3fe57" +dependencies = [ + "either", +] + +[[package]] +name = "itoa" +version = "1.0.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "af150ab688ff2122fcef229be89cb50dd66af9e01a4ff320cc137eecc9bacc38" + +[[package]] +name = "jemalloc-ctl" +version = "0.5.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7cffc705424a344c054e135d12ee591402f4539245e8bbd64e6c9eaa9458b63c" +dependencies = [ + "jemalloc-sys", + "libc", + "paste", +] + +[[package]] +name = "jemalloc-sys" +version = "0.5.4+5.3.0-patched" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ac6c1946e1cea1788cbfde01c993b52a10e2da07f4bac608228d1bed20bfebf2" +dependencies = [ + "cc", + "libc", +] + +[[package]] +name = "jemallocator" +version = "0.5.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a0de374a9f8e63150e6f5e8a60cc14c668226d7a347d8aee1a45766e3c4dd3bc" +dependencies = [ + "jemalloc-sys", + "libc", +] + +[[package]] +name = "jobserver" +version = "0.1.27" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8c37f63953c4c63420ed5fd3d6d398c719489b9f872b9fa683262f8edd363c7d" +dependencies = [ + "libc", +] + +[[package]] +name = "js-sys" +version = "0.3.64" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c5f195fe497f702db0f318b07fdd68edb16955aed830df8363d837542f8f935a" +dependencies = [ + "wasm-bindgen", +] + +[[package]] +name = "jsonrpsee" +version = "0.20.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "affdc52f7596ccb2d7645231fc6163bb314630c989b64998f3699a28b4d5d4dc" +dependencies = [ + "jsonrpsee-client-transport", + "jsonrpsee-core", + "jsonrpsee-http-client", + "jsonrpsee-proc-macros", + "jsonrpsee-server", + "jsonrpsee-types", + "jsonrpsee-wasm-client", + "jsonrpsee-ws-client", + "tokio", + "tracing", +] + +[[package]] +name = "jsonrpsee-client-transport" +version = "0.20.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b5b005c793122d03217da09af68ba9383363caa950b90d3436106df8cabce935" +dependencies = [ + "futures-channel", + "futures-util", + "gloo-net", + "http", + "jsonrpsee-core", + "pin-project", + "rustls-native-certs", + "soketto", + "thiserror", + "tokio", + "tokio-rustls", + "tokio-util", + "tracing", + "url", + "webpki-roots", +] + +[[package]] +name = "jsonrpsee-core" +version = "0.20.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "da2327ba8df2fdbd5e897e2b5ed25ce7f299d345b9736b6828814c3dbd1fd47b" +dependencies = [ + "anyhow", + "async-lock", + "async-trait", + "beef", + "futures-timer", + "futures-util", + "hyper", + "jsonrpsee-types", + "parking_lot 0.12.1", + "rand 0.8.5", + "rustc-hash", + "serde", + "serde_json", + "soketto", + "thiserror", + "tokio", + "tracing", + "wasm-bindgen-futures", +] + +[[package]] +name = "jsonrpsee-http-client" +version = "0.20.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5f80c17f62c7653ce767e3d7288b793dfec920f97067ceb189ebdd3570f2bc20" +dependencies = [ + "async-trait", + "hyper", + "hyper-rustls", + "jsonrpsee-core", + "jsonrpsee-types", + "serde", + "serde_json", + "thiserror", + "tokio", + "tower", + "tracing", + "url", +] + +[[package]] +name = "jsonrpsee-proc-macros" +version = "0.20.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "29110019693a4fa2dbda04876499d098fa16d70eba06b1e6e2b3f1b251419515" +dependencies = [ + "heck", + "proc-macro-crate", + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "jsonrpsee-server" +version = "0.20.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "82c39a00449c9ef3f50b84fc00fc4acba20ef8f559f07902244abf4c15c5ab9c" +dependencies = [ + "futures-util", + "http", + "hyper", + "jsonrpsee-core", + "jsonrpsee-types", + "route-recognizer", + "serde", + "serde_json", + "soketto", + "thiserror", + "tokio", + "tokio-stream", + "tokio-util", + "tower", + "tracing", +] + +[[package]] +name = "jsonrpsee-types" +version = "0.20.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5be0be325642e850ed0bdff426674d2e66b2b7117c9be23a7caef68a2902b7d9" +dependencies = [ + "anyhow", + "beef", + "serde", + "serde_json", + "thiserror", + "tracing", +] + +[[package]] +name = "jsonrpsee-wasm-client" +version = "0.20.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7c7cbb3447cf14fd4d2f407c3cc96e6c9634d5440aa1fbed868a31f3c02b27f0" +dependencies = [ + "jsonrpsee-client-transport", + "jsonrpsee-core", + "jsonrpsee-types", +] + +[[package]] +name = "jsonrpsee-ws-client" +version = "0.20.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bca9cb3933ccae417eb6b08c3448eb1cb46e39834e5b503e395e5e5bd08546c0" +dependencies = [ + "http", + "jsonrpsee-client-transport", + "jsonrpsee-core", + "jsonrpsee-types", + "url", +] + +[[package]] +name = "jsonwebtoken" +version = "8.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6971da4d9c3aa03c3d8f3ff0f4155b534aad021292003895a469716b2a230378" +dependencies = [ + "base64 0.21.5", + "pem", + "ring 0.16.20", + "serde", + "serde_json", + "simple_asn1", +] + +[[package]] +name = "k256" +version = "0.13.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cadb76004ed8e97623117f3df85b17aaa6626ab0b0831e6573f104df16cd1bcc" +dependencies = [ + "cfg-if", + "ecdsa", + "elliptic-curve", + "once_cell", + "sha2", + "signature", +] + +[[package]] +name = "keccak" +version = "0.1.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8f6d5ed8676d904364de097082f4e7d240b571b67989ced0240f08b7f966f940" +dependencies = [ + "cpufeatures", +] + +[[package]] +name = "lazy_static" +version = "1.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" +dependencies = [ + "spin 0.5.2", +] + +[[package]] +name = "lazycell" +version = "1.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "830d08ce1d1d941e6b30645f1a0eb5643013d835ce3779a5fc208261dbe10f55" + +[[package]] +name = "libc" +version = "0.2.149" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a08173bc88b7955d1b3145aa561539096c421ac8debde8cbc3612ec635fee29b" + +[[package]] +name = "libloading" +version = "0.7.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b67380fd3b2fbe7527a606e18729d21c6f3951633d0500574c4dc22d2d638b9f" +dependencies = [ + "cfg-if", + "winapi", +] + +[[package]] +name = "libm" +version = "0.2.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4ec2a862134d2a7d32d7983ddcdd1c4923530833c9f2ea1a44fc5fa473989058" + +[[package]] +name = "libproc" +version = "0.13.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8b18cbf29f8ff3542ba22bdce9ac610fcb75d74bb4e2b306b2a2762242025b4f" +dependencies = [ + "bindgen 0.64.0", + "errno 0.2.8", + "libc", +] + +[[package]] +name = "linked-hash-map" +version = "0.5.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0717cef1bc8b636c6e1c1bbdefc09e6322da8a9321966e8928ef80d20f7f770f" + +[[package]] +name = "linked_hash_set" +version = "0.1.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "47186c6da4d81ca383c7c47c1bfc80f4b95f4720514d860a5407aaf4233f9588" +dependencies = [ + "linked-hash-map", +] + +[[package]] +name = "linux-raw-sys" +version = "0.1.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f051f77a7c8e6957c0696eac88f26b0117e54f52d3fc682ab19397a8812846a4" + +[[package]] +name = "linux-raw-sys" +version = "0.4.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "da2479e8c062e40bf0066ffa0bc823de0a9368974af99c9f6df941d2c231e03f" + +[[package]] +name = "litemap" +version = "0.7.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "77a1a2647d5b7134127971a6de0d533c49de2159167e7f259c427195f87168a1" + +[[package]] +name = "lock_api" +version = "0.4.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3c168f8615b12bc01f9c17e2eb0cc07dcae1940121185446edc3744920e8ef45" +dependencies = [ + "autocfg", + "scopeguard", +] + +[[package]] +name = "log" +version = "0.4.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b5e6163cb8c49088c2c36f57875e58ccd8c87c7427f7fbd50ea6710b2f3f2e8f" + +[[package]] +name = "lru" +version = "0.11.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a4a83fb7698b3643a0e34f9ae6f2e8f0178c0fd42f8b59d493aa271ff3a5bf21" +dependencies = [ + "hashbrown 0.14.2", +] + +[[package]] +name = "lru" +version = "0.12.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1efa59af2ddfad1854ae27d75009d538d0998b4b2fd47083e743ac1a10e46c60" +dependencies = [ + "hashbrown 0.14.2", +] + +[[package]] +name = "lru-cache" +version = "0.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "31e24f1ad8321ca0e8a1e0ac13f23cb668e6f5466c2c57319f6a5cf1cc8e3b1c" +dependencies = [ + "linked-hash-map", +] + +[[package]] +name = "lz4_flex" +version = "0.11.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3ea9b256699eda7b0387ffbc776dd625e28bde3918446381781245b7a50349d8" + +[[package]] +name = "mach2" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6d0d1830bcd151a6fc4aea1369af235b36c1528fe976b8ff678683c9995eade8" +dependencies = [ + "libc", +] + +[[package]] +name = "match_cfg" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ffbee8634e0d45d258acb448e7eaab3fce7a0a467395d4d9f228e3c1f01fb2e4" + +[[package]] +name = "matchers" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8263075bb86c5a1b1427b5ae862e8889656f126e9f77c484496e8b47cf5c5558" +dependencies = [ + "regex-automata 0.1.10", +] + +[[package]] +name = "matches" +version = "0.1.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2532096657941c2fea9c289d370a250971c689d4f143798ff67113ec042024a5" + +[[package]] +name = "memchr" +version = "2.6.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f665ee40bc4a3c5590afb1e9677db74a508659dfd71e126420da8274909a0167" + +[[package]] +name = "memmap2" +version = "0.7.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f49388d20533534cd19360ad3d6a7dadc885944aa802ba3995040c5ec11288c6" +dependencies = [ + "libc", +] + +[[package]] +name = "memoffset" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5a634b1c61a95585bd15607c6ab0c4e5b226e695ff2800ba0cdccddf208c406c" +dependencies = [ + "autocfg", +] + +[[package]] +name = "metrics" +version = "0.21.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fde3af1a009ed76a778cb84fdef9e7dbbdf5775ae3e4cc1f434a6a307f6f76c5" +dependencies = [ + "ahash", + "metrics-macros", + "portable-atomic", +] + +[[package]] +name = "metrics-exporter-prometheus" +version = "0.12.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8a4964177ddfdab1e3a2b37aec7cf320e14169abb0ed73999f558136409178d5" +dependencies = [ + "base64 0.21.5", + "hyper", + "indexmap 1.9.3", + "ipnet", + "metrics", + "metrics-util", + "quanta", + "thiserror", + "tokio", + "tracing", +] + +[[package]] +name = "metrics-macros" +version = "0.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ddece26afd34c31585c74a4db0630c376df271c285d682d1e55012197830b6df" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.38", +] + +[[package]] +name = "metrics-process" +version = "1.0.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1c93f6ad342d3f7bc14724147e2dbc6eb6fdbe5a832ace16ea23b73618e8cc17" +dependencies = [ + "libproc", + "mach2", + "metrics", + "once_cell", + "procfs", + "rlimit", + "windows", +] + +[[package]] +name = "metrics-util" +version = "0.15.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "111cb375987443c3de8d503580b536f77dc8416d32db62d9456db5d93bd7ac47" +dependencies = [ + "aho-corasick 0.7.20", + "crossbeam-epoch", + "crossbeam-utils", + "hashbrown 0.13.2", + "indexmap 1.9.3", + "metrics", + "num_cpus", + "ordered-float", + "quanta", + "radix_trie", + "sketches-ddsketch", +] + +[[package]] +name = "mime" +version = "0.3.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6877bb514081ee2a7ff5ef9de3281f14a4dd4bceac4c09388074a6b5df8a139a" + +[[package]] +name = "mime_guess" +version = "2.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4192263c238a5f0d0c6bfd21f336a313a4ce1c450542449ca191bb657b4642ef" +dependencies = [ + "mime", + "unicase", +] + +[[package]] +name = "minimal-lexical" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "68354c5c6bd36d73ff3feceb05efa59b6acb7626617f4962be322a825e61f79a" + +[[package]] +name = "miniz_oxide" +version = "0.7.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e7810e0be55b428ada41041c41f32c9f1a42817901b4ccf45fa3d4b6561e74c7" +dependencies = [ + "adler", +] + +[[package]] +name = "mio" +version = "0.8.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3dce281c5e46beae905d4de1870d8b1509a9142b62eedf18b443b011ca8343d0" +dependencies = [ + "libc", + "log", + "wasi 0.11.0+wasi-snapshot-preview1", + "windows-sys 0.48.0", +] + +[[package]] +name = "modular-bitfield" +version = "0.11.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a53d79ba8304ac1c4f9eb3b9d281f21f7be9d4626f72ce7df4ad8fbde4f38a74" +dependencies = [ + "modular-bitfield-impl", + "static_assertions", +] + +[[package]] +name = "modular-bitfield-impl" +version = "0.11.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5a7d5f7076603ebc68de2dc6a650ec331a062a13abaa346975be747bbfa4b789" +dependencies = [ + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "more-asserts" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1fafa6961cabd9c63bcd77a45d7e3b7f3b552b70417831fb0f56db717e72407e" + +[[package]] +name = "nibble_vec" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "77a5d83df9f36fe23f0c3648c6bbb8b0298bb5f1939c8f2704431371f4b84d43" +dependencies = [ + "smallvec", +] + +[[package]] +name = "nom" +version = "7.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d273983c5a657a70a3e8f2a01329822f3b8c8172b73826411a55751e404a0a4a" +dependencies = [ + "memchr", + "minimal-lexical", +] + +[[package]] +name = "nu-ansi-term" +version = "0.46.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "77a8165726e8236064dbb45459242600304b42a5ea24ee2948e18e023bf7ba84" +dependencies = [ + "overload", + "winapi", +] + +[[package]] +name = "num" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b05180d69e3da0e530ba2a1dae5110317e49e3b7f3d41be227dc5f92e49ee7af" +dependencies = [ + "num-bigint", + "num-complex", + "num-integer", + "num-iter", + "num-rational", + "num-traits", +] + +[[package]] +name = "num-bigint" +version = "0.4.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "608e7659b5c3d7cba262d894801b9ec9d00de989e8a82bd4bef91d08da45cdc0" +dependencies = [ + "autocfg", + "num-integer", + "num-traits", + "serde", +] + +[[package]] +name = "num-complex" +version = "0.4.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1ba157ca0885411de85d6ca030ba7e2a83a28636056c7c699b07c8b6f7383214" +dependencies = [ + "num-traits", +] + +[[package]] +name = "num-integer" +version = "0.1.45" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "225d3389fb3509a24c93f5c29eb6bde2586b98d9f016636dff58d7c6f7569cd9" +dependencies = [ + "autocfg", + "num-traits", +] + +[[package]] +name = "num-iter" +version = "0.1.43" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7d03e6c028c5dc5cac6e2dec0efda81fc887605bb3d884578bb6d6bf7514e252" +dependencies = [ + "autocfg", + "num-integer", + "num-traits", +] + +[[package]] +name = "num-rational" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0638a1c9d0a3c0914158145bc76cff373a75a627e6ecbfb71cbe6f453a5a19b0" +dependencies = [ + "autocfg", + "num-bigint", + "num-integer", + "num-traits", +] + +[[package]] +name = "num-traits" +version = "0.2.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "39e3200413f237f41ab11ad6d161bc7239c84dcb631773ccd7de3dfe4b5c267c" +dependencies = [ + "autocfg", + "libm", +] + +[[package]] +name = "num_cpus" +version = "1.16.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4161fcb6d602d4d2081af7c3a45852d875a03dd337a6bfdd6e06407b61342a43" +dependencies = [ + "hermit-abi", + "libc", +] + +[[package]] +name = "num_enum" +version = "0.6.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7a015b430d3c108a207fd776d2e2196aaf8b1cf8cf93253e3a097ff3085076a1" +dependencies = [ + "num_enum_derive 0.6.1", +] + +[[package]] +name = "num_enum" +version = "0.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "70bf6736f74634d299d00086f02986875b3c2d924781a6a2cb6c201e73da0ceb" +dependencies = [ + "num_enum_derive 0.7.0", +] + +[[package]] +name = "num_enum_derive" +version = "0.6.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "96667db765a921f7b295ffee8b60472b686a51d4f21c2ee4ffdb94c7013b65a6" +dependencies = [ + "proc-macro-crate", + "proc-macro2", + "quote", + "syn 2.0.38", +] + +[[package]] +name = "num_enum_derive" +version = "0.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "56ea360eafe1022f7cc56cd7b869ed57330fb2453d0c7831d99b74c65d2f5597" +dependencies = [ + "proc-macro-crate", + "proc-macro2", + "quote", + "syn 2.0.38", +] + +[[package]] +name = "num_threads" +version = "0.1.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2819ce041d2ee131036f4fc9d6ae7ae125a3a40e97ba64d04fe799ad9dabbb44" +dependencies = [ + "libc", +] + +[[package]] +name = "object" +version = "0.32.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9cf5f9dd3933bd50a9e1f149ec995f39ae2c496d31fd772c1fd45ebc27e902b0" +dependencies = [ + "memchr", +] + +[[package]] +name = "once_cell" +version = "1.18.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dd8b5dd2ae5ed71462c540258bedcb51965123ad7e7ccf4b9a8cafaa4a63576d" +dependencies = [ + "atomic-polyfill 1.0.3", + "critical-section", +] + +[[package]] +name = "opaque-debug" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "624a8340c38c1b80fd549087862da4ba43e08858af025b236e509b6649fc13d5" + +[[package]] +name = "openssl-probe" +version = "0.1.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ff011a302c396a5197692431fc1948019154afc178baf7d8e37367442a4601cf" + +[[package]] +name = "option-ext" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "04744f49eae99ab78e0d5c0b603ab218f515ea8cfe5a456d7629ad883a3b6e7d" + +[[package]] +name = "ordered-float" +version = "3.9.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f1e1c390732d15f1d48471625cd92d154e66db2c56645e29a9cd26f4699f72dc" +dependencies = [ + "num-traits", +] + +[[package]] +name = "overload" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b15813163c1d831bf4a13c3610c05c0d03b39feb07f7e09fa234dac9b15aaf39" + +[[package]] +name = "page_size" +version = "0.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "30d5b2194ed13191c1999ae0704b7839fb18384fa22e49b57eeaa97d79ce40da" +dependencies = [ + "libc", + "winapi", +] + +[[package]] +name = "parity-scale-codec" +version = "3.6.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0dec8a8073036902368c2cdc0387e85ff9a37054d7e7c98e592145e0c92cd4fb" +dependencies = [ + "arrayvec", + "bitvec", + "byte-slice-cast", + "bytes", + "impl-trait-for-tuples", + "parity-scale-codec-derive", + "serde", +] + +[[package]] +name = "parity-scale-codec-derive" +version = "3.6.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "312270ee71e1cd70289dacf597cab7b207aa107d2f28191c2ae45b2ece18a260" +dependencies = [ + "proc-macro-crate", + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "parity-tokio-ipc" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9981e32fb75e004cc148f5fb70342f393830e0a4aa62e3cc93b50976218d42b6" +dependencies = [ + "futures", + "libc", + "log", + "rand 0.7.3", + "tokio", + "winapi", +] + +[[package]] +name = "parking_lot" +version = "0.11.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7d17b78036a60663b797adeaee46f5c9dfebb86948d1255007a1d6be0271ff99" +dependencies = [ + "instant", + "lock_api", + "parking_lot_core 0.8.6", +] + +[[package]] +name = "parking_lot" +version = "0.12.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3742b2c103b9f06bc9fff0a37ff4912935851bee6d36f3c02bcc755bcfec228f" +dependencies = [ + "lock_api", + "parking_lot_core 0.9.9", +] + +[[package]] +name = "parking_lot_core" +version = "0.8.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "60a2cfe6f0ad2bfc16aefa463b497d5c7a5ecd44a23efa72aa342d90177356dc" +dependencies = [ + "cfg-if", + "instant", + "libc", + "redox_syscall 0.2.16", + "smallvec", + "winapi", +] + +[[package]] +name = "parking_lot_core" +version = "0.9.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4c42a9226546d68acdd9c0a280d17ce19bfe27a46bf68784e4066115788d008e" +dependencies = [ + "cfg-if", + "libc", + "redox_syscall 0.4.1", + "smallvec", + "windows-targets 0.48.5", +] + +[[package]] +name = "paste" +version = "1.0.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "de3145af08024dea9fa9914f381a17b8fc6034dfb00f3a84013f7ff43f29ed4c" + +[[package]] +name = "peeking_take_while" +version = "0.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "19b17cddbe7ec3f8bc800887bab5e717348c95ea2ca0b1bf0837fb964dc67099" + +[[package]] +name = "pem" +version = "1.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a8835c273a76a90455d7344889b0964598e3316e2a79ede8e36f16bdcf2228b8" +dependencies = [ + "base64 0.13.1", +] + +[[package]] +name = "percent-encoding" +version = "2.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9b2a4787296e9989611394c33f193f676704af1686e70b8f8033ab5ba9a35a94" + +[[package]] +name = "pest" +version = "2.7.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ae9cee2a55a544be8b89dc6848072af97a20f2422603c10865be2a42b580fff5" +dependencies = [ + "memchr", + "thiserror", + "ucd-trie", +] + +[[package]] +name = "ph" +version = "0.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "88c6e62e083483e2812a9d2a6eff6b97871302ef4166b5182c6da30624b7e991" +dependencies = [ + "binout", + "bitm", + "dyn_size_of", + "rayon", + "wyhash", +] + +[[package]] +name = "phf" +version = "0.11.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ade2d8b8f33c7333b51bcf0428d37e217e9f32192ae4772156f65063b8ce03dc" +dependencies = [ + "phf_macros", + "phf_shared", +] + +[[package]] +name = "phf_generator" +version = "0.11.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "48e4cc64c2ad9ebe670cb8fd69dd50ae301650392e81c05f9bfcb2d5bdbc24b0" +dependencies = [ + "phf_shared", + "rand 0.8.5", +] + +[[package]] +name = "phf_macros" +version = "0.11.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3444646e286606587e49f3bcf1679b8cef1dc2c5ecc29ddacaffc305180d464b" +dependencies = [ + "phf_generator", + "phf_shared", + "proc-macro2", + "quote", + "syn 2.0.38", +] + +[[package]] +name = "phf_shared" +version = "0.11.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "90fcb95eef784c2ac79119d1dd819e162b5da872ce6f3c3abe1e8ca1c082f72b" +dependencies = [ + "siphasher", +] + +[[package]] +name = "pin-project" +version = "1.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fda4ed1c6c173e3fc7a83629421152e01d7b1f9b7f65fb301e490e8cfc656422" +dependencies = [ + "pin-project-internal", +] + +[[package]] +name = "pin-project-internal" +version = "1.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4359fd9c9171ec6e8c62926d6faaf553a8dc3f64e1507e76da7911b4f6a04405" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.38", +] + +[[package]] +name = "pin-project-lite" +version = "0.2.13" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8afb450f006bf6385ca15ef45d71d2288452bc3683ce2e2cacc0d18e4be60b58" + +[[package]] +name = "pin-utils" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" + +[[package]] +name = "pkcs8" +version = "0.10.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f950b2377845cebe5cf8b5165cb3cc1a5e0fa5cfa3e1f7f55707d8fd82e0a7b7" +dependencies = [ + "der", + "spki", +] + +[[package]] +name = "pkg-config" +version = "0.3.27" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "26072860ba924cbfa98ea39c8c19b4dd6a4a25423dbdf219c1eca91aa0cf6964" + +[[package]] +name = "platforms" +version = "3.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4503fa043bf02cee09a9582e9554b4c6403b2ef55e4612e96561d294419429f8" + +[[package]] +name = "pollster" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "22686f4785f02a4fcc856d3b3bb19bf6c8160d103f7a99cc258bddd0251dc7f2" + +[[package]] +name = "polyval" +version = "0.5.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8419d2b623c7c0896ff2d5d96e2cb4ede590fed28fcc34934f4c33c036e620a1" +dependencies = [ + "cfg-if", + "cpufeatures", + "opaque-debug", + "universal-hash", +] + +[[package]] +name = "portable-atomic" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b559898e0b4931ed2d3b959ab0c2da4d99cc644c4b0b1a35b4d344027f474023" + +[[package]] +name = "postcard" +version = "1.0.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a55c51ee6c0db07e68448e336cf8ea4131a620edefebf9893e759b2d793420f8" +dependencies = [ + "cobs", + "embedded-io", + "heapless", + "serde", +] + +[[package]] +name = "powerfmt" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "439ee305def115ba05938db6eb1644ff94165c5ab5e9420d1c1bcedbba909391" + +[[package]] +name = "ppv-lite86" +version = "0.2.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5b40af805b3121feab8a3c29f04d8ad262fa8e0561883e7653e024ae4479e6de" + +[[package]] +name = "pretty_assertions" +version = "1.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "af7cee1a6c8a5b9208b3cb1061f10c0cb689087b3d8ce85fb9d2dd7a29b6ba66" +dependencies = [ + "diff", + "yansi", +] + +[[package]] +name = "prettyplease" +version = "0.2.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ae005bd773ab59b4725093fd7df83fd7892f7d8eafb48dbd7de6e024e4215f9d" +dependencies = [ + "proc-macro2", + "syn 2.0.38", +] + +[[package]] +name = "primitive-types" +version = "0.12.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0b34d9fd68ae0b74a41b21c03c2f62847aa0ffea044eee893b4c140b37e244e2" +dependencies = [ + "fixed-hash", + "impl-codec", + "uint", +] + +[[package]] +name = "proc-macro-crate" +version = "1.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7f4c021e1093a56626774e81216a4ce732a735e5bad4868a03f3ed65ca0c3919" +dependencies = [ + "once_cell", + "toml_edit 0.19.15", +] + +[[package]] +name = "proc-macro-error" +version = "1.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "da25490ff9892aab3fcf7c36f08cfb902dd3e71ca0f9f9517bea02a73a5ce38c" +dependencies = [ + "proc-macro-error-attr", + "proc-macro2", + "quote", + "syn 1.0.109", + "version_check", +] + +[[package]] +name = "proc-macro-error-attr" +version = "1.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a1be40180e52ecc98ad80b184934baf3d0d29f979574e439af5a55274b35f869" +dependencies = [ + "proc-macro2", + "quote", + "version_check", +] + +[[package]] +name = "proc-macro2" +version = "1.0.69" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "134c189feb4956b20f6f547d2cf727d4c0fe06722b20a0eec87ed445a97f92da" +dependencies = [ + "unicode-ident", +] + +[[package]] +name = "procfs" +version = "0.15.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "943ca7f9f29bab5844ecd8fdb3992c5969b6622bb9609b9502fef9b4310e3f1f" +dependencies = [ + "bitflags 1.3.2", + "byteorder", + "hex", + "lazy_static", + "rustix 0.36.16", +] + +[[package]] +name = "proptest" +version = "1.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7c003ac8c77cb07bb74f5f198bce836a689bcd5a42574612bf14d17bfd08c20e" +dependencies = [ + "bit-set", + "bit-vec", + "bitflags 2.4.1", + "lazy_static", + "num-traits", + "rand 0.8.5", + "rand_chacha 0.3.1", + "rand_xorshift", + "regex-syntax 0.7.5", + "rusty-fork", + "tempfile", + "unarray", +] + +[[package]] +name = "proptest-derive" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9cf16337405ca084e9c78985114633b6827711d22b9e6ef6c6c0d665eb3f0b6e" +dependencies = [ + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "public-ip" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7b4c40db5262d93298c363a299f8bc1b3a956a78eecddba3bc0e58b76e2f419a" +dependencies = [ + "dns-lookup", + "futures-core", + "futures-util", + "http", + "hyper", + "hyper-system-resolver", + "pin-project-lite", + "thiserror", + "tokio", + "tracing", + "tracing-futures", + "trust-dns-client", + "trust-dns-proto 0.20.4", +] + +[[package]] +name = "quanta" +version = "0.11.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a17e662a7a8291a865152364c20c7abc5e60486ab2001e8ec10b24862de0b9ab" +dependencies = [ + "crossbeam-utils", + "libc", + "mach2", + "once_cell", + "raw-cpuid", + "wasi 0.11.0+wasi-snapshot-preview1", + "web-sys", + "winapi", +] + +[[package]] +name = "quick-error" +version = "1.2.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a1d01941d82fa2ab50be1e79e6714289dd7cde78eba4c074bc5a4374f650dfe0" + +[[package]] +name = "quote" +version = "1.0.33" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5267fca4496028628a95160fc423a33e8b2e6af8a5302579e322e4b520293cae" +dependencies = [ + "proc-macro2", +] + +[[package]] +name = "radium" +version = "0.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dc33ff2d4973d518d823d61aa239014831e521c75da58e3df4840d3f47749d09" + +[[package]] +name = "radix_trie" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c069c179fcdc6a2fe24d8d18305cf085fdbd4f922c041943e203685d6a1c58fd" +dependencies = [ + "endian-type", + "nibble_vec", +] + +[[package]] +name = "rand" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6a6b1679d49b24bbfe0c803429aa1874472f50d9b363131f0e89fc356b544d03" +dependencies = [ + "getrandom 0.1.16", + "libc", + "rand_chacha 0.2.2", + "rand_core 0.5.1", + "rand_hc", +] + +[[package]] +name = "rand" +version = "0.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "34af8d1a0e25924bc5b7c43c079c942339d8f0a8b57c39049bef581b46327404" +dependencies = [ + "libc", + "rand_chacha 0.3.1", + "rand_core 0.6.4", +] + +[[package]] +name = "rand_chacha" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f4c8ed856279c9737206bf725bf36935d8666ead7aa69b52be55af369d193402" +dependencies = [ + "ppv-lite86", + "rand_core 0.5.1", +] + +[[package]] +name = "rand_chacha" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e6c10a63a0fa32252be49d21e7709d4d4baf8d231c2dbce1eaa8141b9b127d88" +dependencies = [ + "ppv-lite86", + "rand_core 0.6.4", +] + +[[package]] +name = "rand_core" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "90bde5296fc891b0cef12a6d03ddccc162ce7b2aff54160af9338f8d40df6d19" +dependencies = [ + "getrandom 0.1.16", +] + +[[package]] +name = "rand_core" +version = "0.6.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ec0be4795e2f6a28069bec0b5ff3e2ac9bafc99e6a9a7dc3547996c5c816922c" +dependencies = [ + "getrandom 0.2.10", +] + +[[package]] +name = "rand_hc" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ca3129af7b92a17112d59ad498c6f81eaf463253766b90396d39ea7a39d6613c" +dependencies = [ + "rand_core 0.5.1", +] + +[[package]] +name = "rand_xorshift" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d25bf25ec5ae4a3f1b92f929810509a2f53d7dca2f50b794ff57e3face536c8f" +dependencies = [ + "rand_core 0.6.4", +] + +[[package]] +name = "raw-cpuid" +version = "10.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6c297679cb867470fa8c9f67dbba74a78d78e3e98d7cf2b08d6d71540f797332" +dependencies = [ + "bitflags 1.3.2", +] + +[[package]] +name = "rayon" +version = "1.8.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9c27db03db7734835b3f53954b534c91069375ce6ccaa2e065441e07d9b6cdb1" +dependencies = [ + "either", + "rayon-core", +] + +[[package]] +name = "rayon-core" +version = "1.12.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5ce3fb6ad83f861aac485e76e1985cd109d9a3713802152be56c3b1f0e0658ed" +dependencies = [ + "crossbeam-deque", + "crossbeam-utils", +] + +[[package]] +name = "redox_syscall" +version = "0.2.16" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fb5a58c1855b4b6819d59012155603f0b22ad30cad752600aadfcb695265519a" +dependencies = [ + "bitflags 1.3.2", +] + +[[package]] +name = "redox_syscall" +version = "0.3.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "567664f262709473930a4bf9e51bf2ebf3348f2e748ccc50dea20646858f8f29" +dependencies = [ + "bitflags 1.3.2", +] + +[[package]] +name = "redox_syscall" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4722d768eff46b75989dd134e5c353f0d6296e5aaa3132e776cbdb56be7731aa" +dependencies = [ + "bitflags 1.3.2", +] + +[[package]] +name = "redox_users" +version = "0.4.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b033d837a7cf162d7993aded9304e30a83213c648b6e389db233191f891e5c2b" +dependencies = [ + "getrandom 0.2.10", + "redox_syscall 0.2.16", + "thiserror", +] + +[[package]] +name = "regex" +version = "1.10.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "380b951a9c5e80ddfd6136919eef32310721aa4aacd4889a8d39124b026ab343" +dependencies = [ + "aho-corasick 1.1.2", + "memchr", + "regex-automata 0.4.3", + "regex-syntax 0.8.2", +] + +[[package]] +name = "regex-automata" +version = "0.1.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6c230d73fb8d8c1b9c0b3135c5142a8acee3a0558fb8db5cf1cb65f8d7862132" +dependencies = [ + "regex-syntax 0.6.29", +] + +[[package]] +name = "regex-automata" +version = "0.4.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5f804c7828047e88b2d32e2d7fe5a105da8ee3264f01902f796c8e067dc2483f" +dependencies = [ + "aho-corasick 1.1.2", + "memchr", + "regex-syntax 0.8.2", +] + +[[package]] +name = "regex-syntax" +version = "0.6.29" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f162c6dd7b008981e4d40210aca20b4bd0f9b60ca9271061b07f78537722f2e1" + +[[package]] +name = "regex-syntax" +version = "0.7.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dbb5fb1acd8a1a18b3dd5be62d25485eb770e05afb408a9627d14d451bae12da" + +[[package]] +name = "regex-syntax" +version = "0.8.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c08c74e62047bb2de4ff487b251e4a92e24f48745648451635cec7d591162d9f" + +[[package]] +name = "regress" +version = "0.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "82a9ecfa0cb04d0b04dddb99b8ccf4f66bc8dfd23df694b398570bd8ae3a50fb" +dependencies = [ + "hashbrown 0.13.2", + "memchr", +] + +[[package]] +name = "resolv-conf" +version = "0.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "52e44394d2086d010551b14b53b1f24e31647570cd1deb0379e2c21b329aba00" +dependencies = [ + "hostname", + "quick-error", +] + +[[package]] +name = "reth" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "alloy-rlp", + "aquamarine", + "backon", + "boyer-moore-magiclen", + "clap", + "comfy-table", + "confy", + "const-str", + "crossterm 0.27.0", + "dirs-next", + "eyre", + "fdlimit", + "futures", + "human_bytes", + "humantime", + "hyper", + "itertools 0.11.0", + "jemalloc-ctl", + "jemallocator", + "metrics", + "metrics-exporter-prometheus", + "metrics-process", + "metrics-util", + "pin-project", + "pretty_assertions", + "proptest", + "rand 0.8.5", + "reth-auto-seal-consensus", + "reth-basic-payload-builder", + "reth-beacon-consensus", + "reth-blockchain-tree", + "reth-config", + "reth-consensus-common", + "reth-db", + "reth-discv4", + "reth-downloaders", + "reth-interfaces", + "reth-metrics", + "reth-net-nat", + "reth-network", + "reth-network-api", + "reth-nippy-jar", + "reth-payload-builder", + "reth-primitives", + "reth-provider", + "reth-prune", + "reth-revm", + "reth-revm-inspectors", + "reth-rpc", + "reth-rpc-api", + "reth-rpc-builder", + "reth-rpc-engine-api", + "reth-rpc-types", + "reth-rpc-types-compat", + "reth-snapshot", + "reth-stages", + "reth-tasks", + "reth-tracing", + "reth-transaction-pool", + "reth-trie", + "secp256k1 0.27.0", + "serde", + "serde_json", + "shellexpand", + "tempfile", + "thiserror", + "tokio", + "toml 0.8.5", + "tracing", + "tui", + "vergen", +] + +[[package]] +name = "reth-auto-seal-consensus" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "futures-util", + "reth-beacon-consensus", + "reth-interfaces", + "reth-primitives", + "reth-provider", + "reth-revm", + "reth-stages", + "reth-transaction-pool", + "tokio", + "tokio-stream", + "tracing", +] + +[[package]] +name = "reth-basic-payload-builder" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "alloy-rlp", + "futures-core", + "futures-util", + "metrics", + "reth-interfaces", + "reth-metrics", + "reth-payload-builder", + "reth-primitives", + "reth-provider", + "reth-revm", + "reth-tasks", + "reth-transaction-pool", + "revm", + "tokio", + "tracing", +] + +[[package]] +name = "reth-beacon-consensus" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "futures", + "metrics", + "reth-consensus-common", + "reth-db", + "reth-interfaces", + "reth-metrics", + "reth-payload-builder", + "reth-primitives", + "reth-provider", + "reth-prune", + "reth-rpc-types", + "reth-rpc-types-compat", + "reth-snapshot", + "reth-stages", + "reth-tasks", + "reth-tokio-util", + "schnellru", + "thiserror", + "tokio", + "tokio-stream", + "tracing", +] + +[[package]] +name = "reth-blockchain-tree" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "aquamarine", + "linked_hash_set", + "lru 0.11.1", + "metrics", + "parking_lot 0.12.1", + "reth-db", + "reth-interfaces", + "reth-metrics", + "reth-primitives", + "reth-provider", + "reth-stages", + "tokio", + "tracing", +] + +[[package]] +name = "reth-codecs" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "arbitrary", + "bytes", + "codecs-derive", + "proptest", + "proptest-derive", + "revm-primitives", +] + +[[package]] +name = "reth-config" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "confy", + "reth-discv4", + "reth-downloaders", + "reth-net-nat", + "reth-network", + "reth-primitives", + "reth-stages", + "secp256k1 0.27.0", + "serde", + "serde_json", + "tempfile", +] + +[[package]] +name = "reth-consensus-common" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "reth-interfaces", + "reth-primitives", + "reth-provider", +] + +[[package]] +name = "reth-db" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "arbitrary", + "bytes", + "derive_more", + "eyre", + "futures", + "heapless", + "itertools 0.11.0", + "metrics", + "modular-bitfield", + "page_size", + "parity-scale-codec", + "parking_lot 0.12.1", + "paste", + "postcard", + "proptest", + "proptest-derive", + "rand 0.8.5", + "rayon", + "reth-codecs", + "reth-interfaces", + "reth-libmdbx", + "reth-metrics", + "reth-nippy-jar", + "reth-primitives", + "reth-tracing", + "serde", + "tempfile", + "thiserror", + "tokio-stream", + "vergen", +] + +[[package]] +name = "reth-discv4" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "alloy-rlp", + "discv5", + "enr", + "generic-array", + "parking_lot 0.12.1", + "reth-net-common", + "reth-net-nat", + "reth-primitives", + "rlp", + "secp256k1 0.27.0", + "serde", + "thiserror", + "tokio", + "tokio-stream", + "tracing", +] + +[[package]] +name = "reth-dns-discovery" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "alloy-rlp", + "async-trait", + "data-encoding", + "enr", + "linked_hash_set", + "parking_lot 0.12.1", + "reth-net-common", + "reth-primitives", + "schnellru", + "secp256k1 0.27.0", + "serde", + "serde_with", + "thiserror", + "tokio", + "tokio-stream", + "tracing", + "trust-dns-resolver", +] + +[[package]] +name = "reth-downloaders" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "alloy-rlp", + "futures", + "futures-util", + "itertools 0.11.0", + "metrics", + "pin-project", + "rayon", + "reth-db", + "reth-interfaces", + "reth-metrics", + "reth-primitives", + "reth-tasks", + "tempfile", + "thiserror", + "tokio", + "tokio-stream", + "tokio-util", + "tracing", +] + +[[package]] +name = "reth-ecies" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "aes 0.8.3", + "alloy-rlp", + "block-padding", + "byteorder", + "cipher 0.4.4", + "ctr 0.9.2", + "digest 0.10.7", + "educe", + "futures", + "generic-array", + "hmac", + "pin-project", + "rand 0.8.5", + "reth-net-common", + "reth-primitives", + "secp256k1 0.27.0", + "sha2", + "sha3", + "thiserror", + "tokio", + "tokio-stream", + "tokio-util", + "tracing", + "typenum", +] + +[[package]] +name = "reth-eth-wire" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "alloy-rlp", + "async-trait", + "bytes", + "futures", + "metrics", + "pin-project", + "reth-codecs", + "reth-discv4", + "reth-ecies", + "reth-metrics", + "reth-primitives", + "serde", + "snap", + "thiserror", + "tokio", + "tokio-stream", + "tokio-util", + "tracing", +] + +[[package]] +name = "reth-interfaces" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "async-trait", + "auto_impl", + "clap", + "futures", + "modular-bitfield", + "parity-scale-codec", + "parking_lot 0.12.1", + "rand 0.8.5", + "reth-codecs", + "reth-eth-wire", + "reth-network-api", + "reth-nippy-jar", + "reth-primitives", + "reth-rpc-types", + "revm-primitives", + "secp256k1 0.27.0", + "thiserror", + "tokio", + "tokio-stream", + "tracing", +] + +[[package]] +name = "reth-ipc" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "async-trait", + "bytes", + "futures", + "jsonrpsee", + "parity-tokio-ipc", + "pin-project", + "serde_json", + "thiserror", + "tokio", + "tokio-stream", + "tokio-util", + "tower", + "tracing", +] + +[[package]] +name = "reth-libmdbx" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "bitflags 2.4.1", + "byteorder", + "derive_more", + "indexmap 2.0.2", + "libc", + "parking_lot 0.12.1", + "reth-mdbx-sys", + "thiserror", +] + +[[package]] +name = "reth-mdbx-sys" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "bindgen 0.68.1", + "cc", + "libc", +] + +[[package]] +name = "reth-metrics" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "futures", + "metrics", + "reth-metrics-derive", + "tokio", + "tokio-util", +] + +[[package]] +name = "reth-metrics-derive" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "once_cell", + "proc-macro2", + "quote", + "regex", + "syn 2.0.38", +] + +[[package]] +name = "reth-net-common" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "pin-project", + "reth-primitives", + "tokio", +] + +[[package]] +name = "reth-net-nat" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "igd", + "pin-project-lite", + "public-ip", + "serde_with", + "thiserror", + "tokio", + "tracing", +] + +[[package]] +name = "reth-network" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "alloy-rlp", + "aquamarine", + "async-trait", + "auto_impl", + "enr", + "fnv", + "futures", + "humantime-serde", + "linked-hash-map", + "linked_hash_set", + "metrics", + "parking_lot 0.12.1", + "pin-project", + "rand 0.8.5", + "reth-discv4", + "reth-dns-discovery", + "reth-ecies", + "reth-eth-wire", + "reth-interfaces", + "reth-metrics", + "reth-net-common", + "reth-network-api", + "reth-primitives", + "reth-provider", + "reth-rpc-types", + "reth-tasks", + "reth-tokio-util", + "reth-transaction-pool", + "secp256k1 0.27.0", + "serde", + "serde_json", + "thiserror", + "tokio", + "tokio-stream", + "tokio-util", + "tracing", +] + +[[package]] +name = "reth-network-api" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "async-trait", + "reth-discv4", + "reth-eth-wire", + "reth-primitives", + "reth-rpc-types", + "serde", + "thiserror", + "tokio", +] + +[[package]] +name = "reth-nippy-jar" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "anyhow", + "bincode", + "bytes", + "cuckoofilter", + "hex", + "lz4_flex", + "memmap2", + "ph", + "serde", + "sucds 0.8.1", + "tempfile", + "thiserror", + "tracing", + "tracing-appender", + "zstd 0.12.4", +] + +[[package]] +name = "reth-payload-builder" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "alloy-rlp", + "futures-util", + "metrics", + "reth-interfaces", + "reth-metrics", + "reth-primitives", + "reth-rpc-types", + "reth-rpc-types-compat", + "reth-transaction-pool", + "revm-primitives", + "sha2", + "thiserror", + "tokio", + "tokio-stream", + "tracing", +] + +[[package]] +name = "reth-primitives" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "alloy-primitives", + "alloy-rlp", + "alloy-sol-types", + "arbitrary", + "byteorder", + "bytes", + "c-kzg", + "clap", + "crc", + "derive_more", + "itertools 0.11.0", + "modular-bitfield", + "num_enum 0.7.0", + "once_cell", + "proptest", + "proptest-derive", + "rayon", + "reth-codecs", + "revm", + "revm-primitives", + "secp256k1 0.27.0", + "serde", + "serde_json", + "serde_with", + "sha2", + "strum", + "sucds 0.6.0", + "tempfile", + "thiserror", + "tracing", + "url", + "zstd 0.12.4", +] + +[[package]] +name = "reth-provider" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "alloy-rlp", + "auto_impl", + "itertools 0.11.0", + "parking_lot 0.12.1", + "pin-project", + "rayon", + "reth-db", + "reth-interfaces", + "reth-nippy-jar", + "reth-primitives", + "reth-trie", + "revm", + "tokio", + "tokio-stream", + "tracing", +] + +[[package]] +name = "reth-prune" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "itertools 0.11.0", + "metrics", + "rayon", + "reth-db", + "reth-interfaces", + "reth-metrics", + "reth-primitives", + "reth-provider", + "reth-snapshot", + "reth-tokio-util", + "thiserror", + "tokio", + "tokio-stream", + "tracing", +] + +[[package]] +name = "reth-revm" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "reth-consensus-common", + "reth-interfaces", + "reth-primitives", + "reth-provider", + "reth-revm-inspectors", + "revm", + "tracing", +] + +[[package]] +name = "reth-revm-inspectors" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "alloy-sol-types", + "boa_engine", + "boa_gc", + "reth-primitives", + "reth-rpc-types", + "revm", + "serde", + "serde_json", + "thiserror", + "tokio", +] + +[[package]] +name = "reth-rpc" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "alloy-dyn-abi", + "alloy-primitives", + "alloy-rlp", + "alloy-sol-types", + "async-trait", + "bytes", + "derive_more", + "futures", + "http", + "http-body", + "hyper", + "jsonrpsee", + "jsonwebtoken", + "lazy_static", + "metrics", + "pin-project", + "rand 0.8.5", + "rayon", + "reth-consensus-common", + "reth-interfaces", + "reth-metrics", + "reth-network-api", + "reth-primitives", + "reth-provider", + "reth-revm", + "reth-rpc-api", + "reth-rpc-engine-api", + "reth-rpc-types", + "reth-rpc-types-compat", + "reth-tasks", + "reth-transaction-pool", + "revm", + "revm-primitives", + "schnellru", + "secp256k1 0.27.0", + "serde", + "serde_json", + "thiserror", + "tokio", + "tokio-stream", + "tokio-util", + "tower", + "tracing", + "tracing-futures", +] + +[[package]] +name = "reth-rpc-api" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "jsonrpsee", + "reth-primitives", + "reth-rpc-types", + "serde_json", +] + +[[package]] +name = "reth-rpc-builder" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "hyper", + "jsonrpsee", + "metrics", + "reth-interfaces", + "reth-ipc", + "reth-metrics", + "reth-network-api", + "reth-primitives", + "reth-provider", + "reth-rpc", + "reth-rpc-api", + "reth-rpc-engine-api", + "reth-rpc-types", + "reth-rpc-types-compat", + "reth-tasks", + "reth-transaction-pool", + "serde", + "strum", + "thiserror", + "tower", + "tower-http", + "tracing", +] + +[[package]] +name = "reth-rpc-engine-api" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "async-trait", + "jsonrpsee-core", + "jsonrpsee-types", + "metrics", + "reth-beacon-consensus", + "reth-interfaces", + "reth-metrics", + "reth-payload-builder", + "reth-primitives", + "reth-provider", + "reth-rpc-api", + "reth-rpc-types", + "reth-rpc-types-compat", + "reth-tasks", + "serde", + "thiserror", + "tokio", + "tracing", +] + +[[package]] +name = "reth-rpc-types" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "alloy-primitives", + "alloy-rlp", + "itertools 0.11.0", + "jsonrpsee-types", + "reth-primitives", + "serde", + "serde_json", + "serde_with", + "thiserror", +] + +[[package]] +name = "reth-rpc-types-compat" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "alloy-rlp", + "reth-primitives", + "reth-rpc-types", +] + +[[package]] +name = "reth-snapshot" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "clap", + "reth-db", + "reth-interfaces", + "reth-nippy-jar", + "reth-primitives", + "reth-provider", + "thiserror", + "tokio", + "tracing", +] + +[[package]] +name = "reth-stages" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "aquamarine", + "async-trait", + "futures-util", + "itertools 0.11.0", + "metrics", + "num-traits", + "pin-project", + "rayon", + "reth-codecs", + "reth-db", + "reth-interfaces", + "reth-metrics", + "reth-primitives", + "reth-provider", + "reth-tokio-util", + "reth-trie", + "revm", + "serde", + "thiserror", + "tokio", + "tokio-stream", + "tracing", +] + +[[package]] +name = "reth-tasks" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "dyn-clone", + "futures-util", + "metrics", + "reth-metrics", + "thiserror", + "tokio", + "tracing", + "tracing-futures", +] + +[[package]] +name = "reth-tokio-util" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "tokio", + "tokio-stream", +] + +[[package]] +name = "reth-tracing" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "rolling-file", + "tracing", + "tracing-appender", + "tracing-journald", + "tracing-subscriber", +] + +[[package]] +name = "reth-transaction-pool" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "alloy-rlp", + "aquamarine", + "async-trait", + "auto_impl", + "bitflags 2.4.1", + "fnv", + "futures-util", + "metrics", + "parking_lot 0.12.1", + "paste", + "rand 0.8.5", + "reth-interfaces", + "reth-metrics", + "reth-primitives", + "reth-provider", + "reth-tasks", + "revm", + "serde", + "thiserror", + "tokio", + "tokio-stream", + "tracing", +] + +[[package]] +name = "reth-trie" +version = "0.1.0-alpha.10" +source = "git+https://github.com/paradigmxyz/reth.git#fc4fc936807870b07acb91e1d50c3920b55ff055" +dependencies = [ + "alloy-rlp", + "auto_impl", + "derive_more", + "reth-db", + "reth-interfaces", + "reth-primitives", + "thiserror", + "tokio", + "tracing", +] + +[[package]] +name = "rethdb-reader" +version = "0.1.0" +dependencies = [ + "reth", +] + +[[package]] +name = "revm" +version = "3.5.0" +source = "git+https://github.com/bluealloy/revm?rev=0d78d1eb304a2ce41ddac8f03206f5a316af247b#0d78d1eb304a2ce41ddac8f03206f5a316af247b" +dependencies = [ + "auto_impl", + "revm-interpreter", + "revm-precompile", +] + +[[package]] +name = "revm-interpreter" +version = "1.3.0" +source = "git+https://github.com/bluealloy/revm?rev=0d78d1eb304a2ce41ddac8f03206f5a316af247b#0d78d1eb304a2ce41ddac8f03206f5a316af247b" +dependencies = [ + "revm-primitives", +] + +[[package]] +name = "revm-precompile" +version = "2.2.0" +source = "git+https://github.com/bluealloy/revm?rev=0d78d1eb304a2ce41ddac8f03206f5a316af247b#0d78d1eb304a2ce41ddac8f03206f5a316af247b" +dependencies = [ + "aurora-engine-modexp", + "c-kzg", + "k256", + "once_cell", + "revm-primitives", + "ripemd", + "secp256k1 0.28.0", + "sha2", + "substrate-bn", +] + +[[package]] +name = "revm-primitives" +version = "1.3.0" +source = "git+https://github.com/bluealloy/revm?rev=0d78d1eb304a2ce41ddac8f03206f5a316af247b#0d78d1eb304a2ce41ddac8f03206f5a316af247b" +dependencies = [ + "alloy-primitives", + "alloy-rlp", + "auto_impl", + "bitflags 2.4.1", + "bitvec", + "c-kzg", + "derive_more", + "enumn", + "hashbrown 0.14.2", + "hex", + "once_cell", + "serde", +] + +[[package]] +name = "rfc6979" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f8dd2a808d456c4a54e300a23e9f5a67e122c3024119acbfd73e3bf664491cb2" +dependencies = [ + "hmac", + "subtle", +] + +[[package]] +name = "ring" +version = "0.16.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3053cf52e236a3ed746dfc745aa9cacf1b791d846bdaf412f60a8d7d6e17c8fc" +dependencies = [ + "cc", + "libc", + "once_cell", + "spin 0.5.2", + "untrusted 0.7.1", + "web-sys", + "winapi", +] + +[[package]] +name = "ring" +version = "0.17.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fb0205304757e5d899b9c2e448b867ffd03ae7f988002e47cd24954391394d0b" +dependencies = [ + "cc", + "getrandom 0.2.10", + "libc", + "spin 0.9.8", + "untrusted 0.9.0", + "windows-sys 0.48.0", +] + +[[package]] +name = "ripemd" +version = "0.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bd124222d17ad93a644ed9d011a40f4fb64aa54275c08cc216524a9ea82fb09f" +dependencies = [ + "digest 0.10.7", +] + +[[package]] +name = "rlimit" +version = "0.10.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3560f70f30a0f16d11d01ed078a07740fe6b489667abc7c7b029155d9f21c3d8" +dependencies = [ + "libc", +] + +[[package]] +name = "rlp" +version = "0.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bb919243f34364b6bd2fc10ef797edbfa75f33c252e7998527479c6d6b47e1ec" +dependencies = [ + "bytes", + "rustc-hex", +] + +[[package]] +name = "rolling-file" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8395b4f860856b740f20a296ea2cd4d823e81a2658cf05ef61be22916026a906" +dependencies = [ + "chrono", +] + +[[package]] +name = "route-recognizer" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "afab94fb28594581f62d981211a9a4d53cc8130bbcbbb89a0440d9b8e81a7746" + +[[package]] +name = "ruint" +version = "1.10.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "95294d6e3a6192f3aabf91c38f56505a625aa495533442744185a36d75a790c4" +dependencies = [ + "alloy-rlp", + "arbitrary", + "ark-ff 0.3.0", + "ark-ff 0.4.2", + "bytes", + "fastrlp", + "num-bigint", + "parity-scale-codec", + "primitive-types", + "proptest", + "rand 0.8.5", + "rlp", + "ruint-macro", + "serde", + "valuable", + "zeroize", +] + +[[package]] +name = "ruint-macro" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e666a5496a0b2186dbcd0ff6106e29e093c15591bde62c20d3842007c6978a09" + +[[package]] +name = "rustc-demangle" +version = "0.1.23" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d626bb9dae77e28219937af045c257c28bfd3f69333c512553507f5f9798cb76" + +[[package]] +name = "rustc-hash" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "08d43f7aa6b08d49f382cde6a7982047c3426db949b1424bc4b7ec9ae12c6ce2" + +[[package]] +name = "rustc-hex" +version = "2.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3e75f6a532d0fd9f7f13144f392b6ad56a32696bfcd9c78f797f16bbb6f072d6" + +[[package]] +name = "rustc_version" +version = "0.3.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f0dfe2087c51c460008730de8b57e6a320782fbfb312e1f4d520e6c6fae155ee" +dependencies = [ + "semver 0.11.0", +] + +[[package]] +name = "rustc_version" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bfa0f585226d2e68097d4f95d113b15b83a82e819ab25717ec0590d9584ef366" +dependencies = [ + "semver 1.0.20", +] + +[[package]] +name = "rustix" +version = "0.36.16" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6da3636faa25820d8648e0e31c5d519bbb01f72fdf57131f0f5f7da5fed36eab" +dependencies = [ + "bitflags 1.3.2", + "errno 0.3.5", + "io-lifetimes", + "libc", + "linux-raw-sys 0.1.4", + "windows-sys 0.45.0", +] + +[[package]] +name = "rustix" +version = "0.38.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "67ce50cb2e16c2903e30d1cbccfd8387a74b9d4c938b6a4c5ec6cc7556f7a8a0" +dependencies = [ + "bitflags 2.4.1", + "errno 0.3.5", + "libc", + "linux-raw-sys 0.4.10", + "windows-sys 0.48.0", +] + +[[package]] +name = "rustls" +version = "0.21.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "446e14c5cda4f3f30fe71863c34ec70f5ac79d6087097ad0bb433e1be5edf04c" +dependencies = [ + "log", + "ring 0.17.5", + "rustls-webpki", + "sct", +] + +[[package]] +name = "rustls-native-certs" +version = "0.6.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a9aace74cb666635c918e9c12bc0d348266037aa8eb599b5cba565709a8dff00" +dependencies = [ + "openssl-probe", + "rustls-pemfile", + "schannel", + "security-framework", +] + +[[package]] +name = "rustls-pemfile" +version = "1.0.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2d3987094b1d07b653b7dfdc3f70ce9a1da9c51ac18c1b06b662e4f9a0e9f4b2" +dependencies = [ + "base64 0.21.5", +] + +[[package]] +name = "rustls-webpki" +version = "0.101.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8b6275d1ee7a1cd780b64aca7726599a1dbc893b1e64144529e55c3c2f745765" +dependencies = [ + "ring 0.17.5", + "untrusted 0.9.0", +] + +[[package]] +name = "rustversion" +version = "1.0.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7ffc183a10b4478d04cbbbfc96d0873219d962dd5accaff2ffbd4ceb7df837f4" + +[[package]] +name = "rusty-fork" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cb3dcc6e454c328bb824492db107ab7c0ae8fcffe4ad210136ef014458c1bc4f" +dependencies = [ + "fnv", + "quick-error", + "tempfile", + "wait-timeout", +] + +[[package]] +name = "ryu" +version = "1.0.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1ad4cc8da4ef723ed60bced201181d83791ad433213d8c24efffda1eec85d741" + +[[package]] +name = "ryu-js" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6518fc26bced4d53678a22d6e423e9d8716377def84545fe328236e3af070e7f" + +[[package]] +name = "schannel" +version = "0.1.22" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0c3733bf4cf7ea0880754e19cb5a462007c4a8c1914bff372ccc95b464f1df88" +dependencies = [ + "windows-sys 0.48.0", +] + +[[package]] +name = "schnellru" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "772575a524feeb803e5b0fcbc6dd9f367e579488197c94c6e4023aad2305774d" +dependencies = [ + "ahash", + "cfg-if", + "hashbrown 0.13.2", +] + +[[package]] +name = "scopeguard" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "94143f37725109f92c262ed2cf5e59bce7498c01bcc1502d7b9afe439a4e9f49" + +[[package]] +name = "sct" +version = "0.7.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "da046153aa2352493d6cb7da4b6e5c0c057d8a1d0a9aa8560baffdd945acd414" +dependencies = [ + "ring 0.17.5", + "untrusted 0.9.0", +] + +[[package]] +name = "sec1" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d3e97a565f76233a6003f9f5c54be1d9c5bdfa3eccfb189469f11ec4901c47dc" +dependencies = [ + "base16ct", + "der", + "generic-array", + "pkcs8", + "subtle", + "zeroize", +] + +[[package]] +name = "secp256k1" +version = "0.27.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "25996b82292a7a57ed3508f052cfff8640d38d32018784acd714758b43da9c8f" +dependencies = [ + "rand 0.8.5", + "secp256k1-sys 0.8.1", + "serde", +] + +[[package]] +name = "secp256k1" +version = "0.28.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2acea373acb8c21ecb5a23741452acd2593ed44ee3d343e72baaa143bc89d0d5" +dependencies = [ + "secp256k1-sys 0.9.0", +] + +[[package]] +name = "secp256k1-sys" +version = "0.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "70a129b9e9efbfb223753b9163c4ab3b13cff7fd9c7f010fbac25ab4099fa07e" +dependencies = [ + "cc", +] + +[[package]] +name = "secp256k1-sys" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "09e67c467c38fd24bd5499dc9a18183b31575c12ee549197e3e20d57aa4fe3b7" +dependencies = [ + "cc", +] + +[[package]] +name = "security-framework" +version = "2.9.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "05b64fb303737d99b81884b2c63433e9ae28abebe5eb5045dcdd175dc2ecf4de" +dependencies = [ + "bitflags 1.3.2", + "core-foundation", + "core-foundation-sys", + "libc", + "security-framework-sys", +] + +[[package]] +name = "security-framework-sys" +version = "2.9.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e932934257d3b408ed8f30db49d85ea163bfe74961f017f405b025af298f0c7a" +dependencies = [ + "core-foundation-sys", + "libc", +] + +[[package]] +name = "semver" +version = "0.11.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f301af10236f6df4160f7c3f04eec6dbc70ace82d23326abad5edee88801c6b6" +dependencies = [ + "semver-parser", +] + +[[package]] +name = "semver" +version = "1.0.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "836fa6a3e1e547f9a2c4040802ec865b5d85f4014efe00555d7090a3dcaa1090" + +[[package]] +name = "semver-parser" +version = "0.10.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "00b0bef5b7f9e0df16536d3961cfb6e84331c065b4066afb39768d0e319411f7" +dependencies = [ + "pest", +] + +[[package]] +name = "send_wrapper" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f638d531eccd6e23b980caf34876660d38e265409d8e99b397ab71eb3612fad0" + +[[package]] +name = "serde" +version = "1.0.190" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "91d3c334ca1ee894a2c6f6ad698fe8c435b76d504b13d436f0685d648d6d96f7" +dependencies = [ + "serde_derive", +] + +[[package]] +name = "serde_bytes" +version = "0.11.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ab33ec92f677585af6d88c65593ae2375adde54efdbf16d597f2cbc7a6d368ff" +dependencies = [ + "serde", +] + +[[package]] +name = "serde_derive" +version = "1.0.190" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "67c5609f394e5c2bd7fc51efda478004ea80ef42fee983d5c67a65e34f32c0e3" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.38", +] + +[[package]] +name = "serde_json" +version = "1.0.107" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6b420ce6e3d8bd882e9b243c6eed35dbc9a6110c9769e74b584e0d68d1f20c65" +dependencies = [ + "itoa", + "ryu", + "serde", +] + +[[package]] +name = "serde_spanned" +version = "0.6.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "12022b835073e5b11e90a14f86838ceb1c8fb0325b72416845c487ac0fa95e80" +dependencies = [ + "serde", +] + +[[package]] +name = "serde_with" +version = "3.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "64cd236ccc1b7a29e7e2739f27c0b2dd199804abc4290e32f59f3b68d6405c23" +dependencies = [ + "base64 0.21.5", + "chrono", + "hex", + "indexmap 1.9.3", + "indexmap 2.0.2", + "serde", + "serde_json", + "serde_with_macros", + "time", +] + +[[package]] +name = "serde_with_macros" +version = "3.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "93634eb5f75a2323b16de4748022ac4297f9e76b6dced2be287a099f41b5e788" +dependencies = [ + "darling 0.20.3", + "proc-macro2", + "quote", + "syn 2.0.38", +] + +[[package]] +name = "sha-1" +version = "0.9.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "99cd6713db3cf16b6c84e06321e049a9b9f699826e16096d23bbcc44d15d51a6" +dependencies = [ + "block-buffer 0.9.0", + "cfg-if", + "cpufeatures", + "digest 0.9.0", + "opaque-debug", +] + +[[package]] +name = "sha2" +version = "0.10.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "793db75ad2bcafc3ffa7c68b215fee268f537982cd901d132f89c6343f3a3dc8" +dependencies = [ + "cfg-if", + "cpufeatures", + "digest 0.10.7", +] + +[[package]] +name = "sha3" +version = "0.10.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "75872d278a8f37ef87fa0ddbda7802605cb18344497949862c0d4dcb291eba60" +dependencies = [ + "digest 0.10.7", + "keccak", +] + +[[package]] +name = "sharded-slab" +version = "0.1.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f40ca3c46823713e0d4209592e8d6e826aa57e928f09752619fc696c499637f6" +dependencies = [ + "lazy_static", +] + +[[package]] +name = "shellexpand" +version = "3.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "da03fa3b94cc19e3ebfc88c4229c49d8f08cdbd1228870a45f0ffdf84988e14b" +dependencies = [ + "dirs", +] + +[[package]] +name = "shlex" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a7cee0529a6d40f580e7a5e6c495c8fbfe21b7b52795ed4bb5e62cdf92bc6380" + +[[package]] +name = "signal-hook" +version = "0.3.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8621587d4798caf8eb44879d42e56b9a93ea5dcd315a6487c357130095b62801" +dependencies = [ + "libc", + "signal-hook-registry", +] + +[[package]] +name = "signal-hook-mio" +version = "0.2.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "29ad2e15f37ec9a6cc544097b78a1ec90001e9f71b81338ca39f430adaca99af" +dependencies = [ + "libc", + "mio", + "signal-hook", +] + +[[package]] +name = "signal-hook-registry" +version = "1.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d8229b473baa5980ac72ef434c4415e70c4b5e71b423043adb4ba059f89c99a1" +dependencies = [ + "libc", +] + +[[package]] +name = "signature" +version = "2.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5e1788eed21689f9cf370582dfc467ef36ed9c707f073528ddafa8d83e3b8500" +dependencies = [ + "digest 0.10.7", + "rand_core 0.6.4", +] + +[[package]] +name = "simple_asn1" +version = "0.6.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "adc4e5204eb1910f40f9cfa375f6f05b68c3abac4b6fd879c8ff5e7ae8a0a085" +dependencies = [ + "num-bigint", + "num-traits", + "thiserror", + "time", +] + +[[package]] +name = "siphasher" +version = "0.3.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "38b58827f4464d87d377d175e90bf58eb00fd8716ff0a62f80356b5e61555d0d" + +[[package]] +name = "sketches-ddsketch" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "68a406c1882ed7f29cd5e248c9848a80e7cb6ae0fea82346d2746f2f941c07e1" + +[[package]] +name = "slab" +version = "0.4.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8f92a496fb766b417c996b9c5e57daf2f7ad3b0bebe1ccfca4856390e3d3bb67" +dependencies = [ + "autocfg", +] + +[[package]] +name = "smallvec" +version = "1.11.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "942b4a808e05215192e39f4ab80813e599068285906cc91aa64f923db842bd5a" + +[[package]] +name = "smol_str" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "74212e6bbe9a4352329b2f68ba3130c15a3f26fe88ff22dbdc6cdd58fa85e99c" +dependencies = [ + "serde", +] + +[[package]] +name = "snap" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5e9f0ab6ef7eb7353d9119c170a436d1bf248eea575ac42d19d12f4e34130831" + +[[package]] +name = "socket2" +version = "0.4.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9f7916fc008ca5542385b89a3d3ce689953c143e9304a9bf8beec1de48994c0d" +dependencies = [ + "libc", + "winapi", +] + +[[package]] +name = "socket2" +version = "0.5.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7b5fac59a5cb5dd637972e5fca70daf0523c9067fcdc4842f053dae04a18f8e9" +dependencies = [ + "libc", + "windows-sys 0.48.0", +] + +[[package]] +name = "soketto" +version = "0.7.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "41d1c5305e39e09653383c2c7244f2f78b3bcae37cf50c64cb4789c9f5096ec2" +dependencies = [ + "base64 0.13.1", + "bytes", + "futures", + "http", + "httparse", + "log", + "rand 0.8.5", + "sha-1", +] + +[[package]] +name = "spin" +version = "0.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6e63cff320ae2c57904679ba7cb63280a3dc4613885beafb148ee7bf9aa9042d" + +[[package]] +name = "spin" +version = "0.9.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6980e8d7511241f8acf4aebddbb1ff938df5eebe98691418c4468d0b72a96a67" +dependencies = [ + "lock_api", +] + +[[package]] +name = "spki" +version = "0.7.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9d1e996ef02c474957d681f1b05213dfb0abab947b446a62d37770b23500184a" +dependencies = [ + "base64ct", + "der", +] + +[[package]] +name = "sptr" +version = "0.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3b9b39299b249ad65f3b7e96443bad61c02ca5cd3589f46cb6d610a0fd6c0d6a" + +[[package]] +name = "stable_deref_trait" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a8f112729512f8e442d81f95a8a7ddf2b7c6b8a1a6f509a95864142b30cab2d3" + +[[package]] +name = "static_assertions" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a2eb9349b6444b326872e140eb1cf5e7c522154d69e7a0ffb0fb81c06b37543f" + +[[package]] +name = "strsim" +version = "0.9.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6446ced80d6c486436db5c078dde11a9f73d42b57fb273121e160b84f63d894c" + +[[package]] +name = "strsim" +version = "0.10.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "73473c0e59e6d5812c5dfe2a064a6444949f089e20eec9a2e5506596494e4623" + +[[package]] +name = "strum" +version = "0.25.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "290d54ea6f91c969195bdbcd7442c8c2a2ba87da8bf60a7ee86a235d4bc1e125" +dependencies = [ + "strum_macros", +] + +[[package]] +name = "strum_macros" +version = "0.25.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "23dc1fa9ac9c169a78ba62f0b841814b7abae11bdd047b9c58f893439e309ea0" +dependencies = [ + "heck", + "proc-macro2", + "quote", + "rustversion", + "syn 2.0.38", +] + +[[package]] +name = "substrate-bn" +version = "0.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "72b5bbfa79abbae15dd642ea8176a21a635ff3c00059961d1ea27ad04e5b441c" +dependencies = [ + "byteorder", + "crunchy", + "lazy_static", + "rand 0.8.5", + "rustc-hex", +] + +[[package]] +name = "subtle" +version = "2.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "81cdd64d312baedb58e21336b31bc043b77e01cc99033ce76ef539f78e965ebc" + +[[package]] +name = "sucds" +version = "0.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "64accd20141dfbef67ad83c51d588146cff7810616e1bda35a975be369059533" +dependencies = [ + "anyhow", +] + +[[package]] +name = "sucds" +version = "0.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d53d46182afe6ed822a94c54a532dc0d59691a8f49226bdc4596529ca864cdd6" +dependencies = [ + "anyhow", + "num-traits", +] + +[[package]] +name = "syn" +version = "1.0.109" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "72b64191b275b66ffe2469e8af2c1cfe3bafa67b529ead792a6d0160888b4237" +dependencies = [ + "proc-macro2", + "quote", + "unicode-ident", +] + +[[package]] +name = "syn" +version = "2.0.38" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e96b79aaa137db8f61e26363a0c9b47d8b4ec75da28b7d1d614c2303e232408b" +dependencies = [ + "proc-macro2", + "quote", + "unicode-ident", +] + +[[package]] +name = "syn-solidity" +version = "0.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "86b837ef12ab88835251726eb12237655e61ec8dc8a280085d1961cdc3dfd047" +dependencies = [ + "paste", + "proc-macro2", + "quote", + "syn 2.0.38", +] + +[[package]] +name = "synstructure" +version = "0.13.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "285ba80e733fac80aa4270fbcdf83772a79b80aa35c97075320abfee4a915b06" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.38", + "unicode-xid", +] + +[[package]] +name = "tap" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "55937e1799185b12863d447f42597ed69d9928686b8d88a1df17376a097d8369" + +[[package]] +name = "tempfile" +version = "3.8.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cb94d2f3cc536af71caac6b6fcebf65860b347e7ce0cc9ebe8f70d3e521054ef" +dependencies = [ + "cfg-if", + "fastrand 2.0.1", + "redox_syscall 0.3.5", + "rustix 0.38.20", + "windows-sys 0.48.0", +] + +[[package]] +name = "thin-vec" +version = "0.2.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "aac81b6fd6beb5884b0cf3321b8117e6e5d47ecb6fc89f414cfdcca8b2fe2dd8" + +[[package]] +name = "thiserror" +version = "1.0.50" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f9a7210f5c9a7156bb50aa36aed4c95afb51df0df00713949448cf9e97d382d2" +dependencies = [ + "thiserror-impl", +] + +[[package]] +name = "thiserror-impl" +version = "1.0.50" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "266b2e40bc00e5a6c09c3584011e08b06f123c00362c92b975ba9843aaaa14b8" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.38", +] + +[[package]] +name = "thread_local" +version = "1.1.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3fdd6f064ccff2d6567adcb3873ca630700f00b5ad3f060c25b5dcfd9a4ce152" +dependencies = [ + "cfg-if", + "once_cell", +] + +[[package]] +name = "threadpool" +version = "1.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d050e60b33d41c19108b32cea32164033a9013fe3b46cbd4457559bfbf77afaa" +dependencies = [ + "num_cpus", +] + +[[package]] +name = "time" +version = "0.3.30" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c4a34ab300f2dee6e562c10a046fc05e358b29f9bf92277f30c3c8d82275f6f5" +dependencies = [ + "deranged", + "itoa", + "libc", + "num_threads", + "powerfmt", + "serde", + "time-core", + "time-macros", +] + +[[package]] +name = "time-core" +version = "0.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ef927ca75afb808a4d64dd374f00a2adf8d0fcff8e7b184af886c3c87ec4a3f3" + +[[package]] +name = "time-macros" +version = "0.2.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4ad70d68dba9e1f8aceda7aa6711965dfec1cac869f311a51bd08b3a2ccbce20" +dependencies = [ + "time-core", +] + +[[package]] +name = "tiny-keccak" +version = "2.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2c9d3793400a45f954c52e73d068316d76b6f4e36977e3fcebb13a2721e80237" +dependencies = [ + "crunchy", +] + +[[package]] +name = "tinystr" +version = "0.7.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8faa444297615a4e020acb64146b0603c9c395c03a97c17fd9028816d3b4d63e" +dependencies = [ + "displaydoc", + "serde", + "zerovec", +] + +[[package]] +name = "tinyvec" +version = "1.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "87cc5ceb3875bb20c2890005a4e226a4651264a5c75edb2421b52861a0a0cb50" +dependencies = [ + "tinyvec_macros", +] + +[[package]] +name = "tinyvec_macros" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" + +[[package]] +name = "tokio" +version = "1.33.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4f38200e3ef7995e5ef13baec2f432a6da0aa9ac495b2c0e8f3b7eec2c92d653" +dependencies = [ + "backtrace", + "bytes", + "libc", + "mio", + "num_cpus", + "parking_lot 0.12.1", + "pin-project-lite", + "signal-hook-registry", + "socket2 0.5.5", + "tokio-macros", + "windows-sys 0.48.0", +] + +[[package]] +name = "tokio-macros" +version = "2.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "630bdcf245f78637c13ec01ffae6187cca34625e8c63150d424b59e55af2675e" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.38", +] + +[[package]] +name = "tokio-rustls" +version = "0.24.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c28327cf380ac148141087fbfb9de9d7bd4e84ab5d2c28fbc911d753de8a7081" +dependencies = [ + "rustls", + "tokio", +] + +[[package]] +name = "tokio-stream" +version = "0.1.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "397c988d37662c7dda6d2208364a706264bf3d6138b11d436cbac0ad38832842" +dependencies = [ + "futures-core", + "pin-project-lite", + "tokio", + "tokio-util", +] + +[[package]] +name = "tokio-util" +version = "0.7.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5419f34732d9eb6ee4c3578b7989078579b7f039cbbb9ca2c4da015749371e15" +dependencies = [ + "bytes", + "futures-core", + "futures-io", + "futures-sink", + "pin-project-lite", + "slab", + "tokio", + "tracing", +] + +[[package]] +name = "toml" +version = "0.5.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f4f7f0dd8d50a853a531c426359045b1998f04219d88799810762cd4ad314234" +dependencies = [ + "serde", +] + +[[package]] +name = "toml" +version = "0.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3efaf127c78d5339cc547cce4e4d973bd5e4f56e949a06d091c082ebeef2f800" +dependencies = [ + "serde", + "serde_spanned", + "toml_datetime", + "toml_edit 0.20.5", +] + +[[package]] +name = "toml_datetime" +version = "0.6.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3550f4e9685620ac18a50ed434eb3aec30db8ba93b0287467bca5826ea25baf1" +dependencies = [ + "serde", +] + +[[package]] +name = "toml_edit" +version = "0.19.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1b5bb770da30e5cbfde35a2d7b9b8a2c4b8ef89548a7a6aeab5c9a576e3e7421" +dependencies = [ + "indexmap 2.0.2", + "toml_datetime", + "winnow", +] + +[[package]] +name = "toml_edit" +version = "0.20.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "782bf6c2ddf761c1e7855405e8975472acf76f7f36d0d4328bd3b7a2fae12a85" +dependencies = [ + "indexmap 2.0.2", + "serde", + "serde_spanned", + "toml_datetime", + "winnow", +] + +[[package]] +name = "tower" +version = "0.4.13" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b8fa9be0de6cf49e536ce1851f987bd21a43b771b09473c3549a6c853db37c1c" +dependencies = [ + "futures-core", + "futures-util", + "hdrhistogram", + "indexmap 1.9.3", + "pin-project", + "pin-project-lite", + "rand 0.8.5", + "slab", + "tokio", + "tokio-util", + "tower-layer", + "tower-service", + "tracing", +] + +[[package]] +name = "tower-http" +version = "0.4.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "61c5bb1d698276a2443e5ecfabc1008bf15a36c12e6a7176e7bf089ea9131140" +dependencies = [ + "async-compression", + "base64 0.21.5", + "bitflags 2.4.1", + "bytes", + "futures-core", + "futures-util", + "http", + "http-body", + "http-range-header", + "httpdate", + "iri-string", + "mime", + "mime_guess", + "percent-encoding", + "pin-project-lite", + "tokio", + "tokio-util", + "tower", + "tower-layer", + "tower-service", + "tracing", + "uuid", +] + +[[package]] +name = "tower-layer" +version = "0.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c20c8dbed6283a09604c3e69b4b7eeb54e298b8a600d4d5ecb5ad39de609f1d0" + +[[package]] +name = "tower-service" +version = "0.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b6bc1c9ce2b5135ac7f93c72918fc37feb872bdc6a5533a8b85eb4b86bfdae52" + +[[package]] +name = "tracing" +version = "0.1.40" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c3523ab5a71916ccf420eebdf5521fcef02141234bbc0b8a49f2fdc4544364ef" +dependencies = [ + "log", + "pin-project-lite", + "tracing-attributes", + "tracing-core", +] + +[[package]] +name = "tracing-appender" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "09d48f71a791638519505cefafe162606f706c25592e4bde4d97600c0195312e" +dependencies = [ + "crossbeam-channel", + "time", + "tracing-subscriber", +] + +[[package]] +name = "tracing-attributes" +version = "0.1.27" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.38", +] + +[[package]] +name = "tracing-core" +version = "0.1.32" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c06d3da6113f116aaee68e4d601191614c9053067f9ab7f6edbcb161237daa54" +dependencies = [ + "once_cell", + "valuable", +] + +[[package]] +name = "tracing-futures" +version = "0.2.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "97d095ae15e245a057c8e8451bab9b3ee1e1f68e9ba2b4fbc18d0ac5237835f2" +dependencies = [ + "futures", + "futures-task", + "pin-project", + "tracing", +] + +[[package]] +name = "tracing-journald" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ba316a74e8fc3c3896a850dba2375928a9fa171b085ecddfc7c054d39970f3fd" +dependencies = [ + "libc", + "tracing-core", + "tracing-subscriber", +] + +[[package]] +name = "tracing-log" +version = "0.1.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f751112709b4e791d8ce53e32c4ed2d353565a795ce84da2285393f41557bdf2" +dependencies = [ + "log", + "once_cell", + "tracing-core", +] + +[[package]] +name = "tracing-subscriber" +version = "0.3.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "30a651bc37f915e81f087d86e62a18eec5f79550c7faff886f7090b4ea757c77" +dependencies = [ + "matchers", + "nu-ansi-term", + "once_cell", + "regex", + "sharded-slab", + "smallvec", + "thread_local", + "tracing", + "tracing-core", + "tracing-log", +] + +[[package]] +name = "trust-dns-client" +version = "0.20.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5b4ef9b9bde0559b78a4abb00339143750085f05e5a453efb7b8bef1061f09dc" +dependencies = [ + "cfg-if", + "data-encoding", + "futures-channel", + "futures-util", + "lazy_static", + "log", + "radix_trie", + "rand 0.8.5", + "thiserror", + "time", + "tokio", + "trust-dns-proto 0.20.4", +] + +[[package]] +name = "trust-dns-proto" +version = "0.20.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ca94d4e9feb6a181c690c4040d7a24ef34018d8313ac5044a61d21222ae24e31" +dependencies = [ + "async-trait", + "cfg-if", + "data-encoding", + "enum-as-inner 0.3.4", + "futures-channel", + "futures-io", + "futures-util", + "idna 0.2.3", + "ipnet", + "lazy_static", + "log", + "rand 0.8.5", + "smallvec", + "thiserror", + "tinyvec", + "tokio", + "url", +] + +[[package]] +name = "trust-dns-proto" +version = "0.23.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3119112651c157f4488931a01e586aa459736e9d6046d3bd9105ffb69352d374" +dependencies = [ + "async-trait", + "cfg-if", + "data-encoding", + "enum-as-inner 0.6.0", + "futures-channel", + "futures-io", + "futures-util", + "idna 0.4.0", + "ipnet", + "once_cell", + "rand 0.8.5", + "smallvec", + "thiserror", + "tinyvec", + "tokio", + "tracing", + "url", +] + +[[package]] +name = "trust-dns-resolver" +version = "0.23.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "10a3e6c3aff1718b3c73e395d1f35202ba2ffa847c6a62eea0db8fb4cfe30be6" +dependencies = [ + "cfg-if", + "futures-util", + "ipconfig", + "lru-cache", + "once_cell", + "parking_lot 0.12.1", + "rand 0.8.5", + "resolv-conf", + "smallvec", + "thiserror", + "tokio", + "tracing", + "trust-dns-proto 0.23.2", +] + +[[package]] +name = "try-lock" +version = "0.2.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3528ecfd12c466c6f163363caf2d02a71161dd5e1cc6ae7b34207ea2d42d81ed" + +[[package]] +name = "tui" +version = "0.19.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ccdd26cbd674007e649a272da4475fb666d3aa0ad0531da7136db6fab0e5bad1" +dependencies = [ + "bitflags 1.3.2", + "cassowary", + "crossterm 0.25.0", + "unicode-segmentation", + "unicode-width", +] + +[[package]] +name = "typenum" +version = "1.17.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "42ff0bf0c66b8238c6f3b578df37d0b7848e55df8577b3f74f92a69acceeb825" + +[[package]] +name = "ucd-trie" +version = "0.1.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ed646292ffc8188ef8ea4d1e0e0150fb15a5c2e12ad9b8fc191ae7a8a7f3c4b9" + +[[package]] +name = "uint" +version = "0.9.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "76f64bba2c53b04fcab63c01a7d7427eadc821e3bc48c34dc9ba29c501164b52" +dependencies = [ + "byteorder", + "crunchy", + "hex", + "static_assertions", +] + +[[package]] +name = "unarray" +version = "0.1.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "eaea85b334db583fe3274d12b4cd1880032beab409c0d774be044d4480ab9a94" + +[[package]] +name = "unicase" +version = "2.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f7d2d4dafb69621809a81864c9c1b864479e1235c0dd4e199924b9742439ed89" +dependencies = [ + "version_check", +] + +[[package]] +name = "unicode-bidi" +version = "0.3.13" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "92888ba5573ff080736b3648696b70cafad7d250551175acbaa4e0385b3e1460" + +[[package]] +name = "unicode-ident" +version = "1.0.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3354b9ac3fae1ff6755cb6db53683adb661634f67557942dea4facebec0fee4b" + +[[package]] +name = "unicode-normalization" +version = "0.1.22" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5c5713f0fc4b5db668a2ac63cdb7bb4469d8c9fed047b1d0292cc7b0ce2ba921" +dependencies = [ + "tinyvec", +] + +[[package]] +name = "unicode-segmentation" +version = "1.10.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1dd624098567895118886609431a7c3b8f516e41d30e0643f03d94592a147e36" + +[[package]] +name = "unicode-width" +version = "0.1.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e51733f11c9c4f72aa0c160008246859e340b00807569a0da0e7a1079b27ba85" + +[[package]] +name = "unicode-xid" +version = "0.2.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f962df74c8c05a667b5ee8bcf162993134c104e96440b663c8daa176dc772d8c" + +[[package]] +name = "universal-hash" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8326b2c654932e3e4f9196e69d08fdf7cfd718e1dc6f66b347e6024a0c961402" +dependencies = [ + "generic-array", + "subtle", +] + +[[package]] +name = "untrusted" +version = "0.7.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a156c684c91ea7d62626509bce3cb4e1d9ed5c4d978f7b4352658f96a4c26b4a" + +[[package]] +name = "untrusted" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8ecb6da28b8a351d773b68d5825ac39017e680750f980f3a1a85cd8dd28a47c1" + +[[package]] +name = "url" +version = "2.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "143b538f18257fac9cad154828a57c6bf5157e1aa604d4816b5995bf6de87ae5" +dependencies = [ + "form_urlencoded", + "idna 0.4.0", + "percent-encoding", +] + +[[package]] +name = "utf16_iter" +version = "1.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "52df8b7fb78e7910d776fccf2e42ceaf3604d55e8e7eb2dbd183cb1441d8a692" + +[[package]] +name = "utf8_iter" +version = "1.0.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "64a8922555b9500e3d865caed19330172cd67cbf82203f1a3311d8c305cc9f33" + +[[package]] +name = "utf8parse" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "711b9620af191e0cdc7468a8d14e709c3dcdb115b36f838e601583af800a370a" + +[[package]] +name = "uuid" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "88ad59a7560b41a70d191093a945f0b87bc1deeda46fb237479708a1d6b6cdfc" +dependencies = [ + "getrandom 0.2.10", +] + +[[package]] +name = "valuable" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "830b7e5d4d90034032940e4ace0d9a9a057e7a45cd94e6c007832e39edb82f6d" + +[[package]] +name = "vergen" +version = "8.2.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "85e7dc29b3c54a2ea67ef4f953d5ec0c4085035c0ae2d325be1c0d2144bd9f16" +dependencies = [ + "anyhow", + "rustversion", + "time", +] + +[[package]] +name = "version_check" +version = "0.9.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "49874b5167b65d7193b8aba1567f5c7d93d001cafc34600cee003eda787e483f" + +[[package]] +name = "wait-timeout" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9f200f5b12eb75f8c1ed65abd4b2db8a6e1b138a20de009dacee265a2498f3f6" +dependencies = [ + "libc", +] + +[[package]] +name = "want" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bfa7760aed19e106de2c7c0b581b509f2f25d3dacaf737cb82ac61bc6d760b0e" +dependencies = [ + "try-lock", +] + +[[package]] +name = "wasi" +version = "0.9.0+wasi-snapshot-preview1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cccddf32554fecc6acb585f82a32a72e28b48f8c4c1883ddfeeeaa96f7d8e519" + +[[package]] +name = "wasi" +version = "0.11.0+wasi-snapshot-preview1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" + +[[package]] +name = "wasm-bindgen" +version = "0.2.87" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7706a72ab36d8cb1f80ffbf0e071533974a60d0a308d01a5d0375bf60499a342" +dependencies = [ + "cfg-if", + "wasm-bindgen-macro", +] + +[[package]] +name = "wasm-bindgen-backend" +version = "0.2.87" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5ef2b6d3c510e9625e5fe6f509ab07d66a760f0885d858736483c32ed7809abd" +dependencies = [ + "bumpalo", + "log", + "once_cell", + "proc-macro2", + "quote", + "syn 2.0.38", + "wasm-bindgen-shared", +] + +[[package]] +name = "wasm-bindgen-futures" +version = "0.4.37" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c02dbc21516f9f1f04f187958890d7e6026df8d16540b7ad9492bc34a67cea03" +dependencies = [ + "cfg-if", + "js-sys", + "wasm-bindgen", + "web-sys", +] + +[[package]] +name = "wasm-bindgen-macro" +version = "0.2.87" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dee495e55982a3bd48105a7b947fd2a9b4a8ae3010041b9e0faab3f9cd028f1d" +dependencies = [ + "quote", + "wasm-bindgen-macro-support", +] + +[[package]] +name = "wasm-bindgen-macro-support" +version = "0.2.87" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "54681b18a46765f095758388f2d0cf16eb8d4169b639ab575a8f5693af210c7b" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.38", + "wasm-bindgen-backend", + "wasm-bindgen-shared", +] + +[[package]] +name = "wasm-bindgen-shared" +version = "0.2.87" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ca6ad05a4870b2bf5fe995117d3728437bd27d7cd5f06f13c17443ef369775a1" + +[[package]] +name = "web-sys" +version = "0.3.64" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9b85cbef8c220a6abc02aefd892dfc0fc23afb1c6a426316ec33253a3877249b" +dependencies = [ + "js-sys", + "wasm-bindgen", +] + +[[package]] +name = "webpki-roots" +version = "0.25.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "14247bb57be4f377dfb94c72830b8ce8fc6beac03cf4bf7b9732eadd414123fc" + +[[package]] +name = "which" +version = "4.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "87ba24419a2078cd2b0f2ede2691b6c66d8e47836da3b6db8265ebad47afbfc7" +dependencies = [ + "either", + "home", + "once_cell", + "rustix 0.38.20", +] + +[[package]] +name = "widestring" +version = "1.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "653f141f39ec16bba3c5abe400a0c60da7468261cc2cbf36805022876bc721a8" + +[[package]] +name = "wildmatch" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7f44b95f62d34113cf558c93511ac93027e03e9c29a60dd0fd70e6e025c7270a" + +[[package]] +name = "winapi" +version = "0.3.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5c839a674fcd7a98952e593242ea400abe93992746761e38641405d28b00f419" +dependencies = [ + "winapi-i686-pc-windows-gnu", + "winapi-x86_64-pc-windows-gnu", +] + +[[package]] +name = "winapi-i686-pc-windows-gnu" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ac3b87c63620426dd9b991e5ce0329eff545bccbbb34f3be09ff6fb6ab51b7b6" + +[[package]] +name = "winapi-x86_64-pc-windows-gnu" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "712e227841d057c1ee1cd2fb22fa7e5a5461ae8e48fa2ca79ec42cfc1931183f" + +[[package]] +name = "windows" +version = "0.51.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ca229916c5ee38c2f2bc1e9d8f04df975b4bd93f9955dc69fabb5d91270045c9" +dependencies = [ + "windows-core", + "windows-targets 0.48.5", +] + +[[package]] +name = "windows-core" +version = "0.51.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f1f8cf84f35d2db49a46868f947758c7a1138116f7fac3bc844f43ade1292e64" +dependencies = [ + "windows-targets 0.48.5", +] + +[[package]] +name = "windows-sys" +version = "0.45.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "75283be5efb2831d37ea142365f009c02ec203cd29a3ebecbc093d52315b66d0" +dependencies = [ + "windows-targets 0.42.2", +] + +[[package]] +name = "windows-sys" +version = "0.48.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "677d2418bec65e3338edb076e806bc1ec15693c5d0104683f2efe857f61056a9" +dependencies = [ + "windows-targets 0.48.5", +] + +[[package]] +name = "windows-targets" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8e5180c00cd44c9b1c88adb3693291f1cd93605ded80c250a75d472756b4d071" +dependencies = [ + "windows_aarch64_gnullvm 0.42.2", + "windows_aarch64_msvc 0.42.2", + "windows_i686_gnu 0.42.2", + "windows_i686_msvc 0.42.2", + "windows_x86_64_gnu 0.42.2", + "windows_x86_64_gnullvm 0.42.2", + "windows_x86_64_msvc 0.42.2", +] + +[[package]] +name = "windows-targets" +version = "0.48.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9a2fa6e2155d7247be68c096456083145c183cbbbc2764150dda45a87197940c" +dependencies = [ + "windows_aarch64_gnullvm 0.48.5", + "windows_aarch64_msvc 0.48.5", + "windows_i686_gnu 0.48.5", + "windows_i686_msvc 0.48.5", + "windows_x86_64_gnu 0.48.5", + "windows_x86_64_gnullvm 0.48.5", + "windows_x86_64_msvc 0.48.5", +] + +[[package]] +name = "windows_aarch64_gnullvm" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "597a5118570b68bc08d8d59125332c54f1ba9d9adeedeef5b99b02ba2b0698f8" + +[[package]] +name = "windows_aarch64_gnullvm" +version = "0.48.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2b38e32f0abccf9987a4e3079dfb67dcd799fb61361e53e2882c3cbaf0d905d8" + +[[package]] +name = "windows_aarch64_msvc" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e08e8864a60f06ef0d0ff4ba04124db8b0fb3be5776a5cd47641e942e58c4d43" + +[[package]] +name = "windows_aarch64_msvc" +version = "0.48.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dc35310971f3b2dbbf3f0690a219f40e2d9afcf64f9ab7cc1be722937c26b4bc" + +[[package]] +name = "windows_i686_gnu" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c61d927d8da41da96a81f029489353e68739737d3beca43145c8afec9a31a84f" + +[[package]] +name = "windows_i686_gnu" +version = "0.48.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a75915e7def60c94dcef72200b9a8e58e5091744960da64ec734a6c6e9b3743e" + +[[package]] +name = "windows_i686_msvc" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "44d840b6ec649f480a41c8d80f9c65108b92d89345dd94027bfe06ac444d1060" + +[[package]] +name = "windows_i686_msvc" +version = "0.48.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8f55c233f70c4b27f66c523580f78f1004e8b5a8b659e05a4eb49d4166cca406" + +[[package]] +name = "windows_x86_64_gnu" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8de912b8b8feb55c064867cf047dda097f92d51efad5b491dfb98f6bbb70cb36" + +[[package]] +name = "windows_x86_64_gnu" +version = "0.48.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "53d40abd2583d23e4718fddf1ebec84dbff8381c07cae67ff7768bbf19c6718e" + +[[package]] +name = "windows_x86_64_gnullvm" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "26d41b46a36d453748aedef1486d5c7a85db22e56aff34643984ea85514e94a3" + +[[package]] +name = "windows_x86_64_gnullvm" +version = "0.48.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0b7b52767868a23d5bab768e390dc5f5c55825b6d30b86c844ff2dc7414044cc" + +[[package]] +name = "windows_x86_64_msvc" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9aec5da331524158c6d1a4ac0ab1541149c0b9505fde06423b02f5ef0106b9f0" + +[[package]] +name = "windows_x86_64_msvc" +version = "0.48.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ed94fce61571a4006852b7389a063ab983c02eb1bb37b47f8272ce92d06d9538" + +[[package]] +name = "winnow" +version = "0.5.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a3b801d0e0a6726477cc207f60162da452f3a95adb368399bef20a946e06f65c" +dependencies = [ + "memchr", +] + +[[package]] +name = "winreg" +version = "0.50.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "524e57b2c537c0f9b1e69f1965311ec12182b4122e45035b1508cd24d2adadb1" +dependencies = [ + "cfg-if", + "windows-sys 0.48.0", +] + +[[package]] +name = "write16" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d1890f4022759daae28ed4fe62859b1236caebfc61ede2f63ed4e695f3f6d936" + +[[package]] +name = "writeable" +version = "0.5.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c0af0c3d13faebf8dda0b5256fa7096a2d5ccb662f7b9f54a40fe201077ab1c2" + +[[package]] +name = "wyhash" +version = "0.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "baf6e163c25e3fac820b4b453185ea2dea3b6a3e0a721d4d23d75bd33734c295" +dependencies = [ + "rand_core 0.6.4", +] + +[[package]] +name = "wyz" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "05f360fc0b24296329c78fda852a1e9ae82de9cf7b27dae4b7f62f118f77b9ed" +dependencies = [ + "tap", +] + +[[package]] +name = "xml-rs" +version = "0.8.19" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0fcb9cbac069e033553e8bb871be2fbdffcab578eb25bd0f7c508cedc6dcd75a" + +[[package]] +name = "xmltree" +version = "0.10.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d7d8a75eaf6557bb84a65ace8609883db44a29951042ada9b393151532e41fcb" +dependencies = [ + "xml-rs", +] + +[[package]] +name = "yansi" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "09041cd90cf85f7f8b2df60c646f853b7f535ce68f85244eb6731cf89fa498ec" + +[[package]] +name = "yoke" +version = "0.7.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "61e38c508604d6bbbd292dadb3c02559aa7fff6b654a078a36217cad871636e4" +dependencies = [ + "serde", + "stable_deref_trait", + "yoke-derive", + "zerofrom", +] + +[[package]] +name = "yoke-derive" +version = "0.7.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d5e19fb6ed40002bab5403ffa37e53e0e56f914a4450c8765f533018db1db35f" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.38", + "synstructure", +] + +[[package]] +name = "zerocopy" +version = "0.7.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "81ba595b9f2772fbee2312de30eeb80ec773b4cb2f1e8098db024afadda6c06f" +dependencies = [ + "zerocopy-derive", +] + +[[package]] +name = "zerocopy-derive" +version = "0.7.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "772666c41fb6dceaf520b564b962d738a8e1a83b41bd48945f50837aed78bb1d" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.38", +] + +[[package]] +name = "zerofrom" +version = "0.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "655b0814c5c0b19ade497851070c640773304939a6c0fd5f5fb43da0696d05b7" +dependencies = [ + "zerofrom-derive", +] + +[[package]] +name = "zerofrom-derive" +version = "0.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e6a647510471d372f2e6c2e6b7219e44d8c574d24fdc11c610a61455782f18c3" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.38", + "synstructure", +] + +[[package]] +name = "zeroize" +version = "1.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2a0956f1ba7c7909bfb66c2e9e4124ab6f6482560f6628b5aaeba39207c9aad9" +dependencies = [ + "zeroize_derive", +] + +[[package]] +name = "zeroize_derive" +version = "1.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ce36e65b0d2999d2aafac989fb249189a141aee1f53c612c1f37d72631959f69" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.38", +] + +[[package]] +name = "zerovec" +version = "0.9.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "591691014119b87047ead4dcf3e6adfbf73cb7c38ab6980d4f18a32138f35d46" +dependencies = [ + "serde", + "yoke", + "zerofrom", + "zerovec-derive", +] + +[[package]] +name = "zerovec-derive" +version = "0.9.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7a4a1638a1934450809c2266a70362bfc96cd90550c073f5b8a55014d1010157" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.38", +] + +[[package]] +name = "zstd" +version = "0.12.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1a27595e173641171fc74a1232b7b1c7a7cb6e18222c11e9dfb9888fa424c53c" +dependencies = [ + "zstd-safe 6.0.6", +] + +[[package]] +name = "zstd" +version = "0.13.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bffb3309596d527cfcba7dfc6ed6052f1d39dfbd7c867aa2e865e4a449c10110" +dependencies = [ + "zstd-safe 7.0.0", +] + +[[package]] +name = "zstd-safe" +version = "6.0.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ee98ffd0b48ee95e6c5168188e44a54550b1564d9d530ee21d5f0eaed1069581" +dependencies = [ + "libc", + "zstd-sys", +] + +[[package]] +name = "zstd-safe" +version = "7.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "43747c7422e2924c11144d5229878b98180ef8b06cca4ab5af37afc8a8d8ea3e" +dependencies = [ + "zstd-sys", +] + +[[package]] +name = "zstd-sys" +version = "2.0.9+zstd.1.5.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9e16efa8a874a0481a574084d34cc26fdb3b99627480f785888deb6386506656" +dependencies = [ + "cc", + "pkg-config", +] diff --git a/op-service/rethdb-reader/Cargo.toml b/op-service/rethdb-reader/Cargo.toml new file mode 100644 index 000000000000..389172a68335 --- /dev/null +++ b/op-service/rethdb-reader/Cargo.toml @@ -0,0 +1,12 @@ +[package] +name = "rethdb-reader" +description = "A simple library for reading data through Reth's DB abstractions." +version = "0.1.0" +edition = "2021" + +[lib] +name = "rethdbreader" +crate-type = ["cdylib"] + +[dependencies] +reth = { git = "https://github.com/paradigmxyz/reth.git" } diff --git a/op-service/rethdb-reader/README.md b/op-service/rethdb-reader/README.md new file mode 100644 index 000000000000..e8d9f0d26c02 --- /dev/null +++ b/op-service/rethdb-reader/README.md @@ -0,0 +1,4 @@ +# `rethdb-reader` + +Exported Rust code to be used via FFI in `op-service`'s `sources` package for reading information +directly from the `reth` database. diff --git a/op-service/rethdb-reader/src/lib.rs b/op-service/rethdb-reader/src/lib.rs new file mode 100644 index 000000000000..406df15c1e5b --- /dev/null +++ b/op-service/rethdb-reader/src/lib.rs @@ -0,0 +1,102 @@ +use reth::{ + blockchain_tree::noop::NoopBlockchainTree, + primitives::{ + alloy_primitives::private::alloy_rlp::Encodable, BlockHashOrNumber, ChainSpecBuilder, + }, + providers::{providers::BlockchainProvider, ProviderFactory, ReceiptProvider}, + utils::db::open_db_read_only, +}; +use std::{os::raw::c_char, path::Path, sync::Arc}; + +#[repr(C)] +pub struct ByteArray { + data: *mut u8, + len: usize, +} + +#[repr(C)] +pub struct ByteArrays { + data: *mut ByteArray, + len: usize, +} + +/// Read the receipts for a blockhash from the RETH database directly. +/// +/// WARNING: Will panic on error. +/// TODO: Gracefully return OK status. +#[no_mangle] +pub extern "C" fn read_receipts( + block_hash: *const u8, + block_hash_len: usize, + db_path: *const c_char, +) -> ByteArrays { + // Convert the raw pointer and length back to a Rust slice + let block_hash: [u8; 32] = unsafe { std::slice::from_raw_parts(block_hash, block_hash_len) } + .try_into() + .expect("Block hash must be 32 bytes long"); + + // Convert the *const c_char to a Rust &str + let db_path_str = unsafe { + assert!(!db_path.is_null(), "Null pointer for database path"); + std::ffi::CStr::from_ptr(db_path) + .to_str() + .expect("Invalid UTF-8 for database path") + }; + + let db = open_db_read_only(&Path::new(db_path_str), None).expect("Could not open reth DB"); + let spec = Arc::new(ChainSpecBuilder::mainnet().build()); + let factory = ProviderFactory::new(db, spec.clone()); + + // Create a read-only BlockChainProvider + let provider = BlockchainProvider::new(factory, NoopBlockchainTree::default()) + .expect("Failed to create blockchain provider."); + let receipts = provider + .receipts_by_block(BlockHashOrNumber::Hash(block_hash.into())) + .expect("Could not fetch receipts for block") + .expect("No receipts found for block"); + + // Serialize receipts to RLP for the FFI interface. + let receipts_rlp = receipts + .into_iter() + .map(|r| { + // todo - reduce alloc? + // RLP encode the receipt with a bloom filter. + let mut buf = Vec::default(); + r.with_bloom().encode(&mut buf); + + // Return a pointer to the `buf` and its length + let res = ByteArray { + data: buf.as_mut_ptr(), + len: buf.len(), + }; + + // Forget the `buf` so that its memory isn't freed by the + // borrow checker at the end of this scope + std::mem::forget(buf); + + res + }) + .collect::>(); + + let result = ByteArrays { + data: receipts_rlp.as_ptr() as *mut ByteArray, + len: receipts_rlp.len(), + }; + + // Forget the `receipts_rlp` arr so that its memory isn't freed by the + // borrow checker at the end of this scope + std::mem::forget(receipts_rlp); // Prevent Rust from freeing the memory + + result +} + +/// Free the [ByteArrays] data structure and its sub-components when they are no longer needed. +#[no_mangle] +pub extern "C" fn free_byte_arrays(array: ByteArrays) { + unsafe { + let arrays = Vec::from_raw_parts(array.data, array.len, array.len); + for inner_array in arrays { + let _ = Vec::from_raw_parts(inner_array.data, inner_array.len, inner_array.len); + } + } +} diff --git a/op-service/sources/receipts.go b/op-service/sources/receipts.go index e3c8a8713b06..8e9b80506cb0 100644 --- a/op-service/sources/receipts.go +++ b/op-service/sources/receipts.go @@ -124,6 +124,7 @@ const ( RPCKindBasic RPCProviderKind = "basic" // try only the standard most basic receipt fetching RPCKindAny RPCProviderKind = "any" // try any method available RPCKindStandard RPCProviderKind = "standard" // try standard methods, including newer optimized standard RPC methods + RPCKindRethDB RPCProviderKind = "reth_db" // read data directly from reth's MDBX database ) var RPCProviderKinds = []RPCProviderKind{ @@ -137,6 +138,7 @@ var RPCProviderKinds = []RPCProviderKind{ RPCKindBasic, RPCKindAny, RPCKindStandard, + RPCKindRethDB, } func (kind RPCProviderKind) String() string { @@ -268,6 +270,18 @@ const ( // See: // https://github.com/ledgerwatch/erigon/blob/287a3d1d6c90fc6a7a088b5ae320f93600d5a167/cmd/rpcdaemon/commands/erigon_receipts.go#LL391C24-L391C51 ErigonGetBlockReceiptsByBlockHash + // RethGetBlockReceiptsMDBX is a Reth-specific receipt fetching method. It reads the data directly from reth's database, using their + // generic DB abstractions, rather than requesting it from the RPC provider. + // Available in: + // - Reth + // Method: n/a - does not use RPC. + // Params: + // - Reth: string, hex-encoded block hash + // Returns: + // - Reth: array of RLP-encoded receipts + // See: + // - reth's DB crate documentation: https://github.com/paradigmxyz/reth/blob/main/docs/crates/db.md + RethGetBlockReceiptsMDBX // Other: // - 250 credits, not supported, strictly worse than other options. In quicknode price-table. @@ -297,12 +311,14 @@ func AvailableReceiptsFetchingMethods(kind RPCProviderKind) ReceiptsFetchingMeth case RPCKindBasic: return EthGetTransactionReceiptBatch case RPCKindAny: - // if it's any kind of RPC provider, then try all methods + // if it's any kind of RPC provider, then try all methods (except for RethGetBlockReceiptsMDBX) return AlchemyGetTransactionReceipts | EthGetBlockReceipts | DebugGetRawReceipts | ErigonGetBlockReceiptsByBlockHash | ParityGetBlockReceipts | EthGetTransactionReceiptBatch case RPCKindStandard: return EthGetBlockReceipts | EthGetTransactionReceiptBatch + case RPCKindRethDB: + return RethGetBlockReceiptsMDBX default: return EthGetTransactionReceiptBatch } @@ -313,7 +329,9 @@ func AvailableReceiptsFetchingMethods(kind RPCProviderKind) ReceiptsFetchingMeth func PickBestReceiptsFetchingMethod(kind RPCProviderKind, available ReceiptsFetchingMethod, txCount uint64) ReceiptsFetchingMethod { // If we have optimized methods available, it makes sense to use them, but only if the cost is // lower than fetching transactions one by one with the standard receipts RPC method. - if kind == RPCKindAlchemy { + if kind == RPCKindRethDB { + return RethGetBlockReceiptsMDBX + } else if kind == RPCKindAlchemy { if available&AlchemyGetTransactionReceipts != 0 && txCount > 250/15 { return AlchemyGetTransactionReceipts } @@ -460,6 +478,12 @@ func (job *receiptsFetchingJob) runAltMethod(ctx context.Context, m ReceiptsFetc err = job.client.CallContext(ctx, &result, "eth_getBlockReceipts", job.block.Hash) case ErigonGetBlockReceiptsByBlockHash: err = job.client.CallContext(ctx, &result, "erigon_getBlockReceiptsByBlockHash", job.block.Hash) + case RethGetBlockReceiptsMDBX: + res, err := FetchRethReceipts("placeholder", &job.block.Hash) + if err != nil { + return err + } + result = res default: err = fmt.Errorf("unknown receipt fetching method: %d", uint64(m)) } diff --git a/op-service/sources/reth_db.go b/op-service/sources/reth_db.go new file mode 100644 index 000000000000..160ef3d35c2a --- /dev/null +++ b/op-service/sources/reth_db.go @@ -0,0 +1,62 @@ +package sources + +import ( + "fmt" + + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/core/types" +) + +/* +#cgo LDFLAGS: -L../rethdb-reader/target/release -lrethdbreader +#include +#include + +typedef struct { + uint8_t* data; + size_t len; +} ByteArray; + +typedef struct { + ByteArray* data; + size_t len; +} ByteArrays; + +extern ByteArrays read_receipts(const uint8_t* block_hash, size_t block_hash_len, const char* db_path); +extern void free_byte_arrays(ByteArrays arrays); +*/ +import "C" +import "unsafe" + +// FetchRethReceipts fetches the receipts for the given block hash directly from the Reth Database +// and populates the given results slice pointer with the receipts that were found. +func FetchRethReceipts(dbPath string, blockHash *common.Hash) (types.Receipts, error) { + if blockHash == nil { + return nil, fmt.Errorf("Must provide a block hash to fetch receipts for.") + } + + // Convert the block hash to a C byte array and defer its deallocation + cBlockHash := C.CBytes(blockHash[:]) + defer C.free(cBlockHash) + + // Convert the db path to a C string and defer its deallocation + cDbPath := C.CString(dbPath) + defer C.free(unsafe.Pointer(cDbPath)) + + // Call the C function to fetch the receipts from the Reth Database + byteArrayStruct := C.read_receipts((*C.uint8_t)(cBlockHash), C.size_t(len(blockHash)), cDbPath) + + // Convert the returned receipt RLP byte arrays to decoded Receipts. + data := make(types.Receipts, byteArrayStruct.len) + byteArraySlice := (*[1 << 30]C.ByteArray)(unsafe.Pointer(byteArrayStruct.data))[:byteArrayStruct.len:byteArrayStruct.len] + for i, byteArray := range byteArraySlice { + receipt := types.Receipt{} + receipt.UnmarshalBinary(C.GoBytes(unsafe.Pointer(byteArray.data), C.int(byteArray.len))) + data[i] = &receipt + } + + // Free the memory allocated by the C code + C.free_byte_arrays(byteArrayStruct) + + return data, nil +} diff --git a/op-service/sources/reth_db_test.go b/op-service/sources/reth_db_test.go new file mode 100644 index 000000000000..ecb793895e0d --- /dev/null +++ b/op-service/sources/reth_db_test.go @@ -0,0 +1,22 @@ +package sources + +import ( + "testing" + + "github.com/ethereum/go-ethereum/common" + "github.com/stretchr/testify/require" +) + +func TestRethReceiptsLoad(t *testing.T) { + t.Skip("Skipping test that requires a local L1 Goerli Reth DB") + t.Parallel() + + // block = https://goerli.etherscan.io/block/994113 + blockHash := common.HexToHash("0x6f6f00553e4f74262a9812927afd11c341730c5c9210824fe172367457adb5f6") + res, err := FetchRethReceipts("/path/to/goerli-db", &blockHash) + require.NoError(t, err, "Failed to fetch receipts from Reth DB") + require.Len(t, res, 2, "Expected 2 receipts to be returned") + require.Equal(t, res[0].Type, 0) + require.Equal(t, res[0].CumulativeGasUsed, uint64(93_787)) + require.Equal(t, res[0].Status, uint64(1)) +} From bc5e060e0fccd143a400e02e7f59376d4bae79f3 Mon Sep 17 00:00:00 2001 From: clabby Date: Thu, 26 Oct 2023 21:47:59 -0400 Subject: [PATCH 149/279] Add flag to `op-node` --- op-node/flags/flags.go | 7 +++++++ op-node/node/config.go | 3 +++ op-node/node/node.go | 2 ++ op-node/service.go | 6 ++++++ op-service/sources/eth_client.go | 9 ++++++++- op-service/sources/receipts.go | 19 +++++++++++++------ 6 files changed, 39 insertions(+), 7 deletions(-) diff --git a/op-node/flags/flags.go b/op-node/flags/flags.go index c660db759acd..fa6fcd066db6 100644 --- a/op-node/flags/flags.go +++ b/op-node/flags/flags.go @@ -82,6 +82,12 @@ var ( return &out }(), } + L1RethDBPath = &cli.StringFlag{ + Name: "l1.rethdb", + Usage: "The L1 RethDB path, used to fetch receipts for L1 blocks. Only applicable when using the `reth_db` RPC kind with `l1.rpckind`.", + EnvVars: prefixEnvVars("L1_RETHDB"), + Required: false, + } L1RPCRateLimit = &cli.Float64Flag{ Name: "l1.rpc-rate-limit", Usage: "Optional self-imposed global rate-limit on L1 RPC requests, specified in requests / second. Disabled if set to 0.", @@ -304,6 +310,7 @@ var optionalFlags = []cli.Flag{ RollupHalt, RollupLoadProtocolVersions, CanyonOverrideFlag, + L1RethDBPath, } // Flags contains the list of configuration options available to the binary. diff --git a/op-node/node/config.go b/op-node/node/config.go index 6f15e7d710f9..d590bcfc675f 100644 --- a/op-node/node/config.go +++ b/op-node/node/config.go @@ -60,6 +60,9 @@ type Config struct { // Cancel to request a premature shutdown of the node itself, e.g. when halting. This may be nil. Cancel context.CancelCauseFunc + + // [OPTIONAL] The reth DB path to read receipts from + RethDBPath *string } type RPCConfig struct { diff --git a/op-node/node/node.go b/op-node/node/node.go index 594225a006be..78639fec731b 100644 --- a/op-node/node/node.go +++ b/op-node/node/node.go @@ -156,6 +156,8 @@ func (n *OpNode) initL1(ctx context.Context, cfg *Config) error { return fmt.Errorf("failed to get L1 RPC client: %w", err) } + rpcCfg.EthClientConfig.RethDBPath = cfg.RethDBPath + n.l1Source, err = sources.NewL1Client( client.NewInstrumentedRPC(l1Node, n.metrics), n.log, n.metrics.L1SourceCache, rpcCfg) if err != nil { diff --git a/op-node/service.go b/op-node/service.go index a243cba25390..2776ef180e00 100644 --- a/op-node/service.go +++ b/op-node/service.go @@ -71,6 +71,11 @@ func NewConfig(ctx *cli.Context, log log.Logger) (*node.Config, error) { haltOption = "" } + var rethDBPath *string + if rdb := ctx.String(flags.L1RethDBPath.Name); rdb != "" { + rethDBPath = &rdb + } + cfg := &node.Config{ L1: l1Endpoint, L2: l2Endpoint, @@ -104,6 +109,7 @@ func NewConfig(ctx *cli.Context, log log.Logger) (*node.Config, error) { ConfigPersistence: configPersistence, Sync: *syncConfig, RollupHalt: haltOption, + RethDBPath: rethDBPath, } if err := cfg.LoadPersisted(log); err != nil { diff --git a/op-service/sources/eth_client.go b/op-service/sources/eth_client.go index 553615857908..b679427f5d68 100644 --- a/op-service/sources/eth_client.go +++ b/op-service/sources/eth_client.go @@ -62,6 +62,9 @@ type EthClientConfig struct { // till we re-attempt the user-preferred methods. // If this is 0 then the client does not fall back to less optimal but available methods. MethodResetDuration time.Duration + + // [OPTIONAL] The reth DB path to fetch receipts from + RethDBPath *string } func (c *EthClientConfig) Check() error { @@ -132,6 +135,9 @@ type EthClient struct { // methodResetDuration defines how long we take till we reset lastMethodsReset methodResetDuration time.Duration + + // [OPTIONAL] The reth DB path to fetch receipts from + rethDbPath *string } func (s *EthClient) PickReceiptsMethod(txCount uint64) ReceiptsFetchingMethod { @@ -179,6 +185,7 @@ func NewEthClient(client client.RPC, log log.Logger, metrics caching.Metrics, co availableReceiptMethods: AvailableReceiptsFetchingMethods(config.RPCProviderKind), lastMethodsReset: time.Now(), methodResetDuration: config.MethodResetDuration, + rethDbPath: config.RethDBPath, }, nil } @@ -357,7 +364,7 @@ func (s *EthClient) FetchReceipts(ctx context.Context, blockHash common.Hash) (e job = v } else { txHashes := eth.TransactionsToHashes(txs) - job = NewReceiptsFetchingJob(s, s.client, s.maxBatchSize, eth.ToBlockID(info), info.ReceiptHash(), txHashes) + job = NewReceiptsFetchingJob(s, s.client, s.maxBatchSize, eth.ToBlockID(info), info.ReceiptHash(), txHashes, s.rethDbPath) s.receiptsCache.Add(blockHash, job) } receipts, err := job.Fetch(ctx) diff --git a/op-service/sources/receipts.go b/op-service/sources/receipts.go index 8e9b80506cb0..4c631797051d 100644 --- a/op-service/sources/receipts.go +++ b/op-service/sources/receipts.go @@ -281,7 +281,7 @@ const ( // - Reth: array of RLP-encoded receipts // See: // - reth's DB crate documentation: https://github.com/paradigmxyz/reth/blob/main/docs/crates/db.md - RethGetBlockReceiptsMDBX + RethGetBlockReceipts // Other: // - 250 credits, not supported, strictly worse than other options. In quicknode price-table. @@ -318,7 +318,7 @@ func AvailableReceiptsFetchingMethods(kind RPCProviderKind) ReceiptsFetchingMeth case RPCKindStandard: return EthGetBlockReceipts | EthGetTransactionReceiptBatch case RPCKindRethDB: - return RethGetBlockReceiptsMDBX + return RethGetBlockReceipts default: return EthGetTransactionReceiptBatch } @@ -330,7 +330,7 @@ func PickBestReceiptsFetchingMethod(kind RPCProviderKind, available ReceiptsFetc // If we have optimized methods available, it makes sense to use them, but only if the cost is // lower than fetching transactions one by one with the standard receipts RPC method. if kind == RPCKindRethDB { - return RethGetBlockReceiptsMDBX + return RethGetBlockReceipts } else if kind == RPCKindAlchemy { if available&AlchemyGetTransactionReceipts != 0 && txCount > 250/15 { return AlchemyGetTransactionReceipts @@ -389,11 +389,14 @@ type receiptsFetchingJob struct { fetcher *IterativeBatchCall[common.Hash, *types.Receipt] + // [OPTIONAL] RethDB path to fetch receipts from + rethDbPath *string + result types.Receipts } func NewReceiptsFetchingJob(requester ReceiptsRequester, client rpcClient, maxBatchSize int, block eth.BlockID, - receiptHash common.Hash, txHashes []common.Hash) *receiptsFetchingJob { + receiptHash common.Hash, txHashes []common.Hash, rethDb *string) *receiptsFetchingJob { return &receiptsFetchingJob{ requester: requester, client: client, @@ -401,6 +404,7 @@ func NewReceiptsFetchingJob(requester ReceiptsRequester, client rpcClient, maxBa block: block, receiptHash: receiptHash, txHashes: txHashes, + rethDbPath: rethDb, } } @@ -478,8 +482,11 @@ func (job *receiptsFetchingJob) runAltMethod(ctx context.Context, m ReceiptsFetc err = job.client.CallContext(ctx, &result, "eth_getBlockReceipts", job.block.Hash) case ErigonGetBlockReceiptsByBlockHash: err = job.client.CallContext(ctx, &result, "erigon_getBlockReceiptsByBlockHash", job.block.Hash) - case RethGetBlockReceiptsMDBX: - res, err := FetchRethReceipts("placeholder", &job.block.Hash) + case RethGetBlockReceipts: + if job.rethDbPath == nil { + return fmt.Errorf("reth_db path not set") + } + res, err := FetchRethReceipts(*job.rethDbPath, &job.block.Hash) if err != nil { return err } From 0d752667667a3d8bc796d40630d23ba8c6650e1e Mon Sep 17 00:00:00 2001 From: clabby Date: Thu, 26 Oct 2023 22:33:14 -0400 Subject: [PATCH 150/279] Add error handling in dylib --- op-service/rethdb-reader/src/lib.rs | 136 ++++++++++++++++++---------- op-service/sources/receipts.go | 4 +- op-service/sources/reth_db.go | 19 +++- 3 files changed, 106 insertions(+), 53 deletions(-) diff --git a/op-service/rethdb-reader/src/lib.rs b/op-service/rethdb-reader/src/lib.rs index 406df15c1e5b..b4af373171ec 100644 --- a/op-service/rethdb-reader/src/lib.rs +++ b/op-service/rethdb-reader/src/lib.rs @@ -20,6 +20,38 @@ pub struct ByteArrays { len: usize, } +#[repr(C)] +pub struct ReceiptsResult { + receipts: ByteArrays, + error: bool, +} + +// Implement a default for ByteArrays to be used in error cases +impl Default for ByteArrays { + fn default() -> Self { + ByteArrays { + data: std::ptr::null_mut(), + len: 0, + } + } +} + +impl ReceiptsResult { + pub fn success(receipts: ByteArrays) -> Self { + Self { + receipts, + error: false, + } + } + + pub fn fail() -> Self { + Self { + receipts: ByteArrays::default(), + error: true, + } + } +} + /// Read the receipts for a blockhash from the RETH database directly. /// /// WARNING: Will panic on error. @@ -29,72 +61,82 @@ pub extern "C" fn read_receipts( block_hash: *const u8, block_hash_len: usize, db_path: *const c_char, -) -> ByteArrays { +) -> ReceiptsResult { // Convert the raw pointer and length back to a Rust slice - let block_hash: [u8; 32] = unsafe { std::slice::from_raw_parts(block_hash, block_hash_len) } - .try_into() - .expect("Block hash must be 32 bytes long"); + let Ok(block_hash): Result<[u8; 32], _> = + unsafe { std::slice::from_raw_parts(block_hash, block_hash_len) }.try_into() + else { + return ReceiptsResult::fail(); + }; // Convert the *const c_char to a Rust &str - let db_path_str = unsafe { + let Ok(db_path_str) = unsafe { assert!(!db_path.is_null(), "Null pointer for database path"); std::ffi::CStr::from_ptr(db_path) - .to_str() - .expect("Invalid UTF-8 for database path") + } + .to_str() else { + return ReceiptsResult::fail(); }; - let db = open_db_read_only(&Path::new(db_path_str), None).expect("Could not open reth DB"); + let Ok(db) = open_db_read_only(&Path::new(db_path_str), None) else { + return ReceiptsResult::fail(); + }; let spec = Arc::new(ChainSpecBuilder::mainnet().build()); let factory = ProviderFactory::new(db, spec.clone()); // Create a read-only BlockChainProvider - let provider = BlockchainProvider::new(factory, NoopBlockchainTree::default()) - .expect("Failed to create blockchain provider."); - let receipts = provider - .receipts_by_block(BlockHashOrNumber::Hash(block_hash.into())) - .expect("Could not fetch receipts for block") - .expect("No receipts found for block"); - - // Serialize receipts to RLP for the FFI interface. - let receipts_rlp = receipts - .into_iter() - .map(|r| { - // todo - reduce alloc? - // RLP encode the receipt with a bloom filter. - let mut buf = Vec::default(); - r.with_bloom().encode(&mut buf); - - // Return a pointer to the `buf` and its length - let res = ByteArray { - data: buf.as_mut_ptr(), - len: buf.len(), - }; - - // Forget the `buf` so that its memory isn't freed by the - // borrow checker at the end of this scope - std::mem::forget(buf); - - res - }) - .collect::>(); - - let result = ByteArrays { - data: receipts_rlp.as_ptr() as *mut ByteArray, - len: receipts_rlp.len(), + let Ok(provider) = BlockchainProvider::new(factory, NoopBlockchainTree::default()) else { + return ReceiptsResult::fail(); + }; + + let Ok(receipts) = provider.receipts_by_block(BlockHashOrNumber::Hash(block_hash.into())) + else { + return ReceiptsResult::fail(); }; - // Forget the `receipts_rlp` arr so that its memory isn't freed by the - // borrow checker at the end of this scope - std::mem::forget(receipts_rlp); // Prevent Rust from freeing the memory + if let Some(receipts) = receipts { + let receipts_rlp = receipts + .into_iter() + .map(|r| { + // todo - reduce alloc? + // RLP encode the receipt with a bloom filter. + let mut buf = Vec::default(); + r.with_bloom().encode(&mut buf); - result + // Return a pointer to the `buf` and its length + let res = ByteArray { + data: buf.as_mut_ptr(), + len: buf.len(), + }; + + // Forget the `buf` so that its memory isn't freed by the + // borrow checker at the end of this scope + std::mem::forget(buf); + + res + }) + .collect::>(); + + let result = ByteArrays { + data: receipts_rlp.as_ptr() as *mut ByteArray, + len: receipts_rlp.len(), + }; + + // Forget the `receipts_rlp` arr so that its memory isn't freed by the + // borrow checker at the end of this scope + std::mem::forget(receipts_rlp); // Prevent Rust from freeing the memory + + ReceiptsResult::success(result) + } else { + return ReceiptsResult::fail(); + } } /// Free the [ByteArrays] data structure and its sub-components when they are no longer needed. #[no_mangle] -pub extern "C" fn free_byte_arrays(array: ByteArrays) { +pub extern "C" fn free_byte_arrays(arrays: ByteArrays) { unsafe { - let arrays = Vec::from_raw_parts(array.data, array.len, array.len); + let arrays = Vec::from_raw_parts(arrays.data, arrays.len, arrays.len); for inner_array in arrays { let _ = Vec::from_raw_parts(inner_array.data, inner_array.len, inner_array.len); } diff --git a/op-service/sources/receipts.go b/op-service/sources/receipts.go index 4c631797051d..624379129707 100644 --- a/op-service/sources/receipts.go +++ b/op-service/sources/receipts.go @@ -124,7 +124,7 @@ const ( RPCKindBasic RPCProviderKind = "basic" // try only the standard most basic receipt fetching RPCKindAny RPCProviderKind = "any" // try any method available RPCKindStandard RPCProviderKind = "standard" // try standard methods, including newer optimized standard RPC methods - RPCKindRethDB RPCProviderKind = "reth_db" // read data directly from reth's MDBX database + RPCKindRethDB RPCProviderKind = "reth_db" // read data directly from reth's database ) var RPCProviderKinds = []RPCProviderKind{ @@ -311,7 +311,7 @@ func AvailableReceiptsFetchingMethods(kind RPCProviderKind) ReceiptsFetchingMeth case RPCKindBasic: return EthGetTransactionReceiptBatch case RPCKindAny: - // if it's any kind of RPC provider, then try all methods (except for RethGetBlockReceiptsMDBX) + // if it's any kind of RPC provider, then try all methods (except for RethGetBlockReceipts) return AlchemyGetTransactionReceipts | EthGetBlockReceipts | DebugGetRawReceipts | ErigonGetBlockReceiptsByBlockHash | ParityGetBlockReceipts | EthGetTransactionReceiptBatch diff --git a/op-service/sources/reth_db.go b/op-service/sources/reth_db.go index 160ef3d35c2a..58a4aebe0f5a 100644 --- a/op-service/sources/reth_db.go +++ b/op-service/sources/reth_db.go @@ -11,6 +11,7 @@ import ( #cgo LDFLAGS: -L../rethdb-reader/target/release -lrethdbreader #include #include +#include typedef struct { uint8_t* data; @@ -22,7 +23,13 @@ typedef struct { size_t len; } ByteArrays; -extern ByteArrays read_receipts(const uint8_t* block_hash, size_t block_hash_len, const char* db_path); +// Define ReceiptsResult with a bool for error +typedef struct { + ByteArrays receipts; + bool error; +} ReceiptsResult; + +extern ReceiptsResult read_receipts(const uint8_t* block_hash, size_t block_hash_len, const char* db_path); extern void free_byte_arrays(ByteArrays arrays); */ import "C" @@ -46,9 +53,13 @@ func FetchRethReceipts(dbPath string, blockHash *common.Hash) (types.Receipts, e // Call the C function to fetch the receipts from the Reth Database byteArrayStruct := C.read_receipts((*C.uint8_t)(cBlockHash), C.size_t(len(blockHash)), cDbPath) + if byteArrayStruct.error { + return nil, fmt.Errorf("Error fetching receipts from Reth Database.") + } + // Convert the returned receipt RLP byte arrays to decoded Receipts. - data := make(types.Receipts, byteArrayStruct.len) - byteArraySlice := (*[1 << 30]C.ByteArray)(unsafe.Pointer(byteArrayStruct.data))[:byteArrayStruct.len:byteArrayStruct.len] + data := make(types.Receipts, byteArrayStruct.receipts.len) + byteArraySlice := (*[1 << 30]C.ByteArray)(unsafe.Pointer(byteArrayStruct.receipts.data))[:byteArrayStruct.receipts.len:byteArrayStruct.receipts.len] for i, byteArray := range byteArraySlice { receipt := types.Receipt{} receipt.UnmarshalBinary(C.GoBytes(unsafe.Pointer(byteArray.data), C.int(byteArray.len))) @@ -56,7 +67,7 @@ func FetchRethReceipts(dbPath string, blockHash *common.Hash) (types.Receipts, e } // Free the memory allocated by the C code - C.free_byte_arrays(byteArrayStruct) + C.free_byte_arrays(byteArrayStruct.receipts) return data, nil } From 7d603267486a6bb037104791815b3049e19a083e Mon Sep 17 00:00:00 2001 From: clabby Date: Thu, 26 Oct 2023 23:07:08 -0400 Subject: [PATCH 151/279] test: hydrated receipts --- op-service/rethdb-reader/Cargo.lock | 2 + op-service/rethdb-reader/Cargo.toml | 2 + op-service/rethdb-reader/src/lib.rs | 193 +++++++++++++++++++--------- op-service/sources/reth_db.go | 40 ++---- 4 files changed, 152 insertions(+), 85 deletions(-) diff --git a/op-service/rethdb-reader/Cargo.lock b/op-service/rethdb-reader/Cargo.lock index f0fe36055823..ef918a166b1f 100644 --- a/op-service/rethdb-reader/Cargo.lock +++ b/op-service/rethdb-reader/Cargo.lock @@ -5361,6 +5361,8 @@ name = "rethdb-reader" version = "0.1.0" dependencies = [ "reth", + "serde", + "serde_json", ] [[package]] diff --git a/op-service/rethdb-reader/Cargo.toml b/op-service/rethdb-reader/Cargo.toml index 389172a68335..ac53bea511b8 100644 --- a/op-service/rethdb-reader/Cargo.toml +++ b/op-service/rethdb-reader/Cargo.toml @@ -10,3 +10,5 @@ crate-type = ["cdylib"] [dependencies] reth = { git = "https://github.com/paradigmxyz/reth.git" } +serde = "1.0.190" +serde_json = "1.0.107" diff --git a/op-service/rethdb-reader/src/lib.rs b/op-service/rethdb-reader/src/lib.rs index b4af373171ec..9eb2442a3159 100644 --- a/op-service/rethdb-reader/src/lib.rs +++ b/op-service/rethdb-reader/src/lib.rs @@ -1,52 +1,36 @@ use reth::{ blockchain_tree::noop::NoopBlockchainTree, primitives::{ - alloy_primitives::private::alloy_rlp::Encodable, BlockHashOrNumber, ChainSpecBuilder, + BlockHashOrNumber, ChainSpecBuilder, Receipt, TransactionMeta, TransactionSigned, U128, + U256, U64, }, - providers::{providers::BlockchainProvider, ProviderFactory, ReceiptProvider}, + providers::{providers::BlockchainProvider, BlockReader, ProviderFactory, ReceiptProvider}, + revm::primitives::calc_blob_gasprice, + rpc::types::{Log, TransactionReceipt}, utils::db::open_db_read_only, }; use std::{os::raw::c_char, path::Path, sync::Arc}; -#[repr(C)] -pub struct ByteArray { - data: *mut u8, - len: usize, -} - -#[repr(C)] -pub struct ByteArrays { - data: *mut ByteArray, - len: usize, -} - #[repr(C)] pub struct ReceiptsResult { - receipts: ByteArrays, + data: *mut char, + data_len: usize, error: bool, } -// Implement a default for ByteArrays to be used in error cases -impl Default for ByteArrays { - fn default() -> Self { - ByteArrays { - data: std::ptr::null_mut(), - len: 0, - } - } -} - impl ReceiptsResult { - pub fn success(receipts: ByteArrays) -> Self { + pub fn success(data: *mut char, data_len: usize) -> Self { Self { - receipts, + data, + data_len, error: false, } } pub fn fail() -> Self { Self { - receipts: ByteArrays::default(), + data: std::ptr::null_mut(), + data_len: 0, error: true, } } @@ -89,56 +73,147 @@ pub extern "C" fn read_receipts( return ReceiptsResult::fail(); }; + let Ok(block) = provider.block_by_hash(block_hash.into()) else { + return ReceiptsResult::fail(); + }; + let Ok(receipts) = provider.receipts_by_block(BlockHashOrNumber::Hash(block_hash.into())) else { return ReceiptsResult::fail(); }; - if let Some(receipts) = receipts { - let receipts_rlp = receipts + if let (Some(block), Some(receipts)) = (block, receipts) { + let block_number = block.number; + let base_fee = block.base_fee_per_gas; + let block_hash = block.hash_slow(); + let excess_blob_gas = block.excess_blob_gas; + let Some(receipts) = block + .body .into_iter() - .map(|r| { - // todo - reduce alloc? - // RLP encode the receipt with a bloom filter. - let mut buf = Vec::default(); - r.with_bloom().encode(&mut buf); - - // Return a pointer to the `buf` and its length - let res = ByteArray { - data: buf.as_mut_ptr(), - len: buf.len(), + .zip(receipts.clone()) + .enumerate() + .map(|(idx, (tx, receipt))| { + let meta = TransactionMeta { + tx_hash: tx.hash, + index: idx as u64, + block_hash, + block_number, + base_fee, + excess_blob_gas, }; - - // Forget the `buf` so that its memory isn't freed by the - // borrow checker at the end of this scope - std::mem::forget(buf); - - res + build_transaction_receipt_with_block_receipts(tx, meta, receipt, &receipts) }) - .collect::>(); + .collect::>>() + else { + return ReceiptsResult::fail(); + }; - let result = ByteArrays { - data: receipts_rlp.as_ptr() as *mut ByteArray, - len: receipts_rlp.len(), + // Convert the receipts to JSON for transport + let Ok(mut receipts_json) = serde_json::to_string(&receipts) else { + return ReceiptsResult::fail(); }; - // Forget the `receipts_rlp` arr so that its memory isn't freed by the + let res = + ReceiptsResult::success(receipts_json.as_mut_ptr() as *mut char, receipts_json.len()); + + // Forget the `receipts_json` string so that its memory isn't freed by the // borrow checker at the end of this scope - std::mem::forget(receipts_rlp); // Prevent Rust from freeing the memory + std::mem::forget(receipts_json); // Prevent Rust from freeing the memory - ReceiptsResult::success(result) + res } else { - return ReceiptsResult::fail(); + ReceiptsResult::fail() } } -/// Free the [ByteArrays] data structure and its sub-components when they are no longer needed. +/// Free a string that was allocated in Rust and passed to C. #[no_mangle] -pub extern "C" fn free_byte_arrays(arrays: ByteArrays) { +pub extern "C" fn free_string(string: *mut c_char) { unsafe { - let arrays = Vec::from_raw_parts(arrays.data, arrays.len, arrays.len); - for inner_array in arrays { - let _ = Vec::from_raw_parts(inner_array.data, inner_array.len, inner_array.len); + // Convert the raw pointer back to a CString and let it go out of scope, + // which will deallocate the memory. + if !string.is_null() { + let _ = std::ffi::CString::from_raw(string); + } + } +} + +pub(crate) fn build_transaction_receipt_with_block_receipts( + tx: TransactionSigned, + meta: TransactionMeta, + receipt: Receipt, + all_receipts: &[Receipt], +) -> Option { + let transaction = tx.clone().into_ecrecovered()?; + + // get the previous transaction cumulative gas used + let gas_used = if meta.index == 0 { + receipt.cumulative_gas_used + } else { + let prev_tx_idx = (meta.index - 1) as usize; + all_receipts + .get(prev_tx_idx) + .map(|prev_receipt| receipt.cumulative_gas_used - prev_receipt.cumulative_gas_used) + .unwrap_or_default() + }; + + let mut res_receipt = TransactionReceipt { + transaction_hash: Some(meta.tx_hash), + transaction_index: U64::from(meta.index), + block_hash: Some(meta.block_hash), + block_number: Some(U256::from(meta.block_number)), + from: transaction.signer(), + to: None, + cumulative_gas_used: U256::from(receipt.cumulative_gas_used), + gas_used: Some(U256::from(gas_used)), + contract_address: None, + logs: Vec::with_capacity(receipt.logs.len()), + effective_gas_price: U128::from(transaction.effective_gas_price(meta.base_fee)), + transaction_type: tx.transaction.tx_type().into(), + // TODO pre-byzantium receipts have a post-transaction state root + state_root: None, + logs_bloom: receipt.bloom_slow(), + status_code: if receipt.success { + Some(U64::from(1)) + } else { + Some(U64::from(0)) + }, + + // EIP-4844 fields + blob_gas_price: meta.excess_blob_gas.map(calc_blob_gasprice).map(U128::from), + blob_gas_used: transaction.transaction.blob_gas_used().map(U128::from), + }; + + match tx.transaction.kind() { + reth::primitives::TransactionKind::Create => { + res_receipt.contract_address = + Some(transaction.signer().create(tx.transaction.nonce())); + } + reth::primitives::TransactionKind::Call(addr) => { + res_receipt.to = Some(*addr); } } + + // get number of logs in the block + let mut num_logs = 0; + for prev_receipt in all_receipts.iter().take(meta.index as usize) { + num_logs += prev_receipt.logs.len(); + } + + for (tx_log_idx, log) in receipt.logs.into_iter().enumerate() { + let rpclog = Log { + address: log.address, + topics: log.topics, + data: log.data, + block_hash: Some(meta.block_hash), + block_number: Some(U256::from(meta.block_number)), + transaction_hash: Some(meta.tx_hash), + transaction_index: Some(U256::from(meta.index)), + log_index: Some(U256::from(num_logs + tx_log_idx)), + removed: false, + }; + res_receipt.logs.push(rpclog); + } + + Some(res_receipt) } diff --git a/op-service/sources/reth_db.go b/op-service/sources/reth_db.go index 58a4aebe0f5a..a7c1da5f9dff 100644 --- a/op-service/sources/reth_db.go +++ b/op-service/sources/reth_db.go @@ -1,8 +1,11 @@ package sources import ( + "encoding/json" "fmt" + "unsafe" + "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/core/types" ) @@ -14,26 +17,15 @@ import ( #include typedef struct { - uint8_t* data; - size_t len; -} ByteArray; - -typedef struct { - ByteArray* data; - size_t len; -} ByteArrays; - -// Define ReceiptsResult with a bool for error -typedef struct { - ByteArrays receipts; + char* data; + size_t data_len; bool error; } ReceiptsResult; extern ReceiptsResult read_receipts(const uint8_t* block_hash, size_t block_hash_len, const char* db_path); -extern void free_byte_arrays(ByteArrays arrays); +extern void free_string(char* string); */ import "C" -import "unsafe" // FetchRethReceipts fetches the receipts for the given block hash directly from the Reth Database // and populates the given results slice pointer with the receipts that were found. @@ -51,23 +43,19 @@ func FetchRethReceipts(dbPath string, blockHash *common.Hash) (types.Receipts, e defer C.free(unsafe.Pointer(cDbPath)) // Call the C function to fetch the receipts from the Reth Database - byteArrayStruct := C.read_receipts((*C.uint8_t)(cBlockHash), C.size_t(len(blockHash)), cDbPath) + receiptsResult := C.read_receipts((*C.uint8_t)(cBlockHash), C.size_t(len(blockHash)), cDbPath) - if byteArrayStruct.error { + if receiptsResult.error { return nil, fmt.Errorf("Error fetching receipts from Reth Database.") } - // Convert the returned receipt RLP byte arrays to decoded Receipts. - data := make(types.Receipts, byteArrayStruct.receipts.len) - byteArraySlice := (*[1 << 30]C.ByteArray)(unsafe.Pointer(byteArrayStruct.receipts.data))[:byteArrayStruct.receipts.len:byteArrayStruct.receipts.len] - for i, byteArray := range byteArraySlice { - receipt := types.Receipt{} - receipt.UnmarshalBinary(C.GoBytes(unsafe.Pointer(byteArray.data), C.int(byteArray.len))) - data[i] = &receipt - } + // Convert the returned JSON string to Go string and parse it + receiptsJSON := C.GoStringN(receiptsResult.data, C.int(receiptsResult.data_len)) + var receipts types.Receipts + json.Unmarshal([]byte(receiptsJSON), &receipts) // Free the memory allocated by the C code - C.free_byte_arrays(byteArrayStruct.receipts) + C.free_string(receiptsResult.data) - return data, nil + return receipts, nil } From 4aacef463858f97af7f0853ed2e863773a2d3459 Mon Sep 17 00:00:00 2001 From: clabby Date: Thu, 26 Oct 2023 23:32:12 -0400 Subject: [PATCH 152/279] :broom: --- op-node/node/node.go | 1 + op-service/rethdb-reader/src/lib.rs | 3 ++- op-service/sources/reth_db.go | 4 +++- op-service/sources/reth_db_test.go | 22 ---------------------- 4 files changed, 6 insertions(+), 24 deletions(-) delete mode 100644 op-service/sources/reth_db_test.go diff --git a/op-node/node/node.go b/op-node/node/node.go index 78639fec731b..c2bcedb0b893 100644 --- a/op-node/node/node.go +++ b/op-node/node/node.go @@ -156,6 +156,7 @@ func (n *OpNode) initL1(ctx context.Context, cfg *Config) error { return fmt.Errorf("failed to get L1 RPC client: %w", err) } + // Set the RethDB path in the EthClientConfig, if there is one configured. rpcCfg.EthClientConfig.RethDBPath = cfg.RethDBPath n.l1Source, err = sources.NewL1Client( diff --git a/op-service/rethdb-reader/src/lib.rs b/op-service/rethdb-reader/src/lib.rs index 9eb2442a3159..0d94747ddbd4 100644 --- a/op-service/rethdb-reader/src/lib.rs +++ b/op-service/rethdb-reader/src/lib.rs @@ -138,7 +138,8 @@ pub extern "C" fn free_string(string: *mut c_char) { } } -pub(crate) fn build_transaction_receipt_with_block_receipts( +#[inline(always)] +fn build_transaction_receipt_with_block_receipts( tx: TransactionSigned, meta: TransactionMeta, receipt: Receipt, diff --git a/op-service/sources/reth_db.go b/op-service/sources/reth_db.go index a7c1da5f9dff..d507e2f45f38 100644 --- a/op-service/sources/reth_db.go +++ b/op-service/sources/reth_db.go @@ -52,7 +52,9 @@ func FetchRethReceipts(dbPath string, blockHash *common.Hash) (types.Receipts, e // Convert the returned JSON string to Go string and parse it receiptsJSON := C.GoStringN(receiptsResult.data, C.int(receiptsResult.data_len)) var receipts types.Receipts - json.Unmarshal([]byte(receiptsJSON), &receipts) + if err := json.Unmarshal([]byte(receiptsJSON), &receipts); err != nil { + return nil, err + } // Free the memory allocated by the C code C.free_string(receiptsResult.data) diff --git a/op-service/sources/reth_db_test.go b/op-service/sources/reth_db_test.go deleted file mode 100644 index ecb793895e0d..000000000000 --- a/op-service/sources/reth_db_test.go +++ /dev/null @@ -1,22 +0,0 @@ -package sources - -import ( - "testing" - - "github.com/ethereum/go-ethereum/common" - "github.com/stretchr/testify/require" -) - -func TestRethReceiptsLoad(t *testing.T) { - t.Skip("Skipping test that requires a local L1 Goerli Reth DB") - t.Parallel() - - // block = https://goerli.etherscan.io/block/994113 - blockHash := common.HexToHash("0x6f6f00553e4f74262a9812927afd11c341730c5c9210824fe172367457adb5f6") - res, err := FetchRethReceipts("/path/to/goerli-db", &blockHash) - require.NoError(t, err, "Failed to fetch receipts from Reth DB") - require.Len(t, res, 2, "Expected 2 receipts to be returned") - require.Equal(t, res[0].Type, 0) - require.Equal(t, res[0].CumulativeGasUsed, uint64(93_787)) - require.Equal(t, res[0].Status, uint64(1)) -} From 0a25b054e0f206f0d13e1b9ed3137ab6417d495f Mon Sep 17 00:00:00 2001 From: clabby Date: Thu, 26 Oct 2023 23:39:05 -0400 Subject: [PATCH 153/279] Stub out rethdb receipt fetcher with `rethdb` build tag --- op-service/sources/reth_db.go | 2 ++ op-service/sources/reth_db_stub.go | 13 +++++++++++++ 2 files changed, 15 insertions(+) create mode 100644 op-service/sources/reth_db_stub.go diff --git a/op-service/sources/reth_db.go b/op-service/sources/reth_db.go index d507e2f45f38..71b8d2a78c9f 100644 --- a/op-service/sources/reth_db.go +++ b/op-service/sources/reth_db.go @@ -1,3 +1,5 @@ +//go:build rethdb + package sources import ( diff --git a/op-service/sources/reth_db_stub.go b/op-service/sources/reth_db_stub.go new file mode 100644 index 000000000000..d55db52ef851 --- /dev/null +++ b/op-service/sources/reth_db_stub.go @@ -0,0 +1,13 @@ +//go:build !rethdb + +package sources + +import ( + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/core/types" +) + +// FetchRethReceipts fetches the receipts for the given block hash... +func FetchRethReceipts(dbPath string, blockHash *common.Hash) (types.Receipts, error) { + panic("unimplemented!") +} From 7f12a6dc49baead76cc64e227fb2c289febfa4d5 Mon Sep 17 00:00:00 2001 From: clabby Date: Fri, 27 Oct 2023 00:17:07 -0400 Subject: [PATCH 154/279] Update op-service/sources/reth_db_stub.go --- op-service/sources/reth_db_stub.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/op-service/sources/reth_db_stub.go b/op-service/sources/reth_db_stub.go index d55db52ef851..be685a1d5a9c 100644 --- a/op-service/sources/reth_db_stub.go +++ b/op-service/sources/reth_db_stub.go @@ -7,7 +7,7 @@ import ( "github.com/ethereum/go-ethereum/core/types" ) -// FetchRethReceipts fetches the receipts for the given block hash... +// FetchRethReceipts stub; Not available without `rethdb` build tag. func FetchRethReceipts(dbPath string, blockHash *common.Hash) (types.Receipts, error) { - panic("unimplemented!") + panic("unimplemented! Did you forget to enable the `rethdb` build tag?") } From b2bc54308eac4c0174fc3863e76016e9ab5842f8 Mon Sep 17 00:00:00 2001 From: clabby Date: Fri, 27 Oct 2023 00:17:43 -0400 Subject: [PATCH 155/279] update comment --- op-service/sources/receipts.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/op-service/sources/receipts.go b/op-service/sources/receipts.go index 624379129707..2d60bb959716 100644 --- a/op-service/sources/receipts.go +++ b/op-service/sources/receipts.go @@ -278,7 +278,7 @@ const ( // Params: // - Reth: string, hex-encoded block hash // Returns: - // - Reth: array of RLP-encoded receipts + // - Reth: string, json-ified receipts // See: // - reth's DB crate documentation: https://github.com/paradigmxyz/reth/blob/main/docs/crates/db.md RethGetBlockReceipts From 51da1fcf3fb8c988116577e99f27d543b6b7fa31 Mon Sep 17 00:00:00 2001 From: clabby Date: Fri, 27 Oct 2023 01:08:37 -0400 Subject: [PATCH 156/279] :broom: --- op-service/rethdb-reader/src/lib.rs | 63 ++++++++++++++++------------- 1 file changed, 34 insertions(+), 29 deletions(-) diff --git a/op-service/rethdb-reader/src/lib.rs b/op-service/rethdb-reader/src/lib.rs index 0d94747ddbd4..c7852000115a 100644 --- a/op-service/rethdb-reader/src/lib.rs +++ b/op-service/rethdb-reader/src/lib.rs @@ -1,15 +1,14 @@ use reth::{ blockchain_tree::noop::NoopBlockchainTree, primitives::{ - BlockHashOrNumber, ChainSpecBuilder, Receipt, TransactionMeta, TransactionSigned, U128, - U256, U64, + BlockHashOrNumber, Receipt, TransactionKind, TransactionMeta, TransactionSigned, MAINNET, + U128, U256, U64, }, providers::{providers::BlockchainProvider, BlockReader, ProviderFactory, ReceiptProvider}, - revm::primitives::calc_blob_gasprice, rpc::types::{Log, TransactionReceipt}, utils::db::open_db_read_only, }; -use std::{os::raw::c_char, path::Path, sync::Arc}; +use std::{os::raw::c_char, path::Path}; #[repr(C)] pub struct ReceiptsResult { @@ -38,45 +37,51 @@ impl ReceiptsResult { /// Read the receipts for a blockhash from the RETH database directly. /// -/// WARNING: Will panic on error. -/// TODO: Gracefully return OK status. +/// # Safety +/// - All possible nil pointer dereferences are checked, and the function will return a +/// failing [ReceiptsResult] if any are found. #[no_mangle] -pub extern "C" fn read_receipts( +pub unsafe extern "C" fn read_receipts( block_hash: *const u8, block_hash_len: usize, db_path: *const c_char, ) -> ReceiptsResult { // Convert the raw pointer and length back to a Rust slice - let Ok(block_hash): Result<[u8; 32], _> = - unsafe { std::slice::from_raw_parts(block_hash, block_hash_len) }.try_into() - else { + let Ok(block_hash): Result<[u8; 32], _> = { + if block_hash.is_null() { + return ReceiptsResult::fail(); + } + std::slice::from_raw_parts(block_hash, block_hash_len) + } + .try_into() else { return ReceiptsResult::fail(); }; // Convert the *const c_char to a Rust &str - let Ok(db_path_str) = unsafe { - assert!(!db_path.is_null(), "Null pointer for database path"); + let Ok(db_path_str) = { + if db_path.is_null() { + return ReceiptsResult::fail(); + } std::ffi::CStr::from_ptr(db_path) } .to_str() else { return ReceiptsResult::fail(); }; - let Ok(db) = open_db_read_only(&Path::new(db_path_str), None) else { + let Ok(db) = open_db_read_only(Path::new(db_path_str), None) else { return ReceiptsResult::fail(); }; - let spec = Arc::new(ChainSpecBuilder::mainnet().build()); - let factory = ProviderFactory::new(db, spec.clone()); + let factory = ProviderFactory::new(db, MAINNET.clone()); // Create a read-only BlockChainProvider let Ok(provider) = BlockchainProvider::new(factory, NoopBlockchainTree::default()) else { return ReceiptsResult::fail(); }; + // Fetch the block and the receipts within it let Ok(block) = provider.block_by_hash(block_hash.into()) else { return ReceiptsResult::fail(); }; - let Ok(receipts) = provider.receipts_by_block(BlockHashOrNumber::Hash(block_hash.into())) else { return ReceiptsResult::fail(); @@ -86,7 +91,6 @@ pub extern "C" fn read_receipts( let block_number = block.number; let base_fee = block.base_fee_per_gas; let block_hash = block.hash_slow(); - let excess_blob_gas = block.excess_blob_gas; let Some(receipts) = block .body .into_iter() @@ -99,7 +103,7 @@ pub extern "C" fn read_receipts( block_hash, block_number, base_fee, - excess_blob_gas, + excess_blob_gas: None, }; build_transaction_receipt_with_block_receipts(tx, meta, receipt, &receipts) }) @@ -127,14 +131,15 @@ pub extern "C" fn read_receipts( } /// Free a string that was allocated in Rust and passed to C. +/// +/// # Safety +/// - All possible nil pointer dereferences are checked. #[no_mangle] -pub extern "C" fn free_string(string: *mut c_char) { - unsafe { - // Convert the raw pointer back to a CString and let it go out of scope, - // which will deallocate the memory. - if !string.is_null() { - let _ = std::ffi::CString::from_raw(string); - } +pub unsafe extern "C" fn free_string(string: *mut c_char) { + // Convert the raw pointer back to a CString and let it go out of scope, + // which will deallocate the memory. + if !string.is_null() { + let _ = std::ffi::CString::from_raw(string); } } @@ -181,16 +186,16 @@ fn build_transaction_receipt_with_block_receipts( }, // EIP-4844 fields - blob_gas_price: meta.excess_blob_gas.map(calc_blob_gasprice).map(U128::from), - blob_gas_used: transaction.transaction.blob_gas_used().map(U128::from), + blob_gas_price: None, + blob_gas_used: None, }; match tx.transaction.kind() { - reth::primitives::TransactionKind::Create => { + TransactionKind::Create => { res_receipt.contract_address = Some(transaction.signer().create(tx.transaction.nonce())); } - reth::primitives::TransactionKind::Call(addr) => { + TransactionKind::Call(addr) => { res_receipt.to = Some(*addr); } } From e9a8f81af8b7a2eccf72ee258952fe9d0b13fc31 Mon Sep 17 00:00:00 2001 From: clabby Date: Fri, 27 Oct 2023 01:18:40 -0400 Subject: [PATCH 157/279] Add C header rustdoc --- op-service/rethdb-reader/README.md | 37 ++++++++++++++++++++++++++++- op-service/rethdb-reader/src/lib.rs | 14 +++++++++++ op-service/sources/reth_db_test.go | 16 +++++++++++++ 3 files changed, 66 insertions(+), 1 deletion(-) create mode 100644 op-service/sources/reth_db_test.go diff --git a/op-service/rethdb-reader/README.md b/op-service/rethdb-reader/README.md index e8d9f0d26c02..878cfd2c8662 100644 --- a/op-service/rethdb-reader/README.md +++ b/op-service/rethdb-reader/README.md @@ -1,4 +1,39 @@ # `rethdb-reader` -Exported Rust code to be used via FFI in `op-service`'s `sources` package for reading information +A dylib to be accessed via FFI in `op-service`'s `sources` package for reading information directly from the `reth` database. + +### C Header + +```c +#include +#include +#include +#include +#include + +struct ReceiptsResult { + uint32_t *data; + uintptr_t data_len; + bool error; +}; + +extern "C" { + +/// Read the receipts for a blockhash from the RETH database directly. +/// +/// # Safety +/// - All possible nil pointer dereferences are checked, and the function will return a +/// failing [ReceiptsResult] if any are found. +ReceiptsResult read_receipts(const uint8_t *block_hash, + uintptr_t block_hash_len, + const char *db_path); + +/// Free a string that was allocated in Rust and passed to C. +/// +/// # Safety +/// - All possible nil pointer dereferences are checked. +void free_string(char *string); + +} +``` diff --git a/op-service/rethdb-reader/src/lib.rs b/op-service/rethdb-reader/src/lib.rs index c7852000115a..2382f480623a 100644 --- a/op-service/rethdb-reader/src/lib.rs +++ b/op-service/rethdb-reader/src/lib.rs @@ -1,3 +1,5 @@ +#![doc = include_str!("../README.md")] + use reth::{ blockchain_tree::noop::NoopBlockchainTree, primitives::{ @@ -10,6 +12,12 @@ use reth::{ }; use std::{os::raw::c_char, path::Path}; +/// A [ReceiptsResult] is a wrapper around a JSON string containing serialized [TransactionReceipt]s +/// as well as an error status that is compatible with FFI. +/// +/// # Safety +/// - When the `error` field is false, the `data` pointer is guaranteed to be valid. +/// - When the `error` field is true, the `data` pointer is guaranteed to be null. #[repr(C)] pub struct ReceiptsResult { data: *mut char, @@ -18,6 +26,7 @@ pub struct ReceiptsResult { } impl ReceiptsResult { + /// Constructs a successful [ReceiptsResult] from a JSON string. pub fn success(data: *mut char, data_len: usize) -> Self { Self { data, @@ -26,6 +35,7 @@ impl ReceiptsResult { } } + /// Constructs a failing [ReceiptsResult] with a null pointer to the data. pub fn fail() -> Self { Self { data: std::ptr::null_mut(), @@ -143,6 +153,10 @@ pub unsafe extern "C" fn free_string(string: *mut c_char) { } } +/// Builds a hydrated [TransactionReceipt] from information in the passed transaction, +/// receipt, and block receipts. +/// +/// Returns [None] if the transaction's sender could not be recovered from the signature. #[inline(always)] fn build_transaction_receipt_with_block_receipts( tx: TransactionSigned, diff --git a/op-service/sources/reth_db_test.go b/op-service/sources/reth_db_test.go new file mode 100644 index 000000000000..73048f7be0d2 --- /dev/null +++ b/op-service/sources/reth_db_test.go @@ -0,0 +1,16 @@ +package sources + +import ( + "testing" + + "github.com/ethereum/go-ethereum/common" +) + +func TestRethDBRead(t *testing.T) { + t.Parallel() + + _, err := FetchRethReceipts("/test", &common.Hash{}) + if err != nil { + panic("test") + } +} From 679eb237d63faec1ba861b09664f0586d4039b5a Mon Sep 17 00:00:00 2001 From: clabby Date: Fri, 27 Oct 2023 01:44:34 -0400 Subject: [PATCH 158/279] :broom: dylib --- op-service/rethdb-reader/.gitignore | 3 + op-service/rethdb-reader/Cargo.lock | 1 + op-service/rethdb-reader/Cargo.toml | 1 + op-service/rethdb-reader/README.md | 94 +++++++--- op-service/rethdb-reader/headgen.sh | 16 ++ op-service/rethdb-reader/src/lib.rs | 214 +---------------------- op-service/rethdb-reader/src/receipts.rs | 211 ++++++++++++++++++++++ op-service/sources/reth_db_test.go | 16 -- 8 files changed, 305 insertions(+), 251 deletions(-) create mode 100755 op-service/rethdb-reader/headgen.sh create mode 100644 op-service/rethdb-reader/src/receipts.rs delete mode 100644 op-service/sources/reth_db_test.go diff --git a/op-service/rethdb-reader/.gitignore b/op-service/rethdb-reader/.gitignore index 9004537baafa..96359724ffbc 100644 --- a/op-service/rethdb-reader/.gitignore +++ b/op-service/rethdb-reader/.gitignore @@ -1,2 +1,5 @@ # Target target/ + +# Bindings +rdb.h diff --git a/op-service/rethdb-reader/Cargo.lock b/op-service/rethdb-reader/Cargo.lock index ef918a166b1f..14622c0a8c03 100644 --- a/op-service/rethdb-reader/Cargo.lock +++ b/op-service/rethdb-reader/Cargo.lock @@ -5360,6 +5360,7 @@ dependencies = [ name = "rethdb-reader" version = "0.1.0" dependencies = [ + "anyhow", "reth", "serde", "serde_json", diff --git a/op-service/rethdb-reader/Cargo.toml b/op-service/rethdb-reader/Cargo.toml index ac53bea511b8..c73662970ad1 100644 --- a/op-service/rethdb-reader/Cargo.toml +++ b/op-service/rethdb-reader/Cargo.toml @@ -12,3 +12,4 @@ crate-type = ["cdylib"] reth = { git = "https://github.com/paradigmxyz/reth.git" } serde = "1.0.190" serde_json = "1.0.107" +anyhow = "1.0.75" diff --git a/op-service/rethdb-reader/README.md b/op-service/rethdb-reader/README.md index 878cfd2c8662..430fe8f96b0b 100644 --- a/op-service/rethdb-reader/README.md +++ b/op-service/rethdb-reader/README.md @@ -3,37 +3,81 @@ A dylib to be accessed via FFI in `op-service`'s `sources` package for reading information directly from the `reth` database. +## Developing + +**Building** + +To build the dylib, you must first have the [Rust Toolchain][rust-toolchain] installed. + +```sh +cargo build --release +``` + +**Docs** + +Documentation is available via rustdoc. + +```sh +cargo doc --open +``` + +**Linting** + +```sh +cargo +nightly fmt -- && cargo +nightly clippy --all --all-features -- -D warnings +``` + +**Generating the C header** + +To generate the C header, first install `cbindgen` via `cargo install cbindgen --force`. Then, run the generation script: + +```sh +./headgen.sh +``` + ### C Header +The C header below is generated by `cbindgen`, and it is the interface that consumers of the dylib use to call its exported +functions. Currently, the only exported functions pertain to reading fully hydrated block receipts from the database. + ```c -#include -#include -#include -#include -#include +#include +#include +#include +#include -struct ReceiptsResult { +/** + * A [ReceiptsResult] is a wrapper around a JSON string containing serialized [TransactionReceipt]s + * as well as an error status that is compatible with FFI. + * + * # Safety + * - When the `error` field is false, the `data` pointer is guaranteed to be valid. + * - When the `error` field is true, the `data` pointer is guaranteed to be null. + */ +typedef struct ReceiptsResult { uint32_t *data; uintptr_t data_len; bool error; -}; - -extern "C" { - -/// Read the receipts for a blockhash from the RETH database directly. -/// -/// # Safety -/// - All possible nil pointer dereferences are checked, and the function will return a -/// failing [ReceiptsResult] if any are found. -ReceiptsResult read_receipts(const uint8_t *block_hash, - uintptr_t block_hash_len, - const char *db_path); - -/// Free a string that was allocated in Rust and passed to C. -/// -/// # Safety -/// - All possible nil pointer dereferences are checked. -void free_string(char *string); +} ReceiptsResult; + +/** + * Read the receipts for a blockhash from the RETH database directly. + * + * # Safety + * - All possible nil pointer dereferences are checked, and the function will return a + * failing [ReceiptsResult] if any are found. + */ +struct ReceiptsResult read_receipts(const uint8_t *block_hash, + uintptr_t block_hash_len, + const char *db_path); -} +/** + * Free a string that was allocated in Rust and passed to C. + * + * # Safety + * - All possible nil pointer dereferences are checked. + */ +void free_string(char *string); ``` + +[rust-toolchain]: https://rustup.rs/ diff --git a/op-service/rethdb-reader/headgen.sh b/op-service/rethdb-reader/headgen.sh new file mode 100755 index 000000000000..e7f7daf676d4 --- /dev/null +++ b/op-service/rethdb-reader/headgen.sh @@ -0,0 +1,16 @@ +#!/bin/bash + +set -e + +# Generate rdb.h +cbindgen --crate rethdb-reader --output rdb.h -l C + +# Process README.md to replace the content within the specified code block +awk ' + BEGIN { in_code_block=0; } + /^```c/ { in_code_block=1; print; next; } + /^```/ && in_code_block { in_code_block=0; while ((getline line < "rdb.h") > 0) print line; } + !in_code_block { print; } +' README.md > README.tmp && mv README.tmp README.md + +echo "Generated C header successfully" diff --git a/op-service/rethdb-reader/src/lib.rs b/op-service/rethdb-reader/src/lib.rs index 2382f480623a..1d3f3b0eb10c 100644 --- a/op-service/rethdb-reader/src/lib.rs +++ b/op-service/rethdb-reader/src/lib.rs @@ -1,49 +1,9 @@ #![doc = include_str!("../README.md")] -use reth::{ - blockchain_tree::noop::NoopBlockchainTree, - primitives::{ - BlockHashOrNumber, Receipt, TransactionKind, TransactionMeta, TransactionSigned, MAINNET, - U128, U256, U64, - }, - providers::{providers::BlockchainProvider, BlockReader, ProviderFactory, ReceiptProvider}, - rpc::types::{Log, TransactionReceipt}, - utils::db::open_db_read_only, -}; -use std::{os::raw::c_char, path::Path}; +use receipts::{read_receipts_inner, ReceiptsResult}; +use std::os::raw::c_char; -/// A [ReceiptsResult] is a wrapper around a JSON string containing serialized [TransactionReceipt]s -/// as well as an error status that is compatible with FFI. -/// -/// # Safety -/// - When the `error` field is false, the `data` pointer is guaranteed to be valid. -/// - When the `error` field is true, the `data` pointer is guaranteed to be null. -#[repr(C)] -pub struct ReceiptsResult { - data: *mut char, - data_len: usize, - error: bool, -} - -impl ReceiptsResult { - /// Constructs a successful [ReceiptsResult] from a JSON string. - pub fn success(data: *mut char, data_len: usize) -> Self { - Self { - data, - data_len, - error: false, - } - } - - /// Constructs a failing [ReceiptsResult] with a null pointer to the data. - pub fn fail() -> Self { - Self { - data: std::ptr::null_mut(), - data_len: 0, - error: true, - } - } -} +mod receipts; /// Read the receipts for a blockhash from the RETH database directly. /// @@ -56,88 +16,7 @@ pub unsafe extern "C" fn read_receipts( block_hash_len: usize, db_path: *const c_char, ) -> ReceiptsResult { - // Convert the raw pointer and length back to a Rust slice - let Ok(block_hash): Result<[u8; 32], _> = { - if block_hash.is_null() { - return ReceiptsResult::fail(); - } - std::slice::from_raw_parts(block_hash, block_hash_len) - } - .try_into() else { - return ReceiptsResult::fail(); - }; - - // Convert the *const c_char to a Rust &str - let Ok(db_path_str) = { - if db_path.is_null() { - return ReceiptsResult::fail(); - } - std::ffi::CStr::from_ptr(db_path) - } - .to_str() else { - return ReceiptsResult::fail(); - }; - - let Ok(db) = open_db_read_only(Path::new(db_path_str), None) else { - return ReceiptsResult::fail(); - }; - let factory = ProviderFactory::new(db, MAINNET.clone()); - - // Create a read-only BlockChainProvider - let Ok(provider) = BlockchainProvider::new(factory, NoopBlockchainTree::default()) else { - return ReceiptsResult::fail(); - }; - - // Fetch the block and the receipts within it - let Ok(block) = provider.block_by_hash(block_hash.into()) else { - return ReceiptsResult::fail(); - }; - let Ok(receipts) = provider.receipts_by_block(BlockHashOrNumber::Hash(block_hash.into())) - else { - return ReceiptsResult::fail(); - }; - - if let (Some(block), Some(receipts)) = (block, receipts) { - let block_number = block.number; - let base_fee = block.base_fee_per_gas; - let block_hash = block.hash_slow(); - let Some(receipts) = block - .body - .into_iter() - .zip(receipts.clone()) - .enumerate() - .map(|(idx, (tx, receipt))| { - let meta = TransactionMeta { - tx_hash: tx.hash, - index: idx as u64, - block_hash, - block_number, - base_fee, - excess_blob_gas: None, - }; - build_transaction_receipt_with_block_receipts(tx, meta, receipt, &receipts) - }) - .collect::>>() - else { - return ReceiptsResult::fail(); - }; - - // Convert the receipts to JSON for transport - let Ok(mut receipts_json) = serde_json::to_string(&receipts) else { - return ReceiptsResult::fail(); - }; - - let res = - ReceiptsResult::success(receipts_json.as_mut_ptr() as *mut char, receipts_json.len()); - - // Forget the `receipts_json` string so that its memory isn't freed by the - // borrow checker at the end of this scope - std::mem::forget(receipts_json); // Prevent Rust from freeing the memory - - res - } else { - ReceiptsResult::fail() - } + read_receipts_inner(block_hash, block_hash_len, db_path).unwrap_or(ReceiptsResult::fail()) } /// Free a string that was allocated in Rust and passed to C. @@ -152,88 +31,3 @@ pub unsafe extern "C" fn free_string(string: *mut c_char) { let _ = std::ffi::CString::from_raw(string); } } - -/// Builds a hydrated [TransactionReceipt] from information in the passed transaction, -/// receipt, and block receipts. -/// -/// Returns [None] if the transaction's sender could not be recovered from the signature. -#[inline(always)] -fn build_transaction_receipt_with_block_receipts( - tx: TransactionSigned, - meta: TransactionMeta, - receipt: Receipt, - all_receipts: &[Receipt], -) -> Option { - let transaction = tx.clone().into_ecrecovered()?; - - // get the previous transaction cumulative gas used - let gas_used = if meta.index == 0 { - receipt.cumulative_gas_used - } else { - let prev_tx_idx = (meta.index - 1) as usize; - all_receipts - .get(prev_tx_idx) - .map(|prev_receipt| receipt.cumulative_gas_used - prev_receipt.cumulative_gas_used) - .unwrap_or_default() - }; - - let mut res_receipt = TransactionReceipt { - transaction_hash: Some(meta.tx_hash), - transaction_index: U64::from(meta.index), - block_hash: Some(meta.block_hash), - block_number: Some(U256::from(meta.block_number)), - from: transaction.signer(), - to: None, - cumulative_gas_used: U256::from(receipt.cumulative_gas_used), - gas_used: Some(U256::from(gas_used)), - contract_address: None, - logs: Vec::with_capacity(receipt.logs.len()), - effective_gas_price: U128::from(transaction.effective_gas_price(meta.base_fee)), - transaction_type: tx.transaction.tx_type().into(), - // TODO pre-byzantium receipts have a post-transaction state root - state_root: None, - logs_bloom: receipt.bloom_slow(), - status_code: if receipt.success { - Some(U64::from(1)) - } else { - Some(U64::from(0)) - }, - - // EIP-4844 fields - blob_gas_price: None, - blob_gas_used: None, - }; - - match tx.transaction.kind() { - TransactionKind::Create => { - res_receipt.contract_address = - Some(transaction.signer().create(tx.transaction.nonce())); - } - TransactionKind::Call(addr) => { - res_receipt.to = Some(*addr); - } - } - - // get number of logs in the block - let mut num_logs = 0; - for prev_receipt in all_receipts.iter().take(meta.index as usize) { - num_logs += prev_receipt.logs.len(); - } - - for (tx_log_idx, log) in receipt.logs.into_iter().enumerate() { - let rpclog = Log { - address: log.address, - topics: log.topics, - data: log.data, - block_hash: Some(meta.block_hash), - block_number: Some(U256::from(meta.block_number)), - transaction_hash: Some(meta.tx_hash), - transaction_index: Some(U256::from(meta.index)), - log_index: Some(U256::from(num_logs + tx_log_idx)), - removed: false, - }; - res_receipt.logs.push(rpclog); - } - - Some(res_receipt) -} diff --git a/op-service/rethdb-reader/src/receipts.rs b/op-service/rethdb-reader/src/receipts.rs new file mode 100644 index 000000000000..3f17daff74bd --- /dev/null +++ b/op-service/rethdb-reader/src/receipts.rs @@ -0,0 +1,211 @@ +//! This module contains the logic for reading a block's fully hydrated receipts directly from the +//! [reth] database. + +use anyhow::{anyhow, Result}; +use reth::{ + blockchain_tree::noop::NoopBlockchainTree, + primitives::{ + BlockHashOrNumber, Receipt, TransactionKind, TransactionMeta, TransactionSigned, MAINNET, + U128, U256, U64, + }, + providers::{providers::BlockchainProvider, BlockReader, ProviderFactory, ReceiptProvider}, + rpc::types::{Log, TransactionReceipt}, + utils::db::open_db_read_only, +}; +use std::{ffi::c_char, path::Path}; + +/// A [ReceiptsResult] is a wrapper around a JSON string containing serialized [TransactionReceipt]s +/// as well as an error status that is compatible with FFI. +/// +/// # Safety +/// - When the `error` field is false, the `data` pointer is guaranteed to be valid. +/// - When the `error` field is true, the `data` pointer is guaranteed to be null. +#[repr(C)] +pub struct ReceiptsResult { + data: *mut char, + data_len: usize, + error: bool, +} + +impl ReceiptsResult { + /// Constructs a successful [ReceiptsResult] from a JSON string. + pub fn success(data: *mut char, data_len: usize) -> Self { + Self { + data, + data_len, + error: false, + } + } + + /// Constructs a failing [ReceiptsResult] with a null pointer to the data. + pub fn fail() -> Self { + Self { + data: std::ptr::null_mut(), + data_len: 0, + error: true, + } + } +} + +/// Read the receipts for a blockhash from the RETH database directly. +/// +/// # Safety +/// - All possible nil pointer dereferences are checked, and the function will return a +/// failing [ReceiptsResult] if any are found. +#[inline(always)] +pub(crate) unsafe fn read_receipts_inner( + block_hash: *const u8, + block_hash_len: usize, + db_path: *const c_char, +) -> Result { + // Convert the raw pointer and length back to a Rust slice + let block_hash: [u8; 32] = { + if block_hash.is_null() { + anyhow::bail!("block_hash pointer is null"); + } + std::slice::from_raw_parts(block_hash, block_hash_len) + } + .try_into()?; + + // Convert the *const c_char to a Rust &str + let db_path_str = { + if db_path.is_null() { + anyhow::bail!("db path pointer is null"); + } + std::ffi::CStr::from_ptr(db_path) + } + .to_str()?; + + let db = open_db_read_only(Path::new(db_path_str), None).map_err(|e| anyhow!(e))?; + let factory = ProviderFactory::new(db, MAINNET.clone()); + + // Create a read-only BlockChainProvider + let provider = BlockchainProvider::new(factory, NoopBlockchainTree::default())?; + + // Fetch the block and the receipts within it + let block = provider + .block_by_hash(block_hash.into())? + .ok_or(anyhow!("Failed to fetch block"))?; + let receipts = provider + .receipts_by_block(BlockHashOrNumber::Hash(block_hash.into()))? + .ok_or(anyhow!("Failed to fetch block receipts"))?; + + let block_number = block.number; + let base_fee = block.base_fee_per_gas; + let block_hash = block.hash_slow(); + let receipts = block + .body + .into_iter() + .zip(receipts.clone()) + .enumerate() + .map(|(idx, (tx, receipt))| { + let meta = TransactionMeta { + tx_hash: tx.hash, + index: idx as u64, + block_hash, + block_number, + base_fee, + excess_blob_gas: None, + }; + build_transaction_receipt_with_block_receipts(tx, meta, receipt, &receipts) + }) + .collect::>>() + .ok_or(anyhow!("Failed to build receipts"))?; + + // Convert the receipts to JSON for transport + let mut receipts_json = serde_json::to_string(&receipts)?; + + // Create a ReceiptsResult with a pointer to the json-ified receipts + let res = ReceiptsResult::success(receipts_json.as_mut_ptr() as *mut char, receipts_json.len()); + + // Forget the `receipts_json` string so that its memory isn't freed by the + // borrow checker at the end of this scope + std::mem::forget(receipts_json); // Prevent Rust from freeing the memory + + Ok(res) +} + +/// Builds a hydrated [TransactionReceipt] from information in the passed transaction, +/// receipt, and block receipts. +/// +/// Returns [None] if the transaction's sender could not be recovered from the signature. +#[inline(always)] +fn build_transaction_receipt_with_block_receipts( + tx: TransactionSigned, + meta: TransactionMeta, + receipt: Receipt, + all_receipts: &[Receipt], +) -> Option { + let transaction = tx.clone().into_ecrecovered()?; + + // get the previous transaction cumulative gas used + let gas_used = if meta.index == 0 { + receipt.cumulative_gas_used + } else { + let prev_tx_idx = (meta.index - 1) as usize; + all_receipts + .get(prev_tx_idx) + .map(|prev_receipt| receipt.cumulative_gas_used - prev_receipt.cumulative_gas_used) + .unwrap_or_default() + }; + + let mut res_receipt = TransactionReceipt { + transaction_hash: Some(meta.tx_hash), + transaction_index: U64::from(meta.index), + block_hash: Some(meta.block_hash), + block_number: Some(U256::from(meta.block_number)), + from: transaction.signer(), + to: None, + cumulative_gas_used: U256::from(receipt.cumulative_gas_used), + gas_used: Some(U256::from(gas_used)), + contract_address: None, + logs: Vec::with_capacity(receipt.logs.len()), + effective_gas_price: U128::from(transaction.effective_gas_price(meta.base_fee)), + transaction_type: tx.transaction.tx_type().into(), + // TODO pre-byzantium receipts have a post-transaction state root + state_root: None, + logs_bloom: receipt.bloom_slow(), + status_code: if receipt.success { + Some(U64::from(1)) + } else { + Some(U64::from(0)) + }, + + // EIP-4844 fields + blob_gas_price: None, + blob_gas_used: None, + }; + + match tx.transaction.kind() { + TransactionKind::Create => { + res_receipt.contract_address = + Some(transaction.signer().create(tx.transaction.nonce())); + } + TransactionKind::Call(addr) => { + res_receipt.to = Some(*addr); + } + } + + // get number of logs in the block + let mut num_logs = 0; + for prev_receipt in all_receipts.iter().take(meta.index as usize) { + num_logs += prev_receipt.logs.len(); + } + + for (tx_log_idx, log) in receipt.logs.into_iter().enumerate() { + let rpclog = Log { + address: log.address, + topics: log.topics, + data: log.data, + block_hash: Some(meta.block_hash), + block_number: Some(U256::from(meta.block_number)), + transaction_hash: Some(meta.tx_hash), + transaction_index: Some(U256::from(meta.index)), + log_index: Some(U256::from(num_logs + tx_log_idx)), + removed: false, + }; + res_receipt.logs.push(rpclog); + } + + Some(res_receipt) +} diff --git a/op-service/sources/reth_db_test.go b/op-service/sources/reth_db_test.go deleted file mode 100644 index 73048f7be0d2..000000000000 --- a/op-service/sources/reth_db_test.go +++ /dev/null @@ -1,16 +0,0 @@ -package sources - -import ( - "testing" - - "github.com/ethereum/go-ethereum/common" -) - -func TestRethDBRead(t *testing.T) { - t.Parallel() - - _, err := FetchRethReceipts("/test", &common.Hash{}) - if err != nil { - panic("test") - } -} From c87d94bb0d2bfa2c27762b814da2662fabe50473 Mon Sep 17 00:00:00 2001 From: clabby Date: Fri, 27 Oct 2023 16:53:54 -0400 Subject: [PATCH 159/279] @inphi nits Co-Authored-By: inphi --- op-node/node/config.go | 2 +- op-node/service.go | 7 +------ op-service/rethdb-reader/README.md | 8 ++++---- op-service/rethdb-reader/src/lib.rs | 4 ++-- op-service/sources/eth_client.go | 4 ++-- op-service/sources/receipts.go | 8 ++++---- op-service/sources/reth_db.go | 12 ++++++------ 7 files changed, 20 insertions(+), 25 deletions(-) diff --git a/op-node/node/config.go b/op-node/node/config.go index d590bcfc675f..cb970d5ebfd3 100644 --- a/op-node/node/config.go +++ b/op-node/node/config.go @@ -62,7 +62,7 @@ type Config struct { Cancel context.CancelCauseFunc // [OPTIONAL] The reth DB path to read receipts from - RethDBPath *string + RethDBPath string } type RPCConfig struct { diff --git a/op-node/service.go b/op-node/service.go index 2776ef180e00..2569e7af83b6 100644 --- a/op-node/service.go +++ b/op-node/service.go @@ -71,11 +71,6 @@ func NewConfig(ctx *cli.Context, log log.Logger) (*node.Config, error) { haltOption = "" } - var rethDBPath *string - if rdb := ctx.String(flags.L1RethDBPath.Name); rdb != "" { - rethDBPath = &rdb - } - cfg := &node.Config{ L1: l1Endpoint, L2: l2Endpoint, @@ -109,7 +104,7 @@ func NewConfig(ctx *cli.Context, log log.Logger) (*node.Config, error) { ConfigPersistence: configPersistence, Sync: *syncConfig, RollupHalt: haltOption, - RethDBPath: rethDBPath, + RethDBPath: ctx.String(flags.L1RethDBPath.Name), } if err := cfg.LoadPersisted(log); err != nil { diff --git a/op-service/rethdb-reader/README.md b/op-service/rethdb-reader/README.md index 430fe8f96b0b..2ef313ef413e 100644 --- a/op-service/rethdb-reader/README.md +++ b/op-service/rethdb-reader/README.md @@ -67,9 +67,9 @@ typedef struct ReceiptsResult { * - All possible nil pointer dereferences are checked, and the function will return a * failing [ReceiptsResult] if any are found. */ -struct ReceiptsResult read_receipts(const uint8_t *block_hash, - uintptr_t block_hash_len, - const char *db_path); +struct ReceiptsResult rdb_read_receipts(const uint8_t *block_hash, + uintptr_t block_hash_len, + const char *db_path); /** * Free a string that was allocated in Rust and passed to C. @@ -77,7 +77,7 @@ struct ReceiptsResult read_receipts(const uint8_t *block_hash, * # Safety * - All possible nil pointer dereferences are checked. */ -void free_string(char *string); +void rdb_free_string(char *string); ``` [rust-toolchain]: https://rustup.rs/ diff --git a/op-service/rethdb-reader/src/lib.rs b/op-service/rethdb-reader/src/lib.rs index 1d3f3b0eb10c..20aabb1f1391 100644 --- a/op-service/rethdb-reader/src/lib.rs +++ b/op-service/rethdb-reader/src/lib.rs @@ -11,7 +11,7 @@ mod receipts; /// - All possible nil pointer dereferences are checked, and the function will return a /// failing [ReceiptsResult] if any are found. #[no_mangle] -pub unsafe extern "C" fn read_receipts( +pub unsafe extern "C" fn rdb_read_receipts( block_hash: *const u8, block_hash_len: usize, db_path: *const c_char, @@ -24,7 +24,7 @@ pub unsafe extern "C" fn read_receipts( /// # Safety /// - All possible nil pointer dereferences are checked. #[no_mangle] -pub unsafe extern "C" fn free_string(string: *mut c_char) { +pub unsafe extern "C" fn rdb_free_string(string: *mut c_char) { // Convert the raw pointer back to a CString and let it go out of scope, // which will deallocate the memory. if !string.is_null() { diff --git a/op-service/sources/eth_client.go b/op-service/sources/eth_client.go index b679427f5d68..7a54a5838f85 100644 --- a/op-service/sources/eth_client.go +++ b/op-service/sources/eth_client.go @@ -64,7 +64,7 @@ type EthClientConfig struct { MethodResetDuration time.Duration // [OPTIONAL] The reth DB path to fetch receipts from - RethDBPath *string + RethDBPath string } func (c *EthClientConfig) Check() error { @@ -137,7 +137,7 @@ type EthClient struct { methodResetDuration time.Duration // [OPTIONAL] The reth DB path to fetch receipts from - rethDbPath *string + rethDbPath string } func (s *EthClient) PickReceiptsMethod(txCount uint64) ReceiptsFetchingMethod { diff --git a/op-service/sources/receipts.go b/op-service/sources/receipts.go index 2d60bb959716..f5520624a7f7 100644 --- a/op-service/sources/receipts.go +++ b/op-service/sources/receipts.go @@ -390,13 +390,13 @@ type receiptsFetchingJob struct { fetcher *IterativeBatchCall[common.Hash, *types.Receipt] // [OPTIONAL] RethDB path to fetch receipts from - rethDbPath *string + rethDbPath string result types.Receipts } func NewReceiptsFetchingJob(requester ReceiptsRequester, client rpcClient, maxBatchSize int, block eth.BlockID, - receiptHash common.Hash, txHashes []common.Hash, rethDb *string) *receiptsFetchingJob { + receiptHash common.Hash, txHashes []common.Hash, rethDb string) *receiptsFetchingJob { return &receiptsFetchingJob{ requester: requester, client: client, @@ -483,10 +483,10 @@ func (job *receiptsFetchingJob) runAltMethod(ctx context.Context, m ReceiptsFetc case ErigonGetBlockReceiptsByBlockHash: err = job.client.CallContext(ctx, &result, "erigon_getBlockReceiptsByBlockHash", job.block.Hash) case RethGetBlockReceipts: - if job.rethDbPath == nil { + if job.rethDbPath == "" { return fmt.Errorf("reth_db path not set") } - res, err := FetchRethReceipts(*job.rethDbPath, &job.block.Hash) + res, err := FetchRethReceipts(job.rethDbPath, &job.block.Hash) if err != nil { return err } diff --git a/op-service/sources/reth_db.go b/op-service/sources/reth_db.go index 71b8d2a78c9f..2579b95f3f87 100644 --- a/op-service/sources/reth_db.go +++ b/op-service/sources/reth_db.go @@ -24,8 +24,8 @@ typedef struct { bool error; } ReceiptsResult; -extern ReceiptsResult read_receipts(const uint8_t* block_hash, size_t block_hash_len, const char* db_path); -extern void free_string(char* string); +extern ReceiptsResult rdb_read_receipts(const uint8_t* block_hash, size_t block_hash_len, const char* db_path); +extern void rdb_free_string(char* string); */ import "C" @@ -45,12 +45,15 @@ func FetchRethReceipts(dbPath string, blockHash *common.Hash) (types.Receipts, e defer C.free(unsafe.Pointer(cDbPath)) // Call the C function to fetch the receipts from the Reth Database - receiptsResult := C.read_receipts((*C.uint8_t)(cBlockHash), C.size_t(len(blockHash)), cDbPath) + receiptsResult := C.rdb_read_receipts((*C.uint8_t)(cBlockHash), C.size_t(len(blockHash)), cDbPath) if receiptsResult.error { return nil, fmt.Errorf("Error fetching receipts from Reth Database.") } + // Free the memory allocated by the C code + defer C.rdb_free_string(receiptsResult.data) + // Convert the returned JSON string to Go string and parse it receiptsJSON := C.GoStringN(receiptsResult.data, C.int(receiptsResult.data_len)) var receipts types.Receipts @@ -58,8 +61,5 @@ func FetchRethReceipts(dbPath string, blockHash *common.Hash) (types.Receipts, e return nil, err } - // Free the memory allocated by the C code - C.free_string(receiptsResult.data) - return receipts, nil } From d766780e677e7505e3621145c6e4bb4017793d09 Mon Sep 17 00:00:00 2001 From: clabby Date: Sat, 28 Oct 2023 02:27:47 -0400 Subject: [PATCH 160/279] Remove version pin on `foundryup` in devnet job --- .circleci/config.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index 16a9fb0f5037..c669ebcac3bd 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -989,7 +989,7 @@ jobs: command: | curl -L https://foundry.paradigm.xyz | bash source $HOME/.bashrc - foundryup --version nightly-aa257c2fb50814dfc5da4b3688cd3b95b5e3844d + foundryup echo 'export PATH=$HOME/.foundry/bin:$PATH' >> $BASH_ENV source $HOME/.bashrc forge --version From d91633678077bc01f515d793d8063203b4ad355b Mon Sep 17 00:00:00 2001 From: Maurelian Date: Wed, 27 Sep 2023 13:31:10 -0400 Subject: [PATCH 161/279] feat(ctb): Add outline for Safe liveness checks --- .gitmodules | 3 - .../src/Safe/LivenessGuard.sol | 105 ++++++++++++++ .../src/Safe/LivenessModule.sol | 1 + .../test/SafeLivenessChecking.t.sol | 129 ++++++++++++++++++ 4 files changed, 235 insertions(+), 3 deletions(-) create mode 100644 packages/contracts-bedrock/src/Safe/LivenessGuard.sol create mode 100644 packages/contracts-bedrock/src/Safe/LivenessModule.sol create mode 100644 packages/contracts-bedrock/test/SafeLivenessChecking.t.sol diff --git a/.gitmodules b/.gitmodules index 802f78e7a218..e3bf55582296 100644 --- a/.gitmodules +++ b/.gitmodules @@ -17,6 +17,3 @@ path = packages/contracts-bedrock/lib/safe-contracts url = https://github.com/safe-global/safe-contracts branch = v1.4.0 - - - diff --git a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol new file mode 100644 index 000000000000..a1c99fb689b3 --- /dev/null +++ b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol @@ -0,0 +1,105 @@ +// SPDX-License-Identifier: MIT +pragma solidity 0.8.15; + +import { Safe } from "safe-contracts/Safe.sol"; +import { BaseGuard } from "safe-contracts/base/GuardManager.sol"; +import { SignatureDecoder } from "safe-contracts/common/SignatureDecoder.sol"; +import { Enum } from "safe-contracts/common/Enum.sol"; + +contract LivenessGuard is SignatureDecoder, BaseGuard { + Safe public safe; + mapping(address => uint256) public lastSigned; + + constructor(Safe _safe) { + safe = _safe; + } + + /// @notice We just need to satisfy the BaseGuard interfae, but we don't actually need to use this method. + function checkAfterExecution(bytes32 txHash, bool success) external { + return; + } + + /// @notice This checkTransaction implementation records the most recent time which any owner has signed a + /// transaction. + function checkTransaction( + address to, + uint256 value, + bytes memory data, + Enum.Operation operation, + uint256 safeTxGas, + uint256 baseGas, + uint256 gasPrice, + address gasToken, + address payable refundReceiver, + bytes memory signatures, + address msgSender + ) + external + { + require(msg.sender == address(safe), "LivenessGuard: only Safe can call this function"); + if (to == address(safe) && data[0:4] == bytes4(keccak256("setGuard(address)"))) { + // We can't allow the guard to be disabled, or else the upgrade delay can be bypassed. + // TODO(Maurelian): Figure out how to best address this. + } + + // This is a bit of a hack, maybe just replicate the functionality here rather than calling home + bytes memory txHashData = Safe(payable(msg.sender)).encodeTransactionData( + // Transaction info + to, + value, + data, + operation, + safeTxGas, + // Payment info + baseGas, + gasPrice, + gasToken, + refundReceiver, + // Signature info + Safe(payable(msg.sender)).nonce() // check that this works + ); + address[] memory signers = _getNSigners(keccak256(txHashData), signatures); + for (uint256 i = 0; i < signers.length; i++) { + lastSigned[signers[i]] = block.timestamp; + } + } + + function _getNSigners(bytes32 dataHash, bytes memory signatures) internal returns (address[] memory _owners) { + uint256 numSignatures = signatures.length / 65; // division OK? + _owners = new address[](numSignatures); + + // The following code is extracted from the Safe.checkNSignatures() method. It removes the signature validation + // code, + // and keeps only the parsing code necessary to extract the owner addresses from the signatures. + // We do not double check if the owner derived from a signature is valid. As tHis is handled in + // the final require statement of Safe.checkNSignatures(). + address currentOwner; + uint8 v; + bytes32 r; + bytes32 s; + uint256 i; + for (i = 0; i < numSignatures; i++) { + (v, r, s) = signatureSplit(signatures, i); + if (v == 0) { + // If v is 0 then it is a contract signature + // When handling contract signatures the address of the contract is encoded into r + currentOwner = address(uint160(uint256(r))); + } else if (v == 1) { + // If v is 1 then it is an approved hash + // When handling approved hashes the address of the approver is encoded into r + currentOwner = address(uint160(uint256(r))); + } else if (v > 30) { + // If v > 30 then default va (27,28) has been adjusted for eth_sign flow + // To support eth_sign and similar we adjust v and hash the messageHash with the Ethereum message prefix + // before applying ecrecover + currentOwner = + ecrecover(keccak256(abi.encodePacked("\x19Ethereum Signed Message:\n32", dataHash)), v - 4, r, s); + } else { + // Default is the ecrecover flow with the provided data hash + // Use ecrecover with the messageHash for EOA signatures + currentOwner = ecrecover(dataHash, v, r, s); + } + _owners[i] = currentOwner; + } + } +} diff --git a/packages/contracts-bedrock/src/Safe/LivenessModule.sol b/packages/contracts-bedrock/src/Safe/LivenessModule.sol new file mode 100644 index 000000000000..8b137891791f --- /dev/null +++ b/packages/contracts-bedrock/src/Safe/LivenessModule.sol @@ -0,0 +1 @@ + diff --git a/packages/contracts-bedrock/test/SafeLivenessChecking.t.sol b/packages/contracts-bedrock/test/SafeLivenessChecking.t.sol new file mode 100644 index 000000000000..fff70e690c9d --- /dev/null +++ b/packages/contracts-bedrock/test/SafeLivenessChecking.t.sol @@ -0,0 +1,129 @@ +// SPDX-License-Identifier: MIT +pragma solidity 0.8.15; + +import { Test } from "forge-std/Test.sol"; +import { Safe } from "safe-contracts/Safe.sol"; +import { SafeProxyFactory } from "safe-contracts/proxies/SafeProxyFactory.sol"; +import { ModuleManager } from "safe-contracts/base/ModuleManager.sol"; +import { Enum } from "safe-contracts/common/Enum.sol"; + +import { LivenessGuard } from "src/Safe/LivenessGuard.sol"; + +contract LivnessGuard_TestInit is Test { + struct Signer { + address owner; + uint256 pk; + } + + LivenessGuard livenessGuard; + Safe safe; + string mnemonic = "test test test test test test test test test test test junk"; + + Signer[] signers; + + function newSigner(uint256 index) public returns (Signer memory signer_) { + signer_.pk = vm.deriveKey(mnemonic, uint32(index)); + signer_.owner = vm.addr(signer_.pk); + } + + function signTransaction( + uint256 _pk, + address _to, + uint256 _value, + bytes memory _data + ) + public + view + returns (bytes memory sig_) + { + bytes32 txDataHash; + { + txDataHash = safe.getTransactionHash({ + to: _to, + value: _value, + data: _data, + operation: Enum.Operation.Call, + safeTxGas: 0, + baseGas: 0, + gasPrice: 0, + gasToken: address(0), + refundReceiver: address(0), + _nonce: safe.nonce() + }); + } + + (uint8 v, bytes32 r, bytes32 s) = vm.sign(_pk, txDataHash); + sig_ = abi.encodePacked(v, r, s); + } + + function exec(Signer[] memory _signers, address _to, bytes memory _data) internal { + bytes memory sig; + for (uint256 i; i < _signers.length; i++) { + bytes.concat(sig, signTransaction(_signers[i].pk, address(safe), 0, _data)); + } + safe.execTransaction({ + to: _to, + value: 0, + data: _data, + operation: Enum.Operation.Call, + safeTxGas: 0, + baseGas: 0, + gasPrice: 0, + gasToken: address(0), + refundReceiver: payable(0), + signatures: sig + }); + } + + // @dev Create a new Safe instance with a minimimal proxy and implementation. + function newSafe(Signer[] memory _signers) internal returns (Safe safe_) { + SafeProxyFactory safeProxyFactory = new SafeProxyFactory(); + Safe safeSingleton = new Safe(); + + bytes memory initData = abi.encodeWithSelector( + Safe.setup.selector, _signers, 2, address(0), hex"", address(0), address(0), 0, address(0) + ); + + safe_ = Safe(payable(safeProxyFactory.createProxyWithNonce(address(safeSingleton), initData, block.timestamp))); + } + + function setUp() public { + // Create 3 signers + for (uint256 i; i < 3; i++) { + signers.push(newSigner(i)); + } + + Signer[] memory signers_ = signers; + safe = newSafe(signers_); + livenessGuard = new LivenessGuard(safe); + + // enable the module + bytes memory data = abi.encodeCall(ModuleManager.enableModule, (address(livenessGuard))); + bytes memory sig1 = signTransaction(signers[0].pk, address(safe), 0, data); + bytes memory sig2 = signTransaction(signers[1].pk, address(safe), 0, data); + bytes memory sigs = bytes.concat(sig1, sig2); + safe.execTransaction({ + to: address(safe), + value: 0, + data: data, + operation: Enum.Operation.Call, + safeTxGas: 0, + baseGas: 0, + gasPrice: 0, + gasToken: address(0), + refundReceiver: payable(0), + signatures: sigs + }); + } +} + +contract LivnessGuard_TestCheckTx is LivnessGuard_TestInit { + function test_checkTransaction_succeeds() external { + Signer[] memory signers_ = signers; + exec(signers, address(1111), hex"abba"); + + for (uint256 i; i < signers.length; i++) { + assertEq(livenessGuard.lastSigned(signers[i].owner), block.timestamp); + } + } +} From 55fe9711c81801dba3a6c8eb669d29a17b559f3b Mon Sep 17 00:00:00 2001 From: Maurelian Date: Sat, 30 Sep 2023 23:07:37 -0400 Subject: [PATCH 162/279] safe-tools: vendor in safe-tools lib --- .../CompatibilityFallbackHandler_1_3_0.sol | 212 ++++++++ .../test/safe-tools/SafeTestTools.sol | 467 ++++++++++++++++++ 2 files changed, 679 insertions(+) create mode 100644 packages/contracts-bedrock/test/safe-tools/CompatibilityFallbackHandler_1_3_0.sol create mode 100644 packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol diff --git a/packages/contracts-bedrock/test/safe-tools/CompatibilityFallbackHandler_1_3_0.sol b/packages/contracts-bedrock/test/safe-tools/CompatibilityFallbackHandler_1_3_0.sol new file mode 100644 index 000000000000..68e46e8e88e2 --- /dev/null +++ b/packages/contracts-bedrock/test/safe-tools/CompatibilityFallbackHandler_1_3_0.sol @@ -0,0 +1,212 @@ +// SPDX-License-Identifier: LGPL-3.0-only +pragma solidity >=0.7.0 <0.9.0; + +import "safe-contracts/interfaces/ERC1155TokenReceiver.sol"; +import "safe-contracts/interfaces/ERC721TokenReceiver.sol"; +import "safe-contracts/interfaces/ERC777TokensRecipient.sol"; +import "safe-contracts/interfaces/IERC165.sol"; +import "safe-contracts/interfaces/ISignatureValidator.sol"; +import "safe-contracts/GnosisSafe.sol"; + +/// @dev NOTE: removed VERSION and NAME due to inheritance conflicts +contract DefaultCallbackHandler is ERC1155TokenReceiver, ERC777TokensRecipient, ERC721TokenReceiver, IERC165 { + function onERC1155Received( + address, + address, + uint256, + uint256, + bytes calldata + ) + external + pure + override + returns (bytes4) + { + return 0xf23a6e61; + } + + function onERC1155BatchReceived( + address, + address, + uint256[] calldata, + uint256[] calldata, + bytes calldata + ) + external + pure + override + returns (bytes4) + { + return 0xbc197c81; + } + + function onERC721Received(address, address, uint256, bytes calldata) external pure override returns (bytes4) { + return 0x150b7a02; + } + + function tokensReceived( + address, + address, + address, + uint256, + bytes calldata, + bytes calldata + ) + external + pure + override + { + // We implement this for completeness, doesn't really have any value + } + + function supportsInterface(bytes4 interfaceId) external view virtual override returns (bool) { + return interfaceId == type(ERC1155TokenReceiver).interfaceId + || interfaceId == type(ERC721TokenReceiver).interfaceId || interfaceId == type(IERC165).interfaceId; + } +} + +address constant SENTINEL_MODULES = address(0x1); + +/// @title Compatibility Fallback Handler - fallback handler to provider compatibility between pre 1.3.0 and 1.3.0+ Safe +/// contracts +/// @author Richard Meissner - +contract CompatibilityFallbackHandler is DefaultCallbackHandler, ISignatureValidator { + //keccak256( + // "SafeMessage(bytes message)" + //); + bytes32 private constant SAFE_MSG_TYPEHASH = 0x60b3cbf8b4a223d68d641b3b6ddf9a298e7f33710cf3d3a9d1146b5a6150fbca; + + bytes4 internal constant SIMULATE_SELECTOR = bytes4(keccak256("simulate(address,bytes)")); + + bytes4 internal constant UPDATED_MAGIC_VALUE = 0x1626ba7e; + + /** + * Implementation of ISignatureValidator (see `interfaces/ISignatureValidator.sol`) + * @dev Should return whether the signature provided is valid for the provided data. + * @param _data Arbitrary length data signed on the behalf of address(msg.sender) + * @param _signature Signature byte array associated with _data + * @return a bool upon valid or invalid signature with corresponding _data + */ + function isValidSignature(bytes memory _data, bytes memory _signature) public view override returns (bytes4) { + // Caller should be a Safe + GnosisSafe safe = GnosisSafe(payable(msg.sender)); + bytes32 messageHash = getMessageHashForSafe(safe, _data); + if (_signature.length == 0) { + require(safe.signedMessages(messageHash) != 0, "Hash not approved"); + } else { + safe.checkSignatures(messageHash, _data, _signature); + } + return EIP1271_MAGIC_VALUE; + } + + /// @dev Returns hash of a message that can be signed by owners. + /// @param message Message that should be hashed + /// @return Message hash. + function getMessageHash(bytes memory message) public view returns (bytes32) { + return getMessageHashForSafe(GnosisSafe(payable(msg.sender)), message); + } + + /// @dev Returns hash of a message that can be signed by owners. + /// @param safe Safe to which the message is targeted + /// @param message Message that should be hashed + /// @return Message hash. + function getMessageHashForSafe(GnosisSafe safe, bytes memory message) public view returns (bytes32) { + bytes32 safeMessageHash = keccak256(abi.encode(SAFE_MSG_TYPEHASH, keccak256(message))); + return keccak256(abi.encodePacked(bytes1(0x19), bytes1(0x01), safe.domainSeparator(), safeMessageHash)); + } + + /** + * Implementation of updated EIP-1271 + * @dev Should return whether the signature provided is valid for the provided data. + * The save does not implement the interface since `checkSignatures` is not a view method. + * The method will not perform any state changes (see parameters of `checkSignatures`) + * @param _dataHash Hash of the data signed on the behalf of address(msg.sender) + * @param _signature Signature byte array associated with _dataHash + * @return a bool upon valid or invalid signature with corresponding _dataHash + * @notice See + * https://github.com/gnosis/util-contracts/blob/bb5fe5fb5df6d8400998094fb1b32a178a47c3a1/contracts/StorageAccessible.sol + */ + function isValidSignature(bytes32 _dataHash, bytes calldata _signature) external view returns (bytes4) { + ISignatureValidator validator = ISignatureValidator(msg.sender); + bytes4 value = validator.isValidSignature(abi.encode(_dataHash), _signature); + return (value == EIP1271_MAGIC_VALUE) ? UPDATED_MAGIC_VALUE : bytes4(0); + } + + /// @dev Returns array of first 10 modules. + /// @return Array of modules. + function getModules() external view returns (address[] memory) { + // Caller should be a Safe + GnosisSafe safe = GnosisSafe(payable(msg.sender)); + (address[] memory array,) = safe.getModulesPaginated(SENTINEL_MODULES, 10); + return array; + } + + /** + * @dev Performs a delegetecall on a targetContract in the context of self. + * Internally reverts execution to avoid side effects (making it static). Catches revert and returns encoded result + * as bytes. + * @param targetContract Address of the contract containing the code to execute. + * @param calldataPayload Calldata that should be sent to the target contract (encoded method name and arguments). + */ + function simulate( + address targetContract, + bytes calldata calldataPayload + ) + external + returns (bytes memory response) + { + // Suppress compiler warnings about not using parameters, while allowing + // parameters to keep names for documentation purposes. This does not + // generate code. + targetContract; + calldataPayload; + + // solhint-disable-next-line no-inline-assembly + assembly { + let internalCalldata := mload(0x40) + // Store `simulateAndRevert.selector`. + // String representation is used to force right padding + mstore(internalCalldata, "\xb4\xfa\xba\x09") + // Abuse the fact that both this and the internal methods have the + // same signature, and differ only in symbol name (and therefore, + // selector) and copy calldata directly. This saves us approximately + // 250 bytes of code and 300 gas at runtime over the + // `abi.encodeWithSelector` builtin. + calldatacopy(add(internalCalldata, 0x04), 0x04, sub(calldatasize(), 0x04)) + + // `pop` is required here by the compiler, as top level expressions + // can't have return values in inline assembly. `call` typically + // returns a 0 or 1 value indicated whether or not it reverted, but + // since we know it will always revert, we can safely ignore it. + pop( + call( + gas(), + // address() has been changed to caller() to use the implementation of the Safe + caller(), + 0, + internalCalldata, + calldatasize(), + // The `simulateAndRevert` call always reverts, and + // instead encodes whether or not it was successful in the return + // data. The first 32-byte word of the return data contains the + // `success` value, so write it to memory address 0x00 (which is + // reserved Solidity scratch space and OK to use). + 0x00, + 0x20 + ) + ) + + // Allocate and copy the response bytes, making sure to increment + // the free memory pointer accordingly (in case this method is + // called as an internal function). The remaining `returndata[0x20:]` + // contains the ABI encoded response bytes, so we can just write it + // as is to memory. + let responseSize := sub(returndatasize(), 0x20) + response := mload(0x40) + mstore(0x40, add(response, responseSize)) + returndatacopy(response, 0x20, responseSize) + + if iszero(mload(0x00)) { revert(add(response, 0x20), mload(response)) } + } + } +} diff --git a/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol b/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol new file mode 100644 index 000000000000..e09b6ed80677 --- /dev/null +++ b/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol @@ -0,0 +1,467 @@ +// SPDX-License-Identifier: MIT +pragma solidity >=0.7.0 <0.9.0; + +import "forge-std/Test.sol"; +import "solady/utils/LibSort.sol"; +import "safe-contracts/GnosisSafe.sol"; +import "safe-contracts/proxies/GnosisSafeProxyFactory.sol"; +import "safe-contracts/examples/libraries/SignMessage.sol"; +import "./CompatibilityFallbackHandler_1_3_0.sol"; +import "safe-contracts/examples/libraries/SignMessage.sol"; + +address constant VM_ADDR = 0x7109709ECfa91a80626fF3989D68f67F5b1DD12D; +bytes12 constant ADDR_MASK = 0xffffffffffffffffffffffff; + +function getAddr(uint256 pk) pure returns (address) { + return Vm(VM_ADDR).addr(pk); +} + +function encodeSmartContractWalletAsPK(address addr) pure returns (uint256 encodedPK) { + assembly { + let addr_b32 := addr + encodedPK := or(addr, ADDR_MASK) + } +} + +function decodeSmartContractWalletAsAddress(uint256 pk) pure returns (address decodedAddr) { + assembly { + let addr := shl(96, pk) + decodedAddr := shr(96, addr) + } +} + +function isSmartContractPK(uint256 pk) pure returns (bool isEncoded) { + assembly { + isEncoded := eq(shr(160, pk), shr(160, ADDR_MASK)) + } +} + +library Sort { + function sort(address[] memory arr) public pure returns (address[] memory) { + LibSort.sort(arr); + return arr; + } +} + +function sortPKsByComputedAddress(uint256[] memory _pks) pure returns (uint256[] memory) { + uint256[] memory sortedPKs = new uint256[](_pks.length); + + address[] memory addresses = new address[](_pks.length); + bytes32[2][] memory accounts = new bytes32[2][](_pks.length); + + for (uint256 i; i < _pks.length; i++) { + address signer = getAddr(_pks[i]); + addresses[i] = signer; + accounts[i][0] = bytes32(abi.encode(signer)); + accounts[i][1] = bytes32(_pks[i]); + } + + addresses = Sort.sort(addresses); + + uint256 found; + for (uint256 j; j < addresses.length; j++) { + address signer = addresses[j]; + uint256 pk; + for (uint256 k; k < accounts.length; k++) { + if (address(uint160(uint256(accounts[k][0]))) == signer) { + pk = uint256(accounts[k][1]); + found++; + } + } + + sortedPKs[j] = pk; + } + + if (found < _pks.length) { + revert("SAFETESTTOOLS: issue with private key sorting, please open a ticket on github"); + } + return sortedPKs; +} + +// collapsed interface that includes comapatibilityfallback handler calls +abstract contract DeployedSafe is GnosisSafe, CompatibilityFallbackHandler { } + +struct AdvancedSafeInitParams { + bool includeFallbackHandler; + uint256 saltNonce; + address setupModulesCall_to; + bytes setupModulesCall_data; + uint256 refundAmount; + address refundToken; + address payable refundReceiver; + bytes initData; +} + +struct SafeInstance { + uint256 instanceId; + uint256[] ownerPKs; + address[] owners; + uint256 threshold; + DeployedSafe safe; +} + +library SafeTestLib { + function execTransaction( + SafeInstance memory instance, + address to, + uint256 value, + bytes memory data, + Enum.Operation operation, + uint256 safeTxGas, + uint256 baseGas, + uint256 gasPrice, + address gasToken, + address refundReceiver, + bytes memory signatures + ) + public + returns (bool) + { + if (instance.owners.length == 0) { + revert("SAFETEST: Instance not initialized. Call _setupSafe() to initialize a test safe"); + } + + bytes32 safeTxHash; + { + uint256 _nonce = instance.safe.nonce(); + safeTxHash = instance.safe.getTransactionHash({ + to: to, + value: value, + data: data, + operation: operation, + safeTxGas: safeTxGas, + baseGas: baseGas, + gasPrice: gasPrice, + gasToken: gasToken, + refundReceiver: refundReceiver, + _nonce: _nonce + }); + } + + if (signatures.length == 0) { + for (uint256 i; i < instance.ownerPKs.length; ++i) { + uint256 pk = instance.ownerPKs[i]; + (uint8 v, bytes32 r, bytes32 s) = Vm(VM_ADDR).sign(pk, safeTxHash); + if (isSmartContractPK(pk)) { + v = 0; + address addr = decodeSmartContractWalletAsAddress(pk); + assembly { + r := addr + } + console.logBytes32(r); + } + signatures = bytes.concat(signatures, abi.encodePacked(r, s, v)); + } + } + + return instance.safe.execTransaction({ + to: to, + value: value, + data: data, + operation: operation, + safeTxGas: safeTxGas, + baseGas: baseGas, + gasPrice: gasPrice, + gasToken: gasToken, + refundReceiver: payable(refundReceiver), + signatures: signatures + }); + } + + function execTransaction( + SafeInstance memory instance, + address to, + uint256 value, + bytes memory data, + Enum.Operation operation + ) + public + returns (bool) + { + return execTransaction(instance, to, value, data, operation, 0, 0, 0, address(0), address(0), ""); + } + + /// @dev performs a noraml "call" + function execTransaction( + SafeInstance memory instance, + address to, + uint256 value, + bytes memory data + ) + public + returns (bool) + { + return execTransaction(instance, to, value, data, Enum.Operation.Call, 0, 0, 0, address(0), address(0), ""); + } + + function enableModule(SafeInstance memory instance, address module) public { + execTransaction( + instance, + address(instance.safe), + 0, + abi.encodeWithSelector(ModuleManager.enableModule.selector, module), + Enum.Operation.Call, + 0, + 0, + 0, + address(0), + address(0), + "" + ); + } + + function disableModule(SafeInstance memory instance, address module) public { + (address[] memory modules,) = instance.safe.getModulesPaginated(SENTINEL_MODULES, 1000); + address prevModule = SENTINEL_MODULES; + bool moduleFound; + for (uint256 i; i < modules.length; i++) { + if (modules[i] == module) { + moduleFound = true; + break; + } + prevModule = modules[i]; + } + if (!moduleFound) revert("SAFETESTTOOLS: cannot disable module that is not enabled"); + + execTransaction( + instance, + address(instance.safe), + 0, + abi.encodeWithSelector(ModuleManager.disableModule.selector, prevModule, module), + Enum.Operation.Call, + 0, + 0, + 0, + address(0), + address(0), + "" + ); + } + + function EIP1271Sign(SafeInstance memory instance, bytes memory data) public { + address signMessageLib = address(new SignMessageLib()); + execTransaction({ + instance: instance, + to: signMessageLib, + value: 0, + data: abi.encodeWithSelector(SignMessageLib.signMessage.selector, data), + operation: Enum.Operation.DelegateCall, + safeTxGas: 0, + baseGas: 0, + gasPrice: 0, + gasToken: address(0), + refundReceiver: payable(address(0)), + signatures: "" + }); + } + + function EIP1271Sign(SafeInstance memory instance, bytes32 digest) public { + EIP1271Sign(instance, abi.encodePacked(digest)); + } + + function signTransaction( + SafeInstance memory instance, + uint256 pk, + address to, + uint256 value, + bytes memory data, + Enum.Operation operation, + uint256 safeTxGas, + uint256 baseGas, + uint256 gasPrice, + address gasToken, + address refundReceiver + ) + public + view + returns (uint8 v, bytes32 r, bytes32 s) + { + bytes32 txDataHash; + { + uint256 _nonce = instance.safe.nonce(); + txDataHash = instance.safe.getTransactionHash({ + to: to, + value: value, + data: data, + operation: operation, + safeTxGas: safeTxGas, + baseGas: baseGas, + gasPrice: gasPrice, + gasToken: gasToken, + refundReceiver: refundReceiver, + _nonce: _nonce + }); + } + + (v, r, s) = Vm(VM_ADDR).sign(pk, txDataHash); + } + + function incrementNonce(SafeInstance memory instance) public returns (uint256 newNonce) { + execTransaction(instance, address(0), 0, "", Enum.Operation.Call, 0, 0, 0, address(0), address(0), ""); + return instance.safe.nonce(); + } +} + +contract SafeTestTools { + using SafeTestLib for SafeInstance; + + GnosisSafe internal singleton = new GnosisSafe(); + GnosisSafeProxyFactory internal proxyFactory = new GnosisSafeProxyFactory(); + CompatibilityFallbackHandler internal handler = new CompatibilityFallbackHandler(); + + SafeInstance[] internal instances; + + /// @dev can be called to reinitialize the singleton, proxyFactory and handler. Useful for forking. + function _initializeSafeTools() internal { + singleton = new GnosisSafe(); + proxyFactory = new GnosisSafeProxyFactory(); + handler = new CompatibilityFallbackHandler(); + } + + function _setupSafe( + uint256[] memory ownerPKs, + uint256 threshold, + uint256 initialBalance, + AdvancedSafeInitParams memory advancedParams + ) + public + returns (SafeInstance memory) + { + uint256[] memory sortedPKs = sortPKsByComputedAddress(ownerPKs); + address[] memory owners = new address[](sortedPKs.length); + + for (uint256 i; i < sortedPKs.length; i++) { + if (isSmartContractPK(sortedPKs[i])) { + owners[i] = decodeSmartContractWalletAsAddress(sortedPKs[i]); + } else { + owners[i] = getAddr(sortedPKs[i]); + } + } + // store the initialization parameters + + bytes memory initData = advancedParams.initData.length > 0 + ? advancedParams.initData + : abi.encodeWithSelector( + GnosisSafe.setup.selector, + owners, + threshold, + advancedParams.setupModulesCall_to, + advancedParams.setupModulesCall_data, + advancedParams.includeFallbackHandler ? address(handler) : address(0), + advancedParams.refundToken, + advancedParams.refundAmount, + advancedParams.refundReceiver + ); + + DeployedSafe safe0 = DeployedSafe( + payable( + advancedParams.saltNonce != 0 + ? proxyFactory.createProxyWithNonce(address(singleton), initData, advancedParams.saltNonce) + : proxyFactory.createProxy(address(singleton), initData) + ) + ); + + SafeInstance memory instance0 = SafeInstance({ + instanceId: instances.length, + ownerPKs: sortedPKs, + owners: owners, + threshold: threshold, + // setup safe ecosystem, singleton, proxy factory, fallback handler, and create a new safe + safe: safe0 + }); + instances.push(instance0); + + Vm(VM_ADDR).deal(address(safe0), initialBalance); + + return instance0; + } + + function _setupSafe( + uint256[] memory ownerPKs, + uint256 threshold, + uint256 initialBalance + ) + public + returns (SafeInstance memory) + { + return _setupSafe( + ownerPKs, + threshold, + initialBalance, + AdvancedSafeInitParams({ + includeFallbackHandler: true, + initData: "", + saltNonce: 0, + setupModulesCall_to: address(0), + setupModulesCall_data: "", + refundAmount: 0, + refundToken: address(0), + refundReceiver: payable(address(0)) + }) + ); + } + + function _setupSafe(uint256[] memory ownerPKs, uint256 threshold) public returns (SafeInstance memory) { + return _setupSafe( + ownerPKs, + threshold, + 10000 ether, + AdvancedSafeInitParams({ + includeFallbackHandler: true, + initData: "", + saltNonce: 0, + setupModulesCall_to: address(0), + setupModulesCall_data: "", + refundAmount: 0, + refundToken: address(0), + refundReceiver: payable(address(0)) + }) + ); + } + + function _setupSafe() public returns (SafeInstance memory) { + string[3] memory users; + users[0] = "SAFETEST: Signer 0"; + users[1] = "SAFETEST: Signer 1"; + users[2] = "SAFETEST: Signer 2"; + + uint256[] memory defaultPKs = new uint256[](3); + defaultPKs[0] = 0xac0974bec39a17e36ba4a6b4d238ff944bacb478cbed5efcae784d7bf4f2ff80; + defaultPKs[1] = 0x59c6995e998f97a5a0044966f0945389dc9e86dae88c7a8412f4603b6b78690d; + defaultPKs[2] = 0x5de4111afa1a4b94908f83103eb1f1706367c2e68ca870fc3fb9a804cdab365a; + + for (uint256 i; i < 3; i++) { + Vm(VM_ADDR).label(getAddr(defaultPKs[i]), users[i]); + } + + return _setupSafe( + defaultPKs, + 2, + 10000 ether, + AdvancedSafeInitParams({ + includeFallbackHandler: true, + initData: "", + saltNonce: uint256(keccak256(bytes("SAFE TEST"))), + setupModulesCall_to: address(0), + setupModulesCall_data: "", + refundAmount: 0, + refundToken: address(0), + refundReceiver: payable(address(0)) + }) + ); + } + + function getSafe() public view returns (SafeInstance memory) { + if (instances.length == 0) { + revert("SAFETESTTOOLS: Test Safe has not been deployed, use _setupSafe() calling safe()"); + } + return instances[0]; + } + + function getSafe(address _safe) public view returns (SafeInstance memory) { + for (uint256 i; i < instances.length; ++i) { + if (address(instances[i].safe) == _safe) return instances[i]; + } + revert("SAFETESTTOOLS: Safe instance not found"); + } +} From 5910878d323be4a2d29266832f5a2e51c30db0af Mon Sep 17 00:00:00 2001 From: Maurelian Date: Sat, 30 Sep 2023 23:17:53 -0400 Subject: [PATCH 163/279] safe-tools: Light edits to fix compatibility issues --- .../CompatibilityFallbackHandler_1_3_0.sol | 2 +- .../test/safe-tools/SafeTestTools.sol | 16 ++++++---------- 2 files changed, 7 insertions(+), 11 deletions(-) diff --git a/packages/contracts-bedrock/test/safe-tools/CompatibilityFallbackHandler_1_3_0.sol b/packages/contracts-bedrock/test/safe-tools/CompatibilityFallbackHandler_1_3_0.sol index 68e46e8e88e2..046a0eda43f0 100644 --- a/packages/contracts-bedrock/test/safe-tools/CompatibilityFallbackHandler_1_3_0.sol +++ b/packages/contracts-bedrock/test/safe-tools/CompatibilityFallbackHandler_1_3_0.sol @@ -6,7 +6,7 @@ import "safe-contracts/interfaces/ERC721TokenReceiver.sol"; import "safe-contracts/interfaces/ERC777TokensRecipient.sol"; import "safe-contracts/interfaces/IERC165.sol"; import "safe-contracts/interfaces/ISignatureValidator.sol"; -import "safe-contracts/GnosisSafe.sol"; +import { Safe as GnosisSafe } from "safe-contracts/Safe.sol"; /// @dev NOTE: removed VERSION and NAME due to inheritance conflicts contract DefaultCallbackHandler is ERC1155TokenReceiver, ERC777TokensRecipient, ERC721TokenReceiver, IERC165 { diff --git a/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol b/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol index e09b6ed80677..0b167a61c3d4 100644 --- a/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol +++ b/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol @@ -2,12 +2,12 @@ pragma solidity >=0.7.0 <0.9.0; import "forge-std/Test.sol"; -import "solady/utils/LibSort.sol"; -import "safe-contracts/GnosisSafe.sol"; -import "safe-contracts/proxies/GnosisSafeProxyFactory.sol"; -import "safe-contracts/examples/libraries/SignMessage.sol"; +import "scripts/libraries/LibSort.sol"; +import { ModuleManager, Safe as GnosisSafe } from "safe-contracts/Safe.sol"; +import { SafeProxyFactory as GnosisSafeProxyFactory } from "safe-contracts/proxies/SafeProxyFactory.sol"; +import { Enum } from "safe-contracts/common/Enum.sol"; +import { SignMessageLib } from "safe-contracts/libraries/SignMessageLib.sol"; import "./CompatibilityFallbackHandler_1_3_0.sol"; -import "safe-contracts/examples/libraries/SignMessage.sol"; address constant VM_ADDR = 0x7109709ECfa91a80626fF3989D68f67F5b1DD12D; bytes12 constant ADDR_MASK = 0xffffffffffffffffffffffff; @@ -354,11 +354,7 @@ contract SafeTestTools { ); DeployedSafe safe0 = DeployedSafe( - payable( - advancedParams.saltNonce != 0 - ? proxyFactory.createProxyWithNonce(address(singleton), initData, advancedParams.saltNonce) - : proxyFactory.createProxy(address(singleton), initData) - ) + payable(proxyFactory.createProxyWithNonce(address(singleton), initData, advancedParams.saltNonce)) ); SafeInstance memory instance0 = SafeInstance({ From 3de35b69974b4b230c444f6008ea5f3eb1ff8862 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Sun, 1 Oct 2023 14:31:15 -0400 Subject: [PATCH 164/279] safe-tools: Add setGuard method --- .../test/safe-tools/SafeTestTools.sol | 18 +++++++++++++++++- 1 file changed, 17 insertions(+), 1 deletion(-) diff --git a/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol b/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol index 0b167a61c3d4..6de3906de6fb 100644 --- a/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol +++ b/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol @@ -3,7 +3,7 @@ pragma solidity >=0.7.0 <0.9.0; import "forge-std/Test.sol"; import "scripts/libraries/LibSort.sol"; -import { ModuleManager, Safe as GnosisSafe } from "safe-contracts/Safe.sol"; +import { ModuleManager, GuardManager, Safe as GnosisSafe } from "safe-contracts/Safe.sol"; import { SafeProxyFactory as GnosisSafeProxyFactory } from "safe-contracts/proxies/SafeProxyFactory.sol"; import { Enum } from "safe-contracts/common/Enum.sol"; import { SignMessageLib } from "safe-contracts/libraries/SignMessageLib.sol"; @@ -238,6 +238,22 @@ library SafeTestLib { ); } + function setGuard(SafeInstance memory instance, address guard) public { + execTransaction( + instance, + address(instance.safe), + 0, + abi.encodeWithSelector(GuardManager.setGuard.selector, guard), + Enum.Operation.Call, + 0, + 0, + 0, + address(0), + address(0), + "" + ); + } + function EIP1271Sign(SafeInstance memory instance, bytes memory data) public { address signMessageLib = address(new SignMessageLib()); execTransaction({ From 0d9784bfd2bd69b5ca30c1f27ab9b0c4745d4240 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Sat, 30 Sep 2023 23:36:20 -0400 Subject: [PATCH 165/279] feat(ctb): Simplify check tests with safe-tools --- .../test/LivenessGuard.t.sol | 63 +++++++++ .../test/SafeLivenessChecking.t.sol | 129 ------------------ 2 files changed, 63 insertions(+), 129 deletions(-) create mode 100644 packages/contracts-bedrock/test/LivenessGuard.t.sol delete mode 100644 packages/contracts-bedrock/test/SafeLivenessChecking.t.sol diff --git a/packages/contracts-bedrock/test/LivenessGuard.t.sol b/packages/contracts-bedrock/test/LivenessGuard.t.sol new file mode 100644 index 000000000000..dcfcf52d4c1b --- /dev/null +++ b/packages/contracts-bedrock/test/LivenessGuard.t.sol @@ -0,0 +1,63 @@ +// SPDX-License-Identifier: MIT +pragma solidity 0.8.15; + +import { Test } from "forge-std/Test.sol"; +import { Safe } from "safe-contracts/Safe.sol"; +import { SafeProxyFactory } from "safe-contracts/proxies/SafeProxyFactory.sol"; +import { ModuleManager } from "safe-contracts/base/ModuleManager.sol"; +import { Enum } from "safe-contracts/common/Enum.sol"; +import "test/safe-tools/SafeTestTools.sol"; + +import { LivenessGuard } from "src/Safe/LivenessGuard.sol"; + +// Todo(Maurelian): +// Other tests needed: +// - EIP1271 signatures +// - Signatures from contracts +// - Signatures from non-owners +// - Signers may call directly to prove liveness (must be an owner). +// - Unexpected length of signature data + +contract LivnessGuard_TestInit is Test, SafeTestTools { + using SafeTestLib for SafeInstance; + + LivenessGuard livenessGuard; + SafeInstance safeInstance; + + function setUp() public { + safeInstance = _setupSafe(); + livenessGuard = new LivenessGuard(safeInstance.safe); + safeInstance.enableModule(address(livenessGuard)); + } +} + +contract LivnessGuard_TestCheckTx is LivnessGuard_TestInit { + using SafeTestLib for SafeInstance; + + function test_checkTransaction_succeeds() external { + safeInstance.execTransaction({ to: address(1111), value: 0, data: hex"abba" }); + + // bug signers need to be sorted from low to high + for (uint256 i; i < safeInstance.owners.length; i++) { + assertEq(livenessGuard.lastSigned(safeInstance.owners[i]), block.timestamp); + } + } +} + +contract LivenessGuard_ShowLiveness_Test is LivnessGuard_TestInit { + function test_showLiveness_succeeds() external { + // Cache the caller + address caller = safeInstance.owners[0]; + + // Construct a signers array with just the caller to identify the expected event. + address[] memory signers = new address[](1); + signers[0] = caller; + vm.expectEmit(address(livenessGuard)); + emit SignersRecorded(0x0, signers); + + vm.prank(caller); + livenessGuard.showLiveness(); + + assertEq(livenessGuard.lastSigned(caller), block.timestamp); + } +} diff --git a/packages/contracts-bedrock/test/SafeLivenessChecking.t.sol b/packages/contracts-bedrock/test/SafeLivenessChecking.t.sol deleted file mode 100644 index fff70e690c9d..000000000000 --- a/packages/contracts-bedrock/test/SafeLivenessChecking.t.sol +++ /dev/null @@ -1,129 +0,0 @@ -// SPDX-License-Identifier: MIT -pragma solidity 0.8.15; - -import { Test } from "forge-std/Test.sol"; -import { Safe } from "safe-contracts/Safe.sol"; -import { SafeProxyFactory } from "safe-contracts/proxies/SafeProxyFactory.sol"; -import { ModuleManager } from "safe-contracts/base/ModuleManager.sol"; -import { Enum } from "safe-contracts/common/Enum.sol"; - -import { LivenessGuard } from "src/Safe/LivenessGuard.sol"; - -contract LivnessGuard_TestInit is Test { - struct Signer { - address owner; - uint256 pk; - } - - LivenessGuard livenessGuard; - Safe safe; - string mnemonic = "test test test test test test test test test test test junk"; - - Signer[] signers; - - function newSigner(uint256 index) public returns (Signer memory signer_) { - signer_.pk = vm.deriveKey(mnemonic, uint32(index)); - signer_.owner = vm.addr(signer_.pk); - } - - function signTransaction( - uint256 _pk, - address _to, - uint256 _value, - bytes memory _data - ) - public - view - returns (bytes memory sig_) - { - bytes32 txDataHash; - { - txDataHash = safe.getTransactionHash({ - to: _to, - value: _value, - data: _data, - operation: Enum.Operation.Call, - safeTxGas: 0, - baseGas: 0, - gasPrice: 0, - gasToken: address(0), - refundReceiver: address(0), - _nonce: safe.nonce() - }); - } - - (uint8 v, bytes32 r, bytes32 s) = vm.sign(_pk, txDataHash); - sig_ = abi.encodePacked(v, r, s); - } - - function exec(Signer[] memory _signers, address _to, bytes memory _data) internal { - bytes memory sig; - for (uint256 i; i < _signers.length; i++) { - bytes.concat(sig, signTransaction(_signers[i].pk, address(safe), 0, _data)); - } - safe.execTransaction({ - to: _to, - value: 0, - data: _data, - operation: Enum.Operation.Call, - safeTxGas: 0, - baseGas: 0, - gasPrice: 0, - gasToken: address(0), - refundReceiver: payable(0), - signatures: sig - }); - } - - // @dev Create a new Safe instance with a minimimal proxy and implementation. - function newSafe(Signer[] memory _signers) internal returns (Safe safe_) { - SafeProxyFactory safeProxyFactory = new SafeProxyFactory(); - Safe safeSingleton = new Safe(); - - bytes memory initData = abi.encodeWithSelector( - Safe.setup.selector, _signers, 2, address(0), hex"", address(0), address(0), 0, address(0) - ); - - safe_ = Safe(payable(safeProxyFactory.createProxyWithNonce(address(safeSingleton), initData, block.timestamp))); - } - - function setUp() public { - // Create 3 signers - for (uint256 i; i < 3; i++) { - signers.push(newSigner(i)); - } - - Signer[] memory signers_ = signers; - safe = newSafe(signers_); - livenessGuard = new LivenessGuard(safe); - - // enable the module - bytes memory data = abi.encodeCall(ModuleManager.enableModule, (address(livenessGuard))); - bytes memory sig1 = signTransaction(signers[0].pk, address(safe), 0, data); - bytes memory sig2 = signTransaction(signers[1].pk, address(safe), 0, data); - bytes memory sigs = bytes.concat(sig1, sig2); - safe.execTransaction({ - to: address(safe), - value: 0, - data: data, - operation: Enum.Operation.Call, - safeTxGas: 0, - baseGas: 0, - gasPrice: 0, - gasToken: address(0), - refundReceiver: payable(0), - signatures: sigs - }); - } -} - -contract LivnessGuard_TestCheckTx is LivnessGuard_TestInit { - function test_checkTransaction_succeeds() external { - Signer[] memory signers_ = signers; - exec(signers, address(1111), hex"abba"); - - for (uint256 i; i < signers.length; i++) { - assertEq(livenessGuard.lastSigned(signers[i].owner), block.timestamp); - } - } -} From da17f609bf44895b2fc4a3ee2f82292fce064754 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Sun, 1 Oct 2023 14:38:28 -0400 Subject: [PATCH 166/279] feat(ctb): Working signature recording --- .../src/Safe/LivenessGuard.sol | 26 +++++++++---------- .../test/LivenessGuard.t.sol | 3 +-- 2 files changed, 13 insertions(+), 16 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol index a1c99fb689b3..719259cce153 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol @@ -15,12 +15,11 @@ contract LivenessGuard is SignatureDecoder, BaseGuard { } /// @notice We just need to satisfy the BaseGuard interfae, but we don't actually need to use this method. - function checkAfterExecution(bytes32 txHash, bool success) external { + function checkAfterExecution(bytes32, bool) external pure { return; } - /// @notice This checkTransaction implementation records the most recent time which any owner has signed a - /// transaction. + /// @notice Records the most recent time which any owner has signed a transaction. function checkTransaction( address to, uint256 value, @@ -32,7 +31,7 @@ contract LivenessGuard is SignatureDecoder, BaseGuard { address gasToken, address payable refundReceiver, bytes memory signatures, - address msgSender + address ) external { @@ -43,7 +42,7 @@ contract LivenessGuard is SignatureDecoder, BaseGuard { } // This is a bit of a hack, maybe just replicate the functionality here rather than calling home - bytes memory txHashData = Safe(payable(msg.sender)).encodeTransactionData( + bytes32 txHash = Safe(payable(msg.sender)).getTransactionHash( // Transaction info to, value, @@ -56,23 +55,22 @@ contract LivenessGuard is SignatureDecoder, BaseGuard { gasToken, refundReceiver, // Signature info - Safe(payable(msg.sender)).nonce() // check that this works ); - address[] memory signers = _getNSigners(keccak256(txHashData), signatures); + address[] memory signers = _getNSigners(txHash, signatures); for (uint256 i = 0; i < signers.length; i++) { lastSigned[signers[i]] = block.timestamp; } } - function _getNSigners(bytes32 dataHash, bytes memory signatures) internal returns (address[] memory _owners) { - uint256 numSignatures = signatures.length / 65; // division OK? + /// @notice Exctract the signers from a set of signatures. + function _getNSigners(bytes32 dataHash, bytes memory signatures) internal pure returns (address[] memory _owners) { + uint256 numSignatures = signatures.length / 65; _owners = new address[](numSignatures); - // The following code is extracted from the Safe.checkNSignatures() method. It removes the signature validation - // code, - // and keeps only the parsing code necessary to extract the owner addresses from the signatures. - // We do not double check if the owner derived from a signature is valid. As tHis is handled in - // the final require statement of Safe.checkNSignatures(). + /// The following code is extracted from the Safe.checkNSignatures() method. It removes the signature + /// validation code, and keeps only the parsing code necessary to extract the owner addresses from the + /// signatures. We do not double check if the owner derived from a signature is valid. As this is handled + /// in the final require statement of Safe.checkNSignatures(). address currentOwner; uint8 v; bytes32 r; diff --git a/packages/contracts-bedrock/test/LivenessGuard.t.sol b/packages/contracts-bedrock/test/LivenessGuard.t.sol index dcfcf52d4c1b..c36247a16acd 100644 --- a/packages/contracts-bedrock/test/LivenessGuard.t.sol +++ b/packages/contracts-bedrock/test/LivenessGuard.t.sol @@ -27,7 +27,7 @@ contract LivnessGuard_TestInit is Test, SafeTestTools { function setUp() public { safeInstance = _setupSafe(); livenessGuard = new LivenessGuard(safeInstance.safe); - safeInstance.enableModule(address(livenessGuard)); + safeInstance.setGuard(address(livenessGuard)); } } @@ -37,7 +37,6 @@ contract LivnessGuard_TestCheckTx is LivnessGuard_TestInit { function test_checkTransaction_succeeds() external { safeInstance.execTransaction({ to: address(1111), value: 0, data: hex"abba" }); - // bug signers need to be sorted from low to high for (uint256 i; i < safeInstance.owners.length; i++) { assertEq(livenessGuard.lastSigned(safeInstance.owners[i]), block.timestamp); } From 8e2a6dec82dd3cde592a3589b306256a1dceab5e Mon Sep 17 00:00:00 2001 From: Maurelian Date: Mon, 2 Oct 2023 12:00:31 -0400 Subject: [PATCH 167/279] safe-tools: Add some natspec comments --- .../test/safe-tools/SafeTestTools.sol | 21 ++++++++++++++++++- 1 file changed, 20 insertions(+), 1 deletion(-) diff --git a/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol b/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol index 6de3906de6fb..138e6feb215b 100644 --- a/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol +++ b/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol @@ -12,10 +12,12 @@ import "./CompatibilityFallbackHandler_1_3_0.sol"; address constant VM_ADDR = 0x7109709ECfa91a80626fF3989D68f67F5b1DD12D; bytes12 constant ADDR_MASK = 0xffffffffffffffffffffffff; +/// @dev Get the address from a private key function getAddr(uint256 pk) pure returns (address) { return Vm(VM_ADDR).addr(pk); } +/// @dev Encode a smart contract wallet as a private key function encodeSmartContractWalletAsPK(address addr) pure returns (uint256 encodedPK) { assembly { let addr_b32 := addr @@ -23,6 +25,7 @@ function encodeSmartContractWalletAsPK(address addr) pure returns (uint256 encod } } +/// @dev Decode a smart contract wallet as an address from a private key function decodeSmartContractWalletAsAddress(uint256 pk) pure returns (address decodedAddr) { assembly { let addr := shl(96, pk) @@ -30,6 +33,7 @@ function decodeSmartContractWalletAsAddress(uint256 pk) pure returns (address de } } +/// @dev Checks if a private key is an encoded smart contract address function isSmartContractPK(uint256 pk) pure returns (bool isEncoded) { assembly { isEncoded := eq(shr(160, pk), shr(160, ADDR_MASK)) @@ -37,12 +41,14 @@ function isSmartContractPK(uint256 pk) pure returns (bool isEncoded) { } library Sort { + /// @dev Sorts an array of addresses in place function sort(address[] memory arr) public pure returns (address[] memory) { LibSort.sort(arr); return arr; } } +/// @dev Sorts an array of private keys by the computed address function sortPKsByComputedAddress(uint256[] memory _pks) pure returns (uint256[] memory) { uint256[] memory sortedPKs = new uint256[](_pks.length); @@ -101,6 +107,8 @@ struct SafeInstance { } library SafeTestLib { + /// @dev A wrapper for the full execTransaction method, if no signatures are provided it will + /// generate them for all owners. function execTransaction( SafeInstance memory instance, address to, @@ -168,6 +176,7 @@ library SafeTestLib { }); } + /// @dev Executes either a CALL or DELEGATECALL transaction. function execTransaction( SafeInstance memory instance, address to, @@ -181,7 +190,7 @@ library SafeTestLib { return execTransaction(instance, to, value, data, operation, 0, 0, 0, address(0), address(0), ""); } - /// @dev performs a noraml "call" + /// @dev Executes a CALL transaction. function execTransaction( SafeInstance memory instance, address to, @@ -194,6 +203,7 @@ library SafeTestLib { return execTransaction(instance, to, value, data, Enum.Operation.Call, 0, 0, 0, address(0), address(0), ""); } + /// @dev Enables a module on the Safe. function enableModule(SafeInstance memory instance, address module) public { execTransaction( instance, @@ -210,6 +220,7 @@ library SafeTestLib { ); } + /// @dev Disables a module on the Safe. function disableModule(SafeInstance memory instance, address module) public { (address[] memory modules,) = instance.safe.getModulesPaginated(SENTINEL_MODULES, 1000); address prevModule = SENTINEL_MODULES; @@ -238,6 +249,9 @@ library SafeTestLib { ); } + /// @dev Sets the guard address on the Safe. Unlike modules there can only be one guard, so + /// this method will remove the previous guard. If the guard is set to the 0 address, the + /// guard will be disabled. function setGuard(SafeInstance memory instance, address guard) public { execTransaction( instance, @@ -254,6 +268,7 @@ library SafeTestLib { ); } + /// @dev Signs message data using EIP1271: Standard Signature Validation Method for Contracts function EIP1271Sign(SafeInstance memory instance, bytes memory data) public { address signMessageLib = address(new SignMessageLib()); execTransaction({ @@ -271,10 +286,12 @@ library SafeTestLib { }); } + /// @dev Signs a data hash using EIP1271: Standard Signature Validation Method for Contracts function EIP1271Sign(SafeInstance memory instance, bytes32 digest) public { EIP1271Sign(instance, abi.encodePacked(digest)); } + /// @dev Sign a transaction as a safe owner with a private key. function signTransaction( SafeInstance memory instance, uint256 pk, @@ -312,12 +329,14 @@ library SafeTestLib { (v, r, s) = Vm(VM_ADDR).sign(pk, txDataHash); } + /// @dev Increments the nonce of the Safe by sending an empty transaction. function incrementNonce(SafeInstance memory instance) public returns (uint256 newNonce) { execTransaction(instance, address(0), 0, "", Enum.Operation.Call, 0, 0, 0, address(0), address(0), ""); return instance.safe.nonce(); } } +/// @dev SafeTestTools implements a set of helper functions for testing Safe contracts. contract SafeTestTools { using SafeTestLib for SafeInstance; From 6b67dc7c80a73069c5842b0c477a3841a238e1d4 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Mon, 2 Oct 2023 12:08:40 -0400 Subject: [PATCH 168/279] feat(ctb): Emit SignersRecorded event --- packages/contracts-bedrock/src/Safe/LivenessGuard.sol | 9 ++++++++- packages/contracts-bedrock/test/LivenessGuard.t.sol | 5 +++++ 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol index 719259cce153..ccbebd1e86e2 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol @@ -7,6 +7,10 @@ import { SignatureDecoder } from "safe-contracts/common/SignatureDecoder.sol"; import { Enum } from "safe-contracts/common/Enum.sol"; contract LivenessGuard is SignatureDecoder, BaseGuard { + /// @notice Emitted when a new set of signers is recorded. + /// @param signers An arrary of signer addresses. + event SignersRecorded(bytes32 indexed txHash, address[] signers); + Safe public safe; mapping(address => uint256) public lastSigned; @@ -41,7 +45,8 @@ contract LivenessGuard is SignatureDecoder, BaseGuard { // TODO(Maurelian): Figure out how to best address this. } - // This is a bit of a hack, maybe just replicate the functionality here rather than calling home + // This call will reenter to the Safe which is calling it. This is OK because it is only reading the + // nonce, and using the getTransactionHash() method. bytes32 txHash = Safe(payable(msg.sender)).getTransactionHash( // Transaction info to, @@ -55,11 +60,13 @@ contract LivenessGuard is SignatureDecoder, BaseGuard { gasToken, refundReceiver, // Signature info + Safe(payable(msg.sender)).nonce() - 1 ); address[] memory signers = _getNSigners(txHash, signatures); for (uint256 i = 0; i < signers.length; i++) { lastSigned[signers[i]] = block.timestamp; } + emit SignersRecorded(txHash, signers); } /// @notice Exctract the signers from a set of signatures. diff --git a/packages/contracts-bedrock/test/LivenessGuard.t.sol b/packages/contracts-bedrock/test/LivenessGuard.t.sol index c36247a16acd..198ed9a2dc72 100644 --- a/packages/contracts-bedrock/test/LivenessGuard.t.sol +++ b/packages/contracts-bedrock/test/LivenessGuard.t.sol @@ -21,6 +21,8 @@ import { LivenessGuard } from "src/Safe/LivenessGuard.sol"; contract LivnessGuard_TestInit is Test, SafeTestTools { using SafeTestLib for SafeInstance; + event SignersRecorded(bytes32 indexed txHash, address[] signers); + LivenessGuard livenessGuard; SafeInstance safeInstance; @@ -35,6 +37,9 @@ contract LivnessGuard_TestCheckTx is LivnessGuard_TestInit { using SafeTestLib for SafeInstance; function test_checkTransaction_succeeds() external { + // Don't check topic1 so that we can avoid the ugly txHash calculation. + vm.expectEmit(false, true, true, true, address(livenessGuard)); + emit SignersRecorded(0x0, safeInstance.owners); safeInstance.execTransaction({ to: address(1111), value: 0, data: hex"abba" }); for (uint256 i; i < safeInstance.owners.length; i++) { From 57a5f220a20531ec95bb86326e3a0ad099e9ab8c Mon Sep 17 00:00:00 2001 From: Maurelian Date: Mon, 2 Oct 2023 12:09:20 -0400 Subject: [PATCH 169/279] feat(ctb): Allow signers to call directly to prove liveness From fa0db2a672db989bf09a85ee954ccee1a6a2fb90 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Mon, 2 Oct 2023 12:31:37 -0400 Subject: [PATCH 170/279] feat(ctb): Add showLiveness method --- .../contracts-bedrock/src/Safe/LivenessGuard.sol | 14 ++++++++++++++ .../contracts-bedrock/test/LivenessGuard.t.sol | 2 +- 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol index ccbebd1e86e2..fb32fcb7a187 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol @@ -107,4 +107,18 @@ contract LivenessGuard is SignatureDecoder, BaseGuard { _owners[i] = currentOwner; } } + + /// @notice Enables an owner to demonstrate liveness by calling this method directly. + /// This is useful for owners who have not recently signed a transaction via the Safe. + function showLiveness() external { + require(safe.isOwner(msg.sender), "LivenessGuard: only Safe owners may demontstrate liveness"); + lastSigned[msg.sender] = block.timestamp; + address[] memory signers = new address[](1); + signers[0] = msg.sender; + + // todo(maurelian): Is there any need for this event to be differentiated from the one emitted in + // checkTransaction? + // Technically the 0x0 txHash does serve to identiy a call to this method. + emit SignersRecorded(0x0, signers); + } } diff --git a/packages/contracts-bedrock/test/LivenessGuard.t.sol b/packages/contracts-bedrock/test/LivenessGuard.t.sol index 198ed9a2dc72..823311888932 100644 --- a/packages/contracts-bedrock/test/LivenessGuard.t.sol +++ b/packages/contracts-bedrock/test/LivenessGuard.t.sol @@ -33,7 +33,7 @@ contract LivnessGuard_TestInit is Test, SafeTestTools { } } -contract LivnessGuard_TestCheckTx is LivnessGuard_TestInit { +contract LivnessGuard_CheckTx_Test is LivnessGuard_TestInit { using SafeTestLib for SafeInstance; function test_checkTransaction_succeeds() external { From a56e6c7576adad9340f0b4bfaef19c312629e5e2 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Wed, 4 Oct 2023 01:25:28 -0400 Subject: [PATCH 171/279] feat(ctb): Add liveness module --- .../src/Safe/LivenessGuard.sol | 19 ++- .../src/Safe/LivenessModule.sol | 124 ++++++++++++++++++ .../test/LivenessModule.t.sol | 51 +++++++ 3 files changed, 190 insertions(+), 4 deletions(-) create mode 100644 packages/contracts-bedrock/test/LivenessModule.t.sol diff --git a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol index fb32fcb7a187..c9408d479830 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol @@ -2,7 +2,8 @@ pragma solidity 0.8.15; import { Safe } from "safe-contracts/Safe.sol"; -import { BaseGuard } from "safe-contracts/base/GuardManager.sol"; +import { BaseGuard, GuardManager } from "safe-contracts/base/GuardManager.sol"; +import { ModuleManager } from "safe-contracts/base/ModuleManager.sol"; import { SignatureDecoder } from "safe-contracts/common/SignatureDecoder.sol"; import { Enum } from "safe-contracts/common/Enum.sol"; @@ -40,9 +41,19 @@ contract LivenessGuard is SignatureDecoder, BaseGuard { external { require(msg.sender == address(safe), "LivenessGuard: only Safe can call this function"); - if (to == address(safe) && data[0:4] == bytes4(keccak256("setGuard(address)"))) { - // We can't allow the guard to be disabled, or else the upgrade delay can be bypassed. - // TODO(Maurelian): Figure out how to best address this. + + // There are a number of ways in which we need to constrain this safe so that it cannot remove + // this guard, nor the LivenessModule. + // TODO(Maurelian): Figure out how to best address this. The following is just intended to outline the + // known mathods by which a Safe could remove the liveness checks. + // TODO(Maurelian): Do we _need_ to have this feature at all? + bytes4 dataSig = bytes4(data); + if ( + to == address(safe) + && (dataSig == GuardManager.setGuard.selector || dataSig == ModuleManager.enableModule.selector) + || operation == Enum.Operation.DelegateCall + ) { + revert("LivenessGuard: cannot remove LivenessGuard or LivenessModule"); } // This call will reenter to the Safe which is calling it. This is OK because it is only reading the diff --git a/packages/contracts-bedrock/src/Safe/LivenessModule.sol b/packages/contracts-bedrock/src/Safe/LivenessModule.sol index 8b137891791f..09be58314472 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessModule.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessModule.sol @@ -1 +1,125 @@ +// SPDX-License-Identifier: MIT +pragma solidity 0.8.15; +import { Safe } from "safe-contracts/Safe.sol"; +import { Enum } from "safe-contracts/common/Enum.sol"; +import { OwnerManager } from "safe-contracts/base/OwnerManager.sol"; +import { LivenessGuard } from "src/Safe/LivenessGuard.sol"; + +/// @title LivenessModule +/// @notice This module is intended to be used in conjunction with the LivenessGuard. It should be able to +/// execute a transaction on the Safe in only a small number of cases: +contract LivenessModule { + /// @notice The Safe contract instance + Safe public safe; + + /// @notice The LivenessGuard contract instance + LivenessGuard public livenessGuard; + + /// @notice The interval, in seconds, during which an owner must have demonstrated liveness + uint256 public livenessInterval; + + /// @notice The minimum number of owners before ownership of the safe is transferred to the fallback owner. + uint256 public minOwners; + + /// @notice The fallback owner of the Safe + address public fallbackOwner; + + // Constructor to initialize the Safe and baseModule instances + constructor( + Safe _safe, + LivenessGuard _livenessGuard, + uint256 _livenessInterval, + uint256 _minOwners, + address _fallbackOwner + ) { + safe = _safe; + livenessGuard = _livenessGuard; + livenessInterval = _livenessInterval; + minOwners = _minOwners; + fallbackOwner = _fallbackOwner; + } + + /// @notice This function can be called by anyone to remove an owner that has not signed a transaction + /// during the livness interval. If the number of owners drops below + function removeOwner(address owner) external { + // Check that the owner has not signed a transaction in the last 30 days + require( + livenessGuard.lastSigned(owner) < block.timestamp - livenessInterval, + "LivenessModule: owner has signed recently" + ); + + // Calculate the new threshold + uint256 numOwnersAfter = safe.getOwners().length - 1; + uint256 thresholdAfter = get75PercentThreshold(numOwnersAfter); + if (numOwnersAfter >= 8) { + safe.execTransactionFromModule({ + to: address(safe), + value: 0, + data: abi.encodeCall( + // Call the Safe to remove the owner + OwnerManager.removeOwner, + (getPrevOwner(owner), owner, thresholdAfter) + ), + operation: Enum.Operation.Call + }); + } else { + // The number of owners is dangerously low, so we wish to transfer the ownership of this Safe to a new + // to the fallback owner. + // Remove owners one at a time starting from the last owner. + // Since we're removing them in order, the ordering will remain constant, + // and we shouldn't need to query the list of owners again. + address[] memory owners = safe.getOwners(); + for (uint256 i = owners.length - 1; i >= 0; i--) { + address currentOwner = owners[i]; + if (currentOwner != address(this)) { + safe.execTransactionFromModule({ + to: address(safe), + value: 0, + data: abi.encodeCall( + // Call the Safe to remove the owner + OwnerManager.removeOwner, + ( + getPrevOwner(currentOwner), + currentOwner, + 1 // The threshold is 1 because we are removing all owners except the fallback owner + ) + ), + operation: Enum.Operation.Call + }); + } + } + + // Add the fallback owner as the sole owner of the Safe + safe.execTransactionFromModule({ + to: address(safe), + value: 0, + data: abi.encodeCall(OwnerManager.addOwnerWithThreshold, (fallbackOwner, 1)), + operation: Enum.Operation.Call + }); + + address[] memory ownersAfter = safe.getOwners(); + require( + ownersAfter.length == 1 && ownersAfter[0] == fallbackOwner, + "LivenessModule: fallback owner was not added as the sole owner" + ); + } + } + + /// @notice Get the previous owner in the linked list of owners + function getPrevOwner(address owner) public view returns (address prevOwner_) { + address[] memory owners = safe.getOwners(); + prevOwner_ = address(0); + for (uint256 i = 0; i < owners.length; i++) { + if (owners[i] == owner) { + prevOwner_ = owners[i - 1]; + break; + } + } + } + + /// @notice For a given number of owners, return the lowest threshold which is greater than 75. + function get75PercentThreshold(uint256 _numOwners) public view returns (uint256 threshold_) { + threshold_ = (_numOwners * 75 + 99) / 100; + } +} diff --git a/packages/contracts-bedrock/test/LivenessModule.t.sol b/packages/contracts-bedrock/test/LivenessModule.t.sol new file mode 100644 index 000000000000..151c3aea61b5 --- /dev/null +++ b/packages/contracts-bedrock/test/LivenessModule.t.sol @@ -0,0 +1,51 @@ +// SPDX-License-Identifier: MIT +pragma solidity 0.8.15; + +import { Test, StdUtils } from "forge-std/Test.sol"; +import { Safe } from "safe-contracts/Safe.sol"; +import { SafeProxyFactory } from "safe-contracts/proxies/SafeProxyFactory.sol"; +import { ModuleManager } from "safe-contracts/base/ModuleManager.sol"; +import { Enum } from "safe-contracts/common/Enum.sol"; +import "test/safe-tools/SafeTestTools.sol"; + +import { LivenessModule } from "src/Safe/LivenessModule.sol"; +import { LivenessGuard } from "src/Safe/LivenessGuard.sol"; + +contract LivnessModule_TestInit is Test, SafeTestTools { + using SafeTestLib for SafeInstance; + + event SignersRecorded(bytes32 indexed txHash, address[] signers); + + LivenessModule livenessModule; + LivenessGuard livenessGuard; + SafeInstance safeInstance; + + function makeKeys(uint256 num) public pure returns (uint256[] memory) { + uint256[] memory keys = new uint256[](num); + for (uint256 i; i < num; i++) { + keys[i] = uint256(keccak256(abi.encodePacked(i))); + } + } + + function setUp() public { + // Create a Safe with 10 owners + uint256[] memory keys = makeKeys(10); + safeInstance = _setupSafe(keys, 8); + livenessGuard = new LivenessGuard(safeInstance.safe); + livenessModule = new LivenessModule({ + _safe: safeInstance.safe, + _livenessGuard: livenessGuard, + _livenessInterval: 30 days, + _minOwners: 6, + _fallbackOwner: makeAddr("fallbackOwner") + }); + safeInstance.enableModule(address(livenessModule)); + } +} + +contract LivenessModule_RemoveOwner_Test is LivnessModule_TestInit { + function test_removeOwner_succeeds() external { + vm.warp(block.timestamp + 30 days); + livenessModule.removeOwner(safeInstance.owners[0]); + } +} From f683806754db0b7be95f6013560d650896de4c30 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Wed, 4 Oct 2023 10:06:30 -0400 Subject: [PATCH 172/279] feat(ctb): Add _verifyFinal state in removeOwner() --- .../src/Safe/LivenessModule.sol | 63 +++++++++++++------ 1 file changed, 43 insertions(+), 20 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessModule.sol b/packages/contracts-bedrock/src/Safe/LivenessModule.sol index 09be58314472..6db3bf9bea12 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessModule.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessModule.sol @@ -6,6 +6,9 @@ import { Enum } from "safe-contracts/common/Enum.sol"; import { OwnerManager } from "safe-contracts/base/OwnerManager.sol"; import { LivenessGuard } from "src/Safe/LivenessGuard.sol"; +// TODO(maurelian): remove me +import { console2 as console } from "forge-std/console2.sol"; + /// @title LivenessModule /// @notice This module is intended to be used in conjunction with the LivenessGuard. It should be able to /// execute a transaction on the Safe in only a small number of cases: @@ -50,28 +53,38 @@ contract LivenessModule { ); // Calculate the new threshold - uint256 numOwnersAfter = safe.getOwners().length - 1; - uint256 thresholdAfter = get75PercentThreshold(numOwnersAfter); - if (numOwnersAfter >= 8) { + address[] memory owners = safe.getOwners(); + uint256 numOwners = owners.length - 1; + uint256 thresholdAfter; + if (numOwners > minOwners) { + // Preserves the invariant that the Safe has at least 8 owners + + thresholdAfter = get75PercentThreshold(numOwners); + console.log("removing one owner. numOwners: %s, thresholdAfter: %s", numOwners, thresholdAfter); safe.execTransactionFromModule({ to: address(safe), value: 0, data: abi.encodeCall( // Call the Safe to remove the owner OwnerManager.removeOwner, - (getPrevOwner(owner), owner, thresholdAfter) + (getPrevOwner(owner, owners), owner, thresholdAfter) ), operation: Enum.Operation.Call }); } else { + console.log("removing all owners. numOwnersAfter: %s", numOwners); // The number of owners is dangerously low, so we wish to transfer the ownership of this Safe to a new // to the fallback owner. + + // The threshold will be 1 because we are removing all owners except the fallback owner + thresholdAfter = 1; + // Remove owners one at a time starting from the last owner. // Since we're removing them in order, the ordering will remain constant, // and we shouldn't need to query the list of owners again. - address[] memory owners = safe.getOwners(); for (uint256 i = owners.length - 1; i >= 0; i--) { address currentOwner = owners[i]; + address prevOwner = getPrevOwner(currentOwner, owners); if (currentOwner != address(this)) { safe.execTransactionFromModule({ to: address(safe), @@ -79,11 +92,7 @@ contract LivenessModule { data: abi.encodeCall( // Call the Safe to remove the owner OwnerManager.removeOwner, - ( - getPrevOwner(currentOwner), - currentOwner, - 1 // The threshold is 1 because we are removing all owners except the fallback owner - ) + (prevOwner, currentOwner, 1) ), operation: Enum.Operation.Call }); @@ -97,21 +106,35 @@ contract LivenessModule { data: abi.encodeCall(OwnerManager.addOwnerWithThreshold, (fallbackOwner, 1)), operation: Enum.Operation.Call }); - - address[] memory ownersAfter = safe.getOwners(); - require( - ownersAfter.length == 1 && ownersAfter[0] == fallbackOwner, - "LivenessModule: fallback owner was not added as the sole owner" - ); } + _verifyFinalState(); } - /// @notice Get the previous owner in the linked list of owners - function getPrevOwner(address owner) public view returns (address prevOwner_) { + /// @notice A FREI-PI invariant check enforcing requirements on number of owners and threshold. + function _verifyFinalState() internal view { address[] memory owners = safe.getOwners(); - prevOwner_ = address(0); + uint256 numOwners = owners.length; + require( + (numOwners == 1 && owners[0] == fallbackOwner) || (numOwners >= minOwners), + "LivenessModule: Safe must have at least 1 owner or minOwners" + ); + + // Check that the threshold is correct + uint256 threshold = safe.getThreshold(); + require( + threshold == get75PercentThreshold(numOwners) || (numOwners == 1 && threshold == 1), + "LivenessModule: threshold must be 75% of the number of owners, or 1 if there is only 1 owner" + ); + } + + /// @notice Get the previous owner in the linked list of owners + function getPrevOwner(address owner, address[] memory owners) public pure returns (address prevOwner_) { for (uint256 i = 0; i < owners.length; i++) { if (owners[i] == owner) { + if (i == 0) { + prevOwner_ = address(0x1); // OwnerManager.SENTINEL_OWNERS + break; + } prevOwner_ = owners[i - 1]; break; } @@ -119,7 +142,7 @@ contract LivenessModule { } /// @notice For a given number of owners, return the lowest threshold which is greater than 75. - function get75PercentThreshold(uint256 _numOwners) public view returns (uint256 threshold_) { + function get75PercentThreshold(uint256 _numOwners) public pure returns (uint256 threshold_) { threshold_ = (_numOwners * 75 + 99) / 100; } } From baa0fa5b169660f929a65352b3eb392d6ec4a452 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Wed, 4 Oct 2023 10:07:26 -0400 Subject: [PATCH 173/279] test(ctb): Add test for transferring to fallback owner --- .../test/LivenessModule.t.sol | 18 ++++++++++++++---- 1 file changed, 14 insertions(+), 4 deletions(-) diff --git a/packages/contracts-bedrock/test/LivenessModule.t.sol b/packages/contracts-bedrock/test/LivenessModule.t.sol index 151c3aea61b5..f375ee6af69f 100644 --- a/packages/contracts-bedrock/test/LivenessModule.t.sol +++ b/packages/contracts-bedrock/test/LivenessModule.t.sol @@ -20,10 +20,10 @@ contract LivnessModule_TestInit is Test, SafeTestTools { LivenessGuard livenessGuard; SafeInstance safeInstance; - function makeKeys(uint256 num) public pure returns (uint256[] memory) { - uint256[] memory keys = new uint256[](num); + function makeKeys(uint256 num) public pure returns (uint256[] memory keys_) { + keys_ = new uint256[](num); for (uint256 i; i < num; i++) { - keys[i] = uint256(keccak256(abi.encodePacked(i))); + keys_[i] = uint256(keccak256(abi.encodePacked(i))); } } @@ -44,8 +44,18 @@ contract LivnessModule_TestInit is Test, SafeTestTools { } contract LivenessModule_RemoveOwner_Test is LivnessModule_TestInit { - function test_removeOwner_succeeds() external { + function test_removeOwner_oneOwner_succeeds() external { + uint256 ownersBefore = safeInstance.owners.length; vm.warp(block.timestamp + 30 days); livenessModule.removeOwner(safeInstance.owners[0]); + assertEq(safeInstance.safe.getOwners().length, ownersBefore - 1); + } + + function test_removeOwner_allOwners_succeeds() external { + vm.warp(block.timestamp + 30 days); + // The safe is initialized with 10 owners, so we need to remove 3 to get below the minOwners threshold + livenessModule.removeOwner(safeInstance.owners[0]); + livenessModule.removeOwner(safeInstance.owners[1]); + livenessModule.removeOwner(safeInstance.owners[2]); } } From 0b690016bb051f6b8f31201a22c0d2dc5588e367 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Wed, 4 Oct 2023 10:38:11 -0400 Subject: [PATCH 174/279] chore(ctb): Attribution for safe-tools --- packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol | 3 +++ 1 file changed, 3 insertions(+) diff --git a/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol b/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol index 138e6feb215b..0730a6bc3bc8 100644 --- a/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol +++ b/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol @@ -9,6 +9,9 @@ import { Enum } from "safe-contracts/common/Enum.sol"; import { SignMessageLib } from "safe-contracts/libraries/SignMessageLib.sol"; import "./CompatibilityFallbackHandler_1_3_0.sol"; +// Tools to simplify testing Safe contracts +// Author: Colin Nielsen (https://github.com/colinnielsen/safe-tools) + address constant VM_ADDR = 0x7109709ECfa91a80626fF3989D68f67F5b1DD12D; bytes12 constant ADDR_MASK = 0xffffffffffffffffffffffff; From 0e2030249df3708722270528b3cfe90251c5f811 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Fri, 6 Oct 2023 14:13:38 -0400 Subject: [PATCH 175/279] feat(ctb): Remove notes/poc of guard removal prevention --- .../contracts-bedrock/src/Safe/LivenessGuard.sol | 14 -------------- 1 file changed, 14 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol index c9408d479830..e95f15dfe13e 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol @@ -42,20 +42,6 @@ contract LivenessGuard is SignatureDecoder, BaseGuard { { require(msg.sender == address(safe), "LivenessGuard: only Safe can call this function"); - // There are a number of ways in which we need to constrain this safe so that it cannot remove - // this guard, nor the LivenessModule. - // TODO(Maurelian): Figure out how to best address this. The following is just intended to outline the - // known mathods by which a Safe could remove the liveness checks. - // TODO(Maurelian): Do we _need_ to have this feature at all? - bytes4 dataSig = bytes4(data); - if ( - to == address(safe) - && (dataSig == GuardManager.setGuard.selector || dataSig == ModuleManager.enableModule.selector) - || operation == Enum.Operation.DelegateCall - ) { - revert("LivenessGuard: cannot remove LivenessGuard or LivenessModule"); - } - // This call will reenter to the Safe which is calling it. This is OK because it is only reading the // nonce, and using the getTransactionHash() method. bytes32 txHash = Safe(payable(msg.sender)).getTransactionHash( From 8032387236b0ff41dca66b8f58132b1a1db3ddcc Mon Sep 17 00:00:00 2001 From: Maurelian Date: Fri, 6 Oct 2023 14:14:22 -0400 Subject: [PATCH 176/279] feat(ctb): Make guard.safe var immutable --- packages/contracts-bedrock/src/Safe/LivenessGuard.sol | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol index e95f15dfe13e..6ce1a474c871 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol @@ -12,7 +12,7 @@ contract LivenessGuard is SignatureDecoder, BaseGuard { /// @param signers An arrary of signer addresses. event SignersRecorded(bytes32 indexed txHash, address[] signers); - Safe public safe; + Safe public immutable safe; mapping(address => uint256) public lastSigned; constructor(Safe _safe) { From 53fb127aaa2a16a29ad8002f986fe8ecb77e9251 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Tue, 17 Oct 2023 06:40:51 -0400 Subject: [PATCH 177/279] feat(ctb): Make Safe Module and Guard ISemver --- packages/contracts-bedrock/src/Safe/LivenessGuard.sol | 7 ++++++- packages/contracts-bedrock/src/Safe/LivenessModule.sol | 5 +++++ 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol index 6ce1a474c871..4ec9690c03a8 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol @@ -6,12 +6,17 @@ import { BaseGuard, GuardManager } from "safe-contracts/base/GuardManager.sol"; import { ModuleManager } from "safe-contracts/base/ModuleManager.sol"; import { SignatureDecoder } from "safe-contracts/common/SignatureDecoder.sol"; import { Enum } from "safe-contracts/common/Enum.sol"; +import { ISemver } from "src/universal/ISemver.sol"; -contract LivenessGuard is SignatureDecoder, BaseGuard { +contract LivenessGuard is ISemver, SignatureDecoder, BaseGuard { /// @notice Emitted when a new set of signers is recorded. /// @param signers An arrary of signer addresses. event SignersRecorded(bytes32 indexed txHash, address[] signers); + /// @notice Semantic version. + /// @custom:semver 1.0.0 + string public constant version = "1.0.0"; + Safe public immutable safe; mapping(address => uint256) public lastSigned; diff --git a/packages/contracts-bedrock/src/Safe/LivenessModule.sol b/packages/contracts-bedrock/src/Safe/LivenessModule.sol index 6db3bf9bea12..de44ae5bff05 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessModule.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessModule.sol @@ -5,6 +5,7 @@ import { Safe } from "safe-contracts/Safe.sol"; import { Enum } from "safe-contracts/common/Enum.sol"; import { OwnerManager } from "safe-contracts/base/OwnerManager.sol"; import { LivenessGuard } from "src/Safe/LivenessGuard.sol"; +import { ISemver } from "src/universal/ISemver.sol"; // TODO(maurelian): remove me import { console2 as console } from "forge-std/console2.sol"; @@ -28,6 +29,10 @@ contract LivenessModule { /// @notice The fallback owner of the Safe address public fallbackOwner; + /// @notice Semantic version. + /// @custom:semver 1.0.0 + string public constant version = "1.0.0"; + // Constructor to initialize the Safe and baseModule instances constructor( Safe _safe, From 108f1fe88a9f8e6c2e85f874817911007809abbe Mon Sep 17 00:00:00 2001 From: Maurelian Date: Tue, 17 Oct 2023 06:41:38 -0400 Subject: [PATCH 178/279] feat(ctb): Commenting cleanup --- packages/contracts-bedrock/src/Safe/LivenessModule.sol | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessModule.sol b/packages/contracts-bedrock/src/Safe/LivenessModule.sol index de44ae5bff05..46f1f40a15f5 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessModule.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessModule.sol @@ -12,8 +12,8 @@ import { console2 as console } from "forge-std/console2.sol"; /// @title LivenessModule /// @notice This module is intended to be used in conjunction with the LivenessGuard. It should be able to -/// execute a transaction on the Safe in only a small number of cases: -contract LivenessModule { +/// execute a transaction on the Safe in only a small number of cases. +contract LivenessModule is ISemver { /// @notice The Safe contract instance Safe public safe; From 542bc578ad55926f3e0c31bc33bbe7948f62259c Mon Sep 17 00:00:00 2001 From: Maurelian Date: Tue, 17 Oct 2023 13:46:40 -0400 Subject: [PATCH 179/279] feat(ctb): Add makeAddrsAndKeys to SafeTestTools --- .../test/LivenessModule.t.sol | 9 +----- .../test/safe-tools/SafeTestTools.sol | 30 +++++++++++-------- 2 files changed, 18 insertions(+), 21 deletions(-) diff --git a/packages/contracts-bedrock/test/LivenessModule.t.sol b/packages/contracts-bedrock/test/LivenessModule.t.sol index f375ee6af69f..1276e689362e 100644 --- a/packages/contracts-bedrock/test/LivenessModule.t.sol +++ b/packages/contracts-bedrock/test/LivenessModule.t.sol @@ -20,16 +20,9 @@ contract LivnessModule_TestInit is Test, SafeTestTools { LivenessGuard livenessGuard; SafeInstance safeInstance; - function makeKeys(uint256 num) public pure returns (uint256[] memory keys_) { - keys_ = new uint256[](num); - for (uint256 i; i < num; i++) { - keys_[i] = uint256(keccak256(abi.encodePacked(i))); - } - } - function setUp() public { // Create a Safe with 10 owners - uint256[] memory keys = makeKeys(10); + (, uint256[] memory keys) = makeAddrsAndKeys(10); safeInstance = _setupSafe(keys, 8); livenessGuard = new LivenessGuard(safeInstance.safe); livenessModule = new LivenessModule({ diff --git a/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol b/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol index 0730a6bc3bc8..042b18aeaca0 100644 --- a/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol +++ b/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol @@ -20,6 +20,22 @@ function getAddr(uint256 pk) pure returns (address) { return Vm(VM_ADDR).addr(pk); } +/// @dev Get arrays of addresses and private keys. The arrays are sorted by address, and the addresses are labelled +function makeAddrsAndKeys(uint256 num) returns (address[] memory addrs, uint256[] memory keys) { + keys = new uint256[](num); + addrs = new address[](num); + for (uint256 i; i < num; i++) { + uint256 key = uint256(keccak256(abi.encodePacked(i))); + keys[i] = key; + } + + keys = sortPKsByComputedAddress(keys); + for (uint256 i; i < num; i++) { + addrs[i] = Vm(VM_ADDR).addr(keys[i]); + Vm(VM_ADDR).label(getAddr(keys[i]), string.concat("SAFETEST: Signer ", string(abi.encodePacked(bytes32(i))))); + } +} + /// @dev Encode a smart contract wallet as a private key function encodeSmartContractWalletAsPK(address addr) pure returns (uint256 encodedPK) { assembly { @@ -454,19 +470,7 @@ contract SafeTestTools { } function _setupSafe() public returns (SafeInstance memory) { - string[3] memory users; - users[0] = "SAFETEST: Signer 0"; - users[1] = "SAFETEST: Signer 1"; - users[2] = "SAFETEST: Signer 2"; - - uint256[] memory defaultPKs = new uint256[](3); - defaultPKs[0] = 0xac0974bec39a17e36ba4a6b4d238ff944bacb478cbed5efcae784d7bf4f2ff80; - defaultPKs[1] = 0x59c6995e998f97a5a0044966f0945389dc9e86dae88c7a8412f4603b6b78690d; - defaultPKs[2] = 0x5de4111afa1a4b94908f83103eb1f1706367c2e68ca870fc3fb9a804cdab365a; - - for (uint256 i; i < 3; i++) { - Vm(VM_ADDR).label(getAddr(defaultPKs[i]), users[i]); - } + (, uint256[] memory defaultPKs) = makeAddrsAndKeys(3); return _setupSafe( defaultPKs, From c5b6f88849d3380499dface0f0d1b4349afdd41a Mon Sep 17 00:00:00 2001 From: Maurelian Date: Tue, 17 Oct 2023 14:28:51 -0400 Subject: [PATCH 180/279] refactor(ctb): Move _getNSigners() to inheritable contract --- .../contracts-bedrock/src/Safe/GetSigners.sol | 45 +++++++++++++++++++ .../src/Safe/LivenessGuard.sol | 45 ++----------------- 2 files changed, 49 insertions(+), 41 deletions(-) create mode 100644 packages/contracts-bedrock/src/Safe/GetSigners.sol diff --git a/packages/contracts-bedrock/src/Safe/GetSigners.sol b/packages/contracts-bedrock/src/Safe/GetSigners.sol new file mode 100644 index 000000000000..f10d0424f23e --- /dev/null +++ b/packages/contracts-bedrock/src/Safe/GetSigners.sol @@ -0,0 +1,45 @@ +// SPDX-License-Identifier: MIT +pragma solidity 0.8.15; + +import { SignatureDecoder } from "safe-contracts/common/SignatureDecoder.sol"; + +abstract contract GetSigners is SignatureDecoder { + /// @notice Extract the signers from a set of signatures. + function _getNSigners(bytes32 dataHash, bytes memory signatures) internal pure returns (address[] memory _owners) { + uint256 numSignatures = signatures.length / 65; + _owners = new address[](numSignatures); + + /// The following code is extracted from the Safe.checkNSignatures() method. It removes the signature + /// validation code, and keeps only the parsing code necessary to extract the owner addresses from the + /// signatures. We do not double check if the owner derived from a signature is valid. As this is handled + /// in the final require statement of Safe.checkNSignatures(). + address currentOwner; + uint8 v; + bytes32 r; + bytes32 s; + uint256 i; + for (i = 0; i < numSignatures; i++) { + (v, r, s) = signatureSplit(signatures, i); + if (v == 0) { + // If v is 0 then it is a contract signature + // When handling contract signatures the address of the contract is encoded into r + currentOwner = address(uint160(uint256(r))); + } else if (v == 1) { + // If v is 1 then it is an approved hash + // When handling approved hashes the address of the approver is encoded into r + currentOwner = address(uint160(uint256(r))); + } else if (v > 30) { + // If v > 30 then default va (27,28) has been adjusted for eth_sign flow + // To support eth_sign and similar we adjust v and hash the messageHash with the Ethereum message prefix + // before applying ecrecover + currentOwner = + ecrecover(keccak256(abi.encodePacked("\x19Ethereum Signed Message:\n32", dataHash)), v - 4, r, s); + } else { + // Default is the ecrecover flow with the provided data hash + // Use ecrecover with the messageHash for EOA signatures + currentOwner = ecrecover(dataHash, v, r, s); + } + _owners[i] = currentOwner; + } + } +} diff --git a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol index 4ec9690c03a8..cd69eb357069 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol @@ -4,11 +4,11 @@ pragma solidity 0.8.15; import { Safe } from "safe-contracts/Safe.sol"; import { BaseGuard, GuardManager } from "safe-contracts/base/GuardManager.sol"; import { ModuleManager } from "safe-contracts/base/ModuleManager.sol"; -import { SignatureDecoder } from "safe-contracts/common/SignatureDecoder.sol"; +import { GetSigners } from "src/Safe/GetSigners.sol"; import { Enum } from "safe-contracts/common/Enum.sol"; import { ISemver } from "src/universal/ISemver.sol"; -contract LivenessGuard is ISemver, SignatureDecoder, BaseGuard { +contract LivenessGuard is ISemver, GetSigners, BaseGuard { /// @notice Emitted when a new set of signers is recorded. /// @param signers An arrary of signer addresses. event SignersRecorded(bytes32 indexed txHash, address[] signers); @@ -30,6 +30,8 @@ contract LivenessGuard is ISemver, SignatureDecoder, BaseGuard { } /// @notice Records the most recent time which any owner has signed a transaction. + /// @dev This method is called by the Safe contract, it is critical that it does not revert, otherwise + /// the Safe contract will be unable to execute transactions. function checkTransaction( address to, uint256 value, @@ -71,45 +73,6 @@ contract LivenessGuard is ISemver, SignatureDecoder, BaseGuard { emit SignersRecorded(txHash, signers); } - /// @notice Exctract the signers from a set of signatures. - function _getNSigners(bytes32 dataHash, bytes memory signatures) internal pure returns (address[] memory _owners) { - uint256 numSignatures = signatures.length / 65; - _owners = new address[](numSignatures); - - /// The following code is extracted from the Safe.checkNSignatures() method. It removes the signature - /// validation code, and keeps only the parsing code necessary to extract the owner addresses from the - /// signatures. We do not double check if the owner derived from a signature is valid. As this is handled - /// in the final require statement of Safe.checkNSignatures(). - address currentOwner; - uint8 v; - bytes32 r; - bytes32 s; - uint256 i; - for (i = 0; i < numSignatures; i++) { - (v, r, s) = signatureSplit(signatures, i); - if (v == 0) { - // If v is 0 then it is a contract signature - // When handling contract signatures the address of the contract is encoded into r - currentOwner = address(uint160(uint256(r))); - } else if (v == 1) { - // If v is 1 then it is an approved hash - // When handling approved hashes the address of the approver is encoded into r - currentOwner = address(uint160(uint256(r))); - } else if (v > 30) { - // If v > 30 then default va (27,28) has been adjusted for eth_sign flow - // To support eth_sign and similar we adjust v and hash the messageHash with the Ethereum message prefix - // before applying ecrecover - currentOwner = - ecrecover(keccak256(abi.encodePacked("\x19Ethereum Signed Message:\n32", dataHash)), v - 4, r, s); - } else { - // Default is the ecrecover flow with the provided data hash - // Use ecrecover with the messageHash for EOA signatures - currentOwner = ecrecover(dataHash, v, r, s); - } - _owners[i] = currentOwner; - } - } - /// @notice Enables an owner to demonstrate liveness by calling this method directly. /// This is useful for owners who have not recently signed a transaction via the Safe. function showLiveness() external { From 3f7947b7a61172bf51a0d4d04da422b1c7d2cb5e Mon Sep 17 00:00:00 2001 From: Maurelian Date: Tue, 17 Oct 2023 14:34:30 -0400 Subject: [PATCH 181/279] test(ctb): Add GetSigners diff test --- .../contracts-bedrock/test/GetSigners.t.sol | 45 +++++++++++++++++++ 1 file changed, 45 insertions(+) create mode 100644 packages/contracts-bedrock/test/GetSigners.t.sol diff --git a/packages/contracts-bedrock/test/GetSigners.t.sol b/packages/contracts-bedrock/test/GetSigners.t.sol new file mode 100644 index 000000000000..0eeb34e9c20c --- /dev/null +++ b/packages/contracts-bedrock/test/GetSigners.t.sol @@ -0,0 +1,45 @@ +// SPDX-License-Identifier: MIT +pragma solidity 0.8.15; + +import { Test } from "forge-std/Test.sol"; +import { Safe } from "safe-contracts/Safe.sol"; +import { GetSigners } from "src/Safe/GetSigners.sol"; +import "test/safe-tools/SafeTestTools.sol"; + +import { SignatureDecoder } from "safe-contracts/common/SignatureDecoder.sol"; + +contract GetSigners_Test is Test, SafeTestTools, GetSigners { + /// @dev Test that for a given set of signatures: + /// 1. safe.checkNSignatures() succeeds + /// 2. the getSigners() method returns the expected signers + /// 3. the expected signers are all owners of the safe. + /// Demonstrating these three properties is sufficient to prove that the getSigners() method + /// returns the same signatures as those recovered by safe.checkNSignatures(). + /// todo(maurelian): include tests for EIP1271 signatures, and contract signatures. + function testDiff_getSignaturesVsCheckSignatures_succeeds(uint256 _numSigs, bytes32 _digest) external { + uint256 numSigs = bound(_numSigs, 1, 100); + (, uint256[] memory keys) = makeAddrsAndKeys(numSigs); + SafeInstance memory safeInstance = SafeTestTools._setupSafe(keys, numSigs, 0); + + bytes memory signatures; + for (uint256 i; i < numSigs; i++) { + (uint8 v, bytes32 r, bytes32 s) = vm.sign(keys[i], _digest); + + // Safe signatures are encoded as r, s, v, not v, r, s. + signatures = bytes.concat(signatures, abi.encodePacked(r, s, v)); + } + + // Signature checking on the Safe should succeed. + safeInstance.safe.checkNSignatures(_digest, hex"", signatures, numSigs); + + // Recover the signatures using the getSigners() method. + address[] memory gotSigners = _getNSigners(_digest, signatures); + + // Compare the recovered signers to the expected signers. + assertEq(gotSigners.length, numSigs); + assertEq(gotSigners.length, safeInstance.owners.length); + for (uint256 i; i < numSigs; i++) { + assertEq(safeInstance.owners[i], gotSigners[i]); + } + } +} From 017e0480ccb039a13aa26e1bc25a8160ac0f0ff5 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Thu, 19 Oct 2023 13:14:22 -0400 Subject: [PATCH 182/279] feat(ctb): Record comment about numsignatures calc --- packages/contracts-bedrock/src/Safe/GetSigners.sol | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/packages/contracts-bedrock/src/Safe/GetSigners.sol b/packages/contracts-bedrock/src/Safe/GetSigners.sol index f10d0424f23e..6c7f01862295 100644 --- a/packages/contracts-bedrock/src/Safe/GetSigners.sol +++ b/packages/contracts-bedrock/src/Safe/GetSigners.sol @@ -6,7 +6,8 @@ import { SignatureDecoder } from "safe-contracts/common/SignatureDecoder.sol"; abstract contract GetSigners is SignatureDecoder { /// @notice Extract the signers from a set of signatures. function _getNSigners(bytes32 dataHash, bytes memory signatures) internal pure returns (address[] memory _owners) { - uint256 numSignatures = signatures.length / 65; + uint256 numSignatures = signatures.length / 65; // this is wrong. There can be extra data appended to the + // signatures for contract signatures. We should use SignatureDecoder to parse the signatures. _owners = new address[](numSignatures); /// The following code is extracted from the Safe.checkNSignatures() method. It removes the signature From 5011159887e587b1a3bdd35fcc65e03ad3e21735 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Thu, 19 Oct 2023 14:04:41 -0400 Subject: [PATCH 183/279] test(ctb): Add eth_sign types to GetSigners test --- .../contracts-bedrock/test/GetSigners.t.sol | 68 +++++++++++++++++-- 1 file changed, 62 insertions(+), 6 deletions(-) diff --git a/packages/contracts-bedrock/test/GetSigners.t.sol b/packages/contracts-bedrock/test/GetSigners.t.sol index 0eeb34e9c20c..a01bc88b0e54 100644 --- a/packages/contracts-bedrock/test/GetSigners.t.sol +++ b/packages/contracts-bedrock/test/GetSigners.t.sol @@ -9,6 +9,22 @@ import "test/safe-tools/SafeTestTools.sol"; import { SignatureDecoder } from "safe-contracts/common/SignatureDecoder.sol"; contract GetSigners_Test is Test, SafeTestTools, GetSigners { + struct SigTypeCount { + uint256 numEoaSigs; + uint256 numEthSignSigs; + uint256 numApprovedHashSigs; + uint256 numContractSigs; + } + + enum SigType { + Eoa, + EthSign, + ApprovedHash, + Contract + } + + mapping(uint256 => SigType) public sigTypes; + /// @dev Test that for a given set of signatures: /// 1. safe.checkNSignatures() succeeds /// 2. the getSigners() method returns the expected signers @@ -16,20 +32,60 @@ contract GetSigners_Test is Test, SafeTestTools, GetSigners { /// Demonstrating these three properties is sufficient to prove that the getSigners() method /// returns the same signatures as those recovered by safe.checkNSignatures(). /// todo(maurelian): include tests for EIP1271 signatures, and contract signatures. - function testDiff_getSignaturesVsCheckSignatures_succeeds(uint256 _numSigs, bytes32 _digest) external { - uint256 numSigs = bound(_numSigs, 1, 100); + function testDiff_getSignaturesVsCheckSignatures_succeeds(bytes32 _digest, SigTypeCount memory _split) external { + // Limit the number of each signature type to 25 + uint256 numEoaSigs = bound(_split.numEoaSigs, 1, 25); + uint256 numEthSignSigs = bound(_split.numEthSignSigs, 1, 25); + // uint256 numContractSigs = bound(_split.numContractSigs, 1, 25); + // uint256 numApprovedHashSigs = bound(_split.numApprovedHashSigs, 1, 25); + + // uint256 numSigs = numEoaSigs + numApprovedHashSigs + numContractSigs + numEthSignSigs; + uint256 numSigs = numEoaSigs + numEthSignSigs; + (, uint256[] memory keys) = makeAddrsAndKeys(numSigs); + + // record the signature types for each key + for (uint256 i; i < numSigs; i++) { + // Generate EOA keys for both EOA and ETH Sign signatures + if (i < numEoaSigs) { + sigTypes[keys[i]] = SigType.Eoa; + } else if (i < numEoaSigs + numEthSignSigs) { + sigTypes[keys[i]] = SigType.EthSign; + } else { + // Generate approved hash signatures + // Generate eth_sign signatures + revert("not implemented"); + } + } + + // Now sort the keys array. By doing this after assigning a signature type to each key, + // we ensure that the signature types are randomly ordered. It probably doesn't matter either + // way, but this is more realistic. + keys = sortPKsByComputedAddress(keys); + + // Create a new safeInstance with M=N, so that it requires a signature from each key. SafeInstance memory safeInstance = SafeTestTools._setupSafe(keys, numSigs, 0); + // Create an empty array of signature data bytes memory signatures; - for (uint256 i; i < numSigs; i++) { - (uint8 v, bytes32 r, bytes32 s) = vm.sign(keys[i], _digest); - // Safe signatures are encoded as r, s, v, not v, r, s. - signatures = bytes.concat(signatures, abi.encodePacked(r, s, v)); + // Populate the signatures by iterating over the safeInstance owners list. + // is a requirement for the ordering of signatures in the Safe contract. + for (uint256 i; i < keys.length; i++) { + if (sigTypes[keys[i]] == SigType.Eoa) { + (uint8 v, bytes32 r, bytes32 s) = vm.sign(keys[i], _digest); + // Safe signatures are encoded as r, s, v, not v, r, s. + signatures = bytes.concat(signatures, abi.encodePacked(r, s, v)); + } else if (sigTypes[keys[i]] == SigType.EthSign) { + (uint8 v, bytes32 r, bytes32 s) = + vm.sign(keys[i], keccak256(abi.encodePacked("\x19Ethereum Signed Message:\n32", _digest))); + v += 4; + signatures = bytes.concat(signatures, abi.encodePacked(r, s, v)); + } } // Signature checking on the Safe should succeed. + // temp note: the second arg is the data, which is only used in the contract signatures type. safeInstance.safe.checkNSignatures(_digest, hex"", signatures, numSigs); // Recover the signatures using the getSigners() method. From 0a443dd4cd25f9969c45da3df698c375c3b3ae09 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Thu, 19 Oct 2023 14:21:56 -0400 Subject: [PATCH 184/279] feat(safe-tools): Do not force generated list of keys to be sorted --- packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol | 1 - 1 file changed, 1 deletion(-) diff --git a/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol b/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol index 042b18aeaca0..41aa65fc605b 100644 --- a/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol +++ b/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol @@ -29,7 +29,6 @@ function makeAddrsAndKeys(uint256 num) returns (address[] memory addrs, uint256[ keys[i] = key; } - keys = sortPKsByComputedAddress(keys); for (uint256 i; i < num; i++) { addrs[i] = Vm(VM_ADDR).addr(keys[i]); Vm(VM_ADDR).label(getAddr(keys[i]), string.concat("SAFETEST: Signer ", string(abi.encodePacked(bytes32(i))))); From 411da0fd245265f1752b0005db52f375a3bf1585 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Fri, 20 Oct 2023 14:02:10 -0400 Subject: [PATCH 185/279] refactor(safe-tools): Cache priv-key during sorting loop this change is more readable --- .../contracts-bedrock/test/safe-tools/SafeTestTools.sol | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol b/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol index 41aa65fc605b..acd5cacb0eef 100644 --- a/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol +++ b/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol @@ -67,6 +67,7 @@ library Sort { } /// @dev Sorts an array of private keys by the computed address +/// If the private key is a smart contract wallet, it will be decoded and sorted by the address function sortPKsByComputedAddress(uint256[] memory _pks) pure returns (uint256[] memory) { uint256[] memory sortedPKs = new uint256[](_pks.length); @@ -74,10 +75,14 @@ function sortPKsByComputedAddress(uint256[] memory _pks) pure returns (uint256[] bytes32[2][] memory accounts = new bytes32[2][](_pks.length); for (uint256 i; i < _pks.length; i++) { - address signer = getAddr(_pks[i]); + uint256 pk = _pks[i]; + address signer = getAddr(pk); + if (isSmartContractPK(pk)) { + signer = decodeSmartContractWalletAsAddress(pk); + } addresses[i] = signer; accounts[i][0] = bytes32(abi.encode(signer)); - accounts[i][1] = bytes32(_pks[i]); + accounts[i][1] = bytes32(pk); } addresses = Sort.sort(addresses); From 040c34f7c01a0eaf61e5e42fd78d9a3d9778acc7 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Thu, 19 Oct 2023 15:50:44 -0400 Subject: [PATCH 186/279] test(ctb): Add approved hash checking to GetSigners test working for approvedHashes --- .../contracts-bedrock/test/GetSigners.t.sol | 82 +++++++++---------- 1 file changed, 37 insertions(+), 45 deletions(-) diff --git a/packages/contracts-bedrock/test/GetSigners.t.sol b/packages/contracts-bedrock/test/GetSigners.t.sol index a01bc88b0e54..ed5ce5e767ca 100644 --- a/packages/contracts-bedrock/test/GetSigners.t.sol +++ b/packages/contracts-bedrock/test/GetSigners.t.sol @@ -9,21 +9,19 @@ import "test/safe-tools/SafeTestTools.sol"; import { SignatureDecoder } from "safe-contracts/common/SignatureDecoder.sol"; contract GetSigners_Test is Test, SafeTestTools, GetSigners { - struct SigTypeCount { - uint256 numEoaSigs; - uint256 numEthSignSigs; - uint256 numApprovedHashSigs; - uint256 numContractSigs; - } + bytes4 internal constant EIP1271_MAGIC_VALUE = 0x20c13b0b; - enum SigType { + enum SigTypes { Eoa, EthSign, - ApprovedHash, - Contract + ApprovedHash //, + // Contract } - mapping(uint256 => SigType) public sigTypes; + function sigType(uint256 _key) internal view returns (SigTypes sigType_) { + uint256 t = _key % 3; //4; + sigType_ = SigTypes(t); + } /// @dev Test that for a given set of signatures: /// 1. safe.checkNSignatures() succeeds @@ -32,35 +30,11 @@ contract GetSigners_Test is Test, SafeTestTools, GetSigners { /// Demonstrating these three properties is sufficient to prove that the getSigners() method /// returns the same signatures as those recovered by safe.checkNSignatures(). /// todo(maurelian): include tests for EIP1271 signatures, and contract signatures. - function testDiff_getSignaturesVsCheckSignatures_succeeds(bytes32 _digest, SigTypeCount memory _split) external { + function testDiff_getSignaturesVsCheckSignatures_succeeds(bytes32 _digest, uint256 _numSigs) external { // Limit the number of each signature type to 25 - uint256 numEoaSigs = bound(_split.numEoaSigs, 1, 25); - uint256 numEthSignSigs = bound(_split.numEthSignSigs, 1, 25); - // uint256 numContractSigs = bound(_split.numContractSigs, 1, 25); - // uint256 numApprovedHashSigs = bound(_split.numApprovedHashSigs, 1, 25); - - // uint256 numSigs = numEoaSigs + numApprovedHashSigs + numContractSigs + numEthSignSigs; - uint256 numSigs = numEoaSigs + numEthSignSigs; + uint256 numSigs = bound(_numSigs, 1, 25); (, uint256[] memory keys) = makeAddrsAndKeys(numSigs); - - // record the signature types for each key - for (uint256 i; i < numSigs; i++) { - // Generate EOA keys for both EOA and ETH Sign signatures - if (i < numEoaSigs) { - sigTypes[keys[i]] = SigType.Eoa; - } else if (i < numEoaSigs + numEthSignSigs) { - sigTypes[keys[i]] = SigType.EthSign; - } else { - // Generate approved hash signatures - // Generate eth_sign signatures - revert("not implemented"); - } - } - - // Now sort the keys array. By doing this after assigning a signature type to each key, - // we ensure that the signature types are randomly ordered. It probably doesn't matter either - // way, but this is more realistic. keys = sortPKsByComputedAddress(keys); // Create a new safeInstance with M=N, so that it requires a signature from each key. @@ -69,21 +43,39 @@ contract GetSigners_Test is Test, SafeTestTools, GetSigners { // Create an empty array of signature data bytes memory signatures; - // Populate the signatures by iterating over the safeInstance owners list. - // is a requirement for the ordering of signatures in the Safe contract. + // Populate the signatures by iterating over the keys, and choosing the signature type based + // on the key. + uint8 v; + bytes32 r; + bytes32 s; for (uint256 i; i < keys.length; i++) { - if (sigTypes[keys[i]] == SigType.Eoa) { - (uint8 v, bytes32 r, bytes32 s) = vm.sign(keys[i], _digest); + if (sigType(keys[i]) == SigTypes.Eoa) { + (v, r, s) = vm.sign(keys[i], _digest); // Safe signatures are encoded as r, s, v, not v, r, s. signatures = bytes.concat(signatures, abi.encodePacked(r, s, v)); - } else if (sigTypes[keys[i]] == SigType.EthSign) { - (uint8 v, bytes32 r, bytes32 s) = - vm.sign(keys[i], keccak256(abi.encodePacked("\x19Ethereum Signed Message:\n32", _digest))); + } else if (sigType(keys[i]) == SigTypes.EthSign) { + (v, r, s) = vm.sign(keys[i], keccak256(abi.encodePacked("\x19Ethereum Signed Message:\n32", _digest))); v += 4; signatures = bytes.concat(signatures, abi.encodePacked(r, s, v)); - } + } else if (sigType(keys[i]) == SigTypes.ApprovedHash) { + vm.prank(getAddr(keys[i])); + safeInstance.safe.approveHash(_digest); + v = 1; + s; // s is not checked on approved hash signatures. + r = bytes32(uint256(uint160(getAddr(keys[i])))); + signatures = bytes.concat(signatures, abi.encodePacked(r, s, v)); + } // else if (sigType(keys[i]) == SigTypes.Contract) { + // address addr = decodeSmartContractWalletAsAddress(keys[i]); + // r = bytes32(uint256(uint160(addr))); + // vm.mockCall( + // addr, abi.encodeWithSignature("isValidSignature(bytes,bytes)"), + // abi.encode(EIP1271_MAGIC_VALUE) + // ); + // v = 1; + // s; // s is not checked on approved hash signatures. + // signatures = bytes.concat(signatures, abi.encodePacked(r, s, v)); + // } } - // Signature checking on the Safe should succeed. // temp note: the second arg is the data, which is only used in the contract signatures type. safeInstance.safe.checkNSignatures(_digest, hex"", signatures, numSigs); From 7636b5e44c43e790a9e3cef138a1fc9c4742812b Mon Sep 17 00:00:00 2001 From: Maurelian Date: Fri, 20 Oct 2023 14:05:38 -0400 Subject: [PATCH 187/279] fix(ctb): getNSigners is limited to the threshold This change fixes a bug in getNSigners which incorrectly assumed that the length of the signature data could be used to directly determine the number of signatures provided. This is wrong because contract signatures append additional data used for the EIP1271 signature validation. --- .../contracts-bedrock/src/Safe/GetSigners.sol | 19 ++++++++++++++----- .../src/Safe/LivenessGuard.sol | 6 +++++- 2 files changed, 19 insertions(+), 6 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/GetSigners.sol b/packages/contracts-bedrock/src/Safe/GetSigners.sol index 6c7f01862295..0d9642bf6125 100644 --- a/packages/contracts-bedrock/src/Safe/GetSigners.sol +++ b/packages/contracts-bedrock/src/Safe/GetSigners.sol @@ -5,10 +5,19 @@ import { SignatureDecoder } from "safe-contracts/common/SignatureDecoder.sol"; abstract contract GetSigners is SignatureDecoder { /// @notice Extract the signers from a set of signatures. - function _getNSigners(bytes32 dataHash, bytes memory signatures) internal pure returns (address[] memory _owners) { - uint256 numSignatures = signatures.length / 65; // this is wrong. There can be extra data appended to the - // signatures for contract signatures. We should use SignatureDecoder to parse the signatures. - _owners = new address[](numSignatures); + /// @param dataHash Hash of the data. + /// @param signatures Signature data for identifying signers. + /// @param requiredSignatures Amount of required valid signatures. + function _getNSigners( + bytes32 dataHash, + bytes memory signatures, + uint256 requiredSignatures + ) + internal + pure + returns (address[] memory _owners) + { + _owners = new address[](requiredSignatures); /// The following code is extracted from the Safe.checkNSignatures() method. It removes the signature /// validation code, and keeps only the parsing code necessary to extract the owner addresses from the @@ -19,7 +28,7 @@ abstract contract GetSigners is SignatureDecoder { bytes32 r; bytes32 s; uint256 i; - for (i = 0; i < numSignatures; i++) { + for (i = 0; i < requiredSignatures; i++) { (v, r, s) = signatureSplit(signatures, i); if (v == 0) { // If v is 0 then it is a contract signature diff --git a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol index cd69eb357069..2abe1d3b2be9 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol @@ -66,7 +66,11 @@ contract LivenessGuard is ISemver, GetSigners, BaseGuard { // Signature info Safe(payable(msg.sender)).nonce() - 1 ); - address[] memory signers = _getNSigners(txHash, signatures); + + uint256 threshold = safe.getThreshold(); + address[] memory signers = + _getNSigners({ dataHash: txHash, signatures: signatures, requiredSignatures: threshold }); + for (uint256 i = 0; i < signers.length; i++) { lastSigned[signers[i]] = block.timestamp; } From a9312b79fe5621385038ed78b02c4f4a815793e2 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Fri, 20 Oct 2023 14:06:27 -0400 Subject: [PATCH 188/279] test(ctb): Add contract signatures testing --- .../contracts-bedrock/test/GetSigners.t.sol | 97 +++++++++++-------- 1 file changed, 57 insertions(+), 40 deletions(-) diff --git a/packages/contracts-bedrock/test/GetSigners.t.sol b/packages/contracts-bedrock/test/GetSigners.t.sol index ed5ce5e767ca..03cfd711860c 100644 --- a/packages/contracts-bedrock/test/GetSigners.t.sol +++ b/packages/contracts-bedrock/test/GetSigners.t.sol @@ -14,12 +14,15 @@ contract GetSigners_Test is Test, SafeTestTools, GetSigners { enum SigTypes { Eoa, EthSign, - ApprovedHash //, - // Contract + ApprovedHash, + Contract } - function sigType(uint256 _key) internal view returns (SigTypes sigType_) { - uint256 t = _key % 3; //4; + /// @dev Maps every key to one of the 4 signatures types. + /// This is used in the tests below as a pseudorandom mechanism for determining which + /// signature type to use for each key. + function sigType(uint256 _key) internal pure returns (SigTypes sigType_) { + uint256 t = _key % 4; sigType_ = SigTypes(t); } @@ -29,61 +32,75 @@ contract GetSigners_Test is Test, SafeTestTools, GetSigners { /// 3. the expected signers are all owners of the safe. /// Demonstrating these three properties is sufficient to prove that the getSigners() method /// returns the same signatures as those recovered by safe.checkNSignatures(). - /// todo(maurelian): include tests for EIP1271 signatures, and contract signatures. - function testDiff_getSignaturesVsCheckSignatures_succeeds(bytes32 _digest, uint256 _numSigs) external { - // Limit the number of each signature type to 25 + function testDiff_getSignaturesVsCheckSignatures_succeeds(bytes memory _data, uint256 _numSigs) external { + bytes32 digest = keccak256(_data); + + // Limit the number of signatures to 25 uint256 numSigs = bound(_numSigs, 1, 25); (, uint256[] memory keys) = makeAddrsAndKeys(numSigs); - keys = sortPKsByComputedAddress(keys); + for (uint256 i = 0; i < keys.length; i++) { + if (sigType(keys[i]) == SigTypes.Contract) { + keys[i] = encodeSmartContractWalletAsPK(decodeSmartContractWalletAsAddress(keys[i])); + } + } // Create a new safeInstance with M=N, so that it requires a signature from each key. SafeInstance memory safeInstance = SafeTestTools._setupSafe(keys, numSigs, 0); - // Create an empty array of signature data - bytes memory signatures; - - // Populate the signatures by iterating over the keys, and choosing the signature type based - // on the key. + // Next we will generate signatures by iterating over the keys, and choosing the signature type + // based on the key. uint8 v; bytes32 r; bytes32 s; - for (uint256 i; i < keys.length; i++) { - if (sigType(keys[i]) == SigTypes.Eoa) { - (v, r, s) = vm.sign(keys[i], _digest); - // Safe signatures are encoded as r, s, v, not v, r, s. + uint256 contractSigs; + bytes memory signatures; + uint256[] memory pks = safeInstance.ownerPKs; + for (uint256 i; i < pks.length; i++) { + if (sigType(pks[i]) == SigTypes.Eoa) { + (v, r, s) = vm.sign(pks[i], digest); signatures = bytes.concat(signatures, abi.encodePacked(r, s, v)); - } else if (sigType(keys[i]) == SigTypes.EthSign) { - (v, r, s) = vm.sign(keys[i], keccak256(abi.encodePacked("\x19Ethereum Signed Message:\n32", _digest))); + } else if (sigType(pks[i]) == SigTypes.EthSign) { + (v, r, s) = vm.sign(pks[i], keccak256(abi.encodePacked("\x19Ethereum Signed Message:\n32", digest))); v += 4; signatures = bytes.concat(signatures, abi.encodePacked(r, s, v)); - } else if (sigType(keys[i]) == SigTypes.ApprovedHash) { - vm.prank(getAddr(keys[i])); - safeInstance.safe.approveHash(_digest); + } else if (sigType(pks[i]) == SigTypes.ApprovedHash) { + vm.prank(getAddr(pks[i])); + safeInstance.safe.approveHash(digest); v = 1; - s; // s is not checked on approved hash signatures. - r = bytes32(uint256(uint160(getAddr(keys[i])))); + // s is not checked on approved hash signatures, so we can leave it as zero. + r = bytes32(uint256(uint160(getAddr(pks[i])))); signatures = bytes.concat(signatures, abi.encodePacked(r, s, v)); - } // else if (sigType(keys[i]) == SigTypes.Contract) { - // address addr = decodeSmartContractWalletAsAddress(keys[i]); - // r = bytes32(uint256(uint160(addr))); - // vm.mockCall( - // addr, abi.encodeWithSignature("isValidSignature(bytes,bytes)"), - // abi.encode(EIP1271_MAGIC_VALUE) - // ); - // v = 1; - // s; // s is not checked on approved hash signatures. - // signatures = bytes.concat(signatures, abi.encodePacked(r, s, v)); - // } + } else if (sigType(pks[i]) == SigTypes.Contract) { + contractSigs++; + address addr = decodeSmartContractWalletAsAddress(pks[i]); + r = bytes32(uint256(uint160(addr))); + vm.mockCall( + addr, abi.encodeWithSignature("isValidSignature(bytes,bytes)"), abi.encode(EIP1271_MAGIC_VALUE) + ); + v = 0; + // s needs to point to data that comes after the signatures + s = bytes32(numSigs * 65); + signatures = bytes.concat(signatures, abi.encodePacked(r, s, v)); + } + } + + // For each contract sig, add 64 bytes to the signature data. This is necessary to satisfy + // the validation checks that the Safe contract performs on the value of s on contract + // signatures. The Safe contract checks that s correctly points to additional data appended + // after the signatures, and that the length of the data is within bounds. + for (uint256 i = 0; i < contractSigs; i++) { + signatures = bytes.concat(signatures, abi.encode(32, 1)); } + // Signature checking on the Safe should succeed. - // temp note: the second arg is the data, which is only used in the contract signatures type. - safeInstance.safe.checkNSignatures(_digest, hex"", signatures, numSigs); + safeInstance.safe.checkNSignatures(digest, _data, signatures, numSigs); - // Recover the signatures using the getSigners() method. - address[] memory gotSigners = _getNSigners(_digest, signatures); + // Recover the signatures using the _getNSigners() method. + address[] memory gotSigners = + _getNSigners({ dataHash: digest, signatures: signatures, requiredSignatures: numSigs }); - // Compare the recovered signers to the expected signers. + // Compare the list of recovered signers to the expected signers. assertEq(gotSigners.length, numSigs); assertEq(gotSigners.length, safeInstance.owners.length); for (uint256 i; i < numSigs; i++) { From 0493c40b9a0ae072212af3e43ea4bb30ffe81041 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Fri, 20 Oct 2023 15:12:33 -0400 Subject: [PATCH 189/279] test(ctb): Fixes and improvements to checkTransaction test --- .../contracts-bedrock/test/LivenessGuard.t.sol | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/packages/contracts-bedrock/test/LivenessGuard.t.sol b/packages/contracts-bedrock/test/LivenessGuard.t.sol index 823311888932..d85466081c57 100644 --- a/packages/contracts-bedrock/test/LivenessGuard.t.sol +++ b/packages/contracts-bedrock/test/LivenessGuard.t.sol @@ -2,7 +2,7 @@ pragma solidity 0.8.15; import { Test } from "forge-std/Test.sol"; -import { Safe } from "safe-contracts/Safe.sol"; +import { Safe, OwnerManager } from "safe-contracts/Safe.sol"; import { SafeProxyFactory } from "safe-contracts/proxies/SafeProxyFactory.sol"; import { ModuleManager } from "safe-contracts/base/ModuleManager.sol"; import { Enum } from "safe-contracts/common/Enum.sol"; @@ -37,12 +37,20 @@ contract LivnessGuard_CheckTx_Test is LivnessGuard_TestInit { using SafeTestLib for SafeInstance; function test_checkTransaction_succeeds() external { + // Create an array of the addresses who will sign the transaction. SafeTestTools + // will generate these signatures up to the threshold by iterating over the owners array. + address[] memory signers = new address[](safeInstance.threshold); + signers[0] = safeInstance.owners[0]; + signers[1] = safeInstance.owners[1]; + // Don't check topic1 so that we can avoid the ugly txHash calculation. vm.expectEmit(false, true, true, true, address(livenessGuard)); - emit SignersRecorded(0x0, safeInstance.owners); + emit SignersRecorded(0x0, signers); + vm.expectCall(address(safeInstance.safe), abi.encodeWithSignature("nonce()")); + vm.expectCall(address(safeInstance.safe), abi.encodeCall(OwnerManager.getThreshold, ())); safeInstance.execTransaction({ to: address(1111), value: 0, data: hex"abba" }); - for (uint256 i; i < safeInstance.owners.length; i++) { + for (uint256 i; i < safeInstance.threshold; i++) { assertEq(livenessGuard.lastSigned(safeInstance.owners[i]), block.timestamp); } } From d5600eb8628bf63ffc4b97ecc450e6175d072226 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Fri, 20 Oct 2023 15:34:27 -0400 Subject: [PATCH 190/279] chore(ctb): Remove todo note from LivenessGuard --- packages/contracts-bedrock/src/Safe/LivenessGuard.sol | 3 --- 1 file changed, 3 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol index 2abe1d3b2be9..38f07fe4c684 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol @@ -85,9 +85,6 @@ contract LivenessGuard is ISemver, GetSigners, BaseGuard { address[] memory signers = new address[](1); signers[0] = msg.sender; - // todo(maurelian): Is there any need for this event to be differentiated from the one emitted in - // checkTransaction? - // Technically the 0x0 txHash does serve to identiy a call to this method. emit SignersRecorded(0x0, signers); } } From b6a38049e59a804e4364ae1fe4e94b52a164920d Mon Sep 17 00:00:00 2001 From: Maurelian Date: Fri, 29 Sep 2023 16:48:03 -0400 Subject: [PATCH 191/279] specs: Add specs for Liveness Checking --- specs/safe-liveness-checking.md | 68 +++++++++++++++++++++++++++++++++ 1 file changed, 68 insertions(+) create mode 100644 specs/safe-liveness-checking.md diff --git a/specs/safe-liveness-checking.md b/specs/safe-liveness-checking.md new file mode 100644 index 000000000000..b4d145c50ce2 --- /dev/null +++ b/specs/safe-liveness-checking.md @@ -0,0 +1,68 @@ +# Safe Liveness Checking + + + +**Table of Contents** + +- [Liveness Checking Mechanism](#liveness-checking-mechanism) +- [Liveness checking methodology](#liveness-checking-methodology) + - [The Liveness Guard](#the-liveness-guard) + - [The Liveness Module](#the-liveness-module) + - [Shutdown](#shutdown) + - [Security Properties](#security-properties) + + + +## Liveness Checking Mechanism + +The Security Security Council uses a specially extended Safe multisig contract to ensure that +any loss of access to a signer's keys is identified and addressed within a predictable period of +time. + +## Liveness checking methodology + +This is achieved using two types of contracts which the Safe contract has built-in support for: + +1. **Guard contracts:** can execute pre- and post- transaction checks. +1. **Module contracts:** a contract which is added to the Safe by the signers, and thenceforth is + authorized to execute transactions via the Safe. This means the module must properly implement + auth conditions internally. + +### The Liveness Guard + +For implementing liveness checks a `LivenessGuard` is created which receives the signatures from +each executed transaction, and tracks the latest time at which a transaction was signed by each +signer. This time is made publicly available by calling a `lastSigned(address)(Timestamp)` method. + +Signers may also call the contract directly in order to prove liveness. + +### The Liveness Module + +A `LivenessModule` is also created which does the following: + +1. Has a function `removeSigner()` that anyone may call to specify a signer to be removed from the + Safe. +1. The Module would then check the `LivenessGuard.lastSigned()` to determine if the signer is + eligible for removal. +1. If so, it will call the Safe's `removeSigner()` to remove the non-live signer, and if necessary + reduce the threshold. +1. When a member is removed, the signing parameters are modified such that `M/N` is the lowest ratio + which remains above 75%. These ratios are (9 of 12, 9 of 11, 8 of 10, 7 of 9, 6 of 8). Using + integer math, this can be expressed as `M = (N * 75 + 99) / 100`. + +### Shutdown + +In the unlikely event that the signer set (`N`) is reduced below 8, then (and only then) is a + shutdown mechanism activated which removes the existing signers, and hands control of the + multisig over to a predetermined entity. + +### Security Properties + +1. Signatures are assigned to the correct signer. +2. Non-signers are unable to create a record of having signed. +3. A signer cannot be censored or grieffed such that their signing is not recorded. +4. Signers may demonstrate liveness either by signing a transaction or by calling directly to the + guard. +5. The module implements the correct checks priort to removing a signer. +6. The module sets the correct threshold upon removing a signer. +7. During a shutdown the module correctly removes all signers, and converts the safe to a 1 of 1. From 3033dd632e23e111e0c1f6805b0eb471128d171a Mon Sep 17 00:00:00 2001 From: Maurelian Date: Fri, 20 Oct 2023 16:06:12 -0400 Subject: [PATCH 192/279] feat(ctb): Remove redundant check in LivenessModule test(ctb): Add test for get75PercentThreshold --- packages/contracts-bedrock/.gas-snapshot | 5 ++++ packages/contracts-bedrock/semver-lock.json | 2 ++ .../src/Safe/LivenessModule.sol | 16 ++++++----- .../test/LivenessModule.t.sol | 27 +++++++++++++++++++ 4 files changed, 44 insertions(+), 6 deletions(-) diff --git a/packages/contracts-bedrock/.gas-snapshot b/packages/contracts-bedrock/.gas-snapshot index 449fd2dcba8c..67e01fb5ddb4 100644 --- a/packages/contracts-bedrock/.gas-snapshot +++ b/packages/contracts-bedrock/.gas-snapshot @@ -305,6 +305,11 @@ LegacyERC20ETH_Test:test_transferFrom_doesNotExist_reverts() (gas: 12957) LegacyERC20ETH_Test:test_transfer_doesNotExist_reverts() (gas: 10755) LegacyMessagePasser_Test:test_passMessageToL1_succeeds() (gas: 34524) LibPosition_Test:test_pos_correctness_succeeds() (gas: 38689) +LivenessGuard_ShowLiveness_Test:test_showLiveness_succeeds() (gas: 51339) +LivenessModule_Get75PercentThreshold_Test:test_get75PercentThreshold_Works() (gas: 26339) +LivenessModule_RemoveOwner_Test:test_removeOwner_allOwners_succeeds() (gas: 159764) +LivenessModule_RemoveOwner_Test:test_removeOwner_oneOwner_succeeds() (gas: 109028) +LivnessGuard_CheckTx_Test:test_checkTransaction_succeeds() (gas: 160454) MIPS_Test:test_add_succeeds() (gas: 122932) MIPS_Test:test_addiSign_succeeds() (gas: 122923) MIPS_Test:test_addi_succeeds() (gas: 123120) diff --git a/packages/contracts-bedrock/semver-lock.json b/packages/contracts-bedrock/semver-lock.json index 4c56a35e20aa..07382eba2e82 100644 --- a/packages/contracts-bedrock/semver-lock.json +++ b/packages/contracts-bedrock/semver-lock.json @@ -18,6 +18,8 @@ "src/L2/L2StandardBridge.sol": "0x284ebf5569c75d98f2d1920a276d1116524399355708c4a60ea5892283c56719", "src/L2/L2ToL1MessagePasser.sol": "0xafc710b4d320ef450586d96a61cbd58cac814cb3b0c4fdc280eace3efdcdf321", "src/L2/SequencerFeeVault.sol": "0x883e434a69b4789997a4a9a32060dbbd2e12db6f1970927f1310820336119575", + "src/Safe/LivenessGuard.sol": "0x31b4ecc88c982490243ab42914c3de75e5acfa421ffc0ea0d0f0997dcc0341b5", + "src/Safe/LivenessModule.sol": "0xb8c8178c1f4f78eed4777846a40eda6a3a0c1710085822d92267339ae752799b", "src/dispute/BlockOracle.sol": "0x7e724b1ee0116dfd744f556e6237af449c2f40c6426d6f1462ae2a47589283bb", "src/dispute/DisputeGameFactory.sol": "0xfdfa141408d7f8de7e230ff4bef088e30d0e4d569ca743d60d292abdd21ff270", "src/dispute/FaultDisputeGame.sol": "0x0766707ab32338a6586c2340ddfbfd4e9023eeb9dfa3ef87e4b404fb0260479f", diff --git a/packages/contracts-bedrock/src/Safe/LivenessModule.sol b/packages/contracts-bedrock/src/Safe/LivenessModule.sol index 46f1f40a15f5..20c6bf72e12f 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessModule.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessModule.sol @@ -1,7 +1,7 @@ // SPDX-License-Identifier: MIT pragma solidity 0.8.15; -import { Safe } from "safe-contracts/Safe.sol"; +import { Safe, OwnerManager } from "safe-contracts/Safe.sol"; import { Enum } from "safe-contracts/common/Enum.sol"; import { OwnerManager } from "safe-contracts/base/OwnerManager.sol"; import { LivenessGuard } from "src/Safe/LivenessGuard.sol"; @@ -29,6 +29,9 @@ contract LivenessModule is ISemver { /// @notice The fallback owner of the Safe address public fallbackOwner; + /// @notice The address of the first owner in the linked list of owners + address internal constant SENTINEL_OWNERS = address(0x1); + /// @notice Semantic version. /// @custom:semver 1.0.0 string public constant version = "1.0.0"; @@ -120,15 +123,15 @@ contract LivenessModule is ISemver { address[] memory owners = safe.getOwners(); uint256 numOwners = owners.length; require( - (numOwners == 1 && owners[0] == fallbackOwner) || (numOwners >= minOwners), - "LivenessModule: Safe must have at least 1 owner or minOwners" + (numOwners >= minOwners) || (numOwners == 1 && owners[0] == fallbackOwner), + "LivenessModule: Safe must have the minimum number of owners, or be owned solely by the fallback owner" ); // Check that the threshold is correct uint256 threshold = safe.getThreshold(); require( - threshold == get75PercentThreshold(numOwners) || (numOwners == 1 && threshold == 1), - "LivenessModule: threshold must be 75% of the number of owners, or 1 if there is only 1 owner" + threshold == get75PercentThreshold(numOwners), + "LivenessModule: threshold must be 75% of the number of owners" ); } @@ -137,7 +140,7 @@ contract LivenessModule is ISemver { for (uint256 i = 0; i < owners.length; i++) { if (owners[i] == owner) { if (i == 0) { - prevOwner_ = address(0x1); // OwnerManager.SENTINEL_OWNERS + prevOwner_ = SENTINEL_OWNERS; break; } prevOwner_ = owners[i - 1]; @@ -147,6 +150,7 @@ contract LivenessModule is ISemver { } /// @notice For a given number of owners, return the lowest threshold which is greater than 75. + /// Note: this function returns 1 for numOwners == 1. function get75PercentThreshold(uint256 _numOwners) public pure returns (uint256 threshold_) { threshold_ = (_numOwners * 75 + 99) / 100; } diff --git a/packages/contracts-bedrock/test/LivenessModule.t.sol b/packages/contracts-bedrock/test/LivenessModule.t.sol index 1276e689362e..9e4ebdbf5ca4 100644 --- a/packages/contracts-bedrock/test/LivenessModule.t.sol +++ b/packages/contracts-bedrock/test/LivenessModule.t.sol @@ -36,6 +36,33 @@ contract LivnessModule_TestInit is Test, SafeTestTools { } } +contract LivenessModule_Get75PercentThreshold_Test is LivnessModule_TestInit { + /// @dev check the return values of the get75PercentThreshold function against manually + /// calculated values. + function test_get75PercentThreshold_Works() external { + assertEq(livenessModule.get75PercentThreshold(20), 15); + assertEq(livenessModule.get75PercentThreshold(19), 15); + assertEq(livenessModule.get75PercentThreshold(18), 14); + assertEq(livenessModule.get75PercentThreshold(17), 13); + assertEq(livenessModule.get75PercentThreshold(16), 12); + assertEq(livenessModule.get75PercentThreshold(15), 12); + assertEq(livenessModule.get75PercentThreshold(14), 11); + assertEq(livenessModule.get75PercentThreshold(13), 10); + assertEq(livenessModule.get75PercentThreshold(12), 9); + assertEq(livenessModule.get75PercentThreshold(11), 9); + assertEq(livenessModule.get75PercentThreshold(10), 8); + assertEq(livenessModule.get75PercentThreshold(9), 7); + assertEq(livenessModule.get75PercentThreshold(8), 6); + assertEq(livenessModule.get75PercentThreshold(7), 6); + assertEq(livenessModule.get75PercentThreshold(6), 5); + assertEq(livenessModule.get75PercentThreshold(5), 4); + assertEq(livenessModule.get75PercentThreshold(4), 3); + assertEq(livenessModule.get75PercentThreshold(3), 3); + assertEq(livenessModule.get75PercentThreshold(2), 2); + assertEq(livenessModule.get75PercentThreshold(1), 1); + } +} + contract LivenessModule_RemoveOwner_Test is LivnessModule_TestInit { function test_removeOwner_oneOwner_succeeds() external { uint256 ownersBefore = safeInstance.owners.length; From 0831cd6dc288bd90016847b4ff9d7aa0924ca918 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Mon, 23 Oct 2023 12:50:22 -0400 Subject: [PATCH 193/279] feat(ctb): Ensure adding and removing owners is handled correctly OZ's EnumerableSet library is used to store the set of owners prior to execution, and then to compare with the set of owners after execution, and to add/remove those addresses from the mapping accordingly. --- .../src/Safe/LivenessGuard.sol | 35 +++++++++++++++++-- 1 file changed, 33 insertions(+), 2 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol index 38f07fe4c684..1de36297f70d 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol @@ -7,8 +7,11 @@ import { ModuleManager } from "safe-contracts/base/ModuleManager.sol"; import { GetSigners } from "src/Safe/GetSigners.sol"; import { Enum } from "safe-contracts/common/Enum.sol"; import { ISemver } from "src/universal/ISemver.sol"; +import { EnumerableSet } from "@openzeppelin/contracts/utils/structs/EnumerableSet.sol"; contract LivenessGuard is ISemver, GetSigners, BaseGuard { + using EnumerableSet for EnumerableSet.AddressSet; + /// @notice Emitted when a new set of signers is recorded. /// @param signers An arrary of signer addresses. event SignersRecorded(bytes32 indexed txHash, address[] signers); @@ -20,12 +23,33 @@ contract LivenessGuard is ISemver, GetSigners, BaseGuard { Safe public immutable safe; mapping(address => uint256) public lastSigned; + EnumerableSet.AddressSet private ownersBefore; + constructor(Safe _safe) { safe = _safe; } - /// @notice We just need to satisfy the BaseGuard interfae, but we don't actually need to use this method. - function checkAfterExecution(bytes32, bool) external pure { + /// @notice We use this post execution hook to compare the set of owners before and after. + /// If the set of owners has changed then we: + /// 1. Add new owners to the lastSigned mapping + /// 2. Delete removed owners from the lastSigned mapping + function checkAfterExecution(bytes32, bool) external { + address[] memory ownersAfter = safe.getOwners(); + for (uint256 i = 0; i < ownersAfter.length; i++) { + if (ownersBefore.contains(ownersAfter[i])) { + // This address was already present, no change, remove it from the set. + ownersBefore.remove(ownersAfter[i]); + } else { + // This address is newly added, add it to the lastSigned mapping + lastSigned[ownersAfter[i]] = block.timestamp; + } + // Iterate over ownersSet. Any remaining addresses are no longer an owner, so we delete + // it from the lastSigned mapping. + for (uint256 j = 0; j < ownersBefore.length(); j++) { + address owner = ownersBefore.at(j); + delete lastSigned[owner]; + } + } return; } @@ -49,6 +73,13 @@ contract LivenessGuard is ISemver, GetSigners, BaseGuard { { require(msg.sender == address(safe), "LivenessGuard: only Safe can call this function"); + // Cache the set of owners prior to execution. + // This will be used in the checkAfterExecution method. + address[] memory owners = safe.getOwners(); + for (uint256 i = 0; i < owners.length; i++) { + ownersBefore.add(owners[i]); + } + // This call will reenter to the Safe which is calling it. This is OK because it is only reading the // nonce, and using the getTransactionHash() method. bytes32 txHash = Safe(payable(msg.sender)).getTransactionHash( From 5a1baf8f85c4f6f003c4d55e1b82fe31ee22ba87 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Mon, 23 Oct 2023 12:57:29 -0400 Subject: [PATCH 194/279] feat(ctb): Clean up and comment the checkAfterExecution hook --- .../src/Safe/LivenessGuard.sol | 23 ++++++++++--------- 1 file changed, 12 insertions(+), 11 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol index 1de36297f70d..39da08c2775f 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol @@ -34,21 +34,22 @@ contract LivenessGuard is ISemver, GetSigners, BaseGuard { /// 1. Add new owners to the lastSigned mapping /// 2. Delete removed owners from the lastSigned mapping function checkAfterExecution(bytes32, bool) external { + // Get the current set of owners address[] memory ownersAfter = safe.getOwners(); + + // Iterate over the current owners, and remove one at a time from the ownersBefore set. for (uint256 i = 0; i < ownersAfter.length; i++) { - if (ownersBefore.contains(ownersAfter[i])) { - // This address was already present, no change, remove it from the set. - ownersBefore.remove(ownersAfter[i]); - } else { - // This address is newly added, add it to the lastSigned mapping + // If the value was present, remove() returns true. + if (ownersBefore.remove(ownersAfter[i]) == false) { + // This address was not already an owner, add it to the lastSigned mapping lastSigned[ownersAfter[i]] = block.timestamp; } - // Iterate over ownersSet. Any remaining addresses are no longer an owner, so we delete - // it from the lastSigned mapping. - for (uint256 j = 0; j < ownersBefore.length(); j++) { - address owner = ownersBefore.at(j); - delete lastSigned[owner]; - } + } + // Now iterate over the remaining ownersBefore entries. Any remaining addresses are no longer an owner, so we + // delete them from the lastSigned mapping. + for (uint256 j = 0; j < ownersBefore.length(); j++) { + address owner = ownersBefore.at(j); + delete lastSigned[owner]; } return; } From e8f9e2e6226d90047c2ae73c02ced1509d5114ce Mon Sep 17 00:00:00 2001 From: Maurelian Date: Mon, 23 Oct 2023 16:01:57 -0400 Subject: [PATCH 195/279] specs: Apply suggested edits from review h/t @mds1 Co-authored-by: Matt Solomon --- specs/safe-liveness-checking.md | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/specs/safe-liveness-checking.md b/specs/safe-liveness-checking.md index b4d145c50ce2..8a891d2b6d3d 100644 --- a/specs/safe-liveness-checking.md +++ b/specs/safe-liveness-checking.md @@ -40,7 +40,7 @@ Signers may also call the contract directly in order to prove liveness. A `LivenessModule` is also created which does the following: -1. Has a function `removeSigner()` that anyone may call to specify a signer to be removed from the +1. Has a function `removeOwner()` that anyone may call to specify an owner to be removed from the Safe. 1. The Module would then check the `LivenessGuard.lastSigned()` to determine if the signer is eligible for removal. @@ -58,11 +58,13 @@ In the unlikely event that the signer set (`N`) is reduced below 8, then (and on ### Security Properties +The following security properties must be upheld: + 1. Signatures are assigned to the correct signer. 2. Non-signers are unable to create a record of having signed. -3. A signer cannot be censored or grieffed such that their signing is not recorded. +3. A signer cannot be censored or griefed such that their signing is not recorded. 4. Signers may demonstrate liveness either by signing a transaction or by calling directly to the guard. -5. The module implements the correct checks priort to removing a signer. +5. The module implements the correct checks prior to removing a signer. 6. The module sets the correct threshold upon removing a signer. 7. During a shutdown the module correctly removes all signers, and converts the safe to a 1 of 1. From 38f828d054668ebdcddaa36e4961a16e7e849bab Mon Sep 17 00:00:00 2001 From: Maurelian Date: Mon, 23 Oct 2023 13:09:24 -0400 Subject: [PATCH 196/279] feat(ctb): Rename lastSigned to more accurate lastLive This is more accurate because of the fact that showLiveness can be used without approving a transaction. --- packages/contracts-bedrock/src/Safe/LivenessGuard.sol | 10 +++++----- packages/contracts-bedrock/src/Safe/LivenessModule.sol | 2 +- packages/contracts-bedrock/test/LivenessGuard.t.sol | 4 ++-- specs/safe-liveness-checking.md | 7 +++---- 4 files changed, 11 insertions(+), 12 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol index 39da08c2775f..310e771f27c5 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol @@ -21,8 +21,8 @@ contract LivenessGuard is ISemver, GetSigners, BaseGuard { string public constant version = "1.0.0"; Safe public immutable safe; - mapping(address => uint256) public lastSigned; + mapping(address => uint256) public lastLive; EnumerableSet.AddressSet private ownersBefore; constructor(Safe _safe) { @@ -42,14 +42,14 @@ contract LivenessGuard is ISemver, GetSigners, BaseGuard { // If the value was present, remove() returns true. if (ownersBefore.remove(ownersAfter[i]) == false) { // This address was not already an owner, add it to the lastSigned mapping - lastSigned[ownersAfter[i]] = block.timestamp; + lastLive[ownersAfter[i]] = block.timestamp; } } // Now iterate over the remaining ownersBefore entries. Any remaining addresses are no longer an owner, so we // delete them from the lastSigned mapping. for (uint256 j = 0; j < ownersBefore.length(); j++) { address owner = ownersBefore.at(j); - delete lastSigned[owner]; + delete lastLive[owner]; } return; } @@ -104,7 +104,7 @@ contract LivenessGuard is ISemver, GetSigners, BaseGuard { _getNSigners({ dataHash: txHash, signatures: signatures, requiredSignatures: threshold }); for (uint256 i = 0; i < signers.length; i++) { - lastSigned[signers[i]] = block.timestamp; + lastLive[signers[i]] = block.timestamp; } emit SignersRecorded(txHash, signers); } @@ -113,7 +113,7 @@ contract LivenessGuard is ISemver, GetSigners, BaseGuard { /// This is useful for owners who have not recently signed a transaction via the Safe. function showLiveness() external { require(safe.isOwner(msg.sender), "LivenessGuard: only Safe owners may demontstrate liveness"); - lastSigned[msg.sender] = block.timestamp; + lastLive[msg.sender] = block.timestamp; address[] memory signers = new address[](1); signers[0] = msg.sender; diff --git a/packages/contracts-bedrock/src/Safe/LivenessModule.sol b/packages/contracts-bedrock/src/Safe/LivenessModule.sol index 20c6bf72e12f..a7ccbf42c713 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessModule.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessModule.sol @@ -56,7 +56,7 @@ contract LivenessModule is ISemver { function removeOwner(address owner) external { // Check that the owner has not signed a transaction in the last 30 days require( - livenessGuard.lastSigned(owner) < block.timestamp - livenessInterval, + livenessGuard.lastLive(owner) < block.timestamp - livenessInterval, "LivenessModule: owner has signed recently" ); diff --git a/packages/contracts-bedrock/test/LivenessGuard.t.sol b/packages/contracts-bedrock/test/LivenessGuard.t.sol index d85466081c57..8046fb01b0ad 100644 --- a/packages/contracts-bedrock/test/LivenessGuard.t.sol +++ b/packages/contracts-bedrock/test/LivenessGuard.t.sol @@ -51,7 +51,7 @@ contract LivnessGuard_CheckTx_Test is LivnessGuard_TestInit { safeInstance.execTransaction({ to: address(1111), value: 0, data: hex"abba" }); for (uint256 i; i < safeInstance.threshold; i++) { - assertEq(livenessGuard.lastSigned(safeInstance.owners[i]), block.timestamp); + assertEq(livenessGuard.lastLive(safeInstance.owners[i]), block.timestamp); } } } @@ -70,6 +70,6 @@ contract LivenessGuard_ShowLiveness_Test is LivnessGuard_TestInit { vm.prank(caller); livenessGuard.showLiveness(); - assertEq(livenessGuard.lastSigned(caller), block.timestamp); + assertEq(livenessGuard.lastLive(caller), block.timestamp); } } diff --git a/specs/safe-liveness-checking.md b/specs/safe-liveness-checking.md index 8a891d2b6d3d..1a71cfee83fa 100644 --- a/specs/safe-liveness-checking.md +++ b/specs/safe-liveness-checking.md @@ -32,7 +32,7 @@ This is achieved using two types of contracts which the Safe contract has built- For implementing liveness checks a `LivenessGuard` is created which receives the signatures from each executed transaction, and tracks the latest time at which a transaction was signed by each -signer. This time is made publicly available by calling a `lastSigned(address)(Timestamp)` method. +signer. This time is made publicly available by calling a `lastLive(address)(Timestamp)` method. Signers may also call the contract directly in order to prove liveness. @@ -42,13 +42,12 @@ A `LivenessModule` is also created which does the following: 1. Has a function `removeOwner()` that anyone may call to specify an owner to be removed from the Safe. -1. The Module would then check the `LivenessGuard.lastSigned()` to determine if the signer is +1. The Module would then check the `LivenessGuard.lastLive()` to determine if the signer is eligible for removal. 1. If so, it will call the Safe's `removeSigner()` to remove the non-live signer, and if necessary reduce the threshold. 1. When a member is removed, the signing parameters are modified such that `M/N` is the lowest ratio - which remains above 75%. These ratios are (9 of 12, 9 of 11, 8 of 10, 7 of 9, 6 of 8). Using - integer math, this can be expressed as `M = (N * 75 + 99) / 100`. + which remains above 75%. Using integer math, this can be expressed as `M = (N * 75 + 99) / 100`. ### Shutdown From 2ce7314d481a0d73fe8c1590cd16aa1f0a3994eb Mon Sep 17 00:00:00 2001 From: Maurelian Date: Mon, 23 Oct 2023 13:10:56 -0400 Subject: [PATCH 197/279] feat(ctb): Add missing natspec to LivenessGuard and LivenessModule --- packages/contracts-bedrock/src/Safe/LivenessGuard.sol | 9 +++++++++ packages/contracts-bedrock/src/Safe/LivenessModule.sol | 7 +++++-- 2 files changed, 14 insertions(+), 2 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol index 310e771f27c5..a472c3229d44 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol @@ -20,11 +20,20 @@ contract LivenessGuard is ISemver, GetSigners, BaseGuard { /// @custom:semver 1.0.0 string public constant version = "1.0.0"; + /// @notice The safe account for which this contract will be the guard. Safe public immutable safe; + /// @notice A mapping of the timestamp at which an owner last participated in signing a + /// an executed transaction. mapping(address => uint256) public lastLive; + + /// @notice An enumerable set of addresses used to store the list of owners before execution, + /// and then to update the lastSigned mapping according to changes in the set observed + /// after execution. EnumerableSet.AddressSet private ownersBefore; + /// @notice Constructor. + /// @param _safe The safe account for which this contract will be the guard. constructor(Safe _safe) { safe = _safe; } diff --git a/packages/contracts-bedrock/src/Safe/LivenessModule.sol b/packages/contracts-bedrock/src/Safe/LivenessModule.sol index a7ccbf42c713..795c2fcc85c0 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessModule.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessModule.sol @@ -11,8 +11,11 @@ import { ISemver } from "src/universal/ISemver.sol"; import { console2 as console } from "forge-std/console2.sol"; /// @title LivenessModule -/// @notice This module is intended to be used in conjunction with the LivenessGuard. It should be able to -/// execute a transaction on the Safe in only a small number of cases. +/// @notice This module is intended to be used in conjunction with the LivenessGuard. In the event +/// that an owner of the safe is not recorded by the guard during the liveness interval, +/// the owner will be considered inactive and will be removed from the list of owners. +/// If the number of owners falls below the minimum number of owners, the ownership of the +/// safe will be transferred to the fallback owner. contract LivenessModule is ISemver { /// @notice The Safe contract instance Safe public safe; From aef990f387fca34cff5bf2f71cd1d2a222ceb207 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Mon, 23 Oct 2023 13:27:56 -0400 Subject: [PATCH 198/279] feat(ctb): Make guard state immutable --- .../src/Safe/LivenessModule.sol | 31 ++++++++++++++----- 1 file changed, 23 insertions(+), 8 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessModule.sol b/packages/contracts-bedrock/src/Safe/LivenessModule.sol index 795c2fcc85c0..0954ab7c517f 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessModule.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessModule.sol @@ -18,19 +18,23 @@ import { console2 as console } from "forge-std/console2.sol"; /// safe will be transferred to the fallback owner. contract LivenessModule is ISemver { /// @notice The Safe contract instance - Safe public safe; + Safe public immutable safe; /// @notice The LivenessGuard contract instance - LivenessGuard public livenessGuard; + /// This can be updated by replacing with a new module and switching out the guard. + LivenessGuard public immutable livenessGuard; /// @notice The interval, in seconds, during which an owner must have demonstrated liveness - uint256 public livenessInterval; + /// This can be updated by replacing with a new module. + uint256 public immutable livenessInterval; /// @notice The minimum number of owners before ownership of the safe is transferred to the fallback owner. - uint256 public minOwners; + /// This can be updated by replacing with a new module. + uint256 public immutable minOwners; /// @notice The fallback owner of the Safe - address public fallbackOwner; + /// This can be updated by replacing with a new module. + address public immutable fallbackOwner; /// @notice The address of the first owner in the linked list of owners address internal constant SENTINEL_OWNERS = address(0x1); @@ -55,8 +59,12 @@ contract LivenessModule is ISemver { } /// @notice This function can be called by anyone to remove an owner that has not signed a transaction - /// during the livness interval. If the number of owners drops below + /// during the liveness interval. If the number of owners drops below the minimum, then the + /// ownership of the Safe is transferred to the fallback owner. function removeOwner(address owner) external { + // Check that the guard has not been changed + require(livenessGuard == safe.getGuard(), "LivenessModule: guard has been changed"); + // Check that the owner has not signed a transaction in the last 30 days require( livenessGuard.lastLive(owner) < block.timestamp - livenessInterval, @@ -67,7 +75,7 @@ contract LivenessModule is ISemver { address[] memory owners = safe.getOwners(); uint256 numOwners = owners.length - 1; uint256 thresholdAfter; - if (numOwners > minOwners) { + if (hasMinOwners(numOwners)) { // Preserves the invariant that the Safe has at least 8 owners thresholdAfter = get75PercentThreshold(numOwners); @@ -126,7 +134,7 @@ contract LivenessModule is ISemver { address[] memory owners = safe.getOwners(); uint256 numOwners = owners.length; require( - (numOwners >= minOwners) || (numOwners == 1 && owners[0] == fallbackOwner), + hasMinOwners(numOwners) || (numOwners == 1 && owners[0] == fallbackOwner), "LivenessModule: Safe must have the minimum number of owners, or be owned solely by the fallback owner" ); @@ -157,4 +165,11 @@ contract LivenessModule is ISemver { function get75PercentThreshold(uint256 _numOwners) public pure returns (uint256 threshold_) { threshold_ = (_numOwners * 75 + 99) / 100; } + + /// @notice Check if the number of owners is greater than or equal to the minimum number of owners. + /// @param numOwners The number of owners. + /// @return A boolean indicating if the number of owners is greater than or equal to the minimum number of owners. + function hasMinOwners(uint256 numOwners) public view returns (bool) { + return numOwners >= minOwners; + } } From 20a0c6c432beac9ab595c3c9a335cb5221f8f347 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Mon, 23 Oct 2023 13:58:39 -0400 Subject: [PATCH 199/279] feat(ctb): ensure guard has not changed before calling removeOwner --- packages/contracts-bedrock/src/Safe/LivenessModule.sol | 9 ++++++++- packages/contracts-bedrock/test/LivenessModule.t.sol | 1 + 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessModule.sol b/packages/contracts-bedrock/src/Safe/LivenessModule.sol index 0954ab7c517f..f7efcdb253bb 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessModule.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessModule.sol @@ -36,6 +36,10 @@ contract LivenessModule is ISemver { /// This can be updated by replacing with a new module. address public immutable fallbackOwner; + /// @notice The storage slot used in the safe to store the guard address + /// keccak256("guard_manager.guard.address") + uint256 internal constant GUARD_STORAGE_SLOT = 0x4a204f620c8c5ccdca3fd54d003badd85ba500436a431f0cbda4f558c93c34c8; + /// @notice The address of the first owner in the linked list of owners address internal constant SENTINEL_OWNERS = address(0x1); @@ -63,7 +67,10 @@ contract LivenessModule is ISemver { /// ownership of the Safe is transferred to the fallback owner. function removeOwner(address owner) external { // Check that the guard has not been changed - require(livenessGuard == safe.getGuard(), "LivenessModule: guard has been changed"); + require( + address(livenessGuard) == address(uint160(uint256(bytes32(safe.getStorageAt(GUARD_STORAGE_SLOT, 1))))), + "LivenessModule: guard has been changed" + ); // Check that the owner has not signed a transaction in the last 30 days require( diff --git a/packages/contracts-bedrock/test/LivenessModule.t.sol b/packages/contracts-bedrock/test/LivenessModule.t.sol index 9e4ebdbf5ca4..0e7a262af036 100644 --- a/packages/contracts-bedrock/test/LivenessModule.t.sol +++ b/packages/contracts-bedrock/test/LivenessModule.t.sol @@ -33,6 +33,7 @@ contract LivnessModule_TestInit is Test, SafeTestTools { _fallbackOwner: makeAddr("fallbackOwner") }); safeInstance.enableModule(address(livenessModule)); + safeInstance.setGuard(address(livenessGuard)); } } From da8a1f235c18e70b83f5b4bcaa6a37a76d03a6cd Mon Sep 17 00:00:00 2001 From: Maurelian Date: Mon, 23 Oct 2023 15:14:26 -0400 Subject: [PATCH 200/279] refactor(ctb): Extract execTransactionFromModule calls into helper functions --- .../src/Safe/LivenessModule.sol | 81 ++++++++++--------- 1 file changed, 42 insertions(+), 39 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessModule.sol b/packages/contracts-bedrock/src/Safe/LivenessModule.sol index f7efcdb253bb..3213a2e23ad2 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessModule.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessModule.sol @@ -83,59 +83,64 @@ contract LivenessModule is ISemver { uint256 numOwners = owners.length - 1; uint256 thresholdAfter; if (hasMinOwners(numOwners)) { - // Preserves the invariant that the Safe has at least 8 owners + // Preserves the invariant that the Safe has at least numOwners thresholdAfter = get75PercentThreshold(numOwners); console.log("removing one owner. numOwners: %s, thresholdAfter: %s", numOwners, thresholdAfter); - safe.execTransactionFromModule({ - to: address(safe), - value: 0, - data: abi.encodeCall( - // Call the Safe to remove the owner - OwnerManager.removeOwner, - (getPrevOwner(owner, owners), owner, thresholdAfter) - ), - operation: Enum.Operation.Call - }); + address prevOwner = _getPrevOwner(owner, owners); + // Call the Safe to remove the owner + _removeOwner({ _prevOwner: prevOwner, _owner: owner, _threshold: thresholdAfter }); } else { console.log("removing all owners. numOwnersAfter: %s", numOwners); - // The number of owners is dangerously low, so we wish to transfer the ownership of this Safe to a new + // The number of owners is dangerously low, so we wish to transfer the ownership of this Safe // to the fallback owner. // The threshold will be 1 because we are removing all owners except the fallback owner - thresholdAfter = 1; + // thresholdAfter = 1; // todo: why is this here? We should be able to delete it. // Remove owners one at a time starting from the last owner. - // Since we're removing them in order, the ordering will remain constant, - // and we shouldn't need to query the list of owners again. + // Since we're removing them in order from last to first, the ordering will remain constant, + // and we shouldn't need to query the list of owners again. for (uint256 i = owners.length - 1; i >= 0; i--) { address currentOwner = owners[i]; - address prevOwner = getPrevOwner(currentOwner, owners); + address prevOwner = _getPrevOwner(currentOwner, owners); if (currentOwner != address(this)) { - safe.execTransactionFromModule({ - to: address(safe), - value: 0, - data: abi.encodeCall( - // Call the Safe to remove the owner - OwnerManager.removeOwner, - (prevOwner, currentOwner, 1) - ), - operation: Enum.Operation.Call - }); + // Call the Safe to remove the owner + _removeOwner({ _prevOwner: prevOwner, _owner: currentOwner, _threshold: 1 }); } } // Add the fallback owner as the sole owner of the Safe - safe.execTransactionFromModule({ - to: address(safe), - value: 0, - data: abi.encodeCall(OwnerManager.addOwnerWithThreshold, (fallbackOwner, 1)), - operation: Enum.Operation.Call - }); + _addOwnerWithThreshold({ _owner: fallbackOwner, _threshold: 1 }); } _verifyFinalState(); } + /// @notice Adds the owner `owner` to the Safe and updates the threshold to `_threshold`. + /// @param _owner New owner address. + /// @param _threshold New threshold. + function _addOwnerWithThreshold(address _owner, uint256 _threshold) internal { + safe.execTransactionFromModule({ + to: address(safe), + value: 0, + operation: Enum.Operation.Call, + data: abi.encodeCall(OwnerManager.addOwnerWithThreshold, (_owner, _threshold)) + }); + } + + /// @notice Removes the owner `owner` from the Safe and updates the threshold to `_threshold`. + /// @param _prevOwner Owner that pointed to the owner to be removed in the linked list + /// @param _owner Owner address to be removed. + /// @param _threshold New threshold. + function _removeOwner(address _prevOwner, address _owner, uint256 _threshold) internal { + safe.execTransactionFromModule({ + to: address(safe), + value: 0, + operation: Enum.Operation.Call, + data: abi.encodeCall(OwnerManager.removeOwner, (_prevOwner, _owner, _threshold)) + }); + } + /// @notice A FREI-PI invariant check enforcing requirements on number of owners and threshold. function _verifyFinalState() internal view { address[] memory owners = safe.getOwners(); @@ -154,16 +159,14 @@ contract LivenessModule is ISemver { } /// @notice Get the previous owner in the linked list of owners - function getPrevOwner(address owner, address[] memory owners) public pure returns (address prevOwner_) { + function _getPrevOwner(address owner, address[] memory owners) internal pure returns (address prevOwner_) { for (uint256 i = 0; i < owners.length; i++) { - if (owners[i] == owner) { - if (i == 0) { - prevOwner_ = SENTINEL_OWNERS; - break; - } - prevOwner_ = owners[i - 1]; + if (owners[i] != owner) continue; + if (i == 0) { + prevOwner_ = SENTINEL_OWNERS; break; } + prevOwner_ = owners[i - 1]; } } From 66a0eee4772cfdfc9ceae635842fe4bd6169555c Mon Sep 17 00:00:00 2001 From: Maurelian Date: Mon, 23 Oct 2023 15:22:39 -0400 Subject: [PATCH 201/279] refactor(ctb): Abstract addOwner method to giveToFallbackOwner --- .../contracts-bedrock/src/Safe/LivenessModule.sol | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessModule.sol b/packages/contracts-bedrock/src/Safe/LivenessModule.sol index 3213a2e23ad2..52b96a29b373 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessModule.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessModule.sol @@ -111,20 +111,19 @@ contract LivenessModule is ISemver { } // Add the fallback owner as the sole owner of the Safe - _addOwnerWithThreshold({ _owner: fallbackOwner, _threshold: 1 }); + _giveToFallbackOwner(); } + _verifyFinalState(); } - /// @notice Adds the owner `owner` to the Safe and updates the threshold to `_threshold`. - /// @param _owner New owner address. - /// @param _threshold New threshold. - function _addOwnerWithThreshold(address _owner, uint256 _threshold) internal { + /// @notice Sets the fallback owner as the sole owner of the Safe with a threshold of 1 + function _giveToFallbackOwner() internal { safe.execTransactionFromModule({ to: address(safe), value: 0, operation: Enum.Operation.Call, - data: abi.encodeCall(OwnerManager.addOwnerWithThreshold, (_owner, _threshold)) + data: abi.encodeCall(OwnerManager.addOwnerWithThreshold, (fallbackOwner, 1)) }); } From 139751ad945ac80062c176b053cc86ae57b0cb30 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Mon, 23 Oct 2023 15:23:13 -0400 Subject: [PATCH 202/279] feat(ctb): Clean up removeOwner method on module --- .../contracts-bedrock/src/Safe/LivenessModule.sol | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessModule.sol b/packages/contracts-bedrock/src/Safe/LivenessModule.sol index 52b96a29b373..e1b4d6459767 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessModule.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessModule.sol @@ -72,7 +72,7 @@ contract LivenessModule is ISemver { "LivenessModule: guard has been changed" ); - // Check that the owner has not signed a transaction in the last 30 days + // Check that the owner to remove has not signed a transaction in the last 30 days require( livenessGuard.lastLive(owner) < block.timestamp - livenessInterval, "LivenessModule: owner has signed recently" @@ -83,21 +83,14 @@ contract LivenessModule is ISemver { uint256 numOwners = owners.length - 1; uint256 thresholdAfter; if (hasMinOwners(numOwners)) { - // Preserves the invariant that the Safe has at least numOwners - + // Call the Safe to remove the owner and update the threshold thresholdAfter = get75PercentThreshold(numOwners); - console.log("removing one owner. numOwners: %s, thresholdAfter: %s", numOwners, thresholdAfter); address prevOwner = _getPrevOwner(owner, owners); - // Call the Safe to remove the owner _removeOwner({ _prevOwner: prevOwner, _owner: owner, _threshold: thresholdAfter }); } else { - console.log("removing all owners. numOwnersAfter: %s", numOwners); // The number of owners is dangerously low, so we wish to transfer the ownership of this Safe // to the fallback owner. - // The threshold will be 1 because we are removing all owners except the fallback owner - // thresholdAfter = 1; // todo: why is this here? We should be able to delete it. - // Remove owners one at a time starting from the last owner. // Since we're removing them in order from last to first, the ordering will remain constant, // and we shouldn't need to query the list of owners again. From b157425580013d861231d240e871462dafe7f428 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Mon, 23 Oct 2023 15:23:44 -0400 Subject: [PATCH 203/279] feat(ctb): Add guard check to _verifyState --- packages/contracts-bedrock/src/Safe/LivenessModule.sol | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/packages/contracts-bedrock/src/Safe/LivenessModule.sol b/packages/contracts-bedrock/src/Safe/LivenessModule.sol index e1b4d6459767..c90a81b71479 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessModule.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessModule.sol @@ -148,6 +148,12 @@ contract LivenessModule is ISemver { threshold == get75PercentThreshold(numOwners), "LivenessModule: threshold must be 75% of the number of owners" ); + + // Check that the guard has not been changed + require( + address(livenessGuard) == address(uint160(uint256(bytes32(safe.getStorageAt(GUARD_STORAGE_SLOT, 1))))), + "LivenessModule: guard has been changed" + ); } /// @notice Get the previous owner in the linked list of owners From 4a042355f231d650d465edfefde868f46e84a496 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Mon, 23 Oct 2023 15:48:38 -0400 Subject: [PATCH 204/279] refactor(ctb): Make GetSigners into a library rather than contract This required copy/pasting the SignatureDecoder code from the Safe repo. --- .../contracts-bedrock/src/Safe/GetSigners.sol | 39 +++++++++++++++++-- .../src/Safe/LivenessGuard.sol | 4 +- .../contracts-bedrock/test/GetSigners.t.sol | 4 +- 3 files changed, 40 insertions(+), 7 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/GetSigners.sol b/packages/contracts-bedrock/src/Safe/GetSigners.sol index 0d9642bf6125..7ef639f675b9 100644 --- a/packages/contracts-bedrock/src/Safe/GetSigners.sol +++ b/packages/contracts-bedrock/src/Safe/GetSigners.sol @@ -1,14 +1,47 @@ // SPDX-License-Identifier: MIT pragma solidity 0.8.15; -import { SignatureDecoder } from "safe-contracts/common/SignatureDecoder.sol"; +library GetSigners { + /// @notice Splits signature bytes into `uint8 v, bytes32 r, bytes32 s`. + /// Copied directly from + /// https://github.com/safe-global/safe-contracts/blob/e870f514ad34cd9654c72174d6d4a839e3c6639f/contracts/common/SignatureDecoder.sol + /// @dev Make sure to perform a bounds check for @param pos, to avoid out of bounds access on @param signatures + /// The signature format is a compact form of {bytes32 r}{bytes32 s}{uint8 v} + /// Compact means uint8 is not padded to 32 bytes. + /// @param pos Which signature to read. + /// A prior bounds check of this parameter should be performed, to avoid out of bounds access. + /// @param signatures Concatenated {r, s, v} signatures. + /// @return v Recovery ID or Safe signature type. + /// @return r Output value r of the signature. + /// @return s Output value s of the signature. + function signatureSplit( + bytes memory signatures, + uint256 pos + ) + internal + pure + returns (uint8 v, bytes32 r, bytes32 s) + { + // solhint-disable-next-line no-inline-assembly + assembly { + let signaturePos := mul(0x41, pos) + r := mload(add(signatures, add(signaturePos, 0x20))) + s := mload(add(signatures, add(signaturePos, 0x40))) + /** + * Here we are loading the last 32 bytes, including 31 bytes + * of 's'. There is no 'mload8' to do this. + * 'byte' is not working due to the Solidity parser, so lets + * use the second best option, 'and' + */ + v := and(mload(add(signatures, add(signaturePos, 0x41))), 0xff) + } + } -abstract contract GetSigners is SignatureDecoder { /// @notice Extract the signers from a set of signatures. /// @param dataHash Hash of the data. /// @param signatures Signature data for identifying signers. /// @param requiredSignatures Amount of required valid signatures. - function _getNSigners( + function getNSigners( bytes32 dataHash, bytes memory signatures, uint256 requiredSignatures diff --git a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol index a472c3229d44..2d7cca663f96 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol @@ -9,7 +9,7 @@ import { Enum } from "safe-contracts/common/Enum.sol"; import { ISemver } from "src/universal/ISemver.sol"; import { EnumerableSet } from "@openzeppelin/contracts/utils/structs/EnumerableSet.sol"; -contract LivenessGuard is ISemver, GetSigners, BaseGuard { +contract LivenessGuard is ISemver, BaseGuard { using EnumerableSet for EnumerableSet.AddressSet; /// @notice Emitted when a new set of signers is recorded. @@ -110,7 +110,7 @@ contract LivenessGuard is ISemver, GetSigners, BaseGuard { uint256 threshold = safe.getThreshold(); address[] memory signers = - _getNSigners({ dataHash: txHash, signatures: signatures, requiredSignatures: threshold }); + GetSigners.getNSigners({ dataHash: txHash, signatures: signatures, requiredSignatures: threshold }); for (uint256 i = 0; i < signers.length; i++) { lastLive[signers[i]] = block.timestamp; diff --git a/packages/contracts-bedrock/test/GetSigners.t.sol b/packages/contracts-bedrock/test/GetSigners.t.sol index 03cfd711860c..7b1de0952482 100644 --- a/packages/contracts-bedrock/test/GetSigners.t.sol +++ b/packages/contracts-bedrock/test/GetSigners.t.sol @@ -8,7 +8,7 @@ import "test/safe-tools/SafeTestTools.sol"; import { SignatureDecoder } from "safe-contracts/common/SignatureDecoder.sol"; -contract GetSigners_Test is Test, SafeTestTools, GetSigners { +contract GetSigners_Test is Test, SafeTestTools { bytes4 internal constant EIP1271_MAGIC_VALUE = 0x20c13b0b; enum SigTypes { @@ -98,7 +98,7 @@ contract GetSigners_Test is Test, SafeTestTools, GetSigners { // Recover the signatures using the _getNSigners() method. address[] memory gotSigners = - _getNSigners({ dataHash: digest, signatures: signatures, requiredSignatures: numSigs }); + GetSigners.getNSigners({ dataHash: digest, signatures: signatures, requiredSignatures: numSigs }); // Compare the list of recovered signers to the expected signers. assertEq(gotSigners.length, numSigs); From 2103d61dffd0b12cdf9d035fa66baf31a110c216 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Mon, 23 Oct 2023 16:03:47 -0400 Subject: [PATCH 205/279] refactor(ctb): Extract guard check into _verifyGuard() --- .../contracts-bedrock/src/Safe/LivenessModule.sol | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessModule.sol b/packages/contracts-bedrock/src/Safe/LivenessModule.sol index c90a81b71479..ee7eefea7101 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessModule.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessModule.sol @@ -67,10 +67,7 @@ contract LivenessModule is ISemver { /// ownership of the Safe is transferred to the fallback owner. function removeOwner(address owner) external { // Check that the guard has not been changed - require( - address(livenessGuard) == address(uint160(uint256(bytes32(safe.getStorageAt(GUARD_STORAGE_SLOT, 1))))), - "LivenessModule: guard has been changed" - ); + _verifyGuard(); // Check that the owner to remove has not signed a transaction in the last 30 days require( @@ -142,14 +139,20 @@ contract LivenessModule is ISemver { "LivenessModule: Safe must have the minimum number of owners, or be owned solely by the fallback owner" ); - // Check that the threshold is correct + // Check that the threshold is correct. This check is also correct when there is a single + // owner, because get75PercentThreshold(1) returns 1. uint256 threshold = safe.getThreshold(); require( threshold == get75PercentThreshold(numOwners), "LivenessModule: threshold must be 75% of the number of owners" ); - // Check that the guard has not been changed + // Check that the guard has not been changed. + _verifyGuard(); + } + + /// @notice Reverts if the guard address does not match the expected value. + function _verifyGuard() internal view { require( address(livenessGuard) == address(uint160(uint256(bytes32(safe.getStorageAt(GUARD_STORAGE_SLOT, 1))))), "LivenessModule: guard has been changed" From 7f474593fc2e2d04e33c51f5e6067893e195a644 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Tue, 24 Oct 2023 10:28:05 -0400 Subject: [PATCH 206/279] chore(ctb): Rename lib to SafeSigners --- packages/contracts-bedrock/src/Safe/LivenessGuard.sol | 4 ++-- .../src/Safe/{GetSigners.sol => SafeSigners.sol} | 5 +++-- .../test/{GetSigners.t.sol => SafeSigners.t.sol} | 6 +++--- 3 files changed, 8 insertions(+), 7 deletions(-) rename packages/contracts-bedrock/src/Safe/{GetSigners.sol => SafeSigners.sol} (97%) rename packages/contracts-bedrock/test/{GetSigners.t.sol => SafeSigners.t.sol} (95%) diff --git a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol index 2d7cca663f96..042e47619f7c 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol @@ -4,7 +4,7 @@ pragma solidity 0.8.15; import { Safe } from "safe-contracts/Safe.sol"; import { BaseGuard, GuardManager } from "safe-contracts/base/GuardManager.sol"; import { ModuleManager } from "safe-contracts/base/ModuleManager.sol"; -import { GetSigners } from "src/Safe/GetSigners.sol"; +import { SafeSigners } from "src/Safe/SafeSigners.sol"; import { Enum } from "safe-contracts/common/Enum.sol"; import { ISemver } from "src/universal/ISemver.sol"; import { EnumerableSet } from "@openzeppelin/contracts/utils/structs/EnumerableSet.sol"; @@ -110,7 +110,7 @@ contract LivenessGuard is ISemver, BaseGuard { uint256 threshold = safe.getThreshold(); address[] memory signers = - GetSigners.getNSigners({ dataHash: txHash, signatures: signatures, requiredSignatures: threshold }); + SafeSigners.getNSigners({ dataHash: txHash, signatures: signatures, requiredSignatures: threshold }); for (uint256 i = 0; i < signers.length; i++) { lastLive[signers[i]] = block.timestamp; diff --git a/packages/contracts-bedrock/src/Safe/GetSigners.sol b/packages/contracts-bedrock/src/Safe/SafeSigners.sol similarity index 97% rename from packages/contracts-bedrock/src/Safe/GetSigners.sol rename to packages/contracts-bedrock/src/Safe/SafeSigners.sol index 7ef639f675b9..20033b26c644 100644 --- a/packages/contracts-bedrock/src/Safe/GetSigners.sol +++ b/packages/contracts-bedrock/src/Safe/SafeSigners.sol @@ -1,7 +1,7 @@ // SPDX-License-Identifier: MIT -pragma solidity 0.8.15; +pragma solidity ^0.8.0; -library GetSigners { +library SafeSigners { /// @notice Splits signature bytes into `uint8 v, bytes32 r, bytes32 s`. /// Copied directly from /// https://github.com/safe-global/safe-contracts/blob/e870f514ad34cd9654c72174d6d4a839e3c6639f/contracts/common/SignatureDecoder.sol @@ -41,6 +41,7 @@ library GetSigners { /// @param dataHash Hash of the data. /// @param signatures Signature data for identifying signers. /// @param requiredSignatures Amount of required valid signatures. + /// @return _owners List of unique signers. function getNSigners( bytes32 dataHash, bytes memory signatures, diff --git a/packages/contracts-bedrock/test/GetSigners.t.sol b/packages/contracts-bedrock/test/SafeSigners.t.sol similarity index 95% rename from packages/contracts-bedrock/test/GetSigners.t.sol rename to packages/contracts-bedrock/test/SafeSigners.t.sol index 7b1de0952482..a9edbb1ca52e 100644 --- a/packages/contracts-bedrock/test/GetSigners.t.sol +++ b/packages/contracts-bedrock/test/SafeSigners.t.sol @@ -3,12 +3,12 @@ pragma solidity 0.8.15; import { Test } from "forge-std/Test.sol"; import { Safe } from "safe-contracts/Safe.sol"; -import { GetSigners } from "src/Safe/GetSigners.sol"; +import { SafeSigners } from "src/Safe/SafeSigners.sol"; import "test/safe-tools/SafeTestTools.sol"; import { SignatureDecoder } from "safe-contracts/common/SignatureDecoder.sol"; -contract GetSigners_Test is Test, SafeTestTools { +contract SafeSigners_Test is Test, SafeTestTools { bytes4 internal constant EIP1271_MAGIC_VALUE = 0x20c13b0b; enum SigTypes { @@ -98,7 +98,7 @@ contract GetSigners_Test is Test, SafeTestTools { // Recover the signatures using the _getNSigners() method. address[] memory gotSigners = - GetSigners.getNSigners({ dataHash: digest, signatures: signatures, requiredSignatures: numSigs }); + SafeSigners.getNSigners({ dataHash: digest, signatures: signatures, requiredSignatures: numSigs }); // Compare the list of recovered signers to the expected signers. assertEq(gotSigners.length, numSigs); From 6368229c0609e80cfd2f000fa105afb4e797aa1d Mon Sep 17 00:00:00 2001 From: Maurelian Date: Tue, 24 Oct 2023 10:29:06 -0400 Subject: [PATCH 207/279] refactor(ctb): Make function _isAboveMinOwners --- packages/contracts-bedrock/src/Safe/LivenessModule.sol | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessModule.sol b/packages/contracts-bedrock/src/Safe/LivenessModule.sol index ee7eefea7101..214212485349 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessModule.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessModule.sol @@ -79,7 +79,7 @@ contract LivenessModule is ISemver { address[] memory owners = safe.getOwners(); uint256 numOwners = owners.length - 1; uint256 thresholdAfter; - if (hasMinOwners(numOwners)) { + if (_isAboveMinOwners(numOwners)) { // Call the Safe to remove the owner and update the threshold thresholdAfter = get75PercentThreshold(numOwners); address prevOwner = _getPrevOwner(owner, owners); @@ -135,8 +135,8 @@ contract LivenessModule is ISemver { address[] memory owners = safe.getOwners(); uint256 numOwners = owners.length; require( - hasMinOwners(numOwners) || (numOwners == 1 && owners[0] == fallbackOwner), - "LivenessModule: Safe must have the minimum number of owners, or be owned solely by the fallback owner" + _isAboveMinOwners(numOwners) || (numOwners == 1 && owners[0] == fallbackOwner), + "LivenessModule: Safe must have the minimum number of owners or be owned solely by the fallback owner" ); // Check that the threshold is correct. This check is also correct when there is a single @@ -180,7 +180,7 @@ contract LivenessModule is ISemver { /// @notice Check if the number of owners is greater than or equal to the minimum number of owners. /// @param numOwners The number of owners. /// @return A boolean indicating if the number of owners is greater than or equal to the minimum number of owners. - function hasMinOwners(uint256 numOwners) public view returns (bool) { + function _isAboveMinOwners(uint256 numOwners) internal view returns (bool) { return numOwners >= minOwners; } } From c7d5b2b9d40975d811604df25826792fd6a9552b Mon Sep 17 00:00:00 2001 From: Maurelian Date: Tue, 24 Oct 2023 10:29:41 -0400 Subject: [PATCH 208/279] refactor(ctb): Clean up in LivenessGuard --- .../src/Safe/LivenessGuard.sol | 40 +++++++++---------- 1 file changed, 19 insertions(+), 21 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol index 042e47619f7c..9b32dea66f0e 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol @@ -47,20 +47,21 @@ contract LivenessGuard is ISemver, BaseGuard { address[] memory ownersAfter = safe.getOwners(); // Iterate over the current owners, and remove one at a time from the ownersBefore set. - for (uint256 i = 0; i < ownersAfter.length; i++) { + uint256 ownersAfterLength = ownersAfter.length; + for (uint256 i = 0; i < ownersAfterLength; i++) { // If the value was present, remove() returns true. - if (ownersBefore.remove(ownersAfter[i]) == false) { + address ownerAfter = ownersAfter[i]; + if (ownersBefore.remove(ownerAfter) == false) { // This address was not already an owner, add it to the lastSigned mapping - lastLive[ownersAfter[i]] = block.timestamp; + lastLive[ownerAfter] = block.timestamp; } } // Now iterate over the remaining ownersBefore entries. Any remaining addresses are no longer an owner, so we // delete them from the lastSigned mapping. for (uint256 j = 0; j < ownersBefore.length(); j++) { - address owner = ownersBefore.at(j); - delete lastLive[owner]; + address ownerBefore = ownersBefore.at(j); + delete lastLive[ownerBefore]; } - return; } /// @notice Records the most recent time which any owner has signed a transaction. @@ -92,21 +93,18 @@ contract LivenessGuard is ISemver, BaseGuard { // This call will reenter to the Safe which is calling it. This is OK because it is only reading the // nonce, and using the getTransactionHash() method. - bytes32 txHash = Safe(payable(msg.sender)).getTransactionHash( - // Transaction info - to, - value, - data, - operation, - safeTxGas, - // Payment info - baseGas, - gasPrice, - gasToken, - refundReceiver, - // Signature info - Safe(payable(msg.sender)).nonce() - 1 - ); + bytes32 txHash = Safe(payable(msg.sender)).getTransactionHash({ + to: to, + value: value, + data: data, + operation: operation, + safeTxGas: safeTxGas, + baseGas: baseGas, + gasPrice: gasPrice, + gasToken: gasToken, + refundReceiver: refundReceiver, + _nonce: Safe(payable(msg.sender)).nonce() - 1 + }); uint256 threshold = safe.getThreshold(); address[] memory signers = From b51e604e3afa0b0e0baf7236c84adb9dd25d0887 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Tue, 24 Oct 2023 10:34:40 -0400 Subject: [PATCH 209/279] feat(ctb): Improve commenting on LivenessGuard --- .../src/Safe/LivenessGuard.sol | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol index 9b32dea66f0e..edfdf13b34d0 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol @@ -9,6 +9,12 @@ import { Enum } from "safe-contracts/common/Enum.sol"; import { ISemver } from "src/universal/ISemver.sol"; import { EnumerableSet } from "@openzeppelin/contracts/utils/structs/EnumerableSet.sol"; +/// @title LivenessGuard +/// @notice This Guard contract is used to track the liveness of Safe owners. +/// @dev It keeps track of the last time each owner participated in signing a transaction. +/// If an owner does not participate in a transaction for a certain period of time, they are considered inactive. +/// This Guard is intended to be used in conjunction with the LivenessModule contract, but does +/// not depend on it. contract LivenessGuard is ISemver, BaseGuard { using EnumerableSet for EnumerableSet.AddressSet; @@ -24,11 +30,11 @@ contract LivenessGuard is ISemver, BaseGuard { Safe public immutable safe; /// @notice A mapping of the timestamp at which an owner last participated in signing a - /// an executed transaction. + /// an executed transaction, or called showLiveness. mapping(address => uint256) public lastLive; /// @notice An enumerable set of addresses used to store the list of owners before execution, - /// and then to update the lastSigned mapping according to changes in the set observed + /// and then to update the lastLive mapping according to changes in the set observed /// after execution. EnumerableSet.AddressSet private ownersBefore; @@ -40,8 +46,8 @@ contract LivenessGuard is ISemver, BaseGuard { /// @notice We use this post execution hook to compare the set of owners before and after. /// If the set of owners has changed then we: - /// 1. Add new owners to the lastSigned mapping - /// 2. Delete removed owners from the lastSigned mapping + /// 1. Add new owners to the lastLive mapping + /// 2. Delete removed owners from the lastLive mapping function checkAfterExecution(bytes32, bool) external { // Get the current set of owners address[] memory ownersAfter = safe.getOwners(); @@ -52,12 +58,12 @@ contract LivenessGuard is ISemver, BaseGuard { // If the value was present, remove() returns true. address ownerAfter = ownersAfter[i]; if (ownersBefore.remove(ownerAfter) == false) { - // This address was not already an owner, add it to the lastSigned mapping + // This address was not already an owner, add it to the lastLive mapping lastLive[ownerAfter] = block.timestamp; } } // Now iterate over the remaining ownersBefore entries. Any remaining addresses are no longer an owner, so we - // delete them from the lastSigned mapping. + // delete them from the lastLive mapping. for (uint256 j = 0; j < ownersBefore.length(); j++) { address ownerBefore = ownersBefore.at(j); delete lastLive[ownerBefore]; From 2c7ddd0eb7329beccfa7338d99eba39be9c1b94e Mon Sep 17 00:00:00 2001 From: Maurelian Date: Tue, 24 Oct 2023 11:48:11 -0400 Subject: [PATCH 210/279] refactor(ctb): Make immutables screaming snake in Liveness code --- .../src/Safe/LivenessGuard.sol | 20 +++-- .../src/Safe/LivenessModule.sol | 74 +++++++++++++------ .../test/LivenessGuard.t.sol | 7 ++ .../test/LivenessModule.t.sol | 14 +++- 4 files changed, 85 insertions(+), 30 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol index edfdf13b34d0..011884b3ad68 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol @@ -27,7 +27,7 @@ contract LivenessGuard is ISemver, BaseGuard { string public constant version = "1.0.0"; /// @notice The safe account for which this contract will be the guard. - Safe public immutable safe; + Safe internal immutable SAFE; /// @notice A mapping of the timestamp at which an owner last participated in signing a /// an executed transaction, or called showLiveness. @@ -41,7 +41,7 @@ contract LivenessGuard is ISemver, BaseGuard { /// @notice Constructor. /// @param _safe The safe account for which this contract will be the guard. constructor(Safe _safe) { - safe = _safe; + SAFE = _safe; } /// @notice We use this post execution hook to compare the set of owners before and after. @@ -50,7 +50,7 @@ contract LivenessGuard is ISemver, BaseGuard { /// 2. Delete removed owners from the lastLive mapping function checkAfterExecution(bytes32, bool) external { // Get the current set of owners - address[] memory ownersAfter = safe.getOwners(); + address[] memory ownersAfter = SAFE.getOwners(); // Iterate over the current owners, and remove one at a time from the ownersBefore set. uint256 ownersAfterLength = ownersAfter.length; @@ -88,11 +88,11 @@ contract LivenessGuard is ISemver, BaseGuard { ) external { - require(msg.sender == address(safe), "LivenessGuard: only Safe can call this function"); + require(msg.sender == address(SAFE), "LivenessGuard: only Safe can call this function"); // Cache the set of owners prior to execution. // This will be used in the checkAfterExecution method. - address[] memory owners = safe.getOwners(); + address[] memory owners = SAFE.getOwners(); for (uint256 i = 0; i < owners.length; i++) { ownersBefore.add(owners[i]); } @@ -112,7 +112,7 @@ contract LivenessGuard is ISemver, BaseGuard { _nonce: Safe(payable(msg.sender)).nonce() - 1 }); - uint256 threshold = safe.getThreshold(); + uint256 threshold = SAFE.getThreshold(); address[] memory signers = SafeSigners.getNSigners({ dataHash: txHash, signatures: signatures, requiredSignatures: threshold }); @@ -125,11 +125,17 @@ contract LivenessGuard is ISemver, BaseGuard { /// @notice Enables an owner to demonstrate liveness by calling this method directly. /// This is useful for owners who have not recently signed a transaction via the Safe. function showLiveness() external { - require(safe.isOwner(msg.sender), "LivenessGuard: only Safe owners may demontstrate liveness"); + require(SAFE.isOwner(msg.sender), "LivenessGuard: only Safe owners may demontstrate liveness"); lastLive[msg.sender] = block.timestamp; address[] memory signers = new address[](1); signers[0] = msg.sender; emit SignersRecorded(0x0, signers); } + + /// @notice Getter function for the Safe contract instance + /// @return safe_ The Safe contract instance + function safe() public view returns (Safe safe_) { + safe_ = SAFE; + } } diff --git a/packages/contracts-bedrock/src/Safe/LivenessModule.sol b/packages/contracts-bedrock/src/Safe/LivenessModule.sol index 214212485349..c11f70c6dac4 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessModule.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessModule.sol @@ -18,23 +18,23 @@ import { console2 as console } from "forge-std/console2.sol"; /// safe will be transferred to the fallback owner. contract LivenessModule is ISemver { /// @notice The Safe contract instance - Safe public immutable safe; + Safe internal immutable SAFE; /// @notice The LivenessGuard contract instance /// This can be updated by replacing with a new module and switching out the guard. - LivenessGuard public immutable livenessGuard; + LivenessGuard internal immutable LIVENESS_GUARD; /// @notice The interval, in seconds, during which an owner must have demonstrated liveness /// This can be updated by replacing with a new module. - uint256 public immutable livenessInterval; + uint256 internal immutable LIVENESS_INTERVAL; /// @notice The minimum number of owners before ownership of the safe is transferred to the fallback owner. /// This can be updated by replacing with a new module. - uint256 public immutable minOwners; + uint256 internal immutable MIN_OWNERS; /// @notice The fallback owner of the Safe /// This can be updated by replacing with a new module. - address public immutable fallbackOwner; + address internal immutable FALLBACK_OWNER; /// @notice The storage slot used in the safe to store the guard address /// keccak256("guard_manager.guard.address") @@ -55,11 +55,11 @@ contract LivenessModule is ISemver { uint256 _minOwners, address _fallbackOwner ) { - safe = _safe; - livenessGuard = _livenessGuard; - livenessInterval = _livenessInterval; - minOwners = _minOwners; - fallbackOwner = _fallbackOwner; + SAFE = _safe; + LIVENESS_GUARD = _livenessGuard; + LIVENESS_INTERVAL = _livenessInterval; + MIN_OWNERS = _minOwners; + FALLBACK_OWNER = _fallbackOwner; } /// @notice This function can be called by anyone to remove an owner that has not signed a transaction @@ -71,12 +71,12 @@ contract LivenessModule is ISemver { // Check that the owner to remove has not signed a transaction in the last 30 days require( - livenessGuard.lastLive(owner) < block.timestamp - livenessInterval, + LIVENESS_GUARD.lastLive(owner) < block.timestamp - LIVENESS_INTERVAL, "LivenessModule: owner has signed recently" ); // Calculate the new threshold - address[] memory owners = safe.getOwners(); + address[] memory owners = SAFE.getOwners(); uint256 numOwners = owners.length - 1; uint256 thresholdAfter; if (_isAboveMinOwners(numOwners)) { @@ -109,11 +109,11 @@ contract LivenessModule is ISemver { /// @notice Sets the fallback owner as the sole owner of the Safe with a threshold of 1 function _giveToFallbackOwner() internal { - safe.execTransactionFromModule({ - to: address(safe), + SAFE.execTransactionFromModule({ + to: address(SAFE), value: 0, operation: Enum.Operation.Call, - data: abi.encodeCall(OwnerManager.addOwnerWithThreshold, (fallbackOwner, 1)) + data: abi.encodeCall(OwnerManager.addOwnerWithThreshold, (FALLBACK_OWNER, 1)) }); } @@ -122,8 +122,8 @@ contract LivenessModule is ISemver { /// @param _owner Owner address to be removed. /// @param _threshold New threshold. function _removeOwner(address _prevOwner, address _owner, uint256 _threshold) internal { - safe.execTransactionFromModule({ - to: address(safe), + SAFE.execTransactionFromModule({ + to: address(SAFE), value: 0, operation: Enum.Operation.Call, data: abi.encodeCall(OwnerManager.removeOwner, (_prevOwner, _owner, _threshold)) @@ -132,16 +132,16 @@ contract LivenessModule is ISemver { /// @notice A FREI-PI invariant check enforcing requirements on number of owners and threshold. function _verifyFinalState() internal view { - address[] memory owners = safe.getOwners(); + address[] memory owners = SAFE.getOwners(); uint256 numOwners = owners.length; require( - _isAboveMinOwners(numOwners) || (numOwners == 1 && owners[0] == fallbackOwner), + _isAboveMinOwners(numOwners) || (numOwners == 1 && owners[0] == FALLBACK_OWNER), "LivenessModule: Safe must have the minimum number of owners or be owned solely by the fallback owner" ); // Check that the threshold is correct. This check is also correct when there is a single // owner, because get75PercentThreshold(1) returns 1. - uint256 threshold = safe.getThreshold(); + uint256 threshold = SAFE.getThreshold(); require( threshold == get75PercentThreshold(numOwners), "LivenessModule: threshold must be 75% of the number of owners" @@ -154,7 +154,7 @@ contract LivenessModule is ISemver { /// @notice Reverts if the guard address does not match the expected value. function _verifyGuard() internal view { require( - address(livenessGuard) == address(uint160(uint256(bytes32(safe.getStorageAt(GUARD_STORAGE_SLOT, 1))))), + address(LIVENESS_GUARD) == address(uint160(uint256(bytes32(SAFE.getStorageAt(GUARD_STORAGE_SLOT, 1))))), "LivenessModule: guard has been changed" ); } @@ -181,6 +181,36 @@ contract LivenessModule is ISemver { /// @param numOwners The number of owners. /// @return A boolean indicating if the number of owners is greater than or equal to the minimum number of owners. function _isAboveMinOwners(uint256 numOwners) internal view returns (bool) { - return numOwners >= minOwners; + return numOwners >= MIN_OWNERS; + } + + /// @notice Getter function for the Safe contract instance + /// @return safe_ The Safe contract instance + function safe() public view returns (Safe safe_) { + safe_ = SAFE; + } + + /// @notice Getter function for the LivenessGuard contract instance + /// @return livenessGuard_ The LivenessGuard contract instance + function livenessGuard() public view returns (LivenessGuard livenessGuard_) { + livenessGuard_ = LIVENESS_GUARD; + } + + /// @notice Getter function for the liveness interval + /// @return livenessInterval_ The liveness interval, in seconds + function livenessInterval() public view returns (uint256 livenessInterval_) { + livenessInterval_ = LIVENESS_INTERVAL; + } + + /// @notice Getter function for the minimum number of owners + /// @return minOwners_ The minimum number of owners + function minOwners() public view returns (uint256 minOwners_) { + minOwners_ = MIN_OWNERS; + } + + /// @notice Getter function for the fallback owner + /// @return fallbackOwner_ The fallback owner of the Safe + function fallbackOwner() public view returns (address fallbackOwner_) { + fallbackOwner_ = FALLBACK_OWNER; } } diff --git a/packages/contracts-bedrock/test/LivenessGuard.t.sol b/packages/contracts-bedrock/test/LivenessGuard.t.sol index 8046fb01b0ad..132a36fa5c11 100644 --- a/packages/contracts-bedrock/test/LivenessGuard.t.sol +++ b/packages/contracts-bedrock/test/LivenessGuard.t.sol @@ -33,6 +33,13 @@ contract LivnessGuard_TestInit is Test, SafeTestTools { } } +contract LivenessGuard_Getters_Test is LivnessGuard_TestInit { + function test_getters_works() external { + assertEq(address(livenessGuard.safe()), address(safeInstance.safe)); + assertEq(livenessGuard.lastLive(address(0)), 0); + } +} + contract LivnessGuard_CheckTx_Test is LivnessGuard_TestInit { using SafeTestLib for SafeInstance; diff --git a/packages/contracts-bedrock/test/LivenessModule.t.sol b/packages/contracts-bedrock/test/LivenessModule.t.sol index 0e7a262af036..39a51f6370c8 100644 --- a/packages/contracts-bedrock/test/LivenessModule.t.sol +++ b/packages/contracts-bedrock/test/LivenessModule.t.sol @@ -19,24 +19,36 @@ contract LivnessModule_TestInit is Test, SafeTestTools { LivenessModule livenessModule; LivenessGuard livenessGuard; SafeInstance safeInstance; + address fallbackOwner; function setUp() public { // Create a Safe with 10 owners (, uint256[] memory keys) = makeAddrsAndKeys(10); safeInstance = _setupSafe(keys, 8); livenessGuard = new LivenessGuard(safeInstance.safe); + fallbackOwner = makeAddr("fallbackOwner"); livenessModule = new LivenessModule({ _safe: safeInstance.safe, _livenessGuard: livenessGuard, _livenessInterval: 30 days, _minOwners: 6, - _fallbackOwner: makeAddr("fallbackOwner") + _fallbackOwner: fallbackOwner }); safeInstance.enableModule(address(livenessModule)); safeInstance.setGuard(address(livenessGuard)); } } +contract LivenessModule_Getters_Test is LivnessModule_TestInit { + function test_getters_works() external { + assertEq(address(livenessModule.safe()), address(safeInstance.safe)); + assertEq(address(livenessModule.livenessGuard()), address(livenessGuard)); + assertEq(livenessModule.livenessInterval(), 30 days); + assertEq(livenessModule.minOwners(), 6); + assertEq(livenessModule.fallbackOwner(), fallbackOwner); + } +} + contract LivenessModule_Get75PercentThreshold_Test is LivnessModule_TestInit { /// @dev check the return values of the get75PercentThreshold function against manually /// calculated values. From 26c1ac545abe97f6618f10473333c3e046a91d0d Mon Sep 17 00:00:00 2001 From: Maurelian Date: Tue, 24 Oct 2023 12:56:10 -0400 Subject: [PATCH 211/279] feat(ctb): Document requirements for non-reverting in the Guard Also reorders the functions in the order they are called. --- .../src/Safe/LivenessGuard.sol | 61 ++++++++++--------- 1 file changed, 33 insertions(+), 28 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol index 011884b3ad68..f5d8d71adfa3 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol @@ -15,6 +15,9 @@ import { EnumerableSet } from "@openzeppelin/contracts/utils/structs/EnumerableS /// If an owner does not participate in a transaction for a certain period of time, they are considered inactive. /// This Guard is intended to be used in conjunction with the LivenessModule contract, but does /// not depend on it. +/// Note: Both `checkTransaction` and `checkAfterExecution` are called once each by the Safe contract +/// before and after the execution of a transaction. It is critical that neither function revert, +/// otherwise the Safe contract will be unable to execute a transaction. contract LivenessGuard is ISemver, BaseGuard { using EnumerableSet for EnumerableSet.AddressSet; @@ -44,35 +47,8 @@ contract LivenessGuard is ISemver, BaseGuard { SAFE = _safe; } - /// @notice We use this post execution hook to compare the set of owners before and after. - /// If the set of owners has changed then we: - /// 1. Add new owners to the lastLive mapping - /// 2. Delete removed owners from the lastLive mapping - function checkAfterExecution(bytes32, bool) external { - // Get the current set of owners - address[] memory ownersAfter = SAFE.getOwners(); - - // Iterate over the current owners, and remove one at a time from the ownersBefore set. - uint256 ownersAfterLength = ownersAfter.length; - for (uint256 i = 0; i < ownersAfterLength; i++) { - // If the value was present, remove() returns true. - address ownerAfter = ownersAfter[i]; - if (ownersBefore.remove(ownerAfter) == false) { - // This address was not already an owner, add it to the lastLive mapping - lastLive[ownerAfter] = block.timestamp; - } - } - // Now iterate over the remaining ownersBefore entries. Any remaining addresses are no longer an owner, so we - // delete them from the lastLive mapping. - for (uint256 j = 0; j < ownersBefore.length(); j++) { - address ownerBefore = ownersBefore.at(j); - delete lastLive[ownerBefore]; - } - } - /// @notice Records the most recent time which any owner has signed a transaction. - /// @dev This method is called by the Safe contract, it is critical that it does not revert, otherwise - /// the Safe contract will be unable to execute transactions. + /// @dev Called by the Safe contract before execution of a transaction. function checkTransaction( address to, uint256 value, @@ -122,6 +98,35 @@ contract LivenessGuard is ISemver, BaseGuard { emit SignersRecorded(txHash, signers); } + /// @notice Update the lastLive mapping according to the set of owners before and after execution. + /// @dev Called by the Safe contract after the execution of a transaction. + /// We use this post execution hook to compare the set of owners before and after. + /// If the set of owners has changed then we: + /// 1. Add new owners to the lastLive mapping + /// 2. Delete removed owners from the lastLive mapping + function checkAfterExecution(bytes32, bool) external { + + // Get the current set of owners + address[] memory ownersAfter = SAFE.getOwners(); + + // Iterate over the current owners, and remove one at a time from the ownersBefore set. + uint256 ownersAfterLength = ownersAfter.length; + for (uint256 i = 0; i < ownersAfterLength; i++) { + // If the value was present, remove() returns true. + address ownerAfter = ownersAfter[i]; + if (ownersBefore.remove(ownerAfter) == false) { + // This address was not already an owner, add it to the lastLive mapping + lastLive[ownerAfter] = block.timestamp; + } + } + // Now iterate over the remaining ownersBefore entries. Any remaining addresses are no longer an owner, so we + // delete them from the lastLive mapping. + for (uint256 j = 0; j < ownersBefore.length(); j++) { + address ownerBefore = ownersBefore.at(j); + delete lastLive[ownerBefore]; + } + } + /// @notice Enables an owner to demonstrate liveness by calling this method directly. /// This is useful for owners who have not recently signed a transaction via the Safe. function showLiveness() external { From 8215444c6856a5d4e8aff06cd5dd7126d198a13f Mon Sep 17 00:00:00 2001 From: Maurelian Date: Tue, 24 Oct 2023 12:56:31 -0400 Subject: [PATCH 212/279] feat(ctb): Remove redundant _verifyGuard call --- packages/contracts-bedrock/src/Safe/LivenessModule.sol | 3 --- 1 file changed, 3 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessModule.sol b/packages/contracts-bedrock/src/Safe/LivenessModule.sol index c11f70c6dac4..f97a846104a5 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessModule.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessModule.sol @@ -66,9 +66,6 @@ contract LivenessModule is ISemver { /// during the liveness interval. If the number of owners drops below the minimum, then the /// ownership of the Safe is transferred to the fallback owner. function removeOwner(address owner) external { - // Check that the guard has not been changed - _verifyGuard(); - // Check that the owner to remove has not signed a transaction in the last 30 days require( LIVENESS_GUARD.lastLive(owner) < block.timestamp - LIVENESS_INTERVAL, From 410310eaffb8e3cadb4a52e9378f165783882b5e Mon Sep 17 00:00:00 2001 From: Maurelian Date: Tue, 24 Oct 2023 14:17:07 -0400 Subject: [PATCH 213/279] fix(ctb): Add auth to guard checkAfterExecution Also added regression tests. --- .../src/Safe/LivenessGuard.sol | 4 ++- .../test/LivenessGuard.t.sol | 28 +++++++++++++++++++ 2 files changed, 31 insertions(+), 1 deletion(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol index f5d8d71adfa3..1e2ffe761ca6 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol @@ -60,10 +60,11 @@ contract LivenessGuard is ISemver, BaseGuard { address gasToken, address payable refundReceiver, bytes memory signatures, - address + address msgSender ) external { + msgSender; // silence unused variable warning require(msg.sender == address(SAFE), "LivenessGuard: only Safe can call this function"); // Cache the set of owners prior to execution. @@ -105,6 +106,7 @@ contract LivenessGuard is ISemver, BaseGuard { /// 1. Add new owners to the lastLive mapping /// 2. Delete removed owners from the lastLive mapping function checkAfterExecution(bytes32, bool) external { + require(msg.sender == address(SAFE), "LivenessGuard: only Safe can call this function"); // Get the current set of owners address[] memory ownersAfter = SAFE.getOwners(); diff --git a/packages/contracts-bedrock/test/LivenessGuard.t.sol b/packages/contracts-bedrock/test/LivenessGuard.t.sol index 132a36fa5c11..77ed8d58148a 100644 --- a/packages/contracts-bedrock/test/LivenessGuard.t.sol +++ b/packages/contracts-bedrock/test/LivenessGuard.t.sol @@ -40,6 +40,25 @@ contract LivenessGuard_Getters_Test is LivnessGuard_TestInit { } } +contract LivnessGuard_CheckTx_TestFails is LivnessGuard_TestInit { + function test_checkTransaction_callerIsNotSafe_revert() external { + vm.expectRevert("LivenessGuard: only Safe can call this function"); + livenessGuard.checkTransaction({ + to: address(0), + value: 0, + data: hex"00", + operation: Enum.Operation.Call, + safeTxGas: 0, + baseGas: 0, + gasPrice: 0, + gasToken: address(0), + refundReceiver: payable(address(0)), + signatures: hex"00", + msgSender: address(0) + }); + } +} + contract LivnessGuard_CheckTx_Test is LivnessGuard_TestInit { using SafeTestLib for SafeInstance; @@ -63,6 +82,15 @@ contract LivnessGuard_CheckTx_Test is LivnessGuard_TestInit { } } +contract LivnessGuard_CheckAfterExecution_TestFails is LivnessGuard_TestInit { + function test_checkAfterExecution_callerIsNotSafe_revert() external { + vm.expectRevert("LivenessGuard: only Safe can call this function"); + livenessGuard.checkAfterExecution(bytes32(0), false); + } +} + +contract LivnessGuard_CheckAfterExecution_Test is LivnessGuard_TestInit { } + contract LivenessGuard_ShowLiveness_Test is LivnessGuard_TestInit { function test_showLiveness_succeeds() external { // Cache the caller From 2d7e3ba8e4f8e00b8a5288ece940fefd03307c6a Mon Sep 17 00:00:00 2001 From: Maurelian Date: Tue, 24 Oct 2023 14:18:28 -0400 Subject: [PATCH 214/279] feat(ctb): Fix typo Livness in test files --- .../contracts-bedrock/test/LivenessGuard.t.sol | 14 +++++++------- .../contracts-bedrock/test/LivenessModule.t.sol | 8 ++++---- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/packages/contracts-bedrock/test/LivenessGuard.t.sol b/packages/contracts-bedrock/test/LivenessGuard.t.sol index 77ed8d58148a..35fb2c1e0d68 100644 --- a/packages/contracts-bedrock/test/LivenessGuard.t.sol +++ b/packages/contracts-bedrock/test/LivenessGuard.t.sol @@ -18,7 +18,7 @@ import { LivenessGuard } from "src/Safe/LivenessGuard.sol"; // - Signers may call directly to prove liveness (must be an owner). // - Unexpected length of signature data -contract LivnessGuard_TestInit is Test, SafeTestTools { +contract LivenessGuard_TestInit is Test, SafeTestTools { using SafeTestLib for SafeInstance; event SignersRecorded(bytes32 indexed txHash, address[] signers); @@ -33,14 +33,14 @@ contract LivnessGuard_TestInit is Test, SafeTestTools { } } -contract LivenessGuard_Getters_Test is LivnessGuard_TestInit { +contract LivenessGuard_Getters_Test is LivenessGuard_TestInit { function test_getters_works() external { assertEq(address(livenessGuard.safe()), address(safeInstance.safe)); assertEq(livenessGuard.lastLive(address(0)), 0); } } -contract LivnessGuard_CheckTx_TestFails is LivnessGuard_TestInit { +contract LivenessGuard_CheckTx_TestFails is LivenessGuard_TestInit { function test_checkTransaction_callerIsNotSafe_revert() external { vm.expectRevert("LivenessGuard: only Safe can call this function"); livenessGuard.checkTransaction({ @@ -59,7 +59,7 @@ contract LivnessGuard_CheckTx_TestFails is LivnessGuard_TestInit { } } -contract LivnessGuard_CheckTx_Test is LivnessGuard_TestInit { +contract LivenessGuard_CheckTx_Test is LivenessGuard_TestInit { using SafeTestLib for SafeInstance; function test_checkTransaction_succeeds() external { @@ -82,16 +82,16 @@ contract LivnessGuard_CheckTx_Test is LivnessGuard_TestInit { } } -contract LivnessGuard_CheckAfterExecution_TestFails is LivnessGuard_TestInit { +contract LivenessGuard_CheckAfterExecution_TestFails is LivenessGuard_TestInit { function test_checkAfterExecution_callerIsNotSafe_revert() external { vm.expectRevert("LivenessGuard: only Safe can call this function"); livenessGuard.checkAfterExecution(bytes32(0), false); } } -contract LivnessGuard_CheckAfterExecution_Test is LivnessGuard_TestInit { } +contract LivenessGuard_CheckAfterExecution_Test is LivenessGuard_TestInit { } -contract LivenessGuard_ShowLiveness_Test is LivnessGuard_TestInit { +contract LivenessGuard_ShowLiveness_Test is LivenessGuard_TestInit { function test_showLiveness_succeeds() external { // Cache the caller address caller = safeInstance.owners[0]; diff --git a/packages/contracts-bedrock/test/LivenessModule.t.sol b/packages/contracts-bedrock/test/LivenessModule.t.sol index 39a51f6370c8..04f69980980a 100644 --- a/packages/contracts-bedrock/test/LivenessModule.t.sol +++ b/packages/contracts-bedrock/test/LivenessModule.t.sol @@ -11,7 +11,7 @@ import "test/safe-tools/SafeTestTools.sol"; import { LivenessModule } from "src/Safe/LivenessModule.sol"; import { LivenessGuard } from "src/Safe/LivenessGuard.sol"; -contract LivnessModule_TestInit is Test, SafeTestTools { +contract LivenessModule_TestInit is Test, SafeTestTools { using SafeTestLib for SafeInstance; event SignersRecorded(bytes32 indexed txHash, address[] signers); @@ -39,7 +39,7 @@ contract LivnessModule_TestInit is Test, SafeTestTools { } } -contract LivenessModule_Getters_Test is LivnessModule_TestInit { +contract LivenessModule_Getters_Test is LivenessModule_TestInit { function test_getters_works() external { assertEq(address(livenessModule.safe()), address(safeInstance.safe)); assertEq(address(livenessModule.livenessGuard()), address(livenessGuard)); @@ -49,7 +49,7 @@ contract LivenessModule_Getters_Test is LivnessModule_TestInit { } } -contract LivenessModule_Get75PercentThreshold_Test is LivnessModule_TestInit { +contract LivenessModule_Get75PercentThreshold_Test is LivenessModule_TestInit { /// @dev check the return values of the get75PercentThreshold function against manually /// calculated values. function test_get75PercentThreshold_Works() external { @@ -76,7 +76,7 @@ contract LivenessModule_Get75PercentThreshold_Test is LivnessModule_TestInit { } } -contract LivenessModule_RemoveOwner_Test is LivnessModule_TestInit { +contract LivenessModule_RemoveOwner_Test is LivenessModule_TestInit { function test_removeOwner_oneOwner_succeeds() external { uint256 ownersBefore = safeInstance.owners.length; vm.warp(block.timestamp + 30 days); From a5a723d382e7fbc488c89c75d2308e89cba7b0d8 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Tue, 24 Oct 2023 14:42:17 -0400 Subject: [PATCH 215/279] feat(ctb): numOwnersAfter is a better var name --- packages/contracts-bedrock/src/Safe/LivenessModule.sol | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessModule.sol b/packages/contracts-bedrock/src/Safe/LivenessModule.sol index f97a846104a5..d82dc8b52133 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessModule.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessModule.sol @@ -74,11 +74,11 @@ contract LivenessModule is ISemver { // Calculate the new threshold address[] memory owners = SAFE.getOwners(); - uint256 numOwners = owners.length - 1; + uint256 numOwnersAfter = owners.length - 1; uint256 thresholdAfter; - if (_isAboveMinOwners(numOwners)) { + if (_isAboveMinOwners(numOwnersAfter)) { // Call the Safe to remove the owner and update the threshold - thresholdAfter = get75PercentThreshold(numOwners); + thresholdAfter = get75PercentThreshold(numOwnersAfter); address prevOwner = _getPrevOwner(owner, owners); _removeOwner({ _prevOwner: prevOwner, _owner: owner, _threshold: thresholdAfter }); } else { From 927bcaaa1e04e4e05bbe30b85c3f001eb716a1d1 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Tue, 24 Oct 2023 14:50:26 -0400 Subject: [PATCH 216/279] fix(ctb): Fix off-by-one in removeOwner There was a bug in the remove all owners flow that would leave two owners on the safe. Fixing that bug required using `swapOwners` in order to remove and replace the last signer, since you cannot remove the only owner and then add a new one (even when calling from a module). --- .../src/Safe/LivenessModule.sol | 31 +++++++++++-------- .../test/LivenessModule.t.sol | 16 +++++++--- 2 files changed, 30 insertions(+), 17 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessModule.sol b/packages/contracts-bedrock/src/Safe/LivenessModule.sol index d82dc8b52133..134c80190bc6 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessModule.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessModule.sol @@ -88,29 +88,32 @@ contract LivenessModule is ISemver { // Remove owners one at a time starting from the last owner. // Since we're removing them in order from last to first, the ordering will remain constant, // and we shouldn't need to query the list of owners again. - for (uint256 i = owners.length - 1; i >= 0; i--) { + address prevOwner; + for (uint256 i = owners.length - 1; i > 0; i--) { address currentOwner = owners[i]; - address prevOwner = _getPrevOwner(currentOwner, owners); - if (currentOwner != address(this)) { - // Call the Safe to remove the owner - _removeOwner({ _prevOwner: prevOwner, _owner: currentOwner, _threshold: 1 }); - } + prevOwner = _getPrevOwner(currentOwner, owners); + + // Call the Safe to remove the owner + _removeOwner({ _prevOwner: prevOwner, _owner: currentOwner, _threshold: 1 }); } + prevOwner = _getPrevOwner(owners[0], owners); // Add the fallback owner as the sole owner of the Safe - _giveToFallbackOwner(); + _swapToFallbackOwner({ _prevOwner: prevOwner, _oldOwner: owners[0] }); } _verifyFinalState(); } /// @notice Sets the fallback owner as the sole owner of the Safe with a threshold of 1 - function _giveToFallbackOwner() internal { + /// @param _prevOwner Owner that pointed to the owner to be replaced in the linked list + /// @param _oldOwner Owner address to be replaced. + function _swapToFallbackOwner(address _prevOwner, address _oldOwner) internal { SAFE.execTransactionFromModule({ to: address(SAFE), value: 0, operation: Enum.Operation.Call, - data: abi.encodeCall(OwnerManager.addOwnerWithThreshold, (FALLBACK_OWNER, 1)) + data: abi.encodeCall(OwnerManager.swapOwner, (_prevOwner, _oldOwner, FALLBACK_OWNER)) }); } @@ -157,14 +160,16 @@ contract LivenessModule is ISemver { } /// @notice Get the previous owner in the linked list of owners - function _getPrevOwner(address owner, address[] memory owners) internal pure returns (address prevOwner_) { - for (uint256 i = 0; i < owners.length; i++) { - if (owners[i] != owner) continue; + /// @param _owner The owner whose previous owner we want to find + /// @param _owners The list of owners + function _getPrevOwner(address _owner, address[] memory _owners) internal pure returns (address prevOwner_) { + for (uint256 i = 0; i < _owners.length; i++) { + if (_owners[i] != _owner) continue; if (i == 0) { prevOwner_ = SENTINEL_OWNERS; break; } - prevOwner_ = owners[i - 1]; + prevOwner_ = _owners[i - 1]; } } diff --git a/packages/contracts-bedrock/test/LivenessModule.t.sol b/packages/contracts-bedrock/test/LivenessModule.t.sol index 04f69980980a..6b669442e810 100644 --- a/packages/contracts-bedrock/test/LivenessModule.t.sol +++ b/packages/contracts-bedrock/test/LivenessModule.t.sol @@ -86,9 +86,17 @@ contract LivenessModule_RemoveOwner_Test is LivenessModule_TestInit { function test_removeOwner_allOwners_succeeds() external { vm.warp(block.timestamp + 30 days); - // The safe is initialized with 10 owners, so we need to remove 3 to get below the minOwners threshold - livenessModule.removeOwner(safeInstance.owners[0]); - livenessModule.removeOwner(safeInstance.owners[1]); - livenessModule.removeOwner(safeInstance.owners[2]); + uint256 numOwners = safeInstance.owners.length; + uint256 minOwners = livenessModule.minOwners(); + + // Remove enough owners to trigger the transfer to the fallbackOwner + uint256 numToRemove = numOwners - minOwners + 1; + + for (uint256 i = 0; i < numToRemove; i++) { + livenessModule.removeOwner(safeInstance.owners[i]); + } + assertEq(safeInstance.safe.getOwners().length, 1); + assertEq(safeInstance.safe.getOwners()[0], fallbackOwner); + assertEq(safeInstance.safe.getThreshold(), 1); } } From 7ce3e4dcea9f36d952fbdb8b07ea85a3751c2ac1 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Tue, 24 Oct 2023 15:23:38 -0400 Subject: [PATCH 217/279] refactor(ctb): Only emit a single address from SignerRecorded --- .../contracts-bedrock/src/Safe/LivenessGuard.sol | 12 +++++------- .../contracts-bedrock/test/LivenessGuard.t.sol | 15 +++++++-------- 2 files changed, 12 insertions(+), 15 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol index 1e2ffe761ca6..62b492732246 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol @@ -21,9 +21,9 @@ import { EnumerableSet } from "@openzeppelin/contracts/utils/structs/EnumerableS contract LivenessGuard is ISemver, BaseGuard { using EnumerableSet for EnumerableSet.AddressSet; - /// @notice Emitted when a new set of signers is recorded. - /// @param signers An arrary of signer addresses. - event SignersRecorded(bytes32 indexed txHash, address[] signers); + /// @notice Emitted when a signer is recorded. + /// @param signer The signer's address. + event SignerRecorded(bytes32 indexed txHash, address signer); /// @notice Semantic version. /// @custom:semver 1.0.0 @@ -95,8 +95,8 @@ contract LivenessGuard is ISemver, BaseGuard { for (uint256 i = 0; i < signers.length; i++) { lastLive[signers[i]] = block.timestamp; + emit SignerRecorded(txHash, signers[i]); } - emit SignersRecorded(txHash, signers); } /// @notice Update the lastLive mapping according to the set of owners before and after execution. @@ -134,10 +134,8 @@ contract LivenessGuard is ISemver, BaseGuard { function showLiveness() external { require(SAFE.isOwner(msg.sender), "LivenessGuard: only Safe owners may demontstrate liveness"); lastLive[msg.sender] = block.timestamp; - address[] memory signers = new address[](1); - signers[0] = msg.sender; - emit SignersRecorded(0x0, signers); + emit SignerRecorded(0x0, msg.sender); } /// @notice Getter function for the Safe contract instance diff --git a/packages/contracts-bedrock/test/LivenessGuard.t.sol b/packages/contracts-bedrock/test/LivenessGuard.t.sol index 35fb2c1e0d68..c93872be3a8b 100644 --- a/packages/contracts-bedrock/test/LivenessGuard.t.sol +++ b/packages/contracts-bedrock/test/LivenessGuard.t.sol @@ -21,7 +21,7 @@ import { LivenessGuard } from "src/Safe/LivenessGuard.sol"; contract LivenessGuard_TestInit is Test, SafeTestTools { using SafeTestLib for SafeInstance; - event SignersRecorded(bytes32 indexed txHash, address[] signers); + event SignerRecorded(bytes32 indexed txHash, address signer); LivenessGuard livenessGuard; SafeInstance safeInstance; @@ -69,9 +69,11 @@ contract LivenessGuard_CheckTx_Test is LivenessGuard_TestInit { signers[0] = safeInstance.owners[0]; signers[1] = safeInstance.owners[1]; - // Don't check topic1 so that we can avoid the ugly txHash calculation. - vm.expectEmit(false, true, true, true, address(livenessGuard)); - emit SignersRecorded(0x0, signers); + for (uint256 i; i < signers.length; i++) { + // Don't check topic1 so that we can avoid the ugly txHash calculation. + vm.expectEmit(false, true, true, true, address(livenessGuard)); + emit SignerRecorded(0x0, signers[i]); + } vm.expectCall(address(safeInstance.safe), abi.encodeWithSignature("nonce()")); vm.expectCall(address(safeInstance.safe), abi.encodeCall(OwnerManager.getThreshold, ())); safeInstance.execTransaction({ to: address(1111), value: 0, data: hex"abba" }); @@ -96,11 +98,8 @@ contract LivenessGuard_ShowLiveness_Test is LivenessGuard_TestInit { // Cache the caller address caller = safeInstance.owners[0]; - // Construct a signers array with just the caller to identify the expected event. - address[] memory signers = new address[](1); - signers[0] = caller; vm.expectEmit(address(livenessGuard)); - emit SignersRecorded(0x0, signers); + emit SignerRecorded(0x0, caller); vm.prank(caller); livenessGuard.showLiveness(); From fbc6adede9c133eaec4fe0d7af24c31ef768ad10 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Tue, 24 Oct 2023 15:47:28 -0400 Subject: [PATCH 218/279] docs(ctb): Some commenting fixes --- packages/contracts-bedrock/src/Safe/LivenessModule.sol | 3 +-- packages/contracts-bedrock/src/Safe/SafeSigners.sol | 8 ++++---- .../safe-tools/CompatibilityFallbackHandler_1_3_0.sol | 3 ++- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessModule.sol b/packages/contracts-bedrock/src/Safe/LivenessModule.sol index 134c80190bc6..a631232b080b 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessModule.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessModule.sol @@ -75,10 +75,9 @@ contract LivenessModule is ISemver { // Calculate the new threshold address[] memory owners = SAFE.getOwners(); uint256 numOwnersAfter = owners.length - 1; - uint256 thresholdAfter; if (_isAboveMinOwners(numOwnersAfter)) { // Call the Safe to remove the owner and update the threshold - thresholdAfter = get75PercentThreshold(numOwnersAfter); + uint256 thresholdAfter = get75PercentThreshold(numOwnersAfter); address prevOwner = _getPrevOwner(owner, owners); _removeOwner({ _prevOwner: prevOwner, _owner: owner, _threshold: thresholdAfter }); } else { diff --git a/packages/contracts-bedrock/src/Safe/SafeSigners.sol b/packages/contracts-bedrock/src/Safe/SafeSigners.sol index 20033b26c644..f0ad74fc67ed 100644 --- a/packages/contracts-bedrock/src/Safe/SafeSigners.sol +++ b/packages/contracts-bedrock/src/Safe/SafeSigners.sol @@ -53,10 +53,10 @@ library SafeSigners { { _owners = new address[](requiredSignatures); - /// The following code is extracted from the Safe.checkNSignatures() method. It removes the signature - /// validation code, and keeps only the parsing code necessary to extract the owner addresses from the - /// signatures. We do not double check if the owner derived from a signature is valid. As this is handled - /// in the final require statement of Safe.checkNSignatures(). + // The following code is extracted from the Safe.checkNSignatures() method. It removes the signature + // validation code, and keeps only the parsing code necessary to extract the owner addresses from the + // signatures. We do not double check if the owner derived from a signature is valid. As this is handled + // in the final require statement of Safe.checkNSignatures(). address currentOwner; uint8 v; bytes32 r; diff --git a/packages/contracts-bedrock/test/safe-tools/CompatibilityFallbackHandler_1_3_0.sol b/packages/contracts-bedrock/test/safe-tools/CompatibilityFallbackHandler_1_3_0.sol index 046a0eda43f0..30c002f119f1 100644 --- a/packages/contracts-bedrock/test/safe-tools/CompatibilityFallbackHandler_1_3_0.sol +++ b/packages/contracts-bedrock/test/safe-tools/CompatibilityFallbackHandler_1_3_0.sol @@ -8,7 +8,8 @@ import "safe-contracts/interfaces/IERC165.sol"; import "safe-contracts/interfaces/ISignatureValidator.sol"; import { Safe as GnosisSafe } from "safe-contracts/Safe.sol"; -/// @dev NOTE: removed VERSION and NAME due to inheritance conflicts +/// @author: Colin Nielsen +/// https://github.com/colinnielsen/safe-tools/blob/ce6c654a76d91b619ab7778c77d1a76b3ced6666/src/CompatibilityFallbackHandler_1_3_0.sol contract DefaultCallbackHandler is ERC1155TokenReceiver, ERC777TokensRecipient, ERC721TokenReceiver, IERC165 { function onERC1155Received( address, From 7d974663143e150e7c54ccf0f46664f414e21226 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Tue, 24 Oct 2023 16:13:26 -0400 Subject: [PATCH 219/279] feat(ctb): Naming improvements Standardizing on the term owner rather than signer except in and around calls to _getNSigners. --- packages/contracts-bedrock/src/Safe/LivenessGuard.sol | 10 +++++----- packages/contracts-bedrock/src/Safe/LivenessModule.sol | 4 ++-- packages/contracts-bedrock/test/LivenessGuard.t.sol | 6 +++--- 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol index 62b492732246..2e257697b55c 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol @@ -21,9 +21,9 @@ import { EnumerableSet } from "@openzeppelin/contracts/utils/structs/EnumerableS contract LivenessGuard is ISemver, BaseGuard { using EnumerableSet for EnumerableSet.AddressSet; - /// @notice Emitted when a signer is recorded. - /// @param signer The signer's address. - event SignerRecorded(bytes32 indexed txHash, address signer); + /// @notice Emitted when an owner is recorded. + /// @param owner The owner's address. + event OwnerRecorded(bytes32 indexed txHash, address owner); /// @notice Semantic version. /// @custom:semver 1.0.0 @@ -95,7 +95,7 @@ contract LivenessGuard is ISemver, BaseGuard { for (uint256 i = 0; i < signers.length; i++) { lastLive[signers[i]] = block.timestamp; - emit SignerRecorded(txHash, signers[i]); + emit OwnerRecorded(txHash, signers[i]); } } @@ -135,7 +135,7 @@ contract LivenessGuard is ISemver, BaseGuard { require(SAFE.isOwner(msg.sender), "LivenessGuard: only Safe owners may demontstrate liveness"); lastLive[msg.sender] = block.timestamp; - emit SignerRecorded(0x0, msg.sender); + emit OwnerRecorded(0x0, msg.sender); } /// @notice Getter function for the Safe contract instance diff --git a/packages/contracts-bedrock/src/Safe/LivenessModule.sol b/packages/contracts-bedrock/src/Safe/LivenessModule.sol index a631232b080b..2a492add9749 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessModule.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessModule.sol @@ -147,11 +147,11 @@ contract LivenessModule is ISemver { ); // Check that the guard has not been changed. - _verifyGuard(); + _requireCorrectGuard(); } /// @notice Reverts if the guard address does not match the expected value. - function _verifyGuard() internal view { + function _requireCorrectGuard() internal view { require( address(LIVENESS_GUARD) == address(uint160(uint256(bytes32(SAFE.getStorageAt(GUARD_STORAGE_SLOT, 1))))), "LivenessModule: guard has been changed" diff --git a/packages/contracts-bedrock/test/LivenessGuard.t.sol b/packages/contracts-bedrock/test/LivenessGuard.t.sol index c93872be3a8b..206f27df8e5d 100644 --- a/packages/contracts-bedrock/test/LivenessGuard.t.sol +++ b/packages/contracts-bedrock/test/LivenessGuard.t.sol @@ -21,7 +21,7 @@ import { LivenessGuard } from "src/Safe/LivenessGuard.sol"; contract LivenessGuard_TestInit is Test, SafeTestTools { using SafeTestLib for SafeInstance; - event SignerRecorded(bytes32 indexed txHash, address signer); + event OwnerRecorded(bytes32 indexed txHash, address signer); LivenessGuard livenessGuard; SafeInstance safeInstance; @@ -72,7 +72,7 @@ contract LivenessGuard_CheckTx_Test is LivenessGuard_TestInit { for (uint256 i; i < signers.length; i++) { // Don't check topic1 so that we can avoid the ugly txHash calculation. vm.expectEmit(false, true, true, true, address(livenessGuard)); - emit SignerRecorded(0x0, signers[i]); + emit OwnerRecorded(0x0, signers[i]); } vm.expectCall(address(safeInstance.safe), abi.encodeWithSignature("nonce()")); vm.expectCall(address(safeInstance.safe), abi.encodeCall(OwnerManager.getThreshold, ())); @@ -99,7 +99,7 @@ contract LivenessGuard_ShowLiveness_Test is LivenessGuard_TestInit { address caller = safeInstance.owners[0]; vm.expectEmit(address(livenessGuard)); - emit SignerRecorded(0x0, caller); + emit OwnerRecorded(0x0, caller); vm.prank(caller); livenessGuard.showLiveness(); From 7efcfc3c06f75c3db89182dab8fb4d80ae0136b7 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Wed, 25 Oct 2023 13:22:11 -0400 Subject: [PATCH 220/279] specs: Apply code review suggestions Co-authored-by: Matt Solomon --- specs/safe-liveness-checking.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/specs/safe-liveness-checking.md b/specs/safe-liveness-checking.md index 1a71cfee83fa..5dfb66a701af 100644 --- a/specs/safe-liveness-checking.md +++ b/specs/safe-liveness-checking.md @@ -34,7 +34,7 @@ For implementing liveness checks a `LivenessGuard` is created which receives the each executed transaction, and tracks the latest time at which a transaction was signed by each signer. This time is made publicly available by calling a `lastLive(address)(Timestamp)` method. -Signers may also call the contract directly in order to prove liveness. +Signers may also call the contract's `showLiveness()()` method directly in order to prove liveness. ### The Liveness Module @@ -47,7 +47,7 @@ A `LivenessModule` is also created which does the following: 1. If so, it will call the Safe's `removeSigner()` to remove the non-live signer, and if necessary reduce the threshold. 1. When a member is removed, the signing parameters are modified such that `M/N` is the lowest ratio - which remains above 75%. Using integer math, this can be expressed as `M = (N * 75 + 99) / 100`. + which remains greater than or equal to 75%. Using integer math, this can be expressed as `M = (N * 75 + 99) / 100`. ### Shutdown From ce97143813e4442a41d66ffc8fa0f53e3a8edc0b Mon Sep 17 00:00:00 2001 From: Maurelian Date: Tue, 24 Oct 2023 15:47:28 -0400 Subject: [PATCH 221/279] docs(ctb): Some commenting fixes --- packages/contracts-bedrock/src/Safe/LivenessModule.sol | 3 --- .../test/safe-tools/CompatibilityFallbackHandler_1_3_0.sol | 4 ++-- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessModule.sol b/packages/contracts-bedrock/src/Safe/LivenessModule.sol index 2a492add9749..cb90bb739a04 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessModule.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessModule.sol @@ -7,9 +7,6 @@ import { OwnerManager } from "safe-contracts/base/OwnerManager.sol"; import { LivenessGuard } from "src/Safe/LivenessGuard.sol"; import { ISemver } from "src/universal/ISemver.sol"; -// TODO(maurelian): remove me -import { console2 as console } from "forge-std/console2.sol"; - /// @title LivenessModule /// @notice This module is intended to be used in conjunction with the LivenessGuard. In the event /// that an owner of the safe is not recorded by the guard during the liveness interval, diff --git a/packages/contracts-bedrock/test/safe-tools/CompatibilityFallbackHandler_1_3_0.sol b/packages/contracts-bedrock/test/safe-tools/CompatibilityFallbackHandler_1_3_0.sol index 30c002f119f1..a0efa0e68b26 100644 --- a/packages/contracts-bedrock/test/safe-tools/CompatibilityFallbackHandler_1_3_0.sol +++ b/packages/contracts-bedrock/test/safe-tools/CompatibilityFallbackHandler_1_3_0.sol @@ -8,8 +8,8 @@ import "safe-contracts/interfaces/IERC165.sol"; import "safe-contracts/interfaces/ISignatureValidator.sol"; import { Safe as GnosisSafe } from "safe-contracts/Safe.sol"; -/// @author: Colin Nielsen -/// https://github.com/colinnielsen/safe-tools/blob/ce6c654a76d91b619ab7778c77d1a76b3ced6666/src/CompatibilityFallbackHandler_1_3_0.sol +/// author: Colin Nielsen +/// https://github.com/colinnielsen/safe-tools/blob/ce6c654a76d91b619ab7778c77d1a76b3ced6666/src/CompatibilityFallbackHandler_1_3_0.sol contract DefaultCallbackHandler is ERC1155TokenReceiver, ERC777TokensRecipient, ERC721TokenReceiver, IERC165 { function onERC1155Received( address, From e2eb747af969c8dbe299436cebb96cf60511f86b Mon Sep 17 00:00:00 2001 From: Maurelian Date: Wed, 25 Oct 2023 09:02:51 -0400 Subject: [PATCH 222/279] feat(ctb): Add min owners sanity check in Liveness Module constructor --- packages/contracts-bedrock/src/Safe/LivenessModule.sol | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessModule.sol b/packages/contracts-bedrock/src/Safe/LivenessModule.sol index cb90bb739a04..2bd1729a63ee 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessModule.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessModule.sol @@ -55,8 +55,11 @@ contract LivenessModule is ISemver { SAFE = _safe; LIVENESS_GUARD = _livenessGuard; LIVENESS_INTERVAL = _livenessInterval; - MIN_OWNERS = _minOwners; FALLBACK_OWNER = _fallbackOwner; + MIN_OWNERS = _minOwners; + require( + _minOwners < _safe.getOwners().length, "LivenessModule: minOwners must be less than the number of owners" + ); } /// @notice This function can be called by anyone to remove an owner that has not signed a transaction From 432f22c2bc35365bbb59ad79a5481f47b748e3b9 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Wed, 25 Oct 2023 13:56:04 -0400 Subject: [PATCH 223/279] WIP: Add mermaid graph --- specs/safe-liveness-checking.md | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/specs/safe-liveness-checking.md b/specs/safe-liveness-checking.md index 5dfb66a701af..86cb4054a9d0 100644 --- a/specs/safe-liveness-checking.md +++ b/specs/safe-liveness-checking.md @@ -49,6 +49,25 @@ A `LivenessModule` is also created which does the following: 1. When a member is removed, the signing parameters are modified such that `M/N` is the lowest ratio which remains greater than or equal to 75%. Using integer math, this can be expressed as `M = (N * 75 + 99) / 100`. +### Owner removal call flow + +The following diagram illustrates the flow for removing a single owner. + +```mermaid +sequenceDiagram + participant User + participant LivenessModule + participant LivenessGuard + participant Safe + User->>LivenessModule: removeOwner(owner) + LivenessModule->>LivenessGuard: lastLive(owner) + LivenessModule->>Safe: getOwners() + LivenessModule->>LivenessModule: get75PercentThreshold(numOwnersAfter) + LivenessModule->>LivenessModule: _getPrevOwner(owner, owners) + LivenessModule->>LivenessModule: _removeOwner(prevOwner, owner, thresholdAfter) + LivenessModule->>LivenessModule: _verifyFinalState() +``` + ### Shutdown In the unlikely event that the signer set (`N`) is reduced below 8, then (and only then) is a From 67a17c319db98d3469cc3e149f8e6d23913f5b84 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Wed, 25 Oct 2023 15:31:59 -0400 Subject: [PATCH 224/279] refactor(ctb): Change to removeOwners This commit introduces the ability to remove multiple owners at once in the LivenessModule.sol contract. A new function removeOwners has been added, which accepts arrays of previous owners and owners to be removed. The existing removeOwner function has been updated to use a new internal function _removeOwner. The _getPrevOwner function has been removed as it is no longer needed. --- .../src/Safe/LivenessModule.sol | 101 ++++++++---------- .../test/LivenessModule.t.sol | 57 ++++++++-- 2 files changed, 93 insertions(+), 65 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessModule.sol b/packages/contracts-bedrock/src/Safe/LivenessModule.sol index 2bd1729a63ee..79e764b12449 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessModule.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessModule.sol @@ -37,9 +37,6 @@ contract LivenessModule is ISemver { /// keccak256("guard_manager.guard.address") uint256 internal constant GUARD_STORAGE_SLOT = 0x4a204f620c8c5ccdca3fd54d003badd85ba500436a431f0cbda4f558c93c34c8; - /// @notice The address of the first owner in the linked list of owners - address internal constant SENTINEL_OWNERS = address(0x1); - /// @notice Semantic version. /// @custom:semver 1.0.0 string public constant version = "1.0.0"; @@ -62,71 +59,73 @@ contract LivenessModule is ISemver { ); } + function removeOwners(address[] memory _previousOwners, address[] memory _ownersToRemove) external { + require(_previousOwners.length == _ownersToRemove.length, "LivenessModule: arrays must be the same length"); + for (uint256 i = 0; i < _previousOwners.length; i++) { + _removeOwner(_previousOwners[i], _ownersToRemove[i]); + } + _verifyFinalState(); + } + /// @notice This function can be called by anyone to remove an owner that has not signed a transaction /// during the liveness interval. If the number of owners drops below the minimum, then the /// ownership of the Safe is transferred to the fallback owner. - function removeOwner(address owner) external { - // Check that the owner to remove has not signed a transaction in the last 30 days - require( - LIVENESS_GUARD.lastLive(owner) < block.timestamp - LIVENESS_INTERVAL, - "LivenessModule: owner has signed recently" - ); - + function _removeOwner(address _prevOwner, address _ownerToRemove) internal { // Calculate the new threshold address[] memory owners = SAFE.getOwners(); uint256 numOwnersAfter = owners.length - 1; if (_isAboveMinOwners(numOwnersAfter)) { + // Check that the owner to remove has not signed a transaction in the last 30 days + require( + LIVENESS_GUARD.lastLive(_ownerToRemove) < block.timestamp - LIVENESS_INTERVAL, + "LivenessModule: owner has signed recently" + ); // Call the Safe to remove the owner and update the threshold uint256 thresholdAfter = get75PercentThreshold(numOwnersAfter); - address prevOwner = _getPrevOwner(owner, owners); - _removeOwner({ _prevOwner: prevOwner, _owner: owner, _threshold: thresholdAfter }); + _removeOwnerSafeCall({ _prevOwner: _prevOwner, _owner: _ownerToRemove, _threshold: thresholdAfter }); } else { // The number of owners is dangerously low, so we wish to transfer the ownership of this Safe - // to the fallback owner. - - // Remove owners one at a time starting from the last owner. - // Since we're removing them in order from last to first, the ordering will remain constant, - // and we shouldn't need to query the list of owners again. - address prevOwner; - for (uint256 i = owners.length - 1; i > 0; i--) { - address currentOwner = owners[i]; - prevOwner = _getPrevOwner(currentOwner, owners); - - // Call the Safe to remove the owner - _removeOwner({ _prevOwner: prevOwner, _owner: currentOwner, _threshold: 1 }); + // to the fallback owner. Therefore we no longer need to validate the liveness of the owners + // before removing them. + if (numOwnersAfter == 0) { + // Add the fallback owner as the sole owner of the Safe + _swapToFallbackOwnerSafeCall({ _prevOwner: _prevOwner, _oldOwner: _ownerToRemove }); + } else { + // Remove the owner and set the threshold to 1 + _removeOwnerSafeCall({ _prevOwner: _prevOwner, _owner: _ownerToRemove, _threshold: 1 }); } - - prevOwner = _getPrevOwner(owners[0], owners); - // Add the fallback owner as the sole owner of the Safe - _swapToFallbackOwner({ _prevOwner: prevOwner, _oldOwner: owners[0] }); } - - _verifyFinalState(); } /// @notice Sets the fallback owner as the sole owner of the Safe with a threshold of 1 /// @param _prevOwner Owner that pointed to the owner to be replaced in the linked list /// @param _oldOwner Owner address to be replaced. - function _swapToFallbackOwner(address _prevOwner, address _oldOwner) internal { - SAFE.execTransactionFromModule({ - to: address(SAFE), - value: 0, - operation: Enum.Operation.Call, - data: abi.encodeCall(OwnerManager.swapOwner, (_prevOwner, _oldOwner, FALLBACK_OWNER)) - }); + function _swapToFallbackOwnerSafeCall(address _prevOwner, address _oldOwner) internal { + require( + SAFE.execTransactionFromModule({ + to: address(SAFE), + value: 0, + operation: Enum.Operation.Call, + data: abi.encodeCall(OwnerManager.swapOwner, (_prevOwner, _oldOwner, FALLBACK_OWNER)) + }), + "LivenessModule: failed to swap to fallback owner" + ); } /// @notice Removes the owner `owner` from the Safe and updates the threshold to `_threshold`. /// @param _prevOwner Owner that pointed to the owner to be removed in the linked list /// @param _owner Owner address to be removed. /// @param _threshold New threshold. - function _removeOwner(address _prevOwner, address _owner, uint256 _threshold) internal { - SAFE.execTransactionFromModule({ - to: address(SAFE), - value: 0, - operation: Enum.Operation.Call, - data: abi.encodeCall(OwnerManager.removeOwner, (_prevOwner, _owner, _threshold)) - }); + function _removeOwnerSafeCall(address _prevOwner, address _owner, uint256 _threshold) internal { + require( + SAFE.execTransactionFromModule({ + to: address(SAFE), + value: 0, + operation: Enum.Operation.Call, + data: abi.encodeCall(OwnerManager.removeOwner, (_prevOwner, _owner, _threshold)) + }), + "LivenessModule: failed to remove owner" + ); } /// @notice A FREI-PI invariant check enforcing requirements on number of owners and threshold. @@ -158,20 +157,6 @@ contract LivenessModule is ISemver { ); } - /// @notice Get the previous owner in the linked list of owners - /// @param _owner The owner whose previous owner we want to find - /// @param _owners The list of owners - function _getPrevOwner(address _owner, address[] memory _owners) internal pure returns (address prevOwner_) { - for (uint256 i = 0; i < _owners.length; i++) { - if (_owners[i] != _owner) continue; - if (i == 0) { - prevOwner_ = SENTINEL_OWNERS; - break; - } - prevOwner_ = _owners[i - 1]; - } - } - /// @notice For a given number of owners, return the lowest threshold which is greater than 75. /// Note: this function returns 1 for numOwners == 1. function get75PercentThreshold(uint256 _numOwners) public pure returns (uint256 threshold_) { diff --git a/packages/contracts-bedrock/test/LivenessModule.t.sol b/packages/contracts-bedrock/test/LivenessModule.t.sol index 6b669442e810..c2b9936b7d37 100644 --- a/packages/contracts-bedrock/test/LivenessModule.t.sol +++ b/packages/contracts-bedrock/test/LivenessModule.t.sol @@ -5,26 +5,57 @@ import { Test, StdUtils } from "forge-std/Test.sol"; import { Safe } from "safe-contracts/Safe.sol"; import { SafeProxyFactory } from "safe-contracts/proxies/SafeProxyFactory.sol"; import { ModuleManager } from "safe-contracts/base/ModuleManager.sol"; +import { OwnerManager } from "safe-contracts/base/OwnerManager.sol"; import { Enum } from "safe-contracts/common/Enum.sol"; import "test/safe-tools/SafeTestTools.sol"; import { LivenessModule } from "src/Safe/LivenessModule.sol"; import { LivenessGuard } from "src/Safe/LivenessGuard.sol"; +contract OwnerSimulator is OwnerManager { + constructor(address[] memory _owners, uint256 _threshold) { + setupOwners(_owners, _threshold); + } + + function removeOwnerWrapped(address prevOwner, address owner, uint256 _threshold) public { + OwnerManager(address(this)).removeOwner(prevOwner, owner, _threshold); + } +} + contract LivenessModule_TestInit is Test, SafeTestTools { using SafeTestLib for SafeInstance; + /// @notice The address of the first owner in the linked list of owners + address internal constant SENTINEL_OWNERS = address(0x1); + event SignersRecorded(bytes32 indexed txHash, address[] signers); LivenessModule livenessModule; LivenessGuard livenessGuard; SafeInstance safeInstance; + OwnerSimulator ownerSimulator; address fallbackOwner; + /// @notice Get the previous owner in the linked list of owners + /// @param _owner The owner whose previous owner we want to find + /// @param _owners The list of owners + function _getPrevOwner(address _owner, address[] memory _owners) internal pure returns (address prevOwner_) { + for (uint256 i = 0; i < _owners.length; i++) { + if (_owners[i] != _owner) continue; + if (i == 0) { + prevOwner_ = SENTINEL_OWNERS; + break; + } + prevOwner_ = _owners[i - 1]; + } + } + function setUp() public { // Create a Safe with 10 owners (, uint256[] memory keys) = makeAddrsAndKeys(10); safeInstance = _setupSafe(keys, 8); + ownerSimulator = new OwnerSimulator(safeInstance.owners, 1); + livenessGuard = new LivenessGuard(safeInstance.safe); fallbackOwner = makeAddr("fallbackOwner"); livenessModule = new LivenessModule({ @@ -79,22 +110,34 @@ contract LivenessModule_Get75PercentThreshold_Test is LivenessModule_TestInit { contract LivenessModule_RemoveOwner_Test is LivenessModule_TestInit { function test_removeOwner_oneOwner_succeeds() external { uint256 ownersBefore = safeInstance.owners.length; + address[] memory prevOwners = new address[](1); + address[] memory ownersToRemove = new address[](1); + ownersToRemove[0] = safeInstance.owners[0]; + prevOwners[0] = _getPrevOwner(safeInstance.owners[0], safeInstance.owners); + + address ownerToRemove = safeInstance.owners[0]; vm.warp(block.timestamp + 30 days); - livenessModule.removeOwner(safeInstance.owners[0]); + + livenessModule.removeOwners(prevOwners, ownersToRemove); assertEq(safeInstance.safe.getOwners().length, ownersBefore - 1); } function test_removeOwner_allOwners_succeeds() external { vm.warp(block.timestamp + 30 days); uint256 numOwners = safeInstance.owners.length; - uint256 minOwners = livenessModule.minOwners(); - - // Remove enough owners to trigger the transfer to the fallbackOwner - uint256 numToRemove = numOwners - minOwners + 1; - for (uint256 i = 0; i < numToRemove; i++) { - livenessModule.removeOwner(safeInstance.owners[i]); + address[] memory prevOwners = new address[](numOwners); + address[] memory ownersToRemove = new address[](numOwners); + address[] memory currentOwners; + for (uint256 i = 0; i < numOwners; i++) { + currentOwners = ownerSimulator.getOwners(); + ownersToRemove[i] = safeInstance.owners[i]; + prevOwners[i] = _getPrevOwner(safeInstance.owners[i], currentOwners); + if (currentOwners.length == 1) break; + ownerSimulator.removeOwnerWrapped(prevOwners[i], ownersToRemove[i], 1); } + + livenessModule.removeOwners(prevOwners, ownersToRemove); assertEq(safeInstance.safe.getOwners().length, 1); assertEq(safeInstance.safe.getOwners()[0], fallbackOwner); assertEq(safeInstance.safe.getThreshold(), 1); From 377dcdaf6ce3f6722daafd07f4d7e9b701c495b1 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Wed, 25 Oct 2023 21:23:49 -0400 Subject: [PATCH 225/279] refactor(ctb): Simplify removal logic This refactor removes a few different variable which were unnecessary and added to the confusion. --- .../src/Safe/LivenessModule.sol | 55 +++++++++++-------- .../test/LivenessModule.t.sol | 1 - 2 files changed, 31 insertions(+), 25 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessModule.sol b/packages/contracts-bedrock/src/Safe/LivenessModule.sol index 79e764b12449..11a04252aa67 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessModule.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessModule.sol @@ -61,8 +61,18 @@ contract LivenessModule is ISemver { function removeOwners(address[] memory _previousOwners, address[] memory _ownersToRemove) external { require(_previousOwners.length == _ownersToRemove.length, "LivenessModule: arrays must be the same length"); + + // We will remove at least one owner, so we'll initialize the newOwners count to the current number of owners + // minus one. + uint256 newOwnersCount = SAFE.getOwners().length; for (uint256 i = 0; i < _previousOwners.length; i++) { - _removeOwner(_previousOwners[i], _ownersToRemove[i]); + newOwnersCount--; + _removeOwner({ + _prevOwner: _previousOwners[i], + _ownerToRemove: _ownersToRemove[i], + _newOwnersCount: newOwnersCount, + _newThreshold: get75PercentThreshold(newOwnersCount) + }); } _verifyFinalState(); } @@ -70,30 +80,27 @@ contract LivenessModule is ISemver { /// @notice This function can be called by anyone to remove an owner that has not signed a transaction /// during the liveness interval. If the number of owners drops below the minimum, then the /// ownership of the Safe is transferred to the fallback owner. - function _removeOwner(address _prevOwner, address _ownerToRemove) internal { - // Calculate the new threshold - address[] memory owners = SAFE.getOwners(); - uint256 numOwnersAfter = owners.length - 1; - if (_isAboveMinOwners(numOwnersAfter)) { - // Check that the owner to remove has not signed a transaction in the last 30 days - require( - LIVENESS_GUARD.lastLive(_ownerToRemove) < block.timestamp - LIVENESS_INTERVAL, - "LivenessModule: owner has signed recently" - ); - // Call the Safe to remove the owner and update the threshold - uint256 thresholdAfter = get75PercentThreshold(numOwnersAfter); - _removeOwnerSafeCall({ _prevOwner: _prevOwner, _owner: _ownerToRemove, _threshold: thresholdAfter }); - } else { - // The number of owners is dangerously low, so we wish to transfer the ownership of this Safe - // to the fallback owner. Therefore we no longer need to validate the liveness of the owners - // before removing them. - if (numOwnersAfter == 0) { - // Add the fallback owner as the sole owner of the Safe - _swapToFallbackOwnerSafeCall({ _prevOwner: _prevOwner, _oldOwner: _ownerToRemove }); - } else { - // Remove the owner and set the threshold to 1 - _removeOwnerSafeCall({ _prevOwner: _prevOwner, _owner: _ownerToRemove, _threshold: 1 }); + function _removeOwner( + address _prevOwner, + address _ownerToRemove, + uint256 _newOwnersCount, + uint256 _newThreshold + ) + internal + { + if (_newOwnersCount > 0) { + if (_isAboveMinOwners(_newOwnersCount)) { + // Check that the owner to remove has not signed a transaction in the last 30 days + require( + LIVENESS_GUARD.lastLive(_ownerToRemove) < block.timestamp - LIVENESS_INTERVAL, + "LivenessModule: owner has signed recently" + ); } + // Remove the owner and update the threshold + _removeOwnerSafeCall({ _prevOwner: _prevOwner, _owner: _ownerToRemove, _threshold: _newThreshold }); + } else { + // Add the fallback owner as the sole owner of the Safe + _swapToFallbackOwnerSafeCall({ _prevOwner: _prevOwner, _oldOwner: _ownerToRemove }); } } diff --git a/packages/contracts-bedrock/test/LivenessModule.t.sol b/packages/contracts-bedrock/test/LivenessModule.t.sol index c2b9936b7d37..4aeae7ca9809 100644 --- a/packages/contracts-bedrock/test/LivenessModule.t.sol +++ b/packages/contracts-bedrock/test/LivenessModule.t.sol @@ -115,7 +115,6 @@ contract LivenessModule_RemoveOwner_Test is LivenessModule_TestInit { ownersToRemove[0] = safeInstance.owners[0]; prevOwners[0] = _getPrevOwner(safeInstance.owners[0], safeInstance.owners); - address ownerToRemove = safeInstance.owners[0]; vm.warp(block.timestamp + 30 days); livenessModule.removeOwners(prevOwners, ownersToRemove); From 40c997f13bd7531a009921d2a7301382e3d158c2 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Wed, 25 Oct 2023 21:54:17 -0400 Subject: [PATCH 226/279] refactor(ctb): Reduce branching on pre-removal check --- .../contracts-bedrock/src/Safe/LivenessModule.sol | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessModule.sol b/packages/contracts-bedrock/src/Safe/LivenessModule.sol index 11a04252aa67..7c9134fc9eaf 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessModule.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessModule.sol @@ -88,14 +88,12 @@ contract LivenessModule is ISemver { ) internal { + require( + !_isAboveMinOwners(_newOwnersCount) + || LIVENESS_GUARD.lastLive(_ownerToRemove) < block.timestamp - LIVENESS_INTERVAL, + "LivenessModule: owner has signed recently" + ); if (_newOwnersCount > 0) { - if (_isAboveMinOwners(_newOwnersCount)) { - // Check that the owner to remove has not signed a transaction in the last 30 days - require( - LIVENESS_GUARD.lastLive(_ownerToRemove) < block.timestamp - LIVENESS_INTERVAL, - "LivenessModule: owner has signed recently" - ); - } // Remove the owner and update the threshold _removeOwnerSafeCall({ _prevOwner: _prevOwner, _owner: _ownerToRemove, _threshold: _newThreshold }); } else { From a725a3c28394fc4727097bb377f1fed914f9f982 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Wed, 25 Oct 2023 22:15:18 -0400 Subject: [PATCH 227/279] refactor(ctb): Better helper functions for pre-removal check --- .../src/Safe/LivenessModule.sol | 21 +++++++++---------- 1 file changed, 10 insertions(+), 11 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessModule.sol b/packages/contracts-bedrock/src/Safe/LivenessModule.sol index 7c9134fc9eaf..8f9026d22038 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessModule.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessModule.sol @@ -89,9 +89,8 @@ contract LivenessModule is ISemver { internal { require( - !_isAboveMinOwners(_newOwnersCount) - || LIVENESS_GUARD.lastLive(_ownerToRemove) < block.timestamp - LIVENESS_INTERVAL, - "LivenessModule: owner has signed recently" + _newOwnersCount < MIN_OWNERS || _canRemove(_ownerToRemove), + "LivenessModule: the safe still has sufficient owners, or the owner to remove has signed recently" ); if (_newOwnersCount > 0) { // Remove the owner and update the threshold @@ -133,12 +132,19 @@ contract LivenessModule is ISemver { ); } + /// @notice Checks if the owner can be removed + /// @param _owner The owner to be removed + /// @return canRemove_ bool indicating if the owner can be removed + function _canRemove(address _owner) internal view returns (bool canRemove_) { + canRemove_ = LIVENESS_GUARD.lastLive(_owner) + LIVENESS_INTERVAL < block.timestamp; + } + /// @notice A FREI-PI invariant check enforcing requirements on number of owners and threshold. function _verifyFinalState() internal view { address[] memory owners = SAFE.getOwners(); uint256 numOwners = owners.length; require( - _isAboveMinOwners(numOwners) || (numOwners == 1 && owners[0] == FALLBACK_OWNER), + numOwners >= MIN_OWNERS || (numOwners == 1 && owners[0] == FALLBACK_OWNER), "LivenessModule: Safe must have the minimum number of owners or be owned solely by the fallback owner" ); @@ -168,13 +174,6 @@ contract LivenessModule is ISemver { threshold_ = (_numOwners * 75 + 99) / 100; } - /// @notice Check if the number of owners is greater than or equal to the minimum number of owners. - /// @param numOwners The number of owners. - /// @return A boolean indicating if the number of owners is greater than or equal to the minimum number of owners. - function _isAboveMinOwners(uint256 numOwners) internal view returns (bool) { - return numOwners >= MIN_OWNERS; - } - /// @notice Getter function for the Safe contract instance /// @return safe_ The Safe contract instance function safe() public view returns (Safe safe_) { From 42d4f5e54d9bb513d2b02d0190d4f601ab9e590d Mon Sep 17 00:00:00 2001 From: Maurelian Date: Wed, 25 Oct 2023 22:21:17 -0400 Subject: [PATCH 228/279] refactor(ctb): Move pre-removal check to outer function --- .../src/Safe/LivenessModule.sol | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessModule.sol b/packages/contracts-bedrock/src/Safe/LivenessModule.sol index 8f9026d22038..e330c06f2e4b 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessModule.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessModule.sol @@ -63,15 +63,18 @@ contract LivenessModule is ISemver { require(_previousOwners.length == _ownersToRemove.length, "LivenessModule: arrays must be the same length"); // We will remove at least one owner, so we'll initialize the newOwners count to the current number of owners - // minus one. - uint256 newOwnersCount = SAFE.getOwners().length; + uint256 ownersCount = SAFE.getOwners().length; for (uint256 i = 0; i < _previousOwners.length; i++) { - newOwnersCount--; + ownersCount--; + require( + ownersCount < MIN_OWNERS || _canRemove(_ownersToRemove[i]), + "LivenessModule: the safe still has sufficient owners, or the owner to remove has signed recently" + ); _removeOwner({ _prevOwner: _previousOwners[i], _ownerToRemove: _ownersToRemove[i], - _newOwnersCount: newOwnersCount, - _newThreshold: get75PercentThreshold(newOwnersCount) + _newOwnersCount: ownersCount, + _newThreshold: get75PercentThreshold(ownersCount) }); } _verifyFinalState(); @@ -88,10 +91,6 @@ contract LivenessModule is ISemver { ) internal { - require( - _newOwnersCount < MIN_OWNERS || _canRemove(_ownerToRemove), - "LivenessModule: the safe still has sufficient owners, or the owner to remove has signed recently" - ); if (_newOwnersCount > 0) { // Remove the owner and update the threshold _removeOwnerSafeCall({ _prevOwner: _prevOwner, _owner: _ownerToRemove, _threshold: _newThreshold }); From 199cb31d4e32a5bb6203e117093649deb5f6700b Mon Sep 17 00:00:00 2001 From: Maurelian Date: Wed, 25 Oct 2023 23:03:57 -0400 Subject: [PATCH 229/279] test(ctb): Extract owner prevOwners generation into helper function --- .../src/Safe/LivenessModule.sol | 16 +++++++++--- .../test/LivenessModule.t.sol | 26 ++++++++++++++----- 2 files changed, 31 insertions(+), 11 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessModule.sol b/packages/contracts-bedrock/src/Safe/LivenessModule.sol index e330c06f2e4b..603a56a30745 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessModule.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessModule.sol @@ -59,6 +59,11 @@ contract LivenessModule is ISemver { ); } + /// @notice This function can be called by anyone to remove a set of owners that have not signed a transaction + /// during the liveness interval. If the number of owners drops below the minimum, then all owners + /// must be removed. + /// @param _previousOwners The previous owners in the linked list of owners + /// @param _ownersToRemove The owners to remove function removeOwners(address[] memory _previousOwners, address[] memory _ownersToRemove) external { require(_previousOwners.length == _ownersToRemove.length, "LivenessModule: arrays must be the same length"); @@ -80,9 +85,11 @@ contract LivenessModule is ISemver { _verifyFinalState(); } - /// @notice This function can be called by anyone to remove an owner that has not signed a transaction - /// during the liveness interval. If the number of owners drops below the minimum, then the - /// ownership of the Safe is transferred to the fallback owner. + /// @notice Removes an owner from the Safe and updates the threshold. + /// @param _prevOwner Owner that pointed to the owner to be removed in the linked list + /// @param _ownerToRemove Owner address to be removed. + /// @param _newOwnersCount New number of owners after removal. + /// @param _newThreshold New threshold. function _removeOwner( address _prevOwner, address _ownerToRemove, @@ -95,7 +102,8 @@ contract LivenessModule is ISemver { // Remove the owner and update the threshold _removeOwnerSafeCall({ _prevOwner: _prevOwner, _owner: _ownerToRemove, _threshold: _newThreshold }); } else { - // Add the fallback owner as the sole owner of the Safe + // There is only one owner left. The Safe will not allow a safe with no owners, so we will + // need to swap owners instead. _swapToFallbackOwnerSafeCall({ _prevOwner: _prevOwner, _oldOwner: _ownerToRemove }); } } diff --git a/packages/contracts-bedrock/test/LivenessModule.t.sol b/packages/contracts-bedrock/test/LivenessModule.t.sol index 4aeae7ca9809..fa85646bf201 100644 --- a/packages/contracts-bedrock/test/LivenessModule.t.sol +++ b/packages/contracts-bedrock/test/LivenessModule.t.sol @@ -50,6 +50,23 @@ contract LivenessModule_TestInit is Test, SafeTestTools { } } + /// @notice Given an arrary of owners to remove, this function will return an array of the previous owners + /// in the order that they must be provided to the LivenessMoules's removeOwners() function. + /// Because owners are removed one at a time, and not necessarily in order, we need to simulate + /// the owners list after each removal, in order to identify the correct previous owner. + /// @param _ownersToRemove The owners to remove + /// @return prevOwners_ The previous owners in the linked list + function _getPrevOwners(address[] memory _ownersToRemove) internal returns (address[] memory prevOwners_) { + prevOwners_ = new address[](_ownersToRemove.length); + address[] memory currentOwners; + for (uint256 i = 0; i < _ownersToRemove.length; i++) { + currentOwners = ownerSimulator.getOwners(); + prevOwners_[i] = _getPrevOwner(safeInstance.owners[i], currentOwners); + if (currentOwners.length == 1) break; + ownerSimulator.removeOwnerWrapped(prevOwners_[i], _ownersToRemove[i], 1); + } + } + function setUp() public { // Create a Safe with 10 owners (, uint256[] memory keys) = makeAddrsAndKeys(10); @@ -122,20 +139,15 @@ contract LivenessModule_RemoveOwner_Test is LivenessModule_TestInit { } function test_removeOwner_allOwners_succeeds() external { - vm.warp(block.timestamp + 30 days); uint256 numOwners = safeInstance.owners.length; - address[] memory prevOwners = new address[](numOwners); address[] memory ownersToRemove = new address[](numOwners); - address[] memory currentOwners; for (uint256 i = 0; i < numOwners; i++) { - currentOwners = ownerSimulator.getOwners(); ownersToRemove[i] = safeInstance.owners[i]; - prevOwners[i] = _getPrevOwner(safeInstance.owners[i], currentOwners); - if (currentOwners.length == 1) break; - ownerSimulator.removeOwnerWrapped(prevOwners[i], ownersToRemove[i], 1); } + address[] memory prevOwners = _getPrevOwners(ownersToRemove); + vm.warp(block.timestamp + 30 days); livenessModule.removeOwners(prevOwners, ownersToRemove); assertEq(safeInstance.safe.getOwners().length, 1); assertEq(safeInstance.safe.getOwners()[0], fallbackOwner); From 3f6470e710346ca2483593ecad62fe06c1352d80 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Wed, 25 Oct 2023 23:18:20 -0400 Subject: [PATCH 230/279] test(ctb): Add natspec to test functions --- .../test/LivenessGuard.t.sol | 22 ++++++++++++------- .../test/LivenessModule.t.sol | 11 ++++++++-- 2 files changed, 23 insertions(+), 10 deletions(-) diff --git a/packages/contracts-bedrock/test/LivenessGuard.t.sol b/packages/contracts-bedrock/test/LivenessGuard.t.sol index 206f27df8e5d..73833f8f6fef 100644 --- a/packages/contracts-bedrock/test/LivenessGuard.t.sol +++ b/packages/contracts-bedrock/test/LivenessGuard.t.sol @@ -10,14 +10,6 @@ import "test/safe-tools/SafeTestTools.sol"; import { LivenessGuard } from "src/Safe/LivenessGuard.sol"; -// Todo(Maurelian): -// Other tests needed: -// - EIP1271 signatures -// - Signatures from contracts -// - Signatures from non-owners -// - Signers may call directly to prove liveness (must be an owner). -// - Unexpected length of signature data - contract LivenessGuard_TestInit is Test, SafeTestTools { using SafeTestLib for SafeInstance; @@ -26,6 +18,7 @@ contract LivenessGuard_TestInit is Test, SafeTestTools { LivenessGuard livenessGuard; SafeInstance safeInstance; + /// @dev Sets up the test environment function setUp() public { safeInstance = _setupSafe(); livenessGuard = new LivenessGuard(safeInstance.safe); @@ -34,6 +27,7 @@ contract LivenessGuard_TestInit is Test, SafeTestTools { } contract LivenessGuard_Getters_Test is LivenessGuard_TestInit { + /// @dev Tests that the getters return the correct values function test_getters_works() external { assertEq(address(livenessGuard.safe()), address(safeInstance.safe)); assertEq(livenessGuard.lastLive(address(0)), 0); @@ -41,6 +35,7 @@ contract LivenessGuard_Getters_Test is LivenessGuard_TestInit { } contract LivenessGuard_CheckTx_TestFails is LivenessGuard_TestInit { + /// @dev Tests that the checkTransaction function reverts if the caller is not the Safe function test_checkTransaction_callerIsNotSafe_revert() external { vm.expectRevert("LivenessGuard: only Safe can call this function"); livenessGuard.checkTransaction({ @@ -62,6 +57,7 @@ contract LivenessGuard_CheckTx_TestFails is LivenessGuard_TestInit { contract LivenessGuard_CheckTx_Test is LivenessGuard_TestInit { using SafeTestLib for SafeInstance; + /// @dev Tests that the checkTransaction function succeeds function test_checkTransaction_succeeds() external { // Create an array of the addresses who will sign the transaction. SafeTestTools // will generate these signatures up to the threshold by iterating over the owners array. @@ -85,6 +81,7 @@ contract LivenessGuard_CheckTx_Test is LivenessGuard_TestInit { } contract LivenessGuard_CheckAfterExecution_TestFails is LivenessGuard_TestInit { + /// @dev Tests that the checkAfterExecution function reverts if the caller is not the Safe function test_checkAfterExecution_callerIsNotSafe_revert() external { vm.expectRevert("LivenessGuard: only Safe can call this function"); livenessGuard.checkAfterExecution(bytes32(0), false); @@ -93,7 +90,16 @@ contract LivenessGuard_CheckAfterExecution_TestFails is LivenessGuard_TestInit { contract LivenessGuard_CheckAfterExecution_Test is LivenessGuard_TestInit { } +contract LivenessGuard_ShowLiveness_TestFail is LivenessGuard_TestInit { + /// @dev Tests that the showLiveness function reverts if the caller is not an owner + function test_showLiveness_callIsNotSafeOwner_reverts() external { + vm.expectRevert("LivenessGuard: only Safe owners can call this function"); + livenessGuard.showLiveness(); + } +} + contract LivenessGuard_ShowLiveness_Test is LivenessGuard_TestInit { + /// @dev Tests that the showLiveness function succeeds function test_showLiveness_succeeds() external { // Cache the caller address caller = safeInstance.owners[0]; diff --git a/packages/contracts-bedrock/test/LivenessModule.t.sol b/packages/contracts-bedrock/test/LivenessModule.t.sol index fa85646bf201..5dd4e5811c67 100644 --- a/packages/contracts-bedrock/test/LivenessModule.t.sol +++ b/packages/contracts-bedrock/test/LivenessModule.t.sol @@ -12,6 +12,9 @@ import "test/safe-tools/SafeTestTools.sol"; import { LivenessModule } from "src/Safe/LivenessModule.sol"; import { LivenessGuard } from "src/Safe/LivenessGuard.sol"; +/// @dev A minimal wrapper around the OwnerManager contract. This contract is meant to be initialized with +/// the same owners as a Safe instance, and then used to simulate the resulting owners list +/// after an owner is removed. contract OwnerSimulator is OwnerManager { constructor(address[] memory _owners, uint256 _threshold) { setupOwners(_owners, _threshold); @@ -25,7 +28,7 @@ contract OwnerSimulator is OwnerManager { contract LivenessModule_TestInit is Test, SafeTestTools { using SafeTestLib for SafeInstance; - /// @notice The address of the first owner in the linked list of owners + /// @dev The address of the first owner in the linked list of owners address internal constant SENTINEL_OWNERS = address(0x1); event SignersRecorded(bytes32 indexed txHash, address[] signers); @@ -50,7 +53,7 @@ contract LivenessModule_TestInit is Test, SafeTestTools { } } - /// @notice Given an arrary of owners to remove, this function will return an array of the previous owners + /// @dev Given an array of owners to remove, this function will return an array of the previous owners /// in the order that they must be provided to the LivenessMoules's removeOwners() function. /// Because owners are removed one at a time, and not necessarily in order, we need to simulate /// the owners list after each removal, in order to identify the correct previous owner. @@ -67,6 +70,7 @@ contract LivenessModule_TestInit is Test, SafeTestTools { } } + /// @dev Sets up the test environment function setUp() public { // Create a Safe with 10 owners (, uint256[] memory keys) = makeAddrsAndKeys(10); @@ -88,6 +92,7 @@ contract LivenessModule_TestInit is Test, SafeTestTools { } contract LivenessModule_Getters_Test is LivenessModule_TestInit { + /// @dev Tests if the getters work correctly function test_getters_works() external { assertEq(address(livenessModule.safe()), address(safeInstance.safe)); assertEq(address(livenessModule.livenessGuard()), address(livenessGuard)); @@ -125,6 +130,7 @@ contract LivenessModule_Get75PercentThreshold_Test is LivenessModule_TestInit { } contract LivenessModule_RemoveOwner_Test is LivenessModule_TestInit { + /// @dev Tests if removing one owner works correctly function test_removeOwner_oneOwner_succeeds() external { uint256 ownersBefore = safeInstance.owners.length; address[] memory prevOwners = new address[](1); @@ -138,6 +144,7 @@ contract LivenessModule_RemoveOwner_Test is LivenessModule_TestInit { assertEq(safeInstance.safe.getOwners().length, ownersBefore - 1); } + /// @dev Tests if removing all owners works correctly function test_removeOwner_allOwners_succeeds() external { uint256 numOwners = safeInstance.owners.length; From 69f3cce1faa3048cd7d1b48daff5e8921a6e4688 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Wed, 25 Oct 2023 23:27:32 -0400 Subject: [PATCH 231/279] refactor(ctb): Add _onlySafe function --- .../contracts-bedrock/src/Safe/LivenessGuard.sol | 12 ++++++++---- packages/contracts-bedrock/test/LivenessGuard.t.sol | 2 +- 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol index 2e257697b55c..30825b8c1191 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol @@ -47,6 +47,11 @@ contract LivenessGuard is ISemver, BaseGuard { SAFE = _safe; } + /// @notice Internal function to ensure that only the Safe can call certain functions. + function _onlySafe() internal view { + require(msg.sender == address(SAFE), "LivenessGuard: only Safe can call this function"); + } + /// @notice Records the most recent time which any owner has signed a transaction. /// @dev Called by the Safe contract before execution of a transaction. function checkTransaction( @@ -65,7 +70,7 @@ contract LivenessGuard is ISemver, BaseGuard { external { msgSender; // silence unused variable warning - require(msg.sender == address(SAFE), "LivenessGuard: only Safe can call this function"); + _onlySafe(); // Cache the set of owners prior to execution. // This will be used in the checkAfterExecution method. @@ -106,8 +111,7 @@ contract LivenessGuard is ISemver, BaseGuard { /// 1. Add new owners to the lastLive mapping /// 2. Delete removed owners from the lastLive mapping function checkAfterExecution(bytes32, bool) external { - require(msg.sender == address(SAFE), "LivenessGuard: only Safe can call this function"); - + _onlySafe(); // Get the current set of owners address[] memory ownersAfter = SAFE.getOwners(); @@ -132,7 +136,7 @@ contract LivenessGuard is ISemver, BaseGuard { /// @notice Enables an owner to demonstrate liveness by calling this method directly. /// This is useful for owners who have not recently signed a transaction via the Safe. function showLiveness() external { - require(SAFE.isOwner(msg.sender), "LivenessGuard: only Safe owners may demontstrate liveness"); + require(SAFE.isOwner(msg.sender), "LivenessGuard: only Safe owners may demonstrate liveness"); lastLive[msg.sender] = block.timestamp; emit OwnerRecorded(0x0, msg.sender); diff --git a/packages/contracts-bedrock/test/LivenessGuard.t.sol b/packages/contracts-bedrock/test/LivenessGuard.t.sol index 73833f8f6fef..89531ba42dd1 100644 --- a/packages/contracts-bedrock/test/LivenessGuard.t.sol +++ b/packages/contracts-bedrock/test/LivenessGuard.t.sol @@ -93,7 +93,7 @@ contract LivenessGuard_CheckAfterExecution_Test is LivenessGuard_TestInit { } contract LivenessGuard_ShowLiveness_TestFail is LivenessGuard_TestInit { /// @dev Tests that the showLiveness function reverts if the caller is not an owner function test_showLiveness_callIsNotSafeOwner_reverts() external { - vm.expectRevert("LivenessGuard: only Safe owners can call this function"); + vm.expectRevert("LivenessGuard: only Safe owners may demonstrate liveness"); livenessGuard.showLiveness(); } } From ade16503198d84109dc09395e52b2023bcf57ffb Mon Sep 17 00:00:00 2001 From: Maurelian Date: Wed, 25 Oct 2023 23:28:04 -0400 Subject: [PATCH 232/279] test(ctb): Additional Liveness Module Testing --- .../test/LivenessModule.t.sol | 20 +++++++++++++++++-- 1 file changed, 18 insertions(+), 2 deletions(-) diff --git a/packages/contracts-bedrock/test/LivenessModule.t.sol b/packages/contracts-bedrock/test/LivenessModule.t.sol index 5dd4e5811c67..67579c0a7209 100644 --- a/packages/contracts-bedrock/test/LivenessModule.t.sol +++ b/packages/contracts-bedrock/test/LivenessModule.t.sol @@ -33,6 +33,8 @@ contract LivenessModule_TestInit is Test, SafeTestTools { event SignersRecorded(bytes32 indexed txHash, address[] signers); + uint256 livenessInterval = 30 days; + uint256 minOwners = 6; LivenessModule livenessModule; LivenessGuard livenessGuard; SafeInstance safeInstance; @@ -82,8 +84,8 @@ contract LivenessModule_TestInit is Test, SafeTestTools { livenessModule = new LivenessModule({ _safe: safeInstance.safe, _livenessGuard: livenessGuard, - _livenessInterval: 30 days, - _minOwners: 6, + _livenessInterval: livenessInterval, + _minOwners: minOwners, _fallbackOwner: fallbackOwner }); safeInstance.enableModule(address(livenessModule)); @@ -91,6 +93,20 @@ contract LivenessModule_TestInit is Test, SafeTestTools { } } +contract LivenessModule_Constructor_Test is LivenessModule_TestInit { + /// @dev Tests that the constructor fails if the minOwners is greater than the number of owners + function test_constructor_minOwnersGreaterThanOwners_revert() external { + vm.expectRevert("LivenessModule: minOwners must be less than the number of owners"); + new LivenessModule({ + _safe: safeInstance.safe, + _livenessGuard: livenessGuard, + _livenessInterval: livenessInterval, + _minOwners: 11, + _fallbackOwner: address(0) + }); + } +} + contract LivenessModule_Getters_Test is LivenessModule_TestInit { /// @dev Tests if the getters work correctly function test_getters_works() external { From 10a256ff328f75bd410fe3c92f4049a93827abd8 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Thu, 26 Oct 2023 00:10:26 -0400 Subject: [PATCH 233/279] test(ctb): Guard changed --- .../test/LivenessModule.t.sol | 41 ++++++++++++++++--- 1 file changed, 35 insertions(+), 6 deletions(-) diff --git a/packages/contracts-bedrock/test/LivenessModule.t.sol b/packages/contracts-bedrock/test/LivenessModule.t.sol index 67579c0a7209..ac1bdb444738 100644 --- a/packages/contracts-bedrock/test/LivenessModule.t.sol +++ b/packages/contracts-bedrock/test/LivenessModule.t.sol @@ -67,11 +67,23 @@ contract LivenessModule_TestInit is Test, SafeTestTools { for (uint256 i = 0; i < _ownersToRemove.length; i++) { currentOwners = ownerSimulator.getOwners(); prevOwners_[i] = _getPrevOwner(safeInstance.owners[i], currentOwners); + + // Don't try to remove the last owner if (currentOwners.length == 1) break; ownerSimulator.removeOwnerWrapped(prevOwners_[i], _ownersToRemove[i], 1); } } + /// @dev Removes an owner from the safe + function _removeAnOwner(address _ownerToRemove) internal { + address[] memory prevOwners = new address[](1); + address[] memory ownersToRemove = new address[](1); + ownersToRemove[0] = _ownerToRemove; + prevOwners[0] = _getPrevOwner(_ownerToRemove, safeInstance.owners); + + livenessModule.removeOwners(prevOwners, ownersToRemove); + } + /// @dev Sets up the test environment function setUp() public { // Create a Safe with 10 owners @@ -145,18 +157,35 @@ contract LivenessModule_Get75PercentThreshold_Test is LivenessModule_TestInit { } } +contract LivenessModule_RemoveOwner_TestFail is LivenessModule_TestInit { + using SafeTestLib for SafeInstance; + // "LivenessModule: guard has been changed" + + function test_removeOwner_guardChanged_revert() external { + address[] memory ownersToRemove = new address[](1); + ownersToRemove[0] = safeInstance.owners[0]; + address[] memory prevOwners = _getPrevOwners(ownersToRemove); + + // Change the guard + livenessGuard = new LivenessGuard(safeInstance.safe); + safeInstance.setGuard(address(livenessGuard)); + + vm.warp(block.timestamp + livenessInterval + 1); + vm.expectRevert("LivenessModule: guard has been changed"); + livenessModule.removeOwners(prevOwners, ownersToRemove); + } +} + contract LivenessModule_RemoveOwner_Test is LivenessModule_TestInit { /// @dev Tests if removing one owner works correctly function test_removeOwner_oneOwner_succeeds() external { uint256 ownersBefore = safeInstance.owners.length; - address[] memory prevOwners = new address[](1); - address[] memory ownersToRemove = new address[](1); - ownersToRemove[0] = safeInstance.owners[0]; - prevOwners[0] = _getPrevOwner(safeInstance.owners[0], safeInstance.owners); + address ownerToRemove = safeInstance.owners[0]; - vm.warp(block.timestamp + 30 days); + // vm.warp(block.timestamp + 30 days); + _removeAnOwner(ownerToRemove); - livenessModule.removeOwners(prevOwners, ownersToRemove); + assertFalse(safeInstance.safe.isOwner(ownerToRemove)); assertEq(safeInstance.safe.getOwners().length, ownersBefore - 1); } From 63fceb3908d4aee997cdff7e6222b104b105d0c1 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Thu, 26 Oct 2023 00:48:47 -0400 Subject: [PATCH 234/279] refactor(ctb): Remove _requireGuard It was only used once. --- packages/contracts-bedrock/src/Safe/LivenessModule.sol | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessModule.sol b/packages/contracts-bedrock/src/Safe/LivenessModule.sol index 603a56a30745..859e2aa178f1 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessModule.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessModule.sol @@ -150,6 +150,7 @@ contract LivenessModule is ISemver { function _verifyFinalState() internal view { address[] memory owners = SAFE.getOwners(); uint256 numOwners = owners.length; + // Ensure that the safe is not being left in an unsafe state with too few owners. require( numOwners >= MIN_OWNERS || (numOwners == 1 && owners[0] == FALLBACK_OWNER), "LivenessModule: Safe must have the minimum number of owners or be owned solely by the fallback owner" @@ -163,12 +164,7 @@ contract LivenessModule is ISemver { "LivenessModule: threshold must be 75% of the number of owners" ); - // Check that the guard has not been changed. - _requireCorrectGuard(); - } - - /// @notice Reverts if the guard address does not match the expected value. - function _requireCorrectGuard() internal view { + // Check that the guard has not been changed require( address(LIVENESS_GUARD) == address(uint160(uint256(bytes32(SAFE.getStorageAt(GUARD_STORAGE_SLOT, 1))))), "LivenessModule: guard has been changed" From 7fe44b99827d4c3dacfd0eab785074ef4a8d1d32 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Thu, 26 Oct 2023 00:57:38 -0400 Subject: [PATCH 235/279] test(ctb): Add tests for all reverts in Liveness Module --- .../test/LivenessModule.t.sol | 133 ++++++++++++++++-- 1 file changed, 124 insertions(+), 9 deletions(-) diff --git a/packages/contracts-bedrock/test/LivenessModule.t.sol b/packages/contracts-bedrock/test/LivenessModule.t.sol index ac1bdb444738..e4205f0796ce 100644 --- a/packages/contracts-bedrock/test/LivenessModule.t.sol +++ b/packages/contracts-bedrock/test/LivenessModule.t.sol @@ -33,6 +33,7 @@ contract LivenessModule_TestInit is Test, SafeTestTools { event SignersRecorded(bytes32 indexed txHash, address[] signers); + uint256 initTime = 10; uint256 livenessInterval = 30 days; uint256 minOwners = 6; LivenessModule livenessModule; @@ -86,6 +87,10 @@ contract LivenessModule_TestInit is Test, SafeTestTools { /// @dev Sets up the test environment function setUp() public { + // Set the block timestamp to the initTime, so that signatures recorded in the first block + // are non-zero. + vm.warp(initTime); + // Create a Safe with 10 owners (, uint256[] memory keys) = makeAddrsAndKeys(10); safeInstance = _setupSafe(keys, 8); @@ -107,7 +112,7 @@ contract LivenessModule_TestInit is Test, SafeTestTools { contract LivenessModule_Constructor_Test is LivenessModule_TestInit { /// @dev Tests that the constructor fails if the minOwners is greater than the number of owners - function test_constructor_minOwnersGreaterThanOwners_revert() external { + function test_constructor_minOwnersGreaterThanOwners_reverts() external { vm.expectRevert("LivenessModule: minOwners must be less than the number of owners"); new LivenessModule({ _safe: safeInstance.safe, @@ -117,6 +122,22 @@ contract LivenessModule_Constructor_Test is LivenessModule_TestInit { _fallbackOwner: address(0) }); } + + /// @dev Tests that the constructor fails if the minOwners is greater than the number of owners + function test_constructor_wrongThreshold_reverts() external { + uint256 wrongThreshold = livenessModule.get75PercentThreshold(safeInstance.owners.length) + 1; + vm.mockCall( + address(safeInstance.safe), abi.encodeCall(OwnerManager.getThreshold, ()), abi.encode(wrongThreshold) + ); + vm.expectRevert("LivenessModule: Safe must have a threshold of 75% of the number of owners"); + new LivenessModule({ + _safe: safeInstance.safe, + _livenessGuard: livenessGuard, + _livenessInterval: livenessInterval, + _minOwners: minOwners, + _fallbackOwner: address(0) + }); + } } contract LivenessModule_Getters_Test is LivenessModule_TestInit { @@ -157,11 +178,91 @@ contract LivenessModule_Get75PercentThreshold_Test is LivenessModule_TestInit { } } -contract LivenessModule_RemoveOwner_TestFail is LivenessModule_TestInit { +contract LivenessModule_RemoveOwners_TestFail is LivenessModule_TestInit { using SafeTestLib for SafeInstance; - // "LivenessModule: guard has been changed" - function test_removeOwner_guardChanged_revert() external { + /// @dev Tests with different length owner arrays + function test_removeOwners_differentArrayLengths_reverts() external { + address[] memory ownersToRemove = new address[](1); + address[] memory prevOwners = new address[](2); + vm.expectRevert("LivenessModule: arrays must be the same length"); + livenessModule.removeOwners(prevOwners, ownersToRemove); + } + + /// @dev Test removing an owner which has recently signed a transaction + function test_removeOwners_ownerHasSignedRecently_reverts() external { + /// Will sign a transaction with the first M owners in the owners list + vm.warp(block.timestamp + livenessInterval); + safeInstance.execTransaction({ to: address(1111), value: 0, data: hex"abba" }); + vm.expectRevert( + "LivenessModule: the safe still has sufficient owners, or the owner to remove has signed recently" + ); + _removeAnOwner(safeInstance.owners[0]); + } + + /// @dev Test removing an owner which has recently called showLiveness + function test_removeOwners_ownerHasShownLivenessRecently_reverts() external { + /// Will sign a transaction with the first M owners in the owners list + vm.warp(block.timestamp + livenessInterval); + vm.prank(safeInstance.owners[0]); + livenessGuard.showLiveness(); + vm.expectRevert( + "LivenessModule: the safe still has sufficient owners, or the owner to remove has signed recently" + ); + _removeAnOwner(safeInstance.owners[0]); + } + + /// @dev Test removing an owner with an incorrect previous owner + function test_removeOwners_wrongPreviousOwner_reverts() external { + address[] memory prevOwners = new address[](1); + address[] memory ownersToRemove = new address[](1); + ownersToRemove[0] = safeInstance.owners[0]; + prevOwners[0] = ownersToRemove[0]; // incorrect. + + vm.warp(block.timestamp + livenessInterval); + vm.expectRevert("LivenessModule: failed to remove owner"); + livenessModule.removeOwners(prevOwners, ownersToRemove); + } + + /// @dev Tests if removing all owners works correctly + function test_removeOwners_swapToFallBackOwner_reverts() external { + uint256 numOwners = safeInstance.owners.length; + + address[] memory ownersToRemove = new address[](numOwners); + for (uint256 i = 0; i < numOwners; i++) { + ownersToRemove[i] = safeInstance.owners[i]; + } + address[] memory prevOwners = _getPrevOwners(ownersToRemove); + + // Incorrectly set the final owner to address(0) + ownersToRemove[ownersToRemove.length - 1] = address(0); + + vm.warp(block.timestamp + livenessInterval); + vm.expectRevert("LivenessModule: failed to swap to fallback owner"); + livenessModule.removeOwners(prevOwners, ownersToRemove); + } + + /// @dev Tests if remove owners reverts if it removes too many owners without swapping to the fallback owner + function test_removeOwners_belowMinButNotToFallbackOwner_reverts() external { + // Remove all but one owner + uint256 numOwners = safeInstance.owners.length - 1; + + address[] memory ownersToRemove = new address[](numOwners); + for (uint256 i = 0; i < numOwners; i++) { + ownersToRemove[i] = safeInstance.owners[i]; + } + address[] memory prevOwners = _getPrevOwners(ownersToRemove); + + vm.warp(block.timestamp + livenessInterval); + vm.expectRevert( + "LivenessModule: Safe must have the minimum number of owners or be owned solely by the fallback owner" + ); + livenessModule.removeOwners(prevOwners, ownersToRemove); + } + + /// @dev Tests if remove owners reverts if the current Safe.guard does note match the expected + /// livenessGuard address. + function test_removeOwners_guardChanged_reverts() external { address[] memory ownersToRemove = new address[](1); ownersToRemove[0] = safeInstance.owners[0]; address[] memory prevOwners = _getPrevOwners(ownersToRemove); @@ -174,15 +275,29 @@ contract LivenessModule_RemoveOwner_TestFail is LivenessModule_TestInit { vm.expectRevert("LivenessModule: guard has been changed"); livenessModule.removeOwners(prevOwners, ownersToRemove); } + + function test_removeOwners_invalidThreshold_reverts() external { + address[] memory ownersToRemove = new address[](0); + address[] memory prevOwners = new address[](0); + uint256 wrongThreshold = safeInstance.safe.getThreshold() + 1; + + vm.mockCall( + address(safeInstance.safe), abi.encodeCall(OwnerManager.getThreshold, ()), abi.encode(wrongThreshold) + ); + + vm.warp(block.timestamp + livenessInterval + 1); + vm.expectRevert("LivenessModule: Safe must have a threshold of 75% of the number of owners"); + livenessModule.removeOwners(prevOwners, ownersToRemove); + } } -contract LivenessModule_RemoveOwner_Test is LivenessModule_TestInit { +contract LivenessModule_RemoveOwners_Test is LivenessModule_TestInit { /// @dev Tests if removing one owner works correctly - function test_removeOwner_oneOwner_succeeds() external { + function test_removeOwners_oneOwner_succeeds() external { uint256 ownersBefore = safeInstance.owners.length; address ownerToRemove = safeInstance.owners[0]; - // vm.warp(block.timestamp + 30 days); + vm.warp(block.timestamp + livenessInterval + 1); _removeAnOwner(ownerToRemove); assertFalse(safeInstance.safe.isOwner(ownerToRemove)); @@ -190,7 +305,7 @@ contract LivenessModule_RemoveOwner_Test is LivenessModule_TestInit { } /// @dev Tests if removing all owners works correctly - function test_removeOwner_allOwners_succeeds() external { + function test_removeOwners_allOwners_succeeds() external { uint256 numOwners = safeInstance.owners.length; address[] memory ownersToRemove = new address[](numOwners); @@ -199,7 +314,7 @@ contract LivenessModule_RemoveOwner_Test is LivenessModule_TestInit { } address[] memory prevOwners = _getPrevOwners(ownersToRemove); - vm.warp(block.timestamp + 30 days); + vm.warp(block.timestamp + livenessInterval + 1); livenessModule.removeOwners(prevOwners, ownersToRemove); assertEq(safeInstance.safe.getOwners().length, 1); assertEq(safeInstance.safe.getOwners()[0], fallbackOwner); From a51e82f39657f3eeb01ffdbced3af6f759797402 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Thu, 26 Oct 2023 01:17:59 -0400 Subject: [PATCH 236/279] feat(ctb): Add constructor check on threshold --- packages/contracts-bedrock/src/Safe/LivenessModule.sol | 7 +++++-- packages/contracts-bedrock/test/LivenessModule.t.sol | 6 +++--- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessModule.sol b/packages/contracts-bedrock/src/Safe/LivenessModule.sol index 859e2aa178f1..ed4f5d579de5 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessModule.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessModule.sol @@ -54,8 +54,11 @@ contract LivenessModule is ISemver { LIVENESS_INTERVAL = _livenessInterval; FALLBACK_OWNER = _fallbackOwner; MIN_OWNERS = _minOwners; + address[] memory owners = _safe.getOwners(); + require(_minOwners < owners.length, "LivenessModule: minOwners must be less than the number of owners"); require( - _minOwners < _safe.getOwners().length, "LivenessModule: minOwners must be less than the number of owners" + _safe.getThreshold() == get75PercentThreshold(owners.length), + "LivenessModule: Safe must have a threshold of 75% of the number of owners" ); } @@ -161,7 +164,7 @@ contract LivenessModule is ISemver { uint256 threshold = SAFE.getThreshold(); require( threshold == get75PercentThreshold(numOwners), - "LivenessModule: threshold must be 75% of the number of owners" + "LivenessModule: Safe must have a threshold of 75% of the number of owners" ); // Check that the guard has not been changed diff --git a/packages/contracts-bedrock/test/LivenessModule.t.sol b/packages/contracts-bedrock/test/LivenessModule.t.sol index e4205f0796ce..eeef3734d046 100644 --- a/packages/contracts-bedrock/test/LivenessModule.t.sol +++ b/packages/contracts-bedrock/test/LivenessModule.t.sol @@ -219,7 +219,7 @@ contract LivenessModule_RemoveOwners_TestFail is LivenessModule_TestInit { ownersToRemove[0] = safeInstance.owners[0]; prevOwners[0] = ownersToRemove[0]; // incorrect. - vm.warp(block.timestamp + livenessInterval); + vm.warp(block.timestamp + livenessInterval + 1); vm.expectRevert("LivenessModule: failed to remove owner"); livenessModule.removeOwners(prevOwners, ownersToRemove); } @@ -237,7 +237,7 @@ contract LivenessModule_RemoveOwners_TestFail is LivenessModule_TestInit { // Incorrectly set the final owner to address(0) ownersToRemove[ownersToRemove.length - 1] = address(0); - vm.warp(block.timestamp + livenessInterval); + vm.warp(block.timestamp + livenessInterval + 1); vm.expectRevert("LivenessModule: failed to swap to fallback owner"); livenessModule.removeOwners(prevOwners, ownersToRemove); } @@ -253,7 +253,7 @@ contract LivenessModule_RemoveOwners_TestFail is LivenessModule_TestInit { } address[] memory prevOwners = _getPrevOwners(ownersToRemove); - vm.warp(block.timestamp + livenessInterval); + vm.warp(block.timestamp + livenessInterval + 1); vm.expectRevert( "LivenessModule: Safe must have the minimum number of owners or be owned solely by the fallback owner" ); From 854db4392313f28417bedd4b5fd02e015a6dd82d Mon Sep 17 00:00:00 2001 From: Maurelian Date: Thu, 26 Oct 2023 01:52:15 -0400 Subject: [PATCH 237/279] feat(ctb): LivenessGuard record current owners in constructor --- .../contracts-bedrock/src/Safe/LivenessGuard.sol | 6 ++++++ packages/contracts-bedrock/test/LivenessGuard.t.sol | 12 ++++++++++++ 2 files changed, 18 insertions(+) diff --git a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol index 30825b8c1191..802c98dfed0f 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol @@ -45,6 +45,12 @@ contract LivenessGuard is ISemver, BaseGuard { /// @param _safe The safe account for which this contract will be the guard. constructor(Safe _safe) { SAFE = _safe; + address[] memory owners = _safe.getOwners(); + for (uint256 i = 0; i < owners.length; i++) { + address owner = owners[i]; + lastLive[owner] = block.timestamp; + emit OwnerRecorded(0x0, owner); + } } /// @notice Internal function to ensure that only the Safe can call certain functions. diff --git a/packages/contracts-bedrock/test/LivenessGuard.t.sol b/packages/contracts-bedrock/test/LivenessGuard.t.sol index 89531ba42dd1..6dc9b06393be 100644 --- a/packages/contracts-bedrock/test/LivenessGuard.t.sol +++ b/packages/contracts-bedrock/test/LivenessGuard.t.sol @@ -15,17 +15,29 @@ contract LivenessGuard_TestInit is Test, SafeTestTools { event OwnerRecorded(bytes32 indexed txHash, address signer); + uint256 initTime = 10; LivenessGuard livenessGuard; SafeInstance safeInstance; /// @dev Sets up the test environment function setUp() public { + vm.warp(initTime); safeInstance = _setupSafe(); livenessGuard = new LivenessGuard(safeInstance.safe); safeInstance.setGuard(address(livenessGuard)); } } +contract LivenessGuard_Constructor_Test is LivenessGuard_TestInit { + function test_constructor_works() external { + address[] memory owners = safeInstance.owners; + livenessGuard = new LivenessGuard(safeInstance.safe); + for (uint256 i = 0; i < owners.length; i++) { + assertEq(livenessGuard.lastLive(owners[i]), initTime); + } + } +} + contract LivenessGuard_Getters_Test is LivenessGuard_TestInit { /// @dev Tests that the getters return the correct values function test_getters_works() external { From 57cfa4d351b50abd73e11c5946db0d8104587b78 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Thu, 26 Oct 2023 02:18:43 -0400 Subject: [PATCH 238/279] feat(ctb): Address feedback --- .../src/Safe/LivenessGuard.sol | 8 +++++--- .../contracts-bedrock/test/LivenessGuard.t.sol | 1 + .../contracts-bedrock/test/LivenessModule.t.sol | 17 ++++++----------- .../contracts-bedrock/test/SafeSigners.t.sol | 1 + 4 files changed, 13 insertions(+), 14 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol index 802c98dfed0f..97ced936b42f 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol @@ -81,7 +81,8 @@ contract LivenessGuard is ISemver, BaseGuard { // Cache the set of owners prior to execution. // This will be used in the checkAfterExecution method. address[] memory owners = SAFE.getOwners(); - for (uint256 i = 0; i < owners.length; i++) { + uint256 ownersLength = owners.length; + for (uint256 i = 0; i < ownersLength; i++) { ownersBefore.add(owners[i]); } @@ -133,8 +134,9 @@ contract LivenessGuard is ISemver, BaseGuard { } // Now iterate over the remaining ownersBefore entries. Any remaining addresses are no longer an owner, so we // delete them from the lastLive mapping. - for (uint256 j = 0; j < ownersBefore.length(); j++) { - address ownerBefore = ownersBefore.at(j); + uint256 ownersBeforeLength = ownersBefore.length(); + for (uint256 i = 0; i < ownersBeforeLength; i++) { + address ownerBefore = ownersBefore.at(i); delete lastLive[ownerBefore]; } } diff --git a/packages/contracts-bedrock/test/LivenessGuard.t.sol b/packages/contracts-bedrock/test/LivenessGuard.t.sol index 6dc9b06393be..671a12bcb7ba 100644 --- a/packages/contracts-bedrock/test/LivenessGuard.t.sol +++ b/packages/contracts-bedrock/test/LivenessGuard.t.sol @@ -29,6 +29,7 @@ contract LivenessGuard_TestInit is Test, SafeTestTools { } contract LivenessGuard_Constructor_Test is LivenessGuard_TestInit { + /// @dev Tests that the constructor correctly sets the current time as the lastLive time for each owner function test_constructor_works() external { address[] memory owners = safeInstance.owners; livenessGuard = new LivenessGuard(safeInstance.safe); diff --git a/packages/contracts-bedrock/test/LivenessModule.t.sol b/packages/contracts-bedrock/test/LivenessModule.t.sol index eeef3734d046..7a39c774bf37 100644 --- a/packages/contracts-bedrock/test/LivenessModule.t.sol +++ b/packages/contracts-bedrock/test/LivenessModule.t.sol @@ -20,6 +20,7 @@ contract OwnerSimulator is OwnerManager { setupOwners(_owners, _threshold); } + /// @dev Exposes the OwnerManager's removeOwner function so that anyone may call without needing auth function removeOwnerWrapped(address prevOwner, address owner, uint256 _threshold) public { OwnerManager(address(this)).removeOwner(prevOwner, owner, _threshold); } @@ -194,9 +195,7 @@ contract LivenessModule_RemoveOwners_TestFail is LivenessModule_TestInit { /// Will sign a transaction with the first M owners in the owners list vm.warp(block.timestamp + livenessInterval); safeInstance.execTransaction({ to: address(1111), value: 0, data: hex"abba" }); - vm.expectRevert( - "LivenessModule: the safe still has sufficient owners, or the owner to remove has signed recently" - ); + vm.expectRevert("LivenessModule: the owner to remove has signed recently"); _removeAnOwner(safeInstance.owners[0]); } @@ -206,9 +205,7 @@ contract LivenessModule_RemoveOwners_TestFail is LivenessModule_TestInit { vm.warp(block.timestamp + livenessInterval); vm.prank(safeInstance.owners[0]); livenessGuard.showLiveness(); - vm.expectRevert( - "LivenessModule: the safe still has sufficient owners, or the owner to remove has signed recently" - ); + vm.expectRevert("LivenessModule: the owner to remove has signed recently"); _removeAnOwner(safeInstance.owners[0]); } @@ -242,8 +239,8 @@ contract LivenessModule_RemoveOwners_TestFail is LivenessModule_TestInit { livenessModule.removeOwners(prevOwners, ownersToRemove); } - /// @dev Tests if remove owners reverts if it removes too many owners without swapping to the fallback owner - function test_removeOwners_belowMinButNotToFallbackOwner_reverts() external { + /// @dev Tests if remove owners reverts if it removes too many owners without removing all of them + function test_removeOwners_belowMinButNotEmptied_reverts() external { // Remove all but one owner uint256 numOwners = safeInstance.owners.length - 1; @@ -254,9 +251,7 @@ contract LivenessModule_RemoveOwners_TestFail is LivenessModule_TestInit { address[] memory prevOwners = _getPrevOwners(ownersToRemove); vm.warp(block.timestamp + livenessInterval + 1); - vm.expectRevert( - "LivenessModule: Safe must have the minimum number of owners or be owned solely by the fallback owner" - ); + vm.expectRevert("LivenessModule: must transfer ownership to fallback owner"); livenessModule.removeOwners(prevOwners, ownersToRemove); } diff --git a/packages/contracts-bedrock/test/SafeSigners.t.sol b/packages/contracts-bedrock/test/SafeSigners.t.sol index a9edbb1ca52e..8ad92d09af65 100644 --- a/packages/contracts-bedrock/test/SafeSigners.t.sol +++ b/packages/contracts-bedrock/test/SafeSigners.t.sol @@ -21,6 +21,7 @@ contract SafeSigners_Test is Test, SafeTestTools { /// @dev Maps every key to one of the 4 signatures types. /// This is used in the tests below as a pseudorandom mechanism for determining which /// signature type to use for each key. + /// @param _key The key to map to a signature type. function sigType(uint256 _key) internal pure returns (SigTypes sigType_) { uint256 t = _key % 4; sigType_ = SigTypes(t); From 0cd44702819182121a029f5301e721f8744054d8 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Thu, 26 Oct 2023 02:46:28 -0400 Subject: [PATCH 239/279] specs: Define deployment order for liveness contracts --- packages/contracts-bedrock/test/LivenessModule.t.sol | 2 +- specs/safe-liveness-checking.md | 11 +++++++++++ 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/packages/contracts-bedrock/test/LivenessModule.t.sol b/packages/contracts-bedrock/test/LivenessModule.t.sol index 7a39c774bf37..a872b52e4c17 100644 --- a/packages/contracts-bedrock/test/LivenessModule.t.sol +++ b/packages/contracts-bedrock/test/LivenessModule.t.sol @@ -106,8 +106,8 @@ contract LivenessModule_TestInit is Test, SafeTestTools { _minOwners: minOwners, _fallbackOwner: fallbackOwner }); - safeInstance.enableModule(address(livenessModule)); safeInstance.setGuard(address(livenessGuard)); + safeInstance.enableModule(address(livenessModule)); } } diff --git a/specs/safe-liveness-checking.md b/specs/safe-liveness-checking.md index 86cb4054a9d0..69ec6462dc64 100644 --- a/specs/safe-liveness-checking.md +++ b/specs/safe-liveness-checking.md @@ -86,3 +86,14 @@ The following security properties must be upheld: 5. The module implements the correct checks prior to removing a signer. 6. The module sets the correct threshold upon removing a signer. 7. During a shutdown the module correctly removes all signers, and converts the safe to a 1 of 1. +### Deployment + +The module are guard are intended to be deployed and installed on the safe in the following sequence: + +1. Deploy the guard contract, this will set a timestamp for each existing owner on the Safe. +1. Deploy the module. +1. Enable the module on the safe. +1. Set the guard on the safe. + +This order of operations is necessary to satisfy the constructor checks in the module, and is +intended to prevent owners from being immediately removable. From 7b82cba7147f0fe02e2fe8c4842c8a40ab2f6c7f Mon Sep 17 00:00:00 2001 From: Maurelian Date: Thu, 26 Oct 2023 02:50:21 -0400 Subject: [PATCH 240/279] specs: Add diagram and other details --- packages/contracts-bedrock/.gas-snapshot | 24 ++++++++++--- specs/safe-liveness-checking.md | 46 +++++++++++++++++------- 2 files changed, 53 insertions(+), 17 deletions(-) diff --git a/packages/contracts-bedrock/.gas-snapshot b/packages/contracts-bedrock/.gas-snapshot index 67e01fb5ddb4..3c874406e284 100644 --- a/packages/contracts-bedrock/.gas-snapshot +++ b/packages/contracts-bedrock/.gas-snapshot @@ -305,11 +305,27 @@ LegacyERC20ETH_Test:test_transferFrom_doesNotExist_reverts() (gas: 12957) LegacyERC20ETH_Test:test_transfer_doesNotExist_reverts() (gas: 10755) LegacyMessagePasser_Test:test_passMessageToL1_succeeds() (gas: 34524) LibPosition_Test:test_pos_correctness_succeeds() (gas: 38689) -LivenessGuard_ShowLiveness_Test:test_showLiveness_succeeds() (gas: 51339) +LivenessGuard_CheckAfterExecution_TestFails:test_checkAfterExecution_callerIsNotSafe_revert() (gas: 8553) +LivenessGuard_CheckTx_Test:test_checkTransaction_succeeds() (gas: 234784) +LivenessGuard_CheckTx_TestFails:test_checkTransaction_callerIsNotSafe_revert() (gas: 10343) +LivenessGuard_Constructor_Test:test_constructor_works() (gas: 1163577) +LivenessGuard_Getters_Test:test_getters_works() (gas: 10662) +LivenessGuard_ShowLiveness_Test:test_showLiveness_succeeds() (gas: 29584) +LivenessGuard_ShowLiveness_TestFail:test_showLiveness_callIsNotSafeOwner_reverts() (gas: 18770) +LivenessModule_Constructor_Test:test_constructor_minOwnersGreaterThanOwners_reverts() (gas: 83531) +LivenessModule_Constructor_Test:test_constructor_wrongThreshold_reverts() (gas: 92808) LivenessModule_Get75PercentThreshold_Test:test_get75PercentThreshold_Works() (gas: 26339) -LivenessModule_RemoveOwner_Test:test_removeOwner_allOwners_succeeds() (gas: 159764) -LivenessModule_RemoveOwner_Test:test_removeOwner_oneOwner_succeeds() (gas: 109028) -LivnessGuard_CheckTx_Test:test_checkTransaction_succeeds() (gas: 160454) +LivenessModule_Getters_Test:test_getters_works() (gas: 14807) +LivenessModule_RemoveOwners_Test:test_removeOwners_allOwners_succeeds() (gas: 346743) +LivenessModule_RemoveOwners_Test:test_removeOwners_oneOwner_succeeds() (gas: 127161) +LivenessModule_RemoveOwners_TestFail:test_removeOwners_belowMinButNotEmptied_reverts() (gas: 301067) +LivenessModule_RemoveOwners_TestFail:test_removeOwners_differentArrayLengths_reverts() (gas: 10536) +LivenessModule_RemoveOwners_TestFail:test_removeOwners_guardChanged_reverts() (gas: 1741747) +LivenessModule_RemoveOwners_TestFail:test_removeOwners_invalidThreshold_reverts() (gas: 67238) +LivenessModule_RemoveOwners_TestFail:test_removeOwners_ownerHasShownLivenessRecently_reverts() (gas: 91661) +LivenessModule_RemoveOwners_TestFail:test_removeOwners_ownerHasSignedRecently_reverts() (gas: 638440) +LivenessModule_RemoveOwners_TestFail:test_removeOwners_swapToFallBackOwner_reverts() (gas: 310105) +LivenessModule_RemoveOwners_TestFail:test_removeOwners_wrongPreviousOwner_reverts() (gas: 70465) MIPS_Test:test_add_succeeds() (gas: 122932) MIPS_Test:test_addiSign_succeeds() (gas: 122923) MIPS_Test:test_addi_succeeds() (gas: 123120) diff --git a/specs/safe-liveness-checking.md b/specs/safe-liveness-checking.md index 69ec6462dc64..aa28ea2d10d1 100644 --- a/specs/safe-liveness-checking.md +++ b/specs/safe-liveness-checking.md @@ -8,8 +8,11 @@ - [Liveness checking methodology](#liveness-checking-methodology) - [The Liveness Guard](#the-liveness-guard) - [The Liveness Module](#the-liveness-module) + - [Owner removal call flow](#owner-removal-call-flow) - [Shutdown](#shutdown) - [Security Properties](#security-properties) + - [Interdependency between the Guard and Module](#interdependency-between-the-guard-and-module) + - [Deployment](#deployment) @@ -19,6 +22,11 @@ The Security Security Council uses a specially extended Safe multisig contract t any loss of access to a signer's keys is identified and addressed within a predictable period of time. +This mechanism is intended only to be used to remove signers who have lost access to their keys, or +are otherwise inactive. It is not intended to be used to remove signers who are acting in bad faith, +or any other subjective criteria, such cases should be addressed by governance, and the removal +handled via the standard Safe ownership management functionality. + ## Liveness checking methodology This is achieved using two types of contracts which the Safe contract has built-in support for: @@ -51,7 +59,8 @@ A `LivenessModule` is also created which does the following: ### Owner removal call flow -The following diagram illustrates the flow for removing a single owner. +The following diagram illustrates the flow for removing a single owner. The `verifyFinalState` +box indicates calls to the Safe which ensure the final state is valid. ```mermaid sequenceDiagram @@ -59,18 +68,21 @@ sequenceDiagram participant LivenessModule participant LivenessGuard participant Safe - User->>LivenessModule: removeOwner(owner) + User->>LivenessModule: removeOwners([previousOwner], [owner]) LivenessModule->>LivenessGuard: lastLive(owner) LivenessModule->>Safe: getOwners() - LivenessModule->>LivenessModule: get75PercentThreshold(numOwnersAfter) - LivenessModule->>LivenessModule: _getPrevOwner(owner, owners) - LivenessModule->>LivenessModule: _removeOwner(prevOwner, owner, thresholdAfter) - LivenessModule->>LivenessModule: _verifyFinalState() + LivenessModule->>Safe: removeOwner(previousOwner, owner) + + alt verifyFinalState + LivenessModule->>Safe: getOwners() + LivenessModule->>Safe: getThreshold() + LivenessModule->>Safe: getGuard() + end ``` ### Shutdown -In the unlikely event that the signer set (`N`) is reduced below 8, then (and only then) is a +In the unlikely event that the signer set (`N`) is reduced below the allowed threshold, then (and only then) is a shutdown mechanism activated which removes the existing signers, and hands control of the multisig over to a predetermined entity. @@ -79,13 +91,21 @@ In the unlikely event that the signer set (`N`) is reduced below 8, then (and on The following security properties must be upheld: 1. Signatures are assigned to the correct signer. -2. Non-signers are unable to create a record of having signed. -3. A signer cannot be censored or griefed such that their signing is not recorded. -4. Signers may demonstrate liveness either by signing a transaction or by calling directly to the +1. Non-signers are unable to create a record of having signed. +1. A signer cannot be censored or grieffed such that their signing is not recorded. +1. Signers may demonstrate liveness either by signing a transaction or by calling directly to the guard. -5. The module implements the correct checks prior to removing a signer. -6. The module sets the correct threshold upon removing a signer. -7. During a shutdown the module correctly removes all signers, and converts the safe to a 1 of 1. +1. The module only removes a signer if they have demonstrated liveness during the interval, or + if necessary to convert the safe to a 1 of 1. +1. The module sets the correct 75% threshold upon removing a signer. +1. During a shutdown the module correctly removes all signers, and converts the safe to a 1 of 1. + +### Interdependency between the Guard and Module + +The Guard has no dependency on the Module, and can be used independently to track liveness of +Safe owners. The Module however does have a dependency on the Guard, only one guard contract can +be set on the Safe, and the Module will be unable to function if the Guard is removed. + ### Deployment The module are guard are intended to be deployed and installed on the safe in the following sequence: From f7ecd62be397ccd4283a5f14bf0c73c8670ef8e2 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Thu, 26 Oct 2023 09:54:25 -0400 Subject: [PATCH 241/279] codecov: Ignore all files in contracts-bedrock/test --- codecov.yml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/codecov.yml b/codecov.yml index 953756f48752..f64112c9cf1c 100644 --- a/codecov.yml +++ b/codecov.yml @@ -9,8 +9,9 @@ comment: ignore: - "op-e2e" - - "**/*.t.sol" - "op-bindings/bindings/*.go" + - "**/*.t.sol" + - "packages/contracts-bedrock/test/**/*.sol" - "packages/contracts-bedrock/contracts/vendor/WETH9.sol" - 'packages/contracts-bedrock/contracts/EAS/**/*.sol' coverage: From d52cc302125259a7676cc07d22dee25ccfa06029 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Thu, 26 Oct 2023 11:01:49 -0400 Subject: [PATCH 242/279] test(ctb): Add Guard owner management tests --- .../test/LivenessGuard.t.sol | 38 ++++++++++++++++++- 1 file changed, 36 insertions(+), 2 deletions(-) diff --git a/packages/contracts-bedrock/test/LivenessGuard.t.sol b/packages/contracts-bedrock/test/LivenessGuard.t.sol index 671a12bcb7ba..cf21c0f2f2ac 100644 --- a/packages/contracts-bedrock/test/LivenessGuard.t.sol +++ b/packages/contracts-bedrock/test/LivenessGuard.t.sol @@ -101,8 +101,6 @@ contract LivenessGuard_CheckAfterExecution_TestFails is LivenessGuard_TestInit { } } -contract LivenessGuard_CheckAfterExecution_Test is LivenessGuard_TestInit { } - contract LivenessGuard_ShowLiveness_TestFail is LivenessGuard_TestInit { /// @dev Tests that the showLiveness function reverts if the caller is not an owner function test_showLiveness_callIsNotSafeOwner_reverts() external { @@ -126,3 +124,39 @@ contract LivenessGuard_ShowLiveness_Test is LivenessGuard_TestInit { assertEq(livenessGuard.lastLive(caller), block.timestamp); } } + +contract LivenessGuard_OwnerManagement_Test is LivenessGuard_TestInit { + using SafeTestLib for SafeInstance; + + /// @dev Tests that the guard correctly deletes the owner from the lastLive mapping when it is removed + function test_removeOwner_succeeds() external { + address ownerToRemove = safeInstance.owners[0]; + assertGe(livenessGuard.lastLive(ownerToRemove), 0); + assertTrue(safeInstance.safe.isOwner(ownerToRemove)); + + safeInstance.execTransaction({ + to: address(safeInstance.safe), + value: 0, + data: abi.encodeWithSelector(OwnerManager.removeOwner.selector, SENTINEL_OWNERS, ownerToRemove, 1) + }); + + assertFalse(safeInstance.safe.isOwner(ownerToRemove)); + assertEq(livenessGuard.lastLive(ownerToRemove), 0); + } + + /// @dev Tests that the guard correctly adds an owner to the lastLive mapping when it is added + function test_addOwner_succeeds() external { + address ownerToAdd = makeAddr("new owner"); + assertEq(livenessGuard.lastLive(ownerToAdd), 0); + assertFalse(safeInstance.safe.isOwner(ownerToAdd)); + + safeInstance.execTransaction({ + to: address(safeInstance.safe), + value: 0, + data: abi.encodeWithSelector(OwnerManager.addOwnerWithThreshold.selector, ownerToAdd, 1) + }); + + assertTrue(safeInstance.safe.isOwner(ownerToAdd)); + assertEq(livenessGuard.lastLive(ownerToAdd), block.timestamp); + } +} From 30a444092c5905aebe512c2993b92a08103d30b3 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Thu, 26 Oct 2023 11:17:12 -0400 Subject: [PATCH 243/279] safe-tools: Refactor getPrevOwner() into lib Also make lib functions internal which removes the deploy and linking with solidity libs. --- .../test/LivenessModule.t.sol | 21 +---------- .../test/safe-tools/SafeTestTools.sol | 37 ++++++++++++++----- 2 files changed, 29 insertions(+), 29 deletions(-) diff --git a/packages/contracts-bedrock/test/LivenessModule.t.sol b/packages/contracts-bedrock/test/LivenessModule.t.sol index a872b52e4c17..f4783dbb41d1 100644 --- a/packages/contracts-bedrock/test/LivenessModule.t.sol +++ b/packages/contracts-bedrock/test/LivenessModule.t.sol @@ -29,9 +29,6 @@ contract OwnerSimulator is OwnerManager { contract LivenessModule_TestInit is Test, SafeTestTools { using SafeTestLib for SafeInstance; - /// @dev The address of the first owner in the linked list of owners - address internal constant SENTINEL_OWNERS = address(0x1); - event SignersRecorded(bytes32 indexed txHash, address[] signers); uint256 initTime = 10; @@ -43,20 +40,6 @@ contract LivenessModule_TestInit is Test, SafeTestTools { OwnerSimulator ownerSimulator; address fallbackOwner; - /// @notice Get the previous owner in the linked list of owners - /// @param _owner The owner whose previous owner we want to find - /// @param _owners The list of owners - function _getPrevOwner(address _owner, address[] memory _owners) internal pure returns (address prevOwner_) { - for (uint256 i = 0; i < _owners.length; i++) { - if (_owners[i] != _owner) continue; - if (i == 0) { - prevOwner_ = SENTINEL_OWNERS; - break; - } - prevOwner_ = _owners[i - 1]; - } - } - /// @dev Given an array of owners to remove, this function will return an array of the previous owners /// in the order that they must be provided to the LivenessMoules's removeOwners() function. /// Because owners are removed one at a time, and not necessarily in order, we need to simulate @@ -68,7 +51,7 @@ contract LivenessModule_TestInit is Test, SafeTestTools { address[] memory currentOwners; for (uint256 i = 0; i < _ownersToRemove.length; i++) { currentOwners = ownerSimulator.getOwners(); - prevOwners_[i] = _getPrevOwner(safeInstance.owners[i], currentOwners); + prevOwners_[i] = SafeTestLib.getPrevOwner(safeInstance.owners[i], currentOwners); // Don't try to remove the last owner if (currentOwners.length == 1) break; @@ -81,7 +64,7 @@ contract LivenessModule_TestInit is Test, SafeTestTools { address[] memory prevOwners = new address[](1); address[] memory ownersToRemove = new address[](1); ownersToRemove[0] = _ownerToRemove; - prevOwners[0] = _getPrevOwner(_ownerToRemove, safeInstance.owners); + prevOwners[0] = SafeTestLib.getPrevOwner(_ownerToRemove, safeInstance.owners); livenessModule.removeOwners(prevOwners, ownersToRemove); } diff --git a/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol b/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol index acd5cacb0eef..0573a0a3f01f 100644 --- a/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol +++ b/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol @@ -15,6 +15,9 @@ import "./CompatibilityFallbackHandler_1_3_0.sol"; address constant VM_ADDR = 0x7109709ECfa91a80626fF3989D68f67F5b1DD12D; bytes12 constant ADDR_MASK = 0xffffffffffffffffffffffff; +/// @dev The address of the first owner in the linked list of owners +address constant SENTINEL_OWNERS = address(0x1); + /// @dev Get the address from a private key function getAddr(uint256 pk) pure returns (address) { return Vm(VM_ADDR).addr(pk); @@ -145,7 +148,7 @@ library SafeTestLib { address refundReceiver, bytes memory signatures ) - public + internal returns (bool) { if (instance.owners.length == 0) { @@ -207,7 +210,7 @@ library SafeTestLib { bytes memory data, Enum.Operation operation ) - public + internal returns (bool) { return execTransaction(instance, to, value, data, operation, 0, 0, 0, address(0), address(0), ""); @@ -220,14 +223,14 @@ library SafeTestLib { uint256 value, bytes memory data ) - public + internal returns (bool) { return execTransaction(instance, to, value, data, Enum.Operation.Call, 0, 0, 0, address(0), address(0), ""); } /// @dev Enables a module on the Safe. - function enableModule(SafeInstance memory instance, address module) public { + function enableModule(SafeInstance memory instance, address module) internal { execTransaction( instance, address(instance.safe), @@ -244,7 +247,7 @@ library SafeTestLib { } /// @dev Disables a module on the Safe. - function disableModule(SafeInstance memory instance, address module) public { + function disableModule(SafeInstance memory instance, address module) internal { (address[] memory modules,) = instance.safe.getModulesPaginated(SENTINEL_MODULES, 1000); address prevModule = SENTINEL_MODULES; bool moduleFound; @@ -275,7 +278,7 @@ library SafeTestLib { /// @dev Sets the guard address on the Safe. Unlike modules there can only be one guard, so /// this method will remove the previous guard. If the guard is set to the 0 address, the /// guard will be disabled. - function setGuard(SafeInstance memory instance, address guard) public { + function setGuard(SafeInstance memory instance, address guard) internal { execTransaction( instance, address(instance.safe), @@ -292,7 +295,7 @@ library SafeTestLib { } /// @dev Signs message data using EIP1271: Standard Signature Validation Method for Contracts - function EIP1271Sign(SafeInstance memory instance, bytes memory data) public { + function EIP1271Sign(SafeInstance memory instance, bytes memory data) internal { address signMessageLib = address(new SignMessageLib()); execTransaction({ instance: instance, @@ -310,7 +313,7 @@ library SafeTestLib { } /// @dev Signs a data hash using EIP1271: Standard Signature Validation Method for Contracts - function EIP1271Sign(SafeInstance memory instance, bytes32 digest) public { + function EIP1271Sign(SafeInstance memory instance, bytes32 digest) internal { EIP1271Sign(instance, abi.encodePacked(digest)); } @@ -328,7 +331,7 @@ library SafeTestLib { address gasToken, address refundReceiver ) - public + internal view returns (uint8 v, bytes32 r, bytes32 s) { @@ -353,10 +356,24 @@ library SafeTestLib { } /// @dev Increments the nonce of the Safe by sending an empty transaction. - function incrementNonce(SafeInstance memory instance) public returns (uint256 newNonce) { + function incrementNonce(SafeInstance memory instance) internal returns (uint256 newNonce) { execTransaction(instance, address(0), 0, "", Enum.Operation.Call, 0, 0, 0, address(0), address(0), ""); return instance.safe.nonce(); } + + /// @notice Get the previous owner in the linked list of owners + /// @param _owner The owner whose previous owner we want to find + /// @param _owners The list of owners + function getPrevOwner(address _owner, address[] memory _owners) internal pure returns (address prevOwner_) { + for (uint256 i = 0; i < _owners.length; i++) { + if (_owners[i] != _owner) continue; + if (i == 0) { + prevOwner_ = SENTINEL_OWNERS; + break; + } + prevOwner_ = _owners[i - 1]; + } + } } /// @dev SafeTestTools implements a set of helper functions for testing Safe contracts. From 3787e706ef293ff21ef9094a9c8d511738331724 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Thu, 26 Oct 2023 11:29:40 -0400 Subject: [PATCH 244/279] safe-tools: Refactor getPrevOwners (plural) into lib This also move the OwnerSimulator into the lib, which nicely abstracts that away. --- .../test/LivenessModule.t.sol | 45 +++---------------- .../test/safe-tools/SafeTestTools.sol | 44 +++++++++++++++++- 2 files changed, 48 insertions(+), 41 deletions(-) diff --git a/packages/contracts-bedrock/test/LivenessModule.t.sol b/packages/contracts-bedrock/test/LivenessModule.t.sol index f4783dbb41d1..0558bec70803 100644 --- a/packages/contracts-bedrock/test/LivenessModule.t.sol +++ b/packages/contracts-bedrock/test/LivenessModule.t.sol @@ -12,20 +12,6 @@ import "test/safe-tools/SafeTestTools.sol"; import { LivenessModule } from "src/Safe/LivenessModule.sol"; import { LivenessGuard } from "src/Safe/LivenessGuard.sol"; -/// @dev A minimal wrapper around the OwnerManager contract. This contract is meant to be initialized with -/// the same owners as a Safe instance, and then used to simulate the resulting owners list -/// after an owner is removed. -contract OwnerSimulator is OwnerManager { - constructor(address[] memory _owners, uint256 _threshold) { - setupOwners(_owners, _threshold); - } - - /// @dev Exposes the OwnerManager's removeOwner function so that anyone may call without needing auth - function removeOwnerWrapped(address prevOwner, address owner, uint256 _threshold) public { - OwnerManager(address(this)).removeOwner(prevOwner, owner, _threshold); - } -} - contract LivenessModule_TestInit is Test, SafeTestTools { using SafeTestLib for SafeInstance; @@ -37,28 +23,8 @@ contract LivenessModule_TestInit is Test, SafeTestTools { LivenessModule livenessModule; LivenessGuard livenessGuard; SafeInstance safeInstance; - OwnerSimulator ownerSimulator; address fallbackOwner; - /// @dev Given an array of owners to remove, this function will return an array of the previous owners - /// in the order that they must be provided to the LivenessMoules's removeOwners() function. - /// Because owners are removed one at a time, and not necessarily in order, we need to simulate - /// the owners list after each removal, in order to identify the correct previous owner. - /// @param _ownersToRemove The owners to remove - /// @return prevOwners_ The previous owners in the linked list - function _getPrevOwners(address[] memory _ownersToRemove) internal returns (address[] memory prevOwners_) { - prevOwners_ = new address[](_ownersToRemove.length); - address[] memory currentOwners; - for (uint256 i = 0; i < _ownersToRemove.length; i++) { - currentOwners = ownerSimulator.getOwners(); - prevOwners_[i] = SafeTestLib.getPrevOwner(safeInstance.owners[i], currentOwners); - - // Don't try to remove the last owner - if (currentOwners.length == 1) break; - ownerSimulator.removeOwnerWrapped(prevOwners_[i], _ownersToRemove[i], 1); - } - } - /// @dev Removes an owner from the safe function _removeAnOwner(address _ownerToRemove) internal { address[] memory prevOwners = new address[](1); @@ -78,7 +44,6 @@ contract LivenessModule_TestInit is Test, SafeTestTools { // Create a Safe with 10 owners (, uint256[] memory keys) = makeAddrsAndKeys(10); safeInstance = _setupSafe(keys, 8); - ownerSimulator = new OwnerSimulator(safeInstance.owners, 1); livenessGuard = new LivenessGuard(safeInstance.safe); fallbackOwner = makeAddr("fallbackOwner"); @@ -212,7 +177,7 @@ contract LivenessModule_RemoveOwners_TestFail is LivenessModule_TestInit { for (uint256 i = 0; i < numOwners; i++) { ownersToRemove[i] = safeInstance.owners[i]; } - address[] memory prevOwners = _getPrevOwners(ownersToRemove); + address[] memory prevOwners = safeInstance.getPrevOwners(ownersToRemove); // Incorrectly set the final owner to address(0) ownersToRemove[ownersToRemove.length - 1] = address(0); @@ -231,7 +196,7 @@ contract LivenessModule_RemoveOwners_TestFail is LivenessModule_TestInit { for (uint256 i = 0; i < numOwners; i++) { ownersToRemove[i] = safeInstance.owners[i]; } - address[] memory prevOwners = _getPrevOwners(ownersToRemove); + address[] memory prevOwners = safeInstance.getPrevOwners(ownersToRemove); vm.warp(block.timestamp + livenessInterval + 1); vm.expectRevert("LivenessModule: must transfer ownership to fallback owner"); @@ -243,7 +208,7 @@ contract LivenessModule_RemoveOwners_TestFail is LivenessModule_TestInit { function test_removeOwners_guardChanged_reverts() external { address[] memory ownersToRemove = new address[](1); ownersToRemove[0] = safeInstance.owners[0]; - address[] memory prevOwners = _getPrevOwners(ownersToRemove); + address[] memory prevOwners = safeInstance.getPrevOwners(ownersToRemove); // Change the guard livenessGuard = new LivenessGuard(safeInstance.safe); @@ -270,7 +235,9 @@ contract LivenessModule_RemoveOwners_TestFail is LivenessModule_TestInit { } contract LivenessModule_RemoveOwners_Test is LivenessModule_TestInit { + using SafeTestLib for SafeInstance; /// @dev Tests if removing one owner works correctly + function test_removeOwners_oneOwner_succeeds() external { uint256 ownersBefore = safeInstance.owners.length; address ownerToRemove = safeInstance.owners[0]; @@ -290,7 +257,7 @@ contract LivenessModule_RemoveOwners_Test is LivenessModule_TestInit { for (uint256 i = 0; i < numOwners; i++) { ownersToRemove[i] = safeInstance.owners[i]; } - address[] memory prevOwners = _getPrevOwners(ownersToRemove); + address[] memory prevOwners = safeInstance.getPrevOwners(ownersToRemove); vm.warp(block.timestamp + livenessInterval + 1); livenessModule.removeOwners(prevOwners, ownersToRemove); diff --git a/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol b/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol index 0573a0a3f01f..9b78ff7843d1 100644 --- a/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol +++ b/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol @@ -3,7 +3,7 @@ pragma solidity >=0.7.0 <0.9.0; import "forge-std/Test.sol"; import "scripts/libraries/LibSort.sol"; -import { ModuleManager, GuardManager, Safe as GnosisSafe } from "safe-contracts/Safe.sol"; +import { Safe as GnosisSafe, OwnerManager, ModuleManager, GuardManager } from "safe-contracts/Safe.sol"; import { SafeProxyFactory as GnosisSafeProxyFactory } from "safe-contracts/proxies/SafeProxyFactory.sol"; import { Enum } from "safe-contracts/common/Enum.sol"; import { SignMessageLib } from "safe-contracts/libraries/SignMessageLib.sol"; @@ -110,7 +110,21 @@ function sortPKsByComputedAddress(uint256[] memory _pks) pure returns (uint256[] return sortedPKs; } -// collapsed interface that includes comapatibilityfallback handler calls +/// @dev A minimal wrapper around the OwnerManager contract. This contract is meant to be initialized with +/// the same owners as a Safe instance, and then used to simulate the resulting owners list +/// after an owner is removed. +contract OwnerSimulator is OwnerManager { + constructor(address[] memory _owners, uint256 _threshold) { + setupOwners(_owners, _threshold); + } + + /// @dev Exposes the OwnerManager's removeOwner function so that anyone may call without needing auth + function removeOwnerWrapped(address prevOwner, address owner, uint256 _threshold) public { + OwnerManager(address(this)).removeOwner(prevOwner, owner, _threshold); + } +} + +/// @dev collapsed interface that includes comapatibilityfallback handler calls abstract contract DeployedSafe is GnosisSafe, CompatibilityFallbackHandler { } struct AdvancedSafeInitParams { @@ -374,6 +388,32 @@ library SafeTestLib { prevOwner_ = _owners[i - 1]; } } + + /// @dev Given an array of owners to remove, this function will return an array of the previous owners + /// in the order that they must be provided to the LivenessMoules's removeOwners() function. + /// Because owners are removed one at a time, and not necessarily in order, we need to simulate + /// the owners list after each removal, in order to identify the correct previous owner. + /// @param _ownersToRemove The owners to remove + /// @return prevOwners_ The previous owners in the linked list + function getPrevOwners( + SafeInstance memory instance, + address[] memory _ownersToRemove + ) + internal + returns (address[] memory prevOwners_) + { + OwnerSimulator ownerSimulator = new OwnerSimulator(instance.owners, 1); + prevOwners_ = new address[](_ownersToRemove.length); + address[] memory currentOwners; + for (uint256 i = 0; i < _ownersToRemove.length; i++) { + currentOwners = ownerSimulator.getOwners(); + prevOwners_[i] = SafeTestLib.getPrevOwner(instance.owners[i], currentOwners); + + // Don't try to remove the last owner + if (currentOwners.length == 1) break; + ownerSimulator.removeOwnerWrapped(prevOwners_[i], _ownersToRemove[i], 1); + } + } } /// @dev SafeTestTools implements a set of helper functions for testing Safe contracts. From ab4a40cc8e12edeae68e32b7d641b13cc4681558 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Thu, 26 Oct 2023 11:46:31 -0400 Subject: [PATCH 245/279] safe-tools: Refactor to move free functions into SafeTestLib --- .../test/LivenessGuard.t.sol | 2 +- .../test/LivenessModule.t.sol | 2 +- .../contracts-bedrock/test/SafeSigners.t.sol | 11 +- .../test/safe-tools/SafeTestTools.sol | 212 +++++++++--------- 4 files changed, 116 insertions(+), 111 deletions(-) diff --git a/packages/contracts-bedrock/test/LivenessGuard.t.sol b/packages/contracts-bedrock/test/LivenessGuard.t.sol index cf21c0f2f2ac..679604b0db2f 100644 --- a/packages/contracts-bedrock/test/LivenessGuard.t.sol +++ b/packages/contracts-bedrock/test/LivenessGuard.t.sol @@ -137,7 +137,7 @@ contract LivenessGuard_OwnerManagement_Test is LivenessGuard_TestInit { safeInstance.execTransaction({ to: address(safeInstance.safe), value: 0, - data: abi.encodeWithSelector(OwnerManager.removeOwner.selector, SENTINEL_OWNERS, ownerToRemove, 1) + data: abi.encodeWithSelector(OwnerManager.removeOwner.selector, SafeTestLib.SENTINEL_OWNERS, ownerToRemove, 1) }); assertFalse(safeInstance.safe.isOwner(ownerToRemove)); diff --git a/packages/contracts-bedrock/test/LivenessModule.t.sol b/packages/contracts-bedrock/test/LivenessModule.t.sol index 0558bec70803..718dffff06f3 100644 --- a/packages/contracts-bedrock/test/LivenessModule.t.sol +++ b/packages/contracts-bedrock/test/LivenessModule.t.sol @@ -42,7 +42,7 @@ contract LivenessModule_TestInit is Test, SafeTestTools { vm.warp(initTime); // Create a Safe with 10 owners - (, uint256[] memory keys) = makeAddrsAndKeys(10); + (, uint256[] memory keys) = SafeTestLib.makeAddrsAndKeys(10); safeInstance = _setupSafe(keys, 8); livenessGuard = new LivenessGuard(safeInstance.safe); diff --git a/packages/contracts-bedrock/test/SafeSigners.t.sol b/packages/contracts-bedrock/test/SafeSigners.t.sol index 8ad92d09af65..ffada6c96e09 100644 --- a/packages/contracts-bedrock/test/SafeSigners.t.sol +++ b/packages/contracts-bedrock/test/SafeSigners.t.sol @@ -39,10 +39,11 @@ contract SafeSigners_Test is Test, SafeTestTools { // Limit the number of signatures to 25 uint256 numSigs = bound(_numSigs, 1, 25); - (, uint256[] memory keys) = makeAddrsAndKeys(numSigs); + (, uint256[] memory keys) = SafeTestLib.makeAddrsAndKeys(numSigs); for (uint256 i = 0; i < keys.length; i++) { if (sigType(keys[i]) == SigTypes.Contract) { - keys[i] = encodeSmartContractWalletAsPK(decodeSmartContractWalletAsAddress(keys[i])); + keys[i] = + SafeTestLib.encodeSmartContractWalletAsPK(SafeTestLib.decodeSmartContractWalletAsAddress(keys[i])); } } @@ -66,15 +67,15 @@ contract SafeSigners_Test is Test, SafeTestTools { v += 4; signatures = bytes.concat(signatures, abi.encodePacked(r, s, v)); } else if (sigType(pks[i]) == SigTypes.ApprovedHash) { - vm.prank(getAddr(pks[i])); + vm.prank(SafeTestLib.getAddr(pks[i])); safeInstance.safe.approveHash(digest); v = 1; // s is not checked on approved hash signatures, so we can leave it as zero. - r = bytes32(uint256(uint160(getAddr(pks[i])))); + r = bytes32(uint256(uint160(SafeTestLib.getAddr(pks[i])))); signatures = bytes.concat(signatures, abi.encodePacked(r, s, v)); } else if (sigType(pks[i]) == SigTypes.Contract) { contractSigs++; - address addr = decodeSmartContractWalletAsAddress(pks[i]); + address addr = SafeTestLib.decodeSmartContractWalletAsAddress(pks[i]); r = bytes32(uint256(uint160(addr))); vm.mockCall( addr, abi.encodeWithSignature("isValidSignature(bytes,bytes)"), abi.encode(EIP1271_MAGIC_VALUE) diff --git a/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol b/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol index 9b78ff7843d1..c6935afe6d9c 100644 --- a/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol +++ b/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol @@ -11,104 +11,7 @@ import "./CompatibilityFallbackHandler_1_3_0.sol"; // Tools to simplify testing Safe contracts // Author: Colin Nielsen (https://github.com/colinnielsen/safe-tools) - -address constant VM_ADDR = 0x7109709ECfa91a80626fF3989D68f67F5b1DD12D; -bytes12 constant ADDR_MASK = 0xffffffffffffffffffffffff; - -/// @dev The address of the first owner in the linked list of owners -address constant SENTINEL_OWNERS = address(0x1); - -/// @dev Get the address from a private key -function getAddr(uint256 pk) pure returns (address) { - return Vm(VM_ADDR).addr(pk); -} - -/// @dev Get arrays of addresses and private keys. The arrays are sorted by address, and the addresses are labelled -function makeAddrsAndKeys(uint256 num) returns (address[] memory addrs, uint256[] memory keys) { - keys = new uint256[](num); - addrs = new address[](num); - for (uint256 i; i < num; i++) { - uint256 key = uint256(keccak256(abi.encodePacked(i))); - keys[i] = key; - } - - for (uint256 i; i < num; i++) { - addrs[i] = Vm(VM_ADDR).addr(keys[i]); - Vm(VM_ADDR).label(getAddr(keys[i]), string.concat("SAFETEST: Signer ", string(abi.encodePacked(bytes32(i))))); - } -} - -/// @dev Encode a smart contract wallet as a private key -function encodeSmartContractWalletAsPK(address addr) pure returns (uint256 encodedPK) { - assembly { - let addr_b32 := addr - encodedPK := or(addr, ADDR_MASK) - } -} - -/// @dev Decode a smart contract wallet as an address from a private key -function decodeSmartContractWalletAsAddress(uint256 pk) pure returns (address decodedAddr) { - assembly { - let addr := shl(96, pk) - decodedAddr := shr(96, addr) - } -} - -/// @dev Checks if a private key is an encoded smart contract address -function isSmartContractPK(uint256 pk) pure returns (bool isEncoded) { - assembly { - isEncoded := eq(shr(160, pk), shr(160, ADDR_MASK)) - } -} - -library Sort { - /// @dev Sorts an array of addresses in place - function sort(address[] memory arr) public pure returns (address[] memory) { - LibSort.sort(arr); - return arr; - } -} - -/// @dev Sorts an array of private keys by the computed address -/// If the private key is a smart contract wallet, it will be decoded and sorted by the address -function sortPKsByComputedAddress(uint256[] memory _pks) pure returns (uint256[] memory) { - uint256[] memory sortedPKs = new uint256[](_pks.length); - - address[] memory addresses = new address[](_pks.length); - bytes32[2][] memory accounts = new bytes32[2][](_pks.length); - - for (uint256 i; i < _pks.length; i++) { - uint256 pk = _pks[i]; - address signer = getAddr(pk); - if (isSmartContractPK(pk)) { - signer = decodeSmartContractWalletAsAddress(pk); - } - addresses[i] = signer; - accounts[i][0] = bytes32(abi.encode(signer)); - accounts[i][1] = bytes32(pk); - } - - addresses = Sort.sort(addresses); - - uint256 found; - for (uint256 j; j < addresses.length; j++) { - address signer = addresses[j]; - uint256 pk; - for (uint256 k; k < accounts.length; k++) { - if (address(uint160(uint256(accounts[k][0]))) == signer) { - pk = uint256(accounts[k][1]); - found++; - } - } - - sortedPKs[j] = pk; - } - - if (found < _pks.length) { - revert("SAFETESTTOOLS: issue with private key sorting, please open a ticket on github"); - } - return sortedPKs; -} +// With expanded and improved functionality by OP Labs /// @dev A minimal wrapper around the OwnerManager contract. This contract is meant to be initialized with /// the same owners as a Safe instance, and then used to simulate the resulting owners list @@ -146,7 +49,108 @@ struct SafeInstance { DeployedSafe safe; } +library Sort { + /// @dev Sorts an array of addresses in place + function sort(address[] memory arr) public pure returns (address[] memory) { + LibSort.sort(arr); + return arr; + } +} + library SafeTestLib { + /// @dev The address of foundry's VM contract + address constant VM_ADDR = 0x7109709ECfa91a80626fF3989D68f67F5b1DD12D; + /// @dev The address of the first owner in the linked list of owners + address constant SENTINEL_OWNERS = address(0x1); + + /// @dev Get the address from a private key + function getAddr(uint256 pk) internal pure returns (address) { + return Vm(VM_ADDR).addr(pk); + } + + /// @dev Get arrays of addresses and private keys. The arrays are sorted by address, and the addresses are labelled + function makeAddrsAndKeys(uint256 num) internal returns (address[] memory addrs, uint256[] memory keys) { + keys = new uint256[](num); + addrs = new address[](num); + for (uint256 i; i < num; i++) { + uint256 key = uint256(keccak256(abi.encodePacked(i))); + keys[i] = key; + } + + for (uint256 i; i < num; i++) { + addrs[i] = Vm(VM_ADDR).addr(keys[i]); + Vm(VM_ADDR).label( + getAddr(keys[i]), string.concat("SAFETEST: Signer ", string(abi.encodePacked(bytes32(i)))) + ); + } + } + + bytes12 constant ADDR_MASK = 0xffffffffffffffffffffffff; + + /// @dev Encode a smart contract wallet as a private key + function encodeSmartContractWalletAsPK(address addr) internal pure returns (uint256 encodedPK) { + assembly { + let addr_b32 := addr + encodedPK := or(addr, ADDR_MASK) + } + } + + /// @dev Decode a smart contract wallet as an address from a private key + function decodeSmartContractWalletAsAddress(uint256 pk) internal pure returns (address decodedAddr) { + assembly { + let addr := shl(96, pk) + decodedAddr := shr(96, addr) + } + } + + /// @dev Checks if a private key is an encoded smart contract address + function isSmartContractPK(uint256 pk) internal pure returns (bool isEncoded) { + assembly { + isEncoded := eq(shr(160, pk), shr(160, ADDR_MASK)) + } + } + + /// @dev Sorts an array of private keys by the computed address + /// If the private key is a smart contract wallet, it will be decoded and sorted by the address + function sortPKsByComputedAddress(uint256[] memory _pks) internal pure returns (uint256[] memory) { + uint256[] memory sortedPKs = new uint256[](_pks.length); + + address[] memory addresses = new address[](_pks.length); + bytes32[2][] memory accounts = new bytes32[2][](_pks.length); + + for (uint256 i; i < _pks.length; i++) { + uint256 pk = _pks[i]; + address signer = SafeTestLib.getAddr(pk); + if (isSmartContractPK(pk)) { + signer = decodeSmartContractWalletAsAddress(pk); + } + addresses[i] = signer; + accounts[i][0] = bytes32(abi.encode(signer)); + accounts[i][1] = bytes32(pk); + } + + addresses = Sort.sort(addresses); + + uint256 found; + for (uint256 j; j < addresses.length; j++) { + address signer = addresses[j]; + uint256 pk; + for (uint256 k; k < accounts.length; k++) { + if (address(uint160(uint256(accounts[k][0]))) == signer) { + pk = uint256(accounts[k][1]); + found++; + } + } + + sortedPKs[j] = pk; + } + + if (found < _pks.length) { + revert("SAFETESTTOOLS: issue with private key sorting, please open a ticket on github"); + } + return sortedPKs; + } + /// @dev A wrapper for the full execTransaction method, if no signatures are provided it will /// generate them for all owners. function execTransaction( @@ -442,14 +446,14 @@ contract SafeTestTools { public returns (SafeInstance memory) { - uint256[] memory sortedPKs = sortPKsByComputedAddress(ownerPKs); + uint256[] memory sortedPKs = SafeTestLib.sortPKsByComputedAddress(ownerPKs); address[] memory owners = new address[](sortedPKs.length); for (uint256 i; i < sortedPKs.length; i++) { - if (isSmartContractPK(sortedPKs[i])) { - owners[i] = decodeSmartContractWalletAsAddress(sortedPKs[i]); + if (SafeTestLib.isSmartContractPK(sortedPKs[i])) { + owners[i] = SafeTestLib.decodeSmartContractWalletAsAddress(sortedPKs[i]); } else { - owners[i] = getAddr(sortedPKs[i]); + owners[i] = SafeTestLib.getAddr(sortedPKs[i]); } } // store the initialization parameters @@ -482,7 +486,7 @@ contract SafeTestTools { }); instances.push(instance0); - Vm(VM_ADDR).deal(address(safe0), initialBalance); + Vm(SafeTestLib.VM_ADDR).deal(address(safe0), initialBalance); return instance0; } @@ -531,7 +535,7 @@ contract SafeTestTools { } function _setupSafe() public returns (SafeInstance memory) { - (, uint256[] memory defaultPKs) = makeAddrsAndKeys(3); + (, uint256[] memory defaultPKs) = SafeTestLib.makeAddrsAndKeys(3); return _setupSafe( defaultPKs, From beba38d5a77b738bb4ff531a48426db19589bb8c Mon Sep 17 00:00:00 2001 From: Maurelian Date: Thu, 26 Oct 2023 12:11:09 -0400 Subject: [PATCH 246/279] safe-tools: Reorder functions in SafeTestLib Moving the multiple overriden execTransaction calls to the end is more readable. The other write operations are also now grouped together. --- .../test/safe-tools/SafeTestTools.sol | 208 +++++++++--------- 1 file changed, 104 insertions(+), 104 deletions(-) diff --git a/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol b/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol index c6935afe6d9c..9a5196f5791c 100644 --- a/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol +++ b/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol @@ -151,10 +151,10 @@ library SafeTestLib { return sortedPKs; } - /// @dev A wrapper for the full execTransaction method, if no signatures are provided it will - /// generate them for all owners. - function execTransaction( + /// @dev Sign a transaction as a safe owner with a private key. + function signTransaction( SafeInstance memory instance, + uint256 pk, address to, uint256 value, bytes memory data, @@ -163,20 +163,16 @@ library SafeTestLib { uint256 baseGas, uint256 gasPrice, address gasToken, - address refundReceiver, - bytes memory signatures + address refundReceiver ) internal - returns (bool) + view + returns (uint8 v, bytes32 r, bytes32 s) { - if (instance.owners.length == 0) { - revert("SAFETEST: Instance not initialized. Call _setupSafe() to initialize a test safe"); - } - - bytes32 safeTxHash; + bytes32 txDataHash; { uint256 _nonce = instance.safe.nonce(); - safeTxHash = instance.safe.getTransactionHash({ + txDataHash = instance.safe.getTransactionHash({ to: to, value: value, data: data, @@ -190,61 +186,47 @@ library SafeTestLib { }); } - if (signatures.length == 0) { - for (uint256 i; i < instance.ownerPKs.length; ++i) { - uint256 pk = instance.ownerPKs[i]; - (uint8 v, bytes32 r, bytes32 s) = Vm(VM_ADDR).sign(pk, safeTxHash); - if (isSmartContractPK(pk)) { - v = 0; - address addr = decodeSmartContractWalletAsAddress(pk); - assembly { - r := addr - } - console.logBytes32(r); - } - signatures = bytes.concat(signatures, abi.encodePacked(r, s, v)); + (v, r, s) = Vm(VM_ADDR).sign(pk, txDataHash); + } + + /// @notice Get the previous owner in the linked list of owners + /// @param _owner The owner whose previous owner we want to find + /// @param _owners The list of owners + function getPrevOwner(address _owner, address[] memory _owners) internal pure returns (address prevOwner_) { + for (uint256 i = 0; i < _owners.length; i++) { + if (_owners[i] != _owner) continue; + if (i == 0) { + prevOwner_ = SENTINEL_OWNERS; + break; } + prevOwner_ = _owners[i - 1]; } - - return instance.safe.execTransaction({ - to: to, - value: value, - data: data, - operation: operation, - safeTxGas: safeTxGas, - baseGas: baseGas, - gasPrice: gasPrice, - gasToken: gasToken, - refundReceiver: payable(refundReceiver), - signatures: signatures - }); } - /// @dev Executes either a CALL or DELEGATECALL transaction. - function execTransaction( + /// @dev Given an array of owners to remove, this function will return an array of the previous owners + /// in the order that they must be provided to the LivenessMoules's removeOwners() function. + /// Because owners are removed one at a time, and not necessarily in order, we need to simulate + /// the owners list after each removal, in order to identify the correct previous owner. + /// @param _ownersToRemove The owners to remove + /// @return prevOwners_ The previous owners in the linked list + function getPrevOwners( SafeInstance memory instance, - address to, - uint256 value, - bytes memory data, - Enum.Operation operation + address[] memory _ownersToRemove ) internal - returns (bool) + returns (address[] memory prevOwners_) { - return execTransaction(instance, to, value, data, operation, 0, 0, 0, address(0), address(0), ""); - } + OwnerSimulator ownerSimulator = new OwnerSimulator(instance.owners, 1); + prevOwners_ = new address[](_ownersToRemove.length); + address[] memory currentOwners; + for (uint256 i = 0; i < _ownersToRemove.length; i++) { + currentOwners = ownerSimulator.getOwners(); + prevOwners_[i] = SafeTestLib.getPrevOwner(instance.owners[i], currentOwners); - /// @dev Executes a CALL transaction. - function execTransaction( - SafeInstance memory instance, - address to, - uint256 value, - bytes memory data - ) - internal - returns (bool) - { - return execTransaction(instance, to, value, data, Enum.Operation.Call, 0, 0, 0, address(0), address(0), ""); + // Don't try to remove the last owner + if (currentOwners.length == 1) break; + ownerSimulator.removeOwnerWrapped(prevOwners_[i], _ownersToRemove[i], 1); + } } /// @dev Enables a module on the Safe. @@ -335,10 +317,16 @@ library SafeTestLib { EIP1271Sign(instance, abi.encodePacked(digest)); } - /// @dev Sign a transaction as a safe owner with a private key. - function signTransaction( + /// @dev Increments the nonce of the Safe by sending an empty transaction. + function incrementNonce(SafeInstance memory instance) internal returns (uint256 newNonce) { + execTransaction(instance, address(0), 0, "", Enum.Operation.Call, 0, 0, 0, address(0), address(0), ""); + return instance.safe.nonce(); + } + + /// @dev A wrapper for the full execTransaction method, if no signatures are provided it will + /// generate them for all owners. + function execTransaction( SafeInstance memory instance, - uint256 pk, address to, uint256 value, bytes memory data, @@ -347,16 +335,20 @@ library SafeTestLib { uint256 baseGas, uint256 gasPrice, address gasToken, - address refundReceiver + address refundReceiver, + bytes memory signatures ) internal - view - returns (uint8 v, bytes32 r, bytes32 s) + returns (bool) { - bytes32 txDataHash; + if (instance.owners.length == 0) { + revert("SAFETEST: Instance not initialized. Call _setupSafe() to initialize a test safe"); + } + + bytes32 safeTxHash; { uint256 _nonce = instance.safe.nonce(); - txDataHash = instance.safe.getTransactionHash({ + safeTxHash = instance.safe.getTransactionHash({ to: to, value: value, data: data, @@ -370,53 +362,61 @@ library SafeTestLib { }); } - (v, r, s) = Vm(VM_ADDR).sign(pk, txDataHash); - } - - /// @dev Increments the nonce of the Safe by sending an empty transaction. - function incrementNonce(SafeInstance memory instance) internal returns (uint256 newNonce) { - execTransaction(instance, address(0), 0, "", Enum.Operation.Call, 0, 0, 0, address(0), address(0), ""); - return instance.safe.nonce(); - } - - /// @notice Get the previous owner in the linked list of owners - /// @param _owner The owner whose previous owner we want to find - /// @param _owners The list of owners - function getPrevOwner(address _owner, address[] memory _owners) internal pure returns (address prevOwner_) { - for (uint256 i = 0; i < _owners.length; i++) { - if (_owners[i] != _owner) continue; - if (i == 0) { - prevOwner_ = SENTINEL_OWNERS; - break; + if (signatures.length == 0) { + for (uint256 i; i < instance.ownerPKs.length; ++i) { + uint256 pk = instance.ownerPKs[i]; + (uint8 v, bytes32 r, bytes32 s) = Vm(VM_ADDR).sign(pk, safeTxHash); + if (isSmartContractPK(pk)) { + v = 0; + address addr = decodeSmartContractWalletAsAddress(pk); + assembly { + r := addr + } + console.logBytes32(r); + } + signatures = bytes.concat(signatures, abi.encodePacked(r, s, v)); } - prevOwner_ = _owners[i - 1]; } + + return instance.safe.execTransaction({ + to: to, + value: value, + data: data, + operation: operation, + safeTxGas: safeTxGas, + baseGas: baseGas, + gasPrice: gasPrice, + gasToken: gasToken, + refundReceiver: payable(refundReceiver), + signatures: signatures + }); } - /// @dev Given an array of owners to remove, this function will return an array of the previous owners - /// in the order that they must be provided to the LivenessMoules's removeOwners() function. - /// Because owners are removed one at a time, and not necessarily in order, we need to simulate - /// the owners list after each removal, in order to identify the correct previous owner. - /// @param _ownersToRemove The owners to remove - /// @return prevOwners_ The previous owners in the linked list - function getPrevOwners( + /// @dev Executes either a CALL or DELEGATECALL transaction. + function execTransaction( SafeInstance memory instance, - address[] memory _ownersToRemove + address to, + uint256 value, + bytes memory data, + Enum.Operation operation ) internal - returns (address[] memory prevOwners_) + returns (bool) { - OwnerSimulator ownerSimulator = new OwnerSimulator(instance.owners, 1); - prevOwners_ = new address[](_ownersToRemove.length); - address[] memory currentOwners; - for (uint256 i = 0; i < _ownersToRemove.length; i++) { - currentOwners = ownerSimulator.getOwners(); - prevOwners_[i] = SafeTestLib.getPrevOwner(instance.owners[i], currentOwners); + return execTransaction(instance, to, value, data, operation, 0, 0, 0, address(0), address(0), ""); + } - // Don't try to remove the last owner - if (currentOwners.length == 1) break; - ownerSimulator.removeOwnerWrapped(prevOwners_[i], _ownersToRemove[i], 1); - } + /// @dev Executes a CALL transaction. + function execTransaction( + SafeInstance memory instance, + address to, + uint256 value, + bytes memory data + ) + internal + returns (bool) + { + return execTransaction(instance, to, value, data, Enum.Operation.Call, 0, 0, 0, address(0), address(0), ""); } } From e3ac8519d18dce177a961587e0365422bc943b4c Mon Sep 17 00:00:00 2001 From: Maurelian Date: Thu, 26 Oct 2023 12:30:19 -0400 Subject: [PATCH 247/279] safe-tools: Add owner management methods to SafeTestLib Can now easily add, remove and swap owners. Previous owner identification is handled automatically if not specified. This allows for testing errors by specifying an incorrect prevOwner. --- .../test/safe-tools/SafeTestTools.sol | 34 +++++++++++++++++++ 1 file changed, 34 insertions(+) diff --git a/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol b/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol index 9a5196f5791c..a2dd7bde841c 100644 --- a/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol +++ b/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol @@ -323,6 +323,40 @@ library SafeTestLib { return instance.safe.nonce(); } + /// @dev Adds a new owner to the safe + function addOwnerWithThreshold(SafeInstance memory instance, address owner, uint256 threshold) internal { + execTransaction( + instance, + address(instance.safe), + 0, + abi.encodeWithSelector(OwnerManager.addOwnerWithThreshold.selector, owner, threshold) + ); + } + + /// @dev Removes an owner from the safe. If not provided explictly, the identification of the prevOwner is handled + /// automatically. + function removeOwner(SafeInstance memory instance, address prevOwner, address owner, uint256 threshold) internal { + prevOwner = prevOwner > address(0) ? prevOwner : SafeTestLib.getPrevOwner(owner, instance.owners); + execTransaction( + instance, + address(instance.safe), + 0, + abi.encodeWithSelector(OwnerManager.removeOwner.selector, prevOwner, owner, threshold) + ); + } + + /// @dev Replaces an old owner with a new owner. If not provided explictly, the identification of the prevOwner is + /// handled automatically. + function swapOwner(SafeInstance memory instance, address prevOwner, address oldOwner, address newOwner) internal { + prevOwner = prevOwner > address(0) ? prevOwner : SafeTestLib.getPrevOwner(oldOwner, instance.owners); + execTransaction( + instance, + address(instance.safe), + 0, + abi.encodeWithSelector(OwnerManager.swapOwner.selector, prevOwner, oldOwner, newOwner) + ); + } + /// @dev A wrapper for the full execTransaction method, if no signatures are provided it will /// generate them for all owners. function execTransaction( From 2f3965fcb8f36d984aacc1ef5f819c4478fff3c4 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Thu, 26 Oct 2023 12:33:18 -0400 Subject: [PATCH 248/279] test(ctb): Refactor tests to use owner management lib utils --- .../test/LivenessGuard.t.sol | 31 +++++++++++++------ 1 file changed, 21 insertions(+), 10 deletions(-) diff --git a/packages/contracts-bedrock/test/LivenessGuard.t.sol b/packages/contracts-bedrock/test/LivenessGuard.t.sol index 679604b0db2f..66cdc4189774 100644 --- a/packages/contracts-bedrock/test/LivenessGuard.t.sol +++ b/packages/contracts-bedrock/test/LivenessGuard.t.sol @@ -134,11 +134,7 @@ contract LivenessGuard_OwnerManagement_Test is LivenessGuard_TestInit { assertGe(livenessGuard.lastLive(ownerToRemove), 0); assertTrue(safeInstance.safe.isOwner(ownerToRemove)); - safeInstance.execTransaction({ - to: address(safeInstance.safe), - value: 0, - data: abi.encodeWithSelector(OwnerManager.removeOwner.selector, SafeTestLib.SENTINEL_OWNERS, ownerToRemove, 1) - }); + safeInstance.removeOwner({ prevOwner: address(0), owner: ownerToRemove, threshold: 1 }); assertFalse(safeInstance.safe.isOwner(ownerToRemove)); assertEq(livenessGuard.lastLive(ownerToRemove), 0); @@ -150,11 +146,26 @@ contract LivenessGuard_OwnerManagement_Test is LivenessGuard_TestInit { assertEq(livenessGuard.lastLive(ownerToAdd), 0); assertFalse(safeInstance.safe.isOwner(ownerToAdd)); - safeInstance.execTransaction({ - to: address(safeInstance.safe), - value: 0, - data: abi.encodeWithSelector(OwnerManager.addOwnerWithThreshold.selector, ownerToAdd, 1) - }); + safeInstance.addOwnerWithThreshold({ owner: ownerToAdd, threshold: 1 }); + + assertTrue(safeInstance.safe.isOwner(ownerToAdd)); + assertEq(livenessGuard.lastLive(ownerToAdd), block.timestamp); + } + + /// @dev Tests that the guard correctly adds an owner to the lastLive mapping when it is added + function test_swapOwner_succeeds() external { + address ownerToRemove = safeInstance.owners[0]; + assertGe(livenessGuard.lastLive(ownerToRemove), 0); + assertTrue(safeInstance.safe.isOwner(ownerToRemove)); + + address ownerToAdd = makeAddr("new owner"); + assertEq(livenessGuard.lastLive(ownerToAdd), 0); + assertFalse(safeInstance.safe.isOwner(ownerToAdd)); + + safeInstance.swapOwner({ prevOwner: address(0), oldOwner: ownerToRemove, newOwner: ownerToAdd }); + + assertFalse(safeInstance.safe.isOwner(ownerToRemove)); + assertEq(livenessGuard.lastLive(ownerToRemove), 0); assertTrue(safeInstance.safe.isOwner(ownerToAdd)); assertEq(livenessGuard.lastLive(ownerToAdd), block.timestamp); From 41c9a3d4b768fc306ffe9c24355c512c91457679 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Thu, 26 Oct 2023 13:56:32 -0400 Subject: [PATCH 249/279] refactor(ctb): Simplify require statements by nesting in an if() Rather than using an OR clause we put them inside an if statement. This allows us to clarify the error message and is more testable. --- .../src/Safe/LivenessModule.sol | 20 +++++++++++-------- 1 file changed, 12 insertions(+), 8 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessModule.sol b/packages/contracts-bedrock/src/Safe/LivenessModule.sol index ed4f5d579de5..1f5857f63861 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessModule.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessModule.sol @@ -74,10 +74,10 @@ contract LivenessModule is ISemver { uint256 ownersCount = SAFE.getOwners().length; for (uint256 i = 0; i < _previousOwners.length; i++) { ownersCount--; - require( - ownersCount < MIN_OWNERS || _canRemove(_ownersToRemove[i]), - "LivenessModule: the safe still has sufficient owners, or the owner to remove has signed recently" - ); + if (ownersCount >= MIN_OWNERS) { + require(_canRemove(_ownersToRemove[i]), "LivenessModule: the owner to remove has signed recently"); + } + _removeOwner({ _prevOwner: _previousOwners[i], _ownerToRemove: _ownersToRemove[i], @@ -154,10 +154,14 @@ contract LivenessModule is ISemver { address[] memory owners = SAFE.getOwners(); uint256 numOwners = owners.length; // Ensure that the safe is not being left in an unsafe state with too few owners. - require( - numOwners >= MIN_OWNERS || (numOwners == 1 && owners[0] == FALLBACK_OWNER), - "LivenessModule: Safe must have the minimum number of owners or be owned solely by the fallback owner" - ); + if (numOwners == 1) { + require(owners[0] == FALLBACK_OWNER, "LivenessModule: must transfer ownership to fallback owner"); + } else { + require( + numOwners >= MIN_OWNERS, + "LivenessModule: must remove all owners and transfer to fallback owner if numOwners < minOwners" + ); + } // Check that the threshold is correct. This check is also correct when there is a single // owner, because get75PercentThreshold(1) returns 1. From 71bb67b5b6c772acea4ed90ddd1677df2446519a Mon Sep 17 00:00:00 2001 From: Maurelian Date: Thu, 26 Oct 2023 21:33:21 -0400 Subject: [PATCH 250/279] feat: Address feedback --- .../src/Safe/LivenessGuard.sol | 31 ++++++++++--------- .../src/Safe/LivenessModule.sol | 20 ++++-------- .../test/LivenessGuard.t.sol | 6 ++-- specs/safe-liveness-checking.md | 5 +-- 4 files changed, 28 insertions(+), 34 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol index 97ced936b42f..a39c80551a81 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol @@ -23,7 +23,7 @@ contract LivenessGuard is ISemver, BaseGuard { /// @notice Emitted when an owner is recorded. /// @param owner The owner's address. - event OwnerRecorded(bytes32 indexed txHash, address owner); + event OwnerRecorded(address owner); /// @notice Semantic version. /// @custom:semver 1.0.0 @@ -49,12 +49,18 @@ contract LivenessGuard is ISemver, BaseGuard { for (uint256 i = 0; i < owners.length; i++) { address owner = owners[i]; lastLive[owner] = block.timestamp; - emit OwnerRecorded(0x0, owner); + emit OwnerRecorded(owner); } } + /// @notice Getter function for the Safe contract instance + /// @return safe_ The Safe contract instance + function safe() public view returns (Safe safe_) { + safe_ = SAFE; + } + /// @notice Internal function to ensure that only the Safe can call certain functions. - function _onlySafe() internal view { + function _requireOnlySafe() internal view { require(msg.sender == address(SAFE), "LivenessGuard: only Safe can call this function"); } @@ -76,7 +82,7 @@ contract LivenessGuard is ISemver, BaseGuard { external { msgSender; // silence unused variable warning - _onlySafe(); + _requireOnlySafe(); // Cache the set of owners prior to execution. // This will be used in the checkAfterExecution method. @@ -88,7 +94,7 @@ contract LivenessGuard is ISemver, BaseGuard { // This call will reenter to the Safe which is calling it. This is OK because it is only reading the // nonce, and using the getTransactionHash() method. - bytes32 txHash = Safe(payable(msg.sender)).getTransactionHash({ + bytes32 txHash = SAFE.getTransactionHash({ to: to, value: value, data: data, @@ -98,7 +104,7 @@ contract LivenessGuard is ISemver, BaseGuard { gasPrice: gasPrice, gasToken: gasToken, refundReceiver: refundReceiver, - _nonce: Safe(payable(msg.sender)).nonce() - 1 + _nonce: SAFE.nonce() - 1 }); uint256 threshold = SAFE.getThreshold(); @@ -107,7 +113,7 @@ contract LivenessGuard is ISemver, BaseGuard { for (uint256 i = 0; i < signers.length; i++) { lastLive[signers[i]] = block.timestamp; - emit OwnerRecorded(txHash, signers[i]); + emit OwnerRecorded(signers[i]); } } @@ -118,7 +124,7 @@ contract LivenessGuard is ISemver, BaseGuard { /// 1. Add new owners to the lastLive mapping /// 2. Delete removed owners from the lastLive mapping function checkAfterExecution(bytes32, bool) external { - _onlySafe(); + _requireOnlySafe(); // Get the current set of owners address[] memory ownersAfter = SAFE.getOwners(); @@ -132,6 +138,7 @@ contract LivenessGuard is ISemver, BaseGuard { lastLive[ownerAfter] = block.timestamp; } } + // Now iterate over the remaining ownersBefore entries. Any remaining addresses are no longer an owner, so we // delete them from the lastLive mapping. uint256 ownersBeforeLength = ownersBefore.length(); @@ -147,12 +154,6 @@ contract LivenessGuard is ISemver, BaseGuard { require(SAFE.isOwner(msg.sender), "LivenessGuard: only Safe owners may demonstrate liveness"); lastLive[msg.sender] = block.timestamp; - emit OwnerRecorded(0x0, msg.sender); - } - - /// @notice Getter function for the Safe contract instance - /// @return safe_ The Safe contract instance - function safe() public view returns (Safe safe_) { - safe_ = SAFE; + emit OwnerRecorded(msg.sender); } } diff --git a/packages/contracts-bedrock/src/Safe/LivenessModule.sol b/packages/contracts-bedrock/src/Safe/LivenessModule.sol index 1f5857f63861..225d306250f5 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessModule.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessModule.sol @@ -55,7 +55,7 @@ contract LivenessModule is ISemver { FALLBACK_OWNER = _fallbackOwner; MIN_OWNERS = _minOwners; address[] memory owners = _safe.getOwners(); - require(_minOwners < owners.length, "LivenessModule: minOwners must be less than the number of owners"); + require(_minOwners <= owners.length, "LivenessModule: minOwners must be less than the number of owners"); require( _safe.getThreshold() == get75PercentThreshold(owners.length), "LivenessModule: Safe must have a threshold of 75% of the number of owners" @@ -70,7 +70,7 @@ contract LivenessModule is ISemver { function removeOwners(address[] memory _previousOwners, address[] memory _ownersToRemove) external { require(_previousOwners.length == _ownersToRemove.length, "LivenessModule: arrays must be the same length"); - // We will remove at least one owner, so we'll initialize the newOwners count to the current number of owners + // We will remove at least one owner, so we'll initialize the ownersCount count to the current number of owners uint256 ownersCount = SAFE.getOwners().length; for (uint256 i = 0; i < _previousOwners.length; i++) { ownersCount--; @@ -81,8 +81,7 @@ contract LivenessModule is ISemver { _removeOwner({ _prevOwner: _previousOwners[i], _ownerToRemove: _ownersToRemove[i], - _newOwnersCount: ownersCount, - _newThreshold: get75PercentThreshold(ownersCount) + _newOwnersCount: ownersCount }); } _verifyFinalState(); @@ -92,18 +91,11 @@ contract LivenessModule is ISemver { /// @param _prevOwner Owner that pointed to the owner to be removed in the linked list /// @param _ownerToRemove Owner address to be removed. /// @param _newOwnersCount New number of owners after removal. - /// @param _newThreshold New threshold. - function _removeOwner( - address _prevOwner, - address _ownerToRemove, - uint256 _newOwnersCount, - uint256 _newThreshold - ) - internal - { + function _removeOwner(address _prevOwner, address _ownerToRemove, uint256 _newOwnersCount) internal { if (_newOwnersCount > 0) { + uint256 newThreshold = get75PercentThreshold(_newOwnersCount); // Remove the owner and update the threshold - _removeOwnerSafeCall({ _prevOwner: _prevOwner, _owner: _ownerToRemove, _threshold: _newThreshold }); + _removeOwnerSafeCall({ _prevOwner: _prevOwner, _owner: _ownerToRemove, _threshold: newThreshold }); } else { // There is only one owner left. The Safe will not allow a safe with no owners, so we will // need to swap owners instead. diff --git a/packages/contracts-bedrock/test/LivenessGuard.t.sol b/packages/contracts-bedrock/test/LivenessGuard.t.sol index 66cdc4189774..191efc144013 100644 --- a/packages/contracts-bedrock/test/LivenessGuard.t.sol +++ b/packages/contracts-bedrock/test/LivenessGuard.t.sol @@ -13,7 +13,7 @@ import { LivenessGuard } from "src/Safe/LivenessGuard.sol"; contract LivenessGuard_TestInit is Test, SafeTestTools { using SafeTestLib for SafeInstance; - event OwnerRecorded(bytes32 indexed txHash, address signer); + event OwnerRecorded(address owner); uint256 initTime = 10; LivenessGuard livenessGuard; @@ -81,7 +81,7 @@ contract LivenessGuard_CheckTx_Test is LivenessGuard_TestInit { for (uint256 i; i < signers.length; i++) { // Don't check topic1 so that we can avoid the ugly txHash calculation. vm.expectEmit(false, true, true, true, address(livenessGuard)); - emit OwnerRecorded(0x0, signers[i]); + emit OwnerRecorded(signers[i]); } vm.expectCall(address(safeInstance.safe), abi.encodeWithSignature("nonce()")); vm.expectCall(address(safeInstance.safe), abi.encodeCall(OwnerManager.getThreshold, ())); @@ -116,7 +116,7 @@ contract LivenessGuard_ShowLiveness_Test is LivenessGuard_TestInit { address caller = safeInstance.owners[0]; vm.expectEmit(address(livenessGuard)); - emit OwnerRecorded(0x0, caller); + emit OwnerRecorded(caller); vm.prank(caller); livenessGuard.showLiveness(); diff --git a/specs/safe-liveness-checking.md b/specs/safe-liveness-checking.md index aa28ea2d10d1..604df07bc678 100644 --- a/specs/safe-liveness-checking.md +++ b/specs/safe-liveness-checking.md @@ -92,13 +92,14 @@ The following security properties must be upheld: 1. Signatures are assigned to the correct signer. 1. Non-signers are unable to create a record of having signed. -1. A signer cannot be censored or grieffed such that their signing is not recorded. +1. A signer cannot be censored or griefed such that their signing is not recorded. 1. Signers may demonstrate liveness either by signing a transaction or by calling directly to the guard. 1. The module only removes a signer if they have demonstrated liveness during the interval, or if necessary to convert the safe to a 1 of 1. 1. The module sets the correct 75% threshold upon removing a signer. 1. During a shutdown the module correctly removes all signers, and converts the safe to a 1 of 1. +1. It must be impossible for the guard's checkTransaction or checkAfterExecution to permanently revert given any calldata and the current state. ### Interdependency between the Guard and Module @@ -108,7 +109,7 @@ be set on the Safe, and the Module will be unable to function if the Guard is re ### Deployment -The module are guard are intended to be deployed and installed on the safe in the following sequence: +The module and guard are intended to be deployed and installed on the safe in the following sequence: 1. Deploy the guard contract, this will set a timestamp for each existing owner on the Safe. 1. Deploy the module. From d925a353f5f7e3691fd94e4b97cfb739384b7ef3 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Fri, 27 Oct 2023 00:40:03 -0400 Subject: [PATCH 251/279] safe-tools: Fix address label string concatenation --- packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol b/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol index a2dd7bde841c..cb8c82137d10 100644 --- a/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol +++ b/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol @@ -79,9 +79,7 @@ library SafeTestLib { for (uint256 i; i < num; i++) { addrs[i] = Vm(VM_ADDR).addr(keys[i]); - Vm(VM_ADDR).label( - getAddr(keys[i]), string.concat("SAFETEST: Signer ", string(abi.encodePacked(bytes32(i)))) - ); + Vm(VM_ADDR).label(getAddr(keys[i]), string.concat("SAFETEST: Signer ", Vm(VM_ADDR).toString(i))); } } From 2469eb3a52b85f1c8889f35c6af592e0157e8a92 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Fri, 27 Oct 2023 00:44:01 -0400 Subject: [PATCH 252/279] safe-tools: Break up getPrevOwner() into multiple functions A new pure function getPrevOwnerFromList() accepts a list to search in, this avoids making an extra call which breaks expectRevert tests. --- .../test/LivenessModule.t.sol | 16 ++++--- .../test/safe-tools/SafeTestTools.sol | 43 +++++++++++++++---- 2 files changed, 43 insertions(+), 16 deletions(-) diff --git a/packages/contracts-bedrock/test/LivenessModule.t.sol b/packages/contracts-bedrock/test/LivenessModule.t.sol index 718dffff06f3..9cac377a5188 100644 --- a/packages/contracts-bedrock/test/LivenessModule.t.sol +++ b/packages/contracts-bedrock/test/LivenessModule.t.sol @@ -26,11 +26,11 @@ contract LivenessModule_TestInit is Test, SafeTestTools { address fallbackOwner; /// @dev Removes an owner from the safe - function _removeAnOwner(address _ownerToRemove) internal { + function _removeAnOwner(address _ownerToRemove, address[] memory _owners) internal { address[] memory prevOwners = new address[](1); address[] memory ownersToRemove = new address[](1); ownersToRemove[0] = _ownerToRemove; - prevOwners[0] = SafeTestLib.getPrevOwner(_ownerToRemove, safeInstance.owners); + prevOwners[0] = SafeTestLib.getPrevOwnerFromList(_ownerToRemove, _owners); livenessModule.removeOwners(prevOwners, ownersToRemove); } @@ -141,20 +141,22 @@ contract LivenessModule_RemoveOwners_TestFail is LivenessModule_TestInit { /// @dev Test removing an owner which has recently signed a transaction function test_removeOwners_ownerHasSignedRecently_reverts() external { /// Will sign a transaction with the first M owners in the owners list - vm.warp(block.timestamp + livenessInterval); safeInstance.execTransaction({ to: address(1111), value: 0, data: hex"abba" }); + + address[] memory owners = safeInstance.safe.getOwners(); + vm.expectRevert("LivenessModule: the owner to remove has signed recently"); - _removeAnOwner(safeInstance.owners[0]); + _removeAnOwner(safeInstance.owners[0], owners); } /// @dev Test removing an owner which has recently called showLiveness function test_removeOwners_ownerHasShownLivenessRecently_reverts() external { /// Will sign a transaction with the first M owners in the owners list - vm.warp(block.timestamp + livenessInterval); vm.prank(safeInstance.owners[0]); livenessGuard.showLiveness(); + address[] memory owners = safeInstance.safe.getOwners(); vm.expectRevert("LivenessModule: the owner to remove has signed recently"); - _removeAnOwner(safeInstance.owners[0]); + _removeAnOwner(safeInstance.owners[0], owners); } /// @dev Test removing an owner with an incorrect previous owner @@ -243,7 +245,7 @@ contract LivenessModule_RemoveOwners_Test is LivenessModule_TestInit { address ownerToRemove = safeInstance.owners[0]; vm.warp(block.timestamp + livenessInterval + 1); - _removeAnOwner(ownerToRemove); + _removeAnOwner(ownerToRemove, safeInstance.owners); assertFalse(safeInstance.safe.isOwner(ownerToRemove)); assertEq(safeInstance.safe.getOwners().length, ownersBefore - 1); diff --git a/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol b/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol index cb8c82137d10..76af3b6c6dc1 100644 --- a/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol +++ b/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol @@ -187,17 +187,42 @@ library SafeTestLib { (v, r, s) = Vm(VM_ADDR).sign(pk, txDataHash); } - /// @notice Get the previous owner in the linked list of owners + /// @dev Get the previous owner in the linked list of owners. + /// This version of getPrevOwner will call to the Safe contract to get the current list of owners. + /// Note that this will break vm.expectRevert() tests by making a call which does not revert.. /// @param _owner The owner whose previous owner we want to find - /// @param _owners The list of owners - function getPrevOwner(address _owner, address[] memory _owners) internal pure returns (address prevOwner_) { - for (uint256 i = 0; i < _owners.length; i++) { - if (_owners[i] != _owner) continue; + function getPrevOwner(SafeInstance memory instance, address _owner) internal view returns (address prevOwner_) { + address[] memory owners = instance.safe.getOwners(); + for (uint256 i = 0; i < owners.length; i++) { + if (owners[i] != _owner) continue; if (i == 0) { prevOwner_ = SENTINEL_OWNERS; break; } - prevOwner_ = _owners[i - 1]; + prevOwner_ = owners[i - 1]; + } + } + + /// @dev Get the previous owner in the provided list of owners. + /// This version of getPrevOwner accepts a list of owners, and will return the previous owner. + /// It is useful when testing for a revert, as it avoids the need to call to the Safe contract. + /// @param _owner The owner whose previous owner we want to find + /// @param _ownersList The list of owners to search in + function getPrevOwnerFromList( + address _owner, + address[] memory _ownersList + ) + internal + pure + returns (address prevOwner_) + { + for (uint256 i = 0; i < _ownersList.length; i++) { + if (_ownersList[i] != _owner) continue; + if (i == 0) { + prevOwner_ = SENTINEL_OWNERS; + break; + } + prevOwner_ = _ownersList[i - 1]; } } @@ -219,7 +244,7 @@ library SafeTestLib { address[] memory currentOwners; for (uint256 i = 0; i < _ownersToRemove.length; i++) { currentOwners = ownerSimulator.getOwners(); - prevOwners_[i] = SafeTestLib.getPrevOwner(instance.owners[i], currentOwners); + prevOwners_[i] = SafeTestLib.getPrevOwnerFromList(instance.owners[i], currentOwners); // Don't try to remove the last owner if (currentOwners.length == 1) break; @@ -334,7 +359,7 @@ library SafeTestLib { /// @dev Removes an owner from the safe. If not provided explictly, the identification of the prevOwner is handled /// automatically. function removeOwner(SafeInstance memory instance, address prevOwner, address owner, uint256 threshold) internal { - prevOwner = prevOwner > address(0) ? prevOwner : SafeTestLib.getPrevOwner(owner, instance.owners); + prevOwner = prevOwner > address(0) ? prevOwner : SafeTestLib.getPrevOwner(instance, owner); execTransaction( instance, address(instance.safe), @@ -346,7 +371,7 @@ library SafeTestLib { /// @dev Replaces an old owner with a new owner. If not provided explictly, the identification of the prevOwner is /// handled automatically. function swapOwner(SafeInstance memory instance, address prevOwner, address oldOwner, address newOwner) internal { - prevOwner = prevOwner > address(0) ? prevOwner : SafeTestLib.getPrevOwner(oldOwner, instance.owners); + prevOwner = prevOwner > address(0) ? prevOwner : SafeTestLib.getPrevOwner(instance, oldOwner); execTransaction( instance, address(instance.safe), From 30377e223d609520e0a668e5971ae72a6cb9cff9 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Fri, 27 Oct 2023 09:12:38 -0400 Subject: [PATCH 253/279] refactor(ctb): Move getters to just after constructor --- .../src/Safe/LivenessModule.sol | 72 +++++++++---------- 1 file changed, 36 insertions(+), 36 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessModule.sol b/packages/contracts-bedrock/src/Safe/LivenessModule.sol index 225d306250f5..a8c4f136d0c8 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessModule.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessModule.sol @@ -62,6 +62,42 @@ contract LivenessModule is ISemver { ); } + /// @notice For a given number of owners, return the lowest threshold which is greater than 75. + /// Note: this function returns 1 for numOwners == 1. + function get75PercentThreshold(uint256 _numOwners) public pure returns (uint256 threshold_) { + threshold_ = (_numOwners * 75 + 99) / 100; + } + + /// @notice Getter function for the Safe contract instance + /// @return safe_ The Safe contract instance + function safe() public view returns (Safe safe_) { + safe_ = SAFE; + } + + /// @notice Getter function for the LivenessGuard contract instance + /// @return livenessGuard_ The LivenessGuard contract instance + function livenessGuard() public view returns (LivenessGuard livenessGuard_) { + livenessGuard_ = LIVENESS_GUARD; + } + + /// @notice Getter function for the liveness interval + /// @return livenessInterval_ The liveness interval, in seconds + function livenessInterval() public view returns (uint256 livenessInterval_) { + livenessInterval_ = LIVENESS_INTERVAL; + } + + /// @notice Getter function for the minimum number of owners + /// @return minOwners_ The minimum number of owners + function minOwners() public view returns (uint256 minOwners_) { + minOwners_ = MIN_OWNERS; + } + + /// @notice Getter function for the fallback owner + /// @return fallbackOwner_ The fallback owner of the Safe + function fallbackOwner() public view returns (address fallbackOwner_) { + fallbackOwner_ = FALLBACK_OWNER; + } + /// @notice This function can be called by anyone to remove a set of owners that have not signed a transaction /// during the liveness interval. If the number of owners drops below the minimum, then all owners /// must be removed. @@ -169,40 +205,4 @@ contract LivenessModule is ISemver { "LivenessModule: guard has been changed" ); } - - /// @notice For a given number of owners, return the lowest threshold which is greater than 75. - /// Note: this function returns 1 for numOwners == 1. - function get75PercentThreshold(uint256 _numOwners) public pure returns (uint256 threshold_) { - threshold_ = (_numOwners * 75 + 99) / 100; - } - - /// @notice Getter function for the Safe contract instance - /// @return safe_ The Safe contract instance - function safe() public view returns (Safe safe_) { - safe_ = SAFE; - } - - /// @notice Getter function for the LivenessGuard contract instance - /// @return livenessGuard_ The LivenessGuard contract instance - function livenessGuard() public view returns (LivenessGuard livenessGuard_) { - livenessGuard_ = LIVENESS_GUARD; - } - - /// @notice Getter function for the liveness interval - /// @return livenessInterval_ The liveness interval, in seconds - function livenessInterval() public view returns (uint256 livenessInterval_) { - livenessInterval_ = LIVENESS_INTERVAL; - } - - /// @notice Getter function for the minimum number of owners - /// @return minOwners_ The minimum number of owners - function minOwners() public view returns (uint256 minOwners_) { - minOwners_ = MIN_OWNERS; - } - - /// @notice Getter function for the fallback owner - /// @return fallbackOwner_ The fallback owner of the Safe - function fallbackOwner() public view returns (address fallbackOwner_) { - fallbackOwner_ = FALLBACK_OWNER; - } } From 7b5dfee97f2e791d40a37a6b481de8e25a7e2249 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Fri, 27 Oct 2023 09:24:49 -0400 Subject: [PATCH 254/279] docs(ctb): Improved comments in SafeSigners.sol --- packages/contracts-bedrock/src/Safe/SafeSigners.sol | 13 +++++-------- 1 file changed, 5 insertions(+), 8 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/SafeSigners.sol b/packages/contracts-bedrock/src/Safe/SafeSigners.sol index f0ad74fc67ed..32e65913dda5 100644 --- a/packages/contracts-bedrock/src/Safe/SafeSigners.sol +++ b/packages/contracts-bedrock/src/Safe/SafeSigners.sol @@ -38,10 +38,11 @@ library SafeSigners { } /// @notice Extract the signers from a set of signatures. - /// @param dataHash Hash of the data. - /// @param signatures Signature data for identifying signers. - /// @param requiredSignatures Amount of required valid signatures. - /// @return _owners List of unique signers. + /// This method is based closely on the code in the Safe.checkNSignatures() method. + /// https://github.com/safe-global/safe-contracts/blob/e870f514ad34cd9654c72174d6d4a839e3c6639f/contracts/Safe.sol#L274 + /// It has been modified by removing all signature _validation_ code. We trust the Safe to properly validate + /// the signatures. + /// This method therefore simply extracts the addresses from the signatures. function getNSigners( bytes32 dataHash, bytes memory signatures, @@ -53,10 +54,6 @@ library SafeSigners { { _owners = new address[](requiredSignatures); - // The following code is extracted from the Safe.checkNSignatures() method. It removes the signature - // validation code, and keeps only the parsing code necessary to extract the owner addresses from the - // signatures. We do not double check if the owner derived from a signature is valid. As this is handled - // in the final require statement of Safe.checkNSignatures(). address currentOwner; uint8 v; bytes32 r; From a2b4bae02b8190efd6f1ef68aac4df3d83151d5b Mon Sep 17 00:00:00 2001 From: Maurelian Date: Fri, 27 Oct 2023 09:39:01 -0400 Subject: [PATCH 255/279] specs: Document deployment and updating of the liveness checking system --- specs/safe-liveness-checking.md | 50 +++++++++++++++++++++++++-------- 1 file changed, 39 insertions(+), 11 deletions(-) diff --git a/specs/safe-liveness-checking.md b/specs/safe-liveness-checking.md index 604df07bc678..f1313fa7ccca 100644 --- a/specs/safe-liveness-checking.md +++ b/specs/safe-liveness-checking.md @@ -11,7 +11,7 @@ - [Owner removal call flow](#owner-removal-call-flow) - [Shutdown](#shutdown) - [Security Properties](#security-properties) - - [Interdependency between the Guard and Module](#interdependency-between-the-guard-and-module) + - [Interdependency between the guard and module](#interdependency-between-the-guard-and-module) - [Deployment](#deployment) @@ -48,7 +48,7 @@ Signers may also call the contract's `showLiveness()()` method directly in order A `LivenessModule` is also created which does the following: -1. Has a function `removeOwner()` that anyone may call to specify an owner to be removed from the +1. Has a function `removeOwners()` that anyone may call to specify one or more owners to be removed from the Safe. 1. The Module would then check the `LivenessGuard.lastLive()` to determine if the signer is eligible for removal. @@ -99,22 +99,50 @@ The following security properties must be upheld: if necessary to convert the safe to a 1 of 1. 1. The module sets the correct 75% threshold upon removing a signer. 1. During a shutdown the module correctly removes all signers, and converts the safe to a 1 of 1. -1. It must be impossible for the guard's checkTransaction or checkAfterExecution to permanently revert given any calldata and the current state. +1. It must be impossible for the guard's checkTransaction or checkAfterExecution to permanently + revert given any calldata and the current state. -### Interdependency between the Guard and Module +### Interdependency between the guard and module -The Guard has no dependency on the Module, and can be used independently to track liveness of -Safe owners. The Module however does have a dependency on the Guard, only one guard contract can -be set on the Safe, and the Module will be unable to function if the Guard is removed. +The guard has no dependency on the module, and can be used independently to track liveness of +Safe owners. -### Deployment +This means that the module can be removed or replaced without any affect on the guard. -The module and guard are intended to be deployed and installed on the safe in the following sequence: +The module however does have a dependency on the guard; if the guard is removed from the Safe, then +the module will no longer be functional and calls to its `removeOwners` function will revert. -1. Deploy the guard contract, this will set a timestamp for each existing owner on the Safe. +### Deploying the liveness checking system + +[deploying]: #deploying-the-liveness-checking-system + +The module and guard are intended to be deployed and installed on the safe in the following +sequence: + +1. Deploy the guard contract + 2. The guard's constructor will read the Safe's owners and set a timestamp 1. Deploy the module. -1. Enable the module on the safe. 1. Set the guard on the safe. +1. Enable the module on the safe. This order of operations is necessary to satisfy the constructor checks in the module, and is intended to prevent owners from being immediately removable. + +### Modify the liveness checking system + +Changes to the liveness checking system should be done in the following manner: + +#### Replacing the module + +The module can safely be removed without affecting the operation of the guard. A new module can then +be added. + +#### Replacing the guard + +The safe can only have one guard contract at a time, and if the guard is removed the module will +cease to function. This does not affect the ability of the Safe to operate normally, however the +module should be removed as a best practice. + +If a new guard is added, eg. as a means of upgrading it, then a new module will also need to be +deployed and enabled. Once both the guard and module have been removed, they can be replaced +according to the steps in the [Deployment][deploying] section above. From f2cf018e6504ed2214a7c15bbfee9782f78dcaac Mon Sep 17 00:00:00 2001 From: Maurelian Date: Fri, 27 Oct 2023 11:06:27 -0400 Subject: [PATCH 256/279] refactor(ctb): Extract module tests vm.warp into _warpPastLivenesInterval --- .../test/LivenessModule.t.sol | 19 ++++++++++++------- 1 file changed, 12 insertions(+), 7 deletions(-) diff --git a/packages/contracts-bedrock/test/LivenessModule.t.sol b/packages/contracts-bedrock/test/LivenessModule.t.sol index 9cac377a5188..71734cd9a80d 100644 --- a/packages/contracts-bedrock/test/LivenessModule.t.sol +++ b/packages/contracts-bedrock/test/LivenessModule.t.sol @@ -35,6 +35,11 @@ contract LivenessModule_TestInit is Test, SafeTestTools { livenessModule.removeOwners(prevOwners, ownersToRemove); } + /// @dev Set the current time to after the liveness interval + function _warpPastLivenessInterval() internal { + vm.warp(initTime + livenessInterval + 1); + } + /// @dev Sets up the test environment function setUp() public { // Set the block timestamp to the initTime, so that signatures recorded in the first block @@ -166,7 +171,7 @@ contract LivenessModule_RemoveOwners_TestFail is LivenessModule_TestInit { ownersToRemove[0] = safeInstance.owners[0]; prevOwners[0] = ownersToRemove[0]; // incorrect. - vm.warp(block.timestamp + livenessInterval + 1); + _warpPastLivenessInterval(); vm.expectRevert("LivenessModule: failed to remove owner"); livenessModule.removeOwners(prevOwners, ownersToRemove); } @@ -184,7 +189,7 @@ contract LivenessModule_RemoveOwners_TestFail is LivenessModule_TestInit { // Incorrectly set the final owner to address(0) ownersToRemove[ownersToRemove.length - 1] = address(0); - vm.warp(block.timestamp + livenessInterval + 1); + _warpPastLivenessInterval(); vm.expectRevert("LivenessModule: failed to swap to fallback owner"); livenessModule.removeOwners(prevOwners, ownersToRemove); } @@ -200,7 +205,7 @@ contract LivenessModule_RemoveOwners_TestFail is LivenessModule_TestInit { } address[] memory prevOwners = safeInstance.getPrevOwners(ownersToRemove); - vm.warp(block.timestamp + livenessInterval + 1); + _warpPastLivenessInterval(); vm.expectRevert("LivenessModule: must transfer ownership to fallback owner"); livenessModule.removeOwners(prevOwners, ownersToRemove); } @@ -216,7 +221,7 @@ contract LivenessModule_RemoveOwners_TestFail is LivenessModule_TestInit { livenessGuard = new LivenessGuard(safeInstance.safe); safeInstance.setGuard(address(livenessGuard)); - vm.warp(block.timestamp + livenessInterval + 1); + _warpPastLivenessInterval(); vm.expectRevert("LivenessModule: guard has been changed"); livenessModule.removeOwners(prevOwners, ownersToRemove); } @@ -230,7 +235,7 @@ contract LivenessModule_RemoveOwners_TestFail is LivenessModule_TestInit { address(safeInstance.safe), abi.encodeCall(OwnerManager.getThreshold, ()), abi.encode(wrongThreshold) ); - vm.warp(block.timestamp + livenessInterval + 1); + _warpPastLivenessInterval(); vm.expectRevert("LivenessModule: Safe must have a threshold of 75% of the number of owners"); livenessModule.removeOwners(prevOwners, ownersToRemove); } @@ -244,7 +249,7 @@ contract LivenessModule_RemoveOwners_Test is LivenessModule_TestInit { uint256 ownersBefore = safeInstance.owners.length; address ownerToRemove = safeInstance.owners[0]; - vm.warp(block.timestamp + livenessInterval + 1); + _warpPastLivenessInterval(); _removeAnOwner(ownerToRemove, safeInstance.owners); assertFalse(safeInstance.safe.isOwner(ownerToRemove)); @@ -261,7 +266,7 @@ contract LivenessModule_RemoveOwners_Test is LivenessModule_TestInit { } address[] memory prevOwners = safeInstance.getPrevOwners(ownersToRemove); - vm.warp(block.timestamp + livenessInterval + 1); + _warpPastLivenessInterval(); livenessModule.removeOwners(prevOwners, ownersToRemove); assertEq(safeInstance.safe.getOwners().length, 1); assertEq(safeInstance.safe.getOwners()[0], fallbackOwner); From 24e9a030224f6c7c23a50fb2d2f61c84125a46a3 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Fri, 27 Oct 2023 11:07:03 -0400 Subject: [PATCH 257/279] refactor(ctb): Make canRemove a public function --- .../src/Safe/LivenessModule.sol | 17 +++++++++-------- .../test/LivenessModule.t.sol | 18 ++++++++++++++++++ 2 files changed, 27 insertions(+), 8 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessModule.sol b/packages/contracts-bedrock/src/Safe/LivenessModule.sol index a8c4f136d0c8..66d802306adb 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessModule.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessModule.sol @@ -98,6 +98,14 @@ contract LivenessModule is ISemver { fallbackOwner_ = FALLBACK_OWNER; } + /// @notice Checks if the owner can be removed + /// @param _owner The owner to be removed + /// @return canRemove_ bool indicating if the owner can be removed + function canRemove(address _owner) public view returns (bool canRemove_) { + require(SAFE.isOwner(_owner), "LivenessModule: the owner to remove must be an owner of the Safe"); + canRemove_ = LIVENESS_GUARD.lastLive(_owner) + LIVENESS_INTERVAL < block.timestamp; + } + /// @notice This function can be called by anyone to remove a set of owners that have not signed a transaction /// during the liveness interval. If the number of owners drops below the minimum, then all owners /// must be removed. @@ -111,7 +119,7 @@ contract LivenessModule is ISemver { for (uint256 i = 0; i < _previousOwners.length; i++) { ownersCount--; if (ownersCount >= MIN_OWNERS) { - require(_canRemove(_ownersToRemove[i]), "LivenessModule: the owner to remove has signed recently"); + require(canRemove(_ownersToRemove[i]), "LivenessModule: the owner to remove has signed recently"); } _removeOwner({ @@ -170,13 +178,6 @@ contract LivenessModule is ISemver { ); } - /// @notice Checks if the owner can be removed - /// @param _owner The owner to be removed - /// @return canRemove_ bool indicating if the owner can be removed - function _canRemove(address _owner) internal view returns (bool canRemove_) { - canRemove_ = LIVENESS_GUARD.lastLive(_owner) + LIVENESS_INTERVAL < block.timestamp; - } - /// @notice A FREI-PI invariant check enforcing requirements on number of owners and threshold. function _verifyFinalState() internal view { address[] memory owners = SAFE.getOwners(); diff --git a/packages/contracts-bedrock/test/LivenessModule.t.sol b/packages/contracts-bedrock/test/LivenessModule.t.sol index 71734cd9a80d..de0cb36d85c5 100644 --- a/packages/contracts-bedrock/test/LivenessModule.t.sol +++ b/packages/contracts-bedrock/test/LivenessModule.t.sol @@ -105,6 +105,24 @@ contract LivenessModule_Getters_Test is LivenessModule_TestInit { } } +contract LivenessModule_CanRemove_TestFail is LivenessModule_TestInit { + /// @dev Tests if canRemove work correctly + function test_canRemove_notSafeOwner_reverts() external { + address nonOwner = makeAddr("nonOwner"); + vm.expectRevert("LivenessModule: the owner to remove must be an owner of the Safe"); + livenessModule.canRemove(nonOwner); + } +} + +contract LivenessModule_CanRemove_Test is LivenessModule_TestInit { + /// @dev Tests if canRemove work correctly + function test_canRemove_works() external { + _warpPastLivenessInterval(); + bool canRemove = livenessModule.canRemove(safeInstance.owners[0]); + assertTrue(canRemove); + } +} + contract LivenessModule_Get75PercentThreshold_Test is LivenessModule_TestInit { /// @dev check the return values of the get75PercentThreshold function against manually /// calculated values. From 81a8c1042e23014fedfb850e8a2dbd17e5b5cd4f Mon Sep 17 00:00:00 2001 From: Maurelian Date: Fri, 27 Oct 2023 11:26:08 -0400 Subject: [PATCH 258/279] refactor(ctb): Standardize on not caching the length in a for-loop --- packages/contracts-bedrock/.gas-snapshot | 39 +++++++++++-------- .../src/Safe/LivenessGuard.sol | 7 ++-- 2 files changed, 25 insertions(+), 21 deletions(-) diff --git a/packages/contracts-bedrock/.gas-snapshot b/packages/contracts-bedrock/.gas-snapshot index 3c874406e284..c5315d9f1633 100644 --- a/packages/contracts-bedrock/.gas-snapshot +++ b/packages/contracts-bedrock/.gas-snapshot @@ -306,26 +306,31 @@ LegacyERC20ETH_Test:test_transfer_doesNotExist_reverts() (gas: 10755) LegacyMessagePasser_Test:test_passMessageToL1_succeeds() (gas: 34524) LibPosition_Test:test_pos_correctness_succeeds() (gas: 38689) LivenessGuard_CheckAfterExecution_TestFails:test_checkAfterExecution_callerIsNotSafe_revert() (gas: 8553) -LivenessGuard_CheckTx_Test:test_checkTransaction_succeeds() (gas: 234784) -LivenessGuard_CheckTx_TestFails:test_checkTransaction_callerIsNotSafe_revert() (gas: 10343) -LivenessGuard_Constructor_Test:test_constructor_works() (gas: 1163577) +LivenessGuard_CheckTx_Test:test_checkTransaction_succeeds() (gas: 228306) +LivenessGuard_CheckTx_TestFails:test_checkTransaction_callerIsNotSafe_revert() (gas: 10380) +LivenessGuard_Constructor_Test:test_constructor_works() (gas: 1174506) LivenessGuard_Getters_Test:test_getters_works() (gas: 10662) -LivenessGuard_ShowLiveness_Test:test_showLiveness_succeeds() (gas: 29584) +LivenessGuard_OwnerManagement_Test:test_addOwner_succeeds() (gas: 272723) +LivenessGuard_OwnerManagement_Test:test_removeOwner_succeeds() (gas: 241108) +LivenessGuard_OwnerManagement_Test:test_swapOwner_succeeds() (gas: 279720) +LivenessGuard_ShowLiveness_Test:test_showLiveness_succeeds() (gas: 28831) LivenessGuard_ShowLiveness_TestFail:test_showLiveness_callIsNotSafeOwner_reverts() (gas: 18770) -LivenessModule_Constructor_Test:test_constructor_minOwnersGreaterThanOwners_reverts() (gas: 83531) -LivenessModule_Constructor_Test:test_constructor_wrongThreshold_reverts() (gas: 92808) +LivenessModule_CanRemove_Test:test_canRemove_works() (gas: 33026) +LivenessModule_CanRemove_TestFail:test_canRemove_notSafeOwner_reverts() (gas: 20489) +LivenessModule_Constructor_Test:test_constructor_minOwnersGreaterThanOwners_reverts() (gas: 83623) +LivenessModule_Constructor_Test:test_constructor_wrongThreshold_reverts() (gas: 92901) LivenessModule_Get75PercentThreshold_Test:test_get75PercentThreshold_Works() (gas: 26339) -LivenessModule_Getters_Test:test_getters_works() (gas: 14807) -LivenessModule_RemoveOwners_Test:test_removeOwners_allOwners_succeeds() (gas: 346743) -LivenessModule_RemoveOwners_Test:test_removeOwners_oneOwner_succeeds() (gas: 127161) -LivenessModule_RemoveOwners_TestFail:test_removeOwners_belowMinButNotEmptied_reverts() (gas: 301067) -LivenessModule_RemoveOwners_TestFail:test_removeOwners_differentArrayLengths_reverts() (gas: 10536) -LivenessModule_RemoveOwners_TestFail:test_removeOwners_guardChanged_reverts() (gas: 1741747) -LivenessModule_RemoveOwners_TestFail:test_removeOwners_invalidThreshold_reverts() (gas: 67238) -LivenessModule_RemoveOwners_TestFail:test_removeOwners_ownerHasShownLivenessRecently_reverts() (gas: 91661) -LivenessModule_RemoveOwners_TestFail:test_removeOwners_ownerHasSignedRecently_reverts() (gas: 638440) -LivenessModule_RemoveOwners_TestFail:test_removeOwners_swapToFallBackOwner_reverts() (gas: 310105) -LivenessModule_RemoveOwners_TestFail:test_removeOwners_wrongPreviousOwner_reverts() (gas: 70465) +LivenessModule_Getters_Test:test_getters_works() (gas: 14853) +LivenessModule_RemoveOwners_Test:test_removeOwners_allOwners_succeeds() (gas: 1311942) +LivenessModule_RemoveOwners_Test:test_removeOwners_oneOwner_succeeds() (gas: 130731) +LivenessModule_RemoveOwners_TestFail:test_removeOwners_belowMinButNotEmptied_reverts() (gas: 1265044) +LivenessModule_RemoveOwners_TestFail:test_removeOwners_differentArrayLengths_reverts() (gas: 10547) +LivenessModule_RemoveOwners_TestFail:test_removeOwners_guardChanged_reverts() (gas: 2820629) +LivenessModule_RemoveOwners_TestFail:test_removeOwners_invalidThreshold_reverts() (gas: 69358) +LivenessModule_RemoveOwners_TestFail:test_removeOwners_ownerHasShownLivenessRecently_reverts() (gas: 77817) +LivenessModule_RemoveOwners_TestFail:test_removeOwners_ownerHasSignedRecently_reverts() (gas: 614867) +LivenessModule_RemoveOwners_TestFail:test_removeOwners_swapToFallBackOwner_reverts() (gas: 1273705) +LivenessModule_RemoveOwners_TestFail:test_removeOwners_wrongPreviousOwner_reverts() (gas: 73954) MIPS_Test:test_add_succeeds() (gas: 122932) MIPS_Test:test_addiSign_succeeds() (gas: 122923) MIPS_Test:test_addi_succeeds() (gas: 123120) diff --git a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol index a39c80551a81..ce77c33982d5 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol @@ -87,8 +87,7 @@ contract LivenessGuard is ISemver, BaseGuard { // Cache the set of owners prior to execution. // This will be used in the checkAfterExecution method. address[] memory owners = SAFE.getOwners(); - uint256 ownersLength = owners.length; - for (uint256 i = 0; i < ownersLength; i++) { + for (uint256 i = 0; i < owners.length; i++) { ownersBefore.add(owners[i]); } @@ -141,8 +140,8 @@ contract LivenessGuard is ISemver, BaseGuard { // Now iterate over the remaining ownersBefore entries. Any remaining addresses are no longer an owner, so we // delete them from the lastLive mapping. - uint256 ownersBeforeLength = ownersBefore.length(); - for (uint256 i = 0; i < ownersBeforeLength; i++) { + // uint256 ownersBeforeLength = ownersBefore.length(); + for (uint256 i = 0; i < ownersBefore.length(); i++) { address ownerBefore = ownersBefore.at(i); delete lastLive[ownerBefore]; } From ec882b185f12ab76ed2444faf3515cab6d49d44e Mon Sep 17 00:00:00 2001 From: Maurelian Date: Fri, 27 Oct 2023 11:48:33 -0400 Subject: [PATCH 259/279] refactor(ctb): Fix placement of ownersCount Also adds comments to clarify the logic. --- .../src/Safe/LivenessModule.sol | 27 ++++++++++++++++--- 1 file changed, 24 insertions(+), 3 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessModule.sol b/packages/contracts-bedrock/src/Safe/LivenessModule.sol index 66d802306adb..44c2f1c63d35 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessModule.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessModule.sol @@ -114,19 +114,37 @@ contract LivenessModule is ISemver { function removeOwners(address[] memory _previousOwners, address[] memory _ownersToRemove) external { require(_previousOwners.length == _ownersToRemove.length, "LivenessModule: arrays must be the same length"); - // We will remove at least one owner, so we'll initialize the ownersCount count to the current number of owners + // Initialize the ownersCount count to the current number of owners, so that we can track the number of + // owners in the Safe after each removal. The Safe will revert if an owner cannot be removed, so it is safe + // keep a local count of the number of owners this way. uint256 ownersCount = SAFE.getOwners().length; for (uint256 i = 0; i < _previousOwners.length; i++) { - ownersCount--; + // Validate that the owner can be removed, which means that either: + // 1. the ownersCount is now less than MIN_OWNERS, in which case all owners should be removed regardless + // of liveness, + // 2. the owner has not signed a transaction during the liveness interval. if (ownersCount >= MIN_OWNERS) { require(canRemove(_ownersToRemove[i]), "LivenessModule: the owner to remove has signed recently"); } + // Pre-emptively update our local count of the number of owners. + // This is safe because _removeOwner will bubble up any revert from the Safe if the owner cannot be removed. + ownersCount--; + + // We now attempt remove the owner from the safe. _removeOwner({ _prevOwner: _previousOwners[i], _ownerToRemove: _ownersToRemove[i], _newOwnersCount: ownersCount }); + + // when all owners are removed and the sole owner is the fallback owner, the + // ownersCount variable will be incorrectly set to zero. + // This reflects the fact that all prior owners have been removed. The loop should naturally exit at this + // point, but for safety we detect this condition and force the loop to terminate. + if (ownersCount == 0) { + break; + } } _verifyFinalState(); } @@ -182,7 +200,10 @@ contract LivenessModule is ISemver { function _verifyFinalState() internal view { address[] memory owners = SAFE.getOwners(); uint256 numOwners = owners.length; - // Ensure that the safe is not being left in an unsafe state with too few owners. + + // Ensure that the safe is not being left in a safe state such that either: + // 1. there are at least the minimum number of owners, or + // 2. there is a single owner and that owner is the fallback owner. if (numOwners == 1) { require(owners[0] == FALLBACK_OWNER, "LivenessModule: must transfer ownership to fallback owner"); } else { From d1f965f9738cfb148941979ebd258ae858618bef Mon Sep 17 00:00:00 2001 From: Maurelian Date: Fri, 27 Oct 2023 12:15:27 -0400 Subject: [PATCH 260/279] fix(ctb): Remove remaining entries from ownersBefore --- packages/contracts-bedrock/src/Safe/LivenessGuard.sol | 1 + 1 file changed, 1 insertion(+) diff --git a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol index ce77c33982d5..b6a0bd3d6fd2 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol @@ -144,6 +144,7 @@ contract LivenessGuard is ISemver, BaseGuard { for (uint256 i = 0; i < ownersBefore.length(); i++) { address ownerBefore = ownersBefore.at(i); delete lastLive[ownerBefore]; + ownersBefore.remove(ownerBefore); } } From d759e6ff7c000b50cd96513ef485eefc8968041e Mon Sep 17 00:00:00 2001 From: Maurelian Date: Fri, 27 Oct 2023 12:18:40 -0400 Subject: [PATCH 261/279] fix(ctb): Module constructor allows higher than minimum threshold in Safe Loosening this requirement should provide added flexibility when setting up the liveness system --- packages/contracts-bedrock/src/Safe/LivenessModule.sol | 4 ++-- packages/contracts-bedrock/test/LivenessModule.t.sol | 6 +++--- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessModule.sol b/packages/contracts-bedrock/src/Safe/LivenessModule.sol index 44c2f1c63d35..6f26b0eb12e9 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessModule.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessModule.sol @@ -57,8 +57,8 @@ contract LivenessModule is ISemver { address[] memory owners = _safe.getOwners(); require(_minOwners <= owners.length, "LivenessModule: minOwners must be less than the number of owners"); require( - _safe.getThreshold() == get75PercentThreshold(owners.length), - "LivenessModule: Safe must have a threshold of 75% of the number of owners" + _safe.getThreshold() >= get75PercentThreshold(owners.length), + "LivenessModule: Safe must have a threshold of at least 75% of the number of owners" ); } diff --git a/packages/contracts-bedrock/test/LivenessModule.t.sol b/packages/contracts-bedrock/test/LivenessModule.t.sol index de0cb36d85c5..2c01806f2e31 100644 --- a/packages/contracts-bedrock/test/LivenessModule.t.sol +++ b/packages/contracts-bedrock/test/LivenessModule.t.sol @@ -64,7 +64,7 @@ contract LivenessModule_TestInit is Test, SafeTestTools { } } -contract LivenessModule_Constructor_Test is LivenessModule_TestInit { +contract LivenessModule_Constructor_TestFail is LivenessModule_TestInit { /// @dev Tests that the constructor fails if the minOwners is greater than the number of owners function test_constructor_minOwnersGreaterThanOwners_reverts() external { vm.expectRevert("LivenessModule: minOwners must be less than the number of owners"); @@ -79,11 +79,11 @@ contract LivenessModule_Constructor_Test is LivenessModule_TestInit { /// @dev Tests that the constructor fails if the minOwners is greater than the number of owners function test_constructor_wrongThreshold_reverts() external { - uint256 wrongThreshold = livenessModule.get75PercentThreshold(safeInstance.owners.length) + 1; + uint256 wrongThreshold = livenessModule.get75PercentThreshold(safeInstance.owners.length) - 1; vm.mockCall( address(safeInstance.safe), abi.encodeCall(OwnerManager.getThreshold, ()), abi.encode(wrongThreshold) ); - vm.expectRevert("LivenessModule: Safe must have a threshold of 75% of the number of owners"); + vm.expectRevert("LivenessModule: Safe must have a threshold of at least 75% of the number of owners"); new LivenessModule({ _safe: safeInstance.safe, _livenessGuard: livenessGuard, From 8fd7b65e0c749055402bb07105f627f45ff393a9 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Fri, 27 Oct 2023 12:58:02 -0400 Subject: [PATCH 262/279] specs: Document deployment and updating of liveness module --- specs/safe-liveness-checking.md | 28 +++++++++++++++++++++------- 1 file changed, 21 insertions(+), 7 deletions(-) diff --git a/specs/safe-liveness-checking.md b/specs/safe-liveness-checking.md index f1313fa7ccca..25068268372b 100644 --- a/specs/safe-liveness-checking.md +++ b/specs/safe-liveness-checking.md @@ -4,19 +4,22 @@ **Table of Contents** -- [Liveness Checking Mechanism](#liveness-checking-mechanism) +- [Liveness checking Mechanism](#liveness-checking-mechanism) - [Liveness checking methodology](#liveness-checking-methodology) - - [The Liveness Guard](#the-liveness-guard) - - [The Liveness Module](#the-liveness-module) + - [The liveness guard](#the-liveness-guard) + - [The liveness module](#the-liveness-module) - [Owner removal call flow](#owner-removal-call-flow) - [Shutdown](#shutdown) - [Security Properties](#security-properties) - [Interdependency between the guard and module](#interdependency-between-the-guard-and-module) - - [Deployment](#deployment) + - [Deploying the liveness checking system](#deploying-the-liveness-checking-system) + - [Modify the liveness checking system](#modify-the-liveness-checking-system) + - [Replacing the module](#replacing-the-module) + - [Replacing the guard](#replacing-the-guard) -## Liveness Checking Mechanism +## Liveness checking Mechanism The Security Security Council uses a specially extended Safe multisig contract to ensure that any loss of access to a signer's keys is identified and addressed within a predictable period of @@ -36,7 +39,7 @@ This is achieved using two types of contracts which the Safe contract has built- authorized to execute transactions via the Safe. This means the module must properly implement auth conditions internally. -### The Liveness Guard +### The liveness guard For implementing liveness checks a `LivenessGuard` is created which receives the signatures from each executed transaction, and tracks the latest time at which a transaction was signed by each @@ -44,7 +47,7 @@ signer. This time is made publicly available by calling a `lastLive(address)(Tim Signers may also call the contract's `showLiveness()()` method directly in order to prove liveness. -### The Liveness Module +### The liveness module A `LivenessModule` is also created which does the following: @@ -102,6 +105,10 @@ The following security properties must be upheld: 1. It must be impossible for the guard's checkTransaction or checkAfterExecution to permanently revert given any calldata and the current state. +Note: neither the module nor guard attempt to prevent a quorum of owners from removing either the liveness +module or guard. There are legitimate reasons they might wish to do so. Moreover, if such a quorum +of owners exists, there is no benefit to removing them, as they are defacto 'sufficiently live'. + ### Interdependency between the guard and module The guard has no dependency on the module, and can be used independently to track liveness of @@ -128,6 +135,10 @@ sequence: This order of operations is necessary to satisfy the constructor checks in the module, and is intended to prevent owners from being immediately removable. +Note that changes to the owners set should not be made between the time the module is deployed, and +when it is enabled on the Safe, otherwise the checks made in the module's constructor may be +invalidated. If such changes are made, a new module should be deployed. + ### Modify the liveness checking system Changes to the liveness checking system should be done in the following manner: @@ -137,6 +148,9 @@ Changes to the liveness checking system should be done in the following manner: The module can safely be removed without affecting the operation of the guard. A new module can then be added. +Note: none of the module's parameters are modifiable. In order to update the security properties +enforced by the module, it must be replaced. + #### Replacing the guard The safe can only have one guard contract at a time, and if the guard is removed the module will From ba0c2ac9556e99347a603c1ff54093ca827247e6 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Fri, 27 Oct 2023 00:53:04 -0400 Subject: [PATCH 263/279] test(ctb): Add multi-step add/remove/swap test fix guard? temp: add commented out assertions in multistep test test(ctb): Add pre-checks to owner management fuzz test test(ctb): Add WrappedGuard to expose the ownersBefore set --- .../src/Safe/LivenessGuard.sol | 12 +- .../test/LivenessGuard.t.sol | 150 +++++++++++++++++- .../test/LivenessModule.t.sol | 2 +- 3 files changed, 154 insertions(+), 10 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol index b6a0bd3d6fd2..ae2c5145514b 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessGuard.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessGuard.sol @@ -39,7 +39,7 @@ contract LivenessGuard is ISemver, BaseGuard { /// @notice An enumerable set of addresses used to store the list of owners before execution, /// and then to update the lastLive mapping according to changes in the set observed /// after execution. - EnumerableSet.AddressSet private ownersBefore; + EnumerableSet.AddressSet internal ownersBefore; /// @notice Constructor. /// @param _safe The safe account for which this contract will be the guard. @@ -128,8 +128,7 @@ contract LivenessGuard is ISemver, BaseGuard { address[] memory ownersAfter = SAFE.getOwners(); // Iterate over the current owners, and remove one at a time from the ownersBefore set. - uint256 ownersAfterLength = ownersAfter.length; - for (uint256 i = 0; i < ownersAfterLength; i++) { + for (uint256 i = 0; i < ownersAfter.length; i++) { // If the value was present, remove() returns true. address ownerAfter = ownersAfter[i]; if (ownersBefore.remove(ownerAfter) == false) { @@ -140,9 +139,10 @@ contract LivenessGuard is ISemver, BaseGuard { // Now iterate over the remaining ownersBefore entries. Any remaining addresses are no longer an owner, so we // delete them from the lastLive mapping. - // uint256 ownersBeforeLength = ownersBefore.length(); - for (uint256 i = 0; i < ownersBefore.length(); i++) { - address ownerBefore = ownersBefore.at(i); + // We cache the ownersBefore set before iterating over it, because the remove() method mutates the set. + address[] memory ownersBeforeCache = ownersBefore.values(); + for (uint256 i = 0; i < ownersBeforeCache.length; i++) { + address ownerBefore = ownersBeforeCache[i]; delete lastLive[ownerBefore]; ownersBefore.remove(ownerBefore); } diff --git a/packages/contracts-bedrock/test/LivenessGuard.t.sol b/packages/contracts-bedrock/test/LivenessGuard.t.sol index 191efc144013..7593b6c69b98 100644 --- a/packages/contracts-bedrock/test/LivenessGuard.t.sol +++ b/packages/contracts-bedrock/test/LivenessGuard.t.sol @@ -2,28 +2,42 @@ pragma solidity 0.8.15; import { Test } from "forge-std/Test.sol"; +import { StdUtils } from "forge-std/StdUtils.sol"; +import { StdCheats } from "forge-std/StdCheats.sol"; import { Safe, OwnerManager } from "safe-contracts/Safe.sol"; import { SafeProxyFactory } from "safe-contracts/proxies/SafeProxyFactory.sol"; import { ModuleManager } from "safe-contracts/base/ModuleManager.sol"; import { Enum } from "safe-contracts/common/Enum.sol"; import "test/safe-tools/SafeTestTools.sol"; +import { EnumerableSet } from "@openzeppelin/contracts/utils/structs/EnumerableSet.sol"; import { LivenessGuard } from "src/Safe/LivenessGuard.sol"; +/// @dev A wrapper contract exposing the length of the ownersBefore set in the LivenessGuard. +contract WrappedGuard is LivenessGuard { + using EnumerableSet for EnumerableSet.AddressSet; + + constructor(Safe safe) LivenessGuard(safe) { } + + function ownersBeforeLength() public view returns (uint256) { + return ownersBefore.length(); + } +} + contract LivenessGuard_TestInit is Test, SafeTestTools { using SafeTestLib for SafeInstance; event OwnerRecorded(address owner); uint256 initTime = 10; - LivenessGuard livenessGuard; + WrappedGuard livenessGuard; SafeInstance safeInstance; /// @dev Sets up the test environment function setUp() public { vm.warp(initTime); safeInstance = _setupSafe(); - livenessGuard = new LivenessGuard(safeInstance.safe); + livenessGuard = new WrappedGuard(safeInstance.safe); safeInstance.setGuard(address(livenessGuard)); } } @@ -32,7 +46,7 @@ contract LivenessGuard_Constructor_Test is LivenessGuard_TestInit { /// @dev Tests that the constructor correctly sets the current time as the lastLive time for each owner function test_constructor_works() external { address[] memory owners = safeInstance.owners; - livenessGuard = new LivenessGuard(safeInstance.safe); + livenessGuard = new WrappedGuard(safeInstance.safe); for (uint256 i = 0; i < owners.length; i++) { assertEq(livenessGuard.lastLive(owners[i]), initTime); } @@ -134,7 +148,9 @@ contract LivenessGuard_OwnerManagement_Test is LivenessGuard_TestInit { assertGe(livenessGuard.lastLive(ownerToRemove), 0); assertTrue(safeInstance.safe.isOwner(ownerToRemove)); + assertEq(livenessGuard.ownersBeforeLength(), 0); safeInstance.removeOwner({ prevOwner: address(0), owner: ownerToRemove, threshold: 1 }); + assertEq(livenessGuard.ownersBeforeLength(), 0); assertFalse(safeInstance.safe.isOwner(ownerToRemove)); assertEq(livenessGuard.lastLive(ownerToRemove), 0); @@ -146,7 +162,9 @@ contract LivenessGuard_OwnerManagement_Test is LivenessGuard_TestInit { assertEq(livenessGuard.lastLive(ownerToAdd), 0); assertFalse(safeInstance.safe.isOwner(ownerToAdd)); + assertEq(livenessGuard.ownersBeforeLength(), 0); safeInstance.addOwnerWithThreshold({ owner: ownerToAdd, threshold: 1 }); + assertEq(livenessGuard.ownersBeforeLength(), 0); assertTrue(safeInstance.safe.isOwner(ownerToAdd)); assertEq(livenessGuard.lastLive(ownerToAdd), block.timestamp); @@ -162,7 +180,9 @@ contract LivenessGuard_OwnerManagement_Test is LivenessGuard_TestInit { assertEq(livenessGuard.lastLive(ownerToAdd), 0); assertFalse(safeInstance.safe.isOwner(ownerToAdd)); + assertEq(livenessGuard.ownersBeforeLength(), 0); safeInstance.swapOwner({ prevOwner: address(0), oldOwner: ownerToRemove, newOwner: ownerToAdd }); + assertEq(livenessGuard.ownersBeforeLength(), 0); assertFalse(safeInstance.safe.isOwner(ownerToRemove)); assertEq(livenessGuard.lastLive(ownerToRemove), 0); @@ -171,3 +191,127 @@ contract LivenessGuard_OwnerManagement_Test is LivenessGuard_TestInit { assertEq(livenessGuard.lastLive(ownerToAdd), block.timestamp); } } + +contract LivenessGuard_FuzzOwnerManagement_Test is StdCheats, StdUtils, LivenessGuard_TestInit { + using SafeTestLib for SafeInstance; + + /// @dev Enumerates the possible owner management operations + enum OwnerOp { + Add, + Remove, + Swap + } + + /// @dev Describes a change to be made to the safe + struct OwnerChange { + uint8 operation; // used to choose an OwnerOp + uint256 newThreshold; + uint256 ownerIndex; + } + + /// @dev Maps addresses to private keys + mapping(address => uint256) privateKeys; + + /// @dev Tests that the guard correctly manages the lastLive mapping when owners are added, removed, or swapped + function testFuzz_OwnerManagement_works( + uint256 initialOwners, + uint256 threshold, + OwnerChange[] memory changes + ) + external + { + vm.assume(changes.length < 20); + // Initialize the safe with more owners than changes, to ensure we don't try to remove them all + initialOwners = bound(initialOwners, changes.length, 2 * changes.length); + + // We need at least one owner + initialOwners = initialOwners < 1 ? 1 : initialOwners; + + // Limit the threshold to the number of owners + threshold = bound(threshold, 1, initialOwners); + + // Generate the initial owners and keys and setup the safe + (address[] memory ownerAddrs, uint256[] memory ownerkeys) = SafeTestLib.makeAddrsAndKeys(initialOwners); + // record the private keys for later use + for (uint256 i = 0; i < ownerAddrs.length; i++) { + privateKeys[ownerAddrs[i]] = ownerkeys[i]; + } + + // Create the new safe and register the guard. + SafeInstance memory safeInstance = _setupSafe(ownerkeys, threshold); + livenessGuard = new WrappedGuard(safeInstance.safe); + safeInstance.setGuard(address(livenessGuard)); + + for (uint256 i = 0; i < changes.length; i++) { + OwnerChange memory change = changes[i]; + address[] memory currentOwners = safeInstance.safe.getOwners(); + + // Create a new owner address to add and store the key + (address newOwner, uint256 newKey) = makeAddrAndKey(string.concat("new owner", vm.toString(i))); + privateKeys[newOwner] = newKey; + + OwnerOp op = OwnerOp(bound(change.operation, 0, uint256(type(OwnerOp).max))); + + assertEq(livenessGuard.ownersBeforeLength(), 0); + if (op == OwnerOp.Add) { + assertEq(livenessGuard.lastLive(newOwner), 0); + assertFalse(safeInstance.safe.isOwner(newOwner)); + change.newThreshold = bound(change.newThreshold, 1, currentOwners.length + 1); + + safeInstance.addOwnerWithThreshold(newOwner, change.newThreshold); + + assertTrue(safeInstance.safe.isOwner(newOwner)); + assertEq(livenessGuard.lastLive(newOwner), block.timestamp); + } else { + // Ensure we're removing an owner at an index within bounds + uint256 ownerIndexToRemove = bound(change.ownerIndex, 0, currentOwners.length - 1); + address ownerToRemove = currentOwners[ownerIndexToRemove]; + address prevOwner = safeInstance.getPrevOwner(ownerToRemove); + + if (op == OwnerOp.Remove) { + if (currentOwners.length == 1) continue; + assertGe(livenessGuard.lastLive(ownerToRemove), 0); + assertTrue(safeInstance.safe.isOwner(ownerToRemove)); + change.newThreshold = bound(change.newThreshold, 1, currentOwners.length - 1); + + safeInstance.removeOwner(prevOwner, ownerToRemove, change.newThreshold); + + assertFalse(safeInstance.safe.isOwner(ownerToRemove)); + assertEq(livenessGuard.lastLive(ownerToRemove), 0); + } else if (op == OwnerOp.Swap) { + assertGe(livenessGuard.lastLive(ownerToRemove), 0); + assertTrue(safeInstance.safe.isOwner(ownerToRemove)); + + safeInstance.swapOwner(prevOwner, ownerToRemove, newOwner); + + assertTrue(safeInstance.safe.isOwner(newOwner)); + assertFalse(safeInstance.safe.isOwner(ownerToRemove)); + assertEq(livenessGuard.lastLive(ownerToRemove), 0); + assertEq(livenessGuard.lastLive(newOwner), block.timestamp); + } + } + assertEq(livenessGuard.ownersBeforeLength(), 0); + _refreshOwners(safeInstance); + } + } + + /// @dev Refreshes the owners and ownerPKs arrays in the SafeInstance + function _refreshOwners(SafeInstance memory instance) internal view { + // Get the current owners + instance.owners = instance.safe.getOwners(); + + // Looks up the private key for each owner + uint256[] memory unsortedOwnerPKs = new uint256[](instance.owners.length); + for (uint256 j = 0; j < instance.owners.length; j++) { + unsortedOwnerPKs[j] = privateKeys[instance.owners[j]]; + } + + // Sort the keys by address and store them in the SafeInstance + instance.ownerPKs = SafeTestLib.sortPKsByComputedAddress(unsortedOwnerPKs); + + // Overwrite the SafeInstances owners array with the computed addresses from the ownerPKs array + for (uint256 k; k < instance.owners.length; k++) { + instance.owners[k] = SafeTestLib.getAddr(instance.ownerPKs[k]); + } + } +} diff --git a/packages/contracts-bedrock/test/LivenessModule.t.sol b/packages/contracts-bedrock/test/LivenessModule.t.sol index 2c01806f2e31..59725b1167c8 100644 --- a/packages/contracts-bedrock/test/LivenessModule.t.sol +++ b/packages/contracts-bedrock/test/LivenessModule.t.sol @@ -261,8 +261,8 @@ contract LivenessModule_RemoveOwners_TestFail is LivenessModule_TestInit { contract LivenessModule_RemoveOwners_Test is LivenessModule_TestInit { using SafeTestLib for SafeInstance; - /// @dev Tests if removing one owner works correctly + /// @dev Tests if removing one owner works correctly function test_removeOwners_oneOwner_succeeds() external { uint256 ownersBefore = safeInstance.owners.length; address ownerToRemove = safeInstance.owners[0]; From a1c2987e881fe2657713c4bcfb63a1a6f0282506 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Fri, 27 Oct 2023 15:50:11 -0400 Subject: [PATCH 264/279] safe-tools: Make address label prefix configurable --- packages/contracts-bedrock/test/LivenessGuard.t.sol | 3 ++- packages/contracts-bedrock/test/LivenessModule.t.sol | 2 +- packages/contracts-bedrock/test/SafeSigners.t.sol | 2 +- .../test/safe-tools/SafeTestTools.sol | 12 +++++++++--- 4 files changed, 13 insertions(+), 6 deletions(-) diff --git a/packages/contracts-bedrock/test/LivenessGuard.t.sol b/packages/contracts-bedrock/test/LivenessGuard.t.sol index 7593b6c69b98..643872563092 100644 --- a/packages/contracts-bedrock/test/LivenessGuard.t.sol +++ b/packages/contracts-bedrock/test/LivenessGuard.t.sol @@ -231,7 +231,8 @@ contract LivenessGuard_FuzzOwnerManagement_Test is StdCheats, StdUtils, Liveness threshold = bound(threshold, 1, initialOwners); // Generate the initial owners and keys and setup the safe - (address[] memory ownerAddrs, uint256[] memory ownerkeys) = SafeTestLib.makeAddrsAndKeys(initialOwners); + (address[] memory ownerAddrs, uint256[] memory ownerkeys) = + SafeTestLib.makeAddrsAndKeys("safeTest", initialOwners); // record the private keys for later use for (uint256 i = 0; i < ownerAddrs.length; i++) { privateKeys[ownerAddrs[i]] = ownerkeys[i]; diff --git a/packages/contracts-bedrock/test/LivenessModule.t.sol b/packages/contracts-bedrock/test/LivenessModule.t.sol index 59725b1167c8..ed93691f0ba8 100644 --- a/packages/contracts-bedrock/test/LivenessModule.t.sol +++ b/packages/contracts-bedrock/test/LivenessModule.t.sol @@ -47,7 +47,7 @@ contract LivenessModule_TestInit is Test, SafeTestTools { vm.warp(initTime); // Create a Safe with 10 owners - (, uint256[] memory keys) = SafeTestLib.makeAddrsAndKeys(10); + (, uint256[] memory keys) = SafeTestLib.makeAddrsAndKeys("moduleTest", 10); safeInstance = _setupSafe(keys, 8); livenessGuard = new LivenessGuard(safeInstance.safe); diff --git a/packages/contracts-bedrock/test/SafeSigners.t.sol b/packages/contracts-bedrock/test/SafeSigners.t.sol index ffada6c96e09..b79386a52238 100644 --- a/packages/contracts-bedrock/test/SafeSigners.t.sol +++ b/packages/contracts-bedrock/test/SafeSigners.t.sol @@ -39,7 +39,7 @@ contract SafeSigners_Test is Test, SafeTestTools { // Limit the number of signatures to 25 uint256 numSigs = bound(_numSigs, 1, 25); - (, uint256[] memory keys) = SafeTestLib.makeAddrsAndKeys(numSigs); + (, uint256[] memory keys) = SafeTestLib.makeAddrsAndKeys("getSigsTest", numSigs); for (uint256 i = 0; i < keys.length; i++) { if (sigType(keys[i]) == SigTypes.Contract) { keys[i] = diff --git a/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol b/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol index 76af3b6c6dc1..bb84c48b4225 100644 --- a/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol +++ b/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol @@ -69,7 +69,13 @@ library SafeTestLib { } /// @dev Get arrays of addresses and private keys. The arrays are sorted by address, and the addresses are labelled - function makeAddrsAndKeys(uint256 num) internal returns (address[] memory addrs, uint256[] memory keys) { + function makeAddrsAndKeys( + string memory prefix, + uint256 num + ) + internal + returns (address[] memory addrs, uint256[] memory keys) + { keys = new uint256[](num); addrs = new address[](num); for (uint256 i; i < num; i++) { @@ -79,7 +85,7 @@ library SafeTestLib { for (uint256 i; i < num; i++) { addrs[i] = Vm(VM_ADDR).addr(keys[i]); - Vm(VM_ADDR).label(getAddr(keys[i]), string.concat("SAFETEST: Signer ", Vm(VM_ADDR).toString(i))); + Vm(VM_ADDR).label(getAddr(keys[i]), string.concat(prefix, Vm(VM_ADDR).toString(i))); } } @@ -592,7 +598,7 @@ contract SafeTestTools { } function _setupSafe() public returns (SafeInstance memory) { - (, uint256[] memory defaultPKs) = SafeTestLib.makeAddrsAndKeys(3); + (, uint256[] memory defaultPKs) = SafeTestLib.makeAddrsAndKeys("default", 3); return _setupSafe( defaultPKs, From 5f56bd2cfd9e5b7160cdc7b83635f305d02faa9b Mon Sep 17 00:00:00 2001 From: Maurelian Date: Fri, 27 Oct 2023 15:50:32 -0400 Subject: [PATCH 265/279] safe-tools: Add changeThreshold method --- .../test/safe-tools/SafeTestTools.sol | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol b/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol index bb84c48b4225..6e74b3e6ee37 100644 --- a/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol +++ b/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol @@ -352,6 +352,16 @@ library SafeTestLib { return instance.safe.nonce(); } + /// @dev Adds a new owner to the safe + function changeThreshold(SafeInstance memory instance, uint256 threshold) internal { + execTransaction( + instance, + address(instance.safe), + 0, + abi.encodeWithSelector(OwnerManager.changeThreshold.selector, threshold) + ); + } + /// @dev Adds a new owner to the safe function addOwnerWithThreshold(SafeInstance memory instance, address owner, uint256 threshold) internal { execTransaction( From 774dd1b9fe61af0c76e96cf76ebc01de8dabfa0e Mon Sep 17 00:00:00 2001 From: Maurelian Date: Fri, 27 Oct 2023 16:21:57 -0400 Subject: [PATCH 266/279] safe-tools: Fix bug in getPrevOwners --- .../test/safe-tools/SafeTestTools.sol | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) diff --git a/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol b/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol index 6e74b3e6ee37..1fee1de29ed2 100644 --- a/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol +++ b/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol @@ -199,14 +199,7 @@ library SafeTestLib { /// @param _owner The owner whose previous owner we want to find function getPrevOwner(SafeInstance memory instance, address _owner) internal view returns (address prevOwner_) { address[] memory owners = instance.safe.getOwners(); - for (uint256 i = 0; i < owners.length; i++) { - if (owners[i] != _owner) continue; - if (i == 0) { - prevOwner_ = SENTINEL_OWNERS; - break; - } - prevOwner_ = owners[i - 1]; - } + prevOwner_ = getPrevOwnerFromList(_owner, owners); } /// @dev Get the previous owner in the provided list of owners. @@ -250,7 +243,7 @@ library SafeTestLib { address[] memory currentOwners; for (uint256 i = 0; i < _ownersToRemove.length; i++) { currentOwners = ownerSimulator.getOwners(); - prevOwners_[i] = SafeTestLib.getPrevOwnerFromList(instance.owners[i], currentOwners); + prevOwners_[i] = SafeTestLib.getPrevOwnerFromList(_ownersToRemove[i], currentOwners); // Don't try to remove the last owner if (currentOwners.length == 1) break; From 33c7c8b1f2f68a8e207c63cd85190b76ecc4e194 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Fri, 27 Oct 2023 16:24:02 -0400 Subject: [PATCH 267/279] test(ctb): Fuzz test removing an arbitrary number of owners --- .../test/LivenessModule.t.sol | 97 ++++++++++++++++++- .../test/safe-tools/SafeTestTools.sol | 13 ++- 2 files changed, 107 insertions(+), 3 deletions(-) diff --git a/packages/contracts-bedrock/test/LivenessModule.t.sol b/packages/contracts-bedrock/test/LivenessModule.t.sol index ed93691f0ba8..77d33c0d228e 100644 --- a/packages/contracts-bedrock/test/LivenessModule.t.sol +++ b/packages/contracts-bedrock/test/LivenessModule.t.sol @@ -41,7 +41,7 @@ contract LivenessModule_TestInit is Test, SafeTestTools { } /// @dev Sets up the test environment - function setUp() public { + function setUp() public virtual { // Set the block timestamp to the initTime, so that signatures recorded in the first block // are non-zero. vm.warp(initTime); @@ -291,3 +291,98 @@ contract LivenessModule_RemoveOwners_Test is LivenessModule_TestInit { assertEq(safeInstance.safe.getThreshold(), 1); } } + +/// @dev A copy of LivenessModule.get75PercentThreshold as a free function to use below. +function get75PercentThreshold(uint256 _numOwners) pure returns (uint256 threshold_) { + threshold_ = (_numOwners * 75 + 99) / 100; +} + +contract LivenessModule_RemoveOwnersFuzz_Test is LivenessModule_TestInit { + using SafeTestLib for SafeInstance; + + /// @dev Override the base setUp function, to avoid instantiating an unnecessary Safe + function setUp() public override { + vm.warp(initTime); + fallbackOwner = makeAddr("fallbackOwner"); + } + + /// @dev Tests if removing owners works correctly for various safe configurations and numbeers of live owners + function testFuzz_removeOwners(uint256 _numOwners, uint256 _minOwners, uint256 _numLiveOwners) external { + // _numOwners must be at least 3, so that _minOwners can be set to at least 2 by the following bound() call. + _numOwners = bound(_numOwners, 3, 20); + // _minOwners must be at least 2, otherwise we don't have any range below _minOwners to test the transfer to + // the fallback owner. + _minOwners = bound(_minOwners, _numOwners - 1, _numOwners - 1); + + // Ensure that _numLiveOwners is less than _numOwners so that we can remove at least one owner. + _numLiveOwners = bound(_numLiveOwners, 0, _numOwners - 1); + + // The above bounds are a bit tricky, so we assert that the result is correct + assertTrue(_numOwners > _minOwners && _numOwners > _numLiveOwners && _minOwners >= 2); + + // Create a Safe with _numOwners owners + (, uint256[] memory keys) = SafeTestLib.makeAddrsAndKeys("rmOwnersTest", _numOwners); + uint256 threshold = get75PercentThreshold(_numOwners); + safeInstance = _setupSafe(keys, threshold); + livenessGuard = new LivenessGuard(safeInstance.safe); + livenessModule = new LivenessModule({ + _safe: safeInstance.safe, + _livenessGuard: livenessGuard, + _livenessInterval: livenessInterval, + _minOwners: _minOwners, + _fallbackOwner: fallbackOwner + }); + safeInstance.setGuard(address(livenessGuard)); + safeInstance.enableModule(address(livenessModule)); + + // Warp ahead so that all owners non-live + _warpPastLivenessInterval(); + + // Create an array of live owners, and call showLiveness for each of them + address[] memory liveOwners = new address[](_numLiveOwners); + for (uint256 i = 0; i < _numLiveOwners; i++) { + liveOwners[i] = safeInstance.owners[i]; + vm.prank(safeInstance.owners[i]); + livenessGuard.showLiveness(); + } + + // Create an array of non-live owners + address[] memory nonLiveOwners = new address[](_numOwners - _numLiveOwners); + for (uint256 i = 0; i < _numOwners - _numLiveOwners; i++) { + nonLiveOwners[i] = safeInstance.owners[i + _numLiveOwners]; + } + + address[] memory prevOwners; + if (_numLiveOwners >= _minOwners) { + // The safe will remain above the minimum number of owners, so we can remove only those owners which are not + // live. + prevOwners = safeInstance.getPrevOwners(nonLiveOwners); + livenessModule.removeOwners(prevOwners, nonLiveOwners); + + // Validate the resulting state of the Safe + assertEq(safeInstance.safe.getOwners().length, _numLiveOwners); + assertEq(safeInstance.safe.getThreshold(), get75PercentThreshold(_numLiveOwners)); + for (uint256 i = 0; i < _numLiveOwners; i++) { + assertTrue(safeInstance.safe.isOwner(liveOwners[i])); + } + for (uint256 i = 0; i < nonLiveOwners.length; i++) { + assertFalse(safeInstance.safe.isOwner(nonLiveOwners[i])); + } + } else { + // The safe is below the minimum number of owners, so we can remove all owners, + // but we need to do remove the non-live owners first, so we reverse the owners array, since + // the first owners have signed recently. + address[] memory ownersToRemove = new address[](_numOwners); + for (uint256 i = 0; i < _numOwners; i++) { + ownersToRemove[_numOwners - i - 1] = safeInstance.owners[i]; + } + prevOwners = safeInstance.getPrevOwners(ownersToRemove); + livenessModule.removeOwners(prevOwners, ownersToRemove); + + // Validate the resulting state of the Safe + assertEq(safeInstance.safe.getOwners().length, 1); + assertEq(safeInstance.safe.getOwners()[0], fallbackOwner); + assertEq(safeInstance.safe.getThreshold(), 1); + } + } +} diff --git a/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol b/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol index 1fee1de29ed2..f4ff521ff0c9 100644 --- a/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol +++ b/packages/contracts-bedrock/test/safe-tools/SafeTestTools.sol @@ -212,10 +212,17 @@ library SafeTestLib { address[] memory _ownersList ) internal - pure - returns (address prevOwner_) + view + returns ( + // pure + address prevOwner_ + ) { + // console.log("getPrevOwnerFromList"); for (uint256 i = 0; i < _ownersList.length; i++) { + // console.log(i); + // console.log(_owner); + // console.log("_ownersList[i]:", _ownersList[i]); if (_ownersList[i] != _owner) continue; if (i == 0) { prevOwner_ = SENTINEL_OWNERS; @@ -223,6 +230,8 @@ library SafeTestLib { } prevOwner_ = _ownersList[i - 1]; } + + console.log("prevOwner_:", prevOwner_); } /// @dev Given an array of owners to remove, this function will return an array of the previous owners From eb0e133501a4eb39bb516979f211ebab01ff5d5e Mon Sep 17 00:00:00 2001 From: Maurelian Date: Sat, 28 Oct 2023 11:18:10 -0400 Subject: [PATCH 268/279] test(ctb): Add test for incomplete emptying below minOwners --- .../test/LivenessModule.t.sol | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/packages/contracts-bedrock/test/LivenessModule.t.sol b/packages/contracts-bedrock/test/LivenessModule.t.sol index 77d33c0d228e..c9e72f326134 100644 --- a/packages/contracts-bedrock/test/LivenessModule.t.sol +++ b/packages/contracts-bedrock/test/LivenessModule.t.sol @@ -214,6 +214,24 @@ contract LivenessModule_RemoveOwners_TestFail is LivenessModule_TestInit { /// @dev Tests if remove owners reverts if it removes too many owners without removing all of them function test_removeOwners_belowMinButNotEmptied_reverts() external { + // Remove all but one owner + uint256 numOwners = safeInstance.owners.length - 2; + + address[] memory ownersToRemove = new address[](numOwners); + for (uint256 i = 0; i < numOwners; i++) { + ownersToRemove[i] = safeInstance.owners[i]; + } + address[] memory prevOwners = safeInstance.getPrevOwners(ownersToRemove); + + _warpPastLivenessInterval(); + vm.expectRevert( + "LivenessModule: must remove all owners and transfer to fallback owner if numOwners < minOwners" + ); + livenessModule.removeOwners(prevOwners, ownersToRemove); + } + + /// @dev Tests if remove owners reverts if it removes too many owners transferring to the fallback owner + function test_removeOwners_belowEmptiedButNotFallback_reverts() external { // Remove all but one owner uint256 numOwners = safeInstance.owners.length - 1; From 5c0d3226233cfcaade5a234989228212e4e5e90c Mon Sep 17 00:00:00 2001 From: Maurelian Date: Sat, 28 Oct 2023 15:18:49 -0400 Subject: [PATCH 269/279] test(ctb): Add test for not giving to fallback owner --- .../src/Safe/LivenessModule.sol | 3 +- .../test/LivenessModule.t.sol | 179 ++++++++++++++---- 2 files changed, 141 insertions(+), 41 deletions(-) diff --git a/packages/contracts-bedrock/src/Safe/LivenessModule.sol b/packages/contracts-bedrock/src/Safe/LivenessModule.sol index 6f26b0eb12e9..41f2ed64708e 100644 --- a/packages/contracts-bedrock/src/Safe/LivenessModule.sol +++ b/packages/contracts-bedrock/src/Safe/LivenessModule.sol @@ -213,7 +213,8 @@ contract LivenessModule is ISemver { ); } - // Check that the threshold is correct. This check is also correct when there is a single + // Check that"LivenessModule: must remove all owners and transfer to fallback owner if numOwners < minOwners" + // the threshold is correct. This check is also correct when there is a single // owner, because get75PercentThreshold(1) returns 1. uint256 threshold = SAFE.getThreshold(); require( diff --git a/packages/contracts-bedrock/test/LivenessModule.t.sol b/packages/contracts-bedrock/test/LivenessModule.t.sol index c9e72f326134..d4e1654c4e31 100644 --- a/packages/contracts-bedrock/test/LivenessModule.t.sol +++ b/packages/contracts-bedrock/test/LivenessModule.t.sol @@ -195,7 +195,7 @@ contract LivenessModule_RemoveOwners_TestFail is LivenessModule_TestInit { } /// @dev Tests if removing all owners works correctly - function test_removeOwners_swapToFallBackOwner_reverts() external { + function test_removeOwners_swapToFallbackOwner_reverts() external { uint256 numOwners = safeInstance.owners.length; address[] memory ownersToRemove = new address[](numOwners); @@ -230,8 +230,8 @@ contract LivenessModule_RemoveOwners_TestFail is LivenessModule_TestInit { livenessModule.removeOwners(prevOwners, ownersToRemove); } - /// @dev Tests if remove owners reverts if it removes too many owners transferring to the fallback owner - function test_removeOwners_belowEmptiedButNotFallback_reverts() external { + /// @dev Tests if remove owners reverts if it removes too many owners transferring to the shutDown owner + function test_removeOwners_belowEmptiedButNotShutDown_reverts() external { // Remove all but one owner uint256 numOwners = safeInstance.owners.length - 1; @@ -318,36 +318,66 @@ function get75PercentThreshold(uint256 _numOwners) pure returns (uint256 thresho contract LivenessModule_RemoveOwnersFuzz_Test is LivenessModule_TestInit { using SafeTestLib for SafeInstance; - /// @dev Override the base setUp function, to avoid instantiating an unnecessary Safe + address[] ownersToRemove2; + + /// @dev Options for handling the event that the number of owners remaining is less than minOwners + enum ShutDownBehavior { + Correct, // Correctly removes the owners and transfers to the shutDown owner + DoesNotTransferToFallbackOwner, // Removes all but one owner, and does not transfer to the shutDown owner + DoesNotRemoveAllOwners // Leaves more than one owner when below minOwners + } + + /// @dev This contract inherits the storage layout from the LivenessModule_TestInit contract, but we + /// override the base setUp function, to avoid instantiating an unnecessary Safe and liveness checking system. function setUp() public override { vm.warp(initTime); fallbackOwner = makeAddr("fallbackOwner"); } - /// @dev Tests if removing owners works correctly for various safe configurations and numbeers of live owners - function testFuzz_removeOwners(uint256 _numOwners, uint256 _minOwners, uint256 _numLiveOwners) external { - // _numOwners must be at least 3, so that _minOwners can be set to at least 2 by the following bound() call. - _numOwners = bound(_numOwners, 3, 20); - // _minOwners must be at least 2, otherwise we don't have any range below _minOwners to test the transfer to - // the fallback owner. - _minOwners = bound(_minOwners, _numOwners - 1, _numOwners - 1); + /// @dev Extracts the setup of the test environment into a separate function. + function _prepare( + uint256 _numOwners, + uint256 _minOwners, + uint256 _numLiveOwners + ) + internal + returns (uint256 numOwners_, uint256 minOwners_, uint256 numLiveOwners_) + { + // First we modify the test parameters to ensure that they describe a plausible starting point. + // + // _numOwners must be at least 4, so that _minOwners can be set to at least 3 by the following bound() call. + // Limiting the owner set to 20 helps to keep the runtime of the test reasonable. + console.log("bounding numOwners"); + numOwners_ = bound(_numOwners, 4, 20); + // _minOwners must be at least 3, otherwise we don't have any range below _minOwners in which to test all of the + // ShutDownBehavior options. + console.log("bounding minOwners"); + minOwners_ = bound(_minOwners, 3, numOwners_ - 1); // Ensure that _numLiveOwners is less than _numOwners so that we can remove at least one owner. - _numLiveOwners = bound(_numLiveOwners, 0, _numOwners - 1); - - // The above bounds are a bit tricky, so we assert that the result is correct - assertTrue(_numOwners > _minOwners && _numOwners > _numLiveOwners && _minOwners >= 2); + console.log("bounding numLiveOwners"); + numLiveOwners_ = bound(_numLiveOwners, 0, numOwners_ - 1); + + // The above bounds are a bit tricky, so we assert that the resulting parameters enable us to test all possible + // success and revert cases in the removeOwners function. + assertTrue( + numOwners_ > minOwners_ // + && numOwners_ > numLiveOwners_ + // + && minOwners_ >= 3 + ); + // // Create a Safe with _numOwners owners - (, uint256[] memory keys) = SafeTestLib.makeAddrsAndKeys("rmOwnersTest", _numOwners); - uint256 threshold = get75PercentThreshold(_numOwners); + (, uint256[] memory keys) = SafeTestLib.makeAddrsAndKeys("rmOwnersTest", numOwners_); + uint256 threshold = get75PercentThreshold(numOwners_); safeInstance = _setupSafe(keys, threshold); livenessGuard = new LivenessGuard(safeInstance.safe); livenessModule = new LivenessModule({ _safe: safeInstance.safe, _livenessGuard: livenessGuard, _livenessInterval: livenessInterval, - _minOwners: _minOwners, + _minOwners: minOwners_, _fallbackOwner: fallbackOwner }); safeInstance.setGuard(address(livenessGuard)); @@ -355,52 +385,121 @@ contract LivenessModule_RemoveOwnersFuzz_Test is LivenessModule_TestInit { // Warp ahead so that all owners non-live _warpPastLivenessInterval(); + } + + /// @dev Tests if removing owners works correctly for various safe configurations and numbeers of live owners + function testFuzz_removeOwners( + uint256 _numOwners, + uint256 _minOwners, + uint256 _numLiveOwners, + uint256 _shutDownBehavior, + uint256 _numOwnersToRemoveinShutDown + ) + external + { + // Prepare the test env and test params + (uint256 numOwners, uint256 minOwners, uint256 numLiveOwners) = _prepare(_numOwners, _minOwners, _numLiveOwners); // Create an array of live owners, and call showLiveness for each of them - address[] memory liveOwners = new address[](_numLiveOwners); - for (uint256 i = 0; i < _numLiveOwners; i++) { + address[] memory liveOwners = new address[](numLiveOwners); + for (uint256 i = 0; i < numLiveOwners; i++) { liveOwners[i] = safeInstance.owners[i]; vm.prank(safeInstance.owners[i]); livenessGuard.showLiveness(); } // Create an array of non-live owners - address[] memory nonLiveOwners = new address[](_numOwners - _numLiveOwners); - for (uint256 i = 0; i < _numOwners - _numLiveOwners; i++) { - nonLiveOwners[i] = safeInstance.owners[i + _numLiveOwners]; + address[] memory nonLiveOwners = new address[](numOwners - numLiveOwners); + for (uint256 i = 0; i < numOwners - numLiveOwners; i++) { + nonLiveOwners[i] = safeInstance.owners[i + numLiveOwners]; } address[] memory prevOwners; - if (_numLiveOwners >= _minOwners) { + if (numLiveOwners >= minOwners) { + console.log("No shutdown"); // The safe will remain above the minimum number of owners, so we can remove only those owners which are not // live. prevOwners = safeInstance.getPrevOwners(nonLiveOwners); livenessModule.removeOwners(prevOwners, nonLiveOwners); // Validate the resulting state of the Safe - assertEq(safeInstance.safe.getOwners().length, _numLiveOwners); - assertEq(safeInstance.safe.getThreshold(), get75PercentThreshold(_numLiveOwners)); - for (uint256 i = 0; i < _numLiveOwners; i++) { + assertEq(safeInstance.safe.getOwners().length, numLiveOwners); + assertEq(safeInstance.safe.getThreshold(), get75PercentThreshold(numLiveOwners)); + for (uint256 i = 0; i < numLiveOwners; i++) { assertTrue(safeInstance.safe.isOwner(liveOwners[i])); } for (uint256 i = 0; i < nonLiveOwners.length; i++) { assertFalse(safeInstance.safe.isOwner(nonLiveOwners[i])); } } else { - // The safe is below the minimum number of owners, so we can remove all owners, - // but we need to do remove the non-live owners first, so we reverse the owners array, since - // the first owners have signed recently. - address[] memory ownersToRemove = new address[](_numOwners); - for (uint256 i = 0; i < _numOwners; i++) { - ownersToRemove[_numOwners - i - 1] = safeInstance.owners[i]; + // The number of non-live owners will push the safe below the minimum number of owners. + // We need to test all of the possible ShutDownBehavior options, so we'll create a ShutDownBehavior enum + // from the _shutDownBehavior input. + ShutDownBehavior shutDownBehavior = + ShutDownBehavior(bound(_shutDownBehavior, 0, uint256(type(ShutDownBehavior).max))); + // The safe is below the minimum number of owners. + // The ShutDownBehavior enum determines how we handle this case. + if (shutDownBehavior == ShutDownBehavior.Correct) { + console.log("Correct Shutdown"); + // We remove all owners, and transfer ownership to the shutDown owner. + // but we need to do remove the non-live owners first, so we reverse the owners array, since + // the first owners have signed recently. + address[] memory ownersToRemove = new address[](numOwners); + for (uint256 i = 0; i < numOwners; i++) { + ownersToRemove[numOwners - i - 1] = safeInstance.owners[i]; + } + prevOwners = safeInstance.getPrevOwners(ownersToRemove); + livenessModule.removeOwners(prevOwners, ownersToRemove); + + // Validate the resulting state of the Safe + assertEq(safeInstance.safe.getOwners().length, 1); + assertEq(safeInstance.safe.getOwners()[0], fallbackOwner); + assertEq(safeInstance.safe.getThreshold(), 1); + } else { + // For both of the incorrect behaviors, we need to calculate the number of owners to remove to + // trigger that behavior. We initialize that value here then set it in the if statements below. + uint256 numOwnersToRemoveinShutDown; + if (shutDownBehavior == ShutDownBehavior.DoesNotRemoveAllOwners) { + console.log("Shutdown DoesNotRemoveAllOwners"); + // In the DoesNotRemoveAllOwners case, we should have more than 1 of the pre-existing owners + // remaining + console.log("bounding numOwnersToRemoveinShutDown"); + numOwnersToRemoveinShutDown = + bound(_numOwnersToRemoveinShutDown, numOwners - minOwners + 1, numOwners - 2); + uint256 i = 0; + for (i; i < numOwnersToRemoveinShutDown; i++) { + // Add non-live owners to remove first + if (i < nonLiveOwners.length) { + ownersToRemove2.push(nonLiveOwners[i]); + } else { + // Then add live owners to remove + ownersToRemove2.push(liveOwners[i - nonLiveOwners.length]); + } + } + prevOwners = safeInstance.getPrevOwners(ownersToRemove2); + vm.expectRevert( + "LivenessModule: must remove all owners and transfer to fallback owner if numOwners < minOwners" + ); + livenessModule.removeOwners(prevOwners, ownersToRemove2); + } else if (shutDownBehavior == ShutDownBehavior.DoesNotTransferToFallbackOwner) { + console.log("Shutdown DoesNotTransferToFallbackOwner"); + // In the DoesNotRemoveAllOwners case, we should have exactly 1 pre-existing owners remaining + numOwnersToRemoveinShutDown = numOwners - 1; + uint256 i = 0; + for (i; i < numOwnersToRemoveinShutDown; i++) { + // Add non-live owners to remove first + if (i < nonLiveOwners.length) { + ownersToRemove2.push(nonLiveOwners[i]); + } else { + // Then add live owners to remove + ownersToRemove2.push(liveOwners[i - nonLiveOwners.length]); + } + } + prevOwners = safeInstance.getPrevOwners(ownersToRemove2); + vm.expectRevert("LivenessModule: must transfer ownership to fallback owner"); + livenessModule.removeOwners(prevOwners, ownersToRemove2); + } } - prevOwners = safeInstance.getPrevOwners(ownersToRemove); - livenessModule.removeOwners(prevOwners, ownersToRemove); - - // Validate the resulting state of the Safe - assertEq(safeInstance.safe.getOwners().length, 1); - assertEq(safeInstance.safe.getOwners()[0], fallbackOwner); - assertEq(safeInstance.safe.getThreshold(), 1); } } } From 2cb544372cad0d06119bd16340a63e347b4d78b7 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Sat, 28 Oct 2023 15:28:41 -0400 Subject: [PATCH 270/279] test(ctb): Check all topics in guard.OwnerRecorded --- packages/contracts-bedrock/test/LivenessGuard.t.sol | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/packages/contracts-bedrock/test/LivenessGuard.t.sol b/packages/contracts-bedrock/test/LivenessGuard.t.sol index 643872563092..3a91e8f79c2d 100644 --- a/packages/contracts-bedrock/test/LivenessGuard.t.sol +++ b/packages/contracts-bedrock/test/LivenessGuard.t.sol @@ -93,8 +93,7 @@ contract LivenessGuard_CheckTx_Test is LivenessGuard_TestInit { signers[1] = safeInstance.owners[1]; for (uint256 i; i < signers.length; i++) { - // Don't check topic1 so that we can avoid the ugly txHash calculation. - vm.expectEmit(false, true, true, true, address(livenessGuard)); + vm.expectEmit(address(livenessGuard)); emit OwnerRecorded(signers[i]); } vm.expectCall(address(safeInstance.safe), abi.encodeWithSignature("nonce()")); From dcd5a0f009fe01b731e5881f5848e4d3f81d2ee8 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Sat, 28 Oct 2023 15:46:43 -0400 Subject: [PATCH 271/279] test(ctb): Clean up and add assertions to testFuzz_removeOwners --- .../test/LivenessModule.t.sol | 43 +++++++++++-------- 1 file changed, 25 insertions(+), 18 deletions(-) diff --git a/packages/contracts-bedrock/test/LivenessModule.t.sol b/packages/contracts-bedrock/test/LivenessModule.t.sol index d4e1654c4e31..adc1af45666c 100644 --- a/packages/contracts-bedrock/test/LivenessModule.t.sol +++ b/packages/contracts-bedrock/test/LivenessModule.t.sol @@ -318,7 +318,8 @@ function get75PercentThreshold(uint256 _numOwners) pure returns (uint256 thresho contract LivenessModule_RemoveOwnersFuzz_Test is LivenessModule_TestInit { using SafeTestLib for SafeInstance; - address[] ownersToRemove2; + /// @dev We put this array in storage so that we can more easily populate it using push in the tests below. + address[] ownersToRemove; /// @dev Options for handling the event that the number of owners remaining is less than minOwners enum ShutDownBehavior { @@ -360,13 +361,12 @@ contract LivenessModule_RemoveOwnersFuzz_Test is LivenessModule_TestInit { // The above bounds are a bit tricky, so we assert that the resulting parameters enable us to test all possible // success and revert cases in the removeOwners function. + // This is also necessary to avoid underflows or out of bounds accesses in the test. assertTrue( - numOwners_ > minOwners_ // - && numOwners_ > numLiveOwners_ - // - && minOwners_ >= 3 + numOwners_ > minOwners_ // We need to be able to remove at least one owner + && numOwners_ >= numLiveOwners_ // We can have more live owners than there are owners + && minOwners_ >= 3 // Allows us to test all of the ShutDownBehavior options when removing an owner ); - // // Create a Safe with _numOwners owners (, uint256[] memory keys) = SafeTestLib.makeAddrsAndKeys("rmOwnersTest", numOwners_); @@ -408,7 +408,6 @@ contract LivenessModule_RemoveOwnersFuzz_Test is LivenessModule_TestInit { livenessGuard.showLiveness(); } - // Create an array of non-live owners address[] memory nonLiveOwners = new address[](numOwners - numLiveOwners); for (uint256 i = 0; i < numOwners - numLiveOwners; i++) { nonLiveOwners[i] = safeInstance.owners[i + numLiveOwners]; @@ -443,10 +442,12 @@ contract LivenessModule_RemoveOwnersFuzz_Test is LivenessModule_TestInit { console.log("Correct Shutdown"); // We remove all owners, and transfer ownership to the shutDown owner. // but we need to do remove the non-live owners first, so we reverse the owners array, since - // the first owners have signed recently. - address[] memory ownersToRemove = new address[](numOwners); + // the first owners in the array were the ones to call showLiveness. + // ownersToRemove = new address[](numOwners); for (uint256 i = 0; i < numOwners; i++) { - ownersToRemove[numOwners - i - 1] = safeInstance.owners[i]; + // ownersToRemove[numOwners - i - 1] = safeInstance.owners[i]; + // ownersToRemove[i] = safeInstance.owners[numOwners - i - 1]; + ownersToRemove.push(safeInstance.owners[numOwners - i - 1]); } prevOwners = safeInstance.getPrevOwners(ownersToRemove); livenessModule.removeOwners(prevOwners, ownersToRemove); @@ -470,17 +471,17 @@ contract LivenessModule_RemoveOwnersFuzz_Test is LivenessModule_TestInit { for (i; i < numOwnersToRemoveinShutDown; i++) { // Add non-live owners to remove first if (i < nonLiveOwners.length) { - ownersToRemove2.push(nonLiveOwners[i]); + ownersToRemove.push(nonLiveOwners[i]); } else { // Then add live owners to remove - ownersToRemove2.push(liveOwners[i - nonLiveOwners.length]); + ownersToRemove.push(liveOwners[i - nonLiveOwners.length]); } } - prevOwners = safeInstance.getPrevOwners(ownersToRemove2); + prevOwners = safeInstance.getPrevOwners(ownersToRemove); vm.expectRevert( "LivenessModule: must remove all owners and transfer to fallback owner if numOwners < minOwners" ); - livenessModule.removeOwners(prevOwners, ownersToRemove2); + livenessModule.removeOwners(prevOwners, ownersToRemove); } else if (shutDownBehavior == ShutDownBehavior.DoesNotTransferToFallbackOwner) { console.log("Shutdown DoesNotTransferToFallbackOwner"); // In the DoesNotRemoveAllOwners case, we should have exactly 1 pre-existing owners remaining @@ -489,15 +490,21 @@ contract LivenessModule_RemoveOwnersFuzz_Test is LivenessModule_TestInit { for (i; i < numOwnersToRemoveinShutDown; i++) { // Add non-live owners to remove first if (i < nonLiveOwners.length) { - ownersToRemove2.push(nonLiveOwners[i]); + ownersToRemove.push(nonLiveOwners[i]); } else { // Then add live owners to remove - ownersToRemove2.push(liveOwners[i - nonLiveOwners.length]); + ownersToRemove.push(liveOwners[i - nonLiveOwners.length]); } } - prevOwners = safeInstance.getPrevOwners(ownersToRemove2); + prevOwners = safeInstance.getPrevOwners(ownersToRemove); vm.expectRevert("LivenessModule: must transfer ownership to fallback owner"); - livenessModule.removeOwners(prevOwners, ownersToRemove2); + livenessModule.removeOwners(prevOwners, ownersToRemove); + } + // For both of the incorrect behaviors, verify no change to the Safe state + assertEq(safeInstance.safe.getOwners().length, numOwners); + assertEq(safeInstance.safe.getThreshold(), get75PercentThreshold(numOwners)); + for (uint256 j = 0; j < numOwners; j++) { + assertTrue(safeInstance.safe.isOwner(safeInstance.owners[j])); } } } From ae901333b5cad8066870edae952e307a6d9f3b2c Mon Sep 17 00:00:00 2001 From: Maurelian Date: Sat, 28 Oct 2023 15:58:50 -0400 Subject: [PATCH 272/279] test(ctb): Ensure lastLive increments on signing --- packages/contracts-bedrock/test/LivenessGuard.t.sol | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/packages/contracts-bedrock/test/LivenessGuard.t.sol b/packages/contracts-bedrock/test/LivenessGuard.t.sol index 3a91e8f79c2d..fd4281ea5170 100644 --- a/packages/contracts-bedrock/test/LivenessGuard.t.sol +++ b/packages/contracts-bedrock/test/LivenessGuard.t.sol @@ -92,6 +92,13 @@ contract LivenessGuard_CheckTx_Test is LivenessGuard_TestInit { signers[0] = safeInstance.owners[0]; signers[1] = safeInstance.owners[1]; + // Record the timestamps before the transaction + uint256[] memory beforeTimestamps = new uint256[](safeInstance.owners.length); + + // Jump ahead + uint256 newTimestamp = block.timestamp + 100; + vm.warp(newTimestamp); + for (uint256 i; i < signers.length; i++) { vm.expectEmit(address(livenessGuard)); emit OwnerRecorded(signers[i]); @@ -99,9 +106,10 @@ contract LivenessGuard_CheckTx_Test is LivenessGuard_TestInit { vm.expectCall(address(safeInstance.safe), abi.encodeWithSignature("nonce()")); vm.expectCall(address(safeInstance.safe), abi.encodeCall(OwnerManager.getThreshold, ())); safeInstance.execTransaction({ to: address(1111), value: 0, data: hex"abba" }); - for (uint256 i; i < safeInstance.threshold; i++) { - assertEq(livenessGuard.lastLive(safeInstance.owners[i]), block.timestamp); + uint256 lastLive = livenessGuard.lastLive(safeInstance.owners[i]); + assertGe(lastLive, beforeTimestamps[i]); + assertEq(lastLive, newTimestamp); } } } From 1873257f868a980bf8980d34af3ad8087f30a9c4 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Sat, 28 Oct 2023 16:04:42 -0400 Subject: [PATCH 273/279] test(ctb): testFuzz_OwnerManagement increases timestamp on each action --- packages/contracts-bedrock/test/LivenessGuard.t.sol | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/packages/contracts-bedrock/test/LivenessGuard.t.sol b/packages/contracts-bedrock/test/LivenessGuard.t.sol index fd4281ea5170..e1f7cdd761b2 100644 --- a/packages/contracts-bedrock/test/LivenessGuard.t.sol +++ b/packages/contracts-bedrock/test/LivenessGuard.t.sol @@ -211,9 +211,10 @@ contract LivenessGuard_FuzzOwnerManagement_Test is StdCheats, StdUtils, Liveness /// @dev Describes a change to be made to the safe struct OwnerChange { + uint8 timeDelta; // used to warp the vm uint8 operation; // used to choose an OwnerOp + uint256 ownerIndex; // used to choose the owner to remove or swap out uint256 newThreshold; - uint256 ownerIndex; } /// @dev Maps addresses to private keys @@ -251,6 +252,7 @@ contract LivenessGuard_FuzzOwnerManagement_Test is StdCheats, StdUtils, Liveness safeInstance.setGuard(address(livenessGuard)); for (uint256 i = 0; i < changes.length; i++) { + vm.warp(block.timestamp + changes[i].timeDelta); OwnerChange memory change = changes[i]; address[] memory currentOwners = safeInstance.safe.getOwners(); From 43cb263a7b4d47c25301b3743b8f28c2f62de511 Mon Sep 17 00:00:00 2001 From: Maurelian Date: Sat, 28 Oct 2023 16:05:12 -0400 Subject: [PATCH 274/279] bind and lock and snap --- op-bindings/bindings/mips_more.go | 2 +- op-bindings/bindings/preimageoracle_more.go | 2 +- packages/contracts-bedrock/.gas-snapshot | 35 +++++++++++---------- packages/contracts-bedrock/semver-lock.json | 4 +-- 4 files changed, 22 insertions(+), 21 deletions(-) diff --git a/op-bindings/bindings/mips_more.go b/op-bindings/bindings/mips_more.go index 75570153b367..cab2a3b89d8d 100644 --- a/op-bindings/bindings/mips_more.go +++ b/op-bindings/bindings/mips_more.go @@ -15,7 +15,7 @@ var MIPSStorageLayout = new(solc.StorageLayout) var MIPSDeployedBin = "0x608060405234801561001057600080fd5b50600436106100415760003560e01c8063155633fe146100465780637dc0d1d01461006b578063836e7b32146100af575b600080fd5b610051634000000081565b60405163ffffffff90911681526020015b60405180910390f35b60405173ffffffffffffffffffffffffffffffffffffffff7f0000000000000000000000000000000000000000000000000000000000000000168152602001610062565b6100c26100bd366004611d2e565b6100d0565b604051908152602001610062565b60006100da611c5b565b608081146100e757600080fd5b604051610600146100f757600080fd5b6084871461010457600080fd5b6101a4851461011257600080fd5b8635608052602087013560a052604087013560e090811c60c09081526044890135821c82526048890135821c61010052604c890135821c610120526050890135821c61014052605489013590911c61016052605888013560f890811c610180526059890135901c6101a052605a880135901c6101c0526102006101e0819052606288019060005b60208110156101bd57823560e01c8252600490920191602090910190600101610199565b505050806101200151156101db576101d361061b565b915050610612565b6101408101805160010167ffffffffffffffff16905260608101516000906102039082610737565b9050603f601a82901c16600281148061022257508063ffffffff166003145b156102775760006002836303ffffff1663ffffffff16901b846080015163f00000001617905061026c8263ffffffff1660021461026057601f610263565b60005b60ff16826107f3565b945050505050610612565b6101608301516000908190601f601086901c81169190601587901c16602081106102a3576102a3611da2565b602002015192508063ffffffff851615806102c457508463ffffffff16601c145b156102fb578661016001518263ffffffff16602081106102e6576102e6611da2565b6020020151925050601f600b86901c166103b7565b60208563ffffffff16101561035d578463ffffffff16600c148061032557508463ffffffff16600d145b8061033657508463ffffffff16600e145b15610347578561ffff1692506103b7565b6103568661ffff1660106108e4565b92506103b7565b60288563ffffffff1610158061037957508463ffffffff166022145b8061038a57508463ffffffff166026145b156103b7578661016001518263ffffffff16602081106103ac576103ac611da2565b602002015192508190505b60048563ffffffff16101580156103d4575060088563ffffffff16105b806103e557508463ffffffff166001145b15610404576103f685878487610957565b975050505050505050610612565b63ffffffff6000602087831610610469576104248861ffff1660106108e4565b9095019463fffffffc861661043a816001610737565b915060288863ffffffff161015801561045a57508763ffffffff16603014155b1561046757809250600093505b505b600061047789888885610b67565b63ffffffff9081169150603f8a1690891615801561049c575060088163ffffffff1610155b80156104ae5750601c8163ffffffff16105b1561058b578063ffffffff16600814806104ce57508063ffffffff166009145b15610505576104f38163ffffffff166008146104ea57856104ed565b60005b896107f3565b9b505050505050505050505050610612565b8063ffffffff16600a03610525576104f3858963ffffffff8a16156112f7565b8063ffffffff16600b03610546576104f3858963ffffffff8a1615156112f7565b8063ffffffff16600c0361055d576104f38d6113dd565b60108163ffffffff161015801561057a5750601c8163ffffffff16105b1561058b576104f381898988611914565b8863ffffffff1660381480156105a6575063ffffffff861615155b156105db5760018b61016001518763ffffffff16602081106105ca576105ca611da2565b63ffffffff90921660209290920201525b8363ffffffff1663ffffffff146105f8576105f884600184611b0e565b610604858360016112f7565b9b5050505050505050505050505b95945050505050565b60408051608051815260a051602082015260dc519181019190915260fc51604482015261011c51604882015261013c51604c82015261015c51605082015261017c5160548201526101805161019f5160588301526101a0516101bf5160598401526101d851605a840152600092610200929091606283019190855b60208110156106ba57601c8601518452602090950194600490930192600101610696565b506000835283830384a06000945080600181146106da5760039550610702565b8280156106f257600181146106fb5760029650610700565b60009650610700565b600196505b505b50505081900390207effffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff1660f89190911b17919050565b60008061074383611bb2565b9050600384161561075357600080fd5b6020810190358460051c8160005b601b8110156107b95760208501943583821c6001168015610789576001811461079e576107af565b600084815260208390526040902093506107af565b600082815260208590526040902093505b5050600101610761565b5060805191508181146107d457630badf00d60005260206000fd5b5050601f94909416601c0360031b9390931c63ffffffff169392505050565b60006107fd611c5b565b60809050806060015160040163ffffffff16816080015163ffffffff1614610886576040517f08c379a000000000000000000000000000000000000000000000000000000000815260206004820152601260248201527f6a756d7020696e2064656c617920736c6f74000000000000000000000000000060448201526064015b60405180910390fd5b60608101805160808301805163ffffffff9081169093528583169052908516156108dc57806008018261016001518663ffffffff16602081106108cb576108cb611da2565b63ffffffff90921660209290920201525b61061261061b565b600063ffffffff8381167fffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff80850183169190911c821615159160016020869003821681901b830191861691821b92911b0182610941576000610943565b815b90861663ffffffff16179250505092915050565b6000610961611c5b565b608090506000816060015160040163ffffffff16826080015163ffffffff16146109e7576040517f08c379a000000000000000000000000000000000000000000000000000000000815260206004820152601460248201527f6272616e636820696e2064656c617920736c6f74000000000000000000000000604482015260640161087d565b8663ffffffff1660041480610a0257508663ffffffff166005145b15610a7e5760008261016001518663ffffffff1660208110610a2657610a26611da2565b602002015190508063ffffffff168563ffffffff16148015610a4e57508763ffffffff166004145b80610a7657508063ffffffff168563ffffffff1614158015610a7657508763ffffffff166005145b915050610afb565b8663ffffffff16600603610a9b5760008460030b13159050610afb565b8663ffffffff16600703610ab75760008460030b139050610afb565b8663ffffffff16600103610afb57601f601087901c166000819003610ae05760008560030b1291505b8063ffffffff16600103610af95760008560030b121591505b505b606082018051608084015163ffffffff169091528115610b41576002610b268861ffff1660106108e4565b63ffffffff90811690911b8201600401166080840152610b53565b60808301805160040163ffffffff1690525b610b5b61061b565b98975050505050505050565b6000603f601a86901c16801580610b96575060088163ffffffff1610158015610b965750600f8163ffffffff16105b15610fec57603f86168160088114610bdd5760098114610be657600a8114610bef57600b8114610bf857600c8114610c0157600d8114610c0a57600e8114610c1357610c18565b60209150610c18565b60219150610c18565b602a9150610c18565b602b9150610c18565b60249150610c18565b60259150610c18565b602691505b508063ffffffff16600003610c3f5750505063ffffffff8216601f600686901c161b6112ef565b8063ffffffff16600203610c655750505063ffffffff8216601f600686901c161c6112ef565b8063ffffffff16600303610c9b57601f600688901c16610c9163ffffffff8716821c60208390036108e4565b93505050506112ef565b8063ffffffff16600403610cbd5750505063ffffffff8216601f84161b6112ef565b8063ffffffff16600603610cdf5750505063ffffffff8216601f84161c6112ef565b8063ffffffff16600703610d1257610d098663ffffffff168663ffffffff16901c876020036108e4565b925050506112ef565b8063ffffffff16600803610d2a5785925050506112ef565b8063ffffffff16600903610d425785925050506112ef565b8063ffffffff16600a03610d5a5785925050506112ef565b8063ffffffff16600b03610d725785925050506112ef565b8063ffffffff16600c03610d8a5785925050506112ef565b8063ffffffff16600f03610da25785925050506112ef565b8063ffffffff16601003610dba5785925050506112ef565b8063ffffffff16601103610dd25785925050506112ef565b8063ffffffff16601203610dea5785925050506112ef565b8063ffffffff16601303610e025785925050506112ef565b8063ffffffff16601803610e1a5785925050506112ef565b8063ffffffff16601903610e325785925050506112ef565b8063ffffffff16601a03610e4a5785925050506112ef565b8063ffffffff16601b03610e625785925050506112ef565b8063ffffffff16602003610e7b575050508282016112ef565b8063ffffffff16602103610e94575050508282016112ef565b8063ffffffff16602203610ead575050508183036112ef565b8063ffffffff16602303610ec6575050508183036112ef565b8063ffffffff16602403610edf575050508282166112ef565b8063ffffffff16602503610ef8575050508282176112ef565b8063ffffffff16602603610f11575050508282186112ef565b8063ffffffff16602703610f2b57505050828217196112ef565b8063ffffffff16602a03610f5c578460030b8660030b12610f4d576000610f50565b60015b60ff16925050506112ef565b8063ffffffff16602b03610f84578463ffffffff168663ffffffff1610610f4d576000610f50565b6040517f08c379a000000000000000000000000000000000000000000000000000000000815260206004820152601360248201527f696e76616c696420696e737472756374696f6e00000000000000000000000000604482015260640161087d565b50610f84565b8063ffffffff16601c0361107057603f86166002819003611012575050508282026112ef565b8063ffffffff166020148061102d57508063ffffffff166021145b15610fe6578063ffffffff16602003611044579419945b60005b6380000000871615611066576401fffffffe600197881b169601611047565b92506112ef915050565b8063ffffffff16600f0361109257505065ffffffff0000601083901b166112ef565b8063ffffffff166020036110ce576110c68560031660080260180363ffffffff168463ffffffff16901c60ff1660086108e4565b9150506112ef565b8063ffffffff16602103611103576110c68560021660080260100363ffffffff168463ffffffff16901c61ffff1660106108e4565b8063ffffffff1660220361113257505063ffffffff60086003851602811681811b198416918316901b176112ef565b8063ffffffff1660230361114957829150506112ef565b8063ffffffff1660240361117b578460031660080260180363ffffffff168363ffffffff16901c60ff169150506112ef565b8063ffffffff166025036111ae578460021660080260100363ffffffff168363ffffffff16901c61ffff169150506112ef565b8063ffffffff166026036111e057505063ffffffff60086003851602601803811681811c198416918316901c176112ef565b8063ffffffff1660280361121657505060ff63ffffffff60086003861602601803811682811b9091188316918416901b176112ef565b8063ffffffff1660290361124d57505061ffff63ffffffff60086002861602601003811682811b9091188316918416901b176112ef565b8063ffffffff16602a0361127c57505063ffffffff60086003851602811681811c198316918416901c176112ef565b8063ffffffff16602b0361129357839150506112ef565b8063ffffffff16602e036112c557505063ffffffff60086003851602601803811681811b198316918416901b176112ef565b8063ffffffff166030036112dc57829150506112ef565b8063ffffffff16603803610f8457839150505b949350505050565b6000611301611c5b565b506080602063ffffffff861610611374576040517f08c379a000000000000000000000000000000000000000000000000000000000815260206004820152600e60248201527f76616c6964207265676973746572000000000000000000000000000000000000604482015260640161087d565b63ffffffff8516158015906113865750825b156113ba57838161016001518663ffffffff16602081106113a9576113a9611da2565b63ffffffff90921660209290920201525b60808101805163ffffffff8082166060850152600490910116905261061261061b565b60006113e7611c5b565b506101e051604081015160808083015160a084015160c09094015191936000928392919063ffffffff8616610ffa036114615781610fff81161561143057610fff811661100003015b8363ffffffff166000036114575760e08801805163ffffffff83820116909152955061145b565b8395505b506118d3565b8563ffffffff16610fcd0361147c57634000000094506118d3565b8563ffffffff166110180361149457600194506118d3565b8563ffffffff16611096036114ca57600161012088015260ff83166101008801526114bd61061b565b9998505050505050505050565b8563ffffffff16610fa3036117365763ffffffff8316156118d3577ffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffb63ffffffff8416016116f05760006115258363fffffffc166001610737565b60208901519091508060001a60010361159457604080516000838152336020528d83526060902091527effffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff167f01000000000000000000000000000000000000000000000000000000000000001790505b6040808a015190517fe03110e10000000000000000000000000000000000000000000000000000000081526004810183905263ffffffff9091166024820152600090819073ffffffffffffffffffffffffffffffffffffffff7f0000000000000000000000000000000000000000000000000000000000000000169063e03110e1906044016040805180830381865afa158015611635573d6000803e3d6000fd5b505050506040513d601f19601f820116820180604052508101906116599190611dd1565b91509150600386168060040382811015611671578092505b508186101561167e578591505b8260088302610100031c9250826008828460040303021b9250600180600883600403021b036001806008858560040303021b039150811981169050838119871617955050506116d58663fffffffc16600186611b0e565b60408b018051820163ffffffff169052975061173192505050565b7ffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffd63ffffffff841601611725578094506118d3565b63ffffffff9450600993505b6118d3565b8563ffffffff16610fa4036118275763ffffffff831660011480611760575063ffffffff83166002145b80611771575063ffffffff83166004145b1561177e578094506118d3565b7ffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffa63ffffffff8416016117255760006117be8363fffffffc166001610737565b602089015190915060038416600403838110156117d9578093505b83900360089081029290921c7fffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff600193850293841b0116911b176020880152600060408801529350836118d3565b8563ffffffff16610fd7036118d3578163ffffffff166003036118c75763ffffffff8316158061185d575063ffffffff83166005145b8061186e575063ffffffff83166003145b1561187c57600094506118d3565b63ffffffff831660011480611897575063ffffffff83166002145b806118a8575063ffffffff83166006145b806118b9575063ffffffff83166004145b1561172557600194506118d3565b63ffffffff9450601693505b6101608701805163ffffffff808816604090920191909152905185821660e09091015260808801805180831660608b015260040190911690526114bd61061b565b600061191e611c5b565b506080600063ffffffff871660100361193c575060c0810151611aa5565b8663ffffffff1660110361195b5763ffffffff861660c0830152611aa5565b8663ffffffff16601203611974575060a0810151611aa5565b8663ffffffff166013036119935763ffffffff861660a0830152611aa5565b8663ffffffff166018036119c75763ffffffff600387810b9087900b02602081901c821660c08501521660a0830152611aa5565b8663ffffffff166019036119f85763ffffffff86811681871602602081901c821660c08501521660a0830152611aa5565b8663ffffffff16601a03611a4e578460030b8660030b81611a1b57611a1b611df5565b0763ffffffff1660c0830152600385810b9087900b81611a3d57611a3d611df5565b0563ffffffff1660a0830152611aa5565b8663ffffffff16601b03611aa5578463ffffffff168663ffffffff1681611a7757611a77611df5565b0663ffffffff90811660c084015285811690871681611a9857611a98611df5565b0463ffffffff1660a08301525b63ffffffff841615611ae057808261016001518563ffffffff1660208110611acf57611acf611da2565b63ffffffff90921660209290920201525b60808201805163ffffffff80821660608601526004909101169052611b0361061b565b979650505050505050565b6000611b1983611bb2565b90506003841615611b2957600080fd5b6020810190601f8516601c0360031b83811b913563ffffffff90911b1916178460051c60005b601b811015611ba75760208401933582821c6001168015611b775760018114611b8c57611b9d565b60008581526020839052604090209450611b9d565b600082815260208690526040902094505b5050600101611b4f565b505060805250505050565b60ff8116610380026101a4810190369061052401811015611c55576040517f08c379a000000000000000000000000000000000000000000000000000000000815260206004820152602360248201527f636865636b207468617420746865726520697320656e6f7567682063616c6c6460448201527f6174610000000000000000000000000000000000000000000000000000000000606482015260840161087d565b50919050565b6040805161018081018252600080825260208201819052918101829052606081018290526080810182905260a0810182905260c0810182905260e08101829052610100810182905261012081018290526101408101919091526101608101611cc1611cc6565b905290565b6040518061040001604052806020906020820280368337509192915050565b60008083601f840112611cf757600080fd5b50813567ffffffffffffffff811115611d0f57600080fd5b602083019150836020828501011115611d2757600080fd5b9250929050565b600080600080600060608688031215611d4657600080fd5b853567ffffffffffffffff80821115611d5e57600080fd5b611d6a89838a01611ce5565b90975095506020880135915080821115611d8357600080fd5b50611d9088828901611ce5565b96999598509660400135949350505050565b7f4e487b7100000000000000000000000000000000000000000000000000000000600052603260045260246000fd5b60008060408385031215611de457600080fd5b505080516020909101519092909150565b7f4e487b7100000000000000000000000000000000000000000000000000000000600052601260045260246000fdfea164736f6c634300080f000a" -var MIPSDeployedSourceMap = "1131:40054:126:-:0;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;1710:45;;1745:10;1710:45;;;;;188:10:285;176:23;;;158:42;;146:2;131:18;1710:45:126;;;;;;;;2448:99;;;412:42:285;2534:6:126;400:55:285;382:74;;370:2;355:18;2448:99:126;211:251:285;26025:6379:126;;;;;;:::i;:::-;;:::i;:::-;;;1755:25:285;;;1743:2;1728:18;26025:6379:126;1609:177:285;26025:6379:126;26128:7;26171:18;;:::i;:::-;26318:4;26311:5;26308:15;26298:134;;26412:1;26409;26402:12;26298:134;26468:4;26462:11;26475:10;26459:27;26449:136;;26565:1;26562;26555:12;26449:136;26634:3;26615:17;26612:26;26602:151;;26733:1;26730;26723:12;26602:151;26798:3;26783:13;26780:22;26770:146;;26896:1;26893;26886:12;26770:146;27176:24;;27521:4;27222:20;27579:2;27280:21;;27176:24;27338:18;27222:20;27280:21;;;27176:24;27153:21;27149:52;;;27338:18;27222:20;;;27280:21;;;27176:24;27149:52;;27222:20;;27280:21;;;27176:24;27149:52;;27338:18;27222:20;27280:21;;;27176:24;27149:52;;27338:18;27222:20;27280:21;;;27176:24;27149:52;;27338:18;27222:20;27280:21;;;27176:24;27149:52;;;27338:18;27222:20;27280:21;;;27176:24;27153:21;27149:52;;;27338:18;27222:20;27280:21;;;27176:24;27149:52;;27338:18;27222:20;27280:21;;;27176:24;27149:52;;27338:18;27222:20;28197:10;27338:18;28187:21;;;27280;;;;28295:1;28280:77;28305:2;28302:1;28299:9;28280:77;;;27176:24;;27153:21;27149:52;27222:20;;28353:1;27280:21;;;;27164:2;27338:18;;;;28323:1;28316:9;28280:77;;;28284:14;;;28435:5;:12;;;28431:71;;;28474:13;:11;:13::i;:::-;28467:20;;;;;28431:71;28516:10;;;:15;;28530:1;28516:15;;;;;28601:8;;;;-1:-1:-1;;28593:20:126;;-1:-1:-1;28593:7:126;:20::i;:::-;28579:34;-1:-1:-1;28643:10:126;28651:2;28643:10;;;;28720:1;28710:11;;;:26;;;28725:6;:11;;28735:1;28725:11;28710:26;28706:310;;;28866:13;28935:1;28913:4;28920:10;28913:17;28912:24;;;;28883:5;:12;;;28898:10;28883:25;28882:54;28866:70;;28961:40;28972:6;:11;;28982:1;28972:11;:20;;28990:2;28972:20;;;28986:1;28972:20;28961:40;;28994:6;28961:10;:40::i;:::-;28954:47;;;;;;;;28706:310;29265:15;;;;29060:9;;;;29197:4;29191:2;29183:10;;;29182:19;;;29265:15;29290:2;29282:10;;;29281:19;29265:36;;;;;;;:::i;:::-;;;;;;-1:-1:-1;29330:5:126;29354:11;;;;;:29;;;29369:6;:14;;29379:4;29369:14;29354:29;29350:832;;;29446:5;:15;;;29462:5;29446:22;;;;;;;;;:::i;:::-;;;;;;-1:-1:-1;;29509:4:126;29503:2;29495:10;;;29494:19;29350:832;;;29547:4;29538:6;:13;;;29534:648;;;29668:6;:13;;29678:3;29668:13;:30;;;;29685:6;:13;;29695:3;29685:13;29668:30;:47;;;;29702:6;:13;;29712:3;29702:13;29668:47;29664:253;;;29778:4;29785:6;29778:13;29773:18;;29534:648;;29664:253;29877:21;29880:4;29887:6;29880:13;29895:2;29877;:21::i;:::-;29872:26;;29534:648;;;29951:4;29941:6;:14;;;;:32;;;;29959:6;:14;;29969:4;29959:14;29941:32;:50;;;;29977:6;:14;;29987:4;29977:14;29941:50;29937:245;;;30061:5;:15;;;30077:5;30061:22;;;;;;;;;:::i;:::-;;;;;30056:27;;30162:5;30154:13;;29937:245;30211:1;30201:6;:11;;;;:25;;;;;30225:1;30216:6;:10;;;30201:25;30200:42;;;;30231:6;:11;;30241:1;30231:11;30200:42;30196:125;;;30269:37;30282:6;30290:4;30296:5;30303:2;30269:12;:37::i;:::-;30262:44;;;;;;;;;;;30196:125;30354:13;30335:16;30506:4;30496:14;;;;30492:446;;30575:21;30578:4;30585:6;30578:13;30593:2;30575;:21::i;:::-;30569:27;;;;30633:10;30628:15;;30667:16;30628:15;30681:1;30667:7;:16::i;:::-;30661:22;;30715:4;30705:6;:14;;;;:32;;;;;30723:6;:14;;30733:4;30723:14;;30705:32;30701:223;;;30802:4;30790:16;;30904:1;30896:9;;30701:223;30512:426;30492:446;30971:10;30984:26;30992:4;30998:2;31002;31006:3;30984:7;:26::i;:::-;31013:10;30984:39;;;;-1:-1:-1;31109:4:126;31102:11;;;31141;;;:24;;;;;31164:1;31156:4;:9;;;;31141:24;:39;;;;;31176:4;31169;:11;;;31141:39;31137:860;;;31204:4;:9;;31212:1;31204:9;:22;;;;31217:4;:9;;31225:1;31217:9;31204:22;31200:144;;;31288:37;31299:4;:9;;31307:1;31299:9;:21;;31315:5;31299:21;;;31311:1;31299:21;31322:2;31288:10;:37::i;:::-;31281:44;;;;;;;;;;;;;;;31200:144;31366:4;:11;;31374:3;31366:11;31362:121;;31436:28;31445:5;31452:2;31456:7;;;;31436:8;:28::i;31362:121::-;31504:4;:11;;31512:3;31504:11;31500:121;;31574:28;31583:5;31590:2;31594:7;;;;;31574:8;:28::i;31500:121::-;31691:4;:11;;31699:3;31691:11;31687:93;;31733:28;31747:13;31733;:28::i;31687:93::-;31883:4;31875;:12;;;;:27;;;;;31898:4;31891;:11;;;31875:27;31871:112;;;31933:31;31944:4;31950:2;31954;31958:5;31933:10;:31::i;31871:112::-;32057:6;:14;;32067:4;32057:14;:28;;;;-1:-1:-1;32075:10:126;;;;;32057:28;32053:93;;;32130:1;32105:5;:15;;;32121:5;32105:22;;;;;;;;;:::i;:::-;:26;;;;:22;;;;;;:26;32053:93;32192:9;:26;;32205:13;32192:26;32188:92;;32238:27;32247:9;32258:1;32261:3;32238:8;:27::i;:::-;32361:26;32370:5;32377:3;32382:4;32361:8;:26::i;:::-;32354:33;;;;;;;;;;;;;26025:6379;;;;;;;;:::o;3087:2334::-;3634:4;3628:11;;3550:4;3353:31;3342:43;;3413:13;3353:31;3752:2;3452:13;;3342:43;3359:24;3353:31;3452:13;;;3342:43;;;;3359:24;3353:31;3452:13;;;3342:43;3359:24;3353:31;3452:13;;;3342:43;3359:24;3353:31;3452:13;;;3342:43;3359:24;3353:31;3452:13;;;3342:43;3359:24;3353:31;3452:13;;;3342:43;3413:13;4180:11;3359:24;3353:31;3452:13;;;3342:43;3413:13;4275:11;3359:24;3353:31;3452:13;;;3342:43;3359:24;3353:31;3452:13;;;3342:43;3128:12;;4415:13;;3628:11;;3452:13;;;;4180:11;3128:12;4495:84;4520:2;4517:1;4514:9;4495:84;;;3369:13;3359:24;;3353:31;3342:43;;3373:2;3413:13;;;;4575:1;3452:13;;;;4538:1;4531:9;4495:84;;;4499:14;4642:1;4638:2;4631:13;4737:5;4733:2;4729:14;4722:5;4717:27;4811:1;4797:15;;4832:6;4856:1;4851:273;;;;5191:1;5181:11;;4825:369;;4851:273;4883:8;4941:22;;;;5020:1;5015:22;;;;5107:1;5097:11;;4876:234;;4941:22;4960:1;4950:11;;4941:22;;5015;5034:1;5024:11;;4876:234;;4825:369;-1:-1:-1;;;5317:14:126;;;5300:32;;5360:19;5356:30;5392:3;5388:16;;;;5353:52;;3087:2334;-1:-1:-1;3087:2334:126:o;21746:1831::-;21819:11;21930:14;21947:24;21959:11;21947;:24::i;:::-;21930:41;;22079:1;22072:5;22068:13;22065:33;;;22094:1;22091;22084:12;22065:33;22227:2;22215:15;;;22168:20;22657:5;22654:1;22650:13;22692:4;22728:1;22713:343;22738:2;22735:1;22732:9;22713:343;;;22861:2;22849:15;;;22798:20;22896:12;;;22910:1;22892:20;22933:42;;;;23001:1;22996:42;;;;22885:153;;22933:42;22391:1;22384:12;;;22424:2;22417:13;;;22469:2;22456:16;;22942:31;;22933:42;;22996;22391:1;22384:12;;;22424:2;22417:13;;;22469:2;22456:16;;23005:31;;22885:153;-1:-1:-1;;22756:1:126;22749:9;22713:343;;;22717:14;23166:4;23160:11;23145:26;;23252:7;23246:4;23243:17;23233:124;;23294:10;23291:1;23284:21;23336:2;23333:1;23326:13;23233:124;-1:-1:-1;;23484:2:126;23473:14;;;;23461:10;23457:31;23454:1;23450:39;23518:16;;;;23536:10;23514:33;;21746:1831;-1:-1:-1;;;21746:1831:126:o;18856:823::-;18925:12;19012:18;;:::i;:::-;19080:4;19071:13;;19132:5;:8;;;19143:1;19132:12;19116:28;;:5;:12;;;:28;;;19112:95;;19164:28;;;;;2182:2:285;19164:28:126;;;2164:21:285;2221:2;2201:18;;;2194:30;2260:20;2240:18;;;2233:48;2298:18;;19164:28:126;;;;;;;;19112:95;19296:8;;;;;19329:12;;;;;19318:23;;;;;;;19355:20;;;;;19296:8;19487:13;;;19483:90;;19548:6;19557:1;19548:10;19520:5;:15;;;19536:8;19520:25;;;;;;;;;:::i;:::-;:38;;;;:25;;;;;;:38;19483:90;19649:13;:11;:13::i;2645:339::-;2706:11;2770:18;;;;2779:8;;;;2770:18;;;;;;2769:25;;;;;2786:1;2833:2;:9;;;2827:16;;;;;2826:22;;2825:32;;;;;;;2887:9;;2886:15;2769:25;2944:21;;2964:1;2944:21;;;2955:6;2944:21;2929:11;;;;;:37;;-1:-1:-1;;;2645:339:126;;;;:::o;13732:2026::-;13829:12;13915:18;;:::i;:::-;13983:4;13974:13;;14015:17;14075:5;:8;;;14086:1;14075:12;14059:28;;:5;:12;;;:28;;;14055:97;;14107:30;;;;;2529:2:285;14107:30:126;;;2511:21:285;2568:2;2548:18;;;2541:30;2607:22;2587:18;;;2580:50;2647:18;;14107:30:126;2327:344:285;14055:97:126;14222:7;:12;;14233:1;14222:12;:28;;;;14238:7;:12;;14249:1;14238:12;14222:28;14218:947;;;14270:9;14282:5;:15;;;14298:6;14282:23;;;;;;;;;:::i;:::-;;;;;14270:35;;14346:2;14339:9;;:3;:9;;;:25;;;;;14352:7;:12;;14363:1;14352:12;14339:25;14338:58;;;;14377:2;14370:9;;:3;:9;;;;:25;;;;;14383:7;:12;;14394:1;14383:12;14370:25;14323:73;;14252:159;14218:947;;;14508:7;:12;;14519:1;14508:12;14504:661;;14569:1;14561:3;14555:15;;;;14540:30;;14504:661;;;14673:7;:12;;14684:1;14673:12;14669:496;;14733:1;14726:3;14720:14;;;14705:29;;14669:496;;;14854:7;:12;;14865:1;14854:12;14850:315;;14942:4;14936:2;14927:11;;;14926:20;14912:10;14969:8;;;14965:84;;15029:1;15022:3;15016:14;;;15001:29;;14965:84;15070:3;:8;;15077:1;15070:8;15066:85;;15131:1;15123:3;15117:15;;;;15102:30;;15066:85;14868:297;14850:315;15241:8;;;;;15319:12;;;;15308:23;;;;;15475:178;;;;15566:1;15540:22;15543:5;15551:6;15543:14;15559:2;15540;:22::i;:::-;:27;;;;;;;15526:42;;15535:1;15526:42;15511:57;:12;;;:57;15475:178;;;15622:12;;;;;15637:1;15622:16;15607:31;;;;15475:178;15728:13;:11;:13::i;:::-;15721:20;13732:2026;-1:-1:-1;;;;;;;;13732:2026:126:o;32450:8733::-;32537:10;32599;32607:2;32599:10;;;;32638:11;;;:44;;;32664:1;32654:6;:11;;;;:27;;;;;32678:3;32669:6;:12;;;32654:27;32634:8490;;;32723:4;32716:11;;32847:6;32907:3;32902:25;;;;32982:3;32977:25;;;;33056:3;33051:25;;;;33131:3;33126:25;;;;33205:3;33200:25;;;;33278:3;33273:25;;;;33352:3;33347:25;;;;32840:532;;32902:25;32921:4;32913:12;;32902:25;;32977;32996:4;32988:12;;32977:25;;33051;33070:4;33062:12;;33051:25;;33126;33145:4;33137:12;;33126:25;;33200;33219:4;33211:12;;33200:25;;33273;33292:4;33284:12;;33273:25;;33347;33366:4;33358:12;;32840:532;;33435:4;:12;;33443:4;33435:12;33431:4023;;-1:-1:-1;;;33486:9:126;33478:26;;33499:4;33494:1;33486:9;;;33485:18;33478:26;33471:33;;33431:4023;33572:4;:12;;33580:4;33572:12;33568:3886;;-1:-1:-1;;;33623:9:126;33615:26;;33636:4;33631:1;33623:9;;;33622:18;33615:26;33608:33;;33568:3886;33709:4;:12;;33717:4;33709:12;33705:3749;;33774:4;33769:1;33761:9;;;33760:18;33807:27;33761:9;33810:11;;;;33823:2;:10;;;33807:2;:27::i;:::-;33800:34;;;;;;;33705:3749;33903:4;:12;;33911:4;33903:12;33899:3555;;-1:-1:-1;;;33946:17:126;;;33958:4;33953:9;;33946:17;33939:24;;33899:3555;34032:4;:11;;34040:3;34032:11;34028:3426;;-1:-1:-1;;;34074:17:126;;;34086:4;34081:9;;34074:17;34067:24;;34028:3426;34160:4;:12;;34168:4;34160:12;34156:3298;;34203:21;34212:2;34206:8;;:2;:8;;;;34221:2;34216;:7;34203:2;:21::i;:::-;34196:28;;;;;;34156:3298;34473:4;:12;;34481:4;34473:12;34469:2985;;34516:2;34509:9;;;;;;34469:2985;34587:4;:12;;34595:4;34587:12;34583:2871;;34630:2;34623:9;;;;;;34583:2871;34701:4;:12;;34709:4;34701:12;34697:2757;;34744:2;34737:9;;;;;;34697:2757;34815:4;:12;;34823:4;34815:12;34811:2643;;34858:2;34851:9;;;;;;34811:2643;34932:4;:12;;34940:4;34932:12;34928:2526;;34975:2;34968:9;;;;;;34928:2526;35092:4;:12;;35100:4;35092:12;35088:2366;;35135:2;35128:9;;;;;;35088:2366;35206:4;:12;;35214:4;35206:12;35202:2252;;35249:2;35242:9;;;;;;35202:2252;35320:4;:12;;35328:4;35320:12;35316:2138;;35363:2;35356:9;;;;;;35316:2138;35434:4;:12;;35442:4;35434:12;35430:2024;;35477:2;35470:9;;;;;;35430:2024;35548:4;:12;;35556:4;35548:12;35544:1910;;35591:2;35584:9;;;;;;35544:1910;35662:4;:12;;35670:4;35662:12;35658:1796;;35705:2;35698:9;;;;;;35658:1796;35777:4;:12;;35785:4;35777:12;35773:1681;;35820:2;35813:9;;;;;;35773:1681;35890:4;:12;;35898:4;35890:12;35886:1568;;35933:2;35926:9;;;;;;35886:1568;36004:4;:12;;36012:4;36004:12;36000:1454;;36047:2;36040:9;;;;;;36000:1454;36196:4;:12;;36204:4;36196:12;36192:1262;;-1:-1:-1;;;36240:7:126;;;36232:16;;36192:1262;36317:4;:12;;36325:4;36317:12;36313:1141;;-1:-1:-1;;;36361:7:126;;;36353:16;;36313:1141;36437:4;:12;;36445:4;36437:12;36433:1021;;-1:-1:-1;;;36481:7:126;;;36473:16;;36433:1021;36558:4;:12;;36566:4;36558:12;36554:900;;-1:-1:-1;;;36602:7:126;;;36594:16;;36554:900;36678:4;:12;;36686:4;36678:12;36674:780;;-1:-1:-1;;;36722:7:126;;;36714:16;;36674:780;36797:4;:12;;36805:4;36797:12;36793:661;;-1:-1:-1;;;36841:7:126;;;36833:16;;36793:661;36917:4;:12;;36925:4;36917:12;36913:541;;-1:-1:-1;;;36961:7:126;;;36953:16;;36913:541;37037:4;:12;;37045:4;37037:12;37033:421;;-1:-1:-1;;;37082:7:126;;;37080:10;37073:17;;37033:421;37159:4;:12;;37167:4;37159:12;37155:299;;37220:2;37202:21;;37208:2;37202:21;;;:29;;37230:1;37202:29;;;37226:1;37202:29;37195:36;;;;;;;;37155:299;37301:4;:12;;37309:4;37301:12;37297:157;;37349:2;37344:7;;:2;:7;;;:15;;37358:1;37344:15;;37297:157;37406:29;;;;;2878:2:285;37406:29:126;;;2860:21:285;2917:2;2897:18;;;2890:30;2956:21;2936:18;;;2929:49;2995:18;;37406:29:126;2676:343:285;37297:157:126;32684:4784;32634:8490;;;37524:6;:14;;37534:4;37524:14;37520:3590;;37583:4;37576:11;;37658:3;37650:11;;;37646:549;;-1:-1:-1;;;37703:21:126;;;37689:36;;37646:549;37810:4;:12;;37818:4;37810:12;:28;;;;37826:4;:12;;37834:4;37826:12;37810:28;37806:389;;;37870:4;:12;;37878:4;37870:12;37866:83;;37919:3;;;37866:83;37974:8;38012:127;38024:10;38019:15;;:20;38012:127;;38104:8;38071:3;38104:8;;;;;38071:3;38012:127;;;38171:1;-1:-1:-1;38164:8:126;;-1:-1:-1;;38164:8:126;37520:3590;38262:6;:14;;38272:4;38262:14;38258:2852;;-1:-1:-1;;38307:8:126;38313:2;38307:8;;;;38300:15;;38258:2852;38382:6;:14;;38392:4;38382:14;38378:2732;;38427:42;38445:2;38450:1;38445:6;38455:1;38444:12;38439:2;:17;38431:26;;:3;:26;;;;38461:4;38430:35;38467:1;38427:2;:42::i;:::-;38420:49;;;;;38378:2732;38536:6;:14;;38546:4;38536:14;38532:2578;;38581:45;38599:2;38604:1;38599:6;38609:1;38598:12;38593:2;:17;38585:26;;:3;:26;;;;38615:6;38584:37;38623:2;38581;:45::i;38532:2578::-;38694:6;:14;;38704:4;38694:14;38690:2420;;-1:-1:-1;;38745:21:126;38764:1;38759;38754:6;;38753:12;38745:21;;38802:36;;;38873:5;38868:10;;38745:21;;;;;38867:18;38860:25;;38690:2420;38952:6;:14;;38962:4;38952:14;38948:2162;;38997:3;38990:10;;;;;38948:2162;39068:6;:14;;39078:4;39068:14;39064:2046;;39128:2;39133:1;39128:6;39138:1;39127:12;39122:2;:17;39114:26;;:3;:26;;;;39144:4;39113:35;39106:42;;;;;39064:2046;39217:6;:14;;39227:4;39217:14;39213:1897;;39277:2;39282:1;39277:6;39287:1;39276:12;39271:2;:17;39263:26;;:3;:26;;;;39293:6;39262:37;39255:44;;;;;39213:1897;39368:6;:14;;39378:4;39368:14;39364:1746;;-1:-1:-1;;39419:26:126;39443:1;39438;39433:6;;39432:12;39427:2;:17;39419:26;;39481:41;;;39557:5;39552:10;;39419:26;;;;;39551:18;39544:25;;39364:1746;39637:6;:14;;39647:4;39637:14;39633:1477;;-1:-1:-1;;39694:4:126;39688:34;39720:1;39715;39710:6;;39709:12;39704:2;:17;39688:34;;39778:27;;;39758:48;;;39836:10;;39689:9;;;39688:34;;39835:18;39828:25;;39633:1477;39921:6;:14;;39931:4;39921:14;39917:1193;;-1:-1:-1;;39978:6:126;39972:36;40006:1;40001;39996:6;;39995:12;39990:2;:17;39972:36;;40064:29;;;40044:50;;;40124:10;;39973:11;;;39972:36;;40123:18;40116:25;;39917:1193;40210:6;:14;;40220:4;40210:14;40206:904;;-1:-1:-1;;40261:20:126;40279:1;40274;40269:6;;40268:12;40261:20;;40317:36;;;40389:5;40383:11;;40261:20;;;;;40382:19;40375:26;;40206:904;40469:6;:14;;40479:4;40469:14;40465:645;;40514:2;40507:9;;;;;40465:645;40585:6;:14;;40595:4;40585:14;40581:529;;-1:-1:-1;;40636:25:126;40659:1;40654;40649:6;;40648:12;40643:2;:17;40636:25;;40697:41;;;40774:5;40768:11;;40636:25;;;;;40767:19;40760:26;;40581:529;40853:6;:14;;40863:4;40853:14;40849:261;;40898:3;40891:10;;;;;40849:261;40968:6;:14;;40978:4;40968:14;40964:146;;41013:2;41006:9;;;32450:8733;;;;;;;:::o;19960:782::-;20046:12;20133:18;;:::i;:::-;-1:-1:-1;20201:4:126;20308:2;20296:14;;;;20288:41;;;;;;;3226:2:285;20288:41:126;;;3208:21:285;3265:2;3245:18;;;3238:30;3304:16;3284:18;;;3277:44;3338:18;;20288:41:126;3024:338:285;20288:41:126;20425:14;;;;;;;:30;;;20443:12;20425:30;20421:102;;;20504:4;20475:5;:15;;;20491:9;20475:26;;;;;;;;;:::i;:::-;:33;;;;:26;;;;;;:33;20421:102;20578:12;;;;;20567:23;;;;:8;;;:23;20634:1;20619:16;;;20604:31;;;20712:13;:11;:13::i;5582:7764::-;5646:12;5732:18;;:::i;:::-;-1:-1:-1;5910:15:126;;:18;;;;5800:4;6070:18;;;;6114;;;;6158;;;;;5800:4;;5890:17;;;;6070:18;6114;6248;;;6262:4;6248:18;6244:6792;;6298:2;6327:4;6322:9;;:14;6318:144;;6438:4;6433:9;;6425:4;:18;6419:24;6318:144;6483:2;:7;;6489:1;6483:7;6479:161;;6519:10;;;;;6551:16;;;;;;;;6519:10;-1:-1:-1;6479:161:126;;;6619:2;6614:7;;6479:161;6268:386;6244:6792;;;6756:10;:18;;6770:4;6756:18;6752:6284;;1745:10;6794:14;;6752:6284;;;6892:10;:18;;6906:4;6892:18;6888:6148;;6935:1;6930:6;;6888:6148;;;7060:10;:18;;7074:4;7060:18;7056:5980;;7113:4;7098:12;;;:19;7135:26;;;:14;;;:26;7186:13;:11;:13::i;:::-;7179:20;5582:7764;-1:-1:-1;;;;;;;;;5582:7764:126:o;7056:5980::-;7325:10;:18;;7339:4;7325:18;7321:5715;;7476:14;;;7472:2723;7321:5715;7472:2723;7646:22;;;;;7642:2553;;7771:10;7784:27;7792:2;7797:10;7792:15;7809:1;7784:7;:27::i;:::-;7895:17;;;;7771:40;;-1:-1:-1;7895:17:126;7873:19;8045:14;8064:1;8039:26;8035:146;;1676:4:127;1670:11;;1533:21;1787:15;;;1828:8;1822:4;1815:22;1850:27;;;1996:4;1983:18;;2098:17;;2003:19;1979:44;2025:11;1976:61;8093:65:126;;8035:146;8267:20;;;;;8234:54;;;;;;;;3540:25:285;;;8234:54:126;3601:23:285;;;3581:18;;;3574:51;8203:11:126;;;;8234:19;:6;:19;;;;3513:18:285;;8234:54:126;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;:::i;:::-;8202:86;;;;8515:1;8511:2;8507:10;8612:9;8609:1;8605:17;8694:6;8687:5;8684:17;8681:40;;;8714:5;8704:15;;8681:40;;8797:6;8793:2;8790:14;8787:34;;;8817:2;8807:12;;8787:34;8923:3;8918:1;8910:6;8906:14;8901:3;8897:24;8893:34;8886:41;;9023:3;9019:1;9007:9;8998:6;8995:1;8991:14;8987:30;8983:38;8979:48;8972:55;;9178:1;9174;9170;9158:9;9155:1;9151:17;9147:25;9143:33;9139:41;9305:1;9301;9297;9288:6;9276:9;9273:1;9269:17;9265:30;9261:38;9257:46;9253:54;9235:72;;9436:10;9432:15;9426:4;9422:26;9414:34;;9552:3;9544:4;9540:9;9535:3;9531:19;9528:28;9521:35;;;;9698:33;9707:2;9712:10;9707:15;9724:1;9727:3;9698:8;:33::i;:::-;9753:20;;;:38;;;;;;;;;-1:-1:-1;7642:2553:126;;-1:-1:-1;;;7642:2553:126;;9910:18;;;;;9906:289;;10080:2;10075:7;;7321:5715;;9906:289;10134:10;10129:15;;2053:3;10166:10;;9906:289;7321:5715;;;10324:10;:18;;10338:4;10324:18;10320:2716;;10478:15;;;1824:1;10478:15;;:34;;-1:-1:-1;10497:15:126;;;1859:1;10497:15;10478:34;:57;;;-1:-1:-1;10516:19:126;;;1936:1;10516:19;10478:57;10474:1593;;;10564:2;10559:7;;10320:2716;;10474:1593;10690:23;;;;;10686:1381;;10737:10;10750:27;10758:2;10763:10;10758:15;10775:1;10750:7;:27::i;:::-;10853:17;;;;10737:40;;-1:-1:-1;11096:1:126;11088:10;;11190:1;11186:17;11265:13;;;11262:32;;;11287:5;11281:11;;11262:32;11573:14;;;11379:1;11569:22;;;11565:32;;;;11462:26;11486:1;11371:10;;;11466:18;;;11462:26;11561:43;11367:20;;11669:12;11797:17;;;:23;11865:1;11842:20;;;:24;11375:2;-1:-1:-1;11375:2:126;7321:5715;;10320:2716;12269:10;:18;;12283:4;12269:18;12265:771;;12379:2;:7;;12385:1;12379:7;12375:647;;12472:14;;;;;:40;;-1:-1:-1;12490:22:126;;;1978:1;12490:22;12472:40;:62;;;-1:-1:-1;12516:18:126;;;1897:1;12516:18;12472:62;12468:404;;;12567:1;12562:6;;12375:647;;12468:404;12613:15;;;1824:1;12613:15;;:34;;-1:-1:-1;12632:15:126;;;1859:1;12632:15;12613:34;:61;;;-1:-1:-1;12651:23:126;;;2021:1;12651:23;12613:61;:84;;;-1:-1:-1;12678:19:126;;;1936:1;12678:19;12613:84;12609:263;;;12730:1;12725:6;;7321:5715;;12375:647;12923:10;12918:15;;2087:4;12955:11;;12375:647;13111:15;;;;;:23;;;;:18;;;;:23;;;;13148:15;;:23;;;:18;;;;:23;-1:-1:-1;13237:12:126;;;;13226:23;;;:8;;;:23;13293:1;13278:16;13263:31;;;;;13316:13;:11;:13::i;16084:2480::-;16178:12;16264:18;;:::i;:::-;-1:-1:-1;16332:4:126;16364:10;16472:13;;;16481:4;16472:13;16468:1705;;-1:-1:-1;16511:8:126;;;;16468:1705;;;16630:5;:13;;16639:4;16630:13;16626:1547;;16663:14;;;:8;;;:14;16626:1547;;;16793:5;:13;;16802:4;16793:13;16789:1384;;-1:-1:-1;16832:8:126;;;;16789:1384;;;16951:5;:13;;16960:4;16951:13;16947:1226;;16984:14;;;:8;;;:14;16947:1226;;;17125:5;:13;;17134:4;17125:13;17121:1052;;17252:9;17198:17;17178;;;17198;;;;17178:37;17259:2;17252:9;;;;;17234:8;;;:28;17280:22;:8;;;:22;17121:1052;;;17439:5;:13;;17448:4;17439:13;17435:738;;17506:11;17492;;;17506;;;17492:25;17561:2;17554:9;;;;;17536:8;;;:28;17582:22;:8;;;:22;17435:738;;;17763:5;:13;;17772:4;17763:13;17759:414;;17833:3;17814:23;;17820:3;17814:23;;;;;;;:::i;:::-;;17796:42;;:8;;;:42;17874:23;;;;;;;;;;;;;:::i;:::-;;17856:42;;:8;;;:42;17759:414;;;18067:5;:13;;18076:4;18067:13;18063:110;;18117:3;18111:9;;:3;:9;;;;;;;:::i;:::-;;18100:20;;;;:8;;;:20;18149:9;;;;;;;;;;;:::i;:::-;;18138:20;;:8;;;:20;18063:110;18266:14;;;;18262:85;;18329:3;18300:5;:15;;;18316:9;18300:26;;;;;;;;;:::i;:::-;:32;;;;:26;;;;;;:32;18262:85;18401:12;;;;;18390:23;;;;:8;;;:23;18457:1;18442:16;;;18427:31;;;18534:13;:11;:13::i;:::-;18527:20;16084:2480;-1:-1:-1;;;;;;;16084:2480:126:o;23913:1654::-;24089:14;24106:24;24118:11;24106;:24::i;:::-;24089:41;;24238:1;24231:5;24227:13;24224:33;;;24253:1;24250;24243:12;24224:33;24392:2;24586:15;;;24411:2;24400:14;;24388:10;24384:31;24381:1;24377:39;24542:16;;;24327:20;;24527:10;24516:22;;;24512:27;24502:38;24499:60;25028:5;25025:1;25021:13;25099:1;25084:343;25109:2;25106:1;25103:9;25084:343;;;25232:2;25220:15;;;25169:20;25267:12;;;25281:1;25263:20;25304:42;;;;25372:1;25367:42;;;;25256:153;;25304:42;22391:1;22384:12;;;22424:2;22417:13;;;22469:2;22456:16;;25313:31;;25304:42;;25367;22391:1;22384:12;;;22424:2;22417:13;;;22469:2;22456:16;;25376:31;;25256:153;-1:-1:-1;;25127:1:126;25120:9;25084:343;;;-1:-1:-1;;25526:4:126;25519:18;-1:-1:-1;;;;23913:1654:126:o;20946:586::-;21268:20;;;21292:7;21268:32;21261:3;:40;;;21374:14;;21429:17;;21423:24;;;21415:72;;;;;;;4277:2:285;21415:72:126;;;4259:21:285;4316:2;4296:18;;;4289:30;4355:34;4335:18;;;4328:62;4426:5;4406:18;;;4399:33;4449:19;;21415:72:126;4075:399:285;21415:72:126;21501:14;20946:586;;;:::o;-1:-1:-1:-;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;:::i;:::-;;;;:::o;:::-;;;;;;;;;;;;;;;;;;;;;;;;:::o;467:347:285:-;518:8;528:6;582:3;575:4;567:6;563:17;559:27;549:55;;600:1;597;590:12;549:55;-1:-1:-1;623:20:285;;666:18;655:30;;652:50;;;698:1;695;688:12;652:50;735:4;727:6;723:17;711:29;;787:3;780:4;771:6;763;759:19;755:30;752:39;749:59;;;804:1;801;794:12;749:59;467:347;;;;;:::o;819:785::-;918:6;926;934;942;950;1003:2;991:9;982:7;978:23;974:32;971:52;;;1019:1;1016;1009:12;971:52;1059:9;1046:23;1088:18;1129:2;1121:6;1118:14;1115:34;;;1145:1;1142;1135:12;1115:34;1184:58;1234:7;1225:6;1214:9;1210:22;1184:58;:::i;:::-;1261:8;;-1:-1:-1;1158:84:285;-1:-1:-1;1349:2:285;1334:18;;1321:32;;-1:-1:-1;1365:16:285;;;1362:36;;;1394:1;1391;1384:12;1362:36;;1433:60;1485:7;1474:8;1463:9;1459:24;1433:60;:::i;:::-;819:785;;;;-1:-1:-1;1512:8:285;1594:2;1579:18;1566:32;;819:785;-1:-1:-1;;;;819:785:285:o;1791:184::-;1843:77;1840:1;1833:88;1940:4;1937:1;1930:15;1964:4;1961:1;1954:15;3636:245;3715:6;3723;3776:2;3764:9;3755:7;3751:23;3747:32;3744:52;;;3792:1;3789;3782:12;3744:52;-1:-1:-1;;3815:16:285;;3871:2;3856:18;;;3850:25;3815:16;;3850:25;;-1:-1:-1;3636:245:285:o;3886:184::-;3938:77;3935:1;3928:88;4035:4;4032:1;4025:15;4059:4;4056:1;4049:15" +var MIPSDeployedSourceMap = "1131:40054:135:-:0;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;1710:45;;1745:10;1710:45;;;;;188:10:299;176:23;;;158:42;;146:2;131:18;1710:45:135;;;;;;;;2448:99;;;412:42:299;2534:6:135;400:55:299;382:74;;370:2;355:18;2448:99:135;211:251:299;26025:6379:135;;;;;;:::i;:::-;;:::i;:::-;;;1755:25:299;;;1743:2;1728:18;26025:6379:135;1609:177:299;26025:6379:135;26128:7;26171:18;;:::i;:::-;26318:4;26311:5;26308:15;26298:134;;26412:1;26409;26402:12;26298:134;26468:4;26462:11;26475:10;26459:27;26449:136;;26565:1;26562;26555:12;26449:136;26634:3;26615:17;26612:26;26602:151;;26733:1;26730;26723:12;26602:151;26798:3;26783:13;26780:22;26770:146;;26896:1;26893;26886:12;26770:146;27176:24;;27521:4;27222:20;27579:2;27280:21;;27176:24;27338:18;27222:20;27280:21;;;27176:24;27153:21;27149:52;;;27338:18;27222:20;;;27280:21;;;27176:24;27149:52;;27222:20;;27280:21;;;27176:24;27149:52;;27338:18;27222:20;27280:21;;;27176:24;27149:52;;27338:18;27222:20;27280:21;;;27176:24;27149:52;;27338:18;27222:20;27280:21;;;27176:24;27149:52;;;27338:18;27222:20;27280:21;;;27176:24;27153:21;27149:52;;;27338:18;27222:20;27280:21;;;27176:24;27149:52;;27338:18;27222:20;27280:21;;;27176:24;27149:52;;27338:18;27222:20;28197:10;27338:18;28187:21;;;27280;;;;28295:1;28280:77;28305:2;28302:1;28299:9;28280:77;;;27176:24;;27153:21;27149:52;27222:20;;28353:1;27280:21;;;;27164:2;27338:18;;;;28323:1;28316:9;28280:77;;;28284:14;;;28435:5;:12;;;28431:71;;;28474:13;:11;:13::i;:::-;28467:20;;;;;28431:71;28516:10;;;:15;;28530:1;28516:15;;;;;28601:8;;;;-1:-1:-1;;28593:20:135;;-1:-1:-1;28593:7:135;:20::i;:::-;28579:34;-1:-1:-1;28643:10:135;28651:2;28643:10;;;;28720:1;28710:11;;;:26;;;28725:6;:11;;28735:1;28725:11;28710:26;28706:310;;;28866:13;28935:1;28913:4;28920:10;28913:17;28912:24;;;;28883:5;:12;;;28898:10;28883:25;28882:54;28866:70;;28961:40;28972:6;:11;;28982:1;28972:11;:20;;28990:2;28972:20;;;28986:1;28972:20;28961:40;;28994:6;28961:10;:40::i;:::-;28954:47;;;;;;;;28706:310;29265:15;;;;29060:9;;;;29197:4;29191:2;29183:10;;;29182:19;;;29265:15;29290:2;29282:10;;;29281:19;29265:36;;;;;;;:::i;:::-;;;;;;-1:-1:-1;29330:5:135;29354:11;;;;;:29;;;29369:6;:14;;29379:4;29369:14;29354:29;29350:832;;;29446:5;:15;;;29462:5;29446:22;;;;;;;;;:::i;:::-;;;;;;-1:-1:-1;;29509:4:135;29503:2;29495:10;;;29494:19;29350:832;;;29547:4;29538:6;:13;;;29534:648;;;29668:6;:13;;29678:3;29668:13;:30;;;;29685:6;:13;;29695:3;29685:13;29668:30;:47;;;;29702:6;:13;;29712:3;29702:13;29668:47;29664:253;;;29778:4;29785:6;29778:13;29773:18;;29534:648;;29664:253;29877:21;29880:4;29887:6;29880:13;29895:2;29877;:21::i;:::-;29872:26;;29534:648;;;29951:4;29941:6;:14;;;;:32;;;;29959:6;:14;;29969:4;29959:14;29941:32;:50;;;;29977:6;:14;;29987:4;29977:14;29941:50;29937:245;;;30061:5;:15;;;30077:5;30061:22;;;;;;;;;:::i;:::-;;;;;30056:27;;30162:5;30154:13;;29937:245;30211:1;30201:6;:11;;;;:25;;;;;30225:1;30216:6;:10;;;30201:25;30200:42;;;;30231:6;:11;;30241:1;30231:11;30200:42;30196:125;;;30269:37;30282:6;30290:4;30296:5;30303:2;30269:12;:37::i;:::-;30262:44;;;;;;;;;;;30196:125;30354:13;30335:16;30506:4;30496:14;;;;30492:446;;30575:21;30578:4;30585:6;30578:13;30593:2;30575;:21::i;:::-;30569:27;;;;30633:10;30628:15;;30667:16;30628:15;30681:1;30667:7;:16::i;:::-;30661:22;;30715:4;30705:6;:14;;;;:32;;;;;30723:6;:14;;30733:4;30723:14;;30705:32;30701:223;;;30802:4;30790:16;;30904:1;30896:9;;30701:223;30512:426;30492:446;30971:10;30984:26;30992:4;30998:2;31002;31006:3;30984:7;:26::i;:::-;31013:10;30984:39;;;;-1:-1:-1;31109:4:135;31102:11;;;31141;;;:24;;;;;31164:1;31156:4;:9;;;;31141:24;:39;;;;;31176:4;31169;:11;;;31141:39;31137:860;;;31204:4;:9;;31212:1;31204:9;:22;;;;31217:4;:9;;31225:1;31217:9;31204:22;31200:144;;;31288:37;31299:4;:9;;31307:1;31299:9;:21;;31315:5;31299:21;;;31311:1;31299:21;31322:2;31288:10;:37::i;:::-;31281:44;;;;;;;;;;;;;;;31200:144;31366:4;:11;;31374:3;31366:11;31362:121;;31436:28;31445:5;31452:2;31456:7;;;;31436:8;:28::i;31362:121::-;31504:4;:11;;31512:3;31504:11;31500:121;;31574:28;31583:5;31590:2;31594:7;;;;;31574:8;:28::i;31500:121::-;31691:4;:11;;31699:3;31691:11;31687:93;;31733:28;31747:13;31733;:28::i;31687:93::-;31883:4;31875;:12;;;;:27;;;;;31898:4;31891;:11;;;31875:27;31871:112;;;31933:31;31944:4;31950:2;31954;31958:5;31933:10;:31::i;31871:112::-;32057:6;:14;;32067:4;32057:14;:28;;;;-1:-1:-1;32075:10:135;;;;;32057:28;32053:93;;;32130:1;32105:5;:15;;;32121:5;32105:22;;;;;;;;;:::i;:::-;:26;;;;:22;;;;;;:26;32053:93;32192:9;:26;;32205:13;32192:26;32188:92;;32238:27;32247:9;32258:1;32261:3;32238:8;:27::i;:::-;32361:26;32370:5;32377:3;32382:4;32361:8;:26::i;:::-;32354:33;;;;;;;;;;;;;26025:6379;;;;;;;;:::o;3087:2334::-;3634:4;3628:11;;3550:4;3353:31;3342:43;;3413:13;3353:31;3752:2;3452:13;;3342:43;3359:24;3353:31;3452:13;;;3342:43;;;;3359:24;3353:31;3452:13;;;3342:43;3359:24;3353:31;3452:13;;;3342:43;3359:24;3353:31;3452:13;;;3342:43;3359:24;3353:31;3452:13;;;3342:43;3359:24;3353:31;3452:13;;;3342:43;3413:13;4180:11;3359:24;3353:31;3452:13;;;3342:43;3413:13;4275:11;3359:24;3353:31;3452:13;;;3342:43;3359:24;3353:31;3452:13;;;3342:43;3128:12;;4415:13;;3628:11;;3452:13;;;;4180:11;3128:12;4495:84;4520:2;4517:1;4514:9;4495:84;;;3369:13;3359:24;;3353:31;3342:43;;3373:2;3413:13;;;;4575:1;3452:13;;;;4538:1;4531:9;4495:84;;;4499:14;4642:1;4638:2;4631:13;4737:5;4733:2;4729:14;4722:5;4717:27;4811:1;4797:15;;4832:6;4856:1;4851:273;;;;5191:1;5181:11;;4825:369;;4851:273;4883:8;4941:22;;;;5020:1;5015:22;;;;5107:1;5097:11;;4876:234;;4941:22;4960:1;4950:11;;4941:22;;5015;5034:1;5024:11;;4876:234;;4825:369;-1:-1:-1;;;5317:14:135;;;5300:32;;5360:19;5356:30;5392:3;5388:16;;;;5353:52;;3087:2334;-1:-1:-1;3087:2334:135:o;21746:1831::-;21819:11;21930:14;21947:24;21959:11;21947;:24::i;:::-;21930:41;;22079:1;22072:5;22068:13;22065:33;;;22094:1;22091;22084:12;22065:33;22227:2;22215:15;;;22168:20;22657:5;22654:1;22650:13;22692:4;22728:1;22713:343;22738:2;22735:1;22732:9;22713:343;;;22861:2;22849:15;;;22798:20;22896:12;;;22910:1;22892:20;22933:42;;;;23001:1;22996:42;;;;22885:153;;22933:42;22391:1;22384:12;;;22424:2;22417:13;;;22469:2;22456:16;;22942:31;;22933:42;;22996;22391:1;22384:12;;;22424:2;22417:13;;;22469:2;22456:16;;23005:31;;22885:153;-1:-1:-1;;22756:1:135;22749:9;22713:343;;;22717:14;23166:4;23160:11;23145:26;;23252:7;23246:4;23243:17;23233:124;;23294:10;23291:1;23284:21;23336:2;23333:1;23326:13;23233:124;-1:-1:-1;;23484:2:135;23473:14;;;;23461:10;23457:31;23454:1;23450:39;23518:16;;;;23536:10;23514:33;;21746:1831;-1:-1:-1;;;21746:1831:135:o;18856:823::-;18925:12;19012:18;;:::i;:::-;19080:4;19071:13;;19132:5;:8;;;19143:1;19132:12;19116:28;;:5;:12;;;:28;;;19112:95;;19164:28;;;;;2182:2:299;19164:28:135;;;2164:21:299;2221:2;2201:18;;;2194:30;2260:20;2240:18;;;2233:48;2298:18;;19164:28:135;;;;;;;;19112:95;19296:8;;;;;19329:12;;;;;19318:23;;;;;;;19355:20;;;;;19296:8;19487:13;;;19483:90;;19548:6;19557:1;19548:10;19520:5;:15;;;19536:8;19520:25;;;;;;;;;:::i;:::-;:38;;;;:25;;;;;;:38;19483:90;19649:13;:11;:13::i;2645:339::-;2706:11;2770:18;;;;2779:8;;;;2770:18;;;;;;2769:25;;;;;2786:1;2833:2;:9;;;2827:16;;;;;2826:22;;2825:32;;;;;;;2887:9;;2886:15;2769:25;2944:21;;2964:1;2944:21;;;2955:6;2944:21;2929:11;;;;;:37;;-1:-1:-1;;;2645:339:135;;;;:::o;13732:2026::-;13829:12;13915:18;;:::i;:::-;13983:4;13974:13;;14015:17;14075:5;:8;;;14086:1;14075:12;14059:28;;:5;:12;;;:28;;;14055:97;;14107:30;;;;;2529:2:299;14107:30:135;;;2511:21:299;2568:2;2548:18;;;2541:30;2607:22;2587:18;;;2580:50;2647:18;;14107:30:135;2327:344:299;14055:97:135;14222:7;:12;;14233:1;14222:12;:28;;;;14238:7;:12;;14249:1;14238:12;14222:28;14218:947;;;14270:9;14282:5;:15;;;14298:6;14282:23;;;;;;;;;:::i;:::-;;;;;14270:35;;14346:2;14339:9;;:3;:9;;;:25;;;;;14352:7;:12;;14363:1;14352:12;14339:25;14338:58;;;;14377:2;14370:9;;:3;:9;;;;:25;;;;;14383:7;:12;;14394:1;14383:12;14370:25;14323:73;;14252:159;14218:947;;;14508:7;:12;;14519:1;14508:12;14504:661;;14569:1;14561:3;14555:15;;;;14540:30;;14504:661;;;14673:7;:12;;14684:1;14673:12;14669:496;;14733:1;14726:3;14720:14;;;14705:29;;14669:496;;;14854:7;:12;;14865:1;14854:12;14850:315;;14942:4;14936:2;14927:11;;;14926:20;14912:10;14969:8;;;14965:84;;15029:1;15022:3;15016:14;;;15001:29;;14965:84;15070:3;:8;;15077:1;15070:8;15066:85;;15131:1;15123:3;15117:15;;;;15102:30;;15066:85;14868:297;14850:315;15241:8;;;;;15319:12;;;;15308:23;;;;;15475:178;;;;15566:1;15540:22;15543:5;15551:6;15543:14;15559:2;15540;:22::i;:::-;:27;;;;;;;15526:42;;15535:1;15526:42;15511:57;:12;;;:57;15475:178;;;15622:12;;;;;15637:1;15622:16;15607:31;;;;15475:178;15728:13;:11;:13::i;:::-;15721:20;13732:2026;-1:-1:-1;;;;;;;;13732:2026:135:o;32450:8733::-;32537:10;32599;32607:2;32599:10;;;;32638:11;;;:44;;;32664:1;32654:6;:11;;;;:27;;;;;32678:3;32669:6;:12;;;32654:27;32634:8490;;;32723:4;32716:11;;32847:6;32907:3;32902:25;;;;32982:3;32977:25;;;;33056:3;33051:25;;;;33131:3;33126:25;;;;33205:3;33200:25;;;;33278:3;33273:25;;;;33352:3;33347:25;;;;32840:532;;32902:25;32921:4;32913:12;;32902:25;;32977;32996:4;32988:12;;32977:25;;33051;33070:4;33062:12;;33051:25;;33126;33145:4;33137:12;;33126:25;;33200;33219:4;33211:12;;33200:25;;33273;33292:4;33284:12;;33273:25;;33347;33366:4;33358:12;;32840:532;;33435:4;:12;;33443:4;33435:12;33431:4023;;-1:-1:-1;;;33486:9:135;33478:26;;33499:4;33494:1;33486:9;;;33485:18;33478:26;33471:33;;33431:4023;33572:4;:12;;33580:4;33572:12;33568:3886;;-1:-1:-1;;;33623:9:135;33615:26;;33636:4;33631:1;33623:9;;;33622:18;33615:26;33608:33;;33568:3886;33709:4;:12;;33717:4;33709:12;33705:3749;;33774:4;33769:1;33761:9;;;33760:18;33807:27;33761:9;33810:11;;;;33823:2;:10;;;33807:2;:27::i;:::-;33800:34;;;;;;;33705:3749;33903:4;:12;;33911:4;33903:12;33899:3555;;-1:-1:-1;;;33946:17:135;;;33958:4;33953:9;;33946:17;33939:24;;33899:3555;34032:4;:11;;34040:3;34032:11;34028:3426;;-1:-1:-1;;;34074:17:135;;;34086:4;34081:9;;34074:17;34067:24;;34028:3426;34160:4;:12;;34168:4;34160:12;34156:3298;;34203:21;34212:2;34206:8;;:2;:8;;;;34221:2;34216;:7;34203:2;:21::i;:::-;34196:28;;;;;;34156:3298;34473:4;:12;;34481:4;34473:12;34469:2985;;34516:2;34509:9;;;;;;34469:2985;34587:4;:12;;34595:4;34587:12;34583:2871;;34630:2;34623:9;;;;;;34583:2871;34701:4;:12;;34709:4;34701:12;34697:2757;;34744:2;34737:9;;;;;;34697:2757;34815:4;:12;;34823:4;34815:12;34811:2643;;34858:2;34851:9;;;;;;34811:2643;34932:4;:12;;34940:4;34932:12;34928:2526;;34975:2;34968:9;;;;;;34928:2526;35092:4;:12;;35100:4;35092:12;35088:2366;;35135:2;35128:9;;;;;;35088:2366;35206:4;:12;;35214:4;35206:12;35202:2252;;35249:2;35242:9;;;;;;35202:2252;35320:4;:12;;35328:4;35320:12;35316:2138;;35363:2;35356:9;;;;;;35316:2138;35434:4;:12;;35442:4;35434:12;35430:2024;;35477:2;35470:9;;;;;;35430:2024;35548:4;:12;;35556:4;35548:12;35544:1910;;35591:2;35584:9;;;;;;35544:1910;35662:4;:12;;35670:4;35662:12;35658:1796;;35705:2;35698:9;;;;;;35658:1796;35777:4;:12;;35785:4;35777:12;35773:1681;;35820:2;35813:9;;;;;;35773:1681;35890:4;:12;;35898:4;35890:12;35886:1568;;35933:2;35926:9;;;;;;35886:1568;36004:4;:12;;36012:4;36004:12;36000:1454;;36047:2;36040:9;;;;;;36000:1454;36196:4;:12;;36204:4;36196:12;36192:1262;;-1:-1:-1;;;36240:7:135;;;36232:16;;36192:1262;36317:4;:12;;36325:4;36317:12;36313:1141;;-1:-1:-1;;;36361:7:135;;;36353:16;;36313:1141;36437:4;:12;;36445:4;36437:12;36433:1021;;-1:-1:-1;;;36481:7:135;;;36473:16;;36433:1021;36558:4;:12;;36566:4;36558:12;36554:900;;-1:-1:-1;;;36602:7:135;;;36594:16;;36554:900;36678:4;:12;;36686:4;36678:12;36674:780;;-1:-1:-1;;;36722:7:135;;;36714:16;;36674:780;36797:4;:12;;36805:4;36797:12;36793:661;;-1:-1:-1;;;36841:7:135;;;36833:16;;36793:661;36917:4;:12;;36925:4;36917:12;36913:541;;-1:-1:-1;;;36961:7:135;;;36953:16;;36913:541;37037:4;:12;;37045:4;37037:12;37033:421;;-1:-1:-1;;;37082:7:135;;;37080:10;37073:17;;37033:421;37159:4;:12;;37167:4;37159:12;37155:299;;37220:2;37202:21;;37208:2;37202:21;;;:29;;37230:1;37202:29;;;37226:1;37202:29;37195:36;;;;;;;;37155:299;37301:4;:12;;37309:4;37301:12;37297:157;;37349:2;37344:7;;:2;:7;;;:15;;37358:1;37344:15;;37297:157;37406:29;;;;;2878:2:299;37406:29:135;;;2860:21:299;2917:2;2897:18;;;2890:30;2956:21;2936:18;;;2929:49;2995:18;;37406:29:135;2676:343:299;37297:157:135;32684:4784;32634:8490;;;37524:6;:14;;37534:4;37524:14;37520:3590;;37583:4;37576:11;;37658:3;37650:11;;;37646:549;;-1:-1:-1;;;37703:21:135;;;37689:36;;37646:549;37810:4;:12;;37818:4;37810:12;:28;;;;37826:4;:12;;37834:4;37826:12;37810:28;37806:389;;;37870:4;:12;;37878:4;37870:12;37866:83;;37919:3;;;37866:83;37974:8;38012:127;38024:10;38019:15;;:20;38012:127;;38104:8;38071:3;38104:8;;;;;38071:3;38012:127;;;38171:1;-1:-1:-1;38164:8:135;;-1:-1:-1;;38164:8:135;37520:3590;38262:6;:14;;38272:4;38262:14;38258:2852;;-1:-1:-1;;38307:8:135;38313:2;38307:8;;;;38300:15;;38258:2852;38382:6;:14;;38392:4;38382:14;38378:2732;;38427:42;38445:2;38450:1;38445:6;38455:1;38444:12;38439:2;:17;38431:26;;:3;:26;;;;38461:4;38430:35;38467:1;38427:2;:42::i;:::-;38420:49;;;;;38378:2732;38536:6;:14;;38546:4;38536:14;38532:2578;;38581:45;38599:2;38604:1;38599:6;38609:1;38598:12;38593:2;:17;38585:26;;:3;:26;;;;38615:6;38584:37;38623:2;38581;:45::i;38532:2578::-;38694:6;:14;;38704:4;38694:14;38690:2420;;-1:-1:-1;;38745:21:135;38764:1;38759;38754:6;;38753:12;38745:21;;38802:36;;;38873:5;38868:10;;38745:21;;;;;38867:18;38860:25;;38690:2420;38952:6;:14;;38962:4;38952:14;38948:2162;;38997:3;38990:10;;;;;38948:2162;39068:6;:14;;39078:4;39068:14;39064:2046;;39128:2;39133:1;39128:6;39138:1;39127:12;39122:2;:17;39114:26;;:3;:26;;;;39144:4;39113:35;39106:42;;;;;39064:2046;39217:6;:14;;39227:4;39217:14;39213:1897;;39277:2;39282:1;39277:6;39287:1;39276:12;39271:2;:17;39263:26;;:3;:26;;;;39293:6;39262:37;39255:44;;;;;39213:1897;39368:6;:14;;39378:4;39368:14;39364:1746;;-1:-1:-1;;39419:26:135;39443:1;39438;39433:6;;39432:12;39427:2;:17;39419:26;;39481:41;;;39557:5;39552:10;;39419:26;;;;;39551:18;39544:25;;39364:1746;39637:6;:14;;39647:4;39637:14;39633:1477;;-1:-1:-1;;39694:4:135;39688:34;39720:1;39715;39710:6;;39709:12;39704:2;:17;39688:34;;39778:27;;;39758:48;;;39836:10;;39689:9;;;39688:34;;39835:18;39828:25;;39633:1477;39921:6;:14;;39931:4;39921:14;39917:1193;;-1:-1:-1;;39978:6:135;39972:36;40006:1;40001;39996:6;;39995:12;39990:2;:17;39972:36;;40064:29;;;40044:50;;;40124:10;;39973:11;;;39972:36;;40123:18;40116:25;;39917:1193;40210:6;:14;;40220:4;40210:14;40206:904;;-1:-1:-1;;40261:20:135;40279:1;40274;40269:6;;40268:12;40261:20;;40317:36;;;40389:5;40383:11;;40261:20;;;;;40382:19;40375:26;;40206:904;40469:6;:14;;40479:4;40469:14;40465:645;;40514:2;40507:9;;;;;40465:645;40585:6;:14;;40595:4;40585:14;40581:529;;-1:-1:-1;;40636:25:135;40659:1;40654;40649:6;;40648:12;40643:2;:17;40636:25;;40697:41;;;40774:5;40768:11;;40636:25;;;;;40767:19;40760:26;;40581:529;40853:6;:14;;40863:4;40853:14;40849:261;;40898:3;40891:10;;;;;40849:261;40968:6;:14;;40978:4;40968:14;40964:146;;41013:2;41006:9;;;32450:8733;;;;;;;:::o;19960:782::-;20046:12;20133:18;;:::i;:::-;-1:-1:-1;20201:4:135;20308:2;20296:14;;;;20288:41;;;;;;;3226:2:299;20288:41:135;;;3208:21:299;3265:2;3245:18;;;3238:30;3304:16;3284:18;;;3277:44;3338:18;;20288:41:135;3024:338:299;20288:41:135;20425:14;;;;;;;:30;;;20443:12;20425:30;20421:102;;;20504:4;20475:5;:15;;;20491:9;20475:26;;;;;;;;;:::i;:::-;:33;;;;:26;;;;;;:33;20421:102;20578:12;;;;;20567:23;;;;:8;;;:23;20634:1;20619:16;;;20604:31;;;20712:13;:11;:13::i;5582:7764::-;5646:12;5732:18;;:::i;:::-;-1:-1:-1;5910:15:135;;:18;;;;5800:4;6070:18;;;;6114;;;;6158;;;;;5800:4;;5890:17;;;;6070:18;6114;6248;;;6262:4;6248:18;6244:6792;;6298:2;6327:4;6322:9;;:14;6318:144;;6438:4;6433:9;;6425:4;:18;6419:24;6318:144;6483:2;:7;;6489:1;6483:7;6479:161;;6519:10;;;;;6551:16;;;;;;;;6519:10;-1:-1:-1;6479:161:135;;;6619:2;6614:7;;6479:161;6268:386;6244:6792;;;6756:10;:18;;6770:4;6756:18;6752:6284;;1745:10;6794:14;;6752:6284;;;6892:10;:18;;6906:4;6892:18;6888:6148;;6935:1;6930:6;;6888:6148;;;7060:10;:18;;7074:4;7060:18;7056:5980;;7113:4;7098:12;;;:19;7135:26;;;:14;;;:26;7186:13;:11;:13::i;:::-;7179:20;5582:7764;-1:-1:-1;;;;;;;;;5582:7764:135:o;7056:5980::-;7325:10;:18;;7339:4;7325:18;7321:5715;;7476:14;;;7472:2723;7321:5715;7472:2723;7646:22;;;;;7642:2553;;7771:10;7784:27;7792:2;7797:10;7792:15;7809:1;7784:7;:27::i;:::-;7895:17;;;;7771:40;;-1:-1:-1;7895:17:135;7873:19;8045:14;8064:1;8039:26;8035:146;;1676:4:136;1670:11;;1533:21;1787:15;;;1828:8;1822:4;1815:22;1850:27;;;1996:4;1983:18;;2098:17;;2003:19;1979:44;2025:11;1976:61;8093:65:135;;8035:146;8267:20;;;;;8234:54;;;;;;;;3540:25:299;;;8234:54:135;3601:23:299;;;3581:18;;;3574:51;8203:11:135;;;;8234:19;:6;:19;;;;3513:18:299;;8234:54:135;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;:::i;:::-;8202:86;;;;8515:1;8511:2;8507:10;8612:9;8609:1;8605:17;8694:6;8687:5;8684:17;8681:40;;;8714:5;8704:15;;8681:40;;8797:6;8793:2;8790:14;8787:34;;;8817:2;8807:12;;8787:34;8923:3;8918:1;8910:6;8906:14;8901:3;8897:24;8893:34;8886:41;;9023:3;9019:1;9007:9;8998:6;8995:1;8991:14;8987:30;8983:38;8979:48;8972:55;;9178:1;9174;9170;9158:9;9155:1;9151:17;9147:25;9143:33;9139:41;9305:1;9301;9297;9288:6;9276:9;9273:1;9269:17;9265:30;9261:38;9257:46;9253:54;9235:72;;9436:10;9432:15;9426:4;9422:26;9414:34;;9552:3;9544:4;9540:9;9535:3;9531:19;9528:28;9521:35;;;;9698:33;9707:2;9712:10;9707:15;9724:1;9727:3;9698:8;:33::i;:::-;9753:20;;;:38;;;;;;;;;-1:-1:-1;7642:2553:135;;-1:-1:-1;;;7642:2553:135;;9910:18;;;;;9906:289;;10080:2;10075:7;;7321:5715;;9906:289;10134:10;10129:15;;2053:3;10166:10;;9906:289;7321:5715;;;10324:10;:18;;10338:4;10324:18;10320:2716;;10478:15;;;1824:1;10478:15;;:34;;-1:-1:-1;10497:15:135;;;1859:1;10497:15;10478:34;:57;;;-1:-1:-1;10516:19:135;;;1936:1;10516:19;10478:57;10474:1593;;;10564:2;10559:7;;10320:2716;;10474:1593;10690:23;;;;;10686:1381;;10737:10;10750:27;10758:2;10763:10;10758:15;10775:1;10750:7;:27::i;:::-;10853:17;;;;10737:40;;-1:-1:-1;11096:1:135;11088:10;;11190:1;11186:17;11265:13;;;11262:32;;;11287:5;11281:11;;11262:32;11573:14;;;11379:1;11569:22;;;11565:32;;;;11462:26;11486:1;11371:10;;;11466:18;;;11462:26;11561:43;11367:20;;11669:12;11797:17;;;:23;11865:1;11842:20;;;:24;11375:2;-1:-1:-1;11375:2:135;7321:5715;;10320:2716;12269:10;:18;;12283:4;12269:18;12265:771;;12379:2;:7;;12385:1;12379:7;12375:647;;12472:14;;;;;:40;;-1:-1:-1;12490:22:135;;;1978:1;12490:22;12472:40;:62;;;-1:-1:-1;12516:18:135;;;1897:1;12516:18;12472:62;12468:404;;;12567:1;12562:6;;12375:647;;12468:404;12613:15;;;1824:1;12613:15;;:34;;-1:-1:-1;12632:15:135;;;1859:1;12632:15;12613:34;:61;;;-1:-1:-1;12651:23:135;;;2021:1;12651:23;12613:61;:84;;;-1:-1:-1;12678:19:135;;;1936:1;12678:19;12613:84;12609:263;;;12730:1;12725:6;;7321:5715;;12375:647;12923:10;12918:15;;2087:4;12955:11;;12375:647;13111:15;;;;;:23;;;;:18;;;;:23;;;;13148:15;;:23;;;:18;;;;:23;-1:-1:-1;13237:12:135;;;;13226:23;;;:8;;;:23;13293:1;13278:16;13263:31;;;;;13316:13;:11;:13::i;16084:2480::-;16178:12;16264:18;;:::i;:::-;-1:-1:-1;16332:4:135;16364:10;16472:13;;;16481:4;16472:13;16468:1705;;-1:-1:-1;16511:8:135;;;;16468:1705;;;16630:5;:13;;16639:4;16630:13;16626:1547;;16663:14;;;:8;;;:14;16626:1547;;;16793:5;:13;;16802:4;16793:13;16789:1384;;-1:-1:-1;16832:8:135;;;;16789:1384;;;16951:5;:13;;16960:4;16951:13;16947:1226;;16984:14;;;:8;;;:14;16947:1226;;;17125:5;:13;;17134:4;17125:13;17121:1052;;17252:9;17198:17;17178;;;17198;;;;17178:37;17259:2;17252:9;;;;;17234:8;;;:28;17280:22;:8;;;:22;17121:1052;;;17439:5;:13;;17448:4;17439:13;17435:738;;17506:11;17492;;;17506;;;17492:25;17561:2;17554:9;;;;;17536:8;;;:28;17582:22;:8;;;:22;17435:738;;;17763:5;:13;;17772:4;17763:13;17759:414;;17833:3;17814:23;;17820:3;17814:23;;;;;;;:::i;:::-;;17796:42;;:8;;;:42;17874:23;;;;;;;;;;;;;:::i;:::-;;17856:42;;:8;;;:42;17759:414;;;18067:5;:13;;18076:4;18067:13;18063:110;;18117:3;18111:9;;:3;:9;;;;;;;:::i;:::-;;18100:20;;;;:8;;;:20;18149:9;;;;;;;;;;;:::i;:::-;;18138:20;;:8;;;:20;18063:110;18266:14;;;;18262:85;;18329:3;18300:5;:15;;;18316:9;18300:26;;;;;;;;;:::i;:::-;:32;;;;:26;;;;;;:32;18262:85;18401:12;;;;;18390:23;;;;:8;;;:23;18457:1;18442:16;;;18427:31;;;18534:13;:11;:13::i;:::-;18527:20;16084:2480;-1:-1:-1;;;;;;;16084:2480:135:o;23913:1654::-;24089:14;24106:24;24118:11;24106;:24::i;:::-;24089:41;;24238:1;24231:5;24227:13;24224:33;;;24253:1;24250;24243:12;24224:33;24392:2;24586:15;;;24411:2;24400:14;;24388:10;24384:31;24381:1;24377:39;24542:16;;;24327:20;;24527:10;24516:22;;;24512:27;24502:38;24499:60;25028:5;25025:1;25021:13;25099:1;25084:343;25109:2;25106:1;25103:9;25084:343;;;25232:2;25220:15;;;25169:20;25267:12;;;25281:1;25263:20;25304:42;;;;25372:1;25367:42;;;;25256:153;;25304:42;22391:1;22384:12;;;22424:2;22417:13;;;22469:2;22456:16;;25313:31;;25304:42;;25367;22391:1;22384:12;;;22424:2;22417:13;;;22469:2;22456:16;;25376:31;;25256:153;-1:-1:-1;;25127:1:135;25120:9;25084:343;;;-1:-1:-1;;25526:4:135;25519:18;-1:-1:-1;;;;23913:1654:135:o;20946:586::-;21268:20;;;21292:7;21268:32;21261:3;:40;;;21374:14;;21429:17;;21423:24;;;21415:72;;;;;;;4277:2:299;21415:72:135;;;4259:21:299;4316:2;4296:18;;;4289:30;4355:34;4335:18;;;4328:62;4426:5;4406:18;;;4399:33;4449:19;;21415:72:135;4075:399:299;21415:72:135;21501:14;20946:586;;;:::o;-1:-1:-1:-;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;:::i;:::-;;;;:::o;:::-;;;;;;;;;;;;;;;;;;;;;;;;:::o;467:347:299:-;518:8;528:6;582:3;575:4;567:6;563:17;559:27;549:55;;600:1;597;590:12;549:55;-1:-1:-1;623:20:299;;666:18;655:30;;652:50;;;698:1;695;688:12;652:50;735:4;727:6;723:17;711:29;;787:3;780:4;771:6;763;759:19;755:30;752:39;749:59;;;804:1;801;794:12;749:59;467:347;;;;;:::o;819:785::-;918:6;926;934;942;950;1003:2;991:9;982:7;978:23;974:32;971:52;;;1019:1;1016;1009:12;971:52;1059:9;1046:23;1088:18;1129:2;1121:6;1118:14;1115:34;;;1145:1;1142;1135:12;1115:34;1184:58;1234:7;1225:6;1214:9;1210:22;1184:58;:::i;:::-;1261:8;;-1:-1:-1;1158:84:299;-1:-1:-1;1349:2:299;1334:18;;1321:32;;-1:-1:-1;1365:16:299;;;1362:36;;;1394:1;1391;1384:12;1362:36;;1433:60;1485:7;1474:8;1463:9;1459:24;1433:60;:::i;:::-;819:785;;;;-1:-1:-1;1512:8:299;1594:2;1579:18;1566:32;;819:785;-1:-1:-1;;;;819:785:299:o;1791:184::-;1843:77;1840:1;1833:88;1940:4;1937:1;1930:15;1964:4;1961:1;1954:15;3636:245;3715:6;3723;3776:2;3764:9;3755:7;3751:23;3747:32;3744:52;;;3792:1;3789;3782:12;3744:52;-1:-1:-1;;3815:16:299;;3871:2;3856:18;;;3850:25;3815:16;;3850:25;;-1:-1:-1;3636:245:299:o;3886:184::-;3938:77;3935:1;3928:88;4035:4;4032:1;4025:15;4059:4;4056:1;4049:15" func init() { if err := json.Unmarshal([]byte(MIPSStorageLayoutJSON), MIPSStorageLayout); err != nil { diff --git a/op-bindings/bindings/preimageoracle_more.go b/op-bindings/bindings/preimageoracle_more.go index 87c5194ee3bc..82e64da4c864 100644 --- a/op-bindings/bindings/preimageoracle_more.go +++ b/op-bindings/bindings/preimageoracle_more.go @@ -15,7 +15,7 @@ var PreimageOracleStorageLayout = new(solc.StorageLayout) var PreimageOracleDeployedBin = "0x608060405234801561001057600080fd5b50600436106100725760003560e01c8063e03110e111610050578063e03110e114610106578063e15926111461012e578063fef2b4ed1461014357600080fd5b806361238bde146100775780638542cf50146100b5578063c0c220c9146100f3575b600080fd5b6100a26100853660046104df565b600160209081526000928352604080842090915290825290205481565b6040519081526020015b60405180910390f35b6100e36100c33660046104df565b600260209081526000928352604080842090915290825290205460ff1681565b60405190151581526020016100ac565b6100a2610101366004610501565b610163565b6101196101143660046104df565b610238565b604080519283526020830191909152016100ac565b61014161013c36600461053c565b610329565b005b6100a26101513660046105b8565b60006020819052908152604090205481565b600061016f8686610432565b905061017c836008610600565b8211806101895750602083115b156101c0576040517ffe25498700000000000000000000000000000000000000000000000000000000815260040160405180910390fd5b6000602081815260c085901b82526008959095528251828252600286526040808320858452875280832080547fffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff001660019081179091558484528752808320948352938652838220558181529384905292205592915050565b6000828152600260209081526040808320848452909152812054819060ff166102c1576040517f08c379a000000000000000000000000000000000000000000000000000000000815260206004820152601460248201527f7072652d696d616765206d757374206578697374000000000000000000000000604482015260640160405180910390fd5b50600083815260208181526040909120546102dd816008610600565b6102e8856020610600565b1061030657836102f9826008610600565b6103039190610618565b91505b506000938452600160209081526040808620948652939052919092205492909150565b604435600080600883018611156103485763fe2549876000526004601cfd5b60c083901b6080526088838682378087017ffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff80151908490207effffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff167f02000000000000000000000000000000000000000000000000000000000000001760008181526002602090815260408083208b8452825280832080547fffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff0016600190811790915584845282528083209a83529981528982209390935590815290819052959095209190915550505050565b7f01000000000000000000000000000000000000000000000000000000000000007effffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff8316176104d8818360408051600093845233602052918152606090922091527effffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff167f01000000000000000000000000000000000000000000000000000000000000001790565b9392505050565b600080604083850312156104f257600080fd5b50508035926020909101359150565b600080600080600060a0868803121561051957600080fd5b505083359560208501359550604085013594606081013594506080013592509050565b60008060006040848603121561055157600080fd5b83359250602084013567ffffffffffffffff8082111561057057600080fd5b818601915086601f83011261058457600080fd5b81358181111561059357600080fd5b8760208285010111156105a557600080fd5b6020830194508093505050509250925092565b6000602082840312156105ca57600080fd5b5035919050565b7f4e487b7100000000000000000000000000000000000000000000000000000000600052601160045260246000fd5b60008219821115610613576106136105d1565b500190565b60008282101561062a5761062a6105d1565b50039056fea164736f6c634300080f000a" -var PreimageOracleDeployedSourceMap = "306:3911:128:-:0;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;537:68;;;;;;:::i;:::-;;;;;;;;;;;;;;;;;;;;;;;;;;;;413:25:285;;;401:2;386:18;537:68:128;;;;;;;;680:66;;;;;;:::i;:::-;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;614:14:285;;607:22;589:41;;577:2;562:18;680:66:128;449:187:285;1367:1211:128;;;;;;:::i;:::-;;:::i;789:536::-;;;;;;:::i;:::-;;:::i;:::-;;;;1274:25:285;;;1330:2;1315:18;;1308:34;;;;1247:18;789:536:128;1100:248:285;2620:1595:128;;;;;;:::i;:::-;;:::i;:::-;;419:50;;;;;;:::i;:::-;;;;;;;;;;;;;;;1367:1211;1560:12;1665:51;1694:6;1702:13;1665:28;:51::i;:::-;1658:58;-1:-1:-1;1810:9:128;:5;1818:1;1810:9;:::i;:::-;1796:11;:23;:37;;;;1831:2;1823:5;:10;1796:37;1792:90;;;1856:15;;;;;;;;;;;;;;1792:90;1951:12;2051:4;2044:18;;;2152:3;2148:15;;;2135:29;;2184:4;2177:19;;;;2286:18;;2376:20;;;:14;:20;;;;;;:33;;;;;;;;:40;;;;2412:4;2376:40;;;;;;2426:19;;;;;;;;:32;;;;;;;;;:39;2542:21;;;;;;;;;:29;2391:4;1367:1211;-1:-1:-1;;1367:1211:128:o;789:536::-;865:12;914:20;;;:14;:20;;;;;;;;:29;;;;;;;;;865:12;;914:29;;906:62;;;;;;;2908:2:285;906:62:128;;;2890:21:285;2947:2;2927:18;;;2920:30;2986:22;2966:18;;;2959:50;3026:18;;906:62:128;;;;;;;;-1:-1:-1;1099:14:128;1116:21;;;1087:2;1116:21;;;;;;;;1167:10;1116:21;1176:1;1167:10;:::i;:::-;1151:12;:7;1161:2;1151:12;:::i;:::-;:26;1147:87;;1216:7;1203:10;:6;1212:1;1203:10;:::i;:::-;:20;;;;:::i;:::-;1193:30;;1147:87;-1:-1:-1;1290:19:128;;;;:13;:19;;;;;;;;:28;;;;;;;;;;;;789:536;;-1:-1:-1;789:536:128:o;2620:1595::-;2916:4;2903:18;2721:12;;3045:1;3035:12;;3019:29;;3016:210;;;3120:10;3117:1;3110:21;3210:1;3204:4;3197:15;3016:210;3469:3;3465:14;;;3369:4;3453:27;3500:11;3474:4;3619:16;3500:11;3601:41;3832:29;;;3836:11;3832:29;3826:36;3884:20;;;;4031:19;4024:27;4053:11;4021:44;4084:19;;;;4062:1;4084:19;;;;;;;;:32;;;;;;;;:39;;;;4119:4;4084:39;;;;;;4133:18;;;;;;;;:31;;;;;;;;;:38;;;;4181:20;;;;;;;;;;;:27;;;;-1:-1:-1;;;;2620:1595:128:o;552:449:127:-;835:11;860:19;848:32;;832:49;965:29;832:49;980:13;1676:4;1670:11;;1533:21;1787:15;;;1828:8;1822:4;1815:22;1850:27;;;1996:4;1983:18;;;2098:17;;2003:19;1979:44;2025:11;1976:61;;1455:676;965:29;958:36;552:449;-1:-1:-1;;;552:449:127:o;14:248:285:-;82:6;90;143:2;131:9;122:7;118:23;114:32;111:52;;;159:1;156;149:12;111:52;-1:-1:-1;;182:23:285;;;252:2;237:18;;;224:32;;-1:-1:-1;14:248:285:o;641:454::-;736:6;744;752;760;768;821:3;809:9;800:7;796:23;792:33;789:53;;;838:1;835;828:12;789:53;-1:-1:-1;;861:23:285;;;931:2;916:18;;903:32;;-1:-1:-1;982:2:285;967:18;;954:32;;1033:2;1018:18;;1005:32;;-1:-1:-1;1084:3:285;1069:19;1056:33;;-1:-1:-1;641:454:285;-1:-1:-1;641:454:285:o;1353:659::-;1432:6;1440;1448;1501:2;1489:9;1480:7;1476:23;1472:32;1469:52;;;1517:1;1514;1507:12;1469:52;1553:9;1540:23;1530:33;;1614:2;1603:9;1599:18;1586:32;1637:18;1678:2;1670:6;1667:14;1664:34;;;1694:1;1691;1684:12;1664:34;1732:6;1721:9;1717:22;1707:32;;1777:7;1770:4;1766:2;1762:13;1758:27;1748:55;;1799:1;1796;1789:12;1748:55;1839:2;1826:16;1865:2;1857:6;1854:14;1851:34;;;1881:1;1878;1871:12;1851:34;1926:7;1921:2;1912:6;1908:2;1904:15;1900:24;1897:37;1894:57;;;1947:1;1944;1937:12;1894:57;1978:2;1974;1970:11;1960:21;;2000:6;1990:16;;;;;1353:659;;;;;:::o;2017:180::-;2076:6;2129:2;2117:9;2108:7;2104:23;2100:32;2097:52;;;2145:1;2142;2135:12;2097:52;-1:-1:-1;2168:23:285;;2017:180;-1:-1:-1;2017:180:285:o;2384:184::-;2436:77;2433:1;2426:88;2533:4;2530:1;2523:15;2557:4;2554:1;2547:15;2573:128;2613:3;2644:1;2640:6;2637:1;2634:13;2631:39;;;2650:18;;:::i;:::-;-1:-1:-1;2686:9:285;;2573:128::o;3055:125::-;3095:4;3123:1;3120;3117:8;3114:34;;;3128:18;;:::i;:::-;-1:-1:-1;3165:9:285;;3055:125::o" +var PreimageOracleDeployedSourceMap = "306:3911:137:-:0;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;537:68;;;;;;:::i;:::-;;;;;;;;;;;;;;;;;;;;;;;;;;;;413:25:299;;;401:2;386:18;537:68:137;;;;;;;;680:66;;;;;;:::i;:::-;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;614:14:299;;607:22;589:41;;577:2;562:18;680:66:137;449:187:299;1367:1211:137;;;;;;:::i;:::-;;:::i;789:536::-;;;;;;:::i;:::-;;:::i;:::-;;;;1274:25:299;;;1330:2;1315:18;;1308:34;;;;1247:18;789:536:137;1100:248:299;2620:1595:137;;;;;;:::i;:::-;;:::i;:::-;;419:50;;;;;;:::i;:::-;;;;;;;;;;;;;;;1367:1211;1560:12;1665:51;1694:6;1702:13;1665:28;:51::i;:::-;1658:58;-1:-1:-1;1810:9:137;:5;1818:1;1810:9;:::i;:::-;1796:11;:23;:37;;;;1831:2;1823:5;:10;1796:37;1792:90;;;1856:15;;;;;;;;;;;;;;1792:90;1951:12;2051:4;2044:18;;;2152:3;2148:15;;;2135:29;;2184:4;2177:19;;;;2286:18;;2376:20;;;:14;:20;;;;;;:33;;;;;;;;:40;;;;2412:4;2376:40;;;;;;2426:19;;;;;;;;:32;;;;;;;;;:39;2542:21;;;;;;;;;:29;2391:4;1367:1211;-1:-1:-1;;1367:1211:137:o;789:536::-;865:12;914:20;;;:14;:20;;;;;;;;:29;;;;;;;;;865:12;;914:29;;906:62;;;;;;;2908:2:299;906:62:137;;;2890:21:299;2947:2;2927:18;;;2920:30;2986:22;2966:18;;;2959:50;3026:18;;906:62:137;;;;;;;;-1:-1:-1;1099:14:137;1116:21;;;1087:2;1116:21;;;;;;;;1167:10;1116:21;1176:1;1167:10;:::i;:::-;1151:12;:7;1161:2;1151:12;:::i;:::-;:26;1147:87;;1216:7;1203:10;:6;1212:1;1203:10;:::i;:::-;:20;;;;:::i;:::-;1193:30;;1147:87;-1:-1:-1;1290:19:137;;;;:13;:19;;;;;;;;:28;;;;;;;;;;;;789:536;;-1:-1:-1;789:536:137:o;2620:1595::-;2916:4;2903:18;2721:12;;3045:1;3035:12;;3019:29;;3016:210;;;3120:10;3117:1;3110:21;3210:1;3204:4;3197:15;3016:210;3469:3;3465:14;;;3369:4;3453:27;3500:11;3474:4;3619:16;3500:11;3601:41;3832:29;;;3836:11;3832:29;3826:36;3884:20;;;;4031:19;4024:27;4053:11;4021:44;4084:19;;;;4062:1;4084:19;;;;;;;;:32;;;;;;;;:39;;;;4119:4;4084:39;;;;;;4133:18;;;;;;;;:31;;;;;;;;;:38;;;;4181:20;;;;;;;;;;;:27;;;;-1:-1:-1;;;;2620:1595:137:o;552:449:136:-;835:11;860:19;848:32;;832:49;965:29;832:49;980:13;1676:4;1670:11;;1533:21;1787:15;;;1828:8;1822:4;1815:22;1850:27;;;1996:4;1983:18;;;2098:17;;2003:19;1979:44;2025:11;1976:61;;1455:676;965:29;958:36;552:449;-1:-1:-1;;;552:449:136:o;14:248:299:-;82:6;90;143:2;131:9;122:7;118:23;114:32;111:52;;;159:1;156;149:12;111:52;-1:-1:-1;;182:23:299;;;252:2;237:18;;;224:32;;-1:-1:-1;14:248:299:o;641:454::-;736:6;744;752;760;768;821:3;809:9;800:7;796:23;792:33;789:53;;;838:1;835;828:12;789:53;-1:-1:-1;;861:23:299;;;931:2;916:18;;903:32;;-1:-1:-1;982:2:299;967:18;;954:32;;1033:2;1018:18;;1005:32;;-1:-1:-1;1084:3:299;1069:19;1056:33;;-1:-1:-1;641:454:299;-1:-1:-1;641:454:299:o;1353:659::-;1432:6;1440;1448;1501:2;1489:9;1480:7;1476:23;1472:32;1469:52;;;1517:1;1514;1507:12;1469:52;1553:9;1540:23;1530:33;;1614:2;1603:9;1599:18;1586:32;1637:18;1678:2;1670:6;1667:14;1664:34;;;1694:1;1691;1684:12;1664:34;1732:6;1721:9;1717:22;1707:32;;1777:7;1770:4;1766:2;1762:13;1758:27;1748:55;;1799:1;1796;1789:12;1748:55;1839:2;1826:16;1865:2;1857:6;1854:14;1851:34;;;1881:1;1878;1871:12;1851:34;1926:7;1921:2;1912:6;1908:2;1904:15;1900:24;1897:37;1894:57;;;1947:1;1944;1937:12;1894:57;1978:2;1974;1970:11;1960:21;;2000:6;1990:16;;;;;1353:659;;;;;:::o;2017:180::-;2076:6;2129:2;2117:9;2108:7;2104:23;2100:32;2097:52;;;2145:1;2142;2135:12;2097:52;-1:-1:-1;2168:23:299;;2017:180;-1:-1:-1;2017:180:299:o;2384:184::-;2436:77;2433:1;2426:88;2533:4;2530:1;2523:15;2557:4;2554:1;2547:15;2573:128;2613:3;2644:1;2640:6;2637:1;2634:13;2631:39;;;2650:18;;:::i;:::-;-1:-1:-1;2686:9:299;;2573:128::o;3055:125::-;3095:4;3123:1;3120;3117:8;3114:34;;;3128:18;;:::i;:::-;-1:-1:-1;3165:9:299;;3055:125::o" func init() { if err := json.Unmarshal([]byte(PreimageOracleStorageLayoutJSON), PreimageOracleStorageLayout); err != nil { diff --git a/packages/contracts-bedrock/.gas-snapshot b/packages/contracts-bedrock/.gas-snapshot index c5315d9f1633..06d5f60a3ad8 100644 --- a/packages/contracts-bedrock/.gas-snapshot +++ b/packages/contracts-bedrock/.gas-snapshot @@ -305,31 +305,32 @@ LegacyERC20ETH_Test:test_transferFrom_doesNotExist_reverts() (gas: 12957) LegacyERC20ETH_Test:test_transfer_doesNotExist_reverts() (gas: 10755) LegacyMessagePasser_Test:test_passMessageToL1_succeeds() (gas: 34524) LibPosition_Test:test_pos_correctness_succeeds() (gas: 38689) -LivenessGuard_CheckAfterExecution_TestFails:test_checkAfterExecution_callerIsNotSafe_revert() (gas: 8553) -LivenessGuard_CheckTx_Test:test_checkTransaction_succeeds() (gas: 228306) -LivenessGuard_CheckTx_TestFails:test_checkTransaction_callerIsNotSafe_revert() (gas: 10380) -LivenessGuard_Constructor_Test:test_constructor_works() (gas: 1174506) +LivenessGuard_CheckAfterExecution_TestFails:test_checkAfterExecution_callerIsNotSafe_revert() (gas: 8531) +LivenessGuard_CheckTx_Test:test_checkTransaction_succeeds() (gas: 233535) +LivenessGuard_CheckTx_TestFails:test_checkTransaction_callerIsNotSafe_revert() (gas: 10358) +LivenessGuard_Constructor_Test:test_constructor_works() (gas: 1198965) LivenessGuard_Getters_Test:test_getters_works() (gas: 10662) -LivenessGuard_OwnerManagement_Test:test_addOwner_succeeds() (gas: 272723) -LivenessGuard_OwnerManagement_Test:test_removeOwner_succeeds() (gas: 241108) -LivenessGuard_OwnerManagement_Test:test_swapOwner_succeeds() (gas: 279720) +LivenessGuard_OwnerManagement_Test:test_addOwner_succeeds() (gas: 274366) +LivenessGuard_OwnerManagement_Test:test_removeOwner_succeeds() (gas: 246263) +LivenessGuard_OwnerManagement_Test:test_swapOwner_succeeds() (gas: 284880) LivenessGuard_ShowLiveness_Test:test_showLiveness_succeeds() (gas: 28831) LivenessGuard_ShowLiveness_TestFail:test_showLiveness_callIsNotSafeOwner_reverts() (gas: 18770) LivenessModule_CanRemove_Test:test_canRemove_works() (gas: 33026) LivenessModule_CanRemove_TestFail:test_canRemove_notSafeOwner_reverts() (gas: 20489) -LivenessModule_Constructor_Test:test_constructor_minOwnersGreaterThanOwners_reverts() (gas: 83623) -LivenessModule_Constructor_Test:test_constructor_wrongThreshold_reverts() (gas: 92901) +LivenessModule_Constructor_TestFail:test_constructor_minOwnersGreaterThanOwners_reverts() (gas: 83623) +LivenessModule_Constructor_TestFail:test_constructor_wrongThreshold_reverts() (gas: 92925) LivenessModule_Get75PercentThreshold_Test:test_get75PercentThreshold_Works() (gas: 26339) LivenessModule_Getters_Test:test_getters_works() (gas: 14853) -LivenessModule_RemoveOwners_Test:test_removeOwners_allOwners_succeeds() (gas: 1311942) -LivenessModule_RemoveOwners_Test:test_removeOwners_oneOwner_succeeds() (gas: 130731) -LivenessModule_RemoveOwners_TestFail:test_removeOwners_belowMinButNotEmptied_reverts() (gas: 1265044) -LivenessModule_RemoveOwners_TestFail:test_removeOwners_differentArrayLengths_reverts() (gas: 10547) -LivenessModule_RemoveOwners_TestFail:test_removeOwners_guardChanged_reverts() (gas: 2820629) +LivenessModule_RemoveOwners_Test:test_removeOwners_allOwners_succeeds() (gas: 1326177) +LivenessModule_RemoveOwners_Test:test_removeOwners_oneOwner_succeeds() (gas: 133975) +LivenessModule_RemoveOwners_TestFail:test_removeOwners_belowEmptiedButNotShutDown_reverts() (gas: 1278643) +LivenessModule_RemoveOwners_TestFail:test_removeOwners_belowMinButNotEmptied_reverts() (gas: 1281685) +LivenessModule_RemoveOwners_TestFail:test_removeOwners_differentArrayLengths_reverts() (gas: 10502) +LivenessModule_RemoveOwners_TestFail:test_removeOwners_guardChanged_reverts() (gas: 2839358) LivenessModule_RemoveOwners_TestFail:test_removeOwners_invalidThreshold_reverts() (gas: 69358) -LivenessModule_RemoveOwners_TestFail:test_removeOwners_ownerHasShownLivenessRecently_reverts() (gas: 77817) -LivenessModule_RemoveOwners_TestFail:test_removeOwners_ownerHasSignedRecently_reverts() (gas: 614867) -LivenessModule_RemoveOwners_TestFail:test_removeOwners_swapToFallBackOwner_reverts() (gas: 1273705) +LivenessModule_RemoveOwners_TestFail:test_removeOwners_ownerHasShownLivenessRecently_reverts() (gas: 80971) +LivenessModule_RemoveOwners_TestFail:test_removeOwners_ownerHasSignedRecently_reverts() (gas: 617629) +LivenessModule_RemoveOwners_TestFail:test_removeOwners_swapToFallbackOwner_reverts() (gas: 1288036) LivenessModule_RemoveOwners_TestFail:test_removeOwners_wrongPreviousOwner_reverts() (gas: 73954) MIPS_Test:test_add_succeeds() (gas: 122932) MIPS_Test:test_addiSign_succeeds() (gas: 122923) diff --git a/packages/contracts-bedrock/semver-lock.json b/packages/contracts-bedrock/semver-lock.json index 07382eba2e82..baf78554fc45 100644 --- a/packages/contracts-bedrock/semver-lock.json +++ b/packages/contracts-bedrock/semver-lock.json @@ -18,8 +18,8 @@ "src/L2/L2StandardBridge.sol": "0x284ebf5569c75d98f2d1920a276d1116524399355708c4a60ea5892283c56719", "src/L2/L2ToL1MessagePasser.sol": "0xafc710b4d320ef450586d96a61cbd58cac814cb3b0c4fdc280eace3efdcdf321", "src/L2/SequencerFeeVault.sol": "0x883e434a69b4789997a4a9a32060dbbd2e12db6f1970927f1310820336119575", - "src/Safe/LivenessGuard.sol": "0x31b4ecc88c982490243ab42914c3de75e5acfa421ffc0ea0d0f0997dcc0341b5", - "src/Safe/LivenessModule.sol": "0xb8c8178c1f4f78eed4777846a40eda6a3a0c1710085822d92267339ae752799b", + "src/Safe/LivenessGuard.sol": "0xa08460138c22a337f8f5d3a17e02beffe8136c4dba58935cc5c9c2d7ffe1222c", + "src/Safe/LivenessModule.sol": "0x45621d74ea464c75064f9194261d29d47552cf4a9c4f4b3a733f5df5803fc0dd", "src/dispute/BlockOracle.sol": "0x7e724b1ee0116dfd744f556e6237af449c2f40c6426d6f1462ae2a47589283bb", "src/dispute/DisputeGameFactory.sol": "0xfdfa141408d7f8de7e230ff4bef088e30d0e4d569ca743d60d292abdd21ff270", "src/dispute/FaultDisputeGame.sol": "0x0766707ab32338a6586c2340ddfbfd4e9023eeb9dfa3ef87e4b404fb0260479f", From 0dab5fcf4af4ffb53ba239b86c46b5e2be318bff Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 28 Oct 2023 22:16:13 +0000 Subject: [PATCH 275/279] build(deps): bump viem from 1.17.1 to 1.18.0 Bumps [viem](https://github.com/wagmi-dev/viem) from 1.17.1 to 1.18.0. - [Release notes](https://github.com/wagmi-dev/viem/releases) - [Commits](https://github.com/wagmi-dev/viem/compare/viem@1.17.1...viem@1.18.0) --- updated-dependencies: - dependency-name: viem dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] --- packages/contracts-ts/package.json | 2 +- packages/fee-estimation/package.json | 2 +- packages/sdk/package.json | 2 +- packages/web3js-plugin/package.json | 2 +- pnpm-lock.yaml | 62 ++++++++++++++-------------- 5 files changed, 35 insertions(+), 35 deletions(-) diff --git a/packages/contracts-ts/package.json b/packages/contracts-ts/package.json index 5fbd9a45925d..9cd089f4c3fe 100644 --- a/packages/contracts-ts/package.json +++ b/packages/contracts-ts/package.json @@ -82,6 +82,6 @@ "change-case": "4.1.2", "react": "^18.2.0", "react-dom": "^18.2.0", - "viem": "^1.17.1" + "viem": "^1.18.0" } } diff --git a/packages/fee-estimation/package.json b/packages/fee-estimation/package.json index 28d611323354..6cbc34b80625 100644 --- a/packages/fee-estimation/package.json +++ b/packages/fee-estimation/package.json @@ -44,7 +44,7 @@ "jsdom": "^22.1.0", "tsup": "^7.2.0", "typescript": "^5.2.2", - "viem": "^1.17.1", + "viem": "^1.18.0", "vite": "^4.5.0", "vitest": "^0.34.2" }, diff --git a/packages/sdk/package.json b/packages/sdk/package.json index 73f75eb6ddb7..8588f45b2590 100644 --- a/packages/sdk/package.json +++ b/packages/sdk/package.json @@ -56,7 +56,7 @@ "ts-node": "^10.9.1", "typedoc": "^0.25.2", "typescript": "^5.2.2", - "viem": "^1.17.1", + "viem": "^1.18.0", "vitest": "^0.34.2", "zod": "^3.22.4" }, diff --git a/packages/web3js-plugin/package.json b/packages/web3js-plugin/package.json index 92222f2a37a8..d64bbb6179b0 100644 --- a/packages/web3js-plugin/package.json +++ b/packages/web3js-plugin/package.json @@ -37,7 +37,7 @@ "@vitest/coverage-istanbul": "^0.34.6", "tsup": "^7.2.0", "typescript": "^5.2.2", - "viem": "^1.17.1", + "viem": "^1.18.0", "vite": "^4.5.0", "vitest": "^0.34.1", "zod": "^3.22.4" diff --git a/pnpm-lock.yaml b/pnpm-lock.yaml index 857684d175c7..8152a3e6c086 100644 --- a/pnpm-lock.yaml +++ b/pnpm-lock.yaml @@ -298,11 +298,11 @@ importers: specifier: ^18.2.0 version: 18.2.0(react@18.2.0) viem: - specifier: ^1.17.1 - version: 1.17.1(typescript@5.2.2)(zod@3.22.4) + specifier: ^1.18.0 + version: 1.18.0(typescript@5.2.2)(zod@3.22.4) wagmi: specifier: '>1.0.0' - version: 1.0.1(react-dom@18.2.0)(react@18.2.0)(typescript@5.2.2)(viem@1.17.1) + version: 1.0.1(react-dom@18.2.0)(react@18.2.0)(typescript@5.2.2)(viem@1.18.0) devDependencies: '@eth-optimism/contracts-bedrock': specifier: workspace:* @@ -324,7 +324,7 @@ importers: version: 1.5.2(@wagmi/core@1.4.5)(typescript@5.2.2)(wagmi@1.0.1) '@wagmi/core': specifier: ^1.4.5 - version: 1.4.5(react@18.2.0)(typescript@5.2.2)(viem@1.17.1) + version: 1.4.5(react@18.2.0)(typescript@5.2.2)(viem@1.18.0) abitype: specifier: ^0.10.1 version: 0.10.1(typescript@5.2.2) @@ -438,8 +438,8 @@ importers: specifier: ^5.2.2 version: 5.2.2 viem: - specifier: ^1.17.1 - version: 1.17.1(typescript@5.2.2)(zod@3.22.4) + specifier: ^1.18.0 + version: 1.18.0(typescript@5.2.2)(zod@3.22.4) vite: specifier: ^4.5.0 version: 4.5.0(@types/node@20.8.9) @@ -529,8 +529,8 @@ importers: specifier: ^5.2.2 version: 5.2.2 viem: - specifier: ^1.17.1 - version: 1.17.1(typescript@5.2.2)(zod@3.22.4) + specifier: ^1.18.0 + version: 1.18.0(typescript@5.2.2)(zod@3.22.4) vitest: specifier: ^0.34.2 version: 0.34.2 @@ -569,8 +569,8 @@ importers: specifier: ^5.2.2 version: 5.2.2 viem: - specifier: ^1.17.1 - version: 1.17.1(typescript@5.2.2)(zod@3.22.4) + specifier: ^1.18.0 + version: 1.18.0(typescript@5.2.2)(zod@3.22.4) vite: specifier: ^4.5.0 version: 4.5.0(@types/node@20.8.9) @@ -3219,7 +3219,7 @@ packages: resolution: {integrity: sha512-gYw0ki/EAuV1oSyMxpqandHjnthZjYYy+YWpTAzf8BqfXM3ItcZLpjxfg+3+mXW8HIO+3jw6T9iiqEXsqHaMMw==} dependencies: '@safe-global/safe-gateway-typescript-sdk': 3.7.3 - viem: 1.17.1(typescript@5.2.2)(zod@3.22.4) + viem: 1.18.0(typescript@5.2.2)(zod@3.22.4) transitivePeerDependencies: - bufferutil - encoding @@ -4584,7 +4584,7 @@ packages: wagmi: optional: true dependencies: - '@wagmi/core': 1.4.5(react@18.2.0)(typescript@5.2.2)(viem@1.17.1) + '@wagmi/core': 1.4.5(react@18.2.0)(typescript@5.2.2)(viem@1.18.0) abitype: 0.8.7(typescript@5.2.2)(zod@3.22.3) abort-controller: 3.0.0 bundle-require: 3.1.2(esbuild@0.16.17) @@ -4606,15 +4606,15 @@ packages: picocolors: 1.0.0 prettier: 2.8.8 typescript: 5.2.2 - viem: 1.17.1(typescript@5.2.2)(zod@3.22.3) - wagmi: 1.0.1(react-dom@18.2.0)(react@18.2.0)(typescript@5.2.2)(viem@1.17.1) + viem: 1.18.0(typescript@5.2.2)(zod@3.22.3) + wagmi: 1.0.1(react-dom@18.2.0)(react@18.2.0)(typescript@5.2.2)(viem@1.18.0) zod: 3.22.3 transitivePeerDependencies: - bufferutil - utf-8-validate dev: true - /@wagmi/connectors@1.0.1(@wagmi/chains@0.2.22)(react@18.2.0)(typescript@5.2.2)(viem@1.17.1): + /@wagmi/connectors@1.0.1(@wagmi/chains@0.2.22)(react@18.2.0)(typescript@5.2.2)(viem@1.18.0): resolution: {integrity: sha512-fl01vym19DE1uoE+MlASw5zo3Orr/YXlJRjOKLaKYtV+Q7jOLY4TwHgq7sEMs+JYOvFICFBEAlWNNxidr51AqQ==} peerDependencies: '@wagmi/chains': '>=0.2.0' @@ -4637,7 +4637,7 @@ packages: abitype: 0.8.1(typescript@5.2.2) eventemitter3: 4.0.7 typescript: 5.2.2 - viem: 1.17.1(typescript@5.2.2)(zod@3.22.4) + viem: 1.18.0(typescript@5.2.2)(zod@3.22.4) transitivePeerDependencies: - '@react-native-async-storage/async-storage' - bufferutil @@ -4649,7 +4649,7 @@ packages: - utf-8-validate - zod - /@wagmi/connectors@3.1.3(react@18.2.0)(typescript@5.2.2)(viem@1.17.1): + /@wagmi/connectors@3.1.3(react@18.2.0)(typescript@5.2.2)(viem@1.18.0): resolution: {integrity: sha512-UgwsQKQDFObJVJMf9pDfFoXTv710o4zrTHyhIWKBTMMkLpCMsMxN5+ZaDhBYt/BgoRinfRYQo8uwuwLhxE6Log==} peerDependencies: typescript: '>=5.0.4' @@ -4669,7 +4669,7 @@ packages: abitype: 0.8.7(typescript@5.2.2)(zod@3.22.3) eventemitter3: 4.0.7 typescript: 5.2.2 - viem: 1.17.1(typescript@5.2.2)(zod@3.22.4) + viem: 1.18.0(typescript@5.2.2)(zod@3.22.4) transitivePeerDependencies: - '@react-native-async-storage/async-storage' - '@types/react' @@ -4682,7 +4682,7 @@ packages: - zod dev: true - /@wagmi/core@1.0.1(react@18.2.0)(typescript@5.2.2)(viem@1.17.1): + /@wagmi/core@1.0.1(react@18.2.0)(typescript@5.2.2)(viem@1.18.0): resolution: {integrity: sha512-Zzg4Ob92QMF9NsC+z5/8JZjMn3NCCnwVWGJlv79qRX9mp5Ku40OzJNvqDnjcSGjshe6H0L/KtFZAqTlmu8lT7w==} peerDependencies: typescript: '>=4.9.4' @@ -4692,11 +4692,11 @@ packages: optional: true dependencies: '@wagmi/chains': 0.2.22(typescript@5.2.2) - '@wagmi/connectors': 1.0.1(@wagmi/chains@0.2.22)(react@18.2.0)(typescript@5.2.2)(viem@1.17.1) + '@wagmi/connectors': 1.0.1(@wagmi/chains@0.2.22)(react@18.2.0)(typescript@5.2.2)(viem@1.18.0) abitype: 0.8.1(typescript@5.2.2) eventemitter3: 4.0.7 typescript: 5.2.2 - viem: 1.17.1(typescript@5.2.2)(zod@3.22.4) + viem: 1.18.0(typescript@5.2.2)(zod@3.22.4) zustand: 4.3.9(react@18.2.0) transitivePeerDependencies: - '@react-native-async-storage/async-storage' @@ -4710,7 +4710,7 @@ packages: - utf-8-validate - zod - /@wagmi/core@1.4.5(react@18.2.0)(typescript@5.2.2)(viem@1.17.1): + /@wagmi/core@1.4.5(react@18.2.0)(typescript@5.2.2)(viem@1.18.0): resolution: {integrity: sha512-N9luRb1Uk4tBN9kaYcQSWKE9AsRt/rvZaFt5IZech4JPzNN2sQlfhKd9GEjOXYRDqEPHdDvos7qyBKiDNTz4GA==} peerDependencies: typescript: '>=5.0.4' @@ -4719,11 +4719,11 @@ packages: typescript: optional: true dependencies: - '@wagmi/connectors': 3.1.3(react@18.2.0)(typescript@5.2.2)(viem@1.17.1) + '@wagmi/connectors': 3.1.3(react@18.2.0)(typescript@5.2.2)(viem@1.18.0) abitype: 0.8.7(typescript@5.2.2)(zod@3.22.3) eventemitter3: 4.0.7 typescript: 5.2.2 - viem: 1.17.1(typescript@5.2.2)(zod@3.22.4) + viem: 1.18.0(typescript@5.2.2)(zod@3.22.4) zustand: 4.3.9(react@18.2.0) transitivePeerDependencies: - '@react-native-async-storage/async-storage' @@ -14318,8 +14318,8 @@ packages: vfile-message: 2.0.4 dev: true - /viem@1.17.1(typescript@5.2.2)(zod@3.22.3): - resolution: {integrity: sha512-MSbrfntjgIMKPUPdNJ1pnwT1pDfnOzJnKSLqpafw1q+1k6k6M/jxn09g3WbKefIKIok122DcbmviMow+4FqkAg==} + /viem@1.18.0(typescript@5.2.2)(zod@3.22.3): + resolution: {integrity: sha512-NeKi5RFj7fHdsnk5pojivHFLkTyBWyehxeSE/gSPTDJKCWnR9i+Ra0W++VwN5ghciEG55O8b4RdpYhzGmhnr7A==} peerDependencies: typescript: '>=5.0.4' peerDependenciesMeta: @@ -14341,8 +14341,8 @@ packages: - zod dev: true - /viem@1.17.1(typescript@5.2.2)(zod@3.22.4): - resolution: {integrity: sha512-MSbrfntjgIMKPUPdNJ1pnwT1pDfnOzJnKSLqpafw1q+1k6k6M/jxn09g3WbKefIKIok122DcbmviMow+4FqkAg==} + /viem@1.18.0(typescript@5.2.2)(zod@3.22.4): + resolution: {integrity: sha512-NeKi5RFj7fHdsnk5pojivHFLkTyBWyehxeSE/gSPTDJKCWnR9i+Ra0W++VwN5ghciEG55O8b4RdpYhzGmhnr7A==} peerDependencies: typescript: '>=5.0.4' peerDependenciesMeta: @@ -14835,7 +14835,7 @@ packages: xml-name-validator: 4.0.0 dev: true - /wagmi@1.0.1(react-dom@18.2.0)(react@18.2.0)(typescript@5.2.2)(viem@1.17.1): + /wagmi@1.0.1(react-dom@18.2.0)(react@18.2.0)(typescript@5.2.2)(viem@1.18.0): resolution: {integrity: sha512-+2UkZG9eA3tKqXj1wvlvI8mL0Bcff7Tf5CKfUOyQsdKcY+J5rfwYYya25G+jja57umpHFtfxRaL7xDkNjehrRg==} peerDependencies: react: '>=17.0.0' @@ -14848,12 +14848,12 @@ packages: '@tanstack/query-sync-storage-persister': 4.29.25 '@tanstack/react-query': 4.29.25(react-dom@18.2.0)(react@18.2.0) '@tanstack/react-query-persist-client': 4.29.25(@tanstack/react-query@4.29.25) - '@wagmi/core': 1.0.1(react@18.2.0)(typescript@5.2.2)(viem@1.17.1) + '@wagmi/core': 1.0.1(react@18.2.0)(typescript@5.2.2)(viem@1.18.0) abitype: 0.8.1(typescript@5.2.2) react: 18.2.0 typescript: 5.2.2 use-sync-external-store: 1.2.0(react@18.2.0) - viem: 1.17.1(typescript@5.2.2)(zod@3.22.4) + viem: 1.18.0(typescript@5.2.2)(zod@3.22.4) transitivePeerDependencies: - '@react-native-async-storage/async-storage' - bufferutil From 9a6ed52657001a6d66699f7de86902e10632e643 Mon Sep 17 00:00:00 2001 From: Mark Tyneway Date: Mon, 30 Oct 2023 14:49:29 +0300 Subject: [PATCH 276/279] ci-builder: include solc 0.8.19 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit The other 2 compiler versions are included in `ci-builder` so we should also include `0.8.19`. This will save us from needing to download the compiler in CI many times. The following lines are logged in CI: ``` 3[2K[⠆]3[2Ktalling solc version 0.8.19[⠰]3[2Ktalling solc version 0.8.19[⠔] Installing solc version 0.8.19 3[2K[⠒]3[2Kcessfully installed solc 0.8.19[⠑] Successfully installed solc 0.8.19 ``` These lines are not logged for the other 2 solc versions that are included in `ci-builder` because they are already downloaded. --- ops/docker/ci-builder/Dockerfile | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/ops/docker/ci-builder/Dockerfile b/ops/docker/ci-builder/Dockerfile index ab9f0565c258..b7e2d6a3e3e2 100644 --- a/ops/docker/ci-builder/Dockerfile +++ b/ops/docker/ci-builder/Dockerfile @@ -103,7 +103,8 @@ RUN /bin/sh -c set -eux; \ RUN npm i -g pnpm && npm i -g yarn@1 && pnpm --version && yarn --version RUN svm install 0.5.17 && \ - svm install 0.8.15 + svm install 0.8.15 && \ + svm install 0.8.19 RUN echo "downloading and verifying Codecov uploader" && \ curl https://keybase.io/codecovsecurity/pgp_keys.asc | gpg --no-default-keyring --keyring trustedkeys.gpg --import && \ From 381bbb8104437f2e19316824afba03f227a8d226 Mon Sep 17 00:00:00 2001 From: protolambda Date: Mon, 30 Oct 2023 13:25:26 +0100 Subject: [PATCH 277/279] derive: remove outdated channel-out TODOs --- op-node/rollup/derive/channel_out.go | 3 +-- op-node/rollup/derive/span_channel_out.go | 3 +-- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/op-node/rollup/derive/channel_out.go b/op-node/rollup/derive/channel_out.go index 642c32ac06aa..5eb559678f6a 100644 --- a/op-node/rollup/derive/channel_out.go +++ b/op-node/rollup/derive/channel_out.go @@ -91,7 +91,7 @@ func (co *SingularChannelOut) ID() ChannelID { func NewSingularChannelOut(compress Compressor) (*SingularChannelOut, error) { c := &SingularChannelOut{ - id: ChannelID{}, // TODO: use GUID here instead of fully random data + id: ChannelID{}, frame: 0, rlpLength: 0, compress: compress, @@ -104,7 +104,6 @@ func NewSingularChannelOut(compress Compressor) (*SingularChannelOut, error) { return c, nil } -// TODO: reuse SingularChannelOut for performance func (co *SingularChannelOut) Reset() error { co.frame = 0 co.rlpLength = 0 diff --git a/op-node/rollup/derive/span_channel_out.go b/op-node/rollup/derive/span_channel_out.go index 608257b7bbb3..b509247fb704 100644 --- a/op-node/rollup/derive/span_channel_out.go +++ b/op-node/rollup/derive/span_channel_out.go @@ -34,7 +34,7 @@ func (co *SpanChannelOut) ID() ChannelID { func NewSpanChannelOut(compress Compressor, spanBatchBuilder *SpanBatchBuilder) (*SpanChannelOut, error) { c := &SpanChannelOut{ - id: ChannelID{}, // TODO: use GUID here instead of fully random data + id: ChannelID{}, frame: 0, rlpLength: 0, compress: compress, @@ -49,7 +49,6 @@ func NewSpanChannelOut(compress Compressor, spanBatchBuilder *SpanBatchBuilder) return c, nil } -// TODO: reuse ChannelOut for performance func (co *SpanChannelOut) Reset() error { co.frame = 0 co.rlpLength = 0 From ea8ffecef9a614f912ed9aefda95b1f205da4966 Mon Sep 17 00:00:00 2001 From: protolambda Date: Mon, 30 Oct 2023 13:30:25 +0100 Subject: [PATCH 278/279] op-batcher: remove outdated comment --- op-batcher/batcher/channel_manager.go | 1 - 1 file changed, 1 deletion(-) diff --git a/op-batcher/batcher/channel_manager.go b/op-batcher/batcher/channel_manager.go index 5829e3dc448b..8100a3a2826e 100644 --- a/op-batcher/batcher/channel_manager.go +++ b/op-batcher/batcher/channel_manager.go @@ -59,7 +59,6 @@ func NewChannelManager(log log.Logger, metr metrics.Metricer, cfg ChannelConfig, // Clear clears the entire state of the channel manager. // It is intended to be used before launching op-batcher and after an L2 reorg. -// Must set lastProcessedBlock as current L2 safe head fetched from L2 node. func (s *channelManager) Clear() { s.mu.Lock() defer s.mu.Unlock() From 46f9c8c73022b6e7ec7a5d728abf7a6477b24178 Mon Sep 17 00:00:00 2001 From: cby3149 Date: Tue, 31 Oct 2023 10:12:17 -0700 Subject: [PATCH 279/279] Disable build-info in plug --- package.json | 9 ++- .../@foundry-rs__hardhat-forge@0.1.17.patch | 72 +++++++++++++++++++ pnpm-lock.yaml | 46 ++++++------ 3 files changed, 100 insertions(+), 27 deletions(-) create mode 100644 patches/@foundry-rs__hardhat-forge@0.1.17.patch diff --git a/package.json b/package.json index 9a94eb9d18ff..9e7879317dbf 100644 --- a/package.json +++ b/package.json @@ -66,17 +66,22 @@ "markdownlint-cli2": "0.4.0", "mocha": "^10.2.0", "nx": "17.0.2", + "nx-cloud": "latest", "nyc": "^15.1.0", "patch-package": "^8.0.0", "prettier": "^2.8.0", "prettier-plugin-solidity": "^1.0.0-beta.13", "rimraf": "^5.0.5", "ts-mocha": "^10.0.0", - "typescript": "^5.2.2", - "nx-cloud": "latest" + "typescript": "^5.2.2" }, "dependencies": { "@changesets/cli": "^2.26.0", "@codechecks/client": "^0.1.11" + }, + "pnpm": { + "patchedDependencies": { + "@foundry-rs/hardhat-forge@0.1.17": "patches/@foundry-rs__hardhat-forge@0.1.17.patch" + } } } diff --git a/patches/@foundry-rs__hardhat-forge@0.1.17.patch b/patches/@foundry-rs__hardhat-forge@0.1.17.patch new file mode 100644 index 000000000000..a3bc52d6c41f --- /dev/null +++ b/patches/@foundry-rs__hardhat-forge@0.1.17.patch @@ -0,0 +1,72 @@ +diff --git a/CHANGELOG.md b/CHANGELOG.md +deleted file mode 100644 +index 994275f8d189b585bbdeed8be9292c29931d884f..0000000000000000000000000000000000000000 +diff --git a/dist/src/forge/artifacts.js b/dist/src/forge/artifacts.js +index 84cd0f3aae8ab870af8b1c8081d786c0cb2e0a5d..dee730a84ad37630e3a106bf06955bb5148ad4e5 100644 +--- a/dist/src/forge/artifacts.js ++++ b/dist/src/forge/artifacts.js +@@ -50,8 +50,8 @@ class ForgeArtifacts { + get buildInfos() { + if (this._buildInfos.length === 0) { + for (const buildInfoPath of this.getBuildInfoPathsSync()) { +- const buildInfo = fs_extra_1.default.readJsonSync(buildInfoPath); +- this._buildInfos.push({ buildInfo, buildInfoPath }); ++ // const buildInfo = fs_extra_1.default.readJsonSync(buildInfoPath); ++ this._buildInfos.push({ buildInfoPath }); + } + } + return this._buildInfos; +@@ -133,13 +133,13 @@ class ForgeArtifacts { + const buildInfo = fs_extra_1.default.readJsonSync(buildInfoPath); + // Handle ethers-solc serializing the metadata as a string + // when hardhat serializes it as an object +- for (const contract of Object.values(buildInfo.output.contracts)) { +- for (const output of Object.values(contract)) { +- if (typeof output.metadata === "string") { +- output.metadata = JSON.parse(output.metadata); +- } +- } +- } ++ // for (const contract of Object.values(buildInfo.output.contracts)) { ++ // for (const output of Object.values(contract)) { ++ // if (typeof output.metadata === "string") { ++ // output.metadata = JSON.parse(output.metadata); ++ // } ++ // } ++ // } + return buildInfo; + } + async getArtifactPaths() { +@@ -473,19 +473,19 @@ Please replace "${contractName}" for the correct contract name wherever you are + * to the hardhat artifact + */ + _writeDebugFile(out, sourceName) { +- for (const { buildInfo, buildInfoPath } of this.buildInfos) { +- for (const contract of Object.keys(buildInfo.output.contracts)) { +- if (contract === sourceName) { +- const debugFile = { +- _format: "hh-sol-dbg-1", +- buildInfo: path.relative(path.dirname(out), buildInfoPath), +- }; +- const debug = this._getDebugFilePath(out); +- fs_extra_1.default.writeJsonSync(debug, debugFile, { spaces: 2 }); +- return; +- } +- } +- } ++ // for (const { buildInfo, buildInfoPath } of this.buildInfos) { ++ // for (const contract of Object.keys(buildInfo.output.contracts)) { ++ // if (contract === sourceName) { ++ // const debugFile = { ++ // _format: "hh-sol-dbg-1", ++ // buildInfo: path.relative(path.dirname(out), buildInfoPath), ++ // }; ++ // const debug = this._getDebugFilePath(out); ++ // fs_extra_1.default.writeJsonSync(debug, debugFile, { spaces: 2 }); ++ // return; ++ // } ++ // } ++ // } + } + /** + * Converts a foundry artifact path to a hardhat artifact path \ No newline at end of file diff --git a/pnpm-lock.yaml b/pnpm-lock.yaml index 5edf4130887b..d22a34b39e22 100644 --- a/pnpm-lock.yaml +++ b/pnpm-lock.yaml @@ -4,6 +4,11 @@ settings: autoInstallPeers: true excludeLinksFromLockfile: false +patchedDependencies: + '@foundry-rs/hardhat-forge@0.1.17': + hash: 4kwfanljlllriou4skkzupekji + path: patches/@foundry-rs__hardhat-forge@0.1.17.patch + importers: .: @@ -101,7 +106,7 @@ importers: version: 17.0.2 nx-cloud: specifier: latest - version: 16.4.0 + version: 16.5.2 nyc: specifier: ^15.1.0 version: 15.1.0 @@ -284,7 +289,7 @@ importers: version: 0.1.3 '@foundry-rs/hardhat-forge': specifier: ^0.1.17 - version: 0.1.17(@nomiclabs/hardhat-ethers@2.2.3)(ethereum-waffle@3.4.4)(ethers@5.7.2)(hardhat@2.17.3) + version: 0.1.17(patch_hash=4kwfanljlllriou4skkzupekji)(@nomiclabs/hardhat-ethers@2.2.3)(ethereum-waffle@3.4.4)(ethers@5.7.2)(hardhat@2.17.3) '@nomiclabs/hardhat-ethers': specifier: ^2.0.0 version: 2.2.3(ethers@5.7.2)(hardhat@2.17.3) @@ -2511,7 +2516,7 @@ packages: ts-interface-checker: 0.1.13 dev: true - /@foundry-rs/hardhat-forge@0.1.17(@nomiclabs/hardhat-ethers@2.2.3)(ethereum-waffle@3.4.4)(ethers@5.7.2)(hardhat@2.17.3): + /@foundry-rs/hardhat-forge@0.1.17(patch_hash=4kwfanljlllriou4skkzupekji)(@nomiclabs/hardhat-ethers@2.2.3)(ethereum-waffle@3.4.4)(ethers@5.7.2)(hardhat@2.17.3): resolution: {integrity: sha512-2wxzxA12CQmT11PH/KigyVTNm/4vzsVtzVZow6gwCbC41fTyf73a5qbggHZFRR74JXfmvVSkX1BJitTmdzQvxw==} peerDependencies: '@nomiclabs/hardhat-ethers': ^2.0.0 @@ -2536,6 +2541,7 @@ packages: transitivePeerDependencies: - supports-color dev: true + patched: true /@ganache/ethereum-address@0.1.4: resolution: {integrity: sha512-sTkU0M9z2nZUzDeHRzzGlW724xhMLXo2LeX1hixbnjHWY1Zg1hkqORywVfl+g5uOO8ht8T0v+34IxNxAhmWlbw==} @@ -3220,10 +3226,10 @@ packages: hardhat: 2.18.3(ts-node@10.9.1)(typescript@5.2.2) dev: true - /@nrwl/nx-cloud@16.4.0: - resolution: {integrity: sha512-QitrYK6z9ceagetBlgLMZnC0T85k2JTk+oK0MxZ5p/woclqeYN7SiGNZgMzDq8TjJwt8Fm/MDnsSo3xtufmLBg==} + /@nrwl/nx-cloud@16.5.2: + resolution: {integrity: sha512-oHO5T1HRJsR9mbRd8eUqMBPCgqVZLSbAh3zJoPFmhEmjbM4YB9ePRpgYFT8dRNeZUOUd/8Yt7Pb6EVWOHvpD/w==} dependencies: - nx-cloud: 16.4.0 + nx-cloud: 16.5.2 transitivePeerDependencies: - debug dev: true @@ -8962,7 +8968,7 @@ packages: dependencies: debug: 3.2.7 is-core-module: 2.13.1 - resolve: 1.22.4 + resolve: 1.22.8 transitivePeerDependencies: - supports-color dev: true @@ -13940,7 +13946,7 @@ packages: resolution: {integrity: sha512-/5CMN3T0R4XTj4DcGaexo+roZSdSFW/0AOOTROrjxzCG1wrWXEsGbRKevjlIL+ZDE4sZlJr5ED4YW0yqmkK+eA==} dependencies: hosted-git-info: 2.8.9 - resolve: 1.22.4 + resolve: 1.22.8 semver: 5.7.2 validate-npm-package-license: 3.0.4 @@ -14000,11 +14006,11 @@ packages: resolution: {integrity: sha512-ub5E4+FBPKwAZx0UwIQOjYWGHTEq5sPqHQNRN8Z9e4A7u3Tj1weLJsL59yH9vmvqEtBHaOmT6cYQKIZOxp35FQ==} dev: true - /nx-cloud@16.4.0: - resolution: {integrity: sha512-jbq4hWvDwRlJVpxgMgbmNSkue+6XZSn53R6Vo6qmCAWODJ9KY1BZdZ/9VRL8IX/BRKebVFiXp3SapFB1qPhH8A==} + /nx-cloud@16.5.2: + resolution: {integrity: sha512-1t1Ii9gojl8r/8hFGaZ/ZyYR0Cb0hzvXLCsaFuvg+EJEFdvua3P4cfNya/0bdRrm+7Eb/ITUOskbvYq4TSlyGg==} hasBin: true dependencies: - '@nrwl/nx-cloud': 16.4.0 + '@nrwl/nx-cloud': 16.5.2 axios: 1.1.3 chalk: 4.1.2 dotenv: 10.0.0 @@ -14625,7 +14631,7 @@ packages: semver: 7.5.4 slash: 2.0.0 tmp: 0.0.33 - yaml: 2.3.1 + yaml: 2.3.3 dev: true /path-browserify@1.0.1: @@ -14892,7 +14898,7 @@ packages: optional: true dependencies: lilconfig: 2.1.0 - yaml: 2.3.2 + yaml: 2.3.3 dev: true /postcss@8.4.27: @@ -15713,6 +15719,7 @@ packages: is-core-module: 2.13.1 path-parse: 1.0.7 supports-preserve-symlinks-flag: 1.0.0 + dev: true /resolve@1.22.8: resolution: {integrity: sha512-oKWePCxqpd6FlLvGV1VU0x7bkPmmCNolxzjMf4NczoDnQcIWrAF+cPtZn5i6n+RfD2d9i0tzpKnG6Yk168yIyw==} @@ -15721,7 +15728,6 @@ packages: is-core-module: 2.13.1 path-parse: 1.0.7 supports-preserve-symlinks-flag: 1.0.0 - dev: true /resolve@2.0.0-next.4: resolution: {integrity: sha512-iMDbmAWtfU+MHpxt/I5iWI7cY6YVEZUQ3MBgPQ++XD1PELuJHIl82xBmObyP2KyQmkNB2dsqF7seoQQiAn5yDQ==} @@ -17169,7 +17175,7 @@ packages: glob: 7.2.3 mkdirp: 0.5.6 prettier: 2.8.8 - resolve: 1.22.4 + resolve: 1.22.8 ts-essentials: 1.0.4 dev: true @@ -19434,16 +19440,6 @@ packages: engines: {node: '>= 6'} dev: true - /yaml@2.3.1: - resolution: {integrity: sha512-2eHWfjaoXgTBC2jNM1LRef62VQa0umtvRiDSk6HSzW7RvS5YtkabJrwYLLEKWBc8a5U2PTSCs+dJjUTJdlHsWQ==} - engines: {node: '>= 14'} - dev: true - - /yaml@2.3.2: - resolution: {integrity: sha512-N/lyzTPaJasoDmfV7YTrYCI0G/3ivm/9wdG0aHuheKowWQwGTsK0Eoiw6utmzAnI6pkJa0DUVygvp3spqqEKXg==} - engines: {node: '>= 14'} - dev: true - /yaml@2.3.3: resolution: {integrity: sha512-zw0VAJxgeZ6+++/su5AFoqBbZbrEakwu+X0M5HmcwUiBL7AzcuPKjj5we4xfQLp78LkEMpD0cOnUhmgOVy3KdQ==} engines: {node: '>= 14'}