From cee7d1fb02cf17d06a71e71ef070c139be3ed406 Mon Sep 17 00:00:00 2001 From: Ganesh Vanahalli Date: Mon, 8 Apr 2024 16:41:30 -0500 Subject: [PATCH 001/113] Improve blocks re-execution and make it compatible with --init.then-quit --- blocks_reexecutor/blocks_reexecutor.go | 51 ++++++++++++++++++-------- cmd/nitro/nitro.go | 26 ++++++++++--- system_tests/blocks_reexecutor_test.go | 26 +++++-------- 3 files changed, 65 insertions(+), 38 deletions(-) diff --git a/blocks_reexecutor/blocks_reexecutor.go b/blocks_reexecutor/blocks_reexecutor.go index bb6de00cad..bedea37776 100644 --- a/blocks_reexecutor/blocks_reexecutor.go +++ b/blocks_reexecutor/blocks_reexecutor.go @@ -42,10 +42,9 @@ func (c *Config) Validate() error { } var DefaultConfig = Config{ - Enable: false, - Mode: "random", - Room: runtime.NumCPU(), - BlocksPerThread: 10000, + Enable: false, + Mode: "random", + Room: runtime.NumCPU(), } var TestConfig = Config{ @@ -84,25 +83,38 @@ func New(c *Config, blockchain *core.BlockChain, fatalErrChan chan error) *Block start = chainStart end = chainEnd } - if start < chainStart { - log.Warn("state reexecutor's start block number is lower than genesis, resetting to genesis") + if start < chainStart || start > chainEnd { + log.Warn("invalid state reexecutor's start block number, resetting to genesis", "start", start, "genesis", chainStart) start = chainStart } - if end > chainEnd { - log.Warn("state reexecutor's end block number is greater than latest, resetting to latest") + if end > chainEnd || end < chainStart { + log.Warn("invalid state reexecutor's end block number, resetting to latest", "end", end, "latest", chainEnd) end = chainEnd } if c.Mode == "random" && end != start { - if c.BlocksPerThread > end-start { - c.BlocksPerThread = end - start + // Reexecute a range of 10000 or (non-zero) c.BlocksPerThread number of blocks between start to end picked randomly + rng := uint64(10000) + if c.BlocksPerThread != 0 { + rng = c.BlocksPerThread + } + if rng > end-start { + rng = end - start } - start += uint64(rand.Intn(int(end - start - c.BlocksPerThread + 1))) - end = start + c.BlocksPerThread + start += uint64(rand.Intn(int(end - start - rng + 1))) + end = start + rng } - // inclusive of block reexecution [start, end] + // Inclusive of block reexecution [start, end] if start > 0 { start-- } + // Divide work equally among available threads + if c.BlocksPerThread == 0 { + c.BlocksPerThread = 10000 + work := (end - start) / uint64(c.Room) + if work > 0 { + c.BlocksPerThread = work + } + } return &BlocksReExecutor{ config: c, blockchain: blockchain, @@ -125,11 +137,13 @@ func (s *BlocksReExecutor) LaunchBlocksReExecution(ctx context.Context, currentB } // we don't use state release pattern here // TODO do we want to use release pattern here? - startState, startHeader, _, err := arbitrum.FindLastAvailableState(ctx, s.blockchain, s.stateFor, s.blockchain.GetHeaderByNumber(start), nil, -1) + startState, startHeader, release, err := arbitrum.FindLastAvailableState(ctx, s.blockchain, s.stateFor, s.blockchain.GetHeaderByNumber(start), nil, -1) if err != nil { s.fatalErrChan <- fmt.Errorf("blocksReExecutor failed to get last available state while searching for state at %d, err: %w", start, err) return s.startBlock } + // NoOp + defer release() start = startHeader.Number.Uint64() s.LaunchThread(func(ctx context.Context) { _, err := arbitrum.AdvanceStateUpToBlock(ctx, s.blockchain, startState, s.blockchain.GetHeaderByNumber(currentBlock), startHeader, nil) @@ -169,9 +183,14 @@ func (s *BlocksReExecutor) Impl(ctx context.Context) { log.Info("BlocksReExecutor successfully completed re-execution of blocks against historic state", "stateAt", s.startBlock, "startBlock", s.startBlock+1, "endBlock", end) } -func (s *BlocksReExecutor) Start(ctx context.Context) { +func (s *BlocksReExecutor) Start(ctx context.Context, done chan struct{}) { s.StopWaiter.Start(ctx, s) - s.LaunchThread(s.Impl) + s.LaunchThread(func(ctx context.Context) { + s.Impl(ctx) + if done != nil { + close(done) + } + }) } func (s *BlocksReExecutor) StopAndWait() { diff --git a/cmd/nitro/nitro.go b/cmd/nitro/nitro.go index 997adf9369..59241204f1 100644 --- a/cmd/nitro/nitro.go +++ b/cmd/nitro/nitro.go @@ -494,6 +494,25 @@ func mainImpl() int { return 1 } + fatalErrChan := make(chan error, 10) + + var blocksReExecutor *blocksreexecutor.BlocksReExecutor + if nodeConfig.BlocksReExecutor.Enable && l2BlockChain != nil { + blocksReExecutor = blocksreexecutor.New(&nodeConfig.BlocksReExecutor, l2BlockChain, fatalErrChan) + if nodeConfig.Init.ThenQuit { + success := make(chan struct{}) + blocksReExecutor.Start(ctx, success) + deferFuncs = append(deferFuncs, func() { blocksReExecutor.StopAndWait() }) + select { + case err := <-fatalErrChan: + log.Error("shutting down due to fatal error", "err", err) + defer log.Error("shut down due to fatal error", "err", err) + return 1 + case <-success: + } + } + } + if nodeConfig.Init.ThenQuit && nodeConfig.Init.ResetToMessage < 0 { return 0 } @@ -514,8 +533,6 @@ func mainImpl() int { return 1 } - fatalErrChan := make(chan error, 10) - var valNode *valnode.ValidationNode if sameProcessValidationNodeEnabled { valNode, err = valnode.CreateValidationNode( @@ -644,9 +661,8 @@ func mainImpl() int { // remove previous deferFuncs, StopAndWait closes database and blockchain. deferFuncs = []func(){func() { currentNode.StopAndWait() }} } - if nodeConfig.BlocksReExecutor.Enable && l2BlockChain != nil { - blocksReExecutor := blocksreexecutor.New(&nodeConfig.BlocksReExecutor, l2BlockChain, fatalErrChan) - blocksReExecutor.Start(ctx) + if blocksReExecutor != nil && !nodeConfig.Init.ThenQuit { + blocksReExecutor.Start(ctx, nil) deferFuncs = append(deferFuncs, func() { blocksReExecutor.StopAndWait() }) } diff --git a/system_tests/blocks_reexecutor_test.go b/system_tests/blocks_reexecutor_test.go index c2941ddcc4..66690d1427 100644 --- a/system_tests/blocks_reexecutor_test.go +++ b/system_tests/blocks_reexecutor_test.go @@ -45,16 +45,11 @@ func TestBlocksReExecutorModes(t *testing.T) { } } + // Reexecute blocks at mode full success := make(chan struct{}) + executorFull := blocksreexecutor.New(&blocksreexecutor.TestConfig, blockchain, feedErrChan) + executorFull.Start(ctx, success) - // Reexecute blocks at mode full - go func() { - executorFull := blocksreexecutor.New(&blocksreexecutor.TestConfig, blockchain, feedErrChan) - executorFull.StopWaiter.Start(ctx, executorFull) - executorFull.Impl(ctx) - executorFull.StopAndWait() - success <- struct{}{} - }() select { case err := <-feedErrChan: t.Errorf("error occurred: %v", err) @@ -66,15 +61,12 @@ func TestBlocksReExecutorModes(t *testing.T) { } // Reexecute blocks at mode random - go func() { - c := &blocksreexecutor.TestConfig - c.Mode = "random" - executorRandom := blocksreexecutor.New(c, blockchain, feedErrChan) - executorRandom.StopWaiter.Start(ctx, executorRandom) - executorRandom.Impl(ctx) - executorRandom.StopAndWait() - success <- struct{}{} - }() + success = make(chan struct{}) + c := &blocksreexecutor.TestConfig + c.Mode = "random" + executorRandom := blocksreexecutor.New(c, blockchain, feedErrChan) + executorRandom.Start(ctx, success) + select { case err := <-feedErrChan: t.Errorf("error occurred: %v", err) From f9055c93622bd32fe2f5049bd89600236c4ee689 Mon Sep 17 00:00:00 2001 From: Maciej Kulawik Date: Sat, 20 Apr 2024 02:26:52 +0200 Subject: [PATCH 002/113] add pebble extra options --- arbnode/dataposter/storage_test.go | 3 +- cmd/conf/database.go | 123 +++++++++++++++++++++++++++-- cmd/nitro/init.go | 12 +-- cmd/nitro/nitro.go | 5 +- cmd/pruning/pruning.go | 8 +- execution/gethexec/node.go | 2 +- go-ethereum | 2 +- system_tests/common_test.go | 11 ++- system_tests/das_test.go | 6 +- system_tests/pruning_test.go | 6 +- system_tests/staterecovery_test.go | 4 +- 11 files changed, 152 insertions(+), 30 deletions(-) diff --git a/arbnode/dataposter/storage_test.go b/arbnode/dataposter/storage_test.go index f98c120f38..343efac3c7 100644 --- a/arbnode/dataposter/storage_test.go +++ b/arbnode/dataposter/storage_test.go @@ -19,6 +19,7 @@ import ( "github.com/offchainlabs/nitro/arbnode/dataposter/redis" "github.com/offchainlabs/nitro/arbnode/dataposter/slice" "github.com/offchainlabs/nitro/arbnode/dataposter/storage" + "github.com/offchainlabs/nitro/cmd/conf" "github.com/offchainlabs/nitro/util/arbmath" "github.com/offchainlabs/nitro/util/redisutil" "github.com/offchainlabs/nitro/util/signature" @@ -44,7 +45,7 @@ func newLevelDBStorage(t *testing.T, encF storage.EncoderDecoderF) *dbstorage.St func newPebbleDBStorage(t *testing.T, encF storage.EncoderDecoderF) *dbstorage.Storage { t.Helper() - db, err := rawdb.NewPebbleDBDatabase(path.Join(t.TempDir(), "pebble.db"), 0, 0, "default", false, true) + db, err := rawdb.NewPebbleDBDatabase(path.Join(t.TempDir(), "pebble.db"), 0, 0, "default", false, true, conf.PersistentConfigDefault.Pebble.ExtraOptions()) if err != nil { t.Fatalf("NewPebbleDBDatabase() unexpected error: %v", err) } diff --git a/cmd/conf/database.go b/cmd/conf/database.go index b049375d66..be0c630fa9 100644 --- a/cmd/conf/database.go +++ b/cmd/conf/database.go @@ -8,17 +8,21 @@ import ( "os" "path" "path/filepath" + "runtime" + "time" + "github.com/ethereum/go-ethereum/ethdb/pebble" flag "github.com/spf13/pflag" ) type PersistentConfig struct { - GlobalConfig string `koanf:"global-config"` - Chain string `koanf:"chain"` - LogDir string `koanf:"log-dir"` - Handles int `koanf:"handles"` - Ancient string `koanf:"ancient"` - DBEngine string `koanf:"db-engine"` + GlobalConfig string `koanf:"global-config"` + Chain string `koanf:"chain"` + LogDir string `koanf:"log-dir"` + Handles int `koanf:"handles"` + Ancient string `koanf:"ancient"` + DBEngine string `koanf:"db-engine"` + Pebble PebbleConfig `koanf:"pebble"` } var PersistentConfigDefault = PersistentConfig{ @@ -28,6 +32,7 @@ var PersistentConfigDefault = PersistentConfig{ Handles: 512, Ancient: "", DBEngine: "leveldb", + Pebble: PebbleConfigDefault, } func PersistentConfigAddOptions(prefix string, f *flag.FlagSet) { @@ -37,6 +42,7 @@ func PersistentConfigAddOptions(prefix string, f *flag.FlagSet) { f.Int(prefix+".handles", PersistentConfigDefault.Handles, "number of file descriptor handles to use for the database") f.String(prefix+".ancient", PersistentConfigDefault.Ancient, "directory of ancient where the chain freezer can be opened") f.String(prefix+".db-engine", PersistentConfigDefault.DBEngine, "backing database implementation to use ('leveldb' or 'pebble')") + PebbleConfigAddOptions(prefix+".pebble", f) } func (c *PersistentConfig) ResolveDirectoryNames() error { @@ -96,3 +102,108 @@ func (c *PersistentConfig) Validate() error { } return nil } + +type PebbleConfig struct { + BytesPerSync int `koanf:"bytes-per-sync"` + L0CompactionFileThreshold int `koanf:"l0-compaction-file-threshold"` + L0CompactionThreshold int `koanf:"l0-compaction-threshold"` + L0StopWritesThreshold int `koanf:"l0-stop-writes-threshold"` + LBaseMaxBytes int64 `koanf:"l-base-max-bytes"` + MaxConcurrentCompactions int `koanf:"max-concurrent-compactions"` + DisableAutomaticCompactions bool `koanf:"disable-automatic-compactions"` + WALBytesPerSync int `koanf:"wal-bytes-per-sync"` + WALDir string `koanf:"wal-dir"` + WALMinSyncInterval int `koanf:"wal-min-sync-interval"` + TargetByteDeletionRate int `koanf:"target-byte-deletion-rate"` + Experimental PebbleExperimentalConfig `koaf:"experimental"` +} + +var PebbleConfigDefault = PebbleConfig{ + BytesPerSync: 0, // pebble default will be used + L0CompactionFileThreshold: 0, // pebble default will be used + L0CompactionThreshold: 0, // pebble default will be used + L0StopWritesThreshold: 0, // pebble default will be used + LBaseMaxBytes: 0, // pebble default will be used + MaxConcurrentCompactions: runtime.NumCPU(), + DisableAutomaticCompactions: false, + WALBytesPerSync: 0, // pebble default will be used + WALDir: "", // default will use same dir as for sstables + WALMinSyncInterval: 0, // pebble default will be used + TargetByteDeletionRate: 0, // pebble default will be used + Experimental: PebbleExperimentalConfigDefault, +} + +func PebbleConfigAddOptions(prefix string, f *flag.FlagSet) { + f.Int(prefix+".bytes-per-sync", PebbleConfigDefault.BytesPerSync, "number of bytes to write to a SSTable before calling Sync on it in the background (0 = pebble default)") + f.Int(prefix+".l0-compaction-file-threshold", PebbleConfigDefault.L0CompactionFileThreshold, "count of L0 files necessary to trigger an L0 compaction (0 = pebble default)") + f.Int(prefix+".l0-compaction-threshold", PebbleConfigDefault.L0CompactionThreshold, "amount of L0 read-amplification necessary to trigger an L0 compaction (0 = pebble default)") + f.Int(prefix+".l0-stop-writes-threshold", PebbleConfigDefault.L0StopWritesThreshold, "hard limit on L0 read-amplification, computed as the number of L0 sublevels. Writes are stopped when this threshold is reached (0 = pebble default)") + f.Int64(prefix+".l-base-max-bytes", PebbleConfigDefault.LBaseMaxBytes, "hard limit on L0 read-amplification, computed as the number of L0 sublevels. Writes are stopped when this threshold is reached (0 = pebble default)") + f.Int(prefix+".max-concurrent-compactions", PebbleConfigDefault.MaxConcurrentCompactions, "maximum number of concurrent compactions (0 = pebble default)") + f.Bool(prefix+".disable-automatic-compactions", PebbleConfigDefault.DisableAutomaticCompactions, "disables automatic compactions") + f.Int(prefix+".wal-bytes-per-sync", PebbleConfigDefault.WALBytesPerSync, "number of bytes to write to a write-ahead log (WAL) before calling Sync on it in the backgroud (0 = pebble default)") + f.String(prefix+".wal-dir", PebbleConfigDefault.WALDir, "directory to store write-ahead logs (WALs) in. If empty, WALs will be stored in the same directory as sstables") + f.Int(prefix+".wal-min-sync-interval", PebbleConfigDefault.WALMinSyncInterval, "minimum duration in microseconds between syncs of the WAL. If WAL syncs are requested faster than this interval, they will be artificially delayed.") + PebbleExperimentalConfigAddOptions(".experimental", f) +} + +type PebbleExperimentalConfig struct { + L0CompactionConcurrency int `koanf:"l0-compaction-concurrency"` + CompactionDebtConcurrency uint64 `koanf:"compaction-debt-concurrency"` + ReadCompactionRate int64 `koanf:"read-compaction-rate"` + ReadSamplingMultiplier int64 `koanf:"read-sampling-multiplier"` + MaxWriterConcurrency int `koanf:"max-writer-concurrency"` + ForceWriterParallelism bool `koanf:"force-writer-parallelism"` +} + +var PebbleExperimentalConfigDefault = PebbleExperimentalConfig{ + L0CompactionConcurrency: 0, + CompactionDebtConcurrency: 0, + ReadCompactionRate: 0, + ReadSamplingMultiplier: -1, + MaxWriterConcurrency: 0, + ForceWriterParallelism: false, +} + +func PebbleExperimentalConfigAddOptions(prefix string, f *flag.FlagSet) { + f.Int(prefix+".l0-compaction-concurrency", PebbleExperimentalConfigDefault.L0CompactionConcurrency, "threshold of L0 read-amplification at which compaction concurrency is enabled (if compaction-debt-concurrency was not already exceeded). Every multiple of this value enables another concurrent compaction up to max-concurrent-compactions. (0 = pebble default)") + f.Uint64(prefix+".compaction-debt-concurrency", PebbleExperimentalConfigDefault.CompactionDebtConcurrency, "controls the threshold of compaction debt at which additional compaction concurrency slots are added. For every multiple of this value in compaction debt bytes, an additional concurrent compaction is added. This works \"on top\" of l0-compaction-concurrency, so the higher of the count of compaction concurrency slots as determined by the two options is chosen. (0 = pebble default)") + f.Int64(prefix+".read-compaction-rate", PebbleExperimentalConfigDefault.ReadCompactionRate, "controls the frequency of read triggered compactions by adjusting `AllowedSeeks` in manifest.FileMetadata: AllowedSeeks = FileSize / ReadCompactionRate") + f.Int64(prefix+".read-sampling-multiplier", PebbleExperimentalConfigDefault.ReadSamplingMultiplier, "a multiplier for the readSamplingPeriod in iterator.maybeSampleRead() to control the frequency of read sampling to trigger a read triggered compaction. A value of -1 prevents sampling and disables read triggered compactions. Geth default is -1. The pebble default is 1 << 4. which gets multiplied with a constant of 1 << 16 to yield 1 << 20 (1MB). (0 = pebble default)") + f.Int(prefix+".max-writer-concurrency", PebbleExperimentalConfigDefault.MaxWriterConcurrency, "maximum number of compression workers the compression queue is allowed to use. If max-writer-concurrency > 0, then the Writer will use parallelism, to compress and write blocks to disk. Otherwise, the writer will compress and write blocks to disk synchronously.") + f.Bool(prefix+".force-writer-parallelism", PebbleExperimentalConfigDefault.ForceWriterParallelism, "force parallelism in the sstable Writer for the metamorphic tests. Even with the MaxWriterConcurrency option set, pebble only enables parallelism in the sstable Writer if there is enough CPU available, and this option bypasses that.") +} + +func (c *PebbleConfig) ExtraOptions() *pebble.ExtraOptions { + var maxConcurrentCompactions func() int + if c.MaxConcurrentCompactions > 0 { + maxConcurrentCompactions = func() int { return c.MaxConcurrentCompactions } + } + var walMinSyncInterval func() time.Duration + if c.WALMinSyncInterval > 0 { + walMinSyncInterval = func() time.Duration { + return time.Microsecond * time.Duration(c.WALMinSyncInterval) + } + } + return &pebble.ExtraOptions{ + BytesPerSync: c.BytesPerSync, + L0CompactionFileThreshold: c.L0CompactionFileThreshold, + L0CompactionThreshold: c.L0CompactionThreshold, + L0StopWritesThreshold: c.L0StopWritesThreshold, + LBaseMaxBytes: c.LBaseMaxBytes, + MaxConcurrentCompactions: maxConcurrentCompactions, + DisableAutomaticCompactions: c.DisableAutomaticCompactions, + WALBytesPerSync: c.WALBytesPerSync, + WALDir: c.WALDir, + WALMinSyncInterval: walMinSyncInterval, + TargetByteDeletionRate: c.TargetByteDeletionRate, + Experimental: pebble.ExtraOptionsExperimental{ + L0CompactionConcurrency: c.Experimental.L0CompactionConcurrency, + CompactionDebtConcurrency: c.Experimental.CompactionDebtConcurrency, + ReadCompactionRate: c.Experimental.ReadCompactionRate, + ReadSamplingMultiplier: c.Experimental.ReadSamplingMultiplier, + MaxWriterConcurrency: c.Experimental.MaxWriterConcurrency, + ForceWriterParallelism: c.Experimental.ForceWriterParallelism, + }, + } +} diff --git a/cmd/nitro/init.go b/cmd/nitro/init.go index 6ebfec3bb1..2bae2d9e11 100644 --- a/cmd/nitro/init.go +++ b/cmd/nitro/init.go @@ -159,19 +159,19 @@ func validateBlockChain(blockChain *core.BlockChain, chainConfig *params.ChainCo return nil } -func openInitializeChainDb(ctx context.Context, stack *node.Node, config *NodeConfig, chainId *big.Int, cacheConfig *core.CacheConfig, l1Client arbutil.L1Interface, rollupAddrs chaininfo.RollupAddresses) (ethdb.Database, *core.BlockChain, error) { +func openInitializeChainDb(ctx context.Context, stack *node.Node, config *NodeConfig, chainId *big.Int, cacheConfig *core.CacheConfig, persistentConfig *conf.PersistentConfig, l1Client arbutil.L1Interface, rollupAddrs chaininfo.RollupAddresses) (ethdb.Database, *core.BlockChain, error) { if !config.Init.Force { - if readOnlyDb, err := stack.OpenDatabaseWithFreezer("l2chaindata", 0, 0, "", "l2chaindata/", true); err == nil { + if readOnlyDb, err := stack.OpenDatabaseWithFreezerWithExtraOptions("l2chaindata", 0, 0, "", "l2chaindata/", true, persistentConfig.Pebble.ExtraOptions()); err == nil { if chainConfig := gethexec.TryReadStoredChainConfig(readOnlyDb); chainConfig != nil { readOnlyDb.Close() if !arbmath.BigEquals(chainConfig.ChainID, chainId) { return nil, nil, fmt.Errorf("database has chain ID %v but config has chain ID %v (are you sure this database is for the right chain?)", chainConfig.ChainID, chainId) } - chainDb, err := stack.OpenDatabaseWithFreezer("l2chaindata", config.Execution.Caching.DatabaseCache, config.Persistent.Handles, config.Persistent.Ancient, "l2chaindata/", false) + chainDb, err := stack.OpenDatabaseWithFreezerWithExtraOptions("l2chaindata", config.Execution.Caching.DatabaseCache, config.Persistent.Handles, config.Persistent.Ancient, "l2chaindata/", false, persistentConfig.Pebble.ExtraOptions()) if err != nil { return chainDb, nil, err } - err = pruning.PruneChainDb(ctx, chainDb, stack, &config.Init, cacheConfig, l1Client, rollupAddrs, config.Node.ValidatorRequired()) + err = pruning.PruneChainDb(ctx, chainDb, stack, &config.Init, cacheConfig, persistentConfig, l1Client, rollupAddrs, config.Node.ValidatorRequired()) if err != nil { return chainDb, nil, fmt.Errorf("error pruning: %w", err) } @@ -219,7 +219,7 @@ func openInitializeChainDb(ctx context.Context, stack *node.Node, config *NodeCo var initDataReader statetransfer.InitDataReader = nil - chainDb, err := stack.OpenDatabaseWithFreezer("l2chaindata", config.Execution.Caching.DatabaseCache, config.Persistent.Handles, config.Persistent.Ancient, "l2chaindata/", false) + chainDb, err := stack.OpenDatabaseWithFreezerWithExtraOptions("l2chaindata", config.Execution.Caching.DatabaseCache, config.Persistent.Handles, config.Persistent.Ancient, "l2chaindata/", false, persistentConfig.Pebble.ExtraOptions()) if err != nil { return chainDb, nil, err } @@ -367,7 +367,7 @@ func openInitializeChainDb(ctx context.Context, stack *node.Node, config *NodeCo return chainDb, l2BlockChain, err } - err = pruning.PruneChainDb(ctx, chainDb, stack, &config.Init, cacheConfig, l1Client, rollupAddrs, config.Node.ValidatorRequired()) + err = pruning.PruneChainDb(ctx, chainDb, stack, &config.Init, cacheConfig, persistentConfig, l1Client, rollupAddrs, config.Node.ValidatorRequired()) if err != nil { return chainDb, nil, fmt.Errorf("error pruning: %w", err) } diff --git a/cmd/nitro/nitro.go b/cmd/nitro/nitro.go index 79ecd51ac2..f70d16a25a 100644 --- a/cmd/nitro/nitro.go +++ b/cmd/nitro/nitro.go @@ -177,6 +177,7 @@ func mainImpl() int { nodeConfig.Auth.Apply(&stackConf) nodeConfig.IPC.Apply(&stackConf) nodeConfig.GraphQL.Apply(&stackConf) + if nodeConfig.WS.ExposeAll { stackConf.WSModules = append(stackConf.WSModules, "personal") } @@ -476,7 +477,7 @@ func mainImpl() int { } } - chainDb, l2BlockChain, err := openInitializeChainDb(ctx, stack, nodeConfig, new(big.Int).SetUint64(nodeConfig.Chain.ID), gethexec.DefaultCacheConfigFor(stack, &nodeConfig.Execution.Caching), l1Client, rollupAddrs) + chainDb, l2BlockChain, err := openInitializeChainDb(ctx, stack, nodeConfig, new(big.Int).SetUint64(nodeConfig.Chain.ID), gethexec.DefaultCacheConfigFor(stack, &nodeConfig.Execution.Caching), &nodeConfig.Persistent, l1Client, rollupAddrs) if l2BlockChain != nil { deferFuncs = append(deferFuncs, func() { l2BlockChain.Stop() }) } @@ -487,7 +488,7 @@ func mainImpl() int { return 1 } - arbDb, err := stack.OpenDatabase("arbitrumdata", 0, 0, "arbitrumdata/", false) + arbDb, err := stack.OpenDatabaseWithExtraOptions("arbitrumdata", 0, 0, "arbitrumdata/", false, nodeConfig.Persistent.Pebble.ExtraOptions()) deferFuncs = append(deferFuncs, func() { closeDb(arbDb, "arbDb") }) if err != nil { log.Error("failed to open database", "err", err) diff --git a/cmd/pruning/pruning.go b/cmd/pruning/pruning.go index c483526aa1..363126a49f 100644 --- a/cmd/pruning/pruning.go +++ b/cmd/pruning/pruning.go @@ -80,12 +80,12 @@ func (r *importantRoots) addHeader(header *types.Header, overwrite bool) error { var hashListRegex = regexp.MustCompile("^(0x)?[0-9a-fA-F]{64}(,(0x)?[0-9a-fA-F]{64})*$") // Finds important roots to retain while proving -func findImportantRoots(ctx context.Context, chainDb ethdb.Database, stack *node.Node, initConfig *conf.InitConfig, cacheConfig *core.CacheConfig, l1Client arbutil.L1Interface, rollupAddrs chaininfo.RollupAddresses, validatorRequired bool) ([]common.Hash, error) { +func findImportantRoots(ctx context.Context, chainDb ethdb.Database, stack *node.Node, initConfig *conf.InitConfig, cacheConfig *core.CacheConfig, persistentConfig *conf.PersistentConfig, l1Client arbutil.L1Interface, rollupAddrs chaininfo.RollupAddresses, validatorRequired bool) ([]common.Hash, error) { chainConfig := gethexec.TryReadStoredChainConfig(chainDb) if chainConfig == nil { return nil, errors.New("database doesn't have a chain config (was this node initialized?)") } - arbDb, err := stack.OpenDatabase("arbitrumdata", 0, 0, "arbitrumdata/", true) + arbDb, err := stack.OpenDatabaseWithExtraOptions("arbitrumdata", 0, 0, "arbitrumdata/", true, persistentConfig.Pebble.ExtraOptions()) if err != nil { return nil, err } @@ -232,11 +232,11 @@ func findImportantRoots(ctx context.Context, chainDb ethdb.Database, stack *node return roots.roots, nil } -func PruneChainDb(ctx context.Context, chainDb ethdb.Database, stack *node.Node, initConfig *conf.InitConfig, cacheConfig *core.CacheConfig, l1Client arbutil.L1Interface, rollupAddrs chaininfo.RollupAddresses, validatorRequired bool) error { +func PruneChainDb(ctx context.Context, chainDb ethdb.Database, stack *node.Node, initConfig *conf.InitConfig, cacheConfig *core.CacheConfig, persistentConfig *conf.PersistentConfig, l1Client arbutil.L1Interface, rollupAddrs chaininfo.RollupAddresses, validatorRequired bool) error { if initConfig.Prune == "" { return pruner.RecoverPruning(stack.InstanceDir(), chainDb) } - root, err := findImportantRoots(ctx, chainDb, stack, initConfig, cacheConfig, l1Client, rollupAddrs, validatorRequired) + root, err := findImportantRoots(ctx, chainDb, stack, initConfig, cacheConfig, persistentConfig, l1Client, rollupAddrs, validatorRequired) if err != nil { return fmt.Errorf("failed to find root to retain for pruning: %w", err) } diff --git a/execution/gethexec/node.go b/execution/gethexec/node.go index 54f9ed6fe1..284934245b 100644 --- a/execution/gethexec/node.go +++ b/execution/gethexec/node.go @@ -216,7 +216,7 @@ func CreateExecutionNode( var classicOutbox *ClassicOutboxRetriever if l2BlockChain.Config().ArbitrumChainParams.GenesisBlockNum > 0 { - classicMsgDb, err := stack.OpenDatabase("classic-msg", 0, 0, "classicmsg/", true) + classicMsgDb, err := stack.OpenDatabase("classic-msg", 0, 0, "classicmsg/", true) // TODO can we skip using ExtraOptions here? if err != nil { log.Warn("Classic Msg Database not found", "err", err) classicOutbox = nil diff --git a/go-ethereum b/go-ethereum index daccadb06c..935cb21640 160000 --- a/go-ethereum +++ b/go-ethereum @@ -1 +1 @@ -Subproject commit daccadb06c7bd9ad7e86c74f33ea39d897f0ece4 +Subproject commit 935cb216402c9693faf86d75a7fbb045109ed4a3 diff --git a/system_tests/common_test.go b/system_tests/common_test.go index 7f9f4844fd..4bcf1349e2 100644 --- a/system_tests/common_test.go +++ b/system_tests/common_test.go @@ -21,6 +21,7 @@ import ( "github.com/offchainlabs/nitro/arbstate" "github.com/offchainlabs/nitro/blsSignatures" "github.com/offchainlabs/nitro/cmd/chaininfo" + "github.com/offchainlabs/nitro/cmd/conf" "github.com/offchainlabs/nitro/cmd/genericconf" "github.com/offchainlabs/nitro/das" "github.com/offchainlabs/nitro/deploy" @@ -718,9 +719,10 @@ func createL2BlockChainWithStackConfig( stack, err = node.New(stackConfig) Require(t, err) - chainDb, err := stack.OpenDatabase("l2chaindata", 0, 0, "l2chaindata/", false) + // TODO get pebble.ExtraOptions from conf.PersistentConfig when opening the DBs + chainDb, err := stack.OpenDatabaseWithExtraOptions("l2chaindata", 0, 0, "l2chaindata/", false, conf.PersistentConfigDefault.Pebble.ExtraOptions()) Require(t, err) - arbDb, err := stack.OpenDatabase("arbitrumdata", 0, 0, "arbitrumdata/", false) + arbDb, err := stack.OpenDatabaseWithExtraOptions("arbitrumdata", 0, 0, "arbitrumdata/", false, conf.PersistentConfigDefault.Pebble.ExtraOptions()) Require(t, err) initReader := statetransfer.NewMemoryInitDataReader(&l2info.ArbInitData) @@ -922,9 +924,10 @@ func Create2ndNodeWithConfig( l2stack, err := node.New(stackConfig) Require(t, err) - l2chainDb, err := l2stack.OpenDatabase("l2chaindata", 0, 0, "l2chaindata/", false) + // TODO get pebble.ExtraOptions from conf.PersistentConfig when opening the DBs + l2chainDb, err := l2stack.OpenDatabaseWithExtraOptions("l2chaindata", 0, 0, "l2chaindata/", false, conf.PersistentConfigDefault.Pebble.ExtraOptions()) Require(t, err) - l2arbDb, err := l2stack.OpenDatabase("arbitrumdata", 0, 0, "arbitrumdata/", false) + l2arbDb, err := l2stack.OpenDatabaseWithExtraOptions("arbitrumdata", 0, 0, "arbitrumdata/", false, conf.PersistentConfigDefault.Pebble.ExtraOptions()) Require(t, err) initReader := statetransfer.NewMemoryInitDataReader(l2InitData) diff --git a/system_tests/das_test.go b/system_tests/das_test.go index c4a3c453d8..be0ef9c957 100644 --- a/system_tests/das_test.go +++ b/system_tests/das_test.go @@ -25,6 +25,7 @@ import ( "github.com/offchainlabs/nitro/arbnode" "github.com/offchainlabs/nitro/arbutil" "github.com/offchainlabs/nitro/blsSignatures" + "github.com/offchainlabs/nitro/cmd/conf" "github.com/offchainlabs/nitro/cmd/genericconf" "github.com/offchainlabs/nitro/das" "github.com/offchainlabs/nitro/execution/gethexec" @@ -175,10 +176,11 @@ func TestDASRekey(t *testing.T) { l2stackA, err := node.New(stackConfig) Require(t, err) - l2chainDb, err := l2stackA.OpenDatabase("l2chaindata", 0, 0, "l2chaindata/", false) + // TODO get pebble.ExtraOptions from conf.PersistentConfig + l2chainDb, err := l2stackA.OpenDatabaseWithExtraOptions("l2chaindata", 0, 0, "l2chaindata/", false, conf.PersistentConfigDefault.Pebble.ExtraOptions()) Require(t, err) - l2arbDb, err := l2stackA.OpenDatabase("arbitrumdata", 0, 0, "arbitrumdata/", false) + l2arbDb, err := l2stackA.OpenDatabaseWithExtraOptions("arbitrumdata", 0, 0, "arbitrumdata/", false, conf.PersistentConfigDefault.Pebble.ExtraOptions()) Require(t, err) l2blockchain, err := gethexec.GetBlockChain(l2chainDb, nil, chainConfig, gethexec.ConfigDefaultTest().TxLookupLimit) diff --git a/system_tests/pruning_test.go b/system_tests/pruning_test.go index 8efc8653e6..e83c350804 100644 --- a/system_tests/pruning_test.go +++ b/system_tests/pruning_test.go @@ -65,7 +65,8 @@ func TestPruning(t *testing.T) { stack, err := node.New(builder.l2StackConfig) Require(t, err) defer stack.Close() - chainDb, err := stack.OpenDatabase("l2chaindata", 0, 0, "l2chaindata/", false) + // TODO get pebble.ExtraOptions from conf.PersistentConfig + chainDb, err := stack.OpenDatabaseWithExtraOptions("l2chaindata", 0, 0, "l2chaindata/", false, conf.PersistentConfigDefault.Pebble.ExtraOptions()) Require(t, err) defer chainDb.Close() chainDbEntriesBeforePruning := countStateEntries(chainDb) @@ -89,7 +90,8 @@ func TestPruning(t *testing.T) { initConfig := conf.InitConfigDefault initConfig.Prune = "full" coreCacheConfig := gethexec.DefaultCacheConfigFor(stack, &builder.execConfig.Caching) - err = pruning.PruneChainDb(ctx, chainDb, stack, &initConfig, coreCacheConfig, builder.L1.Client, *builder.L2.ConsensusNode.DeployInfo, false) + persistentConfig := conf.PersistentConfigDefault + err = pruning.PruneChainDb(ctx, chainDb, stack, &initConfig, coreCacheConfig, &persistentConfig, builder.L1.Client, *builder.L2.ConsensusNode.DeployInfo, false) Require(t, err) for _, key := range testKeys { diff --git a/system_tests/staterecovery_test.go b/system_tests/staterecovery_test.go index 632e748da8..9dc1081a7b 100644 --- a/system_tests/staterecovery_test.go +++ b/system_tests/staterecovery_test.go @@ -9,6 +9,7 @@ import ( "github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/node" "github.com/ethereum/go-ethereum/trie" + "github.com/offchainlabs/nitro/cmd/conf" "github.com/offchainlabs/nitro/cmd/staterecovery" "github.com/offchainlabs/nitro/execution/gethexec" ) @@ -49,7 +50,8 @@ func TestRectreateMissingStates(t *testing.T) { stack, err := node.New(builder.l2StackConfig) Require(t, err) defer stack.Close() - chainDb, err := stack.OpenDatabase("l2chaindata", 0, 0, "l2chaindata/", false) + // TODO get pebble.ExtraOptions from conf.PersistentConfig + chainDb, err := stack.OpenDatabaseWithExtraOptions("l2chaindata", 0, 0, "l2chaindata/", false, conf.PersistentConfigDefault.Pebble.ExtraOptions()) Require(t, err) defer chainDb.Close() cacheConfig := gethexec.DefaultCacheConfigFor(stack, &gethexec.DefaultCachingConfig) From 43d6e82020468299029f96426f20dd98635380b2 Mon Sep 17 00:00:00 2001 From: Maciej Kulawik Date: Mon, 22 Apr 2024 16:20:57 +0200 Subject: [PATCH 003/113] update geth --- go-ethereum | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/go-ethereum b/go-ethereum index 935cb21640..9e62e652e2 160000 --- a/go-ethereum +++ b/go-ethereum @@ -1 +1 @@ -Subproject commit 935cb216402c9693faf86d75a7fbb045109ed4a3 +Subproject commit 9e62e652e211a47ad1c71a428b4a7ea6b96ae710 From 8981880ff6d341a9961dcaa0ee4466ee872de339 Mon Sep 17 00:00:00 2001 From: Maciej Kulawik Date: Mon, 22 Apr 2024 16:47:21 +0200 Subject: [PATCH 004/113] fix koanf prefix --- cmd/conf/database.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cmd/conf/database.go b/cmd/conf/database.go index be0c630fa9..8e3759ee73 100644 --- a/cmd/conf/database.go +++ b/cmd/conf/database.go @@ -115,7 +115,7 @@ type PebbleConfig struct { WALDir string `koanf:"wal-dir"` WALMinSyncInterval int `koanf:"wal-min-sync-interval"` TargetByteDeletionRate int `koanf:"target-byte-deletion-rate"` - Experimental PebbleExperimentalConfig `koaf:"experimental"` + Experimental PebbleExperimentalConfig `koanf:"experimental"` } var PebbleConfigDefault = PebbleConfig{ @@ -144,7 +144,7 @@ func PebbleConfigAddOptions(prefix string, f *flag.FlagSet) { f.Int(prefix+".wal-bytes-per-sync", PebbleConfigDefault.WALBytesPerSync, "number of bytes to write to a write-ahead log (WAL) before calling Sync on it in the backgroud (0 = pebble default)") f.String(prefix+".wal-dir", PebbleConfigDefault.WALDir, "directory to store write-ahead logs (WALs) in. If empty, WALs will be stored in the same directory as sstables") f.Int(prefix+".wal-min-sync-interval", PebbleConfigDefault.WALMinSyncInterval, "minimum duration in microseconds between syncs of the WAL. If WAL syncs are requested faster than this interval, they will be artificially delayed.") - PebbleExperimentalConfigAddOptions(".experimental", f) + PebbleExperimentalConfigAddOptions(prefix+".experimental", f) } type PebbleExperimentalConfig struct { From 9d128ea332bbd987b739add12f6d142953595645 Mon Sep 17 00:00:00 2001 From: Maciej Kulawik Date: Tue, 23 Apr 2024 15:30:03 +0200 Subject: [PATCH 005/113] add missing koanf pebble flag --- cmd/conf/database.go | 1 + 1 file changed, 1 insertion(+) diff --git a/cmd/conf/database.go b/cmd/conf/database.go index 8e3759ee73..9264baa843 100644 --- a/cmd/conf/database.go +++ b/cmd/conf/database.go @@ -144,6 +144,7 @@ func PebbleConfigAddOptions(prefix string, f *flag.FlagSet) { f.Int(prefix+".wal-bytes-per-sync", PebbleConfigDefault.WALBytesPerSync, "number of bytes to write to a write-ahead log (WAL) before calling Sync on it in the backgroud (0 = pebble default)") f.String(prefix+".wal-dir", PebbleConfigDefault.WALDir, "directory to store write-ahead logs (WALs) in. If empty, WALs will be stored in the same directory as sstables") f.Int(prefix+".wal-min-sync-interval", PebbleConfigDefault.WALMinSyncInterval, "minimum duration in microseconds between syncs of the WAL. If WAL syncs are requested faster than this interval, they will be artificially delayed.") + f.Int(prefix+".target-byte-deletion-rate", PebbleConfigDefault.TargetByteDeletionRate, "rate (in bytes per second) at which sstable file deletions are limited to (under normal circumstances).") PebbleExperimentalConfigAddOptions(prefix+".experimental", f) } From 7be2e34314ac234149e7408663781ce160817bcf Mon Sep 17 00:00:00 2001 From: Maciej Kulawik Date: Tue, 23 Apr 2024 19:40:19 +0200 Subject: [PATCH 006/113] add pebble layers config --- cmd/conf/database.go | 19 +++++++++++++++++++ go-ethereum | 2 +- 2 files changed, 20 insertions(+), 1 deletion(-) diff --git a/cmd/conf/database.go b/cmd/conf/database.go index 9264baa843..fdf8eed565 100644 --- a/cmd/conf/database.go +++ b/cmd/conf/database.go @@ -116,6 +116,8 @@ type PebbleConfig struct { WALMinSyncInterval int `koanf:"wal-min-sync-interval"` TargetByteDeletionRate int `koanf:"target-byte-deletion-rate"` Experimental PebbleExperimentalConfig `koanf:"experimental"` + TargetFileSize int64 `koanf:"target-file-size"` + TargetFileSizeEqualLayers bool `koanf:"target-file-size-equal-layers"` } var PebbleConfigDefault = PebbleConfig{ @@ -131,6 +133,8 @@ var PebbleConfigDefault = PebbleConfig{ WALMinSyncInterval: 0, // pebble default will be used TargetByteDeletionRate: 0, // pebble default will be used Experimental: PebbleExperimentalConfigDefault, + TargetFileSize: 2 * 1024 * 1024, + TargetFileSizeEqualLayers: true, } func PebbleConfigAddOptions(prefix string, f *flag.FlagSet) { @@ -146,6 +150,8 @@ func PebbleConfigAddOptions(prefix string, f *flag.FlagSet) { f.Int(prefix+".wal-min-sync-interval", PebbleConfigDefault.WALMinSyncInterval, "minimum duration in microseconds between syncs of the WAL. If WAL syncs are requested faster than this interval, they will be artificially delayed.") f.Int(prefix+".target-byte-deletion-rate", PebbleConfigDefault.TargetByteDeletionRate, "rate (in bytes per second) at which sstable file deletions are limited to (under normal circumstances).") PebbleExperimentalConfigAddOptions(prefix+".experimental", f) + f.Int64(prefix+".target-file-size", PebbleConfigDefault.TargetFileSize, "target file size for the level 0") + f.Bool(prefix+".target-file-size-equal-layers", PebbleConfigDefault.TargetFileSizeEqualLayers, "if true same target-file-size will be uses for all layers, otherwise target size for layer n = 2 * target size for layer n - 1") } type PebbleExperimentalConfig struct { @@ -186,6 +192,18 @@ func (c *PebbleConfig) ExtraOptions() *pebble.ExtraOptions { return time.Microsecond * time.Duration(c.WALMinSyncInterval) } } + var levels []pebble.ExtraLevelOptions + if c.TargetFileSize > 0 { + if c.TargetFileSizeEqualLayers { + for i := 0; i < 7; i++ { + levels = append(levels, pebble.ExtraLevelOptions{TargetFileSize: c.TargetFileSize}) + } + } else { + for i := 0; i < 7; i++ { + levels = append(levels, pebble.ExtraLevelOptions{TargetFileSize: c.TargetFileSize << i}) + } + } + } return &pebble.ExtraOptions{ BytesPerSync: c.BytesPerSync, L0CompactionFileThreshold: c.L0CompactionFileThreshold, @@ -206,5 +224,6 @@ func (c *PebbleConfig) ExtraOptions() *pebble.ExtraOptions { MaxWriterConcurrency: c.Experimental.MaxWriterConcurrency, ForceWriterParallelism: c.Experimental.ForceWriterParallelism, }, + Levels: levels, } } diff --git a/go-ethereum b/go-ethereum index 9e62e652e2..d6428a6842 160000 --- a/go-ethereum +++ b/go-ethereum @@ -1 +1 @@ -Subproject commit 9e62e652e211a47ad1c71a428b4a7ea6b96ae710 +Subproject commit d6428a6842a8c7d39821e74662fe3e0af34babd7 From 95422f94ecfd17e2a5e3be49efe1d3fd605d51d6 Mon Sep 17 00:00:00 2001 From: Maciej Kulawik Date: Wed, 24 Apr 2024 15:15:58 +0200 Subject: [PATCH 007/113] add pebble block size and index block size options --- cmd/conf/database.go | 40 +++++++++++++++++++++++++++------------- go-ethereum | 2 +- 2 files changed, 28 insertions(+), 14 deletions(-) diff --git a/cmd/conf/database.go b/cmd/conf/database.go index fdf8eed565..bdaf8c1b73 100644 --- a/cmd/conf/database.go +++ b/cmd/conf/database.go @@ -5,6 +5,7 @@ package conf import ( "fmt" + "math" "os" "path" "path/filepath" @@ -116,8 +117,12 @@ type PebbleConfig struct { WALMinSyncInterval int `koanf:"wal-min-sync-interval"` TargetByteDeletionRate int `koanf:"target-byte-deletion-rate"` Experimental PebbleExperimentalConfig `koanf:"experimental"` - TargetFileSize int64 `koanf:"target-file-size"` - TargetFileSizeEqualLayers bool `koanf:"target-file-size-equal-layers"` + + // level specific + BlockSize int `koanf:"block-size"` + IndexBlockSize int `koanf:"index-block-size"` + TargetFileSize int64 `koanf:"target-file-size"` + TargetFileSizeEqualLevels bool `koanf:"target-file-size-equal-levels"` } var PebbleConfigDefault = PebbleConfig{ @@ -133,8 +138,10 @@ var PebbleConfigDefault = PebbleConfig{ WALMinSyncInterval: 0, // pebble default will be used TargetByteDeletionRate: 0, // pebble default will be used Experimental: PebbleExperimentalConfigDefault, + BlockSize: 4096, + IndexBlockSize: 4096, TargetFileSize: 2 * 1024 * 1024, - TargetFileSizeEqualLayers: true, + TargetFileSizeEqualLevels: true, } func PebbleConfigAddOptions(prefix string, f *flag.FlagSet) { @@ -149,9 +156,11 @@ func PebbleConfigAddOptions(prefix string, f *flag.FlagSet) { f.String(prefix+".wal-dir", PebbleConfigDefault.WALDir, "directory to store write-ahead logs (WALs) in. If empty, WALs will be stored in the same directory as sstables") f.Int(prefix+".wal-min-sync-interval", PebbleConfigDefault.WALMinSyncInterval, "minimum duration in microseconds between syncs of the WAL. If WAL syncs are requested faster than this interval, they will be artificially delayed.") f.Int(prefix+".target-byte-deletion-rate", PebbleConfigDefault.TargetByteDeletionRate, "rate (in bytes per second) at which sstable file deletions are limited to (under normal circumstances).") + f.Int(prefix+".block-size", PebbleConfigDefault.BlockSize, "target uncompressed size in bytes of each table block") + f.Int(prefix+".index-block-size", PebbleConfigDefault.IndexBlockSize, fmt.Sprintf("target uncompressed size in bytes of each index block. When the index block size is larger than this target, two-level indexes are automatically enabled. Setting this option to a large value (such as %d) disables the automatic creation of two-level indexes.", math.MaxInt32)) PebbleExperimentalConfigAddOptions(prefix+".experimental", f) f.Int64(prefix+".target-file-size", PebbleConfigDefault.TargetFileSize, "target file size for the level 0") - f.Bool(prefix+".target-file-size-equal-layers", PebbleConfigDefault.TargetFileSizeEqualLayers, "if true same target-file-size will be uses for all layers, otherwise target size for layer n = 2 * target size for layer n - 1") + f.Bool(prefix+".target-file-size-equal-levels", PebbleConfigDefault.TargetFileSizeEqualLevels, "if true same target-file-size will be uses for all levels, otherwise target size for layer n = 2 * target size for layer n - 1") } type PebbleExperimentalConfig struct { @@ -193,16 +202,16 @@ func (c *PebbleConfig) ExtraOptions() *pebble.ExtraOptions { } } var levels []pebble.ExtraLevelOptions - if c.TargetFileSize > 0 { - if c.TargetFileSizeEqualLayers { - for i := 0; i < 7; i++ { - levels = append(levels, pebble.ExtraLevelOptions{TargetFileSize: c.TargetFileSize}) - } - } else { - for i := 0; i < 7; i++ { - levels = append(levels, pebble.ExtraLevelOptions{TargetFileSize: c.TargetFileSize << i}) - } + for i := 0; i < 7; i++ { + targetFileSize := c.TargetFileSize + if !c.TargetFileSizeEqualLevels { + targetFileSize = targetFileSize << i } + levels = append(levels, pebble.ExtraLevelOptions{ + BlockSize: c.BlockSize, + IndexBlockSize: c.IndexBlockSize, + TargetFileSize: targetFileSize, + }) } return &pebble.ExtraOptions{ BytesPerSync: c.BytesPerSync, @@ -227,3 +236,8 @@ func (c *PebbleConfig) ExtraOptions() *pebble.ExtraOptions { Levels: levels, } } + +func (c *PebbleConfig) Validate() error { + // TODO + return nil +} diff --git a/go-ethereum b/go-ethereum index d6428a6842..509f1114ed 160000 --- a/go-ethereum +++ b/go-ethereum @@ -1 +1 @@ -Subproject commit d6428a6842a8c7d39821e74662fe3e0af34babd7 +Subproject commit 509f1114edd9d4e367cedfe4011ceed5766e3f07 From 69d65fedd619a44e186be83de134e6ae3681d63c Mon Sep 17 00:00:00 2001 From: Maciej Kulawik Date: Wed, 24 Apr 2024 15:38:25 +0200 Subject: [PATCH 008/113] add MemTableStopWritesThreshold pebble option --- cmd/conf/database.go | 4 ++++ go-ethereum | 2 +- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/cmd/conf/database.go b/cmd/conf/database.go index bdaf8c1b73..59a7cafd51 100644 --- a/cmd/conf/database.go +++ b/cmd/conf/database.go @@ -110,6 +110,7 @@ type PebbleConfig struct { L0CompactionThreshold int `koanf:"l0-compaction-threshold"` L0StopWritesThreshold int `koanf:"l0-stop-writes-threshold"` LBaseMaxBytes int64 `koanf:"l-base-max-bytes"` + MemTableStopWritesThreshold int `koanf:"mem-table-stop-writes-threshold"` MaxConcurrentCompactions int `koanf:"max-concurrent-compactions"` DisableAutomaticCompactions bool `koanf:"disable-automatic-compactions"` WALBytesPerSync int `koanf:"wal-bytes-per-sync"` @@ -131,6 +132,7 @@ var PebbleConfigDefault = PebbleConfig{ L0CompactionThreshold: 0, // pebble default will be used L0StopWritesThreshold: 0, // pebble default will be used LBaseMaxBytes: 0, // pebble default will be used + MemTableStopWritesThreshold: 2, MaxConcurrentCompactions: runtime.NumCPU(), DisableAutomaticCompactions: false, WALBytesPerSync: 0, // pebble default will be used @@ -150,6 +152,7 @@ func PebbleConfigAddOptions(prefix string, f *flag.FlagSet) { f.Int(prefix+".l0-compaction-threshold", PebbleConfigDefault.L0CompactionThreshold, "amount of L0 read-amplification necessary to trigger an L0 compaction (0 = pebble default)") f.Int(prefix+".l0-stop-writes-threshold", PebbleConfigDefault.L0StopWritesThreshold, "hard limit on L0 read-amplification, computed as the number of L0 sublevels. Writes are stopped when this threshold is reached (0 = pebble default)") f.Int64(prefix+".l-base-max-bytes", PebbleConfigDefault.LBaseMaxBytes, "hard limit on L0 read-amplification, computed as the number of L0 sublevels. Writes are stopped when this threshold is reached (0 = pebble default)") + f.Int(prefix+".mem-table-stop-writes-threshold", PebbleConfigDefault.MemTableStopWritesThreshold, "hard limit on the number of queued of MemTables") f.Int(prefix+".max-concurrent-compactions", PebbleConfigDefault.MaxConcurrentCompactions, "maximum number of concurrent compactions (0 = pebble default)") f.Bool(prefix+".disable-automatic-compactions", PebbleConfigDefault.DisableAutomaticCompactions, "disables automatic compactions") f.Int(prefix+".wal-bytes-per-sync", PebbleConfigDefault.WALBytesPerSync, "number of bytes to write to a write-ahead log (WAL) before calling Sync on it in the backgroud (0 = pebble default)") @@ -219,6 +222,7 @@ func (c *PebbleConfig) ExtraOptions() *pebble.ExtraOptions { L0CompactionThreshold: c.L0CompactionThreshold, L0StopWritesThreshold: c.L0StopWritesThreshold, LBaseMaxBytes: c.LBaseMaxBytes, + MemTableStopWritesThreshold: c.MemTableStopWritesThreshold, MaxConcurrentCompactions: maxConcurrentCompactions, DisableAutomaticCompactions: c.DisableAutomaticCompactions, WALBytesPerSync: c.WALBytesPerSync, diff --git a/go-ethereum b/go-ethereum index 509f1114ed..040c6f7870 160000 --- a/go-ethereum +++ b/go-ethereum @@ -1 +1 @@ -Subproject commit 509f1114edd9d4e367cedfe4011ceed5766e3f07 +Subproject commit 040c6f787056826112340ce0b4e5b8d43503f20a From c835fea2a840120adfc3459933a7118ae5219265 Mon Sep 17 00:00:00 2001 From: Maciej Kulawik Date: Wed, 24 Apr 2024 16:02:57 +0200 Subject: [PATCH 009/113] update geth --- go-ethereum | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/go-ethereum b/go-ethereum index 040c6f7870..5e8d11c191 160000 --- a/go-ethereum +++ b/go-ethereum @@ -1 +1 @@ -Subproject commit 040c6f787056826112340ce0b4e5b8d43503f20a +Subproject commit 5e8d11c191c4b88e53ca53e69b7854efe89487fd From 04b16998573bcb0ae4bf10c6dd316e7eda004000 Mon Sep 17 00:00:00 2001 From: Maciej Kulawik Date: Thu, 25 Apr 2024 00:48:59 +0200 Subject: [PATCH 010/113] update geth --- go-ethereum | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/go-ethereum b/go-ethereum index 5e8d11c191..07d08fede3 160000 --- a/go-ethereum +++ b/go-ethereum @@ -1 +1 @@ -Subproject commit 5e8d11c191c4b88e53ca53e69b7854efe89487fd +Subproject commit 07d08fede3e5e8bbfbdb3797fad08d94f8c7699a From 08ece6f85c9bf9c2ef393ab7d1cdcf5d53f7cac7 Mon Sep 17 00:00:00 2001 From: Maciej Kulawik Date: Thu, 25 Apr 2024 01:01:52 +0200 Subject: [PATCH 011/113] update geth --- go-ethereum | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/go-ethereum b/go-ethereum index 07d08fede3..31dcc54970 160000 --- a/go-ethereum +++ b/go-ethereum @@ -1 +1 @@ -Subproject commit 07d08fede3e5e8bbfbdb3797fad08d94f8c7699a +Subproject commit 31dcc54970876a09e13820a4a7334f39af38157d From 028fd31cc45f3948420ea6cd76e40251a177edd6 Mon Sep 17 00:00:00 2001 From: Maciej Kulawik Date: Thu, 25 Apr 2024 02:29:22 +0200 Subject: [PATCH 012/113] update geth --- go-ethereum | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/go-ethereum b/go-ethereum index 31dcc54970..a67aac7029 160000 --- a/go-ethereum +++ b/go-ethereum @@ -1 +1 @@ -Subproject commit 31dcc54970876a09e13820a4a7334f39af38157d +Subproject commit a67aac7029db022dd0e078783809e2fedf20de53 From 6d5343d5e2882a30b2fcae7d17f02591289c0f26 Mon Sep 17 00:00:00 2001 From: Maciej Kulawik Date: Fri, 26 Apr 2024 23:23:01 +0200 Subject: [PATCH 013/113] update pebble options descriptions --- cmd/conf/database.go | 20 ++++++++++---------- go-ethereum | 2 +- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/cmd/conf/database.go b/cmd/conf/database.go index 59a7cafd51..1c8b673dd3 100644 --- a/cmd/conf/database.go +++ b/cmd/conf/database.go @@ -147,15 +147,15 @@ var PebbleConfigDefault = PebbleConfig{ } func PebbleConfigAddOptions(prefix string, f *flag.FlagSet) { - f.Int(prefix+".bytes-per-sync", PebbleConfigDefault.BytesPerSync, "number of bytes to write to a SSTable before calling Sync on it in the background (0 = pebble default)") - f.Int(prefix+".l0-compaction-file-threshold", PebbleConfigDefault.L0CompactionFileThreshold, "count of L0 files necessary to trigger an L0 compaction (0 = pebble default)") - f.Int(prefix+".l0-compaction-threshold", PebbleConfigDefault.L0CompactionThreshold, "amount of L0 read-amplification necessary to trigger an L0 compaction (0 = pebble default)") - f.Int(prefix+".l0-stop-writes-threshold", PebbleConfigDefault.L0StopWritesThreshold, "hard limit on L0 read-amplification, computed as the number of L0 sublevels. Writes are stopped when this threshold is reached (0 = pebble default)") - f.Int64(prefix+".l-base-max-bytes", PebbleConfigDefault.LBaseMaxBytes, "hard limit on L0 read-amplification, computed as the number of L0 sublevels. Writes are stopped when this threshold is reached (0 = pebble default)") + f.Int(prefix+".bytes-per-sync", PebbleConfigDefault.BytesPerSync, "number of bytes to write to a SSTable before calling Sync on it in the background") + f.Int(prefix+".l0-compaction-file-threshold", PebbleConfigDefault.L0CompactionFileThreshold, "count of L0 files necessary to trigger an L0 compaction") + f.Int(prefix+".l0-compaction-threshold", PebbleConfigDefault.L0CompactionThreshold, "amount of L0 read-amplification necessary to trigger an L0 compaction") + f.Int(prefix+".l0-stop-writes-threshold", PebbleConfigDefault.L0StopWritesThreshold, "hard limit on L0 read-amplification, computed as the number of L0 sublevels. Writes are stopped when this threshold is reached") + f.Int64(prefix+".l-base-max-bytes", PebbleConfigDefault.LBaseMaxBytes, "The maximum number of bytes for LBase. The base level is the level which L0 is compacted into. The base level is determined dynamically based on the existing data in the LSM. The maximum number of bytes for other levels is computed dynamically based on the base level's maximum size. When the maximum number of bytes for a level is exceeded, compaction is requested.") f.Int(prefix+".mem-table-stop-writes-threshold", PebbleConfigDefault.MemTableStopWritesThreshold, "hard limit on the number of queued of MemTables") - f.Int(prefix+".max-concurrent-compactions", PebbleConfigDefault.MaxConcurrentCompactions, "maximum number of concurrent compactions (0 = pebble default)") + f.Int(prefix+".max-concurrent-compactions", PebbleConfigDefault.MaxConcurrentCompactions, "maximum number of concurrent compactions") f.Bool(prefix+".disable-automatic-compactions", PebbleConfigDefault.DisableAutomaticCompactions, "disables automatic compactions") - f.Int(prefix+".wal-bytes-per-sync", PebbleConfigDefault.WALBytesPerSync, "number of bytes to write to a write-ahead log (WAL) before calling Sync on it in the backgroud (0 = pebble default)") + f.Int(prefix+".wal-bytes-per-sync", PebbleConfigDefault.WALBytesPerSync, "number of bytes to write to a write-ahead log (WAL) before calling Sync on it in the backgroud") f.String(prefix+".wal-dir", PebbleConfigDefault.WALDir, "directory to store write-ahead logs (WALs) in. If empty, WALs will be stored in the same directory as sstables") f.Int(prefix+".wal-min-sync-interval", PebbleConfigDefault.WALMinSyncInterval, "minimum duration in microseconds between syncs of the WAL. If WAL syncs are requested faster than this interval, they will be artificially delayed.") f.Int(prefix+".target-byte-deletion-rate", PebbleConfigDefault.TargetByteDeletionRate, "rate (in bytes per second) at which sstable file deletions are limited to (under normal circumstances).") @@ -185,10 +185,10 @@ var PebbleExperimentalConfigDefault = PebbleExperimentalConfig{ } func PebbleExperimentalConfigAddOptions(prefix string, f *flag.FlagSet) { - f.Int(prefix+".l0-compaction-concurrency", PebbleExperimentalConfigDefault.L0CompactionConcurrency, "threshold of L0 read-amplification at which compaction concurrency is enabled (if compaction-debt-concurrency was not already exceeded). Every multiple of this value enables another concurrent compaction up to max-concurrent-compactions. (0 = pebble default)") - f.Uint64(prefix+".compaction-debt-concurrency", PebbleExperimentalConfigDefault.CompactionDebtConcurrency, "controls the threshold of compaction debt at which additional compaction concurrency slots are added. For every multiple of this value in compaction debt bytes, an additional concurrent compaction is added. This works \"on top\" of l0-compaction-concurrency, so the higher of the count of compaction concurrency slots as determined by the two options is chosen. (0 = pebble default)") + f.Int(prefix+".l0-compaction-concurrency", PebbleExperimentalConfigDefault.L0CompactionConcurrency, "threshold of L0 read-amplification at which compaction concurrency is enabled (if compaction-debt-concurrency was not already exceeded). Every multiple of this value enables another concurrent compaction up to max-concurrent-compactions.") + f.Uint64(prefix+".compaction-debt-concurrency", PebbleExperimentalConfigDefault.CompactionDebtConcurrency, "controls the threshold of compaction debt at which additional compaction concurrency slots are added. For every multiple of this value in compaction debt bytes, an additional concurrent compaction is added. This works \"on top\" of l0-compaction-concurrency, so the higher of the count of compaction concurrency slots as determined by the two options is chosen.") f.Int64(prefix+".read-compaction-rate", PebbleExperimentalConfigDefault.ReadCompactionRate, "controls the frequency of read triggered compactions by adjusting `AllowedSeeks` in manifest.FileMetadata: AllowedSeeks = FileSize / ReadCompactionRate") - f.Int64(prefix+".read-sampling-multiplier", PebbleExperimentalConfigDefault.ReadSamplingMultiplier, "a multiplier for the readSamplingPeriod in iterator.maybeSampleRead() to control the frequency of read sampling to trigger a read triggered compaction. A value of -1 prevents sampling and disables read triggered compactions. Geth default is -1. The pebble default is 1 << 4. which gets multiplied with a constant of 1 << 16 to yield 1 << 20 (1MB). (0 = pebble default)") + f.Int64(prefix+".read-sampling-multiplier", PebbleExperimentalConfigDefault.ReadSamplingMultiplier, "a multiplier for the readSamplingPeriod in iterator.maybeSampleRead() to control the frequency of read sampling to trigger a read triggered compaction. A value of -1 prevents sampling and disables read triggered compactions. Geth default is -1. The pebble default is 1 << 4. which gets multiplied with a constant of 1 << 16 to yield 1 << 20 (1MB).") f.Int(prefix+".max-writer-concurrency", PebbleExperimentalConfigDefault.MaxWriterConcurrency, "maximum number of compression workers the compression queue is allowed to use. If max-writer-concurrency > 0, then the Writer will use parallelism, to compress and write blocks to disk. Otherwise, the writer will compress and write blocks to disk synchronously.") f.Bool(prefix+".force-writer-parallelism", PebbleExperimentalConfigDefault.ForceWriterParallelism, "force parallelism in the sstable Writer for the metamorphic tests. Even with the MaxWriterConcurrency option set, pebble only enables parallelism in the sstable Writer if there is enough CPU available, and this option bypasses that.") } diff --git a/go-ethereum b/go-ethereum index a67aac7029..9f39f194d0 160000 --- a/go-ethereum +++ b/go-ethereum @@ -1 +1 @@ -Subproject commit a67aac7029db022dd0e078783809e2fedf20de53 +Subproject commit 9f39f194d0a5b1ab1a47b1d4f83cd112f18dc4b3 From 6a1f54d59117ef600bd140fd1b50b1b99b1ddc8d Mon Sep 17 00:00:00 2001 From: Tristan Wilson Date: Tue, 30 Apr 2024 18:44:59 +0100 Subject: [PATCH 014/113] Add option for websocket message size limit This plumbs through the websocket message size limit option for all rpc clients. --- cmd/conf/chain.go | 2 ++ util/rpcclient/rpcclient.go | 29 +++++++++++++++++------------ 2 files changed, 19 insertions(+), 12 deletions(-) diff --git a/cmd/conf/chain.go b/cmd/conf/chain.go index 531945b4d6..8ad853e7aa 100644 --- a/cmd/conf/chain.go +++ b/cmd/conf/chain.go @@ -25,6 +25,8 @@ var L1ConnectionConfigDefault = rpcclient.ClientConfig{ Timeout: time.Minute, ConnectionWait: time.Minute, ArgLogLimit: 2048, + // Use geth's unexported wsDefaultReadLimit from rpc/websocket.go + WebsocketMessageSizeLimit: 32 * 1024 * 1024, } var L1ConfigDefault = ParentChainConfig{ diff --git a/util/rpcclient/rpcclient.go b/util/rpcclient/rpcclient.go index 02b41cf15d..cc6f11c986 100644 --- a/util/rpcclient/rpcclient.go +++ b/util/rpcclient/rpcclient.go @@ -21,14 +21,15 @@ import ( ) type ClientConfig struct { - URL string `json:"url,omitempty" koanf:"url"` - JWTSecret string `json:"jwtsecret,omitempty" koanf:"jwtsecret"` - Timeout time.Duration `json:"timeout,omitempty" koanf:"timeout" reload:"hot"` - Retries uint `json:"retries,omitempty" koanf:"retries" reload:"hot"` - ConnectionWait time.Duration `json:"connection-wait,omitempty" koanf:"connection-wait"` - ArgLogLimit uint `json:"arg-log-limit,omitempty" koanf:"arg-log-limit" reload:"hot"` - RetryErrors string `json:"retry-errors,omitempty" koanf:"retry-errors" reload:"hot"` - RetryDelay time.Duration `json:"retry-delay,omitempty" koanf:"retry-delay"` + URL string `json:"url,omitempty" koanf:"url"` + JWTSecret string `json:"jwtsecret,omitempty" koanf:"jwtsecret"` + Timeout time.Duration `json:"timeout,omitempty" koanf:"timeout" reload:"hot"` + Retries uint `json:"retries,omitempty" koanf:"retries" reload:"hot"` + ConnectionWait time.Duration `json:"connection-wait,omitempty" koanf:"connection-wait"` + ArgLogLimit uint `json:"arg-log-limit,omitempty" koanf:"arg-log-limit" reload:"hot"` + RetryErrors string `json:"retry-errors,omitempty" koanf:"retry-errors" reload:"hot"` + RetryDelay time.Duration `json:"retry-delay,omitempty" koanf:"retry-delay"` + WebsocketMessageSizeLimit int64 `json:"websocket-message-size-limit,omitempty" koanf:"websocket-message-size-limit"` retryErrors *regexp.Regexp } @@ -46,8 +47,9 @@ func (c *ClientConfig) Validate() error { type ClientConfigFetcher func() *ClientConfig var TestClientConfig = ClientConfig{ - URL: "self", - JWTSecret: "", + URL: "self", + JWTSecret: "", + WebsocketMessageSizeLimit: 32 * 1024 * 1024, } var DefaultClientConfig = ClientConfig{ @@ -56,6 +58,8 @@ var DefaultClientConfig = ClientConfig{ Retries: 3, RetryErrors: "websocket: close.*|dial tcp .*|.*i/o timeout|.*connection reset by peer|.*connection refused", ArgLogLimit: 2048, + // Use geth's unexported wsDefaultReadLimit from rpc/websocket.go + WebsocketMessageSizeLimit: 32 * 1024 * 1024, } func RPCClientAddOptions(prefix string, f *flag.FlagSet, defaultConfig *ClientConfig) { @@ -67,6 +71,7 @@ func RPCClientAddOptions(prefix string, f *flag.FlagSet, defaultConfig *ClientCo f.Uint(prefix+".retries", defaultConfig.Retries, "number of retries in case of failure(0 mean one attempt)") f.String(prefix+".retry-errors", defaultConfig.RetryErrors, "Errors matching this regular expression are automatically retried") f.Duration(prefix+".retry-delay", defaultConfig.RetryDelay, "delay between retries") + f.Int64(prefix+".websocket-message-size-limit", defaultConfig.WebsocketMessageSizeLimit, "websocket message size limit used by the RPC client. 0 means no limit") } type RpcClient struct { @@ -256,9 +261,9 @@ func (c *RpcClient) Start(ctx_in context.Context) error { var err error var client *rpc.Client if jwt == nil { - client, err = rpc.DialContext(ctx, url) + client, err = rpc.DialOptions(ctx, url, rpc.WithWebsocketMessageSizeLimit(c.config().WebsocketMessageSizeLimit)) } else { - client, err = rpc.DialOptions(ctx, url, rpc.WithHTTPAuth(node.NewJWTAuth([32]byte(*jwt)))) + client, err = rpc.DialOptions(ctx, url, rpc.WithHTTPAuth(node.NewJWTAuth([32]byte(*jwt))), rpc.WithWebsocketMessageSizeLimit(c.config().WebsocketMessageSizeLimit)) } cancelCtx() if err == nil { From 3919a6e8175588c78b8ae7b8bd5f2e3e6ae84253 Mon Sep 17 00:00:00 2001 From: Maciej Kulawik Date: Thu, 2 May 2024 17:39:12 +0200 Subject: [PATCH 015/113] update geth --- go-ethereum | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/go-ethereum b/go-ethereum index 9f39f194d0..3ecb5979ae 160000 --- a/go-ethereum +++ b/go-ethereum @@ -1 +1 @@ -Subproject commit 9f39f194d0a5b1ab1a47b1d4f83cd112f18dc4b3 +Subproject commit 3ecb5979ae489902c97d7146209c35071d167be6 From 991f07d2e3e6c8d1d74368977f463b21d73dc59e Mon Sep 17 00:00:00 2001 From: Maciej Kulawik Date: Thu, 2 May 2024 18:05:05 +0200 Subject: [PATCH 016/113] update geth --- go-ethereum | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/go-ethereum b/go-ethereum index 3ecb5979ae..1aaeef7598 160000 --- a/go-ethereum +++ b/go-ethereum @@ -1 +1 @@ -Subproject commit 3ecb5979ae489902c97d7146209c35071d167be6 +Subproject commit 1aaeef75987a3d4379cf7d876cdf1526d8701884 From 39d33c7a88e01a1e5ca77f2c2ff45d06ede45498 Mon Sep 17 00:00:00 2001 From: Maciej Kulawik Date: Sat, 4 May 2024 00:27:13 +0200 Subject: [PATCH 017/113] update geth --- go-ethereum | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/go-ethereum b/go-ethereum index 1aaeef7598..ac85a19d5f 160000 --- a/go-ethereum +++ b/go-ethereum @@ -1 +1 @@ -Subproject commit 1aaeef75987a3d4379cf7d876cdf1526d8701884 +Subproject commit ac85a19d5f56231076d5bab95504d666b084fa3b From 04b9b373b6fb4e529c6a0b27d6fc847de97ee35d Mon Sep 17 00:00:00 2001 From: Ganesh Vanahalli Date: Tue, 7 May 2024 16:01:27 -0500 Subject: [PATCH 018/113] Block reexecutor should not try to reexecute genesis block --- blocks_reexecutor/blocks_reexecutor.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/blocks_reexecutor/blocks_reexecutor.go b/blocks_reexecutor/blocks_reexecutor.go index bedea37776..0ad4337e0f 100644 --- a/blocks_reexecutor/blocks_reexecutor.go +++ b/blocks_reexecutor/blocks_reexecutor.go @@ -104,7 +104,8 @@ func New(c *Config, blockchain *core.BlockChain, fatalErrChan chan error) *Block end = start + rng } // Inclusive of block reexecution [start, end] - if start > 0 { + // Do not reexecute genesis block i,e chainStart + if start > 0 && start != chainStart { start-- } // Divide work equally among available threads From 836459d68a8021accf285e6e9a503e5266140209 Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Thu, 9 May 2024 16:20:53 -0600 Subject: [PATCH 019/113] init: open database with separate wasm dir --- cmd/nitro/init.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cmd/nitro/init.go b/cmd/nitro/init.go index a45ec054a1..750bf03516 100644 --- a/cmd/nitro/init.go +++ b/cmd/nitro/init.go @@ -172,13 +172,13 @@ func validateBlockChain(blockChain *core.BlockChain, chainConfig *params.ChainCo func openInitializeChainDb(ctx context.Context, stack *node.Node, config *NodeConfig, chainId *big.Int, cacheConfig *core.CacheConfig, l1Client arbutil.L1Interface, rollupAddrs chaininfo.RollupAddresses) (ethdb.Database, *core.BlockChain, error) { if !config.Init.Force { - if readOnlyDb, err := stack.OpenDatabaseWithFreezer("l2chaindata", 0, 0, "", "l2chaindata/", true); err == nil { + if readOnlyDb, err := stack.OpenDatabaseWithFreezerAndWasm("l2chaindata", "wasm", 0, 0, "", "l2chaindata/", true); err == nil { if chainConfig := gethexec.TryReadStoredChainConfig(readOnlyDb); chainConfig != nil { readOnlyDb.Close() if !arbmath.BigEquals(chainConfig.ChainID, chainId) { return nil, nil, fmt.Errorf("database has chain ID %v but config has chain ID %v (are you sure this database is for the right chain?)", chainConfig.ChainID, chainId) } - chainDb, err := stack.OpenDatabaseWithFreezer("l2chaindata", config.Execution.Caching.DatabaseCache, config.Persistent.Handles, config.Persistent.Ancient, "l2chaindata/", false) + chainDb, err := stack.OpenDatabaseWithFreezerAndWasm("l2chaindata", "wasm", config.Execution.Caching.DatabaseCache, config.Persistent.Handles, config.Persistent.Ancient, "l2chaindata/", false) if err != nil { return chainDb, nil, err } @@ -230,7 +230,7 @@ func openInitializeChainDb(ctx context.Context, stack *node.Node, config *NodeCo var initDataReader statetransfer.InitDataReader = nil - chainDb, err := stack.OpenDatabaseWithFreezer("l2chaindata", config.Execution.Caching.DatabaseCache, config.Persistent.Handles, config.Persistent.Ancient, "l2chaindata/", false) + chainDb, err := stack.OpenDatabaseWithFreezerAndWasm("l2chaindata", "wasm", config.Execution.Caching.DatabaseCache, config.Persistent.Handles, config.Persistent.Ancient, "l2chaindata/", false) if err != nil { return chainDb, nil, err } From e03a347e055fe1995cf97621e0e57a91675ca42a Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Thu, 9 May 2024 16:22:05 -0600 Subject: [PATCH 020/113] program: recreate activated stylus entry if needed --- arbos/programs/native.go | 72 ++++++++++++++++++++++++++++++++++---- arbos/programs/programs.go | 8 ++++- arbos/programs/wasm.go | 5 +++ 3 files changed, 78 insertions(+), 7 deletions(-) diff --git a/arbos/programs/native.go b/arbos/programs/native.go index 123dda54ce..2fe4822fb6 100644 --- a/arbos/programs/native.go +++ b/arbos/programs/native.go @@ -24,6 +24,7 @@ import ( "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/core" + "github.com/ethereum/go-ethereum/core/rawdb" "github.com/ethereum/go-ethereum/core/state" "github.com/ethereum/go-ethereum/core/vm" "github.com/ethereum/go-ethereum/log" @@ -53,6 +54,24 @@ func activateProgram( debug bool, burner burn.Burner, ) (*activationInfo, error) { + info, asm, module, err := activateProgramInternal(db, program, codehash, wasm, page_limit, version, debug, burner) + if err != nil { + return nil, err + } + db.ActivateWasm(info.moduleHash, asm, module) + return info, nil +} + +func activateProgramInternal( + db vm.StateDB, + program common.Address, + codehash common.Hash, + wasm []byte, + page_limit uint16, + version uint16, + debug bool, + burner burn.Burner, +) (*activationInfo, []byte, []byte, error) { output := &rustBytes{} asmLen := usize(0) moduleHash := &bytes32{} @@ -78,9 +97,9 @@ func activateProgram( log.Warn("activation failed", "err", err, "msg", msg, "program", program) } if errors.Is(err, vm.ErrExecutionReverted) { - return nil, fmt.Errorf("%w: %s", ErrProgramActivation, msg) + return nil, nil, nil, fmt.Errorf("%w: %s", ErrProgramActivation, msg) } - return nil, err + return nil, nil, nil, err } hash := moduleHash.toHash() @@ -95,13 +114,55 @@ func activateProgram( asmEstimate: uint32(stylusData.asm_estimate), footprint: uint16(stylusData.footprint), } - db.ActivateWasm(hash, asm, module) - return info, err + return info, asm, module, err +} + +func getLocalAsm(statedb vm.StateDB, moduleHash common.Hash, address common.Address, pagelimit uint16, time uint64, debugMode bool, program Program) ([]byte, error) { + localAsm, err := statedb.TryGetActivatedAsm(moduleHash) + if err == nil || len(localAsm) > 0 { + return localAsm, nil + } + + codeHash := statedb.GetCodeHash(address) + burner := burn.NewSystemBurner(nil, false) + + wasm, err := getWasm(statedb, address) + if err != nil { + return nil, err + } + + // we know program is activated, so it must be in correct version and not use too much memory + info, asm, module, err := activateProgramInternal(statedb, address, codeHash, wasm, pagelimit, program.version, debugMode, burner) + + if err != nil { + return nil, err + } + + if info.moduleHash != moduleHash { + return nil, errors.New("failed to re-activate program not found in database") + } + + currentHoursSince := hoursSinceArbitrum(time) + if currentHoursSince > program.activatedAt { + // stylus program is active on-chain, and was activated in the past + // so we store it directly to database + batch := statedb.Database().WasmStore().NewBatch() + rawdb.WriteActivation(batch, moduleHash, asm, module) + if err := batch.Write(); err != nil { + log.Error("failed writing re-activation to state", "address", address, "err", err) + } + } else { + // program activated recently, possibly in this eth_call + // store it to statedb. It will be stored to database if statedb is commited + statedb.ActivateWasm(info.moduleHash, asm, module) + } + return asm, nil } func callProgram( address common.Address, moduleHash common.Hash, + localAsm []byte, scope *vm.ScopeContext, interpreter *vm.EVMInterpreter, tracingInfo *util.TracingInfo, @@ -111,7 +172,6 @@ func callProgram( memoryModel *MemoryModel, ) ([]byte, error) { db := interpreter.Evm().StateDB - asm := db.GetActivatedAsm(moduleHash) debug := stylusParams.debugMode if db, ok := db.(*state.StateDB); ok { @@ -123,7 +183,7 @@ func callProgram( output := &rustBytes{} status := userStatus(C.stylus_call( - goSlice(asm), + goSlice(localAsm), goSlice(calldata), stylusParams.encode(), evmApi.cNative, diff --git a/arbos/programs/programs.go b/arbos/programs/programs.go index 779f2d6c67..b277f5d678 100644 --- a/arbos/programs/programs.go +++ b/arbos/programs/programs.go @@ -205,6 +205,12 @@ func (p Programs) CallProgram( statedb.AddStylusPages(program.footprint) defer statedb.SetStylusPagesOpen(open) + localAsm, err := getLocalAsm(statedb, moduleHash, contract.Address(), params.PageLimit, evm.Context.Time, debugMode, program) + if err != nil { + log.Crit("failed to get local wasm for activated program", "program", contract.Address()) + return nil, err + } + evmData := &evmData{ blockBasefee: common.BigToHash(evm.Context.BaseFee), chainId: evm.ChainConfig().ChainID.Uint64(), @@ -227,7 +233,7 @@ func (p Programs) CallProgram( if contract.CodeAddr != nil { address = *contract.CodeAddr } - return callProgram(address, moduleHash, scope, interpreter, tracingInfo, calldata, evmData, goParams, model) + return callProgram(address, moduleHash, localAsm, scope, interpreter, tracingInfo, calldata, evmData, goParams, model) } func getWasm(statedb vm.StateDB, program common.Address) ([]byte, error) { diff --git a/arbos/programs/wasm.go b/arbos/programs/wasm.go index 77eb7e0f2f..105516dc62 100644 --- a/arbos/programs/wasm.go +++ b/arbos/programs/wasm.go @@ -128,9 +128,14 @@ func startProgram(module uint32) uint32 //go:wasmimport programs send_response func sendResponse(req_id uint32) uint32 +func getLocalAsm(statedb vm.StateDB, moduleHash common.Hash, address common.Address, pagelimit uint16, debugMode bool, program Program) ([]byte, error) { + return nil, nil +} + func callProgram( address common.Address, moduleHash common.Hash, + _localAsm []byte, scope *vm.ScopeContext, interpreter *vm.EVMInterpreter, tracingInfo *util.TracingInfo, From dc706bf5f48307951a5172205d6fe058dd5f9db4 Mon Sep 17 00:00:00 2001 From: Tristan Wilson Date: Thu, 9 May 2024 17:21:29 -0700 Subject: [PATCH 021/113] Increase default websocket size limit --- cmd/conf/chain.go | 13 ++++++------- util/rpcclient/rpcclient.go | 15 +++++++-------- 2 files changed, 13 insertions(+), 15 deletions(-) diff --git a/cmd/conf/chain.go b/cmd/conf/chain.go index 8ad853e7aa..ab9a713287 100644 --- a/cmd/conf/chain.go +++ b/cmd/conf/chain.go @@ -20,13 +20,12 @@ type ParentChainConfig struct { } var L1ConnectionConfigDefault = rpcclient.ClientConfig{ - URL: "", - Retries: 2, - Timeout: time.Minute, - ConnectionWait: time.Minute, - ArgLogLimit: 2048, - // Use geth's unexported wsDefaultReadLimit from rpc/websocket.go - WebsocketMessageSizeLimit: 32 * 1024 * 1024, + URL: "", + Retries: 2, + Timeout: time.Minute, + ConnectionWait: time.Minute, + ArgLogLimit: 2048, + WebsocketMessageSizeLimit: 256 * 1024 * 1024, } var L1ConfigDefault = ParentChainConfig{ diff --git a/util/rpcclient/rpcclient.go b/util/rpcclient/rpcclient.go index cc6f11c986..56aebef396 100644 --- a/util/rpcclient/rpcclient.go +++ b/util/rpcclient/rpcclient.go @@ -49,17 +49,16 @@ type ClientConfigFetcher func() *ClientConfig var TestClientConfig = ClientConfig{ URL: "self", JWTSecret: "", - WebsocketMessageSizeLimit: 32 * 1024 * 1024, + WebsocketMessageSizeLimit: 256 * 1024 * 1024, } var DefaultClientConfig = ClientConfig{ - URL: "self-auth", - JWTSecret: "", - Retries: 3, - RetryErrors: "websocket: close.*|dial tcp .*|.*i/o timeout|.*connection reset by peer|.*connection refused", - ArgLogLimit: 2048, - // Use geth's unexported wsDefaultReadLimit from rpc/websocket.go - WebsocketMessageSizeLimit: 32 * 1024 * 1024, + URL: "self-auth", + JWTSecret: "", + Retries: 3, + RetryErrors: "websocket: close.*|dial tcp .*|.*i/o timeout|.*connection reset by peer|.*connection refused", + ArgLogLimit: 2048, + WebsocketMessageSizeLimit: 256 * 1024 * 1024, } func RPCClientAddOptions(prefix string, f *flag.FlagSet, defaultConfig *ClientConfig) { From a5b9853583e4947d0c1aed5a49778b8718b1dffc Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Thu, 9 May 2024 19:10:13 -0600 Subject: [PATCH 022/113] geth - support separate wasm --- go-ethereum | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/go-ethereum b/go-ethereum index 72f81daa8c..37b6489382 160000 --- a/go-ethereum +++ b/go-ethereum @@ -1 +1 @@ -Subproject commit 72f81daa8c59f044246b6e1f3eca08187edd7417 +Subproject commit 37b6489382bb884dd1216dcb0f6a224ce2ca5fe2 From c9d19bcdc0078dfe7321256bf35fb5085429c251 Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Thu, 9 May 2024 19:32:32 -0600 Subject: [PATCH 023/113] wasm split: fix native --- arbos/programs/native.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/arbos/programs/native.go b/arbos/programs/native.go index 2fe4822fb6..b657472371 100644 --- a/arbos/programs/native.go +++ b/arbos/programs/native.go @@ -54,7 +54,7 @@ func activateProgram( debug bool, burner burn.Burner, ) (*activationInfo, error) { - info, asm, module, err := activateProgramInternal(db, program, codehash, wasm, page_limit, version, debug, burner) + info, asm, module, err := activateProgramInternal(db, program, codehash, wasm, page_limit, version, debug, burner.GasLeft()) if err != nil { return nil, err } @@ -70,7 +70,7 @@ func activateProgramInternal( page_limit uint16, version uint16, debug bool, - burner burn.Burner, + gasLeft *uint64, ) (*activationInfo, []byte, []byte, error) { output := &rustBytes{} asmLen := usize(0) @@ -88,7 +88,7 @@ func activateProgramInternal( &codeHash, moduleHash, stylusData, - (*u64)(burner.GasLeft()), + (*u64)(gasLeft), )) data, msg, err := status.toResult(output.intoBytes(), debug) @@ -124,15 +124,15 @@ func getLocalAsm(statedb vm.StateDB, moduleHash common.Hash, address common.Addr } codeHash := statedb.GetCodeHash(address) - burner := burn.NewSystemBurner(nil, false) wasm, err := getWasm(statedb, address) if err != nil { return nil, err } + unlimitedGas := uint64(0xffffffffffff) // we know program is activated, so it must be in correct version and not use too much memory - info, asm, module, err := activateProgramInternal(statedb, address, codeHash, wasm, pagelimit, program.version, debugMode, burner) + info, asm, module, err := activateProgramInternal(statedb, address, codeHash, wasm, pagelimit, program.version, debugMode, &unlimitedGas) if err != nil { return nil, err From 393d1d01db82bbbc660b34a222d9c73a18dd51a4 Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Thu, 9 May 2024 19:32:56 -0600 Subject: [PATCH 024/113] test wasm split --- system_tests/common_test.go | 4 +- system_tests/program_test.go | 78 ++++++++++++++++++++++++++++++++++++ 2 files changed, 80 insertions(+), 2 deletions(-) diff --git a/system_tests/common_test.go b/system_tests/common_test.go index a9f655ff77..9d461bd48c 100644 --- a/system_tests/common_test.go +++ b/system_tests/common_test.go @@ -772,7 +772,7 @@ func createL2BlockChainWithStackConfig( stack, err = node.New(stackConfig) Require(t, err) - chainDb, err := stack.OpenDatabase("l2chaindata", 0, 0, "l2chaindata/", false) + chainDb, err := stack.OpenDatabaseWithFreezerAndWasm("l2chaindata", "wasm", 0, 0, "ancient", "l2chaindata/", false) Require(t, err) arbDb, err := stack.OpenDatabase("arbitrumdata", 0, 0, "arbitrumdata/", false) Require(t, err) @@ -976,7 +976,7 @@ func Create2ndNodeWithConfig( l2stack, err := node.New(stackConfig) Require(t, err) - l2chainDb, err := l2stack.OpenDatabase("l2chaindata", 0, 0, "l2chaindata/", false) + l2chainDb, err := l2stack.OpenDatabaseWithFreezerAndWasm("l2chaindata", "wasm", 0, 0, "", "l2chaindata/", false) Require(t, err) l2arbDb, err := l2stack.OpenDatabase("arbitrumdata", 0, 0, "arbitrumdata/", false) Require(t, err) diff --git a/system_tests/program_test.go b/system_tests/program_test.go index b20efe0740..079b6c0818 100644 --- a/system_tests/program_test.go +++ b/system_tests/program_test.go @@ -1458,3 +1458,81 @@ func formatTime(duration time.Duration) string { } return fmt.Sprintf("%.2f%s", span, units[unit]) } + +func TestWasmRecreate(t *testing.T) { + builder, auth, cleanup := setupProgramTest(t, true) + ctx := builder.ctx + l2info := builder.L2Info + l2client := builder.L2.Client + defer cleanup() + + storage := deployWasm(t, ctx, auth, l2client, rustFile("storage")) + + zero := common.Hash{} + val := common.HexToHash("0x121233445566") + + // do an onchain call - store value + storeTx := l2info.PrepareTxTo("Owner", &storage, l2info.TransferGas, nil, argsForStorageWrite(zero, val)) + Require(t, l2client.SendTransaction(ctx, storeTx)) + _, err := EnsureTxSucceeded(ctx, l2client, storeTx) + Require(t, err) + + testDir := t.TempDir() + nodeBStack := createStackConfigForTest(testDir) + nodeB, cleanupB := builder.Build2ndNode(t, &SecondNodeParams{stackConfig: nodeBStack}) + + _, err = EnsureTxSucceeded(ctx, nodeB.Client, storeTx) + Require(t, err) + + // make sure reading 2nd value succeeds from 2nd node + loadTx := l2info.PrepareTxTo("Owner", &storage, l2info.TransferGas, nil, argsForStorageRead(zero)) + result, err := arbutil.SendTxAsCall(ctx, nodeB.Client, loadTx, l2info.GetAddress("Owner"), nil, true) + Require(t, err) + if common.BytesToHash(result) != val { + Fatal(t, "got wrong value") + } + // close nodeB + cleanupB() + + // delete wasm dir of nodeB + + wasmPath := filepath.Join(testDir, "system_tests.test", "wasm") + dirContents, err := os.ReadDir(wasmPath) + Require(t, err) + if len(dirContents) == 0 { + Fatal(t, "not contents found before delete") + } + os.RemoveAll(wasmPath) + + // recreate nodeB - using same source dir (wasm deleted) + nodeB, cleanupB = builder.Build2ndNode(t, &SecondNodeParams{stackConfig: nodeBStack}) + + // test nodeB - sees existing transaction + _, err = EnsureTxSucceeded(ctx, nodeB.Client, storeTx) + Require(t, err) + + // test nodeB - answers eth_call (requires reloading wasm) + result, err = arbutil.SendTxAsCall(ctx, nodeB.Client, loadTx, l2info.GetAddress("Owner"), nil, true) + Require(t, err) + if common.BytesToHash(result) != val { + Fatal(t, "got wrong value") + } + + // send new tx (requires wasm) and check nodeB sees it as well + Require(t, l2client.SendTransaction(ctx, loadTx)) + + _, err = EnsureTxSucceeded(ctx, l2client, loadTx) + Require(t, err) + + _, err = EnsureTxSucceeded(ctx, nodeB.Client, loadTx) + Require(t, err) + + cleanupB() + dirContents, err = os.ReadDir(wasmPath) + Require(t, err) + if len(dirContents) == 0 { + Fatal(t, "not contents found before delete") + } + os.RemoveAll(wasmPath) + +} From d56f862d7a9038048c335caa502dbb5d12422805 Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Thu, 9 May 2024 19:45:47 -0600 Subject: [PATCH 025/113] wasm split: fix wasm mplementation --- arbos/programs/wasm.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/arbos/programs/wasm.go b/arbos/programs/wasm.go index 105516dc62..8a67babc16 100644 --- a/arbos/programs/wasm.go +++ b/arbos/programs/wasm.go @@ -128,7 +128,7 @@ func startProgram(module uint32) uint32 //go:wasmimport programs send_response func sendResponse(req_id uint32) uint32 -func getLocalAsm(statedb vm.StateDB, moduleHash common.Hash, address common.Address, pagelimit uint16, debugMode bool, program Program) ([]byte, error) { +func getLocalAsm(statedb vm.StateDB, moduleHash common.Hash, address common.Address, pagelimit uint16, time uint64, debugMode bool, program Program) ([]byte, error) { return nil, nil } From 3ab3acffc33747955ece2ec6fb721b507001fbea Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Fri, 10 May 2024 08:47:24 -0600 Subject: [PATCH 026/113] wasm split: getLocalAsm fixes --- arbos/programs/native.go | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/arbos/programs/native.go b/arbos/programs/native.go index b657472371..e1b464b39f 100644 --- a/arbos/programs/native.go +++ b/arbos/programs/native.go @@ -119,7 +119,7 @@ func activateProgramInternal( func getLocalAsm(statedb vm.StateDB, moduleHash common.Hash, address common.Address, pagelimit uint16, time uint64, debugMode bool, program Program) ([]byte, error) { localAsm, err := statedb.TryGetActivatedAsm(moduleHash) - if err == nil || len(localAsm) > 0 { + if err == nil && len(localAsm) > 0 { return localAsm, nil } @@ -127,19 +127,21 @@ func getLocalAsm(statedb vm.StateDB, moduleHash common.Hash, address common.Addr wasm, err := getWasm(statedb, address) if err != nil { - return nil, err + log.Error("Failed to reactivate program: getWasm", "address", address, "expected moduleHash", moduleHash, "err", err) + return nil, fmt.Errorf("failed to reactivate program address: %v err: %w", address, err) } unlimitedGas := uint64(0xffffffffffff) // we know program is activated, so it must be in correct version and not use too much memory info, asm, module, err := activateProgramInternal(statedb, address, codeHash, wasm, pagelimit, program.version, debugMode, &unlimitedGas) - if err != nil { - return nil, err + log.Error("failed to reactivate program", "address", address, "expected moduleHash", moduleHash, "err", err) + return nil, fmt.Errorf("failed to reactivate program address: %v err: %w", address, err) } if info.moduleHash != moduleHash { - return nil, errors.New("failed to re-activate program not found in database") + log.Error("failed to reactivate program", "address", address, "expected moduleHash", moduleHash, "got", info.moduleHash) + return nil, fmt.Errorf("failed to reactivate program. address: %v, expected ModuleHash: %v", address, moduleHash) } currentHoursSince := hoursSinceArbitrum(time) From 8d903c5997dec617961212271ca20bd9f2578603 Mon Sep 17 00:00:00 2001 From: Diego Ximenes Date: Thu, 2 May 2024 15:34:55 -0300 Subject: [PATCH 027/113] mv MessageWithMetadataAndBlockHash to arbostypes --- arbnode/transaction_streamer.go | 14 +++++++------- arbos/arbostypes/messagewithmeta.go | 5 +++++ broadcaster/broadcaster.go | 7 +------ 3 files changed, 13 insertions(+), 13 deletions(-) diff --git a/arbnode/transaction_streamer.go b/arbnode/transaction_streamer.go index 0d5ae829b0..b8b35186b2 100644 --- a/arbnode/transaction_streamer.go +++ b/arbnode/transaction_streamer.go @@ -465,9 +465,9 @@ func (s *TransactionStreamer) reorg(batch ethdb.Batch, count arbutil.MessageInde return err } - messagesWithBlockHash := make([]broadcaster.MessageWithMetadataAndBlockHash, 0, len(messagesResults)) + messagesWithBlockHash := make([]arbostypes.MessageWithMetadataAndBlockHash, 0, len(messagesResults)) for i := 0; i < len(messagesResults); i++ { - messagesWithBlockHash = append(messagesWithBlockHash, broadcaster.MessageWithMetadataAndBlockHash{ + messagesWithBlockHash = append(messagesWithBlockHash, arbostypes.MessageWithMetadataAndBlockHash{ Message: newMessages[i], BlockHash: &messagesResults[i].BlockHash, }) @@ -1011,11 +1011,11 @@ func (s *TransactionStreamer) WriteMessageFromSequencer( return err } - msgWithBlockHash := broadcaster.MessageWithMetadataAndBlockHash{ + msgWithBlockHash := arbostypes.MessageWithMetadataAndBlockHash{ Message: msgWithMeta, BlockHash: &msgResult.BlockHash, } - s.broadcastMessages([]broadcaster.MessageWithMetadataAndBlockHash{msgWithBlockHash}, pos) + s.broadcastMessages([]arbostypes.MessageWithMetadataAndBlockHash{msgWithBlockHash}, pos) return nil } @@ -1046,7 +1046,7 @@ func (s *TransactionStreamer) writeMessage(pos arbutil.MessageIndex, msg arbosty } func (s *TransactionStreamer) broadcastMessages( - msgs []broadcaster.MessageWithMetadataAndBlockHash, + msgs []arbostypes.MessageWithMetadataAndBlockHash, pos arbutil.MessageIndex, ) { if s.broadcastServer == nil { @@ -1145,11 +1145,11 @@ func (s *TransactionStreamer) ExecuteNextMsg(ctx context.Context, exec execution return false } - msgWithBlockHash := broadcaster.MessageWithMetadataAndBlockHash{ + msgWithBlockHash := arbostypes.MessageWithMetadataAndBlockHash{ Message: *msg, BlockHash: &msgResult.BlockHash, } - s.broadcastMessages([]broadcaster.MessageWithMetadataAndBlockHash{msgWithBlockHash}, pos) + s.broadcastMessages([]arbostypes.MessageWithMetadataAndBlockHash{msgWithBlockHash}, pos) return pos+1 < msgCount } diff --git a/arbos/arbostypes/messagewithmeta.go b/arbos/arbostypes/messagewithmeta.go index a3d4f5e3c3..e1215e0dd5 100644 --- a/arbos/arbostypes/messagewithmeta.go +++ b/arbos/arbostypes/messagewithmeta.go @@ -18,6 +18,11 @@ type MessageWithMetadata struct { DelayedMessagesRead uint64 `json:"delayedMessagesRead"` } +type MessageWithMetadataAndBlockHash struct { + Message MessageWithMetadata + BlockHash *common.Hash +} + var EmptyTestMessageWithMetadata = MessageWithMetadata{ Message: &EmptyTestIncomingMessage, } diff --git a/broadcaster/broadcaster.go b/broadcaster/broadcaster.go index ac5c6c39da..da1de6665e 100644 --- a/broadcaster/broadcaster.go +++ b/broadcaster/broadcaster.go @@ -22,11 +22,6 @@ import ( "github.com/offchainlabs/nitro/wsbroadcastserver" ) -type MessageWithMetadataAndBlockHash struct { - Message arbostypes.MessageWithMetadata - BlockHash *common.Hash -} - type Broadcaster struct { server *wsbroadcastserver.WSBroadcastServer backlog backlog.Backlog @@ -98,7 +93,7 @@ func (b *Broadcaster) BroadcastSingleFeedMessage(bfm *m.BroadcastFeedMessage) { } func (b *Broadcaster) BroadcastMessages( - messagesWithBlockHash []MessageWithMetadataAndBlockHash, + messagesWithBlockHash []arbostypes.MessageWithMetadataAndBlockHash, seq arbutil.MessageIndex, ) (err error) { defer func() { From 1f99ca984c2c3ac9c6f0af11680fe3ee756a7746 Mon Sep 17 00:00:00 2001 From: Diego Ximenes Date: Thu, 2 May 2024 17:09:03 -0300 Subject: [PATCH 028/113] pass MessageWithMetadataAndBlockHashes to writeMessages --- arbnode/inbox_test.go | 20 ++++--- arbnode/inbox_tracker.go | 9 ++- arbnode/seq_coordinator.go | 7 ++- arbnode/transaction_streamer.go | 77 ++++++++++++++----------- execution/gethexec/executionengine.go | 6 +- execution/gethexec/node.go | 2 +- execution/interface.go | 2 +- system_tests/contract_tx_test.go | 26 +++++---- system_tests/reorg_resequencing_test.go | 8 ++- system_tests/seq_coordinator_test.go | 5 +- 10 files changed, 92 insertions(+), 70 deletions(-) diff --git a/arbnode/inbox_test.go b/arbnode/inbox_test.go index 5c879743a4..fbd1dba96a 100644 --- a/arbnode/inbox_test.go +++ b/arbnode/inbox_test.go @@ -128,7 +128,7 @@ func TestTransactionStreamer(t *testing.T) { } state.balances = newBalances - var messages []arbostypes.MessageWithMetadata + var messages []arbostypes.MessageWithMetadataAndBlockHash // TODO replay a random amount of messages too numMessages := rand.Int() % 5 for j := 0; j < numMessages; j++ { @@ -154,16 +154,18 @@ func TestTransactionStreamer(t *testing.T) { l2Message = append(l2Message, arbmath.U256Bytes(value)...) var requestId common.Hash binary.BigEndian.PutUint64(requestId.Bytes()[:8], uint64(i)) - messages = append(messages, arbostypes.MessageWithMetadata{ - Message: &arbostypes.L1IncomingMessage{ - Header: &arbostypes.L1IncomingMessageHeader{ - Kind: arbostypes.L1MessageType_L2Message, - Poster: source, - RequestId: &requestId, + messages = append(messages, arbostypes.MessageWithMetadataAndBlockHash{ + Message: arbostypes.MessageWithMetadata{ + Message: &arbostypes.L1IncomingMessage{ + Header: &arbostypes.L1IncomingMessageHeader{ + Kind: arbostypes.L1MessageType_L2Message, + Poster: source, + RequestId: &requestId, + }, + L2msg: l2Message, }, - L2msg: l2Message, + DelayedMessagesRead: 1, }, - DelayedMessagesRead: 1, }) state.balances[source].Sub(state.balances[source], value) if state.balances[dest] == nil { diff --git a/arbnode/inbox_tracker.go b/arbnode/inbox_tracker.go index ba1b875ec8..e2aa1d5e74 100644 --- a/arbnode/inbox_tracker.go +++ b/arbnode/inbox_tracker.go @@ -652,7 +652,7 @@ func (t *InboxTracker) AddSequencerBatches(ctx context.Context, client arbutil.L pos++ } - var messages []arbostypes.MessageWithMetadata + var messages []arbostypes.MessageWithMetadataAndBlockHash backend := &multiplexerBackend{ batchSeqNum: batches[0].SequenceNumber, batches: batches, @@ -673,7 +673,10 @@ func (t *InboxTracker) AddSequencerBatches(ctx context.Context, client arbutil.L if err != nil { return err } - messages = append(messages, *msg) + msgWithBlockHash := arbostypes.MessageWithMetadataAndBlockHash{ + Message: *msg, + } + messages = append(messages, msgWithBlockHash) batchMessageCounts[batchSeqNum] = currentpos currentpos += 1 } @@ -733,7 +736,7 @@ func (t *InboxTracker) AddSequencerBatches(ctx context.Context, client arbutil.L } var latestTimestamp uint64 if len(messages) > 0 { - latestTimestamp = messages[len(messages)-1].Message.Header.Timestamp + latestTimestamp = messages[len(messages)-1].Message.Message.Header.Timestamp } log.Info( "InboxTracker", diff --git a/arbnode/seq_coordinator.go b/arbnode/seq_coordinator.go index ecf38ddf42..0a27d89d40 100644 --- a/arbnode/seq_coordinator.go +++ b/arbnode/seq_coordinator.go @@ -533,7 +533,7 @@ func (c *SeqCoordinator) update(ctx context.Context) time.Duration { if readUntil > localMsgCount+c.config.MsgPerPoll { readUntil = localMsgCount + c.config.MsgPerPoll } - var messages []arbostypes.MessageWithMetadata + var messages []arbostypes.MessageWithMetadataAndBlockHash msgToRead := localMsgCount var msgReadErr error for msgToRead < readUntil { @@ -592,7 +592,10 @@ func (c *SeqCoordinator) update(ctx context.Context) time.Duration { DelayedMessagesRead: lastDelayedMsg, } } - messages = append(messages, message) + msgWithBlockHash := arbostypes.MessageWithMetadataAndBlockHash{ + Message: message, + } + messages = append(messages, msgWithBlockHash) msgToRead++ } if len(messages) > 0 { diff --git a/arbnode/transaction_streamer.go b/arbnode/transaction_streamer.go index b8b35186b2..411eba965d 100644 --- a/arbnode/transaction_streamer.go +++ b/arbnode/transaction_streamer.go @@ -60,7 +60,7 @@ type TransactionStreamer struct { nextAllowedFeedReorgLog time.Time - broadcasterQueuedMessages []arbostypes.MessageWithMetadata + broadcasterQueuedMessages []arbostypes.MessageWithMetadataAndBlockHash broadcasterQueuedMessagesPos uint64 broadcasterQueuedMessagesActiveReorg bool @@ -371,7 +371,7 @@ func deleteFromRange(ctx context.Context, db ethdb.Database, prefix []byte, star // The insertion mutex must be held. This acquires the reorg mutex. // Note: oldMessages will be empty if reorgHook is nil -func (s *TransactionStreamer) reorg(batch ethdb.Batch, count arbutil.MessageIndex, newMessages []arbostypes.MessageWithMetadata) error { +func (s *TransactionStreamer) reorg(batch ethdb.Batch, count arbutil.MessageIndex, newMessages []arbostypes.MessageWithMetadataAndBlockHash) error { if count == 0 { return errors.New("cannot reorg out init message") } @@ -465,14 +465,14 @@ func (s *TransactionStreamer) reorg(batch ethdb.Batch, count arbutil.MessageInde return err } - messagesWithBlockHash := make([]arbostypes.MessageWithMetadataAndBlockHash, 0, len(messagesResults)) + messagesWithComputedBlockHash := make([]arbostypes.MessageWithMetadataAndBlockHash, 0, len(messagesResults)) for i := 0; i < len(messagesResults); i++ { - messagesWithBlockHash = append(messagesWithBlockHash, arbostypes.MessageWithMetadataAndBlockHash{ - Message: newMessages[i], + messagesWithComputedBlockHash = append(messagesWithComputedBlockHash, arbostypes.MessageWithMetadataAndBlockHash{ + Message: newMessages[i].Message, BlockHash: &messagesResults[i].BlockHash, }) } - s.broadcastMessages(messagesWithBlockHash, count) + s.broadcastMessages(messagesWithComputedBlockHash, count) if s.validator != nil { err = s.validator.Reorg(s.GetContext(), count) @@ -555,7 +555,7 @@ func (s *TransactionStreamer) GetProcessedMessageCount() (arbutil.MessageIndex, return msgCount, nil } -func (s *TransactionStreamer) AddMessages(pos arbutil.MessageIndex, messagesAreConfirmed bool, messages []arbostypes.MessageWithMetadata) error { +func (s *TransactionStreamer) AddMessages(pos arbutil.MessageIndex, messagesAreConfirmed bool, messages []arbostypes.MessageWithMetadataAndBlockHash) error { return s.AddMessagesAndEndBatch(pos, messagesAreConfirmed, messages, nil) } @@ -579,7 +579,7 @@ func (s *TransactionStreamer) AddBroadcastMessages(feedMessages []*m.BroadcastFe return nil } broadcastStartPos := feedMessages[0].SequenceNumber - var messages []arbostypes.MessageWithMetadata + var messages []arbostypes.MessageWithMetadataAndBlockHash broadcastAfterPos := broadcastStartPos for _, feedMessage := range feedMessages { if broadcastAfterPos != feedMessage.SequenceNumber { @@ -588,7 +588,11 @@ func (s *TransactionStreamer) AddBroadcastMessages(feedMessages []*m.BroadcastFe if feedMessage.Message.Message == nil || feedMessage.Message.Message.Header == nil { return fmt.Errorf("invalid feed message at sequence number %v", feedMessage.SequenceNumber) } - messages = append(messages, feedMessage.Message) + msgWithBlockHash := arbostypes.MessageWithMetadataAndBlockHash{ + Message: feedMessage.Message, + BlockHash: feedMessage.BlockHash, + } + messages = append(messages, msgWithBlockHash) broadcastAfterPos++ } @@ -607,7 +611,7 @@ func (s *TransactionStreamer) AddBroadcastMessages(feedMessages []*m.BroadcastFe messages = messages[dups:] broadcastStartPos += arbutil.MessageIndex(dups) if oldMsg != nil { - s.logReorg(broadcastStartPos, oldMsg, &messages[0], false) + s.logReorg(broadcastStartPos, oldMsg, &messages[0].Message, false) } if len(messages) == 0 { // No new messages received @@ -681,16 +685,19 @@ func (s *TransactionStreamer) AddFakeInitMessage() error { } chainIdBytes := arbmath.U256Bytes(s.chainConfig.ChainID) msg := append(append(chainIdBytes, 0), chainConfigJson...) - return s.AddMessages(0, false, []arbostypes.MessageWithMetadata{{ - Message: &arbostypes.L1IncomingMessage{ - Header: &arbostypes.L1IncomingMessageHeader{ - Kind: arbostypes.L1MessageType_Initialize, - RequestId: &common.Hash{}, - L1BaseFee: common.Big0, + return s.AddMessages(0, false, []arbostypes.MessageWithMetadataAndBlockHash{{ + Message: arbostypes.MessageWithMetadata{ + Message: &arbostypes.L1IncomingMessage{ + Header: &arbostypes.L1IncomingMessageHeader{ + Kind: arbostypes.L1MessageType_Initialize, + RequestId: &common.Hash{}, + L1BaseFee: common.Big0, + }, + L2msg: msg, }, - L2msg: msg, + DelayedMessagesRead: 1, }, - DelayedMessagesRead: 1, + BlockHash: nil, }}) } @@ -708,7 +715,7 @@ func endBatch(batch ethdb.Batch) error { return batch.Write() } -func (s *TransactionStreamer) AddMessagesAndEndBatch(pos arbutil.MessageIndex, messagesAreConfirmed bool, messages []arbostypes.MessageWithMetadata, batch ethdb.Batch) error { +func (s *TransactionStreamer) AddMessagesAndEndBatch(pos arbutil.MessageIndex, messagesAreConfirmed bool, messages []arbostypes.MessageWithMetadataAndBlockHash, batch ethdb.Batch) error { if messagesAreConfirmed { // Trim confirmed messages from l1pricedataCache s.TrimCache(pos + arbutil.MessageIndex(len(messages))) @@ -748,7 +755,7 @@ func (s *TransactionStreamer) getPrevPrevDelayedRead(pos arbutil.MessageIndex) ( func (s *TransactionStreamer) countDuplicateMessages( pos arbutil.MessageIndex, - messages []arbostypes.MessageWithMetadata, + messages []arbostypes.MessageWithMetadataAndBlockHash, batch *ethdb.Batch, ) (int, bool, *arbostypes.MessageWithMetadata, error) { curMsg := 0 @@ -768,7 +775,7 @@ func (s *TransactionStreamer) countDuplicateMessages( if err != nil { return 0, false, nil, err } - nextMessage := messages[curMsg] + nextMessage := messages[curMsg].Message wantMessage, err := rlp.EncodeToBytes(nextMessage) if err != nil { return 0, false, nil, err @@ -842,7 +849,7 @@ func (s *TransactionStreamer) logReorg(pos arbutil.MessageIndex, dbMsg *arbostyp } -func (s *TransactionStreamer) addMessagesAndEndBatchImpl(messageStartPos arbutil.MessageIndex, messagesAreConfirmed bool, messages []arbostypes.MessageWithMetadata, batch ethdb.Batch) error { +func (s *TransactionStreamer) addMessagesAndEndBatchImpl(messageStartPos arbutil.MessageIndex, messagesAreConfirmed bool, messages []arbostypes.MessageWithMetadataAndBlockHash, batch ethdb.Batch) error { var confirmedReorg bool var oldMsg *arbostypes.MessageWithMetadata var lastDelayedRead uint64 @@ -860,7 +867,7 @@ func (s *TransactionStreamer) addMessagesAndEndBatchImpl(messageStartPos arbutil return err } if duplicates > 0 { - lastDelayedRead = messages[duplicates-1].DelayedMessagesRead + lastDelayedRead = messages[duplicates-1].Message.DelayedMessagesRead messages = messages[duplicates:] messageStartPos += arbutil.MessageIndex(duplicates) } @@ -898,13 +905,13 @@ func (s *TransactionStreamer) addMessagesAndEndBatchImpl(messageStartPos arbutil return err } if duplicates > 0 { - lastDelayedRead = messages[duplicates-1].DelayedMessagesRead + lastDelayedRead = messages[duplicates-1].Message.DelayedMessagesRead messages = messages[duplicates:] messageStartPos += arbutil.MessageIndex(duplicates) } } if oldMsg != nil { - s.logReorg(messageStartPos, oldMsg, &messages[0], confirmedReorg) + s.logReorg(messageStartPos, oldMsg, &messages[0].Message, confirmedReorg) } if feedReorg { @@ -924,12 +931,12 @@ func (s *TransactionStreamer) addMessagesAndEndBatchImpl(messageStartPos arbutil // Validate delayed message counts of remaining messages for i, msg := range messages { msgPos := messageStartPos + arbutil.MessageIndex(i) - diff := msg.DelayedMessagesRead - lastDelayedRead + diff := msg.Message.DelayedMessagesRead - lastDelayedRead if diff != 0 && diff != 1 { - return fmt.Errorf("attempted to insert jump from %v delayed messages read to %v delayed messages read at message index %v", lastDelayedRead, msg.DelayedMessagesRead, msgPos) + return fmt.Errorf("attempted to insert jump from %v delayed messages read to %v delayed messages read at message index %v", lastDelayedRead, msg.Message.DelayedMessagesRead, msgPos) } - lastDelayedRead = msg.DelayedMessagesRead - if msg.Message == nil { + lastDelayedRead = msg.Message.DelayedMessagesRead + if msg.Message.Message == nil { return fmt.Errorf("attempted to insert nil message at position %v", msgPos) } } @@ -1007,14 +1014,14 @@ func (s *TransactionStreamer) WriteMessageFromSequencer( } } - if err := s.writeMessages(pos, []arbostypes.MessageWithMetadata{msgWithMeta}, nil); err != nil { - return err - } - msgWithBlockHash := arbostypes.MessageWithMetadataAndBlockHash{ Message: msgWithMeta, BlockHash: &msgResult.BlockHash, } + + if err := s.writeMessages(pos, []arbostypes.MessageWithMetadataAndBlockHash{msgWithBlockHash}, nil); err != nil { + return err + } s.broadcastMessages([]arbostypes.MessageWithMetadataAndBlockHash{msgWithBlockHash}, pos) return nil @@ -1059,12 +1066,12 @@ func (s *TransactionStreamer) broadcastMessages( // The mutex must be held, and pos must be the latest message count. // `batch` may be nil, which initializes a new batch. The batch is closed out in this function. -func (s *TransactionStreamer) writeMessages(pos arbutil.MessageIndex, messages []arbostypes.MessageWithMetadata, batch ethdb.Batch) error { +func (s *TransactionStreamer) writeMessages(pos arbutil.MessageIndex, messages []arbostypes.MessageWithMetadataAndBlockHash, batch ethdb.Batch) error { if batch == nil { batch = s.db.NewBatch() } for i, msg := range messages { - err := s.writeMessage(pos+arbutil.MessageIndex(i), msg, batch) + err := s.writeMessage(pos+arbutil.MessageIndex(i), msg.Message, batch) if err != nil { return err } diff --git a/execution/gethexec/executionengine.go b/execution/gethexec/executionengine.go index 38569f44ab..c4fbc04712 100644 --- a/execution/gethexec/executionengine.go +++ b/execution/gethexec/executionengine.go @@ -116,7 +116,7 @@ func (s *ExecutionEngine) GetBatchFetcher() execution.BatchFetcher { return s.consensus } -func (s *ExecutionEngine) Reorg(count arbutil.MessageIndex, newMessages []arbostypes.MessageWithMetadata, oldMessages []*arbostypes.MessageWithMetadata) ([]*execution.MessageResult, error) { +func (s *ExecutionEngine) Reorg(count arbutil.MessageIndex, newMessages []arbostypes.MessageWithMetadataAndBlockHash, oldMessages []*arbostypes.MessageWithMetadata) ([]*execution.MessageResult, error) { if count == 0 { return nil, errors.New("cannot reorg out genesis") } @@ -149,9 +149,9 @@ func (s *ExecutionEngine) Reorg(count arbutil.MessageIndex, newMessages []arbost for i := range newMessages { var msgForPrefetch *arbostypes.MessageWithMetadata if i < len(newMessages)-1 { - msgForPrefetch = &newMessages[i] + msgForPrefetch = &newMessages[i].Message } - msgResult, err := s.digestMessageWithBlockMutex(count+arbutil.MessageIndex(i), &newMessages[i], msgForPrefetch) + msgResult, err := s.digestMessageWithBlockMutex(count+arbutil.MessageIndex(i), &newMessages[i].Message, msgForPrefetch) if err != nil { return nil, err } diff --git a/execution/gethexec/node.go b/execution/gethexec/node.go index ae76b88530..458d6601c5 100644 --- a/execution/gethexec/node.go +++ b/execution/gethexec/node.go @@ -346,7 +346,7 @@ func (n *ExecutionNode) StopAndWait() { func (n *ExecutionNode) DigestMessage(num arbutil.MessageIndex, msg *arbostypes.MessageWithMetadata, msgForPrefetch *arbostypes.MessageWithMetadata) (*execution.MessageResult, error) { return n.ExecEngine.DigestMessage(num, msg, msgForPrefetch) } -func (n *ExecutionNode) Reorg(count arbutil.MessageIndex, newMessages []arbostypes.MessageWithMetadata, oldMessages []*arbostypes.MessageWithMetadata) ([]*execution.MessageResult, error) { +func (n *ExecutionNode) Reorg(count arbutil.MessageIndex, newMessages []arbostypes.MessageWithMetadataAndBlockHash, oldMessages []*arbostypes.MessageWithMetadata) ([]*execution.MessageResult, error) { return n.ExecEngine.Reorg(count, newMessages, oldMessages) } func (n *ExecutionNode) HeadMessageNumber() (arbutil.MessageIndex, error) { diff --git a/execution/interface.go b/execution/interface.go index d2a5b58fe5..66aefe9a5e 100644 --- a/execution/interface.go +++ b/execution/interface.go @@ -31,7 +31,7 @@ var ErrSequencerInsertLockTaken = errors.New("insert lock taken") // always needed type ExecutionClient interface { DigestMessage(num arbutil.MessageIndex, msg *arbostypes.MessageWithMetadata, msgForPrefetch *arbostypes.MessageWithMetadata) (*MessageResult, error) - Reorg(count arbutil.MessageIndex, newMessages []arbostypes.MessageWithMetadata, oldMessages []*arbostypes.MessageWithMetadata) ([]*MessageResult, error) + Reorg(count arbutil.MessageIndex, newMessages []arbostypes.MessageWithMetadataAndBlockHash, oldMessages []*arbostypes.MessageWithMetadata) ([]*MessageResult, error) HeadMessageNumber() (arbutil.MessageIndex, error) HeadMessageNumberSync(t *testing.T) (arbutil.MessageIndex, error) ResultAtPos(pos arbutil.MessageIndex) (*MessageResult, error) diff --git a/system_tests/contract_tx_test.go b/system_tests/contract_tx_test.go index 7d66e516b4..d0f7b153f3 100644 --- a/system_tests/contract_tx_test.go +++ b/system_tests/contract_tx_test.go @@ -69,21 +69,23 @@ func TestContractTxDeploy(t *testing.T) { l2Msg = append(l2Msg, arbmath.U256Bytes(contractTx.Value)...) l2Msg = append(l2Msg, contractTx.Data...) - err = builder.L2.ConsensusNode.TxStreamer.AddMessages(pos, true, []arbostypes.MessageWithMetadata{ + err = builder.L2.ConsensusNode.TxStreamer.AddMessages(pos, true, []arbostypes.MessageWithMetadataAndBlockHash{ { - Message: &arbostypes.L1IncomingMessage{ - Header: &arbostypes.L1IncomingMessageHeader{ - Kind: arbostypes.L1MessageType_L2Message, - Poster: from, - BlockNumber: 0, - Timestamp: 0, - RequestId: &contractTx.RequestId, - L1BaseFee: &big.Int{}, + Message: arbostypes.MessageWithMetadata{ + Message: &arbostypes.L1IncomingMessage{ + Header: &arbostypes.L1IncomingMessageHeader{ + Kind: arbostypes.L1MessageType_L2Message, + Poster: from, + BlockNumber: 0, + Timestamp: 0, + RequestId: &contractTx.RequestId, + L1BaseFee: &big.Int{}, + }, + L2msg: l2Msg, + BatchGasCost: new(uint64), }, - L2msg: l2Msg, - BatchGasCost: new(uint64), + DelayedMessagesRead: delayedMessagesRead, }, - DelayedMessagesRead: delayedMessagesRead, }, }) Require(t, err) diff --git a/system_tests/reorg_resequencing_test.go b/system_tests/reorg_resequencing_test.go index b188504acb..6d5ecd5e6a 100644 --- a/system_tests/reorg_resequencing_test.go +++ b/system_tests/reorg_resequencing_test.go @@ -72,9 +72,11 @@ func TestReorgResequencing(t *testing.T) { }, L2msg: append(builder.L2Info.GetAddress("User4").Bytes(), arbmath.Uint64ToU256Bytes(params.Ether)...), } - err = builder.L2.ConsensusNode.TxStreamer.AddMessages(startMsgCount, true, []arbostypes.MessageWithMetadata{{ - Message: newMessage, - DelayedMessagesRead: prevMessage.DelayedMessagesRead + 1, + err = builder.L2.ConsensusNode.TxStreamer.AddMessages(startMsgCount, true, []arbostypes.MessageWithMetadataAndBlockHash{{ + Message: arbostypes.MessageWithMetadata{ + Message: newMessage, + DelayedMessagesRead: prevMessage.DelayedMessagesRead + 1, + }, }}) Require(t, err) diff --git a/system_tests/seq_coordinator_test.go b/system_tests/seq_coordinator_test.go index 886a0528c7..5e539a8812 100644 --- a/system_tests/seq_coordinator_test.go +++ b/system_tests/seq_coordinator_test.go @@ -91,7 +91,10 @@ func TestRedisSeqCoordinatorPriorities(t *testing.T) { return false } Require(t, err) - Require(t, node.TxStreamer.AddMessages(curMsgs, false, []arbostypes.MessageWithMetadata{emptyMessage})) + emptyMessageWithBlockHash := arbostypes.MessageWithMetadataAndBlockHash{ + Message: emptyMessage, + } + Require(t, node.TxStreamer.AddMessages(curMsgs, false, []arbostypes.MessageWithMetadataAndBlockHash{emptyMessageWithBlockHash})) return true } From bf3c9602d822c50277b705c46e22fc151e72fa78 Mon Sep 17 00:00:00 2001 From: Diego Ximenes Date: Thu, 2 May 2024 17:21:20 -0300 Subject: [PATCH 029/113] rename Message to MessageWithMeta in MessageWithMetadataAndBlockHash --- arbnode/inbox_test.go | 2 +- arbnode/inbox_tracker.go | 4 +-- arbnode/seq_coordinator.go | 2 +- arbnode/transaction_streamer.go | 38 ++++++++++++------------- arbos/arbostypes/messagewithmeta.go | 4 +-- broadcaster/broadcaster.go | 2 +- execution/gethexec/executionengine.go | 4 +-- system_tests/contract_tx_test.go | 2 +- system_tests/reorg_resequencing_test.go | 2 +- system_tests/seq_coordinator_test.go | 2 +- 10 files changed, 31 insertions(+), 31 deletions(-) diff --git a/arbnode/inbox_test.go b/arbnode/inbox_test.go index fbd1dba96a..a5d1554cb1 100644 --- a/arbnode/inbox_test.go +++ b/arbnode/inbox_test.go @@ -155,7 +155,7 @@ func TestTransactionStreamer(t *testing.T) { var requestId common.Hash binary.BigEndian.PutUint64(requestId.Bytes()[:8], uint64(i)) messages = append(messages, arbostypes.MessageWithMetadataAndBlockHash{ - Message: arbostypes.MessageWithMetadata{ + MessageWithMeta: arbostypes.MessageWithMetadata{ Message: &arbostypes.L1IncomingMessage{ Header: &arbostypes.L1IncomingMessageHeader{ Kind: arbostypes.L1MessageType_L2Message, diff --git a/arbnode/inbox_tracker.go b/arbnode/inbox_tracker.go index e2aa1d5e74..2340df8303 100644 --- a/arbnode/inbox_tracker.go +++ b/arbnode/inbox_tracker.go @@ -674,7 +674,7 @@ func (t *InboxTracker) AddSequencerBatches(ctx context.Context, client arbutil.L return err } msgWithBlockHash := arbostypes.MessageWithMetadataAndBlockHash{ - Message: *msg, + MessageWithMeta: *msg, } messages = append(messages, msgWithBlockHash) batchMessageCounts[batchSeqNum] = currentpos @@ -736,7 +736,7 @@ func (t *InboxTracker) AddSequencerBatches(ctx context.Context, client arbutil.L } var latestTimestamp uint64 if len(messages) > 0 { - latestTimestamp = messages[len(messages)-1].Message.Message.Header.Timestamp + latestTimestamp = messages[len(messages)-1].MessageWithMeta.Message.Header.Timestamp } log.Info( "InboxTracker", diff --git a/arbnode/seq_coordinator.go b/arbnode/seq_coordinator.go index 0a27d89d40..2fb8c3244b 100644 --- a/arbnode/seq_coordinator.go +++ b/arbnode/seq_coordinator.go @@ -593,7 +593,7 @@ func (c *SeqCoordinator) update(ctx context.Context) time.Duration { } } msgWithBlockHash := arbostypes.MessageWithMetadataAndBlockHash{ - Message: message, + MessageWithMeta: message, } messages = append(messages, msgWithBlockHash) msgToRead++ diff --git a/arbnode/transaction_streamer.go b/arbnode/transaction_streamer.go index 411eba965d..708dcff41b 100644 --- a/arbnode/transaction_streamer.go +++ b/arbnode/transaction_streamer.go @@ -468,8 +468,8 @@ func (s *TransactionStreamer) reorg(batch ethdb.Batch, count arbutil.MessageInde messagesWithComputedBlockHash := make([]arbostypes.MessageWithMetadataAndBlockHash, 0, len(messagesResults)) for i := 0; i < len(messagesResults); i++ { messagesWithComputedBlockHash = append(messagesWithComputedBlockHash, arbostypes.MessageWithMetadataAndBlockHash{ - Message: newMessages[i].Message, - BlockHash: &messagesResults[i].BlockHash, + MessageWithMeta: newMessages[i].MessageWithMeta, + BlockHash: &messagesResults[i].BlockHash, }) } s.broadcastMessages(messagesWithComputedBlockHash, count) @@ -589,8 +589,8 @@ func (s *TransactionStreamer) AddBroadcastMessages(feedMessages []*m.BroadcastFe return fmt.Errorf("invalid feed message at sequence number %v", feedMessage.SequenceNumber) } msgWithBlockHash := arbostypes.MessageWithMetadataAndBlockHash{ - Message: feedMessage.Message, - BlockHash: feedMessage.BlockHash, + MessageWithMeta: feedMessage.Message, + BlockHash: feedMessage.BlockHash, } messages = append(messages, msgWithBlockHash) broadcastAfterPos++ @@ -611,7 +611,7 @@ func (s *TransactionStreamer) AddBroadcastMessages(feedMessages []*m.BroadcastFe messages = messages[dups:] broadcastStartPos += arbutil.MessageIndex(dups) if oldMsg != nil { - s.logReorg(broadcastStartPos, oldMsg, &messages[0].Message, false) + s.logReorg(broadcastStartPos, oldMsg, &messages[0].MessageWithMeta, false) } if len(messages) == 0 { // No new messages received @@ -686,7 +686,7 @@ func (s *TransactionStreamer) AddFakeInitMessage() error { chainIdBytes := arbmath.U256Bytes(s.chainConfig.ChainID) msg := append(append(chainIdBytes, 0), chainConfigJson...) return s.AddMessages(0, false, []arbostypes.MessageWithMetadataAndBlockHash{{ - Message: arbostypes.MessageWithMetadata{ + MessageWithMeta: arbostypes.MessageWithMetadata{ Message: &arbostypes.L1IncomingMessage{ Header: &arbostypes.L1IncomingMessageHeader{ Kind: arbostypes.L1MessageType_Initialize, @@ -775,7 +775,7 @@ func (s *TransactionStreamer) countDuplicateMessages( if err != nil { return 0, false, nil, err } - nextMessage := messages[curMsg].Message + nextMessage := messages[curMsg].MessageWithMeta wantMessage, err := rlp.EncodeToBytes(nextMessage) if err != nil { return 0, false, nil, err @@ -867,7 +867,7 @@ func (s *TransactionStreamer) addMessagesAndEndBatchImpl(messageStartPos arbutil return err } if duplicates > 0 { - lastDelayedRead = messages[duplicates-1].Message.DelayedMessagesRead + lastDelayedRead = messages[duplicates-1].MessageWithMeta.DelayedMessagesRead messages = messages[duplicates:] messageStartPos += arbutil.MessageIndex(duplicates) } @@ -905,13 +905,13 @@ func (s *TransactionStreamer) addMessagesAndEndBatchImpl(messageStartPos arbutil return err } if duplicates > 0 { - lastDelayedRead = messages[duplicates-1].Message.DelayedMessagesRead + lastDelayedRead = messages[duplicates-1].MessageWithMeta.DelayedMessagesRead messages = messages[duplicates:] messageStartPos += arbutil.MessageIndex(duplicates) } } if oldMsg != nil { - s.logReorg(messageStartPos, oldMsg, &messages[0].Message, confirmedReorg) + s.logReorg(messageStartPos, oldMsg, &messages[0].MessageWithMeta, confirmedReorg) } if feedReorg { @@ -931,12 +931,12 @@ func (s *TransactionStreamer) addMessagesAndEndBatchImpl(messageStartPos arbutil // Validate delayed message counts of remaining messages for i, msg := range messages { msgPos := messageStartPos + arbutil.MessageIndex(i) - diff := msg.Message.DelayedMessagesRead - lastDelayedRead + diff := msg.MessageWithMeta.DelayedMessagesRead - lastDelayedRead if diff != 0 && diff != 1 { - return fmt.Errorf("attempted to insert jump from %v delayed messages read to %v delayed messages read at message index %v", lastDelayedRead, msg.Message.DelayedMessagesRead, msgPos) + return fmt.Errorf("attempted to insert jump from %v delayed messages read to %v delayed messages read at message index %v", lastDelayedRead, msg.MessageWithMeta.DelayedMessagesRead, msgPos) } - lastDelayedRead = msg.Message.DelayedMessagesRead - if msg.Message.Message == nil { + lastDelayedRead = msg.MessageWithMeta.DelayedMessagesRead + if msg.MessageWithMeta.Message == nil { return fmt.Errorf("attempted to insert nil message at position %v", msgPos) } } @@ -1015,8 +1015,8 @@ func (s *TransactionStreamer) WriteMessageFromSequencer( } msgWithBlockHash := arbostypes.MessageWithMetadataAndBlockHash{ - Message: msgWithMeta, - BlockHash: &msgResult.BlockHash, + MessageWithMeta: msgWithMeta, + BlockHash: &msgResult.BlockHash, } if err := s.writeMessages(pos, []arbostypes.MessageWithMetadataAndBlockHash{msgWithBlockHash}, nil); err != nil { @@ -1071,7 +1071,7 @@ func (s *TransactionStreamer) writeMessages(pos arbutil.MessageIndex, messages [ batch = s.db.NewBatch() } for i, msg := range messages { - err := s.writeMessage(pos+arbutil.MessageIndex(i), msg.Message, batch) + err := s.writeMessage(pos+arbutil.MessageIndex(i), msg.MessageWithMeta, batch) if err != nil { return err } @@ -1153,8 +1153,8 @@ func (s *TransactionStreamer) ExecuteNextMsg(ctx context.Context, exec execution } msgWithBlockHash := arbostypes.MessageWithMetadataAndBlockHash{ - Message: *msg, - BlockHash: &msgResult.BlockHash, + MessageWithMeta: *msg, + BlockHash: &msgResult.BlockHash, } s.broadcastMessages([]arbostypes.MessageWithMetadataAndBlockHash{msgWithBlockHash}, pos) diff --git a/arbos/arbostypes/messagewithmeta.go b/arbos/arbostypes/messagewithmeta.go index e1215e0dd5..79b7c4f9d2 100644 --- a/arbos/arbostypes/messagewithmeta.go +++ b/arbos/arbostypes/messagewithmeta.go @@ -19,8 +19,8 @@ type MessageWithMetadata struct { } type MessageWithMetadataAndBlockHash struct { - Message MessageWithMetadata - BlockHash *common.Hash + MessageWithMeta MessageWithMetadata + BlockHash *common.Hash } var EmptyTestMessageWithMetadata = MessageWithMetadata{ diff --git a/broadcaster/broadcaster.go b/broadcaster/broadcaster.go index da1de6665e..ba95f2d8af 100644 --- a/broadcaster/broadcaster.go +++ b/broadcaster/broadcaster.go @@ -104,7 +104,7 @@ func (b *Broadcaster) BroadcastMessages( }() var feedMessages []*m.BroadcastFeedMessage for i, msg := range messagesWithBlockHash { - bfm, err := b.NewBroadcastFeedMessage(msg.Message, seq+arbutil.MessageIndex(i), msg.BlockHash) + bfm, err := b.NewBroadcastFeedMessage(msg.MessageWithMeta, seq+arbutil.MessageIndex(i), msg.BlockHash) if err != nil { return err } diff --git a/execution/gethexec/executionengine.go b/execution/gethexec/executionengine.go index c4fbc04712..b31209b882 100644 --- a/execution/gethexec/executionengine.go +++ b/execution/gethexec/executionengine.go @@ -149,9 +149,9 @@ func (s *ExecutionEngine) Reorg(count arbutil.MessageIndex, newMessages []arbost for i := range newMessages { var msgForPrefetch *arbostypes.MessageWithMetadata if i < len(newMessages)-1 { - msgForPrefetch = &newMessages[i].Message + msgForPrefetch = &newMessages[i].MessageWithMeta } - msgResult, err := s.digestMessageWithBlockMutex(count+arbutil.MessageIndex(i), &newMessages[i].Message, msgForPrefetch) + msgResult, err := s.digestMessageWithBlockMutex(count+arbutil.MessageIndex(i), &newMessages[i].MessageWithMeta, msgForPrefetch) if err != nil { return nil, err } diff --git a/system_tests/contract_tx_test.go b/system_tests/contract_tx_test.go index d0f7b153f3..c4ae326df1 100644 --- a/system_tests/contract_tx_test.go +++ b/system_tests/contract_tx_test.go @@ -71,7 +71,7 @@ func TestContractTxDeploy(t *testing.T) { err = builder.L2.ConsensusNode.TxStreamer.AddMessages(pos, true, []arbostypes.MessageWithMetadataAndBlockHash{ { - Message: arbostypes.MessageWithMetadata{ + MessageWithMeta: arbostypes.MessageWithMetadata{ Message: &arbostypes.L1IncomingMessage{ Header: &arbostypes.L1IncomingMessageHeader{ Kind: arbostypes.L1MessageType_L2Message, diff --git a/system_tests/reorg_resequencing_test.go b/system_tests/reorg_resequencing_test.go index 6d5ecd5e6a..28d1b3bd66 100644 --- a/system_tests/reorg_resequencing_test.go +++ b/system_tests/reorg_resequencing_test.go @@ -73,7 +73,7 @@ func TestReorgResequencing(t *testing.T) { L2msg: append(builder.L2Info.GetAddress("User4").Bytes(), arbmath.Uint64ToU256Bytes(params.Ether)...), } err = builder.L2.ConsensusNode.TxStreamer.AddMessages(startMsgCount, true, []arbostypes.MessageWithMetadataAndBlockHash{{ - Message: arbostypes.MessageWithMetadata{ + MessageWithMeta: arbostypes.MessageWithMetadata{ Message: newMessage, DelayedMessagesRead: prevMessage.DelayedMessagesRead + 1, }, diff --git a/system_tests/seq_coordinator_test.go b/system_tests/seq_coordinator_test.go index 5e539a8812..36c7be7251 100644 --- a/system_tests/seq_coordinator_test.go +++ b/system_tests/seq_coordinator_test.go @@ -92,7 +92,7 @@ func TestRedisSeqCoordinatorPriorities(t *testing.T) { } Require(t, err) emptyMessageWithBlockHash := arbostypes.MessageWithMetadataAndBlockHash{ - Message: emptyMessage, + MessageWithMeta: emptyMessage, } Require(t, node.TxStreamer.AddMessages(curMsgs, false, []arbostypes.MessageWithMetadataAndBlockHash{emptyMessageWithBlockHash})) return true From 44e44c7a10a642fe66eac3159e0712038e57f45f Mon Sep 17 00:00:00 2001 From: Diego Ximenes Date: Thu, 2 May 2024 17:59:42 -0300 Subject: [PATCH 030/113] write block hash alongsing with message with metadata --- arbnode/schema.go | 1 + arbnode/transaction_streamer.go | 31 +++++++++++++++++++++---------- 2 files changed, 22 insertions(+), 10 deletions(-) diff --git a/arbnode/schema.go b/arbnode/schema.go index ddc7cf54fd..2854b7e785 100644 --- a/arbnode/schema.go +++ b/arbnode/schema.go @@ -5,6 +5,7 @@ package arbnode var ( messagePrefix []byte = []byte("m") // maps a message sequence number to a message + blockHashInputFeedPrefix []byte = []byte("b") // maps a message sequence number to a block hash received through the input feed legacyDelayedMessagePrefix []byte = []byte("d") // maps a delayed sequence number to an accumulator and a message as serialized on L1 rlpDelayedMessagePrefix []byte = []byte("e") // maps a delayed sequence number to an accumulator and an RLP encoded message parentChainBlockNumberPrefix []byte = []byte("p") // maps a delayed sequence number to a parent chain block number diff --git a/arbnode/transaction_streamer.go b/arbnode/transaction_streamer.go index 708dcff41b..5debe0c41f 100644 --- a/arbnode/transaction_streamer.go +++ b/arbnode/transaction_streamer.go @@ -775,8 +775,8 @@ func (s *TransactionStreamer) countDuplicateMessages( if err != nil { return 0, false, nil, err } - nextMessage := messages[curMsg].MessageWithMeta - wantMessage, err := rlp.EncodeToBytes(nextMessage) + nextMessage := messages[curMsg] + wantMessage, err := rlp.EncodeToBytes(nextMessage.MessageWithMeta) if err != nil { return 0, false, nil, err } @@ -792,12 +792,12 @@ func (s *TransactionStreamer) countDuplicateMessages( return curMsg, true, nil, nil } var duplicateMessage bool - if nextMessage.Message != nil { - if dbMessageParsed.Message.BatchGasCost == nil || nextMessage.Message.BatchGasCost == nil { + if nextMessage.MessageWithMeta.Message != nil { + if dbMessageParsed.Message.BatchGasCost == nil || nextMessage.MessageWithMeta.Message.BatchGasCost == nil { // Remove both of the batch gas costs and see if the messages still differ - nextMessageCopy := nextMessage + nextMessageCopy := nextMessage.MessageWithMeta nextMessageCopy.Message = new(arbostypes.L1IncomingMessage) - *nextMessageCopy.Message = *nextMessage.Message + *nextMessageCopy.Message = *nextMessage.MessageWithMeta.Message batchGasCostBkup := dbMessageParsed.Message.BatchGasCost dbMessageParsed.Message.BatchGasCost = nil nextMessageCopy.Message.BatchGasCost = nil @@ -805,7 +805,7 @@ func (s *TransactionStreamer) countDuplicateMessages( // Actually this isn't a reorg; only the batch gas costs differed duplicateMessage = true // If possible - update the message in the database to add the gas cost cache. - if batch != nil && nextMessage.Message.BatchGasCost != nil { + if batch != nil && nextMessage.MessageWithMeta.Message.BatchGasCost != nil { if *batch == nil { *batch = s.db.NewBatch() } @@ -1043,9 +1043,20 @@ func (s *TransactionStreamer) PopulateFeedBacklog() error { return s.inboxReader.tracker.PopulateFeedBacklog(s.broadcastServer) } -func (s *TransactionStreamer) writeMessage(pos arbutil.MessageIndex, msg arbostypes.MessageWithMetadata, batch ethdb.Batch) error { +func (s *TransactionStreamer) writeMessage(pos arbutil.MessageIndex, msg arbostypes.MessageWithMetadataAndBlockHash, batch ethdb.Batch) error { + // write message with metadata key := dbKey(messagePrefix, uint64(pos)) - msgBytes, err := rlp.EncodeToBytes(msg) + msgBytes, err := rlp.EncodeToBytes(msg.MessageWithMeta) + if err != nil { + return err + } + if err := batch.Put(key, msgBytes); err != nil { + return err + } + + // write block hash + key = dbKey(blockHashInputFeedPrefix, uint64(pos)) + msgBytes, err = rlp.EncodeToBytes(msg.BlockHash) if err != nil { return err } @@ -1071,7 +1082,7 @@ func (s *TransactionStreamer) writeMessages(pos arbutil.MessageIndex, messages [ batch = s.db.NewBatch() } for i, msg := range messages { - err := s.writeMessage(pos+arbutil.MessageIndex(i), msg.MessageWithMeta, batch) + err := s.writeMessage(pos+arbutil.MessageIndex(i), msg, batch) if err != nil { return err } From da0b605ad80d65ca5ae00c438a412ccf4af8a968 Mon Sep 17 00:00:00 2001 From: Diego Ximenes Date: Thu, 2 May 2024 18:58:59 -0300 Subject: [PATCH 031/113] prune block hashes in db --- arbnode/message_pruner.go | 25 +++++++++++++++++-------- arbnode/message_pruner_test.go | 6 ++++-- arbnode/transaction_streamer.go | 4 ++++ 3 files changed, 25 insertions(+), 10 deletions(-) diff --git a/arbnode/message_pruner.go b/arbnode/message_pruner.go index 31bf1a63ff..c31dbc496d 100644 --- a/arbnode/message_pruner.go +++ b/arbnode/message_pruner.go @@ -23,13 +23,14 @@ import ( type MessagePruner struct { stopwaiter.StopWaiter - transactionStreamer *TransactionStreamer - inboxTracker *InboxTracker - config MessagePrunerConfigFetcher - pruningLock sync.Mutex - lastPruneDone time.Time - cachedPrunedMessages uint64 - cachedPrunedDelayedMessages uint64 + transactionStreamer *TransactionStreamer + inboxTracker *InboxTracker + config MessagePrunerConfigFetcher + pruningLock sync.Mutex + lastPruneDone time.Time + cachedPrunedMessages uint64 + cachedPrunedBlockHashesInputFeed uint64 + cachedPrunedDelayedMessages uint64 } type MessagePrunerConfig struct { @@ -115,7 +116,15 @@ func (m *MessagePruner) prune(ctx context.Context, count arbutil.MessageIndex, g } func (m *MessagePruner) deleteOldMessagesFromDB(ctx context.Context, messageCount arbutil.MessageIndex, delayedMessageCount uint64) error { - prunedKeysRange, err := deleteFromLastPrunedUptoEndKey(ctx, m.transactionStreamer.db, messagePrefix, &m.cachedPrunedMessages, uint64(messageCount)) + prunedKeysRange, err := deleteFromLastPrunedUptoEndKey(ctx, m.transactionStreamer.db, blockHashInputFeedPrefix, &m.cachedPrunedBlockHashesInputFeed, uint64(messageCount)) + if err != nil { + return fmt.Errorf("error deleting last batch messages' block hashes: %w", err) + } + if len(prunedKeysRange) > 0 { + log.Info("Pruned last batch messages' block hashes:", "first pruned key", prunedKeysRange[0], "last pruned key", prunedKeysRange[len(prunedKeysRange)-1]) + } + + prunedKeysRange, err = deleteFromLastPrunedUptoEndKey(ctx, m.transactionStreamer.db, messagePrefix, &m.cachedPrunedMessages, uint64(messageCount)) if err != nil { return fmt.Errorf("error deleting last batch messages: %w", err) } diff --git a/arbnode/message_pruner_test.go b/arbnode/message_pruner_test.go index 0212ed2364..ed85c0ebce 100644 --- a/arbnode/message_pruner_test.go +++ b/arbnode/message_pruner_test.go @@ -22,8 +22,8 @@ func TestMessagePrunerWithPruningEligibleMessagePresent(t *testing.T) { Require(t, err) checkDbKeys(t, messagesCount, transactionStreamerDb, messagePrefix) + checkDbKeys(t, messagesCount, transactionStreamerDb, blockHashInputFeedPrefix) checkDbKeys(t, messagesCount, inboxTrackerDb, rlpDelayedMessagePrefix) - } func TestMessagePrunerTwoHalves(t *testing.T) { @@ -71,16 +71,18 @@ func TestMessagePrunerWithNoPruningEligibleMessagePresent(t *testing.T) { Require(t, err) checkDbKeys(t, uint64(messagesCount), transactionStreamerDb, messagePrefix) + checkDbKeys(t, uint64(messagesCount), transactionStreamerDb, blockHashInputFeedPrefix) checkDbKeys(t, messagesCount, inboxTrackerDb, rlpDelayedMessagePrefix) } func setupDatabase(t *testing.T, messageCount, delayedMessageCount uint64) (ethdb.Database, ethdb.Database, *MessagePruner) { - transactionStreamerDb := rawdb.NewMemoryDatabase() for i := uint64(0); i < uint64(messageCount); i++ { err := transactionStreamerDb.Put(dbKey(messagePrefix, i), []byte{}) Require(t, err) + err = transactionStreamerDb.Put(dbKey(blockHashInputFeedPrefix, i), []byte{}) + Require(t, err) } inboxTrackerDb := rawdb.NewMemoryDatabase() diff --git a/arbnode/transaction_streamer.go b/arbnode/transaction_streamer.go index 5debe0c41f..0d02dc27dc 100644 --- a/arbnode/transaction_streamer.go +++ b/arbnode/transaction_streamer.go @@ -481,6 +481,10 @@ func (s *TransactionStreamer) reorg(batch ethdb.Batch, count arbutil.MessageInde } } + err = deleteStartingAt(s.db, batch, blockHashInputFeedPrefix, uint64ToKey(uint64(count))) + if err != nil { + return err + } err = deleteStartingAt(s.db, batch, messagePrefix, uint64ToKey(uint64(count))) if err != nil { return err From 7ad286d12eeeae3eb8a723c6ce2f5d9f3651fef9 Mon Sep 17 00:00:00 2001 From: Diego Ximenes Date: Fri, 3 May 2024 12:37:42 -0300 Subject: [PATCH 032/113] retrieve block hash and compare it when executing next message --- arbnode/transaction_streamer.go | 46 ++++++++++++++++++++++++++++++--- 1 file changed, 43 insertions(+), 3 deletions(-) diff --git a/arbnode/transaction_streamer.go b/arbnode/transaction_streamer.go index 0d02dc27dc..9da0b3ea17 100644 --- a/arbnode/transaction_streamer.go +++ b/arbnode/transaction_streamer.go @@ -530,6 +530,30 @@ func (s *TransactionStreamer) GetMessage(seqNum arbutil.MessageIndex) (*arbostyp return &message, nil } +func (s *TransactionStreamer) getMessageWithMetadataAndBlockHash(seqNum arbutil.MessageIndex) (*arbostypes.MessageWithMetadataAndBlockHash, error) { + msg, err := s.GetMessage(seqNum) + if err != nil { + return nil, err + } + + key := dbKey(blockHashInputFeedPrefix, uint64(seqNum)) + data, err := s.db.Get(key) + if err != nil { + return nil, err + } + var blockHash *common.Hash + err = rlp.DecodeBytes(data, &blockHash) + if err != nil { + return nil, err + } + + msgWithBlockHash := arbostypes.MessageWithMetadataAndBlockHash{ + MessageWithMeta: *msg, + BlockHash: blockHash, + } + return &msgWithBlockHash, nil +} + // Note: if changed to acquire the mutex, some internal users may need to be updated to a non-locking version. func (s *TransactionStreamer) GetMessageCount() (arbutil.MessageIndex, error) { posBytes, err := s.db.Get(messageCountKey) @@ -1117,6 +1141,20 @@ func (s *TransactionStreamer) ResultAtCount(count arbutil.MessageIndex) (*execut return s.exec.ResultAtPos(count - 1) } +func (s *TransactionStreamer) checkResult(msgResult *execution.MessageResult, expectedBlockHash *common.Hash) { + if expectedBlockHash == nil { + return + } + if msgResult.BlockHash != *expectedBlockHash { + log.Error( + "block_hash_mismatch", + "expected", expectedBlockHash, + "actual", msgResult.BlockHash, + ) + return + } +} + // exposed for testing // return value: true if should be called again immediately func (s *TransactionStreamer) ExecuteNextMsg(ctx context.Context, exec execution.ExecutionSequencer) bool { @@ -1143,7 +1181,7 @@ func (s *TransactionStreamer) ExecuteNextMsg(ctx context.Context, exec execution if pos >= msgCount { return false } - msg, err := s.GetMessage(pos) + msgAndBlockHash, err := s.getMessageWithMetadataAndBlockHash(pos) if err != nil { log.Error("feedOneMsg failed to readMessage", "err", err, "pos", pos) return false @@ -1157,7 +1195,7 @@ func (s *TransactionStreamer) ExecuteNextMsg(ctx context.Context, exec execution } msgForPrefetch = msg } - msgResult, err := s.exec.DigestMessage(pos, msg, msgForPrefetch) + msgResult, err := s.exec.DigestMessage(pos, &msgAndBlockHash.MessageWithMeta, msgForPrefetch) if err != nil { logger := log.Warn if prevMessageCount < msgCount { @@ -1167,8 +1205,10 @@ func (s *TransactionStreamer) ExecuteNextMsg(ctx context.Context, exec execution return false } + s.checkResult(msgResult, msgAndBlockHash.BlockHash) + msgWithBlockHash := arbostypes.MessageWithMetadataAndBlockHash{ - MessageWithMeta: *msg, + MessageWithMeta: msgAndBlockHash.MessageWithMeta, BlockHash: &msgResult.BlockHash, } s.broadcastMessages([]arbostypes.MessageWithMetadataAndBlockHash{msgWithBlockHash}, pos) From 95690fbe1f7ce122ca94e474b78eec1b7c15a802 Mon Sep 17 00:00:00 2001 From: Diego Ximenes Date: Fri, 3 May 2024 15:35:53 -0300 Subject: [PATCH 033/113] use MessageWithMetadata instead of MessageWithMetadataAndBlockHash in places where BlockHash will definitely be nil --- arbnode/inbox_test.go | 20 +++++++-------- arbnode/inbox_tracker.go | 9 +++---- arbnode/seq_coordinator.go | 7 ++--- arbnode/transaction_streamer.go | 34 ++++++++++++++----------- system_tests/contract_tx_test.go | 26 +++++++++---------- system_tests/reorg_resequencing_test.go | 8 +++--- system_tests/seq_coordinator_test.go | 5 +--- 7 files changed, 49 insertions(+), 60 deletions(-) diff --git a/arbnode/inbox_test.go b/arbnode/inbox_test.go index a5d1554cb1..5c879743a4 100644 --- a/arbnode/inbox_test.go +++ b/arbnode/inbox_test.go @@ -128,7 +128,7 @@ func TestTransactionStreamer(t *testing.T) { } state.balances = newBalances - var messages []arbostypes.MessageWithMetadataAndBlockHash + var messages []arbostypes.MessageWithMetadata // TODO replay a random amount of messages too numMessages := rand.Int() % 5 for j := 0; j < numMessages; j++ { @@ -154,18 +154,16 @@ func TestTransactionStreamer(t *testing.T) { l2Message = append(l2Message, arbmath.U256Bytes(value)...) var requestId common.Hash binary.BigEndian.PutUint64(requestId.Bytes()[:8], uint64(i)) - messages = append(messages, arbostypes.MessageWithMetadataAndBlockHash{ - MessageWithMeta: arbostypes.MessageWithMetadata{ - Message: &arbostypes.L1IncomingMessage{ - Header: &arbostypes.L1IncomingMessageHeader{ - Kind: arbostypes.L1MessageType_L2Message, - Poster: source, - RequestId: &requestId, - }, - L2msg: l2Message, + messages = append(messages, arbostypes.MessageWithMetadata{ + Message: &arbostypes.L1IncomingMessage{ + Header: &arbostypes.L1IncomingMessageHeader{ + Kind: arbostypes.L1MessageType_L2Message, + Poster: source, + RequestId: &requestId, }, - DelayedMessagesRead: 1, + L2msg: l2Message, }, + DelayedMessagesRead: 1, }) state.balances[source].Sub(state.balances[source], value) if state.balances[dest] == nil { diff --git a/arbnode/inbox_tracker.go b/arbnode/inbox_tracker.go index 2340df8303..ba1b875ec8 100644 --- a/arbnode/inbox_tracker.go +++ b/arbnode/inbox_tracker.go @@ -652,7 +652,7 @@ func (t *InboxTracker) AddSequencerBatches(ctx context.Context, client arbutil.L pos++ } - var messages []arbostypes.MessageWithMetadataAndBlockHash + var messages []arbostypes.MessageWithMetadata backend := &multiplexerBackend{ batchSeqNum: batches[0].SequenceNumber, batches: batches, @@ -673,10 +673,7 @@ func (t *InboxTracker) AddSequencerBatches(ctx context.Context, client arbutil.L if err != nil { return err } - msgWithBlockHash := arbostypes.MessageWithMetadataAndBlockHash{ - MessageWithMeta: *msg, - } - messages = append(messages, msgWithBlockHash) + messages = append(messages, *msg) batchMessageCounts[batchSeqNum] = currentpos currentpos += 1 } @@ -736,7 +733,7 @@ func (t *InboxTracker) AddSequencerBatches(ctx context.Context, client arbutil.L } var latestTimestamp uint64 if len(messages) > 0 { - latestTimestamp = messages[len(messages)-1].MessageWithMeta.Message.Header.Timestamp + latestTimestamp = messages[len(messages)-1].Message.Header.Timestamp } log.Info( "InboxTracker", diff --git a/arbnode/seq_coordinator.go b/arbnode/seq_coordinator.go index 2fb8c3244b..ecf38ddf42 100644 --- a/arbnode/seq_coordinator.go +++ b/arbnode/seq_coordinator.go @@ -533,7 +533,7 @@ func (c *SeqCoordinator) update(ctx context.Context) time.Duration { if readUntil > localMsgCount+c.config.MsgPerPoll { readUntil = localMsgCount + c.config.MsgPerPoll } - var messages []arbostypes.MessageWithMetadataAndBlockHash + var messages []arbostypes.MessageWithMetadata msgToRead := localMsgCount var msgReadErr error for msgToRead < readUntil { @@ -592,10 +592,7 @@ func (c *SeqCoordinator) update(ctx context.Context) time.Duration { DelayedMessagesRead: lastDelayedMsg, } } - msgWithBlockHash := arbostypes.MessageWithMetadataAndBlockHash{ - MessageWithMeta: message, - } - messages = append(messages, msgWithBlockHash) + messages = append(messages, message) msgToRead++ } if len(messages) > 0 { diff --git a/arbnode/transaction_streamer.go b/arbnode/transaction_streamer.go index 9da0b3ea17..7ff565ec16 100644 --- a/arbnode/transaction_streamer.go +++ b/arbnode/transaction_streamer.go @@ -583,7 +583,7 @@ func (s *TransactionStreamer) GetProcessedMessageCount() (arbutil.MessageIndex, return msgCount, nil } -func (s *TransactionStreamer) AddMessages(pos arbutil.MessageIndex, messagesAreConfirmed bool, messages []arbostypes.MessageWithMetadataAndBlockHash) error { +func (s *TransactionStreamer) AddMessages(pos arbutil.MessageIndex, messagesAreConfirmed bool, messages []arbostypes.MessageWithMetadata) error { return s.AddMessagesAndEndBatch(pos, messagesAreConfirmed, messages, nil) } @@ -713,19 +713,16 @@ func (s *TransactionStreamer) AddFakeInitMessage() error { } chainIdBytes := arbmath.U256Bytes(s.chainConfig.ChainID) msg := append(append(chainIdBytes, 0), chainConfigJson...) - return s.AddMessages(0, false, []arbostypes.MessageWithMetadataAndBlockHash{{ - MessageWithMeta: arbostypes.MessageWithMetadata{ - Message: &arbostypes.L1IncomingMessage{ - Header: &arbostypes.L1IncomingMessageHeader{ - Kind: arbostypes.L1MessageType_Initialize, - RequestId: &common.Hash{}, - L1BaseFee: common.Big0, - }, - L2msg: msg, + return s.AddMessages(0, false, []arbostypes.MessageWithMetadata{{ + Message: &arbostypes.L1IncomingMessage{ + Header: &arbostypes.L1IncomingMessageHeader{ + Kind: arbostypes.L1MessageType_Initialize, + RequestId: &common.Hash{}, + L1BaseFee: common.Big0, }, - DelayedMessagesRead: 1, + L2msg: msg, }, - BlockHash: nil, + DelayedMessagesRead: 1, }}) } @@ -743,12 +740,19 @@ func endBatch(batch ethdb.Batch) error { return batch.Write() } -func (s *TransactionStreamer) AddMessagesAndEndBatch(pos arbutil.MessageIndex, messagesAreConfirmed bool, messages []arbostypes.MessageWithMetadataAndBlockHash, batch ethdb.Batch) error { +func (s *TransactionStreamer) AddMessagesAndEndBatch(pos arbutil.MessageIndex, messagesAreConfirmed bool, messages []arbostypes.MessageWithMetadata, batch ethdb.Batch) error { + messagesWithBlockHash := make([]arbostypes.MessageWithMetadataAndBlockHash, 0, len(messages)) + for _, message := range messages { + messagesWithBlockHash = append(messagesWithBlockHash, arbostypes.MessageWithMetadataAndBlockHash{ + MessageWithMeta: message, + }) + } + if messagesAreConfirmed { // Trim confirmed messages from l1pricedataCache s.TrimCache(pos + arbutil.MessageIndex(len(messages))) s.reorgMutex.RLock() - dups, _, _, err := s.countDuplicateMessages(pos, messages, nil) + dups, _, _, err := s.countDuplicateMessages(pos, messagesWithBlockHash, nil) s.reorgMutex.RUnlock() if err != nil { return err @@ -765,7 +769,7 @@ func (s *TransactionStreamer) AddMessagesAndEndBatch(pos arbutil.MessageIndex, m s.insertionMutex.Lock() defer s.insertionMutex.Unlock() - return s.addMessagesAndEndBatchImpl(pos, messagesAreConfirmed, messages, batch) + return s.addMessagesAndEndBatchImpl(pos, messagesAreConfirmed, messagesWithBlockHash, batch) } func (s *TransactionStreamer) getPrevPrevDelayedRead(pos arbutil.MessageIndex) (uint64, error) { diff --git a/system_tests/contract_tx_test.go b/system_tests/contract_tx_test.go index c4ae326df1..7d66e516b4 100644 --- a/system_tests/contract_tx_test.go +++ b/system_tests/contract_tx_test.go @@ -69,23 +69,21 @@ func TestContractTxDeploy(t *testing.T) { l2Msg = append(l2Msg, arbmath.U256Bytes(contractTx.Value)...) l2Msg = append(l2Msg, contractTx.Data...) - err = builder.L2.ConsensusNode.TxStreamer.AddMessages(pos, true, []arbostypes.MessageWithMetadataAndBlockHash{ + err = builder.L2.ConsensusNode.TxStreamer.AddMessages(pos, true, []arbostypes.MessageWithMetadata{ { - MessageWithMeta: arbostypes.MessageWithMetadata{ - Message: &arbostypes.L1IncomingMessage{ - Header: &arbostypes.L1IncomingMessageHeader{ - Kind: arbostypes.L1MessageType_L2Message, - Poster: from, - BlockNumber: 0, - Timestamp: 0, - RequestId: &contractTx.RequestId, - L1BaseFee: &big.Int{}, - }, - L2msg: l2Msg, - BatchGasCost: new(uint64), + Message: &arbostypes.L1IncomingMessage{ + Header: &arbostypes.L1IncomingMessageHeader{ + Kind: arbostypes.L1MessageType_L2Message, + Poster: from, + BlockNumber: 0, + Timestamp: 0, + RequestId: &contractTx.RequestId, + L1BaseFee: &big.Int{}, }, - DelayedMessagesRead: delayedMessagesRead, + L2msg: l2Msg, + BatchGasCost: new(uint64), }, + DelayedMessagesRead: delayedMessagesRead, }, }) Require(t, err) diff --git a/system_tests/reorg_resequencing_test.go b/system_tests/reorg_resequencing_test.go index 28d1b3bd66..b188504acb 100644 --- a/system_tests/reorg_resequencing_test.go +++ b/system_tests/reorg_resequencing_test.go @@ -72,11 +72,9 @@ func TestReorgResequencing(t *testing.T) { }, L2msg: append(builder.L2Info.GetAddress("User4").Bytes(), arbmath.Uint64ToU256Bytes(params.Ether)...), } - err = builder.L2.ConsensusNode.TxStreamer.AddMessages(startMsgCount, true, []arbostypes.MessageWithMetadataAndBlockHash{{ - MessageWithMeta: arbostypes.MessageWithMetadata{ - Message: newMessage, - DelayedMessagesRead: prevMessage.DelayedMessagesRead + 1, - }, + err = builder.L2.ConsensusNode.TxStreamer.AddMessages(startMsgCount, true, []arbostypes.MessageWithMetadata{{ + Message: newMessage, + DelayedMessagesRead: prevMessage.DelayedMessagesRead + 1, }}) Require(t, err) diff --git a/system_tests/seq_coordinator_test.go b/system_tests/seq_coordinator_test.go index 36c7be7251..886a0528c7 100644 --- a/system_tests/seq_coordinator_test.go +++ b/system_tests/seq_coordinator_test.go @@ -91,10 +91,7 @@ func TestRedisSeqCoordinatorPriorities(t *testing.T) { return false } Require(t, err) - emptyMessageWithBlockHash := arbostypes.MessageWithMetadataAndBlockHash{ - MessageWithMeta: emptyMessage, - } - Require(t, node.TxStreamer.AddMessages(curMsgs, false, []arbostypes.MessageWithMetadataAndBlockHash{emptyMessageWithBlockHash})) + Require(t, node.TxStreamer.AddMessages(curMsgs, false, []arbostypes.MessageWithMetadata{emptyMessage})) return true } From 8671d06f3d69a4a27777ec0370c500601381adac Mon Sep 17 00:00:00 2001 From: Diego Ximenes Date: Wed, 8 May 2024 12:04:22 -0300 Subject: [PATCH 034/113] fix storing and retrieving block hash --- arbnode/transaction_streamer.go | 24 +++++++++++++++++++----- 1 file changed, 19 insertions(+), 5 deletions(-) diff --git a/arbnode/transaction_streamer.go b/arbnode/transaction_streamer.go index 7ff565ec16..0973ca942e 100644 --- a/arbnode/transaction_streamer.go +++ b/arbnode/transaction_streamer.go @@ -536,15 +536,24 @@ func (s *TransactionStreamer) getMessageWithMetadataAndBlockHash(seqNum arbutil. return nil, err } + // get block hash key := dbKey(blockHashInputFeedPrefix, uint64(seqNum)) - data, err := s.db.Get(key) + hasBlockHash, err := s.db.Has(key) if err != nil { return nil, err } var blockHash *common.Hash - err = rlp.DecodeBytes(data, &blockHash) - if err != nil { - return nil, err + if hasBlockHash { + data, err := s.db.Get(key) + if err != nil { + return nil, err + } + var storedBlockHash common.Hash + err = rlp.DecodeBytes(data, &storedBlockHash) + if err != nil { + return nil, err + } + blockHash = &storedBlockHash } msgWithBlockHash := arbostypes.MessageWithMetadataAndBlockHash{ @@ -1087,8 +1096,13 @@ func (s *TransactionStreamer) writeMessage(pos arbutil.MessageIndex, msg arbosty } // write block hash + if msg.BlockHash == nil { + // don't write nil block hash to avoid issues with rlp decoder that + // doesn't produce nil values by default + return nil + } key = dbKey(blockHashInputFeedPrefix, uint64(pos)) - msgBytes, err = rlp.EncodeToBytes(msg.BlockHash) + msgBytes, err = rlp.EncodeToBytes(*msg.BlockHash) if err != nil { return err } From 0fc92af157ceb77d86c0f646d36dd06be3f3105d Mon Sep 17 00:00:00 2001 From: Diego Ximenes Date: Wed, 8 May 2024 12:04:44 -0300 Subject: [PATCH 035/113] check block_hash_mismatch in sequencer system tests --- system_tests/seqfeed_test.go | 85 +++++++++++++++++++++++++++++++++++- 1 file changed, 84 insertions(+), 1 deletion(-) diff --git a/system_tests/seqfeed_test.go b/system_tests/seqfeed_test.go index 749a91e3b1..f9cca03616 100644 --- a/system_tests/seqfeed_test.go +++ b/system_tests/seqfeed_test.go @@ -11,10 +11,16 @@ import ( "testing" "time" + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/log" "github.com/offchainlabs/nitro/arbnode" + "github.com/offchainlabs/nitro/arbos/arbostypes" "github.com/offchainlabs/nitro/broadcastclient" + "github.com/offchainlabs/nitro/broadcaster/backlog" + "github.com/offchainlabs/nitro/broadcaster/message" "github.com/offchainlabs/nitro/relay" "github.com/offchainlabs/nitro/util/signature" + "github.com/offchainlabs/nitro/util/testhelpers" "github.com/offchainlabs/nitro/wsbroadcastserver" ) @@ -38,7 +44,8 @@ func newBroadcastClientConfigTest(port int) *broadcastclient.Config { } func TestSequencerFeed(t *testing.T) { - t.Parallel() + logHandler := testhelpers.InitTestLog(t, log.LvlTrace) + ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -73,6 +80,10 @@ func TestSequencerFeed(t *testing.T) { if l2balance.Cmp(big.NewInt(1e12)) != 0 { t.Fatal("Unexpected balance:", l2balance) } + + if logHandler.WasLogged("block_hash_mismatch") { + t.Fatal("block_hash_mismatch was logged unexpectedly") + } } func TestRelayedSequencerFeed(t *testing.T) { @@ -250,3 +261,75 @@ func TestLyingSequencer(t *testing.T) { func TestLyingSequencerLocalDAS(t *testing.T) { testLyingSequencer(t, "files") } + +func TestBlockHashFeedMismatch(t *testing.T) { + logHandler := testhelpers.InitTestLog(t, log.LvlTrace) + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + backlogConfiFetcher := func() *backlog.Config { + return &backlog.DefaultTestConfig + } + bklg := backlog.NewBacklog(backlogConfiFetcher) + + feedErrChan := make(chan error) + wsBroadcastServer := wsbroadcastserver.NewWSBroadcastServer( + newBroadcasterConfigTest, + bklg, + 412346, + feedErrChan, + ) + err := wsBroadcastServer.Initialize() + if err != nil { + t.Fatal("error initializing wsBroadcastServer:", err) + } + err = wsBroadcastServer.Start(ctx) + if err != nil { + t.Fatal("error starting wsBroadcastServer:", err) + } + defer wsBroadcastServer.StopAndWait() + + port := wsBroadcastServer.ListenerAddr().(*net.TCPAddr).Port + + builder := NewNodeBuilder(ctx).DefaultConfig(t, false) + builder.nodeConfig.Feed.Input = *newBroadcastClientConfigTest(port) + cleanup := builder.Build(t) + defer cleanup() + + poster := common.HexToAddress("0xa4b000000000000000000073657175656e636572") + blockHash := common.HexToHash("0x1111111111111111111111111111111111111111111111111111111111111111") + l2msg := []byte{4, 2, 248, 111, 131, 6, 74, 186, 128, 128, 132, 11, 235, 194, 0, 131, 122, 18, 0, 148, 12, 112, 159, 52, 15, 11, 178, 227, 97, 34, 158, 52, 91, 126, 38, 153, 157, 9, 105, 171, 133, 232, 212, 165, 16, 0, 128, 192, 1, 160, 75, 109, 200, 183, 223, 114, 85, 128, 133, 94, 26, 103, 145, 247, 47, 0, 114, 132, 133, 234, 222, 235, 102, 45, 2, 109, 83, 65, 210, 142, 242, 209, 160, 96, 90, 108, 188, 197, 195, 43, 222, 103, 155, 153, 81, 119, 74, 177, 103, 110, 134, 94, 221, 72, 236, 20, 86, 94, 226, 94, 5, 206, 196, 122, 119} + broadcastMessage := message.BroadcastMessage{ + Version: 1, + Messages: []*message.BroadcastFeedMessage{ + { + // SequenceNumber: 1, + SequenceNumber: 2, + Message: arbostypes.MessageWithMetadata{ + Message: &arbostypes.L1IncomingMessage{ + Header: &arbostypes.L1IncomingMessageHeader{ + Kind: arbostypes.L1MessageType_L2Message, + Poster: poster, + BlockNumber: 29, + Timestamp: 1715136502, + RequestId: nil, + L1BaseFee: big.NewInt(0), + }, + L2msg: l2msg, + }, + DelayedMessagesRead: 1, + }, + BlockHash: &blockHash, + Signature: nil, + }, + }, + } + wsBroadcastServer.Broadcast(&broadcastMessage) + + time.Sleep(time.Second * 2) + + if !logHandler.WasLogged("block_hash_mismatch") { + t.Fatal("Failed to log block_hash_mismatch") + } +} From 202efe1c8b7b8dc19229950d4317e8ca8ba4a1f0 Mon Sep 17 00:00:00 2001 From: Diego Ximenes Date: Wed, 8 May 2024 14:30:58 -0300 Subject: [PATCH 036/113] stores block hash even it is nil, which enables to iterate through range of keys to properly prune a range of block hashes from the db --- arbnode/transaction_streamer.go | 20 ++++++++++++-------- 1 file changed, 12 insertions(+), 8 deletions(-) diff --git a/arbnode/transaction_streamer.go b/arbnode/transaction_streamer.go index 0973ca942e..9e6ae7d92e 100644 --- a/arbnode/transaction_streamer.go +++ b/arbnode/transaction_streamer.go @@ -140,6 +140,12 @@ type L1PriceData struct { currentEstimateOfL1GasPrice uint64 } +// Represents a block's hash in the database. +// Necessary because RLP decoder doesn't produce nil values by default. +type blockHashDBValue struct { + BlockHash *common.Hash `rlp:"nil"` +} + func (s *TransactionStreamer) CurrentEstimateOfL1GasPrice() uint64 { s.cachedL1PriceDataMutex.Lock() defer s.cachedL1PriceDataMutex.Unlock() @@ -548,12 +554,12 @@ func (s *TransactionStreamer) getMessageWithMetadataAndBlockHash(seqNum arbutil. if err != nil { return nil, err } - var storedBlockHash common.Hash - err = rlp.DecodeBytes(data, &storedBlockHash) + var blockHashDBVal blockHashDBValue + err = rlp.DecodeBytes(data, &blockHashDBVal) if err != nil { return nil, err } - blockHash = &storedBlockHash + blockHash = blockHashDBVal.BlockHash } msgWithBlockHash := arbostypes.MessageWithMetadataAndBlockHash{ @@ -1096,13 +1102,11 @@ func (s *TransactionStreamer) writeMessage(pos arbutil.MessageIndex, msg arbosty } // write block hash - if msg.BlockHash == nil { - // don't write nil block hash to avoid issues with rlp decoder that - // doesn't produce nil values by default - return nil + blockHashDBVal := blockHashDBValue{ + BlockHash: msg.BlockHash, } key = dbKey(blockHashInputFeedPrefix, uint64(pos)) - msgBytes, err = rlp.EncodeToBytes(*msg.BlockHash) + msgBytes, err = rlp.EncodeToBytes(blockHashDBVal) if err != nil { return err } From 2a56769f37d138185a7c92dfc9c0802128761037 Mon Sep 17 00:00:00 2001 From: Diego Ximenes Date: Wed, 8 May 2024 15:04:59 -0300 Subject: [PATCH 037/113] add comment on why verifying if block hash key exists in DB before retrieving it --- arbnode/transaction_streamer.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/arbnode/transaction_streamer.go b/arbnode/transaction_streamer.go index 9e6ae7d92e..dc375d642e 100644 --- a/arbnode/transaction_streamer.go +++ b/arbnode/transaction_streamer.go @@ -542,7 +542,9 @@ func (s *TransactionStreamer) getMessageWithMetadataAndBlockHash(seqNum arbutil. return nil, err } - // get block hash + // Get block hash. + // First check if key exists in database so this procedure is backwards compatible + // with databases' snapshots that don't have block hashes stored. key := dbKey(blockHashInputFeedPrefix, uint64(seqNum)) hasBlockHash, err := s.db.Has(key) if err != nil { From b2bb3da34c58597f0045bf077dfdaa7a00571f9e Mon Sep 17 00:00:00 2001 From: Diego Ximenes Date: Wed, 8 May 2024 15:41:35 -0300 Subject: [PATCH 038/113] rm unwanted comment --- system_tests/seqfeed_test.go | 1 - 1 file changed, 1 deletion(-) diff --git a/system_tests/seqfeed_test.go b/system_tests/seqfeed_test.go index f9cca03616..e56514a6d5 100644 --- a/system_tests/seqfeed_test.go +++ b/system_tests/seqfeed_test.go @@ -304,7 +304,6 @@ func TestBlockHashFeedMismatch(t *testing.T) { Version: 1, Messages: []*message.BroadcastFeedMessage{ { - // SequenceNumber: 1, SequenceNumber: 2, Message: arbostypes.MessageWithMetadata{ Message: &arbostypes.L1IncomingMessage{ From 881e2fd34ec60bd2c922d9871c1080c26c7de3f1 Mon Sep 17 00:00:00 2001 From: Diego Ximenes Date: Thu, 9 May 2024 16:23:56 -0300 Subject: [PATCH 039/113] Test that the output feed of a node that isn't the sequencer is properly processed --- system_tests/seq_coordinator_test.go | 31 ++++++++++++++++++++++++++-- 1 file changed, 29 insertions(+), 2 deletions(-) diff --git a/system_tests/seq_coordinator_test.go b/system_tests/seq_coordinator_test.go index 886a0528c7..a069a2d5a1 100644 --- a/system_tests/seq_coordinator_test.go +++ b/system_tests/seq_coordinator_test.go @@ -8,12 +8,14 @@ import ( "errors" "fmt" "math/big" + "net" "testing" "time" "github.com/go-redis/redis/v8" "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/log" "github.com/offchainlabs/nitro/arbnode" "github.com/offchainlabs/nitro/arbos/arbostypes" @@ -21,6 +23,7 @@ import ( "github.com/offchainlabs/nitro/execution" "github.com/offchainlabs/nitro/execution/gethexec" "github.com/offchainlabs/nitro/util/redisutil" + "github.com/offchainlabs/nitro/util/testhelpers" ) func initRedisForTest(t *testing.T, ctx context.Context, redisUrl string, nodeNames []string) { @@ -270,6 +273,8 @@ func TestRedisSeqCoordinatorPriorities(t *testing.T) { } func testCoordinatorMessageSync(t *testing.T, successCase bool) { + logHandler := testhelpers.InitTestLog(t, log.LvlTrace) + ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -304,16 +309,25 @@ func testCoordinatorMessageSync(t *testing.T, successCase bool) { nodeConfigDup := *builder.nodeConfig builder.nodeConfig = &nodeConfigDup - + builder.nodeConfig.Feed.Output = *newBroadcasterConfigTest() builder.nodeConfig.SeqCoordinator.MyUrl = nodeNames[1] if !successCase { builder.nodeConfig.SeqCoordinator.Signer.ECDSA.AcceptSequencer = false builder.nodeConfig.SeqCoordinator.Signer.ECDSA.AllowedAddresses = []string{builder.L2Info.GetAddress("User2").Hex()} } - testClientB, cleanupB := builder.Build2ndNode(t, &SecondNodeParams{nodeConfig: builder.nodeConfig}) defer cleanupB() + // Build nodeBOutputFeedReader. + // nodeB doesn't sequence transactions, but adds messages related to them to its output feed. + // nodeBOutputFeedReader reads those messages from this feed and processes them. + // nodeBOutputFeedReader doesn't read messages from L1 since none of the nodes posts to L1. + nodeBPort := testClientB.ConsensusNode.BroadcastServer.ListenerAddr().(*net.TCPAddr).Port + nodeConfigNodeBOutputFeedReader := arbnode.ConfigDefaultL1NonSequencerTest() + nodeConfigNodeBOutputFeedReader.Feed.Input = *newBroadcastClientConfigTest(nodeBPort) + testClientNodeBOutputFeedReader, cleanupNodeBOutputFeedReader := builder.Build2ndNode(t, &SecondNodeParams{nodeConfig: nodeConfigNodeBOutputFeedReader}) + defer cleanupNodeBOutputFeedReader() + tx := builder.L2Info.PrepareTx("Owner", "User2", builder.L2Info.TransferGas, big.NewInt(1e12), nil) err = builder.L2.Client.SendTransaction(ctx, tx) @@ -330,6 +344,19 @@ func testCoordinatorMessageSync(t *testing.T, successCase bool) { if l2balance.Cmp(big.NewInt(1e12)) != 0 { t.Fatal("Unexpected balance:", l2balance) } + + // check that nodeBOutputFeedReader also processed the transaction + _, err = WaitForTx(ctx, testClientNodeBOutputFeedReader.Client, tx.Hash(), time.Second*5) + Require(t, err) + l2balance, err = testClientNodeBOutputFeedReader.Client.BalanceAt(ctx, builder.L2Info.GetAddress("User2"), nil) + Require(t, err) + if l2balance.Cmp(big.NewInt(1e12)) != 0 { + t.Fatal("Unexpected balance:", l2balance) + } + + if logHandler.WasLogged("block_hash_mismatch") { + t.Fatal("block_hash_mismatch was logged unexpectedly") + } } else { _, err = WaitForTx(ctx, testClientB.Client, tx.Hash(), time.Second) if err == nil { From 0df0c201ce107e666e15f744f6c3d604182c3fb8 Mon Sep 17 00:00:00 2001 From: Diego Ximenes Date: Thu, 9 May 2024 19:16:48 -0300 Subject: [PATCH 040/113] improve get block hash from db --- arbnode/transaction_streamer.go | 23 +++++++++++------------ 1 file changed, 11 insertions(+), 12 deletions(-) diff --git a/arbnode/transaction_streamer.go b/arbnode/transaction_streamer.go index dc375d642e..d9c7fc2163 100644 --- a/arbnode/transaction_streamer.go +++ b/arbnode/transaction_streamer.go @@ -520,6 +520,10 @@ func dbKey(prefix []byte, pos uint64) []byte { return key } +func isErrNotFound(err error) bool { + return errors.Is(err, leveldb.ErrNotFound) || errors.Is(err, pebble.ErrNotFound) +} + // Note: if changed to acquire the mutex, some internal users may need to be updated to a non-locking version. func (s *TransactionStreamer) GetMessage(seqNum arbutil.MessageIndex) (*arbostypes.MessageWithMetadata, error) { key := dbKey(messagePrefix, uint64(seqNum)) @@ -543,25 +547,20 @@ func (s *TransactionStreamer) getMessageWithMetadataAndBlockHash(seqNum arbutil. } // Get block hash. - // First check if key exists in database so this procedure is backwards compatible - // with databases' snapshots that don't have block hashes stored. + // To keep it backwards compatible it is possible that a message related + // to a sequence number exists in the database but the block hash doesn't. key := dbKey(blockHashInputFeedPrefix, uint64(seqNum)) - hasBlockHash, err := s.db.Has(key) - if err != nil { - return nil, err - } var blockHash *common.Hash - if hasBlockHash { - data, err := s.db.Get(key) - if err != nil { - return nil, err - } + data, err := s.db.Get(key) + if err == nil { var blockHashDBVal blockHashDBValue err = rlp.DecodeBytes(data, &blockHashDBVal) if err != nil { return nil, err } blockHash = blockHashDBVal.BlockHash + } else if !isErrNotFound(err) { + return nil, err } msgWithBlockHash := arbostypes.MessageWithMetadataAndBlockHash{ @@ -706,7 +705,7 @@ func (s *TransactionStreamer) AddBroadcastMessages(feedMessages []*m.BroadcastFe if broadcastStartPos > 0 { _, err := s.GetMessage(broadcastStartPos - 1) if err != nil { - if !errors.Is(err, leveldb.ErrNotFound) && !errors.Is(err, pebble.ErrNotFound) { + if !isErrNotFound(err) { return err } // Message before current message doesn't exist in database, so don't add current messages yet From 61c7d376bcbf88bd8694d8383a4281cacecc5400 Mon Sep 17 00:00:00 2001 From: Diego Ximenes Date: Thu, 9 May 2024 20:24:10 -0300 Subject: [PATCH 041/113] improve TestBlockHashFeedMismatch --- system_tests/seqfeed_test.go | 31 ++++++++++++++++++++++++------- 1 file changed, 24 insertions(+), 7 deletions(-) diff --git a/system_tests/seqfeed_test.go b/system_tests/seqfeed_test.go index e56514a6d5..2aa64a801d 100644 --- a/system_tests/seqfeed_test.go +++ b/system_tests/seqfeed_test.go @@ -273,12 +273,11 @@ func TestBlockHashFeedMismatch(t *testing.T) { } bklg := backlog.NewBacklog(backlogConfiFetcher) - feedErrChan := make(chan error) wsBroadcastServer := wsbroadcastserver.NewWSBroadcastServer( newBroadcasterConfigTest, bklg, 412346, - feedErrChan, + nil, ) err := wsBroadcastServer.Initialize() if err != nil { @@ -292,11 +291,16 @@ func TestBlockHashFeedMismatch(t *testing.T) { port := wsBroadcastServer.ListenerAddr().(*net.TCPAddr).Port - builder := NewNodeBuilder(ctx).DefaultConfig(t, false) + builder := NewNodeBuilder(ctx).DefaultConfig(t, true) builder.nodeConfig.Feed.Input = *newBroadcastClientConfigTest(port) cleanup := builder.Build(t) defer cleanup() + testClient := builder.L2 + // related to: + // - builder.L2Info.PrepareTx("Owner", "User2", builder.L2Info.TransferGas, big.NewInt(1e12), nil) + userAccount := "User2" + txHash := common.HexToHash("0x633f62b463cc0e52d842406995fb590654db40aace77bfca863ba0e8d2290f97") poster := common.HexToAddress("0xa4b000000000000000000073657175656e636572") blockHash := common.HexToHash("0x1111111111111111111111111111111111111111111111111111111111111111") l2msg := []byte{4, 2, 248, 111, 131, 6, 74, 186, 128, 128, 132, 11, 235, 194, 0, 131, 122, 18, 0, 148, 12, 112, 159, 52, 15, 11, 178, 227, 97, 34, 158, 52, 91, 126, 38, 153, 157, 9, 105, 171, 133, 232, 212, 165, 16, 0, 128, 192, 1, 160, 75, 109, 200, 183, 223, 114, 85, 128, 133, 94, 26, 103, 145, 247, 47, 0, 114, 132, 133, 234, 222, 235, 102, 45, 2, 109, 83, 65, 210, 142, 242, 209, 160, 96, 90, 108, 188, 197, 195, 43, 222, 103, 155, 153, 81, 119, 74, 177, 103, 110, 134, 94, 221, 72, 236, 20, 86, 94, 226, 94, 5, 206, 196, 122, 119} @@ -304,16 +308,16 @@ func TestBlockHashFeedMismatch(t *testing.T) { Version: 1, Messages: []*message.BroadcastFeedMessage{ { - SequenceNumber: 2, + SequenceNumber: 1, Message: arbostypes.MessageWithMetadata{ Message: &arbostypes.L1IncomingMessage{ Header: &arbostypes.L1IncomingMessageHeader{ Kind: arbostypes.L1MessageType_L2Message, Poster: poster, BlockNumber: 29, - Timestamp: 1715136502, + Timestamp: 1715295980, RequestId: nil, - L1BaseFee: big.NewInt(0), + L1BaseFee: nil, }, L2msg: l2msg, }, @@ -326,8 +330,21 @@ func TestBlockHashFeedMismatch(t *testing.T) { } wsBroadcastServer.Broadcast(&broadcastMessage) - time.Sleep(time.Second * 2) + // By now, even though block hash mismatch, the transaction should still be processed + builder.L2Info.GenerateAccount(userAccount) + _, err = WaitForTx(ctx, testClient.Client, txHash, time.Second*15) + if err != nil { + t.Fatal("error waiting for tx:", err) + } + l2balance, err := testClient.Client.BalanceAt(ctx, builder.L2Info.GetAddress(userAccount), nil) + if err != nil { + t.Fatal("error getting balance:", err) + } + if l2balance.Cmp(big.NewInt(1e12)) != 0 { + t.Fatal("Unexpected balance:", l2balance) + } + // check that block hash mismatch if !logHandler.WasLogged("block_hash_mismatch") { t.Fatal("Failed to log block_hash_mismatch") } From 70818c5f559a0acfbc87f6d08920090df351dff9 Mon Sep 17 00:00:00 2001 From: Diego Ximenes Date: Thu, 9 May 2024 20:31:49 -0300 Subject: [PATCH 042/113] add TestBlockHashFeedNil --- system_tests/seqfeed_test.go | 20 +++++++++++++++----- 1 file changed, 15 insertions(+), 5 deletions(-) diff --git a/system_tests/seqfeed_test.go b/system_tests/seqfeed_test.go index 2aa64a801d..946194f17d 100644 --- a/system_tests/seqfeed_test.go +++ b/system_tests/seqfeed_test.go @@ -262,7 +262,7 @@ func TestLyingSequencerLocalDAS(t *testing.T) { testLyingSequencer(t, "files") } -func TestBlockHashFeedMismatch(t *testing.T) { +func testBlockHashComparison(t *testing.T, blockHash *common.Hash, mustMismatch bool) { logHandler := testhelpers.InitTestLog(t, log.LvlTrace) ctx, cancel := context.WithCancel(context.Background()) @@ -302,7 +302,6 @@ func TestBlockHashFeedMismatch(t *testing.T) { userAccount := "User2" txHash := common.HexToHash("0x633f62b463cc0e52d842406995fb590654db40aace77bfca863ba0e8d2290f97") poster := common.HexToAddress("0xa4b000000000000000000073657175656e636572") - blockHash := common.HexToHash("0x1111111111111111111111111111111111111111111111111111111111111111") l2msg := []byte{4, 2, 248, 111, 131, 6, 74, 186, 128, 128, 132, 11, 235, 194, 0, 131, 122, 18, 0, 148, 12, 112, 159, 52, 15, 11, 178, 227, 97, 34, 158, 52, 91, 126, 38, 153, 157, 9, 105, 171, 133, 232, 212, 165, 16, 0, 128, 192, 1, 160, 75, 109, 200, 183, 223, 114, 85, 128, 133, 94, 26, 103, 145, 247, 47, 0, 114, 132, 133, 234, 222, 235, 102, 45, 2, 109, 83, 65, 210, 142, 242, 209, 160, 96, 90, 108, 188, 197, 195, 43, 222, 103, 155, 153, 81, 119, 74, 177, 103, 110, 134, 94, 221, 72, 236, 20, 86, 94, 226, 94, 5, 206, 196, 122, 119} broadcastMessage := message.BroadcastMessage{ Version: 1, @@ -323,7 +322,7 @@ func TestBlockHashFeedMismatch(t *testing.T) { }, DelayedMessagesRead: 1, }, - BlockHash: &blockHash, + BlockHash: blockHash, Signature: nil, }, }, @@ -344,8 +343,19 @@ func TestBlockHashFeedMismatch(t *testing.T) { t.Fatal("Unexpected balance:", l2balance) } - // check that block hash mismatch - if !logHandler.WasLogged("block_hash_mismatch") { + mismatched := logHandler.WasLogged("block_hash_mismatch") + if mustMismatch && !mismatched { t.Fatal("Failed to log block_hash_mismatch") + } else if !mustMismatch && mismatched { + t.Fatal("block_hash_mismatch was logged unexpectedly") } } + +func TestBlockHashFeedMismatch(t *testing.T) { + blockHash := common.HexToHash("0x1111111111111111111111111111111111111111111111111111111111111111") + testBlockHashComparison(t, &blockHash, true) +} + +func TestBlockHashFeedNil(t *testing.T) { + testBlockHashComparison(t, nil, false) +} From 8326613d0a2365b52fe96d23b450df8a20237505 Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Fri, 10 May 2024 10:33:07 -0600 Subject: [PATCH 043/113] wrap l2chaindata with wasm --- cmd/nitro/init.go | 20 +++++++++++++++----- system_tests/common_test.go | 11 +++++++++-- 2 files changed, 24 insertions(+), 7 deletions(-) diff --git a/cmd/nitro/init.go b/cmd/nitro/init.go index 750bf03516..e9d6a5d3fa 100644 --- a/cmd/nitro/init.go +++ b/cmd/nitro/init.go @@ -172,16 +172,21 @@ func validateBlockChain(blockChain *core.BlockChain, chainConfig *params.ChainCo func openInitializeChainDb(ctx context.Context, stack *node.Node, config *NodeConfig, chainId *big.Int, cacheConfig *core.CacheConfig, l1Client arbutil.L1Interface, rollupAddrs chaininfo.RollupAddresses) (ethdb.Database, *core.BlockChain, error) { if !config.Init.Force { - if readOnlyDb, err := stack.OpenDatabaseWithFreezerAndWasm("l2chaindata", "wasm", 0, 0, "", "l2chaindata/", true); err == nil { + if readOnlyDb, err := stack.OpenDatabaseWithFreezer("l2chaindata", 0, 0, "", "l2chaindata/", true); err == nil { if chainConfig := gethexec.TryReadStoredChainConfig(readOnlyDb); chainConfig != nil { readOnlyDb.Close() if !arbmath.BigEquals(chainConfig.ChainID, chainId) { return nil, nil, fmt.Errorf("database has chain ID %v but config has chain ID %v (are you sure this database is for the right chain?)", chainConfig.ChainID, chainId) } - chainDb, err := stack.OpenDatabaseWithFreezerAndWasm("l2chaindata", "wasm", config.Execution.Caching.DatabaseCache, config.Persistent.Handles, config.Persistent.Ancient, "l2chaindata/", false) + chainData, err := stack.OpenDatabaseWithFreezer("l2chaindata", config.Execution.Caching.DatabaseCache, config.Persistent.Handles, config.Persistent.Ancient, "l2chaindata/", false) if err != nil { - return chainDb, nil, err + return nil, nil, err + } + wasmDb, err := stack.OpenDatabase("wasm", config.Execution.Caching.DatabaseCache, config.Persistent.Handles, "wasm/", false) + if err != nil { + return nil, nil, err } + chainDb := stack.WrapDatabaseWithWasm(chainData, wasmDb) err = pruning.PruneChainDb(ctx, chainDb, stack, &config.Init, cacheConfig, l1Client, rollupAddrs, config.Node.ValidatorRequired()) if err != nil { return chainDb, nil, fmt.Errorf("error pruning: %w", err) @@ -230,10 +235,15 @@ func openInitializeChainDb(ctx context.Context, stack *node.Node, config *NodeCo var initDataReader statetransfer.InitDataReader = nil - chainDb, err := stack.OpenDatabaseWithFreezerAndWasm("l2chaindata", "wasm", config.Execution.Caching.DatabaseCache, config.Persistent.Handles, config.Persistent.Ancient, "l2chaindata/", false) + chainData, err := stack.OpenDatabaseWithFreezer("l2chaindata", config.Execution.Caching.DatabaseCache, config.Persistent.Handles, config.Persistent.Ancient, "l2chaindata/", false) if err != nil { - return chainDb, nil, err + return nil, nil, err + } + wasmDb, err := stack.OpenDatabase("wasm", config.Execution.Caching.DatabaseCache, config.Persistent.Handles, "wasm/", false) + if err != nil { + return nil, nil, err } + chainDb := stack.WrapDatabaseWithWasm(chainData, wasmDb) if config.Init.ImportFile != "" { initDataReader, err = statetransfer.NewJsonInitDataReader(config.Init.ImportFile) diff --git a/system_tests/common_test.go b/system_tests/common_test.go index 9d461bd48c..0ffe3990c3 100644 --- a/system_tests/common_test.go +++ b/system_tests/common_test.go @@ -772,8 +772,11 @@ func createL2BlockChainWithStackConfig( stack, err = node.New(stackConfig) Require(t, err) - chainDb, err := stack.OpenDatabaseWithFreezerAndWasm("l2chaindata", "wasm", 0, 0, "ancient", "l2chaindata/", false) + chainData, err := stack.OpenDatabase("l2chaindata", 0, 0, "l2chaindata/", false) Require(t, err) + wasmData, err := stack.OpenDatabase("wasm", 0, 0, "wasm/", false) + Require(t, err) + chainDb := stack.WrapDatabaseWithWasm(chainData, wasmData) arbDb, err := stack.OpenDatabase("arbitrumdata", 0, 0, "arbitrumdata/", false) Require(t, err) @@ -976,8 +979,12 @@ func Create2ndNodeWithConfig( l2stack, err := node.New(stackConfig) Require(t, err) - l2chainDb, err := l2stack.OpenDatabaseWithFreezerAndWasm("l2chaindata", "wasm", 0, 0, "", "l2chaindata/", false) + l2chainData, err := l2stack.OpenDatabase("l2chaindata", 0, 0, "l2chaindata/", false) + Require(t, err) + wasmData, err := l2stack.OpenDatabase("wasm", 0, 0, "wasm/", false) Require(t, err) + l2chainDb := l2stack.WrapDatabaseWithWasm(l2chainData, wasmData) + l2arbDb, err := l2stack.OpenDatabase("arbitrumdata", 0, 0, "arbitrumdata/", false) Require(t, err) initReader := statetransfer.NewMemoryInitDataReader(l2InitData) From cfd24dcdd83b00610d107fe77234c08247a58ba0 Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Fri, 10 May 2024 10:33:50 -0600 Subject: [PATCH 044/113] geth: update pin --- go-ethereum | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/go-ethereum b/go-ethereum index 37b6489382..45145108b1 160000 --- a/go-ethereum +++ b/go-ethereum @@ -1 +1 @@ -Subproject commit 37b6489382bb884dd1216dcb0f6a224ce2ca5fe2 +Subproject commit 45145108b11e39ea57ee5b0a1238440edd1fa3ab From 1e153dd9c0b5171f058984c696a1f9ce26f8f8ad Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Fri, 10 May 2024 11:53:53 -0600 Subject: [PATCH 045/113] program: compilation fix --- arbos/programs/native.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/arbos/programs/native.go b/arbos/programs/native.go index 4a3b8992cf..1111dc7233 100644 --- a/arbos/programs/native.go +++ b/arbos/programs/native.go @@ -176,7 +176,7 @@ func callProgram( db := interpreter.Evm().StateDB debug := stylusParams.debugMode - if len(asm) == 0 { + if len(localAsm) == 0 { log.Error("missing asm", "program", address, "module", moduleHash) panic("missing asm") } From 566e72cfdd42c9111bf43ac0d9c424f41c27c087 Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Fri, 10 May 2024 15:23:50 -0600 Subject: [PATCH 046/113] cache program could also recalc asm --- arbos/programs/native.go | 12 ++++++++---- arbos/programs/programs.go | 10 +++++++--- arbos/programs/wasm.go | 2 +- 3 files changed, 16 insertions(+), 8 deletions(-) diff --git a/arbos/programs/native.go b/arbos/programs/native.go index 1111dc7233..18f6e8e48d 100644 --- a/arbos/programs/native.go +++ b/arbos/programs/native.go @@ -221,11 +221,15 @@ func handleReqImpl(apiId usize, req_type u32, data *rustSlice, costPtr *u64, out // Caches a program in Rust. We write a record so that we can undo on revert. // For gas estimation and eth_call, we ignore permanent updates and rely on Rust's LRU. -func cacheProgram(db vm.StateDB, module common.Hash, version uint16, debug bool, runMode core.MessageRunMode) { +func cacheProgram(db vm.StateDB, module common.Hash, program Program, params *StylusParams, debug bool, time uint64, runMode core.MessageRunMode) { if runMode == core.MessageCommitMode { - asm := db.GetActivatedAsm(module) - state.CacheWasmRust(asm, module, version, debug) - db.RecordCacheWasm(state.CacheWasm{ModuleHash: module, Version: version, Debug: debug}) + // address is only used for logging + asm, err := getLocalAsm(db, module, common.Address{}, params.PageLimit, time, debug, program) + if err != nil { + panic("unable to recreate wasm") + } + state.CacheWasmRust(asm, module, program.version, debug) + db.RecordCacheWasm(state.CacheWasm{ModuleHash: module, Version: program.version, Debug: debug}) } } diff --git a/arbos/programs/programs.go b/arbos/programs/programs.go index b277f5d678..8395980f68 100644 --- a/arbos/programs/programs.go +++ b/arbos/programs/programs.go @@ -120,14 +120,13 @@ func (p Programs) ActivateProgram(evm *vm.EVM, address common.Address, runMode c return 0, codeHash, common.Hash{}, nil, true, err } - // replace the cached asm + // remove prev asm if cached { oldModuleHash, err := p.moduleHashes.Get(codeHash) if err != nil { return 0, codeHash, common.Hash{}, nil, true, err } evictProgram(statedb, oldModuleHash, currentVersion, debugMode, runMode, expired) - cacheProgram(statedb, info.moduleHash, stylusVersion, debugMode, runMode) } if err := p.moduleHashes.Set(codeHash, info.moduleHash); err != nil { return 0, codeHash, common.Hash{}, nil, true, err @@ -152,6 +151,11 @@ func (p Programs) ActivateProgram(evm *vm.EVM, address common.Address, runMode c activatedAt: hoursSinceArbitrum(time), cached: cached, } + // replace the cached asm + if cached { + cacheProgram(statedb, info.moduleHash, programData, params, debugMode, time, runMode) + } + return stylusVersion, codeHash, info.moduleHash, dataFee, false, p.setProgram(codeHash, programData) } @@ -386,7 +390,7 @@ func (p Programs) SetProgramCached( return err } if cache { - cacheProgram(db, moduleHash, program.version, debug, runMode) + cacheProgram(db, moduleHash, program, params, debug, time, runMode) } else { evictProgram(db, moduleHash, program.version, debug, runMode, expired) } diff --git a/arbos/programs/wasm.go b/arbos/programs/wasm.go index 8a67babc16..ee8da44934 100644 --- a/arbos/programs/wasm.go +++ b/arbos/programs/wasm.go @@ -95,7 +95,7 @@ func activateProgram( } // stub any non-consensus, Rust-side caching updates -func cacheProgram(db vm.StateDB, module common.Hash, version uint16, debug bool, mode core.MessageRunMode) { +func cacheProgram(db vm.StateDB, module common.Hash, program Program, params *StylusParams, debug bool, time uint64, runMode core.MessageRunMode) { } func evictProgram(db vm.StateDB, module common.Hash, version uint16, debug bool, mode core.MessageRunMode, forever bool) { } From 97db9c98fe5d72ba7064310c4ccfa2191cc082be Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Fri, 10 May 2024 15:24:35 -0600 Subject: [PATCH 047/113] geth: update pin for TryGetActivated --- go-ethereum | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/go-ethereum b/go-ethereum index 5ac25eda5d..8d94d2b164 160000 --- a/go-ethereum +++ b/go-ethereum @@ -1 +1 @@ -Subproject commit 5ac25eda5df447c061e33d74a6a85b03f0a7d85d +Subproject commit 8d94d2b164ea3b277a1e87dcdc6f61c5beeaefd8 From de022e8655ada2eaa6ab2f50b12e728ade4ecdb3 Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Fri, 10 May 2024 16:20:02 -0600 Subject: [PATCH 048/113] dbWithWasm moved --- cmd/nitro/init.go | 4 ++-- system_tests/common_test.go | 5 +++-- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/cmd/nitro/init.go b/cmd/nitro/init.go index e9d6a5d3fa..c52c87732c 100644 --- a/cmd/nitro/init.go +++ b/cmd/nitro/init.go @@ -186,7 +186,7 @@ func openInitializeChainDb(ctx context.Context, stack *node.Node, config *NodeCo if err != nil { return nil, nil, err } - chainDb := stack.WrapDatabaseWithWasm(chainData, wasmDb) + chainDb := rawdb.WrapDatabaseWithWasm(chainData, wasmDb) err = pruning.PruneChainDb(ctx, chainDb, stack, &config.Init, cacheConfig, l1Client, rollupAddrs, config.Node.ValidatorRequired()) if err != nil { return chainDb, nil, fmt.Errorf("error pruning: %w", err) @@ -243,7 +243,7 @@ func openInitializeChainDb(ctx context.Context, stack *node.Node, config *NodeCo if err != nil { return nil, nil, err } - chainDb := stack.WrapDatabaseWithWasm(chainData, wasmDb) + chainDb := rawdb.WrapDatabaseWithWasm(chainData, wasmDb) if config.Init.ImportFile != "" { initDataReader, err = statetransfer.NewJsonInitDataReader(config.Init.ImportFile) diff --git a/system_tests/common_test.go b/system_tests/common_test.go index 3e5f6fd228..f6bfde2108 100644 --- a/system_tests/common_test.go +++ b/system_tests/common_test.go @@ -44,6 +44,7 @@ import ( "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common/math" "github.com/ethereum/go-ethereum/core" + "github.com/ethereum/go-ethereum/core/rawdb" "github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/core/vm" "github.com/ethereum/go-ethereum/crypto" @@ -776,7 +777,7 @@ func createL2BlockChainWithStackConfig( Require(t, err) wasmData, err := stack.OpenDatabase("wasm", 0, 0, "wasm/", false) Require(t, err) - chainDb := stack.WrapDatabaseWithWasm(chainData, wasmData) + chainDb := rawdb.WrapDatabaseWithWasm(chainData, wasmData) arbDb, err := stack.OpenDatabase("arbitrumdata", 0, 0, "arbitrumdata/", false) Require(t, err) @@ -983,7 +984,7 @@ func Create2ndNodeWithConfig( Require(t, err) wasmData, err := l2stack.OpenDatabase("wasm", 0, 0, "wasm/", false) Require(t, err) - l2chainDb := l2stack.WrapDatabaseWithWasm(l2chainData, wasmData) + l2chainDb := rawdb.WrapDatabaseWithWasm(l2chainData, wasmData) l2arbDb, err := l2stack.OpenDatabase("arbitrumdata", 0, 0, "arbitrumdata/", false) Require(t, err) From 90697dcab330c509ae3505a5597bb40d06cf9ce4 Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Fri, 10 May 2024 16:20:26 -0600 Subject: [PATCH 049/113] geth: update pin, move dbWithWasm --- go-ethereum | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/go-ethereum b/go-ethereum index 8d94d2b164..f8917436fc 160000 --- a/go-ethereum +++ b/go-ethereum @@ -1 +1 @@ -Subproject commit 8d94d2b164ea3b277a1e87dcdc6f61c5beeaefd8 +Subproject commit f8917436fcfa6a6a2b15c0ec7e6f318687491a8c From fcd2370d2295412d0e8f0361cdaa143a84488c48 Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Fri, 10 May 2024 18:38:55 -0600 Subject: [PATCH 050/113] test-cases/user.wat: add call to storage_load_bytes --- arbitrator/prover/test-cases/dynamic.wat | 3 +-- arbitrator/prover/test-cases/link.wat | 2 +- arbitrator/prover/test-cases/user.wat | 13 +++++++++++++ 3 files changed, 15 insertions(+), 3 deletions(-) diff --git a/arbitrator/prover/test-cases/dynamic.wat b/arbitrator/prover/test-cases/dynamic.wat index 97c55ba80b..8771bde87c 100644 --- a/arbitrator/prover/test-cases/dynamic.wat +++ b/arbitrator/prover/test-cases/dynamic.wat @@ -12,8 +12,7 @@ ;; WAVM Module hash (data (i32.const 0x000) - "\87\12\6b\19\8a\ce\0c\ba\00\6a\ab\9b\b7\45\bb\0a\ac\48\4d\6b\b8\b5\f9\03\a2\99\8f\64\00\9f\e2\04") ;; user - + "\a1\49\cf\81\13\ff\9c\95\f2\c8\c2\a1\42\35\75\36\7d\e8\6d\d4\22\d8\71\14\bb\9e\a4\7b\af\53\5d\d7") ;; user (func $start (local $user i32) (local $internals i32) ;; link in user.wat i32.const 0 diff --git a/arbitrator/prover/test-cases/link.wat b/arbitrator/prover/test-cases/link.wat index e033bf0e98..ef15326481 100644 --- a/arbitrator/prover/test-cases/link.wat +++ b/arbitrator/prover/test-cases/link.wat @@ -30,7 +30,7 @@ (data (i32.const 0x140) "\47\f7\4f\9c\21\51\4f\52\24\ea\d3\37\5c\bf\a9\1b\1a\5f\ef\22\a5\2a\60\30\c5\52\18\90\6b\b1\51\e5") ;; iops (data (i32.const 0x160) - "\87\12\6b\19\8a\ce\0c\ba\00\6a\ab\9b\b7\45\bb\0a\ac\48\4d\6b\b8\b5\f9\03\a2\99\8f\64\00\9f\e2\04") ;; user + "\a1\49\cf\81\13\ff\9c\95\f2\c8\c2\a1\42\35\75\36\7d\e8\6d\d4\22\d8\71\14\bb\9e\a4\7b\af\53\5d\d7") ;; user (data (i32.const 0x180) "\ee\47\08\f6\47\b2\10\88\1f\89\86\e7\e3\79\6b\b2\77\43\f1\4e\ee\cf\45\4a\9b\7c\d7\c4\5b\63\b6\d7") ;; return diff --git a/arbitrator/prover/test-cases/user.wat b/arbitrator/prover/test-cases/user.wat index d159339f66..9ecb4dcc45 100644 --- a/arbitrator/prover/test-cases/user.wat +++ b/arbitrator/prover/test-cases/user.wat @@ -2,6 +2,14 @@ ;; For license information, see https://github.com/OffchainLabs/nitro/blob/master/LICENSE (module + (import "vm_hooks" "storage_load_bytes32" (func $storage_load_bytes32 (param i32 i32))) + + (func $storage_load (result i32) + i32.const 0 + i32.const 32 + call $storage_load_bytes32 + i32.const 0 + ) (func $safe (result i32) i32.const 5 ) @@ -35,6 +43,11 @@ (then (call $out_of_bounds) (return)) ) + (i32.eq (local.get $args_len) (i32.const 32)) + (if + (then (call $storage_load) (return)) + ) + unreachable ) (memory (export "memory") 1 1)) From 5dd329a9da4f9716b88234a36fc76a0ef8388b5a Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Fri, 10 May 2024 18:40:09 -0600 Subject: [PATCH 051/113] prover backoff by data for SwitchThread there is an important difference between 0 and non-zero (and only two non-zero values used) --- arbitrator/prover/src/main.rs | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/arbitrator/prover/src/main.rs b/arbitrator/prover/src/main.rs index 697d178fc7..9ddd5020c8 100644 --- a/arbitrator/prover/src/main.rs +++ b/arbitrator/prover/src/main.rs @@ -263,7 +263,10 @@ fn main() -> Result<()> { if opts.proving_backoff { let mut extra_data = 0; - if matches!(next_opcode, Opcode::ReadInboxMessage | Opcode::ReadPreImage) { + if matches!( + next_opcode, + Opcode::ReadInboxMessage | Opcode::ReadPreImage | Opcode::SwitchThread + ) { extra_data = next_inst.argument_data; } let count_entry = proving_backoff From df5dcb53905eeeccf1d97ddcb86db3ec2b97185a Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Fri, 10 May 2024 18:41:31 -0600 Subject: [PATCH 052/113] arbos: make more types and lgic public, to be used by tests --- arbos/programs/native.go | 16 ++++++++-------- arbos/programs/programs.go | 28 ++++++++++++++-------------- arbos/programs/wasm.go | 25 ++++++++++++++++++------- arbos/programs/wasm_api.go | 8 ++++---- 4 files changed, 44 insertions(+), 33 deletions(-) diff --git a/arbos/programs/native.go b/arbos/programs/native.go index 09989f3380..c44f8f56cb 100644 --- a/arbos/programs/native.go +++ b/arbos/programs/native.go @@ -106,13 +106,13 @@ func callProgram( interpreter *vm.EVMInterpreter, tracingInfo *util.TracingInfo, calldata []byte, - evmData *evmData, - stylusParams *goParams, + evmData *EvmData, + stylusParams *ProgParams, memoryModel *MemoryModel, ) ([]byte, error) { db := interpreter.Evm().StateDB asm := db.GetActivatedAsm(moduleHash) - debug := stylusParams.debugMode + debug := stylusParams.DebugMode if len(asm) == 0 { log.Error("missing asm", "program", address, "module", moduleHash) @@ -236,18 +236,18 @@ func goSlice(slice []byte) C.GoSliceData { } } -func (params *goParams) encode() C.StylusConfig { +func (params *ProgParams) encode() C.StylusConfig { pricing := C.PricingParams{ - ink_price: u32(params.inkPrice.ToUint32()), + ink_price: u32(params.InkPrice.ToUint32()), } return C.StylusConfig{ - version: u16(params.version), - max_depth: u32(params.maxDepth), + version: u16(params.Version), + max_depth: u32(params.MaxDepth), pricing: pricing, } } -func (data *evmData) encode() C.EvmData { +func (data *EvmData) encode() C.EvmData { return C.EvmData{ block_basefee: hashToBytes32(data.blockBasefee), chainid: u64(data.chainId), diff --git a/arbos/programs/programs.go b/arbos/programs/programs.go index 779f2d6c67..3f7bdc39ca 100644 --- a/arbos/programs/programs.go +++ b/arbos/programs/programs.go @@ -181,7 +181,7 @@ func (p Programs) CallProgram( if err != nil { return nil, err } - goParams := p.goParams(program.version, debugMode, params) + goParams := p.progParams(program.version, debugMode, params) l1BlockNumber, err := evm.ProcessingHook.L1BlockNumber(evm.Context) if err != nil { return nil, err @@ -205,7 +205,7 @@ func (p Programs) CallProgram( statedb.AddStylusPages(program.footprint) defer statedb.SetStylusPagesOpen(open) - evmData := &evmData{ + evmData := &EvmData{ blockBasefee: common.BigToHash(evm.Context.BaseFee), chainId: evm.ChainConfig().ChainID.Uint64(), blockCoinbase: evm.Context.Coinbase, @@ -444,23 +444,23 @@ func (p Program) cachedGas(params *StylusParams) uint64 { return am.SaturatingUAdd(base, am.DivCeil(dyno, 100)) } -type goParams struct { - version uint16 - maxDepth uint32 - inkPrice uint24 - debugMode bool +type ProgParams struct { + Version uint16 + MaxDepth uint32 + InkPrice uint24 + DebugMode bool } -func (p Programs) goParams(version uint16, debug bool, params *StylusParams) *goParams { - return &goParams{ - version: version, - maxDepth: params.MaxStackDepth, - inkPrice: params.InkPrice, - debugMode: debug, +func (p Programs) progParams(version uint16, debug bool, params *StylusParams) *ProgParams { + return &ProgParams{ + Version: version, + MaxDepth: params.MaxStackDepth, + InkPrice: params.InkPrice, + DebugMode: debug, } } -type evmData struct { +type EvmData struct { blockBasefee common.Hash chainId uint64 blockCoinbase common.Address diff --git a/arbos/programs/wasm.go b/arbos/programs/wasm.go index 77eb7e0f2f..1e9b5e680b 100644 --- a/arbos/programs/wasm.go +++ b/arbos/programs/wasm.go @@ -135,14 +135,26 @@ func callProgram( interpreter *vm.EVMInterpreter, tracingInfo *util.TracingInfo, calldata []byte, - evmData *evmData, - params *goParams, + evmData *EvmData, + params *ProgParams, memoryModel *MemoryModel, ) ([]byte, error) { reqHandler := newApiClosures(interpreter, tracingInfo, scope, memoryModel) + gasLeft, retData, err := CallProgramLoop(moduleHash, calldata, scope.Contract.Gas, evmData, params, reqHandler) + scope.Contract.Gas = gasLeft + return retData, err +} + +func CallProgramLoop( + moduleHash common.Hash, + calldata []byte, + gas uint64, + evmData *EvmData, + params *ProgParams, + reqHandler RequestHandler) (uint64, []byte, error) { configHandler := params.createHandler() dataHandler := evmData.createHandler() - debug := params.debugMode + debug := params.DebugMode module := newProgram( unsafe.Pointer(&moduleHash[0]), @@ -150,7 +162,7 @@ func callProgram( uint32(len(calldata)), configHandler, dataHandler, - scope.Contract.Gas, + gas, ) reqId := startProgram(module) for { @@ -162,12 +174,11 @@ func callProgram( popProgram() status := userStatus(reqTypeId) gasLeft := arbmath.BytesToUint(reqData[:8]) - scope.Contract.Gas = gasLeft data, msg, err := status.toResult(reqData[8:], debug) if status == userFailure && debug { - log.Warn("program failure", "err", err, "msg", msg, "program", address) + log.Warn("program failure", "err", err, "msg", msg, "moduleHash", moduleHash) } - return data, err + return gasLeft, data, err } reqType := RequestType(reqTypeId - EvmApiMethodReqOffset) diff --git a/arbos/programs/wasm_api.go b/arbos/programs/wasm_api.go index fb0f731402..d7bac056c0 100644 --- a/arbos/programs/wasm_api.go +++ b/arbos/programs/wasm_api.go @@ -38,12 +38,12 @@ func createEvmData( reentrant uint32, ) evmDataHandler -func (params *goParams) createHandler() stylusConfigHandler { - debug := arbmath.BoolToUint32(params.debugMode) - return createStylusConfig(uint32(params.version), params.maxDepth, params.inkPrice.ToUint32(), debug) +func (params *ProgParams) createHandler() stylusConfigHandler { + debug := arbmath.BoolToUint32(params.DebugMode) + return createStylusConfig(uint32(params.Version), params.MaxDepth, params.InkPrice.ToUint32(), debug) } -func (data *evmData) createHandler() evmDataHandler { +func (data *EvmData) createHandler() evmDataHandler { return createEvmData( arbutil.SliceToUnsafePointer(data.blockBasefee[:]), data.chainId, From f2448fede36f35df82db94930ea35b7ca1ebe426 Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Fri, 10 May 2024 18:42:57 -0600 Subject: [PATCH 053/113] test-cases/go: add stylus contract this is the only test that checks 5 recursive stylus calls with one-step-proofs for each relevant SwitchThread --- Makefile | 10 ++++-- arbitrator/prover/test-cases/go/main.go | 44 +++++++++++++++++++++++++ 2 files changed, 51 insertions(+), 3 deletions(-) diff --git a/Makefile b/Makefile index 36b55bed6e..53b89c8d72 100644 --- a/Makefile +++ b/Makefile @@ -289,7 +289,7 @@ $(arbitrator_jit): $(DEP_PREDICATE) $(jit_files) $(arbitrator_cases)/rust/$(wasm32_wasi)/%.wasm: $(arbitrator_cases)/rust/src/bin/%.rs $(arbitrator_cases)/rust/src/lib.rs cargo build --manifest-path $(arbitrator_cases)/rust/Cargo.toml --release --target wasm32-wasi --bin $(patsubst $(arbitrator_cases)/rust/$(wasm32_wasi)/%.wasm,%, $@) -$(arbitrator_cases)/go/testcase.wasm: $(arbitrator_cases)/go/*.go +$(arbitrator_cases)/go/testcase.wasm: $(arbitrator_cases)/go/*.go .make/solgen cd $(arbitrator_cases)/go && GOOS=wasip1 GOARCH=wasm go build -o testcase.wasm $(arbitrator_generated_header): $(DEP_PREDICATE) $(stylus_files) @@ -439,8 +439,12 @@ target/testdata/preimages.bin: contracts/test/prover/proofs/rust-%.json: $(arbitrator_cases)/rust/$(wasm32_wasi)/%.wasm $(prover_bin) $(arbitrator_wasm_libs) target/testdata/preimages.bin $(prover_bin) $< $(arbitrator_wasm_lib_flags) -o $@ -b --allow-hostapi --require-success --inbox-add-stub-headers --inbox $(arbitrator_cases)/rust/data/msg0.bin --inbox $(arbitrator_cases)/rust/data/msg1.bin --delayed-inbox $(arbitrator_cases)/rust/data/msg0.bin --delayed-inbox $(arbitrator_cases)/rust/data/msg1.bin --preimages target/testdata/preimages.bin -contracts/test/prover/proofs/go.json: $(arbitrator_cases)/go/testcase.wasm $(prover_bin) $(arbitrator_wasm_libs) target/testdata/preimages.bin $(arbitrator_tests_link_deps) - $(prover_bin) $< $(arbitrator_wasm_lib_flags) -o $@ -i 50000000 --require-success --preimages target/testdata/preimages.bin +contracts/test/prover/proofs/go.json: $(arbitrator_cases)/go/testcase.wasm $(prover_bin) $(arbitrator_wasm_libs) target/testdata/preimages.bin $(arbitrator_tests_link_deps) $(arbitrator_cases)/user.wasm + $(prover_bin) $< $(arbitrator_wasm_lib_flags) -o $@ -b --require-success --preimages target/testdata/preimages.bin --stylus-modules $(arbitrator_cases)/user.wasm + +# avoid testing user.wasm in onestepproofs. It can only run as stylus program. +contracts/test/prover/proofs/user.json: + echo "[]" > $@ # avoid testing read-inboxmsg-10 in onestepproofs. It's used for go challenge testing. contracts/test/prover/proofs/read-inboxmsg-10.json: diff --git a/arbitrator/prover/test-cases/go/main.go b/arbitrator/prover/test-cases/go/main.go index 0df8010449..1f81553af2 100644 --- a/arbitrator/prover/test-cases/go/main.go +++ b/arbitrator/prover/test-cases/go/main.go @@ -1,6 +1,9 @@ // Copyright 2021-2024, Offchain Labs, Inc. // For license information, see https://github.com/nitro/blob/master/LICENSE +//go:build wasm +// +build wasm + package main import ( @@ -19,6 +22,7 @@ import ( merkletree "github.com/wealdtech/go-merkletree" "github.com/offchainlabs/nitro/arbcompress" + "github.com/offchainlabs/nitro/arbos/programs" "github.com/offchainlabs/nitro/arbutil" "github.com/offchainlabs/nitro/wavmio" ) @@ -69,11 +73,51 @@ const BYTES_PER_FIELD_ELEMENT = 32 var BLS_MODULUS, _ = new(big.Int).SetString("52435875175126190479447740508185965837690552500527637822603658699938581184513", 10) +var stylusModuleHash = common.HexToHash("a149cf8113ff9c95f2c8c2a1423575367de86dd422d87114bb9ea47baf535dd7") // user.wat + +func callStylusProgram(recurse int) { + evmData := programs.EvmData{} + progParams := programs.ProgParams{ + MaxDepth: 10000, + InkPrice: 1, + DebugMode: true, + } + reqHandler := func(req programs.RequestType, input []byte) ([]byte, []byte, uint64) { + fmt.Printf("got request type %d req %v\n", req, input) + if req == programs.GetBytes32 { + if recurse > 0 { + callStylusProgram(recurse - 1) + } + answer := common.Hash{} + return answer[:], nil, 1 + } + + panic("unsupported call") + } + calldata := common.Hash{}.Bytes() + _, _, err := programs.CallProgramLoop( + stylusModuleHash, + calldata, + 160000000, + &evmData, + &progParams, + reqHandler) + if err != nil { + panic(err) + } +} + func main() { fmt.Printf("starting executable with %v arg(s): %v\n", len(os.Args), os.Args) runtime.GC() time.Sleep(time.Second) + fmt.Printf("Stylus test\n") + + callStylusProgram(5) + + fmt.Printf("Stylus test done!\n") + // Data for the tree data := [][]byte{ []byte("Foo"), From 45180a7e8e4b43592edb75b43bbd32d1e3cc6148 Mon Sep 17 00:00:00 2001 From: Jeremy Date: Tue, 14 May 2024 14:28:17 +0800 Subject: [PATCH 054/113] Update testconstants.go --- arbos/programs/testconstants.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/arbos/programs/testconstants.go b/arbos/programs/testconstants.go index 215b5fb8a7..1ab0e6e93b 100644 --- a/arbos/programs/testconstants.go +++ b/arbos/programs/testconstants.go @@ -1,6 +1,9 @@ // Copyright 2024, Offchain Labs, Inc. // For license information, see https://github.com/OffchainLabs/nitro/blob/master/LICENSE +//go:build !wasm +// +build !wasm + package programs // This file exists because cgo isn't allowed in tests From 90374dc51277dd99c6fce14737f0cd17e4406f29 Mon Sep 17 00:00:00 2001 From: Maciej Kulawik Date: Tue, 14 May 2024 12:32:23 +0200 Subject: [PATCH 055/113] add subdirectory to wal-dir to avoid filename collision between dbs, move most pebble options to experimental section --- arbnode/dataposter/storage_test.go | 2 +- cmd/conf/database.go | 171 +++++++++++++++++------------ cmd/nitro/init.go | 6 +- cmd/nitro/nitro.go | 2 +- cmd/pruning/pruning.go | 2 +- system_tests/common_test.go | 8 +- system_tests/das_test.go | 4 +- system_tests/pruning_test.go | 2 +- system_tests/staterecovery_test.go | 2 +- 9 files changed, 113 insertions(+), 86 deletions(-) diff --git a/arbnode/dataposter/storage_test.go b/arbnode/dataposter/storage_test.go index 343efac3c7..e2aa321e0d 100644 --- a/arbnode/dataposter/storage_test.go +++ b/arbnode/dataposter/storage_test.go @@ -45,7 +45,7 @@ func newLevelDBStorage(t *testing.T, encF storage.EncoderDecoderF) *dbstorage.St func newPebbleDBStorage(t *testing.T, encF storage.EncoderDecoderF) *dbstorage.Storage { t.Helper() - db, err := rawdb.NewPebbleDBDatabase(path.Join(t.TempDir(), "pebble.db"), 0, 0, "default", false, true, conf.PersistentConfigDefault.Pebble.ExtraOptions()) + db, err := rawdb.NewPebbleDBDatabase(path.Join(t.TempDir(), "pebble.db"), 0, 0, "default", false, true, conf.PersistentConfigDefault.Pebble.ExtraOptions("pebble")) if err != nil { t.Fatalf("NewPebbleDBDatabase() unexpected error: %v", err) } diff --git a/cmd/conf/database.go b/cmd/conf/database.go index 1c8b673dd3..d60ee51c5b 100644 --- a/cmd/conf/database.go +++ b/cmd/conf/database.go @@ -101,81 +101,85 @@ func (c *PersistentConfig) Validate() error { if c.DBEngine != "leveldb" && c.DBEngine != "pebble" { return fmt.Errorf(`invalid .db-engine choice: %q, allowed "leveldb" or "pebble"`, c.DBEngine) } + if c.DBEngine == "pebble" { + if err := c.Pebble.Validate(); err != nil { + return err + } + } return nil } type PebbleConfig struct { - BytesPerSync int `koanf:"bytes-per-sync"` - L0CompactionFileThreshold int `koanf:"l0-compaction-file-threshold"` - L0CompactionThreshold int `koanf:"l0-compaction-threshold"` - L0StopWritesThreshold int `koanf:"l0-stop-writes-threshold"` - LBaseMaxBytes int64 `koanf:"l-base-max-bytes"` - MemTableStopWritesThreshold int `koanf:"mem-table-stop-writes-threshold"` - MaxConcurrentCompactions int `koanf:"max-concurrent-compactions"` - DisableAutomaticCompactions bool `koanf:"disable-automatic-compactions"` - WALBytesPerSync int `koanf:"wal-bytes-per-sync"` - WALDir string `koanf:"wal-dir"` - WALMinSyncInterval int `koanf:"wal-min-sync-interval"` - TargetByteDeletionRate int `koanf:"target-byte-deletion-rate"` - Experimental PebbleExperimentalConfig `koanf:"experimental"` + MaxConcurrentCompactions int `koanf:"max-concurrent-compactions"` + Experimental PebbleExperimentalConfig `koanf:"experimental"` +} + +var PebbleConfigDefault = PebbleConfig{ + MaxConcurrentCompactions: runtime.NumCPU(), + Experimental: PebbleExperimentalConfigDefault, +} + +func PebbleConfigAddOptions(prefix string, f *flag.FlagSet) { + f.Int(prefix+".max-concurrent-compactions", PebbleConfigDefault.MaxConcurrentCompactions, "maximum number of concurrent compactions") + PebbleExperimentalConfigAddOptions(prefix+".experimental", f) +} + +func (c *PebbleConfig) Validate() error { + if c.MaxConcurrentCompactions < 1 { + return fmt.Errorf("invalid .max-concurrent-compactions value: %d, has to be greater then 0", c.MaxConcurrentCompactions) + } + if err := c.Experimental.Validate(); err != nil { + return err + } + return nil +} + +type PebbleExperimentalConfig struct { + BytesPerSync int `koanf:"bytes-per-sync"` + L0CompactionFileThreshold int `koanf:"l0-compaction-file-threshold"` + L0CompactionThreshold int `koanf:"l0-compaction-threshold"` + L0StopWritesThreshold int `koanf:"l0-stop-writes-threshold"` + LBaseMaxBytes int64 `koanf:"l-base-max-bytes"` + MemTableStopWritesThreshold int `koanf:"mem-table-stop-writes-threshold"` + DisableAutomaticCompactions bool `koanf:"disable-automatic-compactions"` + WALBytesPerSync int `koanf:"wal-bytes-per-sync"` + WALDir string `koanf:"wal-dir"` + WALMinSyncInterval int `koanf:"wal-min-sync-interval"` + TargetByteDeletionRate int `koanf:"target-byte-deletion-rate"` // level specific BlockSize int `koanf:"block-size"` IndexBlockSize int `koanf:"index-block-size"` TargetFileSize int64 `koanf:"target-file-size"` TargetFileSizeEqualLevels bool `koanf:"target-file-size-equal-levels"` + + // pebble experimental + L0CompactionConcurrency int `koanf:"l0-compaction-concurrency"` + CompactionDebtConcurrency uint64 `koanf:"compaction-debt-concurrency"` + ReadCompactionRate int64 `koanf:"read-compaction-rate"` + ReadSamplingMultiplier int64 `koanf:"read-sampling-multiplier"` + MaxWriterConcurrency int `koanf:"max-writer-concurrency"` + ForceWriterParallelism bool `koanf:"force-writer-parallelism"` } -var PebbleConfigDefault = PebbleConfig{ +var PebbleExperimentalConfigDefault = PebbleExperimentalConfig{ BytesPerSync: 0, // pebble default will be used L0CompactionFileThreshold: 0, // pebble default will be used L0CompactionThreshold: 0, // pebble default will be used L0StopWritesThreshold: 0, // pebble default will be used LBaseMaxBytes: 0, // pebble default will be used MemTableStopWritesThreshold: 2, - MaxConcurrentCompactions: runtime.NumCPU(), DisableAutomaticCompactions: false, WALBytesPerSync: 0, // pebble default will be used WALDir: "", // default will use same dir as for sstables WALMinSyncInterval: 0, // pebble default will be used TargetByteDeletionRate: 0, // pebble default will be used - Experimental: PebbleExperimentalConfigDefault, - BlockSize: 4096, - IndexBlockSize: 4096, - TargetFileSize: 2 * 1024 * 1024, - TargetFileSizeEqualLevels: true, -} -func PebbleConfigAddOptions(prefix string, f *flag.FlagSet) { - f.Int(prefix+".bytes-per-sync", PebbleConfigDefault.BytesPerSync, "number of bytes to write to a SSTable before calling Sync on it in the background") - f.Int(prefix+".l0-compaction-file-threshold", PebbleConfigDefault.L0CompactionFileThreshold, "count of L0 files necessary to trigger an L0 compaction") - f.Int(prefix+".l0-compaction-threshold", PebbleConfigDefault.L0CompactionThreshold, "amount of L0 read-amplification necessary to trigger an L0 compaction") - f.Int(prefix+".l0-stop-writes-threshold", PebbleConfigDefault.L0StopWritesThreshold, "hard limit on L0 read-amplification, computed as the number of L0 sublevels. Writes are stopped when this threshold is reached") - f.Int64(prefix+".l-base-max-bytes", PebbleConfigDefault.LBaseMaxBytes, "The maximum number of bytes for LBase. The base level is the level which L0 is compacted into. The base level is determined dynamically based on the existing data in the LSM. The maximum number of bytes for other levels is computed dynamically based on the base level's maximum size. When the maximum number of bytes for a level is exceeded, compaction is requested.") - f.Int(prefix+".mem-table-stop-writes-threshold", PebbleConfigDefault.MemTableStopWritesThreshold, "hard limit on the number of queued of MemTables") - f.Int(prefix+".max-concurrent-compactions", PebbleConfigDefault.MaxConcurrentCompactions, "maximum number of concurrent compactions") - f.Bool(prefix+".disable-automatic-compactions", PebbleConfigDefault.DisableAutomaticCompactions, "disables automatic compactions") - f.Int(prefix+".wal-bytes-per-sync", PebbleConfigDefault.WALBytesPerSync, "number of bytes to write to a write-ahead log (WAL) before calling Sync on it in the backgroud") - f.String(prefix+".wal-dir", PebbleConfigDefault.WALDir, "directory to store write-ahead logs (WALs) in. If empty, WALs will be stored in the same directory as sstables") - f.Int(prefix+".wal-min-sync-interval", PebbleConfigDefault.WALMinSyncInterval, "minimum duration in microseconds between syncs of the WAL. If WAL syncs are requested faster than this interval, they will be artificially delayed.") - f.Int(prefix+".target-byte-deletion-rate", PebbleConfigDefault.TargetByteDeletionRate, "rate (in bytes per second) at which sstable file deletions are limited to (under normal circumstances).") - f.Int(prefix+".block-size", PebbleConfigDefault.BlockSize, "target uncompressed size in bytes of each table block") - f.Int(prefix+".index-block-size", PebbleConfigDefault.IndexBlockSize, fmt.Sprintf("target uncompressed size in bytes of each index block. When the index block size is larger than this target, two-level indexes are automatically enabled. Setting this option to a large value (such as %d) disables the automatic creation of two-level indexes.", math.MaxInt32)) - PebbleExperimentalConfigAddOptions(prefix+".experimental", f) - f.Int64(prefix+".target-file-size", PebbleConfigDefault.TargetFileSize, "target file size for the level 0") - f.Bool(prefix+".target-file-size-equal-levels", PebbleConfigDefault.TargetFileSizeEqualLevels, "if true same target-file-size will be uses for all levels, otherwise target size for layer n = 2 * target size for layer n - 1") -} - -type PebbleExperimentalConfig struct { - L0CompactionConcurrency int `koanf:"l0-compaction-concurrency"` - CompactionDebtConcurrency uint64 `koanf:"compaction-debt-concurrency"` - ReadCompactionRate int64 `koanf:"read-compaction-rate"` - ReadSamplingMultiplier int64 `koanf:"read-sampling-multiplier"` - MaxWriterConcurrency int `koanf:"max-writer-concurrency"` - ForceWriterParallelism bool `koanf:"force-writer-parallelism"` -} + BlockSize: 4096, + IndexBlockSize: 4096, + TargetFileSize: 2 * 1024 * 1024, + TargetFileSizeEqualLevels: true, -var PebbleExperimentalConfigDefault = PebbleExperimentalConfig{ L0CompactionConcurrency: 0, CompactionDebtConcurrency: 0, ReadCompactionRate: 0, @@ -185,6 +189,22 @@ var PebbleExperimentalConfigDefault = PebbleExperimentalConfig{ } func PebbleExperimentalConfigAddOptions(prefix string, f *flag.FlagSet) { + f.Int(prefix+".bytes-per-sync", PebbleExperimentalConfigDefault.BytesPerSync, "number of bytes to write to a SSTable before calling Sync on it in the background") + f.Int(prefix+".l0-compaction-file-threshold", PebbleExperimentalConfigDefault.L0CompactionFileThreshold, "count of L0 files necessary to trigger an L0 compaction") + f.Int(prefix+".l0-compaction-threshold", PebbleExperimentalConfigDefault.L0CompactionThreshold, "amount of L0 read-amplification necessary to trigger an L0 compaction") + f.Int(prefix+".l0-stop-writes-threshold", PebbleExperimentalConfigDefault.L0StopWritesThreshold, "hard limit on L0 read-amplification, computed as the number of L0 sublevels. Writes are stopped when this threshold is reached") + f.Int64(prefix+".l-base-max-bytes", PebbleExperimentalConfigDefault.LBaseMaxBytes, "The maximum number of bytes for LBase. The base level is the level which L0 is compacted into. The base level is determined dynamically based on the existing data in the LSM. The maximum number of bytes for other levels is computed dynamically based on the base level's maximum size. When the maximum number of bytes for a level is exceeded, compaction is requested.") + f.Int(prefix+".mem-table-stop-writes-threshold", PebbleExperimentalConfigDefault.MemTableStopWritesThreshold, "hard limit on the number of queued of MemTables") + f.Bool(prefix+".disable-automatic-compactions", PebbleExperimentalConfigDefault.DisableAutomaticCompactions, "disables automatic compactions") + f.Int(prefix+".wal-bytes-per-sync", PebbleExperimentalConfigDefault.WALBytesPerSync, "number of bytes to write to a write-ahead log (WAL) before calling Sync on it in the backgroud") + f.String(prefix+".wal-dir", PebbleExperimentalConfigDefault.WALDir, "absolute path of directory to store write-ahead logs (WALs) in. If empty, WALs will be stored in the same directory as sstables") + f.Int(prefix+".wal-min-sync-interval", PebbleExperimentalConfigDefault.WALMinSyncInterval, "minimum duration in microseconds between syncs of the WAL. If WAL syncs are requested faster than this interval, they will be artificially delayed.") + f.Int(prefix+".target-byte-deletion-rate", PebbleExperimentalConfigDefault.TargetByteDeletionRate, "rate (in bytes per second) at which sstable file deletions are limited to (under normal circumstances).") + f.Int(prefix+".block-size", PebbleExperimentalConfigDefault.BlockSize, "target uncompressed size in bytes of each table block") + f.Int(prefix+".index-block-size", PebbleExperimentalConfigDefault.IndexBlockSize, fmt.Sprintf("target uncompressed size in bytes of each index block. When the index block size is larger than this target, two-level indexes are automatically enabled. Setting this option to a large value (such as %d) disables the automatic creation of two-level indexes.", math.MaxInt32)) + f.Int64(prefix+".target-file-size", PebbleExperimentalConfigDefault.TargetFileSize, "target file size for the level 0") + f.Bool(prefix+".target-file-size-equal-levels", PebbleExperimentalConfigDefault.TargetFileSizeEqualLevels, "if true same target-file-size will be uses for all levels, otherwise target size for layer n = 2 * target size for layer n - 1") + f.Int(prefix+".l0-compaction-concurrency", PebbleExperimentalConfigDefault.L0CompactionConcurrency, "threshold of L0 read-amplification at which compaction concurrency is enabled (if compaction-debt-concurrency was not already exceeded). Every multiple of this value enables another concurrent compaction up to max-concurrent-compactions.") f.Uint64(prefix+".compaction-debt-concurrency", PebbleExperimentalConfigDefault.CompactionDebtConcurrency, "controls the threshold of compaction debt at which additional compaction concurrency slots are added. For every multiple of this value in compaction debt bytes, an additional concurrent compaction is added. This works \"on top\" of l0-compaction-concurrency, so the higher of the count of compaction concurrency slots as determined by the two options is chosen.") f.Int64(prefix+".read-compaction-rate", PebbleExperimentalConfigDefault.ReadCompactionRate, "controls the frequency of read triggered compactions by adjusting `AllowedSeeks` in manifest.FileMetadata: AllowedSeeks = FileSize / ReadCompactionRate") @@ -193,42 +213,54 @@ func PebbleExperimentalConfigAddOptions(prefix string, f *flag.FlagSet) { f.Bool(prefix+".force-writer-parallelism", PebbleExperimentalConfigDefault.ForceWriterParallelism, "force parallelism in the sstable Writer for the metamorphic tests. Even with the MaxWriterConcurrency option set, pebble only enables parallelism in the sstable Writer if there is enough CPU available, and this option bypasses that.") } -func (c *PebbleConfig) ExtraOptions() *pebble.ExtraOptions { +func (c *PebbleExperimentalConfig) Validate() error { + if !filepath.IsAbs(c.WALDir) { + return fmt.Errorf("invalid .wal-dir directory (%s) - has to be an absolute path", c.WALDir) + } + // TODO + return nil +} + +func (c *PebbleConfig) ExtraOptions(namespace string) *pebble.ExtraOptions { var maxConcurrentCompactions func() int if c.MaxConcurrentCompactions > 0 { maxConcurrentCompactions = func() int { return c.MaxConcurrentCompactions } } var walMinSyncInterval func() time.Duration - if c.WALMinSyncInterval > 0 { + if c.Experimental.WALMinSyncInterval > 0 { walMinSyncInterval = func() time.Duration { - return time.Microsecond * time.Duration(c.WALMinSyncInterval) + return time.Microsecond * time.Duration(c.Experimental.WALMinSyncInterval) } } var levels []pebble.ExtraLevelOptions for i := 0; i < 7; i++ { - targetFileSize := c.TargetFileSize - if !c.TargetFileSizeEqualLevels { + targetFileSize := c.Experimental.TargetFileSize + if !c.Experimental.TargetFileSizeEqualLevels { targetFileSize = targetFileSize << i } levels = append(levels, pebble.ExtraLevelOptions{ - BlockSize: c.BlockSize, - IndexBlockSize: c.IndexBlockSize, + BlockSize: c.Experimental.BlockSize, + IndexBlockSize: c.Experimental.IndexBlockSize, TargetFileSize: targetFileSize, }) } + walDir := c.Experimental.WALDir + if walDir != "" { + walDir = path.Join(walDir, namespace) + } return &pebble.ExtraOptions{ - BytesPerSync: c.BytesPerSync, - L0CompactionFileThreshold: c.L0CompactionFileThreshold, - L0CompactionThreshold: c.L0CompactionThreshold, - L0StopWritesThreshold: c.L0StopWritesThreshold, - LBaseMaxBytes: c.LBaseMaxBytes, - MemTableStopWritesThreshold: c.MemTableStopWritesThreshold, + BytesPerSync: c.Experimental.BytesPerSync, + L0CompactionFileThreshold: c.Experimental.L0CompactionFileThreshold, + L0CompactionThreshold: c.Experimental.L0CompactionThreshold, + L0StopWritesThreshold: c.Experimental.L0StopWritesThreshold, + LBaseMaxBytes: c.Experimental.LBaseMaxBytes, + MemTableStopWritesThreshold: c.Experimental.MemTableStopWritesThreshold, MaxConcurrentCompactions: maxConcurrentCompactions, - DisableAutomaticCompactions: c.DisableAutomaticCompactions, - WALBytesPerSync: c.WALBytesPerSync, - WALDir: c.WALDir, + DisableAutomaticCompactions: c.Experimental.DisableAutomaticCompactions, + WALBytesPerSync: c.Experimental.WALBytesPerSync, + WALDir: walDir, WALMinSyncInterval: walMinSyncInterval, - TargetByteDeletionRate: c.TargetByteDeletionRate, + TargetByteDeletionRate: c.Experimental.TargetByteDeletionRate, Experimental: pebble.ExtraOptionsExperimental{ L0CompactionConcurrency: c.Experimental.L0CompactionConcurrency, CompactionDebtConcurrency: c.Experimental.CompactionDebtConcurrency, @@ -240,8 +272,3 @@ func (c *PebbleConfig) ExtraOptions() *pebble.ExtraOptions { Levels: levels, } } - -func (c *PebbleConfig) Validate() error { - // TODO - return nil -} diff --git a/cmd/nitro/init.go b/cmd/nitro/init.go index 9362154ec0..31ce4b91ea 100644 --- a/cmd/nitro/init.go +++ b/cmd/nitro/init.go @@ -172,13 +172,13 @@ func validateBlockChain(blockChain *core.BlockChain, chainConfig *params.ChainCo func openInitializeChainDb(ctx context.Context, stack *node.Node, config *NodeConfig, chainId *big.Int, cacheConfig *core.CacheConfig, persistentConfig *conf.PersistentConfig, l1Client arbutil.L1Interface, rollupAddrs chaininfo.RollupAddresses) (ethdb.Database, *core.BlockChain, error) { if !config.Init.Force { - if readOnlyDb, err := stack.OpenDatabaseWithFreezerWithExtraOptions("l2chaindata", 0, 0, "", "l2chaindata/", true, persistentConfig.Pebble.ExtraOptions()); err == nil { + if readOnlyDb, err := stack.OpenDatabaseWithFreezerWithExtraOptions("l2chaindata", 0, 0, "", "l2chaindata/", true, persistentConfig.Pebble.ExtraOptions("l2chaindata")); err == nil { if chainConfig := gethexec.TryReadStoredChainConfig(readOnlyDb); chainConfig != nil { readOnlyDb.Close() if !arbmath.BigEquals(chainConfig.ChainID, chainId) { return nil, nil, fmt.Errorf("database has chain ID %v but config has chain ID %v (are you sure this database is for the right chain?)", chainConfig.ChainID, chainId) } - chainDb, err := stack.OpenDatabaseWithFreezerWithExtraOptions("l2chaindata", config.Execution.Caching.DatabaseCache, config.Persistent.Handles, config.Persistent.Ancient, "l2chaindata/", false, persistentConfig.Pebble.ExtraOptions()) + chainDb, err := stack.OpenDatabaseWithFreezerWithExtraOptions("l2chaindata", config.Execution.Caching.DatabaseCache, config.Persistent.Handles, config.Persistent.Ancient, "l2chaindata/", false, persistentConfig.Pebble.ExtraOptions("l2chaindata")) if err != nil { return chainDb, nil, err } @@ -230,7 +230,7 @@ func openInitializeChainDb(ctx context.Context, stack *node.Node, config *NodeCo var initDataReader statetransfer.InitDataReader = nil - chainDb, err := stack.OpenDatabaseWithFreezerWithExtraOptions("l2chaindata", config.Execution.Caching.DatabaseCache, config.Persistent.Handles, config.Persistent.Ancient, "l2chaindata/", false, persistentConfig.Pebble.ExtraOptions()) + chainDb, err := stack.OpenDatabaseWithFreezerWithExtraOptions("l2chaindata", config.Execution.Caching.DatabaseCache, config.Persistent.Handles, config.Persistent.Ancient, "l2chaindata/", false, persistentConfig.Pebble.ExtraOptions("l2chaindata")) if err != nil { return chainDb, nil, err } diff --git a/cmd/nitro/nitro.go b/cmd/nitro/nitro.go index 277afa302a..4ee042d477 100644 --- a/cmd/nitro/nitro.go +++ b/cmd/nitro/nitro.go @@ -489,7 +489,7 @@ func mainImpl() int { return 1 } - arbDb, err := stack.OpenDatabaseWithExtraOptions("arbitrumdata", 0, 0, "arbitrumdata/", false, nodeConfig.Persistent.Pebble.ExtraOptions()) + arbDb, err := stack.OpenDatabaseWithExtraOptions("arbitrumdata", 0, 0, "arbitrumdata/", false, nodeConfig.Persistent.Pebble.ExtraOptions("arbitrumdata")) deferFuncs = append(deferFuncs, func() { closeDb(arbDb, "arbDb") }) if err != nil { log.Error("failed to open database", "err", err) diff --git a/cmd/pruning/pruning.go b/cmd/pruning/pruning.go index 363126a49f..72e7d2c516 100644 --- a/cmd/pruning/pruning.go +++ b/cmd/pruning/pruning.go @@ -85,7 +85,7 @@ func findImportantRoots(ctx context.Context, chainDb ethdb.Database, stack *node if chainConfig == nil { return nil, errors.New("database doesn't have a chain config (was this node initialized?)") } - arbDb, err := stack.OpenDatabaseWithExtraOptions("arbitrumdata", 0, 0, "arbitrumdata/", true, persistentConfig.Pebble.ExtraOptions()) + arbDb, err := stack.OpenDatabaseWithExtraOptions("arbitrumdata", 0, 0, "arbitrumdata/", true, persistentConfig.Pebble.ExtraOptions("arbitrumdata")) if err != nil { return nil, err } diff --git a/system_tests/common_test.go b/system_tests/common_test.go index fd63eb9431..4aa8581bd0 100644 --- a/system_tests/common_test.go +++ b/system_tests/common_test.go @@ -774,9 +774,9 @@ func createL2BlockChainWithStackConfig( Require(t, err) // TODO get pebble.ExtraOptions from conf.PersistentConfig when opening the DBs - chainDb, err := stack.OpenDatabaseWithExtraOptions("l2chaindata", 0, 0, "l2chaindata/", false, conf.PersistentConfigDefault.Pebble.ExtraOptions()) + chainDb, err := stack.OpenDatabaseWithExtraOptions("l2chaindata", 0, 0, "l2chaindata/", false, conf.PersistentConfigDefault.Pebble.ExtraOptions("l2chaindata")) Require(t, err) - arbDb, err := stack.OpenDatabaseWithExtraOptions("arbitrumdata", 0, 0, "arbitrumdata/", false, conf.PersistentConfigDefault.Pebble.ExtraOptions()) + arbDb, err := stack.OpenDatabaseWithExtraOptions("arbitrumdata", 0, 0, "arbitrumdata/", false, conf.PersistentConfigDefault.Pebble.ExtraOptions("arbitrumdata")) Require(t, err) initReader := statetransfer.NewMemoryInitDataReader(&l2info.ArbInitData) @@ -979,9 +979,9 @@ func Create2ndNodeWithConfig( Require(t, err) // TODO get pebble.ExtraOptions from conf.PersistentConfig when opening the DBs - l2chainDb, err := l2stack.OpenDatabaseWithExtraOptions("l2chaindata", 0, 0, "l2chaindata/", false, conf.PersistentConfigDefault.Pebble.ExtraOptions()) + l2chainDb, err := l2stack.OpenDatabaseWithExtraOptions("l2chaindata", 0, 0, "l2chaindata/", false, conf.PersistentConfigDefault.Pebble.ExtraOptions("l2chaindata")) Require(t, err) - l2arbDb, err := l2stack.OpenDatabaseWithExtraOptions("arbitrumdata", 0, 0, "arbitrumdata/", false, conf.PersistentConfigDefault.Pebble.ExtraOptions()) + l2arbDb, err := l2stack.OpenDatabaseWithExtraOptions("arbitrumdata", 0, 0, "arbitrumdata/", false, conf.PersistentConfigDefault.Pebble.ExtraOptions("arbitrumdata")) Require(t, err) initReader := statetransfer.NewMemoryInitDataReader(l2InitData) diff --git a/system_tests/das_test.go b/system_tests/das_test.go index 7495b9a13e..2febadb3d2 100644 --- a/system_tests/das_test.go +++ b/system_tests/das_test.go @@ -179,10 +179,10 @@ func TestDASRekey(t *testing.T) { Require(t, err) // TODO get pebble.ExtraOptions from conf.PersistentConfig - l2chainDb, err := l2stackA.OpenDatabaseWithExtraOptions("l2chaindata", 0, 0, "l2chaindata/", false, conf.PersistentConfigDefault.Pebble.ExtraOptions()) + l2chainDb, err := l2stackA.OpenDatabaseWithExtraOptions("l2chaindata", 0, 0, "l2chaindata/", false, conf.PersistentConfigDefault.Pebble.ExtraOptions("l2chaindata")) Require(t, err) - l2arbDb, err := l2stackA.OpenDatabaseWithExtraOptions("arbitrumdata", 0, 0, "arbitrumdata/", false, conf.PersistentConfigDefault.Pebble.ExtraOptions()) + l2arbDb, err := l2stackA.OpenDatabaseWithExtraOptions("arbitrumdata", 0, 0, "arbitrumdata/", false, conf.PersistentConfigDefault.Pebble.ExtraOptions("arbitrumdata")) Require(t, err) l2blockchain, err := gethexec.GetBlockChain(l2chainDb, nil, chainConfig, gethexec.ConfigDefaultTest().TxLookupLimit) diff --git a/system_tests/pruning_test.go b/system_tests/pruning_test.go index e83c350804..d2453887ee 100644 --- a/system_tests/pruning_test.go +++ b/system_tests/pruning_test.go @@ -66,7 +66,7 @@ func TestPruning(t *testing.T) { Require(t, err) defer stack.Close() // TODO get pebble.ExtraOptions from conf.PersistentConfig - chainDb, err := stack.OpenDatabaseWithExtraOptions("l2chaindata", 0, 0, "l2chaindata/", false, conf.PersistentConfigDefault.Pebble.ExtraOptions()) + chainDb, err := stack.OpenDatabaseWithExtraOptions("l2chaindata", 0, 0, "l2chaindata/", false, conf.PersistentConfigDefault.Pebble.ExtraOptions("l2chaindata")) Require(t, err) defer chainDb.Close() chainDbEntriesBeforePruning := countStateEntries(chainDb) diff --git a/system_tests/staterecovery_test.go b/system_tests/staterecovery_test.go index 9dc1081a7b..459a6e3ee8 100644 --- a/system_tests/staterecovery_test.go +++ b/system_tests/staterecovery_test.go @@ -51,7 +51,7 @@ func TestRectreateMissingStates(t *testing.T) { Require(t, err) defer stack.Close() // TODO get pebble.ExtraOptions from conf.PersistentConfig - chainDb, err := stack.OpenDatabaseWithExtraOptions("l2chaindata", 0, 0, "l2chaindata/", false, conf.PersistentConfigDefault.Pebble.ExtraOptions()) + chainDb, err := stack.OpenDatabaseWithExtraOptions("l2chaindata", 0, 0, "l2chaindata/", false, conf.PersistentConfigDefault.Pebble.ExtraOptions("l2chaindata")) Require(t, err) defer chainDb.Close() cacheConfig := gethexec.DefaultCacheConfigFor(stack, &gethexec.DefaultCachingConfig) From d949c071bee0a9969955dc97834bdd168cabcd95 Mon Sep 17 00:00:00 2001 From: Maciej Kulawik Date: Tue, 14 May 2024 12:41:53 +0200 Subject: [PATCH 056/113] update geth --- go-ethereum | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/go-ethereum b/go-ethereum index ac85a19d5f..6d23a7b7e6 160000 --- a/go-ethereum +++ b/go-ethereum @@ -1 +1 @@ -Subproject commit ac85a19d5f56231076d5bab95504d666b084fa3b +Subproject commit 6d23a7b7e6a99701adf1f69701ad367dec61c08c From cb72afd4ba1977b41bc4587901bea052a31e9f54 Mon Sep 17 00:00:00 2001 From: Maciej Kulawik Date: Tue, 14 May 2024 17:16:28 +0200 Subject: [PATCH 057/113] set PebbleConfig defaults to geth / pebble defaults --- cmd/conf/database.go | 34 +++++++++++++++++----------------- cmd/nitro/nitro.go | 1 - 2 files changed, 17 insertions(+), 18 deletions(-) diff --git a/cmd/conf/database.go b/cmd/conf/database.go index d60ee51c5b..57674ba7f1 100644 --- a/cmd/conf/database.go +++ b/cmd/conf/database.go @@ -163,27 +163,27 @@ type PebbleExperimentalConfig struct { } var PebbleExperimentalConfigDefault = PebbleExperimentalConfig{ - BytesPerSync: 0, // pebble default will be used - L0CompactionFileThreshold: 0, // pebble default will be used - L0CompactionThreshold: 0, // pebble default will be used - L0StopWritesThreshold: 0, // pebble default will be used - LBaseMaxBytes: 0, // pebble default will be used + BytesPerSync: 512 << 10, // 512 KB + L0CompactionFileThreshold: 500, + L0CompactionThreshold: 4, + L0StopWritesThreshold: 12, + LBaseMaxBytes: 64 << 20, // 64 MB MemTableStopWritesThreshold: 2, DisableAutomaticCompactions: false, - WALBytesPerSync: 0, // pebble default will be used - WALDir: "", // default will use same dir as for sstables - WALMinSyncInterval: 0, // pebble default will be used - TargetByteDeletionRate: 0, // pebble default will be used + WALBytesPerSync: 0, // no background syncing + WALDir: "", // use same dir as for sstables + WALMinSyncInterval: 0, // no artificial delay + TargetByteDeletionRate: 0, // deletion pacing disabled - BlockSize: 4096, - IndexBlockSize: 4096, - TargetFileSize: 2 * 1024 * 1024, + BlockSize: 4 << 10, // 4 KB + IndexBlockSize: 4 << 10, // 4 KB + TargetFileSize: 2 << 20, // 2 MB TargetFileSizeEqualLevels: true, - L0CompactionConcurrency: 0, - CompactionDebtConcurrency: 0, - ReadCompactionRate: 0, - ReadSamplingMultiplier: -1, + L0CompactionConcurrency: 10, + CompactionDebtConcurrency: 1 << 30, // 1GB + ReadCompactionRate: 16000, // see ReadSamplingMultiplier comment + ReadSamplingMultiplier: -1, // geth default, disables read sampling and disables read triggered compaction MaxWriterConcurrency: 0, ForceWriterParallelism: false, } @@ -196,7 +196,7 @@ func PebbleExperimentalConfigAddOptions(prefix string, f *flag.FlagSet) { f.Int64(prefix+".l-base-max-bytes", PebbleExperimentalConfigDefault.LBaseMaxBytes, "The maximum number of bytes for LBase. The base level is the level which L0 is compacted into. The base level is determined dynamically based on the existing data in the LSM. The maximum number of bytes for other levels is computed dynamically based on the base level's maximum size. When the maximum number of bytes for a level is exceeded, compaction is requested.") f.Int(prefix+".mem-table-stop-writes-threshold", PebbleExperimentalConfigDefault.MemTableStopWritesThreshold, "hard limit on the number of queued of MemTables") f.Bool(prefix+".disable-automatic-compactions", PebbleExperimentalConfigDefault.DisableAutomaticCompactions, "disables automatic compactions") - f.Int(prefix+".wal-bytes-per-sync", PebbleExperimentalConfigDefault.WALBytesPerSync, "number of bytes to write to a write-ahead log (WAL) before calling Sync on it in the backgroud") + f.Int(prefix+".wal-bytes-per-sync", PebbleExperimentalConfigDefault.WALBytesPerSync, "number of bytes to write to a write-ahead log (WAL) before calling Sync on it in the background") f.String(prefix+".wal-dir", PebbleExperimentalConfigDefault.WALDir, "absolute path of directory to store write-ahead logs (WALs) in. If empty, WALs will be stored in the same directory as sstables") f.Int(prefix+".wal-min-sync-interval", PebbleExperimentalConfigDefault.WALMinSyncInterval, "minimum duration in microseconds between syncs of the WAL. If WAL syncs are requested faster than this interval, they will be artificially delayed.") f.Int(prefix+".target-byte-deletion-rate", PebbleExperimentalConfigDefault.TargetByteDeletionRate, "rate (in bytes per second) at which sstable file deletions are limited to (under normal circumstances).") diff --git a/cmd/nitro/nitro.go b/cmd/nitro/nitro.go index 434f36eeb2..9cf2a1a136 100644 --- a/cmd/nitro/nitro.go +++ b/cmd/nitro/nitro.go @@ -177,7 +177,6 @@ func mainImpl() int { nodeConfig.Auth.Apply(&stackConf) nodeConfig.IPC.Apply(&stackConf) nodeConfig.GraphQL.Apply(&stackConf) - if nodeConfig.WS.ExposeAll { stackConf.WSModules = append(stackConf.WSModules, "personal") } From 03ee1dc52e2f163b569b17b36edb96c65a04d9c2 Mon Sep 17 00:00:00 2001 From: Ganesh Vanahalli Date: Tue, 14 May 2024 10:38:30 -0500 Subject: [PATCH 058/113] address PR comments --- blocks_reexecutor/blocks_reexecutor.go | 26 ++++++++++++++------------ 1 file changed, 14 insertions(+), 12 deletions(-) diff --git a/blocks_reexecutor/blocks_reexecutor.go b/blocks_reexecutor/blocks_reexecutor.go index 0ad4337e0f..a03b29fefd 100644 --- a/blocks_reexecutor/blocks_reexecutor.go +++ b/blocks_reexecutor/blocks_reexecutor.go @@ -25,6 +25,8 @@ type Config struct { EndBlock uint64 `koanf:"end-block"` Room int `koanf:"room"` BlocksPerThread uint64 `koanf:"blocks-per-thread"` + + blocksPerThread uint64 } func (c *Config) Validate() error { @@ -35,8 +37,13 @@ func (c *Config) Validate() error { if c.EndBlock < c.StartBlock { return errors.New("invalid block range for blocks re-execution") } - if c.Room == 0 { - return errors.New("room for blocks re-execution cannot be zero") + if c.Room < 0 { + return errors.New("room for blocks re-execution should be greater than 0") + } + if c.BlocksPerThread != 0 { + c.blocksPerThread = c.BlocksPerThread + } else { + c.blocksPerThread = 10000 } return nil } @@ -52,6 +59,7 @@ var TestConfig = Config{ Mode: "full", Room: runtime.NumCPU(), BlocksPerThread: 10, + blocksPerThread: 10, } func ConfigAddOptions(prefix string, f *flag.FlagSet) { @@ -93,10 +101,7 @@ func New(c *Config, blockchain *core.BlockChain, fatalErrChan chan error) *Block } if c.Mode == "random" && end != start { // Reexecute a range of 10000 or (non-zero) c.BlocksPerThread number of blocks between start to end picked randomly - rng := uint64(10000) - if c.BlocksPerThread != 0 { - rng = c.BlocksPerThread - } + rng := c.blocksPerThread if rng > end-start { rng = end - start } @@ -108,12 +113,11 @@ func New(c *Config, blockchain *core.BlockChain, fatalErrChan chan error) *Block if start > 0 && start != chainStart { start-- } - // Divide work equally among available threads + // Divide work equally among available threads when BlocksPerThread is zero if c.BlocksPerThread == 0 { - c.BlocksPerThread = 10000 work := (end - start) / uint64(c.Room) if work > 0 { - c.BlocksPerThread = work + c.blocksPerThread = work } } return &BlocksReExecutor{ @@ -132,12 +136,10 @@ func New(c *Config, blockchain *core.BlockChain, fatalErrChan chan error) *Block // LaunchBlocksReExecution launches the thread to apply blocks of range [currentBlock-s.config.BlocksPerThread, currentBlock] to the last available valid state func (s *BlocksReExecutor) LaunchBlocksReExecution(ctx context.Context, currentBlock uint64) uint64 { - start := arbmath.SaturatingUSub(currentBlock, s.config.BlocksPerThread) + start := arbmath.SaturatingUSub(currentBlock, s.config.blocksPerThread) if start < s.startBlock { start = s.startBlock } - // we don't use state release pattern here - // TODO do we want to use release pattern here? startState, startHeader, release, err := arbitrum.FindLastAvailableState(ctx, s.blockchain, s.stateFor, s.blockchain.GetHeaderByNumber(start), nil, -1) if err != nil { s.fatalErrChan <- fmt.Errorf("blocksReExecutor failed to get last available state while searching for state at %d, err: %w", start, err) From 952d2253065590dbf846af103360e87c4eacc3e3 Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Tue, 14 May 2024 19:11:19 -0600 Subject: [PATCH 059/113] geth: update pin --- go-ethereum | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/go-ethereum b/go-ethereum index f8917436fc..8048ac4bed 160000 --- a/go-ethereum +++ b/go-ethereum @@ -1 +1 @@ -Subproject commit f8917436fcfa6a6a2b15c0ec7e6f318687491a8c +Subproject commit 8048ac4bed2eda18284e3c022ea5ee4cce771134 From 0c84ac6fe1638dc66fcf8ac5051ea457127063ae Mon Sep 17 00:00:00 2001 From: Nodar Ambroladze Date: Wed, 15 May 2024 15:22:41 +0200 Subject: [PATCH 060/113] Implement tracing and CPU profiling of long running block creations --- execution/gethexec/sequencer.go | 70 ++++++++++++++++++++++++++++++++- 1 file changed, 69 insertions(+), 1 deletion(-) diff --git a/execution/gethexec/sequencer.go b/execution/gethexec/sequencer.go index 23340594c4..da816c212f 100644 --- a/execution/gethexec/sequencer.go +++ b/execution/gethexec/sequencer.go @@ -9,13 +9,17 @@ import ( "fmt" "math" "math/big" + "os" "runtime/debug" + "runtime/pprof" + "runtime/trace" "strconv" "strings" "sync" "sync/atomic" "time" + "github.com/google/uuid" "github.com/offchainlabs/nitro/arbutil" "github.com/offchainlabs/nitro/execution" "github.com/offchainlabs/nitro/util/arbmath" @@ -76,6 +80,7 @@ type SequencerConfig struct { NonceFailureCacheExpiry time.Duration `koanf:"nonce-failure-cache-expiry" reload:"hot"` ExpectedSurplusSoftThreshold string `koanf:"expected-surplus-soft-threshold" reload:"hot"` ExpectedSurplusHardThreshold string `koanf:"expected-surplus-hard-threshold" reload:"hot"` + EnableProfiling bool `koanf:"enable-profiling"` expectedSurplusSoftThreshold int expectedSurplusHardThreshold int } @@ -125,6 +130,7 @@ var DefaultSequencerConfig = SequencerConfig{ NonceFailureCacheExpiry: time.Second, ExpectedSurplusSoftThreshold: "default", ExpectedSurplusHardThreshold: "default", + EnableProfiling: true, } var TestSequencerConfig = SequencerConfig{ @@ -142,6 +148,7 @@ var TestSequencerConfig = SequencerConfig{ NonceFailureCacheExpiry: time.Second, ExpectedSurplusSoftThreshold: "default", ExpectedSurplusHardThreshold: "default", + EnableProfiling: false, } func SequencerConfigAddOptions(prefix string, f *flag.FlagSet) { @@ -159,6 +166,7 @@ func SequencerConfigAddOptions(prefix string, f *flag.FlagSet) { f.Duration(prefix+".nonce-failure-cache-expiry", DefaultSequencerConfig.NonceFailureCacheExpiry, "maximum amount of time to wait for a predecessor before rejecting a tx with nonce too high") f.String(prefix+".expected-surplus-soft-threshold", DefaultSequencerConfig.ExpectedSurplusSoftThreshold, "if expected surplus is lower than this value, warnings are posted") f.String(prefix+".expected-surplus-hard-threshold", DefaultSequencerConfig.ExpectedSurplusHardThreshold, "if expected surplus is lower than this value, new incoming transactions will be denied") + f.Bool(prefix+".enable-profiling", DefaultSequencerConfig.EnableProfiling, "enable CPU profiling and tracing") } type txQueueItem struct { @@ -327,6 +335,7 @@ type Sequencer struct { expectedSurplusMutex sync.RWMutex expectedSurplus int64 expectedSurplusUpdated bool + enableProfiling bool } func NewSequencer(execEngine *ExecutionEngine, l1Reader *headerreader.HeaderReader, configFetcher SequencerConfigFetcher) (*Sequencer, error) { @@ -353,6 +362,7 @@ func NewSequencer(execEngine *ExecutionEngine, l1Reader *headerreader.HeaderRead l1Timestamp: 0, pauseChan: nil, onForwarderSet: make(chan struct{}, 1), + enableProfiling: config.EnableProfiling, } s.nonceFailures = &nonceFailureCache{ containers.NewLruCacheWithOnEvict(config.NonceCacheSize, s.onNonceFailureEvict), @@ -758,6 +768,59 @@ func (s *Sequencer) precheckNonces(queueItems []txQueueItem) []txQueueItem { return outputQueueItems } +func deleteFiles(files ...*os.File) { + for _, f := range files { + if err := os.Remove(f.Name()); err != nil { + log.Error("Error removing file", "name", f.Name()) + } + } +} + +func closeFiles(files ...*os.File) { + for _, f := range files { + if err := os.Remove(f.Name()); err != nil { + log.Error("Error closing file", "name", f.Name()) + } + } +} + +// createBlockWithProfiling runs create block with tracing and CPU profiling +// enabled. If the block creation takes longer than 5 seconds, it keeps both +// and prints out filenames in an error log line. +func (s *Sequencer) createBlockWithProfiling(ctx context.Context) bool { + id := uuid.NewString() + pprofFile, err := os.CreateTemp("", id+".pprof") + if err != nil { + log.Error("Creating temporary file for profiling CPU", "error", err) + return false + } + traceFile, err := os.CreateTemp("", id+".trace") + if err != nil { + log.Error("Creating temporary file for tracing", "error", err) + return false + } + if err := pprof.StartCPUProfile(pprofFile); err != nil { + log.Error("Starting CPU profiling", "error", err) + deleteFiles(pprofFile, traceFile) + return false + } + if err := trace.Start(traceFile); err != nil { + log.Error("Starting tracing", "error", err) + } + start := time.Now() + res := s.createBlock(ctx) + elapsed := time.Since(start) + pprof.StopCPUProfile() + trace.Stop() + closeFiles(pprofFile, traceFile) + if elapsed > 5*time.Second { + log.Error("Block creation took longer than 5 seconds", "pprof", pprofFile.Name()) + return res + } + deleteFiles(pprofFile, traceFile) + return res +} + func (s *Sequencer) createBlock(ctx context.Context) (returnValue bool) { var queueItems []txQueueItem var totalBatchSize int @@ -1088,7 +1151,12 @@ func (s *Sequencer) Start(ctxIn context.Context) error { s.CallIteratively(func(ctx context.Context) time.Duration { nextBlock := time.Now().Add(s.config().MaxBlockSpeed) - madeBlock := s.createBlock(ctx) + var madeBlock bool + if s.enableProfiling { + s.createBlockWithProfiling(ctx) + } else { + madeBlock = s.createBlock(ctx) + } if madeBlock { // Note: this may return a negative duration, but timers are fine with that (they treat negative durations as 0). return time.Until(nextBlock) From a1403d0698b534cbd441d1d75496db95e9eb2bdd Mon Sep 17 00:00:00 2001 From: Nodar Ambroladze Date: Wed, 15 May 2024 15:27:16 +0200 Subject: [PATCH 061/113] Don't abort block creation if profiling fails --- execution/gethexec/sequencer.go | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/execution/gethexec/sequencer.go b/execution/gethexec/sequencer.go index da816c212f..2724095154 100644 --- a/execution/gethexec/sequencer.go +++ b/execution/gethexec/sequencer.go @@ -792,19 +792,17 @@ func (s *Sequencer) createBlockWithProfiling(ctx context.Context) bool { pprofFile, err := os.CreateTemp("", id+".pprof") if err != nil { log.Error("Creating temporary file for profiling CPU", "error", err) - return false } traceFile, err := os.CreateTemp("", id+".trace") if err != nil { log.Error("Creating temporary file for tracing", "error", err) - return false } if err := pprof.StartCPUProfile(pprofFile); err != nil { log.Error("Starting CPU profiling", "error", err) - deleteFiles(pprofFile, traceFile) - return false + deleteFiles(pprofFile) } if err := trace.Start(traceFile); err != nil { + deleteFiles(traceFile) log.Error("Starting tracing", "error", err) } start := time.Now() From 3b33bc4f74356e95bdf98d51ebd3e3df09fe92df Mon Sep 17 00:00:00 2001 From: Nodar Ambroladze Date: Wed, 15 May 2024 15:28:02 +0200 Subject: [PATCH 062/113] Set madeblock correctly --- execution/gethexec/sequencer.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/execution/gethexec/sequencer.go b/execution/gethexec/sequencer.go index 2724095154..4247556905 100644 --- a/execution/gethexec/sequencer.go +++ b/execution/gethexec/sequencer.go @@ -1151,7 +1151,7 @@ func (s *Sequencer) Start(ctxIn context.Context) error { nextBlock := time.Now().Add(s.config().MaxBlockSpeed) var madeBlock bool if s.enableProfiling { - s.createBlockWithProfiling(ctx) + madeBlock = s.createBlockWithProfiling(ctx) } else { madeBlock = s.createBlock(ctx) } From e35c1c0d43cdc45a467767acdac326b1c8adf2ed Mon Sep 17 00:00:00 2001 From: Maciej Kulawik Date: Wed, 15 May 2024 21:02:31 +0200 Subject: [PATCH 063/113] clean up TODOs --- system_tests/common_test.go | 2 -- system_tests/das_test.go | 1 - system_tests/pruning_test.go | 1 - system_tests/staterecovery_test.go | 1 - 4 files changed, 5 deletions(-) diff --git a/system_tests/common_test.go b/system_tests/common_test.go index 7a5296516e..f8ba4c8b77 100644 --- a/system_tests/common_test.go +++ b/system_tests/common_test.go @@ -773,7 +773,6 @@ func createL2BlockChainWithStackConfig( stack, err = node.New(stackConfig) Require(t, err) - // TODO get pebble.ExtraOptions from conf.PersistentConfig when opening the DBs chainDb, err := stack.OpenDatabaseWithExtraOptions("l2chaindata", 0, 0, "l2chaindata/", false, conf.PersistentConfigDefault.Pebble.ExtraOptions("l2chaindata")) Require(t, err) arbDb, err := stack.OpenDatabaseWithExtraOptions("arbitrumdata", 0, 0, "arbitrumdata/", false, conf.PersistentConfigDefault.Pebble.ExtraOptions("arbitrumdata")) @@ -978,7 +977,6 @@ func Create2ndNodeWithConfig( l2stack, err := node.New(stackConfig) Require(t, err) - // TODO get pebble.ExtraOptions from conf.PersistentConfig when opening the DBs l2chainDb, err := l2stack.OpenDatabaseWithExtraOptions("l2chaindata", 0, 0, "l2chaindata/", false, conf.PersistentConfigDefault.Pebble.ExtraOptions("l2chaindata")) Require(t, err) l2arbDb, err := l2stack.OpenDatabaseWithExtraOptions("arbitrumdata", 0, 0, "arbitrumdata/", false, conf.PersistentConfigDefault.Pebble.ExtraOptions("arbitrumdata")) diff --git a/system_tests/das_test.go b/system_tests/das_test.go index 2febadb3d2..11d887315a 100644 --- a/system_tests/das_test.go +++ b/system_tests/das_test.go @@ -178,7 +178,6 @@ func TestDASRekey(t *testing.T) { l2stackA, err := node.New(stackConfig) Require(t, err) - // TODO get pebble.ExtraOptions from conf.PersistentConfig l2chainDb, err := l2stackA.OpenDatabaseWithExtraOptions("l2chaindata", 0, 0, "l2chaindata/", false, conf.PersistentConfigDefault.Pebble.ExtraOptions("l2chaindata")) Require(t, err) diff --git a/system_tests/pruning_test.go b/system_tests/pruning_test.go index d2453887ee..041781ac48 100644 --- a/system_tests/pruning_test.go +++ b/system_tests/pruning_test.go @@ -65,7 +65,6 @@ func TestPruning(t *testing.T) { stack, err := node.New(builder.l2StackConfig) Require(t, err) defer stack.Close() - // TODO get pebble.ExtraOptions from conf.PersistentConfig chainDb, err := stack.OpenDatabaseWithExtraOptions("l2chaindata", 0, 0, "l2chaindata/", false, conf.PersistentConfigDefault.Pebble.ExtraOptions("l2chaindata")) Require(t, err) defer chainDb.Close() diff --git a/system_tests/staterecovery_test.go b/system_tests/staterecovery_test.go index 459a6e3ee8..a20cffc787 100644 --- a/system_tests/staterecovery_test.go +++ b/system_tests/staterecovery_test.go @@ -50,7 +50,6 @@ func TestRectreateMissingStates(t *testing.T) { stack, err := node.New(builder.l2StackConfig) Require(t, err) defer stack.Close() - // TODO get pebble.ExtraOptions from conf.PersistentConfig chainDb, err := stack.OpenDatabaseWithExtraOptions("l2chaindata", 0, 0, "l2chaindata/", false, conf.PersistentConfigDefault.Pebble.ExtraOptions("l2chaindata")) Require(t, err) defer chainDb.Close() From b426fdd5ec0cd6ade88de0eadd17caf7202083cb Mon Sep 17 00:00:00 2001 From: Maciej Kulawik Date: Wed, 15 May 2024 21:26:21 +0200 Subject: [PATCH 064/113] update geth --- go-ethereum | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/go-ethereum b/go-ethereum index 6d23a7b7e6..5b7b36a339 160000 --- a/go-ethereum +++ b/go-ethereum @@ -1 +1 @@ -Subproject commit 6d23a7b7e6a99701adf1f69701ad367dec61c08c +Subproject commit 5b7b36a339ac28d708bca072dc5ec8189ceadac2 From a8254afca62deb6bdcda7533df434093df438734 Mon Sep 17 00:00:00 2001 From: Diego Ximenes Date: Wed, 15 May 2024 19:19:25 -0300 Subject: [PATCH 065/113] avoid hardicoding adresses and l2msg in tests --- execution/gethexec/executionengine.go | 4 +-- system_tests/seqfeed_test.go | 41 ++++++++++++++------------- 2 files changed, 24 insertions(+), 21 deletions(-) diff --git a/execution/gethexec/executionengine.go b/execution/gethexec/executionengine.go index b31209b882..96dca6c63e 100644 --- a/execution/gethexec/executionengine.go +++ b/execution/gethexec/executionengine.go @@ -197,7 +197,7 @@ func (s *ExecutionEngine) NextDelayedMessageNumber() (uint64, error) { return currentHeader.Nonce.Uint64(), nil } -func messageFromTxes(header *arbostypes.L1IncomingMessageHeader, txes types.Transactions, txErrors []error) (*arbostypes.L1IncomingMessage, error) { +func MessageFromTxes(header *arbostypes.L1IncomingMessageHeader, txes types.Transactions, txErrors []error) (*arbostypes.L1IncomingMessage, error) { var l2Message []byte if len(txes) == 1 && txErrors[0] == nil { txBytes, err := txes[0].MarshalBinary() @@ -368,7 +368,7 @@ func (s *ExecutionEngine) sequenceTransactionsWithBlockMutex(header *arbostypes. return nil, nil } - msg, err := messageFromTxes(header, txes, hooks.TxErrors) + msg, err := MessageFromTxes(header, txes, hooks.TxErrors) if err != nil { return nil, err } diff --git a/system_tests/seqfeed_test.go b/system_tests/seqfeed_test.go index 946194f17d..ed0398c40e 100644 --- a/system_tests/seqfeed_test.go +++ b/system_tests/seqfeed_test.go @@ -12,12 +12,15 @@ import ( "time" "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/log" "github.com/offchainlabs/nitro/arbnode" "github.com/offchainlabs/nitro/arbos/arbostypes" + "github.com/offchainlabs/nitro/arbos/l1pricing" "github.com/offchainlabs/nitro/broadcastclient" "github.com/offchainlabs/nitro/broadcaster/backlog" "github.com/offchainlabs/nitro/broadcaster/message" + "github.com/offchainlabs/nitro/execution/gethexec" "github.com/offchainlabs/nitro/relay" "github.com/offchainlabs/nitro/util/signature" "github.com/offchainlabs/nitro/util/testhelpers" @@ -297,41 +300,41 @@ func testBlockHashComparison(t *testing.T, blockHash *common.Hash, mustMismatch defer cleanup() testClient := builder.L2 - // related to: - // - builder.L2Info.PrepareTx("Owner", "User2", builder.L2Info.TransferGas, big.NewInt(1e12), nil) userAccount := "User2" - txHash := common.HexToHash("0x633f62b463cc0e52d842406995fb590654db40aace77bfca863ba0e8d2290f97") - poster := common.HexToAddress("0xa4b000000000000000000073657175656e636572") - l2msg := []byte{4, 2, 248, 111, 131, 6, 74, 186, 128, 128, 132, 11, 235, 194, 0, 131, 122, 18, 0, 148, 12, 112, 159, 52, 15, 11, 178, 227, 97, 34, 158, 52, 91, 126, 38, 153, 157, 9, 105, 171, 133, 232, 212, 165, 16, 0, 128, 192, 1, 160, 75, 109, 200, 183, 223, 114, 85, 128, 133, 94, 26, 103, 145, 247, 47, 0, 114, 132, 133, 234, 222, 235, 102, 45, 2, 109, 83, 65, 210, 142, 242, 209, 160, 96, 90, 108, 188, 197, 195, 43, 222, 103, 155, 153, 81, 119, 74, 177, 103, 110, 134, 94, 221, 72, 236, 20, 86, 94, 226, 94, 5, 206, 196, 122, 119} + builder.L2Info.GenerateAccount(userAccount) + tx := builder.L2Info.PrepareTx("Owner", userAccount, builder.L2Info.TransferGas, big.NewInt(1e12), nil) + l1IncomingMsgHeader := arbostypes.L1IncomingMessageHeader{ + Kind: arbostypes.L1MessageType_L2Message, + Poster: l1pricing.BatchPosterAddress, + BlockNumber: 29, + Timestamp: 1715295980, + RequestId: nil, + L1BaseFee: nil, + } + l1IncomingMsg, err := gethexec.MessageFromTxes( + &l1IncomingMsgHeader, + types.Transactions{tx}, + []error{nil}, + ) + Require(t, err) + broadcastMessage := message.BroadcastMessage{ Version: 1, Messages: []*message.BroadcastFeedMessage{ { SequenceNumber: 1, Message: arbostypes.MessageWithMetadata{ - Message: &arbostypes.L1IncomingMessage{ - Header: &arbostypes.L1IncomingMessageHeader{ - Kind: arbostypes.L1MessageType_L2Message, - Poster: poster, - BlockNumber: 29, - Timestamp: 1715295980, - RequestId: nil, - L1BaseFee: nil, - }, - L2msg: l2msg, - }, + Message: l1IncomingMsg, DelayedMessagesRead: 1, }, BlockHash: blockHash, - Signature: nil, }, }, } wsBroadcastServer.Broadcast(&broadcastMessage) // By now, even though block hash mismatch, the transaction should still be processed - builder.L2Info.GenerateAccount(userAccount) - _, err = WaitForTx(ctx, testClient.Client, txHash, time.Second*15) + _, err = WaitForTx(ctx, testClient.Client, tx.Hash(), time.Second*15) if err != nil { t.Fatal("error waiting for tx:", err) } From b03d7b34eac6f0c4456914ee6b2fbe6ef7ba03a4 Mon Sep 17 00:00:00 2001 From: Diego Ximenes Date: Wed, 15 May 2024 19:39:04 -0300 Subject: [PATCH 066/113] improve log message when computed block hash doesn't match hash provided through input feed --- arbnode/transaction_streamer.go | 10 +++++++--- system_tests/seq_coordinator_test.go | 4 ++-- system_tests/seqfeed_test.go | 6 +++--- 3 files changed, 12 insertions(+), 8 deletions(-) diff --git a/arbnode/transaction_streamer.go b/arbnode/transaction_streamer.go index d9c7fc2163..b79b1aa963 100644 --- a/arbnode/transaction_streamer.go +++ b/arbnode/transaction_streamer.go @@ -146,6 +146,10 @@ type blockHashDBValue struct { BlockHash *common.Hash `rlp:"nil"` } +const ( + BlockHashMismatchLogMsg = "BlockHash from feed doesn't match locally computed hash. Check feed source." +) + func (s *TransactionStreamer) CurrentEstimateOfL1GasPrice() uint64 { s.cachedL1PriceDataMutex.Lock() defer s.cachedL1PriceDataMutex.Unlock() @@ -547,8 +551,8 @@ func (s *TransactionStreamer) getMessageWithMetadataAndBlockHash(seqNum arbutil. } // Get block hash. - // To keep it backwards compatible it is possible that a message related - // to a sequence number exists in the database but the block hash doesn't. + // To keep it backwards compatible, since it is possible that a message related + // to a sequence number exists in the database, but the block hash doesn't. key := dbKey(blockHashInputFeedPrefix, uint64(seqNum)) var blockHash *common.Hash data, err := s.db.Get(key) @@ -1170,7 +1174,7 @@ func (s *TransactionStreamer) checkResult(msgResult *execution.MessageResult, ex } if msgResult.BlockHash != *expectedBlockHash { log.Error( - "block_hash_mismatch", + BlockHashMismatchLogMsg, "expected", expectedBlockHash, "actual", msgResult.BlockHash, ) diff --git a/system_tests/seq_coordinator_test.go b/system_tests/seq_coordinator_test.go index a069a2d5a1..43d55f40c9 100644 --- a/system_tests/seq_coordinator_test.go +++ b/system_tests/seq_coordinator_test.go @@ -354,8 +354,8 @@ func testCoordinatorMessageSync(t *testing.T, successCase bool) { t.Fatal("Unexpected balance:", l2balance) } - if logHandler.WasLogged("block_hash_mismatch") { - t.Fatal("block_hash_mismatch was logged unexpectedly") + if logHandler.WasLogged(arbnode.BlockHashMismatchLogMsg) { + t.Fatal("BlockHashMismatchLogMsg was logged unexpectedly") } } else { _, err = WaitForTx(ctx, testClientB.Client, tx.Hash(), time.Second) diff --git a/system_tests/seqfeed_test.go b/system_tests/seqfeed_test.go index ed0398c40e..589a48d3af 100644 --- a/system_tests/seqfeed_test.go +++ b/system_tests/seqfeed_test.go @@ -346,11 +346,11 @@ func testBlockHashComparison(t *testing.T, blockHash *common.Hash, mustMismatch t.Fatal("Unexpected balance:", l2balance) } - mismatched := logHandler.WasLogged("block_hash_mismatch") + mismatched := logHandler.WasLogged(arbnode.BlockHashMismatchLogMsg) if mustMismatch && !mismatched { - t.Fatal("Failed to log block_hash_mismatch") + t.Fatal("Failed to log BlockHashMismatchLogMsg") } else if !mustMismatch && mismatched { - t.Fatal("block_hash_mismatch was logged unexpectedly") + t.Fatal("BlockHashMismatchLogMsg was logged unexpectedly") } } From 4f72ebb168ad60d8868c0de1d82898ff8f33f9a7 Mon Sep 17 00:00:00 2001 From: Lee Bousfield Date: Wed, 15 May 2024 23:45:23 -0500 Subject: [PATCH 067/113] Fix signed saturating math functions --- util/arbmath/math.go | 44 ++++++++----- util/arbmath/math_fuzz_test.go | 112 +++++++++++++++++++++++++++++++++ util/arbmath/math_test.go | 109 ++++++++++++++++++++++++++++++++ 3 files changed, 250 insertions(+), 15 deletions(-) create mode 100644 util/arbmath/math_fuzz_test.go diff --git a/util/arbmath/math.go b/util/arbmath/math.go index 1c11c6ad58..8f93caa87d 100644 --- a/util/arbmath/math.go +++ b/util/arbmath/math.go @@ -74,14 +74,6 @@ func MaxInt[T Number](values ...T) T { return max } -// AbsValue the absolute value of a number -func AbsValue[T Number](value T) T { - if value < 0 { - return -value // never happens for unsigned types - } - return value -} - // Checks if two ints are sufficiently close to one another func Within[T Unsigned](a, b, bound T) bool { min := MinInt(a, b) @@ -267,14 +259,32 @@ func BigFloatMulByUint(multiplicand *big.Float, multiplier uint64) *big.Float { return new(big.Float).Mul(multiplicand, UintToBigFloat(multiplier)) } +func MaxIntValue[T Integer]() T { + allBits := ^T(0) + if allBits < 0 { + // This is a signed integer + return T((uint64(1) << (8*unsafe.Sizeof(allBits) - 1)) - 1) + } + return allBits +} + +func MinIntValue[T Integer]() T { + allBits := ^T(0) + if allBits < 0 { + // This is a signed integer + return T(uint64(1) << ((8 * unsafe.Sizeof(allBits)) - 1)) + } + return 0 +} + // SaturatingAdd add two integers without overflow func SaturatingAdd[T Signed](a, b T) T { sum := a + b if b > 0 && sum < a { - sum = ^T(0) >> 1 + sum = MaxIntValue[T]() } if b < 0 && sum > a { - sum = (^T(0) >> 1) + 1 + sum = MinIntValue[T]() } return sum } @@ -290,7 +300,11 @@ func SaturatingUAdd[T Unsigned](a, b T) T { // SaturatingSub subtract an int64 from another without overflow func SaturatingSub(minuend, subtrahend int64) int64 { - return SaturatingAdd(minuend, -subtrahend) + if subtrahend == math.MinInt64 { + // The absolute value of MinInt64 is one greater than MaxInt64 + return SaturatingAdd(SaturatingAdd(minuend, math.MaxInt64), 1) + } + return SaturatingAdd(minuend, SaturatingNeg(subtrahend)) } // SaturatingUSub subtract an integer from another without underflow @@ -315,9 +329,9 @@ func SaturatingMul[T Signed](a, b T) T { product := a * b if b != 0 && product/b != a { if (a > 0 && b > 0) || (a < 0 && b < 0) { - product = ^T(0) >> 1 + product = MaxIntValue[T]() } else { - product = (^T(0) >> 1) + 1 + product = MinIntValue[T]() } } return product @@ -367,8 +381,8 @@ func SaturatingCastToUint(value *big.Int) uint64 { // Negates an int without underflow func SaturatingNeg[T Signed](value T) T { - if value == ^T(0) { - return (^T(0)) >> 1 + if value < 0 && value == MinIntValue[T]() { + return MaxIntValue[T]() } return -value } diff --git a/util/arbmath/math_fuzz_test.go b/util/arbmath/math_fuzz_test.go new file mode 100644 index 0000000000..6e27f2b70a --- /dev/null +++ b/util/arbmath/math_fuzz_test.go @@ -0,0 +1,112 @@ +// Copyright 2024, Offchain Labs, Inc. +// For license information, see https://github.com/nitro/blob/master/LICENSE + +package arbmath + +import ( + "math/big" + "testing" +) + +func toBig[T Signed](a T) *big.Int { + return big.NewInt(int64(a)) +} + +func saturatingBigToInt[T Signed](a *big.Int) T { + // MinIntValue and MaxIntValue are already separately tested + if a.Cmp(toBig(MaxIntValue[T]())) > 0 { + return MaxIntValue[T]() + } + if a.Cmp(toBig(MinIntValue[T]())) < 0 { + return MinIntValue[T]() + } + return T(a.Int64()) +} + +func fuzzSaturatingAdd[T Signed](f *testing.F) { + f.Fuzz(func(t *testing.T, a, b T) { + got := SaturatingAdd(a, b) + expected := saturatingBigToInt[T](new(big.Int).Add(toBig(a), toBig(b))) + if got != expected { + t.Errorf("SaturatingAdd(%v, %v) = %v, expected %v", a, b, got, expected) + } + }) +} + +func fuzzSaturatingMul[T Signed](f *testing.F) { + f.Fuzz(func(t *testing.T, a, b T) { + got := SaturatingMul(a, b) + expected := saturatingBigToInt[T](new(big.Int).Mul(toBig(a), toBig(b))) + if got != expected { + t.Errorf("SaturatingMul(%v, %v) = %v, expected %v", a, b, got, expected) + } + }) +} + +func fuzzSaturatingNeg[T Signed](f *testing.F) { + f.Fuzz(func(t *testing.T, a T) { + got := SaturatingNeg(a) + expected := saturatingBigToInt[T](new(big.Int).Neg(toBig(a))) + if got != expected { + t.Errorf("SaturatingNeg(%v) = %v, expected %v", a, got, expected) + } + }) +} + +func FuzzSaturatingAddInt8(f *testing.F) { + fuzzSaturatingAdd[int8](f) +} + +func FuzzSaturatingAddInt16(f *testing.F) { + fuzzSaturatingAdd[int16](f) +} + +func FuzzSaturatingAddInt32(f *testing.F) { + fuzzSaturatingAdd[int32](f) +} + +func FuzzSaturatingAddInt64(f *testing.F) { + fuzzSaturatingAdd[int64](f) +} + +func FuzzSaturatingSub(f *testing.F) { + f.Fuzz(func(t *testing.T, a, b int64) { + got := SaturatingSub(a, b) + expected := saturatingBigToInt[int64](new(big.Int).Sub(toBig(a), toBig(b))) + if got != expected { + t.Errorf("SaturatingSub(%v, %v) = %v, expected %v", a, b, got, expected) + } + }) +} + +func FuzzSaturatingMulInt8(f *testing.F) { + fuzzSaturatingMul[int8](f) +} + +func FuzzSaturatingMulInt16(f *testing.F) { + fuzzSaturatingMul[int16](f) +} + +func FuzzSaturatingMulInt32(f *testing.F) { + fuzzSaturatingMul[int32](f) +} + +func FuzzSaturatingMulInt64(f *testing.F) { + fuzzSaturatingMul[int64](f) +} + +func FuzzSaturatingNegInt8(f *testing.F) { + fuzzSaturatingNeg[int8](f) +} + +func FuzzSaturatingNegInt16(f *testing.F) { + fuzzSaturatingNeg[int16](f) +} + +func FuzzSaturatingNegInt32(f *testing.F) { + fuzzSaturatingNeg[int32](f) +} + +func FuzzSaturatingNegInt64(f *testing.F) { + fuzzSaturatingNeg[int64](f) +} diff --git a/util/arbmath/math_test.go b/util/arbmath/math_test.go index 2e2f14795a..194d6d7c86 100644 --- a/util/arbmath/math_test.go +++ b/util/arbmath/math_test.go @@ -5,6 +5,7 @@ package arbmath import ( "bytes" + "fmt" "math" "math/rand" "testing" @@ -120,6 +121,114 @@ func TestSlices(t *testing.T) { assert_eq(SliceWithRunoff(data, 7, 8), []uint8{}) } +func testMinMaxValues[T Integer](t *testing.T, min T, max T) { + gotMin := MinIntValue[T]() + if gotMin != min { + Fail(t, "expected min", min, "but got", gotMin) + } + gotMax := MaxIntValue[T]() + if gotMax != max { + Fail(t, "expected max", max, "but got", gotMax) + } +} + +func TestMinMaxValues(t *testing.T) { + testMinMaxValues[uint8](t, 0, math.MaxUint8) + testMinMaxValues[uint16](t, 0, math.MaxUint16) + testMinMaxValues[uint32](t, 0, math.MaxUint32) + testMinMaxValues[uint64](t, 0, math.MaxUint64) + testMinMaxValues[int8](t, math.MinInt8, math.MaxInt8) + testMinMaxValues[int16](t, math.MinInt16, math.MaxInt16) + testMinMaxValues[int32](t, math.MinInt32, math.MaxInt32) + testMinMaxValues[int64](t, math.MinInt64, math.MaxInt64) +} + +func TestSaturatingAdd(t *testing.T) { + tests := []struct { + a, b, expected int64 + }{ + {2, 3, 5}, + {-1, -2, -3}, + {math.MaxInt64, 1, math.MaxInt64}, + {math.MaxInt64, math.MaxInt64, math.MaxInt64}, + {math.MinInt64, -1, math.MinInt64}, + {math.MinInt64, math.MinInt64, math.MinInt64}, + } + + for _, tc := range tests { + t.Run(fmt.Sprintf("%v + %v = %v", tc.a, tc.b, tc.expected), func(t *testing.T) { + sum := SaturatingAdd(int64(tc.a), int64(tc.b)) + if sum != tc.expected { + t.Errorf("SaturatingAdd(%v, %v) = %v; want %v", tc.a, tc.b, sum, tc.expected) + } + }) + } +} + +func TestSaturatingSub(t *testing.T) { + tests := []struct { + a, b, expected int64 + }{ + {5, 3, 2}, + {-3, -2, -1}, + {math.MinInt64, 1, math.MinInt64}, + {math.MinInt64, -1, math.MinInt64 + 1}, + {math.MinInt64, math.MinInt64, 0}, + {0, math.MinInt64, math.MaxInt64}, + } + + for _, tc := range tests { + t.Run("", func(t *testing.T) { + sum := SaturatingSub(int64(tc.a), int64(tc.b)) + if sum != tc.expected { + t.Errorf("SaturatingSub(%v, %v) = %v; want %v", tc.a, tc.b, sum, tc.expected) + } + }) + } +} + +func TestSaturatingMul(t *testing.T) { + tests := []struct { + a, b, expected int64 + }{ + {5, 3, 15}, + {-3, -2, 6}, + {math.MaxInt64, 2, math.MaxInt64}, + {math.MinInt64, 2, math.MinInt64}, + } + + for _, tc := range tests { + t.Run(fmt.Sprintf("%v - %v = %v", tc.a, tc.b, tc.expected), func(t *testing.T) { + sum := SaturatingMul(int64(tc.a), int64(tc.b)) + if sum != tc.expected { + t.Errorf("SaturatingMul(%v, %v) = %v; want %v", tc.a, tc.b, sum, tc.expected) + } + }) + } +} + +func TestSaturatingNeg(t *testing.T) { + tests := []struct { + value int64 + expected int64 + }{ + {0, 0}, + {5, -5}, + {-5, 5}, + {math.MinInt64, math.MaxInt64}, + {math.MaxInt64, math.MinInt64 + 1}, + } + + for _, tc := range tests { + t.Run(fmt.Sprintf("-%v = %v", tc.value, tc.expected), func(t *testing.T) { + result := SaturatingNeg(tc.value) + if result != tc.expected { + t.Errorf("SaturatingNeg(%v) = %v: expected %v", tc.value, result, tc.expected) + } + }) + } +} + func Fail(t *testing.T, printables ...interface{}) { t.Helper() testhelpers.FailImpl(t, printables...) From bd880e9c340667c08e9fe9cbb9577b8067198016 Mon Sep 17 00:00:00 2001 From: Lee Bousfield Date: Thu, 16 May 2024 10:37:55 -0500 Subject: [PATCH 068/113] Limit min/max functions to Signed --- util/arbmath/math.go | 30 ++++++++++-------------------- util/arbmath/math_fuzz_test.go | 10 +++++----- util/arbmath/math_test.go | 20 ++++++++------------ 3 files changed, 23 insertions(+), 37 deletions(-) diff --git a/util/arbmath/math.go b/util/arbmath/math.go index 8f93caa87d..d7a0d1f523 100644 --- a/util/arbmath/math.go +++ b/util/arbmath/math.go @@ -259,32 +259,22 @@ func BigFloatMulByUint(multiplicand *big.Float, multiplier uint64) *big.Float { return new(big.Float).Mul(multiplicand, UintToBigFloat(multiplier)) } -func MaxIntValue[T Integer]() T { - allBits := ^T(0) - if allBits < 0 { - // This is a signed integer - return T((uint64(1) << (8*unsafe.Sizeof(allBits) - 1)) - 1) - } - return allBits +func MaxSignedValue[T Signed]() T { + return T((uint64(1) << (8*unsafe.Sizeof(T(0)) - 1)) - 1) } -func MinIntValue[T Integer]() T { - allBits := ^T(0) - if allBits < 0 { - // This is a signed integer - return T(uint64(1) << ((8 * unsafe.Sizeof(allBits)) - 1)) - } - return 0 +func MinSignedValue[T Signed]() T { + return T(uint64(1) << ((8 * unsafe.Sizeof(T(0))) - 1)) } // SaturatingAdd add two integers without overflow func SaturatingAdd[T Signed](a, b T) T { sum := a + b if b > 0 && sum < a { - sum = MaxIntValue[T]() + sum = MaxSignedValue[T]() } if b < 0 && sum > a { - sum = MinIntValue[T]() + sum = MinSignedValue[T]() } return sum } @@ -329,9 +319,9 @@ func SaturatingMul[T Signed](a, b T) T { product := a * b if b != 0 && product/b != a { if (a > 0 && b > 0) || (a < 0 && b < 0) { - product = MaxIntValue[T]() + product = MaxSignedValue[T]() } else { - product = MinIntValue[T]() + product = MinSignedValue[T]() } } return product @@ -381,8 +371,8 @@ func SaturatingCastToUint(value *big.Int) uint64 { // Negates an int without underflow func SaturatingNeg[T Signed](value T) T { - if value < 0 && value == MinIntValue[T]() { - return MaxIntValue[T]() + if value < 0 && value == MinSignedValue[T]() { + return MaxSignedValue[T]() } return -value } diff --git a/util/arbmath/math_fuzz_test.go b/util/arbmath/math_fuzz_test.go index 6e27f2b70a..591d699de0 100644 --- a/util/arbmath/math_fuzz_test.go +++ b/util/arbmath/math_fuzz_test.go @@ -13,12 +13,12 @@ func toBig[T Signed](a T) *big.Int { } func saturatingBigToInt[T Signed](a *big.Int) T { - // MinIntValue and MaxIntValue are already separately tested - if a.Cmp(toBig(MaxIntValue[T]())) > 0 { - return MaxIntValue[T]() + // MinSignedValue and MaxSignedValue are already separately tested + if a.Cmp(toBig(MaxSignedValue[T]())) > 0 { + return MaxSignedValue[T]() } - if a.Cmp(toBig(MinIntValue[T]())) < 0 { - return MinIntValue[T]() + if a.Cmp(toBig(MinSignedValue[T]())) < 0 { + return MinSignedValue[T]() } return T(a.Int64()) } diff --git a/util/arbmath/math_test.go b/util/arbmath/math_test.go index 194d6d7c86..1be60dc58b 100644 --- a/util/arbmath/math_test.go +++ b/util/arbmath/math_test.go @@ -121,26 +121,22 @@ func TestSlices(t *testing.T) { assert_eq(SliceWithRunoff(data, 7, 8), []uint8{}) } -func testMinMaxValues[T Integer](t *testing.T, min T, max T) { - gotMin := MinIntValue[T]() +func testMinMaxSignedValues[T Signed](t *testing.T, min T, max T) { + gotMin := MinSignedValue[T]() if gotMin != min { Fail(t, "expected min", min, "but got", gotMin) } - gotMax := MaxIntValue[T]() + gotMax := MaxSignedValue[T]() if gotMax != max { Fail(t, "expected max", max, "but got", gotMax) } } -func TestMinMaxValues(t *testing.T) { - testMinMaxValues[uint8](t, 0, math.MaxUint8) - testMinMaxValues[uint16](t, 0, math.MaxUint16) - testMinMaxValues[uint32](t, 0, math.MaxUint32) - testMinMaxValues[uint64](t, 0, math.MaxUint64) - testMinMaxValues[int8](t, math.MinInt8, math.MaxInt8) - testMinMaxValues[int16](t, math.MinInt16, math.MaxInt16) - testMinMaxValues[int32](t, math.MinInt32, math.MaxInt32) - testMinMaxValues[int64](t, math.MinInt64, math.MaxInt64) +func TestMinMaxSignedValues(t *testing.T) { + testMinMaxSignedValues[int8](t, math.MinInt8, math.MaxInt8) + testMinMaxSignedValues[int16](t, math.MinInt16, math.MaxInt16) + testMinMaxSignedValues[int32](t, math.MinInt32, math.MaxInt32) + testMinMaxSignedValues[int64](t, math.MinInt64, math.MaxInt64) } func TestSaturatingAdd(t *testing.T) { From bf8f40a57cefea50899bb15caa989c6ec0bc40c6 Mon Sep 17 00:00:00 2001 From: Nodar Ambroladze Date: Thu, 16 May 2024 18:09:52 +0200 Subject: [PATCH 069/113] Skip tls verification when making requests to secure signer from Dataposter --- arbnode/dataposter/data_poster.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/arbnode/dataposter/data_poster.go b/arbnode/dataposter/data_poster.go index 7bc18a2121..04789d0fda 100644 --- a/arbnode/dataposter/data_poster.go +++ b/arbnode/dataposter/data_poster.go @@ -217,6 +217,10 @@ func NewDataPoster(ctx context.Context, opts *DataPosterOpts) (*DataPoster, erro func rpcClient(ctx context.Context, opts *ExternalSignerCfg) (*rpc.Client, error) { tlsCfg := &tls.Config{ MinVersion: tls.VersionTLS12, + // Dataposter verifies that signed transaction was signed by the account + // that it expects to be signed with. So signer is already authenticated + // on application level and does not need to rely on TLS for authentication. + InsecureSkipVerify: true, // #nosec G402 } if opts.ClientCert != "" && opts.ClientPrivateKey != "" { From 761e98dfe76f381b30fac569ce6f16d3b36e6f3c Mon Sep 17 00:00:00 2001 From: Nodar Ambroladze Date: Thu, 16 May 2024 20:23:00 +0200 Subject: [PATCH 070/113] Expose InsecureSkipVerify as a flag in external signer config --- arbnode/dataposter/data_poster.go | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/arbnode/dataposter/data_poster.go b/arbnode/dataposter/data_poster.go index 04789d0fda..35826620f8 100644 --- a/arbnode/dataposter/data_poster.go +++ b/arbnode/dataposter/data_poster.go @@ -220,7 +220,7 @@ func rpcClient(ctx context.Context, opts *ExternalSignerCfg) (*rpc.Client, error // Dataposter verifies that signed transaction was signed by the account // that it expects to be signed with. So signer is already authenticated // on application level and does not need to rely on TLS for authentication. - InsecureSkipVerify: true, // #nosec G402 + InsecureSkipVerify: opts.InsecureSkipVerify, // #nosec G402 } if opts.ClientCert != "" && opts.ClientPrivateKey != "" { @@ -1227,6 +1227,8 @@ type ExternalSignerCfg struct { // (Optional) Client certificate key for mtls. // This is required when client-cert is set. ClientPrivateKey string `koanf:"client-private-key"` + // TLS config option, when enabled skips certificate verification of external signer. + InsecureSkipVerify bool `koanf:"insecure-skip-verify"` } type DangerousConfig struct { @@ -1280,6 +1282,7 @@ func addExternalSignerOptions(prefix string, f *pflag.FlagSet) { f.String(prefix+".root-ca", DefaultDataPosterConfig.ExternalSigner.RootCA, "external signer root CA") f.String(prefix+".client-cert", DefaultDataPosterConfig.ExternalSigner.ClientCert, "rpc client cert") f.String(prefix+".client-private-key", DefaultDataPosterConfig.ExternalSigner.ClientPrivateKey, "rpc client private key") + f.Bool(prefix+".client-private-key", DefaultDataPosterConfig.ExternalSigner.InsecureSkipVerify, "skip TLS certificate verification") } var DefaultDataPosterConfig = DataPosterConfig{ @@ -1301,7 +1304,7 @@ var DefaultDataPosterConfig = DataPosterConfig{ UseNoOpStorage: false, LegacyStorageEncoding: false, Dangerous: DangerousConfig{ClearDBStorage: false}, - ExternalSigner: ExternalSignerCfg{Method: "eth_signTransaction"}, + ExternalSigner: ExternalSignerCfg{Method: "eth_signTransaction", InsecureSkipVerify: true}, MaxFeeCapFormula: "((BacklogOfBatches * UrgencyGWei) ** 2) + ((ElapsedTime/ElapsedTimeBase) ** 2) * ElapsedTimeImportance + TargetPriceGWei", ElapsedTimeBase: 10 * time.Minute, ElapsedTimeImportance: 10, @@ -1334,7 +1337,7 @@ var TestDataPosterConfig = DataPosterConfig{ UseDBStorage: false, UseNoOpStorage: false, LegacyStorageEncoding: false, - ExternalSigner: ExternalSignerCfg{Method: "eth_signTransaction"}, + ExternalSigner: ExternalSignerCfg{Method: "eth_signTransaction", InsecureSkipVerify: true}, MaxFeeCapFormula: "((BacklogOfBatches * UrgencyGWei) ** 2) + ((ElapsedTime/ElapsedTimeBase) ** 2) * ElapsedTimeImportance + TargetPriceGWei", ElapsedTimeBase: 10 * time.Minute, ElapsedTimeImportance: 10, From b8503d8c2581d1c065800b01bc8be7740e41a9af Mon Sep 17 00:00:00 2001 From: Nodar Ambroladze Date: Thu, 16 May 2024 20:24:10 +0200 Subject: [PATCH 071/113] Fix flag initialization --- arbnode/dataposter/data_poster.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/arbnode/dataposter/data_poster.go b/arbnode/dataposter/data_poster.go index 35826620f8..8137cbac60 100644 --- a/arbnode/dataposter/data_poster.go +++ b/arbnode/dataposter/data_poster.go @@ -1282,7 +1282,7 @@ func addExternalSignerOptions(prefix string, f *pflag.FlagSet) { f.String(prefix+".root-ca", DefaultDataPosterConfig.ExternalSigner.RootCA, "external signer root CA") f.String(prefix+".client-cert", DefaultDataPosterConfig.ExternalSigner.ClientCert, "rpc client cert") f.String(prefix+".client-private-key", DefaultDataPosterConfig.ExternalSigner.ClientPrivateKey, "rpc client private key") - f.Bool(prefix+".client-private-key", DefaultDataPosterConfig.ExternalSigner.InsecureSkipVerify, "skip TLS certificate verification") + f.Bool(prefix+".insecure-skip-verify", DefaultDataPosterConfig.ExternalSigner.InsecureSkipVerify, "skip TLS certificate verification") } var DefaultDataPosterConfig = DataPosterConfig{ From 7b7159cb7de9ee78d318a4ea1f4f74df5781fbf7 Mon Sep 17 00:00:00 2001 From: Nodar Ambroladze Date: Thu, 16 May 2024 20:25:59 +0200 Subject: [PATCH 072/113] Keep insecureSkipVerify false by default in prod config --- arbnode/dataposter/data_poster.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/arbnode/dataposter/data_poster.go b/arbnode/dataposter/data_poster.go index 8137cbac60..fb35ac3c8d 100644 --- a/arbnode/dataposter/data_poster.go +++ b/arbnode/dataposter/data_poster.go @@ -1304,7 +1304,7 @@ var DefaultDataPosterConfig = DataPosterConfig{ UseNoOpStorage: false, LegacyStorageEncoding: false, Dangerous: DangerousConfig{ClearDBStorage: false}, - ExternalSigner: ExternalSignerCfg{Method: "eth_signTransaction", InsecureSkipVerify: true}, + ExternalSigner: ExternalSignerCfg{Method: "eth_signTransaction", InsecureSkipVerify: false}, MaxFeeCapFormula: "((BacklogOfBatches * UrgencyGWei) ** 2) + ((ElapsedTime/ElapsedTimeBase) ** 2) * ElapsedTimeImportance + TargetPriceGWei", ElapsedTimeBase: 10 * time.Minute, ElapsedTimeImportance: 10, From f8dcce964279ede50b1795cbc93b6ea0fdf4cb81 Mon Sep 17 00:00:00 2001 From: Lee Bousfield Date: Thu, 16 May 2024 13:35:52 -0500 Subject: [PATCH 073/113] Fix lastUpdateTimeOffset -> ArbitrumStartTime --- arbos/programs/data_pricer.go | 8 +++++--- arbos/programs/programs.go | 4 ++-- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/arbos/programs/data_pricer.go b/arbos/programs/data_pricer.go index b0184d7dc7..ed7c98556d 100644 --- a/arbos/programs/data_pricer.go +++ b/arbos/programs/data_pricer.go @@ -27,12 +27,14 @@ const ( inertiaOffset ) +const ArbitrumStartTime = 1421388000 // the day it all began + const initialDemand = 0 // no demand const InitialHourlyBytes = 1 * (1 << 40) / (365 * 24) // 1Tb total footprint const initialBytesPerSecond = InitialHourlyBytes / (60 * 60) // refill each second -const initialLastUpdateTime = 1421388000 // the day it all began -const initialMinPrice = 82928201 // 5Mb = $1 -const initialInertia = 21360419 // expensive at 1Tb +const initialLastUpdateTime = ArbitrumStartTime +const initialMinPrice = 82928201 // 5Mb = $1 +const initialInertia = 21360419 // expensive at 1Tb func initDataPricer(sto *storage.Storage) { demand := sto.OpenStorageBackedUint32(demandOffset) diff --git a/arbos/programs/programs.go b/arbos/programs/programs.go index d3113ae98d..9d51172986 100644 --- a/arbos/programs/programs.go +++ b/arbos/programs/programs.go @@ -527,12 +527,12 @@ func (status userStatus) toResult(data []byte, debug bool) ([]byte, string, erro // Hours since Arbitrum began, rounded down. func hoursSinceArbitrum(time uint64) uint24 { - return uint24((time - lastUpdateTimeOffset) / 3600) + return uint24((time - ArbitrumStartTime) / 3600) } // Computes program age in seconds from the hours passed since Arbitrum began. func hoursToAge(time uint64, hours uint24) uint64 { seconds := am.SaturatingUMul(uint64(hours), 3600) - activatedAt := am.SaturatingUAdd(lastUpdateTimeOffset, seconds) + activatedAt := am.SaturatingUAdd(ArbitrumStartTime, seconds) return am.SaturatingUSub(time, activatedAt) } From 459106163d80806e556f2c23c896cc74acd0c1d3 Mon Sep 17 00:00:00 2001 From: Nodar Ambroladze Date: Thu, 16 May 2024 21:34:04 +0200 Subject: [PATCH 074/113] Gracefully shutdown consumer on interrupts --- pubsub/consumer.go | 48 ++++++++++++++++++++++++++++++++++++++++--- pubsub/pubsub_test.go | 8 +++++++- 2 files changed, 52 insertions(+), 4 deletions(-) diff --git a/pubsub/consumer.go b/pubsub/consumer.go index 7a5078ee00..af1345f059 100644 --- a/pubsub/consumer.go +++ b/pubsub/consumer.go @@ -5,6 +5,10 @@ import ( "encoding/json" "errors" "fmt" + "os" + "os/signal" + "sync/atomic" + "syscall" "time" "github.com/ethereum/go-ethereum/log" @@ -46,6 +50,10 @@ type Consumer[Request any, Response any] struct { redisStream string redisGroup string cfg *ConsumerConfig + // terminating indicates whether interrupt was received, in which case + // consumer should clean up for graceful shutdown. + terminating atomic.Bool + signals chan os.Signal } type Message[Request any] struct { @@ -57,29 +65,51 @@ func NewConsumer[Request any, Response any](client redis.UniversalClient, stream if streamName == "" { return nil, fmt.Errorf("redis stream name cannot be empty") } - consumer := &Consumer[Request, Response]{ + return &Consumer[Request, Response]{ id: uuid.NewString(), client: client, redisStream: streamName, redisGroup: streamName, // There is 1-1 mapping of redis stream and consumer group. cfg: cfg, - } - return consumer, nil + terminating: atomic.Bool{}, + signals: make(chan os.Signal, 1), + }, nil } // Start starts the consumer to iteratively perform heartbeat in configured intervals. func (c *Consumer[Request, Response]) Start(ctx context.Context) { c.StopWaiter.Start(ctx, c) + c.listenForInterrupt() c.StopWaiter.CallIteratively( func(ctx context.Context) time.Duration { + if !c.terminating.Load() { + log.Trace("Consumer is terminating, stopping heartbeat update") + return time.Hour + } c.heartBeat(ctx) return c.cfg.KeepAliveTimeout / 10 }, ) } +// listenForInterrupt launches a thread that notifies the channel when interrupt +// is received. +func (c *Consumer[Request, Response]) listenForInterrupt() { + signal.Notify(c.signals, syscall.SIGINT, syscall.SIGTERM) + c.StopWaiter.LaunchThread(func(ctx context.Context) { + select { + case sig := <-c.signals: + log.Info("Received interrup", "signal", sig.String()) + case <-ctx.Done(): + log.Info("Context is done", "error", ctx.Err()) + } + c.deleteHeartBeat(ctx) + }) +} + func (c *Consumer[Request, Response]) StopAndWait() { c.StopWaiter.StopAndWait() + c.deleteHeartBeat(c.GetContext()) } func heartBeatKey(id string) string { @@ -90,6 +120,18 @@ func (c *Consumer[Request, Response]) heartBeatKey() string { return heartBeatKey(c.id) } +// deleteHeartBeat deletes the heartbeat to indicate it is being shut down. +func (c *Consumer[Request, Response]) deleteHeartBeat(ctx context.Context) { + c.terminating.Store(true) + if err := c.client.Del(ctx, c.heartBeatKey()).Err(); err != nil { + l := log.Info + if ctx.Err() != nil { + l = log.Error + } + l("Deleting heardbeat", "consumer", c.id, "error", err) + } +} + // heartBeat updates the heartBeat key indicating aliveness. func (c *Consumer[Request, Response]) heartBeat(ctx context.Context) { if err := c.client.Set(ctx, c.heartBeatKey(), time.Now().UnixMilli(), 2*c.cfg.KeepAliveTimeout).Err(); err != nil { diff --git a/pubsub/pubsub_test.go b/pubsub/pubsub_test.go index 31f6d9e20a..11407e686f 100644 --- a/pubsub/pubsub_test.go +++ b/pubsub/pubsub_test.go @@ -4,6 +4,7 @@ import ( "context" "errors" "fmt" + "os" "sort" "testing" @@ -232,7 +233,12 @@ func TestRedisProduce(t *testing.T) { if _, err := consumers[i].Consume(ctx); err != nil { t.Errorf("Error consuming message: %v", err) } - consumers[i].StopAndWait() + // Terminate half of the consumers, send interrupt to others. + if i%2 == 0 { + consumers[i].StopAndWait() + } else { + consumers[i].signals <- os.Interrupt + } } } From f301094d5dd0019c1906e9f98a4cabbc720aea46 Mon Sep 17 00:00:00 2001 From: Lee Bousfield Date: Thu, 16 May 2024 15:03:25 -0500 Subject: [PATCH 075/113] Use SaturatingUSub for hoursSinceArbitrum --- arbos/programs/programs.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/arbos/programs/programs.go b/arbos/programs/programs.go index 9d51172986..6f73e16b85 100644 --- a/arbos/programs/programs.go +++ b/arbos/programs/programs.go @@ -527,7 +527,7 @@ func (status userStatus) toResult(data []byte, debug bool) ([]byte, string, erro // Hours since Arbitrum began, rounded down. func hoursSinceArbitrum(time uint64) uint24 { - return uint24((time - ArbitrumStartTime) / 3600) + return am.SaturatingUUCast[uint24]((am.SaturatingUSub(time, ArbitrumStartTime)) / 3600) } // Computes program age in seconds from the hours passed since Arbitrum began. From f18419b0fb92bf523668d1272c1bd9764c5015ed Mon Sep 17 00:00:00 2001 From: Nodar Ambroladze Date: Thu, 16 May 2024 22:48:37 +0200 Subject: [PATCH 076/113] Delete heartbeat before stopAndWait --- pubsub/consumer.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pubsub/consumer.go b/pubsub/consumer.go index af1345f059..d74d4ef1b2 100644 --- a/pubsub/consumer.go +++ b/pubsub/consumer.go @@ -108,8 +108,8 @@ func (c *Consumer[Request, Response]) listenForInterrupt() { } func (c *Consumer[Request, Response]) StopAndWait() { - c.StopWaiter.StopAndWait() c.deleteHeartBeat(c.GetContext()) + c.StopWaiter.StopAndWait() } func heartBeatKey(id string) string { From 3373c4a5b84010a7cd5f3b47a7f43de5cf46f476 Mon Sep 17 00:00:00 2001 From: Nodar Ambroladze Date: Thu, 16 May 2024 23:18:47 +0200 Subject: [PATCH 077/113] Fix test --- pubsub/consumer.go | 1 - pubsub/pubsub_test.go | 7 +++++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/pubsub/consumer.go b/pubsub/consumer.go index d74d4ef1b2..97ab004764 100644 --- a/pubsub/consumer.go +++ b/pubsub/consumer.go @@ -108,7 +108,6 @@ func (c *Consumer[Request, Response]) listenForInterrupt() { } func (c *Consumer[Request, Response]) StopAndWait() { - c.deleteHeartBeat(c.GetContext()) c.StopWaiter.StopAndWait() } diff --git a/pubsub/pubsub_test.go b/pubsub/pubsub_test.go index 11407e686f..9111c5cf66 100644 --- a/pubsub/pubsub_test.go +++ b/pubsub/pubsub_test.go @@ -7,6 +7,7 @@ import ( "os" "sort" "testing" + "time" "github.com/ethereum/go-ethereum/log" "github.com/go-redis/redis/v8" @@ -202,6 +203,7 @@ func consume(ctx context.Context, t *testing.T, consumers []*Consumer[testReques } func TestRedisProduce(t *testing.T) { + log.SetDefault(log.NewLogger(log.NewTerminalHandlerWithLevel(os.Stderr, log.LevelTrace, true))) t.Parallel() for _, tc := range []struct { name string @@ -213,7 +215,7 @@ func TestRedisProduce(t *testing.T) { }, { name: "some consumers killed, others should take over their work", - killConsumers: false, + killConsumers: true, }, } { t.Run(tc.name, func(t *testing.T) { @@ -233,7 +235,7 @@ func TestRedisProduce(t *testing.T) { if _, err := consumers[i].Consume(ctx); err != nil { t.Errorf("Error consuming message: %v", err) } - // Terminate half of the consumers, send interrupt to others. + //Terminate half of the consumers, send interrupt to others. if i%2 == 0 { consumers[i].StopAndWait() } else { @@ -242,6 +244,7 @@ func TestRedisProduce(t *testing.T) { } } + time.Sleep(time.Second) gotMessages, wantResponses := consume(ctx, t, consumers) gotResponses, err := awaitResponses(ctx, promises) if err != nil { From 3fd412e78e725b0bc850e800741b16effab491b3 Mon Sep 17 00:00:00 2001 From: Nodar Ambroladze Date: Thu, 16 May 2024 23:46:56 +0200 Subject: [PATCH 078/113] Fix lint --- pubsub/pubsub_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pubsub/pubsub_test.go b/pubsub/pubsub_test.go index 9111c5cf66..85314dc29a 100644 --- a/pubsub/pubsub_test.go +++ b/pubsub/pubsub_test.go @@ -235,7 +235,7 @@ func TestRedisProduce(t *testing.T) { if _, err := consumers[i].Consume(ctx); err != nil { t.Errorf("Error consuming message: %v", err) } - //Terminate half of the consumers, send interrupt to others. + // Terminate half of the consumers, send interrupt to others. if i%2 == 0 { consumers[i].StopAndWait() } else { From 804e4fa75860e4dd482f0b1e59844e260d277ba4 Mon Sep 17 00:00:00 2001 From: Diego Ximenes Date: Fri, 17 May 2024 11:25:33 -0300 Subject: [PATCH 079/113] fix: CleanCacheSize from hashdb.Config expects a value defined in bytes, and not as in MB as TrieCleanLimit is defined --- cmd/staterecovery/staterecovery.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmd/staterecovery/staterecovery.go b/cmd/staterecovery/staterecovery.go index 6390826a91..58ad06ad14 100644 --- a/cmd/staterecovery/staterecovery.go +++ b/cmd/staterecovery/staterecovery.go @@ -31,7 +31,7 @@ func RecreateMissingStates(chainDb ethdb.Database, bc *core.BlockChain, cacheCon return fmt.Errorf("start block parent is missing, parent block number: %d", current-1) } hashConfig := *hashdb.Defaults - hashConfig.CleanCacheSize = cacheConfig.TrieCleanLimit + hashConfig.CleanCacheSize = cacheConfig.TrieCleanLimit * 1024 * 1024 trieConfig := &trie.Config{ Preimages: false, HashDB: &hashConfig, From 6884188d20b089f9320b6fc26bad6d049583364f Mon Sep 17 00:00:00 2001 From: Ganesh Vanahalli Date: Fri, 17 May 2024 13:21:12 -0500 Subject: [PATCH 080/113] address PR comments --- blocks_reexecutor/blocks_reexecutor.go | 46 ++++++++++++-------------- 1 file changed, 22 insertions(+), 24 deletions(-) diff --git a/blocks_reexecutor/blocks_reexecutor.go b/blocks_reexecutor/blocks_reexecutor.go index a03b29fefd..f58e0ce00f 100644 --- a/blocks_reexecutor/blocks_reexecutor.go +++ b/blocks_reexecutor/blocks_reexecutor.go @@ -25,8 +25,6 @@ type Config struct { EndBlock uint64 `koanf:"end-block"` Room int `koanf:"room"` BlocksPerThread uint64 `koanf:"blocks-per-thread"` - - blocksPerThread uint64 } func (c *Config) Validate() error { @@ -40,11 +38,6 @@ func (c *Config) Validate() error { if c.Room < 0 { return errors.New("room for blocks re-execution should be greater than 0") } - if c.BlocksPerThread != 0 { - c.blocksPerThread = c.BlocksPerThread - } else { - c.blocksPerThread = 10000 - } return nil } @@ -59,7 +52,6 @@ var TestConfig = Config{ Mode: "full", Room: runtime.NumCPU(), BlocksPerThread: 10, - blocksPerThread: 10, } func ConfigAddOptions(prefix string, f *flag.FlagSet) { @@ -73,13 +65,14 @@ func ConfigAddOptions(prefix string, f *flag.FlagSet) { type BlocksReExecutor struct { stopwaiter.StopWaiter - config *Config - blockchain *core.BlockChain - stateFor arbitrum.StateForHeaderFunction - done chan struct{} - fatalErrChan chan error - startBlock uint64 - currentBlock uint64 + config *Config + blockchain *core.BlockChain + stateFor arbitrum.StateForHeaderFunction + done chan struct{} + fatalErrChan chan error + startBlock uint64 + currentBlock uint64 + blocksPerThread uint64 } func New(c *Config, blockchain *core.BlockChain, fatalErrChan chan error) *BlocksReExecutor { @@ -99,9 +92,13 @@ func New(c *Config, blockchain *core.BlockChain, fatalErrChan chan error) *Block log.Warn("invalid state reexecutor's end block number, resetting to latest", "end", end, "latest", chainEnd) end = chainEnd } + blocksPerThread := uint64(10000) + if c.BlocksPerThread != 0 { + blocksPerThread = c.BlocksPerThread + } if c.Mode == "random" && end != start { // Reexecute a range of 10000 or (non-zero) c.BlocksPerThread number of blocks between start to end picked randomly - rng := c.blocksPerThread + rng := blocksPerThread if rng > end-start { rng = end - start } @@ -117,16 +114,17 @@ func New(c *Config, blockchain *core.BlockChain, fatalErrChan chan error) *Block if c.BlocksPerThread == 0 { work := (end - start) / uint64(c.Room) if work > 0 { - c.blocksPerThread = work + blocksPerThread = work } } return &BlocksReExecutor{ - config: c, - blockchain: blockchain, - currentBlock: end, - startBlock: start, - done: make(chan struct{}, c.Room), - fatalErrChan: fatalErrChan, + config: c, + blockchain: blockchain, + currentBlock: end, + startBlock: start, + blocksPerThread: blocksPerThread, + done: make(chan struct{}, c.Room), + fatalErrChan: fatalErrChan, stateFor: func(header *types.Header) (*state.StateDB, arbitrum.StateReleaseFunc, error) { state, err := blockchain.StateAt(header.Root) return state, arbitrum.NoopStateRelease, err @@ -136,7 +134,7 @@ func New(c *Config, blockchain *core.BlockChain, fatalErrChan chan error) *Block // LaunchBlocksReExecution launches the thread to apply blocks of range [currentBlock-s.config.BlocksPerThread, currentBlock] to the last available valid state func (s *BlocksReExecutor) LaunchBlocksReExecution(ctx context.Context, currentBlock uint64) uint64 { - start := arbmath.SaturatingUSub(currentBlock, s.config.blocksPerThread) + start := arbmath.SaturatingUSub(currentBlock, s.blocksPerThread) if start < s.startBlock { start = s.startBlock } From b5b12e89049de4c334035d979f7734d67e3d36d3 Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Fri, 17 May 2024 15:11:19 -0600 Subject: [PATCH 081/113] Add support for configurable lru cache resize on nitro init --- arbitrator/stylus/src/cache.rs | 4 ++++ arbitrator/stylus/src/lib.rs | 6 ++++++ arbnode/inbox_test.go | 1 + arbos/programs/native.go | 4 ++++ execution/gethexec/blockchain.go | 18 ++++++++++++++++++ execution/gethexec/executionengine.go | 10 ++++++++++ execution/gethexec/node.go | 3 +++ system_tests/recreatestate_rpc_test.go | 2 +- system_tests/staterecovery_test.go | 2 +- 9 files changed, 48 insertions(+), 2 deletions(-) diff --git a/arbitrator/stylus/src/cache.rs b/arbitrator/stylus/src/cache.rs index 2b83c6152f..6a9e677be5 100644 --- a/arbitrator/stylus/src/cache.rs +++ b/arbitrator/stylus/src/cache.rs @@ -66,6 +66,10 @@ impl InitCache { } } + pub fn set_lru_size(size: u32) { + cache!().lru.resize(NonZeroUsize::new(size.try_into().unwrap()).unwrap()) + } + /// Retrieves a cached value, updating items as necessary. pub fn get(module_hash: Bytes32, version: u16, debug: bool) -> Option<(Module, Store)> { let mut cache = cache!(); diff --git a/arbitrator/stylus/src/lib.rs b/arbitrator/stylus/src/lib.rs index 7abfb98bf5..9ccc9829ca 100644 --- a/arbitrator/stylus/src/lib.rs +++ b/arbitrator/stylus/src/lib.rs @@ -212,6 +212,12 @@ pub unsafe extern "C" fn stylus_call( status } +/// resize lru +#[no_mangle] +pub extern "C" fn stylus_cache_lru_resize(size: u32) { + InitCache::set_lru_size(size); +} + /// Caches an activated user program. /// /// # Safety diff --git a/arbnode/inbox_test.go b/arbnode/inbox_test.go index 5c879743a4..594e0cedb5 100644 --- a/arbnode/inbox_test.go +++ b/arbnode/inbox_test.go @@ -65,6 +65,7 @@ func NewTransactionStreamerForTest(t *testing.T, ownerAddress common.Address) (* if err != nil { Fail(t, err) } + execEngine.Initialize(gethexec.DefaultCachingConfig.StylusLRUCache) execSeq := &execClientWrapper{execEngine, t} inbox, err := NewTransactionStreamer(arbDb, bc.Config(), execSeq, nil, make(chan error, 1), transactionStreamerConfigFetcher) if err != nil { diff --git a/arbos/programs/native.go b/arbos/programs/native.go index 7a6c16d866..17068371b1 100644 --- a/arbos/programs/native.go +++ b/arbos/programs/native.go @@ -253,6 +253,10 @@ func init() { } } +func ResizeWasmLruCache(size uint32) { + C.stylus_cache_lru_resize(u32(size)) +} + func (value bytes32) toHash() common.Hash { hash := common.Hash{} for index, b := range value.bytes { diff --git a/execution/gethexec/blockchain.go b/execution/gethexec/blockchain.go index 2a20c3da26..1d5060ca8a 100644 --- a/execution/gethexec/blockchain.go +++ b/execution/gethexec/blockchain.go @@ -37,6 +37,7 @@ type CachingConfig struct { SnapshotRestoreGasLimit uint64 `koanf:"snapshot-restore-gas-limit"` MaxNumberOfBlocksToSkipStateSaving uint32 `koanf:"max-number-of-blocks-to-skip-state-saving"` MaxAmountOfGasToSkipStateSaving uint64 `koanf:"max-amount-of-gas-to-skip-state-saving"` + StylusLRUCache uint32 `koanf:"stylus-lru-cache"` } func CachingConfigAddOptions(prefix string, f *flag.FlagSet) { @@ -51,6 +52,7 @@ func CachingConfigAddOptions(prefix string, f *flag.FlagSet) { f.Uint64(prefix+".snapshot-restore-gas-limit", DefaultCachingConfig.SnapshotRestoreGasLimit, "maximum gas rolled back to recover snapshot") f.Uint32(prefix+".max-number-of-blocks-to-skip-state-saving", DefaultCachingConfig.MaxNumberOfBlocksToSkipStateSaving, "maximum number of blocks to skip state saving to persistent storage (archive node only) -- warning: this option seems to cause issues") f.Uint64(prefix+".max-amount-of-gas-to-skip-state-saving", DefaultCachingConfig.MaxAmountOfGasToSkipStateSaving, "maximum amount of gas in blocks to skip saving state to Persistent storage (archive node only) -- warning: this option seems to cause issues") + f.Uint32(prefix+".stylus-lru-cache", DefaultCachingConfig.StylusLRUCache, "initialized stylus programs to keep in LRU cache") } var DefaultCachingConfig = CachingConfig{ @@ -65,6 +67,22 @@ var DefaultCachingConfig = CachingConfig{ SnapshotRestoreGasLimit: 300_000_000_000, MaxNumberOfBlocksToSkipStateSaving: 0, MaxAmountOfGasToSkipStateSaving: 0, + StylusLRUCache: 256, +} + +var TestCachingConfig = CachingConfig{ + Archive: false, + BlockCount: 128, + BlockAge: 30 * time.Minute, + TrieTimeLimit: time.Hour, + TrieDirtyCache: 1024, + TrieCleanCache: 600, + SnapshotCache: 400, + DatabaseCache: 2048, + SnapshotRestoreGasLimit: 300_000_000_000, + MaxNumberOfBlocksToSkipStateSaving: 0, + MaxAmountOfGasToSkipStateSaving: 0, + StylusLRUCache: 0, } // TODO remove stack from parameters as it is no longer needed here diff --git a/execution/gethexec/executionengine.go b/execution/gethexec/executionengine.go index 38569f44ab..b3ebe80f37 100644 --- a/execution/gethexec/executionengine.go +++ b/execution/gethexec/executionengine.go @@ -1,6 +1,9 @@ // Copyright 2022-2024, Offchain Labs, Inc. // For license information, see https://github.com/OffchainLabs/nitro/blob/master/LICENSE +//go:build !wasm +// +build !wasm + package gethexec /* @@ -28,6 +31,7 @@ import ( "github.com/offchainlabs/nitro/arbos/arbosState" "github.com/offchainlabs/nitro/arbos/arbostypes" "github.com/offchainlabs/nitro/arbos/l1pricing" + "github.com/offchainlabs/nitro/arbos/programs" "github.com/offchainlabs/nitro/arbutil" "github.com/offchainlabs/nitro/execution" "github.com/offchainlabs/nitro/util/arbmath" @@ -72,6 +76,12 @@ func NewExecutionEngine(bc *core.BlockChain) (*ExecutionEngine, error) { }, nil } +func (n *ExecutionEngine) Initialize(rustCacheSize uint32) { + if rustCacheSize != 0 { + programs.ResizeWasmLruCache(rustCacheSize) + } +} + func (s *ExecutionEngine) SetRecorder(recorder *BlockRecorder) { if s.Started() { panic("trying to set recorder after start") diff --git a/execution/gethexec/node.go b/execution/gethexec/node.go index ae76b88530..b7fe1c6e14 100644 --- a/execution/gethexec/node.go +++ b/execution/gethexec/node.go @@ -107,6 +107,7 @@ var ConfigDefault = Config{ func ConfigDefaultNonSequencerTest() *Config { config := ConfigDefault + config.Caching = TestCachingConfig config.ParentChainReader = headerreader.TestConfig config.Sequencer.Enable = false config.Forwarder = DefaultTestForwarderConfig @@ -119,6 +120,7 @@ func ConfigDefaultNonSequencerTest() *Config { func ConfigDefaultTest() *Config { config := ConfigDefault + config.Caching = TestCachingConfig config.Sequencer = TestSequencerConfig config.ParentChainReader = headerreader.TestConfig config.ForwardingTarget = "null" @@ -280,6 +282,7 @@ func (n *ExecutionNode) GetL1GasPriceEstimate() (uint64, error) { } func (n *ExecutionNode) Initialize(ctx context.Context) error { + n.ExecEngine.Initialize(n.ConfigFetcher().Caching.StylusLRUCache) n.ArbInterface.Initialize(n) err := n.Backend.Start() if err != nil { diff --git a/system_tests/recreatestate_rpc_test.go b/system_tests/recreatestate_rpc_test.go index 777ed17961..bf321808de 100644 --- a/system_tests/recreatestate_rpc_test.go +++ b/system_tests/recreatestate_rpc_test.go @@ -449,7 +449,7 @@ func testSkippingSavingStateAndRecreatingAfterRestart(t *testing.T, cacheConfig } func TestSkippingSavingStateAndRecreatingAfterRestart(t *testing.T) { - cacheConfig := gethexec.DefaultCachingConfig + cacheConfig := gethexec.TestCachingConfig cacheConfig.Archive = true cacheConfig.SnapshotCache = 0 // disable snapshots cacheConfig.BlockAge = 0 // use only Caching.BlockCount to keep only last N blocks in dirties cache, no matter how new they are diff --git a/system_tests/staterecovery_test.go b/system_tests/staterecovery_test.go index 632e748da8..02c2623cfa 100644 --- a/system_tests/staterecovery_test.go +++ b/system_tests/staterecovery_test.go @@ -52,7 +52,7 @@ func TestRectreateMissingStates(t *testing.T) { chainDb, err := stack.OpenDatabase("l2chaindata", 0, 0, "l2chaindata/", false) Require(t, err) defer chainDb.Close() - cacheConfig := gethexec.DefaultCacheConfigFor(stack, &gethexec.DefaultCachingConfig) + cacheConfig := gethexec.DefaultCacheConfigFor(stack, &gethexec.TestCachingConfig) bc, err := gethexec.GetBlockChain(chainDb, cacheConfig, builder.chainConfig, builder.execConfig.TxLookupLimit) Require(t, err) err = staterecovery.RecreateMissingStates(chainDb, bc, cacheConfig, 1) From ced4a07273a2de581bc57580468b2fc58e8922b5 Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Fri, 17 May 2024 16:04:11 -0600 Subject: [PATCH 082/113] add tags when creating wasm-wrapped database --- cmd/nitro/init.go | 4 ++-- system_tests/common_test.go | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/cmd/nitro/init.go b/cmd/nitro/init.go index c52c87732c..0b36fcfdaf 100644 --- a/cmd/nitro/init.go +++ b/cmd/nitro/init.go @@ -186,7 +186,7 @@ func openInitializeChainDb(ctx context.Context, stack *node.Node, config *NodeCo if err != nil { return nil, nil, err } - chainDb := rawdb.WrapDatabaseWithWasm(chainData, wasmDb) + chainDb := rawdb.WrapDatabaseWithWasm(chainData, wasmDb, 1) err = pruning.PruneChainDb(ctx, chainDb, stack, &config.Init, cacheConfig, l1Client, rollupAddrs, config.Node.ValidatorRequired()) if err != nil { return chainDb, nil, fmt.Errorf("error pruning: %w", err) @@ -243,7 +243,7 @@ func openInitializeChainDb(ctx context.Context, stack *node.Node, config *NodeCo if err != nil { return nil, nil, err } - chainDb := rawdb.WrapDatabaseWithWasm(chainData, wasmDb) + chainDb := rawdb.WrapDatabaseWithWasm(chainData, wasmDb, 1) if config.Init.ImportFile != "" { initDataReader, err = statetransfer.NewJsonInitDataReader(config.Init.ImportFile) diff --git a/system_tests/common_test.go b/system_tests/common_test.go index f6bfde2108..edc16ffec4 100644 --- a/system_tests/common_test.go +++ b/system_tests/common_test.go @@ -777,7 +777,7 @@ func createL2BlockChainWithStackConfig( Require(t, err) wasmData, err := stack.OpenDatabase("wasm", 0, 0, "wasm/", false) Require(t, err) - chainDb := rawdb.WrapDatabaseWithWasm(chainData, wasmData) + chainDb := rawdb.WrapDatabaseWithWasm(chainData, wasmData, 0) arbDb, err := stack.OpenDatabase("arbitrumdata", 0, 0, "arbitrumdata/", false) Require(t, err) @@ -984,7 +984,7 @@ func Create2ndNodeWithConfig( Require(t, err) wasmData, err := l2stack.OpenDatabase("wasm", 0, 0, "wasm/", false) Require(t, err) - l2chainDb := rawdb.WrapDatabaseWithWasm(l2chainData, wasmData) + l2chainDb := rawdb.WrapDatabaseWithWasm(l2chainData, wasmData, 0) l2arbDb, err := l2stack.OpenDatabase("arbitrumdata", 0, 0, "arbitrumdata/", false) Require(t, err) From 0f30f9f4e6cbd05cc76c6710cefbb24929b75eb9 Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Fri, 17 May 2024 16:05:41 -0600 Subject: [PATCH 083/113] arbitrator: add and use long_term tag --- arbitrator/stylus/src/cache.rs | 63 +++++++++++++++++++-------------- arbitrator/stylus/src/lib.rs | 16 +++++---- arbitrator/stylus/src/native.rs | 9 ++--- 3 files changed, 52 insertions(+), 36 deletions(-) diff --git a/arbitrator/stylus/src/cache.rs b/arbitrator/stylus/src/cache.rs index 6a9e677be5..3a15bc5d6a 100644 --- a/arbitrator/stylus/src/cache.rs +++ b/arbitrator/stylus/src/cache.rs @@ -21,7 +21,7 @@ macro_rules! cache { } pub struct InitCache { - arbos: HashMap, + long_term: HashMap, lru: LruCache, } @@ -59,9 +59,14 @@ impl CacheItem { } impl InitCache { + // current implementation only has one tag that stores to the long_term + // future implementations might have more, but 0 is a reserved tag + // that will never modify long_term state + const ARBOS_TAG: u32 = 1; + fn new(size: usize) -> Self { Self { - arbos: HashMap::new(), + long_term: HashMap::new(), lru: LruCache::new(NonZeroUsize::new(size).unwrap()), } } @@ -76,7 +81,7 @@ impl InitCache { let key = CacheKey::new(module_hash, version, debug); // See if the item is in the long term cache - if let Some(item) = cache.arbos.get(&key) { + if let Some(item) = cache.long_term.get(&key) { return Some(item.data()); } @@ -88,18 +93,27 @@ impl InitCache { } /// Inserts an item into the long term cache, cloning from the LRU cache if able. + /// If long_term_tag is 0 will only insert to LRU pub fn insert( module_hash: Bytes32, module: &[u8], version: u16, + long_term_tag: u32, debug: bool, ) -> Result<(Module, Store)> { let key = CacheKey::new(module_hash, version, debug); // if in LRU, add to ArbOS let mut cache = cache!(); + if let Some(item) = cache.long_term.get(&key) { + return Ok(item.data()) + } if let Some(item) = cache.lru.peek(&key).cloned() { - cache.arbos.insert(key, item.clone()); + if long_term_tag == Self::ARBOS_TAG { + cache.long_term.insert(key, item.clone()); + } else { + cache.lru.promote(&key) + } return Ok(item.data()); } drop(cache); @@ -109,37 +123,34 @@ impl InitCache { let item = CacheItem::new(module, engine); let data = item.data(); - cache!().arbos.insert(key, item); + let mut cache = cache!(); + if long_term_tag != Self::ARBOS_TAG { + cache.lru.put(key, item); + } else { + cache.long_term.insert(key, item); + } Ok(data) } - /// Inserts an item into the short-lived LRU cache. - pub fn insert_lru( - module_hash: Bytes32, - module: &[u8], - version: u16, - debug: bool, - ) -> Result<(Module, Store)> { - let engine = CompileConfig::version(version, debug).engine(); - let module = unsafe { Module::deserialize_unchecked(&engine, module)? }; - - let key = CacheKey::new(module_hash, version, debug); - let item = CacheItem::new(module, engine); - cache!().lru.put(key, item.clone()); - Ok(item.data()) - } - /// Evicts an item in the long-term cache. - pub fn evict(module_hash: Bytes32, version: u16, debug: bool) { + pub fn evict(module_hash: Bytes32, version: u16, long_term_tag: u32, debug: bool) { + if long_term_tag != Self::ARBOS_TAG { + return + } let key = CacheKey::new(module_hash, version, debug); - cache!().arbos.remove(&key); + let mut cache = cache!(); + if let Some(item) = cache.long_term.remove(&key) { + cache.lru.put(key, item); + } } - /// Modifies the cache for reorg, dropping the long-term cache. - pub fn reorg(_block: u64) { + pub fn clear_long_term(long_term_tag: u32) { + if long_term_tag != Self::ARBOS_TAG { + return + } let mut cache = cache!(); let cache = &mut *cache; - for (key, item) in cache.arbos.drain() { + for (key, item) in cache.long_term.drain() { cache.lru.put(key, item); // not all will fit, just a heuristic } } diff --git a/arbitrator/stylus/src/lib.rs b/arbitrator/stylus/src/lib.rs index 9ccc9829ca..6133b6ac34 100644 --- a/arbitrator/stylus/src/lib.rs +++ b/arbitrator/stylus/src/lib.rs @@ -183,6 +183,7 @@ pub unsafe extern "C" fn stylus_call( debug_chain: bool, output: *mut RustBytes, gas: *mut u64, + long_term_tag: u32, ) -> UserOutcomeKind { let module = module.slice(); let calldata = calldata.slice().to_vec(); @@ -193,7 +194,7 @@ pub unsafe extern "C" fn stylus_call( // Safety: module came from compile_user_wasm and we've paid for memory expansion let instance = unsafe { - NativeInstance::deserialize_cached(module, config.version, evm_api, evm_data, debug_chain) + NativeInstance::deserialize_cached(module, config.version, evm_api, evm_data, long_term_tag, debug_chain) }; let mut instance = match instance { Ok(instance) => instance, @@ -223,28 +224,31 @@ pub extern "C" fn stylus_cache_lru_resize(size: u32) { /// # Safety /// /// `module` must represent a valid module produced from `stylus_activate`. +/// arbos_tag: a tag for arbos cache. 0 won't affect real caching +/// currently only if tag==1 caching will be affected #[no_mangle] pub unsafe extern "C" fn stylus_cache_module( module: GoSliceData, module_hash: Bytes32, version: u16, + arbos_tag: u32, debug: bool, ) { - if let Err(error) = InitCache::insert(module_hash, module.slice(), version, debug) { + if let Err(error) = InitCache::insert(module_hash, module.slice(), version, arbos_tag, debug) { panic!("tried to cache invalid asm!: {error}"); } } /// Evicts an activated user program from the init cache. #[no_mangle] -pub extern "C" fn stylus_evict_module(module_hash: Bytes32, version: u16, debug: bool) { - InitCache::evict(module_hash, version, debug); +pub extern "C" fn stylus_evict_module(module_hash: Bytes32, version: u16, arbos_tag: u32, debug: bool) { + InitCache::evict(module_hash, version, arbos_tag, debug); } /// Reorgs the init cache. This will likely never happen. #[no_mangle] -pub extern "C" fn stylus_reorg_vm(block: u64) { - InitCache::reorg(block); +pub extern "C" fn stylus_reorg_vm(_block: u64, arbos_tag: u32) { + InitCache::clear_long_term(arbos_tag); } /// Frees the vector. Does nothing when the vector is null. diff --git a/arbitrator/stylus/src/native.rs b/arbitrator/stylus/src/native.rs index 6d5e4cd2e9..38155818c0 100644 --- a/arbitrator/stylus/src/native.rs +++ b/arbitrator/stylus/src/native.rs @@ -113,6 +113,7 @@ impl> NativeInstance { version: u16, evm: E, evm_data: EvmData, + mut long_term_tag: u32, debug: bool, ) -> Result { let compile = CompileConfig::version(version, debug); @@ -122,10 +123,10 @@ impl> NativeInstance { if let Some((module, store)) = InitCache::get(module_hash, version, debug) { return Self::from_module(module, store, env); } - let (module, store) = match env.evm_data.cached { - true => InitCache::insert(module_hash, module, version, debug)?, - false => InitCache::insert_lru(module_hash, module, version, debug)?, - }; + if !env.evm_data.cached { + long_term_tag = 0; + } + let (module, store) = InitCache::insert(module_hash, module, version, long_term_tag, debug)?; Self::from_module(module, store, env) } From 72f8b9da72a8fb6821e06b970ff85573d565f3f2 Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Fri, 17 May 2024 16:08:59 -0600 Subject: [PATCH 084/113] nitro: use tag for stylus calls --- arbos/programs/native.go | 20 ++++++++++++-------- arbos/programs/programs.go | 7 ++++++- arbos/programs/wasm.go | 1 + arbos/tx_processor.go | 1 + execution/gethexec/executionengine.go | 3 ++- 5 files changed, 22 insertions(+), 10 deletions(-) diff --git a/arbos/programs/native.go b/arbos/programs/native.go index 17068371b1..f24dcac64d 100644 --- a/arbos/programs/native.go +++ b/arbos/programs/native.go @@ -172,6 +172,7 @@ func callProgram( evmData *EvmData, stylusParams *ProgParams, memoryModel *MemoryModel, + arbos_tag uint32, ) ([]byte, error) { db := interpreter.Evm().StateDB debug := stylusParams.DebugMode @@ -198,6 +199,7 @@ func callProgram( cbool(debug), output, (*u64)(&scope.Contract.Gas), + u32(arbos_tag), )) depth := interpreter.Depth() @@ -228,8 +230,9 @@ func cacheProgram(db vm.StateDB, module common.Hash, program Program, params *St if err != nil { panic("unable to recreate wasm") } - state.CacheWasmRust(asm, module, program.version, debug) - db.RecordCacheWasm(state.CacheWasm{ModuleHash: module, Version: program.version, Debug: debug}) + tag := db.Database().WasmCacheTag() + state.CacheWasmRust(asm, module, program.version, tag, debug) + db.RecordCacheWasm(state.CacheWasm{ModuleHash: module, Version: program.version, Tag: tag, Debug: debug}) } } @@ -237,19 +240,20 @@ func cacheProgram(db vm.StateDB, module common.Hash, program Program, params *St // For gas estimation and eth_call, we ignore permanent updates and rely on Rust's LRU. func evictProgram(db vm.StateDB, module common.Hash, version uint16, debug bool, runMode core.MessageRunMode, forever bool) { if runMode == core.MessageCommitMode { - state.EvictWasmRust(module, version, debug) + tag := db.Database().WasmCacheTag() + state.EvictWasmRust(module, version, tag, debug) if !forever { - db.RecordEvictWasm(state.EvictWasm{ModuleHash: module, Version: version, Debug: debug}) + db.RecordEvictWasm(state.EvictWasm{ModuleHash: module, Version: version, Tag: tag, Debug: debug}) } } } func init() { - state.CacheWasmRust = func(asm []byte, moduleHash common.Hash, version uint16, debug bool) { - C.stylus_cache_module(goSlice(asm), hashToBytes32(moduleHash), u16(version), cbool(debug)) + state.CacheWasmRust = func(asm []byte, moduleHash common.Hash, version uint16, tag uint32, debug bool) { + C.stylus_cache_module(goSlice(asm), hashToBytes32(moduleHash), u16(version), u32(tag), cbool(debug)) } - state.EvictWasmRust = func(moduleHash common.Hash, version uint16, debug bool) { - C.stylus_evict_module(hashToBytes32(moduleHash), u16(version), cbool(debug)) + state.EvictWasmRust = func(moduleHash common.Hash, version uint16, tag uint32, debug bool) { + C.stylus_evict_module(hashToBytes32(moduleHash), u16(version), u32(tag), cbool(debug)) } } diff --git a/arbos/programs/programs.go b/arbos/programs/programs.go index 9d51172986..f27d5834bf 100644 --- a/arbos/programs/programs.go +++ b/arbos/programs/programs.go @@ -166,6 +166,7 @@ func (p Programs) CallProgram( tracingInfo *util.TracingInfo, calldata []byte, reentrant bool, + runmode core.MessageRunMode, ) ([]byte, error) { evm := interpreter.Evm() contract := scope.Contract @@ -237,7 +238,11 @@ func (p Programs) CallProgram( if contract.CodeAddr != nil { address = *contract.CodeAddr } - return callProgram(address, moduleHash, localAsm, scope, interpreter, tracingInfo, calldata, evmData, goParams, model) + var arbos_tag uint32 + if runmode == core.MessageCommitMode { + arbos_tag = statedb.Database().WasmCacheTag() + } + return callProgram(address, moduleHash, localAsm, scope, interpreter, tracingInfo, calldata, evmData, goParams, model, arbos_tag) } func getWasm(statedb vm.StateDB, program common.Address) ([]byte, error) { diff --git a/arbos/programs/wasm.go b/arbos/programs/wasm.go index 95f30e83b6..4bc978a2b6 100644 --- a/arbos/programs/wasm.go +++ b/arbos/programs/wasm.go @@ -143,6 +143,7 @@ func callProgram( evmData *EvmData, params *ProgParams, memoryModel *MemoryModel, + _arbos_tag uint32, ) ([]byte, error) { reqHandler := newApiClosures(interpreter, tracingInfo, scope, memoryModel) gasLeft, retData, err := CallProgramLoop(moduleHash, calldata, scope.Contract.Gas, evmData, params, reqHandler) diff --git a/arbos/tx_processor.go b/arbos/tx_processor.go index b5fb64f695..65762fd2d1 100644 --- a/arbos/tx_processor.go +++ b/arbos/tx_processor.go @@ -127,6 +127,7 @@ func (p *TxProcessor) ExecuteWASM(scope *vm.ScopeContext, input []byte, interpre tracingInfo, input, reentrant, + p.RunMode(), ) } diff --git a/execution/gethexec/executionengine.go b/execution/gethexec/executionengine.go index b3ebe80f37..00218c9291 100644 --- a/execution/gethexec/executionengine.go +++ b/execution/gethexec/executionengine.go @@ -147,8 +147,9 @@ func (s *ExecutionEngine) Reorg(count arbutil.MessageIndex, newMessages []arbost return nil, nil } + tag := s.bc.StateCache().WasmCacheTag() // reorg Rust-side VM state - C.stylus_reorg_vm(C.uint64_t(blockNum)) + C.stylus_reorg_vm(C.uint64_t(blockNum), C.uint32_t(tag)) err := s.bc.ReorgToOldBlock(targetBlock) if err != nil { From cd03bf07ed3d7065d8b5a243ac4562f62370774f Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Fri, 17 May 2024 16:09:20 -0600 Subject: [PATCH 085/113] geth: udate pin to support arbos tags --- go-ethereum | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/go-ethereum b/go-ethereum index 8048ac4bed..940fbe020e 160000 --- a/go-ethereum +++ b/go-ethereum @@ -1 +1 @@ -Subproject commit 8048ac4bed2eda18284e3c022ea5ee4cce771134 +Subproject commit 940fbe020e03707365da09de939058944d9e1f5d From 1ed090dcda3ac03c0c46321cb4a309b59dcb87c8 Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Fri, 17 May 2024 17:23:44 -0600 Subject: [PATCH 086/113] cargo fmt --- arbitrator/stylus/src/cache.rs | 12 +++++++----- arbitrator/stylus/src/lib.rs | 16 ++++++++++++++-- arbitrator/stylus/src/native.rs | 3 ++- 3 files changed, 23 insertions(+), 8 deletions(-) diff --git a/arbitrator/stylus/src/cache.rs b/arbitrator/stylus/src/cache.rs index 3a15bc5d6a..06739f2219 100644 --- a/arbitrator/stylus/src/cache.rs +++ b/arbitrator/stylus/src/cache.rs @@ -72,7 +72,9 @@ impl InitCache { } pub fn set_lru_size(size: u32) { - cache!().lru.resize(NonZeroUsize::new(size.try_into().unwrap()).unwrap()) + cache!() + .lru + .resize(NonZeroUsize::new(size.try_into().unwrap()).unwrap()) } /// Retrieves a cached value, updating items as necessary. @@ -106,7 +108,7 @@ impl InitCache { // if in LRU, add to ArbOS let mut cache = cache!(); if let Some(item) = cache.long_term.get(&key) { - return Ok(item.data()) + return Ok(item.data()); } if let Some(item) = cache.lru.peek(&key).cloned() { if long_term_tag == Self::ARBOS_TAG { @@ -135,7 +137,7 @@ impl InitCache { /// Evicts an item in the long-term cache. pub fn evict(module_hash: Bytes32, version: u16, long_term_tag: u32, debug: bool) { if long_term_tag != Self::ARBOS_TAG { - return + return; } let key = CacheKey::new(module_hash, version, debug); let mut cache = cache!(); @@ -146,8 +148,8 @@ impl InitCache { pub fn clear_long_term(long_term_tag: u32) { if long_term_tag != Self::ARBOS_TAG { - return - } + return; + } let mut cache = cache!(); let cache = &mut *cache; for (key, item) in cache.long_term.drain() { diff --git a/arbitrator/stylus/src/lib.rs b/arbitrator/stylus/src/lib.rs index 6133b6ac34..3c53359f8b 100644 --- a/arbitrator/stylus/src/lib.rs +++ b/arbitrator/stylus/src/lib.rs @@ -194,7 +194,14 @@ pub unsafe extern "C" fn stylus_call( // Safety: module came from compile_user_wasm and we've paid for memory expansion let instance = unsafe { - NativeInstance::deserialize_cached(module, config.version, evm_api, evm_data, long_term_tag, debug_chain) + NativeInstance::deserialize_cached( + module, + config.version, + evm_api, + evm_data, + long_term_tag, + debug_chain, + ) }; let mut instance = match instance { Ok(instance) => instance, @@ -241,7 +248,12 @@ pub unsafe extern "C" fn stylus_cache_module( /// Evicts an activated user program from the init cache. #[no_mangle] -pub extern "C" fn stylus_evict_module(module_hash: Bytes32, version: u16, arbos_tag: u32, debug: bool) { +pub extern "C" fn stylus_evict_module( + module_hash: Bytes32, + version: u16, + arbos_tag: u32, + debug: bool, +) { InitCache::evict(module_hash, version, arbos_tag, debug); } diff --git a/arbitrator/stylus/src/native.rs b/arbitrator/stylus/src/native.rs index 38155818c0..2858d59fdc 100644 --- a/arbitrator/stylus/src/native.rs +++ b/arbitrator/stylus/src/native.rs @@ -126,7 +126,8 @@ impl> NativeInstance { if !env.evm_data.cached { long_term_tag = 0; } - let (module, store) = InitCache::insert(module_hash, module, version, long_term_tag, debug)?; + let (module, store) = + InitCache::insert(module_hash, module, version, long_term_tag, debug)?; Self::from_module(module, store, env) } From faa405c6799d852a2a9d7cfb38e7688464627d97 Mon Sep 17 00:00:00 2001 From: Diego Ximenes Date: Mon, 20 May 2024 10:04:41 -0300 Subject: [PATCH 087/113] adjust error and log msg to use 'expected blockhashes' instead of 'last batch messages block hashes' when pruning block hashes from the db --- arbnode/message_pruner.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/arbnode/message_pruner.go b/arbnode/message_pruner.go index c31dbc496d..5d18341a27 100644 --- a/arbnode/message_pruner.go +++ b/arbnode/message_pruner.go @@ -118,10 +118,10 @@ func (m *MessagePruner) prune(ctx context.Context, count arbutil.MessageIndex, g func (m *MessagePruner) deleteOldMessagesFromDB(ctx context.Context, messageCount arbutil.MessageIndex, delayedMessageCount uint64) error { prunedKeysRange, err := deleteFromLastPrunedUptoEndKey(ctx, m.transactionStreamer.db, blockHashInputFeedPrefix, &m.cachedPrunedBlockHashesInputFeed, uint64(messageCount)) if err != nil { - return fmt.Errorf("error deleting last batch messages' block hashes: %w", err) + return fmt.Errorf("error deleting expected block hashes: %w", err) } if len(prunedKeysRange) > 0 { - log.Info("Pruned last batch messages' block hashes:", "first pruned key", prunedKeysRange[0], "last pruned key", prunedKeysRange[len(prunedKeysRange)-1]) + log.Info("Pruned expected block hashes:", "first pruned key", prunedKeysRange[0], "last pruned key", prunedKeysRange[len(prunedKeysRange)-1]) } prunedKeysRange, err = deleteFromLastPrunedUptoEndKey(ctx, m.transactionStreamer.db, messagePrefix, &m.cachedPrunedMessages, uint64(messageCount)) From 7f8d471028d3093c5110dc429d5f5d9fddfe5878 Mon Sep 17 00:00:00 2001 From: Diego Ximenes Date: Mon, 20 May 2024 10:06:20 -0300 Subject: [PATCH 088/113] fix: uses arbnode.BlockHashMismatchLogMsg instead of block_hash_mismatch --- system_tests/seqfeed_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/system_tests/seqfeed_test.go b/system_tests/seqfeed_test.go index 589a48d3af..ab30598b60 100644 --- a/system_tests/seqfeed_test.go +++ b/system_tests/seqfeed_test.go @@ -84,8 +84,8 @@ func TestSequencerFeed(t *testing.T) { t.Fatal("Unexpected balance:", l2balance) } - if logHandler.WasLogged("block_hash_mismatch") { - t.Fatal("block_hash_mismatch was logged unexpectedly") + if logHandler.WasLogged(arbnode.BlockHashMismatchLogMsg) { + t.Fatal("BlockHashMismatchLogMsg was logged unexpectedly") } } From 019581e7e733139c331751ae6485ffbd153f8dd5 Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Mon, 20 May 2024 14:18:26 -0600 Subject: [PATCH 089/113] allowed-wasm-module-roots: accept paths as well --- cmd/nitro/nitro.go | 17 ++++++++++++++++- validator/valnode/valnode.go | 2 +- 2 files changed, 17 insertions(+), 2 deletions(-) diff --git a/cmd/nitro/nitro.go b/cmd/nitro/nitro.go index 9280c3af02..473df21811 100644 --- a/cmd/nitro/nitro.go +++ b/cmd/nitro/nitro.go @@ -6,6 +6,7 @@ package main import ( "context" "crypto/ecdsa" + "encoding/hex" "errors" "fmt" "io" @@ -452,7 +453,21 @@ func mainImpl() int { if len(allowedWasmModuleRoots) > 0 { moduleRootMatched := false for _, root := range allowedWasmModuleRoots { - if common.HexToHash(root) == moduleRoot { + bytes, err := hex.DecodeString(root) + if err == nil { + if common.HexToHash(root) == common.BytesToHash(bytes) { + moduleRootMatched = true + break + } + continue + } + locator, locatorErr := server_common.NewMachineLocator(root) + if err != nil { + log.Warn("allowed-wasm-module-roots: value not a hex nor valid path:", "value", root, "locatorErr", locatorErr, "decodeErr", err) + continue + } + path := locator.GetMachinePath(moduleRoot) + if _, err := os.Stat(path); err == nil { moduleRootMatched = true break } diff --git a/validator/valnode/valnode.go b/validator/valnode/valnode.go index 93a5b37238..972e11189d 100644 --- a/validator/valnode/valnode.go +++ b/validator/valnode/valnode.go @@ -25,7 +25,7 @@ type WasmConfig struct { func WasmConfigAddOptions(prefix string, f *pflag.FlagSet) { f.String(prefix+".root-path", DefaultWasmConfig.RootPath, "path to machine folders, each containing wasm files (machine.wavm.br, replay.wasm)") f.Bool(prefix+".enable-wasmroots-check", DefaultWasmConfig.EnableWasmrootsCheck, "enable check for compatibility of on-chain WASM module root with node") - f.StringSlice(prefix+".allowed-wasm-module-roots", DefaultWasmConfig.AllowedWasmModuleRoots, "list of WASM module roots to check if the on-chain WASM module root belongs to on node startup") + f.StringSlice(prefix+".allowed-wasm-module-roots", DefaultWasmConfig.AllowedWasmModuleRoots, "list of WASM module roots or mahcine base paths to match against on-chain WasmModuleRoot") } var DefaultWasmConfig = WasmConfig{ From 458669ad9a2e003c3c8ec920b24798a378b080c2 Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Mon, 20 May 2024 17:59:45 -0600 Subject: [PATCH 090/113] dockerfile: sort split-validator support nitro has legacy machines and config to check for these wasmModuleRots nitro-validator has split-validation on nitro-dev is based of validator and has latest as well --- Dockerfile | 37 +++++++++++++++++++------------------ scripts/split-val-entry.sh | 2 +- 2 files changed, 20 insertions(+), 19 deletions(-) diff --git a/Dockerfile b/Dockerfile index 5c56b60cc0..f7e26ec084 100644 --- a/Dockerfile +++ b/Dockerfile @@ -203,6 +203,7 @@ COPY ./scripts/download-machine.sh . #RUN ./download-machine.sh consensus-v11 0xf4389b835497a910d7ba3ebfb77aa93da985634f3c052de1290360635be40c4a #RUN ./download-machine.sh consensus-v11.1 0x68e4fe5023f792d4ef584796c84d710303a5e12ea02d6e37e2b5e9c4332507c4 #RUN ./download-machine.sh consensus-v20 0x8b104a2e80ac6165dc58b9048de12f301d70b02a0ab51396c22b4b4b802a16a4 +RUN ./download-machine.sh consensus-v30-rc.1 0x8805d035d5fdb8bb4450f306d9ab82633e2b6316260529cdcaf1b3702afbd5d5 FROM golang:1.21-bookworm as node-builder WORKDIR /workspace @@ -268,11 +269,15 @@ USER user WORKDIR /home/user/ ENTRYPOINT [ "/usr/local/bin/nitro" ] +FROM offchainlabs/nitro-node:v2.3.4-rc.5-b4cc111 as nitro-legacy + FROM nitro-node-slim as nitro-node USER root COPY --from=prover-export /bin/jit /usr/local/bin/ COPY --from=node-builder /workspace/target/bin/daserver /usr/local/bin/ COPY --from=node-builder /workspace/target/bin/datool /usr/local/bin/ +COPY --from=nitro-legacy /home/user/target/machines /home/user/nitro-legacy/machines +RUN rm -rf /workspace/target/legacy-machines/latest RUN export DEBIAN_FRONTEND=noninteractive && \ apt-get update && \ apt-get install -y \ @@ -282,10 +287,23 @@ RUN export DEBIAN_FRONTEND=noninteractive && \ apt-get clean && \ rm -rf /var/lib/apt/lists/* /usr/share/doc/* /var/cache/ldconfig/aux-cache /usr/lib/python3.9/__pycache__/ /usr/lib/python3.9/*/__pycache__/ /var/log/* && \ nitro --version +ENTRYPOINT [ "/usr/local/bin/nitro" , "--validation.wasm.allowed-wasm-module-roots", "/home/user/nitro-legacy/machines,/workspace/machines"] USER user -FROM nitro-node as nitro-node-dev-base +FROM nitro-node as nitro-node-validator +USER root +COPY --from=nitro-legacy /usr/local/bin/nitro-val /home/user/nitro-legacy/bin/nitro-val +COPY --from=nitro-legacy /usr/local/bin/jit /home/user/nitro-legacy/bin/jit +RUN export DEBIAN_FRONTEND=noninteractive && \ + apt-get update && \ + apt-get install -y xxd netcat-traditional && \ + rm -rf /var/lib/apt/lists/* /usr/share/doc/* /var/cache/ldconfig/aux-cache /usr/lib/python3.9/__pycache__/ /usr/lib/python3.9/*/__pycache__/ /var/log/* +COPY scripts/split-val-entry.sh /usr/local/bin +ENTRYPOINT [ "/usr/local/bin/split-val-entry.sh" ] +USER user + +FROM nitro-node-validator as nitro-node-dev USER root # Copy in latest WASM module root RUN rm -f /home/user/target/machines/latest @@ -309,22 +327,5 @@ RUN export DEBIAN_FRONTEND=noninteractive && \ USER user -FROM offchainlabs/nitro-node:v2.3.4-rc.5-b4cc111 as nitro-legacy - -FROM nitro-node-dev-base as nitro-node-dev -USER root - -RUN export DEBIAN_FRONTEND=noninteractive && \ - apt-get update && \ - apt-get install -y xxd netcat-traditional && \ - rm -rf /var/lib/apt/lists/* /usr/share/doc/* /var/cache/ldconfig/aux-cache /usr/lib/python3.9/__pycache__/ /usr/lib/python3.9/*/__pycache__/ /var/log/* -COPY scripts/split-val-entry.sh /usr/local/bin -COPY --from=nitro-legacy /home/user/target/machines /home/user/nitro-legacy/machines -RUN rm -rf /workspace/target/legacy-machines/latest -COPY --from=nitro-legacy /usr/local/bin/nitro-val /home/user/nitro-legacy/bin/nitro-val -COPY --from=nitro-legacy /usr/local/bin/jit /home/user/nitro-legacy/bin/jit -ENTRYPOINT [ "/usr/local/bin/split-val-entry.sh" ] -USER user - FROM nitro-node as nitro-node-default # Just to ensure nitro-node-dist is default diff --git a/scripts/split-val-entry.sh b/scripts/split-val-entry.sh index 6f56a8ec46..1f640f9763 100755 --- a/scripts/split-val-entry.sh +++ b/scripts/split-val-entry.sh @@ -16,4 +16,4 @@ for port in 52000 52001; do done done echo launching nitro-node -/usr/local/bin/nitro --node.block-validator.validation-server-configs-list='[{"jwtsecret":"/tmp/nitro-val.jwt","url":"http://127.0.0.10:52000"}, {"jwtsecret":"/tmp/nitro-val.jwt","url":"http://127.0.0.10:52001"}]' "$@" +/usr/local/bin/nitro --validation.wasm.allowed-wasm-module-roots /home/user/nitro-legacy/machines,/workspace/machines --node.block-validator.validation-server-configs-list='[{"jwtsecret":"/tmp/nitro-val.jwt","url":"http://127.0.0.10:52000"}, {"jwtsecret":"/tmp/nitro-val.jwt","url":"http://127.0.0.10:52001"}]' "$@" From b3693be5a21f45d5e9ec63c681068844995d3dbd Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Mon, 20 May 2024 18:41:22 -0600 Subject: [PATCH 091/113] log when choosing validator --- staker/block_validator.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/staker/block_validator.go b/staker/block_validator.go index e494b3da10..50ccac0471 100644 --- a/staker/block_validator.go +++ b/staker/block_validator.go @@ -1097,13 +1097,18 @@ func (v *BlockValidator) Initialize(ctx context.Context) error { for _, root := range moduleRoots { if v.redisValidator != nil && validator.SpawnerSupportsModule(v.redisValidator, root) { v.chosenValidator[root] = v.redisValidator + log.Info("validator chosen", "WasmMosuleRoot", root, "chosen", "redis") } else { for _, spawner := range v.execSpawners { if validator.SpawnerSupportsModule(spawner, root) { v.chosenValidator[root] = spawner + log.Info("validator chosen", "WasmMosuleRoot", root, "chosen", spawner.Name()) break } } + if v.chosenValidator[root] == nil { + log.Error("validator not found", "WasmMosuleRoot", root) + } } } return nil From c79b98d6bbb3b36a43e089e2fa622c676ae5c1b5 Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Mon, 20 May 2024 20:30:54 -0600 Subject: [PATCH 092/113] fix moduleRoots condition --- cmd/nitro/nitro.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmd/nitro/nitro.go b/cmd/nitro/nitro.go index 473df21811..815257cf7a 100644 --- a/cmd/nitro/nitro.go +++ b/cmd/nitro/nitro.go @@ -462,7 +462,7 @@ func mainImpl() int { continue } locator, locatorErr := server_common.NewMachineLocator(root) - if err != nil { + if locatorErr != nil { log.Warn("allowed-wasm-module-roots: value not a hex nor valid path:", "value", root, "locatorErr", locatorErr, "decodeErr", err) continue } From dc7e874065523970eae4d4f6c1b20f991c2c228b Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Mon, 20 May 2024 20:41:22 -0600 Subject: [PATCH 093/113] Dockerfile: use consensus 30-rc.2 --- Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Dockerfile b/Dockerfile index f7e26ec084..e5718868fa 100644 --- a/Dockerfile +++ b/Dockerfile @@ -203,7 +203,7 @@ COPY ./scripts/download-machine.sh . #RUN ./download-machine.sh consensus-v11 0xf4389b835497a910d7ba3ebfb77aa93da985634f3c052de1290360635be40c4a #RUN ./download-machine.sh consensus-v11.1 0x68e4fe5023f792d4ef584796c84d710303a5e12ea02d6e37e2b5e9c4332507c4 #RUN ./download-machine.sh consensus-v20 0x8b104a2e80ac6165dc58b9048de12f301d70b02a0ab51396c22b4b4b802a16a4 -RUN ./download-machine.sh consensus-v30-rc.1 0x8805d035d5fdb8bb4450f306d9ab82633e2b6316260529cdcaf1b3702afbd5d5 +RUN ./download-machine.sh consensus-v30-rc.2 0xb0de9cb89e4d944ae6023a3b62276e54804c242fd8c4c2d8e6cc4450f5fa8b1b FROM golang:1.21-bookworm as node-builder WORKDIR /workspace From 71c9da7c8cf98933c8d306cf198fcbc269727917 Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Tue, 21 May 2024 08:45:33 -0600 Subject: [PATCH 094/113] Dockerfile: fix path --- Dockerfile | 2 +- scripts/split-val-entry.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/Dockerfile b/Dockerfile index e5718868fa..58976fc6e1 100644 --- a/Dockerfile +++ b/Dockerfile @@ -287,7 +287,7 @@ RUN export DEBIAN_FRONTEND=noninteractive && \ apt-get clean && \ rm -rf /var/lib/apt/lists/* /usr/share/doc/* /var/cache/ldconfig/aux-cache /usr/lib/python3.9/__pycache__/ /usr/lib/python3.9/*/__pycache__/ /var/log/* && \ nitro --version -ENTRYPOINT [ "/usr/local/bin/nitro" , "--validation.wasm.allowed-wasm-module-roots", "/home/user/nitro-legacy/machines,/workspace/machines"] +ENTRYPOINT [ "/usr/local/bin/nitro" , "--validation.wasm.allowed-wasm-module-roots", "/home/user/nitro-legacy/machines,/home/user/target/machines"] USER user diff --git a/scripts/split-val-entry.sh b/scripts/split-val-entry.sh index 1f640f9763..8e1be0f6cc 100755 --- a/scripts/split-val-entry.sh +++ b/scripts/split-val-entry.sh @@ -16,4 +16,4 @@ for port in 52000 52001; do done done echo launching nitro-node -/usr/local/bin/nitro --validation.wasm.allowed-wasm-module-roots /home/user/nitro-legacy/machines,/workspace/machines --node.block-validator.validation-server-configs-list='[{"jwtsecret":"/tmp/nitro-val.jwt","url":"http://127.0.0.10:52000"}, {"jwtsecret":"/tmp/nitro-val.jwt","url":"http://127.0.0.10:52001"}]' "$@" +/usr/local/bin/nitro --validation.wasm.allowed-wasm-module-roots /home/user/nitro-legacy/machines,/home/user/target/machines --node.block-validator.validation-server-configs-list='[{"jwtsecret":"/tmp/nitro-val.jwt","url":"http://127.0.0.10:52000"}, {"jwtsecret":"/tmp/nitro-val.jwt","url":"http://127.0.0.10:52001"}]' "$@" From 153ffa76ca5226b2068d01acef643fee3f0e0fa9 Mon Sep 17 00:00:00 2001 From: Diego Ximenes Date: Tue, 21 May 2024 13:32:07 -0300 Subject: [PATCH 095/113] add apt-get update to wasm-libs-builder --- Dockerfile | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/Dockerfile b/Dockerfile index 5c56b60cc0..19a0b46ebd 100644 --- a/Dockerfile +++ b/Dockerfile @@ -41,7 +41,8 @@ RUN apt-get update && apt-get install -y curl build-essential=12.9 FROM wasm-base as wasm-libs-builder # clang / lld used by soft-float wasm -RUN apt-get install -y clang=1:14.0-55.7~deb12u1 lld=1:14.0-55.7~deb12u1 wabt +RUN apt-get update && \ + apt-get install -y clang=1:14.0-55.7~deb12u1 lld=1:14.0-55.7~deb12u1 wabt # pinned rust 1.75.0 RUN curl --proto '=https' --tlsv1.2 -sSf https://sh.rustup.rs | sh -s -- -y --default-toolchain 1.75.0 --target x86_64-unknown-linux-gnu wasm32-unknown-unknown wasm32-wasi COPY ./Makefile ./ From de58296c1a41da7cf9b4fce82ab5687b4925bd47 Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Tue, 21 May 2024 12:28:32 -0600 Subject: [PATCH 096/113] geth: update --- go-ethereum | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/go-ethereum b/go-ethereum index 940fbe020e..b8d4ced531 160000 --- a/go-ethereum +++ b/go-ethereum @@ -1 +1 @@ -Subproject commit 940fbe020e03707365da09de939058944d9e1f5d +Subproject commit b8d4ced5316c987d095ef1fc3ecb5e8ae0df094d From e8685b359cd82771ec9b5c30900c32e4a142834a Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Tue, 21 May 2024 16:52:49 -0600 Subject: [PATCH 097/113] fix typo --- go-ethereum | 2 +- staker/block_validator.go | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/go-ethereum b/go-ethereum index 8048ac4bed..b8d4ced531 160000 --- a/go-ethereum +++ b/go-ethereum @@ -1 +1 @@ -Subproject commit 8048ac4bed2eda18284e3c022ea5ee4cce771134 +Subproject commit b8d4ced5316c987d095ef1fc3ecb5e8ae0df094d diff --git a/staker/block_validator.go b/staker/block_validator.go index 50ccac0471..027ee78248 100644 --- a/staker/block_validator.go +++ b/staker/block_validator.go @@ -1097,17 +1097,17 @@ func (v *BlockValidator) Initialize(ctx context.Context) error { for _, root := range moduleRoots { if v.redisValidator != nil && validator.SpawnerSupportsModule(v.redisValidator, root) { v.chosenValidator[root] = v.redisValidator - log.Info("validator chosen", "WasmMosuleRoot", root, "chosen", "redis") + log.Info("validator chosen", "WasmModuleRoot", root, "chosen", "redis") } else { for _, spawner := range v.execSpawners { if validator.SpawnerSupportsModule(spawner, root) { v.chosenValidator[root] = spawner - log.Info("validator chosen", "WasmMosuleRoot", root, "chosen", spawner.Name()) + log.Info("validator chosen", "WasmModuleRoot", root, "chosen", spawner.Name()) break } } if v.chosenValidator[root] == nil { - log.Error("validator not found", "WasmMosuleRoot", root) + log.Error("validator not found", "WasmModuleRoot", root) } } } From aaf4d1c8ce1baa12d14b3becaf51510fb687d654 Mon Sep 17 00:00:00 2001 From: Lee Bousfield Date: Tue, 21 May 2024 21:03:29 -0500 Subject: [PATCH 098/113] Fix off-by-one in data poster nonce check --- arbnode/dataposter/data_poster.go | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/arbnode/dataposter/data_poster.go b/arbnode/dataposter/data_poster.go index fb35ac3c8d..34ca9e1483 100644 --- a/arbnode/dataposter/data_poster.go +++ b/arbnode/dataposter/data_poster.go @@ -857,24 +857,23 @@ func (p *DataPoster) sendTx(ctx context.Context, prevTx *storage.QueuedTransacti return fmt.Errorf("couldn't get preceding tx in DataPoster to check if should send tx with nonce %d: %w", newTx.FullTx.Nonce(), err) } if precedingTx != nil { // precedingTx == nil -> the actual preceding tx was already confirmed - var latestBlockNumber, prevBlockNumber, reorgResistantNonce uint64 if precedingTx.FullTx.Type() != newTx.FullTx.Type() || !precedingTx.Sent { - latestBlockNumber, err = p.client.BlockNumber(ctx) + latestBlockNumber, err := p.client.BlockNumber(ctx) if err != nil { return fmt.Errorf("couldn't get block number in DataPoster to check if should send tx with nonce %d: %w", newTx.FullTx.Nonce(), err) } - prevBlockNumber = arbmath.SaturatingUSub(latestBlockNumber, 1) - reorgResistantNonce, err = p.client.NonceAt(ctx, p.Sender(), new(big.Int).SetUint64(prevBlockNumber)) + prevBlockNumber := arbmath.SaturatingUSub(latestBlockNumber, 1) + reorgResistantTxCount, err := p.client.NonceAt(ctx, p.Sender(), new(big.Int).SetUint64(prevBlockNumber)) if err != nil { return fmt.Errorf("couldn't determine reorg resistant nonce in DataPoster to check if should send tx with nonce %d: %w", newTx.FullTx.Nonce(), err) } - if precedingTx.FullTx.Nonce() > reorgResistantNonce { - log.Info("DataPoster is avoiding creating a mempool nonce gap (the tx remains queued and will be retried)", "nonce", newTx.FullTx.Nonce(), "prevType", precedingTx.FullTx.Type(), "type", newTx.FullTx.Type(), "prevSent", precedingTx.Sent) + if newTx.FullTx.Nonce() > reorgResistantTxCount { + log.Info("DataPoster is avoiding creating a mempool nonce gap (the tx remains queued and will be retried)", "nonce", newTx.FullTx.Nonce(), "prevType", precedingTx.FullTx.Type(), "type", newTx.FullTx.Type(), "prevSent", precedingTx.Sent, "latestBlockNumber", latestBlockNumber, "prevBlockNumber", prevBlockNumber, "reorgResistantTxCount", reorgResistantTxCount) return nil } } else { - log.Info("DataPoster will send previously unsent batch tx", "nonce", newTx.FullTx.Nonce(), "prevType", precedingTx.FullTx.Type(), "type", newTx.FullTx.Type(), "prevSent", precedingTx.Sent, "latestBlockNumber", latestBlockNumber, "prevBlockNumber", prevBlockNumber, "reorgResistantNonce", reorgResistantNonce) + log.Info("DataPoster will send previously unsent batch tx", "nonce", newTx.FullTx.Nonce(), "prevType", precedingTx.FullTx.Type(), "type", newTx.FullTx.Type(), "prevSent", precedingTx.Sent) } } } From 345e828b430efff7b66d401abe21759cc0af3abc Mon Sep 17 00:00:00 2001 From: Lee Bousfield Date: Tue, 21 May 2024 21:41:53 -0500 Subject: [PATCH 099/113] Always log when sending previously unsent tx --- arbnode/dataposter/data_poster.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/arbnode/dataposter/data_poster.go b/arbnode/dataposter/data_poster.go index 34ca9e1483..399bc19dbd 100644 --- a/arbnode/dataposter/data_poster.go +++ b/arbnode/dataposter/data_poster.go @@ -857,13 +857,14 @@ func (p *DataPoster) sendTx(ctx context.Context, prevTx *storage.QueuedTransacti return fmt.Errorf("couldn't get preceding tx in DataPoster to check if should send tx with nonce %d: %w", newTx.FullTx.Nonce(), err) } if precedingTx != nil { // precedingTx == nil -> the actual preceding tx was already confirmed + var latestBlockNumber, prevBlockNumber, reorgResistantTxCount uint64 if precedingTx.FullTx.Type() != newTx.FullTx.Type() || !precedingTx.Sent { - latestBlockNumber, err := p.client.BlockNumber(ctx) + latestBlockNumber, err = p.client.BlockNumber(ctx) if err != nil { return fmt.Errorf("couldn't get block number in DataPoster to check if should send tx with nonce %d: %w", newTx.FullTx.Nonce(), err) } - prevBlockNumber := arbmath.SaturatingUSub(latestBlockNumber, 1) - reorgResistantTxCount, err := p.client.NonceAt(ctx, p.Sender(), new(big.Int).SetUint64(prevBlockNumber)) + prevBlockNumber = arbmath.SaturatingUSub(latestBlockNumber, 1) + reorgResistantTxCount, err = p.client.NonceAt(ctx, p.Sender(), new(big.Int).SetUint64(prevBlockNumber)) if err != nil { return fmt.Errorf("couldn't determine reorg resistant nonce in DataPoster to check if should send tx with nonce %d: %w", newTx.FullTx.Nonce(), err) } @@ -872,9 +873,8 @@ func (p *DataPoster) sendTx(ctx context.Context, prevTx *storage.QueuedTransacti log.Info("DataPoster is avoiding creating a mempool nonce gap (the tx remains queued and will be retried)", "nonce", newTx.FullTx.Nonce(), "prevType", precedingTx.FullTx.Type(), "type", newTx.FullTx.Type(), "prevSent", precedingTx.Sent, "latestBlockNumber", latestBlockNumber, "prevBlockNumber", prevBlockNumber, "reorgResistantTxCount", reorgResistantTxCount) return nil } - } else { - log.Info("DataPoster will send previously unsent batch tx", "nonce", newTx.FullTx.Nonce(), "prevType", precedingTx.FullTx.Type(), "type", newTx.FullTx.Type(), "prevSent", precedingTx.Sent) } + log.Debug("DataPoster will send previously unsent batch tx", "nonce", newTx.FullTx.Nonce(), "prevType", precedingTx.FullTx.Type(), "type", newTx.FullTx.Type(), "prevSent", precedingTx.Sent, "latestBlockNumber", latestBlockNumber, "prevBlockNumber", prevBlockNumber, "reorgResistantTxCount", reorgResistantTxCount) } } From 8737d5ccca5c252797af89906f1c5840df93d6ee Mon Sep 17 00:00:00 2001 From: Lee Bousfield Date: Tue, 21 May 2024 21:45:39 -0500 Subject: [PATCH 100/113] Improve previouslySent check --- arbnode/dataposter/data_poster.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/arbnode/dataposter/data_poster.go b/arbnode/dataposter/data_poster.go index 399bc19dbd..5aaef959d8 100644 --- a/arbnode/dataposter/data_poster.go +++ b/arbnode/dataposter/data_poster.go @@ -851,7 +851,8 @@ func (p *DataPoster) sendTx(ctx context.Context, prevTx *storage.QueuedTransacti // different type with a lower nonce. // If we decide not to send this tx yet, just leave it queued and with Sent set to false. // The resending/repricing loop in DataPoster.Start will keep trying. - if !newTx.Sent && newTx.FullTx.Nonce() > 0 { + previouslySent := newTx.Sent || (prevTx != nil && prevTx.Sent) // if we've previously sent this nonce + if !previouslySent && newTx.FullTx.Nonce() > 0 { precedingTx, err := p.queue.Get(ctx, arbmath.SaturatingUSub(newTx.FullTx.Nonce(), 1)) if err != nil { return fmt.Errorf("couldn't get preceding tx in DataPoster to check if should send tx with nonce %d: %w", newTx.FullTx.Nonce(), err) From 65f8bb569adeb743f645412df0e4c80346920c39 Mon Sep 17 00:00:00 2001 From: Maciej Kulawik Date: Wed, 22 May 2024 13:53:13 +0200 Subject: [PATCH 101/113] update geth --- go-ethereum | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/go-ethereum b/go-ethereum index 5b7b36a339..07f6d7a8c1 160000 --- a/go-ethereum +++ b/go-ethereum @@ -1 +1 @@ -Subproject commit 5b7b36a339ac28d708bca072dc5ec8189ceadac2 +Subproject commit 07f6d7a8c149f8752aa8deef4598cfd184a37e94 From 5570fb3e57c574b2004aca9ecd3ad1831bd5ee4c Mon Sep 17 00:00:00 2001 From: Nodar Ambroladze Date: Wed, 22 May 2024 14:18:09 +0200 Subject: [PATCH 102/113] Drop listenForInterrupt, since stopAndWait is already called on sigint --- pubsub/consumer.go | 22 +--------------------- pubsub/pubsub_test.go | 10 +++------- 2 files changed, 4 insertions(+), 28 deletions(-) diff --git a/pubsub/consumer.go b/pubsub/consumer.go index 97ab004764..0288c19e45 100644 --- a/pubsub/consumer.go +++ b/pubsub/consumer.go @@ -5,10 +5,7 @@ import ( "encoding/json" "errors" "fmt" - "os" - "os/signal" "sync/atomic" - "syscall" "time" "github.com/ethereum/go-ethereum/log" @@ -53,7 +50,6 @@ type Consumer[Request any, Response any] struct { // terminating indicates whether interrupt was received, in which case // consumer should clean up for graceful shutdown. terminating atomic.Bool - signals chan os.Signal } type Message[Request any] struct { @@ -72,14 +68,12 @@ func NewConsumer[Request any, Response any](client redis.UniversalClient, stream redisGroup: streamName, // There is 1-1 mapping of redis stream and consumer group. cfg: cfg, terminating: atomic.Bool{}, - signals: make(chan os.Signal, 1), }, nil } // Start starts the consumer to iteratively perform heartbeat in configured intervals. func (c *Consumer[Request, Response]) Start(ctx context.Context) { c.StopWaiter.Start(ctx, c) - c.listenForInterrupt() c.StopWaiter.CallIteratively( func(ctx context.Context) time.Duration { if !c.terminating.Load() { @@ -92,22 +86,8 @@ func (c *Consumer[Request, Response]) Start(ctx context.Context) { ) } -// listenForInterrupt launches a thread that notifies the channel when interrupt -// is received. -func (c *Consumer[Request, Response]) listenForInterrupt() { - signal.Notify(c.signals, syscall.SIGINT, syscall.SIGTERM) - c.StopWaiter.LaunchThread(func(ctx context.Context) { - select { - case sig := <-c.signals: - log.Info("Received interrup", "signal", sig.String()) - case <-ctx.Done(): - log.Info("Context is done", "error", ctx.Err()) - } - c.deleteHeartBeat(ctx) - }) -} - func (c *Consumer[Request, Response]) StopAndWait() { + c.deleteHeartBeat(c.GetParentContext()) c.StopWaiter.StopAndWait() } diff --git a/pubsub/pubsub_test.go b/pubsub/pubsub_test.go index 85314dc29a..cdf5fa1ef6 100644 --- a/pubsub/pubsub_test.go +++ b/pubsub/pubsub_test.go @@ -232,15 +232,11 @@ func TestRedisProduce(t *testing.T) { // Consumer messages in every third consumer but don't ack them to check // that other consumers will claim ownership on those messages. for i := 0; i < len(consumers); i += 3 { + consumers[i].Start(ctx) if _, err := consumers[i].Consume(ctx); err != nil { t.Errorf("Error consuming message: %v", err) } - // Terminate half of the consumers, send interrupt to others. - if i%2 == 0 { - consumers[i].StopAndWait() - } else { - consumers[i].signals <- os.Interrupt - } + consumers[i].StopAndWait() } } @@ -252,7 +248,7 @@ func TestRedisProduce(t *testing.T) { } producer.StopAndWait() for _, c := range consumers { - c.StopWaiter.StopAndWait() + c.StopAndWait() } got, err := mergeValues(gotMessages) if err != nil { From 9a866114c9ea15e6efd0bf4452dfa0ec67cdb3b8 Mon Sep 17 00:00:00 2001 From: Nodar Ambroladze Date: Wed, 22 May 2024 14:55:52 +0200 Subject: [PATCH 103/113] Fix test --- pubsub/pubsub_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/pubsub/pubsub_test.go b/pubsub/pubsub_test.go index cdf5fa1ef6..72504602e3 100644 --- a/pubsub/pubsub_test.go +++ b/pubsub/pubsub_test.go @@ -285,6 +285,7 @@ func TestRedisReproduceDisabled(t *testing.T) { // Consumer messages in every third consumer but don't ack them to check // that other consumers will claim ownership on those messages. for i := 0; i < len(consumers); i += 3 { + consumers[i].Start(ctx) if _, err := consumers[i].Consume(ctx); err != nil { t.Errorf("Error consuming message: %v", err) } From 7d67d8b9263deb7d9492f5cd7b08595b6c99f2b4 Mon Sep 17 00:00:00 2001 From: Nodar Ambroladze Date: Wed, 22 May 2024 16:41:05 +0200 Subject: [PATCH 104/113] Use in memory buffer and dump on disk only if needed --- execution/gethexec/sequencer.go | 53 +++++++++++++-------------------- 1 file changed, 21 insertions(+), 32 deletions(-) diff --git a/execution/gethexec/sequencer.go b/execution/gethexec/sequencer.go index 4247556905..c40669495c 100644 --- a/execution/gethexec/sequencer.go +++ b/execution/gethexec/sequencer.go @@ -4,12 +4,14 @@ package gethexec import ( + "bytes" "context" "errors" "fmt" "math" "math/big" "os" + "path" "runtime/debug" "runtime/pprof" "runtime/trace" @@ -768,41 +770,15 @@ func (s *Sequencer) precheckNonces(queueItems []txQueueItem) []txQueueItem { return outputQueueItems } -func deleteFiles(files ...*os.File) { - for _, f := range files { - if err := os.Remove(f.Name()); err != nil { - log.Error("Error removing file", "name", f.Name()) - } - } -} - -func closeFiles(files ...*os.File) { - for _, f := range files { - if err := os.Remove(f.Name()); err != nil { - log.Error("Error closing file", "name", f.Name()) - } - } -} - // createBlockWithProfiling runs create block with tracing and CPU profiling // enabled. If the block creation takes longer than 5 seconds, it keeps both // and prints out filenames in an error log line. func (s *Sequencer) createBlockWithProfiling(ctx context.Context) bool { - id := uuid.NewString() - pprofFile, err := os.CreateTemp("", id+".pprof") - if err != nil { - log.Error("Creating temporary file for profiling CPU", "error", err) - } - traceFile, err := os.CreateTemp("", id+".trace") - if err != nil { - log.Error("Creating temporary file for tracing", "error", err) - } - if err := pprof.StartCPUProfile(pprofFile); err != nil { + pprofBuf, traceBuf := bytes.NewBuffer(nil), bytes.NewBuffer(nil) + if err := pprof.StartCPUProfile(pprofBuf); err != nil { log.Error("Starting CPU profiling", "error", err) - deleteFiles(pprofFile) } - if err := trace.Start(traceFile); err != nil { - deleteFiles(traceFile) + if err := trace.Start(traceBuf); err != nil { log.Error("Starting tracing", "error", err) } start := time.Now() @@ -810,15 +786,28 @@ func (s *Sequencer) createBlockWithProfiling(ctx context.Context) bool { elapsed := time.Since(start) pprof.StopCPUProfile() trace.Stop() - closeFiles(pprofFile, traceFile) if elapsed > 5*time.Second { - log.Error("Block creation took longer than 5 seconds", "pprof", pprofFile.Name()) + writeAndLog(pprofBuf, traceBuf) return res } - deleteFiles(pprofFile, traceFile) return res } +func writeAndLog(pprof, trace *bytes.Buffer) { + id := uuid.NewString() + pprofFile := path.Join(os.TempDir(), id+".pprof") + if err := os.WriteFile(pprofFile, pprof.Bytes(), 0o644); err != nil { + log.Error("Creating temporary file for pprof", "fileName", pprofFile, "error", err) + return + } + traceFile := path.Join(os.TempDir(), id+".trace") + if err := os.WriteFile(traceFile, trace.Bytes(), 0o644); err != nil { + log.Error("Creating temporary file for trace", "fileName", traceFile, "error", err) + return + } + log.Debug("Block creation took longer than 5 seconds, created pprof and trace files", "pprof", pprofFile, "traceFile", traceFile) +} + func (s *Sequencer) createBlock(ctx context.Context) (returnValue bool) { var queueItems []txQueueItem var totalBatchSize int From 101c339d6776af808ae269cdb44b838f1377d1fd Mon Sep 17 00:00:00 2001 From: Nodar Ambroladze Date: Wed, 22 May 2024 16:48:44 +0200 Subject: [PATCH 105/113] Fix gosec linter error --- execution/gethexec/sequencer.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/execution/gethexec/sequencer.go b/execution/gethexec/sequencer.go index c40669495c..9a94e35f61 100644 --- a/execution/gethexec/sequencer.go +++ b/execution/gethexec/sequencer.go @@ -796,12 +796,12 @@ func (s *Sequencer) createBlockWithProfiling(ctx context.Context) bool { func writeAndLog(pprof, trace *bytes.Buffer) { id := uuid.NewString() pprofFile := path.Join(os.TempDir(), id+".pprof") - if err := os.WriteFile(pprofFile, pprof.Bytes(), 0o644); err != nil { + if err := os.WriteFile(pprofFile, pprof.Bytes(), 0o600); err != nil { log.Error("Creating temporary file for pprof", "fileName", pprofFile, "error", err) return } traceFile := path.Join(os.TempDir(), id+".trace") - if err := os.WriteFile(traceFile, trace.Bytes(), 0o644); err != nil { + if err := os.WriteFile(traceFile, trace.Bytes(), 0o600); err != nil { log.Error("Creating temporary file for trace", "fileName", traceFile, "error", err) return } From 0ce93785e406c3375cb1931297b4e9580e4faf4f Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Wed, 22 May 2024 08:47:57 -0600 Subject: [PATCH 106/113] block_validator: fail but dont segfault if no validator --- staker/block_validator.go | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/staker/block_validator.go b/staker/block_validator.go index 027ee78248..5a511920f2 100644 --- a/staker/block_validator.go +++ b/staker/block_validator.go @@ -791,8 +791,9 @@ validationsLoop: } for _, moduleRoot := range wasmRoots { if v.chosenValidator[moduleRoot] == nil { - v.possiblyFatal(fmt.Errorf("did not find spawner for moduleRoot :%v", moduleRoot)) - continue + notFoundErr := fmt.Errorf("did not find spawner for moduleRoot :%v", moduleRoot) + v.possiblyFatal(notFoundErr) + return nil, notFoundErr } if v.chosenValidator[moduleRoot].Room() == 0 { log.Trace("advanceValidations: no more room", "moduleRoot", moduleRoot) @@ -1107,7 +1108,7 @@ func (v *BlockValidator) Initialize(ctx context.Context) error { } } if v.chosenValidator[root] == nil { - log.Error("validator not found", "WasmModuleRoot", root) + return fmt.Errorf("cannot validate WasmModuleRoot %v", root) } } } From 16c95d730f4614625c9d5a10b88984d06aaac645 Mon Sep 17 00:00:00 2001 From: Nodar Ambroladze Date: Thu, 23 May 2024 17:28:11 +0200 Subject: [PATCH 107/113] Delete heartbeat after stopAndWait --- pubsub/consumer.go | 2 +- system_tests/block_validator_test.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pubsub/consumer.go b/pubsub/consumer.go index 0288c19e45..4b51d24f2d 100644 --- a/pubsub/consumer.go +++ b/pubsub/consumer.go @@ -87,8 +87,8 @@ func (c *Consumer[Request, Response]) Start(ctx context.Context) { } func (c *Consumer[Request, Response]) StopAndWait() { - c.deleteHeartBeat(c.GetParentContext()) c.StopWaiter.StopAndWait() + c.deleteHeartBeat(c.GetParentContext()) } func heartBeatKey(id string) string { diff --git a/system_tests/block_validator_test.go b/system_tests/block_validator_test.go index dfd892a079..debd6d4c7c 100644 --- a/system_tests/block_validator_test.go +++ b/system_tests/block_validator_test.go @@ -72,7 +72,7 @@ func testBlockValidatorSimple(t *testing.T, dasModeString string, workloadLoops redisURL := "" if useRedisStreams { redisURL = redisutil.CreateTestRedis(ctx, t) - validatorConfig.BlockValidator.RedisValidationClientConfig = redis.DefaultValidationClientConfig + validatorConfig.BlockValidator.RedisValidationClientConfig = redis.TestValidationClientConfig validatorConfig.BlockValidator.RedisValidationClientConfig.RedisURL = redisURL } From 8504c5c0ba8303fdf18ce8efc0f94b1e81b47f00 Mon Sep 17 00:00:00 2001 From: Ganesh Vanahalli Date: Thu, 23 May 2024 08:54:02 -0700 Subject: [PATCH 108/113] Update blocks_reexecutor/blocks_reexecutor.go Co-authored-by: Maciej Kulawik <10907694+magicxyyz@users.noreply.github.com> --- blocks_reexecutor/blocks_reexecutor.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/blocks_reexecutor/blocks_reexecutor.go b/blocks_reexecutor/blocks_reexecutor.go index f58e0ce00f..1e4a06fe90 100644 --- a/blocks_reexecutor/blocks_reexecutor.go +++ b/blocks_reexecutor/blocks_reexecutor.go @@ -35,7 +35,7 @@ func (c *Config) Validate() error { if c.EndBlock < c.StartBlock { return errors.New("invalid block range for blocks re-execution") } - if c.Room < 0 { + if c.Room <= 0 { return errors.New("room for blocks re-execution should be greater than 0") } return nil From 56fc8d4b867351b9d2ed7714360389dd5d5b76ee Mon Sep 17 00:00:00 2001 From: Nodar Ambroladze Date: Thu, 23 May 2024 20:50:19 +0200 Subject: [PATCH 109/113] Drop terminating atomic bool --- pubsub/consumer.go | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/pubsub/consumer.go b/pubsub/consumer.go index 4b51d24f2d..c9590de8e6 100644 --- a/pubsub/consumer.go +++ b/pubsub/consumer.go @@ -5,7 +5,6 @@ import ( "encoding/json" "errors" "fmt" - "sync/atomic" "time" "github.com/ethereum/go-ethereum/log" @@ -47,9 +46,6 @@ type Consumer[Request any, Response any] struct { redisStream string redisGroup string cfg *ConsumerConfig - // terminating indicates whether interrupt was received, in which case - // consumer should clean up for graceful shutdown. - terminating atomic.Bool } type Message[Request any] struct { @@ -67,7 +63,6 @@ func NewConsumer[Request any, Response any](client redis.UniversalClient, stream redisStream: streamName, redisGroup: streamName, // There is 1-1 mapping of redis stream and consumer group. cfg: cfg, - terminating: atomic.Bool{}, }, nil } @@ -76,10 +71,6 @@ func (c *Consumer[Request, Response]) Start(ctx context.Context) { c.StopWaiter.Start(ctx, c) c.StopWaiter.CallIteratively( func(ctx context.Context) time.Duration { - if !c.terminating.Load() { - log.Trace("Consumer is terminating, stopping heartbeat update") - return time.Hour - } c.heartBeat(ctx) return c.cfg.KeepAliveTimeout / 10 }, @@ -101,7 +92,6 @@ func (c *Consumer[Request, Response]) heartBeatKey() string { // deleteHeartBeat deletes the heartbeat to indicate it is being shut down. func (c *Consumer[Request, Response]) deleteHeartBeat(ctx context.Context) { - c.terminating.Store(true) if err := c.client.Del(ctx, c.heartBeatKey()).Err(); err != nil { l := log.Info if ctx.Err() != nil { From ab6fa4cae57f99b7dae10ad07033f53216f05ad8 Mon Sep 17 00:00:00 2001 From: Nodar Ambroladze Date: Thu, 23 May 2024 21:14:41 +0200 Subject: [PATCH 110/113] Switch threshold from 5 to 2 seconds --- execution/gethexec/sequencer.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/execution/gethexec/sequencer.go b/execution/gethexec/sequencer.go index 9a94e35f61..dd84c352a4 100644 --- a/execution/gethexec/sequencer.go +++ b/execution/gethexec/sequencer.go @@ -786,7 +786,7 @@ func (s *Sequencer) createBlockWithProfiling(ctx context.Context) bool { elapsed := time.Since(start) pprof.StopCPUProfile() trace.Stop() - if elapsed > 5*time.Second { + if elapsed > 2*time.Second { writeAndLog(pprofBuf, traceBuf) return res } From e44cfafd05066c250a98c1960240ba88a1087a11 Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Thu, 23 May 2024 16:54:43 -0600 Subject: [PATCH 111/113] testnode: update pin --- nitro-testnode | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/nitro-testnode b/nitro-testnode index e530842e58..c334820b2d 160000 --- a/nitro-testnode +++ b/nitro-testnode @@ -1 +1 @@ -Subproject commit e530842e583e2f3543f97a71c3a7cb53f8a10814 +Subproject commit c334820b2dba6dfa4078f81ed242afbbccc19c91 From 570c31d51d692610607096b4f4a4e92ffa5538d0 Mon Sep 17 00:00:00 2001 From: Ganesh Vanahalli Date: Thu, 23 May 2024 16:20:07 -0700 Subject: [PATCH 112/113] update geth pin --- go-ethereum | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/go-ethereum b/go-ethereum index 07f6d7a8c1..f45f6d7560 160000 --- a/go-ethereum +++ b/go-ethereum @@ -1 +1 @@ -Subproject commit 07f6d7a8c149f8752aa8deef4598cfd184a37e94 +Subproject commit f45f6d75601626daf108aa62ea6cb1549d91c528 From dd27ef17d584d607e0972ac7cd12c734ebf1462d Mon Sep 17 00:00:00 2001 From: Lee Bousfield Date: Thu, 23 May 2024 23:31:18 -0500 Subject: [PATCH 113/113] Allow 0x prefix for allowed-wasm-module-roots flag --- cmd/nitro/nitro.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmd/nitro/nitro.go b/cmd/nitro/nitro.go index 26aedfbfb7..427974b34f 100644 --- a/cmd/nitro/nitro.go +++ b/cmd/nitro/nitro.go @@ -453,7 +453,7 @@ func mainImpl() int { if len(allowedWasmModuleRoots) > 0 { moduleRootMatched := false for _, root := range allowedWasmModuleRoots { - bytes, err := hex.DecodeString(root) + bytes, err := hex.DecodeString(strings.TrimPrefix(root, "0x")) if err == nil { if common.HexToHash(root) == common.BytesToHash(bytes) { moduleRootMatched = true