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

@ -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()
}