From aa82f967cb124c9c8825538e7287aa7aea0550f4 Mon Sep 17 00:00:00 2001
From: taratorio <94537774+taratorio@users.noreply.github.com>
Date: Fri, 23 Jan 2026 17:55:58 +1100
Subject: [PATCH] execution/stagedsync: cleanup unused stuff in stages
---
cmd/integration/commands/stages.go | 24 +---
execution/stagedsync/default_stages.go | 15 +--
execution/stagedsync/stage_blockhashes.go | 17 +--
execution/stagedsync/stage_bodies.go | 17 +--
execution/stagedsync/stage_finish.go | 6 +-
execution/stagedsync/stage_headers.go | 33 +-----
.../stagedsync/stage_mining_create_block.go | 6 -
execution/stagedsync/stage_mining_exec.go | 3 -
execution/stagedsync/stage_mining_finish.go | 3 -
execution/stagedsync/stage_senders.go | 15 +--
execution/stagedsync/stage_senders_test.go | 2 +-
execution/stagedsync/stage_snapshots.go | 20 ++--
execution/stagedsync/stage_txlookup.go | 13 ---
execution/stagedsync/stage_witness.go | 23 ++--
.../stagedsync/stage_witness_processing.go | 10 +-
execution/stagedsync/stagebuilder.go | 1 -
execution/stagedsync/stageloop/stageloop.go | 60 ++++++----
execution/stagedsync/stages/stages.go | 2 -
execution/stagedsync/testutil.go | 109 ------------------
execution/stagedsync/types.go | 24 ----
execution/tests/mock/mock_sentry.go | 67 ++++++-----
node/eth/backend.go | 34 +++---
rpc/jsonrpc/eth_call.go | 2 +-
23 files changed, 141 insertions(+), 365 deletions(-)
delete mode 100644 execution/stagedsync/testutil.go
delete mode 100644 execution/stagedsync/types.go
diff --git a/cmd/integration/commands/stages.go b/cmd/integration/commands/stages.go
index 99f811f80de..8beb0ce903c 100644
--- a/cmd/integration/commands/stages.go
+++ b/cmd/integration/commands/stages.go
@@ -31,7 +31,6 @@ import (
"github.com/c2h5oh/datasize"
"github.com/erigontech/mdbx-go/mdbx"
"github.com/erigontech/secp256k1"
- lru "github.com/hashicorp/golang-lru/arc/v2"
"github.com/spf13/cobra"
"golang.org/x/sync/errgroup"
"golang.org/x/sync/semaphore"
@@ -68,7 +67,6 @@ import (
"github.com/erigontech/erigon/execution/state/genesiswrite"
"github.com/erigontech/erigon/execution/tracing"
"github.com/erigontech/erigon/execution/types"
- "github.com/erigontech/erigon/execution/types/accounts"
"github.com/erigontech/erigon/execution/vm"
"github.com/erigontech/erigon/node/debug"
"github.com/erigontech/erigon/node/eth"
@@ -81,7 +79,6 @@ import (
"github.com/erigontech/erigon/p2p"
"github.com/erigontech/erigon/p2p/sentry"
"github.com/erigontech/erigon/p2p/sentry/sentry_multi_client"
- "github.com/erigontech/erigon/polygon/bor"
"github.com/erigontech/erigon/polygon/bor/borcfg"
"github.com/erigontech/erigon/polygon/bridge"
"github.com/erigontech/erigon/polygon/heimdall"
@@ -665,8 +662,8 @@ func stageBodies(db kv.TemporalRwDB, ctx context.Context, logger log.Logger) err
}
u := sync.NewUnwindState(stages.Bodies, s.BlockNumber-unwind, s.BlockNumber, true, false)
- cfg := stagedsync.StageBodiesCfg(db, nil, nil, nil, nil, 0, chainConfig, br, bw)
- if err := stagedsync.UnwindBodiesStage(u, tx, cfg, ctx); err != nil {
+ cfg := stagedsync.StageBodiesCfg(nil, nil, nil, nil, 0, chainConfig, br, bw)
+ if err := stagedsync.UnwindBodiesStage(u, tx, cfg); err != nil {
return err
}
@@ -752,7 +749,7 @@ func stageSenders(db kv.TemporalRwDB, ctx context.Context, logger log.Logger) er
return err
}
- cfg := stagedsync.StageSendersCfg(db, chainConfig, sync.Cfg(), false, tmpdir, pm, br, nil)
+ cfg := stagedsync.StageSendersCfg(chainConfig, sync.Cfg(), false /* badBlockHalt */, tmpdir, pm, br, nil /* hd */)
if unwind > 0 {
u := sync.NewUnwindState(stages.Senders, s.BlockNumber-unwind, s.BlockNumber, true, false)
if err = stagedsync.UnwindSendersStage(u, tx, cfg, ctx); err != nil {
@@ -996,7 +993,6 @@ func stageCustomTrace(db kv.TemporalRwDB, ctx context.Context, logger log.Logger
func stageTxLookup(db kv.TemporalRwDB, ctx context.Context, logger log.Logger) error {
dirs, pm := datadir.New(datadirCli), fromdb.PruneMode(db)
_, _, _, sync := newSync(ctx, db, nil /* miningConfig */, logger)
- chainConfig := fromdb.ChainConfig(db)
must(sync.SetCurrentStage(stages.TxLookup))
if reset {
return db.Update(ctx, rawdbreset.ResetTxLookup)
@@ -1014,7 +1010,7 @@ func stageTxLookup(db kv.TemporalRwDB, ctx context.Context, logger log.Logger) e
logger.Info("Stage", "name", s.ID, "progress", s.BlockNumber)
br, _ := blocksIO(db, logger)
- cfg := stagedsync.StageTxLookupCfg(db, pm, dirs.Tmp, chainConfig.Bor, br)
+ cfg := stagedsync.StageTxLookupCfg(pm, dirs.Tmp, br)
if unwind > 0 {
u := sync.NewUnwindState(stages.TxLookup, s.BlockNumber-unwind, s.BlockNumber, true, false)
err = stagedsync.UnwindTxLookup(u, s, tx, cfg, ctx, logger)
@@ -1256,20 +1252,10 @@ func newSync(ctx context.Context, db kv.TemporalRwDB, miningConfig *buildercfg.M
if err != nil {
panic(err)
}
-
notifications := shards.NewNotifications(nil)
-
- var signatures *lru.ARCCache[common.Hash, accounts.Address]
-
- if bor, ok := engine.(*bor.Bor); ok {
- signatures = bor.Signatures
- }
blockRetire := freezeblocks.NewBlockRetire(estimate.CompressSnapshot.Workers(), dirs, blockReader, blockWriter, db, heimdallStore, bridgeStore, chainConfig, &cfg, notifications.Events, blockSnapBuildSema, logger)
-
- stageList := stageloop.NewDefaultStages(context.Background(), db, p2p.Config{}, &cfg, sentryControlServer, notifications, nil, blockReader, blockRetire, nil, nil,
- signatures, logger, nil)
+ stageList := stageloop.NewDefaultStages(context.Background(), db, p2p.Config{}, &cfg, sentryControlServer, notifications, nil, blockReader, blockRetire, nil, nil, nil)
sync := stagedsync.New(cfg.Sync, stageList, stagedsync.DefaultUnwindOrder, stagedsync.DefaultPruneOrder, logger, stages.ModeApplyingBlocks)
-
return blockRetire, engine, vmConfig, sync
}
diff --git a/execution/stagedsync/default_stages.go b/execution/stagedsync/default_stages.go
index 9f96d662115..fe457dc4916 100644
--- a/execution/stagedsync/default_stages.go
+++ b/execution/stagedsync/default_stages.go
@@ -36,7 +36,8 @@ func DefaultStages(ctx context.Context,
exec ExecuteBlockCfg,
txLookup TxLookupCfg,
finish FinishCfg,
- test bool) []*Stage {
+ test bool,
+) []*Stage {
return []*Stage{
{
ID: stages.Snapshots,
@@ -77,7 +78,7 @@ func DefaultStages(ctx context.Context,
return SpawnBlockHashStage(s, tx, blockHashCfg, ctx, logger)
},
Unwind: func(u *UnwindState, s *StageState, sd *execctx.SharedDomains, tx kv.TemporalRwTx, logger log.Logger) error {
- return UnwindBlockHashStage(u, tx, blockHashCfg, ctx)
+ return UnwindBlockHashStage(u, tx)
},
Prune: func(ctx context.Context, p *PruneState, tx kv.RwTx, timeout time.Duration, logger log.Logger) error {
return nil
@@ -90,7 +91,7 @@ func DefaultStages(ctx context.Context,
return BodiesForward(s, u, ctx, tx, bodies, test, logger)
},
Unwind: func(u *UnwindState, s *StageState, sd *execctx.SharedDomains, tx kv.TemporalRwTx, logger log.Logger) error {
- return UnwindBodiesStage(u, tx, bodies, ctx)
+ return UnwindBodiesStage(u, tx, bodies)
},
Prune: func(ctx context.Context, p *PruneState, tx kv.RwTx, timeout time.Duration, logger log.Logger) error {
return nil
@@ -195,7 +196,7 @@ func PipelineStages(ctx context.Context, snapshots SnapshotsCfg, blockHashCfg Bl
return SpawnBlockHashStage(s, tx, blockHashCfg, ctx, logger)
},
Unwind: func(u *UnwindState, s *StageState, sd *execctx.SharedDomains, tx kv.TemporalRwTx, logger log.Logger) error {
- return UnwindBlockHashStage(u, tx, blockHashCfg, ctx)
+ return UnwindBlockHashStage(u, tx)
},
Prune: func(ctx context.Context, p *PruneState, tx kv.RwTx, timeout time.Duration, logger log.Logger) error {
return nil
@@ -234,7 +235,7 @@ func PipelineStages(ctx context.Context, snapshots SnapshotsCfg, blockHashCfg Bl
ID: stages.WitnessProcessing,
Description: "Process buffered witness data",
Forward: func(badBlockUnwind bool, s *StageState, u Unwinder, sd *execctx.SharedDomains, tx kv.TemporalRwTx, logger log.Logger) error {
- return SpawnStageWitnessProcessing(s, tx, *witnessProcessing, ctx, logger)
+ return SpawnStageWitnessProcessing(tx, *witnessProcessing, logger)
},
Unwind: func(u *UnwindState, s *StageState, sd *execctx.SharedDomains, tx kv.TemporalRwTx, logger log.Logger) error {
return UnwindWitnessProcessingStage(u, s, tx, ctx, *witnessProcessing, logger)
@@ -297,7 +298,7 @@ func StateStages(ctx context.Context, headers HeadersCfg, bodies BodiesCfg, bloc
return nil
},
Unwind: func(u *UnwindState, s *StageState, sd *execctx.SharedDomains, tx kv.TemporalRwTx, logger log.Logger) error {
- return UnwindBodiesStage(u, tx, bodies, ctx)
+ return UnwindBodiesStage(u, tx, bodies)
},
},
{
@@ -307,7 +308,7 @@ func StateStages(ctx context.Context, headers HeadersCfg, bodies BodiesCfg, bloc
return SpawnBlockHashStage(s, tx, blockHashCfg, ctx, logger)
},
Unwind: func(u *UnwindState, s *StageState, sd *execctx.SharedDomains, tx kv.TemporalRwTx, logger log.Logger) error {
- return UnwindBlockHashStage(u, tx, blockHashCfg, ctx)
+ return UnwindBlockHashStage(u, tx)
},
},
{
diff --git a/execution/stagedsync/stage_blockhashes.go b/execution/stagedsync/stage_blockhashes.go
index 0439eb5e363..f5412029605 100644
--- a/execution/stagedsync/stage_blockhashes.go
+++ b/execution/stagedsync/stage_blockhashes.go
@@ -23,23 +23,17 @@ import (
"github.com/erigontech/erigon/common/log/v3"
"github.com/erigontech/erigon/db/kv"
"github.com/erigontech/erigon/db/rawdb/blockio"
- "github.com/erigontech/erigon/execution/chain"
"github.com/erigontech/erigon/execution/stagedsync/stages"
)
type BlockHashesCfg struct {
- db kv.RwDB
- tmpDir string
- cc *chain.Config
-
+ tmpDir string
headerWriter *blockio.BlockWriter
}
-func StageBlockHashesCfg(db kv.RwDB, tmpDir string, cc *chain.Config, headerWriter *blockio.BlockWriter) BlockHashesCfg {
+func StageBlockHashesCfg(tmpDir string, headerWriter *blockio.BlockWriter) BlockHashesCfg {
return BlockHashesCfg{
- db: db,
tmpDir: tmpDir,
- cc: cc,
headerWriter: headerWriter,
}
}
@@ -62,9 +56,6 @@ func SpawnBlockHashStage(s *StageState, tx kv.RwTx, cfg BlockHashesCfg, ctx cont
return nil
}
-func UnwindBlockHashStage(u *UnwindState, tx kv.RwTx, cfg BlockHashesCfg, ctx context.Context) (err error) {
- if err = u.Done(tx); err != nil {
- return fmt.Errorf(" reset: %w", err)
- }
- return nil
+func UnwindBlockHashStage(u *UnwindState, tx kv.RwTx) error {
+ return u.Done(tx)
}
diff --git a/execution/stagedsync/stage_bodies.go b/execution/stagedsync/stage_bodies.go
index 65eda066c75..1081c106a78 100644
--- a/execution/stagedsync/stage_bodies.go
+++ b/execution/stagedsync/stage_bodies.go
@@ -41,7 +41,6 @@ import (
const requestLoopCutOff int = 1
type BodiesCfg struct {
- db kv.RwDB
bd *bodydownload.BodyDownload
bodyReqSend func(context.Context, *bodydownload.BodyRequest) ([64]byte, bool)
penalise func(context.Context, []headerdownload.PenaltyItem)
@@ -52,7 +51,7 @@ type BodiesCfg struct {
blockWriter *blockio.BlockWriter
}
-func StageBodiesCfg(db kv.RwDB, bd *bodydownload.BodyDownload,
+func StageBodiesCfg(bd *bodydownload.BodyDownload,
bodyReqSend func(context.Context, *bodydownload.BodyRequest) ([64]byte, bool), penalise func(context.Context, []headerdownload.PenaltyItem),
blockPropagator bodydownload.BlockPropagator, timeout int,
chanConfig *chain.Config,
@@ -60,7 +59,7 @@ func StageBodiesCfg(db kv.RwDB, bd *bodydownload.BodyDownload,
blockWriter *blockio.BlockWriter,
) BodiesCfg {
return BodiesCfg{
- db: db, bd: bd, bodyReqSend: bodyReqSend, penalise: penalise, blockPropagator: blockPropagator,
+ bd: bd, bodyReqSend: bodyReqSend, penalise: penalise, blockPropagator: blockPropagator,
timeout: timeout, chanConfig: chanConfig, blockReader: blockReader,
blockWriter: blockWriter}
}
@@ -378,15 +377,11 @@ func logWritingBodies(logPrefix string, committed, headerProgress uint64, logger
)
}
-func UnwindBodiesStage(u *UnwindState, tx kv.RwTx, cfg BodiesCfg, ctx context.Context) (err error) {
+func UnwindBodiesStage(u *UnwindState, tx kv.RwTx, cfg BodiesCfg) error {
u.UnwindPoint = max(u.UnwindPoint, cfg.blockReader.FrozenBlocks()) // protect from unwind behind files
- logEvery := time.NewTicker(logInterval)
- defer logEvery.Stop()
- if err := cfg.blockWriter.MakeBodiesNonCanonical(tx, u.UnwindPoint+1); err != nil {
+ err := cfg.blockWriter.MakeBodiesNonCanonical(tx, u.UnwindPoint+1)
+ if err != nil {
return err
}
- if err = u.Done(tx); err != nil {
- return err
- }
- return nil
+ return u.Done(tx)
}
diff --git a/execution/stagedsync/stage_finish.go b/execution/stagedsync/stage_finish.go
index 427c8bababf..6aa3719701b 100644
--- a/execution/stagedsync/stage_finish.go
+++ b/execution/stagedsync/stage_finish.go
@@ -31,17 +31,13 @@ import (
)
type FinishCfg struct {
- db kv.RwDB
- tmpDir string
forkValidator *engine_helpers.ForkValidator
initialCycleStart *time.Time
}
-func StageFinishCfg(db kv.RwDB, tmpDir string, forkValidator *engine_helpers.ForkValidator) FinishCfg {
+func StageFinishCfg(forkValidator *engine_helpers.ForkValidator) FinishCfg {
initialCycleStart := time.Now()
return FinishCfg{
- db: db,
- tmpDir: tmpDir,
forkValidator: forkValidator,
initialCycleStart: &initialCycleStart,
}
diff --git a/execution/stagedsync/stage_headers.go b/execution/stagedsync/stage_headers.go
index 80878fde24c..029995280db 100644
--- a/execution/stagedsync/stage_headers.go
+++ b/execution/stagedsync/stage_headers.go
@@ -25,81 +25,52 @@ import (
"runtime"
"time"
- "github.com/c2h5oh/datasize"
-
"github.com/erigontech/erigon/common"
"github.com/erigontech/erigon/common/dbg"
"github.com/erigontech/erigon/common/hexutil"
"github.com/erigontech/erigon/common/log/v3"
"github.com/erigontech/erigon/db/kv"
"github.com/erigontech/erigon/db/rawdb"
- "github.com/erigontech/erigon/db/rawdb/blockio"
"github.com/erigontech/erigon/db/services"
"github.com/erigontech/erigon/db/state/execctx"
"github.com/erigontech/erigon/diagnostics/diaglib"
"github.com/erigontech/erigon/execution/chain"
"github.com/erigontech/erigon/execution/rlp"
- "github.com/erigontech/erigon/execution/stagedsync/bodydownload"
"github.com/erigontech/erigon/execution/stagedsync/headerdownload"
"github.com/erigontech/erigon/execution/types"
"github.com/erigontech/erigon/node/ethconfig"
- "github.com/erigontech/erigon/node/shards"
)
-// The number of blocks we should be able to re-org sub-second on commodity hardware.
-// See https://hackmd.io/TdJtNs0dS56q-In8h-ShSg
-const ShortPoSReorgThresholdBlocks = 10
-
type HeadersCfg struct {
- db kv.RwDB
hd *headerdownload.HeaderDownload
- bodyDownload *bodydownload.BodyDownload
chainConfig *chain.Config
headerReqSend func(context.Context, *headerdownload.HeaderRequest) ([64]byte, bool)
announceNewHashes func(context.Context, []headerdownload.Announce)
penalize func(context.Context, []headerdownload.PenaltyItem)
- batchSize datasize.ByteSize
noP2PDiscovery bool
- tmpdir string
-
- blockReader services.FullBlockReader
- blockWriter *blockio.BlockWriter
- notifications *shards.Notifications
-
- syncConfig ethconfig.Sync
+ blockReader services.FullBlockReader
+ syncConfig ethconfig.Sync
}
func StageHeadersCfg(
- db kv.RwDB,
headerDownload *headerdownload.HeaderDownload,
- bodyDownload *bodydownload.BodyDownload,
chainConfig *chain.Config,
syncConfig ethconfig.Sync,
headerReqSend func(context.Context, *headerdownload.HeaderRequest) ([64]byte, bool),
announceNewHashes func(context.Context, []headerdownload.Announce),
penalize func(context.Context, []headerdownload.PenaltyItem),
- batchSize datasize.ByteSize,
noP2PDiscovery bool,
blockReader services.FullBlockReader,
- blockWriter *blockio.BlockWriter,
- tmpdir string,
- notifications *shards.Notifications,
) HeadersCfg {
return HeadersCfg{
- db: db,
hd: headerDownload,
- bodyDownload: bodyDownload,
chainConfig: chainConfig,
syncConfig: syncConfig,
headerReqSend: headerReqSend,
announceNewHashes: announceNewHashes,
penalize: penalize,
- batchSize: batchSize,
- tmpdir: tmpdir,
noP2PDiscovery: noP2PDiscovery,
blockReader: blockReader,
- blockWriter: blockWriter,
- notifications: notifications,
}
}
diff --git a/execution/stagedsync/stage_mining_create_block.go b/execution/stagedsync/stage_mining_create_block.go
index 3f0fd399fac..a1eb3eb5b61 100644
--- a/execution/stagedsync/stage_mining_create_block.go
+++ b/execution/stagedsync/stage_mining_create_block.go
@@ -123,30 +123,24 @@ func NewMiningState(cfg *buildercfg.MiningConfig) MiningState {
}
type MiningCreateBlockCfg struct {
- db kv.RwDB
miner MiningState
chainConfig *chain.Config
engine rules.Engine
- tmpdir string
blockBuilderParameters *builder.Parameters
blockReader services.FullBlockReader
}
func StageMiningCreateBlockCfg(
- db kv.RwDB,
miner MiningState,
chainConfig *chain.Config,
engine rules.Engine,
blockBuilderParameters *builder.Parameters,
- tmpdir string,
blockReader services.FullBlockReader,
) MiningCreateBlockCfg {
return MiningCreateBlockCfg{
- db: db,
miner: miner,
chainConfig: chainConfig,
engine: engine,
- tmpdir: tmpdir,
blockBuilderParameters: blockBuilderParameters,
blockReader: blockReader,
}
diff --git a/execution/stagedsync/stage_mining_exec.go b/execution/stagedsync/stage_mining_exec.go
index a2dab61387f..19c8836b796 100644
--- a/execution/stagedsync/stage_mining_exec.go
+++ b/execution/stagedsync/stage_mining_exec.go
@@ -50,7 +50,6 @@ import (
)
type MiningExecCfg struct {
- db kv.RwDB
miningState MiningState
notifier ChainEventNotifier
chainConfig *chain.Config
@@ -64,7 +63,6 @@ type MiningExecCfg struct {
}
func StageMiningExecCfg(
- db kv.RwDB,
miningState MiningState,
notifier ChainEventNotifier,
chainConfig *chain.Config,
@@ -77,7 +75,6 @@ func StageMiningExecCfg(
blockReader services.FullBlockReader,
) MiningExecCfg {
return MiningExecCfg{
- db: db,
miningState: miningState,
notifier: notifier,
chainConfig: chainConfig,
diff --git a/execution/stagedsync/stage_mining_finish.go b/execution/stagedsync/stage_mining_finish.go
index 60fa682b93f..4f87ed04300 100644
--- a/execution/stagedsync/stage_mining_finish.go
+++ b/execution/stagedsync/stage_mining_finish.go
@@ -30,7 +30,6 @@ import (
)
type MiningFinishCfg struct {
- db kv.RwDB
chainConfig *chain.Config
engine rules.Engine
sealCancel chan struct{}
@@ -40,7 +39,6 @@ type MiningFinishCfg struct {
}
func StageMiningFinishCfg(
- db kv.RwDB,
chainConfig *chain.Config,
engine rules.Engine,
miningState MiningState,
@@ -49,7 +47,6 @@ func StageMiningFinishCfg(
latestBlockBuiltStore *builder.LatestBlockBuiltStore,
) MiningFinishCfg {
return MiningFinishCfg{
- db: db,
chainConfig: chainConfig,
engine: engine,
miningState: miningState,
diff --git a/execution/stagedsync/stage_senders.go b/execution/stagedsync/stage_senders.go
index 8b71fc9cb85..76f03daac09 100644
--- a/execution/stagedsync/stage_senders.go
+++ b/execution/stagedsync/stage_senders.go
@@ -48,34 +48,25 @@ import (
)
type SendersCfg struct {
- db kv.RwDB
batchSize int
numOfGoroutines int
- readChLen int
badBlockHalt bool
tmpdir string
- prune prune.Mode
chainConfig *chain.Config
hd *headerdownload.HeaderDownload
blockReader services.FullBlockReader
- syncCfg ethconfig.Sync
}
-func StageSendersCfg(db kv.RwDB, chainCfg *chain.Config, syncCfg ethconfig.Sync, badBlockHalt bool, tmpdir string, prune prune.Mode, blockReader services.FullBlockReader, hd *headerdownload.HeaderDownload) SendersCfg {
+func StageSendersCfg(chainCfg *chain.Config, syncCfg ethconfig.Sync, badBlockHalt bool, tmpdir string, prune prune.Mode, blockReader services.FullBlockReader, hd *headerdownload.HeaderDownload) SendersCfg {
const sendersBatchSize = 1000
-
return SendersCfg{
- db: db,
batchSize: sendersBatchSize,
numOfGoroutines: secp256k1.NumOfContexts(), // we can only be as parallels as our crypto library supports,
- readChLen: 4,
badBlockHalt: badBlockHalt,
tmpdir: tmpdir,
chainConfig: chainCfg,
- prune: prune,
hd: hd,
blockReader: blockReader,
- syncCfg: syncCfg,
}
}
@@ -120,7 +111,7 @@ func SpawnRecoverSendersStage(cfg SendersCfg, s *StageState, u Unwinder, tx kv.R
defer dbg.LogPanic()
defer wg.Done()
// each goroutine gets it's own crypto context to make sure they are really parallel
- recoverSenders(ctx, logPrefix, secp256k1.ContextForThread(threadNo), cfg.chainConfig, jobs, out, quitCh)
+ recoverSenders(ctx, secp256k1.ContextForThread(threadNo), cfg.chainConfig, jobs, out, quitCh)
}(i)
}
@@ -367,7 +358,7 @@ type senderRecoveryJob struct {
err error
}
-func recoverSenders(ctx context.Context, logPrefix string, cryptoContext *secp256k1.Context, config *chain.Config, in, out chan *senderRecoveryJob, quit <-chan struct{}) {
+func recoverSenders(ctx context.Context, cryptoContext *secp256k1.Context, config *chain.Config, in, out chan *senderRecoveryJob, quit <-chan struct{}) {
var job *senderRecoveryJob
var ok bool
for {
diff --git a/execution/stagedsync/stage_senders_test.go b/execution/stagedsync/stage_senders_test.go
index 004764e0253..10d659dc54e 100644
--- a/execution/stagedsync/stage_senders_test.go
+++ b/execution/stagedsync/stage_senders_test.go
@@ -145,7 +145,7 @@ func TestSenders(t *testing.T) {
require.NoError(stages.SaveStageProgress(tx, stages.Bodies, 3))
- cfg := stagedsync.StageSendersCfg(db, chain.TestChainConfig, ethconfig.Defaults.Sync, false, "", prune.Mode{}, br, nil)
+ cfg := stagedsync.StageSendersCfg(chain.TestChainConfig, ethconfig.Defaults.Sync, false, "", prune.Mode{}, br, nil)
err = stagedsync.SpawnRecoverSendersStage(cfg, &stagedsync.StageState{ID: stages.Senders}, nil, tx, 3, m.Ctx, log.New())
require.NoError(err)
diff --git a/execution/stagedsync/stage_snapshots.go b/execution/stagedsync/stage_snapshots.go
index 7f1f29bfaeb..28be6dfdae6 100644
--- a/execution/stagedsync/stage_snapshots.go
+++ b/execution/stagedsync/stage_snapshots.go
@@ -50,21 +50,19 @@ import (
)
type SnapshotsCfg struct {
- db kv.TemporalRwDB
- chainConfig *chain.Config
- dirs datadir.Dirs
-
+ db kv.TemporalRwDB
+ chainConfig *chain.Config
+ dirs datadir.Dirs
blockRetire services.BlockRetire
snapshotDownloader downloader.Client
blockReader services.FullBlockReader
notifier *shards.Notifications
-
- caplin bool
- blobs bool
- caplinState bool
- silkworm *silkworm.Silkworm
- syncConfig ethconfig.Sync
- prune prune.Mode
+ caplin bool
+ blobs bool
+ caplinState bool
+ silkworm *silkworm.Silkworm
+ syncConfig ethconfig.Sync
+ prune prune.Mode
}
// Returns a seeder client for block management, a noop implementation if no downloader is attached.
diff --git a/execution/stagedsync/stage_txlookup.go b/execution/stagedsync/stage_txlookup.go
index 6db2569f878..5c92b5e68d9 100644
--- a/execution/stagedsync/stage_txlookup.go
+++ b/execution/stagedsync/stage_txlookup.go
@@ -32,37 +32,24 @@ import (
"github.com/erigontech/erigon/db/kv/rawdbv3"
"github.com/erigontech/erigon/db/rawdb"
"github.com/erigontech/erigon/db/services"
- "github.com/erigontech/erigon/execution/chain"
"github.com/erigontech/erigon/execution/stagedsync/stages"
"github.com/erigontech/erigon/execution/types"
- "github.com/erigontech/erigon/polygon/bor/borcfg"
)
type TxLookupCfg struct {
- db kv.RwDB
prune prune.Mode
tmpdir string
- borConfig *borcfg.BorConfig
blockReader services.FullBlockReader
}
func StageTxLookupCfg(
- db kv.RwDB,
prune prune.Mode,
tmpdir string,
- borConfigInterface chain.BorConfig,
blockReader services.FullBlockReader,
) TxLookupCfg {
- var borConfig *borcfg.BorConfig
- if borConfigInterface != nil {
- borConfig = borConfigInterface.(*borcfg.BorConfig)
- }
-
return TxLookupCfg{
- db: db,
prune: prune,
tmpdir: tmpdir,
- borConfig: borConfig,
blockReader: blockReader,
}
}
diff --git a/execution/stagedsync/stage_witness.go b/execution/stagedsync/stage_witness.go
index d5f94061294..fc82258fdf9 100644
--- a/execution/stagedsync/stage_witness.go
+++ b/execution/stagedsync/stage_witness.go
@@ -28,13 +28,10 @@ import (
)
type WitnessCfg struct {
- db kv.RwDB
- enableWitnessGeneration bool
- maxWitnessLimit uint64
- chainConfig *chain.Config
- engine rules.Engine
- blockReader services.FullBlockReader
- dirs datadir.Dirs
+ chainConfig *chain.Config
+ engine rules.Engine
+ blockReader services.FullBlockReader
+ dirs datadir.Dirs
}
type WitnessStore struct {
@@ -45,14 +42,12 @@ type WitnessStore struct {
GetHashFn func(n uint64) (common.Hash, error)
}
-func StageWitnessCfg(enableWitnessGeneration bool, maxWitnessLimit uint64, chainConfig *chain.Config, engine rules.Engine, blockReader services.FullBlockReader, dirs datadir.Dirs) WitnessCfg {
+func StageWitnessCfg(chainConfig *chain.Config, engine rules.Engine, blockReader services.FullBlockReader, dirs datadir.Dirs) WitnessCfg {
return WitnessCfg{
- enableWitnessGeneration: enableWitnessGeneration,
- maxWitnessLimit: maxWitnessLimit,
- chainConfig: chainConfig,
- engine: engine,
- blockReader: blockReader,
- dirs: dirs,
+ chainConfig: chainConfig,
+ engine: engine,
+ blockReader: blockReader,
+ dirs: dirs,
}
}
diff --git a/execution/stagedsync/stage_witness_processing.go b/execution/stagedsync/stage_witness_processing.go
index 2ec56b68cd7..e40c5d68b7f 100644
--- a/execution/stagedsync/stage_witness_processing.go
+++ b/execution/stagedsync/stage_witness_processing.go
@@ -73,20 +73,18 @@ func (wb *WitnessBuffer) DrainWitnesses() []WitnessData {
}
type WitnessProcessingCfg struct {
- db kv.RwDB
witnessBuffer *WitnessBuffer
}
-func NewWitnessProcessingCfg(db kv.RwDB, witnessBuffer *WitnessBuffer) WitnessProcessingCfg {
+func NewWitnessProcessingCfg(witnessBuffer *WitnessBuffer) WitnessProcessingCfg {
return WitnessProcessingCfg{
- db: db,
witnessBuffer: witnessBuffer,
}
}
-func StageWitnessProcessingCfg(db kv.RwDB, chainConfig *chain.Config, witnessBuffer *WitnessBuffer) *WitnessProcessingCfg {
+func StageWitnessProcessingCfg(chainConfig *chain.Config, witnessBuffer *WitnessBuffer) *WitnessProcessingCfg {
if chainConfig.Bor != nil && witnessBuffer != nil {
- cfg := NewWitnessProcessingCfg(db, witnessBuffer)
+ cfg := NewWitnessProcessingCfg(witnessBuffer)
return &cfg
}
@@ -94,7 +92,7 @@ func StageWitnessProcessingCfg(db kv.RwDB, chainConfig *chain.Config, witnessBuf
}
// SpawnStageWitnessProcessing processes buffered witness data and stores it in the database
-func SpawnStageWitnessProcessing(s *StageState, tx kv.RwTx, cfg WitnessProcessingCfg, ctx context.Context, logger log.Logger) error {
+func SpawnStageWitnessProcessing(tx kv.RwTx, cfg WitnessProcessingCfg, logger log.Logger) error {
if cfg.witnessBuffer == nil {
return nil
}
diff --git a/execution/stagedsync/stagebuilder.go b/execution/stagedsync/stagebuilder.go
index 42082737b18..d79fad2e335 100644
--- a/execution/stagedsync/stagebuilder.go
+++ b/execution/stagedsync/stagebuilder.go
@@ -44,7 +44,6 @@ func MiningStages(
sendersCfg SendersCfg,
execCfg MiningExecCfg,
finish MiningFinishCfg,
- astridEnabled bool,
) []*Stage {
return []*Stage{
{
diff --git a/execution/stagedsync/stageloop/stageloop.go b/execution/stagedsync/stageloop/stageloop.go
index a513a3ec8f5..b9cd97741d3 100644
--- a/execution/stagedsync/stageloop/stageloop.go
+++ b/execution/stagedsync/stageloop/stageloop.go
@@ -23,15 +23,12 @@ import (
"runtime"
"time"
- lru "github.com/hashicorp/golang-lru/arc/v2"
-
- "github.com/erigontech/erigon/db/downloader"
-
"github.com/erigontech/erigon/common"
"github.com/erigontech/erigon/common/dbg"
"github.com/erigontech/erigon/common/log/v3"
"github.com/erigontech/erigon/common/metrics"
"github.com/erigontech/erigon/db/datadir"
+ "github.com/erigontech/erigon/db/downloader"
"github.com/erigontech/erigon/db/kv"
"github.com/erigontech/erigon/db/kv/membatchwithdb"
"github.com/erigontech/erigon/db/kv/rawdbv3"
@@ -50,7 +47,6 @@ import (
"github.com/erigontech/erigon/execution/tracing"
"github.com/erigontech/erigon/execution/tracing/tracers"
"github.com/erigontech/erigon/execution/types"
- "github.com/erigontech/erigon/execution/types/accounts"
"github.com/erigontech/erigon/execution/vm"
"github.com/erigontech/erigon/node/ethconfig"
"github.com/erigontech/erigon/node/gointerfaces"
@@ -706,8 +702,6 @@ func NewDefaultStages(ctx context.Context,
blockRetire services.BlockRetire,
silkworm *silkworm.Silkworm,
forkValidator *engine_helpers.ForkValidator,
- signatures *lru.ARCCache[common.Hash, accounts.Address],
- logger log.Logger,
tracer *tracers.Tracer,
) []*stagedsync.Stage {
var tracingHooks *tracing.Hooks
@@ -723,13 +717,15 @@ func NewDefaultStages(ctx context.Context,
return stagedsync.DefaultStages(ctx,
stagedsync.StageSnapshotsCfg(db, controlServer.ChainConfig, cfg.Sync, dirs, blockRetire, snapDownloader, blockReader, notifications, cfg.InternalCL && cfg.CaplinConfig.ArchiveBlocks, cfg.CaplinConfig.ArchiveBlobs, cfg.CaplinConfig.ArchiveStates, silkworm, cfg.Prune),
- stagedsync.StageHeadersCfg(db, controlServer.Hd, controlServer.Bd, controlServer.ChainConfig, cfg.Sync, controlServer.SendHeaderRequest, controlServer.PropagateNewBlockHashes, controlServer.Penalize, cfg.BatchSize, p2pCfg.NoDiscovery, blockReader, blockWriter, dirs.Tmp, notifications),
- stagedsync.StageBlockHashesCfg(db, dirs.Tmp, controlServer.ChainConfig, blockWriter),
- stagedsync.StageBodiesCfg(db, controlServer.Bd, controlServer.SendBodyRequest, controlServer.Penalize, controlServer.BroadcastNewBlock, cfg.Sync.BodyDownloadTimeoutSeconds, controlServer.ChainConfig, blockReader, blockWriter),
- stagedsync.StageSendersCfg(db, controlServer.ChainConfig, cfg.Sync, false, dirs.Tmp, cfg.Prune, blockReader, controlServer.Hd),
- stagedsync.StageExecuteBlocksCfg(db, cfg.Prune, cfg.BatchSize, controlServer.ChainConfig, controlServer.Engine, &vm.Config{Tracer: tracingHooks}, notifications, cfg.StateStream, false, dirs, blockReader, controlServer.Hd, cfg.Genesis, cfg.Sync, SilkwormForExecutionStage(silkworm, cfg), cfg.ExperimentalBAL),
- stagedsync.StageTxLookupCfg(db, cfg.Prune, dirs.Tmp, controlServer.ChainConfig.Bor, blockReader),
- stagedsync.StageFinishCfg(db, dirs.Tmp, forkValidator), runInTestMode)
+ stagedsync.StageHeadersCfg(controlServer.Hd, controlServer.ChainConfig, cfg.Sync, controlServer.SendHeaderRequest, controlServer.PropagateNewBlockHashes, controlServer.Penalize, p2pCfg.NoDiscovery, blockReader),
+ stagedsync.StageBlockHashesCfg(dirs.Tmp, blockWriter),
+ stagedsync.StageBodiesCfg(controlServer.Bd, controlServer.SendBodyRequest, controlServer.Penalize, controlServer.BroadcastNewBlock, cfg.Sync.BodyDownloadTimeoutSeconds, controlServer.ChainConfig, blockReader, blockWriter),
+ stagedsync.StageSendersCfg(controlServer.ChainConfig, cfg.Sync, false /* badBlockHalt */, dirs.Tmp, cfg.Prune, blockReader, controlServer.Hd),
+ stagedsync.StageExecuteBlocksCfg(db, cfg.Prune, cfg.BatchSize, controlServer.ChainConfig, controlServer.Engine, &vm.Config{Tracer: tracingHooks}, notifications, cfg.StateStream, false /* badBlockHalt */, dirs, blockReader, controlServer.Hd, cfg.Genesis, cfg.Sync, SilkwormForExecutionStage(silkworm, cfg), cfg.ExperimentalBAL),
+ stagedsync.StageTxLookupCfg(cfg.Prune, dirs.Tmp, blockReader),
+ stagedsync.StageFinishCfg(forkValidator),
+ runInTestMode,
+ )
}
func NewPipelineStages(ctx context.Context,
@@ -759,22 +755,36 @@ func NewPipelineStages(ctx context.Context,
return stagedsync.PipelineStages(ctx,
stagedsync.StageSnapshotsCfg(db, controlServer.ChainConfig, cfg.Sync, dirs, blockRetire, snapDownloader, blockReader, notifications, cfg.InternalCL && cfg.CaplinConfig.ArchiveBlocks, cfg.CaplinConfig.ArchiveBlobs, cfg.CaplinConfig.ArchiveStates, silkworm, cfg.Prune),
- stagedsync.StageBlockHashesCfg(db, dirs.Tmp, controlServer.ChainConfig, blockWriter),
- stagedsync.StageSendersCfg(db, controlServer.ChainConfig, cfg.Sync, false, dirs.Tmp, cfg.Prune, blockReader, controlServer.Hd),
+ stagedsync.StageBlockHashesCfg(dirs.Tmp, blockWriter),
+ stagedsync.StageSendersCfg(controlServer.ChainConfig, cfg.Sync, false /* badBlockHalt */, dirs.Tmp, cfg.Prune, blockReader, controlServer.Hd),
stagedsync.StageExecuteBlocksCfg(db, cfg.Prune, cfg.BatchSize, controlServer.ChainConfig, controlServer.Engine, &vm.Config{Tracer: tracingHooks}, notifications, cfg.StateStream, false, dirs, blockReader, controlServer.Hd, cfg.Genesis, cfg.Sync, SilkwormForExecutionStage(silkworm, cfg), cfg.ExperimentalBAL),
- stagedsync.StageTxLookupCfg(db, cfg.Prune, dirs.Tmp, controlServer.ChainConfig.Bor, blockReader),
- stagedsync.StageFinishCfg(db, dirs.Tmp, forkValidator),
- stagedsync.StageWitnessProcessingCfg(db, controlServer.ChainConfig, controlServer.WitnessBuffer))
+ stagedsync.StageTxLookupCfg(cfg.Prune, dirs.Tmp, blockReader),
+ stagedsync.StageFinishCfg(forkValidator),
+ stagedsync.StageWitnessProcessingCfg(controlServer.ChainConfig, controlServer.WitnessBuffer),
+ )
}
-func NewInMemoryExecution(ctx context.Context, db kv.TemporalRwDB, cfg *ethconfig.Config, controlServer *sentry_multi_client.MultiClient,
- dirs datadir.Dirs, notifications *shards.Notifications, blockReader services.FullBlockReader, blockWriter *blockio.BlockWriter,
- silkworm *silkworm.Silkworm, logger log.Logger) *stagedsync.Sync {
+func NewInMemoryExecution(
+ ctx context.Context,
+ db kv.TemporalRwDB,
+ cfg *ethconfig.Config,
+ controlServer *sentry_multi_client.MultiClient,
+ dirs datadir.Dirs,
+ notifications *shards.Notifications,
+ blockReader services.FullBlockReader,
+ blockWriter *blockio.BlockWriter,
+ silkworm *silkworm.Silkworm,
+ logger log.Logger,
+) *stagedsync.Sync {
return stagedsync.New(
cfg.Sync,
- stagedsync.StateStages(ctx, stagedsync.StageHeadersCfg(db, controlServer.Hd, controlServer.Bd, controlServer.ChainConfig, cfg.Sync, controlServer.SendHeaderRequest, controlServer.PropagateNewBlockHashes, controlServer.Penalize, cfg.BatchSize, false, blockReader, blockWriter, dirs.Tmp, nil),
- stagedsync.StageBodiesCfg(db, controlServer.Bd, controlServer.SendBodyRequest, controlServer.Penalize, controlServer.BroadcastNewBlock, cfg.Sync.BodyDownloadTimeoutSeconds, controlServer.ChainConfig, blockReader, blockWriter), stagedsync.StageBlockHashesCfg(db, dirs.Tmp, controlServer.ChainConfig, blockWriter), stagedsync.StageSendersCfg(db, controlServer.ChainConfig, cfg.Sync, true, dirs.Tmp, cfg.Prune, blockReader, controlServer.Hd),
- stagedsync.StageExecuteBlocksCfg(db, cfg.Prune, cfg.BatchSize, controlServer.ChainConfig, controlServer.Engine, &vm.Config{}, notifications, cfg.StateStream, true, cfg.Dirs, blockReader, controlServer.Hd, cfg.Genesis, cfg.Sync, SilkwormForExecutionStage(silkworm, cfg), cfg.ExperimentalBAL)),
+ stagedsync.StateStages(ctx,
+ stagedsync.StageHeadersCfg(controlServer.Hd, controlServer.ChainConfig, cfg.Sync, controlServer.SendHeaderRequest, controlServer.PropagateNewBlockHashes, controlServer.Penalize, false /* noP2PDiscovery */, blockReader),
+ stagedsync.StageBodiesCfg(controlServer.Bd, controlServer.SendBodyRequest, controlServer.Penalize, controlServer.BroadcastNewBlock, cfg.Sync.BodyDownloadTimeoutSeconds, controlServer.ChainConfig, blockReader, blockWriter),
+ stagedsync.StageBlockHashesCfg(dirs.Tmp, blockWriter),
+ stagedsync.StageSendersCfg(controlServer.ChainConfig, cfg.Sync, true /* badBlockHalt */, dirs.Tmp, cfg.Prune, blockReader, controlServer.Hd),
+ stagedsync.StageExecuteBlocksCfg(db, cfg.Prune, cfg.BatchSize, controlServer.ChainConfig, controlServer.Engine, &vm.Config{}, notifications, cfg.StateStream, true, cfg.Dirs, blockReader, controlServer.Hd, cfg.Genesis, cfg.Sync, SilkwormForExecutionStage(silkworm, cfg), cfg.ExperimentalBAL),
+ ),
stagedsync.StateUnwindOrder,
nil, /* pruneOrder */
logger,
diff --git a/execution/stagedsync/stages/stages.go b/execution/stagedsync/stages/stages.go
index 566871f87a0..1508326161e 100644
--- a/execution/stagedsync/stages/stages.go
+++ b/execution/stagedsync/stages/stages.go
@@ -40,7 +40,6 @@ var (
Senders SyncStage = "Senders" // "From" recovered from signatures, bodies re-written
Execution SyncStage = "Execution" // Executing each block w/o building a trie
CustomTrace SyncStage = "CustomTrace" // Executing each block w/o building a trie
- Translation SyncStage = "Translation" // Translation each marked for translation contract (from EVM to TEVM)
WitnessProcessing SyncStage = "WitnessProcessing" // Process buffered witness data for Polygon chains
TxLookup SyncStage = "TxLookup" // Generating transactions lookup index
Finish SyncStage = "Finish" // Nominal stage after all other stages
@@ -58,7 +57,6 @@ var AllStages = []SyncStage{
Senders,
Execution,
CustomTrace,
- Translation,
TxLookup,
Finish,
}
diff --git a/execution/stagedsync/testutil.go b/execution/stagedsync/testutil.go
deleted file mode 100644
index 9c817167776..00000000000
--- a/execution/stagedsync/testutil.go
+++ /dev/null
@@ -1,109 +0,0 @@
-// Copyright 2024 The Erigon Authors
-// This file is part of Erigon.
-//
-// Erigon is free software: you can redistribute it and/or modify
-// it under the terms of the GNU Lesser General Public License as published by
-// the Free Software Foundation, either version 3 of the License, or
-// (at your option) any later version.
-//
-// Erigon is distributed in the hope that it will be useful,
-// but WITHOUT ANY WARRANTY; without even the implied warranty of
-// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
-// GNU Lesser General Public License for more details.
-//
-// You should have received a copy of the GNU Lesser General Public License
-// along with Erigon. If not, see .
-
-package stagedsync
-
-import (
- "fmt"
- "math/big"
- "testing"
-
- "github.com/holiman/uint256"
-
- "github.com/erigontech/erigon/common"
- "github.com/erigontech/erigon/execution/state"
- "github.com/erigontech/erigon/execution/types/accounts"
-)
-
-const (
- staticCodeStaticIncarnations = iota // no incarnation changes, no code changes
- changeCodeWithIncarnations // code changes with incarnation
- changeCodeIndepenentlyOfIncarnations // code changes with and without incarnation
-)
-
-type testGenHook func(n, from, numberOfBlocks uint64)
-
-func generateBlocks2(t *testing.T, from uint64, numberOfBlocks uint64, blockWriter state.StateWriter, beforeBlock, afterBlock testGenHook, difficulty int) {
- acc1 := accounts.NewAccount()
- acc1.Incarnation = 1
- acc1.Initialised = true
- acc1.Balance.SetUint64(0)
-
- acc2 := accounts.NewAccount()
- acc2.Incarnation = 0
- acc2.Initialised = true
- acc2.Balance.SetUint64(0)
-
- testAccounts := []*accounts.Account{
- &acc1,
- &acc2,
- }
-
- for blockNumber := uint64(1); blockNumber < from+numberOfBlocks; blockNumber++ {
- beforeBlock(blockNumber, from, numberOfBlocks)
- updateIncarnation := difficulty != staticCodeStaticIncarnations && blockNumber%10 == 0
-
- for i, oldAcc := range testAccounts {
- addr := accounts.InternAddress(common.HexToAddress(fmt.Sprintf("0x1234567890%d", i)))
-
- newAcc := oldAcc.SelfCopy()
- newAcc.Balance.SetUint64(blockNumber)
- if updateIncarnation && oldAcc.Incarnation > 0 /* only update for contracts */ {
- newAcc.Incarnation = oldAcc.Incarnation + 1
- }
-
- if blockNumber == 1 && newAcc.Incarnation > 0 {
- if blockNumber >= from {
- if err := blockWriter.CreateContract(addr); err != nil {
- t.Fatal(err)
- }
- }
- }
- if blockNumber == 1 || updateIncarnation || difficulty == changeCodeIndepenentlyOfIncarnations {
- if newAcc.Incarnation > 0 {
- code := fmt.Appendf(nil, "acc-code-%v", blockNumber)
- codeHashValue, _ := common.HashData(code)
- codeHash := accounts.InternCodeHash(codeHashValue)
- if blockNumber >= from {
- if err := blockWriter.UpdateAccountCode(addr, newAcc.Incarnation, codeHash, code); err != nil {
- t.Fatal(err)
- }
- }
- newAcc.CodeHash = codeHash
- }
- }
-
- if newAcc.Incarnation > 0 {
- var oldValue, newValue uint256.Int
- newValue.SetOne()
- var location common.Hash
- location.SetBytes(new(big.Int).SetUint64(blockNumber).Bytes())
- if blockNumber >= from {
- if err := blockWriter.WriteAccountStorage(addr, newAcc.Incarnation, accounts.InternKey(location), oldValue, newValue); err != nil {
- t.Fatal(err)
- }
- }
- }
- if blockNumber >= from {
- if err := blockWriter.UpdateAccountData(addr, oldAcc, newAcc); err != nil {
- t.Fatal(err)
- }
- }
- testAccounts[i] = newAcc
- }
- afterBlock(blockNumber, from, numberOfBlocks)
- }
-}
diff --git a/execution/stagedsync/types.go b/execution/stagedsync/types.go
deleted file mode 100644
index 9c714a3077c..00000000000
--- a/execution/stagedsync/types.go
+++ /dev/null
@@ -1,24 +0,0 @@
-// Copyright 2024 The Erigon Authors
-// This file is part of Erigon.
-//
-// Erigon is free software: you can redistribute it and/or modify
-// it under the terms of the GNU Lesser General Public License as published by
-// the Free Software Foundation, either version 3 of the License, or
-// (at your option) any later version.
-//
-// Erigon is distributed in the hope that it will be useful,
-// but WITHOUT ANY WARRANTY; without even the implied warranty of
-// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
-// GNU Lesser General Public License for more details.
-//
-// You should have received a copy of the GNU Lesser General Public License
-// along with Erigon. If not, see .
-
-package stagedsync
-
-import "github.com/erigontech/erigon/execution/stagedsync/bodydownload"
-
-type DownloaderGlue interface {
- SpawnHeaderDownloadStage([]func() error, *StageState, Unwinder) error
- SpawnBodyDownloadStage(string, string, *StageState, Unwinder, *bodydownload.PrefetchedBlocks) (bool, error)
-}
diff --git a/execution/tests/mock/mock_sentry.go b/execution/tests/mock/mock_sentry.go
index 44de62b4472..aedcc1638c6 100644
--- a/execution/tests/mock/mock_sentry.go
+++ b/execution/tests/mock/mock_sentry.go
@@ -34,13 +34,12 @@ import (
"google.golang.org/grpc"
"google.golang.org/protobuf/types/known/emptypb"
- "github.com/erigontech/erigon/db/downloader"
-
"github.com/erigontech/erigon/common"
"github.com/erigontech/erigon/common/crypto"
"github.com/erigontech/erigon/common/log/v3"
"github.com/erigontech/erigon/db/consensuschain"
"github.com/erigontech/erigon/db/datadir"
+ "github.com/erigontech/erigon/db/downloader"
"github.com/erigontech/erigon/db/kv"
"github.com/erigontech/erigon/db/kv/dbcfg"
"github.com/erigontech/erigon/db/kv/kvcache"
@@ -483,8 +482,9 @@ func MockWithEverything(tb testing.TB, gspec *types.Genesis, key *ecdsa.PrivateK
miningStatePos.MiningConfig.Etherbase = param.SuggestedFeeRecipient
proposingSync := stagedsync.New(
cfg.Sync,
- stagedsync.MiningStages(mock.Ctx,
- stagedsync.StageMiningCreateBlockCfg(mock.DB, miner, mock.ChainConfig, mock.Engine, nil, dirs.Tmp, mock.BlockReader),
+ stagedsync.MiningStages(
+ mock.Ctx,
+ stagedsync.StageMiningCreateBlockCfg(miner, mock.ChainConfig, mock.Engine, nil, mock.BlockReader),
stagedsync.StageExecuteBlocksCfg(
mock.DB,
prune,
@@ -494,21 +494,24 @@ func MockWithEverything(tb testing.TB, gspec *types.Genesis, key *ecdsa.PrivateK
&vm.Config{},
mock.Notifications,
cfg.StateStream,
- /*badBlockHalt*/ false,
+ false, /*badBlockHalt*/
dirs,
mock.BlockReader,
mock.sentriesClient.Hd,
gspec,
cfg.Sync,
nil,
- /*experimentalBAL*/ false,
+ false, /*experimentalBAL*/
),
- stagedsync.StageSendersCfg(mock.DB, mock.ChainConfig, cfg.Sync, false, dirs.Tmp, prune, mock.BlockReader, mock.sentriesClient.Hd),
- stagedsync.StageMiningExecCfg(mock.DB, miner, nil, mock.ChainConfig, mock.Engine, &vm.Config{}, dirs.Tmp, nil, 0, mock.TxPool, mock.BlockReader),
- stagedsync.StageMiningFinishCfg(mock.DB, mock.ChainConfig, mock.Engine, miner, miningCancel, mock.BlockReader, latestBlockBuiltStore),
- false,
- ), stagedsync.MiningUnwindOrder, stagedsync.MiningPruneOrder,
- logger, stages.ModeBlockProduction)
+ stagedsync.StageSendersCfg(mock.ChainConfig, cfg.Sync, false /* badBlockHalt */, dirs.Tmp, prune, mock.BlockReader, mock.sentriesClient.Hd),
+ stagedsync.StageMiningExecCfg(miner, nil, mock.ChainConfig, mock.Engine, &vm.Config{}, dirs.Tmp, nil, 0, mock.TxPool, mock.BlockReader),
+ stagedsync.StageMiningFinishCfg(mock.ChainConfig, mock.Engine, miner, miningCancel, mock.BlockReader, latestBlockBuiltStore),
+ ),
+ stagedsync.MiningUnwindOrder,
+ stagedsync.MiningPruneOrder,
+ logger,
+ stages.ModeBlockProduction,
+ )
// We start the mining step
if err := stageloop.MiningStep(ctx, mock.DB, proposingSync, tmpdir, logger); err != nil {
return nil, err
@@ -520,12 +523,14 @@ func MockWithEverything(tb testing.TB, gspec *types.Genesis, key *ecdsa.PrivateK
blockRetire := freezeblocks.NewBlockRetire(1, dirs, mock.BlockReader, blockWriter, mock.DB, nil, nil, mock.ChainConfig, &cfg, mock.Notifications.Events, nil, logger)
mock.Sync = stagedsync.New(
cfg.Sync,
- stagedsync.DefaultStages(mock.Ctx,
+ stagedsync.DefaultStages(
+ mock.Ctx,
stagedsync.StageSnapshotsCfg(mock.DB, mock.ChainConfig, cfg.Sync, dirs, blockRetire, snapDownloader, mock.BlockReader, mock.Notifications, false, false, false, nil, prune),
- stagedsync.StageHeadersCfg(mock.DB, mock.sentriesClient.Hd, mock.sentriesClient.Bd, mock.ChainConfig, cfg.Sync, sendHeaderRequest, propagateNewBlockHashes, penalize, cfg.BatchSize, false, mock.BlockReader, blockWriter, dirs.Tmp, mock.Notifications),
- stagedsync.StageBlockHashesCfg(mock.DB, mock.Dirs.Tmp, mock.ChainConfig, blockWriter),
- stagedsync.StageBodiesCfg(mock.DB, mock.sentriesClient.Bd, sendBodyRequest, penalize, blockPropagator, cfg.Sync.BodyDownloadTimeoutSeconds, mock.ChainConfig, mock.BlockReader, blockWriter),
- stagedsync.StageSendersCfg(mock.DB, mock.ChainConfig, cfg.Sync, false, dirs.Tmp, prune, mock.BlockReader, mock.sentriesClient.Hd), stagedsync.StageExecuteBlocksCfg(
+ stagedsync.StageHeadersCfg(mock.sentriesClient.Hd, mock.ChainConfig, cfg.Sync, sendHeaderRequest, propagateNewBlockHashes, penalize, false /* noP2PDiscovery */, mock.BlockReader),
+ stagedsync.StageBlockHashesCfg(mock.Dirs.Tmp, blockWriter),
+ stagedsync.StageBodiesCfg(mock.sentriesClient.Bd, sendBodyRequest, penalize, blockPropagator, cfg.Sync.BodyDownloadTimeoutSeconds, mock.ChainConfig, mock.BlockReader, blockWriter),
+ stagedsync.StageSendersCfg(mock.ChainConfig, cfg.Sync, false /* badBlockHalt */, dirs.Tmp, prune, mock.BlockReader, mock.sentriesClient.Hd),
+ stagedsync.StageExecuteBlocksCfg(
mock.DB,
prune,
cfg.BatchSize,
@@ -534,18 +539,23 @@ func MockWithEverything(tb testing.TB, gspec *types.Genesis, key *ecdsa.PrivateK
&vm.Config{},
mock.Notifications,
cfg.StateStream,
- /*badBlockHalt*/ false,
+ false, /*badBlockHalt*/
dirs,
mock.BlockReader,
mock.sentriesClient.Hd,
gspec,
cfg.Sync,
nil,
- /*experimentalBAL*/ false,
- ), stagedsync.StageTxLookupCfg(mock.DB, prune, dirs.Tmp, mock.ChainConfig.Bor, mock.BlockReader), stagedsync.StageFinishCfg(mock.DB, dirs.Tmp, forkValidator), !withPosDownloader),
+ false, /*experimentalBAL*/
+ ),
+ stagedsync.StageTxLookupCfg(prune, dirs.Tmp, mock.BlockReader),
+ stagedsync.StageFinishCfg(forkValidator),
+ !withPosDownloader,
+ ),
stagedsync.DefaultUnwindOrder,
stagedsync.DefaultPruneOrder,
- logger, stages.ModeApplyingBlocks,
+ logger,
+ stages.ModeApplyingBlocks,
)
var tracer *tracers.Tracer
@@ -569,8 +579,9 @@ func MockWithEverything(tb testing.TB, gspec *types.Genesis, key *ecdsa.PrivateK
mock.MiningSync = stagedsync.New(
cfg.Sync,
- stagedsync.MiningStages(mock.Ctx,
- stagedsync.StageMiningCreateBlockCfg(mock.DB, miner, mock.ChainConfig, mock.Engine, nil, dirs.Tmp, mock.BlockReader),
+ stagedsync.MiningStages(
+ mock.Ctx,
+ stagedsync.StageMiningCreateBlockCfg(miner, mock.ChainConfig, mock.Engine, nil, mock.BlockReader),
stagedsync.StageExecuteBlocksCfg(
mock.DB,
prune,
@@ -589,14 +600,14 @@ func MockWithEverything(tb testing.TB, gspec *types.Genesis, key *ecdsa.PrivateK
nil,
/*experimentalBAL*/ false,
),
- stagedsync.StageSendersCfg(mock.DB, mock.ChainConfig, cfg.Sync, false, dirs.Tmp, prune, mock.BlockReader, mock.sentriesClient.Hd),
- stagedsync.StageMiningExecCfg(mock.DB, miner, nil, mock.ChainConfig, mock.Engine, &vm.Config{}, dirs.Tmp, nil, 0, mock.TxPool, mock.BlockReader),
- stagedsync.StageMiningFinishCfg(mock.DB, mock.ChainConfig, mock.Engine, miner, miningCancel, mock.BlockReader, latestBlockBuiltStore),
- false,
+ stagedsync.StageSendersCfg(mock.ChainConfig, cfg.Sync, false /* badBlockHalt */, dirs.Tmp, prune, mock.BlockReader, mock.sentriesClient.Hd),
+ stagedsync.StageMiningExecCfg(miner, nil, mock.ChainConfig, mock.Engine, &vm.Config{}, dirs.Tmp, nil, 0, mock.TxPool, mock.BlockReader),
+ stagedsync.StageMiningFinishCfg(mock.ChainConfig, mock.Engine, miner, miningCancel, mock.BlockReader, latestBlockBuiltStore),
),
stagedsync.MiningUnwindOrder,
stagedsync.MiningPruneOrder,
- logger, stages.ModeBlockProduction,
+ logger,
+ stages.ModeBlockProduction,
)
mock.StreamWg.Add(1)
diff --git a/node/eth/backend.go b/node/eth/backend.go
index bbb38fd6ab7..175e66e0a75 100644
--- a/node/eth/backend.go
+++ b/node/eth/backend.go
@@ -37,7 +37,6 @@ import (
"time"
"github.com/erigontech/mdbx-go/mdbx"
- lru "github.com/hashicorp/golang-lru/arc/v2"
"golang.org/x/sync/errgroup"
"golang.org/x/sync/semaphore"
"google.golang.org/grpc"
@@ -96,7 +95,6 @@ import (
"github.com/erigontech/erigon/execution/state/genesiswrite"
"github.com/erigontech/erigon/execution/tracing/tracers"
"github.com/erigontech/erigon/execution/types"
- "github.com/erigontech/erigon/execution/types/accounts"
"github.com/erigontech/erigon/execution/vm"
"github.com/erigontech/erigon/node"
"github.com/erigontech/erigon/node/direct"
@@ -870,23 +868,15 @@ func New(ctx context.Context, stack *node.Node, config *ethconfig.Config, logger
miner := stagedsync.NewMiningState(&config.Miner)
backend.pendingBlocks = miner.PendingResultCh
-
- var signatures *lru.ARCCache[common.Hash, accounts.Address]
-
- if bor, ok := backend.engine.(*bor.Bor); ok {
- signatures = bor.Signatures
- }
-
- astridEnabled := chainConfig.Bor != nil
-
// proof-of-stake mining
assembleBlockPOS := func(param *builder.Parameters, interrupt *atomic.Bool) (*types.BlockWithReceipts, error) {
miningStatePos := stagedsync.NewMiningState(&config.Miner)
miningStatePos.MiningConfig.Etherbase = param.SuggestedFeeRecipient
proposingSync := stagedsync.New(
config.Sync,
- stagedsync.MiningStages(backend.sentryCtx,
- stagedsync.StageMiningCreateBlockCfg(backend.chainDB, miningStatePos, backend.chainConfig, backend.engine, param, tmpdir, backend.blockReader),
+ stagedsync.MiningStages(
+ backend.sentryCtx,
+ stagedsync.StageMiningCreateBlockCfg(miningStatePos, backend.chainConfig, backend.engine, param, backend.blockReader),
stagedsync.StageExecuteBlocksCfg(
backend.chainDB,
config.Prune,
@@ -896,7 +886,7 @@ func New(ctx context.Context, stack *node.Node, config *ethconfig.Config, logger
&vm.Config{},
backend.notifications,
config.StateStream,
- /*badBlockHalt*/ false,
+ false, /*badBlockHalt*/
dirs,
blockReader,
backend.sentriesClient.Hd,
@@ -905,11 +895,15 @@ func New(ctx context.Context, stack *node.Node, config *ethconfig.Config, logger
stageloop.SilkwormForExecutionStage(backend.silkworm, config),
config.ExperimentalBAL,
),
- stagedsync.StageSendersCfg(backend.chainDB, chainConfig, config.Sync, false, dirs.Tmp, config.Prune, blockReader, backend.sentriesClient.Hd),
- stagedsync.StageMiningExecCfg(backend.chainDB, miningStatePos, backend.notifications.Events, backend.chainConfig, backend.engine, &vm.Config{}, tmpdir, interrupt, param.PayloadId, txnProvider, blockReader),
- stagedsync.StageMiningFinishCfg(backend.chainDB, backend.chainConfig, backend.engine, miningStatePos, backend.miningSealingQuit, backend.blockReader, latestBlockBuiltStore),
- astridEnabled,
- ), stagedsync.MiningUnwindOrder, stagedsync.MiningPruneOrder, logger, stages.ModeBlockProduction)
+ stagedsync.StageSendersCfg(chainConfig, config.Sync, false /* badBlockHalt */, dirs.Tmp, config.Prune, blockReader, backend.sentriesClient.Hd),
+ stagedsync.StageMiningExecCfg(miningStatePos, backend.notifications.Events, backend.chainConfig, backend.engine, &vm.Config{}, tmpdir, interrupt, param.PayloadId, txnProvider, blockReader),
+ stagedsync.StageMiningFinishCfg(backend.chainConfig, backend.engine, miningStatePos, backend.miningSealingQuit, backend.blockReader, latestBlockBuiltStore),
+ ),
+ stagedsync.MiningUnwindOrder,
+ stagedsync.MiningPruneOrder,
+ logger,
+ stages.ModeBlockProduction,
+ )
// We start the mining step
if err := stageloop.MiningStep(ctx, backend.chainDB, proposingSync, tmpdir, logger); err != nil {
return nil, err
@@ -984,7 +978,7 @@ func New(ctx context.Context, stack *node.Node, config *ethconfig.Config, logger
}()
backend.syncStages = stageloop.NewDefaultStages(backend.sentryCtx, backend.chainDB, p2pConfig, config, backend.sentriesClient, backend.notifications, backend.downloaderClient,
- blockReader, blockRetire, backend.silkworm, backend.forkValidator, signatures, logger, tracer)
+ blockReader, blockRetire, backend.silkworm, backend.forkValidator, tracer)
backend.syncUnwindOrder = stagedsync.DefaultUnwindOrder
backend.syncPruneOrder = stagedsync.DefaultPruneOrder
diff --git a/rpc/jsonrpc/eth_call.go b/rpc/jsonrpc/eth_call.go
index 7aed309b2f6..02546ea2e05 100644
--- a/rpc/jsonrpc/eth_call.go
+++ b/rpc/jsonrpc/eth_call.go
@@ -708,7 +708,7 @@ func (api *BaseAPI) getWitness(ctx context.Context, db kv.TemporalRoDB, blockNrO
}
// Unwind to blockNr
- cfg := stagedsync.StageWitnessCfg(true, 0, chainConfig, engine, api._blockReader, api.dirs)
+ cfg := stagedsync.StageWitnessCfg(chainConfig, engine, api._blockReader, api.dirs)
err = stagedsync.RewindStagesForWitness(domains, txBatch2, blockNr, latestBlock, &cfg, regenerateHash, ctx, logger)
if err != nil {
return nil, err