From e03071e2471e64d4145c2548e918acbfcae2ac03 Mon Sep 17 00:00:00 2001 From: Tsvetan Dimitrov Date: Wed, 3 Dec 2025 21:57:52 +0200 Subject: [PATCH 01/20] refactor(vmsync): extract SyncStrategy pattern for static state sync Restructure the vmsync package to separate concerns and prepare for future dynamic state sync support. - Extract `finalizer.go` - VM state finalization logic (`finishSync`, `commitMarkers`) moved to dedicated `finalizer` struct. - Add `SyncStrategy` interface with `Start` method for sync execution. - Implement `staticStrategy` for current sequential sync behavior. - Simplify `client.go` to orchestrate sync lifecycle via strategy. This refactoring enables adding dynamic state sync as a separate strategy implementation without modifying existing static sync logic. resolves #4651 Signed-off-by: Tsvetan Dimitrov (tsvetan.dimitrov@avalabs.org) --- graft/coreth/plugin/evm/vmsync/client.go | 340 ++++++------------ graft/coreth/plugin/evm/vmsync/finalizer.go | 152 ++++++++ .../plugin/evm/vmsync/strategy_static.go | 38 ++ 3 files changed, 309 insertions(+), 221 deletions(-) create mode 100644 graft/coreth/plugin/evm/vmsync/finalizer.go create mode 100644 graft/coreth/plugin/evm/vmsync/strategy_static.go diff --git a/graft/coreth/plugin/evm/vmsync/client.go b/graft/coreth/plugin/evm/vmsync/client.go index 435e5b836560..efd7c4a72076 100644 --- a/graft/coreth/plugin/evm/vmsync/client.go +++ b/graft/coreth/plugin/evm/vmsync/client.go @@ -5,11 +5,10 @@ package vmsync import ( "context" + "errors" "fmt" "sync" - "github.com/ava-labs/libevm/common" - "github.com/ava-labs/libevm/core/types" "github.com/ava-labs/libevm/ethdb" "github.com/ava-labs/libevm/log" @@ -17,23 +16,25 @@ import ( "github.com/ava-labs/avalanchego/database/versiondb" "github.com/ava-labs/avalanchego/graft/coreth/core/state/snapshot" "github.com/ava-labs/avalanchego/graft/coreth/eth" - "github.com/ava-labs/avalanchego/graft/coreth/params" "github.com/ava-labs/avalanchego/graft/coreth/plugin/evm/message" - "github.com/ava-labs/avalanchego/graft/coreth/sync/blocksync" - "github.com/ava-labs/avalanchego/graft/coreth/sync/statesync" "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/avalanchego/snow/engine/snowman/block" "github.com/ava-labs/avalanchego/vms/components/chain" syncpkg "github.com/ava-labs/avalanchego/graft/coreth/sync" + "github.com/ava-labs/avalanchego/graft/coreth/sync/blocksync" syncclient "github.com/ava-labs/avalanchego/graft/coreth/sync/client" + "github.com/ava-labs/avalanchego/graft/coreth/sync/statesync" ) // BlocksToFetch is the number of the block parents the state syncs to. // The last 256 block hashes are necessary to support the BLOCKHASH opcode. const BlocksToFetch = 256 -var stateSyncSummaryKey = []byte("stateSyncSummary") +var ( + errSkipSync = fmt.Errorf("skip sync") + stateSyncSummaryKey = []byte("stateSyncSummary") +) // BlockAcceptor provides a mechanism to update the last accepted block ID during state synchronization. // This interface is used by the state sync process to ensure the blockchain state @@ -42,11 +43,11 @@ type BlockAcceptor interface { PutLastAcceptedID(ids.ID) error } -// EthBlockWrapper can be implemented by a concrete block wrapper type to -// return *types.Block, which is needed to update chain pointers at the -// end of the sync operation. -type EthBlockWrapper interface { - GetEthBlock() *types.Block +// SyncStrategy defines how state sync is executed. +// Implementations handle the sync lifecycle differently based on sync mode. +type SyncStrategy interface { + // Start begins the sync process and blocks until completion or error. + Start(ctx context.Context) error } type ClientConfig struct { @@ -76,21 +77,16 @@ type ClientConfig struct { } type client struct { - *ClientConfig - + config *ClientConfig resumableSummary message.Syncable - - cancel context.CancelFunc - wg sync.WaitGroup - - // State Sync results - summary message.Syncable - err error + cancel context.CancelFunc + wg sync.WaitGroup + err error } func NewClient(config *ClientConfig) Client { return &client{ - ClientConfig: config, + config: config, } } @@ -107,37 +103,37 @@ type Client interface { } // StateSyncEnabled returns [client.enabled], which is set in the chain's config file. -func (client *client) StateSyncEnabled(context.Context) (bool, error) { - return client.Enabled, nil +func (c *client) StateSyncEnabled(context.Context) (bool, error) { + return c.config.Enabled, nil } // GetOngoingSyncStateSummary returns a state summary that was previously started // and not finished, and sets [resumableSummary] if one was found. // Returns [database.ErrNotFound] if no ongoing summary is found or if [client.skipResume] is true. -func (client *client) GetOngoingSyncStateSummary(context.Context) (block.StateSummary, error) { - if client.SkipResume { +func (c *client) GetOngoingSyncStateSummary(context.Context) (block.StateSummary, error) { + if c.config.SkipResume { return nil, database.ErrNotFound } - summaryBytes, err := client.MetadataDB.Get(stateSyncSummaryKey) + summaryBytes, err := c.config.MetadataDB.Get(stateSyncSummaryKey) if err != nil { return nil, err // includes the [database.ErrNotFound] case } - summary, err := client.Parser.Parse(summaryBytes, client.acceptSyncSummary) + summary, err := c.config.Parser.Parse(summaryBytes, c.acceptSyncSummary) if err != nil { return nil, fmt.Errorf("failed to parse saved state sync summary to SyncSummary: %w", err) } - client.resumableSummary = summary + c.resumableSummary = summary return summary, nil } // ClearOngoingSummary clears any marker of an ongoing state sync summary -func (client *client) ClearOngoingSummary() error { - if err := client.MetadataDB.Delete(stateSyncSummaryKey); err != nil { +func (c *client) ClearOngoingSummary() error { + if err := c.config.MetadataDB.Delete(stateSyncSummaryKey); err != nil { return fmt.Errorf("failed to clear ongoing summary: %w", err) } - if err := client.VerDB.Commit(); err != nil { + if err := c.config.VerDB.Commit(); err != nil { return fmt.Errorf("failed to commit db while clearing ongoing summary: %w", err) } @@ -145,121 +141,55 @@ func (client *client) ClearOngoingSummary() error { } // ParseStateSummary parses [summaryBytes] to [commonEng.Summary] -func (client *client) ParseStateSummary(_ context.Context, summaryBytes []byte) (block.StateSummary, error) { - return client.Parser.Parse(summaryBytes, client.acceptSyncSummary) -} - -func (client *client) stateSync(ctx context.Context) error { - // Create and register all syncers. - registry := NewSyncerRegistry() - - if err := client.registerSyncers(registry); err != nil { - return err - } - - // Run all registered syncers sequentially. - return registry.RunSyncerTasks(ctx, client.summary) +func (c *client) ParseStateSummary(_ context.Context, summaryBytes []byte) (block.StateSummary, error) { + return c.config.Parser.Parse(summaryBytes, c.acceptSyncSummary) } -func (client *client) registerSyncers(registry *SyncerRegistry) error { - // Register block syncer. - blockSyncer, err := client.createBlockSyncer(client.summary.GetBlockHash(), client.summary.Height()) - if err != nil { - return fmt.Errorf("failed to create block syncer: %w", err) - } - - codeQueue, err := client.createCodeQueue() - if err != nil { - return fmt.Errorf("failed to create code queue: %w", err) - } - - codeSyncer, err := client.createCodeSyncer(codeQueue.CodeHashes()) - if err != nil { - return fmt.Errorf("failed to create code syncer: %w", err) - } - - stateSyncer, err := client.createEVMSyncer(codeQueue) - if err != nil { - return fmt.Errorf("failed to create EVM state syncer: %w", err) - } - - var atomicSyncer syncpkg.Syncer - if client.Extender != nil { - atomicSyncer, err = client.createAtomicSyncer() - if err != nil { - return fmt.Errorf("failed to create atomic syncer: %w", err) +// acceptSyncSummary returns true if sync will be performed and launches the state sync process +// in a goroutine. +func (c *client) acceptSyncSummary(summary message.Syncable) (block.StateSyncMode, error) { + if err := c.prepareForSync(summary); err != nil { + if errors.Is(err, errSkipSync) { + return block.StateSyncSkipped, nil } + return block.StateSyncSkipped, err } - syncers := []syncpkg.Syncer{ - blockSyncer, - codeSyncer, - stateSyncer, - } - if atomicSyncer != nil { - syncers = append(syncers, atomicSyncer) - } - - for _, s := range syncers { - if err := registry.Register(s); err != nil { - return fmt.Errorf("failed to register %s syncer: %w", s.Name(), err) - } + registry, err := c.newSyncerRegistry(summary) + if err != nil { + return block.StateSyncSkipped, fmt.Errorf("failed to create syncer registry: %w", err) } - return nil -} - -func (client *client) createBlockSyncer(fromHash common.Hash, fromHeight uint64) (syncpkg.Syncer, error) { - return blocksync.NewSyncer( - client.Client, - client.ChainDB, - fromHash, - fromHeight, - BlocksToFetch, - ) -} - -func (client *client) createEVMSyncer(queue *statesync.CodeQueue) (syncpkg.Syncer, error) { - return statesync.NewSyncer( - client.Client, - client.ChainDB, - client.summary.GetBlockRoot(), - queue, - client.RequestSize, + finalizer := newFinalizer( + c.config.Chain, + c.config.State, + c.config.Acceptor, + c.config.VerDB, + c.config.MetadataDB, + c.config.Extender, + c.config.LastAcceptedHeight, ) -} -func (client *client) createCodeQueue() (*statesync.CodeQueue, error) { - return statesync.NewCodeQueue( - client.ChainDB, - client.StateSyncDone, - ) -} + strategy := newStaticStrategy(registry, finalizer, summary) -func (client *client) createCodeSyncer(codeHashes <-chan common.Hash) (syncpkg.Syncer, error) { - return statesync.NewCodeSyncer(client.Client, client.ChainDB, codeHashes) + return c.startAsync(strategy), nil } -func (client *client) createAtomicSyncer() (syncpkg.Syncer, error) { - return client.Extender.CreateSyncer(client.Client, client.VerDB, client.summary) -} - -// acceptSyncSummary returns true if sync will be performed and launches the state sync process -// in a goroutine. -func (client *client) acceptSyncSummary(proposedSummary message.Syncable) (block.StateSyncMode, error) { - isResume := client.resumableSummary != nil && - proposedSummary.GetBlockHash() == client.resumableSummary.GetBlockHash() +// prepareForSync handles resume check and snapshot wipe before sync starts. +func (c *client) prepareForSync(summary message.Syncable) error { + isResume := c.resumableSummary != nil && + summary.GetBlockHash() == c.resumableSummary.GetBlockHash() if !isResume { // Skip syncing if the blockchain is not significantly ahead of local state, // since bootstrapping would be faster. // (Also ensures we don't sync to a height prior to local state.) - if client.LastAcceptedHeight+client.MinBlocks > proposedSummary.Height() { + if c.config.LastAcceptedHeight+c.config.MinBlocks > summary.Height() { log.Info( "last accepted too close to most recent syncable block, skipping state sync", - "lastAccepted", client.LastAcceptedHeight, - "syncableHeight", proposedSummary.Height(), + "lastAccepted", c.config.LastAcceptedHeight, + "syncableHeight", summary.Height(), ) - return block.StateSyncSkipped, nil + return errSkipSync } // Wipe the snapshot completely if we are not resuming from an existing sync, so that we do not @@ -268,140 +198,108 @@ func (client *client) acceptSyncSummary(proposedSummary message.Syncable) (block // sync marker will be wiped, so we do not accidentally resume progress from an incorrect version // of the snapshot. (if switching between versions that come before this change and back this could // lead to the snapshot not being cleaned up correctly) - <-snapshot.WipeSnapshot(client.ChainDB, true) + <-snapshot.WipeSnapshot(c.config.ChainDB, true) // Reset the snapshot generator here so that when state sync completes, snapshots will not attempt to read an // invalid generator. // Note: this must be called after WipeSnapshot is called so that we do not invalidate a partially generated snapshot. - snapshot.ResetSnapshotGeneration(client.ChainDB) + snapshot.ResetSnapshotGeneration(c.config.ChainDB) } - client.summary = proposedSummary // Update the current state sync summary key in the database // Note: this must be performed after WipeSnapshot finishes so that we do not start a state sync // session from a partially wiped snapshot. - if err := client.MetadataDB.Put(stateSyncSummaryKey, proposedSummary.Bytes()); err != nil { - return block.StateSyncSkipped, fmt.Errorf("failed to write state sync summary key to disk: %w", err) + if err := c.config.MetadataDB.Put(stateSyncSummaryKey, summary.Bytes()); err != nil { + return fmt.Errorf("failed to write state sync summary key to disk: %w", err) } - if err := client.VerDB.Commit(); err != nil { - return block.StateSyncSkipped, fmt.Errorf("failed to commit db: %w", err) + if err := c.config.VerDB.Commit(); err != nil { + return fmt.Errorf("failed to commit db: %w", err) } - log.Info("Starting state sync", "summary", proposedSummary) + return nil +} - // create a cancellable ctx for the state sync goroutine +// startAsync launches the sync strategy in a background goroutine. +func (c *client) startAsync(strategy SyncStrategy) block.StateSyncMode { ctx, cancel := context.WithCancel(context.Background()) - client.cancel = cancel - client.wg.Add(1) // track the state sync goroutine so we can wait for it on shutdown + c.cancel = cancel + + c.wg.Add(1) go func() { - defer client.wg.Done() + defer c.wg.Done() defer cancel() - if err := client.stateSync(ctx); err != nil { - client.err = err - } else { - client.err = client.finishSync() + if err := strategy.Start(ctx); err != nil { + c.err = err } // notify engine regardless of whether err == nil, // this error will be propagated to the engine when it calls // vm.SetState(snow.Bootstrapping) - log.Info("stateSync completed, notifying engine", "err", client.err) - close(client.StateSyncDone) + log.Info("state sync completed, notifying engine", "err", c.err) + close(c.config.StateSyncDone) }() - return block.StateSyncStatic, nil + + log.Info("state sync started", "mode", block.StateSyncStatic) + return block.StateSyncStatic } -func (client *client) Shutdown() error { - if client.cancel != nil { - client.cancel() +func (c *client) Shutdown() error { + if c.cancel != nil { + c.cancel() } - client.wg.Wait() // wait for the background goroutine to exit + c.wg.Wait() // wait for the background goroutine to exit return nil } -// finishSync is responsible for updating disk and memory pointers so the VM is prepared -// for bootstrapping. Executes any shared memory operations from the atomic trie to shared memory. -func (client *client) finishSync() error { - stateBlock, err := client.State.GetBlock(context.TODO(), ids.ID(client.summary.GetBlockHash())) - if err != nil { - return fmt.Errorf("could not get block by hash from client state: %s", client.summary.GetBlockHash()) - } +// Error returns a non-nil error if one occurred during the sync. +func (c *client) Error() error { return c.err } - wrapper, ok := stateBlock.(*chain.BlockWrapper) - if !ok { - return fmt.Errorf("could not convert block(%T) to *chain.BlockWrapper", wrapper) - } - wrappedBlock := wrapper.Block +// newSyncerRegistry creates a registry with all required syncers for the given summary. +func (c *client) newSyncerRegistry(summary message.Syncable) (*SyncerRegistry, error) { + registry := NewSyncerRegistry() - evmBlockGetter, ok := wrappedBlock.(EthBlockWrapper) - if !ok { - return fmt.Errorf("could not convert block(%T) to evm.EthBlockWrapper", stateBlock) + blockSyncer, err := blocksync.NewSyncer( + c.config.Client, c.config.ChainDB, + summary.GetBlockHash(), summary.Height(), + BlocksToFetch, + ) + if err != nil { + return nil, fmt.Errorf("failed to create block syncer: %w", err) } - block := evmBlockGetter.GetEthBlock() - - if block.Hash() != client.summary.GetBlockHash() { - return fmt.Errorf("attempted to set last summary block to unexpected block hash: (%s != %s)", block.Hash(), client.summary.GetBlockHash()) - } - if block.NumberU64() != client.summary.Height() { - return fmt.Errorf("attempted to set last summary block to unexpected block number: (%d != %d)", block.NumberU64(), client.summary.Height()) + codeQueue, err := statesync.NewCodeQueue(c.config.ChainDB, c.config.StateSyncDone) + if err != nil { + return nil, fmt.Errorf("failed to create code queue: %w", err) } - // BloomIndexer needs to know that some parts of the chain are not available - // and cannot be indexed. This is done by calling [AddCheckpoint] here. - // Since the indexer uses sections of size [params.BloomBitsBlocks] (= 4096), - // each block is indexed in section number [blockNumber/params.BloomBitsBlocks]. - // To allow the indexer to start with the block we just synced to, - // we create a checkpoint for its parent. - // Note: This requires assuming the synced block height is divisible - // by [params.BloomBitsBlocks]. - parentHeight := block.NumberU64() - 1 - parentHash := block.ParentHash() - client.Chain.BloomIndexer().AddCheckpoint(parentHeight/params.BloomBitsBlocks, parentHash) - - if err := client.Chain.BlockChain().ResetToStateSyncedBlock(block); err != nil { - return err + codeSyncer, err := statesync.NewCodeSyncer(c.config.Client, c.config.ChainDB, codeQueue.CodeHashes()) + if err != nil { + return nil, fmt.Errorf("failed to create code syncer: %w", err) } - if client.Extender != nil { - if err := client.Extender.OnFinishBeforeCommit(client.LastAcceptedHeight, client.summary); err != nil { - return err - } + stateSyncer, err := statesync.NewSyncer( + c.config.Client, c.config.ChainDB, + summary.GetBlockRoot(), + codeQueue, c.config.RequestSize, + ) + if err != nil { + return nil, fmt.Errorf("failed to create EVM state syncer: %w", err) } - if err := client.commitVMMarkers(); err != nil { - return fmt.Errorf("error updating vm markers, height=%d, hash=%s, err=%w", block.NumberU64(), block.Hash(), err) - } + syncers := []syncpkg.Syncer{blockSyncer, codeSyncer, stateSyncer} - if err := client.State.SetLastAcceptedBlock(wrappedBlock); err != nil { - return err + if c.config.Extender != nil { + atomicSyncer, err := c.config.Extender.CreateSyncer(c.config.Client, c.config.VerDB, summary) + if err != nil { + return nil, fmt.Errorf("failed to create atomic syncer: %w", err) + } + syncers = append(syncers, atomicSyncer) } - if client.Extender != nil { - return client.Extender.OnFinishAfterCommit(block.NumberU64()) + for _, s := range syncers { + if err := registry.Register(s); err != nil { + return nil, fmt.Errorf("failed to register %s syncer: %w", s.Name(), err) + } } - return nil -} - -// commitVMMarkers updates the following markers in the VM's database -// and commits them atomically: -// - updates atomic trie so it will have necessary metadata for the last committed root -// - updates atomic trie so it will resume applying operations to shared memory on initialize -// - updates lastAcceptedKey -// - removes state sync progress markers -func (client *client) commitVMMarkers() error { - // Mark the previously last accepted block for the shared memory cursor, so that we will execute shared - // memory operations from the previously last accepted block to [vm.syncSummary] when ApplyToSharedMemory - // is called. - id := ids.ID(client.summary.GetBlockHash()) - if err := client.Acceptor.PutLastAcceptedID(id); err != nil { - return err - } - if err := client.MetadataDB.Delete(stateSyncSummaryKey); err != nil { - return err - } - return client.VerDB.Commit() + return registry, nil } - -// Error returns a non-nil error if one occurred during the sync. -func (client *client) Error() error { return client.err } diff --git a/graft/coreth/plugin/evm/vmsync/finalizer.go b/graft/coreth/plugin/evm/vmsync/finalizer.go new file mode 100644 index 000000000000..b803d4e6fb80 --- /dev/null +++ b/graft/coreth/plugin/evm/vmsync/finalizer.go @@ -0,0 +1,152 @@ +// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package vmsync + +import ( + "context" + "errors" + "fmt" + + "github.com/ava-labs/libevm/core/types" + + "github.com/ava-labs/avalanchego/database" + "github.com/ava-labs/avalanchego/database/versiondb" + "github.com/ava-labs/avalanchego/graft/coreth/eth" + "github.com/ava-labs/avalanchego/graft/coreth/params" + "github.com/ava-labs/avalanchego/graft/coreth/plugin/evm/message" + "github.com/ava-labs/avalanchego/ids" + "github.com/ava-labs/avalanchego/vms/components/chain" + + syncpkg "github.com/ava-labs/avalanchego/graft/coreth/sync" +) + +// EthBlockWrapper can be implemented by a concrete block wrapper type to +// return *types.Block, which is needed to update chain pointers at the +// end of the sync operation. +type EthBlockWrapper interface { + GetEthBlock() *types.Block +} + +var ( + errBlockNotFound = errors.New("block not found in state") + errInvalidBlockType = errors.New("invalid block wrapper type") + errBlockHashMismatch = errors.New("block hash mismatch") + errBlockHeightMismatch = errors.New("block height mismatch") + errFinalizeCancelled = errors.New("finalize cancelled") + errCommitMarkers = errors.New("failed to commit VM markers") +) + +// finalizer handles VM state finalization after sync completes. +type finalizer struct { + chain *eth.Ethereum + state *chain.State + acceptor BlockAcceptor + verDB *versiondb.Database + metadataDB database.Database + extender syncpkg.Extender + lastAcceptedHeight uint64 +} + +// newFinalizer creates a new finalizer with the given dependencies. +func newFinalizer( + chain *eth.Ethereum, + state *chain.State, + acceptor BlockAcceptor, + verDB *versiondb.Database, + metadataDB database.Database, + extender syncpkg.Extender, + lastAcceptedHeight uint64, +) *finalizer { + return &finalizer{ + chain: chain, + state: state, + acceptor: acceptor, + verDB: verDB, + metadataDB: metadataDB, + extender: extender, + lastAcceptedHeight: lastAcceptedHeight, + } +} + +// finalize updates disk and memory pointers so the VM is prepared for bootstrapping. +// Executes any shared memory operations from the atomic trie to shared memory. +func (f *finalizer) finalize(ctx context.Context, summary message.Syncable) error { + stateBlock, err := f.state.GetBlock(ctx, ids.ID(summary.GetBlockHash())) + if err != nil { + return fmt.Errorf("%w: hash=%s", errBlockNotFound, summary.GetBlockHash()) + } + + wrapper, ok := stateBlock.(*chain.BlockWrapper) + if !ok { + return fmt.Errorf("%w: got %T, want *chain.BlockWrapper", errInvalidBlockType, stateBlock) + } + wrappedBlock := wrapper.Block + + evmBlockGetter, ok := wrappedBlock.(EthBlockWrapper) + if !ok { + return fmt.Errorf("%w: got %T, want EthBlockWrapper", errInvalidBlockType, wrappedBlock) + } + + block := evmBlockGetter.GetEthBlock() + + if block.Hash() != summary.GetBlockHash() { + return fmt.Errorf("%w: got %s, want %s", errBlockHashMismatch, block.Hash(), summary.GetBlockHash()) + } + if block.NumberU64() != summary.Height() { + return fmt.Errorf("%w: got %d, want %d", errBlockHeightMismatch, block.NumberU64(), summary.Height()) + } + + // BloomIndexer needs to know that some parts of the chain are not available + // and cannot be indexed. This is done by calling [AddCheckpoint] here. + // Since the indexer uses sections of size [params.BloomBitsBlocks] (= 4096), + // each block is indexed in section number [blockNumber/params.BloomBitsBlocks]. + // To allow the indexer to start with the block we just synced to, + // we create a checkpoint for its parent. + // Note: This requires assuming the synced block height is divisible + // by [params.BloomBitsBlocks]. + parentHeight := block.NumberU64() - 1 + parentHash := block.ParentHash() + f.chain.BloomIndexer().AddCheckpoint(parentHeight/params.BloomBitsBlocks, parentHash) + + if err := ctx.Err(); err != nil { + return fmt.Errorf("%w: %w", errFinalizeCancelled, err) + } + if err := f.chain.BlockChain().ResetToStateSyncedBlock(block); err != nil { + return err + } + + if f.extender != nil { + if err := f.extender.OnFinishBeforeCommit(f.lastAcceptedHeight, summary); err != nil { + return err + } + } + + if err := f.commitMarkers(summary); err != nil { + return fmt.Errorf("%w: height=%d, hash=%s: %w", errCommitMarkers, block.NumberU64(), block.Hash(), err) + } + + if err := f.state.SetLastAcceptedBlock(wrappedBlock); err != nil { + return err + } + + if f.extender != nil { + if err := f.extender.OnFinishAfterCommit(block.NumberU64()); err != nil { + return err + } + } + + return nil +} + +// commitMarkers updates VM database markers atomically. +func (f *finalizer) commitMarkers(summary message.Syncable) error { + id := ids.ID(summary.GetBlockHash()) + if err := f.acceptor.PutLastAcceptedID(id); err != nil { + return err + } + if err := f.metadataDB.Delete(stateSyncSummaryKey); err != nil { + return err + } + return f.verDB.Commit() +} diff --git a/graft/coreth/plugin/evm/vmsync/strategy_static.go b/graft/coreth/plugin/evm/vmsync/strategy_static.go new file mode 100644 index 000000000000..ba1d6c7aae67 --- /dev/null +++ b/graft/coreth/plugin/evm/vmsync/strategy_static.go @@ -0,0 +1,38 @@ +// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package vmsync + +import ( + "context" + + "github.com/ava-labs/avalanchego/graft/coreth/plugin/evm/message" +) + +var _ SyncStrategy = (*staticStrategy)(nil) + +// staticStrategy runs syncers sequentially without block queueing. +// This is the default sync mode where all syncers complete before +// finalization, with no concurrent block processing. +type staticStrategy struct { + registry *SyncerRegistry + finalizer *finalizer + summary message.Syncable +} + +func newStaticStrategy(registry *SyncerRegistry, finalizer *finalizer, summary message.Syncable) *staticStrategy { + return &staticStrategy{ + registry: registry, + finalizer: finalizer, + summary: summary, + } +} + +// Start begins the sync process and blocks until completion or error. +// For static sync, this runs all syncers and then finalizes the VM state. +func (s *staticStrategy) Start(ctx context.Context) error { + if err := s.registry.RunSyncerTasks(ctx, s.summary); err != nil { + return err + } + return s.finalizer.finalize(ctx, s.summary) +} From f8ff4c34fec7a2024fe3c9ae1e411cb5c1e2206a Mon Sep 17 00:00:00 2001 From: Tsvetan Dimitrov Date: Wed, 29 Oct 2025 14:40:55 +0200 Subject: [PATCH 02/20] feat(vmsync): dynamic state sync with coordinator, pivot cadence, and engine-driven target updates - Add Coordinator to orchestrate dynamic state sync, enforce pivot cadence, and manage queue execution. - Introduce engine hook OnEngineAccept to enqueue accepted blocks and advance the sync target. - Implement pivot policy (every N blocks) and idempotence (skip behind/equal, allow same-height reorgs). resolves #1259 Signed-off-by: Tsvetan Dimitrov (tsvetan.dimitrov@avalabs.org) --- graft/coreth/plugin/evm/atomic/sync/syncer.go | 4 + graft/coreth/plugin/evm/vmsync/block_queue.go | 44 ++++ graft/coreth/plugin/evm/vmsync/client.go | 7 + graft/coreth/plugin/evm/vmsync/coordinator.go | 194 ++++++++++++++++++ .../coreth/plugin/evm/vmsync/doubles_test.go | 7 +- graft/coreth/plugin/evm/vmsync/pivot.go | 52 +++++ graft/coreth/plugin/evm/vmsync/registry.go | 12 ++ .../coreth/plugin/evm/vmsync/registry_test.go | 8 +- graft/coreth/plugin/evm/vmsync/sync_target.go | 50 +++++ graft/coreth/plugin/evm/wrapped_block.go | 20 +- graft/coreth/sync/blocksync/syncer.go | 6 + graft/coreth/sync/statesync/code_syncer.go | 4 + graft/coreth/sync/statesync/state_syncer.go | 5 + graft/coreth/sync/types.go | 3 + 14 files changed, 409 insertions(+), 7 deletions(-) create mode 100644 graft/coreth/plugin/evm/vmsync/block_queue.go create mode 100644 graft/coreth/plugin/evm/vmsync/coordinator.go create mode 100644 graft/coreth/plugin/evm/vmsync/pivot.go create mode 100644 graft/coreth/plugin/evm/vmsync/sync_target.go diff --git a/graft/coreth/plugin/evm/atomic/sync/syncer.go b/graft/coreth/plugin/evm/atomic/sync/syncer.go index 17099e545838..8435c98904ea 100644 --- a/graft/coreth/plugin/evm/atomic/sync/syncer.go +++ b/graft/coreth/plugin/evm/atomic/sync/syncer.go @@ -146,6 +146,10 @@ func (s *Syncer) Sync(ctx context.Context) error { return s.syncer.Sync(ctx) } +func (*Syncer) UpdateTarget(_ message.Syncable) error { + return nil +} + // addZeroes returns the big-endian representation of `height`, prefixed with [common.HashLength] zeroes. func addZeroes(height uint64) []byte { // Key format is [height(8 bytes)][blockchainID(32 bytes)]. Start should be the diff --git a/graft/coreth/plugin/evm/vmsync/block_queue.go b/graft/coreth/plugin/evm/vmsync/block_queue.go new file mode 100644 index 000000000000..f16f76445325 --- /dev/null +++ b/graft/coreth/plugin/evm/vmsync/block_queue.go @@ -0,0 +1,44 @@ +// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package vmsync + +import "sync" + +// blockQueue buffers blocks that arrive while the coordinator is in the Running +// state. It is cleared (drained) on UpdateSyncTarget to avoid drops and is +// snapshotted at finalization via DequeueBatch. Enqueue is always allowed; a +// DequeueBatch only captures the current buffered blocks and clears them, and +// new enqueues after the snapshot are not part of that batch. +type blockQueue struct { + mu sync.Mutex + // buffered blocks accumulated before finalization + items []EthBlockWrapper +} + +// newBlockQueue creates a new empty queue. +func newBlockQueue() *blockQueue { + return &blockQueue{} +} + +// Enqueue appends a block to the buffer. Returns true if the block was queued, +// false if the block is nil. +func (q *blockQueue) Enqueue(b EthBlockWrapper) bool { + if b == nil { + return false + } + q.mu.Lock() + defer q.mu.Unlock() + q.items = append(q.items, b) + return true +} + +// DequeueBatch returns the current buffered blocks and clears the buffer. New +// arrivals after the snapshot are not included and remain buffered for later. +func (q *blockQueue) DequeueBatch() []EthBlockWrapper { + q.mu.Lock() + defer q.mu.Unlock() + out := q.items + q.items = nil + return out +} diff --git a/graft/coreth/plugin/evm/vmsync/client.go b/graft/coreth/plugin/evm/vmsync/client.go index efd7c4a72076..90d9afadd56a 100644 --- a/graft/coreth/plugin/evm/vmsync/client.go +++ b/graft/coreth/plugin/evm/vmsync/client.go @@ -74,6 +74,11 @@ type ClientConfig struct { RequestSize uint16 // number of key/value pairs to ask peers for per request Enabled bool SkipResume bool + // DynamicStateSyncEnabled toggles dynamic vs static state sync orchestration. + DynamicStateSyncEnabled bool + + // PivotInterval advances the sync target every N blocks. + PivotInterval uint64 } type client struct { @@ -100,6 +105,8 @@ type Client interface { ClearOngoingSummary() error Shutdown() error Error() error + // OnEngineAccept should be called by the engine when a block is accepted. + OnEngineAccept(EthBlockWrapper) error } // StateSyncEnabled returns [client.enabled], which is set in the chain's config file. diff --git a/graft/coreth/plugin/evm/vmsync/coordinator.go b/graft/coreth/plugin/evm/vmsync/coordinator.go new file mode 100644 index 000000000000..4cc6f077938f --- /dev/null +++ b/graft/coreth/plugin/evm/vmsync/coordinator.go @@ -0,0 +1,194 @@ +// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package vmsync + +import ( + "context" + "errors" + "sync" + "sync/atomic" + + "github.com/ava-labs/libevm/libevm/options" + + "github.com/ava-labs/avalanchego/graft/coreth/plugin/evm/message" +) + +// State represents the lifecycle phases of dynamic state sync orchestration. +type State int + +const ( + StateIdle State = iota + StateInitializing + StateRunning + StateFinalizing + StateExecutingBatch + StateCompleted + StateAborted +) + +var ( + errInvalidTargetType = errors.New("invalid target type") + errInvalidState = errors.New("invalid coordinator state") +) + +// Callbacks allows the coordinator to delegate VM-specific work back to the client. +type Callbacks struct { + // FinalizeVM performs the same actions as finishSync/commitVMMarkers in the client. + FinalizeVM func(target message.Syncable) error + // ApplyBlock processes a single block via the normal import path. + // Uses ExecutableBlock to allow extension-aware execution while avoiding imports here. + ApplyBlock func(EthBlockWrapper) error + // OnDone is called when the coordinator finishes (successfully or with error). + OnDone func(err error) +} + +// Coordinator orchestrates dynamic state sync across multiple syncers. +type Coordinator struct { + // state is managed atomically to allow cheap concurrent checks/updates. + state atomic.Int32 + // target stores the current target [message.Syncable] when [Coordinator.UpdateSyncTarget] is called. + target atomic.Value + queue *blockQueue + syncerRegistry *SyncerRegistry + callbacks Callbacks + // doneOnce ensures [Callbacks.OnDone] is invoked at most once. + doneOnce sync.Once + // cancel cancels the syncers' context (with cause) when aborting or finishing. + cancel context.CancelCauseFunc + + // pivot policy to throttle [Coordinator.UpdateSyncTarget] calls. + pivot *pivot +} + +// CoordinatorOption follows the functional options pattern for Coordinator. +type CoordinatorOption = options.Option[Coordinator] + +// WithPivotInterval configures the interval-based pivot policy. 0 disables it. +func WithPivotInterval(interval uint64) CoordinatorOption { + return options.Func[Coordinator](func(co *Coordinator) { + if interval == 0 { + co.pivot = nil + return + } + co.pivot = newPivotPolicy(interval) + }) +} + +// NewCoordinator constructs a coordinator to orchestrate dynamic state sync across multiple syncers. +func NewCoordinator(syncerRegistry *SyncerRegistry, cbs Callbacks, opts ...CoordinatorOption) *Coordinator { + co := &Coordinator{ + queue: newBlockQueue(), + syncerRegistry: syncerRegistry, + callbacks: cbs, + } + + options.ApplyTo(co, opts...) + + co.state.Store(int32(StateIdle)) + + return co +} + +// Start launches all syncers and returns immediately. Failures are monitored +// in the background and will transition to [StateAborted]. +func (co *Coordinator) Start(ctx context.Context, initial message.Syncable) { + co.state.Store(int32(StateInitializing)) + co.target.Store(initial) + + cctx, cancel := context.WithCancelCause(ctx) + co.cancel = cancel + g := co.syncerRegistry.StartAsync(cctx, initial) + + co.state.Store(int32(StateRunning)) + + go func() { + if err := g.Wait(); err != nil { + co.finish(err) + return + } + // All syncers finished successfully: finalize VM and execute the queued batch + if err := co.ApplyQueuedBatch(); err != nil { + co.finish(err) + return + } + co.finish(nil) + }() +} + +// ApplyQueuedBatch finalizes the VM at the current target and applies the +// current queued batch from the queue. Intended to be called after a +// target update cycle when it's time to apply the queued blocks. +func (co *Coordinator) ApplyQueuedBatch() error { + co.state.Store(int32(StateFinalizing)) + if co.callbacks.FinalizeVM != nil { + loaded := co.target.Load() + current, ok := loaded.(message.Syncable) + if !ok { + return errInvalidTargetType + } + if err := co.callbacks.FinalizeVM(current); err != nil { + return err + } + } + co.state.Store(int32(StateExecutingBatch)) + if co.queue != nil && co.callbacks.ApplyBlock != nil { + for _, b := range co.queue.DequeueBatch() { + if err := co.callbacks.ApplyBlock(b); err != nil { + return err + } + } + } + return nil +} + +// UpdateSyncTarget broadcasts a new target to all updatable syncers. +// It is only valid in the [StateRunning] state. +// Note: no batch execution occurs here. Batches are only executed after +// finalization. +func (co *Coordinator) UpdateSyncTarget(newTarget message.Syncable) error { + if co.CurrentState() != StateRunning { + return errInvalidState + } + // Respect pivot policy if configured. + if co.pivot != nil && !co.pivot.shouldForward(newTarget.Height()) { + return nil + } + co.target.Store(newTarget) + + if err := co.syncerRegistry.UpdateSyncTarget(newTarget); err != nil { + return err + } + if co.pivot != nil { + co.pivot.advance() + } + return nil +} + +// AddBlock appends the block to the queue only while in the Running state. +// Returns true if the block was queued, false if the queue was already sealed +// or the block is nil. +func (co *Coordinator) AddBlock(b EthBlockWrapper) bool { + if b == nil || co.CurrentState() != StateRunning { + return false + } + return co.queue.Enqueue(b) +} + +func (co *Coordinator) CurrentState() State { + return State(co.state.Load()) +} + +func (co *Coordinator) finish(err error) { + if err != nil { + co.state.Store(int32(StateAborted)) + } else { + co.state.Store(int32(StateCompleted)) + } + if co.cancel != nil { + co.cancel(err) + } + if co.callbacks.OnDone != nil { + co.doneOnce.Do(func() { co.callbacks.OnDone(err) }) + } +} diff --git a/graft/coreth/plugin/evm/vmsync/doubles_test.go b/graft/coreth/plugin/evm/vmsync/doubles_test.go index 7c79095df6c7..443ceb8bcdbc 100644 --- a/graft/coreth/plugin/evm/vmsync/doubles_test.go +++ b/graft/coreth/plugin/evm/vmsync/doubles_test.go @@ -9,6 +9,8 @@ import ( "sync" "time" + "github.com/ava-labs/avalanchego/graft/coreth/plugin/evm/message" + syncpkg "github.com/ava-labs/avalanchego/graft/coreth/sync" ) @@ -22,8 +24,9 @@ type FuncSyncer struct { func (f FuncSyncer) Sync(ctx context.Context) error { return f.fn(ctx) } // Name returns the provided name or a default if unspecified. -func (FuncSyncer) Name() string { return "Test Name" } -func (FuncSyncer) ID() string { return "test_id" } +func (FuncSyncer) Name() string { return "Test Name" } +func (FuncSyncer) ID() string { return "test_id" } +func (FuncSyncer) UpdateTarget(_ message.Syncable) error { return nil } var _ syncpkg.Syncer = FuncSyncer{} diff --git a/graft/coreth/plugin/evm/vmsync/pivot.go b/graft/coreth/plugin/evm/vmsync/pivot.go new file mode 100644 index 000000000000..32cb6fe79b9d --- /dev/null +++ b/graft/coreth/plugin/evm/vmsync/pivot.go @@ -0,0 +1,52 @@ +// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package vmsync + +import "sync/atomic" + +// pivot encapsulates the logic for deciding when to forward +// a new sync target based on a fixed block-height interval. It is +// safe for concurrent use. +type pivot struct { + interval uint64 + // nextHeight is the next height threshold at or beyond which we + // should forward an update. A value of 0 means uninitialized. + nextHeight uint64 // accessed atomically +} + +func newPivotPolicy(interval uint64) *pivot { + return &pivot{interval: interval} +} + +// shouldForward reports whether a summary at the given height should be +// forwarded, initializing the next threshold on first use. When it returns +// true, callers should follow up with advance(). +func (p *pivot) shouldForward(height uint64) bool { + if p == nil || p.interval == 0 { + return true + } + next := atomic.LoadUint64(&p.nextHeight) + if next == 0 { + // Round up the initial height to the next multiple of interval. + // Ceil division: ((h + interval - 1) / interval) * interval + h := height + init := ((h + p.interval - 1) / p.interval) * p.interval + // Initialize once - if another goroutine wins, read the established value. + if !atomic.CompareAndSwapUint64(&p.nextHeight, 0, init) { + next = atomic.LoadUint64(&p.nextHeight) + } else { + next = init + } + } + return height >= next +} + +// advance moves the next threshold forward by one interval. Call this +// only after shouldForward has returned true and the update was issued. +func (p *pivot) advance() { + if p == nil || p.interval == 0 { + return + } + atomic.AddUint64(&p.nextHeight, p.interval) +} diff --git a/graft/coreth/plugin/evm/vmsync/registry.go b/graft/coreth/plugin/evm/vmsync/registry.go index 5e9bc6031c3b..071352933ec6 100644 --- a/graft/coreth/plugin/evm/vmsync/registry.go +++ b/graft/coreth/plugin/evm/vmsync/registry.go @@ -102,3 +102,15 @@ func (r *SyncerRegistry) StartAsync(ctx context.Context, summary message.Syncabl return g } + +// UpdateSyncTarget updates the sync target for all syncers. +func (r *SyncerRegistry) UpdateSyncTarget(newTarget message.Syncable) error { + for _, task := range r.syncers { + if err := task.syncer.UpdateTarget(newTarget); err != nil { + log.Error("failed updating sync target", "name", task.name, "err", err) + return err + } + log.Info("updated sync target", "name", task.name, "new_target", newTarget.GetBlockHash().Hex(), "height", newTarget.Height()) + } + return nil +} diff --git a/graft/coreth/plugin/evm/vmsync/registry_test.go b/graft/coreth/plugin/evm/vmsync/registry_test.go index 55f17d15d01f..f29c8e5569e5 100644 --- a/graft/coreth/plugin/evm/vmsync/registry_test.go +++ b/graft/coreth/plugin/evm/vmsync/registry_test.go @@ -38,8 +38,9 @@ func (m *mockSyncer) Sync(context.Context) error { return m.syncError } -func (m *mockSyncer) Name() string { return m.name } -func (m *mockSyncer) ID() string { return m.name } +func (m *mockSyncer) Name() string { return m.name } +func (m *mockSyncer) ID() string { return m.name } +func (*mockSyncer) UpdateTarget(_ message.Syncable) error { return nil } // namedSyncer adapts an existing syncer with a provided name to satisfy Syncer with Name(). type namedSyncer struct { @@ -50,6 +51,9 @@ type namedSyncer struct { func (n *namedSyncer) Sync(ctx context.Context) error { return n.syncer.Sync(ctx) } func (n *namedSyncer) Name() string { return n.name } func (n *namedSyncer) ID() string { return n.name } +func (n *namedSyncer) UpdateTarget(newTarget message.Syncable) error { + return n.syncer.UpdateTarget(newTarget) +} // syncerConfig describes a test syncer setup for RunSyncerTasks table tests. type syncerConfig struct { diff --git a/graft/coreth/plugin/evm/vmsync/sync_target.go b/graft/coreth/plugin/evm/vmsync/sync_target.go new file mode 100644 index 000000000000..32c7850ecc56 --- /dev/null +++ b/graft/coreth/plugin/evm/vmsync/sync_target.go @@ -0,0 +1,50 @@ +// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package vmsync + +import ( + "context" + + "github.com/ava-labs/avalanchego/ids" + "github.com/ava-labs/avalanchego/snow/engine/snowman/block" + "github.com/ava-labs/libevm/common" + + "github.com/ava-labs/avalanchego/graft/coreth/plugin/evm/message" +) + +var _ message.Syncable = (*syncTarget)(nil) + +// syncTarget is a minimal implementation of message.Syncable used internally +// to advance the coordinator's sync target from engine-accepted blocks. +type syncTarget struct { + id ids.ID + hash common.Hash + root common.Hash + height uint64 +} + +// Build a sync target from basic fields. +func newSyncTarget(hash common.Hash, root common.Hash, height uint64) message.Syncable { + var id ids.ID + copy(id[:], hash[:]) + return &syncTarget{ + id: id, + hash: hash, + root: root, + height: height, + } +} + +// message.Syncable +func (s *syncTarget) GetBlockHash() common.Hash { return s.hash } +func (s *syncTarget) GetBlockRoot() common.Hash { return s.root } + +// block.StateSummary +func (s *syncTarget) ID() ids.ID { return s.id } +func (s *syncTarget) Height() uint64 { return s.height } +func (s *syncTarget) Bytes() []byte { return s.hash.Bytes() } +func (*syncTarget) Accept(context.Context) (block.StateSyncMode, error) { + // When used internally to advance targets, we always handle dynamically. + return block.StateSyncDynamic, nil +} diff --git a/graft/coreth/plugin/evm/wrapped_block.go b/graft/coreth/plugin/evm/wrapped_block.go index 2db44fa8b263..62debf36685f 100644 --- a/graft/coreth/plugin/evm/wrapped_block.go +++ b/graft/coreth/plugin/evm/wrapped_block.go @@ -130,11 +130,25 @@ func (b *wrappedBlock) Accept(context.Context) error { // Apply any changes atomically with other pending changes to // the vm's versionDB. // Accept flushes the changes in the batch to the database. - return b.extension.Accept(vdbBatch) + if err := b.extension.Accept(vdbBatch); err != nil { + return err + } + } else { + // If there is no extension, we still need to apply the changes to the versionDB + if err := vdbBatch.Write(); err != nil { + return err + } } - // If there is no extension, we still need to apply the changes to the versionDB - return vdbBatch.Write() + // Notify sync client that engine accepted a block + // TODO(powerslider): probably there could be a better way to do this, but it should be wired here for now. + if client := vm.SyncerClient(); client != nil { + if err := client.OnEngineAccept(b); err != nil { + return fmt.Errorf("could not notify sync client that engine accepted a block: %w", err) + } + } + + return nil } // handlePrecompileAccept calls Accept on any logs generated with an active precompile address that implements diff --git a/graft/coreth/sync/blocksync/syncer.go b/graft/coreth/sync/blocksync/syncer.go index 2f7340741234..0ffa9caa5780 100644 --- a/graft/coreth/sync/blocksync/syncer.go +++ b/graft/coreth/sync/blocksync/syncer.go @@ -13,6 +13,8 @@ import ( "github.com/ava-labs/libevm/ethdb" "github.com/ava-labs/libevm/log" + "github.com/ava-labs/avalanchego/graft/coreth/plugin/evm/message" + syncpkg "github.com/ava-labs/avalanchego/graft/coreth/sync" statesyncclient "github.com/ava-labs/avalanchego/graft/coreth/sync/client" ) @@ -112,3 +114,7 @@ func (s *BlockSyncer) Sync(ctx context.Context) error { log.Info("fetched blocks from peer", "total", blocksToFetch) return batch.Write() } + +func (*BlockSyncer) UpdateTarget(_ message.Syncable) error { + return nil +} diff --git a/graft/coreth/sync/statesync/code_syncer.go b/graft/coreth/sync/statesync/code_syncer.go index 1ac8c47b5d71..98941e61968b 100644 --- a/graft/coreth/sync/statesync/code_syncer.go +++ b/graft/coreth/sync/statesync/code_syncer.go @@ -115,6 +115,10 @@ func (c *CodeSyncer) Sync(ctx context.Context) error { return eg.Wait() } +func (*CodeSyncer) UpdateTarget(_ message.Syncable) error { + return nil +} + // work fulfills any incoming requests from the producer channel by fetching code bytes from the network // and fulfilling them by updating the database. func (c *CodeSyncer) work(ctx context.Context) error { diff --git a/graft/coreth/sync/statesync/state_syncer.go b/graft/coreth/sync/statesync/state_syncer.go index a286431dd1aa..54b741de9c33 100644 --- a/graft/coreth/sync/statesync/state_syncer.go +++ b/graft/coreth/sync/statesync/state_syncer.go @@ -17,6 +17,7 @@ import ( "golang.org/x/sync/errgroup" "github.com/ava-labs/avalanchego/graft/coreth/core/state/snapshot" + "github.com/ava-labs/avalanchego/graft/coreth/plugin/evm/message" syncpkg "github.com/ava-labs/avalanchego/graft/coreth/sync" syncclient "github.com/ava-labs/avalanchego/graft/coreth/sync/client" @@ -167,6 +168,10 @@ func (t *stateSync) Sync(ctx context.Context) error { return eg.Wait() } +func (*stateSync) UpdateTarget(_ message.Syncable) error { + return nil +} + // onStorageTrieFinished is called after a storage trie finishes syncing. func (t *stateSync) onStorageTrieFinished(root common.Hash) error { <-t.triesInProgressSem // allow another trie to start (release the semaphore) diff --git a/graft/coreth/sync/types.go b/graft/coreth/sync/types.go index 04b8b75bdf77..55c4a2b31750 100644 --- a/graft/coreth/sync/types.go +++ b/graft/coreth/sync/types.go @@ -29,6 +29,9 @@ type Syncer interface { // "state_evm_state_sync", "state_atomic_sync"). Implementations should ensure this is unique and // stable across renames for logging/metrics/deduplication. ID() string + + // UpdateTarget updates the syncer's target while running to support dynamic state sync. + UpdateTarget(newTarget message.Syncable) error } // SummaryProvider is an interface for providing state summaries. From 6277327c5972c9ff8270425b9e1f302a4e80a8b2 Mon Sep 17 00:00:00 2001 From: Tsvetan Dimitrov Date: Fri, 31 Oct 2025 15:58:20 +0200 Subject: [PATCH 03/20] fix: move OnEngineAccept --- graft/coreth/plugin/evm/vmsync/client.go | 54 ++++++++++++++++--- graft/coreth/plugin/evm/vmsync/coordinator.go | 2 +- .../plugin/evm/vmsync/strategy_static.go | 20 +++++-- graft/coreth/plugin/evm/wrapped_block.go | 16 +++--- 4 files changed, 75 insertions(+), 17 deletions(-) diff --git a/graft/coreth/plugin/evm/vmsync/client.go b/graft/coreth/plugin/evm/vmsync/client.go index 90d9afadd56a..5176537aaa37 100644 --- a/graft/coreth/plugin/evm/vmsync/client.go +++ b/graft/coreth/plugin/evm/vmsync/client.go @@ -44,10 +44,19 @@ type BlockAcceptor interface { } // SyncStrategy defines how state sync is executed. -// Implementations handle the sync lifecycle differently based on sync mode. +// Implementations handle syncer orchestration and block processing during sync. type SyncStrategy interface { - // Start begins the sync process and blocks until completion or error. - Start(ctx context.Context) error + // Start begins sync and blocks until completion or error. + Start(ctx context.Context, summary message.Syncable) error + + // OnBlockAccepted handles a block accepted during sync. + OnBlockAccepted(EthBlockWrapper) (bool, error) + + // OnBlockRejected handles a block rejected during sync. + OnBlockRejected(EthBlockWrapper) (bool, error) + + // OnBlockVerified handles a block verified during sync. + OnBlockVerified(EthBlockWrapper) (bool, error) } type ClientConfig struct { @@ -84,6 +93,7 @@ type ClientConfig struct { type client struct { config *ClientConfig resumableSummary message.Syncable + strategy SyncStrategy // strategy manages sync execution (static or dynamic) cancel context.CancelFunc wg sync.WaitGroup err error @@ -106,7 +116,14 @@ type Client interface { Shutdown() error Error() error // OnEngineAccept should be called by the engine when a block is accepted. - OnEngineAccept(EthBlockWrapper) error + // Returns true if the block was enqueued for deferred processing, false otherwise. + OnEngineAccept(EthBlockWrapper) (bool, error) + // OnEngineReject should be called by the engine when a block is rejected. + // Returns true if the block was enqueued for deferred processing, false otherwise. + OnEngineReject(EthBlockWrapper) (bool, error) + // OnEngineVerify should be called by the engine when a block is verified. + // Returns true if the block was enqueued for deferred processing, false otherwise. + OnEngineVerify(EthBlockWrapper) (bool, error) } // StateSyncEnabled returns [client.enabled], which is set in the chain's config file. @@ -152,6 +169,31 @@ func (c *client) ParseStateSummary(_ context.Context, summaryBytes []byte) (bloc return c.config.Parser.Parse(summaryBytes, c.acceptSyncSummary) } +// OnEngineAccept delegates to the strategy if active. +func (c *client) OnEngineAccept(b EthBlockWrapper) (bool, error) { + if c.strategy == nil { + return false, nil + } + return c.strategy.OnBlockAccepted(b) +} + +// OnEngineReject delegates to the strategy if active. +func (c *client) OnEngineReject(b EthBlockWrapper) (bool, error) { + if c.strategy == nil { + return false, nil + } + return c.strategy.OnBlockRejected(b) +} + +// OnEngineVerify delegates to the strategy if active. +func (c *client) OnEngineVerify(b EthBlockWrapper) (bool, error) { + if c.strategy == nil { + return false, nil + } + return c.strategy.OnBlockVerified(b) +} + + // acceptSyncSummary returns true if sync will be performed and launches the state sync process // in a goroutine. func (c *client) acceptSyncSummary(summary message.Syncable) (block.StateSyncMode, error) { @@ -177,7 +219,7 @@ func (c *client) acceptSyncSummary(summary message.Syncable) (block.StateSyncMod c.config.LastAcceptedHeight, ) - strategy := newStaticStrategy(registry, finalizer, summary) + strategy := newStaticStrategy(registry, finalizer) return c.startAsync(strategy), nil } @@ -235,7 +277,7 @@ func (c *client) startAsync(strategy SyncStrategy) block.StateSyncMode { defer c.wg.Done() defer cancel() - if err := strategy.Start(ctx); err != nil { + if err := strategy.Start(ctx, c.resumableSummary); err != nil { c.err = err } // notify engine regardless of whether err == nil, diff --git a/graft/coreth/plugin/evm/vmsync/coordinator.go b/graft/coreth/plugin/evm/vmsync/coordinator.go index 4cc6f077938f..b8920f84ea87 100644 --- a/graft/coreth/plugin/evm/vmsync/coordinator.go +++ b/graft/coreth/plugin/evm/vmsync/coordinator.go @@ -107,7 +107,7 @@ func (co *Coordinator) Start(ctx context.Context, initial message.Syncable) { co.finish(err) return } - // All syncers finished successfully: finalize VM and execute the queued batch + // All syncers finished successfully: finalize VM and execute the queued batch. if err := co.ApplyQueuedBatch(); err != nil { co.finish(err) return diff --git a/graft/coreth/plugin/evm/vmsync/strategy_static.go b/graft/coreth/plugin/evm/vmsync/strategy_static.go index ba1d6c7aae67..b5a8f88c4518 100644 --- a/graft/coreth/plugin/evm/vmsync/strategy_static.go +++ b/graft/coreth/plugin/evm/vmsync/strategy_static.go @@ -20,19 +20,33 @@ type staticStrategy struct { summary message.Syncable } -func newStaticStrategy(registry *SyncerRegistry, finalizer *finalizer, summary message.Syncable) *staticStrategy { +func newStaticStrategy(registry *SyncerRegistry, finalizer *finalizer) *staticStrategy { return &staticStrategy{ registry: registry, finalizer: finalizer, - summary: summary, } } // Start begins the sync process and blocks until completion or error. // For static sync, this runs all syncers and then finalizes the VM state. -func (s *staticStrategy) Start(ctx context.Context) error { +func (s *staticStrategy) Start(ctx context.Context, summary message.Syncable) error { if err := s.registry.RunSyncerTasks(ctx, s.summary); err != nil { return err } return s.finalizer.finalize(ctx, s.summary) } + +// OnBlockAccepted is a no-op for static sync since blocks are not queued. +func (s *staticStrategy) OnBlockAccepted(EthBlockWrapper) (bool, error) { + return false, nil +} + +// OnBlockRejected is a no-op for static sync since blocks are not queued. +func (s *staticStrategy) OnBlockRejected(EthBlockWrapper) (bool, error) { + return false, nil +} + +// OnBlockVerified is a no-op for static sync since blocks are not queued. +func (s *staticStrategy) OnBlockVerified(EthBlockWrapper) (bool, error) { + return false, nil +} diff --git a/graft/coreth/plugin/evm/wrapped_block.go b/graft/coreth/plugin/evm/wrapped_block.go index 62debf36685f..d87dbce906d6 100644 --- a/graft/coreth/plugin/evm/wrapped_block.go +++ b/graft/coreth/plugin/evm/wrapped_block.go @@ -97,6 +97,14 @@ func (b *wrappedBlock) Accept(context.Context) error { // practice to cleanup the batch we were modifying in the case of an error. defer vm.versiondb.Abort() + // Notify sync client that engine accepted a block + // TODO(powerslider): probably there could be a better way to do this, but it should be wired here for now. + if client := vm.SyncerClient(); client != nil { + if err := client.OnEngineAccept(b); err != nil { + return fmt.Errorf("could not notify sync client that engine accepted a block: %w", err) + } + } + blkID := b.ID() log.Debug("accepting block", "hash", blkID.Hex(), @@ -140,13 +148,7 @@ func (b *wrappedBlock) Accept(context.Context) error { } } - // Notify sync client that engine accepted a block - // TODO(powerslider): probably there could be a better way to do this, but it should be wired here for now. - if client := vm.SyncerClient(); client != nil { - if err := client.OnEngineAccept(b); err != nil { - return fmt.Errorf("could not notify sync client that engine accepted a block: %w", err) - } - } + return nil } From 430bba6ed311aedbc83708ab8865630bf5b1cbf4 Mon Sep 17 00:00:00 2001 From: Tsvetan Dimitrov Date: Thu, 6 Nov 2025 20:22:06 +0200 Subject: [PATCH 04/20] feat(vmsync): remove stale blocks from queue after sync target updates When UpdateSyncTarget is called, remove all queued blocks with height <= new target height since they will never be executed. This prevents processing blocks that the sync has already advanced past. - Add RemoveBlocksBelowHeight method to blockQueue to filter stale blocks. - Call RemoveBlocksBelowHeight in UpdateSyncTarget after pivot check. - Support accept/reject/verify operations in block queue. - Add OnEngineReject and OnEngineVerify handlers to sync client. - Propagate context through ApplyQueuedBatch for proper cancellation. - Remove unnecessary defer vm.versiondb.Abort() from Accept. - Prevent recursion during batch execution via state check. - Make dequeueBatch private to reduce API surface. resolves #1259 Signed-off-by: Tsvetan Dimitrov (tsvetan.dimitrov@avalabs.org) --- graft/coreth/plugin/evm/vmsync/block_queue.go | 87 ++++++++++++++++--- graft/coreth/plugin/evm/vmsync/client.go | 12 +++ graft/coreth/plugin/evm/vmsync/coordinator.go | 31 +++---- graft/coreth/plugin/evm/vmsync/finalizer.go | 7 -- graft/coreth/plugin/evm/wrapped_block.go | 48 ++++++---- 5 files changed, 132 insertions(+), 53 deletions(-) diff --git a/graft/coreth/plugin/evm/vmsync/block_queue.go b/graft/coreth/plugin/evm/vmsync/block_queue.go index f16f76445325..1bb4dc161fe0 100644 --- a/graft/coreth/plugin/evm/vmsync/block_queue.go +++ b/graft/coreth/plugin/evm/vmsync/block_queue.go @@ -3,17 +3,36 @@ package vmsync -import "sync" +import ( + "context" + "sync" +) -// blockQueue buffers blocks that arrive while the coordinator is in the Running -// state. It is cleared (drained) on UpdateSyncTarget to avoid drops and is -// snapshotted at finalization via DequeueBatch. Enqueue is always allowed; a -// DequeueBatch only captures the current buffered blocks and clears them, and -// new enqueues after the snapshot are not part of that batch. +// BlockOperation represents the type of operation to perform on a block. +type BlockOperation int + +const ( + OpAccept BlockOperation = iota + OpReject + OpVerify +) + +// blockOperation represents a queued block operation. +type blockOperation struct { + block EthBlockWrapper + operation BlockOperation +} + +// blockQueue buffers block operations (accept/reject/verify) that arrive while +// the coordinator is in the Running state. Operations are processed in FIFO order. +// It is cleared (drained) on UpdateSyncTarget to avoid drops and is snapshotted +// at finalization via DequeueBatch. Enqueue is always allowed; a DequeueBatch +// only captures the current buffered operations and clears them, and new enqueues +// after the snapshot are not part of that batch. type blockQueue struct { mu sync.Mutex - // buffered blocks accumulated before finalization - items []EthBlockWrapper + // buffered operations accumulated before finalization + items []blockOperation } // newBlockQueue creates a new empty queue. @@ -21,24 +40,64 @@ func newBlockQueue() *blockQueue { return &blockQueue{} } -// Enqueue appends a block to the buffer. Returns true if the block was queued, -// false if the block is nil. -func (q *blockQueue) Enqueue(b EthBlockWrapper) bool { +// Enqueue appends a block operation to the buffer. Returns true if the operation +// was queued, false if the block is nil. +func (q *blockQueue) Enqueue(b EthBlockWrapper, op BlockOperation) bool { if b == nil { return false } q.mu.Lock() defer q.mu.Unlock() - q.items = append(q.items, b) + q.items = append(q.items, blockOperation{ + block: b, + operation: op, + }) return true } -// DequeueBatch returns the current buffered blocks and clears the buffer. New +// dequeueBatch returns the current buffered operations and clears the buffer. New // arrivals after the snapshot are not included and remain buffered for later. -func (q *blockQueue) DequeueBatch() []EthBlockWrapper { +func (q *blockQueue) dequeueBatch() []blockOperation { q.mu.Lock() defer q.mu.Unlock() out := q.items q.items = nil return out } + +// RemoveBlocksBelowHeight removes all queued blocks with height <= targetHeight. +// This is called after UpdateSyncTarget to remove blocks that will never be executed +// because the sync target has advanced past them. +func (q *blockQueue) RemoveBlocksBelowHeight(targetHeight uint64) { + q.mu.Lock() + defer q.mu.Unlock() + + filtered := q.items[:0] + for _, op := range q.items { + ethBlock := op.block.GetEthBlock() + if ethBlock != nil && ethBlock.NumberU64() > targetHeight { + filtered = append(filtered, op) + } + } + q.items = filtered +} + +// ProcessQueue processes all queued operations in FIFO order. +func (q *blockQueue) ProcessQueue(ctx context.Context) error { + operations := q.dequeueBatch() + for _, op := range operations { + var err error + switch op.operation { + case OpAccept: + err = op.block.Accept(ctx) + case OpReject: + err = op.block.Reject(ctx) + case OpVerify: + err = op.block.Verify(ctx) + } + if err != nil { + return err + } + } + return nil +} diff --git a/graft/coreth/plugin/evm/vmsync/client.go b/graft/coreth/plugin/evm/vmsync/client.go index 5176537aaa37..ac2133534fdf 100644 --- a/graft/coreth/plugin/evm/vmsync/client.go +++ b/graft/coreth/plugin/evm/vmsync/client.go @@ -9,6 +9,7 @@ import ( "fmt" "sync" + "github.com/ava-labs/libevm/core/types" "github.com/ava-labs/libevm/ethdb" "github.com/ava-labs/libevm/log" @@ -59,6 +60,17 @@ type SyncStrategy interface { OnBlockVerified(EthBlockWrapper) (bool, error) } +// EthBlockWrapper can be implemented by a concrete block wrapper type to +// return *types.Block, which is needed to update chain pointers at the +// end of the sync operation. It also provides Accept/Reject/Verify operations +// for deferred processing during dynamic state sync. +type EthBlockWrapper interface { + GetEthBlock() *types.Block + Accept(context.Context) error + Reject(context.Context) error + Verify(context.Context) error +} + type ClientConfig struct { Chain *eth.Ethereum State *chain.State diff --git a/graft/coreth/plugin/evm/vmsync/coordinator.go b/graft/coreth/plugin/evm/vmsync/coordinator.go index b8920f84ea87..c4abacde8299 100644 --- a/graft/coreth/plugin/evm/vmsync/coordinator.go +++ b/graft/coreth/plugin/evm/vmsync/coordinator.go @@ -36,9 +36,6 @@ var ( type Callbacks struct { // FinalizeVM performs the same actions as finishSync/commitVMMarkers in the client. FinalizeVM func(target message.Syncable) error - // ApplyBlock processes a single block via the normal import path. - // Uses ExecutableBlock to allow extension-aware execution while avoiding imports here. - ApplyBlock func(EthBlockWrapper) error // OnDone is called when the coordinator finishes (successfully or with error). OnDone func(err error) } @@ -108,7 +105,7 @@ func (co *Coordinator) Start(ctx context.Context, initial message.Syncable) { return } // All syncers finished successfully: finalize VM and execute the queued batch. - if err := co.ApplyQueuedBatch(); err != nil { + if err := co.ApplyQueuedBatch(cctx); err != nil { co.finish(err) return } @@ -116,10 +113,10 @@ func (co *Coordinator) Start(ctx context.Context, initial message.Syncable) { }() } -// ApplyQueuedBatch finalizes the VM at the current target and applies the -// current queued batch from the queue. Intended to be called after a -// target update cycle when it's time to apply the queued blocks. -func (co *Coordinator) ApplyQueuedBatch() error { +// ApplyQueuedBatch finalizes the VM at the current target and processes the +// queued operations in FIFO order. Intended to be called after a target update +// cycle when it's time to process the queued operations. +func (co *Coordinator) ApplyQueuedBatch(ctx context.Context) error { co.state.Store(int32(StateFinalizing)) if co.callbacks.FinalizeVM != nil { loaded := co.target.Load() @@ -132,11 +129,9 @@ func (co *Coordinator) ApplyQueuedBatch() error { } } co.state.Store(int32(StateExecutingBatch)) - if co.queue != nil && co.callbacks.ApplyBlock != nil { - for _, b := range co.queue.DequeueBatch() { - if err := co.callbacks.ApplyBlock(b); err != nil { - return err - } + if co.queue != nil { + if err := co.queue.ProcessQueue(ctx); err != nil { + return err } } return nil @@ -154,6 +149,12 @@ func (co *Coordinator) UpdateSyncTarget(newTarget message.Syncable) error { if co.pivot != nil && !co.pivot.shouldForward(newTarget.Height()) { return nil } + + // Remove blocks from queue that will never be executed (behind the new target). + if co.queue != nil { + co.queue.RemoveBlocksBelowHeight(newTarget.Height()) + } + co.target.Store(newTarget) if err := co.syncerRegistry.UpdateSyncTarget(newTarget); err != nil { @@ -168,11 +169,11 @@ func (co *Coordinator) UpdateSyncTarget(newTarget message.Syncable) error { // AddBlock appends the block to the queue only while in the Running state. // Returns true if the block was queued, false if the queue was already sealed // or the block is nil. -func (co *Coordinator) AddBlock(b EthBlockWrapper) bool { +func (co *Coordinator) AddBlockOperation(b EthBlockWrapper, op BlockOperation) bool { if b == nil || co.CurrentState() != StateRunning { return false } - return co.queue.Enqueue(b) + return co.queue.Enqueue(b, op) } func (co *Coordinator) CurrentState() State { diff --git a/graft/coreth/plugin/evm/vmsync/finalizer.go b/graft/coreth/plugin/evm/vmsync/finalizer.go index b803d4e6fb80..427e8c972656 100644 --- a/graft/coreth/plugin/evm/vmsync/finalizer.go +++ b/graft/coreth/plugin/evm/vmsync/finalizer.go @@ -21,13 +21,6 @@ import ( syncpkg "github.com/ava-labs/avalanchego/graft/coreth/sync" ) -// EthBlockWrapper can be implemented by a concrete block wrapper type to -// return *types.Block, which is needed to update chain pointers at the -// end of the sync operation. -type EthBlockWrapper interface { - GetEthBlock() *types.Block -} - var ( errBlockNotFound = errors.New("block not found in state") errInvalidBlockType = errors.New("invalid block wrapper type") diff --git a/graft/coreth/plugin/evm/wrapped_block.go b/graft/coreth/plugin/evm/wrapped_block.go index d87dbce906d6..1b6d3bfeaa01 100644 --- a/graft/coreth/plugin/evm/wrapped_block.go +++ b/graft/coreth/plugin/evm/wrapped_block.go @@ -55,6 +55,7 @@ var ( errParentBeaconRootNonEmpty = errors.New("invalid non-empty parentBeaconRoot") errBlobGasUsedNilInCancun = errors.New("blob gas used must not be nil in Cancun") errBlobsNotEnabled = errors.New("blobs not enabled on avalanche networks") + errCouldNotNotifySyncClient = errors.New("could not notify sync client") ) var ( @@ -91,15 +92,9 @@ func wrapBlock(ethBlock *types.Block, vm *VM) (*wrappedBlock, error) { func (b *wrappedBlock) ID() ids.ID { return b.id } // Accept implements the snowman.Block interface -func (b *wrappedBlock) Accept(context.Context) error { - vm := b.vm - // Although returning an error from Accept is considered fatal, it is good - // practice to cleanup the batch we were modifying in the case of an error. - defer vm.versiondb.Abort() - - // Notify sync client that engine accepted a block - // TODO(powerslider): probably there could be a better way to do this, but it should be wired here for now. - if client := vm.SyncerClient(); client != nil { +func (b *wrappedBlock) Accept(ctx context.Context) error { + // Notify sync client that engine accepted a block. + if client := b.vm.SyncerClient(); client != nil { if err := client.OnEngineAccept(b); err != nil { return fmt.Errorf("could not notify sync client that engine accepted a block: %w", err) } @@ -119,17 +114,17 @@ func (b *wrappedBlock) Accept(context.Context) error { if err := b.handlePrecompileAccept(rules); err != nil { return err } - if err := vm.blockChain.Accept(b.ethBlock); err != nil { + if err := b.vm.blockChain.Accept(b.ethBlock); err != nil { return fmt.Errorf("chain could not accept %s: %w", blkID, err) } - if err := vm.PutLastAcceptedID(blkID); err != nil { + if err := b.vm.PutLastAcceptedID(blkID); err != nil { return fmt.Errorf("failed to put %s as the last accepted block: %w", blkID, err) } // Get pending operations on the vm's versionDB so we can apply them atomically // with the block extension's changes. - vdbBatch, err := vm.versiondb.CommitBatch() + vdbBatch, err := b.vm.versiondb.CommitBatch() if err != nil { return fmt.Errorf("could not create commit batch processing block[%s]: %w", blkID, err) } @@ -148,8 +143,6 @@ func (b *wrappedBlock) Accept(context.Context) error { } } - - return nil } @@ -189,7 +182,14 @@ func (b *wrappedBlock) handlePrecompileAccept(rules extras.Rules) error { // Reject implements the snowman.Block interface // If [b] contains an atomic transaction, attempt to re-issue it -func (b *wrappedBlock) Reject(context.Context) error { +func (b *wrappedBlock) Reject(ctx context.Context) error { + // Notify sync client that engine rejected a block. + if client := b.vm.SyncerClient(); client != nil { + if err := client.OnEngineReject(b); err != nil { + return fmt.Errorf("%w: %w", errCouldNotNotifySyncClient, err) + } + } + blkID := b.ID() log.Debug("rejecting block", "hash", blkID.Hex(), @@ -221,7 +221,14 @@ func (b *wrappedBlock) Timestamp() time.Time { } // Verify implements the snowman.Block interface -func (b *wrappedBlock) Verify(context.Context) error { +func (b *wrappedBlock) Verify(ctx context.Context) error { + // Notify sync client that engine verified a block. + if client := b.vm.SyncerClient(); client != nil { + if err := client.OnEngineVerify(b); err != nil { + return fmt.Errorf("%w: %w", errCouldNotNotifySyncClient, err) + } + } + return b.verify(&precompileconfig.PredicateContext{ SnowCtx: b.vm.ctx, ProposerVMBlockCtx: nil, @@ -253,7 +260,14 @@ func (b *wrappedBlock) ShouldVerifyWithContext(context.Context) (bool, error) { } // VerifyWithContext implements the block.WithVerifyContext interface -func (b *wrappedBlock) VerifyWithContext(_ context.Context, proposerVMBlockCtx *block.Context) error { +func (b *wrappedBlock) VerifyWithContext(ctx context.Context, proposerVMBlockCtx *block.Context) error { + // Notify sync client that engine verified a block. + if client := b.vm.SyncerClient(); client != nil { + if err := client.OnEngineVerify(b); err != nil { + return fmt.Errorf("could not notify sync client that engine verified a block: %w", err) + } + } + return b.verify(&precompileconfig.PredicateContext{ SnowCtx: b.vm.ctx, ProposerVMBlockCtx: proposerVMBlockCtx, From f0b79bf4994b8b8904800e0ed887f6b8f85db402 Mon Sep 17 00:00:00 2001 From: Tsvetan Dimitrov Date: Fri, 7 Nov 2025 16:28:19 +0200 Subject: [PATCH 05/20] refactor(vmsync): propagate context throughout dynamic state sync flow - Add context parameter to finishSync() and propagate through stateSyncStatic/Dynamic - Add context parameter to FinalizeVM callback in Coordinator - Add context parameter to ProcessQueuedBlockOperations (renamed from ApplyQueuedBatch) - Add context parameter to executeBlockOperationBatch (moved from blockQueue) - Propagate context through ProcessQueue operations - Add cancellation checks before expensive operations in finishSync() using declarative operation list pattern with runWithCancellationCheck helper. - Add cancellation checks in ProcessQueuedBlockOperations before state transitions. - Add cancellation checks in executeBlockOperationBatch loop using select pattern. - Improve error messages to include operation index and type for better debugging. Refactoring: - Move block operation processing logic from blockQueue to Coordinator (executeBlockOperationBatch) for better separation of concerns. - Simplify blockQueue to be a pure data structure (enqueue, dequeueBatch, removeBelowHeight). - Rename pivot.go to pivot_policy.go for clarity. - Remove cancel function from Coordinator struct, pass as parameter to finish(). Pivot Policy: - Add defaultPivotInterval constant (10000 blocks) in pivot_policy.go. - Apply default pivot interval when WithPivotInterval is not explicitly called. - Update newPivotPolicy to use default when interval is 0. This change enables graceful shutdown of state sync operations and ensures that cancellation signals propagate correctly through all layers of the dynamic state sync orchestration. resolves #1259 Signed-off-by: Tsvetan Dimitrov (tsvetan.dimitrov@avalabs.org) --- graft/coreth/plugin/evm/vmsync/block_queue.go | 33 +--- graft/coreth/plugin/evm/vmsync/client.go | 159 +++++++++--------- graft/coreth/plugin/evm/vmsync/coordinator.go | 93 +++++++--- graft/coreth/plugin/evm/vmsync/finalizer.go | 2 - .../evm/vmsync/{pivot.go => pivot_policy.go} | 20 ++- graft/coreth/plugin/evm/vmsync/registry.go | 1 + graft/coreth/plugin/evm/wrapped_block.go | 57 +++++-- 7 files changed, 209 insertions(+), 156 deletions(-) rename graft/coreth/plugin/evm/vmsync/{pivot.go => pivot_policy.go} (70%) diff --git a/graft/coreth/plugin/evm/vmsync/block_queue.go b/graft/coreth/plugin/evm/vmsync/block_queue.go index 1bb4dc161fe0..28b7800a0a42 100644 --- a/graft/coreth/plugin/evm/vmsync/block_queue.go +++ b/graft/coreth/plugin/evm/vmsync/block_queue.go @@ -3,10 +3,7 @@ package vmsync -import ( - "context" - "sync" -) +import "sync" // BlockOperation represents the type of operation to perform on a block. type BlockOperation int @@ -40,9 +37,9 @@ func newBlockQueue() *blockQueue { return &blockQueue{} } -// Enqueue appends a block operation to the buffer. Returns true if the operation +// enqueue appends a block operation to the buffer. Returns true if the operation // was queued, false if the block is nil. -func (q *blockQueue) Enqueue(b EthBlockWrapper, op BlockOperation) bool { +func (q *blockQueue) enqueue(b EthBlockWrapper, op BlockOperation) bool { if b == nil { return false } @@ -65,10 +62,10 @@ func (q *blockQueue) dequeueBatch() []blockOperation { return out } -// RemoveBlocksBelowHeight removes all queued blocks with height <= targetHeight. +// removeBelowHeight removes all queued blocks with height <= targetHeight. // This is called after UpdateSyncTarget to remove blocks that will never be executed // because the sync target has advanced past them. -func (q *blockQueue) RemoveBlocksBelowHeight(targetHeight uint64) { +func (q *blockQueue) removeBelowHeight(targetHeight uint64) { q.mu.Lock() defer q.mu.Unlock() @@ -81,23 +78,3 @@ func (q *blockQueue) RemoveBlocksBelowHeight(targetHeight uint64) { } q.items = filtered } - -// ProcessQueue processes all queued operations in FIFO order. -func (q *blockQueue) ProcessQueue(ctx context.Context) error { - operations := q.dequeueBatch() - for _, op := range operations { - var err error - switch op.operation { - case OpAccept: - err = op.block.Accept(ctx) - case OpReject: - err = op.block.Reject(ctx) - case OpVerify: - err = op.block.Verify(ctx) - } - if err != nil { - return err - } - } - return nil -} diff --git a/graft/coreth/plugin/evm/vmsync/client.go b/graft/coreth/plugin/evm/vmsync/client.go index ac2133534fdf..5a1e964295fa 100644 --- a/graft/coreth/plugin/evm/vmsync/client.go +++ b/graft/coreth/plugin/evm/vmsync/client.go @@ -5,7 +5,6 @@ package vmsync import ( "context" - "errors" "fmt" "sync" @@ -18,31 +17,21 @@ import ( "github.com/ava-labs/avalanchego/graft/coreth/core/state/snapshot" "github.com/ava-labs/avalanchego/graft/coreth/eth" "github.com/ava-labs/avalanchego/graft/coreth/plugin/evm/message" + "github.com/ava-labs/avalanchego/graft/coreth/sync/blocksync" + "github.com/ava-labs/avalanchego/graft/coreth/sync/statesync" "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/avalanchego/snow/engine/snowman/block" "github.com/ava-labs/avalanchego/vms/components/chain" syncpkg "github.com/ava-labs/avalanchego/graft/coreth/sync" - "github.com/ava-labs/avalanchego/graft/coreth/sync/blocksync" syncclient "github.com/ava-labs/avalanchego/graft/coreth/sync/client" - "github.com/ava-labs/avalanchego/graft/coreth/sync/statesync" ) // BlocksToFetch is the number of the block parents the state syncs to. // The last 256 block hashes are necessary to support the BLOCKHASH opcode. const BlocksToFetch = 256 -var ( - errSkipSync = fmt.Errorf("skip sync") - stateSyncSummaryKey = []byte("stateSyncSummary") -) - -// BlockAcceptor provides a mechanism to update the last accepted block ID during state synchronization. -// This interface is used by the state sync process to ensure the blockchain state -// is properly updated when new blocks are synchronized from the network. -type BlockAcceptor interface { - PutLastAcceptedID(ids.ID) error -} +var stateSyncSummaryKey = []byte("stateSyncSummary") // SyncStrategy defines how state sync is executed. // Implementations handle syncer orchestration and block processing during sync. @@ -60,6 +49,11 @@ type SyncStrategy interface { OnBlockVerified(EthBlockWrapper) (bool, error) } +// BlockAcceptor provides a mechanism to update the last accepted block ID during state synchronization. +type BlockAcceptor interface { + PutLastAcceptedID(ids.ID) error +} + // EthBlockWrapper can be implemented by a concrete block wrapper type to // return *types.Block, which is needed to update chain pointers at the // end of the sync operation. It also provides Accept/Reject/Verify operations @@ -105,10 +99,11 @@ type ClientConfig struct { type client struct { config *ClientConfig resumableSummary message.Syncable - strategy SyncStrategy // strategy manages sync execution (static or dynamic) cancel context.CancelFunc wg sync.WaitGroup err error + stateSyncOnce sync.Once + strategy SyncStrategy // strategy manages sync execution (static or dynamic) } func NewClient(config *ClientConfig) Client { @@ -205,52 +200,40 @@ func (c *client) OnEngineVerify(b EthBlockWrapper) (bool, error) { return c.strategy.OnBlockVerified(b) } +func (c *client) Shutdown() error { + c.signalDone(context.Canceled) + c.wg.Wait() + return nil +} + +// Error returns a non-nil error if one occurred during the sync. +func (c *client) Error() error { + return c.err +} // acceptSyncSummary returns true if sync will be performed and launches the state sync process // in a goroutine. -func (c *client) acceptSyncSummary(summary message.Syncable) (block.StateSyncMode, error) { - if err := c.prepareForSync(summary); err != nil { - if errors.Is(err, errSkipSync) { - return block.StateSyncSkipped, nil - } - return block.StateSyncSkipped, err - } - - registry, err := c.newSyncerRegistry(summary) - if err != nil { - return block.StateSyncSkipped, fmt.Errorf("failed to create syncer registry: %w", err) - } - - finalizer := newFinalizer( - c.config.Chain, - c.config.State, - c.config.Acceptor, - c.config.VerDB, - c.config.MetadataDB, - c.config.Extender, - c.config.LastAcceptedHeight, - ) - - strategy := newStaticStrategy(registry, finalizer) - - return c.startAsync(strategy), nil -} +func (c *client) acceptSyncSummary(proposedSummary message.Syncable) (block.StateSyncMode, error) { + // If dynamic sync is already running, treat new summaries as target updates. + // if ds, ok := c.strategy.(*dynamicStrategy); ok && ds.CurrentState() == StateRunning { + // if err := ds.UpdateSyncTarget(proposedSummary); err != nil { + // return block.StateSyncSkipped, err + // } + // return block.StateSyncDynamic, nil + // } -// prepareForSync handles resume check and snapshot wipe before sync starts. -func (c *client) prepareForSync(summary message.Syncable) error { isResume := c.resumableSummary != nil && - summary.GetBlockHash() == c.resumableSummary.GetBlockHash() + proposedSummary.GetBlockHash() == c.resumableSummary.GetBlockHash() if !isResume { // Skip syncing if the blockchain is not significantly ahead of local state, // since bootstrapping would be faster. - // (Also ensures we don't sync to a height prior to local state.) - if c.config.LastAcceptedHeight+c.config.MinBlocks > summary.Height() { + if c.config.LastAcceptedHeight+c.config.MinBlocks > proposedSummary.Height() { log.Info( "last accepted too close to most recent syncable block, skipping state sync", "lastAccepted", c.config.LastAcceptedHeight, - "syncableHeight", summary.Height(), + "syncableHeight", proposedSummary.Height(), ) - return errSkipSync + return block.StateSyncSkipped, nil } // Wipe the snapshot completely if we are not resuming from an existing sync, so that we do not @@ -266,54 +249,68 @@ func (c *client) prepareForSync(summary message.Syncable) error { snapshot.ResetSnapshotGeneration(c.config.ChainDB) } - // Update the current state sync summary key in the database - // Note: this must be performed after WipeSnapshot finishes so that we do not start a state sync - // session from a partially wiped snapshot. - if err := c.config.MetadataDB.Put(stateSyncSummaryKey, summary.Bytes()); err != nil { - return fmt.Errorf("failed to write state sync summary key to disk: %w", err) + // Update the current state sync summary key in the database. + if err := c.config.MetadataDB.Put(stateSyncSummaryKey, proposedSummary.Bytes()); err != nil { + return block.StateSyncSkipped, fmt.Errorf("failed to write state sync summary key to disk: %w", err) } if err := c.config.VerDB.Commit(); err != nil { - return fmt.Errorf("failed to commit db: %w", err) + return block.StateSyncSkipped, fmt.Errorf("failed to commit db: %w", err) } - return nil -} - -// startAsync launches the sync strategy in a background goroutine. -func (c *client) startAsync(strategy SyncStrategy) block.StateSyncMode { + log.Info("Starting state sync", "summary", proposedSummary.GetBlockHash().Hex(), "height", proposedSummary.Height()) ctx, cancel := context.WithCancel(context.Background()) c.cancel = cancel + registry, err := c.newSyncerRegistry(proposedSummary) + if err != nil { + return block.StateSyncSkipped, err + } + + finalizer := newFinalizer( + c.config.Chain, + c.config.State, + c.config.Acceptor, + c.config.VerDB, + c.config.MetadataDB, + c.config.Extender, + c.config.LastAcceptedHeight, + ) + + var ( + strategy SyncStrategy + mode block.StateSyncMode + ) + if c.config.DynamicStateSyncEnabled { + // strategy = newDynamicStrategy(registry, finalizer, c.PivotInterval) + // mode = block.StateSyncDynamic + } else { + strategy = newStaticStrategy(registry, finalizer) + mode = block.StateSyncStatic + } + + c.strategy = strategy c.wg.Add(1) go func() { defer c.wg.Done() - defer cancel() - - if err := strategy.Start(ctx, c.resumableSummary); err != nil { - c.err = err - } - // notify engine regardless of whether err == nil, - // this error will be propagated to the engine when it calls - // vm.SetState(snow.Bootstrapping) - log.Info("state sync completed, notifying engine", "err", c.err) - close(c.config.StateSyncDone) + err := strategy.Start(ctx, proposedSummary) + c.signalDone(err) }() - log.Info("state sync started", "mode", block.StateSyncStatic) - return block.StateSyncStatic + log.Info("state sync started", "mode", mode.String(), "summary", proposedSummary.GetBlockHash().Hex(), "height", proposedSummary.Height()) + return mode, nil } -func (c *client) Shutdown() error { - if c.cancel != nil { - c.cancel() - } - c.wg.Wait() // wait for the background goroutine to exit - return nil +// signalDone sets the terminal error exactly once, signals completion to the engine. +func (c *client) signalDone(err error) { + c.stateSyncOnce.Do(func() { + c.err = err + if c.cancel != nil { + c.cancel() + } + close(c.config.StateSyncDone) + }) } -// Error returns a non-nil error if one occurred during the sync. -func (c *client) Error() error { return c.err } - // newSyncerRegistry creates a registry with all required syncers for the given summary. func (c *client) newSyncerRegistry(summary message.Syncable) (*SyncerRegistry, error) { registry := NewSyncerRegistry() diff --git a/graft/coreth/plugin/evm/vmsync/coordinator.go b/graft/coreth/plugin/evm/vmsync/coordinator.go index c4abacde8299..e3e48ff30f37 100644 --- a/graft/coreth/plugin/evm/vmsync/coordinator.go +++ b/graft/coreth/plugin/evm/vmsync/coordinator.go @@ -6,6 +6,7 @@ package vmsync import ( "context" "errors" + "fmt" "sync" "sync/atomic" @@ -35,7 +36,8 @@ var ( // Callbacks allows the coordinator to delegate VM-specific work back to the client. type Callbacks struct { // FinalizeVM performs the same actions as finishSync/commitVMMarkers in the client. - FinalizeVM func(target message.Syncable) error + // The context is used for cancellation checks during finalization. + FinalizeVM func(ctx context.Context, target message.Syncable) error // OnDone is called when the coordinator finishes (successfully or with error). OnDone func(err error) } @@ -51,11 +53,11 @@ type Coordinator struct { callbacks Callbacks // doneOnce ensures [Callbacks.OnDone] is invoked at most once. doneOnce sync.Once - // cancel cancels the syncers' context (with cause) when aborting or finishing. - cancel context.CancelCauseFunc // pivot policy to throttle [Coordinator.UpdateSyncTarget] calls. - pivot *pivot + pivot *pivotPolicy + + pivotInterval uint64 } // CoordinatorOption follows the functional options pattern for Coordinator. @@ -64,11 +66,7 @@ type CoordinatorOption = options.Option[Coordinator] // WithPivotInterval configures the interval-based pivot policy. 0 disables it. func WithPivotInterval(interval uint64) CoordinatorOption { return options.Func[Coordinator](func(co *Coordinator) { - if interval == 0 { - co.pivot = nil - return - } - co.pivot = newPivotPolicy(interval) + co.pivotInterval = interval }) } @@ -82,6 +80,7 @@ func NewCoordinator(syncerRegistry *SyncerRegistry, cbs Callbacks, opts ...Coord options.ApplyTo(co, opts...) + co.pivot = newPivotPolicy(co.pivotInterval) co.state.Store(int32(StateIdle)) return co @@ -94,46 +93,57 @@ func (co *Coordinator) Start(ctx context.Context, initial message.Syncable) { co.target.Store(initial) cctx, cancel := context.WithCancelCause(ctx) - co.cancel = cancel g := co.syncerRegistry.StartAsync(cctx, initial) co.state.Store(int32(StateRunning)) go func() { if err := g.Wait(); err != nil { - co.finish(err) + co.finish(cancel, err) return } // All syncers finished successfully: finalize VM and execute the queued batch. - if err := co.ApplyQueuedBatch(cctx); err != nil { - co.finish(err) + if err := co.ProcessQueuedBlockOperations(cctx); err != nil { + co.finish(cancel, err) return } - co.finish(nil) + co.finish(cancel, nil) }() } -// ApplyQueuedBatch finalizes the VM at the current target and processes the +// ProcessQueuedBlockOperations finalizes the VM at the current target and processes the // queued operations in FIFO order. Intended to be called after a target update // cycle when it's time to process the queued operations. -func (co *Coordinator) ApplyQueuedBatch(ctx context.Context) error { +func (co *Coordinator) ProcessQueuedBlockOperations(ctx context.Context) error { + // Check for cancellation before starting finalization phase. + if err := ctx.Err(); err != nil { + return err + } + co.state.Store(int32(StateFinalizing)) + if co.callbacks.FinalizeVM != nil { loaded := co.target.Load() current, ok := loaded.(message.Syncable) if !ok { return errInvalidTargetType } - if err := co.callbacks.FinalizeVM(current); err != nil { + // FinalizeVM should complete atomically. The context is passed for internal + // cancellation checks, but the coordinator expects completion or an error. + if err := co.callbacks.FinalizeVM(ctx, current); err != nil { return err } } + co.state.Store(int32(StateExecutingBatch)) - if co.queue != nil { - if err := co.queue.ProcessQueue(ctx); err != nil { - return err - } + + // Execute queued block operations sequentially. Each operation can be + // cancelled individually, but the batch execution itself is not atomic - partial completion + // is acceptable as operations are idempotent. + if err := co.executeBlockOperationBatch(ctx); err != nil { + return err } + return nil } @@ -141,6 +151,7 @@ func (co *Coordinator) ApplyQueuedBatch(ctx context.Context) error { // It is only valid in the [StateRunning] state. // Note: no batch execution occurs here. Batches are only executed after // finalization. +// Note: Syncers manage cancellation themselves through their Sync() contexts. func (co *Coordinator) UpdateSyncTarget(newTarget message.Syncable) error { if co.CurrentState() != StateRunning { return errInvalidState @@ -151,9 +162,7 @@ func (co *Coordinator) UpdateSyncTarget(newTarget message.Syncable) error { } // Remove blocks from queue that will never be executed (behind the new target). - if co.queue != nil { - co.queue.RemoveBlocksBelowHeight(newTarget.Height()) - } + co.queue.removeBelowHeight(newTarget.Height()) co.target.Store(newTarget) @@ -173,21 +182,49 @@ func (co *Coordinator) AddBlockOperation(b EthBlockWrapper, op BlockOperation) b if b == nil || co.CurrentState() != StateRunning { return false } - return co.queue.Enqueue(b, op) + return co.queue.enqueue(b, op) } func (co *Coordinator) CurrentState() State { return State(co.state.Load()) } -func (co *Coordinator) finish(err error) { +// executeBlockOperationBatch executes all queued block operations in FIFO order. +// Each operation can be cancelled individually, but the batch execution itself +// is not atomic - partial completion is acceptable as operations are idempotent. +func (co *Coordinator) executeBlockOperationBatch(ctx context.Context) error { + operations := co.queue.dequeueBatch() + for i, op := range operations { + select { + case <-ctx.Done(): + return fmt.Errorf("executeBlockOperationBatch cancelled at operation %d/%d: %w", i, len(operations), ctx.Err()) + default: + } + + var err error + switch op.operation { + case OpAccept: + err = op.block.Accept(ctx) + case OpReject: + err = op.block.Reject(ctx) + case OpVerify: + err = op.block.Verify(ctx) + } + if err != nil { + return fmt.Errorf("executeBlockOperationBatch failed at operation %d/%d (%v): %w", i, len(operations), op.operation, err) + } + } + return nil +} + +func (co *Coordinator) finish(cancel context.CancelCauseFunc, err error) { if err != nil { co.state.Store(int32(StateAborted)) } else { co.state.Store(int32(StateCompleted)) } - if co.cancel != nil { - co.cancel(err) + if cancel != nil { + cancel(err) } if co.callbacks.OnDone != nil { co.doneOnce.Do(func() { co.callbacks.OnDone(err) }) diff --git a/graft/coreth/plugin/evm/vmsync/finalizer.go b/graft/coreth/plugin/evm/vmsync/finalizer.go index 427e8c972656..709606d7f77b 100644 --- a/graft/coreth/plugin/evm/vmsync/finalizer.go +++ b/graft/coreth/plugin/evm/vmsync/finalizer.go @@ -8,8 +8,6 @@ import ( "errors" "fmt" - "github.com/ava-labs/libevm/core/types" - "github.com/ava-labs/avalanchego/database" "github.com/ava-labs/avalanchego/database/versiondb" "github.com/ava-labs/avalanchego/graft/coreth/eth" diff --git a/graft/coreth/plugin/evm/vmsync/pivot.go b/graft/coreth/plugin/evm/vmsync/pivot_policy.go similarity index 70% rename from graft/coreth/plugin/evm/vmsync/pivot.go rename to graft/coreth/plugin/evm/vmsync/pivot_policy.go index 32cb6fe79b9d..70f5ab18b53c 100644 --- a/graft/coreth/plugin/evm/vmsync/pivot.go +++ b/graft/coreth/plugin/evm/vmsync/pivot_policy.go @@ -5,24 +5,32 @@ package vmsync import "sync/atomic" -// pivot encapsulates the logic for deciding when to forward +// defaultPivotInterval is the default number of blocks between sync target updates. +const defaultPivotInterval = uint64(10000) + +// pivotPolicy encapsulates the logic for deciding when to forward // a new sync target based on a fixed block-height interval. It is // safe for concurrent use. -type pivot struct { +type pivotPolicy struct { interval uint64 // nextHeight is the next height threshold at or beyond which we // should forward an update. A value of 0 means uninitialized. nextHeight uint64 // accessed atomically } -func newPivotPolicy(interval uint64) *pivot { - return &pivot{interval: interval} +// newPivotPolicy creates a new pivot policy with the given interval. +// If interval is 0, defaultPivotInterval is used. +func newPivotPolicy(interval uint64) *pivotPolicy { + if interval == 0 { + interval = defaultPivotInterval + } + return &pivotPolicy{interval: interval} } // shouldForward reports whether a summary at the given height should be // forwarded, initializing the next threshold on first use. When it returns // true, callers should follow up with advance(). -func (p *pivot) shouldForward(height uint64) bool { +func (p *pivotPolicy) shouldForward(height uint64) bool { if p == nil || p.interval == 0 { return true } @@ -44,7 +52,7 @@ func (p *pivot) shouldForward(height uint64) bool { // advance moves the next threshold forward by one interval. Call this // only after shouldForward has returned true and the update was issued. -func (p *pivot) advance() { +func (p *pivotPolicy) advance() { if p == nil || p.interval == 0 { return } diff --git a/graft/coreth/plugin/evm/vmsync/registry.go b/graft/coreth/plugin/evm/vmsync/registry.go index 071352933ec6..9262e8ac8c2f 100644 --- a/graft/coreth/plugin/evm/vmsync/registry.go +++ b/graft/coreth/plugin/evm/vmsync/registry.go @@ -104,6 +104,7 @@ func (r *SyncerRegistry) StartAsync(ctx context.Context, summary message.Syncabl } // UpdateSyncTarget updates the sync target for all syncers. +// Note: Syncers manage cancellation themselves through their Sync() contexts. func (r *SyncerRegistry) UpdateSyncTarget(newTarget message.Syncable) error { for _, task := range r.syncers { if err := task.syncer.UpdateTarget(newTarget); err != nil { diff --git a/graft/coreth/plugin/evm/wrapped_block.go b/graft/coreth/plugin/evm/wrapped_block.go index 1b6d3bfeaa01..a227a0ff99d6 100644 --- a/graft/coreth/plugin/evm/wrapped_block.go +++ b/graft/coreth/plugin/evm/wrapped_block.go @@ -92,11 +92,19 @@ func wrapBlock(ethBlock *types.Block, vm *VM) (*wrappedBlock, error) { func (b *wrappedBlock) ID() ids.ID { return b.id } // Accept implements the snowman.Block interface -func (b *wrappedBlock) Accept(ctx context.Context) error { +// TODO(powerslider): Propagate context to the sync client. +func (b *wrappedBlock) Accept(_ context.Context) error { // Notify sync client that engine accepted a block. + // If the block was enqueued for deferred processing, skip immediate execution. if client := b.vm.SyncerClient(); client != nil { - if err := client.OnEngineAccept(b); err != nil { - return fmt.Errorf("could not notify sync client that engine accepted a block: %w", err) + deferred, err := client.OnEngineAccept(b) + if err != nil { + return fmt.Errorf("%w: %w", errCouldNotNotifySyncClient, err) + } + if deferred { + // Block was enqueued for deferred processing during dynamic state sync. + // It will be processed later from the queue, so skip immediate execution. + return nil } } @@ -182,12 +190,19 @@ func (b *wrappedBlock) handlePrecompileAccept(rules extras.Rules) error { // Reject implements the snowman.Block interface // If [b] contains an atomic transaction, attempt to re-issue it -func (b *wrappedBlock) Reject(ctx context.Context) error { +func (b *wrappedBlock) Reject(_ context.Context) error { // Notify sync client that engine rejected a block. + // If the block was enqueued for deferred processing, skip immediate execution. if client := b.vm.SyncerClient(); client != nil { - if err := client.OnEngineReject(b); err != nil { + deferred, err := client.OnEngineReject(b) + if err != nil { return fmt.Errorf("%w: %w", errCouldNotNotifySyncClient, err) } + if deferred { + // Block was enqueued for deferred processing during dynamic state sync. + // It will be processed later from the queue, so skip immediate execution. + return nil + } } blkID := b.ID() @@ -221,12 +236,22 @@ func (b *wrappedBlock) Timestamp() time.Time { } // Verify implements the snowman.Block interface -func (b *wrappedBlock) Verify(ctx context.Context) error { +// TODO(powerslider): Propagate context to the sync client. +func (b *wrappedBlock) Verify(_ context.Context) error { // Notify sync client that engine verified a block. + // If the block was enqueued for deferred processing, skip immediate execution. if client := b.vm.SyncerClient(); client != nil { - if err := client.OnEngineVerify(b); err != nil { + deferred, err := client.OnEngineVerify(b) + if err != nil { return fmt.Errorf("%w: %w", errCouldNotNotifySyncClient, err) } + if deferred { + // Block was enqueued for deferred processing during dynamic state sync. + // It will be processed later from the queue, so skip immediate execution. + // Note: Verify may be called multiple times with different contexts, but + // we only enqueue once and process once from the queue. + return nil + } } return b.verify(&precompileconfig.PredicateContext{ @@ -259,12 +284,22 @@ func (b *wrappedBlock) ShouldVerifyWithContext(context.Context) (bool, error) { return false, nil } -// VerifyWithContext implements the block.WithVerifyContext interface -func (b *wrappedBlock) VerifyWithContext(ctx context.Context, proposerVMBlockCtx *block.Context) error { +// VerifyWithContext implements the block.WithVerifyContext interface. +// TODO(powerslider): Propagate context to the sync client. +func (b *wrappedBlock) VerifyWithContext(_ context.Context, proposerVMBlockCtx *block.Context) error { // Notify sync client that engine verified a block. + // If the block was enqueued for deferred processing, skip immediate execution. if client := b.vm.SyncerClient(); client != nil { - if err := client.OnEngineVerify(b); err != nil { - return fmt.Errorf("could not notify sync client that engine verified a block: %w", err) + deferred, err := client.OnEngineVerify(b) + if err != nil { + return fmt.Errorf("%w: %w", errCouldNotNotifySyncClient, err) + } + if deferred { + // Block was enqueued for deferred processing during dynamic state sync. + // It will be processed later from the queue, so skip immediate execution. + // Note: VerifyWithContext may be called multiple times with different contexts, but + // we only enqueue once and process once from the queue. + return nil } } From d953aac939277563f89518bc7cbaa6986f8a79c0 Mon Sep 17 00:00:00 2001 From: Tsvetan Dimitrov Date: Mon, 10 Nov 2025 15:24:33 +0200 Subject: [PATCH 06/20] refactor(vmsync): improve code quality and reduce duplication - Refactor block operation handling and error management to improve maintainability, reduce code duplication, and enhance type safety. --- graft/coreth/plugin/evm/vmsync/block_queue.go | 24 ++++- graft/coreth/plugin/evm/vmsync/client.go | 98 ++++++++++--------- graft/coreth/plugin/evm/vmsync/coordinator.go | 18 ++-- .../coreth/plugin/evm/vmsync/pivot_policy.go | 10 +- .../plugin/evm/vmsync/strategy_static.go | 11 +-- graft/coreth/plugin/evm/vmsync/sync_target.go | 4 +- 6 files changed, 92 insertions(+), 73 deletions(-) diff --git a/graft/coreth/plugin/evm/vmsync/block_queue.go b/graft/coreth/plugin/evm/vmsync/block_queue.go index 28b7800a0a42..9a8fe86beaa3 100644 --- a/graft/coreth/plugin/evm/vmsync/block_queue.go +++ b/graft/coreth/plugin/evm/vmsync/block_queue.go @@ -5,19 +5,33 @@ package vmsync import "sync" -// BlockOperation represents the type of operation to perform on a block. -type BlockOperation int +// BlockOperationType represents the type of operation to perform on a block. +type BlockOperationType int const ( - OpAccept BlockOperation = iota + OpAccept BlockOperationType = iota OpReject OpVerify ) +// String returns the string representation of the block operation. +func (op BlockOperationType) String() string { + switch op { + case OpAccept: + return "accept" + case OpReject: + return "reject" + case OpVerify: + return "verify" + default: + return "unknown" + } +} + // blockOperation represents a queued block operation. type blockOperation struct { block EthBlockWrapper - operation BlockOperation + operation BlockOperationType } // blockQueue buffers block operations (accept/reject/verify) that arrive while @@ -39,7 +53,7 @@ func newBlockQueue() *blockQueue { // enqueue appends a block operation to the buffer. Returns true if the operation // was queued, false if the block is nil. -func (q *blockQueue) enqueue(b EthBlockWrapper, op BlockOperation) bool { +func (q *blockQueue) enqueue(b EthBlockWrapper, op BlockOperationType) bool { if b == nil { return false } diff --git a/graft/coreth/plugin/evm/vmsync/client.go b/graft/coreth/plugin/evm/vmsync/client.go index 5a1e964295fa..e6ca2375b1be 100644 --- a/graft/coreth/plugin/evm/vmsync/client.go +++ b/graft/coreth/plugin/evm/vmsync/client.go @@ -97,18 +97,24 @@ type ClientConfig struct { } type client struct { - config *ClientConfig + *ClientConfig + resumableSummary message.Syncable - cancel context.CancelFunc - wg sync.WaitGroup - err error - stateSyncOnce sync.Once - strategy SyncStrategy // strategy manages sync execution (static or dynamic) + + cancel context.CancelFunc + wg sync.WaitGroup + + // State Sync results + err error + stateSyncOnce sync.Once + + // strategy manages sync execution (static or dynamic) + strategy SyncStrategy } func NewClient(config *ClientConfig) Client { return &client{ - config: config, + ClientConfig: config, } } @@ -135,23 +141,23 @@ type Client interface { // StateSyncEnabled returns [client.enabled], which is set in the chain's config file. func (c *client) StateSyncEnabled(context.Context) (bool, error) { - return c.config.Enabled, nil + return c.Enabled, nil } // GetOngoingSyncStateSummary returns a state summary that was previously started // and not finished, and sets [resumableSummary] if one was found. // Returns [database.ErrNotFound] if no ongoing summary is found or if [client.skipResume] is true. func (c *client) GetOngoingSyncStateSummary(context.Context) (block.StateSummary, error) { - if c.config.SkipResume { + if c.SkipResume { return nil, database.ErrNotFound } - summaryBytes, err := c.config.MetadataDB.Get(stateSyncSummaryKey) + summaryBytes, err := c.MetadataDB.Get(stateSyncSummaryKey) if err != nil { return nil, err // includes the [database.ErrNotFound] case } - summary, err := c.config.Parser.Parse(summaryBytes, c.acceptSyncSummary) + summary, err := c.Parser.Parse(summaryBytes, c.acceptSyncSummary) if err != nil { return nil, fmt.Errorf("failed to parse saved state sync summary to SyncSummary: %w", err) } @@ -161,10 +167,10 @@ func (c *client) GetOngoingSyncStateSummary(context.Context) (block.StateSummary // ClearOngoingSummary clears any marker of an ongoing state sync summary func (c *client) ClearOngoingSummary() error { - if err := c.config.MetadataDB.Delete(stateSyncSummaryKey); err != nil { + if err := c.MetadataDB.Delete(stateSyncSummaryKey); err != nil { return fmt.Errorf("failed to clear ongoing summary: %w", err) } - if err := c.config.VerDB.Commit(); err != nil { + if err := c.VerDB.Commit(); err != nil { return fmt.Errorf("failed to commit db while clearing ongoing summary: %w", err) } @@ -173,7 +179,7 @@ func (c *client) ClearOngoingSummary() error { // ParseStateSummary parses [summaryBytes] to [commonEng.Summary] func (c *client) ParseStateSummary(_ context.Context, summaryBytes []byte) (block.StateSummary, error) { - return c.config.Parser.Parse(summaryBytes, c.acceptSyncSummary) + return c.Parser.Parse(summaryBytes, c.acceptSyncSummary) } // OnEngineAccept delegates to the strategy if active. @@ -215,22 +221,22 @@ func (c *client) Error() error { // in a goroutine. func (c *client) acceptSyncSummary(proposedSummary message.Syncable) (block.StateSyncMode, error) { // If dynamic sync is already running, treat new summaries as target updates. - // if ds, ok := c.strategy.(*dynamicStrategy); ok && ds.CurrentState() == StateRunning { - // if err := ds.UpdateSyncTarget(proposedSummary); err != nil { - // return block.StateSyncSkipped, err - // } - // return block.StateSyncDynamic, nil - // } + if ds, ok := c.strategy.(*dynamicStrategy); ok && ds.CurrentState() == StateRunning { + if err := ds.UpdateSyncTarget(proposedSummary); err != nil { + return block.StateSyncSkipped, err + } + return block.StateSyncDynamic, nil + } isResume := c.resumableSummary != nil && proposedSummary.GetBlockHash() == c.resumableSummary.GetBlockHash() if !isResume { // Skip syncing if the blockchain is not significantly ahead of local state, // since bootstrapping would be faster. - if c.config.LastAcceptedHeight+c.config.MinBlocks > proposedSummary.Height() { + if c.LastAcceptedHeight+c.MinBlocks > proposedSummary.Height() { log.Info( "last accepted too close to most recent syncable block, skipping state sync", - "lastAccepted", c.config.LastAcceptedHeight, + "lastAccepted", c.LastAcceptedHeight, "syncableHeight", proposedSummary.Height(), ) return block.StateSyncSkipped, nil @@ -242,18 +248,18 @@ func (c *client) acceptSyncSummary(proposedSummary message.Syncable) (block.Stat // sync marker will be wiped, so we do not accidentally resume progress from an incorrect version // of the snapshot. (if switching between versions that come before this change and back this could // lead to the snapshot not being cleaned up correctly) - <-snapshot.WipeSnapshot(c.config.ChainDB, true) + <-snapshot.WipeSnapshot(c.ChainDB, true) // Reset the snapshot generator here so that when state sync completes, snapshots will not attempt to read an // invalid generator. // Note: this must be called after WipeSnapshot is called so that we do not invalidate a partially generated snapshot. - snapshot.ResetSnapshotGeneration(c.config.ChainDB) + snapshot.ResetSnapshotGeneration(c.ChainDB) } // Update the current state sync summary key in the database. - if err := c.config.MetadataDB.Put(stateSyncSummaryKey, proposedSummary.Bytes()); err != nil { + if err := c.MetadataDB.Put(stateSyncSummaryKey, proposedSummary.Bytes()); err != nil { return block.StateSyncSkipped, fmt.Errorf("failed to write state sync summary key to disk: %w", err) } - if err := c.config.VerDB.Commit(); err != nil { + if err := c.VerDB.Commit(); err != nil { return block.StateSyncSkipped, fmt.Errorf("failed to commit db: %w", err) } @@ -261,28 +267,28 @@ func (c *client) acceptSyncSummary(proposedSummary message.Syncable) (block.Stat ctx, cancel := context.WithCancel(context.Background()) c.cancel = cancel - registry, err := c.newSyncerRegistry(proposedSummary) + registry, err := newSyncerRegistry(c.ClientConfig, proposedSummary) if err != nil { return block.StateSyncSkipped, err } finalizer := newFinalizer( - c.config.Chain, - c.config.State, - c.config.Acceptor, - c.config.VerDB, - c.config.MetadataDB, - c.config.Extender, - c.config.LastAcceptedHeight, + c.Chain, + c.State, + c.Acceptor, + c.VerDB, + c.MetadataDB, + c.Extender, + c.LastAcceptedHeight, ) var ( strategy SyncStrategy mode block.StateSyncMode ) - if c.config.DynamicStateSyncEnabled { - // strategy = newDynamicStrategy(registry, finalizer, c.PivotInterval) - // mode = block.StateSyncDynamic + if c.DynamicStateSyncEnabled { + strategy = newDynamicStrategy(registry, finalizer, c.PivotInterval) + mode = block.StateSyncDynamic } else { strategy = newStaticStrategy(registry, finalizer) mode = block.StateSyncStatic @@ -307,16 +313,16 @@ func (c *client) signalDone(err error) { if c.cancel != nil { c.cancel() } - close(c.config.StateSyncDone) + close(c.StateSyncDone) }) } // newSyncerRegistry creates a registry with all required syncers for the given summary. -func (c *client) newSyncerRegistry(summary message.Syncable) (*SyncerRegistry, error) { +func newSyncerRegistry(cfg *ClientConfig, summary message.Syncable) (*SyncerRegistry, error) { registry := NewSyncerRegistry() blockSyncer, err := blocksync.NewSyncer( - c.config.Client, c.config.ChainDB, + cfg.Client, cfg.ChainDB, summary.GetBlockHash(), summary.Height(), BlocksToFetch, ) @@ -324,20 +330,20 @@ func (c *client) newSyncerRegistry(summary message.Syncable) (*SyncerRegistry, e return nil, fmt.Errorf("failed to create block syncer: %w", err) } - codeQueue, err := statesync.NewCodeQueue(c.config.ChainDB, c.config.StateSyncDone) + codeQueue, err := statesync.NewCodeQueue(cfg.ChainDB, cfg.StateSyncDone) if err != nil { return nil, fmt.Errorf("failed to create code queue: %w", err) } - codeSyncer, err := statesync.NewCodeSyncer(c.config.Client, c.config.ChainDB, codeQueue.CodeHashes()) + codeSyncer, err := statesync.NewCodeSyncer(cfg.Client, cfg.ChainDB, codeQueue.CodeHashes()) if err != nil { return nil, fmt.Errorf("failed to create code syncer: %w", err) } stateSyncer, err := statesync.NewSyncer( - c.config.Client, c.config.ChainDB, + cfg.Client, cfg.ChainDB, summary.GetBlockRoot(), - codeQueue, c.config.RequestSize, + codeQueue, cfg.RequestSize, ) if err != nil { return nil, fmt.Errorf("failed to create EVM state syncer: %w", err) @@ -345,8 +351,8 @@ func (c *client) newSyncerRegistry(summary message.Syncable) (*SyncerRegistry, e syncers := []syncpkg.Syncer{blockSyncer, codeSyncer, stateSyncer} - if c.config.Extender != nil { - atomicSyncer, err := c.config.Extender.CreateSyncer(c.config.Client, c.config.VerDB, summary) + if cfg.Extender != nil { + atomicSyncer, err := cfg.Extender.CreateSyncer(cfg.Client, cfg.VerDB, summary) if err != nil { return nil, fmt.Errorf("failed to create atomic syncer: %w", err) } diff --git a/graft/coreth/plugin/evm/vmsync/coordinator.go b/graft/coreth/plugin/evm/vmsync/coordinator.go index e3e48ff30f37..0ea46d36b8cf 100644 --- a/graft/coreth/plugin/evm/vmsync/coordinator.go +++ b/graft/coreth/plugin/evm/vmsync/coordinator.go @@ -29,8 +29,10 @@ const ( ) var ( - errInvalidTargetType = errors.New("invalid target type") - errInvalidState = errors.New("invalid coordinator state") + errInvalidTargetType = errors.New("invalid target type") + errInvalidState = errors.New("invalid coordinator state") + errBatchCancelled = errors.New("batch execution cancelled") + errBatchOperationFailed = errors.New("batch operation failed") ) // Callbacks allows the coordinator to delegate VM-specific work back to the client. @@ -157,7 +159,7 @@ func (co *Coordinator) UpdateSyncTarget(newTarget message.Syncable) error { return errInvalidState } // Respect pivot policy if configured. - if co.pivot != nil && !co.pivot.shouldForward(newTarget.Height()) { + if !co.pivot.shouldForward(newTarget.Height()) { return nil } @@ -169,16 +171,14 @@ func (co *Coordinator) UpdateSyncTarget(newTarget message.Syncable) error { if err := co.syncerRegistry.UpdateSyncTarget(newTarget); err != nil { return err } - if co.pivot != nil { - co.pivot.advance() - } + co.pivot.advance() return nil } // AddBlock appends the block to the queue only while in the Running state. // Returns true if the block was queued, false if the queue was already sealed // or the block is nil. -func (co *Coordinator) AddBlockOperation(b EthBlockWrapper, op BlockOperation) bool { +func (co *Coordinator) AddBlockOperation(b EthBlockWrapper, op BlockOperationType) bool { if b == nil || co.CurrentState() != StateRunning { return false } @@ -197,7 +197,7 @@ func (co *Coordinator) executeBlockOperationBatch(ctx context.Context) error { for i, op := range operations { select { case <-ctx.Done(): - return fmt.Errorf("executeBlockOperationBatch cancelled at operation %d/%d: %w", i, len(operations), ctx.Err()) + return fmt.Errorf("operation %d/%d: %w", i+1, len(operations), errors.Join(errBatchCancelled, ctx.Err())) default: } @@ -211,7 +211,7 @@ func (co *Coordinator) executeBlockOperationBatch(ctx context.Context) error { err = op.block.Verify(ctx) } if err != nil { - return fmt.Errorf("executeBlockOperationBatch failed at operation %d/%d (%v): %w", i, len(operations), op.operation, err) + return fmt.Errorf("operation %d/%d (%v): %w", i+1, len(operations), op.operation, errors.Join(errBatchOperationFailed, err)) } } return nil diff --git a/graft/coreth/plugin/evm/vmsync/pivot_policy.go b/graft/coreth/plugin/evm/vmsync/pivot_policy.go index 70f5ab18b53c..ac808368b92c 100644 --- a/graft/coreth/plugin/evm/vmsync/pivot_policy.go +++ b/graft/coreth/plugin/evm/vmsync/pivot_policy.go @@ -15,7 +15,7 @@ type pivotPolicy struct { interval uint64 // nextHeight is the next height threshold at or beyond which we // should forward an update. A value of 0 means uninitialized. - nextHeight uint64 // accessed atomically + nextHeight atomic.Uint64 } // newPivotPolicy creates a new pivot policy with the given interval. @@ -34,15 +34,15 @@ func (p *pivotPolicy) shouldForward(height uint64) bool { if p == nil || p.interval == 0 { return true } - next := atomic.LoadUint64(&p.nextHeight) + next := p.nextHeight.Load() if next == 0 { // Round up the initial height to the next multiple of interval. // Ceil division: ((h + interval - 1) / interval) * interval h := height init := ((h + p.interval - 1) / p.interval) * p.interval // Initialize once - if another goroutine wins, read the established value. - if !atomic.CompareAndSwapUint64(&p.nextHeight, 0, init) { - next = atomic.LoadUint64(&p.nextHeight) + if !p.nextHeight.CompareAndSwap(0, init) { + next = p.nextHeight.Load() } else { next = init } @@ -56,5 +56,5 @@ func (p *pivotPolicy) advance() { if p == nil || p.interval == 0 { return } - atomic.AddUint64(&p.nextHeight, p.interval) + p.nextHeight.Add(p.interval) } diff --git a/graft/coreth/plugin/evm/vmsync/strategy_static.go b/graft/coreth/plugin/evm/vmsync/strategy_static.go index b5a8f88c4518..8f98fc61bc4a 100644 --- a/graft/coreth/plugin/evm/vmsync/strategy_static.go +++ b/graft/coreth/plugin/evm/vmsync/strategy_static.go @@ -17,7 +17,6 @@ var _ SyncStrategy = (*staticStrategy)(nil) type staticStrategy struct { registry *SyncerRegistry finalizer *finalizer - summary message.Syncable } func newStaticStrategy(registry *SyncerRegistry, finalizer *finalizer) *staticStrategy { @@ -30,23 +29,23 @@ func newStaticStrategy(registry *SyncerRegistry, finalizer *finalizer) *staticSt // Start begins the sync process and blocks until completion or error. // For static sync, this runs all syncers and then finalizes the VM state. func (s *staticStrategy) Start(ctx context.Context, summary message.Syncable) error { - if err := s.registry.RunSyncerTasks(ctx, s.summary); err != nil { + if err := s.registry.RunSyncerTasks(ctx, summary); err != nil { return err } - return s.finalizer.finalize(ctx, s.summary) + return s.finalizer.finalize(ctx, summary) } // OnBlockAccepted is a no-op for static sync since blocks are not queued. -func (s *staticStrategy) OnBlockAccepted(EthBlockWrapper) (bool, error) { +func (*staticStrategy) OnBlockAccepted(EthBlockWrapper) (bool, error) { return false, nil } // OnBlockRejected is a no-op for static sync since blocks are not queued. -func (s *staticStrategy) OnBlockRejected(EthBlockWrapper) (bool, error) { +func (*staticStrategy) OnBlockRejected(EthBlockWrapper) (bool, error) { return false, nil } // OnBlockVerified is a no-op for static sync since blocks are not queued. -func (s *staticStrategy) OnBlockVerified(EthBlockWrapper) (bool, error) { +func (*staticStrategy) OnBlockVerified(EthBlockWrapper) (bool, error) { return false, nil } diff --git a/graft/coreth/plugin/evm/vmsync/sync_target.go b/graft/coreth/plugin/evm/vmsync/sync_target.go index 32c7850ecc56..20412ebbca6f 100644 --- a/graft/coreth/plugin/evm/vmsync/sync_target.go +++ b/graft/coreth/plugin/evm/vmsync/sync_target.go @@ -6,11 +6,11 @@ package vmsync import ( "context" - "github.com/ava-labs/avalanchego/ids" - "github.com/ava-labs/avalanchego/snow/engine/snowman/block" "github.com/ava-labs/libevm/common" "github.com/ava-labs/avalanchego/graft/coreth/plugin/evm/message" + "github.com/ava-labs/avalanchego/ids" + "github.com/ava-labs/avalanchego/snow/engine/snowman/block" ) var _ message.Syncable = (*syncTarget)(nil) From 88c53c9da1beef6f53f73efe7fdc0c1d7a116087 Mon Sep 17 00:00:00 2001 From: Tsvetan Dimitrov Date: Mon, 10 Nov 2025 16:23:11 +0200 Subject: [PATCH 07/20] feat(vmsync): allow block enqueuing during batch execution Enable blocks to be enqueued during StateExecutingBatch for processing in the next batch, while preventing recursion by skipping sync target updates during batch execution. Block Enqueuing During Batch Execution: - Update AddBlockOperation to allow enqueuing during both StateRunning and StateExecutingBatch states. - Remove early return check in enqueueBlockOperation that prevented enqueuing during batch execution. - Blocks enqueued during batch execution are automatically processed in the next batch (via dequeueBatch snapshot behavior). Prevent Recursion: - Skip UpdateSyncTarget in OnEngineAccept when state is StateExecutingBatch - Blocks are still enqueued during batch execution, but sync target updates are deferred to prevent recursion. - Add documentation explaining the behavior. Code Simplification: - Simplify finishSync cancellation checks from per-operation checks to single check at beginning. - Operations are not cancellable mid-execution, so single check is sufficient and more efficient. This change ensures blocks arriving during batch execution can be queued for the next batch (solving dependency issues) while maintaining fast consensus-critical paths and preventing recursion. --- graft/coreth/plugin/evm/vmsync/coordinator.go | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/graft/coreth/plugin/evm/vmsync/coordinator.go b/graft/coreth/plugin/evm/vmsync/coordinator.go index 0ea46d36b8cf..1be6ccf3c4ac 100644 --- a/graft/coreth/plugin/evm/vmsync/coordinator.go +++ b/graft/coreth/plugin/evm/vmsync/coordinator.go @@ -175,11 +175,16 @@ func (co *Coordinator) UpdateSyncTarget(newTarget message.Syncable) error { return nil } -// AddBlock appends the block to the queue only while in the Running state. -// Returns true if the block was queued, false if the queue was already sealed -// or the block is nil. +// AddBlockOperation appends the block to the queue while in the Running or +// StateExecutingBatch state. Blocks enqueued during batch execution will be +// processed in the next batch. Returns true if the block was queued, false +// if the queue was already sealed or the block is nil. func (co *Coordinator) AddBlockOperation(b EthBlockWrapper, op BlockOperationType) bool { - if b == nil || co.CurrentState() != StateRunning { + if b == nil { + return false + } + state := co.CurrentState() + if state != StateRunning && state != StateExecutingBatch { return false } return co.queue.enqueue(b, op) From 74f726ebc1c7077f34ff70fd4caa6c91ea712fc8 Mon Sep 17 00:00:00 2001 From: Tsvetan Dimitrov Date: Tue, 11 Nov 2025 14:05:38 +0200 Subject: [PATCH 08/20] feat(sync): add Finalize method to Syncer interface and integrate into sync flow - Add Finalize method to the Syncer interface and implement it across all syncer types. Integrate Finalize calls into both static and dynamic state sync flows to allow syncers to clean up their state before VM finalization. - Dynamic sync: Syncers complete -> Finalize syncers -> Finalize VM -> Execute batch - Static sync: Syncers complete -> Finalize syncers -> Finalize VM --- graft/coreth/plugin/evm/atomic/sync/syncer.go | 4 ++++ graft/coreth/plugin/evm/vmsync/coordinator.go | 6 +++++- graft/coreth/plugin/evm/vmsync/doubles_test.go | 1 + graft/coreth/plugin/evm/vmsync/registry.go | 14 ++++++++++++++ graft/coreth/plugin/evm/vmsync/registry_test.go | 5 +++++ graft/coreth/sync/blocksync/syncer.go | 4 ++++ graft/coreth/sync/statesync/code_syncer.go | 4 ++++ graft/coreth/sync/statesync/state_syncer.go | 4 ++++ graft/coreth/sync/types.go | 3 +++ 9 files changed, 44 insertions(+), 1 deletion(-) diff --git a/graft/coreth/plugin/evm/atomic/sync/syncer.go b/graft/coreth/plugin/evm/atomic/sync/syncer.go index 8435c98904ea..f56c353b4748 100644 --- a/graft/coreth/plugin/evm/atomic/sync/syncer.go +++ b/graft/coreth/plugin/evm/atomic/sync/syncer.go @@ -150,6 +150,10 @@ func (*Syncer) UpdateTarget(_ message.Syncable) error { return nil } +func (*Syncer) Finalize(_ context.Context) error { + return nil +} + // addZeroes returns the big-endian representation of `height`, prefixed with [common.HashLength] zeroes. func addZeroes(height uint64) []byte { // Key format is [height(8 bytes)][blockchainID(32 bytes)]. Start should be the diff --git a/graft/coreth/plugin/evm/vmsync/coordinator.go b/graft/coreth/plugin/evm/vmsync/coordinator.go index 1be6ccf3c4ac..d37b346824c0 100644 --- a/graft/coreth/plugin/evm/vmsync/coordinator.go +++ b/graft/coreth/plugin/evm/vmsync/coordinator.go @@ -104,7 +104,11 @@ func (co *Coordinator) Start(ctx context.Context, initial message.Syncable) { co.finish(cancel, err) return } - // All syncers finished successfully: finalize VM and execute the queued batch. + // All syncers finished successfully: finalize syncers, then finalize VM and execute the queued batch. + if err := co.syncerRegistry.FinalizeAll(cctx); err != nil { + co.finish(cancel, err) + return + } if err := co.ProcessQueuedBlockOperations(cctx); err != nil { co.finish(cancel, err) return diff --git a/graft/coreth/plugin/evm/vmsync/doubles_test.go b/graft/coreth/plugin/evm/vmsync/doubles_test.go index 443ceb8bcdbc..b63746dd571e 100644 --- a/graft/coreth/plugin/evm/vmsync/doubles_test.go +++ b/graft/coreth/plugin/evm/vmsync/doubles_test.go @@ -27,6 +27,7 @@ func (f FuncSyncer) Sync(ctx context.Context) error { return f.fn(ctx) } func (FuncSyncer) Name() string { return "Test Name" } func (FuncSyncer) ID() string { return "test_id" } func (FuncSyncer) UpdateTarget(_ message.Syncable) error { return nil } +func (FuncSyncer) Finalize(_ context.Context) error { return nil } var _ syncpkg.Syncer = FuncSyncer{} diff --git a/graft/coreth/plugin/evm/vmsync/registry.go b/graft/coreth/plugin/evm/vmsync/registry.go index 9262e8ac8c2f..2bc8eac7383d 100644 --- a/graft/coreth/plugin/evm/vmsync/registry.go +++ b/graft/coreth/plugin/evm/vmsync/registry.go @@ -115,3 +115,17 @@ func (r *SyncerRegistry) UpdateSyncTarget(newTarget message.Syncable) error { } return nil } + +// FinalizeAll calls Finalize on all registered syncers. +// This should be called after all syncers have completed their Sync() operations +// and before finalizing the VM state. +func (r *SyncerRegistry) FinalizeAll(ctx context.Context) error { + for _, task := range r.syncers { + if err := task.syncer.Finalize(ctx); err != nil { + log.Error("failed finalizing syncer", "name", task.name, "err", err) + return fmt.Errorf("%s finalize failed: %w", task.name, err) + } + log.Info("finalized syncer", "name", task.name) + } + return nil +} diff --git a/graft/coreth/plugin/evm/vmsync/registry_test.go b/graft/coreth/plugin/evm/vmsync/registry_test.go index f29c8e5569e5..b7665bbcc415 100644 --- a/graft/coreth/plugin/evm/vmsync/registry_test.go +++ b/graft/coreth/plugin/evm/vmsync/registry_test.go @@ -41,6 +41,7 @@ func (m *mockSyncer) Sync(context.Context) error { func (m *mockSyncer) Name() string { return m.name } func (m *mockSyncer) ID() string { return m.name } func (*mockSyncer) UpdateTarget(_ message.Syncable) error { return nil } +func (*mockSyncer) Finalize(_ context.Context) error { return nil } // namedSyncer adapts an existing syncer with a provided name to satisfy Syncer with Name(). type namedSyncer struct { @@ -55,6 +56,10 @@ func (n *namedSyncer) UpdateTarget(newTarget message.Syncable) error { return n.syncer.UpdateTarget(newTarget) } +func (n *namedSyncer) Finalize(ctx context.Context) error { + return n.syncer.Finalize(ctx) +} + // syncerConfig describes a test syncer setup for RunSyncerTasks table tests. type syncerConfig struct { name string diff --git a/graft/coreth/sync/blocksync/syncer.go b/graft/coreth/sync/blocksync/syncer.go index 0ffa9caa5780..10952fa7bb81 100644 --- a/graft/coreth/sync/blocksync/syncer.go +++ b/graft/coreth/sync/blocksync/syncer.go @@ -118,3 +118,7 @@ func (s *BlockSyncer) Sync(ctx context.Context) error { func (*BlockSyncer) UpdateTarget(_ message.Syncable) error { return nil } + +func (*BlockSyncer) Finalize(_ context.Context) error { + return nil +} diff --git a/graft/coreth/sync/statesync/code_syncer.go b/graft/coreth/sync/statesync/code_syncer.go index 98941e61968b..839188293090 100644 --- a/graft/coreth/sync/statesync/code_syncer.go +++ b/graft/coreth/sync/statesync/code_syncer.go @@ -119,6 +119,10 @@ func (*CodeSyncer) UpdateTarget(_ message.Syncable) error { return nil } +func (*CodeSyncer) Finalize(_ context.Context) error { + return nil +} + // work fulfills any incoming requests from the producer channel by fetching code bytes from the network // and fulfilling them by updating the database. func (c *CodeSyncer) work(ctx context.Context) error { diff --git a/graft/coreth/sync/statesync/state_syncer.go b/graft/coreth/sync/statesync/state_syncer.go index 54b741de9c33..f955cbed8e88 100644 --- a/graft/coreth/sync/statesync/state_syncer.go +++ b/graft/coreth/sync/statesync/state_syncer.go @@ -172,6 +172,10 @@ func (*stateSync) UpdateTarget(_ message.Syncable) error { return nil } +func (*stateSync) Finalize(_ context.Context) error { + return nil +} + // onStorageTrieFinished is called after a storage trie finishes syncing. func (t *stateSync) onStorageTrieFinished(root common.Hash) error { <-t.triesInProgressSem // allow another trie to start (release the semaphore) diff --git a/graft/coreth/sync/types.go b/graft/coreth/sync/types.go index 55c4a2b31750..06194a50e15a 100644 --- a/graft/coreth/sync/types.go +++ b/graft/coreth/sync/types.go @@ -32,6 +32,9 @@ type Syncer interface { // UpdateTarget updates the syncer's target while running to support dynamic state sync. UpdateTarget(newTarget message.Syncable) error + + // Finalize is called when the syncer is finished. + Finalize(ctx context.Context) error } // SummaryProvider is an interface for providing state summaries. From d7ad43eba7c9fd573f2eac1b7f0124d75b701d7f Mon Sep 17 00:00:00 2001 From: Tsvetan Dimitrov Date: Tue, 11 Nov 2025 14:50:41 +0200 Subject: [PATCH 09/20] fix(vmsync): prevent double execution and fix race conditions in dynamic state sync Fix critical issues in dynamic state sync that could cause blocks to be processed twice and introduce race conditions. Improve error handling and state consistency throughout the sync flow. Double Execution Prevention: - Change OnEngineAccept/Reject/Verify to return (bool, error) indicating whether block was enqueued for deferred processing - Update wrapped_block.Accept/Reject/Verify to skip immediate execution when block is enqueued during dynamic sync which prevents blocks from being processed both immediately and from queue. Race Condition Fixes: - Add state re-check in UpdateSyncTarget before modifying queue to handle concurrent state transitions. - Prevent UpdateSyncTarget from being called during batch execution to avoid race with removeBelowHeight. State Consistency Improvements: - Set StateAborted on all error paths in ProcessQueuedBlockOperations before returning to ensure consistent state. - Add context checks at critical points (before FinalizeVM, before batch execution) to catch cancellations early. - Ensure state transitions are atomic with error handling. Error Handling Enhancements: - Improve OnEngineAccept error handling when UpdateSyncTarget fails - Return clear error message indicating block was enqueued but sync target update failed. - Reorder operations to check batch execution state before enqueuing. --- graft/coreth/plugin/evm/vmsync/coordinator.go | 25 ++++++++++++++++++- 1 file changed, 24 insertions(+), 1 deletion(-) diff --git a/graft/coreth/plugin/evm/vmsync/coordinator.go b/graft/coreth/plugin/evm/vmsync/coordinator.go index d37b346824c0..70f70828db6a 100644 --- a/graft/coreth/plugin/evm/vmsync/coordinator.go +++ b/graft/coreth/plugin/evm/vmsync/coordinator.go @@ -129,24 +129,39 @@ func (co *Coordinator) ProcessQueuedBlockOperations(ctx context.Context) error { co.state.Store(int32(StateFinalizing)) if co.callbacks.FinalizeVM != nil { + // Check context again before expensive FinalizeVM operation. + if err := ctx.Err(); err != nil { + co.state.Store(int32(StateAborted)) + return err + } + loaded := co.target.Load() current, ok := loaded.(message.Syncable) if !ok { + co.state.Store(int32(StateAborted)) return errInvalidTargetType } // FinalizeVM should complete atomically. The context is passed for internal // cancellation checks, but the coordinator expects completion or an error. if err := co.callbacks.FinalizeVM(ctx, current); err != nil { + co.state.Store(int32(StateAborted)) return err } } + // Check context before transitioning to batch execution state. + if err := ctx.Err(); err != nil { + co.state.Store(int32(StateAborted)) + return err + } + co.state.Store(int32(StateExecutingBatch)) // Execute queued block operations sequentially. Each operation can be // cancelled individually, but the batch execution itself is not atomic - partial completion // is acceptable as operations are idempotent. if err := co.executeBlockOperationBatch(ctx); err != nil { + // State will be set to StateAborted by finish() when error is returned. return err } @@ -154,11 +169,13 @@ func (co *Coordinator) ProcessQueuedBlockOperations(ctx context.Context) error { } // UpdateSyncTarget broadcasts a new target to all updatable syncers. -// It is only valid in the [StateRunning] state. +// It is only valid in the [StateRunning] state. It cannot be called during +// batch execution to prevent removing blocks that are currently being processed. // Note: no batch execution occurs here. Batches are only executed after // finalization. // Note: Syncers manage cancellation themselves through their Sync() contexts. func (co *Coordinator) UpdateSyncTarget(newTarget message.Syncable) error { + // Re-check state after each operation to handle concurrent state transitions. if co.CurrentState() != StateRunning { return errInvalidState } @@ -167,6 +184,12 @@ func (co *Coordinator) UpdateSyncTarget(newTarget message.Syncable) error { return nil } + // Re-check state before modifying queue to ensure we're still in Running state + // and not in batch execution (which would cause a race with removeBelowHeight). + if co.CurrentState() != StateRunning { + return errInvalidState + } + // Remove blocks from queue that will never be executed (behind the new target). co.queue.removeBelowHeight(newTarget.Height()) From 2e6e03396b3704aeedf72dd6b0e568071bddec82 Mon Sep 17 00:00:00 2001 From: Tsvetan Dimitrov Date: Tue, 11 Nov 2025 15:07:18 +0200 Subject: [PATCH 10/20] docs(vmsync): streamline method docs --- graft/coreth/plugin/evm/vmsync/coordinator.go | 32 ++++--------------- 1 file changed, 7 insertions(+), 25 deletions(-) diff --git a/graft/coreth/plugin/evm/vmsync/coordinator.go b/graft/coreth/plugin/evm/vmsync/coordinator.go index 70f70828db6a..3384dc07d162 100644 --- a/graft/coreth/plugin/evm/vmsync/coordinator.go +++ b/graft/coreth/plugin/evm/vmsync/coordinator.go @@ -117,11 +117,9 @@ func (co *Coordinator) Start(ctx context.Context, initial message.Syncable) { }() } -// ProcessQueuedBlockOperations finalizes the VM at the current target and processes the -// queued operations in FIFO order. Intended to be called after a target update -// cycle when it's time to process the queued operations. +// ProcessQueuedBlockOperations finalizes the VM and processes queued block operations +// in FIFO order. Called after syncers complete to finalize state and execute deferred operations. func (co *Coordinator) ProcessQueuedBlockOperations(ctx context.Context) error { - // Check for cancellation before starting finalization phase. if err := ctx.Err(); err != nil { return err } @@ -129,7 +127,6 @@ func (co *Coordinator) ProcessQueuedBlockOperations(ctx context.Context) error { co.state.Store(int32(StateFinalizing)) if co.callbacks.FinalizeVM != nil { - // Check context again before expensive FinalizeVM operation. if err := ctx.Err(); err != nil { co.state.Store(int32(StateAborted)) return err @@ -141,15 +138,12 @@ func (co *Coordinator) ProcessQueuedBlockOperations(ctx context.Context) error { co.state.Store(int32(StateAborted)) return errInvalidTargetType } - // FinalizeVM should complete atomically. The context is passed for internal - // cancellation checks, but the coordinator expects completion or an error. if err := co.callbacks.FinalizeVM(ctx, current); err != nil { co.state.Store(int32(StateAborted)) return err } } - // Check context before transitioning to batch execution state. if err := ctx.Err(); err != nil { co.state.Store(int32(StateAborted)) return err @@ -157,35 +151,24 @@ func (co *Coordinator) ProcessQueuedBlockOperations(ctx context.Context) error { co.state.Store(int32(StateExecutingBatch)) - // Execute queued block operations sequentially. Each operation can be - // cancelled individually, but the batch execution itself is not atomic - partial completion - // is acceptable as operations are idempotent. if err := co.executeBlockOperationBatch(ctx); err != nil { - // State will be set to StateAborted by finish() when error is returned. return err } return nil } -// UpdateSyncTarget broadcasts a new target to all updatable syncers. -// It is only valid in the [StateRunning] state. It cannot be called during -// batch execution to prevent removing blocks that are currently being processed. -// Note: no batch execution occurs here. Batches are only executed after -// finalization. -// Note: Syncers manage cancellation themselves through their Sync() contexts. +// UpdateSyncTarget broadcasts a new target to all syncers and removes stale blocks from queue. +// Only valid in [StateRunning] state. Syncers manage cancellation themselves. func (co *Coordinator) UpdateSyncTarget(newTarget message.Syncable) error { - // Re-check state after each operation to handle concurrent state transitions. if co.CurrentState() != StateRunning { return errInvalidState } - // Respect pivot policy if configured. if !co.pivot.shouldForward(newTarget.Height()) { return nil } - // Re-check state before modifying queue to ensure we're still in Running state - // and not in batch execution (which would cause a race with removeBelowHeight). + // Re-check state before modifying queue to handle concurrent transitions. if co.CurrentState() != StateRunning { return errInvalidState } @@ -221,9 +204,8 @@ func (co *Coordinator) CurrentState() State { return State(co.state.Load()) } -// executeBlockOperationBatch executes all queued block operations in FIFO order. -// Each operation can be cancelled individually, but the batch execution itself -// is not atomic - partial completion is acceptable as operations are idempotent. +// executeBlockOperationBatch executes queued block operations in FIFO order. +// Partial completion is acceptable as operations are idempotent. func (co *Coordinator) executeBlockOperationBatch(ctx context.Context) error { operations := co.queue.dequeueBatch() for i, op := range operations { From 30853a45dfb9dbe44d72fe3cf9c2f3adcf2f925f Mon Sep 17 00:00:00 2001 From: Tsvetan Dimitrov Date: Tue, 2 Dec 2025 15:39:39 +0200 Subject: [PATCH 11/20] test(vmsync): add unit tests for dynamic state sync flow --- .../plugin/evm/vmsync/block_queue_test.go | 76 ++++++++++ graft/coreth/plugin/evm/vmsync/coordinator.go | 9 +- .../plugin/evm/vmsync/coordinator_test.go | 135 ++++++++++++++++++ .../coreth/plugin/evm/vmsync/doubles_test.go | 25 ++++ .../plugin/evm/vmsync/pivot_policy_test.go | 27 ++++ 5 files changed, 266 insertions(+), 6 deletions(-) create mode 100644 graft/coreth/plugin/evm/vmsync/block_queue_test.go create mode 100644 graft/coreth/plugin/evm/vmsync/coordinator_test.go create mode 100644 graft/coreth/plugin/evm/vmsync/pivot_policy_test.go diff --git a/graft/coreth/plugin/evm/vmsync/block_queue_test.go b/graft/coreth/plugin/evm/vmsync/block_queue_test.go new file mode 100644 index 000000000000..f361575af315 --- /dev/null +++ b/graft/coreth/plugin/evm/vmsync/block_queue_test.go @@ -0,0 +1,76 @@ +// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package vmsync + +import ( + "sync" + "testing" + + "github.com/stretchr/testify/require" +) + +func TestBlockQueue_EnqueueAndDequeue(t *testing.T) { + q := newBlockQueue() + + // Nil block should be rejected. + require.False(t, q.enqueue(nil, OpAccept)) + + // Enqueue blocks. + for i := uint64(100); i < 105; i++ { + require.True(t, q.enqueue(newMockBlock(i), OpAccept)) + } + + // Dequeue returns all in FIFO order and clears queue. + batch := q.dequeueBatch() + require.Len(t, batch, 5) + for i, op := range batch { + require.Equal(t, uint64(100+i), op.block.GetEthBlock().NumberU64()) + } + + // Queue is now empty. + require.Empty(t, q.dequeueBatch()) +} + +func TestBlockQueue_RemoveBelowHeight(t *testing.T) { + q := newBlockQueue() + + // Enqueue blocks at heights 100-110. + for i := uint64(100); i <= 110; i++ { + q.enqueue(newMockBlock(i), OpAccept) + } + + // Remove blocks at or below height 105. + q.removeBelowHeight(105) + + // Only blocks > 105 should remain (106, 107, 108, 109, 110). + batch := q.dequeueBatch() + require.Len(t, batch, 5) + require.Equal(t, uint64(106), batch[0].block.GetEthBlock().NumberU64()) +} + +func TestBlockQueue_ConcurrentAccess(t *testing.T) { + t.Parallel() + + q := newBlockQueue() + const numGoroutines = 10 + const numOps = 100 + + var wg sync.WaitGroup + wg.Add(numGoroutines) + + for g := 0; g < numGoroutines; g++ { + go func(id int) { + defer wg.Done() + for i := 0; i < numOps; i++ { + q.enqueue(newMockBlock(uint64(id*numOps+i)), OpAccept) + } + }(g) + } + + wg.Wait() + + // All operations should have been enqueued. + batch := q.dequeueBatch() + require.Len(t, batch, numGoroutines*numOps) +} diff --git a/graft/coreth/plugin/evm/vmsync/coordinator.go b/graft/coreth/plugin/evm/vmsync/coordinator.go index 3384dc07d162..7196c278db83 100644 --- a/graft/coreth/plugin/evm/vmsync/coordinator.go +++ b/graft/coreth/plugin/evm/vmsync/coordinator.go @@ -56,10 +56,9 @@ type Coordinator struct { // doneOnce ensures [Callbacks.OnDone] is invoked at most once. doneOnce sync.Once - // pivot policy to throttle [Coordinator.UpdateSyncTarget] calls. - pivot *pivotPolicy - + // pivotInterval configures the pivot policy throttling. 0 disables throttling. pivotInterval uint64 + pivot *pivotPolicy } // CoordinatorOption follows the functional options pattern for Coordinator. @@ -79,10 +78,7 @@ func NewCoordinator(syncerRegistry *SyncerRegistry, cbs Callbacks, opts ...Coord syncerRegistry: syncerRegistry, callbacks: cbs, } - options.ApplyTo(co, opts...) - - co.pivot = newPivotPolicy(co.pivotInterval) co.state.Store(int32(StateIdle)) return co @@ -93,6 +89,7 @@ func NewCoordinator(syncerRegistry *SyncerRegistry, cbs Callbacks, opts ...Coord func (co *Coordinator) Start(ctx context.Context, initial message.Syncable) { co.state.Store(int32(StateInitializing)) co.target.Store(initial) + co.pivot = newPivotPolicy(co.pivotInterval) cctx, cancel := context.WithCancelCause(ctx) g := co.syncerRegistry.StartAsync(cctx, initial) diff --git a/graft/coreth/plugin/evm/vmsync/coordinator_test.go b/graft/coreth/plugin/evm/vmsync/coordinator_test.go new file mode 100644 index 000000000000..bb720dfcac0d --- /dev/null +++ b/graft/coreth/plugin/evm/vmsync/coordinator_test.go @@ -0,0 +1,135 @@ +// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package vmsync + +import ( + "context" + "errors" + "sync" + "testing" + + "github.com/ava-labs/libevm/common" + "github.com/stretchr/testify/require" + + "github.com/ava-labs/avalanchego/graft/coreth/plugin/evm/message" +) + +func TestCoordinator_StateValidation(t *testing.T) { + co := NewCoordinator(NewSyncerRegistry(), Callbacks{}, WithPivotInterval(1)) + block := newMockBlock(100) + target := newTestSyncTarget(100) + + // States that reject both operations. + for _, state := range []State{StateIdle, StateInitializing, StateFinalizing, StateCompleted, StateAborted} { + co.state.Store(int32(state)) + require.False(t, co.AddBlockOperation(block, OpAccept), "state %d should reject block", state) + require.ErrorIs(t, co.UpdateSyncTarget(target), errInvalidState, "state %d should reject target update", state) + } + + // Running: accepts both. + co.state.Store(int32(StateRunning)) + require.True(t, co.AddBlockOperation(block, OpAccept)) + require.NoError(t, co.UpdateSyncTarget(target)) + + // ExecutingBatch: accepts blocks, rejects target updates. + co.state.Store(int32(StateExecutingBatch)) + require.True(t, co.AddBlockOperation(block, OpAccept)) + require.ErrorIs(t, co.UpdateSyncTarget(target), errInvalidState) + + // Nil block is always rejected. + co.state.Store(int32(StateRunning)) + require.False(t, co.AddBlockOperation(nil, OpAccept)) +} + +func TestCoordinator_UpdateSyncTarget_RemovesStaleBlocks(t *testing.T) { + co := NewCoordinator(NewSyncerRegistry(), Callbacks{}, WithPivotInterval(1)) + co.state.Store(int32(StateRunning)) + + for i := uint64(100); i <= 110; i++ { + co.AddBlockOperation(newMockBlock(i), OpAccept) + } + + require.NoError(t, co.UpdateSyncTarget(newTestSyncTarget(105))) + + batch := co.queue.dequeueBatch() + require.Len(t, batch, 5) // Only 106-110 remain. +} + +func TestCoordinator_Lifecycle(t *testing.T) { + t.Run("completes successfully", func(t *testing.T) { + registry := NewSyncerRegistry() + require.NoError(t, registry.Register(newMockSyncer("test", nil))) + + co, err := runCoordinator(t, registry, Callbacks{ + FinalizeVM: func(context.Context, message.Syncable) error { return nil }, + }) + + require.NoError(t, err) + require.Equal(t, StateCompleted, co.CurrentState()) + }) + + t.Run("aborts on syncer error", func(t *testing.T) { + expectedErr := errors.New("syncer failed") + registry := NewSyncerRegistry() + require.NoError(t, registry.Register(newMockSyncer("failing", expectedErr))) + + co, err := runCoordinator(t, registry, Callbacks{}) + + require.ErrorIs(t, err, expectedErr) + require.Equal(t, StateAborted, co.CurrentState()) + }) +} + +func TestCoordinator_ProcessQueuedBlockOperations(t *testing.T) { + t.Run("executes queued operations", func(t *testing.T) { + co := NewCoordinator(NewSyncerRegistry(), Callbacks{}) + co.state.Store(int32(StateRunning)) + co.target.Store(newTestSyncTarget(100)) + co.AddBlockOperation(newMockBlock(100), OpAccept) + + require.NoError(t, co.ProcessQueuedBlockOperations(t.Context())) + require.Equal(t, StateExecutingBatch, co.CurrentState()) + }) + + t.Run("returns error on block operation failure", func(t *testing.T) { + co := NewCoordinator(NewSyncerRegistry(), Callbacks{}) + co.state.Store(int32(StateRunning)) + co.target.Store(newTestSyncTarget(100)) + + failBlock := newMockBlock(100) + failBlock.acceptErr = errors.New("accept failed") + co.AddBlockOperation(failBlock, OpAccept) + + err := co.ProcessQueuedBlockOperations(t.Context()) + require.ErrorIs(t, err, errBatchOperationFailed) + }) +} + +// runCoordinator starts a coordinator and waits for completion. +func runCoordinator(t *testing.T, registry *SyncerRegistry, cbs Callbacks) (*Coordinator, error) { + t.Helper() + + var ( + errDone error + wg sync.WaitGroup + ) + wg.Add(1) + + cbs.OnDone = func(err error) { + errDone = err + wg.Done() + } + + co := NewCoordinator(registry, cbs) + co.Start(t.Context(), newTestSyncTarget(100)) + wg.Wait() + + return co, errDone +} + +func newTestSyncTarget(height uint64) message.Syncable { + hash := common.BytesToHash([]byte{byte(height)}) + root := common.BytesToHash([]byte{byte(height + 1)}) + return newSyncTarget(hash, root, height) +} diff --git a/graft/coreth/plugin/evm/vmsync/doubles_test.go b/graft/coreth/plugin/evm/vmsync/doubles_test.go index b63746dd571e..9ed1ecc1b65e 100644 --- a/graft/coreth/plugin/evm/vmsync/doubles_test.go +++ b/graft/coreth/plugin/evm/vmsync/doubles_test.go @@ -6,14 +6,39 @@ package vmsync import ( "context" "errors" + "math/big" "sync" "time" + "github.com/ava-labs/libevm/core/types" + "github.com/ava-labs/avalanchego/graft/coreth/plugin/evm/message" syncpkg "github.com/ava-labs/avalanchego/graft/coreth/sync" ) +// mockEthBlockWrapper implements EthBlockWrapper for testing. +type mockEthBlockWrapper struct { + ethBlock *types.Block + acceptErr error + rejectErr error + verifyErr error +} + +func newMockBlock(height uint64) *mockEthBlockWrapper { + header := &types.Header{Number: new(big.Int).SetUint64(height)} + return &mockEthBlockWrapper{ + ethBlock: types.NewBlockWithHeader(header), + } +} + +func (m *mockEthBlockWrapper) GetEthBlock() *types.Block { return m.ethBlock } +func (m *mockEthBlockWrapper) Accept(context.Context) error { return m.acceptErr } +func (m *mockEthBlockWrapper) Reject(context.Context) error { return m.rejectErr } +func (m *mockEthBlockWrapper) Verify(context.Context) error { return m.verifyErr } + +var _ EthBlockWrapper = (*mockEthBlockWrapper)(nil) + // FuncSyncer adapts a function to the simple Syncer shape used in tests. It is // useful for defining small, behavior-driven syncers inline. type FuncSyncer struct { diff --git a/graft/coreth/plugin/evm/vmsync/pivot_policy_test.go b/graft/coreth/plugin/evm/vmsync/pivot_policy_test.go new file mode 100644 index 000000000000..5237a102c0d1 --- /dev/null +++ b/graft/coreth/plugin/evm/vmsync/pivot_policy_test.go @@ -0,0 +1,27 @@ +// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package vmsync + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +func TestPivotPolicy(t *testing.T) { + // Zero interval uses default. + require.Equal(t, defaultPivotInterval, newPivotPolicy(0).interval) + + // Test throttling behavior. + p := newPivotPolicy(100) + + // First call at 150 initializes threshold to ceil(150/100)*100 = 200 + require.False(t, p.shouldForward(150)) // 150 < 200 + require.False(t, p.shouldForward(199)) // 199 < 200 + require.True(t, p.shouldForward(200)) // 200 >= 200 + p.advance() // threshold becomes 300 + + require.False(t, p.shouldForward(250)) // 250 < 300 + require.True(t, p.shouldForward(300)) // 300 >= 300 +} From 5db739ee8e10a0ae62f85b0f91f0da9c3b6c0043 Mon Sep 17 00:00:00 2001 From: Tsvetan Dimitrov Date: Tue, 2 Dec 2025 15:39:39 +0200 Subject: [PATCH 12/20] test(vmsync): add unit tests for dynamic state sync flow --- graft/coreth/plugin/evm/vmsync/coordinator_test.go | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/graft/coreth/plugin/evm/vmsync/coordinator_test.go b/graft/coreth/plugin/evm/vmsync/coordinator_test.go index bb720dfcac0d..678dbdb035fd 100644 --- a/graft/coreth/plugin/evm/vmsync/coordinator_test.go +++ b/graft/coreth/plugin/evm/vmsync/coordinator_test.go @@ -24,7 +24,8 @@ func TestCoordinator_StateValidation(t *testing.T) { for _, state := range []State{StateIdle, StateInitializing, StateFinalizing, StateCompleted, StateAborted} { co.state.Store(int32(state)) require.False(t, co.AddBlockOperation(block, OpAccept), "state %d should reject block", state) - require.ErrorIs(t, co.UpdateSyncTarget(target), errInvalidState, "state %d should reject target update", state) + err := co.UpdateSyncTarget(target) + require.ErrorIs(t, err, errInvalidState, "state %d should reject target update", state) } // Running: accepts both. @@ -35,7 +36,8 @@ func TestCoordinator_StateValidation(t *testing.T) { // ExecutingBatch: accepts blocks, rejects target updates. co.state.Store(int32(StateExecutingBatch)) require.True(t, co.AddBlockOperation(block, OpAccept)) - require.ErrorIs(t, co.UpdateSyncTarget(target), errInvalidState) + err := co.UpdateSyncTarget(target) + require.ErrorIs(t, err, errInvalidState) // Nil block is always rejected. co.state.Store(int32(StateRunning)) From 58abd3ebf0cb3b71840434a9edbda13349d499a5 Mon Sep 17 00:00:00 2001 From: Tsvetan Dimitrov Date: Tue, 2 Dec 2025 15:56:20 +0200 Subject: [PATCH 13/20] refactor(vmsync): introduce SyncStrategy pattern for static/dynamic sync Extract sync orchestration into a strategy pattern to improve code organization and separation of concerns. - Add SyncStrategy interface in client.go for sync orchestration. - Extract VM finalization logic to finalizer.go with sentinel errors. - Add staticStrategy for sequential sync without block queueing. - Add dynamicStrategy wrapping Coordinator for concurrent sync. - Simplify client.go by delegating to strategies. - Simplify sync_target.go by removing redundant ID field. - Move syncer creation to standalone newSyncerRegistry function. --- .../plugin/evm/vmsync/strategy_dynamic.go | 106 ++++++++++++++++++ graft/coreth/plugin/evm/vmsync/sync_target.go | 22 ++-- 2 files changed, 114 insertions(+), 14 deletions(-) create mode 100644 graft/coreth/plugin/evm/vmsync/strategy_dynamic.go diff --git a/graft/coreth/plugin/evm/vmsync/strategy_dynamic.go b/graft/coreth/plugin/evm/vmsync/strategy_dynamic.go new file mode 100644 index 000000000000..6a2529d18fe9 --- /dev/null +++ b/graft/coreth/plugin/evm/vmsync/strategy_dynamic.go @@ -0,0 +1,106 @@ +// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package vmsync + +import ( + "context" + "fmt" + + "github.com/ava-labs/libevm/log" + + "github.com/ava-labs/avalanchego/graft/coreth/plugin/evm/message" +) + +var _ SyncStrategy = (*dynamicStrategy)(nil) + +// dynamicStrategy runs syncers concurrently with block queueing. +// It wraps [Coordinator] to manage the sync lifecycle. +type dynamicStrategy struct { + coordinator *Coordinator +} + +func newDynamicStrategy(registry *SyncerRegistry, finalizer *finalizer, pivotInterval uint64) *dynamicStrategy { + coordinator := NewCoordinator( + registry, + Callbacks{ + FinalizeVM: finalizer.finalize, + OnDone: nil, // Set in Start to capture completion. + }, + WithPivotInterval(pivotInterval), + ) + return &dynamicStrategy{coordinator: coordinator} +} + +// Start launches the coordinator and blocks until sync completes or fails. +func (d *dynamicStrategy) Start(ctx context.Context, summary message.Syncable) error { + done := make(chan error, 1) + + // Wire up OnDone to signal completion. + d.coordinator.callbacks.OnDone = func(err error) { + if err != nil { + log.Error("dynamic state sync completed with error", "err", err) + } else { + log.Info("dynamic state sync completed successfully") + } + done <- err + } + + d.coordinator.Start(ctx, summary) + return <-done +} + +// OnBlockAccepted enqueues the block for deferred processing and updates the sync target. +func (d *dynamicStrategy) OnBlockAccepted(b EthBlockWrapper) (bool, error) { + if d.coordinator.CurrentState() == StateExecutingBatch { + // Still enqueue for the next batch, but don't update target. + return d.enqueue(b, OpAccept), nil + } + + if !d.enqueue(b, OpAccept) { + return false, nil + } + + ethb := b.GetEthBlock() + target := newSyncTarget(ethb.Hash(), ethb.Root(), ethb.NumberU64()) + if err := d.coordinator.UpdateSyncTarget(target); err != nil { + // Block is enqueued but target update failed. + return true, fmt.Errorf("block enqueued but sync target update failed: %w", err) + } + return true, nil +} + +// OnBlockRejected enqueues the block for deferred rejection. +func (d *dynamicStrategy) OnBlockRejected(b EthBlockWrapper) (bool, error) { + return d.enqueue(b, OpReject), nil +} + +// OnBlockVerified enqueues the block for deferred verification. +func (d *dynamicStrategy) OnBlockVerified(b EthBlockWrapper) (bool, error) { + return d.enqueue(b, OpVerify), nil +} + +// enqueue adds a block operation to the coordinator's queue. +func (d *dynamicStrategy) enqueue(b EthBlockWrapper, op BlockOperationType) bool { + ok := d.coordinator.AddBlockOperation(b, op) + if !ok { + if ethb := b.GetEthBlock(); ethb != nil { + log.Warn("could not enqueue block operation", + "hash", ethb.Hash(), + "height", ethb.NumberU64(), + "op", op.String(), + ) + } + } + return ok +} + +// CurrentState returns the coordinator's current state. +func (d *dynamicStrategy) CurrentState() State { + return d.coordinator.CurrentState() +} + +// UpdateSyncTarget updates the coordinator's sync target. +func (d *dynamicStrategy) UpdateSyncTarget(target message.Syncable) error { + return d.coordinator.UpdateSyncTarget(target) +} diff --git a/graft/coreth/plugin/evm/vmsync/sync_target.go b/graft/coreth/plugin/evm/vmsync/sync_target.go index 20412ebbca6f..539e3ed6e9ed 100644 --- a/graft/coreth/plugin/evm/vmsync/sync_target.go +++ b/graft/coreth/plugin/evm/vmsync/sync_target.go @@ -15,33 +15,27 @@ import ( var _ message.Syncable = (*syncTarget)(nil) -// syncTarget is a minimal implementation of message.Syncable used internally +// syncTarget is a minimal implementation of [message.Syncable] used internally // to advance the coordinator's sync target from engine-accepted blocks. +// +// NOTE: Unlike [message.BlockSyncSummary], this is not serializable and should not +// be used for network communication. Only [message.Syncable.GetBlockHash], +// [message.Syncable.GetBlockRoot], and [message.Syncable.Height] are used in practice. +// The other methods are stubs to satisfy the interface. type syncTarget struct { - id ids.ID hash common.Hash root common.Hash height uint64 } -// Build a sync target from basic fields. func newSyncTarget(hash common.Hash, root common.Hash, height uint64) message.Syncable { - var id ids.ID - copy(id[:], hash[:]) - return &syncTarget{ - id: id, - hash: hash, - root: root, - height: height, - } + return &syncTarget{hash: hash, root: root, height: height} } -// message.Syncable func (s *syncTarget) GetBlockHash() common.Hash { return s.hash } func (s *syncTarget) GetBlockRoot() common.Hash { return s.root } -// block.StateSummary -func (s *syncTarget) ID() ids.ID { return s.id } +func (s *syncTarget) ID() ids.ID { return ids.ID(s.hash) } func (s *syncTarget) Height() uint64 { return s.height } func (s *syncTarget) Bytes() []byte { return s.hash.Bytes() } func (*syncTarget) Accept(context.Context) (block.StateSyncMode, error) { From 05f57635607bb58cca23e373be7d0c659ef37960 Mon Sep 17 00:00:00 2001 From: Tsvetan Dimitrov Date: Wed, 3 Dec 2025 23:11:04 +0200 Subject: [PATCH 14/20] refactor(vmsync): use explicit config field instead of type embedding - Replace embedded *ClientConfig with named config field for clearer field access. Convert newSyncerRegistry to client method. --- graft/coreth/plugin/evm/vmsync/client.go | 86 +++++++++++------------- 1 file changed, 39 insertions(+), 47 deletions(-) diff --git a/graft/coreth/plugin/evm/vmsync/client.go b/graft/coreth/plugin/evm/vmsync/client.go index e6ca2375b1be..e567e5d259ba 100644 --- a/graft/coreth/plugin/evm/vmsync/client.go +++ b/graft/coreth/plugin/evm/vmsync/client.go @@ -97,25 +97,17 @@ type ClientConfig struct { } type client struct { - *ClientConfig - + config *ClientConfig resumableSummary message.Syncable - - cancel context.CancelFunc - wg sync.WaitGroup - - // State Sync results - err error - stateSyncOnce sync.Once - - // strategy manages sync execution (static or dynamic) - strategy SyncStrategy + cancel context.CancelFunc + wg sync.WaitGroup + err error + stateSyncOnce sync.Once // ensures only one state sync can be in progress at a time + strategy SyncStrategy // strategy manages sync execution (static or dynamic) } func NewClient(config *ClientConfig) Client { - return &client{ - ClientConfig: config, - } + return &client{config: config} } type Client interface { @@ -141,23 +133,23 @@ type Client interface { // StateSyncEnabled returns [client.enabled], which is set in the chain's config file. func (c *client) StateSyncEnabled(context.Context) (bool, error) { - return c.Enabled, nil + return c.config.Enabled, nil } // GetOngoingSyncStateSummary returns a state summary that was previously started // and not finished, and sets [resumableSummary] if one was found. // Returns [database.ErrNotFound] if no ongoing summary is found or if [client.skipResume] is true. func (c *client) GetOngoingSyncStateSummary(context.Context) (block.StateSummary, error) { - if c.SkipResume { + if c.config.SkipResume { return nil, database.ErrNotFound } - summaryBytes, err := c.MetadataDB.Get(stateSyncSummaryKey) + summaryBytes, err := c.config.MetadataDB.Get(stateSyncSummaryKey) if err != nil { return nil, err // includes the [database.ErrNotFound] case } - summary, err := c.Parser.Parse(summaryBytes, c.acceptSyncSummary) + summary, err := c.config.Parser.Parse(summaryBytes, c.acceptSyncSummary) if err != nil { return nil, fmt.Errorf("failed to parse saved state sync summary to SyncSummary: %w", err) } @@ -167,10 +159,10 @@ func (c *client) GetOngoingSyncStateSummary(context.Context) (block.StateSummary // ClearOngoingSummary clears any marker of an ongoing state sync summary func (c *client) ClearOngoingSummary() error { - if err := c.MetadataDB.Delete(stateSyncSummaryKey); err != nil { + if err := c.config.MetadataDB.Delete(stateSyncSummaryKey); err != nil { return fmt.Errorf("failed to clear ongoing summary: %w", err) } - if err := c.VerDB.Commit(); err != nil { + if err := c.config.VerDB.Commit(); err != nil { return fmt.Errorf("failed to commit db while clearing ongoing summary: %w", err) } @@ -179,7 +171,7 @@ func (c *client) ClearOngoingSummary() error { // ParseStateSummary parses [summaryBytes] to [commonEng.Summary] func (c *client) ParseStateSummary(_ context.Context, summaryBytes []byte) (block.StateSummary, error) { - return c.Parser.Parse(summaryBytes, c.acceptSyncSummary) + return c.config.Parser.Parse(summaryBytes, c.acceptSyncSummary) } // OnEngineAccept delegates to the strategy if active. @@ -233,10 +225,10 @@ func (c *client) acceptSyncSummary(proposedSummary message.Syncable) (block.Stat if !isResume { // Skip syncing if the blockchain is not significantly ahead of local state, // since bootstrapping would be faster. - if c.LastAcceptedHeight+c.MinBlocks > proposedSummary.Height() { + if c.config.LastAcceptedHeight+c.config.MinBlocks > proposedSummary.Height() { log.Info( "last accepted too close to most recent syncable block, skipping state sync", - "lastAccepted", c.LastAcceptedHeight, + "lastAccepted", c.config.LastAcceptedHeight, "syncableHeight", proposedSummary.Height(), ) return block.StateSyncSkipped, nil @@ -248,18 +240,18 @@ func (c *client) acceptSyncSummary(proposedSummary message.Syncable) (block.Stat // sync marker will be wiped, so we do not accidentally resume progress from an incorrect version // of the snapshot. (if switching between versions that come before this change and back this could // lead to the snapshot not being cleaned up correctly) - <-snapshot.WipeSnapshot(c.ChainDB, true) + <-snapshot.WipeSnapshot(c.config.ChainDB, true) // Reset the snapshot generator here so that when state sync completes, snapshots will not attempt to read an // invalid generator. // Note: this must be called after WipeSnapshot is called so that we do not invalidate a partially generated snapshot. - snapshot.ResetSnapshotGeneration(c.ChainDB) + snapshot.ResetSnapshotGeneration(c.config.ChainDB) } // Update the current state sync summary key in the database. - if err := c.MetadataDB.Put(stateSyncSummaryKey, proposedSummary.Bytes()); err != nil { + if err := c.config.MetadataDB.Put(stateSyncSummaryKey, proposedSummary.Bytes()); err != nil { return block.StateSyncSkipped, fmt.Errorf("failed to write state sync summary key to disk: %w", err) } - if err := c.VerDB.Commit(); err != nil { + if err := c.config.VerDB.Commit(); err != nil { return block.StateSyncSkipped, fmt.Errorf("failed to commit db: %w", err) } @@ -267,27 +259,27 @@ func (c *client) acceptSyncSummary(proposedSummary message.Syncable) (block.Stat ctx, cancel := context.WithCancel(context.Background()) c.cancel = cancel - registry, err := newSyncerRegistry(c.ClientConfig, proposedSummary) + registry, err := c.newSyncerRegistry(proposedSummary) if err != nil { return block.StateSyncSkipped, err } finalizer := newFinalizer( - c.Chain, - c.State, - c.Acceptor, - c.VerDB, - c.MetadataDB, - c.Extender, - c.LastAcceptedHeight, + c.config.Chain, + c.config.State, + c.config.Acceptor, + c.config.VerDB, + c.config.MetadataDB, + c.config.Extender, + c.config.LastAcceptedHeight, ) var ( strategy SyncStrategy mode block.StateSyncMode ) - if c.DynamicStateSyncEnabled { - strategy = newDynamicStrategy(registry, finalizer, c.PivotInterval) + if c.config.DynamicStateSyncEnabled { + strategy = newDynamicStrategy(registry, finalizer, c.config.PivotInterval) mode = block.StateSyncDynamic } else { strategy = newStaticStrategy(registry, finalizer) @@ -313,16 +305,16 @@ func (c *client) signalDone(err error) { if c.cancel != nil { c.cancel() } - close(c.StateSyncDone) + close(c.config.StateSyncDone) }) } // newSyncerRegistry creates a registry with all required syncers for the given summary. -func newSyncerRegistry(cfg *ClientConfig, summary message.Syncable) (*SyncerRegistry, error) { +func (c *client) newSyncerRegistry(summary message.Syncable) (*SyncerRegistry, error) { registry := NewSyncerRegistry() blockSyncer, err := blocksync.NewSyncer( - cfg.Client, cfg.ChainDB, + c.config.Client, c.config.ChainDB, summary.GetBlockHash(), summary.Height(), BlocksToFetch, ) @@ -330,20 +322,20 @@ func newSyncerRegistry(cfg *ClientConfig, summary message.Syncable) (*SyncerRegi return nil, fmt.Errorf("failed to create block syncer: %w", err) } - codeQueue, err := statesync.NewCodeQueue(cfg.ChainDB, cfg.StateSyncDone) + codeQueue, err := statesync.NewCodeQueue(c.config.ChainDB, c.config.StateSyncDone) if err != nil { return nil, fmt.Errorf("failed to create code queue: %w", err) } - codeSyncer, err := statesync.NewCodeSyncer(cfg.Client, cfg.ChainDB, codeQueue.CodeHashes()) + codeSyncer, err := statesync.NewCodeSyncer(c.config.Client, c.config.ChainDB, codeQueue.CodeHashes()) if err != nil { return nil, fmt.Errorf("failed to create code syncer: %w", err) } stateSyncer, err := statesync.NewSyncer( - cfg.Client, cfg.ChainDB, + c.config.Client, c.config.ChainDB, summary.GetBlockRoot(), - codeQueue, cfg.RequestSize, + codeQueue, c.config.RequestSize, ) if err != nil { return nil, fmt.Errorf("failed to create EVM state syncer: %w", err) @@ -351,8 +343,8 @@ func newSyncerRegistry(cfg *ClientConfig, summary message.Syncable) (*SyncerRegi syncers := []syncpkg.Syncer{blockSyncer, codeSyncer, stateSyncer} - if cfg.Extender != nil { - atomicSyncer, err := cfg.Extender.CreateSyncer(cfg.Client, cfg.VerDB, summary) + if c.config.Extender != nil { + atomicSyncer, err := c.config.Extender.CreateSyncer(c.config.Client, c.config.VerDB, summary) if err != nil { return nil, fmt.Errorf("failed to create atomic syncer: %w", err) } From a5f7cbc13532f45ab80f577c315ed0838f061a34 Mon Sep 17 00:00:00 2001 From: Tsvetan Dimitrov Date: Wed, 3 Dec 2025 23:19:05 +0200 Subject: [PATCH 15/20] refactor(vmsync): refinements on sync strategy definition --- graft/coreth/plugin/evm/vmsync/client.go | 12 ++++++------ graft/coreth/plugin/evm/vmsync/strategy_static.go | 10 ++++------ 2 files changed, 10 insertions(+), 12 deletions(-) diff --git a/graft/coreth/plugin/evm/vmsync/client.go b/graft/coreth/plugin/evm/vmsync/client.go index efd7c4a72076..c2e77a7c18b8 100644 --- a/graft/coreth/plugin/evm/vmsync/client.go +++ b/graft/coreth/plugin/evm/vmsync/client.go @@ -17,14 +17,14 @@ import ( "github.com/ava-labs/avalanchego/graft/coreth/core/state/snapshot" "github.com/ava-labs/avalanchego/graft/coreth/eth" "github.com/ava-labs/avalanchego/graft/coreth/plugin/evm/message" + "github.com/ava-labs/avalanchego/graft/coreth/sync/blocksync" + "github.com/ava-labs/avalanchego/graft/coreth/sync/statesync" "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/avalanchego/snow/engine/snowman/block" "github.com/ava-labs/avalanchego/vms/components/chain" syncpkg "github.com/ava-labs/avalanchego/graft/coreth/sync" - "github.com/ava-labs/avalanchego/graft/coreth/sync/blocksync" syncclient "github.com/ava-labs/avalanchego/graft/coreth/sync/client" - "github.com/ava-labs/avalanchego/graft/coreth/sync/statesync" ) // BlocksToFetch is the number of the block parents the state syncs to. @@ -32,7 +32,7 @@ import ( const BlocksToFetch = 256 var ( - errSkipSync = fmt.Errorf("skip sync") + errSkipSync = errors.New("skip sync") stateSyncSummaryKey = []byte("stateSyncSummary") ) @@ -47,7 +47,7 @@ type BlockAcceptor interface { // Implementations handle the sync lifecycle differently based on sync mode. type SyncStrategy interface { // Start begins the sync process and blocks until completion or error. - Start(ctx context.Context) error + Start(ctx context.Context, summary message.Syncable) error } type ClientConfig struct { @@ -170,7 +170,7 @@ func (c *client) acceptSyncSummary(summary message.Syncable) (block.StateSyncMod c.config.LastAcceptedHeight, ) - strategy := newStaticStrategy(registry, finalizer, summary) + strategy := newStaticStrategy(registry, finalizer) return c.startAsync(strategy), nil } @@ -228,7 +228,7 @@ func (c *client) startAsync(strategy SyncStrategy) block.StateSyncMode { defer c.wg.Done() defer cancel() - if err := strategy.Start(ctx); err != nil { + if err := strategy.Start(ctx, c.resumableSummary); err != nil { c.err = err } // notify engine regardless of whether err == nil, diff --git a/graft/coreth/plugin/evm/vmsync/strategy_static.go b/graft/coreth/plugin/evm/vmsync/strategy_static.go index ba1d6c7aae67..88afed6309ad 100644 --- a/graft/coreth/plugin/evm/vmsync/strategy_static.go +++ b/graft/coreth/plugin/evm/vmsync/strategy_static.go @@ -17,22 +17,20 @@ var _ SyncStrategy = (*staticStrategy)(nil) type staticStrategy struct { registry *SyncerRegistry finalizer *finalizer - summary message.Syncable } -func newStaticStrategy(registry *SyncerRegistry, finalizer *finalizer, summary message.Syncable) *staticStrategy { +func newStaticStrategy(registry *SyncerRegistry, finalizer *finalizer) *staticStrategy { return &staticStrategy{ registry: registry, finalizer: finalizer, - summary: summary, } } // Start begins the sync process and blocks until completion or error. // For static sync, this runs all syncers and then finalizes the VM state. -func (s *staticStrategy) Start(ctx context.Context) error { - if err := s.registry.RunSyncerTasks(ctx, s.summary); err != nil { +func (s *staticStrategy) Start(ctx context.Context, summary message.Syncable) error { + if err := s.registry.RunSyncerTasks(ctx, summary); err != nil { return err } - return s.finalizer.finalize(ctx, s.summary) + return s.finalizer.finalize(ctx, summary) } From 0283266ae4a69eca623f58998abc57d513423d2c Mon Sep 17 00:00:00 2001 From: Tsvetan Dimitrov Date: Wed, 3 Dec 2025 23:33:07 +0200 Subject: [PATCH 16/20] fix(vmsync): pass summary to startAsync instead of using resumableSummary --- graft/coreth/plugin/evm/vmsync/client.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/graft/coreth/plugin/evm/vmsync/client.go b/graft/coreth/plugin/evm/vmsync/client.go index c2e77a7c18b8..7ad0ef090350 100644 --- a/graft/coreth/plugin/evm/vmsync/client.go +++ b/graft/coreth/plugin/evm/vmsync/client.go @@ -172,7 +172,7 @@ func (c *client) acceptSyncSummary(summary message.Syncable) (block.StateSyncMod strategy := newStaticStrategy(registry, finalizer) - return c.startAsync(strategy), nil + return c.startAsync(strategy, summary), nil } // prepareForSync handles resume check and snapshot wipe before sync starts. @@ -219,7 +219,7 @@ func (c *client) prepareForSync(summary message.Syncable) error { } // startAsync launches the sync strategy in a background goroutine. -func (c *client) startAsync(strategy SyncStrategy) block.StateSyncMode { +func (c *client) startAsync(strategy SyncStrategy, summary message.Syncable) block.StateSyncMode { ctx, cancel := context.WithCancel(context.Background()) c.cancel = cancel @@ -228,7 +228,7 @@ func (c *client) startAsync(strategy SyncStrategy) block.StateSyncMode { defer c.wg.Done() defer cancel() - if err := strategy.Start(ctx, c.resumableSummary); err != nil { + if err := strategy.Start(ctx, summary); err != nil { c.err = err } // notify engine regardless of whether err == nil, From ed5e8e78544f00819f0cdebaac23440e263e7770 Mon Sep 17 00:00:00 2001 From: Tsvetan Dimitrov Date: Tue, 16 Dec 2025 14:56:22 +0200 Subject: [PATCH 17/20] refactor(vmsync): replace finalizer struct with Committer interface - Remove the separate finalizer struct that duplicated fields from ClientConfig. The client now implements the Committer interface directly, allowing strategies to call Commit() without extra indirection. - This simplifies the codebase by eliminating field duplication and follows Go idioms by defining the interface at the point of use. --- graft/coreth/plugin/evm/vmsync/client.go | 121 ++++++++++++-- graft/coreth/plugin/evm/vmsync/finalizer.go | 152 ------------------ .../plugin/evm/vmsync/strategy_static.go | 12 +- 3 files changed, 114 insertions(+), 171 deletions(-) delete mode 100644 graft/coreth/plugin/evm/vmsync/finalizer.go diff --git a/graft/coreth/plugin/evm/vmsync/client.go b/graft/coreth/plugin/evm/vmsync/client.go index 7ad0ef090350..6b99191fd5f8 100644 --- a/graft/coreth/plugin/evm/vmsync/client.go +++ b/graft/coreth/plugin/evm/vmsync/client.go @@ -9,6 +9,7 @@ import ( "fmt" "sync" + "github.com/ava-labs/libevm/core/types" "github.com/ava-labs/libevm/ethdb" "github.com/ava-labs/libevm/log" @@ -16,6 +17,7 @@ import ( "github.com/ava-labs/avalanchego/database/versiondb" "github.com/ava-labs/avalanchego/graft/coreth/core/state/snapshot" "github.com/ava-labs/avalanchego/graft/coreth/eth" + "github.com/ava-labs/avalanchego/graft/coreth/params" "github.com/ava-labs/avalanchego/graft/coreth/plugin/evm/message" "github.com/ava-labs/avalanchego/graft/coreth/sync/blocksync" "github.com/ava-labs/avalanchego/graft/coreth/sync/statesync" @@ -32,10 +34,23 @@ import ( const BlocksToFetch = 256 var ( - errSkipSync = errors.New("skip sync") - stateSyncSummaryKey = []byte("stateSyncSummary") + errSkipSync = errors.New("skip sync") + errBlockNotFound = errors.New("block not found in state") + errInvalidBlockType = errors.New("invalid block wrapper type") + errBlockHashMismatch = errors.New("block hash mismatch") + errBlockHeightMismatch = errors.New("block height mismatch") + errCommitCancelled = errors.New("commit cancelled") + errCommitMarkers = errors.New("failed to commit VM markers") + stateSyncSummaryKey = []byte("stateSyncSummary") ) +// EthBlockWrapper can be implemented by a concrete block wrapper type to +// return *types.Block, which is needed to update chain pointers at the +// end of the sync operation. +type EthBlockWrapper interface { + GetEthBlock() *types.Block +} + // BlockAcceptor provides a mechanism to update the last accepted block ID during state synchronization. // This interface is used by the state sync process to ensure the blockchain state // is properly updated when new blocks are synchronized from the network. @@ -43,6 +58,11 @@ type BlockAcceptor interface { PutLastAcceptedID(ids.ID) error } +// Committer commits sync results to the VM, preparing it for bootstrapping. +type Committer interface { + Commit(ctx context.Context, summary message.Syncable) error +} + // SyncStrategy defines how state sync is executed. // Implementations handle the sync lifecycle differently based on sync mode. type SyncStrategy interface { @@ -50,6 +70,8 @@ type SyncStrategy interface { Start(ctx context.Context, summary message.Syncable) error } +var _ Committer = (*client)(nil) + type ClientConfig struct { Chain *eth.Ethereum State *chain.State @@ -160,17 +182,7 @@ func (c *client) acceptSyncSummary(summary message.Syncable) (block.StateSyncMod return block.StateSyncSkipped, fmt.Errorf("failed to create syncer registry: %w", err) } - finalizer := newFinalizer( - c.config.Chain, - c.config.State, - c.config.Acceptor, - c.config.VerDB, - c.config.MetadataDB, - c.config.Extender, - c.config.LastAcceptedHeight, - ) - - strategy := newStaticStrategy(registry, finalizer) + strategy := newStaticStrategy(registry, c) return c.startAsync(strategy, summary), nil } @@ -253,6 +265,89 @@ func (c *client) Shutdown() error { // Error returns a non-nil error if one occurred during the sync. func (c *client) Error() error { return c.err } +// Commit implements Committer. It updates disk and memory pointers so the VM +// is prepared for bootstrapping. Executes any shared memory operations from +// the atomic trie to shared memory. +func (c *client) Commit(ctx context.Context, summary message.Syncable) error { + stateBlock, err := c.config.State.GetBlock(ctx, ids.ID(summary.GetBlockHash())) + if err != nil { + return fmt.Errorf("%w: hash=%s", errBlockNotFound, summary.GetBlockHash()) + } + + wrapper, ok := stateBlock.(*chain.BlockWrapper) + if !ok { + return fmt.Errorf("%w: got %T, want *chain.BlockWrapper", errInvalidBlockType, stateBlock) + } + wrappedBlock := wrapper.Block + + evmBlockGetter, ok := wrappedBlock.(EthBlockWrapper) + if !ok { + return fmt.Errorf("%w: got %T, want EthBlockWrapper", errInvalidBlockType, wrappedBlock) + } + + block := evmBlockGetter.GetEthBlock() + + if block.Hash() != summary.GetBlockHash() { + return fmt.Errorf("%w: got %s, want %s", errBlockHashMismatch, block.Hash(), summary.GetBlockHash()) + } + if block.NumberU64() != summary.Height() { + return fmt.Errorf("%w: got %d, want %d", errBlockHeightMismatch, block.NumberU64(), summary.Height()) + } + + // BloomIndexer needs to know that some parts of the chain are not available + // and cannot be indexed. This is done by calling [AddCheckpoint] here. + // Since the indexer uses sections of size [params.BloomBitsBlocks] (= 4096), + // each block is indexed in section number [blockNumber/params.BloomBitsBlocks]. + // To allow the indexer to start with the block we just synced to, + // we create a checkpoint for its parent. + // Note: This requires assuming the synced block height is divisible + // by [params.BloomBitsBlocks]. + parentHeight := block.NumberU64() - 1 + parentHash := block.ParentHash() + c.config.Chain.BloomIndexer().AddCheckpoint(parentHeight/params.BloomBitsBlocks, parentHash) + + if err := ctx.Err(); err != nil { + return fmt.Errorf("%w: %w", errCommitCancelled, err) + } + if err := c.config.Chain.BlockChain().ResetToStateSyncedBlock(block); err != nil { + return err + } + + if c.config.Extender != nil { + if err := c.config.Extender.OnFinishBeforeCommit(c.config.LastAcceptedHeight, summary); err != nil { + return err + } + } + + if err := c.commitMarkers(summary); err != nil { + return fmt.Errorf("%w: height=%d, hash=%s: %w", errCommitMarkers, block.NumberU64(), block.Hash(), err) + } + + if err := c.config.State.SetLastAcceptedBlock(wrappedBlock); err != nil { + return err + } + + if c.config.Extender != nil { + if err := c.config.Extender.OnFinishAfterCommit(block.NumberU64()); err != nil { + return err + } + } + + return nil +} + +// commitMarkers updates VM database markers atomically. +func (c *client) commitMarkers(summary message.Syncable) error { + id := ids.ID(summary.GetBlockHash()) + if err := c.config.Acceptor.PutLastAcceptedID(id); err != nil { + return err + } + if err := c.config.MetadataDB.Delete(stateSyncSummaryKey); err != nil { + return err + } + return c.config.VerDB.Commit() +} + // newSyncerRegistry creates a registry with all required syncers for the given summary. func (c *client) newSyncerRegistry(summary message.Syncable) (*SyncerRegistry, error) { registry := NewSyncerRegistry() diff --git a/graft/coreth/plugin/evm/vmsync/finalizer.go b/graft/coreth/plugin/evm/vmsync/finalizer.go deleted file mode 100644 index b803d4e6fb80..000000000000 --- a/graft/coreth/plugin/evm/vmsync/finalizer.go +++ /dev/null @@ -1,152 +0,0 @@ -// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. -// See the file LICENSE for licensing terms. - -package vmsync - -import ( - "context" - "errors" - "fmt" - - "github.com/ava-labs/libevm/core/types" - - "github.com/ava-labs/avalanchego/database" - "github.com/ava-labs/avalanchego/database/versiondb" - "github.com/ava-labs/avalanchego/graft/coreth/eth" - "github.com/ava-labs/avalanchego/graft/coreth/params" - "github.com/ava-labs/avalanchego/graft/coreth/plugin/evm/message" - "github.com/ava-labs/avalanchego/ids" - "github.com/ava-labs/avalanchego/vms/components/chain" - - syncpkg "github.com/ava-labs/avalanchego/graft/coreth/sync" -) - -// EthBlockWrapper can be implemented by a concrete block wrapper type to -// return *types.Block, which is needed to update chain pointers at the -// end of the sync operation. -type EthBlockWrapper interface { - GetEthBlock() *types.Block -} - -var ( - errBlockNotFound = errors.New("block not found in state") - errInvalidBlockType = errors.New("invalid block wrapper type") - errBlockHashMismatch = errors.New("block hash mismatch") - errBlockHeightMismatch = errors.New("block height mismatch") - errFinalizeCancelled = errors.New("finalize cancelled") - errCommitMarkers = errors.New("failed to commit VM markers") -) - -// finalizer handles VM state finalization after sync completes. -type finalizer struct { - chain *eth.Ethereum - state *chain.State - acceptor BlockAcceptor - verDB *versiondb.Database - metadataDB database.Database - extender syncpkg.Extender - lastAcceptedHeight uint64 -} - -// newFinalizer creates a new finalizer with the given dependencies. -func newFinalizer( - chain *eth.Ethereum, - state *chain.State, - acceptor BlockAcceptor, - verDB *versiondb.Database, - metadataDB database.Database, - extender syncpkg.Extender, - lastAcceptedHeight uint64, -) *finalizer { - return &finalizer{ - chain: chain, - state: state, - acceptor: acceptor, - verDB: verDB, - metadataDB: metadataDB, - extender: extender, - lastAcceptedHeight: lastAcceptedHeight, - } -} - -// finalize updates disk and memory pointers so the VM is prepared for bootstrapping. -// Executes any shared memory operations from the atomic trie to shared memory. -func (f *finalizer) finalize(ctx context.Context, summary message.Syncable) error { - stateBlock, err := f.state.GetBlock(ctx, ids.ID(summary.GetBlockHash())) - if err != nil { - return fmt.Errorf("%w: hash=%s", errBlockNotFound, summary.GetBlockHash()) - } - - wrapper, ok := stateBlock.(*chain.BlockWrapper) - if !ok { - return fmt.Errorf("%w: got %T, want *chain.BlockWrapper", errInvalidBlockType, stateBlock) - } - wrappedBlock := wrapper.Block - - evmBlockGetter, ok := wrappedBlock.(EthBlockWrapper) - if !ok { - return fmt.Errorf("%w: got %T, want EthBlockWrapper", errInvalidBlockType, wrappedBlock) - } - - block := evmBlockGetter.GetEthBlock() - - if block.Hash() != summary.GetBlockHash() { - return fmt.Errorf("%w: got %s, want %s", errBlockHashMismatch, block.Hash(), summary.GetBlockHash()) - } - if block.NumberU64() != summary.Height() { - return fmt.Errorf("%w: got %d, want %d", errBlockHeightMismatch, block.NumberU64(), summary.Height()) - } - - // BloomIndexer needs to know that some parts of the chain are not available - // and cannot be indexed. This is done by calling [AddCheckpoint] here. - // Since the indexer uses sections of size [params.BloomBitsBlocks] (= 4096), - // each block is indexed in section number [blockNumber/params.BloomBitsBlocks]. - // To allow the indexer to start with the block we just synced to, - // we create a checkpoint for its parent. - // Note: This requires assuming the synced block height is divisible - // by [params.BloomBitsBlocks]. - parentHeight := block.NumberU64() - 1 - parentHash := block.ParentHash() - f.chain.BloomIndexer().AddCheckpoint(parentHeight/params.BloomBitsBlocks, parentHash) - - if err := ctx.Err(); err != nil { - return fmt.Errorf("%w: %w", errFinalizeCancelled, err) - } - if err := f.chain.BlockChain().ResetToStateSyncedBlock(block); err != nil { - return err - } - - if f.extender != nil { - if err := f.extender.OnFinishBeforeCommit(f.lastAcceptedHeight, summary); err != nil { - return err - } - } - - if err := f.commitMarkers(summary); err != nil { - return fmt.Errorf("%w: height=%d, hash=%s: %w", errCommitMarkers, block.NumberU64(), block.Hash(), err) - } - - if err := f.state.SetLastAcceptedBlock(wrappedBlock); err != nil { - return err - } - - if f.extender != nil { - if err := f.extender.OnFinishAfterCommit(block.NumberU64()); err != nil { - return err - } - } - - return nil -} - -// commitMarkers updates VM database markers atomically. -func (f *finalizer) commitMarkers(summary message.Syncable) error { - id := ids.ID(summary.GetBlockHash()) - if err := f.acceptor.PutLastAcceptedID(id); err != nil { - return err - } - if err := f.metadataDB.Delete(stateSyncSummaryKey); err != nil { - return err - } - return f.verDB.Commit() -} diff --git a/graft/coreth/plugin/evm/vmsync/strategy_static.go b/graft/coreth/plugin/evm/vmsync/strategy_static.go index 88afed6309ad..8b3090ba556b 100644 --- a/graft/coreth/plugin/evm/vmsync/strategy_static.go +++ b/graft/coreth/plugin/evm/vmsync/strategy_static.go @@ -13,24 +13,24 @@ var _ SyncStrategy = (*staticStrategy)(nil) // staticStrategy runs syncers sequentially without block queueing. // This is the default sync mode where all syncers complete before -// finalization, with no concurrent block processing. +// committing results, with no concurrent block processing. type staticStrategy struct { registry *SyncerRegistry - finalizer *finalizer + committer Committer } -func newStaticStrategy(registry *SyncerRegistry, finalizer *finalizer) *staticStrategy { +func newStaticStrategy(registry *SyncerRegistry, committer Committer) *staticStrategy { return &staticStrategy{ registry: registry, - finalizer: finalizer, + committer: committer, } } // Start begins the sync process and blocks until completion or error. -// For static sync, this runs all syncers and then finalizes the VM state. +// For static sync, this runs all syncers and then commits the results to the VM. func (s *staticStrategy) Start(ctx context.Context, summary message.Syncable) error { if err := s.registry.RunSyncerTasks(ctx, summary); err != nil { return err } - return s.finalizer.finalize(ctx, summary) + return s.committer.Commit(ctx, summary) } From e755cb300a67354e53236a40e55ed9e4d7f0027f Mon Sep 17 00:00:00 2001 From: Tsvetan Dimitrov Date: Wed, 17 Dec 2025 17:57:25 +0200 Subject: [PATCH 18/20] refactor(vmsync): rename SyncStrategy to Executor Rename the sync execution interface and implementation for clarity: - Synctrategy -> Executor - Start() -> Execute() - staticStrategy -> staticExecutor - strategy_static.go -> executor_static.go `Executor` better describes the role of running the sync process to completion. --- graft/coreth/plugin/evm/vmsync/client.go | 23 ++++++++++--------- ...{strategy_static.go => executor_static.go} | 18 +++++++-------- 2 files changed, 21 insertions(+), 20 deletions(-) rename graft/coreth/plugin/evm/vmsync/{strategy_static.go => executor_static.go} (52%) diff --git a/graft/coreth/plugin/evm/vmsync/client.go b/graft/coreth/plugin/evm/vmsync/client.go index 6b99191fd5f8..83a36c8a8623 100644 --- a/graft/coreth/plugin/evm/vmsync/client.go +++ b/graft/coreth/plugin/evm/vmsync/client.go @@ -63,11 +63,11 @@ type Committer interface { Commit(ctx context.Context, summary message.Syncable) error } -// SyncStrategy defines how state sync is executed. +// Executor defines how state sync is executed. // Implementations handle the sync lifecycle differently based on sync mode. -type SyncStrategy interface { - // Start begins the sync process and blocks until completion or error. - Start(ctx context.Context, summary message.Syncable) error +type Executor interface { + // Execute runs the sync process and blocks until completion or error. + Execute(ctx context.Context, summary message.Syncable) error } var _ Committer = (*client)(nil) @@ -182,9 +182,9 @@ func (c *client) acceptSyncSummary(summary message.Syncable) (block.StateSyncMod return block.StateSyncSkipped, fmt.Errorf("failed to create syncer registry: %w", err) } - strategy := newStaticStrategy(registry, c) + executor := newStaticExecutor(registry, c) - return c.startAsync(strategy, summary), nil + return c.startAsync(executor, summary), nil } // prepareForSync handles resume check and snapshot wipe before sync starts. @@ -230,8 +230,8 @@ func (c *client) prepareForSync(summary message.Syncable) error { return nil } -// startAsync launches the sync strategy in a background goroutine. -func (c *client) startAsync(strategy SyncStrategy, summary message.Syncable) block.StateSyncMode { +// startAsync launches the sync executor in a background goroutine. +func (c *client) startAsync(executor Executor, summary message.Syncable) block.StateSyncMode { ctx, cancel := context.WithCancel(context.Background()) c.cancel = cancel @@ -240,7 +240,7 @@ func (c *client) startAsync(strategy SyncStrategy, summary message.Syncable) blo defer c.wg.Done() defer cancel() - if err := strategy.Start(ctx, summary); err != nil { + if err := executor.Execute(ctx, summary); err != nil { c.err = err } // notify engine regardless of whether err == nil, @@ -265,8 +265,9 @@ func (c *client) Shutdown() error { // Error returns a non-nil error if one occurred during the sync. func (c *client) Error() error { return c.err } -// Commit implements Committer. It updates disk and memory pointers so the VM -// is prepared for bootstrapping. Executes any shared memory operations from +// Commit implements Committer. It resets the blockchain to the synced block, +// preparing it for execution, and updates disk and memory pointers so the VM +// is ready for bootstrapping. Also executes any shared memory operations from // the atomic trie to shared memory. func (c *client) Commit(ctx context.Context, summary message.Syncable) error { stateBlock, err := c.config.State.GetBlock(ctx, ids.ID(summary.GetBlockHash())) diff --git a/graft/coreth/plugin/evm/vmsync/strategy_static.go b/graft/coreth/plugin/evm/vmsync/executor_static.go similarity index 52% rename from graft/coreth/plugin/evm/vmsync/strategy_static.go rename to graft/coreth/plugin/evm/vmsync/executor_static.go index 8b3090ba556b..ae0a98cdff9b 100644 --- a/graft/coreth/plugin/evm/vmsync/strategy_static.go +++ b/graft/coreth/plugin/evm/vmsync/executor_static.go @@ -9,28 +9,28 @@ import ( "github.com/ava-labs/avalanchego/graft/coreth/plugin/evm/message" ) -var _ SyncStrategy = (*staticStrategy)(nil) +var _ Executor = (*staticExecutor)(nil) -// staticStrategy runs syncers sequentially without block queueing. +// staticExecutor runs syncers sequentially without block queueing. // This is the default sync mode where all syncers complete before // committing results, with no concurrent block processing. -type staticStrategy struct { +type staticExecutor struct { registry *SyncerRegistry committer Committer } -func newStaticStrategy(registry *SyncerRegistry, committer Committer) *staticStrategy { - return &staticStrategy{ +func newStaticExecutor(registry *SyncerRegistry, committer Committer) *staticExecutor { + return &staticExecutor{ registry: registry, committer: committer, } } -// Start begins the sync process and blocks until completion or error. +// Execute runs the sync process and blocks until completion or error. // For static sync, this runs all syncers and then commits the results to the VM. -func (s *staticStrategy) Start(ctx context.Context, summary message.Syncable) error { - if err := s.registry.RunSyncerTasks(ctx, summary); err != nil { +func (e *staticExecutor) Execute(ctx context.Context, summary message.Syncable) error { + if err := e.registry.RunSyncerTasks(ctx, summary); err != nil { return err } - return s.committer.Commit(ctx, summary) + return e.committer.Commit(ctx, summary) } From 745b63a826bd5305a265a9ed777d6862a56d599b Mon Sep 17 00:00:00 2001 From: Tsvetan Dimitrov Date: Fri, 19 Dec 2025 16:18:34 +0200 Subject: [PATCH 19/20] fix(vmsync): add atomic context to extender syncer errors Wrap errors in atomic/sync.Extender.CreateSyncer with atomic-prefixed context (including NewSyncer failures) and keep vmsync caller error message generic to avoid double "atomic" wording. --- graft/coreth/plugin/evm/atomic/sync/extender.go | 8 ++++++-- graft/coreth/plugin/evm/vmsync/client.go | 6 +++--- 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/graft/coreth/plugin/evm/atomic/sync/extender.go b/graft/coreth/plugin/evm/atomic/sync/extender.go index 37b4a35b09f1..537cd2077842 100644 --- a/graft/coreth/plugin/evm/atomic/sync/extender.go +++ b/graft/coreth/plugin/evm/atomic/sync/extender.go @@ -32,10 +32,10 @@ func (a *Extender) Initialize(backend *state.AtomicBackend, trie *state.AtomicTr func (a *Extender) CreateSyncer(client syncclient.LeafClient, verDB *versiondb.Database, summary message.Syncable) (sync.Syncer, error) { atomicSummary, ok := summary.(*Summary) if !ok { - return nil, fmt.Errorf("expected *Summary, got %T", summary) + return nil, fmt.Errorf("atomic sync extender: expected *Summary, got %T", summary) } - return NewSyncer( + syncer, err := NewSyncer( client, verDB, a.trie, @@ -43,6 +43,10 @@ func (a *Extender) CreateSyncer(client syncclient.LeafClient, verDB *versiondb.D atomicSummary.BlockNumber, WithRequestSize(a.requestSize), ) + if err != nil { + return nil, fmt.Errorf("atomic.NewSyncer failed: %w", err) + } + return syncer, nil } // OnFinishBeforeCommit implements the sync.Extender interface by marking the previously last accepted block for the shared memory cursor. diff --git a/graft/coreth/plugin/evm/vmsync/client.go b/graft/coreth/plugin/evm/vmsync/client.go index 83a36c8a8623..0c112d79de09 100644 --- a/graft/coreth/plugin/evm/vmsync/client.go +++ b/graft/coreth/plugin/evm/vmsync/client.go @@ -384,11 +384,11 @@ func (c *client) newSyncerRegistry(summary message.Syncable) (*SyncerRegistry, e syncers := []syncpkg.Syncer{blockSyncer, codeSyncer, stateSyncer} if c.config.Extender != nil { - atomicSyncer, err := c.config.Extender.CreateSyncer(c.config.Client, c.config.VerDB, summary) + extenderSyncer, err := c.config.Extender.CreateSyncer(c.config.Client, c.config.VerDB, summary) if err != nil { - return nil, fmt.Errorf("failed to create atomic syncer: %w", err) + return nil, fmt.Errorf("failed to create extender syncer: %w", err) } - syncers = append(syncers, atomicSyncer) + syncers = append(syncers, extenderSyncer) } for _, s := range syncers { From c038160eb2d4dfbef2142d6eea99b4edf88930dc Mon Sep 17 00:00:00 2001 From: Tsvetan Dimitrov Date: Fri, 19 Dec 2025 16:49:07 +0200 Subject: [PATCH 20/20] fix(vmsync): rename Commiter interface to Acceptor and Commit to AcceptSync --- graft/coreth/plugin/evm/vmsync/client.go | 12 ++++++------ graft/coreth/plugin/evm/vmsync/executor_static.go | 14 +++++++------- 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/graft/coreth/plugin/evm/vmsync/client.go b/graft/coreth/plugin/evm/vmsync/client.go index 0c112d79de09..f25a8c39a17c 100644 --- a/graft/coreth/plugin/evm/vmsync/client.go +++ b/graft/coreth/plugin/evm/vmsync/client.go @@ -58,9 +58,9 @@ type BlockAcceptor interface { PutLastAcceptedID(ids.ID) error } -// Committer commits sync results to the VM, preparing it for bootstrapping. -type Committer interface { - Commit(ctx context.Context, summary message.Syncable) error +// Acceptor applies the results of state sync to the VM, preparing it for bootstrapping. +type Acceptor interface { + AcceptSync(ctx context.Context, summary message.Syncable) error } // Executor defines how state sync is executed. @@ -70,7 +70,7 @@ type Executor interface { Execute(ctx context.Context, summary message.Syncable) error } -var _ Committer = (*client)(nil) +var _ Acceptor = (*client)(nil) type ClientConfig struct { Chain *eth.Ethereum @@ -265,11 +265,11 @@ func (c *client) Shutdown() error { // Error returns a non-nil error if one occurred during the sync. func (c *client) Error() error { return c.err } -// Commit implements Committer. It resets the blockchain to the synced block, +// AcceptSync implements Acceptor. It resets the blockchain to the synced block, // preparing it for execution, and updates disk and memory pointers so the VM // is ready for bootstrapping. Also executes any shared memory operations from // the atomic trie to shared memory. -func (c *client) Commit(ctx context.Context, summary message.Syncable) error { +func (c *client) AcceptSync(ctx context.Context, summary message.Syncable) error { stateBlock, err := c.config.State.GetBlock(ctx, ids.ID(summary.GetBlockHash())) if err != nil { return fmt.Errorf("%w: hash=%s", errBlockNotFound, summary.GetBlockHash()) diff --git a/graft/coreth/plugin/evm/vmsync/executor_static.go b/graft/coreth/plugin/evm/vmsync/executor_static.go index ae0a98cdff9b..cb96517ec976 100644 --- a/graft/coreth/plugin/evm/vmsync/executor_static.go +++ b/graft/coreth/plugin/evm/vmsync/executor_static.go @@ -15,22 +15,22 @@ var _ Executor = (*staticExecutor)(nil) // This is the default sync mode where all syncers complete before // committing results, with no concurrent block processing. type staticExecutor struct { - registry *SyncerRegistry - committer Committer + registry *SyncerRegistry + acceptor Acceptor } -func newStaticExecutor(registry *SyncerRegistry, committer Committer) *staticExecutor { +func newStaticExecutor(registry *SyncerRegistry, acceptor Acceptor) *staticExecutor { return &staticExecutor{ - registry: registry, - committer: committer, + registry: registry, + acceptor: acceptor, } } // Execute runs the sync process and blocks until completion or error. -// For static sync, this runs all syncers and then commits the results to the VM. +// For static sync, this runs all syncers and then accepts the synced state into the VM. func (e *staticExecutor) Execute(ctx context.Context, summary message.Syncable) error { if err := e.registry.RunSyncerTasks(ctx, summary); err != nil { return err } - return e.committer.Commit(ctx, summary) + return e.acceptor.AcceptSync(ctx, summary) }