From 1406a96ac357b8bfb457dfe5671cd5eb22980b3d Mon Sep 17 00:00:00 2001 From: pthmas <9058370+pthmas@users.noreply.github.com> Date: Fri, 16 Jan 2026 10:32:55 +0100 Subject: [PATCH 1/6] basic prunning --- block/internal/executing/executor.go | 17 ++++++ pkg/config/config.go | 11 ++++ pkg/config/defaults.go | 3 ++ pkg/store/keys.go | 6 +++ pkg/store/store.go | 79 ++++++++++++++++++++++++++++ pkg/store/store_test.go | 51 ++++++++++++++++++ pkg/store/types.go | 19 ++++++- 7 files changed, 185 insertions(+), 1 deletion(-) diff --git a/block/internal/executing/executor.go b/block/internal/executing/executor.go index a693e8d5e5..538ceab325 100644 --- a/block/internal/executing/executor.go +++ b/block/internal/executing/executor.go @@ -444,6 +444,23 @@ func (e *Executor) ProduceBlock(ctx context.Context) error { // Update in-memory state after successful commit e.setLastState(newState) + // Run height-based pruning of stored block data if enabled. This is a + // best-effort background maintenance step and should not cause block + // production to fail, but it does run in the critical path and may add + // some latency when large ranges are pruned. + if e.config.Node.PruningEnabled && e.config.Node.PruningKeepRecent > 0 && e.config.Node.PruningInterval > 0 { + if newHeight%e.config.Node.PruningInterval == 0 { + // Compute the prune floor: all heights <= targetHeight are candidates + // for pruning of header/data/signature/index entries. + if newHeight > e.config.Node.PruningKeepRecent { + targetHeight := newHeight - e.config.Node.PruningKeepRecent + if err := e.store.PruneBlocks(e.ctx, targetHeight); err != nil { + e.logger.Error().Err(err).Uint64("target_height", targetHeight).Msg("failed to prune old block data") + } + } + } + } + // broadcast header and data to P2P network g, broadcastCtx := errgroup.WithContext(ctx) g.Go(func() error { return e.headerBroadcaster.WriteToStoreAndBroadcast(broadcastCtx, header) }) diff --git a/pkg/config/config.go b/pkg/config/config.go index 6f8193f1eb..6a3583e9ba 100644 --- a/pkg/config/config.go +++ b/pkg/config/config.go @@ -235,6 +235,13 @@ type NodeConfig struct { // Readiness / health configuration ReadinessWindowSeconds uint64 `mapstructure:"readiness_window_seconds" yaml:"readiness_window_seconds" comment:"Time window in seconds used to calculate ReadinessMaxBlocksBehind based on block time. Default: 15 seconds."` ReadinessMaxBlocksBehind uint64 `mapstructure:"readiness_max_blocks_behind" yaml:"readiness_max_blocks_behind" comment:"How many blocks behind best-known head the node can be and still be considered ready. 0 means must be exactly at head."` + + // Pruning configuration + // When enabled, the node will periodically prune old block data (headers, data, + // signatures, and hash index) from the local store while keeping recent history. + PruningEnabled bool `mapstructure:"pruning_enabled" yaml:"pruning_enabled" comment:"Enable height-based pruning of stored block data. When disabled, all blocks are kept (archive mode)."` + PruningKeepRecent uint64 `mapstructure:"pruning_keep_recent" yaml:"pruning_keep_recent" comment:"Number of most recent blocks to retain. Older blocks will have their header/data/signature removed from the local store. 0 means keep all blocks."` + PruningInterval uint64 `mapstructure:"pruning_interval" yaml:"pruning_interval" comment:"Run pruning every N blocks. Must be >= 1 when pruning is enabled."` } // LogConfig contains all logging configuration parameters @@ -366,6 +373,10 @@ func AddFlags(cmd *cobra.Command) { cmd.Flags().Uint64(FlagReadinessWindowSeconds, def.Node.ReadinessWindowSeconds, "time window in seconds for calculating readiness threshold based on block time (default: 15s)") cmd.Flags().Uint64(FlagReadinessMaxBlocksBehind, def.Node.ReadinessMaxBlocksBehind, "how many blocks behind best-known head the node can be and still be considered ready (0 = must be at head)") cmd.Flags().Duration(FlagScrapeInterval, def.Node.ScrapeInterval.Duration, "interval at which the reaper polls the execution layer for new transactions") + // Pruning configuration flags + cmd.Flags().Bool(FlagPrefixEvnode+"node.pruning_enabled", def.Node.PruningEnabled, "enable height-based pruning of stored block data (headers, data, signatures, index)") + cmd.Flags().Uint64(FlagPrefixEvnode+"node.pruning_keep_recent", def.Node.PruningKeepRecent, "number of most recent blocks to retain when pruning is enabled (0 = keep all)") + cmd.Flags().Uint64(FlagPrefixEvnode+"node.pruning_interval", def.Node.PruningInterval, "run pruning every N blocks (must be >= 1 when pruning is enabled)") // Data Availability configuration flags cmd.Flags().String(FlagDAAddress, def.DA.Address, "DA address (host:port)") diff --git a/pkg/config/defaults.go b/pkg/config/defaults.go index 7c8ba269d8..bf468c3019 100644 --- a/pkg/config/defaults.go +++ b/pkg/config/defaults.go @@ -69,6 +69,9 @@ func DefaultConfig() Config { ReadinessWindowSeconds: defaultReadinessWindowSeconds, ReadinessMaxBlocksBehind: calculateReadinessMaxBlocksBehind(defaultBlockTime.Duration, defaultReadinessWindowSeconds), ScrapeInterval: DurationWrapper{1 * time.Second}, + PruningEnabled: false, + PruningKeepRecent: 0, + PruningInterval: 0, }, DA: DAConfig{ Address: "http://localhost:7980", diff --git a/pkg/store/keys.go b/pkg/store/keys.go index dd989c0e82..ff96fd8955 100644 --- a/pkg/store/keys.go +++ b/pkg/store/keys.go @@ -25,6 +25,12 @@ const ( // LastSubmittedHeaderHeightKey is the key used for persisting the last submitted header height in store. LastSubmittedHeaderHeightKey = "last-submitted-header-height" + // LastPrunedBlockHeightKey is the metadata key used for persisting the last + // pruned block height in the store. All block data (header, data, + // signature, and hash index) for heights <= this value are considered + // pruned and may be missing from the store. + LastPrunedBlockHeightKey = "last-pruned-block-height" + headerPrefix = "h" dataPrefix = "d" signaturePrefix = "c" diff --git a/pkg/store/store.go b/pkg/store/store.go index 972b94e0e4..782ff2821c 100644 --- a/pkg/store/store.go +++ b/pkg/store/store.go @@ -274,6 +274,85 @@ func (s *DefaultStore) Rollback(ctx context.Context, height uint64, aggregator b return nil } +// PruneBlocks removes block data (header, data, signature, and hash index) +// up to and including the given height from the store. It does not modify +// the current chain height or any state snapshots. +// +// This method is intended for long-term storage reduction and is safe to +// call repeatedly with the same or increasing heights. +func (s *DefaultStore) PruneBlocks(ctx context.Context, height uint64) error { + batch, err := s.db.Batch(ctx) + if err != nil { + return fmt.Errorf("failed to create a new batch for pruning: %w", err) + } + + // Track the last successfully pruned height so we can resume across restarts. + var lastPruned uint64 + meta, err := s.GetMetadata(ctx, LastPrunedBlockHeightKey) + if err != nil { + if !errors.Is(err, ds.ErrNotFound) { + return fmt.Errorf("failed to get last pruned height: %w", err) + } + } else if len(meta) == heightLength { + lastPruned, err = decodeHeight(meta) + if err != nil { + return fmt.Errorf("failed to decode last pruned height: %w", err) + } + } + + // Nothing new to prune. + if height <= lastPruned { + return nil + } + + // Delete block data for heights in (lastPruned, height]. + for h := lastPruned + 1; h <= height; h++ { + // Get header blob to compute the hash index key. If header is already + // missing (e.g. due to previous partial pruning), just skip this height. + headerBlob, err := s.db.Get(ctx, ds.NewKey(getHeaderKey(h))) + if err != nil { + if errors.Is(err, ds.ErrNotFound) { + continue + } + return fmt.Errorf("failed to get header at height %d during pruning: %w", h, err) + } + + if err := batch.Delete(ctx, ds.NewKey(getHeaderKey(h))); err != nil { + return fmt.Errorf("failed to delete header at height %d during pruning: %w", h, err) + } + + if err := batch.Delete(ctx, ds.NewKey(getDataKey(h))); err != nil { + if !errors.Is(err, ds.ErrNotFound) { + return fmt.Errorf("failed to delete data at height %d during pruning: %w", h, err) + } + } + + if err := batch.Delete(ctx, ds.NewKey(getSignatureKey(h))); err != nil { + if !errors.Is(err, ds.ErrNotFound) { + return fmt.Errorf("failed to delete signature at height %d during pruning: %w", h, err) + } + } + + headerHash := sha256.Sum256(headerBlob) + if err := batch.Delete(ctx, ds.NewKey(getIndexKey(headerHash[:]))); err != nil { + if !errors.Is(err, ds.ErrNotFound) { + return fmt.Errorf("failed to delete index for height %d during pruning: %w", h, err) + } + } + } + + // Persist the updated last pruned height. + if err := batch.Put(ctx, ds.NewKey(getMetaKey(LastPrunedBlockHeightKey)), encodeHeight(height)); err != nil { + return fmt.Errorf("failed to update last pruned height: %w", err) + } + + if err := batch.Commit(ctx); err != nil { + return fmt.Errorf("failed to commit pruning batch: %w", err) + } + + return nil +} + const heightLength = 8 func encodeHeight(height uint64) []byte { diff --git a/pkg/store/store_test.go b/pkg/store/store_test.go index 8636a0ad0d..b9a796788d 100644 --- a/pkg/store/store_test.go +++ b/pkg/store/store_test.go @@ -734,6 +734,57 @@ func TestRollback(t *testing.T) { require.Equal(rollbackToHeight, state.LastBlockHeight) } +func TestPruneBlocks_RemovesOldBlockDataOnly(t *testing.T) { + t.Parallel() + + ctx := context.Background() + ds, err := NewTestInMemoryKVStore() + require.NoError(t, err) + + s := New(ds).(*DefaultStore) + + // create and store a few blocks with headers, data, signatures and state + batch, err := s.NewBatch(ctx) + require.NoError(t, err) + + var lastState types.State + for h := uint64(1); h <= 5; h++ { + header := &types.SignedHeader{Header: types.Header{BaseHeader: types.BaseHeader{Height: h}}} + data := &types.Data{} + sig := types.Signature([]byte{byte(h)}) + + require.NoError(t, batch.SaveBlockData(header, data, &sig)) + + // fake state snapshot per height + lastState = types.State{LastBlockHeight: h} + require.NoError(t, batch.UpdateState(lastState)) + } + require.NoError(t, batch.SetHeight(5)) + require.NoError(t, batch.Commit()) + + // prune everything up to height 3 + require.NoError(t, s.PruneBlocks(ctx, 3)) + + // old block data should be gone + for h := uint64(1); h <= 3; h++ { + _, _, err := s.GetBlockData(ctx, h) + assert.Error(t, err, "expected block data at height %d to be pruned", h) + } + + // recent block data should remain + for h := uint64(4); h <= 5; h++ { + _, _, err := s.GetBlockData(ctx, h) + assert.NoError(t, err, "expected block data at height %d to be kept", h) + } + + // state snapshots are not pruned by PruneBlocks + for h := uint64(1); h <= 5; h++ { + st, err := s.GetStateAtHeight(ctx, h) + assert.NoError(t, err, "expected state at height %d to remain", h) + assert.Equal(t, h, st.LastBlockHeight) + } +} + // TestRollbackToSameHeight verifies that rollback to same height is a no-op func TestRollbackToSameHeight(t *testing.T) { t.Parallel() diff --git a/pkg/store/types.go b/pkg/store/types.go index bf1cb6ced8..2281e1a4cd 100644 --- a/pkg/store/types.go +++ b/pkg/store/types.go @@ -30,9 +30,15 @@ type Batch interface { } // Store is minimal interface for storing and retrieving blocks, commits and state. +// +// It is composed from three concerns: +// - Reader: read access to blocks, state, and metadata +// - Rollback: consensus rollback logic (used for chain reorgs / recovery) +// - Pruner: long-term height-based pruning of historical block data type Store interface { - Rollback Reader + Rollback + Pruner // SetMetadata saves arbitrary value in the store. // @@ -75,3 +81,14 @@ type Rollback interface { // Aggregator is used to determine if the rollback is performed on the aggregator node. Rollback(ctx context.Context, height uint64, aggregator bool) error } + +// Pruner provides long-term, height-based pruning of historical block data. +// +// Implementations SHOULD be idempotent and safe to call multiple times for +// the same or increasing target heights. +type Pruner interface { + // PruneBlocks removes block data (header, data, signature, and hash index) + // up to and including the given height from the store, without modifying + // state snapshots or the current chain height. + PruneBlocks(ctx context.Context, height uint64) error +} From 27770289e61349bfc5aa4274641fe0b4e0aeb7c4 Mon Sep 17 00:00:00 2001 From: pthmas <9058370+pthmas@users.noreply.github.com> Date: Fri, 23 Jan 2026 17:10:28 +0100 Subject: [PATCH 2/6] prune metadata from ev-node store --- pkg/store/store.go | 12 ++++++++++++ pkg/store/store_test.go | 30 +++++++++++++++++++++++++++++- 2 files changed, 41 insertions(+), 1 deletion(-) diff --git a/pkg/store/store.go b/pkg/store/store.go index 782ff2821c..71678f50f2 100644 --- a/pkg/store/store.go +++ b/pkg/store/store.go @@ -333,6 +333,18 @@ func (s *DefaultStore) PruneBlocks(ctx context.Context, height uint64) error { } } + // Delete per-height DA metadata associated with this height, if any. + if err := batch.Delete(ctx, ds.NewKey(getMetaKey(GetHeightToDAHeightHeaderKey(h)))); err != nil { + if !errors.Is(err, ds.ErrNotFound) { + return fmt.Errorf("failed to delete header DA height metadata at height %d during pruning: %w", h, err) + } + } + if err := batch.Delete(ctx, ds.NewKey(getMetaKey(GetHeightToDAHeightDataKey(h)))); err != nil { + if !errors.Is(err, ds.ErrNotFound) { + return fmt.Errorf("failed to delete data DA height metadata at height %d during pruning: %w", h, err) + } + } + headerHash := sha256.Sum256(headerBlob) if err := batch.Delete(ctx, ds.NewKey(getIndexKey(headerHash[:]))); err != nil { if !errors.Is(err, ds.ErrNotFound) { diff --git a/pkg/store/store_test.go b/pkg/store/store_test.go index b9a796788d..5ba5eecfad 100644 --- a/pkg/store/store_test.go +++ b/pkg/store/store_test.go @@ -743,7 +743,7 @@ func TestPruneBlocks_RemovesOldBlockDataOnly(t *testing.T) { s := New(ds).(*DefaultStore) - // create and store a few blocks with headers, data, signatures and state + // create and store a few blocks with headers, data, signatures, state, and per-height DA metadata batch, err := s.NewBatch(ctx) require.NoError(t, err) @@ -758,6 +758,14 @@ func TestPruneBlocks_RemovesOldBlockDataOnly(t *testing.T) { // fake state snapshot per height lastState = types.State{LastBlockHeight: h} require.NoError(t, batch.UpdateState(lastState)) + + // store fake DA metadata per height + hDaKey := GetHeightToDAHeightHeaderKey(h) + dDaKey := GetHeightToDAHeightDataKey(h) + bz := make([]byte, 8) + binary.LittleEndian.PutUint64(bz, h+100) // arbitrary DA height + require.NoError(t, s.SetMetadata(ctx, hDaKey, bz)) + require.NoError(t, s.SetMetadata(ctx, dDaKey, bz)) } require.NoError(t, batch.SetHeight(5)) require.NoError(t, batch.Commit()) @@ -783,6 +791,26 @@ func TestPruneBlocks_RemovesOldBlockDataOnly(t *testing.T) { assert.NoError(t, err, "expected state at height %d to remain", h) assert.Equal(t, h, st.LastBlockHeight) } + + // per-height DA metadata for pruned heights should be gone + for h := uint64(1); h <= 3; h++ { + hDaKey := GetHeightToDAHeightHeaderKey(h) + dDaKey := GetHeightToDAHeightDataKey(h) + _, err := s.GetMetadata(ctx, hDaKey) + assert.Error(t, err, "expected header DA metadata at height %d to be pruned", h) + _, err = s.GetMetadata(ctx, dDaKey) + assert.Error(t, err, "expected data DA metadata at height %d to be pruned", h) + } + + // per-height DA metadata for unpruned heights should remain + for h := uint64(4); h <= 5; h++ { + hDaKey := GetHeightToDAHeightHeaderKey(h) + dDaKey := GetHeightToDAHeightDataKey(h) + _, err := s.GetMetadata(ctx, hDaKey) + assert.NoError(t, err, "expected header DA metadata at height %d to remain", h) + _, err = s.GetMetadata(ctx, dDaKey) + assert.NoError(t, err, "expected data DA metadata at height %d to remain", h) + } } // TestRollbackToSameHeight verifies that rollback to same height is a no-op From 1d44f27265006aaec298aba8decbdac6208679b3 Mon Sep 17 00:00:00 2001 From: pthmas <9058370+pthmas@users.noreply.github.com> Date: Fri, 23 Jan 2026 17:27:04 +0100 Subject: [PATCH 3/6] wiring prunning config to go-header --- pkg/sync/sync_service.go | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/pkg/sync/sync_service.go b/pkg/sync/sync_service.go index 23db9ce1f4..e062443232 100644 --- a/pkg/sync/sync_service.go +++ b/pkg/sync/sync_service.go @@ -34,7 +34,8 @@ const ( dataSync syncType = "dataSync" ) -// TODO: when we add pruning we can remove this +// Large default trusting period for go-header syncer. This is intentionally +// very long to avoid accidental expiration unless explicitly configured. const ninetyNineYears = 99 * 365 * 24 * time.Hour // SyncService is the P2P Sync Service for blocks and headers. @@ -255,11 +256,21 @@ func (syncService *SyncService[H]) Start(ctx context.Context) error { } // create syncer, must be before initFromP2PWithRetry which calls startSyncer. + syncOpts := []goheadersync.Option{goheadersync.WithBlockTime(syncService.conf.Node.BlockTime.Duration)} + // Map ev-node pruning configuration to go-header's pruning window: we approximate + // "keep N recent heights" as "retain headers for N * blockTime". + if syncService.conf.Node.PruningEnabled && syncService.conf.Node.PruningKeepRecent > 0 { + pruningWindow := syncService.conf.Node.BlockTime.Duration * time.Duration(syncService.conf.Node.PruningKeepRecent) + // Only set a pruning window if the computed duration is positive. + if pruningWindow > 0 { + syncOpts = append(syncOpts, goheadersync.WithPruningWindow(pruningWindow)) + } + } if syncService.syncer, err = newSyncer( syncService.ex, syncService.store, syncService.sub, - []goheadersync.Option{goheadersync.WithBlockTime(syncService.conf.Node.BlockTime.Duration)}, + syncOpts, ); err != nil { return fmt.Errorf("failed to create syncer: %w", err) } @@ -517,7 +528,6 @@ func newSyncer[H header.Header[H]]( ) (*goheadersync.Syncer[H], error) { opts = append(opts, goheadersync.WithMetrics(), - goheadersync.WithPruningWindow(ninetyNineYears), goheadersync.WithTrustingPeriod(ninetyNineYears), ) return goheadersync.NewSyncer(ex, store, sub, opts...) From 2996d988a230a0d5c1a23db273dced28c1007750 Mon Sep 17 00:00:00 2001 From: pthmas <9058370+pthmas@users.noreply.github.com> Date: Mon, 26 Jan 2026 12:42:55 +0100 Subject: [PATCH 4/6] prune evm exec store --- block/internal/executing/executor.go | 11 ++++ core/execution/execution.go | 13 ++++ execution/evm/execution.go | 17 +++++ execution/evm/store.go | 53 +++++++++++++++ execution/evm/store_test.go | 99 ++++++++++++++++++++++++++++ 5 files changed, 193 insertions(+) create mode 100644 execution/evm/store_test.go diff --git a/block/internal/executing/executor.go b/block/internal/executing/executor.go index 538ceab325..74521bce1a 100644 --- a/block/internal/executing/executor.go +++ b/block/internal/executing/executor.go @@ -457,6 +457,17 @@ func (e *Executor) ProduceBlock(ctx context.Context) error { if err := e.store.PruneBlocks(e.ctx, targetHeight); err != nil { e.logger.Error().Err(err).Uint64("target_height", targetHeight).Msg("failed to prune old block data") } + + // If the execution client exposes execution-metadata pruning, + // prune ExecMeta using the same target height. This keeps EVM + // execution metadata aligned with ev-node's block store pruning + // while remaining a no-op for execution environments that don't + // implement ExecMetaPruner (e.g. ABCI-based executors). + if pruner, ok := e.exec.(coreexecutor.ExecMetaPruner); ok { + if err := pruner.PruneExecMeta(e.ctx, targetHeight); err != nil { + e.logger.Error().Err(err).Uint64("target_height", targetHeight).Msg("failed to prune execution metadata") + } + } } } } diff --git a/core/execution/execution.go b/core/execution/execution.go index 276da369e9..1f85e2068a 100644 --- a/core/execution/execution.go +++ b/core/execution/execution.go @@ -161,3 +161,16 @@ type Rollbackable interface { // Rollback resets the execution layer head to the specified height. Rollback(ctx context.Context, targetHeight uint64) error } + +// ExecMetaPruner is an optional interface that execution clients can implement +// to support height-based pruning of their execution metadata. This is used by +// EVM-based execution clients to keep ExecMeta consistent with ev-node's +// pruning window while remaining a no-op for execution environments that +// don't persist per-height metadata in ev-node's datastore. +type ExecMetaPruner interface { + // PruneExecMeta should delete execution metadata for all heights up to and + // including the given height. Implementations should be idempotent and track + // their own progress so that repeated calls with the same or decreasing + // heights are cheap no-ops. + PruneExecMeta(ctx context.Context, height uint64) error +} diff --git a/execution/evm/execution.go b/execution/evm/execution.go index 1a61fdc20f..4033e45762 100644 --- a/execution/evm/execution.go +++ b/execution/evm/execution.go @@ -65,6 +65,12 @@ var _ execution.HeightProvider = (*EngineClient)(nil) // Ensure EngineClient implements the execution.Rollbackable interface var _ execution.Rollbackable = (*EngineClient)(nil) +// Ensure EngineClient implements optional pruning interface when used with +// ev-node's height-based pruning. This enables coordinated pruning of EVM +// ExecMeta alongside ev-node's own block data pruning, while remaining a +// no-op for non-EVM execution environments. +var _ execution.ExecMetaPruner = (*EngineClient)(nil) + // validatePayloadStatus checks the payload status and returns appropriate errors. // It implements the Engine API specification's status handling: // - VALID: Operation succeeded, return nil @@ -265,6 +271,17 @@ func NewEngineExecutionClient( }, nil } +// PruneExecMeta implements execution.ExecMetaPruner by delegating to the +// underlying EVMStore. It is safe to call this multiple times with the same +// or increasing heights; the store tracks its own last-pruned height. +func (c *EngineClient) PruneExecMeta(ctx context.Context, height uint64) error { + if c.store == nil { + return nil + } + + return c.store.PruneExecMeta(ctx, height) +} + // SetLogger allows callers to attach a structured logger. func (c *EngineClient) SetLogger(l zerolog.Logger) { c.logger = l diff --git a/execution/evm/store.go b/execution/evm/store.go index af731ee7c6..51a3056b66 100644 --- a/execution/evm/store.go +++ b/execution/evm/store.go @@ -15,6 +15,11 @@ import ( // Store prefix for execution/evm data - keeps it isolated from other ev-node data const evmStorePrefix = "evm/" +// lastPrunedExecMetaKey is the datastore key used to track the highest +// execution height for which ExecMeta has been pruned. All ExecMeta entries +// for heights <= this value are considered pruned. +const lastPrunedExecMetaKey = evmStorePrefix + "last-pruned-execmeta-height" + // ExecMeta stages const ( ExecStageStarted = "started" @@ -140,6 +145,54 @@ func (s *EVMStore) SaveExecMeta(ctx context.Context, meta *ExecMeta) error { return nil } +// PruneExecMeta removes ExecMeta entries up to and including the given height. +// It is safe to call this multiple times with the same or increasing heights; +// previously pruned ranges will be skipped based on the last-pruned marker. +func (s *EVMStore) PruneExecMeta(ctx context.Context, height uint64) error { + // Load last pruned height, if any. + var lastPruned uint64 + data, err := s.db.Get(ctx, ds.NewKey(lastPrunedExecMetaKey)) + if err != nil { + if !errors.Is(err, ds.ErrNotFound) { + return fmt.Errorf("failed to get last pruned execmeta height: %w", err) + } + } else if len(data) == 8 { + lastPruned = binary.BigEndian.Uint64(data) + } + + // Nothing new to prune. + if height <= lastPruned { + return nil + } + + batch, err := s.db.Batch(ctx) + if err != nil { + return fmt.Errorf("failed to create batch for execmeta pruning: %w", err) + } + + for h := lastPruned + 1; h <= height; h++ { + key := execMetaKey(h) + if err := batch.Delete(ctx, key); err != nil { + if !errors.Is(err, ds.ErrNotFound) { + return fmt.Errorf("failed to delete exec meta at height %d: %w", h, err) + } + } + } + + // Persist updated last pruned height. + buf := make([]byte, 8) + binary.BigEndian.PutUint64(buf, height) + if err := batch.Put(ctx, ds.NewKey(lastPrunedExecMetaKey), buf); err != nil { + return fmt.Errorf("failed to update last pruned execmeta height: %w", err) + } + + if err := batch.Commit(ctx); err != nil { + return fmt.Errorf("failed to commit execmeta pruning batch: %w", err) + } + + return nil +} + // Sync ensures all pending writes are flushed to disk. func (s *EVMStore) Sync(ctx context.Context) error { return s.db.Sync(ctx, ds.NewKey(evmStorePrefix)) diff --git a/execution/evm/store_test.go b/execution/evm/store_test.go new file mode 100644 index 0000000000..64389701f9 --- /dev/null +++ b/execution/evm/store_test.go @@ -0,0 +1,99 @@ +package evm + +import ( + "context" + "encoding/binary" + "testing" + + ds "github.com/ipfs/go-datastore" + dssync "github.com/ipfs/go-datastore/sync" + "github.com/stretchr/testify/require" +) + +// newTestDatastore creates an in-memory datastore for testing. +func newTestDatastore(t *testing.T) ds.Batching { + t.Helper() + // Wrap the in-memory MapDatastore to satisfy the Batching interface. + return dssync.MutexWrap(ds.NewMapDatastore()) +} + +func TestPruneExecMeta_PrunesUpToTargetHeight(t *testing.T) { + t.Parallel() + + ctx := context.Background() + db := newTestDatastore(t) + store := NewEVMStore(db) + + // Seed ExecMeta entries at heights 1..5 + for h := uint64(1); h <= 5; h++ { + meta := &ExecMeta{Height: h} + require.NoError(t, store.SaveExecMeta(ctx, meta)) + } + + // Sanity: all heights should be present + for h := uint64(1); h <= 5; h++ { + meta, err := store.GetExecMeta(ctx, h) + require.NoError(t, err) + require.NotNil(t, meta) + require.Equal(t, h, meta.Height) + } + + // Prune up to height 3 + require.NoError(t, store.PruneExecMeta(ctx, 3)) + + // Heights 1..3 should be gone + for h := uint64(1); h <= 3; h++ { + meta, err := store.GetExecMeta(ctx, h) + require.NoError(t, err) + require.Nil(t, meta) + } + + // Heights 4..5 should remain + for h := uint64(4); h <= 5; h++ { + meta, err := store.GetExecMeta(ctx, h) + require.NoError(t, err) + require.NotNil(t, meta) + } + + // Re-pruning with the same height should be a no-op + require.NoError(t, store.PruneExecMeta(ctx, 3)) +} + +func TestPruneExecMeta_TracksLastPrunedHeight(t *testing.T) { + t.Parallel() + + ctx := context.Background() + db := newTestDatastore(t) + store := NewEVMStore(db) + + // Seed ExecMeta entries at heights 1..5 + for h := uint64(1); h <= 5; h++ { + meta := &ExecMeta{Height: h} + require.NoError(t, store.SaveExecMeta(ctx, meta)) + } + + // First prune up to 2 + require.NoError(t, store.PruneExecMeta(ctx, 2)) + + // Then prune up to 4; heights 3..4 should be deleted in this run + require.NoError(t, store.PruneExecMeta(ctx, 4)) + + // Verify all heights 1..4 are gone, 5 remains + for h := uint64(1); h <= 4; h++ { + meta, err := store.GetExecMeta(ctx, h) + require.NoError(t, err) + require.Nil(t, meta) + } + + meta, err := store.GetExecMeta(ctx, 5) + require.NoError(t, err) + require.NotNil(t, meta) + require.Equal(t, uint64(5), meta.Height) + + // Ensure last-pruned marker is set to 4 + raw, err := db.Get(ctx, ds.NewKey(lastPrunedExecMetaKey)) + require.NoError(t, err) + require.Len(t, raw, 8) + last := binary.BigEndian.Uint64(raw) + require.Equal(t, uint64(4), last) +} From 961b3f7a9b4d5a979ee8fe1c06a03a2b241a953c Mon Sep 17 00:00:00 2001 From: pthmas <9058370+pthmas@users.noreply.github.com> Date: Mon, 26 Jan 2026 15:45:24 +0100 Subject: [PATCH 5/6] add parameters validation --- pkg/config/config.go | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/pkg/config/config.go b/pkg/config/config.go index 6a3583e9ba..1d82354f14 100644 --- a/pkg/config/config.go +++ b/pkg/config/config.go @@ -313,6 +313,20 @@ func (c *Config) Validate() error { c.Node.LazyBlockInterval.Duration, c.Node.BlockTime.Duration) } + // Validate pruning configuration + if c.Node.PruningEnabled { + // When pruning is enabled, pruning_interval must be >= 1 + if c.Node.PruningInterval == 0 { + return fmt.Errorf("pruning_interval must be >= 1 when pruning is enabled") + } + + // When pruning is enabled, keeping 0 blocks is contradictory; use pruning_enabled=false + // for archive mode instead. + if c.Node.PruningKeepRecent == 0 { + return fmt.Errorf("pruning_keep_recent must be > 0 when pruning is enabled; use pruning_enabled=false to keep all blocks") + } + } + return nil } From ed66fe6a6c15b1ee800f7c8a5cd3613789b7fa86 Mon Sep 17 00:00:00 2001 From: pthmas <9058370+pthmas@users.noreply.github.com> Date: Mon, 26 Jan 2026 15:47:32 +0100 Subject: [PATCH 6/6] make error handling consistent --- pkg/store/store.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/pkg/store/store.go b/pkg/store/store.go index 71678f50f2..cd652a5dc3 100644 --- a/pkg/store/store.go +++ b/pkg/store/store.go @@ -318,7 +318,9 @@ func (s *DefaultStore) PruneBlocks(ctx context.Context, height uint64) error { } if err := batch.Delete(ctx, ds.NewKey(getHeaderKey(h))); err != nil { - return fmt.Errorf("failed to delete header at height %d during pruning: %w", h, err) + if !errors.Is(err, ds.ErrNotFound) { + return fmt.Errorf("failed to delete header at height %d during pruning: %w", h, err) + } } if err := batch.Delete(ctx, ds.NewKey(getDataKey(h))); err != nil {