beacon, core, eth, miner: integrate witnesses into production Geth (#30069)

This PR integrates witness-enabled block production, witness-creating
payload execution and stateless cross-validation into the `engine` API.
The purpose of the PR is to enable the following use-cases (for API
details, please see next section):

- Cross validating locally created blocks:
- Call `forkchoiceUpdatedWithWitness` instead of `forkchoiceUpdated` to
trigger witness creation too.
- Call `getPayload` as before to retrieve the new block and also the
above created witness.
- Call `executeStatelessPayload` against another client to
cross-validate the block.

- Cross validating locally processed blocks:
- Call `newPayloadWithWitness` instead of `newPayload` to trigger
witness creation too.
- Call `executeStatelessPayload` against another client to
cross-validate the block.

- Block production for stateless clients (local or MEV builders):
- Call `forkchoiceUpdatedWithWitness` instead of `forkchoiceUpdated` to
trigger witness creation too.
- Call `getPayload` as before to retrieve the new block and also the
above created witness.
- Propagate witnesses across the consensus libp2p network for stateless
Ethereum.

- Stateless validator validation:
- Call `executeStatelessPayload` with the propagated witness to
statelessly validate the block.

*Note, the various `WithWitness` methods could also *just be* an
additional boolean flag on the base methods, but this PR wanted to keep
the methods separate until a final consensus is reached on how to
integrate in production.*

---

The following `engine` API types are introduced:

```go
// StatelessPayloadStatusV1 is the result of a stateless payload execution.
type StatelessPayloadStatusV1 struct {
	Status          string      `json:"status"`
	StateRoot       common.Hash `json:"stateRoot"`
	ReceiptsRoot    common.Hash `json:"receiptsRoot"`
	ValidationError *string     `json:"validationError"`
}
```

- Add `forkchoiceUpdatedWithWitnessV1,2,3` with same params and returns
as `forkchoiceUpdatedV1,2,3`, but triggering a stateless witness
building if block production is requested.
- Extend `getPayloadV2,3` to return `executionPayloadEnvelope` with an
additional `witness` field of type `bytes` iff created via
`forkchoiceUpdatedWithWitnessV2,3`.
- Add `newPayloadWithWitnessV1,2,3,4` with same params and returns as
`newPayloadV1,2,3,4`, but triggering a stateless witness creation during
payload execution to allow cross validating it.
- Extend `payloadStatusV1` with a `witness` field of type `bytes` if
returned by `newPayloadWithWitnessV1,2,3,4`.
- Add `executeStatelessPayloadV1,2,3,4` with same base params as
`newPayloadV1,2,3,4` and one more additional param (`witness`) of type
`bytes`. The method returns `statelessPayloadStatusV1`, which mirrors
`payloadStatusV1` but replaces `latestValidHash` with `stateRoot` and
`receiptRoot`.
This commit is contained in:
Péter Szilágyi 2024-09-20 16:43:42 +03:00 committed by GitHub
parent b018da9d02
commit 9326a118c7
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
26 changed files with 669 additions and 353 deletions

@ -19,12 +19,14 @@ func (e ExecutionPayloadEnvelope) MarshalJSON() ([]byte, error) {
BlockValue *hexutil.Big `json:"blockValue" gencodec:"required"` BlockValue *hexutil.Big `json:"blockValue" gencodec:"required"`
BlobsBundle *BlobsBundleV1 `json:"blobsBundle"` BlobsBundle *BlobsBundleV1 `json:"blobsBundle"`
Override bool `json:"shouldOverrideBuilder"` Override bool `json:"shouldOverrideBuilder"`
Witness *hexutil.Bytes `json:"witness"`
} }
var enc ExecutionPayloadEnvelope var enc ExecutionPayloadEnvelope
enc.ExecutionPayload = e.ExecutionPayload enc.ExecutionPayload = e.ExecutionPayload
enc.BlockValue = (*hexutil.Big)(e.BlockValue) enc.BlockValue = (*hexutil.Big)(e.BlockValue)
enc.BlobsBundle = e.BlobsBundle enc.BlobsBundle = e.BlobsBundle
enc.Override = e.Override enc.Override = e.Override
enc.Witness = e.Witness
return json.Marshal(&enc) return json.Marshal(&enc)
} }
@ -35,6 +37,7 @@ func (e *ExecutionPayloadEnvelope) UnmarshalJSON(input []byte) error {
BlockValue *hexutil.Big `json:"blockValue" gencodec:"required"` BlockValue *hexutil.Big `json:"blockValue" gencodec:"required"`
BlobsBundle *BlobsBundleV1 `json:"blobsBundle"` BlobsBundle *BlobsBundleV1 `json:"blobsBundle"`
Override *bool `json:"shouldOverrideBuilder"` Override *bool `json:"shouldOverrideBuilder"`
Witness *hexutil.Bytes `json:"witness"`
} }
var dec ExecutionPayloadEnvelope var dec ExecutionPayloadEnvelope
if err := json.Unmarshal(input, &dec); err != nil { if err := json.Unmarshal(input, &dec); err != nil {
@ -54,5 +57,8 @@ func (e *ExecutionPayloadEnvelope) UnmarshalJSON(input []byte) error {
if dec.Override != nil { if dec.Override != nil {
e.Override = *dec.Override e.Override = *dec.Override
} }
if dec.Witness != nil {
e.Witness = dec.Witness
}
return nil return nil
} }

@ -94,6 +94,14 @@ type executableDataMarshaling struct {
ExcessBlobGas *hexutil.Uint64 ExcessBlobGas *hexutil.Uint64
} }
// StatelessPayloadStatusV1 is the result of a stateless payload execution.
type StatelessPayloadStatusV1 struct {
Status string `json:"status"`
StateRoot common.Hash `json:"stateRoot"`
ReceiptsRoot common.Hash `json:"receiptsRoot"`
ValidationError *string `json:"validationError"`
}
//go:generate go run github.com/fjl/gencodec -type ExecutionPayloadEnvelope -field-override executionPayloadEnvelopeMarshaling -out gen_epe.go //go:generate go run github.com/fjl/gencodec -type ExecutionPayloadEnvelope -field-override executionPayloadEnvelopeMarshaling -out gen_epe.go
type ExecutionPayloadEnvelope struct { type ExecutionPayloadEnvelope struct {
@ -101,6 +109,7 @@ type ExecutionPayloadEnvelope struct {
BlockValue *big.Int `json:"blockValue" gencodec:"required"` BlockValue *big.Int `json:"blockValue" gencodec:"required"`
BlobsBundle *BlobsBundleV1 `json:"blobsBundle"` BlobsBundle *BlobsBundleV1 `json:"blobsBundle"`
Override bool `json:"shouldOverrideBuilder"` Override bool `json:"shouldOverrideBuilder"`
Witness *hexutil.Bytes `json:"witness"`
} }
type BlobsBundleV1 struct { type BlobsBundleV1 struct {
@ -116,6 +125,7 @@ type executionPayloadEnvelopeMarshaling struct {
type PayloadStatusV1 struct { type PayloadStatusV1 struct {
Status string `json:"status"` Status string `json:"status"`
Witness *hexutil.Bytes `json:"witness"`
LatestValidHash *common.Hash `json:"latestValidHash"` LatestValidHash *common.Hash `json:"latestValidHash"`
ValidationError *string `json:"validationError"` ValidationError *string `json:"validationError"`
} }
@ -198,6 +208,20 @@ func decodeTransactions(enc [][]byte) ([]*types.Transaction, error) {
// Withdrawals value will propagate through the returned block. Empty // Withdrawals value will propagate through the returned block. Empty
// Withdrawals value must be passed via non-nil, length 0 value in data. // Withdrawals value must be passed via non-nil, length 0 value in data.
func ExecutableDataToBlock(data ExecutableData, versionedHashes []common.Hash, beaconRoot *common.Hash) (*types.Block, error) { func ExecutableDataToBlock(data ExecutableData, versionedHashes []common.Hash, beaconRoot *common.Hash) (*types.Block, error) {
block, err := ExecutableDataToBlockNoHash(data, versionedHashes, beaconRoot)
if err != nil {
return nil, err
}
if block.Hash() != data.BlockHash {
return nil, fmt.Errorf("blockhash mismatch, want %x, got %x", data.BlockHash, block.Hash())
}
return block, nil
}
// ExecutableDataToBlockNoHash is analogous to ExecutableDataToBlock, but is used
// for stateless execution, so it skips checking if the executable data hashes to
// the requested hash (stateless has to *compute* the root hash, it's not given).
func ExecutableDataToBlockNoHash(data ExecutableData, versionedHashes []common.Hash, beaconRoot *common.Hash) (*types.Block, error) {
txs, err := decodeTransactions(data.Transactions) txs, err := decodeTransactions(data.Transactions)
if err != nil { if err != nil {
return nil, err return nil, err
@ -267,13 +291,10 @@ func ExecutableDataToBlock(data ExecutableData, versionedHashes []common.Hash, b
ParentBeaconRoot: beaconRoot, ParentBeaconRoot: beaconRoot,
RequestsHash: requestsHash, RequestsHash: requestsHash,
} }
block := types.NewBlockWithHeader(header) return types.NewBlockWithHeader(header).
block = block.WithBody(types.Body{Transactions: txs, Uncles: nil, Withdrawals: data.Withdrawals, Requests: requests}) WithBody(types.Body{Transactions: txs, Uncles: nil, Withdrawals: data.Withdrawals, Requests: requests}).
block = block.WithWitness(data.ExecutionWitness) WithWitness(data.ExecutionWitness),
if block.Hash() != data.BlockHash { nil
return nil, fmt.Errorf("blockhash mismatch, want %x, got %x", data.BlockHash, block.Hash())
}
return block, nil
} }
// BlockToExecutableData constructs the ExecutableData structure by filling the // BlockToExecutableData constructs the ExecutableData structure by filling the

@ -156,7 +156,6 @@ var (
utils.BeaconGenesisRootFlag, utils.BeaconGenesisRootFlag,
utils.BeaconGenesisTimeFlag, utils.BeaconGenesisTimeFlag,
utils.BeaconCheckpointFlag, utils.BeaconCheckpointFlag,
utils.CollectWitnessFlag,
}, utils.NetworkFlags, utils.DatabaseFlags) }, utils.NetworkFlags, utils.DatabaseFlags)
rpcFlags = []cli.Flag{ rpcFlags = []cli.Flag{

@ -600,11 +600,6 @@ var (
Usage: "Disables db compaction after import", Usage: "Disables db compaction after import",
Category: flags.LoggingCategory, Category: flags.LoggingCategory,
} }
CollectWitnessFlag = &cli.BoolFlag{
Name: "collectwitness",
Usage: "Enable state witness generation during block execution. Work in progress flag, don't use.",
Category: flags.MiscCategory,
}
// MISC settings // MISC settings
SyncTargetFlag = &cli.StringFlag{ SyncTargetFlag = &cli.StringFlag{
@ -1767,9 +1762,6 @@ func SetEthConfig(ctx *cli.Context, stack *node.Node, cfg *ethconfig.Config) {
// TODO(fjl): force-enable this in --dev mode // TODO(fjl): force-enable this in --dev mode
cfg.EnablePreimageRecording = ctx.Bool(VMEnableDebugFlag.Name) cfg.EnablePreimageRecording = ctx.Bool(VMEnableDebugFlag.Name)
} }
if ctx.IsSet(CollectWitnessFlag.Name) {
cfg.EnableWitnessCollection = ctx.Bool(CollectWitnessFlag.Name)
}
if ctx.IsSet(RPCGlobalGasCapFlag.Name) { if ctx.IsSet(RPCGlobalGasCapFlag.Name) {
cfg.RPCGasCap = ctx.Uint64(RPCGlobalGasCapFlag.Name) cfg.RPCGasCap = ctx.Uint64(RPCGlobalGasCapFlag.Name)
@ -2194,7 +2186,6 @@ func MakeChain(ctx *cli.Context, stack *node.Node, readonly bool) (*core.BlockCh
} }
vmcfg := vm.Config{ vmcfg := vm.Config{
EnablePreimageRecording: ctx.Bool(VMEnableDebugFlag.Name), EnablePreimageRecording: ctx.Bool(VMEnableDebugFlag.Name),
EnableWitnessCollection: ctx.Bool(CollectWitnessFlag.Name),
} }
if ctx.IsSet(VMTraceFlag.Name) { if ctx.IsSet(VMTraceFlag.Name) {
if name := ctx.String(VMTraceFlag.Name); name != "" { if name := ctx.String(VMTraceFlag.Name); name != "" {

@ -20,10 +20,8 @@ import (
"errors" "errors"
"fmt" "fmt"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/consensus" "github.com/ethereum/go-ethereum/consensus"
"github.com/ethereum/go-ethereum/core/state" "github.com/ethereum/go-ethereum/core/state"
"github.com/ethereum/go-ethereum/core/stateless"
"github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/params" "github.com/ethereum/go-ethereum/params"
"github.com/ethereum/go-ethereum/trie" "github.com/ethereum/go-ethereum/trie"
@ -160,28 +158,6 @@ func (v *BlockValidator) ValidateState(block *types.Block, statedb *state.StateD
return nil return nil
} }
// ValidateWitness cross validates a block execution with stateless remote clients.
//
// Normally we'd distribute the block witness to remote cross validators, wait
// for them to respond and then merge the results. For now, however, it's only
// Geth, so do an internal stateless run.
func (v *BlockValidator) ValidateWitness(witness *stateless.Witness, receiptRoot common.Hash, stateRoot common.Hash) error {
// Run the cross client stateless execution
// TODO(karalabe): Self-stateless for now, swap with other clients
crossReceiptRoot, crossStateRoot, err := ExecuteStateless(v.config, witness)
if err != nil {
return fmt.Errorf("stateless execution failed: %v", err)
}
// Stateless cross execution suceeeded, validate the withheld computed fields
if crossReceiptRoot != receiptRoot {
return fmt.Errorf("cross validator receipt root mismatch (cross: %x local: %x)", crossReceiptRoot, receiptRoot)
}
if crossStateRoot != stateRoot {
return fmt.Errorf("cross validator state root mismatch (cross: %x local: %x)", crossStateRoot, stateRoot)
}
return nil
}
// CalcGasLimit computes the gas limit of the next block after parent. It aims // CalcGasLimit computes the gas limit of the next block after parent. It aims
// to keep the baseline gas close to the provided target, and increase it towards // to keep the baseline gas close to the provided target, and increase it towards
// the target if the baseline gas is lower. // the target if the baseline gas is lower.

@ -201,7 +201,7 @@ func testHeaderVerificationForMerging(t *testing.T, isClique bool) {
t.Fatalf("post-block %d: unexpected result returned: %v", i, result) t.Fatalf("post-block %d: unexpected result returned: %v", i, result)
case <-time.After(25 * time.Millisecond): case <-time.After(25 * time.Millisecond):
} }
chain.InsertBlockWithoutSetHead(postBlocks[i]) chain.InsertBlockWithoutSetHead(postBlocks[i], false)
} }
// Verify the blocks with pre-merge blocks and post-merge blocks // Verify the blocks with pre-merge blocks and post-merge blocks

@ -80,6 +80,7 @@ var (
blockInsertTimer = metrics.NewRegisteredResettingTimer("chain/inserts", nil) blockInsertTimer = metrics.NewRegisteredResettingTimer("chain/inserts", nil)
blockValidationTimer = metrics.NewRegisteredResettingTimer("chain/validation", nil) blockValidationTimer = metrics.NewRegisteredResettingTimer("chain/validation", nil)
blockCrossValidationTimer = metrics.NewRegisteredResettingTimer("chain/crossvalidation", nil)
blockExecutionTimer = metrics.NewRegisteredResettingTimer("chain/execution", nil) blockExecutionTimer = metrics.NewRegisteredResettingTimer("chain/execution", nil)
blockWriteTimer = metrics.NewRegisteredResettingTimer("chain/write", nil) blockWriteTimer = metrics.NewRegisteredResettingTimer("chain/write", nil)
@ -1598,7 +1599,9 @@ func (bc *BlockChain) InsertChain(chain types.Blocks) (int, error) {
return 0, errChainStopped return 0, errChainStopped
} }
defer bc.chainmu.Unlock() defer bc.chainmu.Unlock()
return bc.insertChain(chain, true)
_, n, err := bc.insertChain(chain, true, false) // No witness collection for mass inserts (would get super large)
return n, err
} }
// insertChain is the internal implementation of InsertChain, which assumes that // insertChain is the internal implementation of InsertChain, which assumes that
@ -1609,10 +1612,10 @@ func (bc *BlockChain) InsertChain(chain types.Blocks) (int, error) {
// racey behaviour. If a sidechain import is in progress, and the historic state // racey behaviour. If a sidechain import is in progress, and the historic state
// is imported, but then new canon-head is added before the actual sidechain // is imported, but then new canon-head is added before the actual sidechain
// completes, then the historic state could be pruned again // completes, then the historic state could be pruned again
func (bc *BlockChain) insertChain(chain types.Blocks, setHead bool) (int, error) { func (bc *BlockChain) insertChain(chain types.Blocks, setHead bool, makeWitness bool) (*stateless.Witness, int, error) {
// If the chain is terminating, don't even bother starting up. // If the chain is terminating, don't even bother starting up.
if bc.insertStopped() { if bc.insertStopped() {
return 0, nil return nil, 0, nil
} }
// Start a parallel signature recovery (signer will fluke on fork transition, minimal perf loss) // Start a parallel signature recovery (signer will fluke on fork transition, minimal perf loss)
SenderCacher.RecoverFromBlocks(types.MakeSigner(bc.chainConfig, chain[0].Number(), chain[0].Time()), chain) SenderCacher.RecoverFromBlocks(types.MakeSigner(bc.chainConfig, chain[0].Number(), chain[0].Time()), chain)
@ -1667,7 +1670,7 @@ func (bc *BlockChain) insertChain(chain types.Blocks, setHead bool) (int, error)
for block != nil && bc.skipBlock(err, it) { for block != nil && bc.skipBlock(err, it) {
log.Debug("Writing previously known block", "number", block.Number(), "hash", block.Hash()) log.Debug("Writing previously known block", "number", block.Number(), "hash", block.Hash())
if err := bc.writeKnownBlock(block); err != nil { if err := bc.writeKnownBlock(block); err != nil {
return it.index, err return nil, it.index, err
} }
lastCanon = block lastCanon = block
@ -1681,12 +1684,12 @@ func (bc *BlockChain) insertChain(chain types.Blocks, setHead bool) (int, error)
if setHead { if setHead {
// First block is pruned, insert as sidechain and reorg only if TD grows enough // First block is pruned, insert as sidechain and reorg only if TD grows enough
log.Debug("Pruned ancestor, inserting as sidechain", "number", block.Number(), "hash", block.Hash()) log.Debug("Pruned ancestor, inserting as sidechain", "number", block.Number(), "hash", block.Hash())
return bc.insertSideChain(block, it) return bc.insertSideChain(block, it, makeWitness)
} else { } else {
// We're post-merge and the parent is pruned, try to recover the parent state // We're post-merge and the parent is pruned, try to recover the parent state
log.Debug("Pruned ancestor", "number", block.Number(), "hash", block.Hash()) log.Debug("Pruned ancestor", "number", block.Number(), "hash", block.Hash())
_, err := bc.recoverAncestors(block) _, err := bc.recoverAncestors(block, makeWitness)
return it.index, err return nil, it.index, err
} }
// Some other error(except ErrKnownBlock) occurred, abort. // Some other error(except ErrKnownBlock) occurred, abort.
// ErrKnownBlock is allowed here since some known blocks // ErrKnownBlock is allowed here since some known blocks
@ -1694,7 +1697,7 @@ func (bc *BlockChain) insertChain(chain types.Blocks, setHead bool) (int, error)
case err != nil && !errors.Is(err, ErrKnownBlock): case err != nil && !errors.Is(err, ErrKnownBlock):
stats.ignored += len(it.chain) stats.ignored += len(it.chain)
bc.reportBlock(block, nil, err) bc.reportBlock(block, nil, err)
return it.index, err return nil, it.index, err
} }
// No validation errors for the first block (or chain prefix skipped) // No validation errors for the first block (or chain prefix skipped)
var activeState *state.StateDB var activeState *state.StateDB
@ -1708,6 +1711,9 @@ func (bc *BlockChain) insertChain(chain types.Blocks, setHead bool) (int, error)
} }
}() }()
// Track the singleton witness from this chain insertion (if any)
var witness *stateless.Witness
for ; block != nil && err == nil || errors.Is(err, ErrKnownBlock); block, err = it.next() { for ; block != nil && err == nil || errors.Is(err, ErrKnownBlock); block, err = it.next() {
// If the chain is terminating, stop processing blocks // If the chain is terminating, stop processing blocks
if bc.insertStopped() { if bc.insertStopped() {
@ -1744,7 +1750,7 @@ func (bc *BlockChain) insertChain(chain types.Blocks, setHead bool) (int, error)
"hash", block.Hash(), "number", block.NumberU64()) "hash", block.Hash(), "number", block.NumberU64())
} }
if err := bc.writeKnownBlock(block); err != nil { if err := bc.writeKnownBlock(block); err != nil {
return it.index, err return nil, it.index, err
} }
stats.processed++ stats.processed++
if bc.logger != nil && bc.logger.OnSkippedBlock != nil { if bc.logger != nil && bc.logger.OnSkippedBlock != nil {
@ -1755,13 +1761,11 @@ func (bc *BlockChain) insertChain(chain types.Blocks, setHead bool) (int, error)
Safe: bc.CurrentSafeBlock(), Safe: bc.CurrentSafeBlock(),
}) })
} }
// We can assume that logs are empty here, since the only way for consecutive // We can assume that logs are empty here, since the only way for consecutive
// Clique blocks to have the same state is if there are no transactions. // Clique blocks to have the same state is if there are no transactions.
lastCanon = block lastCanon = block
continue continue
} }
// Retrieve the parent block and it's state to execute on top // Retrieve the parent block and it's state to execute on top
start := time.Now() start := time.Now()
parent := it.previous() parent := it.previous()
@ -1770,7 +1774,7 @@ func (bc *BlockChain) insertChain(chain types.Blocks, setHead bool) (int, error)
} }
statedb, err := state.New(parent.Root, bc.statedb) statedb, err := state.New(parent.Root, bc.statedb)
if err != nil { if err != nil {
return it.index, err return nil, it.index, err
} }
statedb.SetLogger(bc.logger) statedb.SetLogger(bc.logger)
@ -1778,11 +1782,13 @@ func (bc *BlockChain) insertChain(chain types.Blocks, setHead bool) (int, error)
// while processing transactions. Before Byzantium the prefetcher is mostly // while processing transactions. Before Byzantium the prefetcher is mostly
// useless due to the intermediate root hashing after each transaction. // useless due to the intermediate root hashing after each transaction.
if bc.chainConfig.IsByzantium(block.Number()) { if bc.chainConfig.IsByzantium(block.Number()) {
var witness *stateless.Witness // Generate witnesses either if we're self-testing, or if it's the
if bc.vmConfig.EnableWitnessCollection { // only block being inserted. A bit crude, but witnesses are huge,
witness, err = stateless.NewWitness(bc, block) // so we refuse to make an entire chain of them.
if bc.vmConfig.StatelessSelfValidation || (makeWitness && len(chain) == 1) {
witness, err = stateless.NewWitness(block.Header(), bc)
if err != nil { if err != nil {
return it.index, err return nil, it.index, err
} }
} }
statedb.StartPrefetcher("chain", witness) statedb.StartPrefetcher("chain", witness)
@ -1814,7 +1820,7 @@ func (bc *BlockChain) insertChain(chain types.Blocks, setHead bool) (int, error)
res, err := bc.processBlock(block, statedb, start, setHead) res, err := bc.processBlock(block, statedb, start, setHead)
followupInterrupt.Store(true) followupInterrupt.Store(true)
if err != nil { if err != nil {
return it.index, err return nil, it.index, err
} }
// Report the import stats before returning the various results // Report the import stats before returning the various results
stats.processed++ stats.processed++
@ -1831,7 +1837,7 @@ func (bc *BlockChain) insertChain(chain types.Blocks, setHead bool) (int, error)
// After merge we expect few side chains. Simply count // After merge we expect few side chains. Simply count
// all blocks the CL gives us for GC processing time // all blocks the CL gives us for GC processing time
bc.gcproc += res.procTime bc.gcproc += res.procTime
return it.index, nil // Direct block insertion of a single block return witness, it.index, nil // Direct block insertion of a single block
} }
switch res.status { switch res.status {
case CanonStatTy: case CanonStatTy:
@ -1861,7 +1867,7 @@ func (bc *BlockChain) insertChain(chain types.Blocks, setHead bool) (int, error)
} }
} }
stats.ignored += it.remaining() stats.ignored += it.remaining()
return it.index, err return witness, it.index, err
} }
// blockProcessingResult is a summary of block processing // blockProcessingResult is a summary of block processing
@ -1906,13 +1912,36 @@ func (bc *BlockChain) processBlock(block *types.Block, statedb *state.StateDB, s
} }
vtime := time.Since(vstart) vtime := time.Since(vstart)
if witness := statedb.Witness(); witness != nil { // If witnesses was generated and stateless self-validation requested, do
if err = bc.validator.ValidateWitness(witness, block.ReceiptHash(), block.Root()); err != nil { // that now. Self validation should *never* run in production, it's more of
bc.reportBlock(block, res, err) // a tight integration to enable running *all* consensus tests through the
return nil, fmt.Errorf("cross verification failed: %v", err) // witness builder/runner, which would otherwise be impossible due to the
// various invalid chain states/behaviors being contained in those tests.
xvstart := time.Now()
if witness := statedb.Witness(); witness != nil && bc.vmConfig.StatelessSelfValidation {
log.Warn("Running stateless self-validation", "block", block.Number(), "hash", block.Hash())
// Remove critical computed fields from the block to force true recalculation
context := block.Header()
context.Root = common.Hash{}
context.ReceiptHash = common.Hash{}
task := types.NewBlockWithHeader(context).WithBody(*block.Body())
// Run the stateless self-cross-validation
crossStateRoot, crossReceiptRoot, err := ExecuteStateless(bc.chainConfig, task, witness)
if err != nil {
return nil, fmt.Errorf("stateless self-validation failed: %v", err)
}
if crossStateRoot != block.Root() {
return nil, fmt.Errorf("stateless self-validation root mismatch (cross: %x local: %x)", crossStateRoot, block.Root())
}
if crossReceiptRoot != block.ReceiptHash() {
return nil, fmt.Errorf("stateless self-validation receipt root mismatch (cross: %x local: %x)", crossReceiptRoot, block.ReceiptHash())
} }
} }
proctime := time.Since(start) // processing + validation xvtime := time.Since(xvstart)
proctime := time.Since(start) // processing + validation + cross validation
// Update the metrics touched during block processing and validation // Update the metrics touched during block processing and validation
accountReadTimer.Update(statedb.AccountReads) // Account reads are complete(in processing) accountReadTimer.Update(statedb.AccountReads) // Account reads are complete(in processing)
@ -1930,6 +1959,7 @@ func (bc *BlockChain) processBlock(block *types.Block, statedb *state.StateDB, s
trieUpdate := statedb.AccountUpdates + statedb.StorageUpdates // The time spent on tries update trieUpdate := statedb.AccountUpdates + statedb.StorageUpdates // The time spent on tries update
blockExecutionTimer.Update(ptime - (statedb.AccountReads + statedb.StorageReads)) // The time spent on EVM processing blockExecutionTimer.Update(ptime - (statedb.AccountReads + statedb.StorageReads)) // The time spent on EVM processing
blockValidationTimer.Update(vtime - (triehash + trieUpdate)) // The time spent on block validation blockValidationTimer.Update(vtime - (triehash + trieUpdate)) // The time spent on block validation
blockCrossValidationTimer.Update(xvtime) // The time spent on stateless cross validation
// Write the block to the chain and get the status. // Write the block to the chain and get the status.
var ( var (
@ -1964,7 +1994,7 @@ func (bc *BlockChain) processBlock(block *types.Block, statedb *state.StateDB, s
// The method writes all (header-and-body-valid) blocks to disk, then tries to // The method writes all (header-and-body-valid) blocks to disk, then tries to
// switch over to the new chain if the TD exceeded the current chain. // switch over to the new chain if the TD exceeded the current chain.
// insertSideChain is only used pre-merge. // insertSideChain is only used pre-merge.
func (bc *BlockChain) insertSideChain(block *types.Block, it *insertIterator) (int, error) { func (bc *BlockChain) insertSideChain(block *types.Block, it *insertIterator, makeWitness bool) (*stateless.Witness, int, error) {
var ( var (
externTd *big.Int externTd *big.Int
current = bc.CurrentBlock() current = bc.CurrentBlock()
@ -2000,7 +2030,7 @@ func (bc *BlockChain) insertSideChain(block *types.Block, it *insertIterator) (i
// If someone legitimately side-mines blocks, they would still be imported as usual. However, // If someone legitimately side-mines blocks, they would still be imported as usual. However,
// we cannot risk writing unverified blocks to disk when they obviously target the pruning // we cannot risk writing unverified blocks to disk when they obviously target the pruning
// mechanism. // mechanism.
return it.index, errors.New("sidechain ghost-state attack") return nil, it.index, errors.New("sidechain ghost-state attack")
} }
} }
if externTd == nil { if externTd == nil {
@ -2011,7 +2041,7 @@ func (bc *BlockChain) insertSideChain(block *types.Block, it *insertIterator) (i
if !bc.HasBlock(block.Hash(), block.NumberU64()) { if !bc.HasBlock(block.Hash(), block.NumberU64()) {
start := time.Now() start := time.Now()
if err := bc.writeBlockWithoutState(block, externTd); err != nil { if err := bc.writeBlockWithoutState(block, externTd); err != nil {
return it.index, err return nil, it.index, err
} }
log.Debug("Injected sidechain block", "number", block.Number(), "hash", block.Hash(), log.Debug("Injected sidechain block", "number", block.Number(), "hash", block.Hash(),
"diff", block.Difficulty(), "elapsed", common.PrettyDuration(time.Since(start)), "diff", block.Difficulty(), "elapsed", common.PrettyDuration(time.Since(start)),
@ -2028,7 +2058,7 @@ func (bc *BlockChain) insertSideChain(block *types.Block, it *insertIterator) (i
for parent != nil && !bc.HasState(parent.Root) { for parent != nil && !bc.HasState(parent.Root) {
if bc.stateRecoverable(parent.Root) { if bc.stateRecoverable(parent.Root) {
if err := bc.triedb.Recover(parent.Root); err != nil { if err := bc.triedb.Recover(parent.Root); err != nil {
return 0, err return nil, 0, err
} }
break break
} }
@ -2038,7 +2068,7 @@ func (bc *BlockChain) insertSideChain(block *types.Block, it *insertIterator) (i
parent = bc.GetHeader(parent.ParentHash, parent.Number.Uint64()-1) parent = bc.GetHeader(parent.ParentHash, parent.Number.Uint64()-1)
} }
if parent == nil { if parent == nil {
return it.index, errors.New("missing parent") return nil, it.index, errors.New("missing parent")
} }
// Import all the pruned blocks to make the state available // Import all the pruned blocks to make the state available
var ( var (
@ -2057,30 +2087,30 @@ func (bc *BlockChain) insertSideChain(block *types.Block, it *insertIterator) (i
// memory here. // memory here.
if len(blocks) >= 2048 || memory > 64*1024*1024 { if len(blocks) >= 2048 || memory > 64*1024*1024 {
log.Info("Importing heavy sidechain segment", "blocks", len(blocks), "start", blocks[0].NumberU64(), "end", block.NumberU64()) log.Info("Importing heavy sidechain segment", "blocks", len(blocks), "start", blocks[0].NumberU64(), "end", block.NumberU64())
if _, err := bc.insertChain(blocks, true); err != nil { if _, _, err := bc.insertChain(blocks, true, false); err != nil {
return 0, err return nil, 0, err
} }
blocks, memory = blocks[:0], 0 blocks, memory = blocks[:0], 0
// If the chain is terminating, stop processing blocks // If the chain is terminating, stop processing blocks
if bc.insertStopped() { if bc.insertStopped() {
log.Debug("Abort during blocks processing") log.Debug("Abort during blocks processing")
return 0, nil return nil, 0, nil
} }
} }
} }
if len(blocks) > 0 { if len(blocks) > 0 {
log.Info("Importing sidechain segment", "start", blocks[0].NumberU64(), "end", blocks[len(blocks)-1].NumberU64()) log.Info("Importing sidechain segment", "start", blocks[0].NumberU64(), "end", blocks[len(blocks)-1].NumberU64())
return bc.insertChain(blocks, true) return bc.insertChain(blocks, true, makeWitness)
} }
return 0, nil return nil, 0, nil
} }
// recoverAncestors finds the closest ancestor with available state and re-execute // recoverAncestors finds the closest ancestor with available state and re-execute
// all the ancestor blocks since that. // all the ancestor blocks since that.
// recoverAncestors is only used post-merge. // recoverAncestors is only used post-merge.
// We return the hash of the latest block that we could correctly validate. // We return the hash of the latest block that we could correctly validate.
func (bc *BlockChain) recoverAncestors(block *types.Block) (common.Hash, error) { func (bc *BlockChain) recoverAncestors(block *types.Block, makeWitness bool) (common.Hash, error) {
// Gather all the sidechain hashes (full blocks may be memory heavy) // Gather all the sidechain hashes (full blocks may be memory heavy)
var ( var (
hashes []common.Hash hashes []common.Hash
@ -2120,7 +2150,7 @@ func (bc *BlockChain) recoverAncestors(block *types.Block) (common.Hash, error)
} else { } else {
b = bc.GetBlock(hashes[i], numbers[i]) b = bc.GetBlock(hashes[i], numbers[i])
} }
if _, err := bc.insertChain(types.Blocks{b}, false); err != nil { if _, _, err := bc.insertChain(types.Blocks{b}, false, makeWitness && i == 0); err != nil {
return b.ParentHash(), err return b.ParentHash(), err
} }
} }
@ -2336,14 +2366,14 @@ func (bc *BlockChain) reorg(oldHead *types.Header, newHead *types.Block) error {
// The key difference between the InsertChain is it won't do the canonical chain // The key difference between the InsertChain is it won't do the canonical chain
// updating. It relies on the additional SetCanonical call to finalize the entire // updating. It relies on the additional SetCanonical call to finalize the entire
// procedure. // procedure.
func (bc *BlockChain) InsertBlockWithoutSetHead(block *types.Block) error { func (bc *BlockChain) InsertBlockWithoutSetHead(block *types.Block, makeWitness bool) (*stateless.Witness, error) {
if !bc.chainmu.TryLock() { if !bc.chainmu.TryLock() {
return errChainStopped return nil, errChainStopped
} }
defer bc.chainmu.Unlock() defer bc.chainmu.Unlock()
_, err := bc.insertChain(types.Blocks{block}, false) witness, _, err := bc.insertChain(types.Blocks{block}, false, makeWitness)
return err return witness, err
} }
// SetCanonical rewinds the chain to set the new head block as the specified // SetCanonical rewinds the chain to set the new head block as the specified
@ -2357,7 +2387,7 @@ func (bc *BlockChain) SetCanonical(head *types.Block) (common.Hash, error) {
// Re-execute the reorged chain in case the head state is missing. // Re-execute the reorged chain in case the head state is missing.
if !bc.HasState(head.Root()) { if !bc.HasState(head.Root()) {
if latestValidHash, err := bc.recoverAncestors(head); err != nil { if latestValidHash, err := bc.recoverAncestors(head, false); err != nil {
return latestValidHash, err return latestValidHash, err
} }
log.Info("Recovered head state", "number", head.Number(), "hash", head.Hash()) log.Info("Recovered head state", "number", head.Number(), "hash", head.Hash())

@ -3674,7 +3674,7 @@ func testSetCanonical(t *testing.T, scheme string) {
gen.AddTx(tx) gen.AddTx(tx)
}) })
for _, block := range side { for _, block := range side {
err := chain.InsertBlockWithoutSetHead(block) _, err := chain.InsertBlockWithoutSetHead(block, false)
if err != nil { if err != nil {
t.Fatalf("Failed to insert into chain: %v", err) t.Fatalf("Failed to insert into chain: %v", err)
} }

@ -25,21 +25,30 @@ import (
"github.com/ethereum/go-ethereum/core/stateless" "github.com/ethereum/go-ethereum/core/stateless"
"github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/core/vm" "github.com/ethereum/go-ethereum/core/vm"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/params" "github.com/ethereum/go-ethereum/params"
"github.com/ethereum/go-ethereum/trie" "github.com/ethereum/go-ethereum/trie"
"github.com/ethereum/go-ethereum/triedb" "github.com/ethereum/go-ethereum/triedb"
) )
// ExecuteStateless runs a stateless execution based on a witness, verifies // ExecuteStateless runs a stateless execution based on a witness, verifies
// everything it can locally and returns the two computed fields that need the // everything it can locally and returns the state root and receipt root, that
// other side to explicitly check. // need the other side to explicitly check.
// //
// This method is a bit of a sore thumb here, but: // This method is a bit of a sore thumb here, but:
// - It cannot be placed in core/stateless, because state.New prodces a circular dep // - It cannot be placed in core/stateless, because state.New prodces a circular dep
// - It cannot be placed outside of core, because it needs to construct a dud headerchain // - It cannot be placed outside of core, because it needs to construct a dud headerchain
// //
// TODO(karalabe): Would be nice to resolve both issues above somehow and move it. // TODO(karalabe): Would be nice to resolve both issues above somehow and move it.
func ExecuteStateless(config *params.ChainConfig, witness *stateless.Witness) (common.Hash, common.Hash, error) { func ExecuteStateless(config *params.ChainConfig, block *types.Block, witness *stateless.Witness) (common.Hash, common.Hash, error) {
// Sanity check if the supplied block accidentally contains a set root or
// receipt hash. If so, be very loud, but still continue.
if block.Root() != (common.Hash{}) {
log.Error("stateless runner received state root it's expected to calculate (faulty consensus client)", "block", block.Number())
}
if block.ReceiptHash() != (common.Hash{}) {
log.Error("stateless runner received receipt root it's expected to calculate (faulty consensus client)", "block", block.Number())
}
// Create and populate the state database to serve as the stateless backend // Create and populate the state database to serve as the stateless backend
memdb := witness.MakeHashDB() memdb := witness.MakeHashDB()
db, err := state.New(witness.Root(), state.NewDatabase(triedb.NewDatabase(memdb, triedb.HashDefaults), nil)) db, err := state.New(witness.Root(), state.NewDatabase(triedb.NewDatabase(memdb, triedb.HashDefaults), nil))
@ -57,16 +66,15 @@ func ExecuteStateless(config *params.ChainConfig, witness *stateless.Witness) (c
validator := NewBlockValidator(config, nil) // No chain, we only validate the state, not the block validator := NewBlockValidator(config, nil) // No chain, we only validate the state, not the block
// Run the stateless blocks processing and self-validate certain fields // Run the stateless blocks processing and self-validate certain fields
res, err := processor.Process(witness.Block, db, vm.Config{}) res, err := processor.Process(block, db, vm.Config{})
if err != nil { if err != nil {
return common.Hash{}, common.Hash{}, err return common.Hash{}, common.Hash{}, err
} }
if err = validator.ValidateState(witness.Block, db, res, true); err != nil { if err = validator.ValidateState(block, db, res, true); err != nil {
return common.Hash{}, common.Hash{}, err return common.Hash{}, common.Hash{}, err
} }
// Almost everything validated, but receipt and state root needs to be returned // Almost everything validated, but receipt and state root needs to be returned
receiptRoot := types.DeriveSha(res.Receipts, trie.NewStackTrie(nil)) receiptRoot := types.DeriveSha(res.Receipts, trie.NewStackTrie(nil))
stateRoot := db.IntermediateRoot(config.IsEIP158(witness.Block.Number())) stateRoot := db.IntermediateRoot(config.IsEIP158(block.Number()))
return stateRoot, receiptRoot, nil
return receiptRoot, stateRoot, nil
} }

@ -26,6 +26,13 @@ import (
// MakeHashDB imports tries, codes and block hashes from a witness into a new // MakeHashDB imports tries, codes and block hashes from a witness into a new
// hash-based memory db. We could eventually rewrite this into a pathdb, but // hash-based memory db. We could eventually rewrite this into a pathdb, but
// simple is better for now. // simple is better for now.
//
// Note, this hashdb approach is quite strictly self-validating:
// - Headers are persisted keyed by hash, so blockhash will error on junk
// - Codes are persisted keyed by hash, so bytecode lookup will error on junk
// - Trie nodes are persisted keyed by hash, so trie expansion will error on junk
//
// Acceleration structures built would need to explicitly validate the witness.
func (w *Witness) MakeHashDB() ethdb.Database { func (w *Witness) MakeHashDB() ethdb.Database {
var ( var (
memdb = rawdb.NewMemoryDatabase() memdb = rawdb.NewMemoryDatabase()

@ -17,42 +17,31 @@
package stateless package stateless
import ( import (
"bytes"
"errors"
"fmt"
"io" "io"
"slices"
"github.com/ethereum/go-ethereum/common/hexutil"
"github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/rlp" "github.com/ethereum/go-ethereum/rlp"
) )
//go:generate go run github.com/fjl/gencodec -type extWitness -field-override extWitnessMarshalling -out gen_encoding_json.go
// toExtWitness converts our internal witness representation to the consensus one. // toExtWitness converts our internal witness representation to the consensus one.
func (w *Witness) toExtWitness() *extWitness { func (w *Witness) toExtWitness() *extWitness {
ext := &extWitness{ ext := &extWitness{
Block: w.Block,
Headers: w.Headers, Headers: w.Headers,
} }
ext.Codes = make([][]byte, 0, len(w.Codes)) ext.Codes = make([][]byte, 0, len(w.Codes))
for code := range w.Codes { for code := range w.Codes {
ext.Codes = append(ext.Codes, []byte(code)) ext.Codes = append(ext.Codes, []byte(code))
} }
slices.SortFunc(ext.Codes, bytes.Compare)
ext.State = make([][]byte, 0, len(w.State)) ext.State = make([][]byte, 0, len(w.State))
for node := range w.State { for node := range w.State {
ext.State = append(ext.State, []byte(node)) ext.State = append(ext.State, []byte(node))
} }
slices.SortFunc(ext.State, bytes.Compare)
return ext return ext
} }
// fromExtWitness converts the consensus witness format into our internal one. // fromExtWitness converts the consensus witness format into our internal one.
func (w *Witness) fromExtWitness(ext *extWitness) error { func (w *Witness) fromExtWitness(ext *extWitness) error {
w.Block, w.Headers = ext.Block, ext.Headers w.Headers = ext.Headers
w.Codes = make(map[string]struct{}, len(ext.Codes)) w.Codes = make(map[string]struct{}, len(ext.Codes))
for _, code := range ext.Codes { for _, code := range ext.Codes {
@ -62,12 +51,7 @@ func (w *Witness) fromExtWitness(ext *extWitness) error {
for _, node := range ext.State { for _, node := range ext.State {
w.State[string(node)] = struct{}{} w.State[string(node)] = struct{}{}
} }
return w.sanitize() return nil
}
// MarshalJSON marshals a witness as JSON.
func (w *Witness) MarshalJSON() ([]byte, error) {
return w.toExtWitness().MarshalJSON()
} }
// EncodeRLP serializes a witness as RLP. // EncodeRLP serializes a witness as RLP.
@ -75,15 +59,6 @@ func (w *Witness) EncodeRLP(wr io.Writer) error {
return rlp.Encode(wr, w.toExtWitness()) return rlp.Encode(wr, w.toExtWitness())
} }
// UnmarshalJSON unmarshals from JSON.
func (w *Witness) UnmarshalJSON(input []byte) error {
var ext extWitness
if err := ext.UnmarshalJSON(input); err != nil {
return err
}
return w.fromExtWitness(&ext)
}
// DecodeRLP decodes a witness from RLP. // DecodeRLP decodes a witness from RLP.
func (w *Witness) DecodeRLP(s *rlp.Stream) error { func (w *Witness) DecodeRLP(s *rlp.Stream) error {
var ext extWitness var ext extWitness
@ -93,37 +68,9 @@ func (w *Witness) DecodeRLP(s *rlp.Stream) error {
return w.fromExtWitness(&ext) return w.fromExtWitness(&ext)
} }
// sanitize checks for some mandatory fields in the witness after decoding so
// the rest of the code can assume invariants and doesn't have to deal with
// corrupted data.
func (w *Witness) sanitize() error {
// Verify that the "parent" header (i.e. index 0) is available, and is the
// true parent of the block-to-be executed, since we use that to link the
// current block to the pre-state.
if len(w.Headers) == 0 {
return errors.New("parent header (for pre-root hash) missing")
}
for i, header := range w.Headers {
if header == nil {
return fmt.Errorf("witness header nil at position %d", i)
}
}
if w.Headers[0].Hash() != w.Block.ParentHash() {
return fmt.Errorf("parent hash different: witness %v, block parent %v", w.Headers[0].Hash(), w.Block.ParentHash())
}
return nil
}
// extWitness is a witness RLP encoding for transferring across clients. // extWitness is a witness RLP encoding for transferring across clients.
type extWitness struct { type extWitness struct {
Block *types.Block `json:"block" gencodec:"required"` Headers []*types.Header
Headers []*types.Header `json:"headers" gencodec:"required"` Codes [][]byte
Codes [][]byte `json:"codes"` State [][]byte
State [][]byte `json:"state"`
}
// extWitnessMarshalling defines the hex marshalling types for a witness.
type extWitnessMarshalling struct {
Codes []hexutil.Bytes
State []hexutil.Bytes
} }

@ -1,74 +0,0 @@
// Code generated by github.com/fjl/gencodec. DO NOT EDIT.
package stateless
import (
"encoding/json"
"errors"
"github.com/ethereum/go-ethereum/common/hexutil"
"github.com/ethereum/go-ethereum/core/types"
)
var _ = (*extWitnessMarshalling)(nil)
// MarshalJSON marshals as JSON.
func (e extWitness) MarshalJSON() ([]byte, error) {
type extWitness struct {
Block *types.Block `json:"block" gencodec:"required"`
Headers []*types.Header `json:"headers" gencodec:"required"`
Codes []hexutil.Bytes `json:"codes"`
State []hexutil.Bytes `json:"state"`
}
var enc extWitness
enc.Block = e.Block
enc.Headers = e.Headers
if e.Codes != nil {
enc.Codes = make([]hexutil.Bytes, len(e.Codes))
for k, v := range e.Codes {
enc.Codes[k] = v
}
}
if e.State != nil {
enc.State = make([]hexutil.Bytes, len(e.State))
for k, v := range e.State {
enc.State[k] = v
}
}
return json.Marshal(&enc)
}
// UnmarshalJSON unmarshals from JSON.
func (e *extWitness) UnmarshalJSON(input []byte) error {
type extWitness struct {
Block *types.Block `json:"block" gencodec:"required"`
Headers []*types.Header `json:"headers" gencodec:"required"`
Codes []hexutil.Bytes `json:"codes"`
State []hexutil.Bytes `json:"state"`
}
var dec extWitness
if err := json.Unmarshal(input, &dec); err != nil {
return err
}
if dec.Block == nil {
return errors.New("missing required field 'block' for extWitness")
}
e.Block = dec.Block
if dec.Headers == nil {
return errors.New("missing required field 'headers' for extWitness")
}
e.Headers = dec.Headers
if dec.Codes != nil {
e.Codes = make([][]byte, len(dec.Codes))
for k, v := range dec.Codes {
e.Codes[k] = v
}
}
if dec.State != nil {
e.State = make([][]byte, len(dec.State))
for k, v := range dec.State {
e.State[k] = v
}
}
return nil
}

@ -17,16 +17,13 @@
package stateless package stateless
import ( import (
"bytes"
"errors" "errors"
"fmt"
"maps" "maps"
"slices" "slices"
"sync" "sync"
"github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/rlp"
) )
// HeaderReader is an interface to pull in headers in place of block hashes for // HeaderReader is an interface to pull in headers in place of block hashes for
@ -36,10 +33,11 @@ type HeaderReader interface {
GetHeader(hash common.Hash, number uint64) *types.Header GetHeader(hash common.Hash, number uint64) *types.Header
} }
// Witness encompasses a block, state and any other chain data required to apply // Witness encompasses the state required to apply a set of transactions and
// a set of transactions and derive a post state/receipt root. // derive a post state/receipt root.
type Witness struct { type Witness struct {
Block *types.Block // Current block with rootHash and receiptHash zeroed out context *types.Header // Header to which this witness belongs to, with rootHash and receiptHash zeroed out
Headers []*types.Header // Past headers in reverse order (0=parent, 1=parent's-parent, etc). First *must* be set. Headers []*types.Header // Past headers in reverse order (0=parent, 1=parent's-parent, etc). First *must* be set.
Codes map[string]struct{} // Set of bytecodes ran or accessed Codes map[string]struct{} // Set of bytecodes ran or accessed
State map[string]struct{} // Set of MPT state trie nodes (account and storage together) State map[string]struct{} // Set of MPT state trie nodes (account and storage together)
@ -49,24 +47,23 @@ type Witness struct {
} }
// NewWitness creates an empty witness ready for population. // NewWitness creates an empty witness ready for population.
func NewWitness(chain HeaderReader, block *types.Block) (*Witness, error) { func NewWitness(context *types.Header, chain HeaderReader) (*Witness, error) {
// Zero out the result fields to avoid accidentally sending them to the verifier // When building witnesses, retrieve the parent header, which will *always*
header := block.Header() // be included to act as a trustless pre-root hash container
header.Root = common.Hash{} var headers []*types.Header
header.ReceiptHash = common.Hash{} if chain != nil {
parent := chain.GetHeader(context.ParentHash, context.Number.Uint64()-1)
// Retrieve the parent header, which will *always* be included to act as a
// trustless pre-root hash container
parent := chain.GetHeader(block.ParentHash(), block.NumberU64()-1)
if parent == nil { if parent == nil {
return nil, errors.New("failed to retrieve parent header") return nil, errors.New("failed to retrieve parent header")
} }
headers = append(headers, parent)
}
// Create the wtness with a reconstructed gutted out block // Create the wtness with a reconstructed gutted out block
return &Witness{ return &Witness{
Block: types.NewBlockWithHeader(header).WithBody(*block.Body()), context: context,
Headers: headers,
Codes: make(map[string]struct{}), Codes: make(map[string]struct{}),
State: make(map[string]struct{}), State: make(map[string]struct{}),
Headers: []*types.Header{parent},
chain: chain, chain: chain,
}, nil }, nil
} }
@ -76,11 +73,8 @@ func NewWitness(chain HeaderReader, block *types.Block) (*Witness, error) {
// the chain to cover the block being added. // the chain to cover the block being added.
func (w *Witness) AddBlockHash(number uint64) { func (w *Witness) AddBlockHash(number uint64) {
// Keep pulling in headers until this hash is populated // Keep pulling in headers until this hash is populated
for int(w.Block.NumberU64()-number) > len(w.Headers) { for int(w.context.Number.Uint64()-number) > len(w.Headers) {
tail := w.Block.Header() tail := w.Headers[len(w.Headers)-1]
if len(w.Headers) > 0 {
tail = w.Headers[len(w.Headers)-1]
}
w.Headers = append(w.Headers, w.chain.GetHeader(tail.ParentHash, tail.Number.Uint64()-1)) w.Headers = append(w.Headers, w.chain.GetHeader(tail.ParentHash, tail.Number.Uint64()-1))
} }
} }
@ -107,45 +101,16 @@ func (w *Witness) AddState(nodes map[string]struct{}) {
// Copy deep-copies the witness object. Witness.Block isn't deep-copied as it // Copy deep-copies the witness object. Witness.Block isn't deep-copied as it
// is never mutated by Witness // is never mutated by Witness
func (w *Witness) Copy() *Witness { func (w *Witness) Copy() *Witness {
return &Witness{ cpy := &Witness{
Block: w.Block,
Headers: slices.Clone(w.Headers), Headers: slices.Clone(w.Headers),
Codes: maps.Clone(w.Codes), Codes: maps.Clone(w.Codes),
State: maps.Clone(w.State), State: maps.Clone(w.State),
chain: w.chain,
} }
} if w.context != nil {
cpy.context = types.CopyHeader(w.context)
// String prints a human-readable summary containing the total size of the
// witness and the sizes of the underlying components
func (w *Witness) String() string {
blob, _ := rlp.EncodeToBytes(w)
bytesTotal := len(blob)
blob, _ = rlp.EncodeToBytes(w.Block)
bytesBlock := len(blob)
bytesHeaders := 0
for _, header := range w.Headers {
blob, _ = rlp.EncodeToBytes(header)
bytesHeaders += len(blob)
} }
bytesCodes := 0 return cpy
for code := range w.Codes {
bytesCodes += len(code)
}
bytesState := 0
for node := range w.State {
bytesState += len(node)
}
buf := new(bytes.Buffer)
fmt.Fprintf(buf, "Witness #%d: %v\n", w.Block.Number(), common.StorageSize(bytesTotal))
fmt.Fprintf(buf, " block (%4d txs): %10v\n", len(w.Block.Transactions()), common.StorageSize(bytesBlock))
fmt.Fprintf(buf, "%4d headers: %10v\n", len(w.Headers), common.StorageSize(bytesHeaders))
fmt.Fprintf(buf, "%4d trie nodes: %10v\n", len(w.State), common.StorageSize(bytesState))
fmt.Fprintf(buf, "%4d codes: %10v\n", len(w.Codes), common.StorageSize(bytesCodes))
return buf.String()
} }
// Root returns the pre-state root from the first header. // Root returns the pre-state root from the first header.

@ -19,9 +19,7 @@ package core
import ( import (
"sync/atomic" "sync/atomic"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/state" "github.com/ethereum/go-ethereum/core/state"
"github.com/ethereum/go-ethereum/core/stateless"
"github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/core/vm" "github.com/ethereum/go-ethereum/core/vm"
) )
@ -35,9 +33,6 @@ type Validator interface {
// ValidateState validates the given statedb and optionally the process result. // ValidateState validates the given statedb and optionally the process result.
ValidateState(block *types.Block, state *state.StateDB, res *ProcessResult, stateless bool) error ValidateState(block *types.Block, state *state.StateDB, res *ProcessResult, stateless bool) error
// ValidateWitness cross validates a block execution with stateless remote clients.
ValidateWitness(witness *stateless.Witness, receiptRoot common.Hash, stateRoot common.Hash) error
} }
// Prefetcher is an interface for pre-caching transaction signatures and state. // Prefetcher is an interface for pre-caching transaction signatures and state.

@ -33,7 +33,8 @@ type Config struct {
NoBaseFee bool // Forces the EIP-1559 baseFee to 0 (needed for 0 price calls) NoBaseFee bool // Forces the EIP-1559 baseFee to 0 (needed for 0 price calls)
EnablePreimageRecording bool // Enables recording of SHA3/keccak preimages EnablePreimageRecording bool // Enables recording of SHA3/keccak preimages
ExtraEips []int // Additional EIPS that are to be enabled ExtraEips []int // Additional EIPS that are to be enabled
EnableWitnessCollection bool // true if witness collection is enabled
StatelessSelfValidation bool // Generate execution witnesses and self-check against them (testing purpose)
} }
// ScopeContext contains the things that are per-call, such as stack and memory, // ScopeContext contains the things that are per-call, such as stack and memory,

@ -183,7 +183,6 @@ func New(stack *node.Node, config *ethconfig.Config) (*Ethereum, error) {
var ( var (
vmConfig = vm.Config{ vmConfig = vm.Config{
EnablePreimageRecording: config.EnablePreimageRecording, EnablePreimageRecording: config.EnablePreimageRecording,
EnableWitnessCollection: config.EnableWitnessCollection,
} }
cacheConfig = &core.CacheConfig{ cacheConfig = &core.CacheConfig{
TrieCleanLimit: config.TrieCleanCache, TrieCleanLimit: config.TrieCleanCache,

@ -27,7 +27,9 @@ import (
"github.com/ethereum/go-ethereum/beacon/engine" "github.com/ethereum/go-ethereum/beacon/engine"
"github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/common/hexutil" "github.com/ethereum/go-ethereum/common/hexutil"
"github.com/ethereum/go-ethereum/core"
"github.com/ethereum/go-ethereum/core/rawdb" "github.com/ethereum/go-ethereum/core/rawdb"
"github.com/ethereum/go-ethereum/core/stateless"
"github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/eth" "github.com/ethereum/go-ethereum/eth"
"github.com/ethereum/go-ethereum/eth/downloader" "github.com/ethereum/go-ethereum/eth/downloader"
@ -37,6 +39,7 @@ import (
"github.com/ethereum/go-ethereum/node" "github.com/ethereum/go-ethereum/node"
"github.com/ethereum/go-ethereum/params" "github.com/ethereum/go-ethereum/params"
"github.com/ethereum/go-ethereum/params/forks" "github.com/ethereum/go-ethereum/params/forks"
"github.com/ethereum/go-ethereum/rlp"
"github.com/ethereum/go-ethereum/rpc" "github.com/ethereum/go-ethereum/rpc"
) )
@ -82,6 +85,9 @@ var caps = []string{
"engine_forkchoiceUpdatedV1", "engine_forkchoiceUpdatedV1",
"engine_forkchoiceUpdatedV2", "engine_forkchoiceUpdatedV2",
"engine_forkchoiceUpdatedV3", "engine_forkchoiceUpdatedV3",
"engine_forkchoiceUpdatedWithWitnessV1",
"engine_forkchoiceUpdatedWithWitnessV2",
"engine_forkchoiceUpdatedWithWitnessV3",
"engine_exchangeTransitionConfigurationV1", "engine_exchangeTransitionConfigurationV1",
"engine_getPayloadV1", "engine_getPayloadV1",
"engine_getPayloadV2", "engine_getPayloadV2",
@ -91,6 +97,14 @@ var caps = []string{
"engine_newPayloadV2", "engine_newPayloadV2",
"engine_newPayloadV3", "engine_newPayloadV3",
"engine_newPayloadV4", "engine_newPayloadV4",
"engine_newPayloadWithWitnessV1",
"engine_newPayloadWithWitnessV2",
"engine_newPayloadWithWitnessV3",
"engine_newPayloadWithWitnessV4",
"engine_executeStatelessPayloadV1",
"engine_executeStatelessPayloadV2",
"engine_executeStatelessPayloadV3",
"engine_executeStatelessPayloadV4",
"engine_getPayloadBodiesByHashV1", "engine_getPayloadBodiesByHashV1",
"engine_getPayloadBodiesByHashV2", "engine_getPayloadBodiesByHashV2",
"engine_getPayloadBodiesByRangeV1", "engine_getPayloadBodiesByRangeV1",
@ -188,7 +202,7 @@ func (api *ConsensusAPI) ForkchoiceUpdatedV1(update engine.ForkchoiceStateV1, pa
return engine.STATUS_INVALID, engine.InvalidParams.With(errors.New("forkChoiceUpdateV1 called post-shanghai")) return engine.STATUS_INVALID, engine.InvalidParams.With(errors.New("forkChoiceUpdateV1 called post-shanghai"))
} }
} }
return api.forkchoiceUpdated(update, payloadAttributes, engine.PayloadV1) return api.forkchoiceUpdated(update, payloadAttributes, engine.PayloadV1, false)
} }
// ForkchoiceUpdatedV2 is equivalent to V1 with the addition of withdrawals in the payload // ForkchoiceUpdatedV2 is equivalent to V1 with the addition of withdrawals in the payload
@ -211,7 +225,7 @@ func (api *ConsensusAPI) ForkchoiceUpdatedV2(update engine.ForkchoiceStateV1, pa
return engine.STATUS_INVALID, engine.UnsupportedFork.With(errors.New("forkchoiceUpdatedV2 must only be called with paris and shanghai payloads")) return engine.STATUS_INVALID, engine.UnsupportedFork.With(errors.New("forkchoiceUpdatedV2 must only be called with paris and shanghai payloads"))
} }
} }
return api.forkchoiceUpdated(update, params, engine.PayloadV2) return api.forkchoiceUpdated(update, params, engine.PayloadV2, false)
} }
// ForkchoiceUpdatedV3 is equivalent to V2 with the addition of parent beacon block root // ForkchoiceUpdatedV3 is equivalent to V2 with the addition of parent beacon block root
@ -232,10 +246,68 @@ func (api *ConsensusAPI) ForkchoiceUpdatedV3(update engine.ForkchoiceStateV1, pa
// hash, even if params are wrong. To do this we need to split up // hash, even if params are wrong. To do this we need to split up
// forkchoiceUpdate into a function that only updates the head and then a // forkchoiceUpdate into a function that only updates the head and then a
// function that kicks off block construction. // function that kicks off block construction.
return api.forkchoiceUpdated(update, params, engine.PayloadV3) return api.forkchoiceUpdated(update, params, engine.PayloadV3, false)
} }
func (api *ConsensusAPI) forkchoiceUpdated(update engine.ForkchoiceStateV1, payloadAttributes *engine.PayloadAttributes, payloadVersion engine.PayloadVersion) (engine.ForkChoiceResponse, error) { // ForkchoiceUpdatedWithWitnessV1 is analogous to ForkchoiceUpdatedV1, only it
// generates an execution witness too if block building was requested.
func (api *ConsensusAPI) ForkchoiceUpdatedWithWitnessV1(update engine.ForkchoiceStateV1, payloadAttributes *engine.PayloadAttributes) (engine.ForkChoiceResponse, error) {
if payloadAttributes != nil {
if payloadAttributes.Withdrawals != nil || payloadAttributes.BeaconRoot != nil {
return engine.STATUS_INVALID, engine.InvalidParams.With(errors.New("withdrawals and beacon root not supported in V1"))
}
if api.eth.BlockChain().Config().IsShanghai(api.eth.BlockChain().Config().LondonBlock, payloadAttributes.Timestamp) {
return engine.STATUS_INVALID, engine.InvalidParams.With(errors.New("forkChoiceUpdateV1 called post-shanghai"))
}
}
return api.forkchoiceUpdated(update, payloadAttributes, engine.PayloadV1, true)
}
// ForkchoiceUpdatedWithWitnessV2 is analogous to ForkchoiceUpdatedV2, only it
// generates an execution witness too if block building was requested.
func (api *ConsensusAPI) ForkchoiceUpdatedWithWitnessV2(update engine.ForkchoiceStateV1, params *engine.PayloadAttributes) (engine.ForkChoiceResponse, error) {
if params != nil {
if params.BeaconRoot != nil {
return engine.STATUS_INVALID, engine.InvalidPayloadAttributes.With(errors.New("unexpected beacon root"))
}
switch api.eth.BlockChain().Config().LatestFork(params.Timestamp) {
case forks.Paris:
if params.Withdrawals != nil {
return engine.STATUS_INVALID, engine.InvalidPayloadAttributes.With(errors.New("withdrawals before shanghai"))
}
case forks.Shanghai:
if params.Withdrawals == nil {
return engine.STATUS_INVALID, engine.InvalidPayloadAttributes.With(errors.New("missing withdrawals"))
}
default:
return engine.STATUS_INVALID, engine.UnsupportedFork.With(errors.New("forkchoiceUpdatedV2 must only be called with paris and shanghai payloads"))
}
}
return api.forkchoiceUpdated(update, params, engine.PayloadV2, true)
}
// ForkchoiceUpdatedWithWitnessV3 is analogous to ForkchoiceUpdatedV3, only it
// generates an execution witness too if block building was requested.
func (api *ConsensusAPI) ForkchoiceUpdatedWithWitnessV3(update engine.ForkchoiceStateV1, params *engine.PayloadAttributes) (engine.ForkChoiceResponse, error) {
if params != nil {
if params.Withdrawals == nil {
return engine.STATUS_INVALID, engine.InvalidPayloadAttributes.With(errors.New("missing withdrawals"))
}
if params.BeaconRoot == nil {
return engine.STATUS_INVALID, engine.InvalidPayloadAttributes.With(errors.New("missing beacon root"))
}
if api.eth.BlockChain().Config().LatestFork(params.Timestamp) != forks.Cancun && api.eth.BlockChain().Config().LatestFork(params.Timestamp) != forks.Prague {
return engine.STATUS_INVALID, engine.UnsupportedFork.With(errors.New("forkchoiceUpdatedV3 must only be called for cancun payloads"))
}
}
// TODO(matt): the spec requires that fcu is applied when called on a valid
// hash, even if params are wrong. To do this we need to split up
// forkchoiceUpdate into a function that only updates the head and then a
// function that kicks off block construction.
return api.forkchoiceUpdated(update, params, engine.PayloadV3, true)
}
func (api *ConsensusAPI) forkchoiceUpdated(update engine.ForkchoiceStateV1, payloadAttributes *engine.PayloadAttributes, payloadVersion engine.PayloadVersion, payloadWitness bool) (engine.ForkChoiceResponse, error) {
api.forkchoiceLock.Lock() api.forkchoiceLock.Lock()
defer api.forkchoiceLock.Unlock() defer api.forkchoiceLock.Unlock()
@ -378,7 +450,7 @@ func (api *ConsensusAPI) forkchoiceUpdated(update engine.ForkchoiceStateV1, payl
if api.localBlocks.has(id) { if api.localBlocks.has(id) {
return valid(&id), nil return valid(&id), nil
} }
payload, err := api.eth.Miner().BuildPayload(args) payload, err := api.eth.Miner().BuildPayload(args, payloadWitness)
if err != nil { if err != nil {
log.Error("Failed to build payload", "err", err) log.Error("Failed to build payload", "err", err)
return valid(nil), engine.InvalidPayloadAttributes.With(err) return valid(nil), engine.InvalidPayloadAttributes.With(err)
@ -469,7 +541,7 @@ func (api *ConsensusAPI) NewPayloadV1(params engine.ExecutableData) (engine.Payl
if params.Withdrawals != nil { if params.Withdrawals != nil {
return engine.PayloadStatusV1{Status: engine.INVALID}, engine.InvalidParams.With(errors.New("withdrawals not supported in V1")) return engine.PayloadStatusV1{Status: engine.INVALID}, engine.InvalidParams.With(errors.New("withdrawals not supported in V1"))
} }
return api.newPayload(params, nil, nil) return api.newPayload(params, nil, nil, false)
} }
// NewPayloadV2 creates an Eth1 block, inserts it in the chain, and returns the status of the chain. // NewPayloadV2 creates an Eth1 block, inserts it in the chain, and returns the status of the chain.
@ -492,7 +564,7 @@ func (api *ConsensusAPI) NewPayloadV2(params engine.ExecutableData) (engine.Payl
if params.BlobGasUsed != nil { if params.BlobGasUsed != nil {
return engine.PayloadStatusV1{Status: engine.INVALID}, engine.InvalidParams.With(errors.New("non-nil blobGasUsed pre-cancun")) return engine.PayloadStatusV1{Status: engine.INVALID}, engine.InvalidParams.With(errors.New("non-nil blobGasUsed pre-cancun"))
} }
return api.newPayload(params, nil, nil) return api.newPayload(params, nil, nil, false)
} }
// NewPayloadV3 creates an Eth1 block, inserts it in the chain, and returns the status of the chain. // NewPayloadV3 creates an Eth1 block, inserts it in the chain, and returns the status of the chain.
@ -517,9 +589,10 @@ func (api *ConsensusAPI) NewPayloadV3(params engine.ExecutableData, versionedHas
if api.eth.BlockChain().Config().LatestFork(params.Timestamp) != forks.Cancun { if api.eth.BlockChain().Config().LatestFork(params.Timestamp) != forks.Cancun {
return engine.PayloadStatusV1{Status: engine.INVALID}, engine.UnsupportedFork.With(errors.New("newPayloadV3 must only be called for cancun payloads")) return engine.PayloadStatusV1{Status: engine.INVALID}, engine.UnsupportedFork.With(errors.New("newPayloadV3 must only be called for cancun payloads"))
} }
return api.newPayload(params, versionedHashes, beaconRoot) return api.newPayload(params, versionedHashes, beaconRoot, false)
} }
// NewPayloadV4 creates an Eth1 block, inserts it in the chain, and returns the status of the chain.
// NewPayloadV4 creates an Eth1 block, inserts it in the chain, and returns the status of the chain. // NewPayloadV4 creates an Eth1 block, inserts it in the chain, and returns the status of the chain.
func (api *ConsensusAPI) NewPayloadV4(params engine.ExecutableData, versionedHashes []common.Hash, beaconRoot *common.Hash) (engine.PayloadStatusV1, error) { func (api *ConsensusAPI) NewPayloadV4(params engine.ExecutableData, versionedHashes []common.Hash, beaconRoot *common.Hash) (engine.PayloadStatusV1, error) {
if params.Withdrawals == nil { if params.Withdrawals == nil {
@ -545,10 +618,186 @@ func (api *ConsensusAPI) NewPayloadV4(params engine.ExecutableData, versionedHas
if api.eth.BlockChain().Config().LatestFork(params.Timestamp) != forks.Prague { if api.eth.BlockChain().Config().LatestFork(params.Timestamp) != forks.Prague {
return engine.PayloadStatusV1{Status: engine.INVALID}, engine.UnsupportedFork.With(errors.New("newPayloadV4 must only be called for prague payloads")) return engine.PayloadStatusV1{Status: engine.INVALID}, engine.UnsupportedFork.With(errors.New("newPayloadV4 must only be called for prague payloads"))
} }
return api.newPayload(params, versionedHashes, beaconRoot) return api.newPayload(params, versionedHashes, beaconRoot, false)
} }
func (api *ConsensusAPI) newPayload(params engine.ExecutableData, versionedHashes []common.Hash, beaconRoot *common.Hash) (engine.PayloadStatusV1, error) { // NewPayloadWithWitnessV1 is analogous to NewPayloadV1, only it also generates
// and returns a stateless witness after running the payload.
func (api *ConsensusAPI) NewPayloadWithWitnessV1(params engine.ExecutableData) (engine.PayloadStatusV1, error) {
if params.Withdrawals != nil {
return engine.PayloadStatusV1{Status: engine.INVALID}, engine.InvalidParams.With(errors.New("withdrawals not supported in V1"))
}
return api.newPayload(params, nil, nil, true)
}
// NewPayloadWithWitnessV2 is analogous to NewPayloadV2, only it also generates
// and returns a stateless witness after running the payload.
func (api *ConsensusAPI) NewPayloadWithWitnessV2(params engine.ExecutableData) (engine.PayloadStatusV1, error) {
if api.eth.BlockChain().Config().IsCancun(api.eth.BlockChain().Config().LondonBlock, params.Timestamp) {
return engine.PayloadStatusV1{Status: engine.INVALID}, engine.InvalidParams.With(errors.New("can't use newPayloadV2 post-cancun"))
}
if api.eth.BlockChain().Config().LatestFork(params.Timestamp) == forks.Shanghai {
if params.Withdrawals == nil {
return engine.PayloadStatusV1{Status: engine.INVALID}, engine.InvalidParams.With(errors.New("nil withdrawals post-shanghai"))
}
} else {
if params.Withdrawals != nil {
return engine.PayloadStatusV1{Status: engine.INVALID}, engine.InvalidParams.With(errors.New("non-nil withdrawals pre-shanghai"))
}
}
if params.ExcessBlobGas != nil {
return engine.PayloadStatusV1{Status: engine.INVALID}, engine.InvalidParams.With(errors.New("non-nil excessBlobGas pre-cancun"))
}
if params.BlobGasUsed != nil {
return engine.PayloadStatusV1{Status: engine.INVALID}, engine.InvalidParams.With(errors.New("non-nil blobGasUsed pre-cancun"))
}
return api.newPayload(params, nil, nil, true)
}
// NewPayloadWithWitnessV3 is analogous to NewPayloadV3, only it also generates
// and returns a stateless witness after running the payload.
func (api *ConsensusAPI) NewPayloadWithWitnessV3(params engine.ExecutableData, versionedHashes []common.Hash, beaconRoot *common.Hash) (engine.PayloadStatusV1, error) {
if params.Withdrawals == nil {
return engine.PayloadStatusV1{Status: engine.INVALID}, engine.InvalidParams.With(errors.New("nil withdrawals post-shanghai"))
}
if params.ExcessBlobGas == nil {
return engine.PayloadStatusV1{Status: engine.INVALID}, engine.InvalidParams.With(errors.New("nil excessBlobGas post-cancun"))
}
if params.BlobGasUsed == nil {
return engine.PayloadStatusV1{Status: engine.INVALID}, engine.InvalidParams.With(errors.New("nil blobGasUsed post-cancun"))
}
if versionedHashes == nil {
return engine.PayloadStatusV1{Status: engine.INVALID}, engine.InvalidParams.With(errors.New("nil versionedHashes post-cancun"))
}
if beaconRoot == nil {
return engine.PayloadStatusV1{Status: engine.INVALID}, engine.InvalidParams.With(errors.New("nil beaconRoot post-cancun"))
}
if api.eth.BlockChain().Config().LatestFork(params.Timestamp) != forks.Cancun {
return engine.PayloadStatusV1{Status: engine.INVALID}, engine.UnsupportedFork.With(errors.New("newPayloadWithWitnessV3 must only be called for cancun payloads"))
}
return api.newPayload(params, versionedHashes, beaconRoot, true)
}
// NewPayloadWithWitnessV4 is analogous to NewPayloadV4, only it also generates
// and returns a stateless witness after running the payload.
func (api *ConsensusAPI) NewPayloadWithWitnessV4(params engine.ExecutableData, versionedHashes []common.Hash, beaconRoot *common.Hash) (engine.PayloadStatusV1, error) {
if params.Withdrawals == nil {
return engine.PayloadStatusV1{Status: engine.INVALID}, engine.InvalidParams.With(errors.New("nil withdrawals post-shanghai"))
}
if params.ExcessBlobGas == nil {
return engine.PayloadStatusV1{Status: engine.INVALID}, engine.InvalidParams.With(errors.New("nil excessBlobGas post-cancun"))
}
if params.BlobGasUsed == nil {
return engine.PayloadStatusV1{Status: engine.INVALID}, engine.InvalidParams.With(errors.New("nil blobGasUsed post-cancun"))
}
if params.Deposits == nil {
return engine.PayloadStatusV1{Status: engine.INVALID}, engine.InvalidParams.With(errors.New("nil deposits post-prague"))
}
if versionedHashes == nil {
return engine.PayloadStatusV1{Status: engine.INVALID}, engine.InvalidParams.With(errors.New("nil versionedHashes post-cancun"))
}
if beaconRoot == nil {
return engine.PayloadStatusV1{Status: engine.INVALID}, engine.InvalidParams.With(errors.New("nil beaconRoot post-cancun"))
}
if api.eth.BlockChain().Config().LatestFork(params.Timestamp) != forks.Prague {
return engine.PayloadStatusV1{Status: engine.INVALID}, engine.UnsupportedFork.With(errors.New("newPayloadWithWitnessV4 must only be called for prague payloads"))
}
return api.newPayload(params, versionedHashes, beaconRoot, true)
}
// ExecuteStatelessPayloadV1 is analogous to NewPayloadV1, only it operates in
// a stateless mode on top of a provided witness instead of the local database.
func (api *ConsensusAPI) ExecuteStatelessPayloadV1(params engine.ExecutableData, opaqueWitness hexutil.Bytes) (engine.StatelessPayloadStatusV1, error) {
if params.Withdrawals != nil {
return engine.StatelessPayloadStatusV1{Status: engine.INVALID}, engine.InvalidParams.With(errors.New("withdrawals not supported in V1"))
}
return api.executeStatelessPayload(params, nil, nil, opaqueWitness)
}
// ExecuteStatelessPayloadV2 is analogous to NewPayloadV2, only it operates in
// a stateless mode on top of a provided witness instead of the local database.
func (api *ConsensusAPI) ExecuteStatelessPayloadV2(params engine.ExecutableData, opaqueWitness hexutil.Bytes) (engine.StatelessPayloadStatusV1, error) {
if api.eth.BlockChain().Config().IsCancun(api.eth.BlockChain().Config().LondonBlock, params.Timestamp) {
return engine.StatelessPayloadStatusV1{Status: engine.INVALID}, engine.InvalidParams.With(errors.New("can't use newPayloadV2 post-cancun"))
}
if api.eth.BlockChain().Config().LatestFork(params.Timestamp) == forks.Shanghai {
if params.Withdrawals == nil {
return engine.StatelessPayloadStatusV1{Status: engine.INVALID}, engine.InvalidParams.With(errors.New("nil withdrawals post-shanghai"))
}
} else {
if params.Withdrawals != nil {
return engine.StatelessPayloadStatusV1{Status: engine.INVALID}, engine.InvalidParams.With(errors.New("non-nil withdrawals pre-shanghai"))
}
}
if params.ExcessBlobGas != nil {
return engine.StatelessPayloadStatusV1{Status: engine.INVALID}, engine.InvalidParams.With(errors.New("non-nil excessBlobGas pre-cancun"))
}
if params.BlobGasUsed != nil {
return engine.StatelessPayloadStatusV1{Status: engine.INVALID}, engine.InvalidParams.With(errors.New("non-nil blobGasUsed pre-cancun"))
}
return api.executeStatelessPayload(params, nil, nil, opaqueWitness)
}
// ExecuteStatelessPayloadV3 is analogous to NewPayloadV3, only it operates in
// a stateless mode on top of a provided witness instead of the local database.
func (api *ConsensusAPI) ExecuteStatelessPayloadV3(params engine.ExecutableData, versionedHashes []common.Hash, beaconRoot *common.Hash, opaqueWitness hexutil.Bytes) (engine.StatelessPayloadStatusV1, error) {
if params.Withdrawals == nil {
return engine.StatelessPayloadStatusV1{Status: engine.INVALID}, engine.InvalidParams.With(errors.New("nil withdrawals post-shanghai"))
}
if params.ExcessBlobGas == nil {
return engine.StatelessPayloadStatusV1{Status: engine.INVALID}, engine.InvalidParams.With(errors.New("nil excessBlobGas post-cancun"))
}
if params.BlobGasUsed == nil {
return engine.StatelessPayloadStatusV1{Status: engine.INVALID}, engine.InvalidParams.With(errors.New("nil blobGasUsed post-cancun"))
}
if versionedHashes == nil {
return engine.StatelessPayloadStatusV1{Status: engine.INVALID}, engine.InvalidParams.With(errors.New("nil versionedHashes post-cancun"))
}
if beaconRoot == nil {
return engine.StatelessPayloadStatusV1{Status: engine.INVALID}, engine.InvalidParams.With(errors.New("nil beaconRoot post-cancun"))
}
if api.eth.BlockChain().Config().LatestFork(params.Timestamp) != forks.Cancun {
return engine.StatelessPayloadStatusV1{Status: engine.INVALID}, engine.UnsupportedFork.With(errors.New("executeStatelessPayloadV3 must only be called for cancun payloads"))
}
return api.executeStatelessPayload(params, versionedHashes, beaconRoot, opaqueWitness)
}
// ExecuteStatelessPayloadV4 is analogous to NewPayloadV4, only it operates in
// a stateless mode on top of a provided witness instead of the local database.
func (api *ConsensusAPI) ExecuteStatelessPayloadV4(params engine.ExecutableData, versionedHashes []common.Hash, beaconRoot *common.Hash, opaqueWitness hexutil.Bytes) (engine.StatelessPayloadStatusV1, error) {
if params.Withdrawals == nil {
return engine.StatelessPayloadStatusV1{Status: engine.INVALID}, engine.InvalidParams.With(errors.New("nil withdrawals post-shanghai"))
}
if params.ExcessBlobGas == nil {
return engine.StatelessPayloadStatusV1{Status: engine.INVALID}, engine.InvalidParams.With(errors.New("nil excessBlobGas post-cancun"))
}
if params.BlobGasUsed == nil {
return engine.StatelessPayloadStatusV1{Status: engine.INVALID}, engine.InvalidParams.With(errors.New("nil blobGasUsed post-cancun"))
}
if params.Deposits == nil {
return engine.StatelessPayloadStatusV1{Status: engine.INVALID}, engine.InvalidParams.With(errors.New("nil deposits post-prague"))
}
if versionedHashes == nil {
return engine.StatelessPayloadStatusV1{Status: engine.INVALID}, engine.InvalidParams.With(errors.New("nil versionedHashes post-cancun"))
}
if beaconRoot == nil {
return engine.StatelessPayloadStatusV1{Status: engine.INVALID}, engine.InvalidParams.With(errors.New("nil beaconRoot post-cancun"))
}
if api.eth.BlockChain().Config().LatestFork(params.Timestamp) != forks.Prague {
return engine.StatelessPayloadStatusV1{Status: engine.INVALID}, engine.UnsupportedFork.With(errors.New("executeStatelessPayloadV4 must only be called for prague payloads"))
}
return api.executeStatelessPayload(params, versionedHashes, beaconRoot, opaqueWitness)
}
func (api *ConsensusAPI) newPayload(params engine.ExecutableData, versionedHashes []common.Hash, beaconRoot *common.Hash, witness bool) (engine.PayloadStatusV1, error) {
// The locking here is, strictly, not required. Without these locks, this can happen: // The locking here is, strictly, not required. Without these locks, this can happen:
// //
// 1. NewPayload( execdata-N ) is invoked from the CL. It goes all the way down to // 1. NewPayload( execdata-N ) is invoked from the CL. It goes all the way down to
@ -656,8 +905,9 @@ func (api *ConsensusAPI) newPayload(params engine.ExecutableData, versionedHashe
return engine.PayloadStatusV1{Status: engine.ACCEPTED}, nil return engine.PayloadStatusV1{Status: engine.ACCEPTED}, nil
} }
log.Trace("Inserting block without sethead", "hash", block.Hash(), "number", block.Number()) log.Trace("Inserting block without sethead", "hash", block.Hash(), "number", block.Number())
if err := api.eth.BlockChain().InsertBlockWithoutSetHead(block); err != nil { proofs, err := api.eth.BlockChain().InsertBlockWithoutSetHead(block, witness)
log.Warn("NewPayloadV1: inserting block failed", "error", err) if err != nil {
log.Warn("NewPayload: inserting block failed", "error", err)
api.invalidLock.Lock() api.invalidLock.Lock()
api.invalidBlocksHits[block.Hash()] = 1 api.invalidBlocksHits[block.Hash()] = 1
@ -667,7 +917,71 @@ func (api *ConsensusAPI) newPayload(params engine.ExecutableData, versionedHashe
return api.invalid(err, parent.Header()), nil return api.invalid(err, parent.Header()), nil
} }
hash := block.Hash() hash := block.Hash()
return engine.PayloadStatusV1{Status: engine.VALID, LatestValidHash: &hash}, nil
// If witness collection was requested, inject that into the result too
var ow *hexutil.Bytes
if proofs != nil {
ow = new(hexutil.Bytes)
*ow, _ = rlp.EncodeToBytes(proofs)
}
return engine.PayloadStatusV1{Status: engine.VALID, Witness: ow, LatestValidHash: &hash}, nil
}
func (api *ConsensusAPI) executeStatelessPayload(params engine.ExecutableData, versionedHashes []common.Hash, beaconRoot *common.Hash, opaqueWitness hexutil.Bytes) (engine.StatelessPayloadStatusV1, error) {
log.Trace("Engine API request received", "method", "ExecuteStatelessPayload", "number", params.Number, "hash", params.BlockHash)
block, err := engine.ExecutableDataToBlockNoHash(params, versionedHashes, beaconRoot)
if err != nil {
bgu := "nil"
if params.BlobGasUsed != nil {
bgu = strconv.Itoa(int(*params.BlobGasUsed))
}
ebg := "nil"
if params.ExcessBlobGas != nil {
ebg = strconv.Itoa(int(*params.ExcessBlobGas))
}
log.Warn("Invalid ExecuteStatelessPayload params",
"params.Number", params.Number,
"params.ParentHash", params.ParentHash,
"params.BlockHash", params.BlockHash,
"params.StateRoot", params.StateRoot,
"params.FeeRecipient", params.FeeRecipient,
"params.LogsBloom", common.PrettyBytes(params.LogsBloom),
"params.Random", params.Random,
"params.GasLimit", params.GasLimit,
"params.GasUsed", params.GasUsed,
"params.Timestamp", params.Timestamp,
"params.ExtraData", common.PrettyBytes(params.ExtraData),
"params.BaseFeePerGas", params.BaseFeePerGas,
"params.BlobGasUsed", bgu,
"params.ExcessBlobGas", ebg,
"len(params.Transactions)", len(params.Transactions),
"len(params.Withdrawals)", len(params.Withdrawals),
"len(params.Deposits)", len(params.Deposits),
"beaconRoot", beaconRoot,
"error", err)
errorMsg := err.Error()
return engine.StatelessPayloadStatusV1{Status: engine.INVALID, ValidationError: &errorMsg}, nil
}
witness := new(stateless.Witness)
if err := rlp.DecodeBytes(opaqueWitness, witness); err != nil {
log.Warn("Invalid ExecuteStatelessPayload witness", "err", err)
errorMsg := err.Error()
return engine.StatelessPayloadStatusV1{Status: engine.INVALID, ValidationError: &errorMsg}, nil
}
// Stash away the last update to warn the user if the beacon client goes offline
api.lastNewPayloadLock.Lock()
api.lastNewPayloadUpdate = time.Now()
api.lastNewPayloadLock.Unlock()
log.Trace("Executing block statelessly", "number", block.Number(), "hash", params.BlockHash)
stateRoot, receiptRoot, err := core.ExecuteStateless(api.eth.BlockChain().Config(), block, witness)
if err != nil {
log.Warn("ExecuteStatelessPayload: execution failed", "err", err)
errorMsg := err.Error()
return engine.StatelessPayloadStatusV1{Status: engine.INVALID, ValidationError: &errorMsg}, nil
}
return engine.StatelessPayloadStatusV1{Status: engine.VALID, StateRoot: stateRoot, ReceiptsRoot: receiptRoot}, nil
} }
// delayPayloadImport stashes the given block away for import at a later time, // delayPayloadImport stashes the given block away for import at a later time,

@ -507,7 +507,7 @@ func setupBlocks(t *testing.T, ethservice *eth.Ethereum, n int, parent *types.He
} }
payload := getNewPayload(t, api, parent, w, h) payload := getNewPayload(t, api, parent, w, h)
execResp, err := api.newPayload(*payload, []common.Hash{}, h) execResp, err := api.newPayload(*payload, []common.Hash{}, h, false)
if err != nil { if err != nil {
t.Fatalf("can't execute payload: %v", err) t.Fatalf("can't execute payload: %v", err)
} }
@ -684,7 +684,7 @@ func assembleBlock(api *ConsensusAPI, parentHash common.Hash, params *engine.Pay
Withdrawals: params.Withdrawals, Withdrawals: params.Withdrawals,
BeaconRoot: params.BeaconRoot, BeaconRoot: params.BeaconRoot,
} }
payload, err := api.eth.Miner().BuildPayload(args) payload, err := api.eth.Miner().BuildPayload(args, false)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -922,7 +922,7 @@ func TestNewPayloadOnInvalidTerminalBlock(t *testing.T) {
Random: crypto.Keccak256Hash([]byte{byte(1)}), Random: crypto.Keccak256Hash([]byte{byte(1)}),
FeeRecipient: parent.Coinbase(), FeeRecipient: parent.Coinbase(),
} }
payload, err := api.eth.Miner().BuildPayload(args) payload, err := api.eth.Miner().BuildPayload(args, false)
if err != nil { if err != nil {
t.Fatalf("error preparing payload, err=%v", err) t.Fatalf("error preparing payload, err=%v", err)
} }
@ -1704,6 +1704,108 @@ func TestParentBeaconBlockRoot(t *testing.T) {
} }
} }
func TestWitnessCreationAndConsumption(t *testing.T) {
log.SetDefault(log.NewLogger(log.NewTerminalHandlerWithLevel(colorable.NewColorableStderr(), log.LevelTrace, true)))
genesis, blocks := generateMergeChain(10, true)
// Set cancun time to semi-last block + 5 seconds
timestamp := blocks[len(blocks)-2].Time() + 5
genesis.Config.ShanghaiTime = &timestamp
genesis.Config.CancunTime = &timestamp
n, ethservice := startEthService(t, genesis, blocks[:9])
defer n.Close()
api := NewConsensusAPI(ethservice)
// Put the 10th block's tx in the pool and produce a new block
txs := blocks[9].Transactions()
ethservice.TxPool().Add(txs, true, true)
blockParams := engine.PayloadAttributes{
Timestamp: blocks[8].Time() + 5,
Withdrawals: make([]*types.Withdrawal, 0),
BeaconRoot: &common.Hash{42},
}
fcState := engine.ForkchoiceStateV1{
HeadBlockHash: blocks[8].Hash(),
SafeBlockHash: common.Hash{},
FinalizedBlockHash: common.Hash{},
}
_, err := api.ForkchoiceUpdatedWithWitnessV3(fcState, &blockParams)
if err != nil {
t.Fatalf("error preparing payload, err=%v", err)
}
// Give the payload some time to be built
time.Sleep(100 * time.Millisecond)
payloadID := (&miner.BuildPayloadArgs{
Parent: fcState.HeadBlockHash,
Timestamp: blockParams.Timestamp,
FeeRecipient: blockParams.SuggestedFeeRecipient,
Random: blockParams.Random,
Withdrawals: blockParams.Withdrawals,
BeaconRoot: blockParams.BeaconRoot,
Version: engine.PayloadV3,
}).Id()
envelope, err := api.GetPayloadV3(payloadID)
if err != nil {
t.Fatalf("error getting payload, err=%v", err)
}
if len(envelope.ExecutionPayload.Transactions) != blocks[9].Transactions().Len() {
t.Fatalf("invalid number of transactions %d != %d", len(envelope.ExecutionPayload.Transactions), blocks[9].Transactions().Len())
}
if envelope.Witness == nil {
t.Fatalf("witness missing from payload")
}
// Test stateless execution of the created witness
wantStateRoot := envelope.ExecutionPayload.StateRoot
wantReceiptRoot := envelope.ExecutionPayload.ReceiptsRoot
envelope.ExecutionPayload.StateRoot = common.Hash{}
envelope.ExecutionPayload.ReceiptsRoot = common.Hash{}
res, err := api.ExecuteStatelessPayloadV3(*envelope.ExecutionPayload, []common.Hash{}, &common.Hash{42}, *envelope.Witness)
if err != nil {
t.Fatalf("error executing stateless payload witness: %v", err)
}
if res.StateRoot != wantStateRoot {
t.Fatalf("stateless state root mismatch: have %v, want %v", res.StateRoot, wantStateRoot)
}
if res.ReceiptsRoot != wantReceiptRoot {
t.Fatalf("stateless receipt root mismatch: have %v, want %v", res.ReceiptsRoot, wantReceiptRoot)
}
// Test block insertion with witness creation
envelope.ExecutionPayload.StateRoot = wantStateRoot
envelope.ExecutionPayload.ReceiptsRoot = wantReceiptRoot
res2, err := api.NewPayloadWithWitnessV3(*envelope.ExecutionPayload, []common.Hash{}, &common.Hash{42})
if err != nil {
t.Fatalf("error executing stateless payload witness: %v", err)
}
if res2.Witness == nil {
t.Fatalf("witness missing from payload")
}
// Test stateless execution of the created witness
wantStateRoot = envelope.ExecutionPayload.StateRoot
wantReceiptRoot = envelope.ExecutionPayload.ReceiptsRoot
envelope.ExecutionPayload.StateRoot = common.Hash{}
envelope.ExecutionPayload.ReceiptsRoot = common.Hash{}
res, err = api.ExecuteStatelessPayloadV3(*envelope.ExecutionPayload, []common.Hash{}, &common.Hash{42}, *res2.Witness)
if err != nil {
t.Fatalf("error executing stateless payload witness: %v", err)
}
if res.StateRoot != wantStateRoot {
t.Fatalf("stateless state root mismatch: have %v, want %v", res.StateRoot, wantStateRoot)
}
if res.ReceiptsRoot != wantReceiptRoot {
t.Fatalf("stateless receipt root mismatch: have %v, want %v", res.ReceiptsRoot, wantReceiptRoot)
}
}
// TestGetClientVersion verifies the expected version info is returned. // TestGetClientVersion verifies the expected version info is returned.
func TestGetClientVersion(t *testing.T) { func TestGetClientVersion(t *testing.T) {
genesis, preMergeBlocks := generateMergeChain(10, false) genesis, preMergeBlocks := generateMergeChain(10, false)

@ -181,7 +181,7 @@ func (c *SimulatedBeacon) sealBlock(withdrawals []*types.Withdrawal, timestamp u
Withdrawals: withdrawals, Withdrawals: withdrawals,
Random: random, Random: random,
BeaconRoot: &common.Hash{}, BeaconRoot: &common.Hash{},
}, engine.PayloadV3) }, engine.PayloadV3, false)
if err != nil { if err != nil {
return err return err
} }

@ -134,9 +134,6 @@ type Config struct {
// Enables tracking of SHA3 preimages in the VM // Enables tracking of SHA3 preimages in the VM
EnablePreimageRecording bool EnablePreimageRecording bool
// Enables prefetching trie nodes for read operations too
EnableWitnessCollection bool `toml:"-"`
// Enables VM tracing // Enables VM tracing
VMTrace string VMTrace string
VMTraceJsonConfig string VMTraceJsonConfig string

@ -44,7 +44,6 @@ func (c Config) MarshalTOML() (interface{}, error) {
BlobPool blobpool.Config BlobPool blobpool.Config
GPO gasprice.Config GPO gasprice.Config
EnablePreimageRecording bool EnablePreimageRecording bool
EnableWitnessCollection bool `toml:"-"`
VMTrace string VMTrace string
VMTraceJsonConfig string VMTraceJsonConfig string
DocRoot string `toml:"-"` DocRoot string `toml:"-"`
@ -82,7 +81,6 @@ func (c Config) MarshalTOML() (interface{}, error) {
enc.BlobPool = c.BlobPool enc.BlobPool = c.BlobPool
enc.GPO = c.GPO enc.GPO = c.GPO
enc.EnablePreimageRecording = c.EnablePreimageRecording enc.EnablePreimageRecording = c.EnablePreimageRecording
enc.EnableWitnessCollection = c.EnableWitnessCollection
enc.VMTrace = c.VMTrace enc.VMTrace = c.VMTrace
enc.VMTraceJsonConfig = c.VMTraceJsonConfig enc.VMTraceJsonConfig = c.VMTraceJsonConfig
enc.DocRoot = c.DocRoot enc.DocRoot = c.DocRoot
@ -124,7 +122,6 @@ func (c *Config) UnmarshalTOML(unmarshal func(interface{}) error) error {
BlobPool *blobpool.Config BlobPool *blobpool.Config
GPO *gasprice.Config GPO *gasprice.Config
EnablePreimageRecording *bool EnablePreimageRecording *bool
EnableWitnessCollection *bool `toml:"-"`
VMTrace *string VMTrace *string
VMTraceJsonConfig *string VMTraceJsonConfig *string
DocRoot *string `toml:"-"` DocRoot *string `toml:"-"`
@ -219,9 +216,6 @@ func (c *Config) UnmarshalTOML(unmarshal func(interface{}) error) error {
if dec.EnablePreimageRecording != nil { if dec.EnablePreimageRecording != nil {
c.EnablePreimageRecording = *dec.EnablePreimageRecording c.EnablePreimageRecording = *dec.EnablePreimageRecording
} }
if dec.EnableWitnessCollection != nil {
c.EnableWitnessCollection = *dec.EnableWitnessCollection
}
if dec.VMTrace != nil { if dec.VMTrace != nil {
c.VMTrace = *dec.VMTrace c.VMTrace = *dec.VMTrace
} }

@ -126,8 +126,8 @@ func (miner *Miner) SetGasTip(tip *big.Int) error {
} }
// BuildPayload builds the payload according to the provided parameters. // BuildPayload builds the payload according to the provided parameters.
func (miner *Miner) BuildPayload(args *BuildPayloadArgs) (*Payload, error) { func (miner *Miner) BuildPayload(args *BuildPayloadArgs, witness bool) (*Payload, error) {
return miner.buildPayload(args) return miner.buildPayload(args, witness)
} }
// getPending retrieves the pending block based on the current head block. // getPending retrieves the pending block based on the current head block.
@ -156,7 +156,7 @@ func (miner *Miner) getPending() *newPayloadResult {
withdrawals: withdrawal, withdrawals: withdrawal,
beaconRoot: nil, beaconRoot: nil,
noTxs: false, noTxs: false,
}) }, false) // we will never make a witness for a pending block
if ret.err != nil { if ret.err != nil {
return nil return nil
} }

@ -25,6 +25,8 @@ import (
"github.com/ethereum/go-ethereum/beacon/engine" "github.com/ethereum/go-ethereum/beacon/engine"
"github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/common/hexutil"
"github.com/ethereum/go-ethereum/core/stateless"
"github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/log" "github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/params" "github.com/ethereum/go-ethereum/params"
@ -69,7 +71,9 @@ func (args *BuildPayloadArgs) Id() engine.PayloadID {
type Payload struct { type Payload struct {
id engine.PayloadID id engine.PayloadID
empty *types.Block empty *types.Block
emptyWitness *stateless.Witness
full *types.Block full *types.Block
fullWitness *stateless.Witness
sidecars []*types.BlobTxSidecar sidecars []*types.BlobTxSidecar
fullFees *big.Int fullFees *big.Int
stop chan struct{} stop chan struct{}
@ -78,10 +82,11 @@ type Payload struct {
} }
// newPayload initializes the payload object. // newPayload initializes the payload object.
func newPayload(empty *types.Block, id engine.PayloadID) *Payload { func newPayload(empty *types.Block, witness *stateless.Witness, id engine.PayloadID) *Payload {
payload := &Payload{ payload := &Payload{
id: id, id: id,
empty: empty, empty: empty,
emptyWitness: witness,
stop: make(chan struct{}), stop: make(chan struct{}),
} }
log.Info("Starting work on payload", "id", payload.id) log.Info("Starting work on payload", "id", payload.id)
@ -106,6 +111,7 @@ func (payload *Payload) update(r *newPayloadResult, elapsed time.Duration) {
payload.full = r.block payload.full = r.block
payload.fullFees = r.fees payload.fullFees = r.fees
payload.sidecars = r.sidecars payload.sidecars = r.sidecars
payload.fullWitness = r.witness
feesInEther := new(big.Float).Quo(new(big.Float).SetInt(r.fees), big.NewFloat(params.Ether)) feesInEther := new(big.Float).Quo(new(big.Float).SetInt(r.fees), big.NewFloat(params.Ether))
log.Info("Updated payload", log.Info("Updated payload",
@ -135,9 +141,19 @@ func (payload *Payload) Resolve() *engine.ExecutionPayloadEnvelope {
close(payload.stop) close(payload.stop)
} }
if payload.full != nil { if payload.full != nil {
return engine.BlockToExecutableData(payload.full, payload.fullFees, payload.sidecars) envelope := engine.BlockToExecutableData(payload.full, payload.fullFees, payload.sidecars)
if payload.fullWitness != nil {
envelope.Witness = new(hexutil.Bytes)
*envelope.Witness, _ = rlp.EncodeToBytes(payload.fullWitness) // cannot fail
} }
return engine.BlockToExecutableData(payload.empty, big.NewInt(0), nil) return envelope
}
envelope := engine.BlockToExecutableData(payload.empty, big.NewInt(0), nil)
if payload.emptyWitness != nil {
envelope.Witness = new(hexutil.Bytes)
*envelope.Witness, _ = rlp.EncodeToBytes(payload.emptyWitness) // cannot fail
}
return envelope
} }
// ResolveEmpty is basically identical to Resolve, but it expects empty block only. // ResolveEmpty is basically identical to Resolve, but it expects empty block only.
@ -146,7 +162,12 @@ func (payload *Payload) ResolveEmpty() *engine.ExecutionPayloadEnvelope {
payload.lock.Lock() payload.lock.Lock()
defer payload.lock.Unlock() defer payload.lock.Unlock()
return engine.BlockToExecutableData(payload.empty, big.NewInt(0), nil) envelope := engine.BlockToExecutableData(payload.empty, big.NewInt(0), nil)
if payload.emptyWitness != nil {
envelope.Witness = new(hexutil.Bytes)
*envelope.Witness, _ = rlp.EncodeToBytes(payload.emptyWitness) // cannot fail
}
return envelope
} }
// ResolveFull is basically identical to Resolve, but it expects full block only. // ResolveFull is basically identical to Resolve, but it expects full block only.
@ -172,11 +193,16 @@ func (payload *Payload) ResolveFull() *engine.ExecutionPayloadEnvelope {
default: default:
close(payload.stop) close(payload.stop)
} }
return engine.BlockToExecutableData(payload.full, payload.fullFees, payload.sidecars) envelope := engine.BlockToExecutableData(payload.full, payload.fullFees, payload.sidecars)
if payload.fullWitness != nil {
envelope.Witness = new(hexutil.Bytes)
*envelope.Witness, _ = rlp.EncodeToBytes(payload.fullWitness) // cannot fail
}
return envelope
} }
// buildPayload builds the payload according to the provided parameters. // buildPayload builds the payload according to the provided parameters.
func (miner *Miner) buildPayload(args *BuildPayloadArgs) (*Payload, error) { func (miner *Miner) buildPayload(args *BuildPayloadArgs, witness bool) (*Payload, error) {
// Build the initial version with no transaction included. It should be fast // Build the initial version with no transaction included. It should be fast
// enough to run. The empty payload can at least make sure there is something // enough to run. The empty payload can at least make sure there is something
// to deliver for not missing slot. // to deliver for not missing slot.
@ -190,13 +216,12 @@ func (miner *Miner) buildPayload(args *BuildPayloadArgs) (*Payload, error) {
beaconRoot: args.BeaconRoot, beaconRoot: args.BeaconRoot,
noTxs: true, noTxs: true,
} }
empty := miner.generateWork(emptyParams) empty := miner.generateWork(emptyParams, witness)
if empty.err != nil { if empty.err != nil {
return nil, empty.err return nil, empty.err
} }
// Construct a payload object for return. // Construct a payload object for return.
payload := newPayload(empty.block, args.Id()) payload := newPayload(empty.block, empty.witness, args.Id())
// Spin up a routine for updating the payload in background. This strategy // Spin up a routine for updating the payload in background. This strategy
// can maximum the revenue for including transactions with highest fee. // can maximum the revenue for including transactions with highest fee.
@ -226,7 +251,7 @@ func (miner *Miner) buildPayload(args *BuildPayloadArgs) (*Payload, error) {
select { select {
case <-timer.C: case <-timer.C:
start := time.Now() start := time.Now()
r := miner.generateWork(fullParams) r := miner.generateWork(fullParams, witness)
if r.err == nil { if r.err == nil {
payload.update(r, time.Since(start)) payload.update(r, time.Since(start))
} else { } else {

@ -160,7 +160,7 @@ func TestBuildPayload(t *testing.T) {
Random: common.Hash{}, Random: common.Hash{},
FeeRecipient: recipient, FeeRecipient: recipient,
} }
payload, err := w.buildPayload(args) payload, err := w.buildPayload(args, false)
if err != nil { if err != nil {
t.Fatalf("Failed to build payload %v", err) t.Fatalf("Failed to build payload %v", err)
} }

@ -28,6 +28,7 @@ import (
"github.com/ethereum/go-ethereum/consensus/misc/eip4844" "github.com/ethereum/go-ethereum/consensus/misc/eip4844"
"github.com/ethereum/go-ethereum/core" "github.com/ethereum/go-ethereum/core"
"github.com/ethereum/go-ethereum/core/state" "github.com/ethereum/go-ethereum/core/state"
"github.com/ethereum/go-ethereum/core/stateless"
"github.com/ethereum/go-ethereum/core/txpool" "github.com/ethereum/go-ethereum/core/txpool"
"github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/core/vm" "github.com/ethereum/go-ethereum/core/vm"
@ -56,6 +57,8 @@ type environment struct {
receipts []*types.Receipt receipts []*types.Receipt
sidecars []*types.BlobTxSidecar sidecars []*types.BlobTxSidecar
blobs int blobs int
witness *stateless.Witness
} }
const ( const (
@ -73,6 +76,7 @@ type newPayloadResult struct {
sidecars []*types.BlobTxSidecar // collected blobs of blob transactions sidecars []*types.BlobTxSidecar // collected blobs of blob transactions
stateDB *state.StateDB // StateDB after executing the transactions stateDB *state.StateDB // StateDB after executing the transactions
receipts []*types.Receipt // Receipts collected during construction receipts []*types.Receipt // Receipts collected during construction
witness *stateless.Witness // Witness is an optional stateless proof
} }
// generateParams wraps various settings for generating sealing task. // generateParams wraps various settings for generating sealing task.
@ -88,8 +92,8 @@ type generateParams struct {
} }
// generateWork generates a sealing block based on the given parameters. // generateWork generates a sealing block based on the given parameters.
func (miner *Miner) generateWork(params *generateParams) *newPayloadResult { func (miner *Miner) generateWork(params *generateParams, witness bool) *newPayloadResult {
work, err := miner.prepareWork(params) work, err := miner.prepareWork(params, witness)
if err != nil { if err != nil {
return &newPayloadResult{err: err} return &newPayloadResult{err: err}
} }
@ -129,13 +133,14 @@ func (miner *Miner) generateWork(params *generateParams) *newPayloadResult {
sidecars: work.sidecars, sidecars: work.sidecars,
stateDB: work.state, stateDB: work.state,
receipts: work.receipts, receipts: work.receipts,
witness: work.witness,
} }
} }
// prepareWork constructs the sealing task according to the given parameters, // prepareWork constructs the sealing task according to the given parameters,
// either based on the last chain head or specified parent. In this function // either based on the last chain head or specified parent. In this function
// the pending transactions are not filled yet, only the empty task returned. // the pending transactions are not filled yet, only the empty task returned.
func (miner *Miner) prepareWork(genParams *generateParams) (*environment, error) { func (miner *Miner) prepareWork(genParams *generateParams, witness bool) (*environment, error) {
miner.confMu.RLock() miner.confMu.RLock()
defer miner.confMu.RUnlock() defer miner.confMu.RUnlock()
@ -203,7 +208,7 @@ func (miner *Miner) prepareWork(genParams *generateParams) (*environment, error)
// Could potentially happen if starting to mine in an odd state. // Could potentially happen if starting to mine in an odd state.
// Note genParams.coinbase can be different with header.Coinbase // Note genParams.coinbase can be different with header.Coinbase
// since clique algorithm can modify the coinbase field in header. // since clique algorithm can modify the coinbase field in header.
env, err := miner.makeEnv(parent, header, genParams.coinbase) env, err := miner.makeEnv(parent, header, genParams.coinbase, witness)
if err != nil { if err != nil {
log.Error("Failed to create sealing context", "err", err) log.Error("Failed to create sealing context", "err", err)
return nil, err return nil, err
@ -222,18 +227,26 @@ func (miner *Miner) prepareWork(genParams *generateParams) (*environment, error)
} }
// makeEnv creates a new environment for the sealing block. // makeEnv creates a new environment for the sealing block.
func (miner *Miner) makeEnv(parent *types.Header, header *types.Header, coinbase common.Address) (*environment, error) { func (miner *Miner) makeEnv(parent *types.Header, header *types.Header, coinbase common.Address, witness bool) (*environment, error) {
// Retrieve the parent state to execute on top. // Retrieve the parent state to execute on top.
state, err := miner.chain.StateAt(parent.Root) state, err := miner.chain.StateAt(parent.Root)
if err != nil { if err != nil {
return nil, err return nil, err
} }
if witness {
bundle, err := stateless.NewWitness(header, miner.chain)
if err != nil {
return nil, err
}
state.StartPrefetcher("miner", bundle)
}
// Note the passed coinbase may be different with header.Coinbase. // Note the passed coinbase may be different with header.Coinbase.
return &environment{ return &environment{
signer: types.MakeSigner(miner.chainConfig, header.Number, header.Time), signer: types.MakeSigner(miner.chainConfig, header.Number, header.Time),
state: state, state: state,
coinbase: coinbase, coinbase: coinbase,
header: header, header: header,
witness: state.Witness(),
}, nil }, nil
} }

@ -152,7 +152,7 @@ func (t *BlockTest) Run(snapshotter bool, scheme string, witness bool, tracer *t
} }
chain, err := core.NewBlockChain(db, cache, gspec, nil, engine, vm.Config{ chain, err := core.NewBlockChain(db, cache, gspec, nil, engine, vm.Config{
Tracer: tracer, Tracer: tracer,
EnableWitnessCollection: witness, StatelessSelfValidation: witness,
}, nil) }, nil)
if err != nil { if err != nil {
return err return err