From fa44e50c84d2ddf4984555da39d58a104975eafb Mon Sep 17 00:00:00 2001 From: Maciej Kulawik Date: Thu, 14 Dec 2023 23:42:22 +0000 Subject: [PATCH 1/9] add prunning test --- cmd/conf/init.go | 57 +++++++ cmd/nitro/init.go | 288 +--------------------------------- cmd/nitro/nitro.go | 6 +- cmd/prunning/prunning.go | 250 +++++++++++++++++++++++++++++ go-ethereum | 2 +- system_tests/common_test.go | 4 +- system_tests/prunning_test.go | 121 ++++++++++++++ 7 files changed, 439 insertions(+), 289 deletions(-) create mode 100644 cmd/conf/init.go create mode 100644 cmd/prunning/prunning.go create mode 100644 system_tests/prunning_test.go diff --git a/cmd/conf/init.go b/cmd/conf/init.go new file mode 100644 index 0000000000..f228891b90 --- /dev/null +++ b/cmd/conf/init.go @@ -0,0 +1,57 @@ +package conf + +import ( + "time" + + "github.com/spf13/pflag" +) + +type InitConfig struct { + Force bool `koanf:"force"` + Url string `koanf:"url"` + DownloadPath string `koanf:"download-path"` + DownloadPoll time.Duration `koanf:"download-poll"` + DevInit bool `koanf:"dev-init"` + DevInitAddress string `koanf:"dev-init-address"` + DevInitBlockNum uint64 `koanf:"dev-init-blocknum"` + Empty bool `koanf:"empty"` + AccountsPerSync uint `koanf:"accounts-per-sync"` + ImportFile string `koanf:"import-file"` + ThenQuit bool `koanf:"then-quit"` + Prune string `koanf:"prune"` + PruneBloomSize uint64 `koanf:"prune-bloom-size"` + ResetToMessage int64 `koanf:"reset-to-message"` +} + +var InitConfigDefault = InitConfig{ + Force: false, + Url: "", + DownloadPath: "/tmp/", + DownloadPoll: time.Minute, + DevInit: false, + DevInitAddress: "", + DevInitBlockNum: 0, + ImportFile: "", + AccountsPerSync: 100000, + ThenQuit: false, + Prune: "", + PruneBloomSize: 2048, + ResetToMessage: -1, +} + +func InitConfigAddOptions(prefix string, f *pflag.FlagSet) { + f.Bool(prefix+".force", InitConfigDefault.Force, "if true: in case database exists init code will be reexecuted and genesis block compared to database") + f.String(prefix+".url", InitConfigDefault.Url, "url to download initializtion data - will poll if download fails") + f.String(prefix+".download-path", InitConfigDefault.DownloadPath, "path to save temp downloaded file") + f.Duration(prefix+".download-poll", InitConfigDefault.DownloadPoll, "how long to wait between polling attempts") + f.Bool(prefix+".dev-init", InitConfigDefault.DevInit, "init with dev data (1 account with balance) instead of file import") + f.String(prefix+".dev-init-address", InitConfigDefault.DevInitAddress, "Address of dev-account. Leave empty to use the dev-wallet.") + f.Uint64(prefix+".dev-init-blocknum", InitConfigDefault.DevInitBlockNum, "Number of preinit blocks. Must exist in ancient database.") + f.Bool(prefix+".empty", InitConfigDefault.Empty, "init with empty state") + f.Bool(prefix+".then-quit", InitConfigDefault.ThenQuit, "quit after init is done") + f.String(prefix+".import-file", InitConfigDefault.ImportFile, "path for json data to import") + f.Uint(prefix+".accounts-per-sync", InitConfigDefault.AccountsPerSync, "during init - sync database every X accounts. Lower value for low-memory systems. 0 disables.") + f.String(prefix+".prune", InitConfigDefault.Prune, "pruning for a given use: \"full\" for full nodes serving RPC requests, or \"validator\" for validators") + f.Uint64(prefix+".prune-bloom-size", InitConfigDefault.PruneBloomSize, "the amount of memory in megabytes to use for the pruning bloom filter (higher values prune better)") + f.Int64(prefix+".reset-to-message", InitConfigDefault.ResetToMessage, "forces a reset to an old message height. Also set max-reorg-resequence-depth=0 to force re-reading messages") +} diff --git a/cmd/nitro/init.go b/cmd/nitro/init.go index 1427ef161e..745e168375 100644 --- a/cmd/nitro/init.go +++ b/cmd/nitro/init.go @@ -10,93 +10,37 @@ import ( "fmt" "math/big" "os" - "reflect" - "regexp" "runtime" "strings" "sync" "time" + "github.com/offchainlabs/nitro/cmd/conf" + "github.com/offchainlabs/nitro/cmd/prunning" "github.com/offchainlabs/nitro/cmd/util" "github.com/cavaliergopher/grab/v3" extract "github.com/codeclysm/extract/v3" - "github.com/ethereum/go-ethereum/accounts/abi/bind" "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/pruner" "github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/ethdb" "github.com/ethereum/go-ethereum/log" "github.com/ethereum/go-ethereum/node" "github.com/ethereum/go-ethereum/params" - "github.com/ethereum/go-ethereum/rpc" "github.com/offchainlabs/nitro/arbnode" - "github.com/offchainlabs/nitro/arbnode/dataposter/storage" "github.com/offchainlabs/nitro/arbos/arbosState" "github.com/offchainlabs/nitro/arbos/arbostypes" "github.com/offchainlabs/nitro/arbutil" "github.com/offchainlabs/nitro/cmd/chaininfo" "github.com/offchainlabs/nitro/cmd/ipfshelper" "github.com/offchainlabs/nitro/execution/gethexec" - "github.com/offchainlabs/nitro/staker" "github.com/offchainlabs/nitro/statetransfer" - "github.com/spf13/pflag" ) -type InitConfig struct { - Force bool `koanf:"force"` - Url string `koanf:"url"` - DownloadPath string `koanf:"download-path"` - DownloadPoll time.Duration `koanf:"download-poll"` - DevInit bool `koanf:"dev-init"` - DevInitAddress string `koanf:"dev-init-address"` - DevInitBlockNum uint64 `koanf:"dev-init-blocknum"` - Empty bool `koanf:"empty"` - AccountsPerSync uint `koanf:"accounts-per-sync"` - ImportFile string `koanf:"import-file"` - ThenQuit bool `koanf:"then-quit"` - Prune string `koanf:"prune"` - PruneBloomSize uint64 `koanf:"prune-bloom-size"` - ResetToMessage int64 `koanf:"reset-to-message"` -} - -var InitConfigDefault = InitConfig{ - Force: false, - Url: "", - DownloadPath: "/tmp/", - DownloadPoll: time.Minute, - DevInit: false, - DevInitAddress: "", - DevInitBlockNum: 0, - ImportFile: "", - AccountsPerSync: 100000, - ThenQuit: false, - Prune: "", - PruneBloomSize: 2048, - ResetToMessage: -1, -} - -func InitConfigAddOptions(prefix string, f *pflag.FlagSet) { - f.Bool(prefix+".force", InitConfigDefault.Force, "if true: in case database exists init code will be reexecuted and genesis block compared to database") - f.String(prefix+".url", InitConfigDefault.Url, "url to download initializtion data - will poll if download fails") - f.String(prefix+".download-path", InitConfigDefault.DownloadPath, "path to save temp downloaded file") - f.Duration(prefix+".download-poll", InitConfigDefault.DownloadPoll, "how long to wait between polling attempts") - f.Bool(prefix+".dev-init", InitConfigDefault.DevInit, "init with dev data (1 account with balance) instead of file import") - f.String(prefix+".dev-init-address", InitConfigDefault.DevInitAddress, "Address of dev-account. Leave empty to use the dev-wallet.") - f.Uint64(prefix+".dev-init-blocknum", InitConfigDefault.DevInitBlockNum, "Number of preinit blocks. Must exist in ancient database.") - f.Bool(prefix+".empty", InitConfigDefault.Empty, "init with empty state") - f.Bool(prefix+".then-quit", InitConfigDefault.ThenQuit, "quit after init is done") - f.String(prefix+".import-file", InitConfigDefault.ImportFile, "path for json data to import") - f.Uint(prefix+".accounts-per-sync", InitConfigDefault.AccountsPerSync, "during init - sync database every X accounts. Lower value for low-memory systems. 0 disables.") - f.String(prefix+".prune", InitConfigDefault.Prune, "pruning for a given use: \"full\" for full nodes serving RPC requests, or \"validator\" for validators") - f.Uint64(prefix+".prune-bloom-size", InitConfigDefault.PruneBloomSize, "the amount of memory in megabytes to use for the pruning bloom filter (higher values prune better)") - f.Int64(prefix+".reset-to-message", InitConfigDefault.ResetToMessage, "forces a reset to an old message height. Also set max-reorg-resequence-depth=0 to force re-reading messages") -} - -func downloadInit(ctx context.Context, initConfig *InitConfig) (string, error) { +func downloadInit(ctx context.Context, initConfig *conf.InitConfig) (string, error) { if initConfig.Url == "" { return "", nil } @@ -215,228 +159,6 @@ func validateBlockChain(blockChain *core.BlockChain, chainConfig *params.ChainCo return nil } -type importantRoots struct { - chainDb ethdb.Database - roots []common.Hash - heights []uint64 -} - -// The minimum block distance between two important roots -const minRootDistance = 2000 - -// Marks a header as important, and records its root and height. -// If overwrite is true, it'll remove any future roots and replace them with this header. -// If overwrite is false, it'll ignore this header if it has future roots. -func (r *importantRoots) addHeader(header *types.Header, overwrite bool) error { - targetBlockNum := header.Number.Uint64() - for { - if header == nil || header.Root == (common.Hash{}) { - log.Error("missing state of pruning target", "blockNum", targetBlockNum) - return nil - } - exists, err := r.chainDb.Has(header.Root.Bytes()) - if err != nil { - return err - } - if exists { - break - } - num := header.Number.Uint64() - if num%3000 == 0 { - log.Info("looking for old block with state to keep", "current", num, "target", targetBlockNum) - } - // An underflow is fine here because it'll just return nil due to not found - header = rawdb.ReadHeader(r.chainDb, header.ParentHash, num-1) - } - height := header.Number.Uint64() - for len(r.heights) > 0 && r.heights[len(r.heights)-1] > height { - if !overwrite { - return nil - } - r.roots = r.roots[:len(r.roots)-1] - r.heights = r.heights[:len(r.heights)-1] - } - if len(r.heights) > 0 && r.heights[len(r.heights)-1]+minRootDistance > height { - return nil - } - r.roots = append(r.roots, header.Root) - r.heights = append(r.heights, height) - return nil -} - -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, nodeConfig *NodeConfig, cacheConfig *core.CacheConfig, l1Client arbutil.L1Interface, rollupAddrs chaininfo.RollupAddresses) ([]common.Hash, error) { - initConfig := &nodeConfig.Init - 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, "", true) - if err != nil { - return nil, err - } - defer func() { - err := arbDb.Close() - if err != nil { - log.Warn("failed to close arbitrum database after finding pruning targets", "err", err) - } - }() - roots := importantRoots{ - chainDb: chainDb, - } - genesisNum := chainConfig.ArbitrumChainParams.GenesisBlockNum - genesisHash := rawdb.ReadCanonicalHash(chainDb, genesisNum) - genesisHeader := rawdb.ReadHeader(chainDb, genesisHash, genesisNum) - if genesisHeader == nil { - return nil, errors.New("missing L2 genesis block header") - } - err = roots.addHeader(genesisHeader, false) - if err != nil { - return nil, err - } - if initConfig.Prune == "validator" { - if l1Client == nil || reflect.ValueOf(l1Client).IsNil() { - return nil, errors.New("an L1 connection is required for validator pruning") - } - callOpts := bind.CallOpts{ - Context: ctx, - BlockNumber: big.NewInt(int64(rpc.FinalizedBlockNumber)), - } - rollup, err := staker.NewRollupWatcher(rollupAddrs.Rollup, l1Client, callOpts) - if err != nil { - return nil, err - } - latestConfirmedNum, err := rollup.LatestConfirmed(&callOpts) - if err != nil { - return nil, err - } - latestConfirmedNode, err := rollup.LookupNode(ctx, latestConfirmedNum) - if err != nil { - return nil, err - } - confirmedHash := latestConfirmedNode.Assertion.AfterState.GlobalState.BlockHash - confirmedNumber := rawdb.ReadHeaderNumber(chainDb, confirmedHash) - var confirmedHeader *types.Header - if confirmedNumber != nil { - confirmedHeader = rawdb.ReadHeader(chainDb, confirmedHash, *confirmedNumber) - } - if confirmedHeader != nil { - err = roots.addHeader(confirmedHeader, false) - if err != nil { - return nil, err - } - } else { - log.Warn("missing latest confirmed block", "hash", confirmedHash) - } - - validatorDb := rawdb.NewTable(arbDb, storage.BlockValidatorPrefix) - lastValidated, err := staker.ReadLastValidatedInfo(validatorDb) - if err != nil { - return nil, err - } - if lastValidated != nil { - var lastValidatedHeader *types.Header - headerNum := rawdb.ReadHeaderNumber(chainDb, lastValidated.GlobalState.BlockHash) - if headerNum != nil { - lastValidatedHeader = rawdb.ReadHeader(chainDb, lastValidated.GlobalState.BlockHash, *headerNum) - } - if lastValidatedHeader != nil { - err = roots.addHeader(lastValidatedHeader, false) - if err != nil { - return nil, err - } - } else { - log.Warn("missing latest validated block", "hash", lastValidated.GlobalState.BlockHash) - } - } - } else if initConfig.Prune == "full" { - if nodeConfig.Node.ValidatorRequired() { - return nil, errors.New("refusing to prune to full-node level when validator is enabled (you should prune in validator mode)") - } - } else if hashListRegex.MatchString(initConfig.Prune) { - parts := strings.Split(initConfig.Prune, ",") - roots := []common.Hash{genesisHeader.Root} - for _, part := range parts { - root := common.HexToHash(part) - if root == genesisHeader.Root { - // This was already included in the builtin list - continue - } - roots = append(roots, root) - } - return roots, nil - } else { - return nil, fmt.Errorf("unknown pruning mode: \"%v\"", initConfig.Prune) - } - if l1Client != nil { - // Find the latest finalized block and add it as a pruning target - l1Block, err := l1Client.BlockByNumber(ctx, big.NewInt(int64(rpc.FinalizedBlockNumber))) - if err != nil { - return nil, fmt.Errorf("failed to get finalized block: %w", err) - } - l1BlockNum := l1Block.NumberU64() - tracker, err := arbnode.NewInboxTracker(arbDb, nil, nil) - if err != nil { - return nil, err - } - batch, err := tracker.GetBatchCount() - if err != nil { - return nil, err - } - for { - if ctx.Err() != nil { - return nil, ctx.Err() - } - if batch == 0 { - // No batch has been finalized - break - } - batch -= 1 - meta, err := tracker.GetBatchMetadata(batch) - if err != nil { - return nil, err - } - if meta.ParentChainBlock <= l1BlockNum { - signedBlockNum := arbutil.MessageCountToBlockNumber(meta.MessageCount, genesisNum) - blockNum := uint64(signedBlockNum) - l2Hash := rawdb.ReadCanonicalHash(chainDb, blockNum) - l2Header := rawdb.ReadHeader(chainDb, l2Hash, blockNum) - if l2Header == nil { - log.Warn("latest finalized L2 block is unknown", "blockNum", signedBlockNum) - break - } - err = roots.addHeader(l2Header, false) - if err != nil { - return nil, err - } - break - } - } - } - roots.roots = append(roots.roots, common.Hash{}) // the latest snapshot - log.Info("found pruning target blocks", "heights", roots.heights, "roots", roots.roots) - return roots.roots, nil -} - -func pruneChainDb(ctx context.Context, chainDb ethdb.Database, stack *node.Node, nodeConfig *NodeConfig, cacheConfig *core.CacheConfig, l1Client arbutil.L1Interface, rollupAddrs chaininfo.RollupAddresses) error { - config := &nodeConfig.Init - if config.Prune == "" { - return pruner.RecoverPruning(stack.InstanceDir(), chainDb) - } - root, err := findImportantRoots(ctx, chainDb, stack, nodeConfig, cacheConfig, l1Client, rollupAddrs) - if err != nil { - return fmt.Errorf("failed to find root to retain for pruning: %w", err) - } - - pruner, err := pruner.NewPruner(chainDb, pruner.Config{Datadir: stack.InstanceDir(), BloomSize: config.PruneBloomSize}) - if err != nil { - return err - } - return pruner.Prune(root) -} - 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, "", "", true); err == nil { @@ -446,7 +168,7 @@ func openInitializeChainDb(ctx context.Context, stack *node.Node, config *NodeCo if err != nil { return chainDb, nil, err } - err = pruneChainDb(ctx, chainDb, stack, config, cacheConfig, l1Client, rollupAddrs) + err = prunning.PruneChainDb(ctx, chainDb, stack, &config.Init, cacheConfig, l1Client, rollupAddrs, config.Node.ValidatorRequired()) if err != nil { return chainDb, nil, fmt.Errorf("error pruning: %w", err) } @@ -642,7 +364,7 @@ func openInitializeChainDb(ctx context.Context, stack *node.Node, config *NodeCo return chainDb, l2BlockChain, err } - err = pruneChainDb(ctx, chainDb, stack, config, cacheConfig, l1Client, rollupAddrs) + err = prunning.PruneChainDb(ctx, chainDb, stack, &config.Init, cacheConfig, 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 86093a85d9..ca0d3496f0 100644 --- a/cmd/nitro/nitro.go +++ b/cmd/nitro/nitro.go @@ -677,7 +677,7 @@ type NodeConfig struct { MetricsServer genericconf.MetricsServerConfig `koanf:"metrics-server"` PProf bool `koanf:"pprof"` PprofCfg genericconf.PProf `koanf:"pprof-cfg"` - Init InitConfig `koanf:"init"` + Init conf.InitConfig `koanf:"init"` Rpc genericconf.RpcConfig `koanf:"rpc"` } @@ -699,7 +699,7 @@ var NodeConfigDefault = NodeConfig{ GraphQL: genericconf.GraphQLConfigDefault, Metrics: false, MetricsServer: genericconf.MetricsServerConfigDefault, - Init: InitConfigDefault, + Init: conf.InitConfigDefault, Rpc: genericconf.DefaultRpcConfig, PProf: false, PprofCfg: genericconf.PProfDefault, @@ -726,7 +726,7 @@ func NodeConfigAddOptions(f *flag.FlagSet) { f.Bool("pprof", NodeConfigDefault.PProf, "enable pprof") genericconf.PProfAddOptions("pprof-cfg", f) - InitConfigAddOptions("init", f) + conf.InitConfigAddOptions("init", f) genericconf.RpcConfigAddOptions("rpc", f) } diff --git a/cmd/prunning/prunning.go b/cmd/prunning/prunning.go new file mode 100644 index 0000000000..7538c68a10 --- /dev/null +++ b/cmd/prunning/prunning.go @@ -0,0 +1,250 @@ +package prunning + +import ( + "context" + "errors" + "fmt" + "math/big" + "reflect" + "regexp" + "strings" + + "github.com/ethereum/go-ethereum/accounts/abi/bind" + "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/pruner" + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/ethdb" + "github.com/ethereum/go-ethereum/log" + "github.com/ethereum/go-ethereum/node" + "github.com/ethereum/go-ethereum/rpc" + "github.com/offchainlabs/nitro/arbnode" + "github.com/offchainlabs/nitro/arbnode/dataposter/storage" + "github.com/offchainlabs/nitro/arbutil" + "github.com/offchainlabs/nitro/cmd/chaininfo" + "github.com/offchainlabs/nitro/cmd/conf" + "github.com/offchainlabs/nitro/execution/gethexec" + "github.com/offchainlabs/nitro/staker" +) + +type importantRoots struct { + chainDb ethdb.Database + roots []common.Hash + heights []uint64 +} + +// The minimum block distance between two important roots +const minRootDistance = 2000 + +// Marks a header as important, and records its root and height. +// If overwrite is true, it'll remove any future roots and replace them with this header. +// If overwrite is false, it'll ignore this header if it has future roots. +func (r *importantRoots) addHeader(header *types.Header, overwrite bool) error { + targetBlockNum := header.Number.Uint64() + for { + if header == nil || header.Root == (common.Hash{}) { + log.Error("missing state of pruning target", "blockNum", targetBlockNum) + return nil + } + exists, err := r.chainDb.Has(header.Root.Bytes()) + if err != nil { + return err + } + if exists { + break + } + num := header.Number.Uint64() + if num%3000 == 0 { + log.Info("looking for old block with state to keep", "current", num, "target", targetBlockNum) + } + // An underflow is fine here because it'll just return nil due to not found + header = rawdb.ReadHeader(r.chainDb, header.ParentHash, num-1) + } + height := header.Number.Uint64() + for len(r.heights) > 0 && r.heights[len(r.heights)-1] > height { + if !overwrite { + return nil + } + r.roots = r.roots[:len(r.roots)-1] + r.heights = r.heights[:len(r.heights)-1] + } + if len(r.heights) > 0 && r.heights[len(r.heights)-1]+minRootDistance > height { + return nil + } + r.roots = append(r.roots, header.Root) + r.heights = append(r.heights, height) + return nil +} + +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) { + 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, "", true) + if err != nil { + return nil, err + } + defer func() { + err := arbDb.Close() + if err != nil { + log.Warn("failed to close arbitrum database after finding pruning targets", "err", err) + } + }() + roots := importantRoots{ + chainDb: chainDb, + } + genesisNum := chainConfig.ArbitrumChainParams.GenesisBlockNum + genesisHash := rawdb.ReadCanonicalHash(chainDb, genesisNum) + genesisHeader := rawdb.ReadHeader(chainDb, genesisHash, genesisNum) + log.Warn("XXX", "genesisNum", genesisNum, "genesisHash", genesisHash) + if genesisHeader == nil { + return nil, errors.New("missing L2 genesis block header") + } + err = roots.addHeader(genesisHeader, false) + if err != nil { + return nil, err + } + if initConfig.Prune == "validator" { + if l1Client == nil || reflect.ValueOf(l1Client).IsNil() { + return nil, errors.New("an L1 connection is required for validator pruning") + } + callOpts := bind.CallOpts{ + Context: ctx, + BlockNumber: big.NewInt(int64(rpc.FinalizedBlockNumber)), + } + rollup, err := staker.NewRollupWatcher(rollupAddrs.Rollup, l1Client, callOpts) + if err != nil { + return nil, err + } + latestConfirmedNum, err := rollup.LatestConfirmed(&callOpts) + if err != nil { + return nil, err + } + latestConfirmedNode, err := rollup.LookupNode(ctx, latestConfirmedNum) + if err != nil { + return nil, err + } + confirmedHash := latestConfirmedNode.Assertion.AfterState.GlobalState.BlockHash + confirmedNumber := rawdb.ReadHeaderNumber(chainDb, confirmedHash) + var confirmedHeader *types.Header + if confirmedNumber != nil { + confirmedHeader = rawdb.ReadHeader(chainDb, confirmedHash, *confirmedNumber) + } + if confirmedHeader != nil { + err = roots.addHeader(confirmedHeader, false) + if err != nil { + return nil, err + } + } else { + log.Warn("missing latest confirmed block", "hash", confirmedHash) + } + + validatorDb := rawdb.NewTable(arbDb, storage.BlockValidatorPrefix) + lastValidated, err := staker.ReadLastValidatedInfo(validatorDb) + if err != nil { + return nil, err + } + if lastValidated != nil { + var lastValidatedHeader *types.Header + headerNum := rawdb.ReadHeaderNumber(chainDb, lastValidated.GlobalState.BlockHash) + if headerNum != nil { + lastValidatedHeader = rawdb.ReadHeader(chainDb, lastValidated.GlobalState.BlockHash, *headerNum) + } + if lastValidatedHeader != nil { + err = roots.addHeader(lastValidatedHeader, false) + if err != nil { + return nil, err + } + } else { + log.Warn("missing latest validated block", "hash", lastValidated.GlobalState.BlockHash) + } + } + } else if initConfig.Prune == "full" { + if validatorRequired { + return nil, errors.New("refusing to prune to full-node level when validator is enabled (you should prune in validator mode)") + } + } else if hashListRegex.MatchString(initConfig.Prune) { + parts := strings.Split(initConfig.Prune, ",") + roots := []common.Hash{genesisHeader.Root} + for _, part := range parts { + root := common.HexToHash(part) + if root == genesisHeader.Root { + // This was already included in the builtin list + continue + } + roots = append(roots, root) + } + return roots, nil + } else { + return nil, fmt.Errorf("unknown pruning mode: \"%v\"", initConfig.Prune) + } + if l1Client != nil { + // Find the latest finalized block and add it as a pruning target + l1Block, err := l1Client.BlockByNumber(ctx, big.NewInt(int64(rpc.FinalizedBlockNumber))) + if err != nil { + return nil, fmt.Errorf("failed to get finalized block: %w", err) + } + l1BlockNum := l1Block.NumberU64() + tracker, err := arbnode.NewInboxTracker(arbDb, nil, nil) + if err != nil { + return nil, err + } + batch, err := tracker.GetBatchCount() + if err != nil { + return nil, err + } + for { + if ctx.Err() != nil { + return nil, ctx.Err() + } + if batch == 0 { + // No batch has been finalized + break + } + batch -= 1 + meta, err := tracker.GetBatchMetadata(batch) + if err != nil { + return nil, err + } + if meta.ParentChainBlock <= l1BlockNum { + signedBlockNum := arbutil.MessageCountToBlockNumber(meta.MessageCount, genesisNum) + blockNum := uint64(signedBlockNum) + l2Hash := rawdb.ReadCanonicalHash(chainDb, blockNum) + l2Header := rawdb.ReadHeader(chainDb, l2Hash, blockNum) + if l2Header == nil { + log.Warn("latest finalized L2 block is unknown", "blockNum", signedBlockNum) + break + } + err = roots.addHeader(l2Header, false) + if err != nil { + return nil, err + } + break + } + } + } + roots.roots = append(roots.roots, common.Hash{}) // the latest snapshot + log.Info("found pruning target blocks", "heights", roots.heights, "roots", roots.roots) + 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 { + if initConfig.Prune == "" { + return pruner.RecoverPruning(stack.InstanceDir(), chainDb) + } + root, err := findImportantRoots(ctx, chainDb, stack, initConfig, cacheConfig, l1Client, rollupAddrs, validatorRequired) + if err != nil { + return fmt.Errorf("failed to find root to retain for pruning: %w", err) + } + + pruner, err := pruner.NewPruner(chainDb, pruner.Config{Datadir: stack.InstanceDir(), BloomSize: initConfig.PruneBloomSize}) + if err != nil { + return err + } + return pruner.Prune(root) +} diff --git a/go-ethereum b/go-ethereum index b1622e6ac4..1e2855b24d 160000 --- a/go-ethereum +++ b/go-ethereum @@ -1 +1 @@ -Subproject commit b1622e6ac4bf3762aebde92a585de2889d90823f +Subproject commit 1e2855b24d6555c8cfaf471bd9e2c3d19ab5c32c diff --git a/system_tests/common_test.go b/system_tests/common_test.go index 7752fbd34e..937b8980fc 100644 --- a/system_tests/common_test.go +++ b/system_tests/common_test.go @@ -698,7 +698,7 @@ func createL2BlockChainWithStackConfig( chainDb, err := stack.OpenDatabase("chaindb", 0, 0, "", false) Require(t, err) - arbDb, err := stack.OpenDatabase("arbdb", 0, 0, "", false) + arbDb, err := stack.OpenDatabase("arbitrumdata", 0, 0, "", false) Require(t, err) initReader := statetransfer.NewMemoryInitDataReader(&l2info.ArbInitData) @@ -903,7 +903,7 @@ func Create2ndNodeWithConfig( l2chainDb, err := l2stack.OpenDatabase("chaindb", 0, 0, "", false) Require(t, err) - l2arbDb, err := l2stack.OpenDatabase("arbdb", 0, 0, "", false) + l2arbDb, err := l2stack.OpenDatabase("arbitrumdata", 0, 0, "", false) Require(t, err) initReader := statetransfer.NewMemoryInitDataReader(l2InitData) diff --git a/system_tests/prunning_test.go b/system_tests/prunning_test.go new file mode 100644 index 0000000000..aadf3c5e9e --- /dev/null +++ b/system_tests/prunning_test.go @@ -0,0 +1,121 @@ +package arbtest + +import ( + "context" + "testing" + + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/ethdb" + "github.com/ethereum/go-ethereum/node" + "github.com/offchainlabs/nitro/cmd/conf" + "github.com/offchainlabs/nitro/cmd/prunning" + "github.com/offchainlabs/nitro/execution/gethexec" + "github.com/offchainlabs/nitro/util/testhelpers" +) + +func countDbEntries(db ethdb.Iteratee) int { + entries := 0 + it := db.NewIterator(nil, nil) + for it.Next() { + entries++ + } + it.Release() + return entries +} + +func TestPrunning(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + var dataDir string + + func() { + builder := NewNodeBuilder(ctx).DefaultConfig(t, true) + _ = builder.Build(t) + dataDir = builder.dataDir + l2cleanupDone := false + defer func() { + if !l2cleanupDone { + builder.L2.cleanup() + } + builder.L1.cleanup() + }() + builder.L2Info.GenerateAccount("User2") + var txs []*types.Transaction + for i := uint64(0); i < 1000; i++ { + tx := builder.L2Info.PrepareTx("Owner", "User2", builder.L2Info.TransferGas, common.Big1, nil) + txs = append(txs, tx) + err := builder.L2.Client.SendTransaction(ctx, tx) + Require(t, err) + } + for _, tx := range txs { + _, err := builder.L2.EnsureTxSucceeded(tx) + Require(t, err) + } + l2cleanupDone = true + builder.L2.cleanup() + t.Log("stopped l2 node") + + stack, err := node.New(builder.l2StackConfig) + Require(t, err) + defer stack.Close() + chainDb, err := stack.OpenDatabase("chaindb", 0, 0, "", false) + Require(t, err) + defer chainDb.Close() + entriesBeforePrunning := countDbEntries(chainDb) + + prand := testhelpers.NewPseudoRandomDataSource(t, 1) + var testKeys [][]byte + for i := 0; i < 100; i++ { + // generate test keys with length of hash to emulate legacy state trie nodes + testKeys = append(testKeys, prand.GetHash().Bytes()) + } + for _, key := range testKeys { + err = chainDb.Put(key, common.FromHex("0xdeadbeef")) + Require(t, err) + } + for _, key := range testKeys { + if has, _ := chainDb.Has(key); !has { + Fatal(t, "internal test error - failed to check existence of test key") + } + } + + initConfig := conf.InitConfigDefault + initConfig.Prune = "full" + coreCacheConfig := gethexec.DefaultCacheConfigFor(stack, &builder.execConfig.Caching) + err = prunning.PruneChainDb(ctx, chainDb, stack, &initConfig, coreCacheConfig, builder.L1.Client, *builder.L2.ConsensusNode.DeployInfo, false) + Require(t, err) + + for _, key := range testKeys { + if has, _ := chainDb.Has(key); has { + Fatal(t, "test key hasn't been prunned as expected") + } + } + + entriesAfterPrunning := countDbEntries(chainDb) + t.Log("db entries pre-prunning:", entriesBeforePrunning) + t.Log("db entries post-prunning:", entriesAfterPrunning) + + if entriesAfterPrunning >= entriesBeforePrunning { + Fatal(t, "The db doesn't have less entires after prunning then before. Before:", entriesBeforePrunning, "After:", entriesAfterPrunning) + } + }() + builder := NewNodeBuilder(ctx).DefaultConfig(t, true) + builder.dataDir = dataDir + cancel = builder.Build(t) + defer cancel() + + builder.L2Info.GenerateAccount("User2") + var txs []*types.Transaction + for i := uint64(0); i < 10; i++ { + tx := builder.L2Info.PrepareTx("Owner", "User2", builder.L2Info.TransferGas, common.Big1, nil) + txs = append(txs, tx) + err := builder.L2.Client.SendTransaction(ctx, tx) + Require(t, err) + } + for _, tx := range txs { + _, err := builder.L2.EnsureTxSucceeded(tx) + Require(t, err) + } +} From 875c558d9b0350d0d1923a1a7028a95c6107940d Mon Sep 17 00:00:00 2001 From: Maciej Kulawik Date: Fri, 15 Dec 2023 15:29:08 +0000 Subject: [PATCH 2/9] add missing default value of InitConfig.Empty --- cmd/conf/init.go | 1 + 1 file changed, 1 insertion(+) diff --git a/cmd/conf/init.go b/cmd/conf/init.go index f228891b90..bebf1955b7 100644 --- a/cmd/conf/init.go +++ b/cmd/conf/init.go @@ -31,6 +31,7 @@ var InitConfigDefault = InitConfig{ DevInit: false, DevInitAddress: "", DevInitBlockNum: 0, + Empty: false, ImportFile: "", AccountsPerSync: 100000, ThenQuit: false, From ae5c2f28227b7e9ea9353a8ae9c581232bca5aa7 Mon Sep 17 00:00:00 2001 From: Maciej Kulawik Date: Fri, 15 Dec 2023 15:43:18 +0000 Subject: [PATCH 3/9] cleanup imports --- cmd/nitro/init.go | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/cmd/nitro/init.go b/cmd/nitro/init.go index 745e168375..00012ac62d 100644 --- a/cmd/nitro/init.go +++ b/cmd/nitro/init.go @@ -15,10 +15,6 @@ import ( "sync" "time" - "github.com/offchainlabs/nitro/cmd/conf" - "github.com/offchainlabs/nitro/cmd/prunning" - "github.com/offchainlabs/nitro/cmd/util" - "github.com/cavaliergopher/grab/v3" extract "github.com/codeclysm/extract/v3" "github.com/ethereum/go-ethereum/common" @@ -35,7 +31,10 @@ import ( "github.com/offchainlabs/nitro/arbos/arbostypes" "github.com/offchainlabs/nitro/arbutil" "github.com/offchainlabs/nitro/cmd/chaininfo" + "github.com/offchainlabs/nitro/cmd/conf" "github.com/offchainlabs/nitro/cmd/ipfshelper" + "github.com/offchainlabs/nitro/cmd/prunning" + "github.com/offchainlabs/nitro/cmd/util" "github.com/offchainlabs/nitro/execution/gethexec" "github.com/offchainlabs/nitro/statetransfer" ) From 6ca851673ed9b86bdca4c60097712f041509f6c5 Mon Sep 17 00:00:00 2001 From: Maciej Kulawik Date: Tue, 19 Dec 2023 14:33:30 +0000 Subject: [PATCH 4/9] fix arbitrum data db name in das test --- system_tests/das_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/system_tests/das_test.go b/system_tests/das_test.go index 8c9621d57a..6db339521c 100644 --- a/system_tests/das_test.go +++ b/system_tests/das_test.go @@ -179,7 +179,7 @@ func TestDASRekey(t *testing.T) { l2chainDb, err := l2stackA.OpenDatabase("chaindb", 0, 0, "", false) Require(t, err) - l2arbDb, err := l2stackA.OpenDatabase("arbdb", 0, 0, "", false) + l2arbDb, err := l2stackA.OpenDatabase("arbitrumdata", 0, 0, "", false) Require(t, err) l2blockchain, err := gethexec.GetBlockChain(l2chainDb, nil, chainConfig, gethexec.ConfigDefaultTest().TxLookupLimit) From 7a26e97d04ef7a26a9adcd6b53db0afc0bf384d6 Mon Sep 17 00:00:00 2001 From: Maciej Kulawik Date: Wed, 20 Dec 2023 15:03:39 +0000 Subject: [PATCH 5/9] clean up debug log --- cmd/prunning/prunning.go | 1 - 1 file changed, 1 deletion(-) diff --git a/cmd/prunning/prunning.go b/cmd/prunning/prunning.go index 7538c68a10..5efdf8e643 100644 --- a/cmd/prunning/prunning.go +++ b/cmd/prunning/prunning.go @@ -101,7 +101,6 @@ func findImportantRoots(ctx context.Context, chainDb ethdb.Database, stack *node genesisNum := chainConfig.ArbitrumChainParams.GenesisBlockNum genesisHash := rawdb.ReadCanonicalHash(chainDb, genesisNum) genesisHeader := rawdb.ReadHeader(chainDb, genesisHash, genesisNum) - log.Warn("XXX", "genesisNum", genesisNum, "genesisHash", genesisHash) if genesisHeader == nil { return nil, errors.New("missing L2 genesis block header") } From 712bc0ce2658f98b7476375148c72891dd659bb3 Mon Sep 17 00:00:00 2001 From: Maciej Kulawik Date: Wed, 20 Dec 2023 15:10:26 +0000 Subject: [PATCH 6/9] fix typo --- system_tests/prunning_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/system_tests/prunning_test.go b/system_tests/prunning_test.go index aadf3c5e9e..cfb1a4bcf7 100644 --- a/system_tests/prunning_test.go +++ b/system_tests/prunning_test.go @@ -98,7 +98,7 @@ func TestPrunning(t *testing.T) { t.Log("db entries post-prunning:", entriesAfterPrunning) if entriesAfterPrunning >= entriesBeforePrunning { - Fatal(t, "The db doesn't have less entires after prunning then before. Before:", entriesBeforePrunning, "After:", entriesAfterPrunning) + Fatal(t, "The db doesn't have less entries after prunning then before. Before:", entriesBeforePrunning, "After:", entriesAfterPrunning) } }() builder := NewNodeBuilder(ctx).DefaultConfig(t, true) From b005bd0c1cce855ac100e3bddb66739dc0e912c0 Mon Sep 17 00:00:00 2001 From: Maciej Kulawik Date: Wed, 20 Dec 2023 18:24:38 +0000 Subject: [PATCH 7/9] count only state entries in diskdb in prunning test --- system_tests/prunning_test.go | 22 +++++++++++++--------- 1 file changed, 13 insertions(+), 9 deletions(-) diff --git a/system_tests/prunning_test.go b/system_tests/prunning_test.go index cfb1a4bcf7..d522458209 100644 --- a/system_tests/prunning_test.go +++ b/system_tests/prunning_test.go @@ -5,6 +5,7 @@ import ( "testing" "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/core/rawdb" "github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/ethdb" "github.com/ethereum/go-ethereum/node" @@ -14,11 +15,14 @@ import ( "github.com/offchainlabs/nitro/util/testhelpers" ) -func countDbEntries(db ethdb.Iteratee) int { +func countStateEntries(db ethdb.Iteratee) int { entries := 0 it := db.NewIterator(nil, nil) for it.Next() { - entries++ + isCode, _ := rawdb.IsCodeKey(it.Key()) + if len(it.Key()) == common.HashLength || isCode { + entries++ + } } it.Release() return entries @@ -43,7 +47,7 @@ func TestPrunning(t *testing.T) { }() builder.L2Info.GenerateAccount("User2") var txs []*types.Transaction - for i := uint64(0); i < 1000; i++ { + for i := uint64(0); i < 200; i++ { tx := builder.L2Info.PrepareTx("Owner", "User2", builder.L2Info.TransferGas, common.Big1, nil) txs = append(txs, tx) err := builder.L2.Client.SendTransaction(ctx, tx) @@ -63,7 +67,7 @@ func TestPrunning(t *testing.T) { chainDb, err := stack.OpenDatabase("chaindb", 0, 0, "", false) Require(t, err) defer chainDb.Close() - entriesBeforePrunning := countDbEntries(chainDb) + chainDbEntriesBeforePrunning := countStateEntries(chainDb) prand := testhelpers.NewPseudoRandomDataSource(t, 1) var testKeys [][]byte @@ -93,12 +97,12 @@ func TestPrunning(t *testing.T) { } } - entriesAfterPrunning := countDbEntries(chainDb) - t.Log("db entries pre-prunning:", entriesBeforePrunning) - t.Log("db entries post-prunning:", entriesAfterPrunning) + chainDbEntriesAfterPrunning := countStateEntries(chainDb) + t.Log("db entries pre-prunning:", chainDbEntriesBeforePrunning) + t.Log("db entries post-prunning:", chainDbEntriesAfterPrunning) - if entriesAfterPrunning >= entriesBeforePrunning { - Fatal(t, "The db doesn't have less entries after prunning then before. Before:", entriesBeforePrunning, "After:", entriesAfterPrunning) + if chainDbEntriesAfterPrunning >= chainDbEntriesBeforePrunning { + Fatal(t, "The db doesn't have less entries after prunning then before. Before:", chainDbEntriesBeforePrunning, "After:", chainDbEntriesAfterPrunning) } }() builder := NewNodeBuilder(ctx).DefaultConfig(t, true) From 32883f183ceb4073bffec6d6394ddd87aa36281a Mon Sep 17 00:00:00 2001 From: Maciej Kulawik Date: Fri, 22 Dec 2023 20:27:51 +0000 Subject: [PATCH 8/9] fix typo --- cmd/nitro/init.go | 6 +++--- .../prunning.go => pruning/pruning.go} | 2 +- .../{prunning_test.go => pruning_test.go} | 20 +++++++++---------- 3 files changed, 14 insertions(+), 14 deletions(-) rename cmd/{prunning/prunning.go => pruning/pruning.go} (99%) rename system_tests/{prunning_test.go => pruning_test.go} (79%) diff --git a/cmd/nitro/init.go b/cmd/nitro/init.go index 00012ac62d..ada195b5c4 100644 --- a/cmd/nitro/init.go +++ b/cmd/nitro/init.go @@ -33,7 +33,7 @@ import ( "github.com/offchainlabs/nitro/cmd/chaininfo" "github.com/offchainlabs/nitro/cmd/conf" "github.com/offchainlabs/nitro/cmd/ipfshelper" - "github.com/offchainlabs/nitro/cmd/prunning" + "github.com/offchainlabs/nitro/cmd/pruning" "github.com/offchainlabs/nitro/cmd/util" "github.com/offchainlabs/nitro/execution/gethexec" "github.com/offchainlabs/nitro/statetransfer" @@ -167,7 +167,7 @@ func openInitializeChainDb(ctx context.Context, stack *node.Node, config *NodeCo if err != nil { return chainDb, nil, err } - err = prunning.PruneChainDb(ctx, chainDb, stack, &config.Init, cacheConfig, l1Client, rollupAddrs, config.Node.ValidatorRequired()) + 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) } @@ -363,7 +363,7 @@ func openInitializeChainDb(ctx context.Context, stack *node.Node, config *NodeCo return chainDb, l2BlockChain, err } - err = prunning.PruneChainDb(ctx, chainDb, stack, &config.Init, cacheConfig, l1Client, rollupAddrs, config.Node.ValidatorRequired()) + 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) } diff --git a/cmd/prunning/prunning.go b/cmd/pruning/pruning.go similarity index 99% rename from cmd/prunning/prunning.go rename to cmd/pruning/pruning.go index 5efdf8e643..68d89302f0 100644 --- a/cmd/prunning/prunning.go +++ b/cmd/pruning/pruning.go @@ -1,4 +1,4 @@ -package prunning +package pruning import ( "context" diff --git a/system_tests/prunning_test.go b/system_tests/pruning_test.go similarity index 79% rename from system_tests/prunning_test.go rename to system_tests/pruning_test.go index d522458209..e6eefdabd5 100644 --- a/system_tests/prunning_test.go +++ b/system_tests/pruning_test.go @@ -10,7 +10,7 @@ import ( "github.com/ethereum/go-ethereum/ethdb" "github.com/ethereum/go-ethereum/node" "github.com/offchainlabs/nitro/cmd/conf" - "github.com/offchainlabs/nitro/cmd/prunning" + "github.com/offchainlabs/nitro/cmd/pruning" "github.com/offchainlabs/nitro/execution/gethexec" "github.com/offchainlabs/nitro/util/testhelpers" ) @@ -28,7 +28,7 @@ func countStateEntries(db ethdb.Iteratee) int { return entries } -func TestPrunning(t *testing.T) { +func TestPruning(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -67,7 +67,7 @@ func TestPrunning(t *testing.T) { chainDb, err := stack.OpenDatabase("chaindb", 0, 0, "", false) Require(t, err) defer chainDb.Close() - chainDbEntriesBeforePrunning := countStateEntries(chainDb) + chainDbEntriesBeforePruning := countStateEntries(chainDb) prand := testhelpers.NewPseudoRandomDataSource(t, 1) var testKeys [][]byte @@ -88,21 +88,21 @@ func TestPrunning(t *testing.T) { initConfig := conf.InitConfigDefault initConfig.Prune = "full" coreCacheConfig := gethexec.DefaultCacheConfigFor(stack, &builder.execConfig.Caching) - err = prunning.PruneChainDb(ctx, chainDb, stack, &initConfig, coreCacheConfig, builder.L1.Client, *builder.L2.ConsensusNode.DeployInfo, false) + err = pruning.PruneChainDb(ctx, chainDb, stack, &initConfig, coreCacheConfig, builder.L1.Client, *builder.L2.ConsensusNode.DeployInfo, false) Require(t, err) for _, key := range testKeys { if has, _ := chainDb.Has(key); has { - Fatal(t, "test key hasn't been prunned as expected") + Fatal(t, "test key hasn't been pruned as expected") } } - chainDbEntriesAfterPrunning := countStateEntries(chainDb) - t.Log("db entries pre-prunning:", chainDbEntriesBeforePrunning) - t.Log("db entries post-prunning:", chainDbEntriesAfterPrunning) + chainDbEntriesAfterPruning := countStateEntries(chainDb) + t.Log("db entries pre-pruning:", chainDbEntriesBeforePruning) + t.Log("db entries post-pruning:", chainDbEntriesAfterPruning) - if chainDbEntriesAfterPrunning >= chainDbEntriesBeforePrunning { - Fatal(t, "The db doesn't have less entries after prunning then before. Before:", chainDbEntriesBeforePrunning, "After:", chainDbEntriesAfterPrunning) + if chainDbEntriesAfterPruning >= chainDbEntriesBeforePruning { + Fatal(t, "The db doesn't have less entries after pruning then before. Before:", chainDbEntriesBeforePruning, "After:", chainDbEntriesAfterPruning) } }() builder := NewNodeBuilder(ctx).DefaultConfig(t, true) From cb4d3c5374be0d9ed155fe6ed2514b27747f7366 Mon Sep 17 00:00:00 2001 From: Maciej Kulawik Date: Fri, 22 Dec 2023 20:30:11 +0000 Subject: [PATCH 9/9] make only one test tx after pruning --- system_tests/pruning_test.go | 16 +++++----------- 1 file changed, 5 insertions(+), 11 deletions(-) diff --git a/system_tests/pruning_test.go b/system_tests/pruning_test.go index e6eefdabd5..ef82c0466e 100644 --- a/system_tests/pruning_test.go +++ b/system_tests/pruning_test.go @@ -111,15 +111,9 @@ func TestPruning(t *testing.T) { defer cancel() builder.L2Info.GenerateAccount("User2") - var txs []*types.Transaction - for i := uint64(0); i < 10; i++ { - tx := builder.L2Info.PrepareTx("Owner", "User2", builder.L2Info.TransferGas, common.Big1, nil) - txs = append(txs, tx) - err := builder.L2.Client.SendTransaction(ctx, tx) - Require(t, err) - } - for _, tx := range txs { - _, err := builder.L2.EnsureTxSucceeded(tx) - Require(t, err) - } + tx := builder.L2Info.PrepareTx("Owner", "User2", builder.L2Info.TransferGas, common.Big1, nil) + err := builder.L2.Client.SendTransaction(ctx, tx) + Require(t, err) + _, err = builder.L2.EnsureTxSucceeded(tx) + Require(t, err) }