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
48 changed files with 3244 additions and 803 deletions

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