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:
committed by
GitHub
parent
519cf98b69
commit
3038e480f5
@ -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()
|
||||
}
|
||||
|
@ -19,7 +19,10 @@ package catalyst
|
||||
import (
|
||||
"math/big"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/common/hexutil"
|
||||
"github.com/ethereum/go-ethereum/consensus/ethash"
|
||||
"github.com/ethereum/go-ethereum/core"
|
||||
"github.com/ethereum/go-ethereum/core/rawdb"
|
||||
@ -38,10 +41,10 @@ var (
|
||||
// testAddr is the Ethereum address of the tester account.
|
||||
testAddr = crypto.PubkeyToAddress(testKey.PublicKey)
|
||||
|
||||
testBalance = big.NewInt(2e15)
|
||||
testBalance = big.NewInt(2e18)
|
||||
)
|
||||
|
||||
func generateTestChain() (*core.Genesis, []*types.Block) {
|
||||
func generatePreMergeChain(n int) (*core.Genesis, []*types.Block) {
|
||||
db := rawdb.NewMemoryDatabase()
|
||||
config := params.AllEthashProtocolChanges
|
||||
genesis := &core.Genesis{
|
||||
@ -51,177 +54,267 @@ func generateTestChain() (*core.Genesis, []*types.Block) {
|
||||
Timestamp: 9000,
|
||||
BaseFee: big.NewInt(params.InitialBaseFee),
|
||||
}
|
||||
testNonce := uint64(0)
|
||||
generate := func(i int, g *core.BlockGen) {
|
||||
g.OffsetTime(5)
|
||||
g.SetExtra([]byte("test"))
|
||||
}
|
||||
gblock := genesis.ToBlock(db)
|
||||
engine := ethash.NewFaker()
|
||||
blocks, _ := core.GenerateChain(config, gblock, engine, db, 10, generate)
|
||||
blocks = append([]*types.Block{gblock}, blocks...)
|
||||
return genesis, blocks
|
||||
}
|
||||
|
||||
// TODO (MariusVanDerWijden) reenable once engine api is updated to the latest spec
|
||||
/*
|
||||
func generateTestChainWithFork(n int, fork int) (*core.Genesis, []*types.Block, []*types.Block) {
|
||||
if fork >= n {
|
||||
fork = n - 1
|
||||
}
|
||||
db := rawdb.NewMemoryDatabase()
|
||||
config := ¶ms.ChainConfig{
|
||||
ChainID: big.NewInt(1337),
|
||||
HomesteadBlock: big.NewInt(0),
|
||||
EIP150Block: big.NewInt(0),
|
||||
EIP155Block: big.NewInt(0),
|
||||
EIP158Block: big.NewInt(0),
|
||||
ByzantiumBlock: big.NewInt(0),
|
||||
ConstantinopleBlock: big.NewInt(0),
|
||||
PetersburgBlock: big.NewInt(0),
|
||||
IstanbulBlock: big.NewInt(0),
|
||||
MuirGlacierBlock: big.NewInt(0),
|
||||
BerlinBlock: big.NewInt(0),
|
||||
LondonBlock: big.NewInt(0),
|
||||
TerminalTotalDifficulty: big.NewInt(0),
|
||||
Ethash: new(params.EthashConfig),
|
||||
}
|
||||
genesis := &core.Genesis{
|
||||
Config: config,
|
||||
Alloc: core.GenesisAlloc{testAddr: {Balance: testBalance}},
|
||||
ExtraData: []byte("test genesis"),
|
||||
Timestamp: 9000,
|
||||
BaseFee: big.NewInt(params.InitialBaseFee),
|
||||
}
|
||||
generate := func(i int, g *core.BlockGen) {
|
||||
g.OffsetTime(5)
|
||||
g.SetExtra([]byte("test"))
|
||||
}
|
||||
generateFork := func(i int, g *core.BlockGen) {
|
||||
g.OffsetTime(5)
|
||||
g.SetExtra([]byte("testF"))
|
||||
tx, _ := types.SignTx(types.NewTransaction(testNonce, common.HexToAddress("0x9a9070028361F7AAbeB3f2F2Dc07F82C4a98A02a"), big.NewInt(1), params.TxGas, big.NewInt(params.InitialBaseFee*2), nil), types.LatestSigner(config), testKey)
|
||||
g.AddTx(tx)
|
||||
testNonce++
|
||||
}
|
||||
gblock := genesis.ToBlock(db)
|
||||
engine := ethash.NewFaker()
|
||||
blocks, _ := core.GenerateChain(config, gblock, engine, db, n, generate)
|
||||
blocks = append([]*types.Block{gblock}, blocks...)
|
||||
forkedBlocks, _ := core.GenerateChain(config, blocks[fork], engine, db, n-fork, generateFork)
|
||||
return genesis, blocks, forkedBlocks
|
||||
totalDifficulty := big.NewInt(0)
|
||||
for _, b := range blocks {
|
||||
totalDifficulty.Add(totalDifficulty, b.Difficulty())
|
||||
}
|
||||
config.TerminalTotalDifficulty = totalDifficulty
|
||||
return genesis, blocks
|
||||
}
|
||||
*/
|
||||
|
||||
func TestEth2AssembleBlock(t *testing.T) {
|
||||
genesis, blocks := generateTestChain()
|
||||
n, ethservice := startEthService(t, genesis, blocks[1:9])
|
||||
genesis, blocks := generatePreMergeChain(10)
|
||||
n, ethservice := startEthService(t, genesis, blocks)
|
||||
defer n.Close()
|
||||
|
||||
api := newConsensusAPI(ethservice)
|
||||
api := NewConsensusAPI(ethservice, nil)
|
||||
signer := types.NewEIP155Signer(ethservice.BlockChain().Config().ChainID)
|
||||
tx, err := types.SignTx(types.NewTransaction(0, blocks[8].Coinbase(), big.NewInt(1000), params.TxGas, big.NewInt(params.InitialBaseFee), nil), signer, testKey)
|
||||
tx, err := types.SignTx(types.NewTransaction(uint64(10), blocks[9].Coinbase(), big.NewInt(1000), params.TxGas, big.NewInt(params.InitialBaseFee), nil), signer, testKey)
|
||||
if err != nil {
|
||||
t.Fatalf("error signing transaction, err=%v", err)
|
||||
}
|
||||
ethservice.TxPool().AddLocal(tx)
|
||||
blockParams := assembleBlockParams{
|
||||
ParentHash: blocks[8].ParentHash(),
|
||||
Timestamp: blocks[8].Time(),
|
||||
blockParams := AssembleBlockParams{
|
||||
ParentHash: blocks[9].Hash(),
|
||||
Timestamp: blocks[9].Time() + 5,
|
||||
}
|
||||
execData, err := api.AssembleBlock(blockParams)
|
||||
|
||||
execData, err := api.assembleBlock(blockParams)
|
||||
if err != nil {
|
||||
t.Fatalf("error producing block, err=%v", err)
|
||||
}
|
||||
|
||||
if len(execData.Transactions) != 1 {
|
||||
t.Fatalf("invalid number of transactions %d != 1", len(execData.Transactions))
|
||||
}
|
||||
}
|
||||
|
||||
func TestEth2AssembleBlockWithAnotherBlocksTxs(t *testing.T) {
|
||||
genesis, blocks := generateTestChain()
|
||||
n, ethservice := startEthService(t, genesis, blocks[1:9])
|
||||
genesis, blocks := generatePreMergeChain(10)
|
||||
n, ethservice := startEthService(t, genesis, blocks[:9])
|
||||
defer n.Close()
|
||||
|
||||
api := newConsensusAPI(ethservice)
|
||||
api := NewConsensusAPI(ethservice, nil)
|
||||
|
||||
// Put the 10th block's tx in the pool and produce a new block
|
||||
api.addBlockTxs(blocks[9])
|
||||
blockParams := assembleBlockParams{
|
||||
ParentHash: blocks[9].ParentHash(),
|
||||
Timestamp: blocks[9].Time(),
|
||||
api.insertTransactions(blocks[9].Transactions())
|
||||
blockParams := AssembleBlockParams{
|
||||
ParentHash: blocks[8].Hash(),
|
||||
Timestamp: blocks[8].Time() + 5,
|
||||
}
|
||||
execData, err := api.AssembleBlock(blockParams)
|
||||
execData, err := api.assembleBlock(blockParams)
|
||||
if err != nil {
|
||||
t.Fatalf("error producing block, err=%v", err)
|
||||
}
|
||||
|
||||
if len(execData.Transactions) != blocks[9].Transactions().Len() {
|
||||
t.Fatalf("invalid number of transactions %d != 1", len(execData.Transactions))
|
||||
}
|
||||
}
|
||||
|
||||
// TODO (MariusVanDerWijden) reenable once engine api is updated to the latest spec
|
||||
/*
|
||||
func TestEth2NewBlock(t *testing.T) {
|
||||
genesis, blocks, forkedBlocks := generateTestChainWithFork(10, 4)
|
||||
n, ethservice := startEthService(t, genesis, blocks[1:5])
|
||||
func TestSetHeadBeforeTotalDifficulty(t *testing.T) {
|
||||
genesis, blocks := generatePreMergeChain(10)
|
||||
n, ethservice := startEthService(t, genesis, blocks)
|
||||
defer n.Close()
|
||||
|
||||
api := newConsensusAPI(ethservice)
|
||||
for i := 5; i < 10; i++ {
|
||||
p := executableData{
|
||||
ParentHash: ethservice.BlockChain().CurrentBlock().Hash(),
|
||||
Miner: blocks[i].Coinbase(),
|
||||
StateRoot: blocks[i].Root(),
|
||||
GasLimit: blocks[i].GasLimit(),
|
||||
GasUsed: blocks[i].GasUsed(),
|
||||
Transactions: encodeTransactions(blocks[i].Transactions()),
|
||||
ReceiptRoot: blocks[i].ReceiptHash(),
|
||||
LogsBloom: blocks[i].Bloom().Bytes(),
|
||||
BlockHash: blocks[i].Hash(),
|
||||
Timestamp: blocks[i].Time(),
|
||||
Number: uint64(i),
|
||||
}
|
||||
success, err := api.NewBlock(p)
|
||||
if err != nil || !success.Valid {
|
||||
t.Fatalf("Failed to insert block: %v", err)
|
||||
}
|
||||
}
|
||||
api := NewConsensusAPI(ethservice, nil)
|
||||
|
||||
exp := ethservice.BlockChain().CurrentBlock().Hash()
|
||||
|
||||
// Introduce the fork point.
|
||||
lastBlockNum := blocks[4].Number()
|
||||
lastBlock := blocks[4]
|
||||
for i := 0; i < 4; i++ {
|
||||
lastBlockNum.Add(lastBlockNum, big.NewInt(1))
|
||||
p := executableData{
|
||||
ParentHash: lastBlock.Hash(),
|
||||
Miner: forkedBlocks[i].Coinbase(),
|
||||
StateRoot: forkedBlocks[i].Root(),
|
||||
Number: lastBlockNum.Uint64(),
|
||||
GasLimit: forkedBlocks[i].GasLimit(),
|
||||
GasUsed: forkedBlocks[i].GasUsed(),
|
||||
Transactions: encodeTransactions(blocks[i].Transactions()),
|
||||
ReceiptRoot: forkedBlocks[i].ReceiptHash(),
|
||||
LogsBloom: forkedBlocks[i].Bloom().Bytes(),
|
||||
BlockHash: forkedBlocks[i].Hash(),
|
||||
Timestamp: forkedBlocks[i].Time(),
|
||||
}
|
||||
success, err := api.NewBlock(p)
|
||||
if err != nil || !success.Valid {
|
||||
t.Fatalf("Failed to insert forked block #%d: %v", i, err)
|
||||
}
|
||||
lastBlock, err = insertBlockParamsToBlock(ethservice.BlockChain().Config(), lastBlock.Header(), p)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
||||
|
||||
if ethservice.BlockChain().CurrentBlock().Hash() != exp {
|
||||
t.Fatalf("Wrong head after inserting fork %x != %x", exp, ethservice.BlockChain().CurrentBlock().Hash())
|
||||
if err := api.ForkchoiceUpdated(ForkChoiceParams{HeadBlockHash: blocks[5].Hash()}); err == nil {
|
||||
t.Errorf("fork choice updated before total terminal difficulty should fail")
|
||||
}
|
||||
}
|
||||
*/
|
||||
|
||||
func TestEth2PrepareAndGetPayload(t *testing.T) {
|
||||
genesis, blocks := generatePreMergeChain(10)
|
||||
n, ethservice := startEthService(t, genesis, blocks[:9])
|
||||
defer n.Close()
|
||||
|
||||
api := NewConsensusAPI(ethservice, nil)
|
||||
|
||||
// Put the 10th block's tx in the pool and produce a new block
|
||||
api.insertTransactions(blocks[9].Transactions())
|
||||
blockParams := AssembleBlockParams{
|
||||
ParentHash: blocks[8].Hash(),
|
||||
Timestamp: blocks[8].Time() + 5,
|
||||
}
|
||||
respID, err := api.PreparePayload(blockParams)
|
||||
if err != nil {
|
||||
t.Fatalf("error preparing payload, err=%v", err)
|
||||
}
|
||||
execData, err := api.GetPayload(hexutil.Uint64(respID.PayloadID))
|
||||
if err != nil {
|
||||
t.Fatalf("error getting payload, err=%v", err)
|
||||
}
|
||||
if len(execData.Transactions) != blocks[9].Transactions().Len() {
|
||||
t.Fatalf("invalid number of transactions %d != 1", len(execData.Transactions))
|
||||
}
|
||||
}
|
||||
|
||||
func checkLogEvents(t *testing.T, logsCh <-chan []*types.Log, rmLogsCh <-chan core.RemovedLogsEvent, wantNew, wantRemoved int) {
|
||||
t.Helper()
|
||||
|
||||
if len(logsCh) != wantNew {
|
||||
t.Fatalf("wrong number of log events: got %d, want %d", len(logsCh), wantNew)
|
||||
}
|
||||
if len(rmLogsCh) != wantRemoved {
|
||||
t.Fatalf("wrong number of removed log events: got %d, want %d", len(rmLogsCh), wantRemoved)
|
||||
}
|
||||
// Drain events.
|
||||
for i := 0; i < len(logsCh); i++ {
|
||||
<-logsCh
|
||||
}
|
||||
for i := 0; i < len(rmLogsCh); i++ {
|
||||
<-rmLogsCh
|
||||
}
|
||||
}
|
||||
|
||||
func TestEth2NewBlock(t *testing.T) {
|
||||
genesis, preMergeBlocks := generatePreMergeChain(10)
|
||||
n, ethservice := startEthService(t, genesis, preMergeBlocks)
|
||||
ethservice.Merger().ReachTTD()
|
||||
defer n.Close()
|
||||
|
||||
var (
|
||||
api = NewConsensusAPI(ethservice, nil)
|
||||
parent = preMergeBlocks[len(preMergeBlocks)-1]
|
||||
|
||||
// This EVM code generates a log when the contract is created.
|
||||
logCode = common.Hex2Bytes("60606040525b7f24ec1d3ff24c2f6ff210738839dbc339cd45a5294d85c79361016243157aae7b60405180905060405180910390a15b600a8060416000396000f360606040526008565b00")
|
||||
)
|
||||
// The event channels.
|
||||
newLogCh := make(chan []*types.Log, 10)
|
||||
rmLogsCh := make(chan core.RemovedLogsEvent, 10)
|
||||
ethservice.BlockChain().SubscribeLogsEvent(newLogCh)
|
||||
ethservice.BlockChain().SubscribeRemovedLogsEvent(rmLogsCh)
|
||||
|
||||
for i := 0; i < 10; i++ {
|
||||
statedb, _ := ethservice.BlockChain().StateAt(parent.Root())
|
||||
nonce := statedb.GetNonce(testAddr)
|
||||
tx, _ := types.SignTx(types.NewContractCreation(nonce, new(big.Int), 1000000, big.NewInt(2*params.InitialBaseFee), logCode), types.LatestSigner(ethservice.BlockChain().Config()), testKey)
|
||||
ethservice.TxPool().AddLocal(tx)
|
||||
|
||||
execData, err := api.assembleBlock(AssembleBlockParams{
|
||||
ParentHash: parent.Hash(),
|
||||
Timestamp: parent.Time() + 5,
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatalf("Failed to create the executable data %v", err)
|
||||
}
|
||||
block, err := ExecutableDataToBlock(*execData)
|
||||
if err != nil {
|
||||
t.Fatalf("Failed to convert executable data to block %v", err)
|
||||
}
|
||||
newResp, err := api.ExecutePayload(*execData)
|
||||
if err != nil || newResp.Status != "VALID" {
|
||||
t.Fatalf("Failed to insert block: %v", err)
|
||||
}
|
||||
if ethservice.BlockChain().CurrentBlock().NumberU64() != block.NumberU64()-1 {
|
||||
t.Fatalf("Chain head shouldn't be updated")
|
||||
}
|
||||
checkLogEvents(t, newLogCh, rmLogsCh, 0, 0)
|
||||
|
||||
if err := api.ForkchoiceUpdated(ForkChoiceParams{HeadBlockHash: block.Hash(), FinalizedBlockHash: block.Hash()}); err != nil {
|
||||
t.Fatalf("Failed to insert block: %v", err)
|
||||
}
|
||||
if ethservice.BlockChain().CurrentBlock().NumberU64() != block.NumberU64() {
|
||||
t.Fatalf("Chain head should be updated")
|
||||
}
|
||||
checkLogEvents(t, newLogCh, rmLogsCh, 1, 0)
|
||||
|
||||
parent = block
|
||||
}
|
||||
|
||||
// Introduce fork chain
|
||||
var (
|
||||
head = ethservice.BlockChain().CurrentBlock().NumberU64()
|
||||
)
|
||||
parent = preMergeBlocks[len(preMergeBlocks)-1]
|
||||
for i := 0; i < 10; i++ {
|
||||
execData, err := api.assembleBlock(AssembleBlockParams{
|
||||
ParentHash: parent.Hash(),
|
||||
Timestamp: parent.Time() + 6,
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatalf("Failed to create the executable data %v", err)
|
||||
}
|
||||
block, err := ExecutableDataToBlock(*execData)
|
||||
if err != nil {
|
||||
t.Fatalf("Failed to convert executable data to block %v", err)
|
||||
}
|
||||
newResp, err := api.ExecutePayload(*execData)
|
||||
if err != nil || newResp.Status != "VALID" {
|
||||
t.Fatalf("Failed to insert block: %v", err)
|
||||
}
|
||||
if ethservice.BlockChain().CurrentBlock().NumberU64() != head {
|
||||
t.Fatalf("Chain head shouldn't be updated")
|
||||
}
|
||||
|
||||
if err := api.ConsensusValidated(ConsensusValidatedParams{BlockHash: block.Hash(), Status: "VALID"}); err != nil {
|
||||
t.Fatalf("Failed to insert block: %v", err)
|
||||
}
|
||||
if err := api.ForkchoiceUpdated(ForkChoiceParams{FinalizedBlockHash: block.Hash(), HeadBlockHash: block.Hash()}); err != nil {
|
||||
t.Fatalf("Failed to insert block: %v", err)
|
||||
}
|
||||
if ethservice.BlockChain().CurrentBlock().NumberU64() != block.NumberU64() {
|
||||
t.Fatalf("Chain head should be updated")
|
||||
}
|
||||
parent, head = block, block.NumberU64()
|
||||
}
|
||||
}
|
||||
|
||||
func TestEth2DeepReorg(t *testing.T) {
|
||||
// TODO (MariusVanDerWijden) TestEth2DeepReorg is currently broken, because it tries to reorg
|
||||
// before the totalTerminalDifficulty threshold
|
||||
/*
|
||||
genesis, preMergeBlocks := generatePreMergeChain(core.TriesInMemory * 2)
|
||||
n, ethservice := startEthService(t, genesis, preMergeBlocks)
|
||||
defer n.Close()
|
||||
|
||||
var (
|
||||
api = NewConsensusAPI(ethservice, nil)
|
||||
parent = preMergeBlocks[len(preMergeBlocks)-core.TriesInMemory-1]
|
||||
head = ethservice.BlockChain().CurrentBlock().NumberU64()
|
||||
)
|
||||
if ethservice.BlockChain().HasBlockAndState(parent.Hash(), parent.NumberU64()) {
|
||||
t.Errorf("Block %d not pruned", parent.NumberU64())
|
||||
}
|
||||
for i := 0; i < 10; i++ {
|
||||
execData, err := api.assembleBlock(AssembleBlockParams{
|
||||
ParentHash: parent.Hash(),
|
||||
Timestamp: parent.Time() + 5,
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatalf("Failed to create the executable data %v", err)
|
||||
}
|
||||
block, err := ExecutableDataToBlock(ethservice.BlockChain().Config(), parent.Header(), *execData)
|
||||
if err != nil {
|
||||
t.Fatalf("Failed to convert executable data to block %v", err)
|
||||
}
|
||||
newResp, err := api.ExecutePayload(*execData)
|
||||
if err != nil || newResp.Status != "VALID" {
|
||||
t.Fatalf("Failed to insert block: %v", err)
|
||||
}
|
||||
if ethservice.BlockChain().CurrentBlock().NumberU64() != head {
|
||||
t.Fatalf("Chain head shouldn't be updated")
|
||||
}
|
||||
if err := api.setHead(block.Hash()); err != nil {
|
||||
t.Fatalf("Failed to set head: %v", err)
|
||||
}
|
||||
if ethservice.BlockChain().CurrentBlock().NumberU64() != block.NumberU64() {
|
||||
t.Fatalf("Chain head should be updated")
|
||||
}
|
||||
parent, head = block, block.NumberU64()
|
||||
}
|
||||
*/
|
||||
}
|
||||
|
||||
// startEthService creates a full node instance for testing.
|
||||
func startEthService(t *testing.T, genesis *core.Genesis, blocks []*types.Block) (*node.Node, *eth.Ethereum) {
|
||||
@ -232,7 +325,7 @@ func startEthService(t *testing.T, genesis *core.Genesis, blocks []*types.Block)
|
||||
t.Fatal("can't create node:", err)
|
||||
}
|
||||
|
||||
ethcfg := ðconfig.Config{Genesis: genesis, Ethash: ethash.Config{PowMode: ethash.ModeFake}}
|
||||
ethcfg := ðconfig.Config{Genesis: genesis, Ethash: ethash.Config{PowMode: ethash.ModeFake}, TrieTimeout: time.Minute, TrieDirtyCache: 256, TrieCleanCache: 256}
|
||||
ethservice, err := eth.New(n, ethcfg)
|
||||
if err != nil {
|
||||
t.Fatal("can't create eth service:", err)
|
||||
@ -245,6 +338,61 @@ func startEthService(t *testing.T, genesis *core.Genesis, blocks []*types.Block)
|
||||
t.Fatal("can't import test blocks:", err)
|
||||
}
|
||||
ethservice.SetEtherbase(testAddr)
|
||||
ethservice.SetSynced()
|
||||
|
||||
return n, ethservice
|
||||
}
|
||||
|
||||
func TestFullAPI(t *testing.T) {
|
||||
genesis, preMergeBlocks := generatePreMergeChain(10)
|
||||
n, ethservice := startEthService(t, genesis, preMergeBlocks)
|
||||
ethservice.Merger().ReachTTD()
|
||||
defer n.Close()
|
||||
var (
|
||||
api = NewConsensusAPI(ethservice, nil)
|
||||
parent = ethservice.BlockChain().CurrentBlock()
|
||||
// This EVM code generates a log when the contract is created.
|
||||
logCode = common.Hex2Bytes("60606040525b7f24ec1d3ff24c2f6ff210738839dbc339cd45a5294d85c79361016243157aae7b60405180905060405180910390a15b600a8060416000396000f360606040526008565b00")
|
||||
)
|
||||
for i := 0; i < 10; i++ {
|
||||
statedb, _ := ethservice.BlockChain().StateAt(parent.Root())
|
||||
nonce := statedb.GetNonce(testAddr)
|
||||
tx, _ := types.SignTx(types.NewContractCreation(nonce, new(big.Int), 1000000, big.NewInt(2*params.InitialBaseFee), logCode), types.LatestSigner(ethservice.BlockChain().Config()), testKey)
|
||||
ethservice.TxPool().AddLocal(tx)
|
||||
|
||||
params := AssembleBlockParams{
|
||||
ParentHash: parent.Hash(),
|
||||
Timestamp: parent.Time() + 1,
|
||||
Random: crypto.Keccak256Hash([]byte{byte(i)}),
|
||||
FeeRecipient: parent.Coinbase(),
|
||||
}
|
||||
resp, err := api.PreparePayload(params)
|
||||
if err != nil {
|
||||
t.Fatalf("can't prepare payload: %v", err)
|
||||
}
|
||||
payload, err := api.GetPayload(hexutil.Uint64(resp.PayloadID))
|
||||
if err != nil {
|
||||
t.Fatalf("can't get payload: %v", err)
|
||||
}
|
||||
execResp, err := api.ExecutePayload(*payload)
|
||||
if err != nil {
|
||||
t.Fatalf("can't execute payload: %v", err)
|
||||
}
|
||||
if execResp.Status != VALID.Status {
|
||||
t.Fatalf("invalid status: %v", execResp.Status)
|
||||
}
|
||||
|
||||
if err := api.ConsensusValidated(ConsensusValidatedParams{BlockHash: payload.BlockHash, Status: VALID.Status}); err != nil {
|
||||
t.Fatalf("failed to validate consensus: %v", err)
|
||||
}
|
||||
|
||||
if err := api.ForkchoiceUpdated(ForkChoiceParams{HeadBlockHash: payload.BlockHash, FinalizedBlockHash: payload.BlockHash}); err != nil {
|
||||
t.Fatalf("Failed to insert block: %v", err)
|
||||
}
|
||||
if ethservice.BlockChain().CurrentBlock().NumberU64() != payload.Number {
|
||||
t.Fatalf("Chain head should be updated")
|
||||
}
|
||||
parent = ethservice.BlockChain().CurrentBlock()
|
||||
|
||||
}
|
||||
}
|
||||
|
@ -17,16 +17,20 @@
|
||||
package catalyst
|
||||
|
||||
import (
|
||||
"math/big"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/common/hexutil"
|
||||
)
|
||||
|
||||
//go:generate go run github.com/fjl/gencodec -type assembleBlockParams -field-override assembleBlockParamsMarshaling -out gen_blockparams.go
|
||||
//go:generate go run github.com/fjl/gencodec -type AssembleBlockParams -field-override assembleBlockParamsMarshaling -out gen_blockparams.go
|
||||
|
||||
// Structure described at https://hackmd.io/T9x2mMA4S7us8tJwEB3FDQ
|
||||
type assembleBlockParams struct {
|
||||
ParentHash common.Hash `json:"parentHash" gencodec:"required"`
|
||||
Timestamp uint64 `json:"timestamp" gencodec:"required"`
|
||||
// Structure described at https://github.com/ethereum/execution-apis/pull/74
|
||||
type AssembleBlockParams struct {
|
||||
ParentHash common.Hash `json:"parentHash" gencodec:"required"`
|
||||
Timestamp uint64 `json:"timestamp" gencodec:"required"`
|
||||
Random common.Hash `json:"random" gencodec:"required"`
|
||||
FeeRecipient common.Address `json:"feeRecipient" gencodec:"required"`
|
||||
}
|
||||
|
||||
// JSON type overrides for assembleBlockParams.
|
||||
@ -34,37 +38,67 @@ type assembleBlockParamsMarshaling struct {
|
||||
Timestamp hexutil.Uint64
|
||||
}
|
||||
|
||||
//go:generate go run github.com/fjl/gencodec -type executableData -field-override executableDataMarshaling -out gen_ed.go
|
||||
//go:generate go run github.com/fjl/gencodec -type ExecutableData -field-override executableDataMarshaling -out gen_ed.go
|
||||
|
||||
// Structure described at https://notes.ethereum.org/@n0ble/rayonism-the-merge-spec#Parameters1
|
||||
type executableData struct {
|
||||
BlockHash common.Hash `json:"blockHash" gencodec:"required"`
|
||||
ParentHash common.Hash `json:"parentHash" gencodec:"required"`
|
||||
Miner common.Address `json:"miner" gencodec:"required"`
|
||||
StateRoot common.Hash `json:"stateRoot" gencodec:"required"`
|
||||
Number uint64 `json:"number" gencodec:"required"`
|
||||
GasLimit uint64 `json:"gasLimit" gencodec:"required"`
|
||||
GasUsed uint64 `json:"gasUsed" gencodec:"required"`
|
||||
Timestamp uint64 `json:"timestamp" gencodec:"required"`
|
||||
ReceiptRoot common.Hash `json:"receiptsRoot" gencodec:"required"`
|
||||
LogsBloom []byte `json:"logsBloom" gencodec:"required"`
|
||||
Transactions [][]byte `json:"transactions" gencodec:"required"`
|
||||
// Structure described at https://github.com/ethereum/execution-apis/pull/74/files
|
||||
type ExecutableData struct {
|
||||
BlockHash common.Hash `json:"blockHash" gencodec:"required"`
|
||||
ParentHash common.Hash `json:"parentHash" gencodec:"required"`
|
||||
Coinbase common.Address `json:"coinbase" gencodec:"required"`
|
||||
StateRoot common.Hash `json:"stateRoot" gencodec:"required"`
|
||||
ReceiptRoot common.Hash `json:"receiptRoot" gencodec:"required"`
|
||||
LogsBloom []byte `json:"logsBloom" gencodec:"required"`
|
||||
Random common.Hash `json:"random" gencodec:"required"`
|
||||
Number uint64 `json:"blockNumber" gencodec:"required"`
|
||||
GasLimit uint64 `json:"gasLimit" gencodec:"required"`
|
||||
GasUsed uint64 `json:"gasUsed" gencodec:"required"`
|
||||
Timestamp uint64 `json:"timestamp" gencodec:"required"`
|
||||
ExtraData []byte `json:"extraData" gencodec:"required"`
|
||||
BaseFeePerGas *big.Int `json:"baseFeePerGas" gencodec:"required"`
|
||||
Transactions [][]byte `json:"transactions" gencodec:"required"`
|
||||
}
|
||||
|
||||
// JSON type overrides for executableData.
|
||||
type executableDataMarshaling struct {
|
||||
Number hexutil.Uint64
|
||||
GasLimit hexutil.Uint64
|
||||
GasUsed hexutil.Uint64
|
||||
Timestamp hexutil.Uint64
|
||||
LogsBloom hexutil.Bytes
|
||||
Transactions []hexutil.Bytes
|
||||
Number hexutil.Uint64
|
||||
GasLimit hexutil.Uint64
|
||||
GasUsed hexutil.Uint64
|
||||
Timestamp hexutil.Uint64
|
||||
BaseFeePerGas *hexutil.Big
|
||||
ExtraData hexutil.Bytes
|
||||
LogsBloom hexutil.Bytes
|
||||
Transactions []hexutil.Bytes
|
||||
}
|
||||
|
||||
type newBlockResponse struct {
|
||||
//go:generate go run github.com/fjl/gencodec -type PayloadResponse -field-override payloadResponseMarshaling -out gen_payload.go
|
||||
|
||||
type PayloadResponse struct {
|
||||
PayloadID uint64 `json:"payloadId"`
|
||||
}
|
||||
|
||||
// JSON type overrides for payloadResponse.
|
||||
type payloadResponseMarshaling struct {
|
||||
PayloadID hexutil.Uint64
|
||||
}
|
||||
|
||||
type NewBlockResponse struct {
|
||||
Valid bool `json:"valid"`
|
||||
}
|
||||
|
||||
type genericResponse struct {
|
||||
type GenericResponse struct {
|
||||
Success bool `json:"success"`
|
||||
}
|
||||
|
||||
type GenericStringResponse struct {
|
||||
Status string `json:"status"`
|
||||
}
|
||||
|
||||
type ConsensusValidatedParams struct {
|
||||
BlockHash common.Hash `json:"blockHash"`
|
||||
Status string `json:"status"`
|
||||
}
|
||||
|
||||
type ForkChoiceParams struct {
|
||||
HeadBlockHash common.Hash `json:"headBlockHash"`
|
||||
FinalizedBlockHash common.Hash `json:"finalizedBlockHash"`
|
||||
}
|
||||
|
@ -13,34 +13,48 @@ import (
|
||||
var _ = (*assembleBlockParamsMarshaling)(nil)
|
||||
|
||||
// MarshalJSON marshals as JSON.
|
||||
func (a assembleBlockParams) MarshalJSON() ([]byte, error) {
|
||||
type assembleBlockParams struct {
|
||||
ParentHash common.Hash `json:"parentHash" gencodec:"required"`
|
||||
Timestamp hexutil.Uint64 `json:"timestamp" gencodec:"required"`
|
||||
func (a AssembleBlockParams) MarshalJSON() ([]byte, error) {
|
||||
type AssembleBlockParams struct {
|
||||
ParentHash common.Hash `json:"parentHash" gencodec:"required"`
|
||||
Timestamp hexutil.Uint64 `json:"timestamp" gencodec:"required"`
|
||||
Random common.Hash `json:"random" gencodec:"required"`
|
||||
FeeRecipient common.Address `json:"feeRecipient" gencodec:"required"`
|
||||
}
|
||||
var enc assembleBlockParams
|
||||
var enc AssembleBlockParams
|
||||
enc.ParentHash = a.ParentHash
|
||||
enc.Timestamp = hexutil.Uint64(a.Timestamp)
|
||||
enc.Random = a.Random
|
||||
enc.FeeRecipient = a.FeeRecipient
|
||||
return json.Marshal(&enc)
|
||||
}
|
||||
|
||||
// UnmarshalJSON unmarshals from JSON.
|
||||
func (a *assembleBlockParams) UnmarshalJSON(input []byte) error {
|
||||
type assembleBlockParams struct {
|
||||
ParentHash *common.Hash `json:"parentHash" gencodec:"required"`
|
||||
Timestamp *hexutil.Uint64 `json:"timestamp" gencodec:"required"`
|
||||
func (a *AssembleBlockParams) UnmarshalJSON(input []byte) error {
|
||||
type AssembleBlockParams struct {
|
||||
ParentHash *common.Hash `json:"parentHash" gencodec:"required"`
|
||||
Timestamp *hexutil.Uint64 `json:"timestamp" gencodec:"required"`
|
||||
Random *common.Hash `json:"random" gencodec:"required"`
|
||||
FeeRecipient *common.Address `json:"feeRecipient" gencodec:"required"`
|
||||
}
|
||||
var dec assembleBlockParams
|
||||
var dec AssembleBlockParams
|
||||
if err := json.Unmarshal(input, &dec); err != nil {
|
||||
return err
|
||||
}
|
||||
if dec.ParentHash == nil {
|
||||
return errors.New("missing required field 'parentHash' for assembleBlockParams")
|
||||
return errors.New("missing required field 'parentHash' for AssembleBlockParams")
|
||||
}
|
||||
a.ParentHash = *dec.ParentHash
|
||||
if dec.Timestamp == nil {
|
||||
return errors.New("missing required field 'timestamp' for assembleBlockParams")
|
||||
return errors.New("missing required field 'timestamp' for AssembleBlockParams")
|
||||
}
|
||||
a.Timestamp = uint64(*dec.Timestamp)
|
||||
if dec.Random == nil {
|
||||
return errors.New("missing required field 'random' for AssembleBlockParams")
|
||||
}
|
||||
a.Random = *dec.Random
|
||||
if dec.FeeRecipient == nil {
|
||||
return errors.New("missing required field 'feeRecipient' for AssembleBlockParams")
|
||||
}
|
||||
a.FeeRecipient = *dec.FeeRecipient
|
||||
return nil
|
||||
}
|
||||
|
@ -5,6 +5,7 @@ package catalyst
|
||||
import (
|
||||
"encoding/json"
|
||||
"errors"
|
||||
"math/big"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/common/hexutil"
|
||||
@ -13,31 +14,37 @@ import (
|
||||
var _ = (*executableDataMarshaling)(nil)
|
||||
|
||||
// MarshalJSON marshals as JSON.
|
||||
func (e executableData) MarshalJSON() ([]byte, error) {
|
||||
type executableData struct {
|
||||
BlockHash common.Hash `json:"blockHash" gencodec:"required"`
|
||||
ParentHash common.Hash `json:"parentHash" gencodec:"required"`
|
||||
Miner common.Address `json:"miner" gencodec:"required"`
|
||||
StateRoot common.Hash `json:"stateRoot" gencodec:"required"`
|
||||
Number hexutil.Uint64 `json:"number" gencodec:"required"`
|
||||
GasLimit hexutil.Uint64 `json:"gasLimit" gencodec:"required"`
|
||||
GasUsed hexutil.Uint64 `json:"gasUsed" gencodec:"required"`
|
||||
Timestamp hexutil.Uint64 `json:"timestamp" gencodec:"required"`
|
||||
ReceiptRoot common.Hash `json:"receiptsRoot" gencodec:"required"`
|
||||
LogsBloom hexutil.Bytes `json:"logsBloom" gencodec:"required"`
|
||||
Transactions []hexutil.Bytes `json:"transactions" gencodec:"required"`
|
||||
func (e ExecutableData) MarshalJSON() ([]byte, error) {
|
||||
type ExecutableData struct {
|
||||
BlockHash common.Hash `json:"blockHash" gencodec:"required"`
|
||||
ParentHash common.Hash `json:"parentHash" gencodec:"required"`
|
||||
Coinbase common.Address `json:"coinbase" gencodec:"required"`
|
||||
StateRoot common.Hash `json:"stateRoot" gencodec:"required"`
|
||||
ReceiptRoot common.Hash `json:"receiptRoot" gencodec:"required"`
|
||||
LogsBloom hexutil.Bytes `json:"logsBloom" gencodec:"required"`
|
||||
Random common.Hash `json:"random" gencodec:"required"`
|
||||
Number hexutil.Uint64 `json:"blockNumber" gencodec:"required"`
|
||||
GasLimit hexutil.Uint64 `json:"gasLimit" gencodec:"required"`
|
||||
GasUsed hexutil.Uint64 `json:"gasUsed" gencodec:"required"`
|
||||
Timestamp hexutil.Uint64 `json:"timestamp" gencodec:"required"`
|
||||
ExtraData hexutil.Bytes `json:"extraData" gencodec:"required"`
|
||||
BaseFeePerGas *hexutil.Big `json:"baseFeePerGas" gencodec:"required"`
|
||||
Transactions []hexutil.Bytes `json:"transactions" gencodec:"required"`
|
||||
}
|
||||
var enc executableData
|
||||
var enc ExecutableData
|
||||
enc.BlockHash = e.BlockHash
|
||||
enc.ParentHash = e.ParentHash
|
||||
enc.Miner = e.Miner
|
||||
enc.Coinbase = e.Coinbase
|
||||
enc.StateRoot = e.StateRoot
|
||||
enc.ReceiptRoot = e.ReceiptRoot
|
||||
enc.LogsBloom = e.LogsBloom
|
||||
enc.Random = e.Random
|
||||
enc.Number = hexutil.Uint64(e.Number)
|
||||
enc.GasLimit = hexutil.Uint64(e.GasLimit)
|
||||
enc.GasUsed = hexutil.Uint64(e.GasUsed)
|
||||
enc.Timestamp = hexutil.Uint64(e.Timestamp)
|
||||
enc.ReceiptRoot = e.ReceiptRoot
|
||||
enc.LogsBloom = e.LogsBloom
|
||||
enc.ExtraData = e.ExtraData
|
||||
enc.BaseFeePerGas = (*hexutil.Big)(e.BaseFeePerGas)
|
||||
if e.Transactions != nil {
|
||||
enc.Transactions = make([]hexutil.Bytes, len(e.Transactions))
|
||||
for k, v := range e.Transactions {
|
||||
@ -48,66 +55,81 @@ func (e executableData) MarshalJSON() ([]byte, error) {
|
||||
}
|
||||
|
||||
// UnmarshalJSON unmarshals from JSON.
|
||||
func (e *executableData) UnmarshalJSON(input []byte) error {
|
||||
type executableData struct {
|
||||
BlockHash *common.Hash `json:"blockHash" gencodec:"required"`
|
||||
ParentHash *common.Hash `json:"parentHash" gencodec:"required"`
|
||||
Miner *common.Address `json:"miner" gencodec:"required"`
|
||||
StateRoot *common.Hash `json:"stateRoot" gencodec:"required"`
|
||||
Number *hexutil.Uint64 `json:"number" gencodec:"required"`
|
||||
GasLimit *hexutil.Uint64 `json:"gasLimit" gencodec:"required"`
|
||||
GasUsed *hexutil.Uint64 `json:"gasUsed" gencodec:"required"`
|
||||
Timestamp *hexutil.Uint64 `json:"timestamp" gencodec:"required"`
|
||||
ReceiptRoot *common.Hash `json:"receiptsRoot" gencodec:"required"`
|
||||
LogsBloom *hexutil.Bytes `json:"logsBloom" gencodec:"required"`
|
||||
Transactions []hexutil.Bytes `json:"transactions" gencodec:"required"`
|
||||
func (e *ExecutableData) UnmarshalJSON(input []byte) error {
|
||||
type ExecutableData struct {
|
||||
BlockHash *common.Hash `json:"blockHash" gencodec:"required"`
|
||||
ParentHash *common.Hash `json:"parentHash" gencodec:"required"`
|
||||
Coinbase *common.Address `json:"coinbase" gencodec:"required"`
|
||||
StateRoot *common.Hash `json:"stateRoot" gencodec:"required"`
|
||||
ReceiptRoot *common.Hash `json:"receiptRoot" gencodec:"required"`
|
||||
LogsBloom *hexutil.Bytes `json:"logsBloom" gencodec:"required"`
|
||||
Random *common.Hash `json:"random" gencodec:"required"`
|
||||
Number *hexutil.Uint64 `json:"blockNumber" gencodec:"required"`
|
||||
GasLimit *hexutil.Uint64 `json:"gasLimit" gencodec:"required"`
|
||||
GasUsed *hexutil.Uint64 `json:"gasUsed" gencodec:"required"`
|
||||
Timestamp *hexutil.Uint64 `json:"timestamp" gencodec:"required"`
|
||||
ExtraData *hexutil.Bytes `json:"extraData" gencodec:"required"`
|
||||
BaseFeePerGas *hexutil.Big `json:"baseFeePerGas" gencodec:"required"`
|
||||
Transactions []hexutil.Bytes `json:"transactions" gencodec:"required"`
|
||||
}
|
||||
var dec executableData
|
||||
var dec ExecutableData
|
||||
if err := json.Unmarshal(input, &dec); err != nil {
|
||||
return err
|
||||
}
|
||||
if dec.BlockHash == nil {
|
||||
return errors.New("missing required field 'blockHash' for executableData")
|
||||
return errors.New("missing required field 'blockHash' for ExecutableData")
|
||||
}
|
||||
e.BlockHash = *dec.BlockHash
|
||||
if dec.ParentHash == nil {
|
||||
return errors.New("missing required field 'parentHash' for executableData")
|
||||
return errors.New("missing required field 'parentHash' for ExecutableData")
|
||||
}
|
||||
e.ParentHash = *dec.ParentHash
|
||||
if dec.Miner == nil {
|
||||
return errors.New("missing required field 'miner' for executableData")
|
||||
if dec.Coinbase == nil {
|
||||
return errors.New("missing required field 'coinbase' for ExecutableData")
|
||||
}
|
||||
e.Miner = *dec.Miner
|
||||
e.Coinbase = *dec.Coinbase
|
||||
if dec.StateRoot == nil {
|
||||
return errors.New("missing required field 'stateRoot' for executableData")
|
||||
return errors.New("missing required field 'stateRoot' for ExecutableData")
|
||||
}
|
||||
e.StateRoot = *dec.StateRoot
|
||||
if dec.Number == nil {
|
||||
return errors.New("missing required field 'number' for executableData")
|
||||
}
|
||||
e.Number = uint64(*dec.Number)
|
||||
if dec.GasLimit == nil {
|
||||
return errors.New("missing required field 'gasLimit' for executableData")
|
||||
}
|
||||
e.GasLimit = uint64(*dec.GasLimit)
|
||||
if dec.GasUsed == nil {
|
||||
return errors.New("missing required field 'gasUsed' for executableData")
|
||||
}
|
||||
e.GasUsed = uint64(*dec.GasUsed)
|
||||
if dec.Timestamp == nil {
|
||||
return errors.New("missing required field 'timestamp' for executableData")
|
||||
}
|
||||
e.Timestamp = uint64(*dec.Timestamp)
|
||||
if dec.ReceiptRoot == nil {
|
||||
return errors.New("missing required field 'receiptsRoot' for executableData")
|
||||
return errors.New("missing required field 'receiptRoot' for ExecutableData")
|
||||
}
|
||||
e.ReceiptRoot = *dec.ReceiptRoot
|
||||
if dec.LogsBloom == nil {
|
||||
return errors.New("missing required field 'logsBloom' for executableData")
|
||||
return errors.New("missing required field 'logsBloom' for ExecutableData")
|
||||
}
|
||||
e.LogsBloom = *dec.LogsBloom
|
||||
if dec.Random == nil {
|
||||
return errors.New("missing required field 'random' for ExecutableData")
|
||||
}
|
||||
e.Random = *dec.Random
|
||||
if dec.Number == nil {
|
||||
return errors.New("missing required field 'blockNumber' for ExecutableData")
|
||||
}
|
||||
e.Number = uint64(*dec.Number)
|
||||
if dec.GasLimit == nil {
|
||||
return errors.New("missing required field 'gasLimit' for ExecutableData")
|
||||
}
|
||||
e.GasLimit = uint64(*dec.GasLimit)
|
||||
if dec.GasUsed == nil {
|
||||
return errors.New("missing required field 'gasUsed' for ExecutableData")
|
||||
}
|
||||
e.GasUsed = uint64(*dec.GasUsed)
|
||||
if dec.Timestamp == nil {
|
||||
return errors.New("missing required field 'timestamp' for ExecutableData")
|
||||
}
|
||||
e.Timestamp = uint64(*dec.Timestamp)
|
||||
if dec.ExtraData == nil {
|
||||
return errors.New("missing required field 'extraData' for ExecutableData")
|
||||
}
|
||||
e.ExtraData = *dec.ExtraData
|
||||
if dec.BaseFeePerGas == nil {
|
||||
return errors.New("missing required field 'baseFeePerGas' for ExecutableData")
|
||||
}
|
||||
e.BaseFeePerGas = (*big.Int)(dec.BaseFeePerGas)
|
||||
if dec.Transactions == nil {
|
||||
return errors.New("missing required field 'transactions' for executableData")
|
||||
return errors.New("missing required field 'transactions' for ExecutableData")
|
||||
}
|
||||
e.Transactions = make([][]byte, len(dec.Transactions))
|
||||
for k, v := range dec.Transactions {
|
||||
|
36
eth/catalyst/gen_payload.go
Normal file
36
eth/catalyst/gen_payload.go
Normal file
@ -0,0 +1,36 @@
|
||||
// Code generated by github.com/fjl/gencodec. DO NOT EDIT.
|
||||
|
||||
package catalyst
|
||||
|
||||
import (
|
||||
"encoding/json"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common/hexutil"
|
||||
)
|
||||
|
||||
var _ = (*payloadResponseMarshaling)(nil)
|
||||
|
||||
// MarshalJSON marshals as JSON.
|
||||
func (p PayloadResponse) MarshalJSON() ([]byte, error) {
|
||||
type PayloadResponse struct {
|
||||
PayloadID hexutil.Uint64 `json:"payloadId"`
|
||||
}
|
||||
var enc PayloadResponse
|
||||
enc.PayloadID = hexutil.Uint64(p.PayloadID)
|
||||
return json.Marshal(&enc)
|
||||
}
|
||||
|
||||
// UnmarshalJSON unmarshals from JSON.
|
||||
func (p *PayloadResponse) UnmarshalJSON(input []byte) error {
|
||||
type PayloadResponse struct {
|
||||
PayloadID *hexutil.Uint64 `json:"payloadId"`
|
||||
}
|
||||
var dec PayloadResponse
|
||||
if err := json.Unmarshal(input, &dec); err != nil {
|
||||
return err
|
||||
}
|
||||
if dec.PayloadID != nil {
|
||||
p.PayloadID = uint64(*dec.PayloadID)
|
||||
}
|
||||
return nil
|
||||
}
|
Reference in New Issue
Block a user