beacon/blsync: support for deneb fork (#29180)

This adds support for the Deneb beacon chain fork, and fork handling
in general, to the beacon chain light client implementation.

Co-authored-by: Zsolt Felfoldi <zsfelfoldi@gmail.com>
pull/29304/head
Felix Lange 6 months ago committed by GitHub
parent 04bf1c802f
commit bca6c40709
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
  1. 89
      beacon/blsync/block_sync.go
  2. 78
      beacon/blsync/block_sync_test.go
  3. 36
      beacon/blsync/client.go
  4. 9
      beacon/blsync/config.go
  5. 147
      beacon/blsync/engineclient.go
  6. 49
      beacon/light/api/light_api.go
  7. 110
      beacon/types/beacon_block.go
  8. 77
      beacon/types/beacon_block_test.go
  9. 12
      beacon/types/config.go
  10. 80
      beacon/types/exec_header.go
  11. 144
      beacon/types/exec_payload.go
  12. 11
      beacon/types/header.go
  13. 11
      beacon/types/light_sync.go
  14. 1703
      beacon/types/testdata/block_capella.json
  15. 2644
      beacon/types/testdata/block_deneb.json
  16. 69
      cmd/blsync/engine_api.go
  17. 9
      cmd/blsync/main.go
  18. 13
      cmd/geth/config.go
  19. 88
      eth/catalyst/blsync.go
  20. 4
      go.mod
  21. 13
      go.sum

@ -17,35 +17,25 @@
package blsync
import (
"fmt"
"math/big"
"github.com/ethereum/go-ethereum/beacon/engine"
"github.com/ethereum/go-ethereum/beacon/light/request"
"github.com/ethereum/go-ethereum/beacon/light/sync"
"github.com/ethereum/go-ethereum/beacon/types"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/common/lru"
ctypes "github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/event"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/trie"
"github.com/holiman/uint256"
"github.com/protolambda/zrnt/eth2/beacon/capella"
"github.com/protolambda/zrnt/eth2/configs"
"github.com/protolambda/ztyp/tree"
)
// beaconBlockSync implements request.Module; it fetches the beacon blocks belonging
// to the validated and prefetch heads.
type beaconBlockSync struct {
recentBlocks *lru.Cache[common.Hash, *capella.BeaconBlock]
recentBlocks *lru.Cache[common.Hash, *types.BeaconBlock]
locked map[common.Hash]request.ServerAndID
serverHeads map[request.Server]common.Hash
headTracker headTracker
lastHeadInfo types.HeadInfo
chainHeadFeed *event.Feed
chainHeadFeed event.FeedOf[types.ChainHeadEvent]
}
type headTracker interface {
@ -55,16 +45,19 @@ type headTracker interface {
}
// newBeaconBlockSync returns a new beaconBlockSync.
func newBeaconBlockSync(headTracker headTracker, chainHeadFeed *event.Feed) *beaconBlockSync {
func newBeaconBlockSync(headTracker headTracker) *beaconBlockSync {
return &beaconBlockSync{
headTracker: headTracker,
chainHeadFeed: chainHeadFeed,
recentBlocks: lru.NewCache[common.Hash, *capella.BeaconBlock](10),
recentBlocks: lru.NewCache[common.Hash, *types.BeaconBlock](10),
locked: make(map[common.Hash]request.ServerAndID),
serverHeads: make(map[request.Server]common.Hash),
}
}
func (s *beaconBlockSync) SubscribeChainHead(ch chan<- types.ChainHeadEvent) event.Subscription {
return s.chainHeadFeed.Subscribe(ch)
}
// Process implements request.Module.
func (s *beaconBlockSync) Process(requester request.Requester, events []request.Event) {
for _, event := range events {
@ -73,7 +66,7 @@ func (s *beaconBlockSync) Process(requester request.Requester, events []request.
sid, req, resp := event.RequestInfo()
blockRoot := common.Hash(req.(sync.ReqBeaconBlock))
if resp != nil {
s.recentBlocks.Add(blockRoot, resp.(*capella.BeaconBlock))
s.recentBlocks.Add(blockRoot, resp.(*types.BeaconBlock))
}
if s.locked[blockRoot] == sid {
delete(s.locked, blockRoot)
@ -112,63 +105,11 @@ func (s *beaconBlockSync) tryRequestBlock(requester request.Requester, blockRoot
}
}
func blockHeadInfo(block *capella.BeaconBlock) types.HeadInfo {
func blockHeadInfo(block *types.BeaconBlock) types.HeadInfo {
if block == nil {
return types.HeadInfo{}
}
return types.HeadInfo{Slot: uint64(block.Slot), BlockRoot: beaconBlockHash(block)}
}
// beaconBlockHash calculates the hash of a beacon block.
func beaconBlockHash(beaconBlock *capella.BeaconBlock) common.Hash {
return common.Hash(beaconBlock.HashTreeRoot(configs.Mainnet, tree.GetHashFn()))
}
// getExecBlock extracts the execution block from the beacon block's payload.
func getExecBlock(beaconBlock *capella.BeaconBlock) (*ctypes.Block, error) {
payload := &beaconBlock.Body.ExecutionPayload
txs := make([]*ctypes.Transaction, len(payload.Transactions))
for i, opaqueTx := range payload.Transactions {
var tx ctypes.Transaction
if err := tx.UnmarshalBinary(opaqueTx); err != nil {
return nil, fmt.Errorf("failed to parse tx %d: %v", i, err)
}
txs[i] = &tx
}
withdrawals := make([]*ctypes.Withdrawal, len(payload.Withdrawals))
for i, w := range payload.Withdrawals {
withdrawals[i] = &ctypes.Withdrawal{
Index: uint64(w.Index),
Validator: uint64(w.ValidatorIndex),
Address: common.Address(w.Address),
Amount: uint64(w.Amount),
}
}
wroot := ctypes.DeriveSha(ctypes.Withdrawals(withdrawals), trie.NewStackTrie(nil))
execHeader := &ctypes.Header{
ParentHash: common.Hash(payload.ParentHash),
UncleHash: ctypes.EmptyUncleHash,
Coinbase: common.Address(payload.FeeRecipient),
Root: common.Hash(payload.StateRoot),
TxHash: ctypes.DeriveSha(ctypes.Transactions(txs), trie.NewStackTrie(nil)),
ReceiptHash: common.Hash(payload.ReceiptsRoot),
Bloom: ctypes.Bloom(payload.LogsBloom),
Difficulty: common.Big0,
Number: new(big.Int).SetUint64(uint64(payload.BlockNumber)),
GasLimit: uint64(payload.GasLimit),
GasUsed: uint64(payload.GasUsed),
Time: uint64(payload.Timestamp),
Extra: []byte(payload.ExtraData),
MixDigest: common.Hash(payload.PrevRandao), // reused in merge
Nonce: ctypes.BlockNonce{}, // zero
BaseFee: (*uint256.Int)(&payload.BaseFeePerGas).ToBig(),
WithdrawalsHash: &wroot,
}
execBlock := ctypes.NewBlockWithHeader(execHeader).WithBody(txs, nil).WithWithdrawals(withdrawals)
if execBlockHash := execBlock.Hash(); execBlockHash != common.Hash(payload.BlockHash) {
return execBlock, fmt.Errorf("Sanity check failed, payload hash does not match (expected %x, got %x)", common.Hash(payload.BlockHash), execBlockHash)
}
return execBlock, nil
return types.HeadInfo{Slot: block.Slot(), BlockRoot: block.Root()}
}
func (s *beaconBlockSync) updateEventFeed() {
@ -190,14 +131,16 @@ func (s *beaconBlockSync) updateEventFeed() {
return
}
s.lastHeadInfo = headInfo
// new head block and finality info available; extract executable data and send event to feed
execBlock, err := getExecBlock(headBlock)
execBlock, err := headBlock.ExecutionPayload()
if err != nil {
log.Error("Error extracting execution block from validated beacon block", "error", err)
return
}
s.chainHeadFeed.Send(types.ChainHeadEvent{
HeadBlock: engine.BlockToExecutableData(execBlock, nil, nil).ExecutionPayload,
Finalized: common.Hash(finality.Finalized.PayloadHeader.BlockHash),
BeaconHead: head.Header,
Block: execBlock,
Finalized: finality.Finalized.PayloadHeader.BlockHash(),
})
}

@ -23,70 +23,69 @@ import (
"github.com/ethereum/go-ethereum/beacon/light/sync"
"github.com/ethereum/go-ethereum/beacon/types"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/event"
"github.com/protolambda/zrnt/eth2/beacon/capella"
"github.com/protolambda/zrnt/eth2/configs"
"github.com/protolambda/ztyp/tree"
zrntcommon "github.com/protolambda/zrnt/eth2/beacon/common"
"github.com/protolambda/zrnt/eth2/beacon/deneb"
)
var (
testServer1 = "testServer1"
testServer2 = "testServer2"
testBlock1 = &capella.BeaconBlock{
testBlock1 = types.NewBeaconBlock(&deneb.BeaconBlock{
Slot: 123,
Body: capella.BeaconBlockBody{
ExecutionPayload: capella.ExecutionPayload{BlockNumber: 456},
Body: deneb.BeaconBlockBody{
ExecutionPayload: deneb.ExecutionPayload{
BlockNumber: 456,
BlockHash: zrntcommon.Hash32(common.HexToHash("905ac721c4058d9ed40b27b6b9c1bdd10d4333e4f3d9769100bf9dfb80e5d1f6")),
},
}
testBlock2 = &capella.BeaconBlock{
},
})
testBlock2 = types.NewBeaconBlock(&deneb.BeaconBlock{
Slot: 124,
Body: capella.BeaconBlockBody{
ExecutionPayload: capella.ExecutionPayload{BlockNumber: 457},
Body: deneb.BeaconBlockBody{
ExecutionPayload: deneb.ExecutionPayload{
BlockNumber: 457,
BlockHash: zrntcommon.Hash32(common.HexToHash("011703f39c664efc1c6cf5f49ca09b595581eec572d4dfddd3d6179a9e63e655")),
},
}
},
})
)
func init() {
eb1, _ := getExecBlock(testBlock1)
testBlock1.Body.ExecutionPayload.BlockHash = tree.Root(eb1.Hash())
eb2, _ := getExecBlock(testBlock2)
testBlock2.Body.ExecutionPayload.BlockHash = tree.Root(eb2.Hash())
}
func TestBlockSync(t *testing.T) {
ht := &testHeadTracker{}
eventFeed := new(event.Feed)
blockSync := newBeaconBlockSync(ht, eventFeed)
blockSync := newBeaconBlockSync(ht)
headCh := make(chan types.ChainHeadEvent, 16)
eventFeed.Subscribe(headCh)
blockSync.SubscribeChainHead(headCh)
ts := sync.NewTestScheduler(t, blockSync)
ts.AddServer(testServer1, 1)
ts.AddServer(testServer2, 1)
expHeadBlock := func(tci int, expHead *capella.BeaconBlock) {
expHeadBlock := func(expHead *types.BeaconBlock) {
t.Helper()
var expNumber, headNumber uint64
if expHead != nil {
expNumber = uint64(expHead.Body.ExecutionPayload.BlockNumber)
p, _ := expHead.ExecutionPayload()
expNumber = p.NumberU64()
}
select {
case event := <-headCh:
headNumber = event.HeadBlock.Number
headNumber = event.Block.NumberU64()
default:
}
if headNumber != expNumber {
t.Errorf("Wrong head block in test case #%d (expected block number %d, got %d)", tci, expNumber, headNumber)
t.Errorf("Wrong head block, expected block number %d, got %d)", expNumber, headNumber)
}
}
// no block requests expected until head tracker knows about a head
ts.Run(1)
expHeadBlock(1, nil)
expHeadBlock(nil)
// set block 1 as prefetch head, announced by server 2
head1 := blockHeadInfo(testBlock1)
ht.prefetch = head1
ts.ServerEvent(sync.EvNewHead, testServer2, head1)
// expect request to server 2 which has announced the head
ts.Run(2, testServer2, sync.ReqBeaconBlock(head1.BlockRoot))
@ -95,12 +94,12 @@ func TestBlockSync(t *testing.T) {
ts.AddAllowance(testServer2, 1)
ts.Run(3)
// head block still not expected as the fetched block is not the validated head yet
expHeadBlock(3, nil)
expHeadBlock(nil)
// set as validated head, expect no further requests but block 1 set as head block
ht.validated.Header = blockHeader(testBlock1)
ht.validated.Header = testBlock1.Header()
ts.Run(4)
expHeadBlock(4, testBlock1)
expHeadBlock(testBlock1)
// set block 2 as prefetch head, announced by server 1
head2 := blockHeadInfo(testBlock2)
@ -114,26 +113,16 @@ func TestBlockSync(t *testing.T) {
ts.Run(6)
// set as validated head before retrieving block; now it's assumed to be available from server 2 too
ht.validated.Header = blockHeader(testBlock2)
ht.validated.Header = testBlock2.Header()
// expect req2 retry to server 2
ts.Run(7, testServer2, sync.ReqBeaconBlock(head2.BlockRoot))
// now head block should be unavailable again
expHeadBlock(4, nil)
expHeadBlock(nil)
// valid response, now head block should be block 2 immediately as it is already validated
ts.RequestEvent(request.EvResponse, ts.Request(7, 1), testBlock2)
ts.Run(8)
expHeadBlock(5, testBlock2)
}
func blockHeader(block *capella.BeaconBlock) types.Header {
return types.Header{
Slot: uint64(block.Slot),
ProposerIndex: uint64(block.ProposerIndex),
ParentRoot: common.Hash(block.ParentRoot),
StateRoot: common.Hash(block.StateRoot),
BodyRoot: common.Hash(block.Body.HashTreeRoot(configs.Mainnet, tree.GetHashFn())),
}
expHeadBlock(testBlock2)
}
type testHeadTracker struct {
@ -151,9 +140,10 @@ func (h *testHeadTracker) ValidatedHead() (types.SignedHeader, bool) {
// TODO add test case for finality
func (h *testHeadTracker) ValidatedFinality() (types.FinalityUpdate, bool) {
finalized := types.NewExecutionHeader(new(deneb.ExecutionPayloadHeader))
return types.FinalityUpdate{
Attested: types.HeaderWithExecProof{Header: h.validated.Header},
Finalized: types.HeaderWithExecProof{PayloadHeader: &capella.ExecutionPayloadHeader{}},
Finalized: types.HeaderWithExecProof{PayloadHeader: finalized},
Signature: h.validated.Signature,
SignatureSlot: h.validated.SignatureSlot,
}, h.validated.Header != (types.Header{})

@ -28,14 +28,20 @@ import (
"github.com/ethereum/go-ethereum/common/mclock"
"github.com/ethereum/go-ethereum/ethdb/memorydb"
"github.com/ethereum/go-ethereum/event"
"github.com/ethereum/go-ethereum/rpc"
"github.com/urfave/cli/v2"
)
type Client struct {
scheduler *request.Scheduler
chainHeadFeed *event.Feed
urls []string
customHeader map[string]string
chainConfig *lightClientConfig
scheduler *request.Scheduler
blockSync *beaconBlockSync
engineRPC *rpc.Client
chainHeadSub event.Subscription
engineClient *engineClient
}
func NewClient(ctx *cli.Context) *Client {
@ -53,6 +59,7 @@ func NewClient(ctx *cli.Context) *Client {
}
customHeader[strings.TrimSpace(kv[0])] = strings.TrimSpace(kv[1])
}
// create data structures
var (
db = memorydb.New()
@ -63,11 +70,10 @@ func NewClient(ctx *cli.Context) *Client {
headSync := sync.NewHeadSync(headTracker, committeeChain)
// set up scheduler and sync modules
chainHeadFeed := new(event.Feed)
scheduler := request.NewScheduler()
checkpointInit := sync.NewCheckpointInit(committeeChain, chainConfig.Checkpoint)
forwardSync := sync.NewForwardUpdateSync(committeeChain)
beaconBlockSync := newBeaconBlockSync(headTracker, chainHeadFeed)
beaconBlockSync := newBeaconBlockSync(headTracker)
scheduler.RegisterTarget(headTracker)
scheduler.RegisterTarget(committeeChain)
scheduler.RegisterModule(checkpointInit, "checkpointInit")
@ -79,25 +85,31 @@ func NewClient(ctx *cli.Context) *Client {
scheduler: scheduler,
urls: ctx.StringSlice(utils.BeaconApiFlag.Name),
customHeader: customHeader,
chainHeadFeed: chainHeadFeed,
chainConfig: &chainConfig,
blockSync: beaconBlockSync,
}
}
// SubscribeChainHeadEvent allows callers to subscribe a provided channel to new
// head updates.
func (c *Client) SubscribeChainHeadEvent(ch chan<- types.ChainHeadEvent) event.Subscription {
return c.chainHeadFeed.Subscribe(ch)
func (c *Client) SetEngineRPC(engine *rpc.Client) {
c.engineRPC = engine
}
func (c *Client) Start() {
func (c *Client) Start() error {
headCh := make(chan types.ChainHeadEvent, 16)
c.chainHeadSub = c.blockSync.SubscribeChainHead(headCh)
c.engineClient = startEngineClient(c.chainConfig, c.engineRPC, headCh)
c.scheduler.Start()
// register server(s)
for _, url := range c.urls {
beaconApi := api.NewBeaconLightApi(url, c.customHeader)
c.scheduler.RegisterServer(request.NewServer(api.NewApiServer(beaconApi), &mclock.System{}))
}
return nil
}
func (c *Client) Stop() {
func (c *Client) Stop() error {
c.engineClient.stop()
c.chainHeadSub.Unsubscribe()
c.scheduler.Stop()
return nil
}

@ -39,7 +39,8 @@ var (
AddFork("GENESIS", 0, []byte{0, 0, 0, 0}).
AddFork("ALTAIR", 74240, []byte{1, 0, 0, 0}).
AddFork("BELLATRIX", 144896, []byte{2, 0, 0, 0}).
AddFork("CAPELLA", 194048, []byte{3, 0, 0, 0}),
AddFork("CAPELLA", 194048, []byte{3, 0, 0, 0}).
AddFork("DENEB", 269568, []byte{4, 0, 0, 0}),
Checkpoint: common.HexToHash("0x388be41594ec7d6a6894f18c73f3469f07e2c19a803de4755d335817ed8e2e5a"),
}
@ -51,7 +52,8 @@ var (
AddFork("GENESIS", 0, []byte{144, 0, 0, 105}).
AddFork("ALTAIR", 50, []byte{144, 0, 0, 112}).
AddFork("BELLATRIX", 100, []byte{144, 0, 0, 113}).
AddFork("CAPELLA", 56832, []byte{144, 0, 0, 114}),
AddFork("CAPELLA", 56832, []byte{144, 0, 0, 114}).
AddFork("DENEB", 132608, []byte{144, 0, 0, 115}),
Checkpoint: common.HexToHash("0x1005a6d9175e96bfbce4d35b80f468e9bff0b674e1e861d16e09e10005a58e81"),
}
@ -63,7 +65,8 @@ var (
AddFork("GENESIS", 0, []byte{0, 0, 16, 32}).
AddFork("ALTAIR", 36660, []byte{1, 0, 16, 32}).
AddFork("BELLATRIX", 112260, []byte{2, 0, 16, 32}).
AddFork("CAPELLA", 162304, []byte{3, 0, 16, 32}),
AddFork("CAPELLA", 162304, []byte{3, 0, 16, 32}).
AddFork("DENEB", 231680, []byte{4, 0, 16, 32}),
Checkpoint: common.HexToHash("0x53a0f4f0a378e2c4ae0a9ee97407eb69d0d737d8d8cd0a5fb1093f42f7b81c49"),
}
)

@ -0,0 +1,147 @@
// Copyright 2024 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 blsync
import (
"context"
"strings"
"sync"
"time"
"github.com/ethereum/go-ethereum/beacon/engine"
"github.com/ethereum/go-ethereum/beacon/types"
"github.com/ethereum/go-ethereum/common"
ctypes "github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/rpc"
)
type engineClient struct {
config *lightClientConfig
rpc *rpc.Client
rootCtx context.Context
cancelRoot context.CancelFunc
wg sync.WaitGroup
}
func startEngineClient(config *lightClientConfig, rpc *rpc.Client, headCh <-chan types.ChainHeadEvent) *engineClient {
ctx, cancel := context.WithCancel(context.Background())
ec := &engineClient{
config: config,
rpc: rpc,
rootCtx: ctx,
cancelRoot: cancel,
}
ec.wg.Add(1)
go ec.updateLoop(headCh)
return ec
}
func (ec *engineClient) stop() {
ec.cancelRoot()
ec.wg.Wait()
}
func (ec *engineClient) updateLoop(headCh <-chan types.ChainHeadEvent) {
defer ec.wg.Done()
for {
select {
case <-ec.rootCtx.Done():
return
case event := <-headCh:
if ec.rpc == nil { // dry run, no engine API specified
log.Info("New execution block retrieved", "number", event.Block.NumberU64(), "hash", event.Block.Hash(), "finalized", event.Finalized)
continue
}
fork := ec.config.ForkAtEpoch(event.BeaconHead.Epoch())
forkName := strings.ToLower(fork.Name)
if status, err := ec.callNewPayload(forkName, event); err == nil {
log.Info("Successful NewPayload", "number", event.Block.NumberU64(), "hash", event.Block.Hash(), "status", status)
} else {
log.Error("Failed NewPayload", "number", event.Block.NumberU64(), "hash", event.Block.Hash(), "error", err)
}
if status, err := ec.callForkchoiceUpdated(forkName, event); err == nil {
log.Info("Successful ForkchoiceUpdated", "head", event.Block.Hash(), "status", status)
} else {
log.Error("Failed ForkchoiceUpdated", "head", event.Block.Hash(), "error", err)
}
}
}
}
func (ec *engineClient) callNewPayload(fork string, event types.ChainHeadEvent) (string, error) {
execData := engine.BlockToExecutableData(event.Block, nil, nil).ExecutionPayload
var (
method string
params = []any{execData}
)
switch fork {
case "deneb":
method = "engine_newPayloadV3"
parentBeaconRoot := event.BeaconHead.ParentRoot
blobHashes := collectBlobHashes(event.Block)
params = append(params, blobHashes, parentBeaconRoot)
case "capella":
method = "engine_newPayloadV2"
default:
method = "engine_newPayloadV1"
}
ctx, cancel := context.WithTimeout(ec.rootCtx, time.Second*5)
defer cancel()
var resp engine.PayloadStatusV1
err := ec.rpc.CallContext(ctx, &resp, method, params...)
return resp.Status, err
}
func collectBlobHashes(b *ctypes.Block) []common.Hash {
list := make([]common.Hash, 0)
for _, tx := range b.Transactions() {
list = append(list, tx.BlobHashes()...)
}
return list
}
func (ec *engineClient) callForkchoiceUpdated(fork string, event types.ChainHeadEvent) (string, error) {
update := engine.ForkchoiceStateV1{
HeadBlockHash: event.Block.Hash(),
SafeBlockHash: event.Finalized,
FinalizedBlockHash: event.Finalized,
}
var method string
switch fork {
case "deneb":
method = "engine_forkchoiceUpdatedV3"
case "capella":
method = "engine_forkchoiceUpdatedV2"
default:
method = "engine_forkchoiceUpdatedV1"
}
ctx, cancel := context.WithTimeout(ec.rootCtx, time.Second*5)
defer cancel()
var resp engine.ForkChoiceResponse
err := ec.rpc.CallContext(ctx, &resp, method, update, nil)
return resp.PayloadStatus.Status, err
}

@ -30,9 +30,6 @@ import (
"github.com/ethereum/go-ethereum/beacon/types"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/common/hexutil"
"github.com/protolambda/zrnt/eth2/beacon/capella"
"github.com/protolambda/zrnt/eth2/configs"
"github.com/protolambda/ztyp/tree"
)
var (
@ -69,7 +66,7 @@ type jsonBeaconHeader struct {
type jsonHeaderWithExecProof struct {
Beacon types.Header `json:"beacon"`
Execution *capella.ExecutionPayloadHeader `json:"execution"`
Execution json.RawMessage `json:"execution"`
ExecutionBranch merkle.Values `json:"execution_branch"`
}
@ -244,33 +241,44 @@ func (api *BeaconLightApi) GetFinalityUpdate() (types.FinalityUpdate, error) {
func decodeFinalityUpdate(enc []byte) (types.FinalityUpdate, error) {
var data struct {
Version string
Data struct {
Attested jsonHeaderWithExecProof `json:"attested_header"`
Finalized jsonHeaderWithExecProof `json:"finalized_header"`
FinalityBranch merkle.Values `json:"finality_branch"`
Aggregate types.SyncAggregate `json:"sync_aggregate"`
SignatureSlot common.Decimal `json:"signature_slot"`
} `json:"data"`
}
}
if err := json.Unmarshal(enc, &data); err != nil {
return types.FinalityUpdate{}, err
}
// Decode the execution payload headers.
attestedExecHeader, err := types.ExecutionHeaderFromJSON(data.Version, data.Data.Attested.Execution)
if err != nil {
return types.FinalityUpdate{}, fmt.Errorf("invalid attested header: %v", err)
}
finalizedExecHeader, err := types.ExecutionHeaderFromJSON(data.Version, data.Data.Finalized.Execution)
if err != nil {
return types.FinalityUpdate{}, fmt.Errorf("invalid finalized header: %v", err)
}
// Perform sanity checks.
if len(data.Data.Aggregate.Signers) != params.SyncCommitteeBitmaskSize {
return types.FinalityUpdate{}, errors.New("invalid sync_committee_bits length")
}
if len(data.Data.Aggregate.Signature) != params.BLSSignatureSize {
return types.FinalityUpdate{}, errors.New("invalid sync_committee_signature length")
}
return types.FinalityUpdate{
Attested: types.HeaderWithExecProof{
Header: data.Data.Attested.Beacon,
PayloadHeader: data.Data.Attested.Execution,
PayloadHeader: attestedExecHeader,
PayloadBranch: data.Data.Attested.ExecutionBranch,
},
Finalized: types.HeaderWithExecProof{
Header: data.Data.Finalized.Beacon,
PayloadHeader: data.Data.Finalized.Execution,
PayloadHeader: finalizedExecHeader,
PayloadBranch: data.Data.Finalized.ExecutionBranch,
},
FinalityBranch: data.Data.FinalityBranch,
@ -359,27 +367,30 @@ func (api *BeaconLightApi) GetCheckpointData(checkpointHash common.Hash) (*types
return checkpoint, nil
}
func (api *BeaconLightApi) GetBeaconBlock(blockRoot common.Hash) (*capella.BeaconBlock, error) {
func (api *BeaconLightApi) GetBeaconBlock(blockRoot common.Hash) (*types.BeaconBlock, error) {
resp, err := api.httpGetf("/eth/v2/beacon/blocks/0x%x", blockRoot)
if err != nil {
return nil, err
}
var beaconBlockMessage struct {
Version string
Data struct {
Message capella.BeaconBlock `json:"message"`
} `json:"data"`
Message json.RawMessage `json:"message"`
}
}
if err := json.Unmarshal(resp, &beaconBlockMessage); err != nil {
return nil, fmt.Errorf("invalid block json data: %v", err)
}
beaconBlock := new(capella.BeaconBlock)
*beaconBlock = beaconBlockMessage.Data.Message
root := common.Hash(beaconBlock.HashTreeRoot(configs.Mainnet, tree.GetHashFn()))
if root != blockRoot {
return nil, fmt.Errorf("Beacon block root hash mismatch (expected: %x, got: %x)", blockRoot, root)
block, err := types.BlockFromJSON(beaconBlockMessage.Version, beaconBlockMessage.Data.Message)
if err != nil {
return nil, err
}
computedRoot := block.Root()
if computedRoot != blockRoot {
return nil, fmt.Errorf("Beacon block root hash mismatch (expected: %x, got: %x)", blockRoot, computedRoot)
}
return beaconBlock, nil
return block, nil
}
func decodeHeadEvent(enc []byte) (uint64, common.Hash, error) {
@ -456,7 +467,7 @@ func (api *BeaconLightApi) StartHeadListener(listener HeadEventListener) func()
select {
case event, ok := <-stream.Events:
if !ok {
break
return
}
switch event.Event() {
case "head":
@ -482,7 +493,7 @@ func (api *BeaconLightApi) StartHeadListener(listener HeadEventListener) func()
}
case err, ok := <-stream.Errors:
if !ok {
break
return
}
listener.OnError(err)
}

@ -0,0 +1,110 @@
// Copyright 2024 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 types
import (
"encoding/json"
"fmt"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/types"
"github.com/protolambda/zrnt/eth2/beacon/capella"
zrntcommon "github.com/protolambda/zrnt/eth2/beacon/common"
"github.com/protolambda/zrnt/eth2/beacon/deneb"
"github.com/protolambda/zrnt/eth2/configs"
"github.com/protolambda/ztyp/tree"
)
type blockObject interface {
HashTreeRoot(spec *zrntcommon.Spec, hFn tree.HashFn) zrntcommon.Root
Header(spec *zrntcommon.Spec) *zrntcommon.BeaconBlockHeader
}
// BeaconBlock represents a full block in the beacon chain.
type BeaconBlock struct {
blockObj blockObject
}
// BlockFromJSON decodes a beacon block from JSON.
func BlockFromJSON(forkName string, data []byte) (*BeaconBlock, error) {
var obj blockObject
switch forkName {
case "deneb":
obj = new(deneb.BeaconBlock)
case "capella":
obj = new(capella.BeaconBlock)
default:
return nil, fmt.Errorf("unsupported fork: " + forkName)
}
if err := json.Unmarshal(data, obj); err != nil {
return nil, err
}
return &BeaconBlock{obj}, nil
}
// NewBeaconBlock wraps a ZRNT block.
func NewBeaconBlock(obj blockObject) *BeaconBlock {
switch obj := obj.(type) {
case *capella.BeaconBlock:
return &BeaconBlock{obj}
case *deneb.BeaconBlock:
return &BeaconBlock{obj}
default:
panic(fmt.Errorf("unsupported block type %T", obj))
}
}
// Slot returns the slot number of the block.
func (b *BeaconBlock) Slot() uint64 {
switch obj := b.blockObj.(type) {
case *capella.BeaconBlock:
return uint64(obj.Slot)
case *deneb.BeaconBlock:
return uint64(obj.Slot)
default:
panic(fmt.Errorf("unsupported block type %T", b.blockObj))
}
}
// ExecutionPayload parses and returns the execution payload of the block.
func (b *BeaconBlock) ExecutionPayload() (*types.Block, error) {
switch obj := b.blockObj.(type) {
case *capella.BeaconBlock:
return convertPayload(&obj.Body.ExecutionPayload, &obj.ParentRoot)
case *deneb.BeaconBlock:
return convertPayload(&obj.Body.ExecutionPayload, &obj.ParentRoot)
default:
panic(fmt.Errorf("unsupported block type %T", b.blockObj))
}
}
// Header returns the block's header data.
func (b *BeaconBlock) Header() Header {
switch obj := b.blockObj.(type) {
case *capella.BeaconBlock:
return headerFromZRNT(obj.Header(configs.Mainnet))
case *deneb.BeaconBlock:
return headerFromZRNT(obj.Header(configs.Mainnet))
default:
panic(fmt.Errorf("unsupported block type %T", b.blockObj))
}
}
// Root computes the SSZ root hash of the block.
func (b *BeaconBlock) Root() common.Hash {
return common.Hash(b.blockObj.HashTreeRoot(configs.Mainnet, tree.GetHashFn()))
}

@ -0,0 +1,77 @@
// Copyright 2024 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 types
import (
"os"
"path/filepath"
"testing"
"github.com/ethereum/go-ethereum/common"
)
func TestBlockFromJSON(t *testing.T) {
type blocktest struct {
file string
version string
wantSlot uint64
wantBlockNumber uint64
wantBlockHash common.Hash
}
tests := []blocktest{
{
file: "block_deneb.json",
version: "deneb",
wantSlot: 8631513,
wantBlockNumber: 19431837,
wantBlockHash: common.HexToHash("0x4cf7d9108fc01b50023ab7cab9b372a96068fddcadec551630393b65acb1f34c"),
},
{
file: "block_capella.json",
version: "capella",
wantSlot: 7378495,
wantBlockNumber: 18189758,
wantBlockHash: common.HexToHash("0x802acf5c350f4252e31d83c431fcb259470250fa0edf49e8391cfee014239820"),
},
}
for _, test := range tests {
t.Run(test.file, func(t *testing.T) {
data, err := os.ReadFile(filepath.Join("testdata", test.file))
if err != nil {
t.Fatal(err)
}
beaconBlock, err := BlockFromJSON(test.version, data)
if err != nil {
t.Fatal(err)
}
if beaconBlock.Slot() != test.wantSlot {
t.Errorf("wrong slot number %d", beaconBlock.Slot())
}
execBlock, err := beaconBlock.ExecutionPayload()
if err != nil {
t.Fatalf("payload extraction failed: %v", err)
}
if execBlock.NumberU64() != test.wantBlockNumber {
t.Errorf("wrong block number: %v", execBlock.NumberU64())
}
if execBlock.Hash() != test.wantBlockHash {
t.Errorf("wrong block hash: %v", execBlock.Hash())
}
})
}
}

@ -37,7 +37,7 @@ const syncCommitteeDomain = 7
// Fork describes a single beacon chain fork and also stores the calculated
// signature domain used after this fork.
type Fork struct {
// Name of the fork in the chain config (config.yaml) file{
// Name of the fork in the chain config (config.yaml) file
Name string
// Epoch when given fork version is activated
@ -110,6 +110,16 @@ type ChainConfig struct {
Forks Forks
}
// ForkAtEpoch returns the latest active fork at the given epoch.
func (c *ChainConfig) ForkAtEpoch(epoch uint64) Fork {
for i := len(c.Forks) - 1; i >= 0; i-- {
if c.Forks[i].Epoch <= epoch {
return *c.Forks[i]
}
}
return Fork{}
}
// AddFork adds a new item to the list of forks.
func (c *ChainConfig) AddFork(name string, epoch uint64, version []byte) *ChainConfig {
fork := &Fork{

@ -0,0 +1,80 @@
// Copyright 2024 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 types
import (
"encoding/json"
"fmt"
"github.com/ethereum/go-ethereum/beacon/merkle"
"github.com/ethereum/go-ethereum/common"
"github.com/protolambda/zrnt/eth2/beacon/capella"
zrntcommon "github.com/protolambda/zrnt/eth2/beacon/common"
"github.com/protolambda/zrnt/eth2/beacon/deneb"
"github.com/protolambda/ztyp/tree"
)
type headerObject interface {
HashTreeRoot(hFn tree.HashFn) zrntcommon.Root
}
type ExecutionHeader struct {
obj headerObject
}
// HeaderFromJSON decodes an execution header from JSON data provided by
// the beacon chain API.
func ExecutionHeaderFromJSON(forkName string, data []byte) (*ExecutionHeader, error) {
var obj headerObject
switch forkName {
case "capella":
obj = new(capella.ExecutionPayloadHeader)
case "deneb":
obj = new(deneb.ExecutionPayloadHeader)
default:
return nil, fmt.Errorf("unsupported fork: " + forkName)
}
if err := json.Unmarshal(data, obj); err != nil {
return nil, err
}
return &ExecutionHeader{obj: obj}, nil
}
func NewExecutionHeader(obj headerObject) *ExecutionHeader {
switch obj.(type) {
case *capella.ExecutionPayloadHeader:
case *deneb.ExecutionPayloadHeader:
default:
panic(fmt.Errorf("unsupported ExecutionPayloadHeader type %T", obj))
}
return &ExecutionHeader{obj: obj}
}
func (eh *ExecutionHeader) PayloadRoot() merkle.Value {
return merkle.Value(eh.obj.HashTreeRoot(tree.GetHashFn()))
}
func (eh *ExecutionHeader) BlockHash() common.Hash {
switch obj := eh.obj.(type) {
case *capella.ExecutionPayloadHeader:
return common.Hash(obj.BlockHash)
case *deneb.ExecutionPayloadHeader:
return common.Hash(obj.BlockHash)
default:
panic(fmt.Errorf("unsupported ExecutionPayloadHeader type %T", obj))
}
}

@ -0,0 +1,144 @@
// Copyright 2024 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 types
import (
"fmt"
"math/big"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/trie"
"github.com/holiman/uint256"
"github.com/protolambda/zrnt/eth2/beacon/capella"
zrntcommon "github.com/protolambda/zrnt/eth2/beacon/common"
"github.com/protolambda/zrnt/eth2/beacon/deneb"
)
type payloadType interface {
*capella.ExecutionPayload | *deneb.ExecutionPayload
}
// convertPayload converts a beacon chain execution payload to types.Block.
func convertPayload[T payloadType](payload T, parentRoot *zrntcommon.Root) (*types.Block, error) {
var (
header types.Header
transactions []*types.Transaction
withdrawals []*types.Withdrawal
expectedHash [32]byte
err error
)
switch p := any(payload).(type) {
case *capella.ExecutionPayload:
convertCapellaHeader(p, &header)
transactions, err = convertTransactions(p.Transactions, &header)
if err != nil {
return nil, err
}
withdrawals = convertWithdrawals(p.Withdrawals, &header)
expectedHash = p.BlockHash
case *deneb.ExecutionPayload:
convertDenebHeader(p, common.Hash(*parentRoot), &header)
transactions, err = convertTransactions(p.Transactions, &header)
if err != nil {
return nil, err
}
withdrawals = convertWithdrawals(p.Withdrawals, &header)
expectedHash = p.BlockHash
default:
panic("unsupported block type")
}
block := types.NewBlockWithHeader(&header)
block = block.WithBody(transactions, nil)
block = block.WithWithdrawals(withdrawals)
hash := block.Hash()
if hash != expectedHash {
return block, fmt.Errorf("Sanity check failed, payload hash does not match (expected %x, got %x)", expectedHash, hash)
}
return block, nil
}
func convertCapellaHeader(payload *capella.ExecutionPayload, h *types.Header) {
// note: h.TxHash is set in convertTransactions
h.ParentHash = common.Hash(payload.ParentHash)
h.UncleHash = types.EmptyUncleHash
h.Coinbase = common.Address(payload.FeeRecipient)
h.Root = common.Hash(payload.StateRoot)
h.ReceiptHash = common.Hash(payload.ReceiptsRoot)
h.Bloom = types.Bloom(payload.LogsBloom)
h.Difficulty = common.Big0
h.Number = new(big.Int).SetUint64(uint64(payload.BlockNumber))
h.GasLimit = uint64(payload.GasLimit)
h.GasUsed = uint64(payload.GasUsed)
h.Time = uint64(payload.Timestamp)
h.Extra = []byte(payload.ExtraData)
h.MixDigest = common.Hash(payload.PrevRandao)
h.Nonce = types.BlockNonce{}
h.BaseFee = (*uint256.Int)(&payload.BaseFeePerGas).ToBig()
}
func convertDenebHeader(payload *deneb.ExecutionPayload, parentRoot common.Hash, h *types.Header) {
// note: h.TxHash is set in convertTransactions
h.ParentHash = common.Hash(payload.ParentHash)
h.UncleHash = types.EmptyUncleHash
h.Coinbase = common.Address(payload.FeeRecipient)
h.Root = common.Hash(payload.StateRoot)
h.ReceiptHash = common.Hash(payload.ReceiptsRoot)
h.Bloom = types.Bloom(payload.LogsBloom)
h.Difficulty = common.Big0
h.Number = new(big.Int).SetUint64(uint64(payload.BlockNumber))
h.GasLimit = uint64(payload.GasLimit)
h.GasUsed = uint64(payload.GasUsed)
h.Time = uint64(payload.Timestamp)
h.Extra = []byte(payload.ExtraData)
h.MixDigest = common.Hash(payload.PrevRandao)
h.Nonce = types.BlockNonce{}
h.BaseFee = (*uint256.Int)(&payload.BaseFeePerGas).ToBig()
// new in deneb
h.BlobGasUsed = (*uint64)(&payload.BlobGasUsed)
h.ExcessBlobGas = (*uint64)(&payload.ExcessBlobGas)
h.ParentBeaconRoot = &parentRoot
}
func convertTransactions(list zrntcommon.PayloadTransactions, execHeader *types.Header) ([]*types.Transaction, error) {
txs := make([]*types.Transaction, len(list))
for i, opaqueTx := range list {
var tx types.Transaction
if err := tx.UnmarshalBinary(opaqueTx); err != nil {
return nil, fmt.Errorf("failed to parse tx %d: %v", i, err)
}
txs[i] = &tx
}
execHeader.TxHash = types.DeriveSha(types.Transactions(txs), trie.NewStackTrie(nil))
return txs, nil
}
func convertWithdrawals(list zrntcommon.Withdrawals, execHeader *types.Header) []*types.Withdrawal {
withdrawals := make([]*types.Withdrawal, len(list))
for i, w := range list {
withdrawals[i] = &types.Withdrawal{
Index: uint64(w.Index),
Validator: uint64(w.ValidatorIndex),
Address: common.Address(w.Address),
Amount: uint64(w.Amount),
}
}
wroot := types.DeriveSha(types.Withdrawals(withdrawals), trie.NewStackTrie(nil))
execHeader.WithdrawalsHash = &wroot
return withdrawals
}

@ -24,6 +24,7 @@ import (
"github.com/ethereum/go-ethereum/beacon/merkle"
"github.com/ethereum/go-ethereum/beacon/params"
"github.com/ethereum/go-ethereum/common"
zrntcommon "github.com/protolambda/zrnt/eth2/beacon/common"
)
//go:generate go run github.com/fjl/gencodec -type Header -field-override headerMarshaling -out gen_header_json.go
@ -57,6 +58,16 @@ type Header struct {
BodyRoot common.Hash `gencodec:"required" json:"body_root"`
}
func headerFromZRNT(zh *zrntcommon.BeaconBlockHeader) Header {
return Header{
Slot: uint64(zh.Slot),
ProposerIndex: uint64(zh.ProposerIndex),
ParentRoot: common.Hash(zh.ParentRoot),
StateRoot: common.Hash(zh.StateRoot),
BodyRoot: common.Hash(zh.BodyRoot),
}
}
// headerMarshaling is a field type overrides for gencodec.
type headerMarshaling struct {
Slot common.Decimal

@ -20,12 +20,10 @@ import (
"errors"
"fmt"
"github.com/ethereum/go-ethereum/beacon/engine"
"github.com/ethereum/go-ethereum/beacon/merkle"
"github.com/ethereum/go-ethereum/beacon/params"
"github.com/ethereum/go-ethereum/common"
"github.com/protolambda/zrnt/eth2/beacon/capella"
"github.com/protolambda/ztyp/tree"
"github.com/ethereum/go-ethereum/core/types"
)
// HeadInfo represents an unvalidated new head announcement.
@ -146,12 +144,12 @@ func (u UpdateScore) BetterThan(w UpdateScore) bool {
type HeaderWithExecProof struct {
Header
PayloadHeader *capella.ExecutionPayloadHeader
PayloadHeader *ExecutionHeader
PayloadBranch merkle.Values
}
func (h *HeaderWithExecProof) Validate() error {
payloadRoot := merkle.Value(h.PayloadHeader.HashTreeRoot(tree.GetHashFn()))
payloadRoot := h.PayloadHeader.PayloadRoot()
return merkle.VerifyProof(h.BodyRoot, params.BodyIndexExecPayload, h.PayloadBranch, payloadRoot)
}
@ -187,6 +185,7 @@ func (u *FinalityUpdate) Validate() error {
// latest accepted head of the beacon chain, along with the hash of the latest
// finalized execution block.
type ChainHeadEvent struct {
HeadBlock *engine.ExecutableData
BeaconHead Header
Block *types.Block
Finalized common.Hash
}

File diff suppressed because it is too large Load Diff

File diff suppressed because one or more lines are too long

@ -1,69 +0,0 @@
// Copyright 2024 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 main
import (
"context"
"time"
"github.com/ethereum/go-ethereum/beacon/engine"
"github.com/ethereum/go-ethereum/beacon/types"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/rpc"
)
func updateEngineApi(client *rpc.Client, headCh chan types.ChainHeadEvent) {
for event := range headCh {
if client == nil { // dry run, no engine API specified
log.Info("New execution block retrieved", "block number", event.HeadBlock.Number, "block hash", event.HeadBlock.BlockHash, "finalized block hash", event.Finalized)
} else {
if status, err := callNewPayloadV2(client, event.HeadBlock); err == nil {
log.Info("Successful NewPayload", "block number", event.HeadBlock.Number, "block hash", event.HeadBlock.BlockHash, "status", status)
} else {
log.Error("Failed NewPayload", "block number", event.HeadBlock.Number, "block hash", event.HeadBlock.BlockHash, "error", err)
}
if status, err := callForkchoiceUpdatedV1(client, event.HeadBlock.BlockHash, event.Finalized); err == nil {
log.Info("Successful ForkchoiceUpdated", "head", event.HeadBlock.BlockHash, "status", status)
} else {
log.Error("Failed ForkchoiceUpdated", "head", event.HeadBlock.BlockHash, "error", err)
}
}
}
}
func callNewPayloadV2(client *rpc.Client, execData *engine.ExecutableData) (string, error) {
var resp engine.PayloadStatusV1
ctx, cancel := context.WithTimeout(context.Background(), time.Second*5)
err := client.CallContext(ctx, &resp, "engine_newPayloadV2", execData)
cancel()
return resp.Status, err
}
func callForkchoiceUpdatedV1(client *rpc.Client, headHash, finalizedHash common.Hash) (string, error) {
var resp engine.ForkChoiceResponse
update := engine.ForkchoiceStateV1{
HeadBlockHash: headHash,
SafeBlockHash: finalizedHash,
FinalizedBlockHash: finalizedHash,
}
ctx, cancel := context.WithTimeout(context.Background(), time.Second*5)
err := client.CallContext(ctx, &resp, "engine_forkchoiceUpdatedV1", update, nil)
cancel()
return resp.PayloadStatus.Status, err
}

@ -23,7 +23,6 @@ import (
"os"
"github.com/ethereum/go-ethereum/beacon/blsync"
"github.com/ethereum/go-ethereum/beacon/types"
"github.com/ethereum/go-ethereum/cmd/utils"
"github.com/ethereum/go-ethereum/internal/flags"
"github.com/ethereum/go-ethereum/log"
@ -87,16 +86,14 @@ func sync(ctx *cli.Context) error {
verbosity := log.FromLegacyLevel(ctx.Int(verbosityFlag.Name))
log.SetDefault(log.NewLogger(log.NewTerminalHandlerWithLevel(output, verbosity, usecolor)))
headCh := make(chan types.ChainHeadEvent, 16)
// set up blsync
client := blsync.NewClient(ctx)
sub := client.SubscribeChainHeadEvent(headCh)
go updateEngineApi(makeRPCClient(ctx), headCh)
client.SetEngineRPC(makeRPCClient(ctx))
client.Start()
// run until stopped
<-ctx.Done()
client.Stop()
sub.Unsubscribe()
close(headCh)
return nil
}

@ -44,6 +44,7 @@ import (
"github.com/ethereum/go-ethereum/metrics"
"github.com/ethereum/go-ethereum/node"
"github.com/ethereum/go-ethereum/params"
"github.com/ethereum/go-ethereum/rpc"
"github.com/naoina/toml"
"github.com/urfave/cli/v2"
)
@ -213,9 +214,9 @@ func makeFullNode(ctx *cli.Context) (*node.Node, ethapi.Backend) {
}
utils.RegisterFullSyncTester(stack, eth, common.BytesToHash(hex))
}
// Start the dev mode if requested, or launch the engine API for
// interacting with external consensus client.
if ctx.IsSet(utils.DeveloperFlag.Name) {
// Start dev mode.
simBeacon, err := catalyst.NewSimulatedBeacon(ctx.Uint64(utils.DeveloperPeriodFlag.Name), eth)
if err != nil {
utils.Fatalf("failed to register dev mode catalyst service: %v", err)
@ -223,8 +224,14 @@ func makeFullNode(ctx *cli.Context) (*node.Node, ethapi.Backend) {
catalyst.RegisterSimulatedBeaconAPIs(stack, simBeacon)
stack.RegisterLifecycle(simBeacon)
} else if ctx.IsSet(utils.BeaconApiFlag.Name) {
stack.RegisterLifecycle(catalyst.NewBlsync(blsync.NewClient(ctx), eth))
// Start blsync mode.
srv := rpc.NewServer()
srv.RegisterName("engine", catalyst.NewConsensusAPI(eth))
blsyncer := blsync.NewClient(ctx)
blsyncer.SetEngineRPC(rpc.DialInProc(srv))
stack.RegisterLifecycle(blsyncer)
} else {
// Launch the engine API for interacting with external consensus client.
err := catalyst.Register(stack, eth)
if err != nil {
utils.Fatalf("failed to register catalyst service: %v", err)

@ -1,88 +0,0 @@
// Copyright 2024 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 catalyst
import (
"github.com/ethereum/go-ethereum/beacon/engine"
"github.com/ethereum/go-ethereum/beacon/types"
"github.com/ethereum/go-ethereum/eth"
"github.com/ethereum/go-ethereum/event"
"github.com/ethereum/go-ethereum/log"
)
// Blsync tracks the head of the beacon chain through the beacon light client
// and drives the local node via ConsensusAPI.
type Blsync struct {
engine *ConsensusAPI
client Client
headCh chan types.ChainHeadEvent
headSub event.Subscription
quitCh chan struct{}
}
type Client interface {
SubscribeChainHeadEvent(ch chan<- types.ChainHeadEvent) event.Subscription
Start()
Stop()
}
// NewBlsync creates a new beacon light syncer.
func NewBlsync(client Client, eth *eth.Ethereum) *Blsync {
return &Blsync{
engine: newConsensusAPIWithoutHeartbeat(eth),
client: client,
headCh: make(chan types.ChainHeadEvent, 16),
quitCh: make(chan struct{}),
}
}
// Start starts underlying beacon light client and the sync logic for driving
// the local node.
func (b *Blsync) Start() error {
log.Info("Beacon light sync started")
b.headSub = b.client.SubscribeChainHeadEvent(b.headCh)
go b.client.Start()
for {
select {
case <-b.quitCh:
return nil
case head := <-b.headCh:
if _, err := b.engine.NewPayloadV2(*head.HeadBlock); err != nil {
log.Error("failed to send new payload", "err", err)
continue
}
update := engine.ForkchoiceStateV1{
HeadBlockHash: head.HeadBlock.BlockHash,
SafeBlockHash: head.Finalized, //TODO pass finalized or empty hash here?
FinalizedBlockHash: head.Finalized,
}
if _, err := b.engine.ForkchoiceUpdatedV1(update, nil); err != nil {
log.Error("failed to send forkchoice updated", "err", err)
continue
}
}
}
}
// Stop signals to the light client and syncer to exit.
func (b *Blsync) Stop() error {
b.client.Stop()
close(b.quitCh)
return nil
}

@ -54,8 +54,8 @@ require (
github.com/naoina/toml v0.1.2-0.20170918210437-9fafd6967416
github.com/olekukonko/tablewriter v0.0.5
github.com/peterh/liner v1.1.1-0.20190123174540-a2c9a5303de7
github.com/protolambda/bls12-381-util v0.0.0-20220416220906-d8552aa452c7
github.com/protolambda/zrnt v0.30.0
github.com/protolambda/bls12-381-util v0.1.0
github.com/protolambda/zrnt v0.32.2
github.com/protolambda/ztyp v0.2.2
github.com/rs/cors v1.7.0
github.com/shirou/gopsutil v3.21.4-0.20210419000835-c7a38de76ee5+incompatible

@ -249,7 +249,6 @@ github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaS
github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY=
github.com/golang/protobuf v1.5.4 h1:i7eJL8qZTpSEXOPTxNKhASYpMn+8e5Q6AdndVa1dWek=
github.com/golang/protobuf v1.5.4/go.mod h1:lnTiLA8Wa4RWRcIUkrtSVa5nRhsEGBg48fD6rSs7xps=
github.com/golang/snappy v0.0.3/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q=
github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q=
github.com/golang/snappy v0.0.5-0.20220116011046-fa5810519dcb h1:PBC98N2aIaM3XXiurYmW7fx4GZkL8feAMVq7nEjURHk=
github.com/golang/snappy v0.0.5-0.20220116011046-fa5810519dcb/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q=
@ -395,7 +394,6 @@ github.com/mattn/go-runewidth v0.0.13/go.mod h1:Jdepj2loyihRzMpdS35Xk/zdY8IAYHsh
github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0=
github.com/matttproud/golang_protobuf_extensions v1.0.2-0.20181231171920-c182affec369 h1:I0XW9+e1XWDxdcEniV4rQAIOPUGDq67JSCiRCgGCZLI=
github.com/matttproud/golang_protobuf_extensions v1.0.2-0.20181231171920-c182affec369/go.mod h1:BSXmuO+STAnVfrANrmjBb36TMTDstsz7MSK+HVaYKv4=
github.com/minio/sha256-simd v0.1.0/go.mod h1:2FMWW+8GMoPweT6+pI63m9YE3Lmw4J71hV56Chs1E/U=
github.com/minio/sha256-simd v1.0.0 h1:v1ta+49hkWZyvaKwrQB8elexRqm6Y0aMLjCNsrYxo6g=
github.com/minio/sha256-simd v1.0.0/go.mod h1:OuYzVNI5vcoYIAmbIvHPl3N3jUzVedXbKy5RFepssQM=
github.com/mitchellh/mapstructure v1.4.1 h1:CpVNEelQCZBooIPDn+AR3NpivK/TIKU8bDxdASFVQag=
@ -467,12 +465,10 @@ github.com/prometheus/procfs v0.1.3/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4O
github.com/prometheus/procfs v0.6.0/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA=
github.com/prometheus/procfs v0.7.3 h1:4jVXhlkAyzOScmCkXBTOLRLTz8EeU+eyjrwB/EPq0VU=
github.com/prometheus/procfs v0.7.3/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA=
github.com/protolambda/bls12-381-util v0.0.0-20210720105258-a772f2aac13e/go.mod h1:MPZvj2Pr0N8/dXyTPS5REeg2sdLG7t8DRzC1rLv925w=
github.com/protolambda/bls12-381-util v0.0.0-20220416220906-d8552aa452c7 h1:cZC+usqsYgHtlBaGulVnZ1hfKAi8iWtujBnRLQE698c=
github.com/protolambda/bls12-381-util v0.0.0-20220416220906-d8552aa452c7/go.mod h1:IToEjHuttnUzwZI5KBSM/LOOW3qLbbrHOEfp3SbECGY=
github.com/protolambda/messagediff v1.4.0/go.mod h1:LboJp0EwIbJsePYpzh5Op/9G1/4mIztMRYzzwR0dR2M=
github.com/protolambda/zrnt v0.30.0 h1:pHEn69ZgaDFGpLGGYG1oD7DvYI7RDirbMBPfbC+8p4g=
github.com/protolambda/zrnt v0.30.0/go.mod h1:qcdX9CXFeVNCQK/q0nswpzhd+31RHMk2Ax/2lMsJ4Jw=
github.com/protolambda/bls12-381-util v0.1.0 h1:05DU2wJN7DTU7z28+Q+zejXkIsA/MF8JZQGhtBZZiWk=
github.com/protolambda/bls12-381-util v0.1.0/go.mod h1:cdkysJTRpeFeuUVx/TXGDQNMTiRAalk1vQw3TYTHcE4=
github.com/protolambda/zrnt v0.32.2 h1:KZ48T+3UhsPXNdtE/5QEvGc9DGjUaRI17nJaoznoIaM=
github.com/protolambda/zrnt v0.32.2/go.mod h1:A0fezkp9Tt3GBLATSPIbuY4ywYESyAuc/FFmPKg8Lqs=
github.com/protolambda/ztyp v0.2.2 h1:rVcL3vBu9W/aV646zF6caLS/dyn9BN8NYiuJzicLNyY=
github.com/protolambda/ztyp v0.2.2/go.mod h1:9bYgKGqg3wJqT9ac1gI2hnVb0STQq7p/1lapqrqY1dU=
github.com/prysmaticlabs/gohashtree v0.0.1-alpha.0.20220714111606-acbb2962fb48 h1:cSo6/vk8YpvkLbk9v3FO97cakNmUoxwi2KMP8hd5WIw=
@ -868,7 +864,6 @@ gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY=
gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ=
gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM=
gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM=
gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA=
gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM=
honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4=

Loading…
Cancel
Save