all: core rework for the merge transition (#23761)

* all: work for eth1/2 transtition

* consensus/beacon, eth: change beacon difficulty to 0

* eth: updates

* all: add terminalBlockDifficulty config, fix rebasing issues

* eth: implemented merge interop spec

* internal/ethapi: update to v1.0.0.alpha.2

                                                                 This commit updates the code to the new spec, moving payloadId into
                                                                 it's own object. It also fixes an issue with finalizing an empty blockhash.
                                                                 It also properly sets the basefee

* all: sync polishes, other fixes + refactors

* core, eth: correct semantics for LeavePoW, EnterPoS

* core: fixed rebasing artifacts

* core: light: performance improvements

* core: use keyed field (f)

* core: eth: fix compilation issues + tests

* eth/catalyst: dbetter error codes

* all: move Merger to consensus/, remove reliance on it in bc

* all: renamed EnterPoS and LeavePoW to ReachTDD and FinalizePoS

* core: make mergelogs a function

* core: use InsertChain instead of InsertBlock

* les: drop merger from lightchain object

* consensus: add merger

* core: recoverAncestors in catalyst mode

* core: fix nitpick

* all: removed merger from beacon, use TTD, nitpicks

* consensus: eth: add docstring, removed unnecessary code duplication

* consensus/beacon: better comment

* all: easy to fix nitpicks by karalabe

* consensus/beacon: verify known headers to be sure

* core: comments

* core: eth: don't drop peers who advertise blocks, nitpicks

* core: never add beacon blocks to the future queue

* core: fixed nitpicks

* consensus/beacon: simplify IsTTDReached check

* consensus/beacon: correct IsTTDReached check

Co-authored-by: rjl493456442 <garyrong0905@gmail.com>
Co-authored-by: Péter Szilágyi <peterke@gmail.com>
This commit is contained in:
Marius van der Wijden 2021-11-26 12:23:02 +01:00 committed by GitHub
parent 519cf98b69
commit 3038e480f5
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
48 changed files with 3244 additions and 803 deletions

View File

@ -32,7 +32,6 @@ import (
"github.com/ethereum/go-ethereum/accounts/scwallet"
"github.com/ethereum/go-ethereum/accounts/usbwallet"
"github.com/ethereum/go-ethereum/cmd/utils"
"github.com/ethereum/go-ethereum/eth/catalyst"
"github.com/ethereum/go-ethereum/eth/ethconfig"
"github.com/ethereum/go-ethereum/internal/ethapi"
"github.com/ethereum/go-ethereum/log"
@ -159,17 +158,10 @@ func makeFullNode(ctx *cli.Context) (*node.Node, ethapi.Backend) {
if ctx.GlobalIsSet(utils.OverrideArrowGlacierFlag.Name) {
cfg.Eth.OverrideArrowGlacier = new(big.Int).SetUint64(ctx.GlobalUint64(utils.OverrideArrowGlacierFlag.Name))
}
backend, eth := utils.RegisterEthService(stack, &cfg.Eth)
// Configure catalyst.
if ctx.GlobalBool(utils.CatalystFlag.Name) {
if eth == nil {
utils.Fatalf("Catalyst does not work in light client mode.")
}
if err := catalyst.Register(stack, eth); err != nil {
utils.Fatalf("%v", err)
}
if ctx.GlobalIsSet(utils.OverrideTerminalTotalDifficulty.Name) {
cfg.Eth.Genesis.Config.TerminalTotalDifficulty = new(big.Int).SetUint64(ctx.GlobalUint64(utils.OverrideTerminalTotalDifficulty.Name))
}
backend, _ := utils.RegisterEthService(stack, &cfg.Eth, ctx.GlobalBool(utils.CatalystFlag.Name))
// Configure GraphQL if requested
if ctx.GlobalIsSet(utils.GraphQLEnabledFlag.Name) {

View File

@ -72,6 +72,7 @@ var (
utils.USBFlag,
utils.SmartCardDaemonPathFlag,
utils.OverrideArrowGlacierFlag,
utils.OverrideTerminalTotalDifficulty,
utils.EthashCacheDirFlag,
utils.EthashCachesInMemoryFlag,
utils.EthashCachesOnDiskFlag,

View File

@ -45,6 +45,7 @@ import (
"github.com/ethereum/go-ethereum/core/vm"
"github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/eth"
"github.com/ethereum/go-ethereum/eth/catalyst"
"github.com/ethereum/go-ethereum/eth/downloader"
"github.com/ethereum/go-ethereum/eth/ethconfig"
"github.com/ethereum/go-ethereum/eth/gasprice"
@ -248,6 +249,10 @@ var (
Name: "override.arrowglacier",
Usage: "Manually specify Arrow Glacier fork-block, overriding the bundled setting",
}
OverrideTerminalTotalDifficulty = cli.Uint64Flag{
Name: "override.terminaltotaldifficulty",
Usage: "Manually specify TerminalTotalDifficulty, overriding the bundled setting",
}
// Light server and client settings
LightServeFlag = cli.IntFlag{
Name: "light.serve",
@ -1196,7 +1201,7 @@ func SetP2PConfig(ctx *cli.Context, cfg *p2p.Config) {
cfg.NetRestrict = list
}
if ctx.GlobalBool(DeveloperFlag.Name) || ctx.GlobalBool(CatalystFlag.Name) {
if ctx.GlobalBool(DeveloperFlag.Name) {
// --dev mode can't use p2p networking.
cfg.MaxPeers = 0
cfg.ListenAddr = ""
@ -1705,13 +1710,18 @@ func SetDNSDiscoveryDefaults(cfg *ethconfig.Config, genesis common.Hash) {
// RegisterEthService adds an Ethereum client to the stack.
// The second return value is the full node instance, which may be nil if the
// node is running as a light client.
func RegisterEthService(stack *node.Node, cfg *ethconfig.Config) (ethapi.Backend, *eth.Ethereum) {
func RegisterEthService(stack *node.Node, cfg *ethconfig.Config, isCatalyst bool) (ethapi.Backend, *eth.Ethereum) {
if cfg.SyncMode == downloader.LightSync {
backend, err := les.New(stack, cfg)
if err != nil {
Fatalf("Failed to register the Ethereum service: %v", err)
}
stack.RegisterAPIs(tracers.APIs(backend.ApiBackend))
if isCatalyst {
if err := catalyst.RegisterLight(stack, backend); err != nil {
Fatalf("Failed to register the catalyst service: %v", err)
}
}
return backend.ApiBackend, nil
}
backend, err := eth.New(stack, cfg)
@ -1724,6 +1734,11 @@ func RegisterEthService(stack *node.Node, cfg *ethconfig.Config) (ethapi.Backend
Fatalf("Failed to create the LES server: %v", err)
}
}
if isCatalyst {
if err := catalyst.Register(stack, backend); err != nil {
Fatalf("Failed to register the catalyst service: %v", err)
}
}
stack.RegisterAPIs(tracers.APIs(backend.APIBackend))
return backend.APIBackend, backend
}

View File

@ -0,0 +1,376 @@
// Copyright 2021 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package beacon
import (
"errors"
"fmt"
"math/big"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/consensus"
"github.com/ethereum/go-ethereum/consensus/misc"
"github.com/ethereum/go-ethereum/core/state"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/rpc"
"github.com/ethereum/go-ethereum/trie"
)
// Proof-of-stake protocol constants.
var (
beaconDifficulty = common.Big0 // The default block difficulty in the beacon consensus
beaconNonce = types.EncodeNonce(0) // The default block nonce in the beacon consensus
)
// Various error messages to mark blocks invalid. These should be private to
// prevent engine specific errors from being referenced in the remainder of the
// codebase, inherently breaking if the engine is swapped out. Please put common
// error types into the consensus package.
var (
errTooManyUncles = errors.New("too many uncles")
errInvalidMixDigest = errors.New("invalid mix digest")
errInvalidNonce = errors.New("invalid nonce")
errInvalidUncleHash = errors.New("invalid uncle hash")
)
// Beacon is a consensus engine that combines the eth1 consensus and proof-of-stake
// algorithm. There is a special flag inside to decide whether to use legacy consensus
// rules or new rules. The transition rule is described in the eth1/2 merge spec.
// https://github.com/ethereum/EIPs/blob/master/EIPS/eip-3675.md
//
// The beacon here is a half-functional consensus engine with partial functions which
// is only used for necessary consensus checks. The legacy consensus engine can be any
// engine implements the consensus interface (except the beacon itself).
type Beacon struct {
ethone consensus.Engine // Original consensus engine used in eth1, e.g. ethash or clique
}
// New creates a consensus engine with the given embedded eth1 engine.
func New(ethone consensus.Engine) *Beacon {
if _, ok := ethone.(*Beacon); ok {
panic("nested consensus engine")
}
return &Beacon{ethone: ethone}
}
// Author implements consensus.Engine, returning the verified author of the block.
func (beacon *Beacon) Author(header *types.Header) (common.Address, error) {
if !beacon.IsPoSHeader(header) {
return beacon.ethone.Author(header)
}
return header.Coinbase, nil
}
// VerifyHeader checks whether a header conforms to the consensus rules of the
// stock Ethereum consensus engine.
func (beacon *Beacon) VerifyHeader(chain consensus.ChainHeaderReader, header *types.Header, seal bool) error {
reached, _ := IsTTDReached(chain, header.ParentHash, header.Number.Uint64()-1)
if !reached {
return beacon.ethone.VerifyHeader(chain, header, seal)
}
// Short circuit if the parent is not known
parent := chain.GetHeader(header.ParentHash, header.Number.Uint64()-1)
if parent == nil {
return consensus.ErrUnknownAncestor
}
// Sanity checks passed, do a proper verification
return beacon.verifyHeader(chain, header, parent)
}
// VerifyHeaders is similar to VerifyHeader, but verifies a batch of headers
// concurrently. The method returns a quit channel to abort the operations and
// a results channel to retrieve the async verifications.
// VerifyHeaders expect the headers to be ordered and continuous.
func (beacon *Beacon) VerifyHeaders(chain consensus.ChainHeaderReader, headers []*types.Header, seals []bool) (chan<- struct{}, <-chan error) {
if !beacon.IsPoSHeader(headers[len(headers)-1]) {
return beacon.ethone.VerifyHeaders(chain, headers, seals)
}
var (
preHeaders []*types.Header
postHeaders []*types.Header
preSeals []bool
)
for index, header := range headers {
if beacon.IsPoSHeader(header) {
preHeaders = headers[:index]
postHeaders = headers[index:]
preSeals = seals[:index]
break
}
}
// All the headers have passed the transition point, use new rules.
if len(preHeaders) == 0 {
return beacon.verifyHeaders(chain, headers, nil)
}
// The transition point exists in the middle, separate the headers
// into two batches and apply different verification rules for them.
var (
abort = make(chan struct{})
results = make(chan error, len(headers))
)
go func() {
var (
old, new, out = 0, len(preHeaders), 0
errors = make([]error, len(headers))
done = make([]bool, len(headers))
oldDone, oldResult = beacon.ethone.VerifyHeaders(chain, preHeaders, preSeals)
newDone, newResult = beacon.verifyHeaders(chain, postHeaders, preHeaders[len(preHeaders)-1])
)
for {
for ; done[out]; out++ {
results <- errors[out]
if out == len(headers)-1 {
return
}
}
select {
case err := <-oldResult:
errors[old], done[old] = err, true
old++
case err := <-newResult:
errors[new], done[new] = err, true
new++
case <-abort:
close(oldDone)
close(newDone)
return
}
}
}()
return abort, results
}
// VerifyUncles verifies that the given block's uncles conform to the consensus
// rules of the Ethereum consensus engine.
func (beacon *Beacon) VerifyUncles(chain consensus.ChainReader, block *types.Block) error {
if !beacon.IsPoSHeader(block.Header()) {
return beacon.ethone.VerifyUncles(chain, block)
}
// Verify that there is no uncle block. It's explicitly disabled in the beacon
if len(block.Uncles()) > 0 {
return errTooManyUncles
}
return nil
}
// verifyHeader checks whether a header conforms to the consensus rules of the
// stock Ethereum consensus engine. The difference between the beacon and classic is
// (a) The following fields are expected to be constants:
// - difficulty is expected to be 0
// - nonce is expected to be 0
// - unclehash is expected to be Hash(emptyHeader)
// to be the desired constants
// (b) the timestamp is not verified anymore
// (c) the extradata is limited to 32 bytes
func (beacon *Beacon) verifyHeader(chain consensus.ChainHeaderReader, header, parent *types.Header) error {
// Ensure that the header's extra-data section is of a reasonable size
if len(header.Extra) > 32 {
return fmt.Errorf("extra-data longer than 32 bytes (%d)", len(header.Extra))
}
// Verify the seal parts. Ensure the mixhash, nonce and uncle hash are the expected value.
if header.MixDigest != (common.Hash{}) {
return errInvalidMixDigest
}
if header.Nonce != beaconNonce {
return errInvalidNonce
}
if header.UncleHash != types.EmptyUncleHash {
return errInvalidUncleHash
}
// Verify the block's difficulty to ensure it's the default constant
if beaconDifficulty.Cmp(header.Difficulty) != 0 {
return fmt.Errorf("invalid difficulty: have %v, want %v", header.Difficulty, beaconDifficulty)
}
// Verify that the gas limit is <= 2^63-1
cap := uint64(0x7fffffffffffffff)
if header.GasLimit > cap {
return fmt.Errorf("invalid gasLimit: have %v, max %v", header.GasLimit, cap)
}
// Verify that the gasUsed is <= gasLimit
if header.GasUsed > header.GasLimit {
return fmt.Errorf("invalid gasUsed: have %d, gasLimit %d", header.GasUsed, header.GasLimit)
}
// Verify that the block number is parent's +1
if diff := new(big.Int).Sub(header.Number, parent.Number); diff.Cmp(common.Big1) != 0 {
return consensus.ErrInvalidNumber
}
// Verify the header's EIP-1559 attributes.
return misc.VerifyEip1559Header(chain.Config(), parent, header)
}
// verifyHeaders is similar to verifyHeader, but verifies a batch of headers
// concurrently. The method returns a quit channel to abort the operations and
// a results channel to retrieve the async verifications. An additional parent
// header will be passed if the relevant header is not in the database yet.
func (beacon *Beacon) verifyHeaders(chain consensus.ChainHeaderReader, headers []*types.Header, ancestor *types.Header) (chan<- struct{}, <-chan error) {
var (
abort = make(chan struct{})
results = make(chan error, len(headers))
)
go func() {
for i, header := range headers {
var parent *types.Header
if i == 0 {
if ancestor != nil {
parent = ancestor
} else {
parent = chain.GetHeader(headers[0].ParentHash, headers[0].Number.Uint64()-1)
}
} else if headers[i-1].Hash() == headers[i].ParentHash {
parent = headers[i-1]
}
if parent == nil {
select {
case <-abort:
return
case results <- consensus.ErrUnknownAncestor:
}
continue
}
err := beacon.verifyHeader(chain, header, parent)
select {
case <-abort:
return
case results <- err:
}
}
}()
return abort, results
}
// Prepare implements consensus.Engine, initializing the difficulty field of a
// header to conform to the beacon protocol. The changes are done inline.
func (beacon *Beacon) Prepare(chain consensus.ChainHeaderReader, header *types.Header) error {
// Transition isn't triggered yet, use the legacy rules for preparation.
reached, err := IsTTDReached(chain, header.ParentHash, header.Number.Uint64()-1)
if err != nil {
return err
}
if !reached {
return beacon.ethone.Prepare(chain, header)
}
header.Difficulty = beaconDifficulty
return nil
}
// Finalize implements consensus.Engine, setting the final state on the header
func (beacon *Beacon) Finalize(chain consensus.ChainHeaderReader, header *types.Header, state *state.StateDB, txs []*types.Transaction, uncles []*types.Header) {
// Finalize is different with Prepare, it can be used in both block generation
// and verification. So determine the consensus rules by header type.
if !beacon.IsPoSHeader(header) {
beacon.ethone.Finalize(chain, header, state, txs, uncles)
return
}
// The block reward is no longer handled here. It's done by the
// external consensus engine.
header.Root = state.IntermediateRoot(true)
}
// FinalizeAndAssemble implements consensus.Engine, setting the final state and
// assembling the block.
func (beacon *Beacon) FinalizeAndAssemble(chain consensus.ChainHeaderReader, header *types.Header, state *state.StateDB, txs []*types.Transaction, uncles []*types.Header, receipts []*types.Receipt) (*types.Block, error) {
// FinalizeAndAssemble is different with Prepare, it can be used in both block
// generation and verification. So determine the consensus rules by header type.
if !beacon.IsPoSHeader(header) {
return beacon.ethone.FinalizeAndAssemble(chain, header, state, txs, uncles, receipts)
}
// Finalize and assemble the block
beacon.Finalize(chain, header, state, txs, uncles)
return types.NewBlock(header, txs, uncles, receipts, trie.NewStackTrie(nil)), nil
}
// Seal generates a new sealing request for the given input block and pushes
// the result into the given channel.
//
// Note, the method returns immediately and will send the result async. More
// than one result may also be returned depending on the consensus algorithm.
func (beacon *Beacon) Seal(chain consensus.ChainHeaderReader, block *types.Block, results chan<- *types.Block, stop <-chan struct{}) error {
if !beacon.IsPoSHeader(block.Header()) {
return beacon.ethone.Seal(chain, block, results, stop)
}
// The seal verification is done by the external consensus engine,
// return directly without pushing any block back. In another word
// beacon won't return any result by `results` channel which may
// blocks the receiver logic forever.
return nil
}
// SealHash returns the hash of a block prior to it being sealed.
func (beacon *Beacon) SealHash(header *types.Header) common.Hash {
return beacon.ethone.SealHash(header)
}
// CalcDifficulty is the difficulty adjustment algorithm. It returns
// the difficulty that a new block should have when created at time
// given the parent block's time and difficulty.
func (beacon *Beacon) CalcDifficulty(chain consensus.ChainHeaderReader, time uint64, parent *types.Header) *big.Int {
// Transition isn't triggered yet, use the legacy rules for calculation
if reached, _ := IsTTDReached(chain, parent.Hash(), parent.Number.Uint64()); !reached {
return beacon.ethone.CalcDifficulty(chain, time, parent)
}
return beaconDifficulty
}
// APIs implements consensus.Engine, returning the user facing RPC APIs.
func (beacon *Beacon) APIs(chain consensus.ChainHeaderReader) []rpc.API {
return beacon.ethone.APIs(chain)
}
// Close shutdowns the consensus engine
func (beacon *Beacon) Close() error {
return beacon.ethone.Close()
}
// IsPoSHeader reports the header belongs to the PoS-stage with some special fields.
// This function is not suitable for a part of APIs like Prepare or CalcDifficulty
// because the header difficulty is not set yet.
func (beacon *Beacon) IsPoSHeader(header *types.Header) bool {
if header.Difficulty == nil {
panic("IsPoSHeader called with invalid difficulty")
}
return header.Difficulty.Cmp(beaconDifficulty) == 0
}
// InnerEngine returns the embedded eth1 consensus engine.
func (beacon *Beacon) InnerEngine() consensus.Engine {
return beacon.ethone
}
// SetThreads updates the mining threads. Delegate the call
// to the eth1 engine if it's threaded.
func (beacon *Beacon) SetThreads(threads int) {
type threaded interface {
SetThreads(threads int)
}
if th, ok := beacon.ethone.(threaded); ok {
th.SetThreads(threads)
}
}
// IsTTDReached checks if the TotalTerminalDifficulty has been surpassed on the `parentHash` block.
// It depends on the parentHash already being stored in the database.
// If the parentHash is not stored in the database a UnknownAncestor error is returned.
func IsTTDReached(chain consensus.ChainHeaderReader, parentHash common.Hash, number uint64) (bool, error) {
if chain.Config().TerminalTotalDifficulty == nil {
return false, nil
}
td := chain.GetTd(parentHash, number)
if td == nil {
return false, consensus.ErrUnknownAncestor
}
return td.Cmp(chain.Config().TerminalTotalDifficulty) >= 0, nil
}

View File

@ -44,6 +44,9 @@ type ChainHeaderReader interface {
// GetHeaderByHash retrieves a block header from the database by its hash.
GetHeaderByHash(hash common.Hash) *types.Header
// GetTd retrieves the total difficulty from the database by hash and number.
GetTd(hash common.Hash, number uint64) *big.Int
}
// ChainReader defines a small collection of methods needed to access the local

110
consensus/merger.go Normal file
View File

@ -0,0 +1,110 @@
// Copyright 2021 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package consensus
import (
"fmt"
"sync"
"github.com/ethereum/go-ethereum/core/rawdb"
"github.com/ethereum/go-ethereum/ethdb"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/rlp"
)
// transitionStatus describes the status of eth1/2 transition. This switch
// between modes is a one-way action which is triggered by corresponding
// consensus-layer message.
type transitionStatus struct {
LeftPoW bool // The flag is set when the first NewHead message received
EnteredPoS bool // The flag is set when the first FinalisedBlock message received
}
// Merger is an internal help structure used to track the eth1/2 transition status.
// It's a common structure can be used in both full node and light client.
type Merger struct {
db ethdb.KeyValueStore
status transitionStatus
mu sync.RWMutex
}
// NewMerger creates a new Merger which stores its transition status in the provided db.
func NewMerger(db ethdb.KeyValueStore) *Merger {
var status transitionStatus
blob := rawdb.ReadTransitionStatus(db)
if len(blob) != 0 {
if err := rlp.DecodeBytes(blob, &status); err != nil {
log.Crit("Failed to decode the transition status", "err", err)
}
}
return &Merger{
db: db,
status: status,
}
}
// ReachTTD is called whenever the first NewHead message received
// from the consensus-layer.
func (m *Merger) ReachTTD() {
m.mu.Lock()
defer m.mu.Unlock()
if m.status.LeftPoW {
return
}
m.status = transitionStatus{LeftPoW: true}
blob, err := rlp.EncodeToBytes(m.status)
if err != nil {
panic(fmt.Sprintf("Failed to encode the transition status: %v", err))
}
rawdb.WriteTransitionStatus(m.db, blob)
log.Info("Left PoW stage")
}
// FinalizePoS is called whenever the first FinalisedBlock message received
// from the consensus-layer.
func (m *Merger) FinalizePoS() {
m.mu.Lock()
defer m.mu.Unlock()
if m.status.EnteredPoS {
return
}
m.status = transitionStatus{LeftPoW: true, EnteredPoS: true}
blob, err := rlp.EncodeToBytes(m.status)
if err != nil {
panic(fmt.Sprintf("Failed to encode the transition status: %v", err))
}
rawdb.WriteTransitionStatus(m.db, blob)
log.Info("Entered PoS stage")
}
// TDDReached reports whether the chain has left the PoW stage.
func (m *Merger) TDDReached() bool {
m.mu.RLock()
defer m.mu.RUnlock()
return m.status.LeftPoW
}
// PoSFinalized reports whether the chain has entered the PoS stage.
func (m *Merger) PoSFinalized() bool {
m.mu.RLock()
defer m.mu.RUnlock()
return m.status.EnteredPoS
}

View File

@ -17,14 +17,21 @@
package core
import (
"encoding/json"
"math/big"
"runtime"
"testing"
"time"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/consensus"
"github.com/ethereum/go-ethereum/consensus/beacon"
"github.com/ethereum/go-ethereum/consensus/clique"
"github.com/ethereum/go-ethereum/consensus/ethash"
"github.com/ethereum/go-ethereum/core/rawdb"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/core/vm"
"github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/params"
)
@ -76,6 +83,172 @@ func TestHeaderVerification(t *testing.T) {
}
}
func TestHeaderVerificationForMergingClique(t *testing.T) { testHeaderVerificationForMerging(t, true) }
func TestHeaderVerificationForMergingEthash(t *testing.T) { testHeaderVerificationForMerging(t, false) }
// Tests the verification for eth1/2 merging, including pre-merge and post-merge
func testHeaderVerificationForMerging(t *testing.T, isClique bool) {
var (
testdb = rawdb.NewMemoryDatabase()
preBlocks []*types.Block
postBlocks []*types.Block
runEngine consensus.Engine
chainConfig *params.ChainConfig
merger = consensus.NewMerger(rawdb.NewMemoryDatabase())
)
if isClique {
var (
key, _ = crypto.HexToECDSA("b71c71a67e1177ad4e901695e1b4b9ee17ae16c6668d313eac2f96dbcda3f291")
addr = crypto.PubkeyToAddress(key.PublicKey)
engine = clique.New(params.AllCliqueProtocolChanges.Clique, testdb)
)
genspec := &Genesis{
ExtraData: make([]byte, 32+common.AddressLength+crypto.SignatureLength),
Alloc: map[common.Address]GenesisAccount{
addr: {Balance: big.NewInt(1)},
},
BaseFee: big.NewInt(params.InitialBaseFee),
}
copy(genspec.ExtraData[32:], addr[:])
genesis := genspec.MustCommit(testdb)
genEngine := beacon.New(engine)
preBlocks, _ = GenerateChain(params.AllCliqueProtocolChanges, genesis, genEngine, testdb, 8, nil)
td := 0
for i, block := range preBlocks {
header := block.Header()
if i > 0 {
header.ParentHash = preBlocks[i-1].Hash()
}
header.Extra = make([]byte, 32+crypto.SignatureLength)
header.Difficulty = big.NewInt(2)
sig, _ := crypto.Sign(genEngine.SealHash(header).Bytes(), key)
copy(header.Extra[len(header.Extra)-crypto.SignatureLength:], sig)
preBlocks[i] = block.WithSeal(header)
// calculate td
td += int(block.Difficulty().Uint64())
}
config := *params.AllCliqueProtocolChanges
config.TerminalTotalDifficulty = big.NewInt(int64(td))
postBlocks, _ = GenerateChain(&config, preBlocks[len(preBlocks)-1], genEngine, testdb, 8, nil)
chainConfig = &config
runEngine = beacon.New(engine)
} else {
gspec := &Genesis{Config: params.TestChainConfig}
genesis := gspec.MustCommit(testdb)
genEngine := beacon.New(ethash.NewFaker())
preBlocks, _ = GenerateChain(params.TestChainConfig, genesis, genEngine, testdb, 8, nil)
td := 0
for _, block := range preBlocks {
// calculate td
td += int(block.Difficulty().Uint64())
}
config := *params.TestChainConfig
config.TerminalTotalDifficulty = big.NewInt(int64(td))
postBlocks, _ = GenerateChain(params.TestChainConfig, preBlocks[len(preBlocks)-1], genEngine, testdb, 8, nil)
chainConfig = &config
runEngine = beacon.New(ethash.NewFaker())
}
preHeaders := make([]*types.Header, len(preBlocks))
for i, block := range preBlocks {
preHeaders[i] = block.Header()
blob, _ := json.Marshal(block.Header())
t.Logf("Log header before the merging %d: %v", block.NumberU64(), string(blob))
}
postHeaders := make([]*types.Header, len(postBlocks))
for i, block := range postBlocks {
postHeaders[i] = block.Header()
blob, _ := json.Marshal(block.Header())
t.Logf("Log header after the merging %d: %v", block.NumberU64(), string(blob))
}
// Run the header checker for blocks one-by-one, checking for both valid and invalid nonces
chain, _ := NewBlockChain(testdb, nil, chainConfig, runEngine, vm.Config{}, nil, nil)
defer chain.Stop()
// Verify the blocks before the merging
for i := 0; i < len(preBlocks); i++ {
_, results := runEngine.VerifyHeaders(chain, []*types.Header{preHeaders[i]}, []bool{true})
// Wait for the verification result
select {
case result := <-results:
if result != nil {
t.Errorf("test %d: verification failed %v", i, result)
}
case <-time.After(time.Second):
t.Fatalf("test %d: verification timeout", i)
}
// Make sure no more data is returned
select {
case result := <-results:
t.Fatalf("test %d: unexpected result returned: %v", i, result)
case <-time.After(25 * time.Millisecond):
}
chain.InsertChain(preBlocks[i : i+1])
}
// Make the transition
merger.ReachTTD()
merger.FinalizePoS()
// Verify the blocks after the merging
for i := 0; i < len(postBlocks); i++ {
_, results := runEngine.VerifyHeaders(chain, []*types.Header{postHeaders[i]}, []bool{true})
// Wait for the verification result
select {
case result := <-results:
if result != nil {
t.Errorf("test %d: verification failed %v", i, result)
}
case <-time.After(time.Second):
t.Fatalf("test %d: verification timeout", i)
}
// Make sure no more data is returned
select {
case result := <-results:
t.Fatalf("test %d: unexpected result returned: %v", i, result)
case <-time.After(25 * time.Millisecond):
}
chain.InsertBlockWithoutSetHead(postBlocks[i])
}
// Verify the blocks with pre-merge blocks and post-merge blocks
var (
headers []*types.Header
seals []bool
)
for _, block := range preBlocks {
headers = append(headers, block.Header())
seals = append(seals, true)
}
for _, block := range postBlocks {
headers = append(headers, block.Header())
seals = append(seals, true)
}
_, results := runEngine.VerifyHeaders(chain, headers, seals)
for i := 0; i < len(headers); i++ {
select {
case result := <-results:
if result != nil {
t.Errorf("test %d: verification failed %v", i, result)
}
case <-time.After(time.Second):
t.Fatalf("test %d: verification timeout", i)
}
}
// Make sure no more data is returned
select {
case result := <-results:
t.Fatalf("unexpected result returned: %v", result)
case <-time.After(25 * time.Millisecond):
}
}
// Tests that concurrent header verification works, for both good and bad blocks.
func TestHeaderConcurrentVerification2(t *testing.T) { testHeaderConcurrentVerification(t, 2) }
func TestHeaderConcurrentVerification8(t *testing.T) { testHeaderConcurrentVerification(t, 8) }

View File

@ -22,7 +22,6 @@ import (
"fmt"
"io"
"math/big"
mrand "math/rand"
"sort"
"sync"
"sync/atomic"
@ -208,15 +207,14 @@ type BlockChain struct {
validator Validator // Block and state validator interface
prefetcher Prefetcher
processor Processor // Block transaction processor interface
forker *ForkChoice
vmConfig vm.Config
shouldPreserve func(*types.Block) bool // Function used to determine whether should preserve the given block.
}
// NewBlockChain returns a fully initialised block chain using information
// available in the database. It initialises the default Ethereum Validator and
// Processor.
func NewBlockChain(db ethdb.Database, cacheConfig *CacheConfig, chainConfig *params.ChainConfig, engine consensus.Engine, vmConfig vm.Config, shouldPreserve func(block *types.Block) bool, txLookupLimit *uint64) (*BlockChain, error) {
// available in the database. It initialises the default Ethereum Validator
// and Processor.
func NewBlockChain(db ethdb.Database, cacheConfig *CacheConfig, chainConfig *params.ChainConfig, engine consensus.Engine, vmConfig vm.Config, shouldPreserve func(header *types.Header) bool, txLookupLimit *uint64) (*BlockChain, error) {
if cacheConfig == nil {
cacheConfig = defaultCacheConfig
}
@ -237,18 +235,18 @@ func NewBlockChain(db ethdb.Database, cacheConfig *CacheConfig, chainConfig *par
Journal: cacheConfig.TrieCleanJournal,
Preimages: cacheConfig.Preimages,
}),
quit: make(chan struct{}),
chainmu: syncx.NewClosableMutex(),
shouldPreserve: shouldPreserve,
bodyCache: bodyCache,
bodyRLPCache: bodyRLPCache,
receiptsCache: receiptsCache,
blockCache: blockCache,
txLookupCache: txLookupCache,
futureBlocks: futureBlocks,
engine: engine,
vmConfig: vmConfig,
quit: make(chan struct{}),
chainmu: syncx.NewClosableMutex(),
bodyCache: bodyCache,
bodyRLPCache: bodyRLPCache,
receiptsCache: receiptsCache,
blockCache: blockCache,
txLookupCache: txLookupCache,
futureBlocks: futureBlocks,
engine: engine,
vmConfig: vmConfig,
}
bc.forker = NewForkChoice(bc, shouldPreserve)
bc.validator = NewBlockValidator(chainConfig, bc, engine)
bc.prefetcher = newStatePrefetcher(chainConfig, bc, engine)
bc.processor = NewStateProcessor(chainConfig, bc, engine)
@ -382,7 +380,7 @@ func NewBlockChain(db ethdb.Database, cacheConfig *CacheConfig, chainConfig *par
// Start future block processor.
bc.wg.Add(1)
go bc.futureBlocksLoop()
go bc.updateFutureBlocks()
// Start tx indexer/unindexer.
if txLookupLimit != nil {
@ -877,12 +875,6 @@ const (
SideStatTy
)
// numberHash is just a container for a number and a hash, to represent a block
type numberHash struct {
number uint64
hash common.Hash
}
// InsertReceiptChain attempts to complete an already existing header chain with
// transaction and receipt data.
func (bc *BlockChain) InsertReceiptChain(blockChain types.Blocks, receiptChain []types.Receipts, ancientLimit uint64) (int, error) {
@ -928,13 +920,17 @@ func (bc *BlockChain) InsertReceiptChain(blockChain types.Blocks, receiptChain [
// Rewind may have occurred, skip in that case.
if bc.CurrentHeader().Number.Cmp(head.Number()) >= 0 {
currentFastBlock, td := bc.CurrentFastBlock(), bc.GetTd(head.Hash(), head.NumberU64())
if bc.GetTd(currentFastBlock.Hash(), currentFastBlock.NumberU64()).Cmp(td) < 0 {
rawdb.WriteHeadFastBlockHash(bc.db, head.Hash())
bc.currentFastBlock.Store(head)
headFastBlockGauge.Update(int64(head.NumberU64()))
return true
reorg, err := bc.forker.ReorgNeeded(bc.CurrentFastBlock().Header(), head.Header())
if err != nil {
log.Warn("Reorg failed", "err", err)
return false
} else if !reorg {
return false
}
rawdb.WriteHeadFastBlockHash(bc.db, head.Hash())
bc.currentFastBlock.Store(head)
headFastBlockGauge.Update(int64(head.NumberU64()))
return true
}
return false
}
@ -1181,30 +1177,15 @@ func (bc *BlockChain) writeKnownBlock(block *types.Block) error {
return nil
}
// WriteBlockWithState writes the block and all associated state to the database.
func (bc *BlockChain) WriteBlockWithState(block *types.Block, receipts []*types.Receipt, logs []*types.Log, state *state.StateDB, emitHeadEvent bool) (status WriteStatus, err error) {
if !bc.chainmu.TryLock() {
return NonStatTy, errInsertionInterrupted
}
defer bc.chainmu.Unlock()
return bc.writeBlockWithState(block, receipts, logs, state, emitHeadEvent)
}
// writeBlockWithState writes the block and all associated state to the database,
// but is expects the chain mutex to be held.
func (bc *BlockChain) writeBlockWithState(block *types.Block, receipts []*types.Receipt, logs []*types.Log, state *state.StateDB, emitHeadEvent bool) (status WriteStatus, err error) {
if bc.insertStopped() {
return NonStatTy, errInsertionInterrupted
}
// writeBlockWithState writes block, metadata and corresponding state data to the
// database.
func (bc *BlockChain) writeBlockWithState(block *types.Block, receipts []*types.Receipt, logs []*types.Log, state *state.StateDB) error {
// Calculate the total difficulty of the block
ptd := bc.GetTd(block.ParentHash(), block.NumberU64()-1)
if ptd == nil {
return NonStatTy, consensus.ErrUnknownAncestor
return consensus.ErrUnknownAncestor
}
// Make sure no inconsistent state is leaked during insertion
currentBlock := bc.CurrentBlock()
localTd := bc.GetTd(currentBlock.Hash(), currentBlock.NumberU64())
externTd := new(big.Int).Add(block.Difficulty(), ptd)
// Irrelevant of the canonical status, write the block itself to the database.
@ -1222,15 +1203,13 @@ func (bc *BlockChain) writeBlockWithState(block *types.Block, receipts []*types.
// Commit all cached state changes into underlying memory database.
root, err := state.Commit(bc.chainConfig.IsEIP158(block.Number()))
if err != nil {
return NonStatTy, err
return err
}
triedb := bc.stateCache.TrieDB()
// If we're running an archive node, always flush
if bc.cacheConfig.TrieDirtyDisabled {
if err := triedb.Commit(root, false, nil); err != nil {
return NonStatTy, err
}
return triedb.Commit(root, false, nil)
} else {
// Full but not archive node, do proper garbage collection
triedb.Reference(root, common.Hash{}) // metadata reference to keep trie alive
@ -1278,23 +1257,30 @@ func (bc *BlockChain) writeBlockWithState(block *types.Block, receipts []*types.
}
}
}
// If the total difficulty is higher than our known, add it to the canonical chain
// Second clause in the if statement reduces the vulnerability to selfish mining.
// Please refer to http://www.cs.cornell.edu/~ie53/publications/btcProcFC.pdf
reorg := externTd.Cmp(localTd) > 0
currentBlock = bc.CurrentBlock()
if !reorg && externTd.Cmp(localTd) == 0 {
// Split same-difficulty blocks by number, then preferentially select
// the block generated by the local miner as the canonical block.
if block.NumberU64() < currentBlock.NumberU64() {
reorg = true
} else if block.NumberU64() == currentBlock.NumberU64() {
var currentPreserve, blockPreserve bool
if bc.shouldPreserve != nil {
currentPreserve, blockPreserve = bc.shouldPreserve(currentBlock), bc.shouldPreserve(block)
}
reorg = !currentPreserve && (blockPreserve || mrand.Float64() < 0.5)
}
return nil
}
// WriteBlockWithState writes the block and all associated state to the database.
func (bc *BlockChain) WriteBlockAndSetHead(block *types.Block, receipts []*types.Receipt, logs []*types.Log, state *state.StateDB, emitHeadEvent bool) (status WriteStatus, err error) {
if !bc.chainmu.TryLock() {
return NonStatTy, errChainStopped
}
defer bc.chainmu.Unlock()
return bc.writeBlockAndSetHead(block, receipts, logs, state, emitHeadEvent)
}
// writeBlockAndSetHead writes the block and all associated state to the database,
// and also it applies the given block as the new chain head. This function expects
// the chain mutex to be held.
func (bc *BlockChain) writeBlockAndSetHead(block *types.Block, receipts []*types.Receipt, logs []*types.Log, state *state.StateDB, emitHeadEvent bool) (status WriteStatus, err error) {
if err := bc.writeBlockWithState(block, receipts, logs, state); err != nil {
return NonStatTy, err
}
currentBlock := bc.CurrentBlock()
reorg, err := bc.forker.ReorgNeeded(currentBlock.Header(), block.Header())
if err != nil {
return NonStatTy, err
}
if reorg {
// Reorganise the chain if the parent is not the head block
@ -1320,7 +1306,7 @@ func (bc *BlockChain) writeBlockWithState(block *types.Block, receipts []*types.
}
// In theory we should fire a ChainHeadEvent when we inject
// a canonical block, but sometimes we can insert a batch of
// canonicial blocks. Avoid firing too much ChainHeadEvents,
// canonicial blocks. Avoid firing too many ChainHeadEvents,
// we will fire an accumulated ChainHeadEvent and disable fire
// event here.
if emitHeadEvent {
@ -1335,11 +1321,18 @@ func (bc *BlockChain) writeBlockWithState(block *types.Block, receipts []*types.
// addFutureBlock checks if the block is within the max allowed window to get
// accepted for future processing, and returns an error if the block is too far
// ahead and was not added.
//
// TODO after the transition, the future block shouldn't be kept. Because
// it's not checked in the Geth side anymore.
func (bc *BlockChain) addFutureBlock(block *types.Block) error {
max := uint64(time.Now().Unix() + maxTimeFutureBlocks)
if block.Time() > max {
return fmt.Errorf("future block timestamp %v > allowed %v", block.Time(), max)
}
if block.Difficulty().Cmp(common.Big0) == 0 {
// Never add PoS blocks into the future queue
return nil
}
bc.futureBlocks.Add(block.Hash(), block)
return nil
}
@ -1347,15 +1340,12 @@ func (bc *BlockChain) addFutureBlock(block *types.Block) error {
// InsertChain attempts to insert the given batch of blocks in to the canonical
// chain or, otherwise, create a fork. If an error is returned it will return
// the index number of the failing block as well an error describing what went
// wrong.
//
// After insertion is done, all accumulated events will be fired.
// wrong. After insertion is done, all accumulated events will be fired.
func (bc *BlockChain) InsertChain(chain types.Blocks) (int, error) {
// Sanity check that we have something meaningful to import
if len(chain) == 0 {
return 0, nil
}
bc.blockProcFeed.Send(true)
defer bc.blockProcFeed.Send(false)
@ -1374,26 +1364,12 @@ func (bc *BlockChain) InsertChain(chain types.Blocks) (int, error) {
prev.Hash().Bytes()[:4], i, block.NumberU64(), block.Hash().Bytes()[:4], block.ParentHash().Bytes()[:4])
}
}
// Pre-check passed, start the full block imports.
// Pre-checks passed, start the full block imports
if !bc.chainmu.TryLock() {
return 0, errChainStopped
}
defer bc.chainmu.Unlock()
return bc.insertChain(chain, true)
}
// InsertChainWithoutSealVerification works exactly the same
// except for seal verification, seal verification is omitted
func (bc *BlockChain) InsertChainWithoutSealVerification(block *types.Block) (int, error) {
bc.blockProcFeed.Send(true)
defer bc.blockProcFeed.Send(false)
if !bc.chainmu.TryLock() {
return 0, errChainStopped
}
defer bc.chainmu.Unlock()
return bc.insertChain(types.Blocks([]*types.Block{block}), false)
return bc.insertChain(chain, true, true)
}
// insertChain is the internal implementation of InsertChain, which assumes that
@ -1404,7 +1380,7 @@ func (bc *BlockChain) InsertChainWithoutSealVerification(block *types.Block) (in
// 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
// completes, then the historic state could be pruned again
func (bc *BlockChain) insertChain(chain types.Blocks, verifySeals bool) (int, error) {
func (bc *BlockChain) insertChain(chain types.Blocks, verifySeals, setHead bool) (int, error) {
// If the chain is terminating, don't even bother starting up.
if bc.insertStopped() {
return 0, nil
@ -1446,14 +1422,23 @@ func (bc *BlockChain) insertChain(chain types.Blocks, verifySeals bool) (int, er
// from the canonical chain, which has not been verified.
// Skip all known blocks that are behind us.
var (
current = bc.CurrentBlock()
localTd = bc.GetTd(current.Hash(), current.NumberU64())
externTd = bc.GetTd(block.ParentHash(), block.NumberU64()-1) // The first block can't be nil
reorg bool
current = bc.CurrentBlock()
)
for block != nil && bc.skipBlock(err, it) {
externTd = new(big.Int).Add(externTd, block.Difficulty())
if localTd.Cmp(externTd) < 0 {
break
reorg, err = bc.forker.ReorgNeeded(current.Header(), block.Header())
if err != nil {
return it.index, err
}
if reorg {
// Switch to import mode if the forker says the reorg is necessary
// and also the block is not on the canonical chain.
// In eth2 the forker always returns true for reorg decision (blindly trusting
// the external consensus engine), but in order to prevent the unnecessary
// reorgs when importing known blocks, the special case is handled here.
if block.NumberU64() > current.NumberU64() || bc.GetCanonicalHash(block.NumberU64()) != block.Hash() {
break
}
}
log.Debug("Ignoring already known block", "number", block.Number(), "hash", block.Hash())
stats.ignored++
@ -1480,11 +1465,17 @@ func (bc *BlockChain) insertChain(chain types.Blocks, verifySeals bool) (int, er
// Falls through to the block import
}
switch {
// First block is pruned, insert as sidechain and reorg only if TD grows enough
// First block is pruned
case errors.Is(err, consensus.ErrPrunedAncestor):
log.Debug("Pruned ancestor, inserting as sidechain", "number", block.Number(), "hash", block.Hash())
return bc.insertSideChain(block, it)
if setHead {
// 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())
return bc.insertSideChain(block, it)
} else {
// 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())
return it.index, bc.recoverAncestors(block)
}
// First block is future, shove it (and all children) to the future queue (unknown ancestor)
case errors.Is(err, consensus.ErrFutureBlock) || (errors.Is(err, consensus.ErrUnknownAncestor) && bc.futureBlocks.Contains(it.first().ParentHash())):
for block != nil && (it.index == 0 || errors.Is(err, consensus.ErrUnknownAncestor)) {
@ -1639,12 +1630,17 @@ func (bc *BlockChain) insertChain(chain types.Blocks, verifySeals bool) (int, er
// Update the metrics touched during block validation
accountHashTimer.Update(statedb.AccountHashes) // Account hashes are complete, we can mark them
storageHashTimer.Update(statedb.StorageHashes) // Storage hashes are complete, we can mark them
blockValidationTimer.Update(time.Since(substart) - (statedb.AccountHashes + statedb.StorageHashes - triehash))
// Write the block to the chain and get the status.
substart = time.Now()
status, err := bc.writeBlockWithState(block, receipts, logs, statedb, false)
var status WriteStatus
if !setHead {
// Don't set the head, only insert the block
err = bc.writeBlockWithState(block, receipts, logs, statedb)
} else {
status, err = bc.writeBlockAndSetHead(block, receipts, logs, statedb, false)
}
atomic.StoreUint32(&followupInterrupt, 1)
if err != nil {
return it.index, err
@ -1657,6 +1653,12 @@ func (bc *BlockChain) insertChain(chain types.Blocks, verifySeals bool) (int, er
blockWriteTimer.Update(time.Since(substart) - statedb.AccountCommits - statedb.StorageCommits - statedb.SnapshotCommits)
blockInsertTimer.UpdateSince(start)
if !setHead {
// We did not setHead, so we don't have any stats to update
log.Info("Inserted block", "number", block.Number(), "hash", block.Hash(), "txs", len(block.Transactions()), "elapsed", common.PrettyDuration(time.Since(start)))
return it.index, nil
}
switch status {
case CanonStatTy:
log.Debug("Inserted new block", "number", block.Number(), "hash", block.Hash(),
@ -1715,10 +1717,12 @@ func (bc *BlockChain) insertChain(chain types.Blocks, verifySeals bool) (int, er
//
// 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.
// insertSideChain is only used pre-merge.
func (bc *BlockChain) insertSideChain(block *types.Block, it *insertIterator) (int, error) {
var (
externTd *big.Int
current = bc.CurrentBlock()
externTd *big.Int
lastBlock = block
current = bc.CurrentBlock()
)
// The first sidechain block error is already verified to be ErrPrunedAncestor.
// Since we don't import them here, we expect ErrUnknownAncestor for the remaining
@ -1769,6 +1773,7 @@ func (bc *BlockChain) insertSideChain(block *types.Block, it *insertIterator) (i
"txs", len(block.Transactions()), "gas", block.GasUsed(), "uncles", len(block.Uncles()),
"root", block.Root())
}
lastBlock = block
}
// At this point, we've written all sidechain blocks to database. Loop ended
// either on some other error or all were processed. If there was some other
@ -1776,8 +1781,12 @@ func (bc *BlockChain) insertSideChain(block *types.Block, it *insertIterator) (i
//
// If the externTd was larger than our local TD, we now need to reimport the previous
// blocks to regenerate the required state
localTd := bc.GetTd(current.Hash(), current.NumberU64())
if localTd.Cmp(externTd) > 0 {
reorg, err := bc.forker.ReorgNeeded(current.Header(), lastBlock.Header())
if err != nil {
return it.index, err
}
if !reorg {
localTd := bc.GetTd(current.Hash(), current.NumberU64())
log.Info("Sidechain written to disk", "start", it.first().NumberU64(), "end", it.previous().Number, "sidetd", externTd, "localtd", localTd)
return it.index, err
}
@ -1813,7 +1822,7 @@ func (bc *BlockChain) insertSideChain(block *types.Block, it *insertIterator) (i
// memory here.
if len(blocks) >= 2048 || memory > 64*1024*1024 {
log.Info("Importing heavy sidechain segment", "blocks", len(blocks), "start", blocks[0].NumberU64(), "end", block.NumberU64())
if _, err := bc.insertChain(blocks, false); err != nil {
if _, err := bc.insertChain(blocks, false, true); err != nil {
return 0, err
}
blocks, memory = blocks[:0], 0
@ -1827,14 +1836,98 @@ func (bc *BlockChain) insertSideChain(block *types.Block, it *insertIterator) (i
}
if len(blocks) > 0 {
log.Info("Importing sidechain segment", "start", blocks[0].NumberU64(), "end", blocks[len(blocks)-1].NumberU64())
return bc.insertChain(blocks, false)
return bc.insertChain(blocks, false, true)
}
return 0, nil
}
// recoverAncestors finds the closest ancestor with available state and re-execute
// all the ancestor blocks since that.
// recoverAncestors is only used post-merge.
func (bc *BlockChain) recoverAncestors(block *types.Block) error {
// Gather all the sidechain hashes (full blocks may be memory heavy)
var (
hashes []common.Hash
numbers []uint64
parent = block
)
for parent != nil && !bc.HasState(parent.Root()) {
hashes = append(hashes, parent.Hash())
numbers = append(numbers, parent.NumberU64())
parent = bc.GetBlock(parent.ParentHash(), parent.NumberU64()-1)
// If the chain is terminating, stop iteration
if bc.insertStopped() {
log.Debug("Abort during blocks iteration")
return errInsertionInterrupted
}
}
if parent == nil {
return errors.New("missing parent")
}
// Import all the pruned blocks to make the state available
for i := len(hashes) - 1; i >= 0; i-- {
// If the chain is terminating, stop processing blocks
if bc.insertStopped() {
log.Debug("Abort during blocks processing")
return errInsertionInterrupted
}
var b *types.Block
if i == 0 {
b = block
} else {
b = bc.GetBlock(hashes[i], numbers[i])
}
if _, err := bc.insertChain(types.Blocks{b}, false, false); err != nil {
return err
}
}
return nil
}
// collectLogs collects the logs that were generated or removed during
// the processing of the block that corresponds with the given hash.
// These logs are later announced as deleted or reborn.
func (bc *BlockChain) collectLogs(hash common.Hash, removed bool) []*types.Log {
number := bc.hc.GetBlockNumber(hash)
if number == nil {
return nil
}
receipts := rawdb.ReadReceipts(bc.db, hash, *number, bc.chainConfig)
var logs []*types.Log
for _, receipt := range receipts {
for _, log := range receipt.Logs {
l := *log
if removed {
l.Removed = true
}
logs = append(logs, &l)
}
}
return logs
}
// mergeLogs returns a merged log slice with specified sort order.
func mergeLogs(logs [][]*types.Log, reverse bool) []*types.Log {
var ret []*types.Log
if reverse {
for i := len(logs) - 1; i >= 0; i-- {
ret = append(ret, logs[i]...)
}
} else {
for i := 0; i < len(logs); i++ {
ret = append(ret, logs[i]...)
}
}
return ret
}
// reorg takes two blocks, an old chain and a new chain and will reconstruct the
// blocks and inserts them to be part of the new canonical chain and accumulates
// potential missing transactions and post an event about them.
// Note the new head block won't be processed here, callers need to handle it
// externally.
func (bc *BlockChain) reorg(oldBlock, newBlock *types.Block) error {
var (
newChain types.Blocks
@ -1846,49 +1939,6 @@ func (bc *BlockChain) reorg(oldBlock, newBlock *types.Block) error {
deletedLogs [][]*types.Log
rebirthLogs [][]*types.Log
// collectLogs collects the logs that were generated or removed during
// the processing of the block that corresponds with the given hash.
// These logs are later announced as deleted or reborn
collectLogs = func(hash common.Hash, removed bool) {
number := bc.hc.GetBlockNumber(hash)
if number == nil {
return
}
receipts := rawdb.ReadReceipts(bc.db, hash, *number, bc.chainConfig)
var logs []*types.Log
for _, receipt := range receipts {
for _, log := range receipt.Logs {
l := *log
if removed {
l.Removed = true
}
logs = append(logs, &l)
}
}
if len(logs) > 0 {
if removed {
deletedLogs = append(deletedLogs, logs)
} else {
rebirthLogs = append(rebirthLogs, logs)
}
}
}
// mergeLogs returns a merged log slice with specified sort order.
mergeLogs = func(logs [][]*types.Log, reverse bool) []*types.Log {
var ret []*types.Log
if reverse {
for i := len(logs) - 1; i >= 0; i-- {
ret = append(ret, logs[i]...)
}
} else {
for i := 0; i < len(logs); i++ {
ret = append(ret, logs[i]...)
}
}
return ret
}
)
// Reduce the longer chain to the same number as the shorter one
if oldBlock.NumberU64() > newBlock.NumberU64() {
@ -1896,7 +1946,12 @@ func (bc *BlockChain) reorg(oldBlock, newBlock *types.Block) error {
for ; oldBlock != nil && oldBlock.NumberU64() != newBlock.NumberU64(); oldBlock = bc.GetBlock(oldBlock.ParentHash(), oldBlock.NumberU64()-1) {
oldChain = append(oldChain, oldBlock)
deletedTxs = append(deletedTxs, oldBlock.Transactions()...)
collectLogs(oldBlock.Hash(), true)
// Collect deleted logs for notification
logs := bc.collectLogs(oldBlock.Hash(), true)
if len(logs) > 0 {
deletedLogs = append(deletedLogs, logs)
}
}
} else {
// New chain is longer, stash all blocks away for subsequent insertion
@ -1921,8 +1976,12 @@ func (bc *BlockChain) reorg(oldBlock, newBlock *types.Block) error {
// Remove an old block as well as stash away a new block
oldChain = append(oldChain, oldBlock)
deletedTxs = append(deletedTxs, oldBlock.Transactions()...)
collectLogs(oldBlock.Hash(), true)
// Collect deleted logs for notification
logs := bc.collectLogs(oldBlock.Hash(), true)
if len(logs) > 0 {
deletedLogs = append(deletedLogs, logs)
}
newChain = append(newChain, newBlock)
// Step back with both chains
@ -1948,8 +2007,15 @@ func (bc *BlockChain) reorg(oldBlock, newBlock *types.Block) error {
blockReorgAddMeter.Mark(int64(len(newChain)))
blockReorgDropMeter.Mark(int64(len(oldChain)))
blockReorgMeter.Mark(1)
} else if len(newChain) > 0 {
// Special case happens in the post merge stage that current head is
// the ancestor of new head while these two blocks are not consecutive
log.Info("Extend chain", "add", len(newChain), "number", newChain[0].NumberU64(), "hash", newChain[0].Hash())
blockReorgAddMeter.Mark(int64(len(newChain)))
} else {
log.Error("Impossible reorg, please file an issue", "oldnum", oldBlock.Number(), "oldhash", oldBlock.Hash(), "newnum", newBlock.Number(), "newhash", newBlock.Hash())
// len(newChain) == 0 && len(oldChain) > 0
// rewind the canonical chain to a lower point.
log.Error("Impossible reorg, please file an issue", "oldnum", oldBlock.Number(), "oldhash", oldBlock.Hash(), "oldblocks", len(oldChain), "newnum", newBlock.Number(), "newhash", newBlock.Hash(), "newblocks", len(newChain))
}
// Insert the new chain(except the head block(reverse order)),
// taking care of the proper incremental order.
@ -1958,8 +2024,10 @@ func (bc *BlockChain) reorg(oldBlock, newBlock *types.Block) error {
bc.writeHeadBlock(newChain[i])
// Collect reborn logs due to chain reorg
collectLogs(newChain[i].Hash(), false)
logs := bc.collectLogs(newChain[i].Hash(), false)
if len(logs) > 0 {
rebirthLogs = append(rebirthLogs, logs)
}
// Collect the new added transactions.
addedTxs = append(addedTxs, newChain[i].Transactions()...)
}
@ -1999,12 +2067,54 @@ func (bc *BlockChain) reorg(oldBlock, newBlock *types.Block) error {
return nil
}
// futureBlocksLoop processes the 'future block' queue.
func (bc *BlockChain) futureBlocksLoop() {
defer bc.wg.Done()
// InsertBlockWithoutSetHead executes the block, runs the necessary verification
// upon it and then persist the block and the associate state into the database.
// The key difference between the InsertChain is it won't do the canonical chain
// updating. It relies on the additional SetChainHead call to finalize the entire
// procedure.
func (bc *BlockChain) InsertBlockWithoutSetHead(block *types.Block) error {
if !bc.chainmu.TryLock() {
return errChainStopped
}
defer bc.chainmu.Unlock()
_, err := bc.insertChain(types.Blocks{block}, true, false)
return err
}
// SetChainHead rewinds the chain to set the new head block as the specified
// block. It's possible that after the reorg the relevant state of head
// is missing. It can be fixed by inserting a new block which triggers
// the re-execution.
func (bc *BlockChain) SetChainHead(newBlock *types.Block) error {
if !bc.chainmu.TryLock() {
return errChainStopped
}
defer bc.chainmu.Unlock()
// Run the reorg if necessary and set the given block as new head.
if newBlock.ParentHash() != bc.CurrentBlock().Hash() {
if err := bc.reorg(bc.CurrentBlock(), newBlock); err != nil {
return err
}
}
bc.writeHeadBlock(newBlock)
// Emit events
logs := bc.collectLogs(newBlock.Hash(), false)
bc.chainFeed.Send(ChainEvent{Block: newBlock, Hash: newBlock.Hash(), Logs: logs})
if len(logs) > 0 {
bc.logsFeed.Send(logs)
}
bc.chainHeadFeed.Send(ChainHeadEvent{Block: newBlock})
log.Info("Set the chain head", "number", newBlock.Number(), "hash", newBlock.Hash())
return nil
}
func (bc *BlockChain) updateFutureBlocks() {
futureTimer := time.NewTicker(5 * time.Second)
defer futureTimer.Stop()
defer bc.wg.Done()
for {
select {
case <-futureTimer.C:
@ -2185,6 +2295,6 @@ func (bc *BlockChain) InsertHeaderChain(chain []*types.Header, checkFreq int) (i
return 0, errChainStopped
}
defer bc.chainmu.Unlock()
_, err := bc.hc.InsertHeaderChain(chain, start)
_, err := bc.hc.InsertHeaderChain(chain, start, bc.forker)
return 0, err
}

View File

@ -1829,7 +1829,7 @@ func testRepair(t *testing.T, tt *rewindTest, snapshots bool) {
// Pull the plug on the database, simulating a hard crash
db.Close()
// Start a new blockchain back up and see where the repait leads us
// Start a new blockchain back up and see where the repair leads us
db, err = rawdb.NewLevelDBDatabaseWithFreezer(datadir, 0, 0, datadir, "", false)
if err != nil {
t.Fatalf("Failed to reopen persistent database: %v", err)

View File

@ -28,7 +28,9 @@ import (
"time"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/common/math"
"github.com/ethereum/go-ethereum/consensus"
"github.com/ethereum/go-ethereum/consensus/beacon"
"github.com/ethereum/go-ethereum/consensus/ethash"
"github.com/ethereum/go-ethereum/core/rawdb"
"github.com/ethereum/go-ethereum/core/state"
@ -211,6 +213,55 @@ func TestLastBlock(t *testing.T) {
}
}
// Test inserts the blocks/headers after the fork choice rule is changed.
// The chain is reorged to whatever specified.
func testInsertAfterMerge(t *testing.T, blockchain *BlockChain, i, n int, full bool) {
// Copy old chain up to #i into a new db
db, blockchain2, err := newCanonical(ethash.NewFaker(), i, full)
if err != nil {
t.Fatal("could not make new canonical in testFork", err)
}
defer blockchain2.Stop()
// Assert the chains have the same header/block at #i
var hash1, hash2 common.Hash
if full {
hash1 = blockchain.GetBlockByNumber(uint64(i)).Hash()
hash2 = blockchain2.GetBlockByNumber(uint64(i)).Hash()
} else {
hash1 = blockchain.GetHeaderByNumber(uint64(i)).Hash()
hash2 = blockchain2.GetHeaderByNumber(uint64(i)).Hash()
}
if hash1 != hash2 {
t.Errorf("chain content mismatch at %d: have hash %v, want hash %v", i, hash2, hash1)
}
// Extend the newly created chain
if full {
blockChainB := makeBlockChain(blockchain2.CurrentBlock(), n, ethash.NewFaker(), db, forkSeed)
if _, err := blockchain2.InsertChain(blockChainB); err != nil {
t.Fatalf("failed to insert forking chain: %v", err)
}
if blockchain2.CurrentBlock().NumberU64() != blockChainB[len(blockChainB)-1].NumberU64() {
t.Fatalf("failed to reorg to the given chain")
}
if blockchain2.CurrentBlock().Hash() != blockChainB[len(blockChainB)-1].Hash() {
t.Fatalf("failed to reorg to the given chain")
}
} else {
headerChainB := makeHeaderChain(blockchain2.CurrentHeader(), n, ethash.NewFaker(), db, forkSeed)
if _, err := blockchain2.InsertHeaderChain(headerChainB, 1); err != nil {
t.Fatalf("failed to insert forking chain: %v", err)
}
if blockchain2.CurrentHeader().Number.Uint64() != headerChainB[len(headerChainB)-1].Number.Uint64() {
t.Fatalf("failed to reorg to the given chain")
}
if blockchain2.CurrentHeader().Hash() != headerChainB[len(headerChainB)-1].Hash() {
t.Fatalf("failed to reorg to the given chain")
}
}
}
// Tests that given a starting canonical chain of a given size, it can be extended
// with various length chains.
func TestExtendCanonicalHeaders(t *testing.T) { testExtendCanonical(t, false) }
@ -239,6 +290,25 @@ func testExtendCanonical(t *testing.T, full bool) {
testFork(t, processor, length, 10, full, better)
}
// Tests that given a starting canonical chain of a given size, it can be extended
// with various length chains.
func TestExtendCanonicalHeadersAfterMerge(t *testing.T) { testExtendCanonicalAfterMerge(t, false) }
func TestExtendCanonicalBlocksAfterMerge(t *testing.T) { testExtendCanonicalAfterMerge(t, true) }
func testExtendCanonicalAfterMerge(t *testing.T, full bool) {
length := 5
// Make first chain starting from genesis
_, processor, err := newCanonical(ethash.NewFaker(), length, full)
if err != nil {
t.Fatalf("failed to make new canonical chain: %v", err)
}
defer processor.Stop()
testInsertAfterMerge(t, processor, length, 1, full)
testInsertAfterMerge(t, processor, length, 10, full)
}
// Tests that given a starting canonical chain of a given size, creating shorter
// forks do not take canonical ownership.
func TestShorterForkHeaders(t *testing.T) { testShorterFork(t, false) }
@ -269,6 +339,29 @@ func testShorterFork(t *testing.T, full bool) {
testFork(t, processor, 5, 4, full, worse)
}
// Tests that given a starting canonical chain of a given size, creating shorter
// forks do not take canonical ownership.
func TestShorterForkHeadersAfterMerge(t *testing.T) { testShorterForkAfterMerge(t, false) }
func TestShorterForkBlocksAfterMerge(t *testing.T) { testShorterForkAfterMerge(t, true) }
func testShorterForkAfterMerge(t *testing.T, full bool) {
length := 10
// Make first chain starting from genesis
_, processor, err := newCanonical(ethash.NewFaker(), length, full)
if err != nil {
t.Fatalf("failed to make new canonical chain: %v", err)
}
defer processor.Stop()
testInsertAfterMerge(t, processor, 0, 3, full)
testInsertAfterMerge(t, processor, 0, 7, full)
testInsertAfterMerge(t, processor, 1, 1, full)
testInsertAfterMerge(t, processor, 1, 7, full)
testInsertAfterMerge(t, processor, 5, 3, full)
testInsertAfterMerge(t, processor, 5, 4, full)
}
// Tests that given a starting canonical chain of a given size, creating longer
// forks do take canonical ownership.
func TestLongerForkHeaders(t *testing.T) { testLongerFork(t, false) }
@ -284,19 +377,35 @@ func testLongerFork(t *testing.T, full bool) {
}
defer processor.Stop()
// Define the difficulty comparator
better := func(td1, td2 *big.Int) {
if td2.Cmp(td1) <= 0 {
t.Errorf("total difficulty mismatch: have %v, expected more than %v", td2, td1)
}
testInsertAfterMerge(t, processor, 0, 11, full)
testInsertAfterMerge(t, processor, 0, 15, full)
testInsertAfterMerge(t, processor, 1, 10, full)
testInsertAfterMerge(t, processor, 1, 12, full)
testInsertAfterMerge(t, processor, 5, 6, full)
testInsertAfterMerge(t, processor, 5, 8, full)
}
// Tests that given a starting canonical chain of a given size, creating longer
// forks do take canonical ownership.
func TestLongerForkHeadersAfterMerge(t *testing.T) { testLongerForkAfterMerge(t, false) }
func TestLongerForkBlocksAfterMerge(t *testing.T) { testLongerForkAfterMerge(t, true) }
func testLongerForkAfterMerge(t *testing.T, full bool) {
length := 10
// Make first chain starting from genesis
_, processor, err := newCanonical(ethash.NewFaker(), length, full)
if err != nil {
t.Fatalf("failed to make new canonical chain: %v", err)
}
// Sum of numbers must be greater than `length` for this to be a longer fork
testFork(t, processor, 0, 11, full, better)
testFork(t, processor, 0, 15, full, better)
testFork(t, processor, 1, 10, full, better)
testFork(t, processor, 1, 12, full, better)
testFork(t, processor, 5, 6, full, better)
testFork(t, processor, 5, 8, full, better)
defer processor.Stop()
testInsertAfterMerge(t, processor, 0, 11, full)
testInsertAfterMerge(t, processor, 0, 15, full)
testInsertAfterMerge(t, processor, 1, 10, full)
testInsertAfterMerge(t, processor, 1, 12, full)
testInsertAfterMerge(t, processor, 5, 6, full)
testInsertAfterMerge(t, processor, 5, 8, full)
}
// Tests that given a starting canonical chain of a given size, creating equal
@ -329,6 +438,29 @@ func testEqualFork(t *testing.T, full bool) {
testFork(t, processor, 9, 1, full, equal)
}
// Tests that given a starting canonical chain of a given size, creating equal
// forks do take canonical ownership.
func TestEqualForkHeadersAfterMerge(t *testing.T) { testEqualForkAfterMerge(t, false) }
func TestEqualForkBlocksAfterMerge(t *testing.T) { testEqualForkAfterMerge(t, true) }
func testEqualForkAfterMerge(t *testing.T, full bool) {
length := 10
// Make first chain starting from genesis
_, processor, err := newCanonical(ethash.NewFaker(), length, full)
if err != nil {
t.Fatalf("failed to make new canonical chain: %v", err)
}
defer processor.Stop()
testInsertAfterMerge(t, processor, 0, 10, full)
testInsertAfterMerge(t, processor, 1, 9, full)
testInsertAfterMerge(t, processor, 2, 8, full)
testInsertAfterMerge(t, processor, 5, 5, full)
testInsertAfterMerge(t, processor, 6, 4, full)
testInsertAfterMerge(t, processor, 9, 1, full)
}
// Tests that chains missing links do not get accepted by the processor.
func TestBrokenHeaderChain(t *testing.T) { testBrokenChain(t, false) }
func TestBrokenBlockChain(t *testing.T) { testBrokenChain(t, true) }
@ -1801,21 +1933,56 @@ func TestLowDiffLongChain(t *testing.T) {
// - C is canon chain, containing blocks [G..Cn..Cm]
// - A common ancestor is placed at prune-point + blocksBetweenCommonAncestorAndPruneblock
// - The sidechain S is prepended with numCanonBlocksInSidechain blocks from the canon chain
func testSideImport(t *testing.T, numCanonBlocksInSidechain, blocksBetweenCommonAncestorAndPruneblock int) {
//
// The mergePoint can be these values:
// -1: the transition won't happen
// 0: the transition happens since genesis
// 1: the transition happens after some chain segments
func testSideImport(t *testing.T, numCanonBlocksInSidechain, blocksBetweenCommonAncestorAndPruneblock int, mergePoint int) {
// Copy the TestChainConfig so we can modify it during tests
chainConfig := *params.TestChainConfig
// Generate a canonical chain to act as the main dataset
engine := ethash.NewFaker()
db := rawdb.NewMemoryDatabase()
genesis := (&Genesis{BaseFee: big.NewInt(params.InitialBaseFee)}).MustCommit(db)
var (
merger = consensus.NewMerger(rawdb.NewMemoryDatabase())
genEngine = beacon.New(ethash.NewFaker())
runEngine = beacon.New(ethash.NewFaker())
db = rawdb.NewMemoryDatabase()
key, _ = crypto.HexToECDSA("b71c71a67e1177ad4e901695e1b4b9ee17ae16c6668d313eac2f96dbcda3f291")
addr = crypto.PubkeyToAddress(key.PublicKey)
nonce = uint64(0)
gspec = &Genesis{
Config: &chainConfig,
Alloc: GenesisAlloc{addr: {Balance: big.NewInt(math.MaxInt64)}},
BaseFee: big.NewInt(params.InitialBaseFee),
}
signer = types.LatestSigner(gspec.Config)
genesis, _ = gspec.Commit(db)
)
// Generate and import the canonical chain
blocks, _ := GenerateChain(params.TestChainConfig, genesis, engine, db, 2*TriesInMemory, nil)
diskdb := rawdb.NewMemoryDatabase()
(&Genesis{BaseFee: big.NewInt(params.InitialBaseFee)}).MustCommit(diskdb)
chain, err := NewBlockChain(diskdb, nil, params.TestChainConfig, engine, vm.Config{}, nil, nil)
gspec.MustCommit(diskdb)
chain, err := NewBlockChain(diskdb, nil, &chainConfig, runEngine, vm.Config{}, nil, nil)
if err != nil {
t.Fatalf("failed to create tester chain: %v", err)
}
// Activate the transition since genesis if required
if mergePoint == 0 {
merger.ReachTTD()
merger.FinalizePoS()
// Set the terminal total difficulty in the config
gspec.Config.TerminalTotalDifficulty = big.NewInt(0)
}
blocks, _ := GenerateChain(&chainConfig, genesis, genEngine, db, 2*TriesInMemory, func(i int, gen *BlockGen) {
tx, err := types.SignTx(types.NewTransaction(nonce, common.HexToAddress("deadbeef"), big.NewInt(100), 21000, big.NewInt(int64(i+1)*params.GWei), nil), signer, key)
if err != nil {
t.Fatalf("failed to create tx: %v", err)
}
gen.AddTx(tx)
nonce++
})
if n, err := chain.InsertChain(blocks); err != nil {
t.Fatalf("block %d: failed to insert into chain: %v", n, err)
}
@ -1832,6 +1999,15 @@ func testSideImport(t *testing.T, numCanonBlocksInSidechain, blocksBetweenCommon
if !chain.HasBlockAndState(firstNonPrunedBlock.Hash(), firstNonPrunedBlock.NumberU64()) {
t.Errorf("Block %d pruned", firstNonPrunedBlock.NumberU64())
}
// Activate the transition in the middle of the chain
if mergePoint == 1 {
merger.ReachTTD()
merger.FinalizePoS()
// Set the terminal total difficulty in the config
gspec.Config.TerminalTotalDifficulty = big.NewInt(int64(len(blocks)))
}
// Generate the sidechain
// First block should be a known block, block after should be a pruned block. So
// canon(pruned), side, side...
@ -1839,7 +2015,7 @@ func testSideImport(t *testing.T, numCanonBlocksInSidechain, blocksBetweenCommon
// Generate fork chain, make it longer than canon
parentIndex := lastPrunedIndex + blocksBetweenCommonAncestorAndPruneblock
parent := blocks[parentIndex]
fork, _ := GenerateChain(params.TestChainConfig, parent, engine, db, 2*TriesInMemory, func(i int, b *BlockGen) {
fork, _ := GenerateChain(&chainConfig, parent, genEngine, db, 2*TriesInMemory, func(i int, b *BlockGen) {
b.SetCoinbase(common.Address{2})
})
// Prepend the parent(s)
@ -1848,9 +2024,9 @@ func testSideImport(t *testing.T, numCanonBlocksInSidechain, blocksBetweenCommon
sidechain = append(sidechain, blocks[parentIndex+1-i])
}
sidechain = append(sidechain, fork...)
_, err = chain.InsertChain(sidechain)
n, err := chain.InsertChain(sidechain)
if err != nil {
t.Errorf("Got error, %v", err)
t.Errorf("Got error, %v number %d - %d", err, sidechain[n].NumberU64(), n)
}
head := chain.CurrentBlock()
if got := fork[len(fork)-1].Hash(); got != head.Hash() {
@ -1871,11 +2047,28 @@ func TestPrunedImportSide(t *testing.T) {
//glogger := log.NewGlogHandler(log.StreamHandler(os.Stdout, log.TerminalFormat(false)))
//glogger.Verbosity(3)
//log.Root().SetHandler(log.Handler(glogger))
testSideImport(t, 3, 3)
testSideImport(t, 3, -3)
testSideImport(t, 10, 0)
testSideImport(t, 1, 10)
testSideImport(t, 1, -10)
testSideImport(t, 3, 3, -1)
testSideImport(t, 3, -3, -1)
testSideImport(t, 10, 0, -1)
testSideImport(t, 1, 10, -1)
testSideImport(t, 1, -10, -1)
}
func TestPrunedImportSideWithMerging(t *testing.T) {
//glogger := log.NewGlogHandler(log.StreamHandler(os.Stdout, log.TerminalFormat(false)))
//glogger.Verbosity(3)
//log.Root().SetHandler(log.Handler(glogger))
testSideImport(t, 3, 3, 0)
testSideImport(t, 3, -3, 0)
testSideImport(t, 10, 0, 0)
testSideImport(t, 1, 10, 0)
testSideImport(t, 1, -10, 0)
testSideImport(t, 3, 3, 1)
testSideImport(t, 3, -3, 1)
testSideImport(t, 10, 0, 1)
testSideImport(t, 1, 10, 1)
testSideImport(t, 1, -10, 1)
}
func TestInsertKnownHeaders(t *testing.T) { testInsertKnownChainData(t, "headers") }
@ -2003,6 +2196,179 @@ func testInsertKnownChainData(t *testing.T, typ string) {
asserter(t, blocks2[len(blocks2)-1])
}
func TestInsertKnownHeadersWithMerging(t *testing.T) {
testInsertKnownChainDataWithMerging(t, "headers", 0)
}
func TestInsertKnownReceiptChainWithMerging(t *testing.T) {
testInsertKnownChainDataWithMerging(t, "receipts", 0)
}
func TestInsertKnownBlocksWithMerging(t *testing.T) {
testInsertKnownChainDataWithMerging(t, "blocks", 0)
}
func TestInsertKnownHeadersAfterMerging(t *testing.T) {
testInsertKnownChainDataWithMerging(t, "headers", 1)
}
func TestInsertKnownReceiptChainAfterMerging(t *testing.T) {
testInsertKnownChainDataWithMerging(t, "receipts", 1)
}
func TestInsertKnownBlocksAfterMerging(t *testing.T) {
testInsertKnownChainDataWithMerging(t, "blocks", 1)
}
// mergeHeight can be assigned in these values:
// 0: means the merging is applied since genesis
// 1: means the merging is applied after the first segment
func testInsertKnownChainDataWithMerging(t *testing.T, typ string, mergeHeight int) {
// Copy the TestChainConfig so we can modify it during tests
chainConfig := *params.TestChainConfig
var (
db = rawdb.NewMemoryDatabase()
genesis = (&Genesis{BaseFee: big.NewInt(params.InitialBaseFee), Config: &chainConfig}).MustCommit(db)
runMerger = consensus.NewMerger(db)
runEngine = beacon.New(ethash.NewFaker())
genEngine = beacon.New(ethash.NewFaker())
)
applyMerge := func(engine *beacon.Beacon, height int) {
if engine != nil {
runMerger.FinalizePoS()
// Set the terminal total difficulty in the config
chainConfig.TerminalTotalDifficulty = big.NewInt(int64(height))
}
}
// Apply merging since genesis
if mergeHeight == 0 {
applyMerge(genEngine, 0)
}
blocks, receipts := GenerateChain(&chainConfig, genesis, genEngine, db, 32, func(i int, b *BlockGen) { b.SetCoinbase(common.Address{1}) })
// Apply merging after the first segment
if mergeHeight == 1 {
applyMerge(genEngine, len(blocks))
}
// Longer chain and shorter chain
blocks2, receipts2 := GenerateChain(&chainConfig, blocks[len(blocks)-1], genEngine, db, 65, func(i int, b *BlockGen) { b.SetCoinbase(common.Address{1}) })
blocks3, receipts3 := GenerateChain(&chainConfig, blocks[len(blocks)-1], genEngine, db, 64, func(i int, b *BlockGen) {
b.SetCoinbase(common.Address{1})
b.OffsetTime(-9) // Time shifted, difficulty shouldn't be changed
})
// Import the shared chain and the original canonical one
dir, err := ioutil.TempDir("", "")
if err != nil {
t.Fatalf("failed to create temp freezer dir: %v", err)
}
defer os.Remove(dir)
chaindb, err := rawdb.NewDatabaseWithFreezer(rawdb.NewMemoryDatabase(), dir, "", false)
if err != nil {
t.Fatalf("failed to create temp freezer db: %v", err)
}
(&Genesis{BaseFee: big.NewInt(params.InitialBaseFee)}).MustCommit(chaindb)
defer os.RemoveAll(dir)
chain, err := NewBlockChain(chaindb, nil, &chainConfig, runEngine, vm.Config{}, nil, nil)
if err != nil {
t.Fatalf("failed to create tester chain: %v", err)
}
var (
inserter func(blocks []*types.Block, receipts []types.Receipts) error
asserter func(t *testing.T, block *types.Block)
)
if typ == "headers" {
inserter = func(blocks []*types.Block, receipts []types.Receipts) error {
headers := make([]*types.Header, 0, len(blocks))
for _, block := range blocks {
headers = append(headers, block.Header())
}
_, err := chain.InsertHeaderChain(headers, 1)
return err
}
asserter = func(t *testing.T, block *types.Block) {
if chain.CurrentHeader().Hash() != block.Hash() {
t.Fatalf("current head header mismatch, have %v, want %v", chain.CurrentHeader().Hash().Hex(), block.Hash().Hex())
}
}
} else if typ == "receipts" {
inserter = func(blocks []*types.Block, receipts []types.Receipts) error {
headers := make([]*types.Header, 0, len(blocks))
for _, block := range blocks {
headers = append(headers, block.Header())
}
_, err := chain.InsertHeaderChain(headers, 1)
if err != nil {
return err
}
_, err = chain.InsertReceiptChain(blocks, receipts, 0)
return err
}
asserter = func(t *testing.T, block *types.Block) {
if chain.CurrentFastBlock().Hash() != block.Hash() {
t.Fatalf("current head fast block mismatch, have %v, want %v", chain.CurrentFastBlock().Hash().Hex(), block.Hash().Hex())
}
}
} else {
inserter = func(blocks []*types.Block, receipts []types.Receipts) error {
_, err := chain.InsertChain(blocks)
return err
}
asserter = func(t *testing.T, block *types.Block) {
if chain.CurrentBlock().Hash() != block.Hash() {
t.Fatalf("current head block mismatch, have %v, want %v", chain.CurrentBlock().Hash().Hex(), block.Hash().Hex())
}
}
}
// Apply merging since genesis if required
if mergeHeight == 0 {
applyMerge(runEngine, 0)
}
if err := inserter(blocks, receipts); err != nil {
t.Fatalf("failed to insert chain data: %v", err)
}
// Reimport the chain data again. All the imported
// chain data are regarded "known" data.
if err := inserter(blocks, receipts); err != nil {
t.Fatalf("failed to insert chain data: %v", err)
}
asserter(t, blocks[len(blocks)-1])
// Import a long canonical chain with some known data as prefix.
rollback := blocks[len(blocks)/2].NumberU64()
chain.SetHead(rollback - 1)
if err := inserter(blocks, receipts); err != nil {
t.Fatalf("failed to insert chain data: %v", err)
}
asserter(t, blocks[len(blocks)-1])
// Apply merging after the first segment
if mergeHeight == 1 {
applyMerge(runEngine, len(blocks))
}
// Import a longer chain with some known data as prefix.
if err := inserter(append(blocks, blocks2...), append(receipts, receipts2...)); err != nil {
t.Fatalf("failed to insert chain data: %v", err)
}
asserter(t, blocks2[len(blocks2)-1])
// Import a shorter chain with some known data as prefix.
// The reorg is expected since the fork choice rule is
// already changed.
if err := inserter(append(blocks, blocks3...), append(receipts, receipts3...)); err != nil {
t.Fatalf("failed to insert chain data: %v", err)
}
// The head shouldn't change.
asserter(t, blocks3[len(blocks3)-1])
// Reimport the longer chain again, the reorg is still expected
chain.SetHead(rollback - 1)
if err := inserter(append(blocks, blocks2...), append(receipts, receipts2...)); err != nil {
t.Fatalf("failed to insert chain data: %v", err)
}
asserter(t, blocks2[len(blocks2)-1])
}
// getLongAndShortChains returns two chains: A is longer, B is heavier.
func getLongAndShortChains() (bc *BlockChain, longChain []*types.Block, heavyChain []*types.Block, err error) {
// Generate a canonical chain to act as the main dataset
@ -2483,6 +2849,7 @@ func TestSideImportPrunedBlocks(t *testing.T) {
// Generate and import the canonical chain
blocks, _ := GenerateChain(params.TestChainConfig, genesis, engine, db, 2*TriesInMemory, nil)
diskdb := rawdb.NewMemoryDatabase()
(&Genesis{BaseFee: big.NewInt(params.InitialBaseFee)}).MustCommit(diskdb)
chain, err := NewBlockChain(diskdb, nil, params.TestChainConfig, engine, vm.Config{}, nil, nil)
if err != nil {

View File

@ -205,6 +205,18 @@ func GenerateChain(config *params.ChainConfig, parent *types.Block, engine conse
b := &BlockGen{i: i, chain: blocks, parent: parent, statedb: statedb, config: config, engine: engine}
b.header = makeHeader(chainreader, parent, statedb, b.engine)
// Set the difficulty for clique block. The chain maker doesn't have access
// to a chain, so the difficulty will be left unset (nil). Set it here to the
// correct value.
if b.header.Difficulty == nil {
if config.TerminalTotalDifficulty == nil {
// Clique chain
b.header.Difficulty = big.NewInt(2)
} else {
// Post-merge chain
b.header.Difficulty = big.NewInt(0)
}
}
// Mutate the state and block according to any hard-fork specs
if daoBlock := config.DAOForkBlock; daoBlock != nil {
limit := new(big.Int).Add(daoBlock, params.DAOForkExtraRange)
@ -313,3 +325,4 @@ func (cr *fakeChainReader) GetHeaderByNumber(number uint64) *types.Header
func (cr *fakeChainReader) GetHeaderByHash(hash common.Hash) *types.Header { return nil }
func (cr *fakeChainReader) GetHeader(hash common.Hash, number uint64) *types.Header { return nil }
func (cr *fakeChainReader) GetBlock(hash common.Hash, number uint64) *types.Block { return nil }
func (cr *fakeChainReader) GetTd(hash common.Hash, number uint64) *big.Int { return nil }

108
core/forkchoice.go Normal file
View File

@ -0,0 +1,108 @@
// Copyright 2021 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package core
import (
crand "crypto/rand"
"errors"
"math/big"
mrand "math/rand"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/common/math"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/params"
)
// ChainReader defines a small collection of methods needed to access the local
// blockchain during header verification. It's implemented by both blockchain
// and lightchain.
type ChainReader interface {
// Config retrieves the header chain's chain configuration.
Config() *params.ChainConfig
// GetTd returns the total difficulty of a local block.
GetTd(common.Hash, uint64) *big.Int
}
// ForkChoice is the fork chooser based on the highest total difficulty of the
// chain(the fork choice used in the eth1) and the external fork choice (the fork
// choice used in the eth2). This main goal of this ForkChoice is not only for
// offering fork choice during the eth1/2 merge phase, but also keep the compatibility
// for all other proof-of-work networks.
type ForkChoice struct {
chain ChainReader
rand *mrand.Rand
// preserve is a helper function used in td fork choice.
// Miners will prefer to choose the local mined block if the
// local td is equal to the extern one. It can be nil for light
// client
preserve func(header *types.Header) bool
}
func NewForkChoice(chainReader ChainReader, preserve func(header *types.Header) bool) *ForkChoice {
// Seed a fast but crypto originating random generator
seed, err := crand.Int(crand.Reader, big.NewInt(math.MaxInt64))
if err != nil {
log.Crit("Failed to initialize random seed", "err", err)
}
return &ForkChoice{
chain: chainReader,
rand: mrand.New(mrand.NewSource(seed.Int64())),
preserve: preserve,
}
}
// ReorgNeeded returns whether the reorg should be applied
// based on the given external header and local canonical chain.
// In the td mode, the new head is chosen if the corresponding
// total difficulty is higher. In the extern mode, the trusted
// header is always selected as the head.
func (f *ForkChoice) ReorgNeeded(current *types.Header, header *types.Header) (bool, error) {
var (
localTD = f.chain.GetTd(current.Hash(), current.Number.Uint64())
externTd = f.chain.GetTd(header.Hash(), header.Number.Uint64())
)
if localTD == nil || externTd == nil {
return false, errors.New("missing td")
}
// Accept the new header as the chain head if the transition
// is already triggered. We assume all the headers after the
// transition come from the trusted consensus layer.
if ttd := f.chain.Config().TerminalTotalDifficulty; ttd != nil && ttd.Cmp(externTd) <= 0 {
return true, nil
}
// If the total difficulty is higher than our known, add it to the canonical chain
// Second clause in the if statement reduces the vulnerability to selfish mining.
// Please refer to http://www.cs.cornell.edu/~ie53/publications/btcProcFC.pdf
reorg := externTd.Cmp(localTD) > 0
if !reorg && externTd.Cmp(localTD) == 0 {
number, headNumber := header.Number.Uint64(), current.Number.Uint64()
if number < headNumber {
reorg = true
} else if number == headNumber {
var currentPreserve, externPreserve bool
if f.preserve != nil {
currentPreserve, externPreserve = f.preserve(current), f.preserve(header)
}
reorg = !currentPreserve && (externPreserve || f.rand.Float64() < 0.5)
}
}
return reorg, nil
}

View File

@ -155,10 +155,10 @@ func (e *GenesisMismatchError) Error() string {
//
// The returned chain configuration is never nil.
func SetupGenesisBlock(db ethdb.Database, genesis *Genesis) (*params.ChainConfig, common.Hash, error) {
return SetupGenesisBlockWithOverride(db, genesis, nil)
return SetupGenesisBlockWithOverride(db, genesis, nil, nil)
}
func SetupGenesisBlockWithOverride(db ethdb.Database, genesis *Genesis, overrideArrowGlacier *big.Int) (*params.ChainConfig, common.Hash, error) {
func SetupGenesisBlockWithOverride(db ethdb.Database, genesis *Genesis, overrideArrowGlacier, overrideTerminalTotalDifficulty *big.Int) (*params.ChainConfig, common.Hash, error) {
if genesis != nil && genesis.Config == nil {
return params.AllEthashProtocolChanges, common.Hash{}, errGenesisNoConfig
}
@ -207,6 +207,9 @@ func SetupGenesisBlockWithOverride(db ethdb.Database, genesis *Genesis, override
if overrideArrowGlacier != nil {
newcfg.ArrowGlacierBlock = overrideArrowGlacier
}
if overrideTerminalTotalDifficulty != nil {
newcfg.TerminalTotalDifficulty = overrideTerminalTotalDifficulty
}
if err := newcfg.CheckConfigForkOrder(); err != nil {
return newcfg, common.Hash{}, err
}

View File

@ -49,15 +49,14 @@ const (
// HeaderChain is responsible for maintaining the header chain including the
// header query and updating.
//
// The components maintained by headerchain includes: (1) total difficult
// The components maintained by headerchain includes: (1) total difficulty
// (2) header (3) block hash -> number mapping (4) canonical number -> hash mapping
// and (5) head header flag.
//
// It is not thread safe either, the encapsulating chain structures should do
// the necessary mutex locking/unlocking.
type HeaderChain struct {
config *params.ChainConfig
config *params.ChainConfig
chainDb ethdb.Database
genesisHeader *types.Header
@ -86,7 +85,6 @@ func NewHeaderChain(chainDb ethdb.Database, config *params.ChainConfig, engine c
if err != nil {
return nil, err
}
hc := &HeaderChain{
config: config,
chainDb: chainDb,
@ -97,12 +95,10 @@ func NewHeaderChain(chainDb ethdb.Database, config *params.ChainConfig, engine c
rand: mrand.New(mrand.NewSource(seed.Int64())),
engine: engine,
}
hc.genesisHeader = hc.GetHeaderByNumber(0)
if hc.genesisHeader == nil {
return nil, ErrNoGenesis
}
hc.currentHeader.Store(hc.genesisHeader)
if head := rawdb.ReadHeadBlockHash(chainDb); head != (common.Hash{}) {
if chead := hc.GetHeaderByHash(head); chead != nil {
@ -111,7 +107,6 @@ func NewHeaderChain(chainDb ethdb.Database, config *params.ChainConfig, engine c
}
hc.currentHeaderHash = hc.CurrentHeader().Hash()
headHeaderGauge.Update(hc.CurrentHeader().Number.Int64())
return hc, nil
}
@ -137,35 +132,93 @@ type headerWriteResult struct {
lastHeader *types.Header
}
// WriteHeaders writes a chain of headers into the local chain, given that the parents
// are already known. If the total difficulty of the newly inserted chain becomes
// greater than the current known TD, the canonical chain is reorged.
//
// Note: This method is not concurrent-safe with inserting blocks simultaneously
// into the chain, as side effects caused by reorganisations cannot be emulated
// without the real blocks. Hence, writing headers directly should only be done
// in two scenarios: pure-header mode of operation (light clients), or properly
// separated header/block phases (non-archive clients).
func (hc *HeaderChain) writeHeaders(headers []*types.Header) (result *headerWriteResult, err error) {
// Reorg reorgs the local canonical chain into the specified chain. The reorg
// can be classified into two cases: (a) extend the local chain (b) switch the
// head to the given header.
func (hc *HeaderChain) Reorg(headers []*types.Header) error {
// Short circuit if nothing to reorg.
if len(headers) == 0 {
return &headerWriteResult{}, nil
return nil
}
// If the parent of the (first) block is already the canon header,
// we don't have to go backwards to delete canon blocks, but simply
// pile them onto the existing chain. Otherwise, do the necessary
// reorgs.
var (
first = headers[0]
last = headers[len(headers)-1]
batch = hc.chainDb.NewBatch()
)
if first.ParentHash != hc.currentHeaderHash {
// Delete any canonical number assignments above the new head
for i := last.Number.Uint64() + 1; ; i++ {
hash := rawdb.ReadCanonicalHash(hc.chainDb, i)
if hash == (common.Hash{}) {
break
}
rawdb.DeleteCanonicalHash(batch, i)
}
// Overwrite any stale canonical number assignments, going
// backwards from the first header in this import until the
// cross link between two chains.
var (
header = first
headNumber = header.Number.Uint64()
headHash = header.Hash()
)
for rawdb.ReadCanonicalHash(hc.chainDb, headNumber) != headHash {
rawdb.WriteCanonicalHash(batch, headHash, headNumber)
if headNumber == 0 {
break // It shouldn't be reached
}
headHash, headNumber = header.ParentHash, header.Number.Uint64()-1
header = hc.GetHeader(headHash, headNumber)
if header == nil {
return fmt.Errorf("missing parent %d %x", headNumber, headHash)
}
}
}
// Extend the canonical chain with the new headers
for i := 0; i < len(headers)-1; i++ {
hash := headers[i+1].ParentHash // Save some extra hashing
num := headers[i].Number.Uint64()
rawdb.WriteCanonicalHash(batch, hash, num)
rawdb.WriteHeadHeaderHash(batch, hash)
}
// Write the last header
hash := headers[len(headers)-1].Hash()
num := headers[len(headers)-1].Number.Uint64()
rawdb.WriteCanonicalHash(batch, hash, num)
rawdb.WriteHeadHeaderHash(batch, hash)
if err := batch.Write(); err != nil {
return err
}
// Last step update all in-memory head header markers
hc.currentHeaderHash = last.Hash()
hc.currentHeader.Store(types.CopyHeader(last))
headHeaderGauge.Update(last.Number.Int64())
return nil
}
// WriteHeaders writes a chain of headers into the local chain, given that the
// parents are already known. The chain head header won't be updated in this
// function, the additional setChainHead is expected in order to finish the entire
// procedure.
func (hc *HeaderChain) WriteHeaders(headers []*types.Header) (int, error) {
if len(headers) == 0 {
return 0, nil
}
ptd := hc.GetTd(headers[0].ParentHash, headers[0].Number.Uint64()-1)
if ptd == nil {
return &headerWriteResult{}, consensus.ErrUnknownAncestor
return 0, consensus.ErrUnknownAncestor
}
var (
lastNumber = headers[0].Number.Uint64() - 1 // Last successfully imported number
lastHash = headers[0].ParentHash // Last imported header hash
newTD = new(big.Int).Set(ptd) // Total difficulty of inserted chain
lastHeader *types.Header
inserted []numberHash // Ephemeral lookup of number/hash for the chain
firstInserted = -1 // Index of the first non-ignored header
newTD = new(big.Int).Set(ptd) // Total difficulty of inserted chain
inserted []rawdb.NumberHash // Ephemeral lookup of number/hash for the chain
parentKnown = true // Set to true to force hc.HasHeader check the first iteration
batch = hc.chainDb.NewBatch()
)
batch := hc.chainDb.NewBatch()
parentKnown := true // Set to true to force hc.HasHeader check the first iteration
for i, header := range headers {
var hash common.Hash
// The headers have already been validated at this point, so we already
@ -188,116 +241,67 @@ func (hc *HeaderChain) writeHeaders(headers []*types.Header) (result *headerWrit
hc.tdCache.Add(hash, new(big.Int).Set(newTD))
rawdb.WriteHeader(batch, header)
inserted = append(inserted, numberHash{number, hash})
inserted = append(inserted, rawdb.NumberHash{Number: number, Hash: hash})
hc.headerCache.Add(hash, header)
hc.numberCache.Add(hash, number)
if firstInserted < 0 {
firstInserted = i
}
}
parentKnown = alreadyKnown
lastHeader, lastHash, lastNumber = header, hash, number
}
// Skip the slow disk write of all headers if interrupted.
if hc.procInterrupt() {
log.Debug("Premature abort during headers import")
return &headerWriteResult{}, errors.New("aborted")
return 0, errors.New("aborted")
}
// Commit to disk!
if err := batch.Write(); err != nil {
log.Crit("Failed to write headers", "error", err)
}
batch.Reset()
return len(inserted), nil
}
// writeHeadersAndSetHead writes a batch of block headers and applies the last
// header as the chain head if the fork choicer says it's ok to update the chain.
// Note: This method is not concurrent-safe with inserting blocks simultaneously
// into the chain, as side effects caused by reorganisations cannot be emulated
// without the real blocks. Hence, writing headers directly should only be done
// in two scenarios: pure-header mode of operation (light clients), or properly
// separated header/block phases (non-archive clients).
func (hc *HeaderChain) writeHeadersAndSetHead(headers []*types.Header, forker *ForkChoice) (*headerWriteResult, error) {
inserted, err := hc.WriteHeaders(headers)
if err != nil {
return nil, err
}
var (
head = hc.CurrentHeader().Number.Uint64()
localTD = hc.GetTd(hc.currentHeaderHash, head)
status = SideStatTy
lastHeader = headers[len(headers)-1]
lastHash = headers[len(headers)-1].Hash()
result = &headerWriteResult{
status: NonStatTy,
ignored: len(headers) - inserted,
imported: inserted,
lastHash: lastHash,
lastHeader: lastHeader,
}
)
// If the total difficulty is higher than our known, add it to the canonical chain
// Second clause in the if statement reduces the vulnerability to selfish mining.
// Please refer to http://www.cs.cornell.edu/~ie53/publications/btcProcFC.pdf
reorg := newTD.Cmp(localTD) > 0
if !reorg && newTD.Cmp(localTD) == 0 {
if lastNumber < head {
reorg = true
} else if lastNumber == head {
reorg = mrand.Float64() < 0.5
// Ask the fork choicer if the reorg is necessary
if reorg, err := forker.ReorgNeeded(hc.CurrentHeader(), lastHeader); err != nil {
return nil, err
} else if !reorg {
if inserted != 0 {
result.status = SideStatTy
}
return result, nil
}
// If the parent of the (first) block is already the canon header,
// we don't have to go backwards to delete canon blocks, but
// simply pile them onto the existing chain
chainAlreadyCanon := headers[0].ParentHash == hc.currentHeaderHash
if reorg {
// If the header can be added into canonical chain, adjust the
// header chain markers(canonical indexes and head header flag).
//
// Note all markers should be written atomically.
markerBatch := batch // we can reuse the batch to keep allocs down
if !chainAlreadyCanon {
// Delete any canonical number assignments above the new head
for i := lastNumber + 1; ; i++ {
hash := rawdb.ReadCanonicalHash(hc.chainDb, i)
if hash == (common.Hash{}) {
break
}
rawdb.DeleteCanonicalHash(markerBatch, i)
}
// Overwrite any stale canonical number assignments, going
// backwards from the first header in this import
var (
headHash = headers[0].ParentHash // inserted[0].parent?
headNumber = headers[0].Number.Uint64() - 1 // inserted[0].num-1 ?
headHeader = hc.GetHeader(headHash, headNumber)
)
for rawdb.ReadCanonicalHash(hc.chainDb, headNumber) != headHash {
rawdb.WriteCanonicalHash(markerBatch, headHash, headNumber)
headHash = headHeader.ParentHash
headNumber = headHeader.Number.Uint64() - 1
headHeader = hc.GetHeader(headHash, headNumber)
}
// If some of the older headers were already known, but obtained canon-status
// during this import batch, then we need to write that now
// Further down, we continue writing the staus for the ones that
// were not already known
for i := 0; i < firstInserted; i++ {
hash := headers[i].Hash()
num := headers[i].Number.Uint64()
rawdb.WriteCanonicalHash(markerBatch, hash, num)
rawdb.WriteHeadHeaderHash(markerBatch, hash)
}
}
// Extend the canonical chain with the new headers
for _, hn := range inserted {
rawdb.WriteCanonicalHash(markerBatch, hn.hash, hn.number)
rawdb.WriteHeadHeaderHash(markerBatch, hn.hash)
}
if err := markerBatch.Write(); err != nil {
log.Crit("Failed to write header markers into disk", "err", err)
}
markerBatch.Reset()
// Last step update all in-memory head header markers
hc.currentHeaderHash = lastHash
hc.currentHeader.Store(types.CopyHeader(lastHeader))
headHeaderGauge.Update(lastHeader.Number.Int64())
// Chain status is canonical since this insert was a reorg.
// Note that all inserts which have higher TD than existing are 'reorg'.
status = CanonStatTy
// Special case, all the inserted headers are already on the canonical
// header chain, skip the reorg operation.
if hc.GetCanonicalHash(lastHeader.Number.Uint64()) == lastHash && lastHeader.Number.Uint64() <= hc.CurrentHeader().Number.Uint64() {
return result, nil
}
if len(inserted) == 0 {
status = NonStatTy
// Apply the reorg operation
if err := hc.Reorg(headers); err != nil {
return nil, err
}
return &headerWriteResult{
status: status,
ignored: len(headers) - len(inserted),
imported: len(inserted),
lastHash: lastHash,
lastHeader: lastHeader,
}, nil
result.status = CanonStatTy
return result, nil
}
func (hc *HeaderChain) ValidateHeaderChain(chain []*types.Header, checkFreq int) (int, error) {
@ -357,7 +361,7 @@ func (hc *HeaderChain) ValidateHeaderChain(chain []*types.Header, checkFreq int)
return 0, nil
}
// InsertHeaderChain inserts the given headers.
// InsertHeaderChain inserts the given headers and does the reorganisations.
//
// The validity of the headers is NOT CHECKED by this method, i.e. they need to be
// validated by ValidateHeaderChain before calling InsertHeaderChain.
@ -367,20 +371,19 @@ func (hc *HeaderChain) ValidateHeaderChain(chain []*types.Header, checkFreq int)
//
// The returned 'write status' says if the inserted headers are part of the canonical chain
// or a side chain.
func (hc *HeaderChain) InsertHeaderChain(chain []*types.Header, start time.Time) (WriteStatus, error) {
func (hc *HeaderChain) InsertHeaderChain(chain []*types.Header, start time.Time, forker *ForkChoice) (WriteStatus, error) {
if hc.procInterrupt() {
return 0, errors.New("aborted")
}
res, err := hc.writeHeaders(chain)
res, err := hc.writeHeadersAndSetHead(chain, forker)
if err != nil {
return 0, err
}
// Report some public statistics so the user has a clue what's going on
context := []interface{}{
"count", res.imported,
"elapsed", common.PrettyDuration(time.Since(start)),
}
if err != nil {
context = append(context, "err", err)
}
if last := res.lastHeader; last != nil {
context = append(context, "number", last.Number, "hash", res.lastHash)
if timestamp := time.Unix(int64(last.Time), 0); time.Since(timestamp) > time.Minute {

View File

@ -51,10 +51,10 @@ func verifyUnbrokenCanonchain(hc *HeaderChain) error {
return nil
}
func testInsert(t *testing.T, hc *HeaderChain, chain []*types.Header, wantStatus WriteStatus, wantErr error) {
func testInsert(t *testing.T, hc *HeaderChain, chain []*types.Header, wantStatus WriteStatus, wantErr error, forker *ForkChoice) {
t.Helper()
status, err := hc.InsertHeaderChain(chain, time.Now())
status, err := hc.InsertHeaderChain(chain, time.Now(), forker)
if status != wantStatus {
t.Errorf("wrong write status from InsertHeaderChain: got %v, want %v", status, wantStatus)
}
@ -80,37 +80,38 @@ func TestHeaderInsertion(t *testing.T) {
}
// chain A: G->A1->A2...A128
chainA := makeHeaderChain(genesis.Header(), 128, ethash.NewFaker(), db, 10)
// chain B: G->A1->B2...B128
// chain B: G->A1->B1...B128
chainB := makeHeaderChain(chainA[0], 128, ethash.NewFaker(), db, 10)
log.Root().SetHandler(log.StdoutHandler)
forker := NewForkChoice(hc, nil)
// Inserting 64 headers on an empty chain, expecting
// 1 callbacks, 1 canon-status, 0 sidestatus,
testInsert(t, hc, chainA[:64], CanonStatTy, nil)
testInsert(t, hc, chainA[:64], CanonStatTy, nil, forker)
// Inserting 64 identical headers, expecting
// 0 callbacks, 0 canon-status, 0 sidestatus,
testInsert(t, hc, chainA[:64], NonStatTy, nil)
testInsert(t, hc, chainA[:64], NonStatTy, nil, forker)
// Inserting the same some old, some new headers
// 1 callbacks, 1 canon, 0 side
testInsert(t, hc, chainA[32:96], CanonStatTy, nil)
testInsert(t, hc, chainA[32:96], CanonStatTy, nil, forker)
// Inserting side blocks, but not overtaking the canon chain
testInsert(t, hc, chainB[0:32], SideStatTy, nil)
testInsert(t, hc, chainB[0:32], SideStatTy, nil, forker)
// Inserting more side blocks, but we don't have the parent
testInsert(t, hc, chainB[34:36], NonStatTy, consensus.ErrUnknownAncestor)
testInsert(t, hc, chainB[34:36], NonStatTy, consensus.ErrUnknownAncestor, forker)
// Inserting more sideblocks, overtaking the canon chain
testInsert(t, hc, chainB[32:97], CanonStatTy, nil)
testInsert(t, hc, chainB[32:97], CanonStatTy, nil, forker)
// Inserting more A-headers, taking back the canonicality
testInsert(t, hc, chainA[90:100], CanonStatTy, nil)
testInsert(t, hc, chainA[90:100], CanonStatTy, nil, forker)
// And B becomes canon again
testInsert(t, hc, chainB[97:107], CanonStatTy, nil)
testInsert(t, hc, chainB[97:107], CanonStatTy, nil, forker)
// And B becomes even longer
testInsert(t, hc, chainB[107:128], CanonStatTy, nil)
testInsert(t, hc, chainB[107:128], CanonStatTy, nil, forker)
}

View File

@ -138,3 +138,16 @@ func PopUncleanShutdownMarker(db ethdb.KeyValueStore) {
log.Warn("Failed to clear unclean-shutdown marker", "err", err)
}
}
// ReadTransitionStatus retrieves the eth2 transition status from the database
func ReadTransitionStatus(db ethdb.KeyValueReader) []byte {
data, _ := db.Get(transitionStatusKey)
return data
}
// WriteTransitionStatus stores the eth2 transition status to the database
func WriteTransitionStatus(db ethdb.KeyValueWriter, data []byte) {
if err := db.Put(transitionStatusKey, data); err != nil {
log.Crit("Failed to store the eth2 transition status", "err", err)
}
}

View File

@ -395,7 +395,7 @@ func InspectDatabase(db ethdb.Database, keyPrefix, keyStart []byte) error {
databaseVersionKey, headHeaderKey, headBlockKey, headFastBlockKey, lastPivotKey,
fastTrieProgressKey, snapshotDisabledKey, SnapshotRootKey, snapshotJournalKey,
snapshotGeneratorKey, snapshotRecoveryKey, txIndexTailKey, fastTxLookupLimitKey,
uncleanShutdownKey, badBlockKey,
uncleanShutdownKey, badBlockKey, transitionStatusKey,
} {
if bytes.Equal(key, meta) {
metadata.Add(size)

View File

@ -75,6 +75,9 @@ var (
// uncleanShutdownKey tracks the list of local crashes
uncleanShutdownKey = []byte("unclean-shutdown") // config prefix for the db
// transitionStatusKey tracks the eth2 transition status.
transitionStatusKey = []byte("eth2-transition")
// Data item prefixes (use single byte to avoid mixing data types, avoid `i`, used for indexes).
headerPrefix = []byte("h") // headerPrefix + num (uint64 big endian) + hash -> header
headerTDSuffix = []byte("t") // headerPrefix + num (uint64 big endian) + hash + headerTDSuffix -> td

View File

@ -85,6 +85,12 @@ type Header struct {
// BaseFee was added by EIP-1559 and is ignored in legacy headers.
BaseFee *big.Int `json:"baseFeePerGas" rlp:"optional"`
/*
TODO (MariusVanDerWijden) Add this field once needed
// Random was added during the merge and contains the BeaconState randomness
Random common.Hash `json:"random" rlp:"optional"`
*/
}
// field type overrides for gencodec

View File

@ -353,7 +353,7 @@ func (b *EthAPIBackend) StartMining(threads int) error {
}
func (b *EthAPIBackend) StateAtBlock(ctx context.Context, block *types.Block, reexec uint64, base *state.StateDB, checkLive, preferDisk bool) (*state.StateDB, error) {
return b.eth.stateAtBlock(block, reexec, base, checkLive, preferDisk)
return b.eth.StateAtBlock(block, reexec, base, checkLive, preferDisk)
}
func (b *EthAPIBackend) StateAtTransaction(ctx context.Context, block *types.Block, txIndex int, reexec uint64) (core.Message, vm.BlockContext, *state.StateDB, error) {

View File

@ -30,6 +30,7 @@ import (
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/common/hexutil"
"github.com/ethereum/go-ethereum/consensus"
"github.com/ethereum/go-ethereum/consensus/beacon"
"github.com/ethereum/go-ethereum/consensus/clique"
"github.com/ethereum/go-ethereum/core"
"github.com/ethereum/go-ethereum/core/bloombits"
@ -71,6 +72,7 @@ type Ethereum struct {
handler *handler
ethDialCandidates enode.Iterator
snapDialCandidates enode.Iterator
merger *consensus.Merger
// DB interfaces
chainDb ethdb.Database // Block chain database
@ -131,7 +133,7 @@ func New(stack *node.Node, config *ethconfig.Config) (*Ethereum, error) {
if err != nil {
return nil, err
}
chainConfig, genesisHash, genesisErr := core.SetupGenesisBlockWithOverride(chainDb, config.Genesis, config.OverrideArrowGlacier)
chainConfig, genesisHash, genesisErr := core.SetupGenesisBlockWithOverride(chainDb, config.Genesis, config.OverrideArrowGlacier, config.OverrideTerminalTotalDifficulty)
if _, ok := genesisErr.(*params.ConfigCompatError); genesisErr != nil && !ok {
return nil, genesisErr
}
@ -140,8 +142,10 @@ func New(stack *node.Node, config *ethconfig.Config) (*Ethereum, error) {
if err := pruner.RecoverPruning(stack.ResolvePath(""), chainDb, stack.ResolvePath(config.TrieCleanCacheJournal)); err != nil {
log.Error("Failed to recover state", "error", err)
}
merger := consensus.NewMerger(chainDb)
eth := &Ethereum{
config: config,
merger: merger,
chainDb: chainDb,
eventMux: stack.EventMux(),
accountManager: stack.AccountManager(),
@ -215,6 +219,7 @@ func New(stack *node.Node, config *ethconfig.Config) (*Ethereum, error) {
Database: chainDb,
Chain: eth.blockchain,
TxPool: eth.txPool,
Merger: merger,
Network: config.NetworkId,
Sync: config.SyncMode,
BloomCache: uint64(cacheLimit),
@ -225,7 +230,7 @@ func New(stack *node.Node, config *ethconfig.Config) (*Ethereum, error) {
return nil, err
}
eth.miner = miner.New(eth, &config.Miner, chainConfig, eth.EventMux(), eth.engine, eth.isLocalBlock)
eth.miner = miner.New(eth, &config.Miner, chainConfig, eth.EventMux(), eth.engine, eth.isLocalBlock, merger)
eth.miner.SetExtra(makeExtraData(config.Miner.ExtraData))
eth.APIBackend = &EthAPIBackend{stack.Config().ExtRPCEnabled(), stack.Config().AllowUnprotectedTxs, eth, nil}
@ -256,6 +261,7 @@ func New(stack *node.Node, config *ethconfig.Config) (*Ethereum, error) {
stack.RegisterAPIs(eth.APIs())
stack.RegisterProtocols(eth.Protocols())
stack.RegisterLifecycle(eth)
// Check for unclean shutdown
if uncleanShutdowns, discards, err := rawdb.PushUncleanShutdownMarker(chainDb); err != nil {
log.Error("Could not update unclean-shutdown-marker list", "error", err)
@ -378,10 +384,10 @@ func (s *Ethereum) Etherbase() (eb common.Address, err error) {
//
// We regard two types of accounts as local miner account: etherbase
// and accounts specified via `txpool.locals` flag.
func (s *Ethereum) isLocalBlock(block *types.Block) bool {
author, err := s.engine.Author(block.Header())
func (s *Ethereum) isLocalBlock(header *types.Header) bool {
author, err := s.engine.Author(header)
if err != nil {
log.Warn("Failed to retrieve block author", "number", block.NumberU64(), "hash", block.Hash(), "err", err)
log.Warn("Failed to retrieve block author", "number", header.Number.Uint64(), "hash", header.Hash(), "err", err)
return false
}
// Check whether the given address is etherbase.
@ -404,7 +410,7 @@ func (s *Ethereum) isLocalBlock(block *types.Block) bool {
// shouldPreserve checks whether we should preserve the given block
// during the chain reorg depending on whether the author of block
// is a local account.
func (s *Ethereum) shouldPreserve(block *types.Block) bool {
func (s *Ethereum) shouldPreserve(header *types.Header) bool {
// The reason we need to disable the self-reorg preserving for clique
// is it can be probable to introduce a deadlock.
//
@ -424,7 +430,7 @@ func (s *Ethereum) shouldPreserve(block *types.Block) bool {
if _, ok := s.engine.(*clique.Clique); ok {
return false
}
return s.isLocalBlock(block)
return s.isLocalBlock(header)
}
// SetEtherbase sets the mining reward address.
@ -465,13 +471,21 @@ func (s *Ethereum) StartMining(threads int) error {
log.Error("Cannot start mining without etherbase", "err", err)
return fmt.Errorf("etherbase missing: %v", err)
}
if clique, ok := s.engine.(*clique.Clique); ok {
var cli *clique.Clique
if c, ok := s.engine.(*clique.Clique); ok {
cli = c
} else if cl, ok := s.engine.(*beacon.Beacon); ok {
if c, ok := cl.InnerEngine().(*clique.Clique); ok {
cli = c
}
}
if cli != nil {
wallet, err := s.accountManager.Find(accounts.Account{Address: eb})
if wallet == nil || err != nil {
log.Error("Etherbase account unavailable locally", "err", err)
return fmt.Errorf("signer missing: %v", err)
}
clique.Authorize(eb, wallet.SignData)
cli.Authorize(eb, wallet.SignData)
}
// If mining is started, we can disable the transaction rejection mechanism
// introduced to speed sync times.
@ -508,8 +522,14 @@ func (s *Ethereum) ChainDb() ethdb.Database { return s.chainDb }
func (s *Ethereum) IsListening() bool { return true } // Always listening
func (s *Ethereum) Downloader() *downloader.Downloader { return s.handler.downloader }
func (s *Ethereum) Synced() bool { return atomic.LoadUint32(&s.handler.acceptTxs) == 1 }
func (s *Ethereum) SetSynced() { atomic.StoreUint32(&s.handler.acceptTxs, 1) }
func (s *Ethereum) ArchiveMode() bool { return s.config.NoPruning }
func (s *Ethereum) BloomIndexer() *core.ChainIndexer { return s.bloomIndexer }
func (s *Ethereum) Merger() *consensus.Merger { return s.merger }
func (s *Ethereum) SyncMode() downloader.SyncMode {
mode, _ := s.handler.chainSync.modeAndLocalHead()
return mode
}
// Protocols returns all the currently configured
// network protocols to start.

View File

@ -18,17 +18,22 @@
package catalyst
import (
"bytes"
"errors"
"fmt"
"math/big"
"time"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/common/hexutil"
"github.com/ethereum/go-ethereum/consensus"
"github.com/ethereum/go-ethereum/consensus/beacon"
"github.com/ethereum/go-ethereum/consensus/misc"
"github.com/ethereum/go-ethereum/core"
"github.com/ethereum/go-ethereum/core/state"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/eth"
"github.com/ethereum/go-ethereum/les"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/node"
chainParams "github.com/ethereum/go-ethereum/params"
@ -36,31 +41,78 @@ import (
"github.com/ethereum/go-ethereum/trie"
)
// Register adds catalyst APIs to the node.
func Register(stack *node.Node, backend *eth.Ethereum) error {
chainconfig := backend.BlockChain().Config()
if chainconfig.TerminalTotalDifficulty == nil {
return errors.New("catalyst started without valid total difficulty")
}
var (
VALID = GenericStringResponse{"VALID"}
INVALID = GenericStringResponse{"INVALID"}
SYNCING = GenericStringResponse{"SYNCING"}
UnknownHeader = rpc.CustomError{Code: -32000, Message: "unknown header"}
UnknownPayload = rpc.CustomError{Code: -32001, Message: "unknown payload"}
)
log.Warn("Catalyst mode enabled")
// Register adds catalyst APIs to the full node.
func Register(stack *node.Node, backend *eth.Ethereum) error {
log.Warn("Catalyst mode enabled", "protocol", "eth")
stack.RegisterAPIs([]rpc.API{
{
Namespace: "consensus",
Namespace: "engine",
Version: "1.0",
Service: newConsensusAPI(backend),
Service: NewConsensusAPI(backend, nil),
Public: true,
},
})
return nil
}
type consensusAPI struct {
eth *eth.Ethereum
// RegisterLight adds catalyst APIs to the light client.
func RegisterLight(stack *node.Node, backend *les.LightEthereum) error {
log.Warn("Catalyst mode enabled", "protocol", "les")
stack.RegisterAPIs([]rpc.API{
{
Namespace: "engine",
Version: "1.0",
Service: NewConsensusAPI(nil, backend),
Public: true,
},
})
return nil
}
func newConsensusAPI(eth *eth.Ethereum) *consensusAPI {
return &consensusAPI{eth: eth}
type ConsensusAPI struct {
light bool
eth *eth.Ethereum
les *les.LightEthereum
engine consensus.Engine // engine is the post-merge consensus engine, only for block creation
preparedBlocks map[int]*ExecutableData
}
func NewConsensusAPI(eth *eth.Ethereum, les *les.LightEthereum) *ConsensusAPI {
var engine consensus.Engine
if eth == nil {
if les.BlockChain().Config().TerminalTotalDifficulty == nil {
panic("Catalyst started without valid total difficulty")
}
if b, ok := les.Engine().(*beacon.Beacon); ok {
engine = beacon.New(b.InnerEngine())
} else {
engine = beacon.New(les.Engine())
}
} else {
if eth.BlockChain().Config().TerminalTotalDifficulty == nil {
panic("Catalyst started without valid total difficulty")
}
if b, ok := eth.Engine().(*beacon.Beacon); ok {
engine = beacon.New(b.InnerEngine())
} else {
engine = beacon.New(eth.Engine())
}
}
return &ConsensusAPI{
light: eth == nil,
eth: eth,
les: les,
engine: engine,
preparedBlocks: make(map[int]*ExecutableData),
}
}
// blockExecutionEnv gathers all the data required to execute
@ -89,8 +141,24 @@ func (env *blockExecutionEnv) commitTransaction(tx *types.Transaction, coinbase
return nil
}
func (api *consensusAPI) makeEnv(parent *types.Block, header *types.Header) (*blockExecutionEnv, error) {
state, err := api.eth.BlockChain().StateAt(parent.Root())
func (api *ConsensusAPI) makeEnv(parent *types.Block, header *types.Header) (*blockExecutionEnv, error) {
// The parent state might be missing. It can be the special scenario
// that consensus layer tries to build a new block based on the very
// old side chain block and the relevant state is already pruned. So
// try to retrieve the live state from the chain, if it's not existent,
// do the necessary recovery work.
var (
err error
state *state.StateDB
)
if api.eth.BlockChain().HasState(parent.Root()) {
state, err = api.eth.BlockChain().StateAt(parent.Root())
} else {
// The maximum acceptable reorg depth can be limited by the
// finalised block somehow. TODO(rjl493456442) fix the hard-
// coded number here later.
state, err = api.eth.StateAtBlock(parent, 1000, nil, false, false)
}
if err != nil {
return nil, err
}
@ -103,9 +171,96 @@ func (api *consensusAPI) makeEnv(parent *types.Block, header *types.Header) (*bl
return env, nil
}
func (api *ConsensusAPI) PreparePayload(params AssembleBlockParams) (*PayloadResponse, error) {
data, err := api.assembleBlock(params)
if err != nil {
return nil, err
}
id := len(api.preparedBlocks)
api.preparedBlocks[id] = data
return &PayloadResponse{PayloadID: uint64(id)}, nil
}
func (api *ConsensusAPI) GetPayload(PayloadID hexutil.Uint64) (*ExecutableData, error) {
data, ok := api.preparedBlocks[int(PayloadID)]
if !ok {
return nil, &UnknownPayload
}
return data, nil
}
// ConsensusValidated is called to mark a block as valid, so
// that data that is no longer needed can be removed.
func (api *ConsensusAPI) ConsensusValidated(params ConsensusValidatedParams) error {
switch params.Status {
case VALID.Status:
return nil
case INVALID.Status:
// TODO (MariusVanDerWijden) delete the block from the bc
return nil
default:
return errors.New("invalid params.status")
}
}
func (api *ConsensusAPI) ForkchoiceUpdated(params ForkChoiceParams) error {
var emptyHash = common.Hash{}
if !bytes.Equal(params.HeadBlockHash[:], emptyHash[:]) {
if err := api.checkTerminalTotalDifficulty(params.HeadBlockHash); err != nil {
return err
}
return api.setHead(params.HeadBlockHash)
}
return nil
}
// ExecutePayload creates an Eth1 block, inserts it in the chain, and returns the status of the chain.
func (api *ConsensusAPI) ExecutePayload(params ExecutableData) (GenericStringResponse, error) {
block, err := ExecutableDataToBlock(params)
if err != nil {
return INVALID, err
}
if api.light {
parent := api.les.BlockChain().GetHeaderByHash(params.ParentHash)
if parent == nil {
return INVALID, fmt.Errorf("could not find parent %x", params.ParentHash)
}
if err = api.les.BlockChain().InsertHeader(block.Header()); err != nil {
return INVALID, err
}
return VALID, nil
}
if !api.eth.BlockChain().HasBlock(block.ParentHash(), block.NumberU64()-1) {
/*
TODO (MariusVanDerWijden) reenable once sync is merged
if err := api.eth.Downloader().BeaconSync(api.eth.SyncMode(), block.Header()); err != nil {
return SYNCING, err
}
*/
return SYNCING, nil
}
parent := api.eth.BlockChain().GetBlockByHash(params.ParentHash)
td := api.eth.BlockChain().GetTd(parent.Hash(), block.NumberU64()-1)
ttd := api.eth.BlockChain().Config().TerminalTotalDifficulty
if td.Cmp(ttd) < 0 {
return INVALID, fmt.Errorf("can not execute payload on top of block with low td got: %v threshold %v", td, ttd)
}
if err := api.eth.BlockChain().InsertBlockWithoutSetHead(block); err != nil {
return INVALID, err
}
merger := api.merger()
if !merger.TDDReached() {
merger.ReachTTD()
}
return VALID, nil
}
// AssembleBlock creates a new block, inserts it into the chain, and returns the "execution
// data" required for eth2 clients to process the new block.
func (api *consensusAPI) AssembleBlock(params assembleBlockParams) (*executableData, error) {
func (api *ConsensusAPI) assembleBlock(params AssembleBlockParams) (*ExecutableData, error) {
if api.light {
return nil, errors.New("not supported")
}
log.Info("Producing block", "parentHash", params.ParentHash)
bc := api.eth.BlockChain()
@ -115,45 +270,34 @@ func (api *consensusAPI) AssembleBlock(params assembleBlockParams) (*executableD
return nil, fmt.Errorf("cannot assemble block with unknown parent %s", params.ParentHash)
}
pool := api.eth.TxPool()
if parent.Time() >= params.Timestamp {
return nil, fmt.Errorf("child timestamp lower than parent's: %d >= %d", parent.Time(), params.Timestamp)
if params.Timestamp < parent.Time() {
return nil, fmt.Errorf("child timestamp lower than parent's: %d < %d", params.Timestamp, parent.Time())
}
if now := uint64(time.Now().Unix()); params.Timestamp > now+1 {
wait := time.Duration(params.Timestamp-now) * time.Second
log.Info("Producing block too far in the future", "wait", common.PrettyDuration(wait))
time.Sleep(wait)
}
pending := pool.Pending(true)
coinbase, err := api.eth.Etherbase()
if err != nil {
return nil, err
diff := time.Duration(params.Timestamp-now) * time.Second
log.Warn("Producing block too far in the future", "diff", common.PrettyDuration(diff))
}
pending := api.eth.TxPool().Pending(true)
coinbase := params.FeeRecipient
num := parent.Number()
header := &types.Header{
ParentHash: parent.Hash(),
Number: num.Add(num, common.Big1),
Coinbase: coinbase,
GasLimit: parent.GasLimit(), // Keep the gas limit constant in this prototype
Extra: []byte{},
Extra: []byte{}, // TODO (MariusVanDerWijden) properly set extra data
Time: params.Timestamp,
}
if config := api.eth.BlockChain().Config(); config.IsLondon(header.Number) {
header.BaseFee = misc.CalcBaseFee(config, parent.Header())
}
err = api.eth.Engine().Prepare(bc, header)
if err != nil {
if err := api.engine.Prepare(bc, header); err != nil {
return nil, err
}
env, err := api.makeEnv(parent, header)
if err != nil {
return nil, err
}
var (
signer = types.MakeSigner(bc.Config(), header.Number)
txHeap = types.NewTransactionsByPriceAndNonce(signer, pending, nil)
@ -204,25 +348,12 @@ func (api *consensusAPI) AssembleBlock(params assembleBlockParams) (*executableD
txHeap.Shift()
}
}
// Create the block.
block, err := api.eth.Engine().FinalizeAndAssemble(bc, header, env.state, transactions, nil /* uncles */, env.receipts)
block, err := api.engine.FinalizeAndAssemble(bc, header, env.state, transactions, nil /* uncles */, env.receipts)
if err != nil {
return nil, err
}
return &executableData{
BlockHash: block.Hash(),
ParentHash: block.ParentHash(),
Miner: block.Coinbase(),
StateRoot: block.Root(),
Number: block.NumberU64(),
GasLimit: block.GasLimit(),
GasUsed: block.GasUsed(),
Timestamp: block.Time(),
ReceiptRoot: block.ReceiptHash(),
LogsBloom: block.Bloom().Bytes(),
Transactions: encodeTransactions(block.Transactions()),
}, nil
return BlockToExecutableData(block, params.Random), nil
}
func encodeTransactions(txs []*types.Transaction) [][]byte {
@ -245,66 +376,129 @@ func decodeTransactions(enc [][]byte) ([]*types.Transaction, error) {
return txs, nil
}
func insertBlockParamsToBlock(config *chainParams.ChainConfig, parent *types.Header, params executableData) (*types.Block, error) {
func ExecutableDataToBlock(params ExecutableData) (*types.Block, error) {
txs, err := decodeTransactions(params.Transactions)
if err != nil {
return nil, err
}
if len(params.ExtraData) > 32 {
return nil, fmt.Errorf("invalid extradata length: %v", len(params.ExtraData))
}
number := big.NewInt(0)
number.SetUint64(params.Number)
header := &types.Header{
ParentHash: params.ParentHash,
UncleHash: types.EmptyUncleHash,
Coinbase: params.Miner,
Coinbase: params.Coinbase,
Root: params.StateRoot,
TxHash: types.DeriveSha(types.Transactions(txs), trie.NewStackTrie(nil)),
ReceiptHash: params.ReceiptRoot,
Bloom: types.BytesToBloom(params.LogsBloom),
Difficulty: big.NewInt(1),
Difficulty: common.Big0,
Number: number,
GasLimit: params.GasLimit,
GasUsed: params.GasUsed,
Time: params.Timestamp,
}
if config.IsLondon(number) {
header.BaseFee = misc.CalcBaseFee(config, parent)
BaseFee: params.BaseFeePerGas,
Extra: params.ExtraData,
// TODO (MariusVanDerWijden) add params.Random to header once required
}
block := types.NewBlockWithHeader(header).WithBody(txs, nil /* uncles */)
if block.Hash() != params.BlockHash {
return nil, fmt.Errorf("blockhash mismatch, want %x, got %x", params.BlockHash, block.Hash())
}
return block, nil
}
// NewBlock creates an Eth1 block, inserts it in the chain, and either returns true,
// or false + an error. This is a bit redundant for go, but simplifies things on the
// eth2 side.
func (api *consensusAPI) NewBlock(params executableData) (*newBlockResponse, error) {
parent := api.eth.BlockChain().GetBlockByHash(params.ParentHash)
if parent == nil {
return &newBlockResponse{false}, fmt.Errorf("could not find parent %x", params.ParentHash)
func BlockToExecutableData(block *types.Block, random common.Hash) *ExecutableData {
return &ExecutableData{
BlockHash: block.Hash(),
ParentHash: block.ParentHash(),
Coinbase: block.Coinbase(),
StateRoot: block.Root(),
Number: block.NumberU64(),
GasLimit: block.GasLimit(),
GasUsed: block.GasUsed(),
BaseFeePerGas: block.BaseFee(),
Timestamp: block.Time(),
ReceiptRoot: block.ReceiptHash(),
LogsBloom: block.Bloom().Bytes(),
Transactions: encodeTransactions(block.Transactions()),
Random: random,
ExtraData: block.Extra(),
}
block, err := insertBlockParamsToBlock(api.eth.BlockChain().Config(), parent.Header(), params)
if err != nil {
return nil, err
}
_, err = api.eth.BlockChain().InsertChainWithoutSealVerification(block)
return &newBlockResponse{err == nil}, err
}
// Used in tests to add a the list of transactions from a block to the tx pool.
func (api *consensusAPI) addBlockTxs(block *types.Block) error {
for _, tx := range block.Transactions() {
func (api *ConsensusAPI) insertTransactions(txs types.Transactions) error {
for _, tx := range txs {
api.eth.TxPool().AddLocal(tx)
}
return nil
}
// FinalizeBlock is called to mark a block as synchronized, so
// that data that is no longer needed can be removed.
func (api *consensusAPI) FinalizeBlock(blockHash common.Hash) (*genericResponse, error) {
return &genericResponse{true}, nil
func (api *ConsensusAPI) checkTerminalTotalDifficulty(head common.Hash) error {
// shortcut if we entered PoS already
if api.merger().PoSFinalized() {
return nil
}
// make sure the parent has enough terminal total difficulty
newHeadBlock := api.eth.BlockChain().GetBlockByHash(head)
if newHeadBlock == nil {
return &UnknownHeader
}
parent := api.eth.BlockChain().GetBlockByHash(newHeadBlock.ParentHash())
if parent == nil {
return fmt.Errorf("parent unavailable: %v", newHeadBlock.ParentHash())
}
td := api.eth.BlockChain().GetTd(parent.Hash(), parent.NumberU64())
if td != nil && td.Cmp(api.eth.BlockChain().Config().TerminalTotalDifficulty) < 0 {
return errors.New("total difficulty not reached yet")
}
return nil
}
// SetHead is called to perform a force choice.
func (api *consensusAPI) SetHead(newHead common.Hash) (*genericResponse, error) {
return &genericResponse{true}, nil
// setHead is called to perform a force choice.
func (api *ConsensusAPI) setHead(newHead common.Hash) error {
// Trigger the transition if it's the first `NewHead` event.
merger := api.merger()
if !merger.PoSFinalized() {
merger.FinalizePoS()
}
log.Info("Setting head", "head", newHead)
if api.light {
headHeader := api.les.BlockChain().CurrentHeader()
if headHeader.Hash() == newHead {
return nil
}
newHeadHeader := api.les.BlockChain().GetHeaderByHash(newHead)
if newHeadHeader == nil {
return &UnknownHeader
}
if err := api.les.BlockChain().SetChainHead(newHeadHeader); err != nil {
return err
}
return nil
}
headBlock := api.eth.BlockChain().CurrentBlock()
if headBlock.Hash() == newHead {
return nil
}
newHeadBlock := api.eth.BlockChain().GetBlockByHash(newHead)
if newHeadBlock == nil {
return &UnknownHeader
}
if err := api.eth.BlockChain().SetChainHead(newHeadBlock); err != nil {
return err
}
api.eth.SetSynced()
return nil
}
// Helper function, return the merger instance.
func (api *ConsensusAPI) merger() *consensus.Merger {
if api.light {
return api.les.Merger()
}
return api.eth.Merger()
}

View File

@ -19,7 +19,10 @@ package catalyst
import (
"math/big"
"testing"
"time"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/common/hexutil"
"github.com/ethereum/go-ethereum/consensus/ethash"
"github.com/ethereum/go-ethereum/core"
"github.com/ethereum/go-ethereum/core/rawdb"
@ -38,10 +41,10 @@ var (
// testAddr is the Ethereum address of the tester account.
testAddr = crypto.PubkeyToAddress(testKey.PublicKey)
testBalance = big.NewInt(2e15)
testBalance = big.NewInt(2e18)
)
func generateTestChain() (*core.Genesis, []*types.Block) {
func generatePreMergeChain(n int) (*core.Genesis, []*types.Block) {
db := rawdb.NewMemoryDatabase()
config := params.AllEthashProtocolChanges
genesis := &core.Genesis{
@ -51,177 +54,267 @@ func generateTestChain() (*core.Genesis, []*types.Block) {
Timestamp: 9000,
BaseFee: big.NewInt(params.InitialBaseFee),
}
testNonce := uint64(0)
generate := func(i int, g *core.BlockGen) {
g.OffsetTime(5)
g.SetExtra([]byte("test"))
}
gblock := genesis.ToBlock(db)
engine := ethash.NewFaker()
blocks, _ := core.GenerateChain(config, gblock, engine, db, 10, generate)
blocks = append([]*types.Block{gblock}, blocks...)
return genesis, blocks
}
// TODO (MariusVanDerWijden) reenable once engine api is updated to the latest spec
/*
func generateTestChainWithFork(n int, fork int) (*core.Genesis, []*types.Block, []*types.Block) {
if fork >= n {
fork = n - 1
}
db := rawdb.NewMemoryDatabase()
config := &params.ChainConfig{
ChainID: big.NewInt(1337),
HomesteadBlock: big.NewInt(0),
EIP150Block: big.NewInt(0),
EIP155Block: big.NewInt(0),
EIP158Block: big.NewInt(0),
ByzantiumBlock: big.NewInt(0),
ConstantinopleBlock: big.NewInt(0),
PetersburgBlock: big.NewInt(0),
IstanbulBlock: big.NewInt(0),
MuirGlacierBlock: big.NewInt(0),
BerlinBlock: big.NewInt(0),
LondonBlock: big.NewInt(0),
TerminalTotalDifficulty: big.NewInt(0),
Ethash: new(params.EthashConfig),
}
genesis := &core.Genesis{
Config: config,
Alloc: core.GenesisAlloc{testAddr: {Balance: testBalance}},
ExtraData: []byte("test genesis"),
Timestamp: 9000,
BaseFee: big.NewInt(params.InitialBaseFee),
}
generate := func(i int, g *core.BlockGen) {
g.OffsetTime(5)
g.SetExtra([]byte("test"))
}
generateFork := func(i int, g *core.BlockGen) {
g.OffsetTime(5)
g.SetExtra([]byte("testF"))
tx, _ := types.SignTx(types.NewTransaction(testNonce, common.HexToAddress("0x9a9070028361F7AAbeB3f2F2Dc07F82C4a98A02a"), big.NewInt(1), params.TxGas, big.NewInt(params.InitialBaseFee*2), nil), types.LatestSigner(config), testKey)
g.AddTx(tx)
testNonce++
}
gblock := genesis.ToBlock(db)
engine := ethash.NewFaker()
blocks, _ := core.GenerateChain(config, gblock, engine, db, n, generate)
blocks = append([]*types.Block{gblock}, blocks...)
forkedBlocks, _ := core.GenerateChain(config, blocks[fork], engine, db, n-fork, generateFork)
return genesis, blocks, forkedBlocks
totalDifficulty := big.NewInt(0)
for _, b := range blocks {
totalDifficulty.Add(totalDifficulty, b.Difficulty())
}
config.TerminalTotalDifficulty = totalDifficulty
return genesis, blocks
}
*/
func TestEth2AssembleBlock(t *testing.T) {
genesis, blocks := generateTestChain()
n, ethservice := startEthService(t, genesis, blocks[1:9])
genesis, blocks := generatePreMergeChain(10)
n, ethservice := startEthService(t, genesis, blocks)
defer n.Close()
api := newConsensusAPI(ethservice)
api := NewConsensusAPI(ethservice, nil)
signer := types.NewEIP155Signer(ethservice.BlockChain().Config().ChainID)
tx, err := types.SignTx(types.NewTransaction(0, blocks[8].Coinbase(), big.NewInt(1000), params.TxGas, big.NewInt(params.InitialBaseFee), nil), signer, testKey)
tx, err := types.SignTx(types.NewTransaction(uint64(10), blocks[9].Coinbase(), big.NewInt(1000), params.TxGas, big.NewInt(params.InitialBaseFee), nil), signer, testKey)
if err != nil {
t.Fatalf("error signing transaction, err=%v", err)
}
ethservice.TxPool().AddLocal(tx)
blockParams := assembleBlockParams{
ParentHash: blocks[8].ParentHash(),
Timestamp: blocks[8].Time(),
blockParams := AssembleBlockParams{
ParentHash: blocks[9].Hash(),
Timestamp: blocks[9].Time() + 5,
}
execData, err := api.AssembleBlock(blockParams)
execData, err := api.assembleBlock(blockParams)
if err != nil {
t.Fatalf("error producing block, err=%v", err)
}
if len(execData.Transactions) != 1 {
t.Fatalf("invalid number of transactions %d != 1", len(execData.Transactions))
}
}
func TestEth2AssembleBlockWithAnotherBlocksTxs(t *testing.T) {
genesis, blocks := generateTestChain()
n, ethservice := startEthService(t, genesis, blocks[1:9])
genesis, blocks := generatePreMergeChain(10)
n, ethservice := startEthService(t, genesis, blocks[:9])
defer n.Close()
api := newConsensusAPI(ethservice)
api := NewConsensusAPI(ethservice, nil)
// Put the 10th block's tx in the pool and produce a new block
api.addBlockTxs(blocks[9])
blockParams := assembleBlockParams{
ParentHash: blocks[9].ParentHash(),
Timestamp: blocks[9].Time(),
api.insertTransactions(blocks[9].Transactions())
blockParams := AssembleBlockParams{
ParentHash: blocks[8].Hash(),
Timestamp: blocks[8].Time() + 5,
}
execData, err := api.AssembleBlock(blockParams)
execData, err := api.assembleBlock(blockParams)
if err != nil {
t.Fatalf("error producing block, err=%v", err)
}
if len(execData.Transactions) != blocks[9].Transactions().Len() {
t.Fatalf("invalid number of transactions %d != 1", len(execData.Transactions))
}
}
// TODO (MariusVanDerWijden) reenable once engine api is updated to the latest spec
/*
func TestEth2NewBlock(t *testing.T) {
genesis, blocks, forkedBlocks := generateTestChainWithFork(10, 4)
n, ethservice := startEthService(t, genesis, blocks[1:5])
func TestSetHeadBeforeTotalDifficulty(t *testing.T) {
genesis, blocks := generatePreMergeChain(10)
n, ethservice := startEthService(t, genesis, blocks)
defer n.Close()
api := newConsensusAPI(ethservice)
for i := 5; i < 10; i++ {
p := executableData{
ParentHash: ethservice.BlockChain().CurrentBlock().Hash(),
Miner: blocks[i].Coinbase(),
StateRoot: blocks[i].Root(),
GasLimit: blocks[i].GasLimit(),
GasUsed: blocks[i].GasUsed(),
Transactions: encodeTransactions(blocks[i].Transactions()),
ReceiptRoot: blocks[i].ReceiptHash(),
LogsBloom: blocks[i].Bloom().Bytes(),
BlockHash: blocks[i].Hash(),
Timestamp: blocks[i].Time(),
Number: uint64(i),
}
success, err := api.NewBlock(p)
if err != nil || !success.Valid {
t.Fatalf("Failed to insert block: %v", err)
}
}
api := NewConsensusAPI(ethservice, nil)
exp := ethservice.BlockChain().CurrentBlock().Hash()
// Introduce the fork point.
lastBlockNum := blocks[4].Number()
lastBlock := blocks[4]
for i := 0; i < 4; i++ {
lastBlockNum.Add(lastBlockNum, big.NewInt(1))
p := executableData{
ParentHash: lastBlock.Hash(),
Miner: forkedBlocks[i].Coinbase(),
StateRoot: forkedBlocks[i].Root(),
Number: lastBlockNum.Uint64(),
GasLimit: forkedBlocks[i].GasLimit(),
GasUsed: forkedBlocks[i].GasUsed(),
Transactions: encodeTransactions(blocks[i].Transactions()),
ReceiptRoot: forkedBlocks[i].ReceiptHash(),
LogsBloom: forkedBlocks[i].Bloom().Bytes(),
BlockHash: forkedBlocks[i].Hash(),
Timestamp: forkedBlocks[i].Time(),
}
success, err := api.NewBlock(p)
if err != nil || !success.Valid {
t.Fatalf("Failed to insert forked block #%d: %v", i, err)
}
lastBlock, err = insertBlockParamsToBlock(ethservice.BlockChain().Config(), lastBlock.Header(), p)
if err != nil {
t.Fatal(err)
}
}
if ethservice.BlockChain().CurrentBlock().Hash() != exp {
t.Fatalf("Wrong head after inserting fork %x != %x", exp, ethservice.BlockChain().CurrentBlock().Hash())
if err := api.ForkchoiceUpdated(ForkChoiceParams{HeadBlockHash: blocks[5].Hash()}); err == nil {
t.Errorf("fork choice updated before total terminal difficulty should fail")
}
}
*/
func TestEth2PrepareAndGetPayload(t *testing.T) {
genesis, blocks := generatePreMergeChain(10)
n, ethservice := startEthService(t, genesis, blocks[:9])
defer n.Close()
api := NewConsensusAPI(ethservice, nil)
// Put the 10th block's tx in the pool and produce a new block
api.insertTransactions(blocks[9].Transactions())
blockParams := AssembleBlockParams{
ParentHash: blocks[8].Hash(),
Timestamp: blocks[8].Time() + 5,
}
respID, err := api.PreparePayload(blockParams)
if err != nil {
t.Fatalf("error preparing payload, err=%v", err)
}
execData, err := api.GetPayload(hexutil.Uint64(respID.PayloadID))
if err != nil {
t.Fatalf("error getting payload, err=%v", err)
}
if len(execData.Transactions) != blocks[9].Transactions().Len() {
t.Fatalf("invalid number of transactions %d != 1", len(execData.Transactions))
}
}
func checkLogEvents(t *testing.T, logsCh <-chan []*types.Log, rmLogsCh <-chan core.RemovedLogsEvent, wantNew, wantRemoved int) {
t.Helper()
if len(logsCh) != wantNew {
t.Fatalf("wrong number of log events: got %d, want %d", len(logsCh), wantNew)
}
if len(rmLogsCh) != wantRemoved {
t.Fatalf("wrong number of removed log events: got %d, want %d", len(rmLogsCh), wantRemoved)
}
// Drain events.
for i := 0; i < len(logsCh); i++ {
<-logsCh
}
for i := 0; i < len(rmLogsCh); i++ {
<-rmLogsCh
}
}
func TestEth2NewBlock(t *testing.T) {
genesis, preMergeBlocks := generatePreMergeChain(10)
n, ethservice := startEthService(t, genesis, preMergeBlocks)
ethservice.Merger().ReachTTD()
defer n.Close()
var (
api = NewConsensusAPI(ethservice, nil)
parent = preMergeBlocks[len(preMergeBlocks)-1]
// This EVM code generates a log when the contract is created.
logCode = common.Hex2Bytes("60606040525b7f24ec1d3ff24c2f6ff210738839dbc339cd45a5294d85c79361016243157aae7b60405180905060405180910390a15b600a8060416000396000f360606040526008565b00")
)
// The event channels.
newLogCh := make(chan []*types.Log, 10)
rmLogsCh := make(chan core.RemovedLogsEvent, 10)
ethservice.BlockChain().SubscribeLogsEvent(newLogCh)
ethservice.BlockChain().SubscribeRemovedLogsEvent(rmLogsCh)
for i := 0; i < 10; i++ {
statedb, _ := ethservice.BlockChain().StateAt(parent.Root())
nonce := statedb.GetNonce(testAddr)
tx, _ := types.SignTx(types.NewContractCreation(nonce, new(big.Int), 1000000, big.NewInt(2*params.InitialBaseFee), logCode), types.LatestSigner(ethservice.BlockChain().Config()), testKey)
ethservice.TxPool().AddLocal(tx)
execData, err := api.assembleBlock(AssembleBlockParams{
ParentHash: parent.Hash(),
Timestamp: parent.Time() + 5,
})
if err != nil {
t.Fatalf("Failed to create the executable data %v", err)
}
block, err := ExecutableDataToBlock(*execData)
if err != nil {
t.Fatalf("Failed to convert executable data to block %v", err)
}
newResp, err := api.ExecutePayload(*execData)
if err != nil || newResp.Status != "VALID" {
t.Fatalf("Failed to insert block: %v", err)
}
if ethservice.BlockChain().CurrentBlock().NumberU64() != block.NumberU64()-1 {
t.Fatalf("Chain head shouldn't be updated")
}
checkLogEvents(t, newLogCh, rmLogsCh, 0, 0)
if err := api.ForkchoiceUpdated(ForkChoiceParams{HeadBlockHash: block.Hash(), FinalizedBlockHash: block.Hash()}); err != nil {
t.Fatalf("Failed to insert block: %v", err)
}
if ethservice.BlockChain().CurrentBlock().NumberU64() != block.NumberU64() {
t.Fatalf("Chain head should be updated")
}
checkLogEvents(t, newLogCh, rmLogsCh, 1, 0)
parent = block
}
// Introduce fork chain
var (
head = ethservice.BlockChain().CurrentBlock().NumberU64()
)
parent = preMergeBlocks[len(preMergeBlocks)-1]
for i := 0; i < 10; i++ {
execData, err := api.assembleBlock(AssembleBlockParams{
ParentHash: parent.Hash(),
Timestamp: parent.Time() + 6,
})
if err != nil {
t.Fatalf("Failed to create the executable data %v", err)
}
block, err := ExecutableDataToBlock(*execData)
if err != nil {
t.Fatalf("Failed to convert executable data to block %v", err)
}
newResp, err := api.ExecutePayload(*execData)
if err != nil || newResp.Status != "VALID" {
t.Fatalf("Failed to insert block: %v", err)
}
if ethservice.BlockChain().CurrentBlock().NumberU64() != head {
t.Fatalf("Chain head shouldn't be updated")
}
if err := api.ConsensusValidated(ConsensusValidatedParams{BlockHash: block.Hash(), Status: "VALID"}); err != nil {
t.Fatalf("Failed to insert block: %v", err)
}
if err := api.ForkchoiceUpdated(ForkChoiceParams{FinalizedBlockHash: block.Hash(), HeadBlockHash: block.Hash()}); err != nil {
t.Fatalf("Failed to insert block: %v", err)
}
if ethservice.BlockChain().CurrentBlock().NumberU64() != block.NumberU64() {
t.Fatalf("Chain head should be updated")
}
parent, head = block, block.NumberU64()
}
}
func TestEth2DeepReorg(t *testing.T) {
// TODO (MariusVanDerWijden) TestEth2DeepReorg is currently broken, because it tries to reorg
// before the totalTerminalDifficulty threshold
/*
genesis, preMergeBlocks := generatePreMergeChain(core.TriesInMemory * 2)
n, ethservice := startEthService(t, genesis, preMergeBlocks)
defer n.Close()
var (
api = NewConsensusAPI(ethservice, nil)
parent = preMergeBlocks[len(preMergeBlocks)-core.TriesInMemory-1]
head = ethservice.BlockChain().CurrentBlock().NumberU64()
)
if ethservice.BlockChain().HasBlockAndState(parent.Hash(), parent.NumberU64()) {
t.Errorf("Block %d not pruned", parent.NumberU64())
}
for i := 0; i < 10; i++ {
execData, err := api.assembleBlock(AssembleBlockParams{
ParentHash: parent.Hash(),
Timestamp: parent.Time() + 5,
})
if err != nil {
t.Fatalf("Failed to create the executable data %v", err)
}
block, err := ExecutableDataToBlock(ethservice.BlockChain().Config(), parent.Header(), *execData)
if err != nil {
t.Fatalf("Failed to convert executable data to block %v", err)
}
newResp, err := api.ExecutePayload(*execData)
if err != nil || newResp.Status != "VALID" {
t.Fatalf("Failed to insert block: %v", err)
}
if ethservice.BlockChain().CurrentBlock().NumberU64() != head {
t.Fatalf("Chain head shouldn't be updated")
}
if err := api.setHead(block.Hash()); err != nil {
t.Fatalf("Failed to set head: %v", err)
}
if ethservice.BlockChain().CurrentBlock().NumberU64() != block.NumberU64() {
t.Fatalf("Chain head should be updated")
}
parent, head = block, block.NumberU64()
}
*/
}
// startEthService creates a full node instance for testing.
func startEthService(t *testing.T, genesis *core.Genesis, blocks []*types.Block) (*node.Node, *eth.Ethereum) {
@ -232,7 +325,7 @@ func startEthService(t *testing.T, genesis *core.Genesis, blocks []*types.Block)
t.Fatal("can't create node:", err)
}
ethcfg := &ethconfig.Config{Genesis: genesis, Ethash: ethash.Config{PowMode: ethash.ModeFake}}
ethcfg := &ethconfig.Config{Genesis: genesis, Ethash: ethash.Config{PowMode: ethash.ModeFake}, TrieTimeout: time.Minute, TrieDirtyCache: 256, TrieCleanCache: 256}
ethservice, err := eth.New(n, ethcfg)
if err != nil {
t.Fatal("can't create eth service:", err)
@ -245,6 +338,61 @@ func startEthService(t *testing.T, genesis *core.Genesis, blocks []*types.Block)
t.Fatal("can't import test blocks:", err)
}
ethservice.SetEtherbase(testAddr)
ethservice.SetSynced()
return n, ethservice
}
func TestFullAPI(t *testing.T) {
genesis, preMergeBlocks := generatePreMergeChain(10)
n, ethservice := startEthService(t, genesis, preMergeBlocks)
ethservice.Merger().ReachTTD()
defer n.Close()
var (
api = NewConsensusAPI(ethservice, nil)
parent = ethservice.BlockChain().CurrentBlock()
// This EVM code generates a log when the contract is created.
logCode = common.Hex2Bytes("60606040525b7f24ec1d3ff24c2f6ff210738839dbc339cd45a5294d85c79361016243157aae7b60405180905060405180910390a15b600a8060416000396000f360606040526008565b00")
)
for i := 0; i < 10; i++ {
statedb, _ := ethservice.BlockChain().StateAt(parent.Root())
nonce := statedb.GetNonce(testAddr)
tx, _ := types.SignTx(types.NewContractCreation(nonce, new(big.Int), 1000000, big.NewInt(2*params.InitialBaseFee), logCode), types.LatestSigner(ethservice.BlockChain().Config()), testKey)
ethservice.TxPool().AddLocal(tx)
params := AssembleBlockParams{
ParentHash: parent.Hash(),
Timestamp: parent.Time() + 1,
Random: crypto.Keccak256Hash([]byte{byte(i)}),
FeeRecipient: parent.Coinbase(),
}
resp, err := api.PreparePayload(params)
if err != nil {
t.Fatalf("can't prepare payload: %v", err)
}
payload, err := api.GetPayload(hexutil.Uint64(resp.PayloadID))
if err != nil {
t.Fatalf("can't get payload: %v", err)
}
execResp, err := api.ExecutePayload(*payload)
if err != nil {
t.Fatalf("can't execute payload: %v", err)
}
if execResp.Status != VALID.Status {
t.Fatalf("invalid status: %v", execResp.Status)
}
if err := api.ConsensusValidated(ConsensusValidatedParams{BlockHash: payload.BlockHash, Status: VALID.Status}); err != nil {
t.Fatalf("failed to validate consensus: %v", err)
}
if err := api.ForkchoiceUpdated(ForkChoiceParams{HeadBlockHash: payload.BlockHash, FinalizedBlockHash: payload.BlockHash}); err != nil {
t.Fatalf("Failed to insert block: %v", err)
}
if ethservice.BlockChain().CurrentBlock().NumberU64() != payload.Number {
t.Fatalf("Chain head should be updated")
}
parent = ethservice.BlockChain().CurrentBlock()
}
}

View File

@ -17,16 +17,20 @@
package catalyst
import (
"math/big"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/common/hexutil"
)
//go:generate go run github.com/fjl/gencodec -type assembleBlockParams -field-override assembleBlockParamsMarshaling -out gen_blockparams.go
//go:generate go run github.com/fjl/gencodec -type AssembleBlockParams -field-override assembleBlockParamsMarshaling -out gen_blockparams.go
// Structure described at https://hackmd.io/T9x2mMA4S7us8tJwEB3FDQ
type assembleBlockParams struct {
ParentHash common.Hash `json:"parentHash" gencodec:"required"`
Timestamp uint64 `json:"timestamp" gencodec:"required"`
// Structure described at https://github.com/ethereum/execution-apis/pull/74
type AssembleBlockParams struct {
ParentHash common.Hash `json:"parentHash" gencodec:"required"`
Timestamp uint64 `json:"timestamp" gencodec:"required"`
Random common.Hash `json:"random" gencodec:"required"`
FeeRecipient common.Address `json:"feeRecipient" gencodec:"required"`
}
// JSON type overrides for assembleBlockParams.
@ -34,37 +38,67 @@ type assembleBlockParamsMarshaling struct {
Timestamp hexutil.Uint64
}
//go:generate go run github.com/fjl/gencodec -type executableData -field-override executableDataMarshaling -out gen_ed.go
//go:generate go run github.com/fjl/gencodec -type ExecutableData -field-override executableDataMarshaling -out gen_ed.go
// Structure described at https://notes.ethereum.org/@n0ble/rayonism-the-merge-spec#Parameters1
type executableData struct {
BlockHash common.Hash `json:"blockHash" gencodec:"required"`
ParentHash common.Hash `json:"parentHash" gencodec:"required"`
Miner common.Address `json:"miner" gencodec:"required"`
StateRoot common.Hash `json:"stateRoot" gencodec:"required"`
Number uint64 `json:"number" gencodec:"required"`
GasLimit uint64 `json:"gasLimit" gencodec:"required"`
GasUsed uint64 `json:"gasUsed" gencodec:"required"`
Timestamp uint64 `json:"timestamp" gencodec:"required"`
ReceiptRoot common.Hash `json:"receiptsRoot" gencodec:"required"`
LogsBloom []byte `json:"logsBloom" gencodec:"required"`
Transactions [][]byte `json:"transactions" gencodec:"required"`
// Structure described at https://github.com/ethereum/execution-apis/pull/74/files
type ExecutableData struct {
BlockHash common.Hash `json:"blockHash" gencodec:"required"`
ParentHash common.Hash `json:"parentHash" gencodec:"required"`
Coinbase common.Address `json:"coinbase" gencodec:"required"`
StateRoot common.Hash `json:"stateRoot" gencodec:"required"`
ReceiptRoot common.Hash `json:"receiptRoot" gencodec:"required"`
LogsBloom []byte `json:"logsBloom" gencodec:"required"`
Random common.Hash `json:"random" gencodec:"required"`
Number uint64 `json:"blockNumber" gencodec:"required"`
GasLimit uint64 `json:"gasLimit" gencodec:"required"`
GasUsed uint64 `json:"gasUsed" gencodec:"required"`
Timestamp uint64 `json:"timestamp" gencodec:"required"`
ExtraData []byte `json:"extraData" gencodec:"required"`
BaseFeePerGas *big.Int `json:"baseFeePerGas" gencodec:"required"`
Transactions [][]byte `json:"transactions" gencodec:"required"`
}
// JSON type overrides for executableData.
type executableDataMarshaling struct {
Number hexutil.Uint64
GasLimit hexutil.Uint64
GasUsed hexutil.Uint64
Timestamp hexutil.Uint64
LogsBloom hexutil.Bytes
Transactions []hexutil.Bytes
Number hexutil.Uint64
GasLimit hexutil.Uint64
GasUsed hexutil.Uint64
Timestamp hexutil.Uint64
BaseFeePerGas *hexutil.Big
ExtraData hexutil.Bytes
LogsBloom hexutil.Bytes
Transactions []hexutil.Bytes
}
type newBlockResponse struct {
//go:generate go run github.com/fjl/gencodec -type PayloadResponse -field-override payloadResponseMarshaling -out gen_payload.go
type PayloadResponse struct {
PayloadID uint64 `json:"payloadId"`
}
// JSON type overrides for payloadResponse.
type payloadResponseMarshaling struct {
PayloadID hexutil.Uint64
}
type NewBlockResponse struct {
Valid bool `json:"valid"`
}
type genericResponse struct {
type GenericResponse struct {
Success bool `json:"success"`
}
type GenericStringResponse struct {
Status string `json:"status"`
}
type ConsensusValidatedParams struct {
BlockHash common.Hash `json:"blockHash"`
Status string `json:"status"`
}
type ForkChoiceParams struct {
HeadBlockHash common.Hash `json:"headBlockHash"`
FinalizedBlockHash common.Hash `json:"finalizedBlockHash"`
}

View File

@ -13,34 +13,48 @@ import (
var _ = (*assembleBlockParamsMarshaling)(nil)
// MarshalJSON marshals as JSON.
func (a assembleBlockParams) MarshalJSON() ([]byte, error) {
type assembleBlockParams struct {
ParentHash common.Hash `json:"parentHash" gencodec:"required"`
Timestamp hexutil.Uint64 `json:"timestamp" gencodec:"required"`
func (a AssembleBlockParams) MarshalJSON() ([]byte, error) {
type AssembleBlockParams struct {
ParentHash common.Hash `json:"parentHash" gencodec:"required"`
Timestamp hexutil.Uint64 `json:"timestamp" gencodec:"required"`
Random common.Hash `json:"random" gencodec:"required"`
FeeRecipient common.Address `json:"feeRecipient" gencodec:"required"`
}
var enc assembleBlockParams
var enc AssembleBlockParams
enc.ParentHash = a.ParentHash
enc.Timestamp = hexutil.Uint64(a.Timestamp)
enc.Random = a.Random
enc.FeeRecipient = a.FeeRecipient
return json.Marshal(&enc)
}
// UnmarshalJSON unmarshals from JSON.
func (a *assembleBlockParams) UnmarshalJSON(input []byte) error {
type assembleBlockParams struct {
ParentHash *common.Hash `json:"parentHash" gencodec:"required"`
Timestamp *hexutil.Uint64 `json:"timestamp" gencodec:"required"`
func (a *AssembleBlockParams) UnmarshalJSON(input []byte) error {
type AssembleBlockParams struct {
ParentHash *common.Hash `json:"parentHash" gencodec:"required"`
Timestamp *hexutil.Uint64 `json:"timestamp" gencodec:"required"`
Random *common.Hash `json:"random" gencodec:"required"`
FeeRecipient *common.Address `json:"feeRecipient" gencodec:"required"`
}
var dec assembleBlockParams
var dec AssembleBlockParams
if err := json.Unmarshal(input, &dec); err != nil {
return err
}
if dec.ParentHash == nil {
return errors.New("missing required field 'parentHash' for assembleBlockParams")
return errors.New("missing required field 'parentHash' for AssembleBlockParams")
}
a.ParentHash = *dec.ParentHash
if dec.Timestamp == nil {
return errors.New("missing required field 'timestamp' for assembleBlockParams")
return errors.New("missing required field 'timestamp' for AssembleBlockParams")
}
a.Timestamp = uint64(*dec.Timestamp)
if dec.Random == nil {
return errors.New("missing required field 'random' for AssembleBlockParams")
}
a.Random = *dec.Random
if dec.FeeRecipient == nil {
return errors.New("missing required field 'feeRecipient' for AssembleBlockParams")
}
a.FeeRecipient = *dec.FeeRecipient
return nil
}

View File

@ -5,6 +5,7 @@ package catalyst
import (
"encoding/json"
"errors"
"math/big"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/common/hexutil"
@ -13,31 +14,37 @@ import (
var _ = (*executableDataMarshaling)(nil)
// MarshalJSON marshals as JSON.
func (e executableData) MarshalJSON() ([]byte, error) {
type executableData struct {
BlockHash common.Hash `json:"blockHash" gencodec:"required"`
ParentHash common.Hash `json:"parentHash" gencodec:"required"`
Miner common.Address `json:"miner" gencodec:"required"`
StateRoot common.Hash `json:"stateRoot" gencodec:"required"`
Number hexutil.Uint64 `json:"number" gencodec:"required"`
GasLimit hexutil.Uint64 `json:"gasLimit" gencodec:"required"`
GasUsed hexutil.Uint64 `json:"gasUsed" gencodec:"required"`
Timestamp hexutil.Uint64 `json:"timestamp" gencodec:"required"`
ReceiptRoot common.Hash `json:"receiptsRoot" gencodec:"required"`
LogsBloom hexutil.Bytes `json:"logsBloom" gencodec:"required"`
Transactions []hexutil.Bytes `json:"transactions" gencodec:"required"`
func (e ExecutableData) MarshalJSON() ([]byte, error) {
type ExecutableData struct {
BlockHash common.Hash `json:"blockHash" gencodec:"required"`
ParentHash common.Hash `json:"parentHash" gencodec:"required"`
Coinbase common.Address `json:"coinbase" gencodec:"required"`
StateRoot common.Hash `json:"stateRoot" gencodec:"required"`
ReceiptRoot common.Hash `json:"receiptRoot" gencodec:"required"`
LogsBloom hexutil.Bytes `json:"logsBloom" gencodec:"required"`
Random common.Hash `json:"random" gencodec:"required"`
Number hexutil.Uint64 `json:"blockNumber" gencodec:"required"`
GasLimit hexutil.Uint64 `json:"gasLimit" gencodec:"required"`
GasUsed hexutil.Uint64 `json:"gasUsed" gencodec:"required"`
Timestamp hexutil.Uint64 `json:"timestamp" gencodec:"required"`
ExtraData hexutil.Bytes `json:"extraData" gencodec:"required"`
BaseFeePerGas *hexutil.Big `json:"baseFeePerGas" gencodec:"required"`
Transactions []hexutil.Bytes `json:"transactions" gencodec:"required"`
}
var enc executableData
var enc ExecutableData
enc.BlockHash = e.BlockHash
enc.ParentHash = e.ParentHash
enc.Miner = e.Miner
enc.Coinbase = e.Coinbase
enc.StateRoot = e.StateRoot
enc.ReceiptRoot = e.ReceiptRoot
enc.LogsBloom = e.LogsBloom
enc.Random = e.Random
enc.Number = hexutil.Uint64(e.Number)
enc.GasLimit = hexutil.Uint64(e.GasLimit)
enc.GasUsed = hexutil.Uint64(e.GasUsed)
enc.Timestamp = hexutil.Uint64(e.Timestamp)
enc.ReceiptRoot = e.ReceiptRoot
enc.LogsBloom = e.LogsBloom
enc.ExtraData = e.ExtraData
enc.BaseFeePerGas = (*hexutil.Big)(e.BaseFeePerGas)
if e.Transactions != nil {
enc.Transactions = make([]hexutil.Bytes, len(e.Transactions))
for k, v := range e.Transactions {
@ -48,66 +55,81 @@ func (e executableData) MarshalJSON() ([]byte, error) {
}
// UnmarshalJSON unmarshals from JSON.
func (e *executableData) UnmarshalJSON(input []byte) error {
type executableData struct {
BlockHash *common.Hash `json:"blockHash" gencodec:"required"`
ParentHash *common.Hash `json:"parentHash" gencodec:"required"`
Miner *common.Address `json:"miner" gencodec:"required"`
StateRoot *common.Hash `json:"stateRoot" gencodec:"required"`
Number *hexutil.Uint64 `json:"number" gencodec:"required"`
GasLimit *hexutil.Uint64 `json:"gasLimit" gencodec:"required"`
GasUsed *hexutil.Uint64 `json:"gasUsed" gencodec:"required"`
Timestamp *hexutil.Uint64 `json:"timestamp" gencodec:"required"`
ReceiptRoot *common.Hash `json:"receiptsRoot" gencodec:"required"`
LogsBloom *hexutil.Bytes `json:"logsBloom" gencodec:"required"`
Transactions []hexutil.Bytes `json:"transactions" gencodec:"required"`
func (e *ExecutableData) UnmarshalJSON(input []byte) error {
type ExecutableData struct {
BlockHash *common.Hash `json:"blockHash" gencodec:"required"`
ParentHash *common.Hash `json:"parentHash" gencodec:"required"`
Coinbase *common.Address `json:"coinbase" gencodec:"required"`
StateRoot *common.Hash `json:"stateRoot" gencodec:"required"`
ReceiptRoot *common.Hash `json:"receiptRoot" gencodec:"required"`
LogsBloom *hexutil.Bytes `json:"logsBloom" gencodec:"required"`
Random *common.Hash `json:"random" gencodec:"required"`
Number *hexutil.Uint64 `json:"blockNumber" gencodec:"required"`
GasLimit *hexutil.Uint64 `json:"gasLimit" gencodec:"required"`
GasUsed *hexutil.Uint64 `json:"gasUsed" gencodec:"required"`
Timestamp *hexutil.Uint64 `json:"timestamp" gencodec:"required"`
ExtraData *hexutil.Bytes `json:"extraData" gencodec:"required"`
BaseFeePerGas *hexutil.Big `json:"baseFeePerGas" gencodec:"required"`
Transactions []hexutil.Bytes `json:"transactions" gencodec:"required"`
}
var dec executableData
var dec ExecutableData
if err := json.Unmarshal(input, &dec); err != nil {
return err
}
if dec.BlockHash == nil {
return errors.New("missing required field 'blockHash' for executableData")
return errors.New("missing required field 'blockHash' for ExecutableData")
}
e.BlockHash = *dec.BlockHash
if dec.ParentHash == nil {
return errors.New("missing required field 'parentHash' for executableData")
return errors.New("missing required field 'parentHash' for ExecutableData")
}
e.ParentHash = *dec.ParentHash
if dec.Miner == nil {
return errors.New("missing required field 'miner' for executableData")
if dec.Coinbase == nil {
return errors.New("missing required field 'coinbase' for ExecutableData")
}
e.Miner = *dec.Miner
e.Coinbase = *dec.Coinbase
if dec.StateRoot == nil {
return errors.New("missing required field 'stateRoot' for executableData")
return errors.New("missing required field 'stateRoot' for ExecutableData")
}
e.StateRoot = *dec.StateRoot
if dec.Number == nil {
return errors.New("missing required field 'number' for executableData")
}
e.Number = uint64(*dec.Number)
if dec.GasLimit == nil {
return errors.New("missing required field 'gasLimit' for executableData")
}
e.GasLimit = uint64(*dec.GasLimit)
if dec.GasUsed == nil {
return errors.New("missing required field 'gasUsed' for executableData")
}
e.GasUsed = uint64(*dec.GasUsed)
if dec.Timestamp == nil {
return errors.New("missing required field 'timestamp' for executableData")
}
e.Timestamp = uint64(*dec.Timestamp)
if dec.ReceiptRoot == nil {
return errors.New("missing required field 'receiptsRoot' for executableData")
return errors.New("missing required field 'receiptRoot' for ExecutableData")
}
e.ReceiptRoot = *dec.ReceiptRoot
if dec.LogsBloom == nil {
return errors.New("missing required field 'logsBloom' for executableData")
return errors.New("missing required field 'logsBloom' for ExecutableData")
}
e.LogsBloom = *dec.LogsBloom
if dec.Random == nil {
return errors.New("missing required field 'random' for ExecutableData")
}
e.Random = *dec.Random
if dec.Number == nil {
return errors.New("missing required field 'blockNumber' for ExecutableData")
}
e.Number = uint64(*dec.Number)
if dec.GasLimit == nil {
return errors.New("missing required field 'gasLimit' for ExecutableData")
}
e.GasLimit = uint64(*dec.GasLimit)
if dec.GasUsed == nil {
return errors.New("missing required field 'gasUsed' for ExecutableData")
}
e.GasUsed = uint64(*dec.GasUsed)
if dec.Timestamp == nil {
return errors.New("missing required field 'timestamp' for ExecutableData")
}
e.Timestamp = uint64(*dec.Timestamp)
if dec.ExtraData == nil {
return errors.New("missing required field 'extraData' for ExecutableData")
}
e.ExtraData = *dec.ExtraData
if dec.BaseFeePerGas == nil {
return errors.New("missing required field 'baseFeePerGas' for ExecutableData")
}
e.BaseFeePerGas = (*big.Int)(dec.BaseFeePerGas)
if dec.Transactions == nil {
return errors.New("missing required field 'transactions' for executableData")
return errors.New("missing required field 'transactions' for ExecutableData")
}
e.Transactions = make([][]byte, len(dec.Transactions))
for k, v := range dec.Transactions {

View File

@ -0,0 +1,36 @@
// Code generated by github.com/fjl/gencodec. DO NOT EDIT.
package catalyst
import (
"encoding/json"
"github.com/ethereum/go-ethereum/common/hexutil"
)
var _ = (*payloadResponseMarshaling)(nil)
// MarshalJSON marshals as JSON.
func (p PayloadResponse) MarshalJSON() ([]byte, error) {
type PayloadResponse struct {
PayloadID hexutil.Uint64 `json:"payloadId"`
}
var enc PayloadResponse
enc.PayloadID = hexutil.Uint64(p.PayloadID)
return json.Marshal(&enc)
}
// UnmarshalJSON unmarshals from JSON.
func (p *PayloadResponse) UnmarshalJSON(input []byte) error {
type PayloadResponse struct {
PayloadID *hexutil.Uint64 `json:"payloadId"`
}
var dec PayloadResponse
if err := json.Unmarshal(input, &dec); err != nil {
return err
}
if dec.PayloadID != nil {
p.PayloadID = uint64(*dec.PayloadID)
}
return nil
}

View File

@ -1720,6 +1720,9 @@ func (d *Downloader) importBlockResults(results []*fetchResult) error {
for i, result := range results {
blocks[i] = types.NewBlockWithHeader(result.Header).WithBody(result.Transactions, result.Uncles)
}
// Downloaded blocks are always regarded as trusted after the
// transition. Because the downloaded chain is guided by the
// consensus-layer.
if index, err := d.blockchain.InsertChain(blocks); err != nil {
if index < len(results) {
log.Debug("Downloaded item processing failed", "number", results[index].Header.Number, "hash", results[index].Header.Hash(), "err", err)

View File

@ -27,6 +27,7 @@ import (
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/consensus"
"github.com/ethereum/go-ethereum/consensus/beacon"
"github.com/ethereum/go-ethereum/consensus/clique"
"github.com/ethereum/go-ethereum/consensus/ethash"
"github.com/ethereum/go-ethereum/core"
@ -204,35 +205,39 @@ type Config struct {
// Arrow Glacier block override (TODO: remove after the fork)
OverrideArrowGlacier *big.Int `toml:",omitempty"`
// OverrideTerminalTotalDifficulty (TODO: remove after the fork)
OverrideTerminalTotalDifficulty *big.Int `toml:",omitempty"`
}
// CreateConsensusEngine creates a consensus engine for the given chain configuration.
func CreateConsensusEngine(stack *node.Node, chainConfig *params.ChainConfig, config *ethash.Config, notify []string, noverify bool, db ethdb.Database) consensus.Engine {
// If proof-of-authority is requested, set it up
var engine consensus.Engine
if chainConfig.Clique != nil {
return clique.New(chainConfig.Clique, db)
engine = clique.New(chainConfig.Clique, db)
} else {
switch config.PowMode {
case ethash.ModeFake:
log.Warn("Ethash used in fake mode")
case ethash.ModeTest:
log.Warn("Ethash used in test mode")
case ethash.ModeShared:
log.Warn("Ethash used in shared mode")
}
engine = ethash.New(ethash.Config{
PowMode: config.PowMode,
CacheDir: stack.ResolvePath(config.CacheDir),
CachesInMem: config.CachesInMem,
CachesOnDisk: config.CachesOnDisk,
CachesLockMmap: config.CachesLockMmap,
DatasetDir: config.DatasetDir,
DatasetsInMem: config.DatasetsInMem,
DatasetsOnDisk: config.DatasetsOnDisk,
DatasetsLockMmap: config.DatasetsLockMmap,
NotifyFull: config.NotifyFull,
}, notify, noverify)
engine.(*ethash.Ethash).SetThreads(-1) // Disable CPU mining
}
// Otherwise assume proof-of-work
switch config.PowMode {
case ethash.ModeFake:
log.Warn("Ethash used in fake mode")
case ethash.ModeTest:
log.Warn("Ethash used in test mode")
case ethash.ModeShared:
log.Warn("Ethash used in shared mode")
}
engine := ethash.New(ethash.Config{
PowMode: config.PowMode,
CacheDir: stack.ResolvePath(config.CacheDir),
CachesInMem: config.CachesInMem,
CachesOnDisk: config.CachesOnDisk,
CachesLockMmap: config.CachesLockMmap,
DatasetDir: config.DatasetDir,
DatasetsInMem: config.DatasetsInMem,
DatasetsOnDisk: config.DatasetsOnDisk,
DatasetsLockMmap: config.DatasetsLockMmap,
NotifyFull: config.NotifyFull,
}, notify, noverify)
engine.SetThreads(-1) // Disable CPU mining
return engine
return beacon.New(engine)
}

View File

@ -18,48 +18,49 @@ import (
// MarshalTOML marshals as TOML.
func (c Config) MarshalTOML() (interface{}, error) {
type Config struct {
Genesis *core.Genesis `toml:",omitempty"`
NetworkId uint64
SyncMode downloader.SyncMode
EthDiscoveryURLs []string
SnapDiscoveryURLs []string
NoPruning bool
NoPrefetch bool
TxLookupLimit uint64 `toml:",omitempty"`
Whitelist map[uint64]common.Hash `toml:"-"`
LightServ int `toml:",omitempty"`
LightIngress int `toml:",omitempty"`
LightEgress int `toml:",omitempty"`
LightPeers int `toml:",omitempty"`
LightNoPrune bool `toml:",omitempty"`
LightNoSyncServe bool `toml:",omitempty"`
SyncFromCheckpoint bool `toml:",omitempty"`
UltraLightServers []string `toml:",omitempty"`
UltraLightFraction int `toml:",omitempty"`
UltraLightOnlyAnnounce bool `toml:",omitempty"`
SkipBcVersionCheck bool `toml:"-"`
DatabaseHandles int `toml:"-"`
DatabaseCache int
DatabaseFreezer string
TrieCleanCache int
TrieCleanCacheJournal string `toml:",omitempty"`
TrieCleanCacheRejournal time.Duration `toml:",omitempty"`
TrieDirtyCache int
TrieTimeout time.Duration
SnapshotCache int
Preimages bool
Miner miner.Config
Ethash ethash.Config
TxPool core.TxPoolConfig
GPO gasprice.Config
EnablePreimageRecording bool
DocRoot string `toml:"-"`
RPCGasCap uint64
RPCEVMTimeout time.Duration
RPCTxFeeCap float64
Checkpoint *params.TrustedCheckpoint `toml:",omitempty"`
CheckpointOracle *params.CheckpointOracleConfig `toml:",omitempty"`
OverrideArrowGlacier *big.Int `toml:",omitempty"`
Genesis *core.Genesis `toml:",omitempty"`
NetworkId uint64
SyncMode downloader.SyncMode
EthDiscoveryURLs []string
SnapDiscoveryURLs []string
NoPruning bool
NoPrefetch bool
TxLookupLimit uint64 `toml:",omitempty"`
Whitelist map[uint64]common.Hash `toml:"-"`
LightServ int `toml:",omitempty"`
LightIngress int `toml:",omitempty"`
LightEgress int `toml:",omitempty"`
LightPeers int `toml:",omitempty"`
LightNoPrune bool `toml:",omitempty"`
LightNoSyncServe bool `toml:",omitempty"`
SyncFromCheckpoint bool `toml:",omitempty"`
UltraLightServers []string `toml:",omitempty"`
UltraLightFraction int `toml:",omitempty"`
UltraLightOnlyAnnounce bool `toml:",omitempty"`
SkipBcVersionCheck bool `toml:"-"`
DatabaseHandles int `toml:"-"`
DatabaseCache int
DatabaseFreezer string
TrieCleanCache int
TrieCleanCacheJournal string `toml:",omitempty"`
TrieCleanCacheRejournal time.Duration `toml:",omitempty"`
TrieDirtyCache int
TrieTimeout time.Duration
SnapshotCache int
Preimages bool
Miner miner.Config
Ethash ethash.Config
TxPool core.TxPoolConfig
GPO gasprice.Config
EnablePreimageRecording bool
DocRoot string `toml:"-"`
RPCGasCap uint64
RPCEVMTimeout time.Duration
RPCTxFeeCap float64
Checkpoint *params.TrustedCheckpoint `toml:",omitempty"`
CheckpointOracle *params.CheckpointOracleConfig `toml:",omitempty"`
OverrideArrowGlacier *big.Int `toml:",omitempty"`
OverrideTerminalTotalDifficulty *big.Int `toml:",omitempty"`
}
var enc Config
enc.Genesis = c.Genesis
@ -104,54 +105,56 @@ func (c Config) MarshalTOML() (interface{}, error) {
enc.Checkpoint = c.Checkpoint
enc.CheckpointOracle = c.CheckpointOracle
enc.OverrideArrowGlacier = c.OverrideArrowGlacier
enc.OverrideTerminalTotalDifficulty = c.OverrideTerminalTotalDifficulty
return &enc, nil
}
// UnmarshalTOML unmarshals from TOML.
func (c *Config) UnmarshalTOML(unmarshal func(interface{}) error) error {
type Config struct {
Genesis *core.Genesis `toml:",omitempty"`
NetworkId *uint64
SyncMode *downloader.SyncMode
EthDiscoveryURLs []string
SnapDiscoveryURLs []string
NoPruning *bool
NoPrefetch *bool
TxLookupLimit *uint64 `toml:",omitempty"`
Whitelist map[uint64]common.Hash `toml:"-"`
LightServ *int `toml:",omitempty"`
LightIngress *int `toml:",omitempty"`
LightEgress *int `toml:",omitempty"`
LightPeers *int `toml:",omitempty"`
LightNoPrune *bool `toml:",omitempty"`
LightNoSyncServe *bool `toml:",omitempty"`
SyncFromCheckpoint *bool `toml:",omitempty"`
UltraLightServers []string `toml:",omitempty"`
UltraLightFraction *int `toml:",omitempty"`
UltraLightOnlyAnnounce *bool `toml:",omitempty"`
SkipBcVersionCheck *bool `toml:"-"`
DatabaseHandles *int `toml:"-"`
DatabaseCache *int
DatabaseFreezer *string
TrieCleanCache *int
TrieCleanCacheJournal *string `toml:",omitempty"`
TrieCleanCacheRejournal *time.Duration `toml:",omitempty"`
TrieDirtyCache *int
TrieTimeout *time.Duration
SnapshotCache *int
Preimages *bool
Miner *miner.Config
Ethash *ethash.Config
TxPool *core.TxPoolConfig
GPO *gasprice.Config
EnablePreimageRecording *bool
DocRoot *string `toml:"-"`
RPCGasCap *uint64
RPCEVMTimeout *time.Duration
RPCTxFeeCap *float64
Checkpoint *params.TrustedCheckpoint `toml:",omitempty"`
CheckpointOracle *params.CheckpointOracleConfig `toml:",omitempty"`
OverrideArrowGlacier *big.Int `toml:",omitempty"`
Genesis *core.Genesis `toml:",omitempty"`
NetworkId *uint64
SyncMode *downloader.SyncMode
EthDiscoveryURLs []string
SnapDiscoveryURLs []string
NoPruning *bool
NoPrefetch *bool
TxLookupLimit *uint64 `toml:",omitempty"`
Whitelist map[uint64]common.Hash `toml:"-"`
LightServ *int `toml:",omitempty"`
LightIngress *int `toml:",omitempty"`
LightEgress *int `toml:",omitempty"`
LightPeers *int `toml:",omitempty"`
LightNoPrune *bool `toml:",omitempty"`
LightNoSyncServe *bool `toml:",omitempty"`
SyncFromCheckpoint *bool `toml:",omitempty"`
UltraLightServers []string `toml:",omitempty"`
UltraLightFraction *int `toml:",omitempty"`
UltraLightOnlyAnnounce *bool `toml:",omitempty"`
SkipBcVersionCheck *bool `toml:"-"`
DatabaseHandles *int `toml:"-"`
DatabaseCache *int
DatabaseFreezer *string
TrieCleanCache *int
TrieCleanCacheJournal *string `toml:",omitempty"`
TrieCleanCacheRejournal *time.Duration `toml:",omitempty"`
TrieDirtyCache *int
TrieTimeout *time.Duration
SnapshotCache *int
Preimages *bool
Miner *miner.Config
Ethash *ethash.Config
TxPool *core.TxPoolConfig
GPO *gasprice.Config
EnablePreimageRecording *bool
DocRoot *string `toml:"-"`
RPCGasCap *uint64
RPCEVMTimeout *time.Duration
RPCTxFeeCap *float64
Checkpoint *params.TrustedCheckpoint `toml:",omitempty"`
CheckpointOracle *params.CheckpointOracleConfig `toml:",omitempty"`
OverrideArrowGlacier *big.Int `toml:",omitempty"`
OverrideTerminalTotalDifficulty *big.Int `toml:",omitempty"`
}
var dec Config
if err := unmarshal(&dec); err != nil {
@ -283,5 +286,8 @@ func (c *Config) UnmarshalTOML(unmarshal func(interface{}) error) error {
if dec.OverrideArrowGlacier != nil {
c.OverrideArrowGlacier = dec.OverrideArrowGlacier
}
if dec.OverrideTerminalTotalDifficulty != nil {
c.OverrideTerminalTotalDifficulty = dec.OverrideTerminalTotalDifficulty
}
return nil
}

View File

@ -144,7 +144,7 @@ func newTestBackend(t *testing.T, londonBlock *big.Int, pending bool) *testBacke
// Construct testing chain
diskdb := rawdb.NewMemoryDatabase()
gspec.Commit(diskdb)
chain, err := core.NewBlockChain(diskdb, &core.CacheConfig{TrieCleanNoPrefetch: true}, &config, engine, vm.Config{}, nil, nil)
chain, err := core.NewBlockChain(diskdb, &core.CacheConfig{TrieCleanNoPrefetch: true}, gspec.Config, engine, vm.Config{}, nil, nil)
if err != nil {
t.Fatalf("Failed to create local chain, %v", err)
}

View File

@ -25,6 +25,8 @@ import (
"time"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/consensus"
"github.com/ethereum/go-ethereum/consensus/beacon"
"github.com/ethereum/go-ethereum/core"
"github.com/ethereum/go-ethereum/core/forkid"
"github.com/ethereum/go-ethereum/core/types"
@ -79,6 +81,7 @@ type handlerConfig struct {
Database ethdb.Database // Database for direct sync insertions
Chain *core.BlockChain // Blockchain to serve data from
TxPool txPool // Transaction pool to propagate from
Merger *consensus.Merger // The manager for eth1/2 transition
Network uint64 // Network identifier to adfvertise
Sync downloader.SyncMode // Whether to fast or full sync
BloomCache uint64 // Megabytes to alloc for fast sync bloom
@ -108,6 +111,7 @@ type handler struct {
blockFetcher *fetcher.BlockFetcher
txFetcher *fetcher.TxFetcher
peers *peerSet
merger *consensus.Merger
eventMux *event.TypeMux
txsCh chan core.NewTxsEvent
@ -138,6 +142,7 @@ func newHandler(config *handlerConfig) (*handler, error) {
txpool: config.TxPool,
chain: config.Chain,
peers: newPeerSet(),
merger: config.Merger,
whitelist: config.Whitelist,
quitSync: make(chan struct{}),
}
@ -186,12 +191,41 @@ func newHandler(config *handlerConfig) (*handler, error) {
// Construct the fetcher (short sync)
validator := func(header *types.Header) error {
// All the block fetcher activities should be disabled
// after the transition. Print the warning log.
if h.merger.PoSFinalized() {
log.Warn("Unexpected validation activity", "hash", header.Hash(), "number", header.Number)
return errors.New("unexpected behavior after transition")
}
// Reject all the PoS style headers in the first place. No matter
// the chain has finished the transition or not, the PoS headers
// should only come from the trusted consensus layer instead of
// p2p network.
if beacon, ok := h.chain.Engine().(*beacon.Beacon); ok {
if beacon.IsPoSHeader(header) {
return errors.New("unexpected post-merge header")
}
}
return h.chain.Engine().VerifyHeader(h.chain, header, true)
}
heighter := func() uint64 {
return h.chain.CurrentBlock().NumberU64()
}
inserter := func(blocks types.Blocks) (int, error) {
// All the block fetcher activities should be disabled
// after the transition. Print the warning log.
if h.merger.PoSFinalized() {
var ctx []interface{}
ctx = append(ctx, "blocks", len(blocks))
if len(blocks) > 0 {
ctx = append(ctx, "firsthash", blocks[0].Hash())
ctx = append(ctx, "firstnumber", blocks[0].Number())
ctx = append(ctx, "lasthash", blocks[len(blocks)-1].Hash())
ctx = append(ctx, "lastnumber", blocks[len(blocks)-1].Number())
}
log.Warn("Unexpected insertion activity", ctx...)
return 0, errors.New("unexpected behavior after transition")
}
// If sync hasn't reached the checkpoint yet, deny importing weird blocks.
//
// Ideally we would also compare the head block's timestamp and similarly reject
@ -211,6 +245,29 @@ func newHandler(config *handlerConfig) (*handler, error) {
log.Warn("Fast syncing, discarded propagated block", "number", blocks[0].Number(), "hash", blocks[0].Hash())
return 0, nil
}
if h.merger.TDDReached() {
// The blocks from the p2p network is regarded as untrusted
// after the transition. In theory block gossip should be disabled
// entirely whenever the transition is started. But in order to
// handle the transition boundary reorg in the consensus-layer,
// the legacy blocks are still accepted, but only for the terminal
// pow blocks. Spec: https://github.com/ethereum/EIPs/blob/master/EIPS/eip-3675.md#halt-the-importing-of-pow-blocks
for i, block := range blocks {
ptd := h.chain.GetTd(block.ParentHash(), block.NumberU64()-1)
if ptd == nil {
return 0, nil
}
td := new(big.Int).Add(ptd, block.Difficulty())
if !h.chain.Config().IsTerminalPoWBlock(ptd, td) {
log.Info("Filtered out non-termimal pow block", "number", block.NumberU64(), "hash", block.Hash())
return 0, nil
}
if err := h.chain.InsertBlockWithoutSetHead(block); err != nil {
return i, err
}
}
return 0, nil
}
n, err := h.chain.InsertChain(blocks)
if err == nil {
atomic.StoreUint32(&h.acceptTxs, 1) // Mark initial sync done on any fetcher import
@ -432,6 +489,17 @@ func (h *handler) Stop() {
// BroadcastBlock will either propagate a block to a subset of its peers, or
// will only announce its availability (depending what's requested).
func (h *handler) BroadcastBlock(block *types.Block, propagate bool) {
// Disable the block propagation if the chain has already entered the PoS
// stage. The block propagation is delegated to the consensus layer.
if h.merger.PoSFinalized() {
return
}
// Disable the block propagation if it's the post-merge block.
if beacon, ok := h.chain.Engine().(*beacon.Beacon); ok {
if beacon.IsPoSHeader(block.Header()) {
return
}
}
hash := block.Hash()
peers := h.peers.peersWithoutBlock(hash)

View File

@ -180,6 +180,14 @@ func (h *ethHandler) handleBodies(peer *eth.Peer, txs [][]*types.Transaction, un
// handleBlockAnnounces is invoked from a peer's message handler when it transmits a
// batch of block announcements for the local node to process.
func (h *ethHandler) handleBlockAnnounces(peer *eth.Peer, hashes []common.Hash, numbers []uint64) error {
// Drop all incoming block announces from the p2p network if
// the chain already entered the pos stage and disconnect the
// remote peer.
if h.merger.PoSFinalized() {
// TODO (MariusVanDerWijden) drop non-updated peers after the merge
return nil
// return errors.New("unexpected block announces")
}
// Schedule all the unknown hashes for retrieval
var (
unknownHashes = make([]common.Hash, 0, len(hashes))
@ -200,6 +208,14 @@ func (h *ethHandler) handleBlockAnnounces(peer *eth.Peer, hashes []common.Hash,
// handleBlockBroadcast is invoked from a peer's message handler when it transmits a
// block broadcast for the local node to process.
func (h *ethHandler) handleBlockBroadcast(peer *eth.Peer, block *types.Block, td *big.Int) error {
// Drop all incoming block announces from the p2p network if
// the chain already entered the pos stage and disconnect the
// remote peer.
if h.merger.PoSFinalized() {
// TODO (MariusVanDerWijden) drop non-updated peers after the merge
return nil
// return errors.New("unexpected block announces")
}
// Schedule the block for import
h.blockFetcher.Enqueue(peer.ID(), block)

View File

@ -25,6 +25,7 @@ import (
"time"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/consensus"
"github.com/ethereum/go-ethereum/consensus/ethash"
"github.com/ethereum/go-ethereum/core"
"github.com/ethereum/go-ethereum/core/forkid"
@ -115,6 +116,7 @@ func testForkIDSplit(t *testing.T, protocol uint) {
Database: dbNoFork,
Chain: chainNoFork,
TxPool: newTestTxPool(),
Merger: consensus.NewMerger(rawdb.NewMemoryDatabase()),
Network: 1,
Sync: downloader.FullSync,
BloomCache: 1,
@ -123,6 +125,7 @@ func testForkIDSplit(t *testing.T, protocol uint) {
Database: dbProFork,
Chain: chainProFork,
TxPool: newTestTxPool(),
Merger: consensus.NewMerger(rawdb.NewMemoryDatabase()),
Network: 1,
Sync: downloader.FullSync,
BloomCache: 1,

View File

@ -22,6 +22,7 @@ import (
"sync"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/consensus"
"github.com/ethereum/go-ethereum/consensus/ethash"
"github.com/ethereum/go-ethereum/core"
"github.com/ethereum/go-ethereum/core/rawdb"
@ -149,6 +150,7 @@ func newTestHandlerWithBlocks(blocks int) *testHandler {
Database: db,
Chain: chain,
TxPool: txpool,
Merger: consensus.NewMerger(rawdb.NewMemoryDatabase()),
Network: 1,
Sync: downloader.FastSync,
BloomCache: 1,

View File

@ -30,7 +30,7 @@ import (
"github.com/ethereum/go-ethereum/trie"
)
// stateAtBlock retrieves the state database associated with a certain block.
// StateAtBlock retrieves the state database associated with a certain block.
// If no state is locally available for the given block, a number of blocks
// are attempted to be reexecuted to generate the desired state. The optional
// base layer statedb can be passed then it's regarded as the statedb of the
@ -45,7 +45,7 @@ import (
// storing trash persistently
// - preferDisk: this arg can be used by the caller to signal that even though the 'base' is provided,
// it would be preferrable to start from a fresh state, if we have it on disk.
func (eth *Ethereum) stateAtBlock(block *types.Block, reexec uint64, base *state.StateDB, checkLive bool, preferDisk bool) (statedb *state.StateDB, err error) {
func (eth *Ethereum) StateAtBlock(block *types.Block, reexec uint64, base *state.StateDB, checkLive bool, preferDisk bool) (statedb *state.StateDB, err error) {
var (
current *types.Block
database state.Database
@ -171,7 +171,7 @@ func (eth *Ethereum) stateAtTransaction(block *types.Block, txIndex int, reexec
}
// Lookup the statedb of parent block from the live database,
// otherwise regenerate it on the flight.
statedb, err := eth.stateAtBlock(parent, reexec, nil, true, false)
statedb, err := eth.StateAtBlock(parent, reexec, nil, true, false)
if err != nil {
return nil, vm.BlockContext{}, nil, err
}

View File

@ -145,7 +145,10 @@ func (cs *chainSyncer) nextSyncOp() *chainSyncOp {
if cs.doneCh != nil {
return nil // Sync already running.
}
// Disable the td based sync trigger after the transition
if cs.handler.merger.TDDReached() {
return nil
}
// Ensure we're at minimum peer count.
minPeers := defaultMinSyncPeers
if cs.forced {

View File

@ -63,6 +63,7 @@ type LightEthereum struct {
serverPool *vfc.ServerPool
serverPoolIterator enode.Iterator
pruner *pruner
merger *consensus.Merger
bloomRequests chan chan *bloombits.Retrieval // Channel receiving bloom data retrieval requests
bloomIndexer *core.ChainIndexer // Bloom indexer operating during block imports
@ -88,13 +89,14 @@ func New(stack *node.Node, config *ethconfig.Config) (*LightEthereum, error) {
if err != nil {
return nil, err
}
chainConfig, genesisHash, genesisErr := core.SetupGenesisBlockWithOverride(chainDb, config.Genesis, config.OverrideArrowGlacier)
chainConfig, genesisHash, genesisErr := core.SetupGenesisBlockWithOverride(chainDb, config.Genesis, config.OverrideArrowGlacier, config.OverrideTerminalTotalDifficulty)
if _, isCompat := genesisErr.(*params.ConfigCompatError); genesisErr != nil && !isCompat {
return nil, genesisErr
}
log.Info("Initialised chain configuration", "config", chainConfig)
peers := newServerPeerSet()
merger := consensus.NewMerger(chainDb)
leth := &LightEthereum{
lesCommons: lesCommons{
genesis: genesisHash,
@ -109,6 +111,7 @@ func New(stack *node.Node, config *ethconfig.Config) (*LightEthereum, error) {
eventMux: stack.EventMux(),
reqDist: newRequestDistributor(peers, &mclock.System{}),
accountManager: stack.AccountManager(),
merger: merger,
engine: ethconfig.CreateConsensusEngine(stack, chainConfig, &config.Ethash, nil, false, chainDb),
bloomRequests: make(chan chan *bloombits.Retrieval),
bloomIndexer: core.NewBloomIndexer(chainDb, params.BloomBitsBlocksClient, params.HelperTrieConfirmations),
@ -332,6 +335,7 @@ func (s *LightEthereum) Engine() consensus.Engine { return s.engine }
func (s *LightEthereum) LesVersion() int { return int(ClientProtocolVersions[0]) }
func (s *LightEthereum) Downloader() *downloader.Downloader { return s.handler.downloader }
func (s *LightEthereum) EventMux() *event.TypeMux { return s.eventMux }
func (s *LightEthereum) Merger() *consensus.Merger { return s.merger }
// Protocols returns all the currently configured network protocols to start.
func (s *LightEthereum) Protocols() []p2p.Protocol {

View File

@ -143,11 +143,13 @@ func (h *clientHandler) handle(p *serverPeer, noInitAnnounce bool) error {
connectionTimer.Update(time.Duration(mclock.Now() - connectedAt))
serverConnectionGauge.Update(int64(h.backend.peers.len()))
}()
// It's mainly used in testing which requires discarding initial
// signal to prevent syncing.
if !noInitAnnounce {
// Discard all the announces after the transition
// Also discarding initial signal to prevent syncing during testing.
if !(noInitAnnounce || h.backend.merger.TDDReached()) {
h.fetcher.announce(p, &announceData{Hash: p.headInfo.Hash, Number: p.headInfo.Number, Td: p.headInfo.Td})
}
// Mark the peer starts to be served.
atomic.StoreUint32(&p.serving, 1)
defer atomic.StoreUint32(&p.serving, 0)
@ -212,7 +214,11 @@ func (h *clientHandler) handleMsg(p *serverPeer) error {
// Update peer head information first and then notify the announcement
p.updateHead(req.Hash, req.Number, req.Td)
h.fetcher.announce(p, &req)
// Discard all the announces after the transition
if !h.backend.merger.TDDReached() {
h.fetcher.announce(p, &req)
}
}
case msg.Code == BlockHeadersMsg:
p.Log().Trace("Received block header response message")

View File

@ -71,8 +71,8 @@ type fetcherPeer struct {
// These following two fields can track the latest announces
// from the peer with limited size for caching. We hold the
// assumption that all enqueued announces are td-monotonic.
announces map[common.Hash]*announce // Announcement map
announcesList []common.Hash // FIFO announces list
announces map[common.Hash]*announce // Announcement map
fifo []common.Hash // FIFO announces list
}
// addAnno enqueues an new trusted announcement. If the queued announces overflow,
@ -87,15 +87,15 @@ func (fp *fetcherPeer) addAnno(anno *announce) {
return
}
fp.announces[hash] = anno
fp.announcesList = append(fp.announcesList, hash)
fp.fifo = append(fp.fifo, hash)
// Evict oldest if the announces are oversized.
if len(fp.announcesList)-cachedAnnosThreshold > 0 {
for i := 0; i < len(fp.announcesList)-cachedAnnosThreshold; i++ {
delete(fp.announces, fp.announcesList[i])
if len(fp.fifo)-cachedAnnosThreshold > 0 {
for i := 0; i < len(fp.fifo)-cachedAnnosThreshold; i++ {
delete(fp.announces, fp.fifo[i])
}
copy(fp.announcesList, fp.announcesList[len(fp.announcesList)-cachedAnnosThreshold:])
fp.announcesList = fp.announcesList[:cachedAnnosThreshold]
copy(fp.fifo, fp.fifo[len(fp.fifo)-cachedAnnosThreshold:])
fp.fifo = fp.fifo[:cachedAnnosThreshold]
}
}
@ -106,8 +106,8 @@ func (fp *fetcherPeer) forwardAnno(td *big.Int) []*announce {
cutset int
evicted []*announce
)
for ; cutset < len(fp.announcesList); cutset++ {
anno := fp.announces[fp.announcesList[cutset]]
for ; cutset < len(fp.fifo); cutset++ {
anno := fp.announces[fp.fifo[cutset]]
if anno == nil {
continue // In theory it should never ever happen
}
@ -118,8 +118,8 @@ func (fp *fetcherPeer) forwardAnno(td *big.Int) []*announce {
delete(fp.announces, anno.data.Hash)
}
if cutset > 0 {
copy(fp.announcesList, fp.announcesList[cutset:])
fp.announcesList = fp.announcesList[:len(fp.announcesList)-cutset]
copy(fp.fifo, fp.fifo[cutset:])
fp.fifo = fp.fifo[:len(fp.fifo)-cutset]
}
return evicted
}

View File

@ -33,6 +33,7 @@ import (
"github.com/ethereum/go-ethereum/accounts/abi/bind/backends"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/common/mclock"
"github.com/ethereum/go-ethereum/consensus"
"github.com/ethereum/go-ethereum/consensus/ethash"
"github.com/ethereum/go-ethereum/contracts/checkpointoracle/contract"
"github.com/ethereum/go-ethereum/core"
@ -239,6 +240,7 @@ func newTestClientHandler(backend *backends.SimulatedBackend, odr *LesOdr, index
engine: engine,
blockchain: chain,
eventMux: evmux,
merger: consensus.NewMerger(rawdb.NewMemoryDatabase()),
}
client.handler = newClientHandler(ulcServers, ulcFraction, nil, client)

View File

@ -59,6 +59,7 @@ type LightChain struct {
chainHeadFeed event.Feed
scope event.SubscriptionScope
genesisBlock *types.Block
forker *core.ForkChoice
bodyCache *lru.Cache // Cache for the most recent block bodies
bodyRLPCache *lru.Cache // Cache for the most recent block bodies in RLP encoded format
@ -92,6 +93,7 @@ func NewLightChain(odr OdrBackend, config *params.ChainConfig, engine consensus.
blockCache: blockCache,
engine: engine,
}
bc.forker = core.NewForkChoice(bc, nil)
var err error
bc.hc, err = core.NewHeaderChain(odr.Database(), config, bc.engine, bc.getProcInterrupt)
if err != nil {
@ -369,6 +371,42 @@ func (lc *LightChain) postChainEvents(events []interface{}) {
}
}
func (lc *LightChain) InsertHeader(header *types.Header) error {
// Verify the header first before obtaining the lock
headers := []*types.Header{header}
if _, err := lc.hc.ValidateHeaderChain(headers, 100); err != nil {
return err
}
// Make sure only one thread manipulates the chain at once
lc.chainmu.Lock()
defer lc.chainmu.Unlock()
lc.wg.Add(1)
defer lc.wg.Done()
_, err := lc.hc.WriteHeaders(headers)
log.Info("Inserted header", "number", header.Number, "hash", header.Hash())
return err
}
func (lc *LightChain) SetChainHead(header *types.Header) error {
lc.chainmu.Lock()
defer lc.chainmu.Unlock()
lc.wg.Add(1)
defer lc.wg.Done()
if err := lc.hc.Reorg([]*types.Header{header}); err != nil {
return err
}
// Emit events
block := types.NewBlockWithHeader(header)
lc.chainFeed.Send(core.ChainEvent{Block: block, Hash: block.Hash()})
lc.chainHeadFeed.Send(core.ChainHeadEvent{Block: block})
log.Info("Set the chain head", "number", block.Number(), "hash", block.Hash())
return nil
}
// InsertHeaderChain attempts to insert the given header chain in to the local
// chain, possibly creating a reorg. If an error is returned, it will return the
// index number of the failing header as well an error describing what went wrong.
@ -396,25 +434,23 @@ func (lc *LightChain) InsertHeaderChain(chain []*types.Header, checkFreq int) (i
lc.wg.Add(1)
defer lc.wg.Done()
status, err := lc.hc.InsertHeaderChain(chain, start)
status, err := lc.hc.InsertHeaderChain(chain, start, lc.forker)
if err != nil || len(chain) == 0 {
return 0, err
}
// Create chain event for the new head block of this insertion.
var (
events = make([]interface{}, 0, 1)
lastHeader = chain[len(chain)-1]
block = types.NewBlockWithHeader(lastHeader)
)
switch status {
case core.CanonStatTy:
events = append(events, core.ChainEvent{Block: block, Hash: block.Hash()})
lc.chainFeed.Send(core.ChainEvent{Block: block, Hash: block.Hash()})
lc.chainHeadFeed.Send(core.ChainHeadEvent{Block: block})
case core.SideStatTy:
events = append(events, core.ChainSideEvent{Block: block})
lc.chainSideFeed.Send(core.ChainSideEvent{Block: block})
}
lc.postChainEvents(events)
return 0, err
}

View File

@ -68,7 +68,7 @@ type Miner struct {
wg sync.WaitGroup
}
func New(eth Backend, config *Config, chainConfig *params.ChainConfig, mux *event.TypeMux, engine consensus.Engine, isLocalBlock func(block *types.Block) bool) *Miner {
func New(eth Backend, config *Config, chainConfig *params.ChainConfig, mux *event.TypeMux, engine consensus.Engine, isLocalBlock func(header *types.Header) bool, merger *consensus.Merger) *Miner {
miner := &Miner{
eth: eth,
mux: mux,
@ -76,7 +76,7 @@ func New(eth Backend, config *Config, chainConfig *params.ChainConfig, mux *even
exitCh: make(chan struct{}),
startCh: make(chan common.Address),
stopCh: make(chan struct{}),
worker: newWorker(config, chainConfig, engine, eth, mux, isLocalBlock, true),
worker: newWorker(config, chainConfig, engine, eth, mux, isLocalBlock, true, merger),
}
miner.wg.Add(1)
go miner.update()

View File

@ -22,6 +22,7 @@ import (
"time"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/consensus"
"github.com/ethereum/go-ethereum/consensus/clique"
"github.com/ethereum/go-ethereum/core"
"github.com/ethereum/go-ethereum/core/rawdb"
@ -245,6 +246,7 @@ func createMiner(t *testing.T) (*Miner, *event.TypeMux) {
// Create consensus engine
engine := clique.New(chainConfig.Clique, chainDB)
// Create Ethereum backend
merger := consensus.NewMerger(rawdb.NewMemoryDatabase())
bc, err := core.NewBlockChain(chainDB, nil, chainConfig, engine, vm.Config{}, nil, nil)
if err != nil {
t.Fatalf("can't create new chain %v", err)
@ -257,5 +259,5 @@ func createMiner(t *testing.T) (*Miner, *event.TypeMux) {
// Create event Mux
mux := new(event.TypeMux)
// Create Miner
return New(backend, &config, chainConfig, mux, engine, nil), mux
return New(backend, &config, chainConfig, mux, engine, nil, merger), mux
}

507
miner/stress/beacon/main.go Normal file
View File

@ -0,0 +1,507 @@
// Copyright 2021 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
// This file contains a miner stress test for the eth1/2 transition
package main
import (
"crypto/ecdsa"
"errors"
"io/ioutil"
"math/big"
"math/rand"
"os"
"path/filepath"
"time"
"github.com/ethereum/go-ethereum/accounts/keystore"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/common/fdlimit"
"github.com/ethereum/go-ethereum/common/hexutil"
"github.com/ethereum/go-ethereum/consensus/ethash"
"github.com/ethereum/go-ethereum/core"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/eth"
"github.com/ethereum/go-ethereum/eth/catalyst"
"github.com/ethereum/go-ethereum/eth/downloader"
"github.com/ethereum/go-ethereum/eth/ethconfig"
"github.com/ethereum/go-ethereum/les"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/miner"
"github.com/ethereum/go-ethereum/node"
"github.com/ethereum/go-ethereum/p2p"
"github.com/ethereum/go-ethereum/p2p/enode"
"github.com/ethereum/go-ethereum/params"
)
type nodetype int
const (
legacyMiningNode nodetype = iota
legacyNormalNode
eth2MiningNode
eth2NormalNode
eth2LightClient
)
func (typ nodetype) String() string {
switch typ {
case legacyMiningNode:
return "legacyMiningNode"
case legacyNormalNode:
return "legacyNormalNode"
case eth2MiningNode:
return "eth2MiningNode"
case eth2NormalNode:
return "eth2NormalNode"
case eth2LightClient:
return "eth2LightClient"
default:
return "undefined"
}
}
var (
// transitionDifficulty is the target total difficulty for transition
transitionDifficulty = new(big.Int).Mul(big.NewInt(20), params.MinimumDifficulty)
// blockInterval is the time interval for creating a new eth2 block
blockInterval = time.Second * 3
blockIntervalInt = 3
// finalizationDist is the block distance for finalizing block
finalizationDist = 10
)
type ethNode struct {
typ nodetype
api *catalyst.ConsensusAPI
ethBackend *eth.Ethereum
lesBackend *les.LightEthereum
stack *node.Node
enode *enode.Node
}
func newNode(typ nodetype, genesis *core.Genesis, enodes []*enode.Node) *ethNode {
var (
err error
api *catalyst.ConsensusAPI
stack *node.Node
ethBackend *eth.Ethereum
lesBackend *les.LightEthereum
)
// Start the node and wait until it's up
if typ == eth2LightClient {
stack, lesBackend, api, err = makeLightNode(genesis)
} else {
stack, ethBackend, api, err = makeFullNode(genesis)
}
if err != nil {
panic(err)
}
for stack.Server().NodeInfo().Ports.Listener == 0 {
time.Sleep(250 * time.Millisecond)
}
// Connect the node to all the previous ones
for _, n := range enodes {
stack.Server().AddPeer(n)
}
enode := stack.Server().Self()
// Inject the signer key and start sealing with it
stack.AccountManager().AddBackend(keystore.NewPlaintextKeyStore("beacon-stress"))
store := stack.AccountManager().Backends(keystore.KeyStoreType)[0].(*keystore.KeyStore)
if _, err := store.NewAccount(""); err != nil {
panic(err)
}
return &ethNode{
typ: typ,
api: api,
ethBackend: ethBackend,
lesBackend: lesBackend,
stack: stack,
enode: enode,
}
}
func (n *ethNode) assembleBlock(parentHash common.Hash, parentTimestamp uint64) (*catalyst.ExecutableData, error) {
if n.typ != eth2MiningNode {
return nil, errors.New("invalid node type")
}
payload, err := n.api.PreparePayload(catalyst.AssembleBlockParams{
ParentHash: parentHash,
Timestamp: uint64(time.Now().Unix()),
})
if err != nil {
return nil, err
}
return n.api.GetPayload(hexutil.Uint64(payload.PayloadID))
}
func (n *ethNode) insertBlock(eb catalyst.ExecutableData) error {
if !eth2types(n.typ) {
return errors.New("invalid node type")
}
newResp, err := n.api.ExecutePayload(eb)
if err != nil {
return err
} else if newResp.Status != "VALID" {
return errors.New("failed to insert block")
}
return nil
}
func (n *ethNode) insertBlockAndSetHead(parent *types.Header, ed catalyst.ExecutableData) error {
if !eth2types(n.typ) {
return errors.New("invalid node type")
}
if err := n.insertBlock(ed); err != nil {
return err
}
block, err := catalyst.ExecutableDataToBlock(ed)
if err != nil {
return err
}
if err := n.api.ConsensusValidated(catalyst.ConsensusValidatedParams{BlockHash: block.Hash(), Status: "VALID"}); err != nil {
return err
}
return nil
}
type nodeManager struct {
genesis *core.Genesis
genesisBlock *types.Block
nodes []*ethNode
enodes []*enode.Node
close chan struct{}
}
func newNodeManager(genesis *core.Genesis) *nodeManager {
return &nodeManager{
close: make(chan struct{}),
genesis: genesis,
genesisBlock: genesis.ToBlock(nil),
}
}
func (mgr *nodeManager) createNode(typ nodetype) {
node := newNode(typ, mgr.genesis, mgr.enodes)
mgr.nodes = append(mgr.nodes, node)
mgr.enodes = append(mgr.enodes, node.enode)
}
func (mgr *nodeManager) getNodes(typ nodetype) []*ethNode {
var ret []*ethNode
for _, node := range mgr.nodes {
if node.typ == typ {
ret = append(ret, node)
}
}
return ret
}
func (mgr *nodeManager) startMining() {
for _, node := range append(mgr.getNodes(eth2MiningNode), mgr.getNodes(legacyMiningNode)...) {
if err := node.ethBackend.StartMining(1); err != nil {
panic(err)
}
}
}
func (mgr *nodeManager) shutdown() {
close(mgr.close)
for _, node := range mgr.nodes {
node.stack.Close()
}
}
func (mgr *nodeManager) run() {
if len(mgr.nodes) == 0 {
return
}
chain := mgr.nodes[0].ethBackend.BlockChain()
sink := make(chan core.ChainHeadEvent, 1024)
sub := chain.SubscribeChainHeadEvent(sink)
defer sub.Unsubscribe()
var (
transitioned bool
parentBlock *types.Block
waitFinalise []*types.Block
)
timer := time.NewTimer(0)
defer timer.Stop()
<-timer.C // discard the initial tick
// Handle the by default transition.
if transitionDifficulty.Sign() == 0 {
transitioned = true
parentBlock = mgr.genesisBlock
timer.Reset(blockInterval)
log.Info("Enable the transition by default")
}
// Handle the block finalization.
checkFinalise := func() {
if parentBlock == nil {
return
}
if len(waitFinalise) == 0 {
return
}
oldest := waitFinalise[0]
if oldest.NumberU64() > parentBlock.NumberU64() {
return
}
distance := parentBlock.NumberU64() - oldest.NumberU64()
if int(distance) < finalizationDist {
return
}
nodes := mgr.getNodes(eth2MiningNode)
nodes = append(nodes, mgr.getNodes(eth2NormalNode)...)
nodes = append(nodes, mgr.getNodes(eth2LightClient)...)
for _, node := range append(nodes) {
node.api.ConsensusValidated(catalyst.ConsensusValidatedParams{BlockHash: oldest.Hash(), Status: catalyst.VALID.Status})
}
log.Info("Finalised eth2 block", "number", oldest.NumberU64(), "hash", oldest.Hash())
waitFinalise = waitFinalise[1:]
}
for {
checkFinalise()
select {
case <-mgr.close:
return
case ev := <-sink:
if transitioned {
continue
}
td := chain.GetTd(ev.Block.Hash(), ev.Block.NumberU64())
if td.Cmp(transitionDifficulty) < 0 {
continue
}
transitioned, parentBlock = true, ev.Block
timer.Reset(blockInterval)
log.Info("Transition difficulty reached", "td", td, "target", transitionDifficulty, "number", ev.Block.NumberU64(), "hash", ev.Block.Hash())
case <-timer.C:
producers := mgr.getNodes(eth2MiningNode)
if len(producers) == 0 {
continue
}
hash, timestamp := parentBlock.Hash(), parentBlock.Time()
if parentBlock.NumberU64() == 0 {
timestamp = uint64(time.Now().Unix()) - uint64(blockIntervalInt)
}
ed, err := producers[0].assembleBlock(hash, timestamp)
if err != nil {
log.Error("Failed to assemble the block", "err", err)
continue
}
block, _ := catalyst.ExecutableDataToBlock(*ed)
nodes := mgr.getNodes(eth2MiningNode)
nodes = append(nodes, mgr.getNodes(eth2NormalNode)...)
nodes = append(nodes, mgr.getNodes(eth2LightClient)...)
for _, node := range nodes {
if err := node.insertBlockAndSetHead(parentBlock.Header(), *ed); err != nil {
log.Error("Failed to insert block", "type", node.typ, "err", err)
}
}
log.Info("Create and insert eth2 block", "number", ed.Number)
parentBlock = block
waitFinalise = append(waitFinalise, block)
timer.Reset(blockInterval)
}
}
}
func main() {
log.Root().SetHandler(log.LvlFilterHandler(log.LvlInfo, log.StreamHandler(os.Stderr, log.TerminalFormat(true))))
fdlimit.Raise(2048)
// Generate a batch of accounts to seal and fund with
faucets := make([]*ecdsa.PrivateKey, 16)
for i := 0; i < len(faucets); i++ {
faucets[i], _ = crypto.GenerateKey()
}
// Pre-generate the ethash mining DAG so we don't race
ethash.MakeDataset(1, filepath.Join(os.Getenv("HOME"), ".ethash"))
// Create an Ethash network based off of the Ropsten config
genesis := makeGenesis(faucets)
manager := newNodeManager(genesis)
defer manager.shutdown()
manager.createNode(eth2NormalNode)
manager.createNode(eth2MiningNode)
manager.createNode(legacyMiningNode)
manager.createNode(legacyNormalNode)
manager.createNode(eth2LightClient)
// Iterate over all the nodes and start mining
time.Sleep(3 * time.Second)
if transitionDifficulty.Sign() != 0 {
manager.startMining()
}
go manager.run()
// Start injecting transactions from the faucets like crazy
time.Sleep(3 * time.Second)
nonces := make([]uint64, len(faucets))
for {
// Pick a random mining node
nodes := manager.getNodes(eth2MiningNode)
index := rand.Intn(len(faucets))
node := nodes[index%len(nodes)]
// Create a self transaction and inject into the pool
tx, err := types.SignTx(types.NewTransaction(nonces[index], crypto.PubkeyToAddress(faucets[index].PublicKey), new(big.Int), 21000, big.NewInt(100000000000+rand.Int63n(65536)), nil), types.HomesteadSigner{}, faucets[index])
if err != nil {
panic(err)
}
if err := node.ethBackend.TxPool().AddLocal(tx); err != nil {
panic(err)
}
nonces[index]++
// Wait if we're too saturated
if pend, _ := node.ethBackend.TxPool().Stats(); pend > 2048 {
time.Sleep(100 * time.Millisecond)
}
}
}
// makeGenesis creates a custom Ethash genesis block based on some pre-defined
// faucet accounts.
func makeGenesis(faucets []*ecdsa.PrivateKey) *core.Genesis {
genesis := core.DefaultRopstenGenesisBlock()
genesis.Difficulty = params.MinimumDifficulty
genesis.GasLimit = 25000000
genesis.Config.ChainID = big.NewInt(18)
genesis.Config.EIP150Hash = common.Hash{}
genesis.BaseFee = big.NewInt(params.InitialBaseFee)
genesis.Config.TerminalTotalDifficulty = transitionDifficulty
genesis.Alloc = core.GenesisAlloc{}
for _, faucet := range faucets {
genesis.Alloc[crypto.PubkeyToAddress(faucet.PublicKey)] = core.GenesisAccount{
Balance: new(big.Int).Exp(big.NewInt(2), big.NewInt(128), nil),
}
}
return genesis
}
func makeFullNode(genesis *core.Genesis) (*node.Node, *eth.Ethereum, *catalyst.ConsensusAPI, error) {
// Define the basic configurations for the Ethereum node
datadir, _ := ioutil.TempDir("", "")
config := &node.Config{
Name: "geth",
Version: params.Version,
DataDir: datadir,
P2P: p2p.Config{
ListenAddr: "0.0.0.0:0",
NoDiscovery: true,
MaxPeers: 25,
},
UseLightweightKDF: true,
}
// Create the node and configure a full Ethereum node on it
stack, err := node.New(config)
if err != nil {
return nil, nil, nil, err
}
econfig := &ethconfig.Config{
Genesis: genesis,
NetworkId: genesis.Config.ChainID.Uint64(),
SyncMode: downloader.FullSync,
DatabaseCache: 256,
DatabaseHandles: 256,
TxPool: core.DefaultTxPoolConfig,
GPO: ethconfig.Defaults.GPO,
Ethash: ethconfig.Defaults.Ethash,
Miner: miner.Config{
GasFloor: genesis.GasLimit * 9 / 10,
GasCeil: genesis.GasLimit * 11 / 10,
GasPrice: big.NewInt(1),
Recommit: 10 * time.Second, // Disable the recommit
},
LightServ: 100,
LightPeers: 10,
LightNoSyncServe: true,
}
ethBackend, err := eth.New(stack, econfig)
if err != nil {
return nil, nil, nil, err
}
_, err = les.NewLesServer(stack, ethBackend, econfig)
if err != nil {
log.Crit("Failed to create the LES server", "err", err)
}
err = stack.Start()
return stack, ethBackend, catalyst.NewConsensusAPI(ethBackend, nil), err
}
func makeLightNode(genesis *core.Genesis) (*node.Node, *les.LightEthereum, *catalyst.ConsensusAPI, error) {
// Define the basic configurations for the Ethereum node
datadir, _ := ioutil.TempDir("", "")
config := &node.Config{
Name: "geth",
Version: params.Version,
DataDir: datadir,
P2P: p2p.Config{
ListenAddr: "0.0.0.0:0",
NoDiscovery: true,
MaxPeers: 25,
},
UseLightweightKDF: true,
}
// Create the node and configure a full Ethereum node on it
stack, err := node.New(config)
if err != nil {
return nil, nil, nil, err
}
lesBackend, err := les.New(stack, &ethconfig.Config{
Genesis: genesis,
NetworkId: genesis.Config.ChainID.Uint64(),
SyncMode: downloader.LightSync,
DatabaseCache: 256,
DatabaseHandles: 256,
TxPool: core.DefaultTxPoolConfig,
GPO: ethconfig.Defaults.GPO,
Ethash: ethconfig.Defaults.Ethash,
LightPeers: 10,
})
if err != nil {
return nil, nil, nil, err
}
err = stack.Start()
return stack, lesBackend, catalyst.NewConsensusAPI(nil, lesBackend), err
}
func eth2types(typ nodetype) bool {
if typ == eth2LightClient || typ == eth2NormalNode || typ == eth2MiningNode {
return true
}
return false
}

View File

@ -128,6 +128,7 @@ type worker struct {
engine consensus.Engine
eth Backend
chain *core.BlockChain
merger *consensus.Merger
// Feeds
pendingLogsFeed event.Feed
@ -181,7 +182,7 @@ type worker struct {
noempty uint32
// External functions
isLocalBlock func(block *types.Block) bool // Function used to determine whether the specified block is mined by local miner.
isLocalBlock func(header *types.Header) bool // Function used to determine whether the specified block is mined by local miner.
// Test hooks
newTaskHook func(*task) // Method to call upon receiving a new sealing task.
@ -190,7 +191,7 @@ type worker struct {
resubmitHook func(time.Duration, time.Duration) // Method to call upon updating resubmitting interval.
}
func newWorker(config *Config, chainConfig *params.ChainConfig, engine consensus.Engine, eth Backend, mux *event.TypeMux, isLocalBlock func(*types.Block) bool, init bool) *worker {
func newWorker(config *Config, chainConfig *params.ChainConfig, engine consensus.Engine, eth Backend, mux *event.TypeMux, isLocalBlock func(header *types.Header) bool, init bool, merger *consensus.Merger) *worker {
worker := &worker{
config: config,
chainConfig: chainConfig,
@ -198,6 +199,7 @@ func newWorker(config *Config, chainConfig *params.ChainConfig, engine consensus
eth: eth,
mux: mux,
chain: eth.BlockChain(),
merger: merger,
isLocalBlock: isLocalBlock,
localUncles: make(map[common.Hash]*types.Block),
remoteUncles: make(map[common.Hash]*types.Block),
@ -472,7 +474,7 @@ func (w *worker) mainLoop() {
continue
}
// Add side block to possible uncle block set depending on the author.
if w.isLocalBlock != nil && w.isLocalBlock(ev.Block) {
if w.isLocalBlock != nil && w.isLocalBlock(ev.Block.Header()) {
w.localUncles[ev.Block.Hash()] = ev.Block
} else {
w.remoteUncles[ev.Block.Hash()] = ev.Block
@ -657,7 +659,7 @@ func (w *worker) resultLoop() {
logs = append(logs, receipt.Logs...)
}
// Commit block and state to database.
_, err := w.chain.WriteBlockWithState(block, receipts, logs, task.state, true)
_, err := w.chain.WriteBlockAndSetHead(block, receipts, logs, task.state, true)
if err != nil {
log.Error("Failed writing block to chain", "err", err)
continue
@ -1037,7 +1039,8 @@ func (w *worker) commit(uncles []*types.Header, interval func(), update bool, st
if err != nil {
return err
}
if w.isRunning() {
if w.isRunning() && !w.merger.TDDReached() {
if interval != nil {
interval()
}

View File

@ -197,7 +197,7 @@ func (b *testWorkerBackend) newRandomTx(creation bool) *types.Transaction {
func newTestWorker(t *testing.T, chainConfig *params.ChainConfig, engine consensus.Engine, db ethdb.Database, blocks int) (*worker, *testWorkerBackend) {
backend := newTestWorkerBackend(t, chainConfig, engine, db, blocks)
backend.txPool.AddLocals(pendingTxs)
w := newWorker(testConfig, chainConfig, engine, backend, new(event.TypeMux), nil, false)
w := newWorker(testConfig, chainConfig, engine, backend, new(event.TypeMux), nil, false, consensus.NewMerger(rawdb.NewMemoryDatabase()))
w.setEtherbase(testBankAddress)
return w, backend
}

View File

@ -54,6 +54,7 @@ var (
_ Error = new(invalidRequestError)
_ Error = new(invalidMessageError)
_ Error = new(invalidParamsError)
_ Error = new(CustomError)
)
const defaultErrorCode = -32000
@ -101,3 +102,12 @@ type invalidParamsError struct{ message string }
func (e *invalidParamsError) ErrorCode() int { return -32602 }
func (e *invalidParamsError) Error() string { return e.message }
type CustomError struct {
Code int
Message string
}
func (e *CustomError) ErrorCode() int { return e.Code }
func (e *CustomError) Error() string { return e.Message }