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:
parent
b018da9d02
commit
9326a118c7
@ -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 {
|
||||||
@ -115,9 +124,10 @@ type executionPayloadEnvelopeMarshaling struct {
|
|||||||
}
|
}
|
||||||
|
|
||||||
type PayloadStatusV1 struct {
|
type PayloadStatusV1 struct {
|
||||||
Status string `json:"status"`
|
Status string `json:"status"`
|
||||||
LatestValidHash *common.Hash `json:"latestValidHash"`
|
Witness *hexutil.Bytes `json:"witness"`
|
||||||
ValidationError *string `json:"validationError"`
|
LatestValidHash *common.Hash `json:"latestValidHash"`
|
||||||
|
ValidationError *string `json:"validationError"`
|
||||||
}
|
}
|
||||||
|
|
||||||
type TransitionConfigurationV1 struct {
|
type TransitionConfigurationV1 struct {
|
||||||
@ -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
|
||||||
|
@ -78,10 +78,11 @@ var (
|
|||||||
snapshotCommitTimer = metrics.NewRegisteredResettingTimer("chain/snapshot/commits", nil)
|
snapshotCommitTimer = metrics.NewRegisteredResettingTimer("chain/snapshot/commits", nil)
|
||||||
triedbCommitTimer = metrics.NewRegisteredResettingTimer("chain/triedb/commits", nil)
|
triedbCommitTimer = metrics.NewRegisteredResettingTimer("chain/triedb/commits", nil)
|
||||||
|
|
||||||
blockInsertTimer = metrics.NewRegisteredResettingTimer("chain/inserts", nil)
|
blockInsertTimer = metrics.NewRegisteredResettingTimer("chain/inserts", nil)
|
||||||
blockValidationTimer = metrics.NewRegisteredResettingTimer("chain/validation", nil)
|
blockValidationTimer = metrics.NewRegisteredResettingTimer("chain/validation", nil)
|
||||||
blockExecutionTimer = metrics.NewRegisteredResettingTimer("chain/execution", nil)
|
blockCrossValidationTimer = metrics.NewRegisteredResettingTimer("chain/crossvalidation", nil)
|
||||||
blockWriteTimer = metrics.NewRegisteredResettingTimer("chain/write", nil)
|
blockExecutionTimer = metrics.NewRegisteredResettingTimer("chain/execution", nil)
|
||||||
|
blockWriteTimer = metrics.NewRegisteredResettingTimer("chain/write", nil)
|
||||||
|
|
||||||
blockReorgMeter = metrics.NewRegisteredMeter("chain/reorg/executes", nil)
|
blockReorgMeter = metrics.NewRegisteredMeter("chain/reorg/executes", nil)
|
||||||
blockReorgAddMeter = metrics.NewRegisteredMeter("chain/reorg/add", nil)
|
blockReorgAddMeter = metrics.NewRegisteredMeter("chain/reorg/add", 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
|
if parent == nil {
|
||||||
// trustless pre-root hash container
|
return nil, errors.New("failed to retrieve parent header")
|
||||||
parent := chain.GetHeader(block.ParentHash(), block.NumberU64()-1)
|
}
|
||||||
if parent == nil {
|
headers = append(headers, parent)
|
||||||
return nil, errors.New("failed to retrieve parent header")
|
|
||||||
}
|
}
|
||||||
// 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 = ×tamp
|
||||||
|
genesis.Config.CancunTime = ×tamp
|
||||||
|
|
||||||
|
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"
|
||||||
@ -67,22 +69,25 @@ func (args *BuildPayloadArgs) Id() engine.PayloadID {
|
|||||||
// the revenue. Therefore, the empty-block here is always available and full-block
|
// the revenue. Therefore, the empty-block here is always available and full-block
|
||||||
// will be set/updated afterwards.
|
// will be set/updated afterwards.
|
||||||
type Payload struct {
|
type Payload struct {
|
||||||
id engine.PayloadID
|
id engine.PayloadID
|
||||||
empty *types.Block
|
empty *types.Block
|
||||||
full *types.Block
|
emptyWitness *stateless.Witness
|
||||||
sidecars []*types.BlobTxSidecar
|
full *types.Block
|
||||||
fullFees *big.Int
|
fullWitness *stateless.Witness
|
||||||
stop chan struct{}
|
sidecars []*types.BlobTxSidecar
|
||||||
lock sync.Mutex
|
fullFees *big.Int
|
||||||
cond *sync.Cond
|
stop chan struct{}
|
||||||
|
lock sync.Mutex
|
||||||
|
cond *sync.Cond
|
||||||
}
|
}
|
||||||
|
|
||||||
// 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,
|
||||||
stop: make(chan struct{}),
|
emptyWitness: witness,
|
||||||
|
stop: make(chan struct{}),
|
||||||
}
|
}
|
||||||
log.Info("Starting work on payload", "id", payload.id)
|
log.Info("Starting work on payload", "id", payload.id)
|
||||||
payload.cond = sync.NewCond(&payload.lock)
|
payload.cond = sync.NewCond(&payload.lock)
|
||||||
@ -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 envelope
|
||||||
}
|
}
|
||||||
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
|
||||||
}
|
}
|
||||||
|
|
||||||
// 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
|
||||||
|
Loading…
Reference in New Issue
Block a user