From d47c875e006b001840969887f181f7ca03360abf Mon Sep 17 00:00:00 2001 From: Austin Larson Date: Thu, 20 Feb 2025 11:40:49 -0500 Subject: [PATCH 01/21] Prototyping block queueing --- core/blockchain.go | 7 + plugin/evm/block.go | 28 +- plugin/evm/statesync/queue.go | 92 +++++++ plugin/evm/syncervm_client.go | 464 +++++++++++++++++++++++++++++++--- 4 files changed, 552 insertions(+), 39 deletions(-) create mode 100644 plugin/evm/statesync/queue.go diff --git a/core/blockchain.go b/core/blockchain.go index 6998fbb57a..844df1e188 100644 --- a/core/blockchain.go +++ b/core/blockchain.go @@ -1067,6 +1067,8 @@ func (bc *BlockChain) Reject(block *types.Block) error { bc.chainmu.Lock() defer bc.chainmu.Unlock() + log.Error("Rejecting block", "number", block.Number(), "hash", block.Hash()) + // Reject Trie if err := bc.stateManager.RejectTrie(block); err != nil { return fmt.Errorf("unable to reject trie: %w", err) @@ -1251,6 +1253,11 @@ func (bc *BlockChain) insertBlock(block *types.Block, writes bool) error { err := bc.engine.VerifyHeader(bc, block.Header()) if err == nil { err = bc.validator.ValidateBody(block) + if err != nil { + log.Error("Body validation failed", "err", err, "number", block.Number(), "hash", block.Hash()) + } + } else { + log.Error("Header verification failed", "err", err, "number", block.Number(), "hash", block.Hash()) } switch { diff --git a/plugin/evm/block.go b/plugin/evm/block.go index bcc7e7e995..52df5dd318 100644 --- a/plugin/evm/block.go +++ b/plugin/evm/block.go @@ -139,7 +139,19 @@ func (b *Block) ID() ids.ID { return b.id } func (b *Block) AtomicTxs() []*atomic.Tx { return b.atomicTxs } // Accept implements the snowman.Block interface -func (b *Block) Accept(context.Context) error { +func (b *Block) Accept(ctx context.Context) error { + if b.vm.StateSyncClient.AsyncReceive() { + log.Debug("Queueing block for later processing", "block", b.ID(), "height", b.Height()) + return b.vm.StateSyncClient.QueueAcceptBlock(b) + } + return b.accept(ctx) +} + +func (b *Block) AcceptDuringSync(ctx context.Context) error { + return b.accept(ctx) +} + +func (b *Block) accept(context.Context) error { vm := b.vm // Although returning an error from Accept is considered fatal, it is good @@ -271,6 +283,20 @@ func (b *Block) syntacticVerify() error { // Verify implements the snowman.Block interface func (b *Block) Verify(context.Context) error { + + // If currently dynamically syncing, we should simply queue the block for later processing + if b.vm.StateSyncClient.AsyncReceive() { + log.Debug("Queueing block for later processing", "block", b.ID(), "height", b.Height()) + return b.vm.StateSyncClient.QueueVerifyBlock(b) + } + + return b.verify(&precompileconfig.PredicateContext{ + SnowCtx: b.vm.ctx, + ProposerVMBlockCtx: nil, + }, true) +} + +func (b *Block) VerifyDuringSync(context.Context) error { return b.verify(&precompileconfig.PredicateContext{ SnowCtx: b.vm.ctx, ProposerVMBlockCtx: nil, diff --git a/plugin/evm/statesync/queue.go b/plugin/evm/statesync/queue.go new file mode 100644 index 0000000000..5e3b1bf453 --- /dev/null +++ b/plugin/evm/statesync/queue.go @@ -0,0 +1,92 @@ +// (c) 2019-2020, Ava Labs, Inc. +// +// This file is although named queue.go, is not the same as the eth/downloader/queue.go file. +// Instead, this is simply the protected array for downloader (yeah not even a queue). +// It might be helpful to keep this in case we move computation here +// +// It is distributed under a license compatible with the licensing terms of the +// original code from which it is derived. + +package statesync + +import ( + "fmt" + "sync" +) + +type Executable interface { + ExitQueue() error +} + +type Queue[K Executable] struct { + buffer []*K + l sync.RWMutex + nextPos int + compare func(*K, *K) int + closed bool +} + +func NewQueue[K Executable](size int, compare func(*K, *K) int) *Queue[K] { + return &Queue[K]{ + buffer: make([]*K, size), + compare: compare, + } +} + +func (q *Queue[K]) Insert(h *K) error { + q.l.Lock() + defer q.l.Unlock() + + if q.nextPos >= len(q.buffer) { + return fmt.Errorf("queue is full, cannot insert") + } + + q.buffer[q.nextPos] = h + q.nextPos++ + + return nil +} + +func (q *Queue[K]) Flush(max *K, close bool) error { + q.l.Lock() + defer q.l.Unlock() + + newBuffer := make([]*K, len(q.buffer)) + newPos := 0 + + for i := 0; i < q.nextPos; i++ { + // If the item is greater than max, postpone + elem := q.buffer[i] + if max != nil && q.compare(elem, max) < 0 { + newBuffer[newPos] = q.buffer[i] + newPos++ + } else { + if err := (*elem).ExitQueue(); err != nil { + return fmt.Errorf("error executing item: %w", err) + } + } + } + + q.buffer = newBuffer + q.nextPos = newPos + + if close { + q.closed = true + } + + return nil +} + +func (q *Queue[K]) Len() int { + q.l.RLock() + defer q.l.RUnlock() + + return q.nextPos +} + +func (q *Queue[K]) Closed() bool { + q.l.RLock() + defer q.l.RUnlock() + + return q.closed +} diff --git a/plugin/evm/syncervm_client.go b/plugin/evm/syncervm_client.go index 8a3438108e..8bdbd73f5c 100644 --- a/plugin/evm/syncervm_client.go +++ b/plugin/evm/syncervm_client.go @@ -8,11 +8,14 @@ import ( "fmt" "sync" + // "time" + "github.com/ava-labs/avalanchego/database" "github.com/ava-labs/avalanchego/database/versiondb" "github.com/ava-labs/avalanchego/ids" commonEng "github.com/ava-labs/avalanchego/snow/engine/common" "github.com/ava-labs/avalanchego/snow/engine/snowman/block" + "github.com/ava-labs/avalanchego/utils" "github.com/ava-labs/avalanchego/vms/components/chain" "github.com/ava-labs/coreth/core/rawdb" "github.com/ava-labs/coreth/core/state/snapshot" @@ -32,7 +35,9 @@ import ( const ( // State sync fetches [parentsToGet] parents of the block it syncs to. // The last 256 block hashes are necessary to support the BLOCKHASH opcode. - parentsToGet = 256 + parentsToGet = 256 + pivotInterval = 128 + bufferSize = 6 * 60 * 30 // 2 * pivotInterval // extra space to be careful ) var stateSyncSummaryKey = []byte("stateSyncSummary") @@ -79,6 +84,10 @@ type stateSyncerClient struct { // State Sync results syncSummary message.SyncSummary stateSyncErr error + + // Testing dynamic sync + syncing utils.Atomic[bool] + dl *downloader } func NewStateSyncClient(config *stateSyncClientConfig) StateSyncClient { @@ -97,6 +106,11 @@ type StateSyncClient interface { ClearOngoingSummary() error Shutdown() error Error() error + + // Methods to try to enable dynamic state sync + AsyncReceive() bool + QueueVerifyBlock(*Block) error + QueueAcceptBlock(*Block) error } // Syncer represents a step in state sync, @@ -108,6 +122,43 @@ type Syncer interface { Done() <-chan error } +// AsyncReceive returns true if the client is ready to receive a message from the engine +// Should return true if syncing and useUpstream is true, i.e. currently dynamicaling syncing +func (client *stateSyncerClient) AsyncReceive() bool { + return client.useUpstream && client.syncing.Get() && client.dl != nil && !client.dl.execQueue.Closed() +} + +func (client *stateSyncerClient) QueueVerifyBlock(b *Block) error { + if !client.AsyncReceive() { + return fmt.Errorf("cannot queue block when not using upstream syncing") + } + + verify := func(a *Block) error { + return a.VerifyDuringSync(context.TODO()) // is this right? + } + return client.dl.execQueue.Insert(&queueElement{b, verify}) +} + +func (client *stateSyncerClient) QueueAcceptBlock(b *Block) error { + if !client.AsyncReceive() { + return fmt.Errorf("cannot queue block when not using upstream syncing") + } + + accept := func(a *Block) error { + return a.AcceptDuringSync(context.TODO()) + } + + if err := client.dl.execQueue.Insert(&queueElement{b, accept}); err != nil { + return err + } + + // If the block is the pivot, signal the state syncer to start + if b.Height() >= client.dl.pivotBlock.Height() { + client.dl.newPivot <- b + } + return nil +} + // StateSyncEnabled returns [client.enabled], which is set in the chain's config file. func (client *stateSyncerClient) StateSyncEnabled(context.Context) (bool, error) { return client.enabled, nil @@ -159,29 +210,14 @@ func (client *stateSyncerClient) stateSync(ctx context.Context) error { } // Sync the EVM trie and then the atomic trie. These steps could be done - // in parallel or in the opposite order. Keeping them serial for simplicity for now. - if client.useUpstream { - log.Warn("Using upstream state syncer (untested)") - syncer := snap.NewSyncer(client.chaindb, rawdb.HashScheme) - p2pClient := client.network.NewClient(ethstatesync.ProtocolID) - if len(client.stateSyncNodes) > 0 { - for _, nodeID := range client.stateSyncNodes { - syncer.Register(ethstatesync.NewOutboundPeer(nodeID, syncer, p2pClient)) - } - } else { - client.network.AddConnector(ethstatesync.NewConnector(syncer, p2pClient)) - } - if err := syncer.Sync(client.syncSummary.BlockRoot, convertReadOnlyToBidirectional(ctx.Done())); err != nil { - return err - } - log.Info("Upstream state syncer completed") - } else { - if err := client.syncStateTrie(ctx); err != nil { - return err - } + // in parallel or in the opposite order in the static case + // For dynamic, much simpler to do atomic trie first + + if err := client.syncAtomicTrie(ctx); err != nil { + return err } - return client.syncAtomicTrie(ctx) + return client.syncStateTrie(ctx) } func convertReadOnlyToBidirectional[T any](readOnly <-chan T) chan T { @@ -244,10 +280,35 @@ func (client *stateSyncerClient) acceptSyncSummary(proposedSummary message.SyncS 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 + if client.useUpstream { + // Must first find first pivot block to signal bootstrapper + stateBlock, err := client.state.GetBlock(context.TODO(), ids.ID(client.syncSummary.BlockHash)) + if err != nil { + return block.StateSyncDynamic, fmt.Errorf("could not get block by hash from client state: %s", client.syncSummary.BlockHash) + } + wrapper, ok := stateBlock.(*chain.BlockWrapper) + if !ok { + return block.StateSyncDynamic, fmt.Errorf("could not convert block(%T) to *chain.BlockWrapper", wrapper) + } + evmBlock, ok := wrapper.Block.(*Block) + if !ok { + return block.StateSyncDynamic, fmt.Errorf("could not convert block(%T) to evm.Block", stateBlock) + } + + if err := client.state.SetLastAcceptedBlock(evmBlock); err != nil { + return block.StateSyncDynamic, err + } + + // Set downloader using pivot + client.dl = newDownloader(client.chaindb, evmBlock) + + log.Info("Set LastAcceptedBlock to first pivot with height", evmBlock.Height(), "timestamp", evmBlock.Timestamp()) + } go func() { defer client.wg.Done() defer cancel() + client.syncing.Set(true) if err := client.stateSync(ctx); err != nil { client.stateSyncErr = err } else { @@ -258,7 +319,12 @@ func (client *stateSyncerClient) acceptSyncSummary(proposedSummary message.SyncS // vm.SetState(snow.Bootstrapping) log.Info("stateSync completed, notifying engine", "err", client.stateSyncErr) client.toEngine <- commonEng.StateSyncDone + client.syncing.Set(false) }() + + if client.useUpstream { + return block.StateSyncDynamic, nil + } return block.StateSyncStatic, nil } @@ -329,24 +395,30 @@ func (client *stateSyncerClient) syncAtomicTrie(ctx context.Context) error { func (client *stateSyncerClient) syncStateTrie(ctx context.Context) error { log.Info("state sync: sync starting", "root", client.syncSummary.BlockRoot) - evmSyncer, err := statesync.NewStateSyncer(&statesync.StateSyncerConfig{ - Client: client.client, - Root: client.syncSummary.BlockRoot, - BatchSize: ethdb.IdealBatchSize, - DB: client.chaindb, - MaxOutstandingCodeHashes: statesync.DefaultMaxOutstandingCodeHashes, - NumCodeFetchingWorkers: statesync.DefaultNumCodeFetchingWorkers, - RequestSize: client.stateSyncRequestSize, - }) - if err != nil { - return err - } - if err := evmSyncer.Start(ctx); err != nil { + + if client.useUpstream { + log.Warn("Using upstream state syncer (untested)") + return client.upstreamSyncStateTrie(ctx) + } else { + evmSyncer, err := statesync.NewStateSyncer(&statesync.StateSyncerConfig{ + Client: client.client, + Root: client.syncSummary.BlockRoot, + BatchSize: ethdb.IdealBatchSize, + DB: client.chaindb, + MaxOutstandingCodeHashes: statesync.DefaultMaxOutstandingCodeHashes, + NumCodeFetchingWorkers: statesync.DefaultNumCodeFetchingWorkers, + RequestSize: client.stateSyncRequestSize, + }) + if err != nil { + return err + } + if err := evmSyncer.Start(ctx); err != nil { + return err + } + err = <-evmSyncer.Done() + log.Info("state sync: sync finished", "root", client.syncSummary.BlockRoot, "err", err) return err } - err = <-evmSyncer.Done() - log.Info("state sync: sync finished", "root", client.syncSummary.BlockRoot, "err", err) - return err } func (client *stateSyncerClient) Shutdown() error { @@ -441,3 +513,319 @@ func (client *stateSyncerClient) updateVMMarkers() error { // Error returns a non-nil error if one occurred during the sync. func (client *stateSyncerClient) Error() error { return client.stateSyncErr } + +// upstreamSyncStateTrie syncs the state trie using the upstream state syncer +func (client *stateSyncerClient) upstreamSyncStateTrie(ctx context.Context) error { + p2pClient := client.network.NewClient(ethstatesync.ProtocolID) + if len(client.stateSyncNodes) > 0 { + for _, nodeID := range client.stateSyncNodes { + client.dl.SnapSyncer.Register(ethstatesync.NewOutboundPeer(nodeID, client.dl.SnapSyncer, p2pClient)) + } + } else { + client.network.AddConnector(ethstatesync.NewConnector(client.dl.SnapSyncer, p2pClient)) + } + if err := client.dl.SnapSyncer.Sync(client.syncSummary.BlockRoot, convertReadOnlyToBidirectional(ctx.Done())); err != nil { + return err + } + log.Info("Upstream state syncer completed, moving to catch-up") + + // Now that we have synced the state trie to static pivot, Verify and Accept all pending blocks + // Incoming blocks are still appended to buffer until emptied + err := client.dl.execQueue.Flush(nil, true) + return err +} + +func (e queueElement) ExitQueue() error { + return e.exec(e.block) +} + +type queueElement struct { + block *Block + exec func(*Block) error +} + +var _ ethstatesync.Executable = &queueElement{} + +type downloader struct { + pivotLock sync.RWMutex + pivotBlock *Block + execQueue *ethstatesync.Queue[queueElement] + SnapSyncer *snap.Syncer + + stateSyncStart chan *stateSync + newPivot chan *Block + quitCh chan struct{} // Quit channel to signal termination + quitLock sync.Mutex // Lock to prevent double closes +} + +func newDownloader(chaindb ethdb.Database, firstPivot *Block) *downloader { + compare := func(a, b *queueElement) int { + return int(a.block.Height()) - int(b.block.Height()) + } + + d := &downloader{ + pivotBlock: firstPivot, + execQueue: ethstatesync.NewQueue[queueElement](bufferSize, compare), + SnapSyncer: snap.NewSyncer(chaindb, rawdb.HashScheme), + stateSyncStart: make(chan *stateSync), + quitCh: make(chan struct{}), + newPivot: make(chan *Block), + } + + // go d.stateFetcher() + + return d +} + +/* +// stateFetcher manages the active state sync and accepts requests +// on its behalf. +func (d *downloader) stateFetcher() { + for { + select { + case s := <-d.stateSyncStart: + for next := s; next != nil; { + next = d.runStateSync(next) + } + case <-d.quitCh: + return + case p:= <-d.newPivot: + d.pivotLock.Lock() + d.pivotBlock = p + d.pivotLock.Unlock() + + } + } +} + + +// processSnapSyncContent takes fetch results from the queue and writes them to the +// database. It also controls the synchronisation of state nodes of the pivot block. +func (d *downloader) processSnapSyncContent() error { + // Start syncing state of the reported head block. This should get us most of + // the state of the pivot block. + d.pivotLock.RLock() + sync := d.syncState(d.pivotBlock.ethBlock.Hash()) + d.pivotLock.RUnlock() + + defer func() { + // The `sync` object is replaced every time the pivot moves. We need to + // defer close the very last active one, hence the lazy evaluation vs. + // calling defer sync.Cancel() !!! + sync.Cancel() + }() + + // closeOnErr := func(s *stateSync) { + // if err := s.Wait(); err != nil && err != errCancelStateFetch && err != errCanceled && err != snap.ErrCancelled { + // d.queue.Close() // wake up Results + // } + // } + // go closeOnErr(sync) + + // To cater for moving pivot points, track the pivot block and subsequently + // accumulated download results separately. + // + // These will be nil up to the point where we reach the pivot, and will only + // be set temporarily if the synced blocks are piling up, but the pivot is + // still busy downloading. In that case, we need to occasionally check for + // pivot moves, so need to unblock the loop. These fields will accumulate + // the results in the meantime. + // + // Note, there's no issue with memory piling up since after 64 blocks the + // pivot will forcefully move so these accumulators will be dropped. + var ( + oldPivot *fetchResult // Locked in pivot block, might change eventually + oldTail []*fetchResult // Downloaded content after the pivot + timer = time.NewTimer(time.Second) + ) + defer timer.Stop() + + for { + // Wait for the next batch of downloaded data to be available. If we have + // not yet reached the pivot point, wait blockingly as there's no need to + // spin-loop check for pivot moves. If we reached the pivot but have not + // yet processed it, check for results async, so we might notice pivot + // moves while state syncing. If the pivot was passed fully, block again + // as there's no more reason to check for pivot moves at all. + results := d.queue.Results(oldPivot == nil) + if len(results) == 0 { + // If pivot sync is done, stop + if d.committed.Load() { + d.reportSnapSyncProgress(true) + return sync.Cancel() + } + // If sync failed, stop + select { + case <-d.cancelCh: + sync.Cancel() + return errCanceled + default: + } + } + if d.chainInsertHook != nil { + d.chainInsertHook(results) + } + d.reportSnapSyncProgress(false) + + // If we haven't downloaded the pivot block yet, check pivot staleness + // notifications from the header downloader + d.pivotLock.RLock() + pivot := d.pivotHeader + d.pivotLock.RUnlock() + + if oldPivot == nil { // no results piling up, we can move the pivot + if !d.committed.Load() { // not yet passed the pivot, we can move the pivot + if pivot.Root != sync.root { // pivot position changed, we can move the pivot + sync.Cancel() + sync = d.syncState(pivot.Root) + + go closeOnErr(sync) + } + } + } else { // results already piled up, consume before handling pivot move + results = append(append([]*fetchResult{oldPivot}, oldTail...), results...) + } + // Split around the pivot block and process the two sides via snap/full sync + if !d.committed.Load() { + latest := results[len(results)-1].Header + // If the height is above the pivot block by 2 sets, it means the pivot + // become stale in the network, and it was garbage collected, move to a + // new pivot. + // + // Note, we have `reorgProtHeaderDelay` number of blocks withheld, Those + // need to be taken into account, otherwise we're detecting the pivot move + // late and will drop peers due to unavailable state!!! + if height := latest.Number.Uint64(); height >= pivot.Number.Uint64()+2*uint64(fsMinFullBlocks)-uint64(reorgProtHeaderDelay) { + log.Warn("Pivot became stale, moving", "old", pivot.Number.Uint64(), "new", height-uint64(fsMinFullBlocks)+uint64(reorgProtHeaderDelay)) + pivot = results[len(results)-1-fsMinFullBlocks+reorgProtHeaderDelay].Header // must exist as lower old pivot is uncommitted + + d.pivotLock.Lock() + d.pivotHeader = pivot + d.pivotLock.Unlock() + + // Write out the pivot into the database so a rollback beyond it will + // reenable snap sync + rawdb.WriteLastPivotNumber(d.stateDB, pivot.Number.Uint64()) + } + } + P, beforeP, afterP := splitAroundPivot(pivot.Number.Uint64(), results) + if err := d.commitSnapSyncData(beforeP, sync); err != nil { + return err + } + if P != nil { + // If new pivot block found, cancel old state retrieval and restart + if oldPivot != P { + sync.Cancel() + sync = d.syncState(P.Header.Root) + + go closeOnErr(sync) + oldPivot = P + } + // Wait for completion, occasionally checking for pivot staleness + timer.Reset(time.Second) + select { + case <-sync.done: + if sync.err != nil { + return sync.err + } + if err := d.commitPivotBlock(P); err != nil { + return err + } + oldPivot = nil + + case <-timer.C: + oldTail = afterP + continue + } + } + // Fast sync done, pivot commit done, full import + if err := d.importBlockResults(afterP); err != nil { + return err + } + } +} + +// syncState starts downloading state with the given root hash. +func (d *downloader) syncState(root common.Hash) *stateSync { + // Create the state sync + s := newStateSync(d, root) + select { + case d.stateSyncStart <- s: + // If we tell the statesync to restart with a new root, we also need + // to wait for it to actually also start -- when old requests have timed + // out or been delivered + <-s.started + case <-d.quitCh: + s.err = errCancelStateFetch + close(s.done) + } + return s +} +// runStateSync runs a state synchronisation until it completes or another root +// hash is requested to be switched over to. +func (d *downloader) runStateSync(s *stateSync) *stateSync { + log.Trace("State sync starting", "root", s.root) + + go s.run() + defer s.Cancel() + + for { + select { + case next := <-d.stateSyncStart: + return next + + case <-s.done: + return nil + } + } +} +*/ +// stateSync schedules requests for downloading a particular state trie defined +// by a given state root. +type stateSync struct { + d *downloader // Downloader instance to access and manage current peerset + root common.Hash // State root currently being synced + + started chan struct{} // Started is signalled once the sync loop starts + cancel chan struct{} // Channel to signal a termination request + cancelOnce sync.Once // Ensures cancel only ever gets called once + done chan struct{} // Channel to signal termination completion + err error // Any error hit during sync (set before completion) +} + +/* +// newStateSync creates a new state trie download scheduler. This method does not +// yet start the sync. The user needs to call run to initiate. +func newStateSync(d *downloader, root common.Hash) *stateSync { + return &stateSync{ + d: d, + root: root, + cancel: make(chan struct{}), + done: make(chan struct{}), + started: make(chan struct{}), + } +} + +// run starts the task assignment and response processing loop, blocking until +// it finishes, and finally notifying any goroutines waiting for the loop to +// finish. +func (s *stateSync) run() { + close(s.started) + s.err = s.d.SnapSyncer.Sync(s.root, s.cancel) + close(s.done) +} + +// Wait blocks until the sync is done or canceled. +func (s *stateSync) Wait() error { + <-s.done + return s.err +} + +// Cancel cancels the sync and waits until it has shut down. +func (s *stateSync) Cancel() error { + s.cancelOnce.Do(func() { + close(s.cancel) + }) + return s.Wait() +} +*/ From 4ec6e3bd2bd3ce2c4bb15b3cd80cf409989f7229 Mon Sep 17 00:00:00 2001 From: Austin Larson Date: Fri, 21 Feb 2025 11:58:26 -0500 Subject: [PATCH 02/21] Prototyped mechanism to switch pivot --- plugin/evm/block.go | 62 +++++++----- plugin/evm/syncervm_client.go | 181 ++++++++-------------------------- plugin/evm/vm.go | 11 ++- 3 files changed, 85 insertions(+), 169 deletions(-) diff --git a/plugin/evm/block.go b/plugin/evm/block.go index 52df5dd318..c153ac70d0 100644 --- a/plugin/evm/block.go +++ b/plugin/evm/block.go @@ -283,6 +283,24 @@ func (b *Block) syntacticVerify() error { // Verify implements the snowman.Block interface func (b *Block) Verify(context.Context) error { + log.Debug("Verifying block without context", "block", b.ID(), "height", b.Height()) + if err := b.syntacticVerify(); err != nil { + return fmt.Errorf("syntactic block verification failed: %w", err) + } + + predicateContext := &precompileconfig.PredicateContext{ + SnowCtx: b.vm.ctx, + ProposerVMBlockCtx: nil, + } + // Only enforce predicates if the chain has already bootstrapped. + // If the chain is still bootstrapping, we can assume that all blocks we are verifying have + // been accepted by the network (so the predicate was validated by the network when the + // block was originally verified). + if b.vm.bootstrapped.Get() { + if err := b.verifyPredicates(predicateContext); err != nil { + return fmt.Errorf("failed to verify predicates: %w", err) + } + } // If currently dynamically syncing, we should simply queue the block for later processing if b.vm.StateSyncClient.AsyncReceive() { @@ -290,17 +308,11 @@ func (b *Block) Verify(context.Context) error { return b.vm.StateSyncClient.QueueVerifyBlock(b) } - return b.verify(&precompileconfig.PredicateContext{ - SnowCtx: b.vm.ctx, - ProposerVMBlockCtx: nil, - }, true) + return b.verify(true) } func (b *Block) VerifyDuringSync(context.Context) error { - return b.verify(&precompileconfig.PredicateContext{ - SnowCtx: b.vm.ctx, - ProposerVMBlockCtx: nil, - }, true) + return b.verify(true) } // ShouldVerifyWithContext implements the block.WithVerifyContext interface @@ -329,30 +341,15 @@ func (b *Block) ShouldVerifyWithContext(context.Context) (bool, error) { // VerifyWithContext implements the block.WithVerifyContext interface func (b *Block) VerifyWithContext(ctx context.Context, proposerVMBlockCtx *block.Context) error { - return b.verify(&precompileconfig.PredicateContext{ - SnowCtx: b.vm.ctx, - ProposerVMBlockCtx: proposerVMBlockCtx, - }, true) -} - -// Verify the block is valid. -// Enforces that the predicates are valid within [predicateContext]. -// Writes the block details to disk and the state to the trie manager iff writes=true. -func (b *Block) verify(predicateContext *precompileconfig.PredicateContext, writes bool) error { - if predicateContext.ProposerVMBlockCtx != nil { - log.Debug("Verifying block with context", "block", b.ID(), "height", b.Height()) - } else { - log.Debug("Verifying block without context", "block", b.ID(), "height", b.Height()) - } + log.Debug("Verifying block with context", "block", b.ID(), "height", b.Height()) if err := b.syntacticVerify(); err != nil { return fmt.Errorf("syntactic block verification failed: %w", err) } - // verify UTXOs named in import txs are present in shared memory. - if err := b.verifyUTXOsPresent(); err != nil { - return err + predicateContext := &precompileconfig.PredicateContext{ + SnowCtx: b.vm.ctx, + ProposerVMBlockCtx: proposerVMBlockCtx, } - // Only enforce predicates if the chain has already bootstrapped. // If the chain is still bootstrapping, we can assume that all blocks we are verifying have // been accepted by the network (so the predicate was validated by the network when the @@ -362,6 +359,17 @@ func (b *Block) verify(predicateContext *precompileconfig.PredicateContext, writ return fmt.Errorf("failed to verify predicates: %w", err) } } + return b.verify(true) +} + +// Verify the block is valid. +// Enforces that the predicates are valid within [predicateContext]. +// Writes the block details to disk and the state to the trie manager iff writes=true. +func (b *Block) verify(writes bool) error { + // verify UTXOs named in import txs are present in shared memory. + if err := b.verifyUTXOsPresent(); err != nil { + return err + } // The engine may call VerifyWithContext multiple times on the same block with different contexts. // Since the engine will only call Accept/Reject once, we should only call InsertBlockManual once. diff --git a/plugin/evm/syncervm_client.go b/plugin/evm/syncervm_client.go index 8bdbd73f5c..9a475ad442 100644 --- a/plugin/evm/syncervm_client.go +++ b/plugin/evm/syncervm_client.go @@ -130,6 +130,7 @@ func (client *stateSyncerClient) AsyncReceive() bool { func (client *stateSyncerClient) QueueVerifyBlock(b *Block) error { if !client.AsyncReceive() { + log.Info("Queueing block for later verify", "block", b.ID(), "height", b.Height()) return fmt.Errorf("cannot queue block when not using upstream syncing") } @@ -141,11 +142,12 @@ func (client *stateSyncerClient) QueueVerifyBlock(b *Block) error { func (client *stateSyncerClient) QueueAcceptBlock(b *Block) error { if !client.AsyncReceive() { + log.Info("Queueing block for later accept", "block", b.ID(), "height", b.Height()) return fmt.Errorf("cannot queue block when not using upstream syncing") } accept := func(a *Block) error { - return a.AcceptDuringSync(context.TODO()) + return a.AcceptDuringSync(context.TODO()) // yeah gotta fix this too } if err := client.dl.execQueue.Insert(&queueElement{b, accept}); err != nil { @@ -153,9 +155,13 @@ func (client *stateSyncerClient) QueueAcceptBlock(b *Block) error { } // If the block is the pivot, signal the state syncer to start - if b.Height() >= client.dl.pivotBlock.Height() { + client.dl.pivotLock.Lock() + if b.Height() >= client.dl.pivotBlock.Height()+pivotInterval { + client.dl.pivotBlock = b client.dl.newPivot <- b } + client.dl.pivotLock.Unlock() + return nil } @@ -295,10 +301,23 @@ func (client *stateSyncerClient) acceptSyncSummary(proposedSummary message.SyncS return block.StateSyncDynamic, fmt.Errorf("could not convert block(%T) to evm.Block", stateBlock) } + b := evmBlock.ethBlock + parentHeight := b.NumberU64() - 1 + parentHash := b.ParentHash() + client.chain.BloomIndexer().AddCheckpoint(parentHeight/params.BloomBitsBlocks, parentHash) + + if err := client.updateVMMarkers(); err != nil { + return block.StateSyncDynamic, fmt.Errorf("error updating vm markers, height=%d, hash=%s, err=%w", b.NumberU64(), b.Hash(), err) + } + if err := client.state.SetLastAcceptedBlock(evmBlock); err != nil { return block.StateSyncDynamic, err } + if err := client.atomicBackend.ApplyToSharedMemory(b.NumberU64()); err != nil { + return block.StateSyncDynamic, err + } + // Set downloader using pivot client.dl = newDownloader(client.chaindb, evmBlock) @@ -524,7 +543,7 @@ func (client *stateSyncerClient) upstreamSyncStateTrie(ctx context.Context) erro } else { client.network.AddConnector(ethstatesync.NewConnector(client.dl.SnapSyncer, p2pClient)) } - if err := client.dl.SnapSyncer.Sync(client.syncSummary.BlockRoot, convertReadOnlyToBidirectional(ctx.Done())); err != nil { + if err := client.dl.SnapSync(); err != nil { return err } log.Info("Upstream state syncer completed, moving to catch-up") @@ -555,7 +574,7 @@ type downloader struct { stateSyncStart chan *stateSync newPivot chan *Block quitCh chan struct{} // Quit channel to signal termination - quitLock sync.Mutex // Lock to prevent double closes + // quitLock sync.Mutex // Lock to prevent double closes } func newDownloader(chaindb ethdb.Database, firstPivot *Block) *downloader { @@ -572,12 +591,11 @@ func newDownloader(chaindb ethdb.Database, firstPivot *Block) *downloader { newPivot: make(chan *Block), } - // go d.stateFetcher() + go d.stateFetcher() return d } -/* // stateFetcher manages the active state sync and accepts requests // on its behalf. func (d *downloader) stateFetcher() { @@ -589,19 +607,13 @@ func (d *downloader) stateFetcher() { } case <-d.quitCh: return - case p:= <-d.newPivot: - d.pivotLock.Lock() - d.pivotBlock = p - d.pivotLock.Unlock() - } } } - // processSnapSyncContent takes fetch results from the queue and writes them to the // database. It also controls the synchronisation of state nodes of the pivot block. -func (d *downloader) processSnapSyncContent() error { +func (d *downloader) SnapSync() error { // Start syncing state of the reported head block. This should get us most of // the state of the pivot block. d.pivotLock.RLock() @@ -615,132 +627,20 @@ func (d *downloader) processSnapSyncContent() error { sync.Cancel() }() - // closeOnErr := func(s *stateSync) { - // if err := s.Wait(); err != nil && err != errCancelStateFetch && err != errCanceled && err != snap.ErrCancelled { - // d.queue.Close() // wake up Results - // } - // } - // go closeOnErr(sync) - - // To cater for moving pivot points, track the pivot block and subsequently - // accumulated download results separately. - // - // These will be nil up to the point where we reach the pivot, and will only - // be set temporarily if the synced blocks are piling up, but the pivot is - // still busy downloading. In that case, we need to occasionally check for - // pivot moves, so need to unblock the loop. These fields will accumulate - // the results in the meantime. - // - // Note, there's no issue with memory piling up since after 64 blocks the - // pivot will forcefully move so these accumulators will be dropped. - var ( - oldPivot *fetchResult // Locked in pivot block, might change eventually - oldTail []*fetchResult // Downloaded content after the pivot - timer = time.NewTimer(time.Second) - ) - defer timer.Stop() - for { - // Wait for the next batch of downloaded data to be available. If we have - // not yet reached the pivot point, wait blockingly as there's no need to - // spin-loop check for pivot moves. If we reached the pivot but have not - // yet processed it, check for results async, so we might notice pivot - // moves while state syncing. If the pivot was passed fully, block again - // as there's no more reason to check for pivot moves at all. - results := d.queue.Results(oldPivot == nil) - if len(results) == 0 { - // If pivot sync is done, stop - if d.committed.Load() { - d.reportSnapSyncProgress(true) - return sync.Cancel() - } - // If sync failed, stop - select { - case <-d.cancelCh: - sync.Cancel() - return errCanceled - default: - } - } - if d.chainInsertHook != nil { - d.chainInsertHook(results) - } - d.reportSnapSyncProgress(false) - - // If we haven't downloaded the pivot block yet, check pivot staleness - // notifications from the header downloader - d.pivotLock.RLock() - pivot := d.pivotHeader - d.pivotLock.RUnlock() - - if oldPivot == nil { // no results piling up, we can move the pivot - if !d.committed.Load() { // not yet passed the pivot, we can move the pivot - if pivot.Root != sync.root { // pivot position changed, we can move the pivot - sync.Cancel() - sync = d.syncState(pivot.Root) - - go closeOnErr(sync) - } - } - } else { // results already piled up, consume before handling pivot move - results = append(append([]*fetchResult{oldPivot}, oldTail...), results...) - } - // Split around the pivot block and process the two sides via snap/full sync - if !d.committed.Load() { - latest := results[len(results)-1].Header - // If the height is above the pivot block by 2 sets, it means the pivot - // become stale in the network, and it was garbage collected, move to a - // new pivot. - // - // Note, we have `reorgProtHeaderDelay` number of blocks withheld, Those - // need to be taken into account, otherwise we're detecting the pivot move - // late and will drop peers due to unavailable state!!! - if height := latest.Number.Uint64(); height >= pivot.Number.Uint64()+2*uint64(fsMinFullBlocks)-uint64(reorgProtHeaderDelay) { - log.Warn("Pivot became stale, moving", "old", pivot.Number.Uint64(), "new", height-uint64(fsMinFullBlocks)+uint64(reorgProtHeaderDelay)) - pivot = results[len(results)-1-fsMinFullBlocks+reorgProtHeaderDelay].Header // must exist as lower old pivot is uncommitted - - d.pivotLock.Lock() - d.pivotHeader = pivot - d.pivotLock.Unlock() - - // Write out the pivot into the database so a rollback beyond it will - // reenable snap sync - rawdb.WriteLastPivotNumber(d.stateDB, pivot.Number.Uint64()) - } - } - P, beforeP, afterP := splitAroundPivot(pivot.Number.Uint64(), results) - if err := d.commitSnapSyncData(beforeP, sync); err != nil { - return err - } - if P != nil { - // If new pivot block found, cancel old state retrieval and restart - if oldPivot != P { - sync.Cancel() - sync = d.syncState(P.Header.Root) - - go closeOnErr(sync) - oldPivot = P - } - // Wait for completion, occasionally checking for pivot staleness - timer.Reset(time.Second) - select { - case <-sync.done: - if sync.err != nil { - return sync.err - } - if err := d.commitPivotBlock(P); err != nil { - return err - } - oldPivot = nil - - case <-timer.C: - oldTail = afterP - continue + select { + // If stateSync is ended, clear queue and return + // If err, just return so we can see it + case <-sync.done: + if sync.err != nil { + return sync.err } - } - // Fast sync done, pivot commit done, full import - if err := d.importBlockResults(afterP); err != nil { - return err + return d.execQueue.Flush(nil, true) + case np := <-d.newPivot: + // If a new pivot block is found, cancel the current state sync and + // start a new one. + sync.Cancel() + sync = d.syncState(np.ethBlock.Hash()) } } } @@ -756,11 +656,12 @@ func (d *downloader) syncState(root common.Hash) *stateSync { // out or been delivered <-s.started case <-d.quitCh: - s.err = errCancelStateFetch + s.err = fmt.Errorf("errCancelStateFetch") //errCancelStateFetch close(s.done) } return s } + // runStateSync runs a state synchronisation until it completes or another root // hash is requested to be switched over to. func (d *downloader) runStateSync(s *stateSync) *stateSync { @@ -779,7 +680,7 @@ func (d *downloader) runStateSync(s *stateSync) *stateSync { } } } -*/ + // stateSync schedules requests for downloading a particular state trie defined // by a given state root. type stateSync struct { @@ -793,7 +694,6 @@ type stateSync struct { err error // Any error hit during sync (set before completion) } -/* // newStateSync creates a new state trie download scheduler. This method does not // yet start the sync. The user needs to call run to initiate. func newStateSync(d *downloader, root common.Hash) *stateSync { @@ -828,4 +728,3 @@ func (s *stateSync) Cancel() error { }) return s.Wait() } -*/ diff --git a/plugin/evm/vm.go b/plugin/evm/vm.go index cadafd80a1..78e4c0ec63 100644 --- a/plugin/evm/vm.go +++ b/plugin/evm/vm.go @@ -1313,7 +1313,16 @@ func (vm *VM) buildBlockWithContext(ctx context.Context, proposerVMBlockCtx *blo // We call verify without writes here to avoid generating a reference // to the blk state root in the triedb when we are going to call verify // again from the consensus engine with writes enabled. - if err := blk.verify(predicateCtx, false /*=writes*/); err != nil { + if err := blk.syntacticVerify(); err != nil { + vm.mempool.CancelCurrentTxs() + return nil, fmt.Errorf("syntactic block verification failed: %w", err) + } + if err := blk.verifyPredicates(predicateCtx); err != nil { + vm.mempool.CancelCurrentTxs() + return nil, fmt.Errorf("failed to verify predicates: %w", err) + } + + if err := blk.verify(false /*=writes*/); err != nil { vm.mempool.CancelCurrentTxs() return nil, fmt.Errorf("block failed verification due to: %w", err) } From 2d4cfdb58c40a5959b6a3f2643fe8a57b0418d4b Mon Sep 17 00:00:00 2001 From: Austin Larson Date: Fri, 21 Feb 2025 14:59:52 -0500 Subject: [PATCH 03/21] Added more logging to see results --- plugin/evm/block.go | 17 ++++++++++++++++- plugin/evm/syncervm_client.go | 31 ++++++++++++++++++++++++++++--- 2 files changed, 44 insertions(+), 4 deletions(-) diff --git a/plugin/evm/block.go b/plugin/evm/block.go index c153ac70d0..5be9c8fe45 100644 --- a/plugin/evm/block.go +++ b/plugin/evm/block.go @@ -148,6 +148,7 @@ func (b *Block) Accept(ctx context.Context) error { } func (b *Block) AcceptDuringSync(ctx context.Context) error { + log.Info("Accepting block during sync", "block", b.ID(), "height", b.Height()) return b.accept(ctx) } @@ -235,7 +236,20 @@ func (b *Block) handlePrecompileAccept(rules extras.Rules) error { // Reject implements the snowman.Block interface // If [b] contains an atomic transaction, attempt to re-issue it -func (b *Block) Reject(context.Context) error { +func (b *Block) Reject(ctx context.Context) error { + + if b.vm.StateSyncClient.AsyncReceive() { + log.Debug("Queueing block for later reject", "block", b.ID(), "height", b.Height()) + return b.vm.StateSyncClient.QueueRejectBlock(b) + } + return b.reject(ctx) +} + +func (b *Block) RejectDuringSync(ctx context.Context) error { + return b.reject(ctx) +} + +func (b *Block) reject(context.Context) error { log.Debug(fmt.Sprintf("Rejecting block %s (%s) at height %d", b.ID().Hex(), b.ID(), b.Height())) for _, tx := range b.atomicTxs { // Re-issue the transaction in the mempool, continue even if it fails @@ -312,6 +326,7 @@ func (b *Block) Verify(context.Context) error { } func (b *Block) VerifyDuringSync(context.Context) error { + log.Info("Verifying block during sync", "block", b.ID(), "height", b.Height()) return b.verify(true) } diff --git a/plugin/evm/syncervm_client.go b/plugin/evm/syncervm_client.go index 9a475ad442..03c0061990 100644 --- a/plugin/evm/syncervm_client.go +++ b/plugin/evm/syncervm_client.go @@ -111,6 +111,7 @@ type StateSyncClient interface { AsyncReceive() bool QueueVerifyBlock(*Block) error QueueAcceptBlock(*Block) error + QueueRejectBlock(*Block) error } // Syncer represents a step in state sync, @@ -130,7 +131,6 @@ func (client *stateSyncerClient) AsyncReceive() bool { func (client *stateSyncerClient) QueueVerifyBlock(b *Block) error { if !client.AsyncReceive() { - log.Info("Queueing block for later verify", "block", b.ID(), "height", b.Height()) return fmt.Errorf("cannot queue block when not using upstream syncing") } @@ -142,7 +142,6 @@ func (client *stateSyncerClient) QueueVerifyBlock(b *Block) error { func (client *stateSyncerClient) QueueAcceptBlock(b *Block) error { if !client.AsyncReceive() { - log.Info("Queueing block for later accept", "block", b.ID(), "height", b.Height()) return fmt.Errorf("cannot queue block when not using upstream syncing") } @@ -157,6 +156,7 @@ func (client *stateSyncerClient) QueueAcceptBlock(b *Block) error { // If the block is the pivot, signal the state syncer to start client.dl.pivotLock.Lock() if b.Height() >= client.dl.pivotBlock.Height()+pivotInterval { + log.Info("Setting new pivot block", "hash", b.ID(), "height", b.Height(), "timestamp", b.Timestamp()) client.dl.pivotBlock = b client.dl.newPivot <- b } @@ -165,6 +165,29 @@ func (client *stateSyncerClient) QueueAcceptBlock(b *Block) error { return nil } +func (client *stateSyncerClient) QueueRejectBlock(b *Block) error { + if !client.AsyncReceive() { + return fmt.Errorf("cannot queue block when not using upstream syncing") + } + + reject := func(a *Block) error { + return a.RejectDuringSync(context.TODO()) // yeah gotta fix this too + } + + if err := client.dl.execQueue.Insert(&queueElement{b, reject}); err != nil { + return err + } + + // If the block is the pivot, signal the state syncer to start + client.dl.pivotLock.RLock() + defer client.dl.pivotLock.RUnlock() + if b.Height() == client.dl.pivotBlock.Height() { + return fmt.Errorf("cannot reject pivot block") + } + + return nil +} + // StateSyncEnabled returns [client.enabled], which is set in the chain's config file. func (client *stateSyncerClient) StateSyncEnabled(context.Context) (bool, error) { return client.enabled, nil @@ -543,6 +566,7 @@ func (client *stateSyncerClient) upstreamSyncStateTrie(ctx context.Context) erro } else { client.network.AddConnector(ethstatesync.NewConnector(client.dl.SnapSyncer, p2pClient)) } + if err := client.dl.SnapSync(); err != nil { return err } @@ -635,7 +659,7 @@ func (d *downloader) SnapSync() error { if sync.err != nil { return sync.err } - return d.execQueue.Flush(nil, true) + return d.execQueue.Flush(nil, true) // might need to provide a cancle channel case np := <-d.newPivot: // If a new pivot block is found, cancel the current state sync and // start a new one. @@ -711,6 +735,7 @@ func newStateSync(d *downloader, root common.Hash) *stateSync { // finish. func (s *stateSync) run() { close(s.started) + log.Info("Starting new sync") s.err = s.d.SnapSyncer.Sync(s.root, s.cancel) close(s.done) } From 706962dc8c194b3fb46d7c49e1679774aa064a08 Mon Sep 17 00:00:00 2001 From: Austin Larson Date: Fri, 21 Feb 2025 16:00:08 -0500 Subject: [PATCH 04/21] Fixed hash passed to Syncer, will flush on pivot change --- plugin/evm/syncervm_client.go | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/plugin/evm/syncervm_client.go b/plugin/evm/syncervm_client.go index 03c0061990..7bdc45881e 100644 --- a/plugin/evm/syncervm_client.go +++ b/plugin/evm/syncervm_client.go @@ -641,7 +641,7 @@ func (d *downloader) SnapSync() error { // Start syncing state of the reported head block. This should get us most of // the state of the pivot block. d.pivotLock.RLock() - sync := d.syncState(d.pivotBlock.ethBlock.Hash()) + sync := d.syncState(d.pivotBlock.ethBlock.Root()) d.pivotLock.RUnlock() defer func() { @@ -664,7 +664,8 @@ func (d *downloader) SnapSync() error { // If a new pivot block is found, cancel the current state sync and // start a new one. sync.Cancel() - sync = d.syncState(np.ethBlock.Hash()) + d.execQueue.Flush(&queueElement{np, nil}, false) + sync = d.syncState(np.ethBlock.Root()) } } } From 1f2fddbc5d52738707094e2e73761c2681956ac3 Mon Sep 17 00:00:00 2001 From: Austin Larson Date: Mon, 24 Feb 2025 16:21:51 -0500 Subject: [PATCH 05/21] Added fixes from draft PR, changed incoming verification and acceptance to only include minimum for atomic state without queuing, provided protection against race between atomic sync and bootstrapping atomic verification --- plugin/evm/block.go | 87 ++++++++++++++++++----- plugin/evm/statesync/queue.go | 3 +- plugin/evm/syncervm_client.go | 128 ++++++++++------------------------ 3 files changed, 108 insertions(+), 110 deletions(-) diff --git a/plugin/evm/block.go b/plugin/evm/block.go index 5be9c8fe45..bf85ea6ac9 100644 --- a/plugin/evm/block.go +++ b/plugin/evm/block.go @@ -141,15 +141,43 @@ func (b *Block) AtomicTxs() []*atomic.Tx { return b.atomicTxs } // Accept implements the snowman.Block interface func (b *Block) Accept(ctx context.Context) error { if b.vm.StateSyncClient.AsyncReceive() { - log.Debug("Queueing block for later processing", "block", b.ID(), "height", b.Height()) - return b.vm.StateSyncClient.QueueAcceptBlock(b) + return b.acceptDuringSync() } return b.accept(ctx) } -func (b *Block) AcceptDuringSync(ctx context.Context) error { +func (b *Block) acceptDuringSync() error { + vm := b.vm + // First check to see if we should pivot + if err := b.vm.StateSyncClient.CheckPivot(b); err != nil { + return err + } + + // 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() log.Info("Accepting block during sync", "block", b.ID(), "height", b.Height()) - return b.accept(ctx) + if err := vm.acceptedBlockDB.Put(lastAcceptedKey, b.id[:]); err != nil { + return fmt.Errorf("failed to put %s as the last accepted block: %w", b.ID(), err) + } + + // Update VM state for atomic txs in this block. This includes updating the + // atomic tx repo, atomic trie, and shared memory. + atomicState, err := b.vm.atomicBackend.GetVerifiedAtomicState(common.Hash(b.ID())) + if err != nil { + // should never occur since [b] must be verified before calling Accept + return err + } + // Get pending operations on the vm's versionDB so we can apply them atomically + // with the shared memory changes. + vdbBatch, err := b.vm.versiondb.CommitBatch() + if err != nil { + return fmt.Errorf("could not create commit batch processing block[%s]: %w", b.ID(), err) + } + + // Apply any shared memory changes atomically with other pending changes to + // the vm's versionDB. + return atomicState.Accept(vdbBatch, nil) } func (b *Block) accept(context.Context) error { @@ -237,18 +265,13 @@ func (b *Block) handlePrecompileAccept(rules extras.Rules) error { // Reject implements the snowman.Block interface // If [b] contains an atomic transaction, attempt to re-issue it func (b *Block) Reject(ctx context.Context) error { - if b.vm.StateSyncClient.AsyncReceive() { - log.Debug("Queueing block for later reject", "block", b.ID(), "height", b.Height()) - return b.vm.StateSyncClient.QueueRejectBlock(b) + log.Error("Called Reject for block during dynamic state sync", "block", b.ID(), "height", b.Height()) + return errors.New("cannot reject block during dynamic state sync") } return b.reject(ctx) } -func (b *Block) RejectDuringSync(ctx context.Context) error { - return b.reject(ctx) -} - func (b *Block) reject(context.Context) error { log.Debug(fmt.Sprintf("Rejecting block %s (%s) at height %d", b.ID().Hex(), b.ID(), b.Height())) for _, tx := range b.atomicTxs { @@ -318,16 +341,48 @@ func (b *Block) Verify(context.Context) error { // If currently dynamically syncing, we should simply queue the block for later processing if b.vm.StateSyncClient.AsyncReceive() { - log.Debug("Queueing block for later processing", "block", b.ID(), "height", b.Height()) - return b.vm.StateSyncClient.QueueVerifyBlock(b) + return b.verifyDuringSync() } return b.verify(true) } -func (b *Block) VerifyDuringSync(context.Context) error { - log.Info("Verifying block during sync", "block", b.ID(), "height", b.Height()) - return b.verify(true) +func (b *Block) verifyDuringSync() error { + log.Debug("Verifying block during sync", "block", b.ID(), "height", b.Height()) + var ( + block = b.ethBlock + header = block.Header() + vm = b.vm + rules = vm.chainConfig.Rules(header.Number, params.IsMergeTODO, header.Time) + rulesExtra = *params.GetRulesExtra(rules) + ) + + txs, err := atomic.ExtractAtomicTxs(block.ExtData(), rulesExtra.IsApricotPhase5, atomic.Codec) + if err != nil { + return err + } + + // If [atomicBackend] is nil, the VM is still initializing and is reprocessing accepted blocks. + if vm.atomicBackend != nil { + if vm.atomicBackend.IsBonus(block.NumberU64(), block.Hash()) { + log.Info("skipping atomic tx verification on bonus block", "block", block.Hash()) + } else { + // Verify [txs] do not conflict with themselves or ancestor blocks. + if err := vm.verifyTxs(txs, block.ParentHash(), block.BaseFee(), block.NumberU64(), rulesExtra); err != nil { + return err + } + } + // Update the atomic backend with [txs] from this block. + // + // Note: The atomic trie canonically contains the duplicate operations + // from any bonus blocks. + _, err := vm.atomicBackend.InsertTxs(block.Hash(), block.NumberU64(), block.ParentHash(), txs) + if err != nil { + return err + } + } + + return nil } // ShouldVerifyWithContext implements the block.WithVerifyContext interface diff --git a/plugin/evm/statesync/queue.go b/plugin/evm/statesync/queue.go index 5e3b1bf453..b629fcdf89 100644 --- a/plugin/evm/statesync/queue.go +++ b/plugin/evm/statesync/queue.go @@ -1,4 +1,5 @@ -// (c) 2019-2020, Ava Labs, Inc. +// (c) 2021-2024, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. // // This file is although named queue.go, is not the same as the eth/downloader/queue.go file. // Instead, this is simply the protected array for downloader (yeah not even a queue). diff --git a/plugin/evm/syncervm_client.go b/plugin/evm/syncervm_client.go index 7bdc45881e..1e30d2ced5 100644 --- a/plugin/evm/syncervm_client.go +++ b/plugin/evm/syncervm_client.go @@ -5,11 +5,10 @@ package evm import ( "context" + "errors" "fmt" "sync" - // "time" - "github.com/ava-labs/avalanchego/database" "github.com/ava-labs/avalanchego/database/versiondb" "github.com/ava-labs/avalanchego/ids" @@ -37,7 +36,6 @@ const ( // The last 256 block hashes are necessary to support the BLOCKHASH opcode. parentsToGet = 256 pivotInterval = 128 - bufferSize = 6 * 60 * 30 // 2 * pivotInterval // extra space to be careful ) var stateSyncSummaryKey = []byte("stateSyncSummary") @@ -85,9 +83,10 @@ type stateSyncerClient struct { syncSummary message.SyncSummary stateSyncErr error - // Testing dynamic sync - syncing utils.Atomic[bool] - dl *downloader + // Dynamic sync + syncing utils.Atomic[bool] + dl *downloader + atomicLock sync.Mutex // to prevent writing during atomic sync } func NewStateSyncClient(config *stateSyncClientConfig) StateSyncClient { @@ -107,11 +106,9 @@ type StateSyncClient interface { Shutdown() error Error() error - // Methods to try to enable dynamic state sync + // Methods to enable dynamic state sync AsyncReceive() bool - QueueVerifyBlock(*Block) error - QueueAcceptBlock(*Block) error - QueueRejectBlock(*Block) error + CheckPivot(b *Block) error } // Syncer represents a step in state sync, @@ -126,63 +123,21 @@ type Syncer interface { // AsyncReceive returns true if the client is ready to receive a message from the engine // Should return true if syncing and useUpstream is true, i.e. currently dynamicaling syncing func (client *stateSyncerClient) AsyncReceive() bool { - return client.useUpstream && client.syncing.Get() && client.dl != nil && !client.dl.execQueue.Closed() + // Block until atomic sync is completed for bootstrapping + client.atomicLock.Lock() + client.atomicLock.Unlock() + return client.useUpstream && client.syncing.Get() && client.dl != nil } -func (client *stateSyncerClient) QueueVerifyBlock(b *Block) error { - if !client.AsyncReceive() { - return fmt.Errorf("cannot queue block when not using upstream syncing") - } - - verify := func(a *Block) error { - return a.VerifyDuringSync(context.TODO()) // is this right? - } - return client.dl.execQueue.Insert(&queueElement{b, verify}) -} - -func (client *stateSyncerClient) QueueAcceptBlock(b *Block) error { - if !client.AsyncReceive() { - return fmt.Errorf("cannot queue block when not using upstream syncing") - } - - accept := func(a *Block) error { - return a.AcceptDuringSync(context.TODO()) // yeah gotta fix this too - } - - if err := client.dl.execQueue.Insert(&queueElement{b, accept}); err != nil { - return err - } - - // If the block is the pivot, signal the state syncer to start - client.dl.pivotLock.Lock() +func (client *stateSyncerClient) CheckPivot(b *Block) error { + // No lock necessary, as this is protected by the engine if b.Height() >= client.dl.pivotBlock.Height()+pivotInterval { log.Info("Setting new pivot block", "hash", b.ID(), "height", b.Height(), "timestamp", b.Timestamp()) client.dl.pivotBlock = b client.dl.newPivot <- b - } - client.dl.pivotLock.Unlock() - - return nil -} - -func (client *stateSyncerClient) QueueRejectBlock(b *Block) error { - if !client.AsyncReceive() { - return fmt.Errorf("cannot queue block when not using upstream syncing") - } - - reject := func(a *Block) error { - return a.RejectDuringSync(context.TODO()) // yeah gotta fix this too - } - - if err := client.dl.execQueue.Insert(&queueElement{b, reject}); err != nil { - return err - } - - // If the block is the pivot, signal the state syncer to start - client.dl.pivotLock.RLock() - defer client.dl.pivotLock.RUnlock() - if b.Height() == client.dl.pivotBlock.Height() { - return fmt.Errorf("cannot reject pivot block") + } else if b.Height() <= client.dl.pivotBlock.Height() { + log.Warn("Received block with height less than pivot block", "hash", b.ID(), "height", b.Height(), "timestamp", b.Timestamp()) + return errors.New("received block with height less than pivot block") } return nil @@ -245,10 +200,12 @@ func (client *stateSyncerClient) stateSync(ctx context.Context) error { if err := client.syncAtomicTrie(ctx); err != nil { return err } + client.atomicLock.Unlock() return client.syncStateTrie(ctx) } +/* Unused func convertReadOnlyToBidirectional[T any](readOnly <-chan T) chan T { bidirectional := make(chan T) @@ -260,7 +217,7 @@ func convertReadOnlyToBidirectional[T any](readOnly <-chan T) chan T { }() return bidirectional -} +}*/ // acceptSyncSummary returns true if sync will be performed and launches the state sync process // in a goroutine. @@ -305,6 +262,9 @@ func (client *stateSyncerClient) acceptSyncSummary(proposedSummary message.SyncS log.Info("Starting state sync", "summary", proposedSummary) + // Lock the atomic trie to prevent writes during the atomic sync from dynamic syncing + client.atomicLock.Lock() + // create a cancellable ctx for the state sync goroutine ctx, cancel := context.WithCancel(context.Background()) client.cancel = cancel @@ -313,15 +273,15 @@ func (client *stateSyncerClient) acceptSyncSummary(proposedSummary message.SyncS // Must first find first pivot block to signal bootstrapper stateBlock, err := client.state.GetBlock(context.TODO(), ids.ID(client.syncSummary.BlockHash)) if err != nil { - return block.StateSyncDynamic, fmt.Errorf("could not get block by hash from client state: %s", client.syncSummary.BlockHash) + return block.StateSyncSkipped, fmt.Errorf("could not get block by hash from client state: %s", client.syncSummary.BlockHash) } wrapper, ok := stateBlock.(*chain.BlockWrapper) if !ok { - return block.StateSyncDynamic, fmt.Errorf("could not convert block(%T) to *chain.BlockWrapper", wrapper) + return block.StateSyncSkipped, fmt.Errorf("could not convert block(%T) to *chain.BlockWrapper", wrapper) } evmBlock, ok := wrapper.Block.(*Block) if !ok { - return block.StateSyncDynamic, fmt.Errorf("could not convert block(%T) to evm.Block", stateBlock) + return block.StateSyncSkipped, fmt.Errorf("could not convert block(%T) to evm.Block", stateBlock) } b := evmBlock.ethBlock @@ -330,15 +290,15 @@ func (client *stateSyncerClient) acceptSyncSummary(proposedSummary message.SyncS client.chain.BloomIndexer().AddCheckpoint(parentHeight/params.BloomBitsBlocks, parentHash) if err := client.updateVMMarkers(); err != nil { - return block.StateSyncDynamic, fmt.Errorf("error updating vm markers, height=%d, hash=%s, err=%w", b.NumberU64(), b.Hash(), err) + return block.StateSyncSkipped, fmt.Errorf("error updating vm markers, height=%d, hash=%s, err=%w", b.NumberU64(), b.Hash(), err) } if err := client.state.SetLastAcceptedBlock(evmBlock); err != nil { - return block.StateSyncDynamic, err + return block.StateSyncSkipped, err } if err := client.atomicBackend.ApplyToSharedMemory(b.NumberU64()); err != nil { - return block.StateSyncDynamic, err + return block.StateSyncSkipped, err } // Set downloader using pivot @@ -570,12 +530,8 @@ func (client *stateSyncerClient) upstreamSyncStateTrie(ctx context.Context) erro if err := client.dl.SnapSync(); err != nil { return err } - log.Info("Upstream state syncer completed, moving to catch-up") - - // Now that we have synced the state trie to static pivot, Verify and Accept all pending blocks - // Incoming blocks are still appended to buffer until emptied - err := client.dl.execQueue.Flush(nil, true) - return err + log.Info("Upstream state syncer completed") + return nil } func (e queueElement) ExitQueue() error { @@ -590,9 +546,7 @@ type queueElement struct { var _ ethstatesync.Executable = &queueElement{} type downloader struct { - pivotLock sync.RWMutex pivotBlock *Block - execQueue *ethstatesync.Queue[queueElement] SnapSyncer *snap.Syncer stateSyncStart chan *stateSync @@ -602,13 +556,8 @@ type downloader struct { } func newDownloader(chaindb ethdb.Database, firstPivot *Block) *downloader { - compare := func(a, b *queueElement) int { - return int(a.block.Height()) - int(b.block.Height()) - } - d := &downloader{ pivotBlock: firstPivot, - execQueue: ethstatesync.NewQueue[queueElement](bufferSize, compare), SnapSyncer: snap.NewSyncer(chaindb, rawdb.HashScheme), stateSyncStart: make(chan *stateSync), quitCh: make(chan struct{}), @@ -640,9 +589,7 @@ func (d *downloader) stateFetcher() { func (d *downloader) SnapSync() error { // Start syncing state of the reported head block. This should get us most of // the state of the pivot block. - d.pivotLock.RLock() sync := d.syncState(d.pivotBlock.ethBlock.Root()) - d.pivotLock.RUnlock() defer func() { // The `sync` object is replaced every time the pivot moves. We need to @@ -656,16 +603,12 @@ func (d *downloader) SnapSync() error { // If stateSync is ended, clear queue and return // If err, just return so we can see it case <-sync.done: - if sync.err != nil { - return sync.err - } - return d.execQueue.Flush(nil, true) // might need to provide a cancle channel - case np := <-d.newPivot: + return sync.err + case newPivot := <-d.newPivot: // If a new pivot block is found, cancel the current state sync and // start a new one. sync.Cancel() - d.execQueue.Flush(&queueElement{np, nil}, false) - sync = d.syncState(np.ethBlock.Root()) + sync = d.syncState(newPivot.ethBlock.Root()) } } } @@ -681,7 +624,7 @@ func (d *downloader) syncState(root common.Hash) *stateSync { // out or been delivered <-s.started case <-d.quitCh: - s.err = fmt.Errorf("errCancelStateFetch") //errCancelStateFetch + s.err = errors.New("errCancelStateFetch") //errCancelStateFetch from geth close(s.done) } return s @@ -690,7 +633,7 @@ func (d *downloader) syncState(root common.Hash) *stateSync { // runStateSync runs a state synchronisation until it completes or another root // hash is requested to be switched over to. func (d *downloader) runStateSync(s *stateSync) *stateSync { - log.Trace("State sync starting", "root", s.root) + log.Debug("State sync starting", "root", s.root) go s.run() defer s.Cancel() @@ -736,7 +679,6 @@ func newStateSync(d *downloader, root common.Hash) *stateSync { // finish. func (s *stateSync) run() { close(s.started) - log.Info("Starting new sync") s.err = s.d.SnapSyncer.Sync(s.root, s.cancel) close(s.done) } From f2c8035b65492603b67397c62b1fb3eba522cfbd Mon Sep 17 00:00:00 2001 From: Austin Larson Date: Tue, 25 Feb 2025 12:08:19 -0500 Subject: [PATCH 06/21] Commented out idea how to fix issue --- core/blockchain.go | 20 ++++++++++++++++++++ plugin/evm/block.go | 7 +++++++ 2 files changed, 27 insertions(+) diff --git a/core/blockchain.go b/core/blockchain.go index 844df1e188..ad3d998565 100644 --- a/core/blockchain.go +++ b/core/blockchain.go @@ -1144,6 +1144,26 @@ func (bc *BlockChain) writeBlockAndSetHead(block *types.Block, parentRoot common return nil } +// InsertBlockDuringSync writes the block to the database without receipts or state. +// skipping pruning management for simplicity +func (bc *BlockChain) InsertBlockDuringSync(block *types.Block) error { + // Write the block to the database + batch := bc.db.NewBatch() + rawdb.WriteBlock(batch, block) + if err := batch.Write(); err != nil { + return fmt.Errorf("failed to write block during sync: %w", err) + } + + // If [block] represents a new tip of the canonical chain, we optimistically add it before + // setPreference is called. Otherwise, we consider it a side chain block. + if bc.newTip(block) { + bc.writeCanonicalBlockWithLogs(block, nil) + } else { + bc.chainSideFeed.Send(ChainSideEvent{Block: block}) + } + return nil +} + // writeBlockWithState writes the block and all associated state to the database, // but it expects the chain mutex to be held. func (bc *BlockChain) writeBlockWithState(block *types.Block, parentRoot common.Hash, receipts []*types.Receipt, state *state.StateDB) error { diff --git a/plugin/evm/block.go b/plugin/evm/block.go index bf85ea6ac9..4fa1b0913a 100644 --- a/plugin/evm/block.go +++ b/plugin/evm/block.go @@ -362,6 +362,13 @@ func (b *Block) verifyDuringSync() error { return err } + // This is my idea how to fix it + /* + if err := vm.blockChain.InsertBlockDuringSync(block); err != nil { + return err + } + */ + // If [atomicBackend] is nil, the VM is still initializing and is reprocessing accepted blocks. if vm.atomicBackend != nil { if vm.atomicBackend.IsBonus(block.NumberU64(), block.Hash()) { From 58a03c80101b342ef5fe9e9ec22ccba0a8df14a2 Mon Sep 17 00:00:00 2001 From: Austin Larson Date: Tue, 25 Feb 2025 14:26:54 -0500 Subject: [PATCH 07/21] Added logging used on test node --- core/blockchain.go | 10 +++++----- core/blockchain_reader.go | 9 ++++++++- plugin/evm/block.go | 25 ++++++++++++++++++------- plugin/evm/syncervm_client.go | 2 +- plugin/evm/vm.go | 3 +++ 5 files changed, 35 insertions(+), 14 deletions(-) diff --git a/core/blockchain.go b/core/blockchain.go index ad3d998565..6a354df0a3 100644 --- a/core/blockchain.go +++ b/core/blockchain.go @@ -1156,11 +1156,11 @@ func (bc *BlockChain) InsertBlockDuringSync(block *types.Block) error { // If [block] represents a new tip of the canonical chain, we optimistically add it before // setPreference is called. Otherwise, we consider it a side chain block. - if bc.newTip(block) { - bc.writeCanonicalBlockWithLogs(block, nil) - } else { - bc.chainSideFeed.Send(ChainSideEvent{Block: block}) - } + // if bc.newTip(block) { + // bc.writeCanonicalBlockWithLogs(block, nil) + // } else { + // bc.chainSideFeed.Send(ChainSideEvent{Block: block}) + // } return nil } diff --git a/core/blockchain_reader.go b/core/blockchain_reader.go index 70465df4b9..78f85c7462 100644 --- a/core/blockchain_reader.go +++ b/core/blockchain_reader.go @@ -36,6 +36,7 @@ import ( "github.com/ava-labs/libevm/common" "github.com/ava-labs/libevm/core/vm" "github.com/ava-labs/libevm/event" + "github.com/ava-labs/libevm/log" "github.com/ava-labs/libevm/triedb" ) @@ -98,12 +99,18 @@ func (bc *BlockChain) GetBody(hash common.Hash) *types.Body { // HasBlock checks if a block is fully present in the database or not. func (bc *BlockChain) HasBlock(hash common.Hash, number uint64) bool { if bc.blockCache.Contains(hash) { + log.Info("Found in cache") return true } if !bc.HasHeader(hash, number) { + log.Info("HasHeader failed") return false } - return rawdb.HasBody(bc.db, hash, number) + if !rawdb.HasBody(bc.db, hash, number) { + log.Info("HasBody failed") + return false + } + return true } // HasFastBlock checks if a fast block is fully present in the database or not. diff --git a/plugin/evm/block.go b/plugin/evm/block.go index 4fa1b0913a..c0fb553360 100644 --- a/plugin/evm/block.go +++ b/plugin/evm/block.go @@ -177,7 +177,13 @@ func (b *Block) acceptDuringSync() error { // Apply any shared memory changes atomically with other pending changes to // the vm's versionDB. - return atomicState.Accept(vdbBatch, nil) + if err := atomicState.Accept(vdbBatch, nil); err != nil { + return err + } + + log.Info("Returning from accept without error") + + return nil } func (b *Block) accept(context.Context) error { @@ -362,12 +368,9 @@ func (b *Block) verifyDuringSync() error { return err } - // This is my idea how to fix it - /* - if err := vm.blockChain.InsertBlockDuringSync(block); err != nil { - return err - } - */ + if err := vm.blockChain.InsertBlockDuringSync(block); err != nil { + return err + } // If [atomicBackend] is nil, the VM is still initializing and is reprocessing accepted blocks. if vm.atomicBackend != nil { @@ -389,6 +392,8 @@ func (b *Block) verifyDuringSync() error { } } + log.Info("Returning from verify without error") + return nil } @@ -436,6 +441,12 @@ func (b *Block) VerifyWithContext(ctx context.Context, proposerVMBlockCtx *block return fmt.Errorf("failed to verify predicates: %w", err) } } + + // If currently dynamically syncing, we should simply queue the block for later processing + if b.vm.StateSyncClient.AsyncReceive() { + return b.verifyDuringSync() + } + return b.verify(true) } diff --git a/plugin/evm/syncervm_client.go b/plugin/evm/syncervm_client.go index 1e30d2ced5..cc8dd22053 100644 --- a/plugin/evm/syncervm_client.go +++ b/plugin/evm/syncervm_client.go @@ -304,7 +304,7 @@ func (client *stateSyncerClient) acceptSyncSummary(proposedSummary message.SyncS // Set downloader using pivot client.dl = newDownloader(client.chaindb, evmBlock) - log.Info("Set LastAcceptedBlock to first pivot with height", evmBlock.Height(), "timestamp", evmBlock.Timestamp()) + log.Info("Set LastAcceptedBlock to first pivot", "height", evmBlock.ID(), evmBlock.Height(), "timestamp", evmBlock.Timestamp()) } go func() { defer client.wg.Done() diff --git a/plugin/evm/vm.go b/plugin/evm/vm.go index 78e4c0ec63..e418e08bc2 100644 --- a/plugin/evm/vm.go +++ b/plugin/evm/vm.go @@ -1642,16 +1642,19 @@ func (vm *VM) verifyTx(tx *atomic.Tx, parentHash common.Hash, baseFee *big.Int, func (vm *VM) verifyTxs(txs []*atomic.Tx, parentHash common.Hash, baseFee *big.Int, height uint64, rules extras.Rules) error { // Ensure that the parent was verified and inserted correctly. if !vm.blockChain.HasBlock(parentHash, height-1) { + log.Error("First errRejectedParent in verifyTxs") return errRejectedParent } ancestorID := ids.ID(parentHash) + log.Info("Ancestor", "ID", ancestorID.String()) // If the ancestor is unknown, then the parent failed verification when // it was called. // If the ancestor is rejected, then this block shouldn't be inserted // into the canonical chain because the parent will be missing. ancestorInf, err := vm.GetBlockInternal(context.TODO(), ancestorID) if err != nil { + log.Info("Second errRejectedParent in verifyTxs") return errRejectedParent } ancestor, ok := ancestorInf.(*Block) From 370167f10cb6644c50a5d2f91654cabbd5feb85c Mon Sep 17 00:00:00 2001 From: Austin Larson Date: Tue, 25 Feb 2025 15:06:45 -0500 Subject: [PATCH 08/21] Updated logging, ignore SetPreference, only update atomic state on reject --- core/blockchain.go | 10 +++++----- core/blockchain_reader.go | 9 +-------- plugin/evm/block.go | 23 +++++++++++++++++++---- plugin/evm/vm.go | 8 +++++--- 4 files changed, 30 insertions(+), 20 deletions(-) diff --git a/core/blockchain.go b/core/blockchain.go index 6a354df0a3..ad3d998565 100644 --- a/core/blockchain.go +++ b/core/blockchain.go @@ -1156,11 +1156,11 @@ func (bc *BlockChain) InsertBlockDuringSync(block *types.Block) error { // If [block] represents a new tip of the canonical chain, we optimistically add it before // setPreference is called. Otherwise, we consider it a side chain block. - // if bc.newTip(block) { - // bc.writeCanonicalBlockWithLogs(block, nil) - // } else { - // bc.chainSideFeed.Send(ChainSideEvent{Block: block}) - // } + if bc.newTip(block) { + bc.writeCanonicalBlockWithLogs(block, nil) + } else { + bc.chainSideFeed.Send(ChainSideEvent{Block: block}) + } return nil } diff --git a/core/blockchain_reader.go b/core/blockchain_reader.go index 78f85c7462..70465df4b9 100644 --- a/core/blockchain_reader.go +++ b/core/blockchain_reader.go @@ -36,7 +36,6 @@ import ( "github.com/ava-labs/libevm/common" "github.com/ava-labs/libevm/core/vm" "github.com/ava-labs/libevm/event" - "github.com/ava-labs/libevm/log" "github.com/ava-labs/libevm/triedb" ) @@ -99,18 +98,12 @@ func (bc *BlockChain) GetBody(hash common.Hash) *types.Body { // HasBlock checks if a block is fully present in the database or not. func (bc *BlockChain) HasBlock(hash common.Hash, number uint64) bool { if bc.blockCache.Contains(hash) { - log.Info("Found in cache") return true } if !bc.HasHeader(hash, number) { - log.Info("HasHeader failed") return false } - if !rawdb.HasBody(bc.db, hash, number) { - log.Info("HasBody failed") - return false - } - return true + return rawdb.HasBody(bc.db, hash, number) } // HasFastBlock checks if a fast block is fully present in the database or not. diff --git a/plugin/evm/block.go b/plugin/evm/block.go index c0fb553360..242f44c8d6 100644 --- a/plugin/evm/block.go +++ b/plugin/evm/block.go @@ -181,7 +181,7 @@ func (b *Block) acceptDuringSync() error { return err } - log.Info("Returning from accept without error") + log.Info("Returning from accept without error", "block", b.ID(), "height", b.Height()) return nil } @@ -272,12 +272,27 @@ func (b *Block) handlePrecompileAccept(rules extras.Rules) error { // If [b] contains an atomic transaction, attempt to re-issue it func (b *Block) Reject(ctx context.Context) error { if b.vm.StateSyncClient.AsyncReceive() { - log.Error("Called Reject for block during dynamic state sync", "block", b.ID(), "height", b.Height()) - return errors.New("cannot reject block during dynamic state sync") + log.Warn("Called Reject for block during dynamic state sync", "block", b.ID(), "height", b.Height()) + return b.rejectDuringSync(ctx) } return b.reject(ctx) } +func (b *Block) rejectDuringSync(ctx context.Context) error { + atomicState, err := b.vm.atomicBackend.GetVerifiedAtomicState(common.Hash(b.ID())) + if err != nil { + // should never occur since [b] must be verified before calling Reject + log.Error("Should never happen because block must be verified before calling Reject", "block", b.ID(), "height", b.Height()) + return err + } + if err := atomicState.Reject(); err != nil { + return err + } + + log.Info("Returning from reject without error", "block", b.ID(), "height", b.Height()) + return nil +} + func (b *Block) reject(context.Context) error { log.Debug(fmt.Sprintf("Rejecting block %s (%s) at height %d", b.ID().Hex(), b.ID(), b.Height())) for _, tx := range b.atomicTxs { @@ -392,7 +407,7 @@ func (b *Block) verifyDuringSync() error { } } - log.Info("Returning from verify without error") + log.Info("Returning from verify without error", "block", b.ID(), "height", b.Height()) return nil } diff --git a/plugin/evm/vm.go b/plugin/evm/vm.go index e418e08bc2..3826d63ade 100644 --- a/plugin/evm/vm.go +++ b/plugin/evm/vm.go @@ -1407,6 +1407,11 @@ func (vm *VM) SetPreference(ctx context.Context, blkID ids.ID) error { return fmt.Errorf("failed to set preference to %s: %w", blkID, err) } + if vm.StateSyncClient.AsyncReceive() { + log.Warn("cannot set preference while state sync is in progress", "block", blkID) + return nil + } + return vm.blockChain.SetPreference(block.(*Block).ethBlock) } @@ -1642,19 +1647,16 @@ func (vm *VM) verifyTx(tx *atomic.Tx, parentHash common.Hash, baseFee *big.Int, func (vm *VM) verifyTxs(txs []*atomic.Tx, parentHash common.Hash, baseFee *big.Int, height uint64, rules extras.Rules) error { // Ensure that the parent was verified and inserted correctly. if !vm.blockChain.HasBlock(parentHash, height-1) { - log.Error("First errRejectedParent in verifyTxs") return errRejectedParent } ancestorID := ids.ID(parentHash) - log.Info("Ancestor", "ID", ancestorID.String()) // If the ancestor is unknown, then the parent failed verification when // it was called. // If the ancestor is rejected, then this block shouldn't be inserted // into the canonical chain because the parent will be missing. ancestorInf, err := vm.GetBlockInternal(context.TODO(), ancestorID) if err != nil { - log.Info("Second errRejectedParent in verifyTxs") return errRejectedParent } ancestor, ok := ancestorInf.(*Block) From 89822abfbe90ddca58b89a4a86a2263a83932bd3 Mon Sep 17 00:00:00 2001 From: Austin Larson Date: Wed, 26 Feb 2025 14:10:58 -0500 Subject: [PATCH 09/21] Added simple queue to downloader for incoming blocks, flushing on pivot. The operation of the downloader is too tightly coupled with syncer client, especially the use of locks - this should be fixed --- plugin/evm/block.go | 56 ++++++--- plugin/evm/syncervm_client.go | 219 +++++++++++++++++++--------------- 2 files changed, 163 insertions(+), 112 deletions(-) diff --git a/plugin/evm/block.go b/plugin/evm/block.go index 242f44c8d6..0cd3157fa2 100644 --- a/plugin/evm/block.go +++ b/plugin/evm/block.go @@ -139,19 +139,15 @@ func (b *Block) ID() ids.ID { return b.id } func (b *Block) AtomicTxs() []*atomic.Tx { return b.atomicTxs } // Accept implements the snowman.Block interface -func (b *Block) Accept(ctx context.Context) error { +func (b *Block) Accept(context.Context) error { if b.vm.StateSyncClient.AsyncReceive() { - return b.acceptDuringSync() + return b.vm.StateSyncClient.QueueBlockOrPivot(b, acceptSyncBlockRequest) } - return b.accept(ctx) + return b.accept() } func (b *Block) acceptDuringSync() error { vm := b.vm - // First check to see if we should pivot - if err := b.vm.StateSyncClient.CheckPivot(b); err != nil { - return err - } // 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. @@ -186,7 +182,7 @@ func (b *Block) acceptDuringSync() error { return nil } -func (b *Block) accept(context.Context) error { +func (b *Block) accept() error { vm := b.vm // Although returning an error from Accept is considered fatal, it is good @@ -273,12 +269,12 @@ func (b *Block) handlePrecompileAccept(rules extras.Rules) error { func (b *Block) Reject(ctx context.Context) error { if b.vm.StateSyncClient.AsyncReceive() { log.Warn("Called Reject for block during dynamic state sync", "block", b.ID(), "height", b.Height()) - return b.rejectDuringSync(ctx) + return b.vm.StateSyncClient.QueueBlockOrPivot(b, rejectSyncBlockRequest) } - return b.reject(ctx) + return b.reject() } -func (b *Block) rejectDuringSync(ctx context.Context) error { +func (b *Block) rejectDuringSync() error { atomicState, err := b.vm.atomicBackend.GetVerifiedAtomicState(common.Hash(b.ID())) if err != nil { // should never occur since [b] must be verified before calling Reject @@ -293,7 +289,7 @@ func (b *Block) rejectDuringSync(ctx context.Context) error { return nil } -func (b *Block) reject(context.Context) error { +func (b *Block) reject() error { log.Debug(fmt.Sprintf("Rejecting block %s (%s) at height %d", b.ID().Hex(), b.ID(), b.Height())) for _, tx := range b.atomicTxs { // Re-issue the transaction in the mempool, continue even if it fails @@ -360,9 +356,9 @@ func (b *Block) Verify(context.Context) error { } } - // If currently dynamically syncing, we should simply queue the block for later processing + // If currently dynamically syncing, we should simply postpone execution if b.vm.StateSyncClient.AsyncReceive() { - return b.verifyDuringSync() + return b.vm.StateSyncClient.QueueBlockOrPivot(b, verifySyncBlockRequest) } return b.verify(true) @@ -457,9 +453,9 @@ func (b *Block) VerifyWithContext(ctx context.Context, proposerVMBlockCtx *block } } - // If currently dynamically syncing, we should simply queue the block for later processing + // If currently dynamically syncing, we should postpone execution if b.vm.StateSyncClient.AsyncReceive() { - return b.verifyDuringSync() + return b.vm.StateSyncClient.QueueBlockOrPivot(b, verifySyncBlockRequest) } return b.verify(true) @@ -558,6 +554,34 @@ func (b *Block) verifyUTXOsPresent() error { return nil } +func (b *Block) ExecuteSyncRequest(req syncBlockRequest, final bool) error { + if final { + switch req { + case verifySyncBlockRequest: + return b.verify(true) + case acceptSyncBlockRequest: + return b.accept() + case rejectSyncBlockRequest: + return b.reject() + default: + // Should never happen + return fmt.Errorf("Unable to perform block operation of %d", req) + } + } else { + switch req { + case verifySyncBlockRequest: + return b.verifyDuringSync() + case acceptSyncBlockRequest: + return b.acceptDuringSync() + case rejectSyncBlockRequest: + return b.rejectDuringSync() + default: + // Should never happen + return fmt.Errorf("Unable to perform block operation of %d", req) + } + } +} + // Bytes implements the snowman.Block interface func (b *Block) Bytes() []byte { res, err := rlp.EncodeToBytes(b.ethBlock) diff --git a/plugin/evm/syncervm_client.go b/plugin/evm/syncervm_client.go index cc8dd22053..235c8a3c4a 100644 --- a/plugin/evm/syncervm_client.go +++ b/plugin/evm/syncervm_client.go @@ -34,8 +34,21 @@ import ( const ( // State sync fetches [parentsToGet] parents of the block it syncs to. // The last 256 block hashes are necessary to support the BLOCKHASH opcode. - parentsToGet = 256 + parentsToGet = 256 + + // Dynamic state switches state root occasionally + // Buffer must be large enough to pivotInterval = 128 + bufferSize = 3 * pivotInterval +) + +type syncBlockRequest uint8 + +const ( + // Constants to identify block requests + verifySyncBlockRequest syncBlockRequest = iota + 1 + acceptSyncBlockRequest + rejectSyncBlockRequest ) var stateSyncSummaryKey = []byte("stateSyncSummary") @@ -84,9 +97,9 @@ type stateSyncerClient struct { stateSyncErr error // Dynamic sync - syncing utils.Atomic[bool] - dl *downloader - atomicLock sync.Mutex // to prevent writing during atomic sync + syncing utils.Atomic[bool] + dl *downloader + queueLock sync.Mutex // to prevent writing during atomic sync } func NewStateSyncClient(config *stateSyncClientConfig) StateSyncClient { @@ -108,7 +121,7 @@ type StateSyncClient interface { // Methods to enable dynamic state sync AsyncReceive() bool - CheckPivot(b *Block) error + QueueBlockOrPivot(*Block, syncBlockRequest) error } // Syncer represents a step in state sync, @@ -124,23 +137,13 @@ type Syncer interface { // Should return true if syncing and useUpstream is true, i.e. currently dynamicaling syncing func (client *stateSyncerClient) AsyncReceive() bool { // Block until atomic sync is completed for bootstrapping - client.atomicLock.Lock() - client.atomicLock.Unlock() + client.queueLock.Lock() + client.queueLock.Unlock() return client.useUpstream && client.syncing.Get() && client.dl != nil } -func (client *stateSyncerClient) CheckPivot(b *Block) error { - // No lock necessary, as this is protected by the engine - if b.Height() >= client.dl.pivotBlock.Height()+pivotInterval { - log.Info("Setting new pivot block", "hash", b.ID(), "height", b.Height(), "timestamp", b.Timestamp()) - client.dl.pivotBlock = b - client.dl.newPivot <- b - } else if b.Height() <= client.dl.pivotBlock.Height() { - log.Warn("Received block with height less than pivot block", "hash", b.ID(), "height", b.Height(), "timestamp", b.Timestamp()) - return errors.New("received block with height less than pivot block") - } - - return nil +func (client *stateSyncerClient) QueueBlockOrPivot(b *Block, req syncBlockRequest) error { + return client.dl.QueueBlockOrPivot(b, req) } // StateSyncEnabled returns [client.enabled], which is set in the chain's config file. @@ -189,6 +192,8 @@ func (client *stateSyncerClient) ParseStateSummary(_ context.Context, summaryByt // stateSync blockingly performs the state sync for the EVM state and the atomic state // to [client.syncSummary]. returns an error if one occurred. func (client *stateSyncerClient) stateSync(ctx context.Context) error { + client.syncing.Set(true) + defer client.syncing.Set(false) if err := client.syncBlocks(ctx, client.syncSummary.BlockHash, client.syncSummary.BlockNumber, parentsToGet); err != nil { return err } @@ -200,25 +205,11 @@ func (client *stateSyncerClient) stateSync(ctx context.Context) error { if err := client.syncAtomicTrie(ctx); err != nil { return err } - client.atomicLock.Unlock() + client.queueLock.Unlock() return client.syncStateTrie(ctx) } -/* Unused -func convertReadOnlyToBidirectional[T any](readOnly <-chan T) chan T { - bidirectional := make(chan T) - - go func() { - defer close(bidirectional) - for value := range readOnly { - bidirectional <- value - } - }() - - return bidirectional -}*/ - // acceptSyncSummary returns true if sync will be performed and launches the state sync process // in a goroutine. func (client *stateSyncerClient) acceptSyncSummary(proposedSummary message.SyncSummary) (block.StateSyncMode, error) { @@ -262,46 +253,19 @@ func (client *stateSyncerClient) acceptSyncSummary(proposedSummary message.SyncS log.Info("Starting state sync", "summary", proposedSummary) - // Lock the atomic trie to prevent writes during the atomic sync from dynamic syncing - client.atomicLock.Lock() + // Lock the atomic trie to prevent pivots during the atomic sync from dynamic syncing + client.queueLock.Lock() // create a cancellable ctx for the state sync goroutine 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 if client.useUpstream { - // Must first find first pivot block to signal bootstrapper - stateBlock, err := client.state.GetBlock(context.TODO(), ids.ID(client.syncSummary.BlockHash)) - if err != nil { - return block.StateSyncSkipped, fmt.Errorf("could not get block by hash from client state: %s", client.syncSummary.BlockHash) - } - wrapper, ok := stateBlock.(*chain.BlockWrapper) - if !ok { - return block.StateSyncSkipped, fmt.Errorf("could not convert block(%T) to *chain.BlockWrapper", wrapper) - } - evmBlock, ok := wrapper.Block.(*Block) - if !ok { - return block.StateSyncSkipped, fmt.Errorf("could not convert block(%T) to evm.Block", stateBlock) - } - - b := evmBlock.ethBlock - parentHeight := b.NumberU64() - 1 - parentHash := b.ParentHash() - client.chain.BloomIndexer().AddCheckpoint(parentHeight/params.BloomBitsBlocks, parentHash) - - if err := client.updateVMMarkers(); err != nil { - return block.StateSyncSkipped, fmt.Errorf("error updating vm markers, height=%d, hash=%s, err=%w", b.NumberU64(), b.Hash(), err) - } - - if err := client.state.SetLastAcceptedBlock(evmBlock); err != nil { - return block.StateSyncSkipped, err - } - - if err := client.atomicBackend.ApplyToSharedMemory(b.NumberU64()); err != nil { - return block.StateSyncSkipped, err - } + // I don't think all of this is necessary/correct, but I'm not sure which parts can be dropped + client.finishSync(client.syncSummary.BlockHash) // Set downloader using pivot + evmBlock := client.getEVMBlockFromHash(client.syncSummary.BlockHash) client.dl = newDownloader(client.chaindb, evmBlock) log.Info("Set LastAcceptedBlock to first pivot", "height", evmBlock.ID(), evmBlock.Height(), "timestamp", evmBlock.Timestamp()) @@ -310,18 +274,24 @@ func (client *stateSyncerClient) acceptSyncSummary(proposedSummary message.SyncS defer client.wg.Done() defer cancel() - client.syncing.Set(true) if err := client.stateSync(ctx); err != nil { client.stateSyncErr = err } else { - client.stateSyncErr = client.finishSync() + if client.useUpstream { + client.stateSyncErr = client.finishSync(client.dl.pivotBlock.ethBlock.Hash()) + + // should be locked from final, opens for regular block operations + // if any error occurs during sync, mutex will not be locked + client.dl.bufferLock.Unlock() + } else { + client.stateSyncErr = client.finishSync(client.syncSummary.BlockHash) + } } // 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.stateSyncErr) client.toEngine <- commonEng.StateSyncDone - client.syncing.Set(false) }() if client.useUpstream { @@ -431,32 +401,37 @@ func (client *stateSyncerClient) Shutdown() error { 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 *stateSyncerClient) finishSync() error { +func (client *stateSyncerClient) getEVMBlockFromHash(blockHash common.Hash) *Block { + // Must first find first pivot block to signal bootstrapper stateBlock, err := client.state.GetBlock(context.TODO(), ids.ID(client.syncSummary.BlockHash)) if err != nil { - return fmt.Errorf("could not get block by hash from client state: %s", client.syncSummary.BlockHash) + log.Error("could not get block by hash from client state", "hash", client.syncSummary.BlockHash) + return nil } - wrapper, ok := stateBlock.(*chain.BlockWrapper) if !ok { - return fmt.Errorf("could not convert block(%T) to *chain.BlockWrapper", wrapper) + log.Error("could not convert block to *chain.BlockWrapper", "type", wrapper, "hash", blockHash) + return nil } evmBlock, ok := wrapper.Block.(*Block) if !ok { - return fmt.Errorf("could not convert block(%T) to evm.Block", stateBlock) + log.Error("could not convert block(%T) to evm.Block", stateBlock) + return nil } - block := evmBlock.ethBlock + return evmBlock +} - if block.Hash() != client.syncSummary.BlockHash { - return fmt.Errorf("attempted to set last summary block to unexpected block hash: (%s != %s)", block.Hash(), client.syncSummary.BlockHash) - } - if block.NumberU64() != client.syncSummary.BlockNumber { - return fmt.Errorf("attempted to set last summary block to unexpected block number: (%d != %d)", block.NumberU64(), client.syncSummary.BlockNumber) +// 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 *stateSyncerClient) finishSync(blockHash common.Hash) error { + evmBlock := client.getEVMBlockFromHash(blockHash) + if evmBlock == nil { + return fmt.Errorf("Could not get evmBlock form hash %s", blockHash) } + block := evmBlock.ethBlock + // 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), @@ -521,10 +496,10 @@ func (client *stateSyncerClient) upstreamSyncStateTrie(ctx context.Context) erro p2pClient := client.network.NewClient(ethstatesync.ProtocolID) if len(client.stateSyncNodes) > 0 { for _, nodeID := range client.stateSyncNodes { - client.dl.SnapSyncer.Register(ethstatesync.NewOutboundPeer(nodeID, client.dl.SnapSyncer, p2pClient)) + client.dl.snapSyncer.Register(ethstatesync.NewOutboundPeer(nodeID, client.dl.snapSyncer, p2pClient)) } } else { - client.network.AddConnector(ethstatesync.NewConnector(client.dl.SnapSyncer, p2pClient)) + client.network.AddConnector(ethstatesync.NewConnector(client.dl.snapSyncer, p2pClient)) } if err := client.dl.SnapSync(); err != nil { @@ -534,20 +509,17 @@ func (client *stateSyncerClient) upstreamSyncStateTrie(ctx context.Context) erro return nil } -func (e queueElement) ExitQueue() error { - return e.exec(e.block) -} - type queueElement struct { block *Block - exec func(*Block) error + req syncBlockRequest } -var _ ethstatesync.Executable = &queueElement{} - type downloader struct { - pivotBlock *Block - SnapSyncer *snap.Syncer + pivotBlock *Block + snapSyncer *snap.Syncer + blockBuffer []*queueElement + bufferLen int + bufferLock sync.Mutex stateSyncStart chan *stateSync newPivot chan *Block @@ -558,7 +530,8 @@ type downloader struct { func newDownloader(chaindb ethdb.Database, firstPivot *Block) *downloader { d := &downloader{ pivotBlock: firstPivot, - SnapSyncer: snap.NewSyncer(chaindb, rawdb.HashScheme), + snapSyncer: snap.NewSyncer(chaindb, rawdb.HashScheme), + blockBuffer: make([]*queueElement, bufferSize), stateSyncStart: make(chan *stateSync), quitCh: make(chan struct{}), newPivot: make(chan *Block), @@ -584,6 +557,56 @@ func (d *downloader) stateFetcher() { } } +// QueueBlock queues a block for processing by the state syncer. +// This assumes the queue lock is NOT held +func (d *downloader) QueueBlockOrPivot(b *Block, req syncBlockRequest) error { + d.bufferLock.Lock() + defer d.bufferLock.Unlock() + if d.bufferLen >= len(d.blockBuffer) { + return errors.New("Snap sync queue overflow") + } + + d.blockBuffer[d.bufferLen] = &queueElement{b, req} + d.bufferLen++ + + if b.Height() >= d.pivotBlock.Height()+pivotInterval { + log.Info("Setting new pivot block", "hash", b.ID(), "height", b.Height(), "timestamp", b.Timestamp()) + + // Reset pivot first in other goroutine + d.pivotBlock = b + d.newPivot <- b + + // Clear queue + d.flushQueue(false) + } else if b.Height() <= d.pivotBlock.Height() { + log.Warn("Received block with height less than pivot block", "hash", b.ID(), "height", b.Height(), "timestamp", b.Timestamp()) + return errors.New("received block with height less than pivot block") + } + return nil +} + +// Clears queue of blocks. Assumes no elements are past pivot and bufferLock is held +// If `final`, executes blocks as normal. Otherwise executes only atomic operations +func (d *downloader) flushQueue(final bool) error { + for i, elem := range d.blockBuffer { + if i >= d.bufferLen { + return nil + } + + if err := elem.block.ExecuteSyncRequest(elem.req, final); err != nil { + if final { + // EXTREMELY hacky solution to release lock in final case on error - should fix later + d.bufferLock.Unlock() + } + return err + } + } + + d.bufferLen = 0 + + return nil +} + // processSnapSyncContent takes fetch results from the queue and writes them to the // database. It also controls the synchronisation of state nodes of the pivot block. func (d *downloader) SnapSync() error { @@ -603,7 +626,11 @@ func (d *downloader) SnapSync() error { // If stateSync is ended, clear queue and return // If err, just return so we can see it case <-sync.done: - return sync.err + if sync.err != nil { + return sync.err + } + d.bufferLock.Lock() // unlocked by syncer client once we can normally process blocks + return d.flushQueue(true) case newPivot := <-d.newPivot: // If a new pivot block is found, cancel the current state sync and // start a new one. @@ -679,7 +706,7 @@ func newStateSync(d *downloader, root common.Hash) *stateSync { // finish. func (s *stateSync) run() { close(s.started) - s.err = s.d.SnapSyncer.Sync(s.root, s.cancel) + s.err = s.d.snapSyncer.Sync(s.root, s.cancel) close(s.done) } From b6d29d1260a89f044cf0649162cbac2317731416 Mon Sep 17 00:00:00 2001 From: Austin Larson Date: Wed, 26 Feb 2025 16:08:53 -0500 Subject: [PATCH 10/21] Adjusted reindexing for buffer, adding additional error checking --- plugin/evm/syncervm_client.go | 25 ++++++++++++++++++------- 1 file changed, 18 insertions(+), 7 deletions(-) diff --git a/plugin/evm/syncervm_client.go b/plugin/evm/syncervm_client.go index 235c8a3c4a..5deff4149b 100644 --- a/plugin/evm/syncervm_client.go +++ b/plugin/evm/syncervm_client.go @@ -261,12 +261,20 @@ func (client *stateSyncerClient) acceptSyncSummary(proposedSummary message.SyncS client.cancel = cancel client.wg.Add(1) // track the state sync goroutine so we can wait for it on shutdown if client.useUpstream { - // I don't think all of this is necessary/correct, but I'm not sure which parts can be dropped - client.finishSync(client.syncSummary.BlockHash) - // Set downloader using pivot evmBlock := client.getEVMBlockFromHash(client.syncSummary.BlockHash) client.dl = newDownloader(client.chaindb, evmBlock) + ethBlock := evmBlock.ethBlock + if err := client.updateVMMarkers(); err != nil { + return block.StateSyncSkipped, fmt.Errorf("error updating vm markers height=%d, hash=%s, err=%w", ethBlock.NumberU64(), ethBlock.Hash(), err) + } + + if err := client.state.SetLastAcceptedBlock(evmBlock); err != nil { + return block.StateSyncSkipped, err + } + if err := client.atomicBackend.ApplyToSharedMemory(ethBlock.NumberU64()); err != nil { + return block.StateSyncSkipped, err + } log.Info("Set LastAcceptedBlock to first pivot", "height", evmBlock.ID(), evmBlock.Height(), "timestamp", evmBlock.Timestamp()) } @@ -569,7 +577,7 @@ func (d *downloader) QueueBlockOrPivot(b *Block, req syncBlockRequest) error { d.blockBuffer[d.bufferLen] = &queueElement{b, req} d.bufferLen++ - if b.Height() >= d.pivotBlock.Height()+pivotInterval { + if req == acceptSyncBlockRequest && b.Height() >= d.pivotBlock.Height()+pivotInterval { log.Info("Setting new pivot block", "hash", b.ID(), "height", b.Height(), "timestamp", b.Timestamp()) // Reset pivot first in other goroutine @@ -577,7 +585,10 @@ func (d *downloader) QueueBlockOrPivot(b *Block, req syncBlockRequest) error { d.newPivot <- b // Clear queue - d.flushQueue(false) + if err := d.flushQueue(false); err != nil { + return err + } + d.bufferLen = 0 } else if b.Height() <= d.pivotBlock.Height() { log.Warn("Received block with height less than pivot block", "hash", b.ID(), "height", b.Height(), "timestamp", b.Timestamp()) return errors.New("received block with height less than pivot block") @@ -587,6 +598,7 @@ func (d *downloader) QueueBlockOrPivot(b *Block, req syncBlockRequest) error { // Clears queue of blocks. Assumes no elements are past pivot and bufferLock is held // If `final`, executes blocks as normal. Otherwise executes only atomic operations +// To avoid duplicating actions, should adjust length at higher level func (d *downloader) flushQueue(final bool) error { for i, elem := range d.blockBuffer { if i >= d.bufferLen { @@ -601,8 +613,7 @@ func (d *downloader) flushQueue(final bool) error { return err } } - - d.bufferLen = 0 + d.bufferLen = 0 // this is not always sufficient to adjust the length? I think scoping issue return nil } From 344bec900ad5085e7faa403a9707a32fe1e7c9e3 Mon Sep 17 00:00:00 2001 From: Austin Larson Date: Wed, 26 Feb 2025 17:05:03 -0500 Subject: [PATCH 11/21] Fixed buffer len error, added close channel, refactored for readability --- plugin/evm/block.go | 209 +++++++++++++------------- plugin/evm/downloader.go | 241 ++++++++++++++++++++++++++++++ plugin/evm/syncervm_client.go | 270 +++------------------------------- 3 files changed, 368 insertions(+), 352 deletions(-) create mode 100644 plugin/evm/downloader.go diff --git a/plugin/evm/block.go b/plugin/evm/block.go index 0cd3157fa2..01fa506050 100644 --- a/plugin/evm/block.go +++ b/plugin/evm/block.go @@ -35,6 +35,15 @@ var ( var errMissingUTXOs = errors.New("missing UTXOs") +type syncBlockRequest uint8 + +const ( + // Constants to identify block requests + verifySyncBlockRequest syncBlockRequest = iota + 1 + acceptSyncBlockRequest + rejectSyncBlockRequest +) + // readMainnetBonusBlocks returns maps of bonus block numbers to block IDs. // Note bonus blocks are indexed in the atomic trie. func readMainnetBonusBlocks() (map[uint64]ids.ID, error) { @@ -146,17 +155,35 @@ func (b *Block) Accept(context.Context) error { return b.accept() } -func (b *Block) acceptDuringSync() error { +func (b *Block) accept() 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() - log.Info("Accepting block during sync", "block", b.ID(), "height", b.Height()) + + log.Debug(fmt.Sprintf("Accepting block %s (%s) at height %d", b.ID().Hex(), b.ID(), b.Height())) + + // Call Accept for relevant precompile logs. Note we do this prior to + // calling Accept on the blockChain so any side effects (eg warp signatures) + // take place before the accepted log is emitted to subscribers. + rules := b.vm.chainConfig.Rules(b.ethBlock.Number(), params.IsMergeTODO, b.ethBlock.Timestamp()) + if err := b.handlePrecompileAccept(*params.GetRulesExtra(rules)); err != nil { + return err + } + if err := vm.blockChain.Accept(b.ethBlock); err != nil { + return fmt.Errorf("chain could not accept %s: %w", b.ID(), err) + } + if err := vm.acceptedBlockDB.Put(lastAcceptedKey, b.id[:]); err != nil { return fmt.Errorf("failed to put %s as the last accepted block: %w", b.ID(), err) } + for _, tx := range b.atomicTxs { + // Remove the accepted transaction from the mempool + vm.mempool.RemoveTx(tx) + } + // Update VM state for atomic txs in this block. This includes updating the // atomic tx repo, atomic trie, and shared memory. atomicState, err := b.vm.atomicBackend.GetVerifiedAtomicState(common.Hash(b.ID())) @@ -173,44 +200,20 @@ func (b *Block) acceptDuringSync() error { // Apply any shared memory changes atomically with other pending changes to // the vm's versionDB. - if err := atomicState.Accept(vdbBatch, nil); err != nil { - return err - } - - log.Info("Returning from accept without error", "block", b.ID(), "height", b.Height()) - - return nil + return atomicState.Accept(vdbBatch, nil) } -func (b *Block) accept() error { +func (b *Block) acceptDuringSync() 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() - - log.Debug(fmt.Sprintf("Accepting block %s (%s) at height %d", b.ID().Hex(), b.ID(), b.Height())) - - // Call Accept for relevant precompile logs. Note we do this prior to - // calling Accept on the blockChain so any side effects (eg warp signatures) - // take place before the accepted log is emitted to subscribers. - rules := b.vm.chainConfig.Rules(b.ethBlock.Number(), params.IsMergeTODO, b.ethBlock.Timestamp()) - if err := b.handlePrecompileAccept(*params.GetRulesExtra(rules)); err != nil { - return err - } - if err := vm.blockChain.Accept(b.ethBlock); err != nil { - return fmt.Errorf("chain could not accept %s: %w", b.ID(), err) - } - + log.Info("Accepting block during sync", "block", b.ID(), "height", b.Height()) if err := vm.acceptedBlockDB.Put(lastAcceptedKey, b.id[:]); err != nil { return fmt.Errorf("failed to put %s as the last accepted block: %w", b.ID(), err) } - for _, tx := range b.atomicTxs { - // Remove the accepted transaction from the mempool - vm.mempool.RemoveTx(tx) - } - // Update VM state for atomic txs in this block. This includes updating the // atomic tx repo, atomic trie, and shared memory. atomicState, err := b.vm.atomicBackend.GetVerifiedAtomicState(common.Hash(b.ID())) @@ -227,7 +230,13 @@ func (b *Block) accept() error { // Apply any shared memory changes atomically with other pending changes to // the vm's versionDB. - return atomicState.Accept(vdbBatch, nil) + if err := atomicState.Accept(vdbBatch, nil); err != nil { + return err + } + + log.Info("Returning from accept without error", "block", b.ID(), "height", b.Height()) + + return nil } // handlePrecompileAccept calls Accept on any logs generated with an active precompile address that implements @@ -274,21 +283,6 @@ func (b *Block) Reject(ctx context.Context) error { return b.reject() } -func (b *Block) rejectDuringSync() error { - atomicState, err := b.vm.atomicBackend.GetVerifiedAtomicState(common.Hash(b.ID())) - if err != nil { - // should never occur since [b] must be verified before calling Reject - log.Error("Should never happen because block must be verified before calling Reject", "block", b.ID(), "height", b.Height()) - return err - } - if err := atomicState.Reject(); err != nil { - return err - } - - log.Info("Returning from reject without error", "block", b.ID(), "height", b.Height()) - return nil -} - func (b *Block) reject() error { log.Debug(fmt.Sprintf("Rejecting block %s (%s) at height %d", b.ID().Hex(), b.ID(), b.Height())) for _, tx := range b.atomicTxs { @@ -309,6 +303,21 @@ func (b *Block) reject() error { return b.vm.blockChain.Reject(b.ethBlock) } +func (b *Block) rejectDuringSync() error { + atomicState, err := b.vm.atomicBackend.GetVerifiedAtomicState(common.Hash(b.ID())) + if err != nil { + // should never occur since [b] must be verified before calling Reject + log.Error("Should never happen because block must be verified before calling Reject", "block", b.ID(), "height", b.Height()) + return err + } + if err := atomicState.Reject(); err != nil { + return err + } + + log.Info("Returning from reject without error", "block", b.ID(), "height", b.Height()) + return nil +} + // Parent implements the snowman.Block interface func (b *Block) Parent() ids.ID { return ids.ID(b.ethBlock.ParentHash()) @@ -324,17 +333,6 @@ func (b *Block) Timestamp() time.Time { return time.Unix(int64(b.ethBlock.Time()), 0) } -// syntacticVerify verifies that a *Block is well-formed. -func (b *Block) syntacticVerify() error { - if b == nil || b.ethBlock == nil { - return errInvalidBlock - } - - header := b.ethBlock.Header() - rules := b.vm.chainConfig.Rules(header.Number, params.IsMergeTODO, header.Time) - return b.vm.syntacticBlockValidator.SyntacticVerify(b, rules) -} - // Verify implements the snowman.Block interface func (b *Block) Verify(context.Context) error { log.Debug("Verifying block without context", "block", b.ID(), "height", b.Height()) @@ -364,50 +362,6 @@ func (b *Block) Verify(context.Context) error { return b.verify(true) } -func (b *Block) verifyDuringSync() error { - log.Debug("Verifying block during sync", "block", b.ID(), "height", b.Height()) - var ( - block = b.ethBlock - header = block.Header() - vm = b.vm - rules = vm.chainConfig.Rules(header.Number, params.IsMergeTODO, header.Time) - rulesExtra = *params.GetRulesExtra(rules) - ) - - txs, err := atomic.ExtractAtomicTxs(block.ExtData(), rulesExtra.IsApricotPhase5, atomic.Codec) - if err != nil { - return err - } - - if err := vm.blockChain.InsertBlockDuringSync(block); err != nil { - return err - } - - // If [atomicBackend] is nil, the VM is still initializing and is reprocessing accepted blocks. - if vm.atomicBackend != nil { - if vm.atomicBackend.IsBonus(block.NumberU64(), block.Hash()) { - log.Info("skipping atomic tx verification on bonus block", "block", block.Hash()) - } else { - // Verify [txs] do not conflict with themselves or ancestor blocks. - if err := vm.verifyTxs(txs, block.ParentHash(), block.BaseFee(), block.NumberU64(), rulesExtra); err != nil { - return err - } - } - // Update the atomic backend with [txs] from this block. - // - // Note: The atomic trie canonically contains the duplicate operations - // from any bonus blocks. - _, err := vm.atomicBackend.InsertTxs(block.Hash(), block.NumberU64(), block.ParentHash(), txs) - if err != nil { - return err - } - } - - log.Info("Returning from verify without error", "block", b.ID(), "height", b.Height()) - - return nil -} - // ShouldVerifyWithContext implements the block.WithVerifyContext interface func (b *Block) ShouldVerifyWithContext(context.Context) (bool, error) { rules := params.GetRulesExtra(b.vm.chainConfig.Rules(b.ethBlock.Number(), params.IsMergeTODO, b.ethBlock.Timestamp())) @@ -491,6 +445,61 @@ func (b *Block) verify(writes bool) error { return err } +func (b *Block) verifyDuringSync() error { + log.Debug("Verifying block during sync", "block", b.ID(), "height", b.Height()) + var ( + block = b.ethBlock + header = block.Header() + vm = b.vm + rules = vm.chainConfig.Rules(header.Number, params.IsMergeTODO, header.Time) + rulesExtra = *params.GetRulesExtra(rules) + ) + + txs, err := atomic.ExtractAtomicTxs(block.ExtData(), rulesExtra.IsApricotPhase5, atomic.Codec) + if err != nil { + return err + } + + if err := vm.blockChain.InsertBlockDuringSync(block); err != nil { + return err + } + + // If [atomicBackend] is nil, the VM is still initializing and is reprocessing accepted blocks. + if vm.atomicBackend != nil { + if vm.atomicBackend.IsBonus(block.NumberU64(), block.Hash()) { + log.Info("skipping atomic tx verification on bonus block", "block", block.Hash()) + } else { + // Verify [txs] do not conflict with themselves or ancestor blocks. + if err := vm.verifyTxs(txs, block.ParentHash(), block.BaseFee(), block.NumberU64(), rulesExtra); err != nil { + return err + } + } + // Update the atomic backend with [txs] from this block. + // + // Note: The atomic trie canonically contains the duplicate operations + // from any bonus blocks. + _, err := vm.atomicBackend.InsertTxs(block.Hash(), block.NumberU64(), block.ParentHash(), txs) + if err != nil { + return err + } + } + + log.Info("Returning from verify without error", "block", b.ID(), "height", b.Height()) + + return nil +} + +// syntacticVerify verifies that a *Block is well-formed. +func (b *Block) syntacticVerify() error { + if b == nil || b.ethBlock == nil { + return errInvalidBlock + } + + header := b.ethBlock.Header() + rules := b.vm.chainConfig.Rules(header.Number, params.IsMergeTODO, header.Time) + return b.vm.syntacticBlockValidator.SyntacticVerify(b, rules) +} + // verifyPredicates verifies the predicates in the block are valid according to predicateContext. func (b *Block) verifyPredicates(predicateContext *precompileconfig.PredicateContext) error { rules := b.vm.chainConfig.Rules(b.ethBlock.Number(), params.IsMergeTODO, b.ethBlock.Timestamp()) diff --git a/plugin/evm/downloader.go b/plugin/evm/downloader.go new file mode 100644 index 0000000000..b300de4594 --- /dev/null +++ b/plugin/evm/downloader.go @@ -0,0 +1,241 @@ +// (c) 2021-2025, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package evm + +import ( + "errors" + "sync" + + "github.com/ava-labs/coreth/core/rawdb" + "github.com/ava-labs/coreth/eth/protocols/snap" + "github.com/ava-labs/libevm/common" + "github.com/ava-labs/libevm/ethdb" + "github.com/ava-labs/libevm/log" +) + +const ( + // Dynamic state switches state root occasionally + // Buffer must be large enough to + pivotInterval = 128 + bufferSize = 3 * pivotInterval +) + +type queueElement struct { + block *Block + req syncBlockRequest +} + +type downloader struct { + pivotBlock *Block + snapSyncer *snap.Syncer + blockBuffer []*queueElement + bufferLen int + bufferLock sync.Mutex + + stateSyncStart chan *stateSync + newPivot chan *Block + quitCh chan struct{} // Quit channel to signal termination + // quitLock sync.Mutex // Lock to prevent double closes +} + +func newDownloader(chaindb ethdb.Database, firstPivot *Block) *downloader { + d := &downloader{ + pivotBlock: firstPivot, + snapSyncer: snap.NewSyncer(chaindb, rawdb.HashScheme), + blockBuffer: make([]*queueElement, bufferSize), + stateSyncStart: make(chan *stateSync), + quitCh: make(chan struct{}), + newPivot: make(chan *Block), + } + + go d.stateFetcher() + + return d +} + +// stateFetcher manages the active state sync and accepts requests +// on its behalf. +func (d *downloader) stateFetcher() { + for { + select { + case s := <-d.stateSyncStart: + for next := s; next != nil; { + next = d.runStateSync(next) + } + case <-d.quitCh: + return + } + } +} + +// QueueBlock queues a block for processing by the state syncer. +// This assumes the queue lock is NOT held +func (d *downloader) QueueBlockOrPivot(b *Block, req syncBlockRequest) error { + d.bufferLock.Lock() + defer d.bufferLock.Unlock() + if d.bufferLen >= len(d.blockBuffer) { + return errors.New("Snap sync queue overflow") + } + + d.blockBuffer[d.bufferLen] = &queueElement{b, req} + d.bufferLen++ + + if req == acceptSyncBlockRequest && b.Height() >= d.pivotBlock.Height()+pivotInterval { + log.Info("Setting new pivot block", "hash", b.ID(), "height", b.Height(), "timestamp", b.Timestamp()) + + // Reset pivot first in other goroutine + d.pivotBlock = b + d.newPivot <- b + + // Clear queue + if err := d.flushQueue(false); err != nil { + close(d.quitCh) + return err + } + } else if b.Height() <= d.pivotBlock.Height() { + close(d.quitCh) + log.Warn("Received block with height less than pivot block", "hash", b.ID(), "height", b.Height(), "timestamp", b.Timestamp()) + return errors.New("received block with height less than pivot block") + } + return nil +} + +// Clears queue of blocks. Assumes no elements are past pivot and bufferLock is held +// If `final`, executes blocks as normal. Otherwise executes only atomic operations +// To avoid duplicating actions, should adjust length at higher level +func (d *downloader) flushQueue(final bool) error { + defer func() { d.bufferLen = 0 }() + for i, elem := range d.blockBuffer { + if i >= d.bufferLen { + return nil + } + + if err := elem.block.ExecuteSyncRequest(elem.req, final); err != nil { + if final { + // EXTREMELY hacky solution to release lock in final case on error - should fix later + d.bufferLock.Unlock() + } + return err + } + } + + return nil +} + +// processSnapSyncContent takes fetch results from the queue and writes them to the +// database. It also controls the synchronisation of state nodes of the pivot block. +func (d *downloader) SnapSync() error { + // Start syncing state of the reported head block. This should get us most of + // the state of the pivot block. + sync := d.syncState(d.pivotBlock.ethBlock.Root()) + + defer func() { + // The `sync` object is replaced every time the pivot moves. We need to + // defer close the very last active one, hence the lazy evaluation vs. + // calling defer sync.Cancel() !!! + sync.Cancel() + }() + + for { + select { + // If stateSync is ended, clear queue and return + // If err, just return so we can see it + case <-sync.done: + if sync.err != nil { + return sync.err + } + d.bufferLock.Lock() // unlocked by syncer client once we can normally process blocks + return d.flushQueue(true) + case newPivot := <-d.newPivot: + // If a new pivot block is found, cancel the current state sync and + // start a new one. + sync.Cancel() + sync = d.syncState(newPivot.ethBlock.Root()) + } + } +} + +// syncState starts downloading state with the given root hash. +func (d *downloader) syncState(root common.Hash) *stateSync { + // Create the state sync + s := newStateSync(d, root) + select { + case d.stateSyncStart <- s: + // If we tell the statesync to restart with a new root, we also need + // to wait for it to actually also start -- when old requests have timed + // out or been delivered + <-s.started + case <-d.quitCh: + s.err = errors.New("errCancelStateFetch") //errCancelStateFetch from geth + close(s.done) + } + return s +} + +// runStateSync runs a state synchronisation until it completes or another root +// hash is requested to be switched over to. +func (d *downloader) runStateSync(s *stateSync) *stateSync { + log.Debug("State sync starting", "root", s.root) + + go s.run() + defer s.Cancel() + + for { + select { + case next := <-d.stateSyncStart: + return next + + case <-s.done: + return nil + } + } +} + +// stateSync schedules requests for downloading a particular state trie defined +// by a given state root. +type stateSync struct { + d *downloader // Downloader instance to access and manage current peerset + root common.Hash // State root currently being synced + + started chan struct{} // Started is signalled once the sync loop starts + cancel chan struct{} // Channel to signal a termination request + cancelOnce sync.Once // Ensures cancel only ever gets called once + done chan struct{} // Channel to signal termination completion + err error // Any error hit during sync (set before completion) +} + +// newStateSync creates a new state trie download scheduler. This method does not +// yet start the sync. The user needs to call run to initiate. +func newStateSync(d *downloader, root common.Hash) *stateSync { + return &stateSync{ + d: d, + root: root, + cancel: make(chan struct{}), + done: make(chan struct{}), + started: make(chan struct{}), + } +} + +// run starts the task assignment and response processing loop, blocking until +// it finishes, and finally notifying any goroutines waiting for the loop to +// finish. +func (s *stateSync) run() { + close(s.started) + s.err = s.d.snapSyncer.Sync(s.root, s.cancel) + close(s.done) +} + +// Wait blocks until the sync is done or canceled. +func (s *stateSync) Wait() error { + <-s.done + return s.err +} + +// Cancel cancels the sync and waits until it has shut down. +func (s *stateSync) Cancel() error { + s.cancelOnce.Do(func() { + close(s.cancel) + }) + return s.Wait() +} diff --git a/plugin/evm/syncervm_client.go b/plugin/evm/syncervm_client.go index 5deff4149b..d64092b06c 100644 --- a/plugin/evm/syncervm_client.go +++ b/plugin/evm/syncervm_client.go @@ -5,7 +5,6 @@ package evm import ( "context" - "errors" "fmt" "sync" @@ -19,7 +18,6 @@ import ( "github.com/ava-labs/coreth/core/rawdb" "github.com/ava-labs/coreth/core/state/snapshot" "github.com/ava-labs/coreth/eth" - "github.com/ava-labs/coreth/eth/protocols/snap" "github.com/ava-labs/coreth/params" "github.com/ava-labs/coreth/peer" "github.com/ava-labs/coreth/plugin/evm/message" @@ -35,20 +33,6 @@ const ( // State sync fetches [parentsToGet] parents of the block it syncs to. // The last 256 block hashes are necessary to support the BLOCKHASH opcode. parentsToGet = 256 - - // Dynamic state switches state root occasionally - // Buffer must be large enough to - pivotInterval = 128 - bufferSize = 3 * pivotInterval -) - -type syncBlockRequest uint8 - -const ( - // Constants to identify block requests - verifySyncBlockRequest syncBlockRequest = iota + 1 - acceptSyncBlockRequest - rejectSyncBlockRequest ) var stateSyncSummaryKey = []byte("stateSyncSummary") @@ -401,6 +385,24 @@ func (client *stateSyncerClient) syncStateTrie(ctx context.Context) error { } } +// upstreamSyncStateTrie syncs the state trie using the upstream state syncer +func (client *stateSyncerClient) upstreamSyncStateTrie(ctx context.Context) error { + p2pClient := client.network.NewClient(ethstatesync.ProtocolID) + if len(client.stateSyncNodes) > 0 { + for _, nodeID := range client.stateSyncNodes { + client.dl.snapSyncer.Register(ethstatesync.NewOutboundPeer(nodeID, client.dl.snapSyncer, p2pClient)) + } + } else { + client.network.AddConnector(ethstatesync.NewConnector(client.dl.snapSyncer, p2pClient)) + } + + if err := client.dl.SnapSync(); err != nil { + return err + } + log.Info("Upstream state syncer completed") + return nil +} + func (client *stateSyncerClient) Shutdown() error { if client.cancel != nil { client.cancel() @@ -498,239 +500,3 @@ func (client *stateSyncerClient) updateVMMarkers() error { // Error returns a non-nil error if one occurred during the sync. func (client *stateSyncerClient) Error() error { return client.stateSyncErr } - -// upstreamSyncStateTrie syncs the state trie using the upstream state syncer -func (client *stateSyncerClient) upstreamSyncStateTrie(ctx context.Context) error { - p2pClient := client.network.NewClient(ethstatesync.ProtocolID) - if len(client.stateSyncNodes) > 0 { - for _, nodeID := range client.stateSyncNodes { - client.dl.snapSyncer.Register(ethstatesync.NewOutboundPeer(nodeID, client.dl.snapSyncer, p2pClient)) - } - } else { - client.network.AddConnector(ethstatesync.NewConnector(client.dl.snapSyncer, p2pClient)) - } - - if err := client.dl.SnapSync(); err != nil { - return err - } - log.Info("Upstream state syncer completed") - return nil -} - -type queueElement struct { - block *Block - req syncBlockRequest -} - -type downloader struct { - pivotBlock *Block - snapSyncer *snap.Syncer - blockBuffer []*queueElement - bufferLen int - bufferLock sync.Mutex - - stateSyncStart chan *stateSync - newPivot chan *Block - quitCh chan struct{} // Quit channel to signal termination - // quitLock sync.Mutex // Lock to prevent double closes -} - -func newDownloader(chaindb ethdb.Database, firstPivot *Block) *downloader { - d := &downloader{ - pivotBlock: firstPivot, - snapSyncer: snap.NewSyncer(chaindb, rawdb.HashScheme), - blockBuffer: make([]*queueElement, bufferSize), - stateSyncStart: make(chan *stateSync), - quitCh: make(chan struct{}), - newPivot: make(chan *Block), - } - - go d.stateFetcher() - - return d -} - -// stateFetcher manages the active state sync and accepts requests -// on its behalf. -func (d *downloader) stateFetcher() { - for { - select { - case s := <-d.stateSyncStart: - for next := s; next != nil; { - next = d.runStateSync(next) - } - case <-d.quitCh: - return - } - } -} - -// QueueBlock queues a block for processing by the state syncer. -// This assumes the queue lock is NOT held -func (d *downloader) QueueBlockOrPivot(b *Block, req syncBlockRequest) error { - d.bufferLock.Lock() - defer d.bufferLock.Unlock() - if d.bufferLen >= len(d.blockBuffer) { - return errors.New("Snap sync queue overflow") - } - - d.blockBuffer[d.bufferLen] = &queueElement{b, req} - d.bufferLen++ - - if req == acceptSyncBlockRequest && b.Height() >= d.pivotBlock.Height()+pivotInterval { - log.Info("Setting new pivot block", "hash", b.ID(), "height", b.Height(), "timestamp", b.Timestamp()) - - // Reset pivot first in other goroutine - d.pivotBlock = b - d.newPivot <- b - - // Clear queue - if err := d.flushQueue(false); err != nil { - return err - } - d.bufferLen = 0 - } else if b.Height() <= d.pivotBlock.Height() { - log.Warn("Received block with height less than pivot block", "hash", b.ID(), "height", b.Height(), "timestamp", b.Timestamp()) - return errors.New("received block with height less than pivot block") - } - return nil -} - -// Clears queue of blocks. Assumes no elements are past pivot and bufferLock is held -// If `final`, executes blocks as normal. Otherwise executes only atomic operations -// To avoid duplicating actions, should adjust length at higher level -func (d *downloader) flushQueue(final bool) error { - for i, elem := range d.blockBuffer { - if i >= d.bufferLen { - return nil - } - - if err := elem.block.ExecuteSyncRequest(elem.req, final); err != nil { - if final { - // EXTREMELY hacky solution to release lock in final case on error - should fix later - d.bufferLock.Unlock() - } - return err - } - } - d.bufferLen = 0 // this is not always sufficient to adjust the length? I think scoping issue - - return nil -} - -// processSnapSyncContent takes fetch results from the queue and writes them to the -// database. It also controls the synchronisation of state nodes of the pivot block. -func (d *downloader) SnapSync() error { - // Start syncing state of the reported head block. This should get us most of - // the state of the pivot block. - sync := d.syncState(d.pivotBlock.ethBlock.Root()) - - defer func() { - // The `sync` object is replaced every time the pivot moves. We need to - // defer close the very last active one, hence the lazy evaluation vs. - // calling defer sync.Cancel() !!! - sync.Cancel() - }() - - for { - select { - // If stateSync is ended, clear queue and return - // If err, just return so we can see it - case <-sync.done: - if sync.err != nil { - return sync.err - } - d.bufferLock.Lock() // unlocked by syncer client once we can normally process blocks - return d.flushQueue(true) - case newPivot := <-d.newPivot: - // If a new pivot block is found, cancel the current state sync and - // start a new one. - sync.Cancel() - sync = d.syncState(newPivot.ethBlock.Root()) - } - } -} - -// syncState starts downloading state with the given root hash. -func (d *downloader) syncState(root common.Hash) *stateSync { - // Create the state sync - s := newStateSync(d, root) - select { - case d.stateSyncStart <- s: - // If we tell the statesync to restart with a new root, we also need - // to wait for it to actually also start -- when old requests have timed - // out or been delivered - <-s.started - case <-d.quitCh: - s.err = errors.New("errCancelStateFetch") //errCancelStateFetch from geth - close(s.done) - } - return s -} - -// runStateSync runs a state synchronisation until it completes or another root -// hash is requested to be switched over to. -func (d *downloader) runStateSync(s *stateSync) *stateSync { - log.Debug("State sync starting", "root", s.root) - - go s.run() - defer s.Cancel() - - for { - select { - case next := <-d.stateSyncStart: - return next - - case <-s.done: - return nil - } - } -} - -// stateSync schedules requests for downloading a particular state trie defined -// by a given state root. -type stateSync struct { - d *downloader // Downloader instance to access and manage current peerset - root common.Hash // State root currently being synced - - started chan struct{} // Started is signalled once the sync loop starts - cancel chan struct{} // Channel to signal a termination request - cancelOnce sync.Once // Ensures cancel only ever gets called once - done chan struct{} // Channel to signal termination completion - err error // Any error hit during sync (set before completion) -} - -// newStateSync creates a new state trie download scheduler. This method does not -// yet start the sync. The user needs to call run to initiate. -func newStateSync(d *downloader, root common.Hash) *stateSync { - return &stateSync{ - d: d, - root: root, - cancel: make(chan struct{}), - done: make(chan struct{}), - started: make(chan struct{}), - } -} - -// run starts the task assignment and response processing loop, blocking until -// it finishes, and finally notifying any goroutines waiting for the loop to -// finish. -func (s *stateSync) run() { - close(s.started) - s.err = s.d.snapSyncer.Sync(s.root, s.cancel) - close(s.done) -} - -// Wait blocks until the sync is done or canceled. -func (s *stateSync) Wait() error { - <-s.done - return s.err -} - -// Cancel cancels the sync and waits until it has shut down. -func (s *stateSync) Cancel() error { - s.cancelOnce.Do(func() { - close(s.cancel) - }) - return s.Wait() -} From a5c1b0ce92a8c279e090fe42ac14637627e30c8a Mon Sep 17 00:00:00 2001 From: Austin Larson Date: Thu, 27 Feb 2025 11:37:08 -0500 Subject: [PATCH 12/21] Adjusted logging to be more informative --- eth/protocols/snap/sync.go | 2 +- plugin/evm/block.go | 7 +++---- plugin/evm/downloader.go | 2 ++ 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/eth/protocols/snap/sync.go b/eth/protocols/snap/sync.go index 3148641f16..755ad64352 100644 --- a/eth/protocols/snap/sync.go +++ b/eth/protocols/snap/sync.go @@ -3070,7 +3070,7 @@ func (s *Syncer) reportSyncProgress(force bool) { storage = fmt.Sprintf("%v@%v", log.FormatLogfmtUint64(s.storageSynced), s.storageBytes.TerminalString()) bytecode = fmt.Sprintf("%v@%v", log.FormatLogfmtUint64(s.bytecodeSynced), s.bytecodeBytes.TerminalString()) ) - log.Info("Syncing: state download in progress", "synced", progress, "state", synced, + log.Info("Syncing: state download in progress", "synced", progress, "root", s.root, "state", synced, "accounts", accounts, "slots", storage, "codes", bytecode, "eta", common.PrettyDuration(estTime-elapsed)) } diff --git a/plugin/evm/block.go b/plugin/evm/block.go index 01fa506050..335daca007 100644 --- a/plugin/evm/block.go +++ b/plugin/evm/block.go @@ -209,7 +209,6 @@ func (b *Block) acceptDuringSync() error { // 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() - log.Info("Accepting block during sync", "block", b.ID(), "height", b.Height()) if err := vm.acceptedBlockDB.Put(lastAcceptedKey, b.id[:]); err != nil { return fmt.Errorf("failed to put %s as the last accepted block: %w", b.ID(), err) } @@ -234,7 +233,7 @@ func (b *Block) acceptDuringSync() error { return err } - log.Info("Returning from accept without error", "block", b.ID(), "height", b.Height()) + log.Debug("Returning from accept without error", "block", b.ID(), "height", b.Height()) return nil } @@ -314,7 +313,7 @@ func (b *Block) rejectDuringSync() error { return err } - log.Info("Returning from reject without error", "block", b.ID(), "height", b.Height()) + log.Debug("Returning from reject without error", "block", b.ID(), "height", b.Height()) return nil } @@ -484,7 +483,7 @@ func (b *Block) verifyDuringSync() error { } } - log.Info("Returning from verify without error", "block", b.ID(), "height", b.Height()) + log.Debug("Returning from verify without error", "block", b.ID(), "height", b.Height()) return nil } diff --git a/plugin/evm/downloader.go b/plugin/evm/downloader.go index b300de4594..947f0b0cf3 100644 --- a/plugin/evm/downloader.go +++ b/plugin/evm/downloader.go @@ -81,6 +81,8 @@ func (d *downloader) QueueBlockOrPivot(b *Block, req syncBlockRequest) error { d.blockBuffer[d.bufferLen] = &queueElement{b, req} d.bufferLen++ + log.Debug("Received queue request", "hash", b.ID(), "height", b.Height(), "req", req, "timestamp", b.Timestamp()) + if req == acceptSyncBlockRequest && b.Height() >= d.pivotBlock.Height()+pivotInterval { log.Info("Setting new pivot block", "hash", b.ID(), "height", b.Height(), "timestamp", b.Timestamp()) From e2e5e6ecd25ec110a81ec5b090efa314011d4de6 Mon Sep 17 00:00:00 2001 From: Austin Larson Date: Thu, 27 Feb 2025 13:36:27 -0500 Subject: [PATCH 13/21] Added additional logging, changed to pivot on multiple of interval --- plugin/evm/downloader.go | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/plugin/evm/downloader.go b/plugin/evm/downloader.go index 947f0b0cf3..ce460ceb1b 100644 --- a/plugin/evm/downloader.go +++ b/plugin/evm/downloader.go @@ -75,16 +75,22 @@ func (d *downloader) QueueBlockOrPivot(b *Block, req syncBlockRequest) error { d.bufferLock.Lock() defer d.bufferLock.Unlock() if d.bufferLen >= len(d.blockBuffer) { + close(d.quitCh) return errors.New("Snap sync queue overflow") } d.blockBuffer[d.bufferLen] = &queueElement{b, req} d.bufferLen++ - log.Debug("Received queue request", "hash", b.ID(), "height", b.Height(), "req", req, "timestamp", b.Timestamp()) + // Should change to debug prior to production + log.Info("Received queue request", "hash", b.ID(), "height", b.Height(), "req", req, "timestamp", b.Timestamp()) - if req == acceptSyncBlockRequest && b.Height() >= d.pivotBlock.Height()+pivotInterval { + // If on pivot interval, we should pivot (regardless of whether the queue is full) + if req == acceptSyncBlockRequest && b.Height()%pivotInterval == 0 { log.Info("Setting new pivot block", "hash", b.ID(), "height", b.Height(), "timestamp", b.Timestamp()) + if b.Height() <= d.pivotBlock.Height() { + log.Warn("Received pivot with height <= pivot block", "old hash", b.ID(), "old height", b.Height(), "timestamp", b.Timestamp()) + } // Reset pivot first in other goroutine d.pivotBlock = b @@ -95,11 +101,8 @@ func (d *downloader) QueueBlockOrPivot(b *Block, req syncBlockRequest) error { close(d.quitCh) return err } - } else if b.Height() <= d.pivotBlock.Height() { - close(d.quitCh) - log.Warn("Received block with height less than pivot block", "hash", b.ID(), "height", b.Height(), "timestamp", b.Timestamp()) - return errors.New("received block with height less than pivot block") } + return nil } From b1f59bdf71d91641736906dbe38bf799e3fd73b5 Mon Sep 17 00:00:00 2001 From: Austin Larson Date: Thu, 27 Feb 2025 15:53:52 -0500 Subject: [PATCH 14/21] Big refactor of dynamic sync stuff - likely has bugs --- plugin/evm/block.go | 64 +++++-------- plugin/evm/statesync/connector.go | 60 +++---------- plugin/evm/{ => statesync}/downloader.go | 110 ++++++++++++++++------- plugin/evm/syncervm_client.go | 47 ++++++---- 4 files changed, 143 insertions(+), 138 deletions(-) rename plugin/evm/{ => statesync}/downloader.go (64%) diff --git a/plugin/evm/block.go b/plugin/evm/block.go index 335daca007..8de8797c01 100644 --- a/plugin/evm/block.go +++ b/plugin/evm/block.go @@ -20,6 +20,7 @@ import ( "github.com/ava-labs/coreth/params" "github.com/ava-labs/coreth/params/extras" "github.com/ava-labs/coreth/plugin/evm/atomic" + "github.com/ava-labs/coreth/plugin/evm/statesync" "github.com/ava-labs/coreth/precompile/precompileconfig" "github.com/ava-labs/coreth/predicate" @@ -35,15 +36,6 @@ var ( var errMissingUTXOs = errors.New("missing UTXOs") -type syncBlockRequest uint8 - -const ( - // Constants to identify block requests - verifySyncBlockRequest syncBlockRequest = iota + 1 - acceptSyncBlockRequest - rejectSyncBlockRequest -) - // readMainnetBonusBlocks returns maps of bonus block numbers to block IDs. // Note bonus blocks are indexed in the atomic trie. func readMainnetBonusBlocks() (map[uint64]ids.ID, error) { @@ -150,7 +142,7 @@ func (b *Block) AtomicTxs() []*atomic.Tx { return b.atomicTxs } // Accept implements the snowman.Block interface func (b *Block) Accept(context.Context) error { if b.vm.StateSyncClient.AsyncReceive() { - return b.vm.StateSyncClient.QueueBlockOrPivot(b, acceptSyncBlockRequest) + return b.vm.StateSyncClient.QueueBlockOrPivot(b, statesync.AcceptSyncBlockRequest) } return b.accept() } @@ -203,7 +195,11 @@ func (b *Block) accept() error { return atomicState.Accept(vdbBatch, nil) } -func (b *Block) acceptDuringSync() error { +func (b *Block) acceptDuringSync(final bool) error { + if final { + return b.accept() + } + vm := b.vm // Although returning an error from Accept is considered fatal, it is good @@ -277,7 +273,7 @@ func (b *Block) handlePrecompileAccept(rules extras.Rules) error { func (b *Block) Reject(ctx context.Context) error { if b.vm.StateSyncClient.AsyncReceive() { log.Warn("Called Reject for block during dynamic state sync", "block", b.ID(), "height", b.Height()) - return b.vm.StateSyncClient.QueueBlockOrPivot(b, rejectSyncBlockRequest) + return b.vm.StateSyncClient.QueueBlockOrPivot(b, statesync.RejectSyncBlockRequest) } return b.reject() } @@ -302,7 +298,11 @@ func (b *Block) reject() error { return b.vm.blockChain.Reject(b.ethBlock) } -func (b *Block) rejectDuringSync() error { +func (b *Block) rejectDuringSync(final bool) error { + if final { + return b.reject() + } + atomicState, err := b.vm.atomicBackend.GetVerifiedAtomicState(common.Hash(b.ID())) if err != nil { // should never occur since [b] must be verified before calling Reject @@ -355,7 +355,7 @@ func (b *Block) Verify(context.Context) error { // If currently dynamically syncing, we should simply postpone execution if b.vm.StateSyncClient.AsyncReceive() { - return b.vm.StateSyncClient.QueueBlockOrPivot(b, verifySyncBlockRequest) + return b.vm.StateSyncClient.QueueBlockOrPivot(b, statesync.VerifySyncBlockRequest) } return b.verify(true) @@ -408,7 +408,7 @@ func (b *Block) VerifyWithContext(ctx context.Context, proposerVMBlockCtx *block // If currently dynamically syncing, we should postpone execution if b.vm.StateSyncClient.AsyncReceive() { - return b.vm.StateSyncClient.QueueBlockOrPivot(b, verifySyncBlockRequest) + return b.vm.StateSyncClient.QueueBlockOrPivot(b, statesync.VerifySyncBlockRequest) } return b.verify(true) @@ -444,7 +444,11 @@ func (b *Block) verify(writes bool) error { return err } -func (b *Block) verifyDuringSync() error { +func (b *Block) verifyDuringSync(final bool) error { + if final { + return b.verify(true) + } + log.Debug("Verifying block during sync", "block", b.ID(), "height", b.Height()) var ( block = b.ethBlock @@ -562,34 +566,6 @@ func (b *Block) verifyUTXOsPresent() error { return nil } -func (b *Block) ExecuteSyncRequest(req syncBlockRequest, final bool) error { - if final { - switch req { - case verifySyncBlockRequest: - return b.verify(true) - case acceptSyncBlockRequest: - return b.accept() - case rejectSyncBlockRequest: - return b.reject() - default: - // Should never happen - return fmt.Errorf("Unable to perform block operation of %d", req) - } - } else { - switch req { - case verifySyncBlockRequest: - return b.verifyDuringSync() - case acceptSyncBlockRequest: - return b.acceptDuringSync() - case rejectSyncBlockRequest: - return b.rejectDuringSync() - default: - // Should never happen - return fmt.Errorf("Unable to perform block operation of %d", req) - } - } -} - // Bytes implements the snowman.Block interface func (b *Block) Bytes() []byte { res, err := rlp.EncodeToBytes(b.ethBlock) diff --git a/plugin/evm/statesync/connector.go b/plugin/evm/statesync/connector.go index 813f1f87ea..dcf1ca196c 100644 --- a/plugin/evm/statesync/connector.go +++ b/plugin/evm/statesync/connector.go @@ -32,33 +32,33 @@ var ( ) type Connector struct { - sync *snap.Syncer - sender *p2p.Client + downloader *Downloader + sender *p2p.Client } -func NewConnector(sync *snap.Syncer, sender *p2p.Client) *Connector { - return &Connector{sync: sync, sender: sender} +func NewConnector(downloader *Downloader, sender *p2p.Client) *Connector { + return &Connector{downloader: downloader, sender: sender} } func (c *Connector) Connected(ctx context.Context, nodeID ids.NodeID, version *version.Application) error { - return c.sync.Register(NewOutboundPeer(nodeID, c.sync, c.sender)) + return c.downloader.SnapSyncer.Register(NewOutboundPeer(nodeID, c.downloader, c.sender)) } func (c *Connector) Disconnected(ctx context.Context, nodeID ids.NodeID) error { - return c.sync.Unregister(nodeID.String()) + return c.downloader.SnapSyncer.Unregister(nodeID.String()) } type outbound struct { - peerID ids.NodeID - sync *snap.Syncer - sender *p2p.Client + peerID ids.NodeID + downloader *Downloader + sender *p2p.Client } -func NewOutboundPeer(nodeID ids.NodeID, sync *snap.Syncer, sender *p2p.Client) *snap.Peer { +func NewOutboundPeer(nodeID ids.NodeID, downloader *Downloader, sender *p2p.Client) *snap.Peer { return snap.NewFakePeer(protocolVersion, nodeID.String(), &outbound{ - peerID: nodeID, - sync: sync, - sender: sender, + peerID: nodeID, + downloader: downloader, + sender: sender, }) } @@ -136,37 +136,5 @@ func (o *outbound) RunPeer(*snap.Peer, snap.Handler) error { panic("not expected func (o *outbound) PeerInfo(id enode.ID) interface{} { panic("not expected to be called") } func (o *outbound) Handle(peer *snap.Peer, packet snap.Packet) error { - d := &Downloader{SnapSyncer: o.sync} - return d.DeliverSnapPacket(peer, packet) -} - -// Downloader is copied from eth/downloader/downloader.go -type Downloader struct { - SnapSyncer *snap.Syncer -} - -// DeliverSnapPacket is invoked from a peer's message handler when it transmits a -// data packet for the local node to consume. -func (d *Downloader) DeliverSnapPacket(peer *snap.Peer, packet snap.Packet) error { - switch packet := packet.(type) { - case *snap.AccountRangePacket: - hashes, accounts, err := packet.Unpack() - if err != nil { - return err - } - return d.SnapSyncer.OnAccounts(peer, packet.ID, hashes, accounts, packet.Proof) - - case *snap.StorageRangesPacket: - hashset, slotset := packet.Unpack() - return d.SnapSyncer.OnStorage(peer, packet.ID, hashset, slotset, packet.Proof) - - case *snap.ByteCodesPacket: - return d.SnapSyncer.OnByteCodes(peer, packet.ID, packet.Codes) - - case *snap.TrieNodesPacket: - return d.SnapSyncer.OnTrieNodes(peer, packet.ID, packet.Nodes) - - default: - return fmt.Errorf("unexpected snap packet type: %T", packet) - } + return o.downloader.DeliverSnapPacket(peer, packet) } diff --git a/plugin/evm/downloader.go b/plugin/evm/statesync/downloader.go similarity index 64% rename from plugin/evm/downloader.go rename to plugin/evm/statesync/downloader.go index ce460ceb1b..c4eaaebf75 100644 --- a/plugin/evm/downloader.go +++ b/plugin/evm/statesync/downloader.go @@ -1,19 +1,30 @@ // (c) 2021-2025, Ava Labs, Inc. All rights reserved. // See the file LICENSE for licensing terms. -package evm +package statesync import ( "errors" + "fmt" "sync" "github.com/ava-labs/coreth/core/rawdb" + "github.com/ava-labs/coreth/core/types" "github.com/ava-labs/coreth/eth/protocols/snap" "github.com/ava-labs/libevm/common" "github.com/ava-labs/libevm/ethdb" "github.com/ava-labs/libevm/log" ) +type SyncBlockRequest uint8 + +const ( + // Constants to identify block requests + VerifySyncBlockRequest SyncBlockRequest = iota + 1 + AcceptSyncBlockRequest + RejectSyncBlockRequest +) + const ( // Dynamic state switches state root occasionally // Buffer must be large enough to @@ -22,31 +33,33 @@ const ( ) type queueElement struct { - block *Block - req syncBlockRequest + block *types.Block + req SyncBlockRequest + resolver func(bool) error } -type downloader struct { - pivotBlock *Block - snapSyncer *snap.Syncer +type Downloader struct { + pivotBlock *types.Block + SnapSyncer *snap.Syncer blockBuffer []*queueElement bufferLen int - bufferLock sync.Mutex + bufferLock *sync.Mutex stateSyncStart chan *stateSync - newPivot chan *Block + newPivot chan *types.Block quitCh chan struct{} // Quit channel to signal termination // quitLock sync.Mutex // Lock to prevent double closes } -func newDownloader(chaindb ethdb.Database, firstPivot *Block) *downloader { - d := &downloader{ +func NewDownloader(chaindb ethdb.Database, firstPivot *types.Block, bufferLock *sync.Mutex) *Downloader { + d := &Downloader{ pivotBlock: firstPivot, - snapSyncer: snap.NewSyncer(chaindb, rawdb.HashScheme), + SnapSyncer: snap.NewSyncer(chaindb, rawdb.HashScheme), blockBuffer: make([]*queueElement, bufferSize), stateSyncStart: make(chan *stateSync), quitCh: make(chan struct{}), - newPivot: make(chan *Block), + newPivot: make(chan *types.Block), + bufferLock: bufferLock, } go d.stateFetcher() @@ -56,7 +69,7 @@ func newDownloader(chaindb ethdb.Database, firstPivot *Block) *downloader { // stateFetcher manages the active state sync and accepts requests // on its behalf. -func (d *downloader) stateFetcher() { +func (d *Downloader) stateFetcher() { for { select { case s := <-d.stateSyncStart: @@ -69,9 +82,20 @@ func (d *downloader) stateFetcher() { } } +// Returns the current pivot +func (d *Downloader) Pivot() *types.Block { + return d.pivotBlock +} + +// Opens bufferLock to allow block requests to go through +func (d *Downloader) Close() { + d.bufferLock.TryLock() + d.bufferLock.Unlock() +} + // QueueBlock queues a block for processing by the state syncer. // This assumes the queue lock is NOT held -func (d *downloader) QueueBlockOrPivot(b *Block, req syncBlockRequest) error { +func (d *Downloader) QueueBlockOrPivot(b *types.Block, req SyncBlockRequest, resolver func(bool) error) error { d.bufferLock.Lock() defer d.bufferLock.Unlock() if d.bufferLen >= len(d.blockBuffer) { @@ -79,17 +103,17 @@ func (d *downloader) QueueBlockOrPivot(b *Block, req syncBlockRequest) error { return errors.New("Snap sync queue overflow") } - d.blockBuffer[d.bufferLen] = &queueElement{b, req} + d.blockBuffer[d.bufferLen] = &queueElement{b, req, resolver} d.bufferLen++ // Should change to debug prior to production - log.Info("Received queue request", "hash", b.ID(), "height", b.Height(), "req", req, "timestamp", b.Timestamp()) + log.Info("Received queue request", "hash", b.Hash(), "height", b.Number(), "req", req, "timestamp", b.Timestamp()) // If on pivot interval, we should pivot (regardless of whether the queue is full) - if req == acceptSyncBlockRequest && b.Height()%pivotInterval == 0 { - log.Info("Setting new pivot block", "hash", b.ID(), "height", b.Height(), "timestamp", b.Timestamp()) - if b.Height() <= d.pivotBlock.Height() { - log.Warn("Received pivot with height <= pivot block", "old hash", b.ID(), "old height", b.Height(), "timestamp", b.Timestamp()) + if req == AcceptSyncBlockRequest && b.NumberU64()%pivotInterval == 0 { + log.Info("Setting new pivot block", "hash", b.Hash(), "height", b.NumberU64(), "timestamp", b.Timestamp()) + if b.NumberU64() <= d.pivotBlock.NumberU64() { + log.Warn("Received pivot with height <= pivot block", "old hash", b.Hash(), "old height", b.NumberU64(), "timestamp", b.Timestamp()) } // Reset pivot first in other goroutine @@ -109,14 +133,14 @@ func (d *downloader) QueueBlockOrPivot(b *Block, req syncBlockRequest) error { // Clears queue of blocks. Assumes no elements are past pivot and bufferLock is held // If `final`, executes blocks as normal. Otherwise executes only atomic operations // To avoid duplicating actions, should adjust length at higher level -func (d *downloader) flushQueue(final bool) error { +func (d *Downloader) flushQueue(final bool) error { defer func() { d.bufferLen = 0 }() for i, elem := range d.blockBuffer { if i >= d.bufferLen { return nil } - if err := elem.block.ExecuteSyncRequest(elem.req, final); err != nil { + if err := elem.resolver(final); err != nil { if final { // EXTREMELY hacky solution to release lock in final case on error - should fix later d.bufferLock.Unlock() @@ -130,10 +154,10 @@ func (d *downloader) flushQueue(final bool) error { // processSnapSyncContent takes fetch results from the queue and writes them to the // database. It also controls the synchronisation of state nodes of the pivot block. -func (d *downloader) SnapSync() error { +func (d *Downloader) SnapSync() error { // Start syncing state of the reported head block. This should get us most of // the state of the pivot block. - sync := d.syncState(d.pivotBlock.ethBlock.Root()) + sync := d.syncState(d.pivotBlock.Root()) defer func() { // The `sync` object is replaced every time the pivot moves. We need to @@ -156,13 +180,13 @@ func (d *downloader) SnapSync() error { // If a new pivot block is found, cancel the current state sync and // start a new one. sync.Cancel() - sync = d.syncState(newPivot.ethBlock.Root()) + sync = d.syncState(newPivot.Root()) } } } // syncState starts downloading state with the given root hash. -func (d *downloader) syncState(root common.Hash) *stateSync { +func (d *Downloader) syncState(root common.Hash) *stateSync { // Create the state sync s := newStateSync(d, root) select { @@ -180,7 +204,7 @@ func (d *downloader) syncState(root common.Hash) *stateSync { // runStateSync runs a state synchronisation until it completes or another root // hash is requested to be switched over to. -func (d *downloader) runStateSync(s *stateSync) *stateSync { +func (d *Downloader) runStateSync(s *stateSync) *stateSync { log.Debug("State sync starting", "root", s.root) go s.run() @@ -200,7 +224,7 @@ func (d *downloader) runStateSync(s *stateSync) *stateSync { // stateSync schedules requests for downloading a particular state trie defined // by a given state root. type stateSync struct { - d *downloader // Downloader instance to access and manage current peerset + d *Downloader // Downloader instance to access and manage current peerset root common.Hash // State root currently being synced started chan struct{} // Started is signalled once the sync loop starts @@ -212,7 +236,7 @@ type stateSync struct { // newStateSync creates a new state trie download scheduler. This method does not // yet start the sync. The user needs to call run to initiate. -func newStateSync(d *downloader, root common.Hash) *stateSync { +func newStateSync(d *Downloader, root common.Hash) *stateSync { return &stateSync{ d: d, root: root, @@ -227,7 +251,7 @@ func newStateSync(d *downloader, root common.Hash) *stateSync { // finish. func (s *stateSync) run() { close(s.started) - s.err = s.d.snapSyncer.Sync(s.root, s.cancel) + s.err = s.d.SnapSyncer.Sync(s.root, s.cancel) close(s.done) } @@ -244,3 +268,29 @@ func (s *stateSync) Cancel() error { }) return s.Wait() } + +// DeliverSnapPacket is invoked from a peer's message handler when it transmits a +// data packet for the local node to consume. +func (d *Downloader) DeliverSnapPacket(peer *snap.Peer, packet snap.Packet) error { + switch packet := packet.(type) { + case *snap.AccountRangePacket: + hashes, accounts, err := packet.Unpack() + if err != nil { + return err + } + return d.SnapSyncer.OnAccounts(peer, packet.ID, hashes, accounts, packet.Proof) + + case *snap.StorageRangesPacket: + hashset, slotset := packet.Unpack() + return d.SnapSyncer.OnStorage(peer, packet.ID, hashset, slotset, packet.Proof) + + case *snap.ByteCodesPacket: + return d.SnapSyncer.OnByteCodes(peer, packet.ID, packet.Codes) + + case *snap.TrieNodesPacket: + return d.SnapSyncer.OnTrieNodes(peer, packet.ID, packet.Nodes) + + default: + return fmt.Errorf("unexpected snap packet type: %T", packet) + } +} diff --git a/plugin/evm/syncervm_client.go b/plugin/evm/syncervm_client.go index d64092b06c..86c3a8ab91 100644 --- a/plugin/evm/syncervm_client.go +++ b/plugin/evm/syncervm_client.go @@ -81,9 +81,9 @@ type stateSyncerClient struct { stateSyncErr error // Dynamic sync - syncing utils.Atomic[bool] - dl *downloader - queueLock sync.Mutex // to prevent writing during atomic sync + syncing utils.Atomic[bool] + dl *ethstatesync.Downloader + downloaderLock sync.Mutex // to prevent writing during atomic sync } func NewStateSyncClient(config *stateSyncClientConfig) StateSyncClient { @@ -105,7 +105,7 @@ type StateSyncClient interface { // Methods to enable dynamic state sync AsyncReceive() bool - QueueBlockOrPivot(*Block, syncBlockRequest) error + QueueBlockOrPivot(*Block, ethstatesync.SyncBlockRequest) error } // Syncer represents a step in state sync, @@ -121,13 +121,27 @@ type Syncer interface { // Should return true if syncing and useUpstream is true, i.e. currently dynamicaling syncing func (client *stateSyncerClient) AsyncReceive() bool { // Block until atomic sync is completed for bootstrapping - client.queueLock.Lock() - client.queueLock.Unlock() + client.downloaderLock.Lock() + client.downloaderLock.Unlock() return client.useUpstream && client.syncing.Get() && client.dl != nil } -func (client *stateSyncerClient) QueueBlockOrPivot(b *Block, req syncBlockRequest) error { - return client.dl.QueueBlockOrPivot(b, req) +func (client *stateSyncerClient) QueueBlockOrPivot(b *Block, req ethstatesync.SyncBlockRequest) error { + return client.dl.QueueBlockOrPivot(b.ethBlock, req, getSyncBlockHandler(b, req)) +} + +func getSyncBlockHandler(b *Block, req ethstatesync.SyncBlockRequest) func(bool) error { + switch req { + case ethstatesync.AcceptSyncBlockRequest: + return func(final bool) error { return b.acceptDuringSync(final) } + case ethstatesync.RejectSyncBlockRequest: + return func(final bool) error { return b.rejectDuringSync(final) } + case ethstatesync.VerifySyncBlockRequest: + return func(final bool) error { return b.verifyDuringSync(final) } + default: + log.Error("Invalid statesync.SyncBlockRequest", "ID", req) + } + return func(final bool) error { return nil } } // StateSyncEnabled returns [client.enabled], which is set in the chain's config file. @@ -189,7 +203,7 @@ func (client *stateSyncerClient) stateSync(ctx context.Context) error { if err := client.syncAtomicTrie(ctx); err != nil { return err } - client.queueLock.Unlock() + client.downloaderLock.Unlock() return client.syncStateTrie(ctx) } @@ -238,7 +252,7 @@ func (client *stateSyncerClient) acceptSyncSummary(proposedSummary message.SyncS log.Info("Starting state sync", "summary", proposedSummary) // Lock the atomic trie to prevent pivots during the atomic sync from dynamic syncing - client.queueLock.Lock() + client.downloaderLock.Lock() // create a cancellable ctx for the state sync goroutine ctx, cancel := context.WithCancel(context.Background()) @@ -247,7 +261,7 @@ func (client *stateSyncerClient) acceptSyncSummary(proposedSummary message.SyncS if client.useUpstream { // Set downloader using pivot evmBlock := client.getEVMBlockFromHash(client.syncSummary.BlockHash) - client.dl = newDownloader(client.chaindb, evmBlock) + client.dl = ethstatesync.NewDownloader(client.chaindb, evmBlock.ethBlock, &client.downloaderLock) ethBlock := evmBlock.ethBlock if err := client.updateVMMarkers(); err != nil { return block.StateSyncSkipped, fmt.Errorf("error updating vm markers height=%d, hash=%s, err=%w", ethBlock.NumberU64(), ethBlock.Hash(), err) @@ -265,16 +279,13 @@ func (client *stateSyncerClient) acceptSyncSummary(proposedSummary message.SyncS go func() { defer client.wg.Done() defer cancel() + defer client.dl.Close() if err := client.stateSync(ctx); err != nil { client.stateSyncErr = err } else { if client.useUpstream { - client.stateSyncErr = client.finishSync(client.dl.pivotBlock.ethBlock.Hash()) - - // should be locked from final, opens for regular block operations - // if any error occurs during sync, mutex will not be locked - client.dl.bufferLock.Unlock() + client.stateSyncErr = client.finishSync(client.dl.Pivot().Hash()) } else { client.stateSyncErr = client.finishSync(client.syncSummary.BlockHash) } @@ -390,10 +401,10 @@ func (client *stateSyncerClient) upstreamSyncStateTrie(ctx context.Context) erro p2pClient := client.network.NewClient(ethstatesync.ProtocolID) if len(client.stateSyncNodes) > 0 { for _, nodeID := range client.stateSyncNodes { - client.dl.snapSyncer.Register(ethstatesync.NewOutboundPeer(nodeID, client.dl.snapSyncer, p2pClient)) + client.dl.SnapSyncer.Register(ethstatesync.NewOutboundPeer(nodeID, client.dl, p2pClient)) } } else { - client.network.AddConnector(ethstatesync.NewConnector(client.dl.snapSyncer, p2pClient)) + client.network.AddConnector(ethstatesync.NewConnector(client.dl, p2pClient)) } if err := client.dl.SnapSync(); err != nil { From 4f914b538b0dff05ba01cbc27aeca6ebc8af21a8 Mon Sep 17 00:00:00 2001 From: Austin Larson Date: Thu, 27 Feb 2025 16:14:02 -0500 Subject: [PATCH 15/21] Fixes error where dl may be nil if not initialized --- plugin/evm/syncervm_client.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/plugin/evm/syncervm_client.go b/plugin/evm/syncervm_client.go index 86c3a8ab91..a83bbb451b 100644 --- a/plugin/evm/syncervm_client.go +++ b/plugin/evm/syncervm_client.go @@ -279,7 +279,9 @@ func (client *stateSyncerClient) acceptSyncSummary(proposedSummary message.SyncS go func() { defer client.wg.Done() defer cancel() - defer client.dl.Close() + if client.useUpstream { + defer client.dl.Close() + } if err := client.stateSync(ctx); err != nil { client.stateSyncErr = err From 23cd38c4cca7723441bc7ca651917568090863ca Mon Sep 17 00:00:00 2001 From: Austin Larson Date: Thu, 27 Feb 2025 16:34:26 -0500 Subject: [PATCH 16/21] Reduce diff for unnecesary logs --- core/blockchain.go | 7 --- plugin/evm/block.go | 3 +- plugin/evm/statesync/downloader.go | 5 +- plugin/evm/statesync/queue.go | 93 ------------------------------ plugin/evm/syncervm_client.go | 2 +- 5 files changed, 5 insertions(+), 105 deletions(-) delete mode 100644 plugin/evm/statesync/queue.go diff --git a/core/blockchain.go b/core/blockchain.go index ad3d998565..a67a9c0518 100644 --- a/core/blockchain.go +++ b/core/blockchain.go @@ -1067,8 +1067,6 @@ func (bc *BlockChain) Reject(block *types.Block) error { bc.chainmu.Lock() defer bc.chainmu.Unlock() - log.Error("Rejecting block", "number", block.Number(), "hash", block.Hash()) - // Reject Trie if err := bc.stateManager.RejectTrie(block); err != nil { return fmt.Errorf("unable to reject trie: %w", err) @@ -1273,11 +1271,6 @@ func (bc *BlockChain) insertBlock(block *types.Block, writes bool) error { err := bc.engine.VerifyHeader(bc, block.Header()) if err == nil { err = bc.validator.ValidateBody(block) - if err != nil { - log.Error("Body validation failed", "err", err, "number", block.Number(), "hash", block.Hash()) - } - } else { - log.Error("Header verification failed", "err", err, "number", block.Number(), "hash", block.Hash()) } switch { diff --git a/plugin/evm/block.go b/plugin/evm/block.go index 8de8797c01..c9bd9f5b33 100644 --- a/plugin/evm/block.go +++ b/plugin/evm/block.go @@ -270,9 +270,8 @@ func (b *Block) handlePrecompileAccept(rules extras.Rules) error { // Reject implements the snowman.Block interface // If [b] contains an atomic transaction, attempt to re-issue it -func (b *Block) Reject(ctx context.Context) error { +func (b *Block) Reject(context.Context) error { if b.vm.StateSyncClient.AsyncReceive() { - log.Warn("Called Reject for block during dynamic state sync", "block", b.ID(), "height", b.Height()) return b.vm.StateSyncClient.QueueBlockOrPivot(b, statesync.RejectSyncBlockRequest) } return b.reject() diff --git a/plugin/evm/statesync/downloader.go b/plugin/evm/statesync/downloader.go index c4eaaebf75..9cc1112341 100644 --- a/plugin/evm/statesync/downloader.go +++ b/plugin/evm/statesync/downloader.go @@ -87,9 +87,10 @@ func (d *Downloader) Pivot() *types.Block { return d.pivotBlock } -// Opens bufferLock to allow block requests to go through +// Opens bufferLock to allow block requests to go through after finalizing the sync func (d *Downloader) Close() { d.bufferLock.TryLock() + d.flushQueue(true) d.bufferLock.Unlock() } @@ -175,7 +176,7 @@ func (d *Downloader) SnapSync() error { return sync.err } d.bufferLock.Lock() // unlocked by syncer client once we can normally process blocks - return d.flushQueue(true) + return nil case newPivot := <-d.newPivot: // If a new pivot block is found, cancel the current state sync and // start a new one. diff --git a/plugin/evm/statesync/queue.go b/plugin/evm/statesync/queue.go deleted file mode 100644 index b629fcdf89..0000000000 --- a/plugin/evm/statesync/queue.go +++ /dev/null @@ -1,93 +0,0 @@ -// (c) 2021-2024, Ava Labs, Inc. All rights reserved. -// See the file LICENSE for licensing terms. -// -// This file is although named queue.go, is not the same as the eth/downloader/queue.go file. -// Instead, this is simply the protected array for downloader (yeah not even a queue). -// It might be helpful to keep this in case we move computation here -// -// It is distributed under a license compatible with the licensing terms of the -// original code from which it is derived. - -package statesync - -import ( - "fmt" - "sync" -) - -type Executable interface { - ExitQueue() error -} - -type Queue[K Executable] struct { - buffer []*K - l sync.RWMutex - nextPos int - compare func(*K, *K) int - closed bool -} - -func NewQueue[K Executable](size int, compare func(*K, *K) int) *Queue[K] { - return &Queue[K]{ - buffer: make([]*K, size), - compare: compare, - } -} - -func (q *Queue[K]) Insert(h *K) error { - q.l.Lock() - defer q.l.Unlock() - - if q.nextPos >= len(q.buffer) { - return fmt.Errorf("queue is full, cannot insert") - } - - q.buffer[q.nextPos] = h - q.nextPos++ - - return nil -} - -func (q *Queue[K]) Flush(max *K, close bool) error { - q.l.Lock() - defer q.l.Unlock() - - newBuffer := make([]*K, len(q.buffer)) - newPos := 0 - - for i := 0; i < q.nextPos; i++ { - // If the item is greater than max, postpone - elem := q.buffer[i] - if max != nil && q.compare(elem, max) < 0 { - newBuffer[newPos] = q.buffer[i] - newPos++ - } else { - if err := (*elem).ExitQueue(); err != nil { - return fmt.Errorf("error executing item: %w", err) - } - } - } - - q.buffer = newBuffer - q.nextPos = newPos - - if close { - q.closed = true - } - - return nil -} - -func (q *Queue[K]) Len() int { - q.l.RLock() - defer q.l.RUnlock() - - return q.nextPos -} - -func (q *Queue[K]) Closed() bool { - q.l.RLock() - defer q.l.RUnlock() - - return q.closed -} diff --git a/plugin/evm/syncervm_client.go b/plugin/evm/syncervm_client.go index a83bbb451b..df2ebcd800 100644 --- a/plugin/evm/syncervm_client.go +++ b/plugin/evm/syncervm_client.go @@ -274,7 +274,7 @@ func (client *stateSyncerClient) acceptSyncSummary(proposedSummary message.SyncS return block.StateSyncSkipped, err } - log.Info("Set LastAcceptedBlock to first pivot", "height", evmBlock.ID(), evmBlock.Height(), "timestamp", evmBlock.Timestamp()) + log.Info("Set LastAcceptedBlock as first pivot", "height", evmBlock.ID(), evmBlock.Height(), "timestamp", evmBlock.Timestamp()) } go func() { defer client.wg.Done() From 2213833fe74ff954e2b3d26e30c34751ec92e2e1 Mon Sep 17 00:00:00 2001 From: Austin Larson Date: Fri, 28 Feb 2025 10:30:51 -0500 Subject: [PATCH 17/21] Fixed some logging --- plugin/evm/statesync/downloader.go | 10 ++++------ plugin/evm/syncervm_client.go | 3 ++- 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/plugin/evm/statesync/downloader.go b/plugin/evm/statesync/downloader.go index 9cc1112341..d695992551 100644 --- a/plugin/evm/statesync/downloader.go +++ b/plugin/evm/statesync/downloader.go @@ -89,7 +89,7 @@ func (d *Downloader) Pivot() *types.Block { // Opens bufferLock to allow block requests to go through after finalizing the sync func (d *Downloader) Close() { - d.bufferLock.TryLock() + d.bufferLock.TryLock() // this should be a no-op d.flushQueue(true) d.bufferLock.Unlock() } @@ -172,14 +172,12 @@ func (d *Downloader) SnapSync() error { // If stateSync is ended, clear queue and return // If err, just return so we can see it case <-sync.done: - if sync.err != nil { - return sync.err - } - d.bufferLock.Lock() // unlocked by syncer client once we can normally process blocks - return nil + d.bufferLock.Lock() // unlocked in Close() + return sync.err case newPivot := <-d.newPivot: // If a new pivot block is found, cancel the current state sync and // start a new one. + log.Debug("Pivot block updated to", "hash", d.pivotBlock.Root(), d.pivotBlock.NumberU64()) sync.Cancel() sync = d.syncState(newPivot.Root()) } diff --git a/plugin/evm/syncervm_client.go b/plugin/evm/syncervm_client.go index df2ebcd800..e4dc331e07 100644 --- a/plugin/evm/syncervm_client.go +++ b/plugin/evm/syncervm_client.go @@ -274,7 +274,7 @@ func (client *stateSyncerClient) acceptSyncSummary(proposedSummary message.SyncS return block.StateSyncSkipped, err } - log.Info("Set LastAcceptedBlock as first pivot", "height", evmBlock.ID(), evmBlock.Height(), "timestamp", evmBlock.Timestamp()) + log.Info("Set LastAcceptedBlock as first pivot", "id", evmBlock.ID(), "height", evmBlock.Height(), "timestamp", evmBlock.Timestamp()) } go func() { defer client.wg.Done() @@ -287,6 +287,7 @@ func (client *stateSyncerClient) acceptSyncSummary(proposedSummary message.SyncS client.stateSyncErr = err } else { if client.useUpstream { + // finish sync on final pivot, Close() will clear queue as if bootstrapping from static sync client.stateSyncErr = client.finishSync(client.dl.Pivot().Hash()) } else { client.stateSyncErr = client.finishSync(client.syncSummary.BlockHash) From ed8bef697f20429e96993b31899dc81e7bb33411 Mon Sep 17 00:00:00 2001 From: Austin Larson Date: Fri, 28 Feb 2025 15:10:19 -0500 Subject: [PATCH 18/21] Added a probably superfluous lock and logging for debugging --- plugin/evm/statesync/downloader.go | 9 ++++++++- plugin/evm/syncervm_client.go | 8 +++++++- 2 files changed, 15 insertions(+), 2 deletions(-) diff --git a/plugin/evm/statesync/downloader.go b/plugin/evm/statesync/downloader.go index d695992551..f782c184e7 100644 --- a/plugin/evm/statesync/downloader.go +++ b/plugin/evm/statesync/downloader.go @@ -40,6 +40,7 @@ type queueElement struct { type Downloader struct { pivotBlock *types.Block + pivotLock sync.RWMutex SnapSyncer *snap.Syncer blockBuffer []*queueElement bufferLen int @@ -84,6 +85,8 @@ func (d *Downloader) stateFetcher() { // Returns the current pivot func (d *Downloader) Pivot() *types.Block { + d.pivotLock.RLock() + defer d.pivotLock.RUnlock() return d.pivotBlock } @@ -118,11 +121,14 @@ func (d *Downloader) QueueBlockOrPivot(b *types.Block, req SyncBlockRequest, res } // Reset pivot first in other goroutine + d.pivotLock.Lock() d.pivotBlock = b + d.pivotLock.Unlock() d.newPivot <- b // Clear queue if err := d.flushQueue(false); err != nil { + log.Error("Issue flushing queue", "err", err) close(d.quitCh) return err } @@ -172,12 +178,13 @@ func (d *Downloader) SnapSync() error { // If stateSync is ended, clear queue and return // If err, just return so we can see it case <-sync.done: + log.Info("Sync completed with", "err", sync.err) d.bufferLock.Lock() // unlocked in Close() return sync.err case newPivot := <-d.newPivot: // If a new pivot block is found, cancel the current state sync and // start a new one. - log.Debug("Pivot block updated to", "hash", d.pivotBlock.Root(), d.pivotBlock.NumberU64()) + log.Debug("Pivot block updated to", "hash", d.pivotBlock.Root(), "height", d.pivotBlock.NumberU64()) sync.Cancel() sync = d.syncState(newPivot.Root()) } diff --git a/plugin/evm/syncervm_client.go b/plugin/evm/syncervm_client.go index e4dc331e07..d21086b755 100644 --- a/plugin/evm/syncervm_client.go +++ b/plugin/evm/syncervm_client.go @@ -280,16 +280,22 @@ func (client *stateSyncerClient) acceptSyncSummary(proposedSummary message.SyncS defer client.wg.Done() defer cancel() if client.useUpstream { - defer client.dl.Close() + defer func() { + log.Debug("Closing downloader in defer") + client.dl.Close() + }() } if err := client.stateSync(ctx); err != nil { + log.Error("Returned from stateSync with error") client.stateSyncErr = err } else { if client.useUpstream { + log.Info("No state sync error, final pivot", "hash", client.dl.Pivot().Hash(), "height", client.dl.Pivot().NumberU64()) // finish sync on final pivot, Close() will clear queue as if bootstrapping from static sync client.stateSyncErr = client.finishSync(client.dl.Pivot().Hash()) } else { + log.Error("Called incorrect finishSync?????", "useUpstream", client.useUpstream) client.stateSyncErr = client.finishSync(client.syncSummary.BlockHash) } } From 0dbbd8fc43f639a8d6a37fd869ec3ea4bcbbc1a2 Mon Sep 17 00:00:00 2001 From: Austin Larson Date: Fri, 28 Feb 2025 16:48:14 -0500 Subject: [PATCH 19/21] Changed atomic lock to use channel, adjusted logging --- plugin/evm/statesync/downloader.go | 2 +- plugin/evm/syncervm_client.go | 21 ++++++++++++++------- 2 files changed, 15 insertions(+), 8 deletions(-) diff --git a/plugin/evm/statesync/downloader.go b/plugin/evm/statesync/downloader.go index f782c184e7..f11ff9b6ed 100644 --- a/plugin/evm/statesync/downloader.go +++ b/plugin/evm/statesync/downloader.go @@ -184,7 +184,7 @@ func (d *Downloader) SnapSync() error { case newPivot := <-d.newPivot: // If a new pivot block is found, cancel the current state sync and // start a new one. - log.Debug("Pivot block updated to", "hash", d.pivotBlock.Root(), "height", d.pivotBlock.NumberU64()) + log.Debug("Pivot block updated to", "hash", d.Pivot().Root(), "height", d.Pivot().NumberU64()) sync.Cancel() sync = d.syncState(newPivot.Root()) } diff --git a/plugin/evm/syncervm_client.go b/plugin/evm/syncervm_client.go index d21086b755..b98488276a 100644 --- a/plugin/evm/syncervm_client.go +++ b/plugin/evm/syncervm_client.go @@ -83,12 +83,14 @@ type stateSyncerClient struct { // Dynamic sync syncing utils.Atomic[bool] dl *ethstatesync.Downloader - downloaderLock sync.Mutex // to prevent writing during atomic sync + downloaderLock sync.Mutex + atomicDone chan struct{} // to prevent writing during atomic sync } func NewStateSyncClient(config *stateSyncClientConfig) StateSyncClient { return &stateSyncerClient{ stateSyncClientConfig: config, + atomicDone: make(chan struct{}), } } @@ -120,16 +122,24 @@ type Syncer interface { // AsyncReceive returns true if the client is ready to receive a message from the engine // Should return true if syncing and useUpstream is true, i.e. currently dynamicaling syncing func (client *stateSyncerClient) AsyncReceive() bool { - // Block until atomic sync is completed for bootstrapping + // Block until atomic sync is completed for bootstrapping and after sync completes until blockchain updates client.downloaderLock.Lock() client.downloaderLock.Unlock() return client.useUpstream && client.syncing.Get() && client.dl != nil } func (client *stateSyncerClient) QueueBlockOrPivot(b *Block, req ethstatesync.SyncBlockRequest) error { - return client.dl.QueueBlockOrPivot(b.ethBlock, req, getSyncBlockHandler(b, req)) + // Wait for atomic sync to be done prior to queueing + <-client.atomicDone + err := client.dl.QueueBlockOrPivot(b.ethBlock, req, getSyncBlockHandler(b, req)) + if err != nil { + log.Error("Queue failed", "error", err) + } + return err } +// Depending on the request type, returns the hook to properly handle the block +// If final, will run normal operation. Otherwise, will only perform atomic ops func getSyncBlockHandler(b *Block, req ethstatesync.SyncBlockRequest) func(bool) error { switch req { case ethstatesync.AcceptSyncBlockRequest: @@ -203,7 +213,6 @@ func (client *stateSyncerClient) stateSync(ctx context.Context) error { if err := client.syncAtomicTrie(ctx); err != nil { return err } - client.downloaderLock.Unlock() return client.syncStateTrie(ctx) } @@ -251,9 +260,6 @@ func (client *stateSyncerClient) acceptSyncSummary(proposedSummary message.SyncS log.Info("Starting state sync", "summary", proposedSummary) - // Lock the atomic trie to prevent pivots during the atomic sync from dynamic syncing - client.downloaderLock.Lock() - // create a cancellable ctx for the state sync goroutine ctx, cancel := context.WithCancel(context.Background()) client.cancel = cancel @@ -373,6 +379,7 @@ func (client *stateSyncerClient) syncAtomicTrie(ctx context.Context) error { return err } err = <-atomicSyncer.Done() + close(client.atomicDone) log.Info("atomic tx: sync finished", "root", client.syncSummary.AtomicRoot, "err", err) return err } From 933894268958b25c366c4721daf240b1dee11504 Mon Sep 17 00:00:00 2001 From: Austin Larson Date: Mon, 3 Mar 2025 10:23:12 -0500 Subject: [PATCH 20/21] Added additional logging for debugging --- core/blockchain.go | 3 +++ plugin/evm/syncervm_client.go | 2 ++ 2 files changed, 5 insertions(+) diff --git a/core/blockchain.go b/core/blockchain.go index a67a9c0518..13a06dec90 100644 --- a/core/blockchain.go +++ b/core/blockchain.go @@ -663,6 +663,7 @@ func (bc *BlockChain) SenderCacher() *TxSenderCacher { // assumes that the chain manager mutex is held. func (bc *BlockChain) loadLastState(lastAcceptedHash common.Hash) error { // Initialize genesis state + log.Debug("Called loadLastState with", "hash", lastAcceptedHash) if lastAcceptedHash == (common.Hash{}) { return bc.loadGenesisState() } @@ -677,6 +678,7 @@ func (bc *BlockChain) loadLastState(lastAcceptedHash common.Hash) error { if headBlock == nil { return fmt.Errorf("could not load head block %s", head.Hex()) } + log.Debug("Put head block in loadLastState as", "hash", headBlock.Hash(), "height", headBlock.NumberU64()) // Everything seems to be fine, set as the head block bc.currentBlock.Store(headBlock.Header()) @@ -2059,6 +2061,7 @@ func (bc *BlockChain) gatherBlockRootsAboveLastAccepted() map[common.Hash]struct // in-memory and on disk current block pointers to [block]. // Only should be called after state sync has completed. func (bc *BlockChain) ResetToStateSyncedBlock(block *types.Block) error { + log.Debug("Called ResetToStateSyncedBlock with", "hash", block.Hash(), "height", block.NumberU64()) bc.chainmu.Lock() defer bc.chainmu.Unlock() diff --git a/plugin/evm/syncervm_client.go b/plugin/evm/syncervm_client.go index b98488276a..6c3ea3b9b8 100644 --- a/plugin/evm/syncervm_client.go +++ b/plugin/evm/syncervm_client.go @@ -462,12 +462,14 @@ func (client *stateSyncerClient) getEVMBlockFromHash(blockHash common.Hash) *Blo // 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 *stateSyncerClient) finishSync(blockHash common.Hash) error { + log.Debug("Called finishSync with", "hash", blockHash) evmBlock := client.getEVMBlockFromHash(blockHash) if evmBlock == nil { return fmt.Errorf("Could not get evmBlock form hash %s", blockHash) } block := evmBlock.ethBlock + log.Debug("Found block for finishSync", "hash", block.Hash(), "height", block.NumberU64()) // BloomIndexer needs to know that some parts of the chain are not available // and cannot be indexed. This is done by calling [AddCheckpoint] here. From bd29ed82f7f5dd08612c9d0cc7117774e5edeb0b Mon Sep 17 00:00:00 2001 From: Austin Larson Date: Mon, 3 Mar 2025 15:11:51 -0500 Subject: [PATCH 21/21] Fixed bug in finding evmBlock for specific hash --- plugin/evm/syncervm_client.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/plugin/evm/syncervm_client.go b/plugin/evm/syncervm_client.go index 6c3ea3b9b8..30bf997822 100644 --- a/plugin/evm/syncervm_client.go +++ b/plugin/evm/syncervm_client.go @@ -440,9 +440,9 @@ func (client *stateSyncerClient) Shutdown() error { func (client *stateSyncerClient) getEVMBlockFromHash(blockHash common.Hash) *Block { // Must first find first pivot block to signal bootstrapper - stateBlock, err := client.state.GetBlock(context.TODO(), ids.ID(client.syncSummary.BlockHash)) + stateBlock, err := client.state.GetBlock(context.TODO(), ids.ID(blockHash)) if err != nil { - log.Error("could not get block by hash from client state", "hash", client.syncSummary.BlockHash) + log.Error("could not get block by hash from client state", "hash", blockHash) return nil } wrapper, ok := stateBlock.(*chain.BlockWrapper)